From c1dc09d186a692892ff91104c59f26ae22a7ef35 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Mon, 8 Jan 2024 14:00:07 +0530 Subject: [PATCH 01/54] Checkpointing poc work --- extensions-core/multi-stage-query/pom.xml | 6 + .../druid/msq/compaction/MSQCompaction.java | 113 ++++++++++++++++++ .../compaction/MSQCompactionDruidModule.java | 35 ++++++ .../msq/compaction/MSQCompactionProvider.java | 17 +++ .../coordinator/duty/CompactSegments.java | 7 +- .../duty/CompactionClientNative.java | 19 +++ .../duty/CompactionClientProvider.java | 10 ++ .../coordinator/duty/CompactionStrategy.java | 15 +++ 8 files changed, 220 insertions(+), 2 deletions(-) create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompaction.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompactionDruidModule.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompactionProvider.java create mode 100644 server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionClientNative.java create mode 100644 server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionClientProvider.java create mode 100644 server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionStrategy.java diff --git a/extensions-core/multi-stage-query/pom.xml b/extensions-core/multi-stage-query/pom.xml index 59184f2441ba..4615258694db 100644 --- a/extensions-core/multi-stage-query/pom.xml +++ b/extensions-core/multi-stage-query/pom.xml @@ -265,6 +265,12 @@ test-jar test + + org.apache.druid + druid-processing + 29.0.0-SNAPSHOT + compile + diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompaction.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompaction.java new file mode 100644 index 000000000000..e974b6bba3c2 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompaction.java @@ -0,0 +1,113 @@ +package org.apache.druid.msq.compaction; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonTypeName; +import org.apache.druid.client.indexing.ClientCompactionTaskQuery; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.data.input.impl.DimensionSchema; +import org.apache.druid.msq.indexing.MSQControllerTask; +import org.apache.druid.msq.indexing.MSQSpec; +import org.apache.druid.msq.indexing.MSQTuningConfig; +import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination; +import org.apache.druid.msq.indexing.destination.MSQDestination; +import org.apache.druid.msq.indexing.destination.TaskReportMSQDestination; +import org.apache.druid.query.Druids; +import org.apache.druid.query.Query; +import org.apache.druid.query.dimension.DefaultDimensionSpec; +import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.rpc.indexing.OverlordClient; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.server.coordinator.duty.CompactionClient; +import org.apache.druid.sql.calcite.planner.ColumnMappings; + +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +@JsonTypeName() +public class MSQCompaction implements CompactionClient +{ + @JacksonInject + OverlordClient overlordClient; + + @Override + public String submitCompactionTask(ClientCompactionTaskQuery compactionParams) + { + +/* + GroupByQuery.Builder builder = new GroupByQuery.Builder().setGranularity(compactionParams.getGranularitySpec() + .getSegmentGranularity()) + .setInterval(compactionParams.getIoConfig() + .getInputSpec() + .getInterval()) + .setDataSource(compactionParams.getDataSource()) + .setDimensions(compactionParams.getDimensionsSpec() + .getDimensions() + .stream() + .map(d -> new DefaultDimensionSpec( + d.getName(), + d.getName() + )) + .collect(Collectors.toList())); + + if (compactionParams.getMetricsSpec() != null) { + builder.setAggregatorSpecs(compactionParams.getMetricsSpec()); + } +*/ + Druids.ScanQueryBuilder builder = new Druids.ScanQueryBuilder() + .dataSource(compactionParams.getDataSource()) + .columns() + .intervals(new MultipleIntervalSegmentSpec(Collections.singletonList(compactionParams.getIoConfig() + .getInputSpec() + .getInterval()))); + + + Query query = builder.build(); + + RowSignature.Builder rowSignatureBuilder = RowSignature.builder(); + + rowSignatureBuilder.addTimeColumn(); + + for (DimensionSchema ds : compactionParams.getDimensionsSpec().getDimensions()) { + rowSignatureBuilder.add(ds.getName(), ColumnType.fromString(ds.getTypeName())); + } + + MSQDestination msqDestination = new DataSourceMSQDestination( + compactionParams.getDataSource() + "-compacted", + compactionParams.getGranularitySpec() + .getSegmentGranularity(), + null, + null + ); + + MSQSpec msqSpec = MSQSpec.builder() + .query(query) + .columnMappings(ColumnMappings.identity(rowSignatureBuilder.build())) + .destination(msqDestination) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .build(); + + + final String taskId = compactionParams.getId(); + + + MSQControllerTask controllerTask = + new MSQControllerTask( + taskId, + msqSpec, + null, + null, + null, + null, + null, + compactionParams.getContext() + ); + + FutureUtils.getUnchecked(overlordClient.runTask(taskId, controllerTask), true); + + return taskId; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompactionDruidModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompactionDruidModule.java new file mode 100644 index 000000000000..286a6120cc80 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompactionDruidModule.java @@ -0,0 +1,35 @@ +package org.apache.druid.msq.compaction; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.google.inject.Binder; +import org.apache.druid.guice.Jerseys; +import org.apache.druid.guice.LifecycleModule; +import org.apache.druid.initialization.DruidModule; +import org.apache.druid.msq.sql.resources.SqlTaskResource; + +import java.util.Collections; +import java.util.List; + +public class MSQCompactionDruidModule implements DruidModule +{ + + public static final String SCHEME = "msq"; + + @Override + public void configure(Binder binder) + { + // Force eager initialization. +// LifecycleModule.register(binder, MSQCompaction.class); +// Jerseys.addResource(binder, MSQCompaction.class); + } + + @Override + public List getJacksonModules() + { +// return Collections.emptyList(); + return Collections.singletonList(new SimpleModule(this.getClass().getSimpleName()).registerSubtypes( + MSQCompactionProvider.class)); + } + +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompactionProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompactionProvider.java new file mode 100644 index 000000000000..a76127429ad2 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompactionProvider.java @@ -0,0 +1,17 @@ +package org.apache.druid.msq.compaction; + +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.inject.Provider; +import org.apache.druid.server.coordinator.duty.CompactionClient; +import org.apache.druid.server.coordinator.duty.CompactionClientProvider; + +@JsonTypeName(MSQCompactionDruidModule.SCHEME) +public class MSQCompactionProvider implements CompactionClientProvider +{ +// @JacksonInject +// ClientCompactionTaskQuery compactionTaskQuery; + @Override + public CompactionClient get(){ + return new MSQCompaction(); + } +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index bb88b86dbf84..737379b8a2cb 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java @@ -94,6 +94,8 @@ public class CompactSegments implements CoordinatorCustomDuty // read by HTTP threads processing Coordinator API calls. private final AtomicReference> autoCompactionSnapshotPerDataSource = new AtomicReference<>(); + @JacksonInject private CompactionClient compactionClient; + @Inject @JsonCreator public CompactSegments( @@ -648,7 +650,7 @@ private String compactSegments( final String taskId = IdUtils.newTaskId(idPrefix, ClientCompactionTaskQuery.TYPE, dataSource, null); final Granularity segmentGranularity = granularitySpec == null ? null : granularitySpec.getSegmentGranularity(); - final ClientTaskQuery taskPayload = new ClientCompactionTaskQuery( + final ClientCompactionTaskQuery taskPayload = new ClientCompactionTaskQuery( taskId, dataSource, new ClientCompactionIOConfig( @@ -662,7 +664,8 @@ private String compactSegments( transformSpec, context ); - FutureUtils.getUnchecked(overlordClient.runTask(taskId, taskPayload), true); +// FutureUtils.getUnchecked(overlordClient.runTask(taskId, taskPayload), true); + compactionClient.submitCompactionTask(taskPayload); return taskId; } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionClientNative.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionClientNative.java new file mode 100644 index 000000000000..bafb8fe0865e --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionClientNative.java @@ -0,0 +1,19 @@ +package org.apache.druid.server.coordinator.duty; + +import com.fasterxml.jackson.annotation.JacksonInject; +import org.apache.druid.client.indexing.ClientCompactionTaskQuery; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.rpc.indexing.OverlordClient; + +public class CompactionClientNative implements CompactionClient +{ + @JacksonInject + OverlordClient overlordClient; + @Override + public String submitCompactionTask(ClientCompactionTaskQuery taskPayload) + { + + FutureUtils.getUnchecked(overlordClient.runTask(taskPayload.getId(), taskPayload), true); + return taskPayload.getId(); + } +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionClientProvider.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionClientProvider.java new file mode 100644 index 000000000000..0c648e2b8fce --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionClientProvider.java @@ -0,0 +1,10 @@ +package org.apache.druid.server.coordinator.duty; + + +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.google.inject.Provider; + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "compactionType") +public interface CompactionClientProvider extends Provider +{ +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionStrategy.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionStrategy.java new file mode 100644 index 000000000000..b3b5923c6c65 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionStrategy.java @@ -0,0 +1,15 @@ +package org.apache.druid.server.coordinator.duty; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD}) +@Retention(RetentionPolicy.RUNTIME) +@BindingAnnotation +public @interface CompactionStrategy +{ +} From d11dcf450bf8b7fd93ea9496b245ee4cc342443f Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Mon, 8 Jan 2024 14:01:01 +0530 Subject: [PATCH 02/54] Add CompactionClient --- .../coordinator/duty/CompactionClient.java | 27 +++++++++++++++++++ 1 file changed, 27 insertions(+) create mode 100644 server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionClient.java diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionClient.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionClient.java new file mode 100644 index 000000000000..906badb031bb --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionClient.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.coordinator.duty; + +import org.apache.druid.client.indexing.ClientCompactionTaskQuery; + +public interface CompactionClient +{ + String submitCompactionTask(ClientCompactionTaskQuery compactionParams); +} From a861c57e86b91dbf77724413d9d78a6c62b11864 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Wed, 24 Jan 2024 09:31:26 +0530 Subject: [PATCH 03/54] Compaction POC working version --- .../druid/msq/compaction/MSQCompaction.java | 74 ++++++++++++++----- .../msq/compaction/MSQCompactionProvider.java | 4 +- .../MSQCompactionDruidModule.java | 13 +++- ...rg.apache.druid.initialization.DruidModule | 1 + .../server/coordinator/DruidCoordinator.java | 8 +- .../coordinator/duty/CompactSegments.java | 18 ++++- .../coordinator/duty/CompactionClient.java | 2 + .../duty/CompactionClientNative.java | 6 ++ .../duty/CompactionClientProvider.java | 2 +- .../coordinator/DruidCoordinatorTest.java | 10 +-- .../CoordinatorSimulationBuilder.java | 2 +- 11 files changed, 105 insertions(+), 35 deletions(-) rename extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/{compaction => guice}/MSQCompactionDruidModule.java (57%) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompaction.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompaction.java index e974b6bba3c2..a6ed46cf62bd 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompaction.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompaction.java @@ -1,38 +1,49 @@ package org.apache.druid.msq.compaction; import com.fasterxml.jackson.annotation.JacksonInject; -import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import org.apache.druid.client.indexing.ClientCompactionTaskQuery; import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.data.input.impl.DimensionSchema; +import org.apache.druid.java.util.common.Intervals; import org.apache.druid.msq.indexing.MSQControllerTask; import org.apache.druid.msq.indexing.MSQSpec; import org.apache.druid.msq.indexing.MSQTuningConfig; import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination; import org.apache.druid.msq.indexing.destination.MSQDestination; -import org.apache.druid.msq.indexing.destination.TaskReportMSQDestination; import org.apache.druid.query.Druids; import org.apache.druid.query.Query; -import org.apache.druid.query.dimension.DefaultDimensionSpec; -import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.server.coordinator.duty.CompactionClient; +import org.apache.druid.sql.calcite.parser.DruidSqlReplace; import org.apache.druid.sql.calcite.planner.ColumnMappings; +import org.apache.druid.sql.calcite.rel.DruidQuery; +import org.joda.time.Interval; +import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.stream.Collectors; -@JsonTypeName() +//@JsonTypeName() public class MSQCompaction implements CompactionClient { + public MSQCompaction(){ + System.out.println("Initializing MSQCompaction"); + } @JacksonInject OverlordClient overlordClient; + @Override + public void setOverlordClient(OverlordClient overlordClient) + { + this.overlordClient = overlordClient; + } + @Override public String submitCompactionTask(ClientCompactionTaskQuery compactionParams) { @@ -57,30 +68,50 @@ public String submitCompactionTask(ClientCompactionTaskQuery compactionParams) builder.setAggregatorSpecs(compactionParams.getMetricsSpec()); } */ - Druids.ScanQueryBuilder builder = new Druids.ScanQueryBuilder() - .dataSource(compactionParams.getDataSource()) - .columns() - .intervals(new MultipleIntervalSegmentSpec(Collections.singletonList(compactionParams.getIoConfig() - .getInputSpec() - .getInterval()))); - - - Query query = builder.build(); - +// String escapedJson = "[{\"name\":\"added\",\"type\":\"LONG\"},{\"name\":\"channel\",\"type\":\"STRING\"},{\"name\":\"cityName\",\"type\":\"STRING\"},{\"name\":\"comment\",\"type\":\"STRING\"},{\"name\":\"countryIsoCode\",\"type\":\"STRING\"},{\"name\":\"countryName\",\"type\":\"STRING\"},{\"name\":\"deleted\",\"type\":\"LONG\"},{\"name\":\"delta\",\"type\":\"LONG\"},{\"name\":\"isAnonymous\",\"type\":\"STRING\"},{\"name\":\"isMinor\",\"type\":\"STRING\"},{\"name\":\"isNew\",\"type\":\"STRING\"},{\"name\":\"isRobot\",\"type\":\"STRING\"},{\"name\":\"isUnpatrolled\",\"type\":\"STRING\"},{\"name\":\"metroCode\",\"type\":\"LONG\"},{\"name\":\"namespace\",\"type\":\"STRING\"},{\"name\":\"page\",\"type\":\"STRING\"},{\"name\":\"regionIsoCode\",\"type\":\"STRING\"},{\"name\":\"regionName\",\"type\":\"STRING\"},{\"name\":\"user\",\"type\":\"STRING\"},{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"time\",\"type\":\"LONG\"}]"; RowSignature.Builder rowSignatureBuilder = RowSignature.builder(); - rowSignatureBuilder.addTimeColumn(); +// rowSignatureBuilder.addTimeColumn(); + List columns = new ArrayList<>(); for (DimensionSchema ds : compactionParams.getDimensionsSpec().getDimensions()) { rowSignatureBuilder.add(ds.getName(), ColumnType.fromString(ds.getTypeName())); + columns.add(ds.getName()); } + Interval replaceInterval = compactionParams.getIoConfig() + .getInputSpec() + .getInterval(); + + MultipleIntervalSegmentSpec multipleIntervalSegmentSpecFull = new MultipleIntervalSegmentSpec(Collections.singletonList( + Intervals.ETERNITY)); + MultipleIntervalSegmentSpec multipleIntervalSegmentSpecQuery = new MultipleIntervalSegmentSpec(Collections.singletonList(compactionParams.getIoConfig() + .getInputSpec() + .getInterval())); + + + String escapedJson = "[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"added\",\"type\":\"LONG\"},{\"name\":\"channel\",\"type\":\"STRING\"},{\"name\":\"cityName\",\"type\":\"STRING\"},{\"name\":\"comment\",\"type\":\"STRING\"},{\"name\":\"countryIsoCode\",\"type\":\"STRING\"},{\"name\":\"countryName\",\"type\":\"STRING\"},{\"name\":\"deleted\",\"type\":\"LONG\"},{\"name\":\"delta\",\"type\":\"LONG\"},{\"name\":\"isAnonymous\",\"type\":\"STRING\"},{\"name\":\"isMinor\",\"type\":\"STRING\"},{\"name\":\"isNew\",\"type\":\"STRING\"},{\"name\":\"isRobot\",\"type\":\"STRING\"},{\"name\":\"isUnpatrolled\",\"type\":\"STRING\"},{\"name\":\"metroCode\",\"type\":\"LONG\"},{\"name\":\"namespace\",\"type\":\"STRING\"},{\"name\":\"page\",\"type\":\"STRING\"},{\"name\":\"regionIsoCode\",\"type\":\"STRING\"},{\"name\":\"regionName\",\"type\":\"STRING\"},{\"name\":\"user\",\"type\":\"STRING\"}]"; + Druids.ScanQueryBuilder builder = new Druids.ScanQueryBuilder() + .dataSource(compactionParams.getDataSource()) + .columns(columns) +// .columns("__time", "added", "channel", "cityName", "comment", "countryIsoCode", "countryName", "deleted", "delta", "isAnonymous", "isMinor", "isNew", "isRobot", "isUnpatrolled", "metroCode", "namespace", "page", "regionIsoCode", "regionName", "user") + .intervals(multipleIntervalSegmentSpecQuery) + .legacy(false) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .context(ImmutableMap.of(DruidQuery.CTX_SCAN_SIGNATURE, escapedJson, DruidSqlReplace.SQL_REPLACE_TIME_CHUNKS, replaceInterval.toString(), "sqlInsertSegmentGranularity", "\"HOUR\"")); + + +// QuerySegmentSpec (intervals) for SQL initiated reingest: MultipleIntervalSegmentSpec{intervals=[-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z]} + Query query = builder.build(); + + MSQDestination msqDestination = new DataSourceMSQDestination( - compactionParams.getDataSource() + "-compacted", + compactionParams.getDataSource(), compactionParams.getGranularitySpec() .getSegmentGranularity(), null, - null + ImmutableList.of(replaceInterval) + ); MSQSpec msqSpec = MSQSpec.builder() @@ -88,11 +119,16 @@ public String submitCompactionTask(ClientCompactionTaskQuery compactionParams) .columnMappings(ColumnMappings.identity(rowSignatureBuilder.build())) .destination(msqDestination) .tuningConfig(MSQTuningConfig.defaultConfig()) + .build(); final String taskId = compactionParams.getId(); +// Map context = compactionParams.getContext(); +// context.put(DruidQuery.CTX_SCAN_SIGNATURE, msqSpec.getColumnMappings()); + + MSQControllerTask controllerTask = new MSQControllerTask( diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompactionProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompactionProvider.java index a76127429ad2..033ed63ddfa7 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompactionProvider.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompactionProvider.java @@ -1,11 +1,11 @@ package org.apache.druid.msq.compaction; import com.fasterxml.jackson.annotation.JsonTypeName; -import com.google.inject.Provider; +import org.apache.druid.msq.guice.MSQCompactionDruidModule; import org.apache.druid.server.coordinator.duty.CompactionClient; import org.apache.druid.server.coordinator.duty.CompactionClientProvider; -@JsonTypeName(MSQCompactionDruidModule.SCHEME) +//@JsonTypeName(MSQCompactionDruidModule.SCHEME) public class MSQCompactionProvider implements CompactionClientProvider { // @JacksonInject diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompactionDruidModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQCompactionDruidModule.java similarity index 57% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompactionDruidModule.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQCompactionDruidModule.java index 286a6120cc80..5e5c95db489a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompactionDruidModule.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQCompactionDruidModule.java @@ -1,12 +1,17 @@ -package org.apache.druid.msq.compaction; +package org.apache.druid.msq.guice; import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.inject.Binder; import org.apache.druid.guice.Jerseys; +import org.apache.druid.guice.LazySingleton; import org.apache.druid.guice.LifecycleModule; import org.apache.druid.initialization.DruidModule; +import org.apache.druid.msq.compaction.MSQCompaction; +import org.apache.druid.msq.compaction.MSQCompactionProvider; +import org.apache.druid.msq.sql.MSQTaskSqlEngine; import org.apache.druid.msq.sql.resources.SqlTaskResource; +import org.apache.druid.server.coordinator.duty.CompactionClient; import java.util.Collections; import java.util.List; @@ -22,14 +27,16 @@ public void configure(Binder binder) // Force eager initialization. // LifecycleModule.register(binder, MSQCompaction.class); // Jerseys.addResource(binder, MSQCompaction.class); + binder.bind(CompactionClient.class).toProvider(MSQCompactionProvider.class).in(LazySingleton.class); } @Override public List getJacksonModules() { // return Collections.emptyList(); - return Collections.singletonList(new SimpleModule(this.getClass().getSimpleName()).registerSubtypes( - MSQCompactionProvider.class)); +// return Collections.singletonList(new SimpleModule(this.getClass().getSimpleName()).registerSubtypes( +// MSQCompactionProvider.class)); + return Collections.emptyList(); } } diff --git a/extensions-core/multi-stage-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-core/multi-stage-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule index cabd131fb758..5c7962e2e69d 100644 --- a/extensions-core/multi-stage-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule +++ b/extensions-core/multi-stage-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule @@ -18,3 +18,4 @@ org.apache.druid.msq.guice.MSQIndexingModule org.apache.druid.msq.guice.MSQDurableStorageModule org.apache.druid.msq.guice.MSQSqlModule org.apache.druid.msq.guice.SqlTaskModule +org.apache.druid.msq.guice.MSQCompactionDruidModule \ No newline at end of file diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java index 5558d204e81a..51752d27e166 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java @@ -57,6 +57,7 @@ import org.apache.druid.server.coordinator.duty.BalanceSegments; import org.apache.druid.server.coordinator.duty.CollectSegmentAndServerStats; import org.apache.druid.server.coordinator.duty.CompactSegments; +import org.apache.druid.server.coordinator.duty.CompactionClient; import org.apache.druid.server.coordinator.duty.CoordinatorCustomDutyGroup; import org.apache.druid.server.coordinator.duty.CoordinatorCustomDutyGroups; import org.apache.druid.server.coordinator.duty.CoordinatorDuty; @@ -137,6 +138,7 @@ public class DruidCoordinator private final ServiceEmitter emitter; private final OverlordClient overlordClient; + private final CompactionClient compactionClient; private final ScheduledExecutorFactory executorFactory; private final Map dutyGroupExecutors = new HashMap<>(); private final LoadQueueTaskMaster taskMaster; @@ -185,7 +187,8 @@ public DruidCoordinator( BalancerStrategyFactory balancerStrategyFactory, LookupCoordinatorManager lookupCoordinatorManager, @Coordinator DruidLeaderSelector coordLeaderSelector, - CompactionSegmentSearchPolicy compactionSegmentSearchPolicy + CompactionSegmentSearchPolicy compactionSegmentSearchPolicy, + CompactionClient compactionClient ) { this.config = config; @@ -193,6 +196,7 @@ public DruidCoordinator( this.serverInventoryView = serverInventoryView; this.emitter = emitter; this.overlordClient = overlordClient; + this.compactionClient = compactionClient; this.taskMaster = taskMaster; this.serviceAnnouncer = serviceAnnouncer; this.self = self; @@ -561,7 +565,7 @@ CompactSegments initializeCompactSegmentsDuty(CompactionSegmentSearchPolicy comp { List compactSegmentsDutyFromCustomGroups = getCompactSegmentsDutyFromCustomGroups(); if (compactSegmentsDutyFromCustomGroups.isEmpty()) { - return new CompactSegments(compactionSegmentSearchPolicy, overlordClient); + return new CompactSegments(compactionSegmentSearchPolicy, overlordClient, compactionClient); } else { if (compactSegmentsDutyFromCustomGroups.size() > 1) { log.warn( diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index 737379b8a2cb..17e5f4ec1a9b 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java @@ -94,17 +94,31 @@ public class CompactSegments implements CoordinatorCustomDuty // read by HTTP threads processing Coordinator API calls. private final AtomicReference> autoCompactionSnapshotPerDataSource = new AtomicReference<>(); - @JacksonInject private CompactionClient compactionClient; + private final CompactionClient compactionClient; + + public CompactSegments( + @JacksonInject CompactionSegmentSearchPolicy policy, + @JacksonInject OverlordClient overlordClient + ) + { + this.policy = policy; + this.overlordClient = overlordClient; + this.compactionClient = null; + resetCompactionSnapshot(); + } @Inject @JsonCreator public CompactSegments( @JacksonInject CompactionSegmentSearchPolicy policy, - @JacksonInject OverlordClient overlordClient + @JacksonInject OverlordClient overlordClient, + @JacksonInject CompactionClient compactionClient ) { this.policy = policy; this.overlordClient = overlordClient; + this.compactionClient = compactionClient; + this.compactionClient.setOverlordClient(overlordClient); resetCompactionSnapshot(); } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionClient.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionClient.java index 906badb031bb..c29dc6cf2d90 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionClient.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionClient.java @@ -20,8 +20,10 @@ package org.apache.druid.server.coordinator.duty; import org.apache.druid.client.indexing.ClientCompactionTaskQuery; +import org.apache.druid.rpc.indexing.OverlordClient; public interface CompactionClient { String submitCompactionTask(ClientCompactionTaskQuery compactionParams); + void setOverlordClient(OverlordClient overlordClient); } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionClientNative.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionClientNative.java index bafb8fe0865e..b7c6673c8fe0 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionClientNative.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionClientNative.java @@ -16,4 +16,10 @@ public String submitCompactionTask(ClientCompactionTaskQuery taskPayload) FutureUtils.getUnchecked(overlordClient.runTask(taskPayload.getId(), taskPayload), true); return taskPayload.getId(); } + + @Override + public void setOverlordClient(OverlordClient overlordClient) + { + this.overlordClient = overlordClient; + } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionClientProvider.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionClientProvider.java index 0c648e2b8fce..ae6c8c481d3f 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionClientProvider.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionClientProvider.java @@ -4,7 +4,7 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.google.inject.Provider; -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "compactionType") +//@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "compactionType") public interface CompactionClientProvider extends Provider { } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java index a9e85ca1b23e..e7662a0af530 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java @@ -180,7 +180,7 @@ public void setUp() throws Exception new CostBalancerStrategyFactory(), EasyMock.createNiceMock(LookupCoordinatorManager.class), new TestDruidLeaderSelector(), - null + null, null ); } @@ -606,7 +606,7 @@ public void testCompactSegmentsDutyWhenCustomDutyGroupEmpty() new CostBalancerStrategyFactory(), EasyMock.createNiceMock(LookupCoordinatorManager.class), new TestDruidLeaderSelector(), - null + null, null ); // Since CompactSegments is not enabled in Custom Duty Group, then CompactSegments must be created in IndexingServiceDuties List indexingDuties = coordinator.makeIndexingServiceDuties(); @@ -645,7 +645,7 @@ public void testInitializeCompactSegmentsDutyWhenCustomDutyGroupDoesNotContainsC new CostBalancerStrategyFactory(), EasyMock.createNiceMock(LookupCoordinatorManager.class), new TestDruidLeaderSelector(), - null + null, null ); // Since CompactSegments is not enabled in Custom Duty Group, then CompactSegments must be created in IndexingServiceDuties List indexingDuties = coordinator.makeIndexingServiceDuties(); @@ -684,7 +684,7 @@ public void testInitializeCompactSegmentsDutyWhenCustomDutyGroupContainsCompactS new CostBalancerStrategyFactory(), EasyMock.createNiceMock(LookupCoordinatorManager.class), new TestDruidLeaderSelector(), - null + null, null ); // Since CompactSegments is enabled in Custom Duty Group, then CompactSegments must not be created in IndexingServiceDuties List indexingDuties = coordinator.makeIndexingServiceDuties(); @@ -788,7 +788,7 @@ public void testCoordinatorCustomDutyGroupsRunAsExpected() throws Exception new CostBalancerStrategyFactory(), EasyMock.createNiceMock(LookupCoordinatorManager.class), new TestDruidLeaderSelector(), - null + null, null ); coordinator.start(); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java index f341ebf5df99..7d811a367a48 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java @@ -208,7 +208,7 @@ public CoordinatorSimulation build() createBalancerStrategy(env), env.lookupCoordinatorManager, env.leaderSelector, - COMPACTION_SEGMENT_SEARCH_POLICY + COMPACTION_SEGMENT_SEARCH_POLICY, null ); return new SimulationImpl(coordinator, env); From 959a9b39366f06a98f591202a8280ffea96c4f3c Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Mon, 29 Jan 2024 12:06:59 +0530 Subject: [PATCH 04/54] Cleanup --- .../druid/msq/compaction/MSQCompaction.java | 62 +++++++------------ .../msq/compaction/MSQCompactionProvider.java | 2 - .../msq/guice/MSQCompactionDruidModule.java | 6 -- 3 files changed, 21 insertions(+), 49 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompaction.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompaction.java index a6ed46cf62bd..705801908cfe 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompaction.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompaction.java @@ -29,7 +29,6 @@ import java.util.Collections; import java.util.List; -//@JsonTypeName() public class MSQCompaction implements CompactionClient { public MSQCompaction(){ @@ -48,30 +47,8 @@ public void setOverlordClient(OverlordClient overlordClient) public String submitCompactionTask(ClientCompactionTaskQuery compactionParams) { -/* - GroupByQuery.Builder builder = new GroupByQuery.Builder().setGranularity(compactionParams.getGranularitySpec() - .getSegmentGranularity()) - .setInterval(compactionParams.getIoConfig() - .getInputSpec() - .getInterval()) - .setDataSource(compactionParams.getDataSource()) - .setDimensions(compactionParams.getDimensionsSpec() - .getDimensions() - .stream() - .map(d -> new DefaultDimensionSpec( - d.getName(), - d.getName() - )) - .collect(Collectors.toList())); - - if (compactionParams.getMetricsSpec() != null) { - builder.setAggregatorSpecs(compactionParams.getMetricsSpec()); - } -*/ -// String escapedJson = "[{\"name\":\"added\",\"type\":\"LONG\"},{\"name\":\"channel\",\"type\":\"STRING\"},{\"name\":\"cityName\",\"type\":\"STRING\"},{\"name\":\"comment\",\"type\":\"STRING\"},{\"name\":\"countryIsoCode\",\"type\":\"STRING\"},{\"name\":\"countryName\",\"type\":\"STRING\"},{\"name\":\"deleted\",\"type\":\"LONG\"},{\"name\":\"delta\",\"type\":\"LONG\"},{\"name\":\"isAnonymous\",\"type\":\"STRING\"},{\"name\":\"isMinor\",\"type\":\"STRING\"},{\"name\":\"isNew\",\"type\":\"STRING\"},{\"name\":\"isRobot\",\"type\":\"STRING\"},{\"name\":\"isUnpatrolled\",\"type\":\"STRING\"},{\"name\":\"metroCode\",\"type\":\"LONG\"},{\"name\":\"namespace\",\"type\":\"STRING\"},{\"name\":\"page\",\"type\":\"STRING\"},{\"name\":\"regionIsoCode\",\"type\":\"STRING\"},{\"name\":\"regionName\",\"type\":\"STRING\"},{\"name\":\"user\",\"type\":\"STRING\"},{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"time\",\"type\":\"LONG\"}]"; - RowSignature.Builder rowSignatureBuilder = RowSignature.builder(); +RowSignature.Builder rowSignatureBuilder = RowSignature.builder(); -// rowSignatureBuilder.addTimeColumn(); List columns = new ArrayList<>(); for (DimensionSchema ds : compactionParams.getDimensionsSpec().getDimensions()) { @@ -83,25 +60,35 @@ public String submitCompactionTask(ClientCompactionTaskQuery compactionParams) .getInputSpec() .getInterval(); - MultipleIntervalSegmentSpec multipleIntervalSegmentSpecFull = new MultipleIntervalSegmentSpec(Collections.singletonList( - Intervals.ETERNITY)); - MultipleIntervalSegmentSpec multipleIntervalSegmentSpecQuery = new MultipleIntervalSegmentSpec(Collections.singletonList(compactionParams.getIoConfig() - .getInputSpec() - .getInterval())); + MultipleIntervalSegmentSpec multipleIntervalSegmentSpecQuery = new MultipleIntervalSegmentSpec(Collections.singletonList( + replaceInterval)); + String escapedSignatureJson = "[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"added\",\"type\":\"LONG\"}," + + "{\"name\":\"channel\",\"type\":\"STRING\"},{\"name\":\"cityName\",\"type\":\"STRING\"}," + + "{\"name\":\"comment\",\"type\":\"STRING\"},{\"name\":\"countryIsoCode\",\"type\":\"STRING\"}," + + "{\"name\":\"countryName\",\"type\":\"STRING\"},{\"name\":\"deleted\",\"type\":\"LONG\"}," + + "{\"name\":\"delta\",\"type\":\"LONG\"},{\"name\":\"isAnonymous\",\"type\":\"STRING\"}," + + "{\"name\":\"isMinor\",\"type\":\"STRING\"},{\"name\":\"isNew\",\"type\":\"STRING\"}," + + "{\"name\":\"isRobot\",\"type\":\"STRING\"},{\"name\":\"isUnpatrolled\",\"type\":\"STRING\"}," + + "{\"name\":\"metroCode\",\"type\":\"LONG\"},{\"name\":\"namespace\",\"type\":\"STRING\"}," + + "{\"name\":\"page\",\"type\":\"STRING\"},{\"name\":\"regionIsoCode\",\"type\":\"STRING\"}," + + "{\"name\":\"regionName\",\"type\":\"STRING\"},{\"name\":\"user\",\"type\":\"STRING\"}]"; - String escapedJson = "[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"added\",\"type\":\"LONG\"},{\"name\":\"channel\",\"type\":\"STRING\"},{\"name\":\"cityName\",\"type\":\"STRING\"},{\"name\":\"comment\",\"type\":\"STRING\"},{\"name\":\"countryIsoCode\",\"type\":\"STRING\"},{\"name\":\"countryName\",\"type\":\"STRING\"},{\"name\":\"deleted\",\"type\":\"LONG\"},{\"name\":\"delta\",\"type\":\"LONG\"},{\"name\":\"isAnonymous\",\"type\":\"STRING\"},{\"name\":\"isMinor\",\"type\":\"STRING\"},{\"name\":\"isNew\",\"type\":\"STRING\"},{\"name\":\"isRobot\",\"type\":\"STRING\"},{\"name\":\"isUnpatrolled\",\"type\":\"STRING\"},{\"name\":\"metroCode\",\"type\":\"LONG\"},{\"name\":\"namespace\",\"type\":\"STRING\"},{\"name\":\"page\",\"type\":\"STRING\"},{\"name\":\"regionIsoCode\",\"type\":\"STRING\"},{\"name\":\"regionName\",\"type\":\"STRING\"},{\"name\":\"user\",\"type\":\"STRING\"}]"; Druids.ScanQueryBuilder builder = new Druids.ScanQueryBuilder() .dataSource(compactionParams.getDataSource()) .columns(columns) -// .columns("__time", "added", "channel", "cityName", "comment", "countryIsoCode", "countryName", "deleted", "delta", "isAnonymous", "isMinor", "isNew", "isRobot", "isUnpatrolled", "metroCode", "namespace", "page", "regionIsoCode", "regionName", "user") .intervals(multipleIntervalSegmentSpecQuery) .legacy(false) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .context(ImmutableMap.of(DruidQuery.CTX_SCAN_SIGNATURE, escapedJson, DruidSqlReplace.SQL_REPLACE_TIME_CHUNKS, replaceInterval.toString(), "sqlInsertSegmentGranularity", "\"HOUR\"")); - + .context(ImmutableMap.of( + DruidQuery.CTX_SCAN_SIGNATURE, + escapedSignatureJson, + DruidSqlReplace.SQL_REPLACE_TIME_CHUNKS, + replaceInterval.toString(), + "sqlInsertSegmentGranularity", + "\"HOUR\"" + )); -// QuerySegmentSpec (intervals) for SQL initiated reingest: MultipleIntervalSegmentSpec{intervals=[-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z]} Query query = builder.build(); @@ -119,17 +106,10 @@ public String submitCompactionTask(ClientCompactionTaskQuery compactionParams) .columnMappings(ColumnMappings.identity(rowSignatureBuilder.build())) .destination(msqDestination) .tuningConfig(MSQTuningConfig.defaultConfig()) - .build(); - final String taskId = compactionParams.getId(); -// Map context = compactionParams.getContext(); -// context.put(DruidQuery.CTX_SCAN_SIGNATURE, msqSpec.getColumnMappings()); - - - MSQControllerTask controllerTask = new MSQControllerTask( taskId, diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompactionProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompactionProvider.java index 033ed63ddfa7..68bbf5c34bec 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompactionProvider.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompactionProvider.java @@ -1,7 +1,5 @@ package org.apache.druid.msq.compaction; -import com.fasterxml.jackson.annotation.JsonTypeName; -import org.apache.druid.msq.guice.MSQCompactionDruidModule; import org.apache.druid.server.coordinator.duty.CompactionClient; import org.apache.druid.server.coordinator.duty.CompactionClientProvider; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQCompactionDruidModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQCompactionDruidModule.java index 5e5c95db489a..63bcb66668bb 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQCompactionDruidModule.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQCompactionDruidModule.java @@ -1,16 +1,10 @@ package org.apache.druid.msq.guice; import com.fasterxml.jackson.databind.Module; -import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.inject.Binder; -import org.apache.druid.guice.Jerseys; import org.apache.druid.guice.LazySingleton; -import org.apache.druid.guice.LifecycleModule; import org.apache.druid.initialization.DruidModule; -import org.apache.druid.msq.compaction.MSQCompaction; import org.apache.druid.msq.compaction.MSQCompactionProvider; -import org.apache.druid.msq.sql.MSQTaskSqlEngine; -import org.apache.druid.msq.sql.resources.SqlTaskResource; import org.apache.druid.server.coordinator.duty.CompactionClient; import java.util.Collections; From 5290bdc46a3d8452490c13b332ad2769e86f4afa Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Mon, 29 Jan 2024 12:08:06 +0530 Subject: [PATCH 05/54] Cleanup 2 --- .../apache/druid/msq/guice/MSQCompactionDruidModule.java | 6 ------ 1 file changed, 6 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQCompactionDruidModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQCompactionDruidModule.java index 63bcb66668bb..42c200f788b1 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQCompactionDruidModule.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQCompactionDruidModule.java @@ -18,18 +18,12 @@ public class MSQCompactionDruidModule implements DruidModule @Override public void configure(Binder binder) { - // Force eager initialization. -// LifecycleModule.register(binder, MSQCompaction.class); -// Jerseys.addResource(binder, MSQCompaction.class); binder.bind(CompactionClient.class).toProvider(MSQCompactionProvider.class).in(LazySingleton.class); } @Override public List getJacksonModules() { -// return Collections.emptyList(); -// return Collections.singletonList(new SimpleModule(this.getClass().getSimpleName()).registerSubtypes( -// MSQCompactionProvider.class)); return Collections.emptyList(); } From 65a03436a4de618e63e08cee8c3d2579fe3abe77 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Fri, 5 Apr 2024 17:40:48 +0530 Subject: [PATCH 06/54] Separate out native and msq-based compaction flows and refine query converters --- .run/Coordinator w_ MSQ.run.xml | 19 ++ extensions-core/multi-stage-query/pom.xml | 6 - .../msq/compaction/CompactionToMSQImpl.java | 304 ++++++++++++++++++ .../CompactionToMSQImplProvider.java | 15 + .../druid/msq/compaction/MSQCompaction.java | 129 -------- .../msq/compaction/MSQCompactionProvider.java | 15 - .../msq/guice/MSQCompactionDruidModule.java | 6 +- .../druid/indexing/common/TaskToolbox.java | 23 +- .../indexing/common/TaskToolboxFactory.java | 7 +- .../indexing/common/task/CompactionTask.java | 175 +++++----- .../indexing/common/task/CompactionToMSQ.java | 19 ++ .../common/task/CompactionToMSQProvider.java | 9 + .../indexing/common/TaskToolboxTest.java | 3 +- ...penderatorDriverRealtimeIndexTaskTest.java | 3 +- .../common/task/CompactionTaskTest.java | 47 +-- .../task/CompactionTuningConfigTest.java | 3 + .../common/task/RealtimeIndexTaskTest.java | 3 +- .../SingleTaskBackgroundRunnerTest.java | 3 +- .../indexing/overlord/TaskLifecycleTest.java | 3 +- .../overlord/TestTaskToolboxFactory.java | 13 +- .../SeekableStreamIndexTaskTestBase.java | 3 +- .../worker/WorkerTaskManagerTest.java | 3 +- .../worker/WorkerTaskMonitorTest.java | 3 +- .../partitions/SecondaryPartitionType.java | 4 +- .../server/coordinator/DruidCoordinator.java | 8 +- .../coordinator/duty/CompactSegments.java | 23 +- .../coordinator/duty/CompactionClient.java | 29 -- .../duty/CompactionClientNative.java | 25 -- .../duty/CompactionClientProvider.java | 10 - .../coordinator/DruidCoordinatorTest.java | 10 +- .../CoordinatorSimulationBuilder.java | 2 +- 31 files changed, 567 insertions(+), 358 deletions(-) create mode 100644 .run/Coordinator w_ MSQ.run.xml create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQImpl.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQImplProvider.java delete mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompaction.java delete mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompactionProvider.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionToMSQ.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionToMSQProvider.java delete mode 100644 server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionClient.java delete mode 100644 server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionClientNative.java delete mode 100644 server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionClientProvider.java diff --git a/.run/Coordinator w_ MSQ.run.xml b/.run/Coordinator w_ MSQ.run.xml new file mode 100644 index 000000000000..074b578ea722 --- /dev/null +++ b/.run/Coordinator w_ MSQ.run.xml @@ -0,0 +1,19 @@ + + + + \ No newline at end of file diff --git a/extensions-core/multi-stage-query/pom.xml b/extensions-core/multi-stage-query/pom.xml index 8d2b501eac48..58e1976d556c 100644 --- a/extensions-core/multi-stage-query/pom.xml +++ b/extensions-core/multi-stage-query/pom.xml @@ -295,12 +295,6 @@ test-jar test - - org.apache.druid - druid-processing - 29.0.0-SNAPSHOT - compile - diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQImpl.java new file mode 100644 index 000000000000..d9f4480daa48 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQImpl.java @@ -0,0 +1,304 @@ +package org.apache.druid.msq.compaction; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.data.input.impl.DimensionSchema; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; +import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.indexer.partitions.SecondaryPartitionType; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.task.CompactionTask; +import org.apache.druid.indexing.common.task.CompactionToMSQ; +import org.apache.druid.indexing.common.task.CurrentSubTaskHolder; +import org.apache.druid.indexing.common.task.Tasks; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.granularity.AllGranularity; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.msq.indexing.MSQControllerTask; +import org.apache.druid.msq.indexing.MSQSpec; +import org.apache.druid.msq.indexing.MSQTuningConfig; +import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination; +import org.apache.druid.msq.indexing.destination.MSQDestination; +import org.apache.druid.query.Druids; +import org.apache.druid.query.Query; +import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.dimension.DefaultDimensionSpec; +import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; +import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.sql.calcite.parser.DruidSqlInsert; +import org.apache.druid.sql.calcite.parser.DruidSqlReplace; +import org.apache.druid.sql.calcite.planner.ColumnMappings; +import org.joda.time.Interval; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class CompactionToMSQImpl implements CompactionToMSQ +{ + private static final Logger log = new Logger(CompactionToMSQImpl.class); + private static final Granularity DEFAULT_SEGMENT_GRANULARITY = Granularities.ALL; + + + public CompactionToMSQImpl() + { + System.out.println("Initializing MSQCompaction"); + } + + List getColumns(DimensionsSpec dimensionSpec) + { + + List columns = new ArrayList<>(); + for (DimensionSchema ds : dimensionSpec.getDimensions()) { + columns.add(ds.getName()); + } + return columns; + } + + RowSignature getRowSignature(DimensionsSpec dimensionSpec) + { + RowSignature.Builder rowSignatureBuilder = RowSignature.builder(); + + for (DimensionSchema ds : dimensionSpec.getDimensions()) { + rowSignatureBuilder.add(ds.getName(), ColumnType.fromString(ds.getTypeName())); + } + return rowSignatureBuilder.build(); + } + + public MultipleIntervalSegmentSpec createMultipleIntervalSpec(Interval interval) + { + return new MultipleIntervalSegmentSpec(Collections.singletonList(interval)); + + } + + public List getOrderBySpec(PartitionsSpec partitionSpec) + { + if (partitionSpec.getType() == SecondaryPartitionType.RANGE) { + List dimensions = ((DimensionRangePartitionsSpec) partitionSpec).getPartitionDimensions(); + return dimensions.stream() + .map(dim -> new OrderByColumnSpec(dim, OrderByColumnSpec.Direction.ASCENDING)) + .collect(Collectors.toList()); + } + return Collections.emptyList(); + } + + public Query buildScanQuery(CompactionTask compactionTask, Interval interval, DataSchema dataSchema) + { + return new Druids.ScanQueryBuilder().dataSource(dataSchema.getDataSource()) + .columns(getColumns(dataSchema.getDimensionsSpec())) + .intervals(createMultipleIntervalSpec(interval)) + .legacy(false) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .filters(dataSchema.getTransformSpec().getFilter()) + .context(compactionTask.getContext()) + .build(); + } + + public Query buildGroupByQuery(CompactionTask compactionTask, Interval interval, DataSchema dataSchema) + { + + DimFilter dimFilter = dataSchema.getTransformSpec().getFilter(); + + GroupByQuery.Builder builder = new GroupByQuery.Builder() + .setDataSource(new TableDataSource(compactionTask.getDataSource())) + .setVirtualColumns(VirtualColumns.EMPTY) + .setDimFilter(dimFilter) + .setGranularity(dataSchema.getGranularitySpec().getQueryGranularity() != null + ? dataSchema.getGranularitySpec() + .getQueryGranularity() + : new AllGranularity() + ) + .setDimensions(dataSchema.getDimensionsSpec() + .getDimensions() + .stream() + .map(d -> new DefaultDimensionSpec(d.getName(), d.getName())) + .collect(Collectors.toList()) + ) + .setAggregatorSpecs(Arrays.asList(dataSchema.getAggregators())) + .setContext(compactionTask.getContext()) + .setInterval(interval); + + if (compactionTask.getTuningConfig() != null && compactionTask.getTuningConfig().getPartitionsSpec() != null) { + getOrderBySpec(compactionTask.getTuningConfig().getPartitionsSpec()).forEach(builder::addOrderByColumn); + } + return builder.build(); + } + + @Override + public TaskStatus createAndRunMSQControllerTasks( + CompactionTask compactionTask, TaskToolbox taskToolbox, List> intervalDataSchemas + ) throws Exception + { + + List msqControllerTasks = new ArrayList<>(); + + for (Pair intervalDataSchema : intervalDataSchemas) { + Query query; + Interval interval = intervalDataSchema.lhs; + DataSchema ds = intervalDataSchema.rhs; + final Interval replaceInterval = compactionTask.getIoConfig() + .getInputSpec() + .findInterval(compactionTask.getDataSource()); +// this.escapedRowSignature = "[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"added\",\"type\":\"LONG\"}," +// + "{\"name\":\"channel\",\"type\":\"STRING\"},{\"name\":\"cityName\",\"type\":\"STRING\"}," +// + "{\"name\":\"comment\",\"type\":\"STRING\"},{\"name\":\"countryIsoCode\",\"type\":\"STRING\"}," +// + "{\"name\":\"countryName\",\"type\":\"STRING\"},{\"name\":\"deleted\",\"type\":\"LONG\"}," +// + "{\"name\":\"delta\",\"type\":\"LONG\"},{\"name\":\"isAnonymous\",\"type\":\"STRING\"}," +// + "{\"name\":\"isMinor\",\"type\":\"STRING\"},{\"name\":\"isNew\",\"type\":\"STRING\"}," +// + "{\"name\":\"isRobot\",\"type\":\"STRING\"},{\"name\":\"isUnpatrolled\",\"type\":\"STRING\"}," +// + "{\"name\":\"metroCode\",\"type\":\"LONG\"},{\"name\":\"namespace\",\"type\":\"STRING\"}," +// + "{\"name\":\"page\",\"type\":\"STRING\"},{\"name\":\"regionIsoCode\",\"type\":\"STRING\"}," +// + "{\"name\":\"regionName\",\"type\":\"STRING\"},{\"name\":\"user\",\"type\":\"STRING\"}]"; + + boolean isGroupBy = ds != null && ds.getAggregators().length > 0; + + // CTX_SCAN_SIGNATURE is deprecated and should no longer be reqd +// compactionTask.getContext().putAll(ImmutableMap.of(DruidQuery.CTX_SCAN_SIGNATURE, +// new ObjectMapper().writeValueAsString(getRowSignature(ds.getDimensionsSpec())), + Map context = ImmutableMap.copyOf(compactionTask.getContext()); + // TODO (vishesh): check why the default is day in specific classes whereas ALL in query maker + context.put( + DruidSqlInsert.SQL_INSERT_SEGMENT_GRANULARITY, + (ds.getGranularitySpec() != null) + ? ds.getGranularitySpec().getSegmentGranularity() + : DEFAULT_SEGMENT_GRANULARITY + ); + context.put(DruidSqlReplace.SQL_REPLACE_TIME_CHUNKS, interval.toString()); + context.put(Tasks.STORE_COMPACTION_STATE_KEY, true); + + if (!isGroupBy) { + query = buildScanQuery(compactionTask, interval, ds); + } else { + query = buildGroupByQuery(compactionTask, interval, ds); + } + + MSQDestination msqDestination = new DataSourceMSQDestination( + ds.getDataSource(), + ds.getGranularitySpec().getSegmentGranularity(), + null, + ImmutableList.of(replaceInterval) + + ); + + PartitionsSpec partitionsSpec = compactionTask.getTuningConfig().getPartitionsSpec(); + + Integer rowsPerSegment = null; + + if (partitionsSpec instanceof DimensionRangePartitionsSpec) { + rowsPerSegment = ((DimensionRangePartitionsSpec) partitionsSpec).getTargetRowsPerSegment(); + + } + + MSQTuningConfig msqTuningConfig = new MSQTuningConfig( + null, + null, + rowsPerSegment, + compactionTask.getTuningConfig().getIndexSpec() + ); + + MSQSpec msqSpec = MSQSpec.builder() + .query(query) + .columnMappings(ColumnMappings.identity(getRowSignature(ds.getDimensionsSpec()))) + .destination(msqDestination) + .tuningConfig(msqTuningConfig) + .build(); + + MSQControllerTask controllerTask = getMsqControllerTask(compactionTask, msqSpec); + + msqControllerTasks.add(controllerTask); + } +// if (msqControllerTasks.isEmpty()){ +// log.warn("Can't find segments from inputSpec[%s], nothing to do.", +// ioConfig.getInputSpec()) +// } + return runSubtasks( + msqControllerTasks, + taskToolbox, + compactionTask.getCurrentSubTaskHolder(), + compactionTask.getId() + ); + } + + private TaskStatus runSubtasks( + List tasks, + TaskToolbox toolbox, + CurrentSubTaskHolder currentSubTaskHolder, + String compactionTaskId + ) throws JsonProcessingException + { + final int totalNumSpecs = tasks.size(); + log.info("Generated [%d] compaction task specs", totalNumSpecs); + + int failCnt = 0; +// Map completionReports = new HashMap<>(); + for (MSQControllerTask eachTask : tasks) { + final String json = toolbox.getJsonMapper().writerWithDefaultPrettyPrinter().writeValueAsString(eachTask); + if (!currentSubTaskHolder.setTask(eachTask)) { + String errMsg = "Task was asked to stop. Finish as failed."; + log.info(errMsg); + return TaskStatus.failure(compactionTaskId, errMsg); + } + try { + if (eachTask.isReady(toolbox.getTaskActionClient())) { + log.info("Running MSQControllerTask: " + json); + final TaskStatus eachResult = eachTask.run(toolbox); + if (!eachResult.isSuccess()) { + failCnt++; + log.warn("Failed to run MSQControllerTask: [%s].\nTrying the next MSQControllerTask.", json); + } + + } else { + failCnt++; + log.warn("MSQControllerTask is not ready: [%s].\nTrying the next MSQControllerTask.", json); + } + } + catch (Exception e) { + failCnt++; + log.warn(e, "Failed to run MSQControllerTask: [%s].\nTrying the next MSQControllerTask.", json); + } + } + String msg = StringUtils.format( + "Ran [%d] MSQControllerTasks, [%d] succeeded, [%d] failed", + totalNumSpecs, + totalNumSpecs - failCnt, + failCnt + ); + +// toolbox.getTaskReportFileWriter().write(compactionTaskId, completionReports); + log.info(msg); + return failCnt == 0 ? TaskStatus.success(compactionTaskId) : TaskStatus.failure(compactionTaskId, msg); + } + + + private static MSQControllerTask getMsqControllerTask(CompactionTask compactionTask, MSQSpec msqSpec) + { + final String taskId = compactionTask.getId(); + + return new MSQControllerTask( + taskId, + msqSpec, + null, + null, + null, + null, + null, + compactionTask.getContext() + ); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQImplProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQImplProvider.java new file mode 100644 index 000000000000..5d720aca8824 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQImplProvider.java @@ -0,0 +1,15 @@ +package org.apache.druid.msq.compaction; + +import org.apache.druid.indexing.common.task.CompactionToMSQ; +import org.apache.druid.indexing.common.task.CompactionToMSQProvider; + +//@JsonTypeName(MSQCompactionDruidModule.SCHEME) +public class CompactionToMSQImplProvider implements CompactionToMSQProvider +{ +// @JacksonInject +// ClientCompactionTaskQuery compactionTaskQuery; + @Override + public CompactionToMSQ get(){ + return new CompactionToMSQImpl(); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompaction.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompaction.java deleted file mode 100644 index 705801908cfe..000000000000 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompaction.java +++ /dev/null @@ -1,129 +0,0 @@ -package org.apache.druid.msq.compaction; - -import com.fasterxml.jackson.annotation.JacksonInject; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import org.apache.druid.client.indexing.ClientCompactionTaskQuery; -import org.apache.druid.common.guava.FutureUtils; -import org.apache.druid.data.input.impl.DimensionSchema; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.msq.indexing.MSQControllerTask; -import org.apache.druid.msq.indexing.MSQSpec; -import org.apache.druid.msq.indexing.MSQTuningConfig; -import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination; -import org.apache.druid.msq.indexing.destination.MSQDestination; -import org.apache.druid.query.Druids; -import org.apache.druid.query.Query; -import org.apache.druid.query.scan.ScanQuery; -import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; -import org.apache.druid.rpc.indexing.OverlordClient; -import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.server.coordinator.duty.CompactionClient; -import org.apache.druid.sql.calcite.parser.DruidSqlReplace; -import org.apache.druid.sql.calcite.planner.ColumnMappings; -import org.apache.druid.sql.calcite.rel.DruidQuery; -import org.joda.time.Interval; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - -public class MSQCompaction implements CompactionClient -{ - public MSQCompaction(){ - System.out.println("Initializing MSQCompaction"); - } - @JacksonInject - OverlordClient overlordClient; - - @Override - public void setOverlordClient(OverlordClient overlordClient) - { - this.overlordClient = overlordClient; - } - - @Override - public String submitCompactionTask(ClientCompactionTaskQuery compactionParams) - { - -RowSignature.Builder rowSignatureBuilder = RowSignature.builder(); - - List columns = new ArrayList<>(); - - for (DimensionSchema ds : compactionParams.getDimensionsSpec().getDimensions()) { - rowSignatureBuilder.add(ds.getName(), ColumnType.fromString(ds.getTypeName())); - columns.add(ds.getName()); - } - - Interval replaceInterval = compactionParams.getIoConfig() - .getInputSpec() - .getInterval(); - - MultipleIntervalSegmentSpec multipleIntervalSegmentSpecQuery = new MultipleIntervalSegmentSpec(Collections.singletonList( - replaceInterval)); - - String escapedSignatureJson = "[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"added\",\"type\":\"LONG\"}," - + "{\"name\":\"channel\",\"type\":\"STRING\"},{\"name\":\"cityName\",\"type\":\"STRING\"}," - + "{\"name\":\"comment\",\"type\":\"STRING\"},{\"name\":\"countryIsoCode\",\"type\":\"STRING\"}," - + "{\"name\":\"countryName\",\"type\":\"STRING\"},{\"name\":\"deleted\",\"type\":\"LONG\"}," - + "{\"name\":\"delta\",\"type\":\"LONG\"},{\"name\":\"isAnonymous\",\"type\":\"STRING\"}," - + "{\"name\":\"isMinor\",\"type\":\"STRING\"},{\"name\":\"isNew\",\"type\":\"STRING\"}," - + "{\"name\":\"isRobot\",\"type\":\"STRING\"},{\"name\":\"isUnpatrolled\",\"type\":\"STRING\"}," - + "{\"name\":\"metroCode\",\"type\":\"LONG\"},{\"name\":\"namespace\",\"type\":\"STRING\"}," - + "{\"name\":\"page\",\"type\":\"STRING\"},{\"name\":\"regionIsoCode\",\"type\":\"STRING\"}," - + "{\"name\":\"regionName\",\"type\":\"STRING\"},{\"name\":\"user\",\"type\":\"STRING\"}]"; - - Druids.ScanQueryBuilder builder = new Druids.ScanQueryBuilder() - .dataSource(compactionParams.getDataSource()) - .columns(columns) - .intervals(multipleIntervalSegmentSpecQuery) - .legacy(false) - .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .context(ImmutableMap.of( - DruidQuery.CTX_SCAN_SIGNATURE, - escapedSignatureJson, - DruidSqlReplace.SQL_REPLACE_TIME_CHUNKS, - replaceInterval.toString(), - "sqlInsertSegmentGranularity", - "\"HOUR\"" - )); - - Query query = builder.build(); - - - MSQDestination msqDestination = new DataSourceMSQDestination( - compactionParams.getDataSource(), - compactionParams.getGranularitySpec() - .getSegmentGranularity(), - null, - ImmutableList.of(replaceInterval) - - ); - - MSQSpec msqSpec = MSQSpec.builder() - .query(query) - .columnMappings(ColumnMappings.identity(rowSignatureBuilder.build())) - .destination(msqDestination) - .tuningConfig(MSQTuningConfig.defaultConfig()) - .build(); - - final String taskId = compactionParams.getId(); - - MSQControllerTask controllerTask = - new MSQControllerTask( - taskId, - msqSpec, - null, - null, - null, - null, - null, - compactionParams.getContext() - ); - - FutureUtils.getUnchecked(overlordClient.runTask(taskId, controllerTask), true); - - return taskId; - } -} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompactionProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompactionProvider.java deleted file mode 100644 index 68bbf5c34bec..000000000000 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/MSQCompactionProvider.java +++ /dev/null @@ -1,15 +0,0 @@ -package org.apache.druid.msq.compaction; - -import org.apache.druid.server.coordinator.duty.CompactionClient; -import org.apache.druid.server.coordinator.duty.CompactionClientProvider; - -//@JsonTypeName(MSQCompactionDruidModule.SCHEME) -public class MSQCompactionProvider implements CompactionClientProvider -{ -// @JacksonInject -// ClientCompactionTaskQuery compactionTaskQuery; - @Override - public CompactionClient get(){ - return new MSQCompaction(); - } -} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQCompactionDruidModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQCompactionDruidModule.java index 42c200f788b1..42ef45783b5a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQCompactionDruidModule.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQCompactionDruidModule.java @@ -3,9 +3,9 @@ import com.fasterxml.jackson.databind.Module; import com.google.inject.Binder; import org.apache.druid.guice.LazySingleton; +import org.apache.druid.indexing.common.task.CompactionToMSQ; import org.apache.druid.initialization.DruidModule; -import org.apache.druid.msq.compaction.MSQCompactionProvider; -import org.apache.druid.server.coordinator.duty.CompactionClient; +import org.apache.druid.msq.compaction.CompactionToMSQImplProvider; import java.util.Collections; import java.util.List; @@ -18,7 +18,7 @@ public class MSQCompactionDruidModule implements DruidModule @Override public void configure(Binder binder) { - binder.bind(CompactionClient.class).toProvider(MSQCompactionProvider.class).in(LazySingleton.class); + binder.bind(CompactionToMSQ.class).toProvider(CompactionToMSQImplProvider.class).in(LazySingleton.class); } @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java index e828e5e3f07f..8564c64cf934 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java @@ -35,6 +35,7 @@ import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.config.TaskConfig; +import org.apache.druid.indexing.common.task.CompactionToMSQ; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTaskClientProvider; import org.apache.druid.indexing.common.task.batch.parallel.ShuffleClient; import org.apache.druid.indexing.worker.shuffle.IntermediaryDataManager; @@ -133,6 +134,9 @@ public class TaskToolbox private final String attemptId; private final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; + + private final CompactionToMSQ compactionToMSQ; + public TaskToolbox( SegmentLoaderConfig segmentLoaderConfig, TaskConfig config, @@ -174,7 +178,8 @@ public TaskToolbox( ShuffleClient shuffleClient, TaskLogPusher taskLogPusher, String attemptId, - CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig, + CompactionToMSQ compactionToMSQ ) { this.segmentLoaderConfig = segmentLoaderConfig; @@ -219,6 +224,7 @@ public TaskToolbox( this.taskLogPusher = taskLogPusher; this.attemptId = attemptId; this.centralizedDatasourceSchemaConfig = centralizedDatasourceSchemaConfig; + this.compactionToMSQ = compactionToMSQ; } public SegmentLoaderConfig getSegmentLoaderConfig() @@ -496,6 +502,11 @@ public CentralizedDatasourceSchemaConfig getCentralizedTableSchemaConfig() return centralizedDatasourceSchemaConfig; } + public CompactionToMSQ getCompactionToMSQ() + { + return compactionToMSQ; + } + /** * Create {@link AdjustedRuntimeInfo} based on the given {@link RuntimeInfo} and {@link AppenderatorsManager}. This * is a way to allow code to properly apportion the amount of processors and heap available to the entire JVM. @@ -563,6 +574,7 @@ public static class Builder private TaskLogPusher taskLogPusher; private String attemptId; private CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; + private CompactionToMSQ compactionToMSQ; public Builder() { @@ -609,6 +621,7 @@ public Builder(TaskToolbox other) this.supervisorTaskClientProvider = other.supervisorTaskClientProvider; this.shuffleClient = other.shuffleClient; this.centralizedDatasourceSchemaConfig = other.centralizedDatasourceSchemaConfig; + this.compactionToMSQ = other.compactionToMSQ; } public Builder config(final SegmentLoaderConfig segmentLoaderConfig) @@ -857,6 +870,11 @@ public Builder centralizedTableSchemaConfig(final CentralizedDatasourceSchemaCon return this; } + public Builder compactionToMSQ(final CompactionToMSQ compactionToMSQ){ + this.compactionToMSQ = compactionToMSQ; + return this; + } + public TaskToolbox build() { return new TaskToolbox( @@ -900,7 +918,8 @@ public TaskToolbox build() shuffleClient, taskLogPusher, attemptId, - centralizedDatasourceSchemaConfig + centralizedDatasourceSchemaConfig, + compactionToMSQ ); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java index 6f009d0b889e..e7ed053d4c2c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java @@ -36,6 +36,7 @@ import org.apache.druid.guice.annotations.RemoteChatHandler; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; import org.apache.druid.indexing.common.config.TaskConfig; +import org.apache.druid.indexing.common.task.CompactionToMSQ; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTaskClientProvider; @@ -116,6 +117,7 @@ public class TaskToolboxFactory private final TaskLogPusher taskLogPusher; private final String attemptId; private final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; + private final CompactionToMSQ compactionToMSQ; @Inject public TaskToolboxFactory( @@ -158,7 +160,8 @@ public TaskToolboxFactory( ShuffleClient shuffleClient, TaskLogPusher taskLogPusher, @AttemptId String attemptId, - CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig, + CompactionToMSQ compactionToMSQ ) { this.segmentLoaderConfig = segmentLoadConfig; @@ -201,6 +204,7 @@ public TaskToolboxFactory( this.taskLogPusher = taskLogPusher; this.attemptId = attemptId; this.centralizedDatasourceSchemaConfig = centralizedDatasourceSchemaConfig; + this.compactionToMSQ = compactionToMSQ; } public TaskToolbox build(Task task) @@ -265,6 +269,7 @@ public TaskToolbox build(TaskConfig config, Task task) .taskLogPusher(taskLogPusher) .attemptId(attemptId) .centralizedTableSchemaConfig(centralizedDatasourceSchemaConfig) + .compactionToMSQ(compactionToMSQ) .build(); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 59a0a499f917..690c77268c57 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -25,6 +25,7 @@ import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonInclude.Include; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonProcessingException; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.collect.BiMap; @@ -290,7 +291,8 @@ static CompactionTuningConfig getTuningConfig(TuningConfig tuningConfig) parallelIndexTuningConfig.getMaxSavedParseExceptions(), parallelIndexTuningConfig.getMaxColumnsToMerge(), parallelIndexTuningConfig.getAwaitSegmentAvailabilityTimeoutMillis(), - parallelIndexTuningConfig.getNumPersistThreads() + parallelIndexTuningConfig.getNumPersistThreads(), + null ); } else if (tuningConfig instanceof IndexTuningConfig) { final IndexTuningConfig indexTuningConfig = (IndexTuningConfig) tuningConfig; @@ -325,7 +327,8 @@ static CompactionTuningConfig getTuningConfig(TuningConfig tuningConfig) indexTuningConfig.getMaxSavedParseExceptions(), indexTuningConfig.getMaxColumnsToMerge(), indexTuningConfig.getAwaitSegmentAvailabilityTimeoutMillis(), - indexTuningConfig.getNumPersistThreads() + indexTuningConfig.getNumPersistThreads(), + null ); } else { throw new ISE( @@ -458,53 +461,77 @@ void emitCompactIngestionModeMetrics( @Override public TaskStatus runTask(TaskToolbox toolbox) throws Exception { - // emit metric for compact ingestion mode: emitCompactIngestionModeMetrics(toolbox.getEmitter(), ioConfig.isDropExisting()); - final List ingestionSpecs = createIngestionSchema( + final List> intervalDataSchemas = createDataSchemasForIntervals( UTC_CLOCK, toolbox, getTaskLockHelper().getLockGranularityToUse(), - ioConfig, segmentProvider, - partitionConfigurationManager, dimensionsSpec, transformSpec, metricsSpec, granularitySpec, - toolbox.getCoordinatorClient(), - segmentCacheManagerFactory, getMetricBuilder() ); - final List indexTaskSpecs = IntStream - .range(0, ingestionSpecs.size()) - .mapToObj(i -> { - // The ID of SubtaskSpecs is used as the base sequenceName in segment allocation protocol. - // The indexing tasks generated by the compaction task should use different sequenceNames - // so that they can allocate valid segment IDs with no duplication. - ParallelIndexIngestionSpec ingestionSpec = ingestionSpecs.get(i); - final String baseSequenceName = createIndexTaskSpecId(i); - return newTask(baseSequenceName, ingestionSpec); - }) - .collect(Collectors.toList()); - - if (indexTaskSpecs.isEmpty()) { - String msg = StringUtils.format( - "Can't find segments from inputSpec[%s], nothing to do.", - ioConfig.getInputSpec() - ); - log.warn(msg); - return TaskStatus.failure(getId(), msg); + + // TODO: Uncomment the condition +// if (tuningConfig.useMSQ && toolbox.getCompactionToMSQ() != null) { + if (toolbox.getCompactionToMSQ() != null) { + // Switch to using MSQ for compaction + registerResourceCloserOnAbnormalExit(currentSubTaskHolder); + return toolbox.getCompactionToMSQ() + .createAndRunMSQControllerTasks(this, toolbox, intervalDataSchemas); } else { + final List ingestionSpecs = createIngestionSpecs( + intervalDataSchemas, + toolbox, + ioConfig, + partitionConfigurationManager, + toolbox.getCoordinatorClient(), + segmentCacheManagerFactory + ); + + List subtasks = IntStream + .range(0, ingestionSpecs.size()) + .mapToObj(i -> { + // The ID of SubtaskSpecs is used as the base sequenceName in segment allocation protocol. + // The indexing tasks generated by the compaction task should use different sequenceNames + // so that they can allocate valid segment IDs with no duplication. + ParallelIndexIngestionSpec ingestionSpec = ingestionSpecs.get(i); + final String baseSequenceName = createIndexTaskSpecId(i); + return newTask(baseSequenceName, ingestionSpec); + }) + .collect(Collectors.toList()); + + if (subtasks.isEmpty()) { + String msg = StringUtils.format("Can't find segments from inputSpec[%s], nothing to do.", + ioConfig.getInputSpec() + ); + log.warn(msg); + return TaskStatus.failure(getId(), msg); + } registerResourceCloserOnAbnormalExit(currentSubTaskHolder); - final int totalNumSpecs = indexTaskSpecs.size(); - log.info("Generated [%d] compaction task specs", totalNumSpecs); + return runParallelIndexSubtasks(subtasks, toolbox); + } + } + + private TaskStatus runParallelIndexSubtasks(List tasks, TaskToolbox toolbox) + throws JsonProcessingException + { + + // TODO(vishesh): Remove below line. +// System.out.println(new ObjectMapper().writeValueAsString(tasks)); +// return TaskStatus.failure(getId(), "Forced failure"); + final int totalNumSpecs = tasks.size(); + log.info("Generated [%d] compaction task specs", totalNumSpecs); + int failCnt = 0; Map completionReports = new HashMap<>(); - for (int i = 0; i < indexTaskSpecs.size(); i++) { - ParallelIndexSupervisorTask eachSpec = indexTaskSpecs.get(i); + for (int i = 0; i < tasks.size(); i++) { + ParallelIndexSupervisorTask eachSpec = tasks.get(i); final String json = toolbox.getJsonMapper().writerWithDefaultPrettyPrinter().writeValueAsString(eachSpec); if (!currentSubTaskHolder.setTask(eachSpec)) { String errMsg = "Task was asked to stop. Finish as failed."; @@ -542,7 +569,6 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception toolbox.getTaskReportFileWriter().write(getId(), completionReports); log.info(msg); return failCnt == 0 ? TaskStatus.success(getId()) : TaskStatus.failure(getId(), msg); - } } @VisibleForTesting @@ -581,19 +607,15 @@ private String createIndexTaskSpecId(int i) * @return an empty list if input segments don't exist. Otherwise, a generated ingestionSpec. */ @VisibleForTesting - static List createIngestionSchema( + static List> createDataSchemasForIntervals( final Clock clock, final TaskToolbox toolbox, final LockGranularity lockGranularityInUse, - final CompactionIOConfig ioConfig, final SegmentProvider segmentProvider, - final PartitionConfigurationManager partitionConfigurationManager, @Nullable final DimensionsSpec dimensionsSpec, @Nullable final ClientCompactionTaskTransformSpec transformSpec, @Nullable final AggregatorFactory[] metricsSpec, @Nullable final ClientCompactionTaskGranularitySpec granularitySpec, - final CoordinatorClient coordinatorClient, - final SegmentCacheManagerFactory segmentCacheManagerFactory, final ServiceMetricEvent.Builder metricBuilder ) throws IOException { @@ -607,10 +629,8 @@ static List createIngestionSchema( return Collections.emptyList(); } - final CompactionTuningConfig compactionTuningConfig = partitionConfigurationManager.computeTuningConfig(); - if (granularitySpec == null || granularitySpec.getSegmentGranularity() == null) { - final List specs = new ArrayList<>(); + List> dataSchemas = new ArrayList<>(); // original granularity final Map> intervalToSegments = new TreeMap<>( @@ -664,23 +684,11 @@ static List createIngestionSchema( : granularitySpec.withSegmentGranularity(segmentGranularityToUse) ); - specs.add( - new ParallelIndexIngestionSpec( - dataSchema, - createIoConfig( - toolbox, - dataSchema, - interval, - coordinatorClient, - segmentCacheManagerFactory, - ioConfig - ), - compactionTuningConfig - ) - ); + // TODO: Based on a function flag, either create parallel index spec OR just aggregate up dataSchemas. We should + // pass a list of schemas to MSQ to have one-to-one mapping between an MSQ controller task and a schema + dataSchemas.add(Pair.of(interval, dataSchema)); } - - return specs; + return dataSchemas; } else { // given segment granularity final DataSchema dataSchema = createDataSchema( @@ -704,24 +712,36 @@ static List createIngestionSchema( metricsSpec, granularitySpec ); - - return Collections.singletonList( - new ParallelIndexIngestionSpec( - dataSchema, - createIoConfig( - toolbox, - dataSchema, - segmentProvider.interval, - coordinatorClient, - segmentCacheManagerFactory, - ioConfig - ), - compactionTuningConfig - ) - ); + return Collections.singletonList(Pair.of(segmentProvider.interval, dataSchema)); } } + private static List createIngestionSpecs( + List> dataschemas, + final TaskToolbox toolbox, + final CompactionIOConfig ioConfig, + final PartitionConfigurationManager partitionConfigurationManager, + final CoordinatorClient coordinatorClient, + final SegmentCacheManagerFactory segmentCacheManagerFactory + ) + { + final CompactionTuningConfig compactionTuningConfig = partitionConfigurationManager.computeTuningConfig(); + + return dataschemas.stream().map((dataSchema) -> new ParallelIndexIngestionSpec( + dataSchema.rhs, + createIoConfig(toolbox, + dataSchema.rhs, + dataSchema.lhs, + coordinatorClient, + segmentCacheManagerFactory, + ioConfig + ), + compactionTuningConfig + ) + + ).collect(Collectors.toList()); + } + private static ParallelIndexIOConfig createIoConfig( TaskToolbox toolbox, DataSchema dataSchema, @@ -1365,6 +1385,7 @@ public CompactionTask build() public static class CompactionTuningConfig extends ParallelIndexTuningConfig { public static final String TYPE = "compaction"; + public final Boolean useMSQ; public static CompactionTuningConfig defaultConfig() { @@ -1399,7 +1420,9 @@ public static CompactionTuningConfig defaultConfig() null, null, 0L, - null + null, + // TODO vishesh: Move this default to false later + true ); } @@ -1436,7 +1459,8 @@ public CompactionTuningConfig( @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions, @JsonProperty("maxColumnsToMerge") @Nullable Integer maxColumnsToMerge, @JsonProperty("awaitSegmentAvailabilityTimeoutMillis") @Nullable Long awaitSegmentAvailabilityTimeoutMillis, - @JsonProperty("numPersistThreads") @Nullable Integer numPersistThreads + @JsonProperty("numPersistThreads") @Nullable Integer numPersistThreads, + @JsonProperty("useMSQ") @Nullable Boolean useMSQ ) { super( @@ -1478,6 +1502,8 @@ public CompactionTuningConfig( awaitSegmentAvailabilityTimeoutMillis == null || awaitSegmentAvailabilityTimeoutMillis == 0, "awaitSegmentAvailabilityTimeoutMillis is not supported for Compcation Task" ); + // TODO vishesh: Move this default to false later + this.useMSQ = useMSQ != null ? useMSQ: true; } @Override @@ -1514,7 +1540,8 @@ public CompactionTuningConfig withPartitionsSpec(PartitionsSpec partitionsSpec) getMaxSavedParseExceptions(), getMaxColumnsToMerge(), getAwaitSegmentAvailabilityTimeoutMillis(), - getNumPersistThreads() + getNumPersistThreads(), + null ); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionToMSQ.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionToMSQ.java new file mode 100644 index 000000000000..e6e4d759b2dc --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionToMSQ.java @@ -0,0 +1,19 @@ +package org.apache.druid.indexing.common.task; + +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.segment.indexing.DataSchema; +import org.joda.time.Interval; + +import java.util.List; + +public interface CompactionToMSQ +{ + TaskStatus createAndRunMSQControllerTasks( + CompactionTask compactionTask, + TaskToolbox taskToolbox, + List> dataSchemas + ) throws Exception; + +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionToMSQProvider.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionToMSQProvider.java new file mode 100644 index 000000000000..73c8232fad5a --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionToMSQProvider.java @@ -0,0 +1,9 @@ +package org.apache.druid.indexing.common.task; + + +import com.google.inject.Provider; + +//@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "compactionType") +public interface CompactionToMSQProvider extends Provider +{ +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java index e1ac9482436b..e63ce24d41bc 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java @@ -157,7 +157,8 @@ public void setUp() throws IOException null, null, "1", - CentralizedDatasourceSchemaConfig.create() + CentralizedDatasourceSchemaConfig.create(), + null ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java index ef769d73006d..bcd26d383d40 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java @@ -1647,7 +1647,8 @@ public void close() null, null, "1", - CentralizedDatasourceSchemaConfig.create() + CentralizedDatasourceSchemaConfig.create(), + null ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index c90c08349c4d..3cadb5528ddd 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -172,7 +172,7 @@ @RunWith(MockitoJUnitRunner.class) public class CompactionTaskTest { - private static final long SEGMENT_SIZE_BYTES = 100; + /*private static final long SEGMENT_SIZE_BYTES = 100; private static final int NUM_ROWS_PER_SEGMENT = 10; private static final String DATA_SOURCE = "dataSource"; private static final String TIMESTAMP_COLUMN = "timestamp"; @@ -367,6 +367,7 @@ private static CompactionTask.CompactionTuningConfig createTuningConfig() null, null, null, + null, null ); } @@ -789,6 +790,7 @@ public void testGetTuningConfigWithIndexTuningConfig() null, null, null, + null, null ); @@ -875,6 +877,7 @@ public void testGetTuningConfigWithParallelIndexTuningConfig() null, null, null, + null, null ); @@ -952,7 +955,7 @@ public void testSegmentProviderFindSegmentsWithEmptySegmentsThrowException() @Test public void testCreateIngestionSchema() throws IOException { - final List ingestionSpecs = CompactionTask.createIngestionSchema( + final List ingestionSpecs = CompactionTask.createDataSchemas( clock, toolbox, LockGranularity.TIME_CHUNK, @@ -1026,9 +1029,10 @@ public void testCreateIngestionSchemaWithTargetPartitionSize() throws IOExceptio null, null, null, + null, null ); - final List ingestionSpecs = CompactionTask.createIngestionSchema( + final List ingestionSpecs = CompactionTask.createDataSchemas( clock, toolbox, LockGranularity.TIME_CHUNK, @@ -1103,9 +1107,10 @@ public void testCreateIngestionSchemaWithMaxTotalRows() throws IOException null, null, null, + null, null ); - final List ingestionSpecs = CompactionTask.createIngestionSchema( + final List ingestionSpecs = CompactionTask.createDataSchemas( clock, toolbox, LockGranularity.TIME_CHUNK, @@ -1180,9 +1185,10 @@ public void testCreateIngestionSchemaWithNumShards() throws IOException null, null, null, + null, null ); - final List ingestionSpecs = CompactionTask.createIngestionSchema( + final List ingestionSpecs = CompactionTask.createDataSchemas( clock, toolbox, LockGranularity.TIME_CHUNK, @@ -1248,7 +1254,7 @@ public void testCreateIngestionSchemaWithCustomDimensionsSpec() throws IOExcepti ) ); - final List ingestionSpecs = CompactionTask.createIngestionSchema( + final List ingestionSpecs = CompactionTask.createDataSchemas( clock, toolbox, LockGranularity.TIME_CHUNK, @@ -1294,7 +1300,7 @@ public void testCreateIngestionSchemaWithCustomMetricsSpec() throws IOException new DoubleMaxAggregatorFactory("custom_double_max", "agg_4") }; - final List ingestionSpecs = CompactionTask.createIngestionSchema( + final List ingestionSpecs = CompactionTask.createDataSchemas( clock, toolbox, LockGranularity.TIME_CHUNK, @@ -1333,7 +1339,7 @@ public void testCreateIngestionSchemaWithCustomMetricsSpec() throws IOException @Test public void testCreateIngestionSchemaWithCustomSegments() throws IOException { - final List ingestionSpecs = CompactionTask.createIngestionSchema( + final List ingestionSpecs = CompactionTask.createDataSchemas( clock, toolbox, LockGranularity.TIME_CHUNK, @@ -1378,7 +1384,7 @@ public void testCreateIngestionSchemaWithDifferentSegmentSet() throws IOExceptio Collections.sort(segments); // Remove one segment in the middle segments.remove(segments.size() / 2); - CompactionTask.createIngestionSchema( + CompactionTask.createDataSchemas( clock, toolbox, LockGranularity.TIME_CHUNK, @@ -1404,7 +1410,7 @@ public void testMissingMetadata() throws IOException final TestIndexIO indexIO = (TestIndexIO) toolbox.getIndexIO(); indexIO.removeMetadata(Iterables.getFirst(indexIO.getQueryableIndexMap().keySet(), null)); final List segments = new ArrayList<>(SEGMENTS); - CompactionTask.createIngestionSchema( + CompactionTask.createDataSchemas( clock, toolbox, LockGranularity.TIME_CHUNK, @@ -1442,7 +1448,7 @@ public void testEmptyInterval() @Test public void testSegmentGranularityAndNullQueryGranularity() throws IOException { - final List ingestionSpecs = CompactionTask.createIngestionSchema( + final List ingestionSpecs = CompactionTask.createDataSchemas( clock, toolbox, LockGranularity.TIME_CHUNK, @@ -1482,7 +1488,7 @@ public void testSegmentGranularityAndNullQueryGranularity() throws IOException @Test public void testQueryGranularityAndNullSegmentGranularity() throws IOException { - final List ingestionSpecs = CompactionTask.createIngestionSchema( + final List ingestionSpecs = CompactionTask.createDataSchemas( clock, toolbox, LockGranularity.TIME_CHUNK, @@ -1520,7 +1526,7 @@ public void testQueryGranularityAndNullSegmentGranularity() throws IOException @Test public void testQueryGranularityAndSegmentGranularityNonNull() throws IOException { - final List ingestionSpecs = CompactionTask.createIngestionSchema( + final List ingestionSpecs = CompactionTask.createDataSchemas( clock, toolbox, LockGranularity.TIME_CHUNK, @@ -1565,7 +1571,7 @@ public void testQueryGranularityAndSegmentGranularityNonNull() throws IOExceptio @Test public void testNullGranularitySpec() throws IOException { - final List ingestionSpecs = CompactionTask.createIngestionSchema( + final List ingestionSpecs = CompactionTask.createDataSchemas( clock, toolbox, LockGranularity.TIME_CHUNK, @@ -1604,7 +1610,7 @@ public void testNullGranularitySpec() throws IOException public void testGranularitySpecWithNullQueryGranularityAndNullSegmentGranularity() throws IOException { - final List ingestionSpecs = CompactionTask.createIngestionSchema( + final List ingestionSpecs = CompactionTask.createDataSchemas( clock, toolbox, LockGranularity.TIME_CHUNK, @@ -1643,7 +1649,7 @@ public void testGranularitySpecWithNullQueryGranularityAndNullSegmentGranularity public void testGranularitySpecWithNotNullRollup() throws IOException { - final List ingestionSpecs = CompactionTask.createIngestionSchema( + final List ingestionSpecs = CompactionTask.createDataSchemas( clock, toolbox, LockGranularity.TIME_CHUNK, @@ -1669,7 +1675,7 @@ public void testGranularitySpecWithNotNullRollup() public void testGranularitySpecWithNullRollup() throws IOException { - final List ingestionSpecs = CompactionTask.createIngestionSchema( + final List ingestionSpecs = CompactionTask.createDataSchemas( clock, toolbox, LockGranularity.TIME_CHUNK, @@ -1842,6 +1848,7 @@ private void assertIngestionSchema( null, null, null, + null, null ), expectedSegmentGranularity, @@ -2155,11 +2162,11 @@ public SettableColumnValueSelector makeNewSettableColumnValueSelector() } - /** + *//** * The compaction task spec in 0.16.0 except for the tuningConfig. * The original spec accepts only {@link IndexTuningConfig}, but this class acceps any type of tuningConfig for * testing. - */ + *//* private static class OldCompactionTaskWithAnyTuningConfigType extends AbstractTask { private final Interval interval; @@ -2334,5 +2341,5 @@ public ColumnType getColumnType() { return ColumnType.ofComplex(ExtensionDimensionHandler.TYPE_NAME); } - } + }*/ } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTuningConfigTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTuningConfigTest.java index 6831ea4adb60..d36ea8d13816 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTuningConfigTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTuningConfigTest.java @@ -103,6 +103,7 @@ public void testSerdeWithNonZeroAwaitSegmentAvailabilityTimeoutMillis() null, null, 5L, + null, null ); } @@ -146,6 +147,7 @@ public void testSerdeWithZeroAwaitSegmentAvailabilityTimeoutMillis() null, null, 0L, + null, null ); Assert.assertEquals(0L, tuningConfig.getAwaitSegmentAvailabilityTimeoutMillis()); @@ -190,6 +192,7 @@ public void testSerdeWithNullAwaitSegmentAvailabilityTimeoutMillis() null, null, null, + null, null ); Assert.assertEquals(0L, tuningConfig.getAwaitSegmentAvailabilityTimeoutMillis()); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java index 43253a10bccc..693b21459666 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -1020,7 +1020,8 @@ public void close() null, null, "1", - CentralizedDatasourceSchemaConfig.create() + CentralizedDatasourceSchemaConfig.create(), + null ); return toolboxFactory.build(task); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java index 350bc745036b..757409e74aa9 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java @@ -137,7 +137,8 @@ public void setup() throws IOException null, null, "1", - CentralizedDatasourceSchemaConfig.create() + CentralizedDatasourceSchemaConfig.create(), + null ); runner = new SingleTaskBackgroundRunner( toolboxFactory, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java index 852ea0df02dc..6b6278af9ef0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java @@ -666,7 +666,8 @@ public void announceSegment(DataSegment segment) null, null, "1", - CentralizedDatasourceSchemaConfig.create() + CentralizedDatasourceSchemaConfig.create(), + null ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskToolboxFactory.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskToolboxFactory.java index 6082664d36bb..8f658d104bf8 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskToolboxFactory.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskToolboxFactory.java @@ -36,6 +36,7 @@ import org.apache.druid.indexing.common.actions.TaskActionClientFactory; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.config.TaskConfigBuilder; +import org.apache.druid.indexing.common.task.CompactionToMSQ; import org.apache.druid.indexing.common.task.NoopTestTaskReportFileWriter; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTaskClientProvider; import org.apache.druid.indexing.common.task.batch.parallel.ShuffleClient; @@ -117,7 +118,8 @@ public TestTaskToolboxFactory( bob.shuffleClient, bob.taskLogPusher, bob.attemptId, - bob.centralizedDatasourceSchemaConfig + bob.centralizedDatasourceSchemaConfig, + bob.compactionToMSQ ); } @@ -162,6 +164,7 @@ public static class Builder private TaskLogPusher taskLogPusher; private String attemptId; private CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; + private CompactionToMSQ compactionToMSQ; public Builder setConfig(TaskConfig config) { @@ -391,9 +394,15 @@ public Builder setAttemptId(String attemptId) return this; } - public void setCentralizedTableSchemaConfig(CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig) + public Builder setCentralizedTableSchemaConfig(CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig) { this.centralizedDatasourceSchemaConfig = centralizedDatasourceSchemaConfig; + return this; + } + + public Builder setCompactionToMSQ(CompactionToMSQ compactionToMSQ){ + this.compactionToMSQ = compactionToMSQ; + return this; } } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java index 2db5da143edd..3a8a3f43444c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java @@ -704,7 +704,8 @@ public void close() null, null, "1", - CentralizedDatasourceSchemaConfig.create() + CentralizedDatasourceSchemaConfig.create(), + null ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java index 1ccc956ca03f..cef1ed414f1a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java @@ -168,7 +168,8 @@ private WorkerTaskManager createWorkerTaskManager() null, null, "1", - CentralizedDatasourceSchemaConfig.create() + CentralizedDatasourceSchemaConfig.create(), + null ), taskConfig, location diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java index 4e1a801979c5..3ac280fec20e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java @@ -210,7 +210,8 @@ private WorkerTaskMonitor createTaskMonitor() null, null, "1", - CentralizedDatasourceSchemaConfig.create() + CentralizedDatasourceSchemaConfig.create(), + null ), taskConfig, new NoopServiceEmitter(), diff --git a/processing/src/main/java/org/apache/druid/indexer/partitions/SecondaryPartitionType.java b/processing/src/main/java/org/apache/druid/indexer/partitions/SecondaryPartitionType.java index a8f1a1f85396..45a38c65815e 100644 --- a/processing/src/main/java/org/apache/druid/indexer/partitions/SecondaryPartitionType.java +++ b/processing/src/main/java/org/apache/druid/indexer/partitions/SecondaryPartitionType.java @@ -43,8 +43,8 @@ public enum SecondaryPartitionType /** * Range partitioning partitions segments in the same time chunk based on the value range of the partition dimension. * - * @see SingleDimensionPartitionsSpec - * @see org.apache.druid.timeline.partition.SingleDimensionShardSpec + * @see DimensionRangePartitionsSpec + * @see org.apache.druid.timeline.partition.DimensionRangeShardSpec */ RANGE } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java index 0be4ed211aab..a78e085fc9ed 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java @@ -57,7 +57,6 @@ import org.apache.druid.server.coordinator.duty.BalanceSegments; import org.apache.druid.server.coordinator.duty.CollectSegmentAndServerStats; import org.apache.druid.server.coordinator.duty.CompactSegments; -import org.apache.druid.server.coordinator.duty.CompactionClient; import org.apache.druid.server.coordinator.duty.CoordinatorCustomDutyGroup; import org.apache.druid.server.coordinator.duty.CoordinatorCustomDutyGroups; import org.apache.druid.server.coordinator.duty.CoordinatorDuty; @@ -138,7 +137,6 @@ public class DruidCoordinator private final ServiceEmitter emitter; private final OverlordClient overlordClient; - private final CompactionClient compactionClient; private final ScheduledExecutorFactory executorFactory; private final Map dutyGroupExecutors = new HashMap<>(); private final LoadQueueTaskMaster taskMaster; @@ -187,8 +185,7 @@ public DruidCoordinator( BalancerStrategyFactory balancerStrategyFactory, LookupCoordinatorManager lookupCoordinatorManager, @Coordinator DruidLeaderSelector coordLeaderSelector, - CompactionSegmentSearchPolicy compactionSegmentSearchPolicy, - CompactionClient compactionClient + CompactionSegmentSearchPolicy compactionSegmentSearchPolicy ) { this.config = config; @@ -196,7 +193,6 @@ public DruidCoordinator( this.serverInventoryView = serverInventoryView; this.emitter = emitter; this.overlordClient = overlordClient; - this.compactionClient = compactionClient; this.taskMaster = taskMaster; this.serviceAnnouncer = serviceAnnouncer; this.self = self; @@ -583,7 +579,7 @@ CompactSegments initializeCompactSegmentsDuty(CompactionSegmentSearchPolicy comp { List compactSegmentsDutyFromCustomGroups = getCompactSegmentsDutyFromCustomGroups(); if (compactSegmentsDutyFromCustomGroups.isEmpty()) { - return new CompactSegments(compactionSegmentSearchPolicy, overlordClient, compactionClient); + return new CompactSegments(compactionSegmentSearchPolicy, overlordClient); } else { if (compactSegmentsDutyFromCustomGroups.size() > 1) { log.warn( diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index 17e5f4ec1a9b..bb88b86dbf84 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java @@ -94,31 +94,15 @@ public class CompactSegments implements CoordinatorCustomDuty // read by HTTP threads processing Coordinator API calls. private final AtomicReference> autoCompactionSnapshotPerDataSource = new AtomicReference<>(); - private final CompactionClient compactionClient; - - public CompactSegments( - @JacksonInject CompactionSegmentSearchPolicy policy, - @JacksonInject OverlordClient overlordClient - ) - { - this.policy = policy; - this.overlordClient = overlordClient; - this.compactionClient = null; - resetCompactionSnapshot(); - } - @Inject @JsonCreator public CompactSegments( @JacksonInject CompactionSegmentSearchPolicy policy, - @JacksonInject OverlordClient overlordClient, - @JacksonInject CompactionClient compactionClient + @JacksonInject OverlordClient overlordClient ) { this.policy = policy; this.overlordClient = overlordClient; - this.compactionClient = compactionClient; - this.compactionClient.setOverlordClient(overlordClient); resetCompactionSnapshot(); } @@ -664,7 +648,7 @@ private String compactSegments( final String taskId = IdUtils.newTaskId(idPrefix, ClientCompactionTaskQuery.TYPE, dataSource, null); final Granularity segmentGranularity = granularitySpec == null ? null : granularitySpec.getSegmentGranularity(); - final ClientCompactionTaskQuery taskPayload = new ClientCompactionTaskQuery( + final ClientTaskQuery taskPayload = new ClientCompactionTaskQuery( taskId, dataSource, new ClientCompactionIOConfig( @@ -678,8 +662,7 @@ private String compactSegments( transformSpec, context ); -// FutureUtils.getUnchecked(overlordClient.runTask(taskId, taskPayload), true); - compactionClient.submitCompactionTask(taskPayload); + FutureUtils.getUnchecked(overlordClient.runTask(taskId, taskPayload), true); return taskId; } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionClient.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionClient.java deleted file mode 100644 index c29dc6cf2d90..000000000000 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionClient.java +++ /dev/null @@ -1,29 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.server.coordinator.duty; - -import org.apache.druid.client.indexing.ClientCompactionTaskQuery; -import org.apache.druid.rpc.indexing.OverlordClient; - -public interface CompactionClient -{ - String submitCompactionTask(ClientCompactionTaskQuery compactionParams); - void setOverlordClient(OverlordClient overlordClient); -} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionClientNative.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionClientNative.java deleted file mode 100644 index b7c6673c8fe0..000000000000 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionClientNative.java +++ /dev/null @@ -1,25 +0,0 @@ -package org.apache.druid.server.coordinator.duty; - -import com.fasterxml.jackson.annotation.JacksonInject; -import org.apache.druid.client.indexing.ClientCompactionTaskQuery; -import org.apache.druid.common.guava.FutureUtils; -import org.apache.druid.rpc.indexing.OverlordClient; - -public class CompactionClientNative implements CompactionClient -{ - @JacksonInject - OverlordClient overlordClient; - @Override - public String submitCompactionTask(ClientCompactionTaskQuery taskPayload) - { - - FutureUtils.getUnchecked(overlordClient.runTask(taskPayload.getId(), taskPayload), true); - return taskPayload.getId(); - } - - @Override - public void setOverlordClient(OverlordClient overlordClient) - { - this.overlordClient = overlordClient; - } -} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionClientProvider.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionClientProvider.java deleted file mode 100644 index ae6c8c481d3f..000000000000 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionClientProvider.java +++ /dev/null @@ -1,10 +0,0 @@ -package org.apache.druid.server.coordinator.duty; - - -import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.google.inject.Provider; - -//@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "compactionType") -public interface CompactionClientProvider extends Provider -{ -} diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java index 6dc9ced4b858..63ce8f234168 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java @@ -180,7 +180,7 @@ public void setUp() throws Exception new CostBalancerStrategyFactory(), EasyMock.createNiceMock(LookupCoordinatorManager.class), new TestDruidLeaderSelector(), - null, null + null ); } @@ -606,7 +606,7 @@ public void testCompactSegmentsDutyWhenCustomDutyGroupEmpty() new CostBalancerStrategyFactory(), EasyMock.createNiceMock(LookupCoordinatorManager.class), new TestDruidLeaderSelector(), - null, null + null ); // Since CompactSegments is not enabled in Custom Duty Group, then CompactSegments must be created in IndexingServiceDuties List indexingDuties = coordinator.makeIndexingServiceDuties(); @@ -645,7 +645,7 @@ public void testInitializeCompactSegmentsDutyWhenCustomDutyGroupDoesNotContainsC new CostBalancerStrategyFactory(), EasyMock.createNiceMock(LookupCoordinatorManager.class), new TestDruidLeaderSelector(), - null, null + null ); // Since CompactSegments is not enabled in Custom Duty Group, then CompactSegments must be created in IndexingServiceDuties List indexingDuties = coordinator.makeIndexingServiceDuties(); @@ -684,7 +684,7 @@ public void testInitializeCompactSegmentsDutyWhenCustomDutyGroupContainsCompactS new CostBalancerStrategyFactory(), EasyMock.createNiceMock(LookupCoordinatorManager.class), new TestDruidLeaderSelector(), - null, null + null ); // Since CompactSegments is enabled in Custom Duty Group, then CompactSegments must not be created in IndexingServiceDuties List indexingDuties = coordinator.makeIndexingServiceDuties(); @@ -788,7 +788,7 @@ public void testCoordinatorCustomDutyGroupsRunAsExpected() throws Exception new CostBalancerStrategyFactory(), EasyMock.createNiceMock(LookupCoordinatorManager.class), new TestDruidLeaderSelector(), - null, null + null ); coordinator.start(); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java index 7d811a367a48..f341ebf5df99 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java @@ -208,7 +208,7 @@ public CoordinatorSimulation build() createBalancerStrategy(env), env.lookupCoordinatorManager, env.leaderSelector, - COMPACTION_SEGMENT_SEARCH_POLICY, null + COMPACTION_SEGMENT_SEARCH_POLICY ); return new SimulationImpl(coordinator, env); From 9b9128f88f44be5af6636dc2420e81757608a3b1 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Mon, 8 Apr 2024 16:32:06 +0530 Subject: [PATCH 07/54] Add engine field compaction config validation --- .../NewestSegmentFirstPolicyBenchmark.java | 1 + .../msq/compaction/CompactionToMSQImpl.java | 11 ++-- .../indexing/common/task/CompactionTask.java | 2 +- .../druid/testing/utils/CompactionUtil.java | 5 ++ .../duty/ITAutoCompactionTest.java | 1 + .../duty/ITAutoCompactionUpgradeTest.java | 1 + .../CoordinatorCompactionConfig.java | 39 +++++++----- .../DataSourceCompactionConfig.java | 61 ++++++++++++++++++- .../CoordinatorCompactionConfigsResource.java | 23 ++++++- .../DataSourceCompactionConfigTest.java | 42 ++++++++----- .../NewestSegmentFirstIteratorTest.java | 9 +++ .../compact/NewestSegmentFirstPolicyTest.java | 1 + .../coordinator/duty/CompactSegmentsTest.java | 19 +++++- .../duty/KillCompactionConfigTest.java | 9 ++- ...rdinatorCompactionConfigsResourceTest.java | 15 +++-- 15 files changed, 193 insertions(+), 46 deletions(-) diff --git a/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java b/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java index 52a6e0d97570..37f7b665927c 100644 --- a/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java @@ -104,6 +104,7 @@ public void setup() null, null, null, + null, null ) ); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQImpl.java index d9f4480daa48..c3d6ee9200de 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQImpl.java @@ -7,6 +7,7 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; +import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexer.partitions.SecondaryPartitionType; import org.apache.druid.indexing.common.TaskToolbox; @@ -196,13 +197,13 @@ public TaskStatus createAndRunMSQControllerTasks( ); - PartitionsSpec partitionsSpec = compactionTask.getTuningConfig().getPartitionsSpec(); - Integer rowsPerSegment = null; + if (compactionTask.getTuningConfig() != null){ + PartitionsSpec partitionsSpec = compactionTask.getTuningConfig().getPartitionsSpec(); - if (partitionsSpec instanceof DimensionRangePartitionsSpec) { - rowsPerSegment = ((DimensionRangePartitionsSpec) partitionsSpec).getTargetRowsPerSegment(); - + if (partitionsSpec instanceof DimensionRangePartitionsSpec) { + rowsPerSegment = ((DimensionRangePartitionsSpec) partitionsSpec).getTargetRowsPerSegment(); + } } MSQTuningConfig msqTuningConfig = new MSQTuningConfig( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 690c77268c57..7449842c129d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -684,7 +684,7 @@ static List> createDataSchemasForIntervals( : granularitySpec.withSegmentGranularity(segmentGranularityToUse) ); - // TODO: Based on a function flag, either create parallel index spec OR just aggregate up dataSchemas. We should + // TODO(vishesh): Based on a function flag, either create parallel index spec OR just aggregate up dataSchemas. We should // pass a list of schemas to MSQ to have one-to-one mapping between an MSQ controller task and a schema dataSchemas.add(Pair.of(interval, dataSchema)); } diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/CompactionUtil.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/CompactionUtil.java index a57a404b61ea..a51f5122bcd9 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/CompactionUtil.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/CompactionUtil.java @@ -20,7 +20,11 @@ package org.apache.druid.testing.utils; import org.apache.druid.data.input.MaxSizeSplitHintSpec; +import org.apache.druid.error.InvalidInput; +import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; +import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.UserCompactionTaskIOConfig; import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; @@ -75,6 +79,7 @@ public static DataSourceCompactionConfig createCompactionConfig( null, null, new UserCompactionTaskIOConfig(true), + null, null ); } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java index 26df03e0d81f..3abcb4af9822 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java @@ -1648,6 +1648,7 @@ private void submitCompactionConfig( metricsSpec, transformSpec, !dropExisting ? null : new UserCompactionTaskIOConfig(true), + null, null ); compactionResource.submitCompactionConfig(compactionConfig); diff --git a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionUpgradeTest.java b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionUpgradeTest.java index 9c32cc5055f3..9e1b54143bbd 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionUpgradeTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionUpgradeTest.java @@ -103,6 +103,7 @@ public void testUpgradeAutoCompactionConfigurationWhenConfigurationFromOlderVers null, null, new UserCompactionTaskIOConfig(true), + null, null ); compactionResource.submitCompactionConfig(compactionConfig); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfig.java index 368917b1e8d6..829298eb3aab 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfig.java @@ -34,11 +34,13 @@ public class CoordinatorCompactionConfig private static final double DEFAULT_COMPACTION_TASK_RATIO = 0.1; private static final int DEFAILT_MAX_COMPACTION_TASK_SLOTS = Integer.MAX_VALUE; private static final boolean DEFAULT_USE_AUTO_SCALE_SLOTS = false; + private static final DataSourceCompactionConfig.Engine DEFAULT_COMPACTION_ENGINE = DataSourceCompactionConfig.Engine.NATIVE; private final List compactionConfigs; private final double compactionTaskSlotRatio; private final int maxCompactionTaskSlots; private final boolean useAutoScaleSlots; + private final DataSourceCompactionConfig.Engine engine; public static CoordinatorCompactionConfig from( CoordinatorCompactionConfig baseConfig, @@ -49,7 +51,8 @@ public static CoordinatorCompactionConfig from( compactionConfigs, baseConfig.compactionTaskSlotRatio, baseConfig.maxCompactionTaskSlots, - baseConfig.useAutoScaleSlots + baseConfig.useAutoScaleSlots, + baseConfig.engine ); } @@ -57,25 +60,27 @@ public static CoordinatorCompactionConfig from( CoordinatorCompactionConfig baseConfig, @Nullable Double compactionTaskSlotRatio, @Nullable Integer maxCompactionTaskSlots, - @Nullable Boolean useAutoScaleSlots + @Nullable Boolean useAutoScaleSlots, + @Nullable DataSourceCompactionConfig.Engine engine ) { return new CoordinatorCompactionConfig( baseConfig.compactionConfigs, compactionTaskSlotRatio == null ? baseConfig.compactionTaskSlotRatio : compactionTaskSlotRatio, maxCompactionTaskSlots == null ? baseConfig.maxCompactionTaskSlots : maxCompactionTaskSlots, - useAutoScaleSlots == null ? baseConfig.useAutoScaleSlots : useAutoScaleSlots + useAutoScaleSlots == null ? baseConfig.useAutoScaleSlots : useAutoScaleSlots, + engine == null ? baseConfig.engine : engine ); } public static CoordinatorCompactionConfig from(List compactionConfigs) { - return new CoordinatorCompactionConfig(compactionConfigs, null, null, null); + return new CoordinatorCompactionConfig(compactionConfigs, null, null, null, null); } public static CoordinatorCompactionConfig empty() { - return new CoordinatorCompactionConfig(ImmutableList.of(), null, null, null); + return new CoordinatorCompactionConfig(ImmutableList.of(), null, null, null, null); } @JsonCreator @@ -83,19 +88,17 @@ public CoordinatorCompactionConfig( @JsonProperty("compactionConfigs") List compactionConfigs, @JsonProperty("compactionTaskSlotRatio") @Nullable Double compactionTaskSlotRatio, @JsonProperty("maxCompactionTaskSlots") @Nullable Integer maxCompactionTaskSlots, - @JsonProperty("useAutoScaleSlots") @Nullable Boolean useAutoScaleSlots + @JsonProperty("useAutoScaleSlots") @Nullable Boolean useAutoScaleSlots, + @JsonProperty("engine") @Nullable final DataSourceCompactionConfig.Engine engine ) { this.compactionConfigs = compactionConfigs; - this.compactionTaskSlotRatio = compactionTaskSlotRatio == null ? - DEFAULT_COMPACTION_TASK_RATIO : + this.compactionTaskSlotRatio = compactionTaskSlotRatio == null ? DEFAULT_COMPACTION_TASK_RATIO : compactionTaskSlotRatio; - this.maxCompactionTaskSlots = maxCompactionTaskSlots == null ? - DEFAILT_MAX_COMPACTION_TASK_SLOTS : + this.maxCompactionTaskSlots = maxCompactionTaskSlots == null ? DEFAILT_MAX_COMPACTION_TASK_SLOTS : maxCompactionTaskSlots; - this.useAutoScaleSlots = useAutoScaleSlots == null ? - DEFAULT_USE_AUTO_SCALE_SLOTS : - useAutoScaleSlots; + this.useAutoScaleSlots = useAutoScaleSlots == null ? DEFAULT_USE_AUTO_SCALE_SLOTS : useAutoScaleSlots; + this.engine = engine == null ? DEFAULT_COMPACTION_ENGINE : engine; } @JsonProperty @@ -122,6 +125,12 @@ public boolean isUseAutoScaleSlots() return useAutoScaleSlots; } + @JsonProperty + public DataSourceCompactionConfig.Engine getEngine() + { + return engine; + } + @Override public boolean equals(Object o) { @@ -135,13 +144,14 @@ public boolean equals(Object o) return Double.compare(that.compactionTaskSlotRatio, compactionTaskSlotRatio) == 0 && maxCompactionTaskSlots == that.maxCompactionTaskSlots && useAutoScaleSlots == that.useAutoScaleSlots && + engine == that.engine && Objects.equals(compactionConfigs, that.compactionConfigs); } @Override public int hashCode() { - return Objects.hash(compactionConfigs, compactionTaskSlotRatio, maxCompactionTaskSlots, useAutoScaleSlots); + return Objects.hash(compactionConfigs, compactionTaskSlotRatio, maxCompactionTaskSlots, useAutoScaleSlots, engine); } @Override @@ -152,6 +162,7 @@ public String toString() ", compactionTaskSlotRatio=" + compactionTaskSlotRatio + ", maxCompactionTaskSlots=" + maxCompactionTaskSlots + ", useAutoScaleSlots=" + useAutoScaleSlots + + ", engine=" + engine + '}'; } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java index da89040f50a4..115a42bb92e1 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java @@ -22,6 +22,10 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import org.apache.druid.error.InvalidInput; +import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; +import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.aggregation.AggregatorFactory; import org.joda.time.Period; @@ -55,6 +59,7 @@ public class DataSourceCompactionConfig private final UserCompactionTaskTransformConfig transformSpec; private final UserCompactionTaskIOConfig ioConfig; private final Map taskContext; + private Engine engine; @JsonCreator public DataSourceCompactionConfig( @@ -69,7 +74,8 @@ public DataSourceCompactionConfig( @JsonProperty("metricsSpec") @Nullable AggregatorFactory[] metricsSpec, @JsonProperty("transformSpec") @Nullable UserCompactionTaskTransformConfig transformSpec, @JsonProperty("ioConfig") @Nullable UserCompactionTaskIOConfig ioConfig, - @JsonProperty("taskContext") @Nullable Map taskContext + @JsonProperty("taskContext") @Nullable Map taskContext, + @JsonProperty("engine") @Nullable Engine engine ) { this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource"); @@ -88,6 +94,7 @@ public DataSourceCompactionConfig( this.dimensionsSpec = dimensionsSpec; this.transformSpec = transformSpec; this.taskContext = taskContext; + this.engine = engine; } @JsonProperty @@ -171,6 +178,13 @@ public Map getTaskContext() return taskContext; } + @JsonProperty + @Nullable + public Engine getEngine() + { + return engine; + } + @Override public boolean equals(Object o) { @@ -192,6 +206,7 @@ public boolean equals(Object o) Arrays.equals(metricsSpec, that.metricsSpec) && Objects.equals(transformSpec, that.transformSpec) && Objects.equals(ioConfig, that.ioConfig) && + Objects.equals(engine, that.engine) && Objects.equals(taskContext, that.taskContext); } @@ -209,9 +224,51 @@ public int hashCode() dimensionsSpec, transformSpec, ioConfig, - taskContext + taskContext, + engine ); result = 31 * result + Arrays.hashCode(metricsSpec); return result; } + + public void updateEngineAndValidate(Engine defaultEngine) + { + boolean usingDefault = false; + if (engine == null) { + engine = defaultEngine; + usingDefault = true; + } + if (engine == Engine.MSQ) { + if (tuningConfig != null) { + PartitionsSpec partitionsSpec = tuningConfig.getPartitionsSpec(); + if (partitionsSpec != null && !(partitionsSpec instanceof DimensionRangePartitionsSpec)) { + throw InvalidInput.exception( + "Invalid partition spec type[%s] for MSQ compaction engine %s. Type must be DynamicRangePartitionSpec.", + partitionsSpec.getClass(), usingDefault ? "set as default" : "specified in spec" + ); + } + } + if (maxRowsPerSegment != null) { + throw InvalidInput.exception( + "MaxRowsPerSegment[%d] field not supported for MSQ compaction engine.", + maxRowsPerSegment + ); + } + } + } + + public enum Engine + { + NATIVE, + MSQ; + + @JsonCreator + public static Engine fromString(String name) + { + if (name == null) { + return null; + } + return valueOf(StringUtils.toUpperCase(name)); + } + } } diff --git a/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java index 57833506f093..9254314311d7 100644 --- a/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java +++ b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java @@ -101,7 +101,27 @@ public Response setCompactionTaskLimit( current, compactionTaskSlotRatio, maxCompactionTaskSlots, - useAutoScaleSlots + useAutoScaleSlots, + null + ); + return updateConfigHelper(operator, AuthorizationUtils.buildAuditInfo(req)); + } + + @POST + @Path("/engine") + @Consumes(MediaType.APPLICATION_JSON) + public Response setCompactionEngine( + @QueryParam("engine") DataSourceCompactionConfig.Engine engine, + @Context HttpServletRequest req + ) + { + UnaryOperator operator = + current -> CoordinatorCompactionConfig.from( + current, + null, + null, + null, + engine ); return updateConfigHelper(operator, AuthorizationUtils.buildAuditInfo(req)); } @@ -119,6 +139,7 @@ public Response addOrUpdateCompactionConfig( .getCompactionConfigs() .stream() .collect(Collectors.toMap(DataSourceCompactionConfig::getDataSource, Function.identity())); + newConfig.updateEngineAndValidate(current.getEngine()); newConfigs.put(newConfig.getDataSource(), newConfig); newCompactionConfig = CoordinatorCompactionConfig.from(current, ImmutableList.copyOf(newConfigs.values())); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigTest.java index e3f8c01c3dcb..6eb4e91db306 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigTest.java @@ -69,7 +69,8 @@ public void testSerdeBasic() throws IOException null, null, null, - ImmutableMap.of("key", "val") + ImmutableMap.of("key", "val"), + null ); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -99,7 +100,8 @@ public void testSerdeWithMaxRowsPerSegment() throws IOException null, null, null, - ImmutableMap.of("key", "val") + ImmutableMap.of("key", "val"), + null ); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -148,7 +150,8 @@ public void testSerdeWithMaxTotalRows() throws IOException null, null, null, - ImmutableMap.of("key", "val") + ImmutableMap.of("key", "val"), + null ); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -197,7 +200,8 @@ public void testSerdeMaxTotalRowsWithMaxRowsPerSegment() throws IOException null, null, null, - ImmutableMap.of("key", "val") + ImmutableMap.of("key", "val"), + null ); final String json = OBJECT_MAPPER.writeValueAsString(config); @@ -305,7 +309,8 @@ public void testSerdeGranularitySpec() throws IOException null, null, null, - ImmutableMap.of("key", "val") + ImmutableMap.of("key", "val"), + null ); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -335,7 +340,8 @@ public void testSerdeGranularitySpecWithQueryGranularity() throws Exception null, null, null, - ImmutableMap.of("key", "val") + ImmutableMap.of("key", "val"), + null ); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -368,7 +374,8 @@ public void testSerdeWithNullGranularitySpec() throws IOException null, null, null, - ImmutableMap.of("key", "val") + ImmutableMap.of("key", "val"), + null ); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -398,7 +405,8 @@ public void testSerdeGranularitySpecWithNullValues() throws IOException null, null, null, - ImmutableMap.of("key", "val") + ImmutableMap.of("key", "val"), + null ); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -428,7 +436,8 @@ public void testSerdeGranularitySpecWithRollup() throws IOException null, null, null, - ImmutableMap.of("key", "val") + ImmutableMap.of("key", "val"), + null ); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -461,7 +470,8 @@ public void testSerdeIOConfigWithNonNullDropExisting() throws IOException null, null, new UserCompactionTaskIOConfig(true), - ImmutableMap.of("key", "val") + ImmutableMap.of("key", "val"), + null ); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -492,7 +502,8 @@ public void testSerdeIOConfigWithNullDropExisting() throws IOException null, null, new UserCompactionTaskIOConfig(null), - ImmutableMap.of("key", "val") + ImmutableMap.of("key", "val"), + null ); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -523,7 +534,8 @@ public void testSerdeDimensionsSpec() throws IOException null, null, null, - ImmutableMap.of("key", "val") + ImmutableMap.of("key", "val"), + null ); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -554,7 +566,8 @@ public void testSerdeTransformSpec() throws IOException null, new UserCompactionTaskTransformConfig(new SelectorDimFilter("dim1", "foo", null)), null, - ImmutableMap.of("key", "val") + ImmutableMap.of("key", "val"), + null ); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -584,7 +597,8 @@ public void testSerdeMetricsSpec() throws IOException new AggregatorFactory[] {new CountAggregatorFactory("cnt")}, null, null, - ImmutableMap.of("key", "val") + ImmutableMap.of("key", "val"), + null ); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstIteratorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstIteratorTest.java index b4ea5d69e003..9c96e6fcdd8b 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstIteratorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstIteratorTest.java @@ -96,6 +96,7 @@ public void testFindPartitionsSpecFromConfigWithNullTuningConfigReturnDynamicPar null, null, null, + null, null ); Assert.assertEquals( @@ -141,6 +142,7 @@ public void testFindPartitionsSpecFromConfigWithNullMaxTotalRowsReturnLongMaxVal null, null, null, + null, null ); Assert.assertEquals( @@ -186,6 +188,7 @@ public void testFindPartitionsSpecFromConfigWithNonNullMaxTotalRowsReturnGivenVa null, null, null, + null, null ); Assert.assertEquals( @@ -231,6 +234,7 @@ public void testFindPartitionsSpecFromConfigWithNonNullMaxRowsPerSegmentReturnGi null, null, null, + null, null ); Assert.assertEquals( @@ -276,6 +280,7 @@ public void testFindPartitionsSpecFromConfigWithDeprecatedMaxRowsPerSegmentAndMa null, null, null, + null, null ); Assert.assertEquals( @@ -321,6 +326,7 @@ public void testFindPartitionsSpecFromConfigWithDeprecatedMaxRowsPerSegmentAndPa null, null, null, + null, null ); Assert.assertEquals( @@ -366,6 +372,7 @@ public void testFindPartitionsSpecFromConfigWithDeprecatedMaxTotalRowsAndPartiti null, null, null, + null, null ); Assert.assertEquals( @@ -411,6 +418,7 @@ public void testFindPartitionsSpecFromConfigWithHashPartitionsSpec() null, null, null, + null, null ); Assert.assertEquals( @@ -456,6 +464,7 @@ public void testFindPartitionsSpecFromConfigWithRangePartitionsSpec() null, null, null, + null, null ); Assert.assertEquals( diff --git a/server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicyTest.java b/server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicyTest.java index dda1cb1af137..31e269f50f41 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicyTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicyTest.java @@ -2023,6 +2023,7 @@ private DataSourceCompactionConfig createCompactionConfig( metricsSpec, transformSpec, null, + null, null ); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java index 43b1c50c9698..787e5b6ea22d 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java @@ -712,6 +712,7 @@ public void testCompactWithoutGranularitySpec() null, null, null, + null, null ) ); @@ -769,6 +770,7 @@ public void testCompactWithNotNullIOConfig() null, null, new UserCompactionTaskIOConfig(true), + null, null ) ); @@ -818,6 +820,7 @@ public void testCompactWithNullIOConfig() null, null, null, + null, null ) ); @@ -867,6 +870,7 @@ public void testCompactWithGranularitySpec() null, null, null, + null, null ) ); @@ -927,6 +931,7 @@ public void testCompactWithDimensionSpec() null, null, null, + null, null ) ); @@ -979,6 +984,7 @@ public void testCompactWithoutDimensionSpec() null, null, null, + null, null ) ); @@ -1028,6 +1034,7 @@ public void testCompactWithRollupInGranularitySpec() null, null, null, + null, null ) ); @@ -1137,6 +1144,7 @@ public void testCompactWithGranularitySpecConflictWithActiveCompactionTask() null, null, null, + null, null ) ); @@ -1252,6 +1260,7 @@ public void testCompactWithTransformSpec() null, new UserCompactionTaskTransformConfig(new SelectorDimFilter("dim1", "foo", null)), null, + null, null ) ); @@ -1302,6 +1311,7 @@ public void testCompactWithoutCustomSpecs() null, null, null, + null, null ) ); @@ -1354,6 +1364,7 @@ public void testCompactWithMetricsSpec() aggregatorFactories, null, null, + null, null ) ); @@ -1434,6 +1445,7 @@ public void testDetermineSegmentGranularityFromSegmentsToCompact() null, null, null, + null, null ) ); @@ -1520,6 +1532,7 @@ public void testDetermineSegmentGranularityFromSegmentGranularityInCompactionCon null, null, null, + null, null ) ); @@ -1577,6 +1590,7 @@ public void testCompactWithMetricsSpecShouldSetPreserveExistingMetricsTrue() new AggregatorFactory[] {new CountAggregatorFactory("cnt")}, null, null, + null, null ) ); @@ -1629,6 +1643,7 @@ public void testCompactWithoutMetricsSpecShouldSetPreserveExistingMetricsFalse() null, null, null, + null, null ) ); @@ -1781,7 +1796,8 @@ private CoordinatorRunStats doCompactSegments( compactionConfigs, numCompactionTaskSlots == null ? null : 1.0, // 100% when numCompactionTaskSlots is not null numCompactionTaskSlots, - useAutoScaleSlots + useAutoScaleSlots, + null ) ) .build(); @@ -1942,6 +1958,7 @@ private List createCompactionConfigs(@Nullable Integ null, null, null, + null, null ) ); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/KillCompactionConfigTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/KillCompactionConfigTest.java index d331204525a4..03809389baec 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/KillCompactionConfigTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/KillCompactionConfigTest.java @@ -192,7 +192,8 @@ public void testRunRemoveInactiveDatasourceCompactionConfig() null, null, null, - ImmutableMap.of("key", "val") + ImmutableMap.of("key", "val"), + null ); DataSourceCompactionConfig activeDatasourceConfig = new DataSourceCompactionConfig( @@ -207,7 +208,8 @@ public void testRunRemoveInactiveDatasourceCompactionConfig() null, null, null, - ImmutableMap.of("key", "val") + ImmutableMap.of("key", "val"), + null ); CoordinatorCompactionConfig originalCurrentConfig = CoordinatorCompactionConfig.from(ImmutableList.of(inactiveDatasourceConfig, activeDatasourceConfig)); byte[] originalCurrentConfigBytes = {1, 2, 3}; @@ -293,7 +295,8 @@ public void testRunRetryForRetryableException() null, null, null, - ImmutableMap.of("key", "val") + ImmutableMap.of("key", "val"), + null ); CoordinatorCompactionConfig originalCurrentConfig = CoordinatorCompactionConfig.from(ImmutableList.of(inactiveDatasourceConfig)); diff --git a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java index c31364ac9fd9..6a5cf9b1cc26 100644 --- a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java @@ -62,7 +62,8 @@ public class CoordinatorCompactionConfigsResourceTest null, null, null, - ImmutableMap.of("key", "val") + ImmutableMap.of("key", "val"), + null ); private static final DataSourceCompactionConfig NEW_CONFIG = new DataSourceCompactionConfig( "newDataSource", @@ -76,7 +77,8 @@ public class CoordinatorCompactionConfigsResourceTest null, null, null, - ImmutableMap.of("key", "val") + ImmutableMap.of("key", "val"), + null ); private static final byte[] OLD_CONFIG_IN_BYTES = {1, 2, 3}; @@ -189,7 +191,8 @@ public void testAddOrUpdateCompactionConfigWithExistingConfig() null, null, null, - ImmutableMap.of("key", "val") + ImmutableMap.of("key", "val"), + null ); Response result = coordinatorCompactionConfigsResource.addOrUpdateCompactionConfig( newConfig, @@ -230,7 +233,8 @@ public void testDeleteCompactionConfigWithExistingConfig() null, null, null, - ImmutableMap.of("key", "val") + ImmutableMap.of("key", "val"), + null ); final CoordinatorCompactionConfig originalConfig = CoordinatorCompactionConfig.from(ImmutableList.of(toDelete)); Mockito.when(mockJacksonConfigManager.convertByteToConfig( @@ -388,7 +392,8 @@ public void testAddOrUpdateCompactionConfigWithoutExistingConfig() null, null, null, - ImmutableMap.of("key", "val") + ImmutableMap.of("key", "val"), + null ); String author = "maytas"; String comment = "hello"; From dfaef5d1f270f72be54a1ee1e9b32fcd539e1b96 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Mon, 15 Apr 2024 10:17:01 +0530 Subject: [PATCH 08/54] Compaction to scan query on msq --- .../msq/compaction/CompactionToMSQImpl.java | 164 ++++++++++++------ .../CompactionToMSQImplProvider.java | 15 -- .../msq/guice/MSQCompactionDruidModule.java | 15 +- .../druid/msq/indexing/MSQControllerTask.java | 3 +- .../druid/msq/sql/MSQTaskQueryMaker.java | 1 + .../msq/indexing/MSQControllerTaskTest.java | 1 + .../resources/SqlStatementResourceTest.java | 2 + .../indexing/common/task/CompactionTask.java | 40 +++-- .../druid/indexing/common/task/Engine.java | 39 +++++ .../ClientCompactionTaskQuerySerdeTest.java | 11 +- .../indexing/ClientCompactionTaskQuery.java | 21 ++- .../DataSourceCompactionConfig.java | 65 +++++-- .../coordinator/duty/CompactSegments.java | 31 +++- .../CoordinatorCompactionConfigsResource.java | 5 +- .../coordinator/duty/CompactSegmentsTest.java | 5 +- 15 files changed, 300 insertions(+), 118 deletions(-) delete mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQImplProvider.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/Engine.java diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQImpl.java index c3d6ee9200de..59940546ad10 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQImpl.java @@ -1,13 +1,15 @@ package org.apache.druid.msq.compaction; import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.InvalidInput; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; -import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexer.partitions.SecondaryPartitionType; import org.apache.druid.indexing.common.TaskToolbox; @@ -15,6 +17,7 @@ import org.apache.druid.indexing.common.task.CompactionToMSQ; import org.apache.druid.indexing.common.task.CurrentSubTaskHolder; import org.apache.druid.indexing.common.task.Tasks; +import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.AllGranularity; @@ -26,8 +29,10 @@ import org.apache.druid.msq.indexing.MSQTuningConfig; import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination; import org.apache.druid.msq.indexing.destination.MSQDestination; +import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.Druids; import org.apache.druid.query.Query; +import org.apache.druid.query.QueryContext; import org.apache.druid.query.TableDataSource; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.filter.DimFilter; @@ -35,6 +40,7 @@ import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -47,6 +53,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -56,27 +63,25 @@ public class CompactionToMSQImpl implements CompactionToMSQ private static final Logger log = new Logger(CompactionToMSQImpl.class); private static final Granularity DEFAULT_SEGMENT_GRANULARITY = Granularities.ALL; + final OverlordClient overlordClient; + final ObjectMapper jsonMapper; - public CompactionToMSQImpl() - { - System.out.println("Initializing MSQCompaction"); - } - List getColumns(DimensionsSpec dimensionSpec) + public CompactionToMSQImpl( + final OverlordClient overlordClient, + final ObjectMapper jsonMapper + ) { - - List columns = new ArrayList<>(); - for (DimensionSchema ds : dimensionSpec.getDimensions()) { - columns.add(ds.getName()); - } - return columns; + this.overlordClient = overlordClient; + this.jsonMapper = jsonMapper; + System.out.println("Initializing MSQCompaction"); } - RowSignature getRowSignature(DimensionsSpec dimensionSpec) + RowSignature getRowSignature(DataSchema dataSchema) { RowSignature.Builder rowSignatureBuilder = RowSignature.builder(); - - for (DimensionSchema ds : dimensionSpec.getDimensions()) { + rowSignatureBuilder.add(dataSchema.getTimestampSpec().getTimestampColumn(), ColumnType.LONG); + for (DimensionSchema ds : dataSchema.getDimensionsSpec().getDimensions()) { rowSignatureBuilder.add(ds.getName(), ColumnType.fromString(ds.getTypeName())); } return rowSignatureBuilder.build(); @@ -101,8 +106,10 @@ public List getOrderBySpec(PartitionsSpec partitionSpec) public Query buildScanQuery(CompactionTask compactionTask, Interval interval, DataSchema dataSchema) { + RowSignature rowSignature = getRowSignature(dataSchema); return new Druids.ScanQueryBuilder().dataSource(dataSchema.getDataSource()) - .columns(getColumns(dataSchema.getDimensionsSpec())) + .columns(rowSignature.getColumnNames()) + .columnTypes(rowSignature.getColumnTypes()) .intervals(createMultipleIntervalSpec(interval)) .legacy(false) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) @@ -148,6 +155,7 @@ public TaskStatus createAndRunMSQControllerTasks( { List msqControllerTasks = new ArrayList<>(); + QueryContext sqlQueryContext = new QueryContext(compactionTask.getContext()); for (Pair intervalDataSchema : intervalDataSchemas) { Query query; @@ -172,14 +180,28 @@ public TaskStatus createAndRunMSQControllerTasks( // CTX_SCAN_SIGNATURE is deprecated and should no longer be reqd // compactionTask.getContext().putAll(ImmutableMap.of(DruidQuery.CTX_SCAN_SIGNATURE, // new ObjectMapper().writeValueAsString(getRowSignature(ds.getDimensionsSpec())), - Map context = ImmutableMap.copyOf(compactionTask.getContext()); + Map context = new HashMap<>(compactionTask.getContext()); // TODO (vishesh): check why the default is day in specific classes whereas ALL in query maker - context.put( - DruidSqlInsert.SQL_INSERT_SEGMENT_GRANULARITY, - (ds.getGranularitySpec() != null) - ? ds.getGranularitySpec().getSegmentGranularity() - : DEFAULT_SEGMENT_GRANULARITY - ); + try { + context.put( + DruidSqlInsert.SQL_INSERT_SEGMENT_GRANULARITY, + (ds != null && ds.getGranularitySpec() != null) + ? jsonMapper.writeValueAsString(ds.getGranularitySpec().getSegmentGranularity()) + : jsonMapper.writeValueAsString(DEFAULT_SEGMENT_GRANULARITY + ) + ); + } + catch (JsonProcessingException e) { + throw DruidException.defensive() + .build( + e, + "Unable to deserialize the DEFAULT_SEGMENT_GRANULARITY in MSQTaskQueryMaker. " + + "This shouldn't have happened since the DEFAULT_SEGMENT_GRANULARITY object is guaranteed to be " + + "serializable. Please raise an issue in case you are seeing this message while executing a query." + ); + + } + context.put(DruidSqlReplace.SQL_REPLACE_TIME_CHUNKS, interval.toString()); context.put(Tasks.STORE_COMPACTION_STATE_KEY, true); @@ -197,32 +219,86 @@ public TaskStatus createAndRunMSQControllerTasks( ); +// // TODO (vishesh) : There is some logic related to finalize aggs. Check if there's a case +// // for non-finalize aggs for compaction -- esp. user specified. +// final int maxNumTasks = 2; + /* + For assistance computing return types if !finalizeAggregations. + final Map aggregationIntermediateTypeMap = + finalizeAggregations ? null : buildAggregationIntermediateTypeMap(druidQuery); + + final List sqlTypeNames = new ArrayList<>(); + final List columnTypeList = new ArrayList<>(); + final List columnMappings = QueryUtils.buildColumnMappings(fieldMapping, druidQuery); + + for (final org.apache.calcite.util.Pair entry : fieldMapping) { + final String queryColumn = druidQuery.getOutputRowSignature().getColumnName(entry.getKey()); + + final SqlTypeName sqlTypeName; + + if (!finalizeAggregations && aggregationIntermediateTypeMap.containsKey(queryColumn)) { + final ColumnType druidType = aggregationIntermediateTypeMap.get(queryColumn); + sqlTypeName = new RowSignatures.ComplexSqlType(SqlTypeName.OTHER, druidType, true).getSqlTypeName(); + } else { + sqlTypeName = druidQuery.getOutputRowType().getFieldList().get(entry.getKey()).getType().getSqlTypeName(); + } + sqlTypeNames.add(sqlTypeName); + columnTypeList.add(druidQuery.getOutputRowSignature().getColumnType(queryColumn).orElse(ColumnType.STRING)); + } + */ + + // Pick-up MSQ related context params, if any, from the compaction context itself. + final int maxNumTasks = MultiStageQueryContext.getMaxNumTasks(sqlQueryContext); + if (maxNumTasks < 2) { + throw InvalidInput.exception( + "MSQ context maxNumTasks [%,d] cannot be less than 2, since at least 1 controller and 1 worker is necessary", + maxNumTasks + ); + } + + // This parameter is used internally for the number of worker tasks only, so we subtract 1 + final int maxNumWorkers = maxNumTasks - 1; + final int maxRowsInMemory = MultiStageQueryContext.getRowsInMemory(sqlQueryContext); + final boolean finalizeAggregations = MultiStageQueryContext.isFinalizeAggregations(sqlQueryContext); + + Integer rowsPerSegment = null; if (compactionTask.getTuningConfig() != null){ PartitionsSpec partitionsSpec = compactionTask.getTuningConfig().getPartitionsSpec(); - - if (partitionsSpec instanceof DimensionRangePartitionsSpec) { - rowsPerSegment = ((DimensionRangePartitionsSpec) partitionsSpec).getTargetRowsPerSegment(); + if (partitionsSpec != null) { + rowsPerSegment = partitionsSpec.getMaxRowsPerSegment(); } } MSQTuningConfig msqTuningConfig = new MSQTuningConfig( - null, - null, + maxNumWorkers, + maxRowsInMemory, rowsPerSegment, compactionTask.getTuningConfig().getIndexSpec() ); MSQSpec msqSpec = MSQSpec.builder() .query(query) - .columnMappings(ColumnMappings.identity(getRowSignature(ds.getDimensionsSpec()))) + .columnMappings(ColumnMappings.identity(getRowSignature(ds))) .destination(msqDestination) + // TODO (vishesh): Investiage how this is populated + .assignmentStrategy(MultiStageQueryContext.getAssignmentStrategy(sqlQueryContext)) .tuningConfig(msqTuningConfig) .build(); - MSQControllerTask controllerTask = getMsqControllerTask(compactionTask, msqSpec); - - msqControllerTasks.add(controllerTask); + MSQControllerTask controllerTask = new MSQControllerTask( + compactionTask.getId(), + compactionTask.getId(), + msqSpec.withOverriddenContext(context), + null, + context, + null, + null, + null, + null + ); + MSQControllerTask serdeControllerTask = jsonMapper.readerFor(MSQControllerTask.class).readValue(jsonMapper.writeValueAsString(controllerTask)); + msqControllerTasks.add(serdeControllerTask); } // if (msqControllerTasks.isEmpty()){ // log.warn("Can't find segments from inputSpec[%s], nothing to do.", @@ -243,11 +319,12 @@ private TaskStatus runSubtasks( String compactionTaskId ) throws JsonProcessingException { + final int totalNumSpecs = tasks.size(); log.info("Generated [%d] compaction task specs", totalNumSpecs); int failCnt = 0; -// Map completionReports = new HashMap<>(); + for (MSQControllerTask eachTask : tasks) { final String json = toolbox.getJsonMapper().writerWithDefaultPrettyPrinter().writeValueAsString(eachTask); if (!currentSubTaskHolder.setTask(eachTask)) { @@ -258,6 +335,7 @@ private TaskStatus runSubtasks( try { if (eachTask.isReady(toolbox.getTaskActionClient())) { log.info("Running MSQControllerTask: " + json); + // TODO (vishesh) : Check handling of multi-controller tasks -- to be triggered via overlord final TaskStatus eachResult = eachTask.run(toolbox); if (!eachResult.isSuccess()) { failCnt++; @@ -281,25 +359,7 @@ private TaskStatus runSubtasks( failCnt ); -// toolbox.getTaskReportFileWriter().write(compactionTaskId, completionReports); log.info(msg); return failCnt == 0 ? TaskStatus.success(compactionTaskId) : TaskStatus.failure(compactionTaskId, msg); } - - - private static MSQControllerTask getMsqControllerTask(CompactionTask compactionTask, MSQSpec msqSpec) - { - final String taskId = compactionTask.getId(); - - return new MSQControllerTask( - taskId, - msqSpec, - null, - null, - null, - null, - null, - compactionTask.getContext() - ); - } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQImplProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQImplProvider.java deleted file mode 100644 index 5d720aca8824..000000000000 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQImplProvider.java +++ /dev/null @@ -1,15 +0,0 @@ -package org.apache.druid.msq.compaction; - -import org.apache.druid.indexing.common.task.CompactionToMSQ; -import org.apache.druid.indexing.common.task.CompactionToMSQProvider; - -//@JsonTypeName(MSQCompactionDruidModule.SCHEME) -public class CompactionToMSQImplProvider implements CompactionToMSQProvider -{ -// @JacksonInject -// ClientCompactionTaskQuery compactionTaskQuery; - @Override - public CompactionToMSQ get(){ - return new CompactionToMSQImpl(); - } -} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQCompactionDruidModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQCompactionDruidModule.java index 42ef45783b5a..33da0106961b 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQCompactionDruidModule.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQCompactionDruidModule.java @@ -1,11 +1,14 @@ package org.apache.druid.msq.guice; import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Binder; +import com.google.inject.Provides; import org.apache.druid.guice.LazySingleton; import org.apache.druid.indexing.common.task.CompactionToMSQ; import org.apache.druid.initialization.DruidModule; -import org.apache.druid.msq.compaction.CompactionToMSQImplProvider; +import org.apache.druid.msq.compaction.CompactionToMSQImpl; +import org.apache.druid.rpc.indexing.OverlordClient; import java.util.Collections; import java.util.List; @@ -18,7 +21,8 @@ public class MSQCompactionDruidModule implements DruidModule @Override public void configure(Binder binder) { - binder.bind(CompactionToMSQ.class).toProvider(CompactionToMSQImplProvider.class).in(LazySingleton.class); +// binder.bind(CompactionToMSQ.class).toProvider(CompactionToMSQImplProvider.class).in(LazySingleton.class); + binder.bind(CompactionToMSQ.class).to(CompactionToMSQImpl.class).in(LazySingleton.class); } @Override @@ -27,4 +31,11 @@ public List getJacksonModules() return Collections.emptyList(); } + @Provides + @LazySingleton + CompactionToMSQImpl getCompactionToMSQImpl(final OverlordClient overlordClient, + final ObjectMapper jsonMapper){ + return new CompactionToMSQImpl(overlordClient, jsonMapper); + } + } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java index 7eb455ca8424..eff98cf1395a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java @@ -114,6 +114,7 @@ public class MSQControllerTask extends AbstractTask implements ClientTaskQuery @JsonCreator public MSQControllerTask( @JsonProperty("id") @Nullable String id, + @JsonProperty("groupId") @Nullable String groupId, @JsonProperty("spec") MSQSpec querySpec, @JsonProperty("sqlQuery") @Nullable String sqlQuery, @JsonProperty("sqlQueryContext") @Nullable Map sqlQueryContext, @@ -125,7 +126,7 @@ public MSQControllerTask( { super( id != null ? id : MSQTasks.controllerTaskId(null), - id, + groupId != null ? groupId : id, null, getDataSourceForTaskMetadata(querySpec), context diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java index 4d7762607985..73a46991220f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java @@ -283,6 +283,7 @@ public QueryResponse runQuery(final DruidQuery druidQuery) final MSQControllerTask controllerTask = new MSQControllerTask( taskId, + null, querySpec.withOverriddenContext(nativeQueryContext), MSQTaskQueryMakerUtils.maskSensitiveJsonKeys(plannerContext.getSql()), plannerContext.queryContextMap(), diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQControllerTaskTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQControllerTaskTest.java index 6aaf21ee3bde..b57f006e396f 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQControllerTaskTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQControllerTaskTest.java @@ -60,6 +60,7 @@ public class MSQControllerTaskTest public void testGetInputSourceResources() { MSQControllerTask msqWorkerTask = new MSQControllerTask( + null, null, MSQ_SPEC, null, diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlStatementResourceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlStatementResourceTest.java index 3d41b46825a4..62f719ae812e 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlStatementResourceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlStatementResourceTest.java @@ -147,6 +147,7 @@ public class SqlStatementResourceTest extends MSQTestBase private static final MSQControllerTask MSQ_CONTROLLER_SELECT_PAYLOAD = new MSQControllerTask( ACCEPTED_SELECT_MSQ_QUERY, + null, MSQSpec.builder() .query(QUERY) .columnMappings( @@ -187,6 +188,7 @@ public class SqlStatementResourceTest extends MSQTestBase private static final MSQControllerTask MSQ_CONTROLLER_INSERT_PAYLOAD = new MSQControllerTask( ACCEPTED_SELECT_MSQ_QUERY, + null, MSQSpec.builder() .query(QUERY) .columnMappings( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 7449842c129d..8743bb8351e4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -169,7 +169,8 @@ public class CompactionTask extends AbstractBatchIndexTask private final SegmentProvider segmentProvider; @JsonIgnore private final PartitionConfigurationManager partitionConfigurationManager; - + @Nullable + private final Engine engine; @JsonIgnore private final SegmentCacheManagerFactory segmentCacheManagerFactory; @@ -197,6 +198,7 @@ public CompactionTask( @JsonProperty("granularitySpec") @Nullable final ClientCompactionTaskGranularitySpec granularitySpec, @JsonProperty("tuningConfig") @Nullable final TuningConfig tuningConfig, @JsonProperty("context") @Nullable final Map context, + @JsonProperty("engine") @Nullable final Engine engine, @JacksonInject SegmentCacheManagerFactory segmentCacheManagerFactory ) { @@ -250,6 +252,7 @@ public CompactionTask( this.tuningConfig = tuningConfig != null ? getTuningConfig(tuningConfig) : null; this.segmentProvider = new SegmentProvider(dataSource, this.ioConfig.getInputSpec()); this.partitionConfigurationManager = new PartitionConfigurationManager(this.tuningConfig); + this.engine = engine; this.segmentCacheManagerFactory = segmentCacheManagerFactory; } @@ -397,6 +400,12 @@ public ParallelIndexTuningConfig getTuningConfig() return tuningConfig; } + @JsonProperty + public Engine getEngine() + { + return engine; + } + @Override public String getType() { @@ -476,13 +485,13 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception getMetricBuilder() ); - // TODO: Uncomment the condition -// if (tuningConfig.useMSQ && toolbox.getCompactionToMSQ() != null) { - if (toolbox.getCompactionToMSQ() != null) { - // Switch to using MSQ for compaction - registerResourceCloserOnAbnormalExit(currentSubTaskHolder); - return toolbox.getCompactionToMSQ() - .createAndRunMSQControllerTasks(this, toolbox, intervalDataSchemas); + // TODO: Remove below test + log.info ("Test log"); + log.info("Compacting using engine[%s]", engine); + if (engine == Engine.MSQ) { + registerResourceCloserOnAbnormalExit(currentSubTaskHolder); + return toolbox.getCompactionToMSQ() + .createAndRunMSQControllerTasks(this, toolbox, intervalDataSchemas); } else { final List ingestionSpecs = createIngestionSpecs( intervalDataSchemas, @@ -520,14 +529,9 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception private TaskStatus runParallelIndexSubtasks(List tasks, TaskToolbox toolbox) throws JsonProcessingException { - - // TODO(vishesh): Remove below line. -// System.out.println(new ObjectMapper().writeValueAsString(tasks)); -// return TaskStatus.failure(getId(), "Forced failure"); final int totalNumSpecs = tasks.size(); log.info("Generated [%d] compaction task specs", totalNumSpecs); - int failCnt = 0; Map completionReports = new HashMap<>(); for (int i = 0; i < tasks.size(); i++) { @@ -1270,6 +1274,7 @@ public static class Builder private TuningConfig tuningConfig; @Nullable private Map context; + private Engine engine; public Builder( String dataSource, @@ -1352,6 +1357,14 @@ public Builder context(Map context) return this; } + public Builder engine(Engine engine) + { + this.engine = engine; + return this; + } + + + public CompactionTask build() { return new CompactionTask( @@ -1369,6 +1382,7 @@ public CompactionTask build() granularitySpec, tuningConfig, context, + engine, segmentCacheManagerFactory ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Engine.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Engine.java new file mode 100644 index 000000000000..f91ea1434d49 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Engine.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task; + +import com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.druid.java.util.common.StringUtils; + +/** + * Engine to be used for a compaction task. + * Should be synchronized with {@link org.apache.druid.server.coordinator.DataSourceCompactionConfig.Engine}. + */ +public enum Engine +{ + NATIVE, + MSQ; + + @JsonCreator + public static Engine fromString(String name) + { + return valueOf(StringUtils.toUpperCase(name)); + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java index fc581d4954b9..898735244f29 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java @@ -63,6 +63,7 @@ import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; +import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.server.security.AuthorizerMapper; import org.joda.time.Duration; @@ -127,7 +128,8 @@ public void testClientCompactionTaskQueryToCompactionTask() throws IOException new ClientCompactionTaskDimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), new AggregatorFactory[] {new CountAggregatorFactory("cnt")}, new ClientCompactionTaskTransformSpec(new SelectorDimFilter("dim1", "foo", null)), - ImmutableMap.of("key", "value") + ImmutableMap.of("key", "value"), + DataSourceCompactionConfig.Engine.MSQ ); final byte[] json = mapper.writeValueAsBytes(query); @@ -233,6 +235,10 @@ public void testClientCompactionTaskQueryToCompactionTask() throws IOException query.getMetricsSpec(), task.getMetricsSpec() ); + Assert.assertEquals( + query.getEngine().toString(), + task.getEngine().toString() + ); } @Test @@ -345,7 +351,8 @@ public void testCompactionTaskToClientCompactionTaskQuery() throws IOException new ClientCompactionTaskDimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), new AggregatorFactory[] {new CountAggregatorFactory("cnt")}, new ClientCompactionTaskTransformSpec(new SelectorDimFilter("dim1", "foo", null)), - new HashMap<>() + new HashMap<>(), + null ); final byte[] json = mapper.writeValueAsBytes(task); diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQuery.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQuery.java index 5873bd229dbe..b6beac37a708 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQuery.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQuery.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import javax.annotation.Nullable; import java.util.Arrays; @@ -46,6 +47,7 @@ public class ClientCompactionTaskQuery implements ClientTaskQuery private final AggregatorFactory[] metricsSpec; private final ClientCompactionTaskTransformSpec transformSpec; private final Map context; + private final DataSourceCompactionConfig.Engine engine; @JsonCreator public ClientCompactionTaskQuery( @@ -57,8 +59,9 @@ public ClientCompactionTaskQuery( @JsonProperty("dimensionsSpec") ClientCompactionTaskDimensionsSpec dimensionsSpec, @JsonProperty("metricsSpec") AggregatorFactory[] metrics, @JsonProperty("transformSpec") ClientCompactionTaskTransformSpec transformSpec, - @JsonProperty("context") Map context - ) + @JsonProperty("context") Map context, + @JsonProperty("engine") DataSourceCompactionConfig.Engine engine + ) { this.id = Preconditions.checkNotNull(id, "id"); this.dataSource = dataSource; @@ -69,6 +72,7 @@ public ClientCompactionTaskQuery( this.metricsSpec = metrics; this.transformSpec = transformSpec; this.context = context; + this.engine = engine; } @JsonProperty @@ -135,6 +139,12 @@ public Map getContext() return context; } + @JsonProperty + public DataSourceCompactionConfig.Engine getEngine() + { + return engine; + } + @Override public boolean equals(Object o) { @@ -153,7 +163,8 @@ public boolean equals(Object o) Objects.equals(dimensionsSpec, that.dimensionsSpec) && Arrays.equals(metricsSpec, that.metricsSpec) && Objects.equals(transformSpec, that.transformSpec) && - Objects.equals(context, that.context); + Objects.equals(context, that.context) && + Objects.equals(engine, that.engine); } @Override @@ -167,7 +178,8 @@ public int hashCode() granularitySpec, dimensionsSpec, transformSpec, - context + context, + engine ); result = 31 * result + Arrays.hashCode(metricsSpec); return result; @@ -186,6 +198,7 @@ public String toString() ", metricsSpec=" + Arrays.toString(metricsSpec) + ", transformSpec=" + transformSpec + ", context=" + context + + ", engine=" + engine + '}'; } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java index 115a42bb92e1..8b737703b2f2 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java @@ -24,6 +24,7 @@ import com.google.common.base.Preconditions; import org.apache.druid.error.InvalidInput; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; +import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.aggregation.AggregatorFactory; @@ -36,7 +37,9 @@ public class DataSourceCompactionConfig { - /** Must be synced with Tasks.DEFAULT_MERGE_TASK_PRIORITY */ + /** + * Must be synced with Tasks.DEFAULT_MERGE_TASK_PRIORITY + */ public static final int DEFAULT_COMPACTION_TASK_PRIORITY = 25; // Approx. 100TB. Chosen instead of Long.MAX_VALUE to avoid overflow on web-console and other clients private static final long DEFAULT_INPUT_SEGMENT_SIZE_BYTES = 100_000_000_000_000L; @@ -59,7 +62,7 @@ public class DataSourceCompactionConfig private final UserCompactionTaskTransformConfig transformSpec; private final UserCompactionTaskIOConfig ioConfig; private final Map taskContext; - private Engine engine; + private final Engine engine; @JsonCreator public DataSourceCompactionConfig( @@ -231,32 +234,56 @@ public int hashCode() return result; } - public void updateEngineAndValidate(Engine defaultEngine) + public static DataSourceCompactionConfig from(Engine defaultEngine, DataSourceCompactionConfig currentConfig) { - boolean usingDefault = false; - if (engine == null) { - engine = defaultEngine; - usingDefault = true; + Engine newEngine = currentConfig.getEngine(); + String engineSourceLog = "specified in spec"; + if (newEngine == null) { + newEngine = defaultEngine; + engineSourceLog = "set as default"; } - if (engine == Engine.MSQ) { - if (tuningConfig != null) { - PartitionsSpec partitionsSpec = tuningConfig.getPartitionsSpec(); - if (partitionsSpec != null && !(partitionsSpec instanceof DimensionRangePartitionsSpec)) { + if (newEngine == Engine.MSQ) { + if (currentConfig.getTuningConfig() != null) { + PartitionsSpec partitionsSpec = currentConfig.getTuningConfig().getPartitionsSpec(); + + if (partitionsSpec != null && !(partitionsSpec instanceof DimensionRangePartitionsSpec + || partitionsSpec instanceof DynamicPartitionsSpec)) { throw InvalidInput.exception( - "Invalid partition spec type[%s] for MSQ compaction engine %s. Type must be DynamicRangePartitionSpec.", - partitionsSpec.getClass(), usingDefault ? "set as default" : "specified in spec" + "Invalid partition spec type[%s] for MSQ compaction engine %s." + + " Type must be either DynamicPartitionsSpec or DynamicRangePartitionsSpec .", + partitionsSpec.getClass(), + engineSourceLog + ); + } + if (partitionsSpec instanceof DynamicPartitionsSpec + && ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows() != null) { + throw InvalidInput.exception( + "maxTotalRows[%d] in DynamicPartitionsSpec not supported for MSQ compaction engine %s.", + ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows(), engineSourceLog ); } - } - if (maxRowsPerSegment != null) { - throw InvalidInput.exception( - "MaxRowsPerSegment[%d] field not supported for MSQ compaction engine.", - maxRowsPerSegment - ); } } + return new DataSourceCompactionConfig( + currentConfig.getDataSource(), + currentConfig.getTaskPriority(), + currentConfig.getInputSegmentSizeBytes(), + currentConfig.getMaxRowsPerSegment(), + currentConfig.getSkipOffsetFromLatest(), + currentConfig.getTuningConfig(), + currentConfig.getGranularitySpec(), + currentConfig.getDimensionsSpec(), + currentConfig.getMetricsSpec(), + currentConfig.getTransformSpec(), + currentConfig.getIoConfig(), + currentConfig.getTaskContext(), + newEngine + ); } + /** + * Engine to be used for a compaction task. + */ public enum Engine { NATIVE, diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index bb88b86dbf84..76f071672edb 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java @@ -168,7 +168,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) intervalsToSkipCompaction.computeIfAbsent(status.getDataSource(), k -> new ArrayList<>()) .add(interval); - busyCompactionTaskSlots += findMaxNumTaskSlotsUsedByOneCompactionTask( + busyCompactionTaskSlots += findMaxNumTaskSlotsUsedByOneNativeCompactionTask( compactionTaskQuery.getTuningConfig() ); } @@ -284,7 +284,7 @@ private Map> getLockedIntervals( * the given tuningConfig. */ @VisibleForTesting - static int findMaxNumTaskSlotsUsedByOneCompactionTask(@Nullable ClientCompactionTaskQueryTuningConfig tuningConfig) + static int findMaxNumTaskSlotsUsedByOneNativeCompactionTask(@Nullable ClientCompactionTaskQueryTuningConfig tuningConfig) { if (isParallelMode(tuningConfig)) { @Nullable @@ -465,6 +465,22 @@ private int submitCompactionTasks( } } + DataSourceCompactionConfig.Engine engine = config.getEngine(); + + Map autoCompactionContext = config.getTaskContext(); + int numCurrentCompactionTasksAndSubtasks; + if (engine == DataSourceCompactionConfig.Engine.MSQ && autoCompactionContext!= null && !autoCompactionContext.containsKey("maxNumTasks")) { + autoCompactionContext = newAutoCompactionContext(autoCompactionContext); + numCurrentCompactionTasksAndSubtasks = numAvailableCompactionTaskSlots; + // todo (vishesh): there was a plan to use auto strategy for task slots calculation. How to introduce that + // and then how to retrieve that number to manage underneath calculation + // also check if there is a need to clone the map here + autoCompactionContext.putIfAbsent("maxNumTasks", numCurrentCompactionTasksAndSubtasks); + } + else { + numCurrentCompactionTasksAndSubtasks = findMaxNumTaskSlotsUsedByOneNativeCompactionTask(config.getTuningConfig()); + } + final String taskId = compactSegments( "coordinator-issued", segmentsToCompact, @@ -479,7 +495,8 @@ private int submitCompactionTasks( config.getMetricsSpec(), transformSpec, dropExisting, - newAutoCompactionContext(config.getTaskContext()) + autoCompactionContext, + engine ); LOG.info( @@ -489,7 +506,7 @@ private int submitCompactionTasks( LOG.debugSegments(segmentsToCompact, "Compacting segments"); // Count the compaction task itself + its sub tasks numSubmittedTasks++; - numCompactionTasksAndSubtasks += findMaxNumTaskSlotsUsedByOneCompactionTask(config.getTuningConfig()); + numCompactionTasksAndSubtasks += numCurrentCompactionTasksAndSubtasks; } LOG.info("Submitted a total of [%d] compaction tasks.", numSubmittedTasks); @@ -632,7 +649,8 @@ private String compactSegments( @Nullable AggregatorFactory[] metricsSpec, @Nullable ClientCompactionTaskTransformSpec transformSpec, @Nullable Boolean dropExisting, - @Nullable Map context + @Nullable Map context, + @Nullable DataSourceCompactionConfig.Engine engine ) { Preconditions.checkArgument(!segments.isEmpty(), "Expect non-empty segments to compact"); @@ -660,7 +678,8 @@ private String compactSegments( dimensionsSpec, metricsSpec, transformSpec, - context + context, + engine ); FutureUtils.getUnchecked(overlordClient.runTask(taskId, taskPayload), true); return taskId; diff --git a/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java index 9254314311d7..c2a26cc18811 100644 --- a/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java +++ b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java @@ -123,6 +123,7 @@ public Response setCompactionEngine( null, engine ); + // TODO (vishesh): add the check to loop over all configs and validate if this update is compatible. return updateConfigHelper(operator, AuthorizationUtils.buildAuditInfo(req)); } @@ -139,8 +140,8 @@ public Response addOrUpdateCompactionConfig( .getCompactionConfigs() .stream() .collect(Collectors.toMap(DataSourceCompactionConfig::getDataSource, Function.identity())); - newConfig.updateEngineAndValidate(current.getEngine()); - newConfigs.put(newConfig.getDataSource(), newConfig); + DataSourceCompactionConfig updateConfig = DataSourceCompactionConfig.from(current.getEngine(), newConfig); + newConfigs.put(updateConfig.getDataSource(), updateConfig); newCompactionConfig = CoordinatorCompactionConfig.from(current, ImmutableList.copyOf(newConfigs.values())); return newCompactionConfig; diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java index 787e5b6ea22d..76021c1b3b2f 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java @@ -1088,6 +1088,7 @@ public void testCompactWithGranularitySpecConflictWithActiveCompactionTask() null, null, null, + null, null ) ); @@ -2189,7 +2190,7 @@ public void testFindMaxNumTaskSlotsUsedByOneCompactionTaskWhenIsParallelMode() ClientCompactionTaskQueryTuningConfig tuningConfig = Mockito.mock(ClientCompactionTaskQueryTuningConfig.class); Mockito.when(tuningConfig.getPartitionsSpec()).thenReturn(Mockito.mock(PartitionsSpec.class)); Mockito.when(tuningConfig.getMaxNumConcurrentSubTasks()).thenReturn(2); - Assert.assertEquals(3, CompactSegments.findMaxNumTaskSlotsUsedByOneCompactionTask(tuningConfig)); + Assert.assertEquals(3, CompactSegments.findMaxNumTaskSlotsUsedByOneNativeCompactionTask(tuningConfig)); } @Test @@ -2198,7 +2199,7 @@ public void testFindMaxNumTaskSlotsUsedByOneCompactionTaskWhenIsSequentialMode() ClientCompactionTaskQueryTuningConfig tuningConfig = Mockito.mock(ClientCompactionTaskQueryTuningConfig.class); Mockito.when(tuningConfig.getPartitionsSpec()).thenReturn(Mockito.mock(PartitionsSpec.class)); Mockito.when(tuningConfig.getMaxNumConcurrentSubTasks()).thenReturn(1); - Assert.assertEquals(1, CompactSegments.findMaxNumTaskSlotsUsedByOneCompactionTask(tuningConfig)); + Assert.assertEquals(1, CompactSegments.findMaxNumTaskSlotsUsedByOneNativeCompactionTask(tuningConfig)); } } From 1deef3e070c62a2754f82bcc53380b345463bc00 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Tue, 16 Apr 2024 11:07:55 +0530 Subject: [PATCH 09/54] Code cleanup --- .run/Coordinator w_ MSQ.run.xml | 19 -- ...Impl.java => CompactionToMSQTaskImpl.java} | 264 +++++++++-------- .../msq/guice/MSQCompactionDruidModule.java | 38 ++- .../druid/msq/indexing/MSQControllerTask.java | 3 +- .../druid/msq/sql/MSQTaskQueryMaker.java | 1 - .../msq/indexing/MSQControllerTaskTest.java | 1 - .../resources/SqlStatementResourceTest.java | 2 - .../druid/indexing/common/TaskToolbox.java | 23 +- .../indexing/common/TaskToolboxFactory.java | 10 +- .../indexing/common/task/CompactionTask.java | 71 ++--- .../common/task/CompactionToMSQProvider.java | 9 - ...ionToMSQ.java => CompactionToMSQTask.java} | 8 +- .../common/task/CompactionTaskTest.java | 269 ++++++++++++------ .../task/CompactionTuningConfigTest.java | 3 - .../overlord/TestTaskToolboxFactory.java | 10 +- .../druid/testing/utils/CompactionUtil.java | 4 - .../CoordinatorCompactionConfig.java | 15 +- .../DataSourceCompactionConfig.java | 14 +- .../coordinator/duty/CompactSegments.java | 22 +- .../coordinator/duty/CompactionStrategy.java | 15 - .../CoordinatorCompactionConfigsResource.java | 27 +- .../DataSourceCompactionConfigTest.java | 56 ++-- .../coordinator/duty/CompactSegmentsTest.java | 3 +- .../duty/KillCompactionConfigTest.java | 12 +- ...rdinatorCompactionConfigsResourceTest.java | 20 +- 25 files changed, 462 insertions(+), 457 deletions(-) delete mode 100644 .run/Coordinator w_ MSQ.run.xml rename extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/{CompactionToMSQImpl.java => CompactionToMSQTaskImpl.java} (55%) delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionToMSQProvider.java rename indexing-service/src/main/java/org/apache/druid/indexing/common/task/{CompactionToMSQ.java => CompactionToMSQTask.java} (65%) delete mode 100644 server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionStrategy.java diff --git a/.run/Coordinator w_ MSQ.run.xml b/.run/Coordinator w_ MSQ.run.xml deleted file mode 100644 index 074b578ea722..000000000000 --- a/.run/Coordinator w_ MSQ.run.xml +++ /dev/null @@ -1,19 +0,0 @@ - - - - \ No newline at end of file diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQTaskImpl.java similarity index 55% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQImpl.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQTaskImpl.java index 59940546ad10..1146abde4756 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQTaskImpl.java @@ -1,24 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.druid.msq.compaction; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import org.apache.druid.data.input.impl.DimensionSchema; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.error.DruidException; import org.apache.druid.error.InvalidInput; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; +import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexer.partitions.SecondaryPartitionType; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.task.CompactionTask; -import org.apache.druid.indexing.common.task.CompactionToMSQ; +import org.apache.druid.indexing.common.task.CompactionToMSQTask; import org.apache.druid.indexing.common.task.CurrentSubTaskHolder; import org.apache.druid.indexing.common.task.Tasks; -import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; -import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.NonnullPair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.AllGranularity; import org.apache.druid.java.util.common.granularity.Granularities; @@ -38,7 +55,6 @@ import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; -import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.VirtualColumns; @@ -58,26 +74,20 @@ import java.util.Map; import java.util.stream.Collectors; -public class CompactionToMSQImpl implements CompactionToMSQ +public class CompactionToMSQTaskImpl implements CompactionToMSQTask { - private static final Logger log = new Logger(CompactionToMSQImpl.class); + private static final Logger log = new Logger(CompactionToMSQTaskImpl.class); private static final Granularity DEFAULT_SEGMENT_GRANULARITY = Granularities.ALL; - final OverlordClient overlordClient; final ObjectMapper jsonMapper; - - public CompactionToMSQImpl( - final OverlordClient overlordClient, - final ObjectMapper jsonMapper - ) + public CompactionToMSQTaskImpl(final OverlordClient overlordClient, final ObjectMapper jsonMapper) { this.overlordClient = overlordClient; this.jsonMapper = jsonMapper; - System.out.println("Initializing MSQCompaction"); } - RowSignature getRowSignature(DataSchema dataSchema) + private static RowSignature getRowSignature(DataSchema dataSchema) { RowSignature.Builder rowSignatureBuilder = RowSignature.builder(); rowSignatureBuilder.add(dataSchema.getTimestampSpec().getTimestampColumn(), ColumnType.LONG); @@ -87,13 +97,12 @@ RowSignature getRowSignature(DataSchema dataSchema) return rowSignatureBuilder.build(); } - public MultipleIntervalSegmentSpec createMultipleIntervalSpec(Interval interval) + private static MultipleIntervalSegmentSpec createMultipleIntervalSpec(Interval interval) { return new MultipleIntervalSegmentSpec(Collections.singletonList(interval)); - } - public List getOrderBySpec(PartitionsSpec partitionSpec) + private static List getOrderBySpec(PartitionsSpec partitionSpec) { if (partitionSpec.getType() == SecondaryPartitionType.RANGE) { List dimensions = ((DimensionRangePartitionsSpec) partitionSpec).getPartitionDimensions(); @@ -104,7 +113,7 @@ public List getOrderBySpec(PartitionsSpec partitionSpec) return Collections.emptyList(); } - public Query buildScanQuery(CompactionTask compactionTask, Interval interval, DataSchema dataSchema) + private static Query buildScanQuery(CompactionTask compactionTask, Interval interval, DataSchema dataSchema) { RowSignature rowSignature = getRowSignature(dataSchema); return new Druids.ScanQueryBuilder().dataSource(dataSchema.getDataSource()) @@ -112,13 +121,12 @@ public Query buildScanQuery(CompactionTask compactionTask, Interval interval, .columnTypes(rowSignature.getColumnTypes()) .intervals(createMultipleIntervalSpec(interval)) .legacy(false) - .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .filters(dataSchema.getTransformSpec().getFilter()) .context(compactionTask.getContext()) .build(); } - public Query buildGroupByQuery(CompactionTask compactionTask, Interval interval, DataSchema dataSchema) + private static Query buildGroupByQuery(CompactionTask compactionTask, Interval interval, DataSchema dataSchema) { DimFilter dimFilter = dataSchema.getTransformSpec().getFilter(); @@ -149,161 +157,93 @@ public Query buildGroupByQuery(CompactionTask compactionTask, Interval interv } @Override - public TaskStatus createAndRunMSQControllerTasks( - CompactionTask compactionTask, TaskToolbox taskToolbox, List> intervalDataSchemas + public TaskStatus createAndRunMSQTasks( + CompactionTask compactionTask, + TaskToolbox taskToolbox, + List> intervalDataSchemas ) throws Exception { List msqControllerTasks = new ArrayList<>(); - QueryContext sqlQueryContext = new QueryContext(compactionTask.getContext()); + QueryContext compactionTaskContext = new QueryContext(compactionTask.getContext()); - for (Pair intervalDataSchema : intervalDataSchemas) { + for (NonnullPair intervalDataSchema : intervalDataSchemas) { Query query; Interval interval = intervalDataSchema.lhs; DataSchema ds = intervalDataSchema.rhs; final Interval replaceInterval = compactionTask.getIoConfig() .getInputSpec() .findInterval(compactionTask.getDataSource()); -// this.escapedRowSignature = "[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"added\",\"type\":\"LONG\"}," -// + "{\"name\":\"channel\",\"type\":\"STRING\"},{\"name\":\"cityName\",\"type\":\"STRING\"}," -// + "{\"name\":\"comment\",\"type\":\"STRING\"},{\"name\":\"countryIsoCode\",\"type\":\"STRING\"}," -// + "{\"name\":\"countryName\",\"type\":\"STRING\"},{\"name\":\"deleted\",\"type\":\"LONG\"}," -// + "{\"name\":\"delta\",\"type\":\"LONG\"},{\"name\":\"isAnonymous\",\"type\":\"STRING\"}," -// + "{\"name\":\"isMinor\",\"type\":\"STRING\"},{\"name\":\"isNew\",\"type\":\"STRING\"}," -// + "{\"name\":\"isRobot\",\"type\":\"STRING\"},{\"name\":\"isUnpatrolled\",\"type\":\"STRING\"}," -// + "{\"name\":\"metroCode\",\"type\":\"LONG\"},{\"name\":\"namespace\",\"type\":\"STRING\"}," -// + "{\"name\":\"page\",\"type\":\"STRING\"},{\"name\":\"regionIsoCode\",\"type\":\"STRING\"}," -// + "{\"name\":\"regionName\",\"type\":\"STRING\"},{\"name\":\"user\",\"type\":\"STRING\"}]"; - - boolean isGroupBy = ds != null && ds.getAggregators().length > 0; - - // CTX_SCAN_SIGNATURE is deprecated and should no longer be reqd -// compactionTask.getContext().putAll(ImmutableMap.of(DruidQuery.CTX_SCAN_SIGNATURE, -// new ObjectMapper().writeValueAsString(getRowSignature(ds.getDimensionsSpec())), - Map context = new HashMap<>(compactionTask.getContext()); - // TODO (vishesh): check why the default is day in specific classes whereas ALL in query maker - try { - context.put( - DruidSqlInsert.SQL_INSERT_SEGMENT_GRANULARITY, - (ds != null && ds.getGranularitySpec() != null) - ? jsonMapper.writeValueAsString(ds.getGranularitySpec().getSegmentGranularity()) - : jsonMapper.writeValueAsString(DEFAULT_SEGMENT_GRANULARITY - ) - ); - } - catch (JsonProcessingException e) { - throw DruidException.defensive() - .build( - e, - "Unable to deserialize the DEFAULT_SEGMENT_GRANULARITY in MSQTaskQueryMaker. " - + "This shouldn't have happened since the DEFAULT_SEGMENT_GRANULARITY object is guaranteed to be " - + "serializable. Please raise an issue in case you are seeing this message while executing a query." - ); - - } - - context.put(DruidSqlReplace.SQL_REPLACE_TIME_CHUNKS, interval.toString()); - context.put(Tasks.STORE_COMPACTION_STATE_KEY, true); - if (!isGroupBy) { - query = buildScanQuery(compactionTask, interval, ds); - } else { - query = buildGroupByQuery(compactionTask, interval, ds); - } + final List segmentSortOrder = MultiStageQueryContext.getSortOrder(compactionTaskContext); MSQDestination msqDestination = new DataSourceMSQDestination( ds.getDataSource(), ds.getGranularitySpec().getSegmentGranularity(), - null, + segmentSortOrder, ImmutableList.of(replaceInterval) - ); -// // TODO (vishesh) : There is some logic related to finalize aggs. Check if there's a case -// // for non-finalize aggs for compaction -- esp. user specified. -// final int maxNumTasks = 2; - /* - For assistance computing return types if !finalizeAggregations. - final Map aggregationIntermediateTypeMap = - finalizeAggregations ? null : buildAggregationIntermediateTypeMap(druidQuery); - - final List sqlTypeNames = new ArrayList<>(); - final List columnTypeList = new ArrayList<>(); - final List columnMappings = QueryUtils.buildColumnMappings(fieldMapping, druidQuery); - - for (final org.apache.calcite.util.Pair entry : fieldMapping) { - final String queryColumn = druidQuery.getOutputRowSignature().getColumnName(entry.getKey()); - - final SqlTypeName sqlTypeName; - - if (!finalizeAggregations && aggregationIntermediateTypeMap.containsKey(queryColumn)) { - final ColumnType druidType = aggregationIntermediateTypeMap.get(queryColumn); - sqlTypeName = new RowSignatures.ComplexSqlType(SqlTypeName.OTHER, druidType, true).getSqlTypeName(); - } else { - sqlTypeName = druidQuery.getOutputRowType().getFieldList().get(entry.getKey()).getType().getSqlTypeName(); - } - sqlTypeNames.add(sqlTypeName); - columnTypeList.add(druidQuery.getOutputRowSignature().getColumnType(queryColumn).orElse(ColumnType.STRING)); - } - */ - - // Pick-up MSQ related context params, if any, from the compaction context itself. - final int maxNumTasks = MultiStageQueryContext.getMaxNumTasks(sqlQueryContext); + // Transfer MSQ-related context params, if any, from the compaction context itself. + final int maxNumTasks = MultiStageQueryContext.getMaxNumTasks(compactionTaskContext); if (maxNumTasks < 2) { throw InvalidInput.exception( - "MSQ context maxNumTasks [%,d] cannot be less than 2, since at least 1 controller and 1 worker is necessary", + "MSQ context maxNumTasks [%,d] cannot be less than 2, " + + "since at least 1 controller and 1 worker is necessary", maxNumTasks ); } - // This parameter is used internally for the number of worker tasks only, so we subtract 1 final int maxNumWorkers = maxNumTasks - 1; - final int maxRowsInMemory = MultiStageQueryContext.getRowsInMemory(sqlQueryContext); - final boolean finalizeAggregations = MultiStageQueryContext.isFinalizeAggregations(sqlQueryContext); - + final int maxRowsInMemory = MultiStageQueryContext.getRowsInMemory(compactionTaskContext); - Integer rowsPerSegment = null; - if (compactionTask.getTuningConfig() != null){ - PartitionsSpec partitionsSpec = compactionTask.getTuningConfig().getPartitionsSpec(); - if (partitionsSpec != null) { - rowsPerSegment = partitionsSpec.getMaxRowsPerSegment(); - } + boolean isGroupBy = ds.getAggregators().length > 0; + if (!isGroupBy) { + query = buildScanQuery(compactionTask, interval, ds); + } else { + query = buildGroupByQuery(compactionTask, interval, ds); } - MSQTuningConfig msqTuningConfig = new MSQTuningConfig( + MSQTuningConfig msqTuningConfig = getMsqTuningConfig( + compactionTask, maxNumWorkers, - maxRowsInMemory, - rowsPerSegment, - compactionTask.getTuningConfig().getIndexSpec() + maxRowsInMemory ); MSQSpec msqSpec = MSQSpec.builder() .query(query) .columnMappings(ColumnMappings.identity(getRowSignature(ds))) .destination(msqDestination) - // TODO (vishesh): Investiage how this is populated - .assignmentStrategy(MultiStageQueryContext.getAssignmentStrategy(sqlQueryContext)) + .assignmentStrategy(MultiStageQueryContext.getAssignmentStrategy(compactionTaskContext)) .tuningConfig(msqTuningConfig) .build(); + Map msqControllerTaskContext = createMSQTaskContext(compactionTask, interval, ds); + MSQControllerTask controllerTask = new MSQControllerTask( compactionTask.getId(), - compactionTask.getId(), - msqSpec.withOverriddenContext(context), + msqSpec.withOverriddenContext(msqControllerTaskContext), null, - context, + msqControllerTaskContext, null, null, null, null ); - MSQControllerTask serdeControllerTask = jsonMapper.readerFor(MSQControllerTask.class).readValue(jsonMapper.writeValueAsString(controllerTask)); - msqControllerTasks.add(serdeControllerTask); + + // Doing a serde roundtrip for MSQControllerTask as the "injector" field of this class is supposed to be injected + // by the mapper. + MSQControllerTask serdedMSQControllerTask = jsonMapper.readerFor(MSQControllerTask.class) + .readValue(jsonMapper.writeValueAsString(controllerTask)); + msqControllerTasks.add(serdedMSQControllerTask); + } + + if (msqControllerTasks.isEmpty()) { + log.warn( + "Can't find segments from inputSpec[%s], nothing to do.", + compactionTask.getIoConfig().getInputSpec() + ); } -// if (msqControllerTasks.isEmpty()){ -// log.warn("Can't find segments from inputSpec[%s], nothing to do.", -// ioConfig.getInputSpec()) -// } return runSubtasks( msqControllerTasks, taskToolbox, @@ -312,16 +252,69 @@ public TaskStatus createAndRunMSQControllerTasks( ); } - private TaskStatus runSubtasks( + private static MSQTuningConfig getMsqTuningConfig( + CompactionTask compactionTask, + int maxNumWorkers, + int maxRowsInMemory + ) + { + Integer rowsPerSegment = null; + if (compactionTask.getTuningConfig() != null) { + PartitionsSpec partitionsSpec = compactionTask.getTuningConfig().getPartitionsSpec(); + if (partitionsSpec instanceof DynamicPartitionsSpec) { + rowsPerSegment = partitionsSpec.getMaxRowsPerSegment(); + } else if (partitionsSpec instanceof DimensionRangePartitionsSpec) { + DimensionRangePartitionsSpec dimensionRangePartitionsSpec = (DimensionRangePartitionsSpec) partitionsSpec; + rowsPerSegment = dimensionRangePartitionsSpec.getTargetRowsPerSegment() != null + ? dimensionRangePartitionsSpec.getTargetRowsPerSegment() + : dimensionRangePartitionsSpec.getMaxRowsPerSegment(); + } + } + + return new MSQTuningConfig( + maxNumWorkers, + maxRowsInMemory, + rowsPerSegment, + compactionTask.getTuningConfig().getIndexSpec() + ); + } + + private Map createMSQTaskContext(CompactionTask compactionTask, Interval interval, DataSchema ds) + { + Map context = new HashMap<>(compactionTask.getContext()); + try { + context.put( + DruidSqlInsert.SQL_INSERT_SEGMENT_GRANULARITY, + jsonMapper.writeValueAsString(ds.getGranularitySpec() != null + ? ds.getGranularitySpec() + .getSegmentGranularity() + : DEFAULT_SEGMENT_GRANULARITY) + ); + } + catch (JsonProcessingException e) { + throw DruidException + .defensive() + .build( + e, + "Unable to deserialize the DEFAULT_SEGMENT_GRANULARITY in CompactionToMSQTaskImpl. " + + "This shouldn't have happened since the DEFAULT_SEGMENT_GRANULARITY object is guaranteed to be " + + "serializable. Please raise an issue in case you are seeing this message while executing a query." + ); + } + context.put(DruidSqlReplace.SQL_REPLACE_TIME_CHUNKS, interval.toString()); + context.put(Tasks.STORE_COMPACTION_STATE_KEY, true); + return context; + } + + private static TaskStatus runSubtasks( List tasks, TaskToolbox toolbox, CurrentSubTaskHolder currentSubTaskHolder, String compactionTaskId ) throws JsonProcessingException { - final int totalNumSpecs = tasks.size(); - log.info("Generated [%d] compaction task specs", totalNumSpecs); + log.info("Generated [%d] MSQControllerTask specs", totalNumSpecs); int failCnt = 0; @@ -335,13 +328,15 @@ private TaskStatus runSubtasks( try { if (eachTask.isReady(toolbox.getTaskActionClient())) { log.info("Running MSQControllerTask: " + json); - // TODO (vishesh) : Check handling of multi-controller tasks -- to be triggered via overlord + // Currently multiple MSQControllerTasks, if created, are started serially in-place, just as multiple + // ParallelIndexSupervisorTasks for native compaction. A better strategy may be to trigger them via the + // overlord. In that case, the subtask statuses would need to be tracked to determine the final status of the + // compaction task. final TaskStatus eachResult = eachTask.run(toolbox); if (!eachResult.isSuccess()) { failCnt++; log.warn("Failed to run MSQControllerTask: [%s].\nTrying the next MSQControllerTask.", json); } - } else { failCnt++; log.warn("MSQControllerTask is not ready: [%s].\nTrying the next MSQControllerTask.", json); @@ -358,7 +353,6 @@ private TaskStatus runSubtasks( totalNumSpecs - failCnt, failCnt ); - log.info(msg); return failCnt == 0 ? TaskStatus.success(compactionTaskId) : TaskStatus.failure(compactionTaskId, msg); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQCompactionDruidModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQCompactionDruidModule.java index 33da0106961b..1cfeac77b5c1 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQCompactionDruidModule.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQCompactionDruidModule.java @@ -1,3 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.druid.msq.guice; import com.fasterxml.jackson.databind.Module; @@ -5,9 +24,9 @@ import com.google.inject.Binder; import com.google.inject.Provides; import org.apache.druid.guice.LazySingleton; -import org.apache.druid.indexing.common.task.CompactionToMSQ; +import org.apache.druid.indexing.common.task.CompactionToMSQTask; import org.apache.druid.initialization.DruidModule; -import org.apache.druid.msq.compaction.CompactionToMSQImpl; +import org.apache.druid.msq.compaction.CompactionToMSQTaskImpl; import org.apache.druid.rpc.indexing.OverlordClient; import java.util.Collections; @@ -16,13 +35,10 @@ public class MSQCompactionDruidModule implements DruidModule { - public static final String SCHEME = "msq"; - @Override public void configure(Binder binder) { -// binder.bind(CompactionToMSQ.class).toProvider(CompactionToMSQImplProvider.class).in(LazySingleton.class); - binder.bind(CompactionToMSQ.class).to(CompactionToMSQImpl.class).in(LazySingleton.class); + binder.bind(CompactionToMSQTask.class).to(CompactionToMSQTaskImpl.class).in(LazySingleton.class); } @Override @@ -33,9 +49,11 @@ public List getJacksonModules() @Provides @LazySingleton - CompactionToMSQImpl getCompactionToMSQImpl(final OverlordClient overlordClient, - final ObjectMapper jsonMapper){ - return new CompactionToMSQImpl(overlordClient, jsonMapper); + CompactionToMSQTaskImpl getCompactionToMSQImpl( + final OverlordClient overlordClient, + final ObjectMapper jsonMapper + ) + { + return new CompactionToMSQTaskImpl(overlordClient, jsonMapper); } - } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java index eff98cf1395a..7eb455ca8424 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java @@ -114,7 +114,6 @@ public class MSQControllerTask extends AbstractTask implements ClientTaskQuery @JsonCreator public MSQControllerTask( @JsonProperty("id") @Nullable String id, - @JsonProperty("groupId") @Nullable String groupId, @JsonProperty("spec") MSQSpec querySpec, @JsonProperty("sqlQuery") @Nullable String sqlQuery, @JsonProperty("sqlQueryContext") @Nullable Map sqlQueryContext, @@ -126,7 +125,7 @@ public MSQControllerTask( { super( id != null ? id : MSQTasks.controllerTaskId(null), - groupId != null ? groupId : id, + id, null, getDataSourceForTaskMetadata(querySpec), context diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java index 73a46991220f..4d7762607985 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java @@ -283,7 +283,6 @@ public QueryResponse runQuery(final DruidQuery druidQuery) final MSQControllerTask controllerTask = new MSQControllerTask( taskId, - null, querySpec.withOverriddenContext(nativeQueryContext), MSQTaskQueryMakerUtils.maskSensitiveJsonKeys(plannerContext.getSql()), plannerContext.queryContextMap(), diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQControllerTaskTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQControllerTaskTest.java index b57f006e396f..6aaf21ee3bde 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQControllerTaskTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQControllerTaskTest.java @@ -60,7 +60,6 @@ public class MSQControllerTaskTest public void testGetInputSourceResources() { MSQControllerTask msqWorkerTask = new MSQControllerTask( - null, null, MSQ_SPEC, null, diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlStatementResourceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlStatementResourceTest.java index 04d54d3de5b9..b80e59223f78 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlStatementResourceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlStatementResourceTest.java @@ -139,7 +139,6 @@ public class SqlStatementResourceTest extends MSQTestBase private static final MSQControllerTask MSQ_CONTROLLER_SELECT_PAYLOAD = new MSQControllerTask( ACCEPTED_SELECT_MSQ_QUERY, - null, MSQSpec.builder() .query(QUERY) .columnMappings( @@ -180,7 +179,6 @@ public class SqlStatementResourceTest extends MSQTestBase private static final MSQControllerTask MSQ_CONTROLLER_INSERT_PAYLOAD = new MSQControllerTask( ACCEPTED_SELECT_MSQ_QUERY, - null, MSQSpec.builder() .query(QUERY) .columnMappings( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java index 412c941863a8..b8ec6d401357 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java @@ -36,7 +36,7 @@ import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.config.TaskConfig; -import org.apache.druid.indexing.common.task.CompactionToMSQ; +import org.apache.druid.indexing.common.task.CompactionToMSQTask; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTaskClientProvider; import org.apache.druid.indexing.common.task.batch.parallel.ShuffleClient; import org.apache.druid.indexing.worker.shuffle.IntermediaryDataManager; @@ -135,8 +135,7 @@ public class TaskToolbox private final String attemptId; private final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; - - private final CompactionToMSQ compactionToMSQ; + private final CompactionToMSQTask compactionToMSQTask; public TaskToolbox( SegmentLoaderConfig segmentLoaderConfig, @@ -180,7 +179,7 @@ public TaskToolbox( TaskLogPusher taskLogPusher, String attemptId, CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig, - CompactionToMSQ compactionToMSQ + CompactionToMSQTask compactionToMSQTask ) { this.segmentLoaderConfig = segmentLoaderConfig; @@ -225,7 +224,7 @@ public TaskToolbox( this.taskLogPusher = taskLogPusher; this.attemptId = attemptId; this.centralizedDatasourceSchemaConfig = centralizedDatasourceSchemaConfig; - this.compactionToMSQ = compactionToMSQ; + this.compactionToMSQTask = compactionToMSQTask; } public SegmentLoaderConfig getSegmentLoaderConfig() @@ -503,9 +502,9 @@ public CentralizedDatasourceSchemaConfig getCentralizedTableSchemaConfig() return centralizedDatasourceSchemaConfig; } - public CompactionToMSQ getCompactionToMSQ() + public CompactionToMSQTask getCompactionToMSQ() { - return compactionToMSQ; + return compactionToMSQTask; } /** @@ -575,7 +574,7 @@ public static class Builder private TaskLogPusher taskLogPusher; private String attemptId; private CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; - private CompactionToMSQ compactionToMSQ; + private CompactionToMSQTask compactionToMSQTask; public Builder() { @@ -622,7 +621,7 @@ public Builder(TaskToolbox other) this.supervisorTaskClientProvider = other.supervisorTaskClientProvider; this.shuffleClient = other.shuffleClient; this.centralizedDatasourceSchemaConfig = other.centralizedDatasourceSchemaConfig; - this.compactionToMSQ = other.compactionToMSQ; + this.compactionToMSQTask = other.compactionToMSQTask; } public Builder config(final SegmentLoaderConfig segmentLoaderConfig) @@ -871,8 +870,8 @@ public Builder centralizedTableSchemaConfig(final CentralizedDatasourceSchemaCon return this; } - public Builder compactionToMSQ(final CompactionToMSQ compactionToMSQ){ - this.compactionToMSQ = compactionToMSQ; + public Builder compactionToMSQ(final CompactionToMSQTask compactionToMSQTask){ + this.compactionToMSQTask = compactionToMSQTask; return this; } @@ -920,7 +919,7 @@ public TaskToolbox build() taskLogPusher, attemptId, centralizedDatasourceSchemaConfig, - compactionToMSQ + compactionToMSQTask ); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java index 9e307ecb995a..8205250f1309 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java @@ -37,7 +37,7 @@ import org.apache.druid.indexer.report.TaskReportFileWriter; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; import org.apache.druid.indexing.common.config.TaskConfig; -import org.apache.druid.indexing.common.task.CompactionToMSQ; +import org.apache.druid.indexing.common.task.CompactionToMSQTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTaskClientProvider; @@ -118,7 +118,7 @@ public class TaskToolboxFactory private final TaskLogPusher taskLogPusher; private final String attemptId; private final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; - private final CompactionToMSQ compactionToMSQ; + private final CompactionToMSQTask compactionToMSQTask; @Inject public TaskToolboxFactory( @@ -162,7 +162,7 @@ public TaskToolboxFactory( TaskLogPusher taskLogPusher, @AttemptId String attemptId, CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig, - CompactionToMSQ compactionToMSQ + CompactionToMSQTask compactionToMSQTask ) { this.segmentLoaderConfig = segmentLoadConfig; @@ -205,7 +205,7 @@ public TaskToolboxFactory( this.taskLogPusher = taskLogPusher; this.attemptId = attemptId; this.centralizedDatasourceSchemaConfig = centralizedDatasourceSchemaConfig; - this.compactionToMSQ = compactionToMSQ; + this.compactionToMSQTask = compactionToMSQTask; } public TaskToolbox build(Task task) @@ -270,7 +270,7 @@ public TaskToolbox build(TaskConfig config, Task task) .taskLogPusher(taskLogPusher) .attemptId(attemptId) .centralizedTableSchemaConfig(centralizedDatasourceSchemaConfig) - .compactionToMSQ(compactionToMSQ) + .compactionToMSQ(compactionToMSQTask) .build(); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index fabea21b808c..0720462e8675 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -294,8 +294,7 @@ static CompactionTuningConfig getTuningConfig(TuningConfig tuningConfig) parallelIndexTuningConfig.getMaxSavedParseExceptions(), parallelIndexTuningConfig.getMaxColumnsToMerge(), parallelIndexTuningConfig.getAwaitSegmentAvailabilityTimeoutMillis(), - parallelIndexTuningConfig.getNumPersistThreads(), - null + parallelIndexTuningConfig.getNumPersistThreads() ); } else if (tuningConfig instanceof IndexTuningConfig) { final IndexTuningConfig indexTuningConfig = (IndexTuningConfig) tuningConfig; @@ -330,8 +329,7 @@ static CompactionTuningConfig getTuningConfig(TuningConfig tuningConfig) indexTuningConfig.getMaxSavedParseExceptions(), indexTuningConfig.getMaxColumnsToMerge(), indexTuningConfig.getAwaitSegmentAvailabilityTimeoutMillis(), - indexTuningConfig.getNumPersistThreads(), - null + indexTuningConfig.getNumPersistThreads() ); } else { throw new ISE( @@ -470,11 +468,10 @@ void emitCompactIngestionModeMetrics( @Override public TaskStatus runTask(TaskToolbox toolbox) throws Exception { - // emit metric for compact ingestion mode: emitCompactIngestionModeMetrics(toolbox.getEmitter(), ioConfig.isDropExisting()); - final List> intervalDataSchemas = createDataSchemasForIntervals( + final List> intervalDataSchemas = createDataSchemasForIntervals( UTC_CLOCK, toolbox, getTaskLockHelper().getLockGranularityToUse(), @@ -486,13 +483,10 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception getMetricBuilder() ); - // TODO: Remove below test - log.info ("Test log"); - log.info("Compacting using engine[%s]", engine); if (engine == Engine.MSQ) { registerResourceCloserOnAbnormalExit(currentSubTaskHolder); return toolbox.getCompactionToMSQ() - .createAndRunMSQControllerTasks(this, toolbox, intervalDataSchemas); + .createAndRunMSQTasks(this, toolbox, intervalDataSchemas); } else { final List ingestionSpecs = createIngestionSpecs( intervalDataSchemas, @@ -535,8 +529,8 @@ private TaskStatus runParallelIndexSubtasks(List ta int failCnt = 0; final TaskReport.ReportMap completionReports = new TaskReport.ReportMap(); - for (int i = 0; i < indexTaskSpecs.size(); i++) { - ParallelIndexSupervisorTask eachSpec = indexTaskSpecs.get(i); + for (int i = 0; i < tasks.size(); i++) { + ParallelIndexSupervisorTask eachSpec = tasks.get(i); final String json = toolbox.getJsonMapper().writerWithDefaultPrettyPrinter().writeValueAsString(eachSpec); if (!currentSubTaskHolder.setTask(eachSpec)) { String errMsg = "Task was asked to stop. Finish as failed."; @@ -553,11 +547,9 @@ private TaskStatus runParallelIndexSubtasks(List ta } String reportKeySuffix = "_" + i; - Optional.ofNullable(eachSpec.getCompletionReports()).ifPresent( - reports -> completionReports.putAll( - CollectionUtils.mapKeys(reports, key -> key + reportKeySuffix) - ) - ); + Optional.ofNullable(eachSpec.getCompletionReports()) + .ifPresent(reports -> completionReports.putAll( + CollectionUtils.mapKeys(reports, key -> key + reportKeySuffix))); } else { failCnt++; log.warn("indexSpec is not ready: [%s].\nTrying the next indexSpec.", json); @@ -608,13 +600,14 @@ private String createIndexTaskSpecId(int i) return StringUtils.format("%s_%d", getId(), i); } + /** - * Generate {@link ParallelIndexIngestionSpec} from input segments. - * - * @return an empty list if input segments don't exist. Otherwise, a generated ingestionSpec. + * Generate dataschema for segments in each interval + * @return + * @throws IOException */ @VisibleForTesting - static List> createDataSchemasForIntervals( + static List> createDataSchemasForIntervals( final Clock clock, final TaskToolbox toolbox, final LockGranularity lockGranularityInUse, @@ -632,12 +625,12 @@ static List> createDataSchemasForIntervals( lockGranularityInUse ); - if (timelineSegments.size() == 0) { + if (timelineSegments.isEmpty()) { return Collections.emptyList(); } if (granularitySpec == null || granularitySpec.getSegmentGranularity() == null) { - List> dataSchemas = new ArrayList<>(); + List> intervalToDataSchemaList = new ArrayList<>(); // original granularity final Map> intervalToSegments = new TreeMap<>( @@ -690,12 +683,9 @@ static List> createDataSchemasForIntervals( ? new ClientCompactionTaskGranularitySpec(segmentGranularityToUse, null, null) : granularitySpec.withSegmentGranularity(segmentGranularityToUse) ); - - // TODO(vishesh): Based on a function flag, either create parallel index spec OR just aggregate up dataSchemas. We should - // pass a list of schemas to MSQ to have one-to-one mapping between an MSQ controller task and a schema - dataSchemas.add(Pair.of(interval, dataSchema)); + intervalToDataSchemaList.add(new NonnullPair<>(interval, dataSchema)); } - return dataSchemas; + return intervalToDataSchemaList; } else { // given segment granularity final DataSchema dataSchema = createDataSchema( @@ -719,12 +709,18 @@ static List> createDataSchemasForIntervals( metricsSpec, granularitySpec ); - return Collections.singletonList(Pair.of(segmentProvider.interval, dataSchema)); + return Collections.singletonList(new NonnullPair<>(segmentProvider.interval, dataSchema)); } } - private static List createIngestionSpecs( - List> dataschemas, + /** + * Generate {@link ParallelIndexIngestionSpec} from input dataschemas. + * + * @return an empty list if input segments don't exist. Otherwise, a generated ingestionSpec. + */ + @VisibleForTesting + static List createIngestionSpecs( + List> dataschemas, final TaskToolbox toolbox, final CompactionIOConfig ioConfig, final PartitionConfigurationManager partitionConfigurationManager, @@ -1399,7 +1395,6 @@ public CompactionTask build() public static class CompactionTuningConfig extends ParallelIndexTuningConfig { public static final String TYPE = "compaction"; - public final Boolean useMSQ; public static CompactionTuningConfig defaultConfig() { @@ -1434,9 +1429,7 @@ public static CompactionTuningConfig defaultConfig() null, null, 0L, - null, - // TODO vishesh: Move this default to false later - true + null ); } @@ -1473,8 +1466,7 @@ public CompactionTuningConfig( @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions, @JsonProperty("maxColumnsToMerge") @Nullable Integer maxColumnsToMerge, @JsonProperty("awaitSegmentAvailabilityTimeoutMillis") @Nullable Long awaitSegmentAvailabilityTimeoutMillis, - @JsonProperty("numPersistThreads") @Nullable Integer numPersistThreads, - @JsonProperty("useMSQ") @Nullable Boolean useMSQ + @JsonProperty("numPersistThreads") @Nullable Integer numPersistThreads ) { super( @@ -1516,8 +1508,6 @@ public CompactionTuningConfig( awaitSegmentAvailabilityTimeoutMillis == null || awaitSegmentAvailabilityTimeoutMillis == 0, "awaitSegmentAvailabilityTimeoutMillis is not supported for Compcation Task" ); - // TODO vishesh: Move this default to false later - this.useMSQ = useMSQ != null ? useMSQ: true; } @Override @@ -1554,8 +1544,7 @@ public CompactionTuningConfig withPartitionsSpec(PartitionsSpec partitionsSpec) getMaxSavedParseExceptions(), getMaxColumnsToMerge(), getAwaitSegmentAvailabilityTimeoutMillis(), - getNumPersistThreads(), - null + getNumPersistThreads() ); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionToMSQProvider.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionToMSQProvider.java deleted file mode 100644 index 73c8232fad5a..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionToMSQProvider.java +++ /dev/null @@ -1,9 +0,0 @@ -package org.apache.druid.indexing.common.task; - - -import com.google.inject.Provider; - -//@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "compactionType") -public interface CompactionToMSQProvider extends Provider -{ -} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionToMSQ.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionToMSQTask.java similarity index 65% rename from indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionToMSQ.java rename to indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionToMSQTask.java index e6e4d759b2dc..29c999495cdb 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionToMSQ.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionToMSQTask.java @@ -2,18 +2,18 @@ import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.NonnullPair; import org.apache.druid.segment.indexing.DataSchema; import org.joda.time.Interval; import java.util.List; -public interface CompactionToMSQ +public interface CompactionToMSQTask { - TaskStatus createAndRunMSQControllerTasks( + TaskStatus createAndRunMSQTasks( CompactionTask compactionTask, TaskToolbox taskToolbox, - List> dataSchemas + List> dataSchemas ) throws Exception; } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index 3cadb5528ddd..a42d30938ca6 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -80,6 +80,7 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.NonnullPair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; @@ -172,7 +173,7 @@ @RunWith(MockitoJUnitRunner.class) public class CompactionTaskTest { - /*private static final long SEGMENT_SIZE_BYTES = 100; + private static final long SEGMENT_SIZE_BYTES = 100; private static final int NUM_ROWS_PER_SEGMENT = 10; private static final String DATA_SOURCE = "dataSource"; private static final String TIMESTAMP_COLUMN = "timestamp"; @@ -367,7 +368,6 @@ private static CompactionTask.CompactionTuningConfig createTuningConfig() null, null, null, - null, null ); } @@ -790,7 +790,6 @@ public void testGetTuningConfigWithIndexTuningConfig() null, null, null, - null, null ); @@ -877,7 +876,6 @@ public void testGetTuningConfigWithParallelIndexTuningConfig() null, null, null, - null, null ); @@ -955,21 +953,27 @@ public void testSegmentProviderFindSegmentsWithEmptySegmentsThrowException() @Test public void testCreateIngestionSchema() throws IOException { - final List ingestionSpecs = CompactionTask.createDataSchemas( + final List> dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( clock, toolbox, LockGranularity.TIME_CHUNK, - new CompactionIOConfig(null, false, null), new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), - new PartitionConfigurationManager(TUNING_CONFIG), null, null, null, null, - COORDINATOR_CLIENT, - segmentCacheManagerFactory, METRIC_BUILDER ); + + final List ingestionSpecs = CompactionTask.createIngestionSpecs( + dataSchemasForIntervals, + toolbox, + new CompactionIOConfig(null, false, null), + new PartitionConfigurationManager(TUNING_CONFIG), + COORDINATOR_CLIENT, + segmentCacheManagerFactory + ); + final List expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(); ingestionSpecs.sort( @@ -1029,24 +1033,29 @@ public void testCreateIngestionSchemaWithTargetPartitionSize() throws IOExceptio null, null, null, - null, null ); - final List ingestionSpecs = CompactionTask.createDataSchemas( + final List> dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( clock, toolbox, LockGranularity.TIME_CHUNK, - new CompactionIOConfig(null, false, null), new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), - new PartitionConfigurationManager(tuningConfig), null, null, null, null, - COORDINATOR_CLIENT, - segmentCacheManagerFactory, METRIC_BUILDER ); + + final List ingestionSpecs = CompactionTask.createIngestionSpecs( + dataSchemasForIntervals, + toolbox, + new CompactionIOConfig(null, false, null), + new PartitionConfigurationManager(tuningConfig), + COORDINATOR_CLIENT, + segmentCacheManagerFactory + ); + final List expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(); ingestionSpecs.sort( @@ -1107,24 +1116,29 @@ public void testCreateIngestionSchemaWithMaxTotalRows() throws IOException null, null, null, - null, null ); - final List ingestionSpecs = CompactionTask.createDataSchemas( + final List> dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( clock, toolbox, LockGranularity.TIME_CHUNK, - new CompactionIOConfig(null, false, null), new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), - new PartitionConfigurationManager(tuningConfig), null, null, null, null, - COORDINATOR_CLIENT, - segmentCacheManagerFactory, METRIC_BUILDER ); + + final List ingestionSpecs = CompactionTask.createIngestionSpecs( + dataSchemasForIntervals, + toolbox, + new CompactionIOConfig(null, false, null), + new PartitionConfigurationManager(tuningConfig), + COORDINATOR_CLIENT, + segmentCacheManagerFactory + ); + final List expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(); ingestionSpecs.sort( @@ -1185,24 +1199,28 @@ public void testCreateIngestionSchemaWithNumShards() throws IOException null, null, null, - null, null ); - final List ingestionSpecs = CompactionTask.createDataSchemas( + final List> dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( clock, toolbox, LockGranularity.TIME_CHUNK, - new CompactionIOConfig(null, false, null), new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), - new PartitionConfigurationManager(tuningConfig), null, null, null, null, - COORDINATOR_CLIENT, - segmentCacheManagerFactory, METRIC_BUILDER ); + + final List ingestionSpecs = CompactionTask.createIngestionSpecs( + dataSchemasForIntervals, + toolbox, + new CompactionIOConfig(null, false, null), + new PartitionConfigurationManager(tuningConfig), + COORDINATOR_CLIENT, + segmentCacheManagerFactory + ); final List expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(); ingestionSpecs.sort( @@ -1254,22 +1272,27 @@ public void testCreateIngestionSchemaWithCustomDimensionsSpec() throws IOExcepti ) ); - final List ingestionSpecs = CompactionTask.createDataSchemas( + final List> dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( clock, toolbox, LockGranularity.TIME_CHUNK, - new CompactionIOConfig(null, false, null), new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), - new PartitionConfigurationManager(TUNING_CONFIG), customSpec, null, null, null, - COORDINATOR_CLIENT, - segmentCacheManagerFactory, METRIC_BUILDER ); + final List ingestionSpecs = CompactionTask.createIngestionSpecs( + dataSchemasForIntervals, + toolbox, + new CompactionIOConfig(null, false, null), + new PartitionConfigurationManager(TUNING_CONFIG), + COORDINATOR_CLIENT, + segmentCacheManagerFactory + ); + ingestionSpecs.sort( (s1, s2) -> Comparators.intervalsByStartThenEnd().compare( s1.getDataSchema().getGranularitySpec().inputIntervals().get(0), @@ -1300,22 +1323,27 @@ public void testCreateIngestionSchemaWithCustomMetricsSpec() throws IOException new DoubleMaxAggregatorFactory("custom_double_max", "agg_4") }; - final List ingestionSpecs = CompactionTask.createDataSchemas( + final List> dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( clock, toolbox, LockGranularity.TIME_CHUNK, - new CompactionIOConfig(null, false, null), new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), - new PartitionConfigurationManager(TUNING_CONFIG), null, null, customMetricsSpec, null, - COORDINATOR_CLIENT, - segmentCacheManagerFactory, METRIC_BUILDER ); + final List ingestionSpecs = CompactionTask.createIngestionSpecs( + dataSchemasForIntervals, + toolbox, + new CompactionIOConfig(null, false, null), + new PartitionConfigurationManager(TUNING_CONFIG), + COORDINATOR_CLIENT, + segmentCacheManagerFactory + ); + final List expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(); ingestionSpecs.sort( @@ -1339,21 +1367,27 @@ public void testCreateIngestionSchemaWithCustomMetricsSpec() throws IOException @Test public void testCreateIngestionSchemaWithCustomSegments() throws IOException { - final List ingestionSpecs = CompactionTask.createDataSchemas( + final List> dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( clock, toolbox, LockGranularity.TIME_CHUNK, - new CompactionIOConfig(null, false, null), - new SegmentProvider(DATA_SOURCE, SpecificSegmentsSpec.fromSegments(SEGMENTS)), - new PartitionConfigurationManager(TUNING_CONFIG), + new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), null, null, null, null, - COORDINATOR_CLIENT, - segmentCacheManagerFactory, METRIC_BUILDER ); + + final List ingestionSpecs = CompactionTask.createIngestionSpecs( + dataSchemasForIntervals, + toolbox, + new CompactionIOConfig(null, false, null), + new PartitionConfigurationManager(TUNING_CONFIG), + COORDINATOR_CLIENT, + segmentCacheManagerFactory + ); + final List expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(); ingestionSpecs.sort( @@ -1384,21 +1418,26 @@ public void testCreateIngestionSchemaWithDifferentSegmentSet() throws IOExceptio Collections.sort(segments); // Remove one segment in the middle segments.remove(segments.size() / 2); - CompactionTask.createDataSchemas( + final List> dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( clock, toolbox, LockGranularity.TIME_CHUNK, - new CompactionIOConfig(null, false, null), new SegmentProvider(DATA_SOURCE, SpecificSegmentsSpec.fromSegments(segments)), - new PartitionConfigurationManager(TUNING_CONFIG), null, null, null, null, - COORDINATOR_CLIENT, - segmentCacheManagerFactory, METRIC_BUILDER ); + + CompactionTask.createIngestionSpecs( + dataSchemasForIntervals, + toolbox, + new CompactionIOConfig(null, false, null), + new PartitionConfigurationManager(TUNING_CONFIG), + COORDINATOR_CLIENT, + segmentCacheManagerFactory + ); } @Test @@ -1410,21 +1449,26 @@ public void testMissingMetadata() throws IOException final TestIndexIO indexIO = (TestIndexIO) toolbox.getIndexIO(); indexIO.removeMetadata(Iterables.getFirst(indexIO.getQueryableIndexMap().keySet(), null)); final List segments = new ArrayList<>(SEGMENTS); - CompactionTask.createDataSchemas( + final List> dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( clock, toolbox, LockGranularity.TIME_CHUNK, - new CompactionIOConfig(null, false, null), new SegmentProvider(DATA_SOURCE, SpecificSegmentsSpec.fromSegments(segments)), - new PartitionConfigurationManager(TUNING_CONFIG), null, null, null, null, - COORDINATOR_CLIENT, - segmentCacheManagerFactory, METRIC_BUILDER ); + + CompactionTask.createIngestionSpecs( + dataSchemasForIntervals, + toolbox, + new CompactionIOConfig(null, false, null), + new PartitionConfigurationManager(TUNING_CONFIG), + COORDINATOR_CLIENT, + segmentCacheManagerFactory + ); } @Test @@ -1448,21 +1492,26 @@ public void testEmptyInterval() @Test public void testSegmentGranularityAndNullQueryGranularity() throws IOException { - final List ingestionSpecs = CompactionTask.createDataSchemas( + final List> dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( clock, toolbox, LockGranularity.TIME_CHUNK, - new CompactionIOConfig(null, false, null), new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), - new PartitionConfigurationManager(TUNING_CONFIG), null, null, null, new ClientCompactionTaskGranularitySpec(new PeriodGranularity(Period.months(3), null, null), null, null), - COORDINATOR_CLIENT, - segmentCacheManagerFactory, METRIC_BUILDER ); + + final List ingestionSpecs = CompactionTask.createIngestionSpecs( + dataSchemasForIntervals, + toolbox, + new CompactionIOConfig(null, false, null), + new PartitionConfigurationManager(TUNING_CONFIG), + COORDINATOR_CLIENT, + segmentCacheManagerFactory + ); final List expectedDimensionsSpec = ImmutableList.of( new DimensionsSpec(getDimensionSchema(new DoubleDimensionSchema("string_to_double"))) ); @@ -1488,21 +1537,25 @@ public void testSegmentGranularityAndNullQueryGranularity() throws IOException @Test public void testQueryGranularityAndNullSegmentGranularity() throws IOException { - final List ingestionSpecs = CompactionTask.createDataSchemas( + final List> dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( clock, toolbox, LockGranularity.TIME_CHUNK, - new CompactionIOConfig(null, false, null), new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), - new PartitionConfigurationManager(TUNING_CONFIG), null, null, null, new ClientCompactionTaskGranularitySpec(null, new PeriodGranularity(Period.months(3), null, null), null), - COORDINATOR_CLIENT, - segmentCacheManagerFactory, METRIC_BUILDER ); + final List ingestionSpecs = CompactionTask.createIngestionSpecs( + dataSchemasForIntervals, + toolbox, + new CompactionIOConfig(null, false, null), + new PartitionConfigurationManager(TUNING_CONFIG), + COORDINATOR_CLIENT, + segmentCacheManagerFactory + ); final List expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(); ingestionSpecs.sort( @@ -1526,13 +1579,11 @@ public void testQueryGranularityAndNullSegmentGranularity() throws IOException @Test public void testQueryGranularityAndSegmentGranularityNonNull() throws IOException { - final List ingestionSpecs = CompactionTask.createDataSchemas( + final List> dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( clock, toolbox, LockGranularity.TIME_CHUNK, - new CompactionIOConfig(null, false, null), new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), - new PartitionConfigurationManager(TUNING_CONFIG), null, null, null, @@ -1541,10 +1592,19 @@ public void testQueryGranularityAndSegmentGranularityNonNull() throws IOExceptio new PeriodGranularity(Period.months(3), null, null), null ), - COORDINATOR_CLIENT, - segmentCacheManagerFactory, METRIC_BUILDER ); + + final List ingestionSpecs = CompactionTask.createIngestionSpecs( + dataSchemasForIntervals, + toolbox, + new CompactionIOConfig(null, false, null), + new PartitionConfigurationManager(TUNING_CONFIG), + COORDINATOR_CLIENT, + segmentCacheManagerFactory + ); + + final List expectedDimensionsSpec = ImmutableList.of( new DimensionsSpec(getDimensionSchema(new DoubleDimensionSchema("string_to_double"))) ); @@ -1571,21 +1631,28 @@ public void testQueryGranularityAndSegmentGranularityNonNull() throws IOExceptio @Test public void testNullGranularitySpec() throws IOException { - final List ingestionSpecs = CompactionTask.createDataSchemas( + final List> dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( clock, toolbox, LockGranularity.TIME_CHUNK, - new CompactionIOConfig(null, false, null), new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), - new PartitionConfigurationManager(TUNING_CONFIG), null, null, null, null, - COORDINATOR_CLIENT, - segmentCacheManagerFactory, METRIC_BUILDER ); + + final List ingestionSpecs = CompactionTask.createIngestionSpecs( + dataSchemasForIntervals, + toolbox, + new CompactionIOConfig(null, false, null), + new PartitionConfigurationManager(TUNING_CONFIG), + COORDINATOR_CLIENT, + segmentCacheManagerFactory + ); + + final List expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(); ingestionSpecs.sort( @@ -1610,21 +1677,28 @@ public void testNullGranularitySpec() throws IOException public void testGranularitySpecWithNullQueryGranularityAndNullSegmentGranularity() throws IOException { - final List ingestionSpecs = CompactionTask.createDataSchemas( + final List> dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( clock, toolbox, LockGranularity.TIME_CHUNK, - new CompactionIOConfig(null, false, null), new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), - new PartitionConfigurationManager(TUNING_CONFIG), null, null, null, new ClientCompactionTaskGranularitySpec(null, null, null), - COORDINATOR_CLIENT, - segmentCacheManagerFactory, METRIC_BUILDER ); + + final List ingestionSpecs = CompactionTask.createIngestionSpecs( + dataSchemasForIntervals, + toolbox, + new CompactionIOConfig(null, false, null), + new PartitionConfigurationManager(TUNING_CONFIG), + COORDINATOR_CLIENT, + segmentCacheManagerFactory + ); + + final List expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(); ingestionSpecs.sort( @@ -1649,22 +1723,27 @@ public void testGranularitySpecWithNullQueryGranularityAndNullSegmentGranularity public void testGranularitySpecWithNotNullRollup() throws IOException { - final List ingestionSpecs = CompactionTask.createDataSchemas( + final List> dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( clock, toolbox, LockGranularity.TIME_CHUNK, - new CompactionIOConfig(null, false, null), new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), - new PartitionConfigurationManager(TUNING_CONFIG), null, null, null, new ClientCompactionTaskGranularitySpec(null, null, true), - COORDINATOR_CLIENT, - segmentCacheManagerFactory, METRIC_BUILDER ); + final List ingestionSpecs = CompactionTask.createIngestionSpecs( + dataSchemasForIntervals, + toolbox, + new CompactionIOConfig(null, false, null), + new PartitionConfigurationManager(TUNING_CONFIG), + COORDINATOR_CLIENT, + segmentCacheManagerFactory + ); + Assert.assertEquals(6, ingestionSpecs.size()); for (ParallelIndexIngestionSpec indexIngestionSpec : ingestionSpecs) { Assert.assertTrue(indexIngestionSpec.getDataSchema().getGranularitySpec().isRollup()); @@ -1675,21 +1754,28 @@ public void testGranularitySpecWithNotNullRollup() public void testGranularitySpecWithNullRollup() throws IOException { - final List ingestionSpecs = CompactionTask.createDataSchemas( + final List> dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( clock, toolbox, LockGranularity.TIME_CHUNK, - new CompactionIOConfig(null, false, null), new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), - new PartitionConfigurationManager(TUNING_CONFIG), null, null, null, new ClientCompactionTaskGranularitySpec(null, null, null), - COORDINATOR_CLIENT, - segmentCacheManagerFactory, METRIC_BUILDER ); + + final List ingestionSpecs = CompactionTask.createIngestionSpecs( + dataSchemasForIntervals, + toolbox, + new CompactionIOConfig(null, false, null), + new PartitionConfigurationManager(TUNING_CONFIG), + COORDINATOR_CLIENT, + segmentCacheManagerFactory + ); + + Assert.assertEquals(6, ingestionSpecs.size()); for (ParallelIndexIngestionSpec indexIngestionSpec : ingestionSpecs) { //Expect false since rollup value in metadata of existing segments are null @@ -1848,7 +1934,6 @@ private void assertIngestionSchema( null, null, null, - null, null ), expectedSegmentGranularity, @@ -2162,11 +2247,11 @@ public SettableColumnValueSelector makeNewSettableColumnValueSelector() } - *//** + /** * The compaction task spec in 0.16.0 except for the tuningConfig. * The original spec accepts only {@link IndexTuningConfig}, but this class acceps any type of tuningConfig for * testing. - *//* + */ private static class OldCompactionTaskWithAnyTuningConfigType extends AbstractTask { private final Interval interval; @@ -2341,5 +2426,5 @@ public ColumnType getColumnType() { return ColumnType.ofComplex(ExtensionDimensionHandler.TYPE_NAME); } - }*/ + } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTuningConfigTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTuningConfigTest.java index d36ea8d13816..6831ea4adb60 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTuningConfigTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTuningConfigTest.java @@ -103,7 +103,6 @@ public void testSerdeWithNonZeroAwaitSegmentAvailabilityTimeoutMillis() null, null, 5L, - null, null ); } @@ -147,7 +146,6 @@ public void testSerdeWithZeroAwaitSegmentAvailabilityTimeoutMillis() null, null, 0L, - null, null ); Assert.assertEquals(0L, tuningConfig.getAwaitSegmentAvailabilityTimeoutMillis()); @@ -192,7 +190,6 @@ public void testSerdeWithNullAwaitSegmentAvailabilityTimeoutMillis() null, null, null, - null, null ); Assert.assertEquals(0L, tuningConfig.getAwaitSegmentAvailabilityTimeoutMillis()); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskToolboxFactory.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskToolboxFactory.java index 5edef5dada0a..2995f83f6838 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskToolboxFactory.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskToolboxFactory.java @@ -36,7 +36,7 @@ import org.apache.druid.indexing.common.actions.TaskActionClientFactory; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.config.TaskConfigBuilder; -import org.apache.druid.indexing.common.task.CompactionToMSQ; +import org.apache.druid.indexing.common.task.CompactionToMSQTask; import org.apache.druid.indexing.common.task.NoopTestTaskReportFileWriter; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTaskClientProvider; import org.apache.druid.indexing.common.task.batch.parallel.ShuffleClient; @@ -119,7 +119,7 @@ public TestTaskToolboxFactory( bob.taskLogPusher, bob.attemptId, bob.centralizedDatasourceSchemaConfig, - bob.compactionToMSQ + bob.compactionToMSQTask ); } @@ -164,7 +164,7 @@ public static class Builder private TaskLogPusher taskLogPusher; private String attemptId; private CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; - private CompactionToMSQ compactionToMSQ; + private CompactionToMSQTask compactionToMSQTask; public Builder setConfig(TaskConfig config) { @@ -400,8 +400,8 @@ public Builder setCentralizedTableSchemaConfig(CentralizedDatasourceSchemaConfig return this; } - public Builder setCompactionToMSQ(CompactionToMSQ compactionToMSQ){ - this.compactionToMSQ = compactionToMSQ; + public Builder setCompactionToMSQ(CompactionToMSQTask compactionToMSQTask){ + this.compactionToMSQTask = compactionToMSQTask; return this; } } diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/CompactionUtil.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/CompactionUtil.java index a51f5122bcd9..63316e76687d 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/CompactionUtil.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/CompactionUtil.java @@ -20,11 +20,7 @@ package org.apache.druid.testing.utils; import org.apache.druid.data.input.MaxSizeSplitHintSpec; -import org.apache.druid.error.InvalidInput; -import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; -import org.apache.druid.indexer.partitions.PartitionsSpec; -import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.UserCompactionTaskIOConfig; import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfig.java index 6c97f81eaa77..e03772c534dc 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfig.java @@ -59,27 +59,25 @@ public static CoordinatorCompactionConfig from( CoordinatorCompactionConfig baseConfig, @Nullable Double compactionTaskSlotRatio, @Nullable Integer maxCompactionTaskSlots, - @Nullable Boolean useAutoScaleSlots, - @Nullable DataSourceCompactionConfig.Engine engine + @Nullable Boolean useAutoScaleSlots ) { return new CoordinatorCompactionConfig( baseConfig.compactionConfigs, compactionTaskSlotRatio == null ? baseConfig.compactionTaskSlotRatio : compactionTaskSlotRatio, maxCompactionTaskSlots == null ? baseConfig.maxCompactionTaskSlots : maxCompactionTaskSlots, - useAutoScaleSlots == null ? baseConfig.useAutoScaleSlots : useAutoScaleSlots, - engine == null ? baseConfig.engine : engine + useAutoScaleSlots == null ? baseConfig.useAutoScaleSlots : useAutoScaleSlots ); } public static CoordinatorCompactionConfig from(List compactionConfigs) { - return new CoordinatorCompactionConfig(compactionConfigs, null, null, null, null); + return new CoordinatorCompactionConfig(compactionConfigs, null, null, null); } public static CoordinatorCompactionConfig empty() { - return new CoordinatorCompactionConfig(ImmutableList.of(), null, null, null, null); + return new CoordinatorCompactionConfig(ImmutableList.of(), null, null, null); } @JsonCreator @@ -87,8 +85,7 @@ public CoordinatorCompactionConfig( @JsonProperty("compactionConfigs") List compactionConfigs, @JsonProperty("compactionTaskSlotRatio") @Nullable Double compactionTaskSlotRatio, @JsonProperty("maxCompactionTaskSlots") @Nullable Integer maxCompactionTaskSlots, - @JsonProperty("useAutoScaleSlots") @Nullable Boolean useAutoScaleSlots, - @JsonProperty("engine") @Nullable final DataSourceCompactionConfig.Engine engine + @JsonProperty("useAutoScaleSlots") @Nullable Boolean useAutoScaleSlots ) { this.compactionConfigs = compactionConfigs; @@ -99,7 +96,7 @@ public CoordinatorCompactionConfig( DEFAULT_MAX_COMPACTION_TASK_SLOTS : maxCompactionTaskSlots; this.useAutoScaleSlots = useAutoScaleSlots == null ? DEFAULT_USE_AUTO_SCALE_SLOTS : useAutoScaleSlots; - this.engine = engine == null ? DEFAULT_COMPACTION_ENGINE : engine; + this.engine = DEFAULT_COMPACTION_ENGINE; } @JsonProperty diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java index 8b737703b2f2..b08d56d78481 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java @@ -37,9 +37,7 @@ public class DataSourceCompactionConfig { - /** - * Must be synced with Tasks.DEFAULT_MERGE_TASK_PRIORITY - */ + /** Must be synced with Tasks.DEFAULT_MERGE_TASK_PRIORITY */ public static final int DEFAULT_COMPACTION_TASK_PRIORITY = 25; // Approx. 100TB. Chosen instead of Long.MAX_VALUE to avoid overflow on web-console and other clients private static final long DEFAULT_INPUT_SEGMENT_SIZE_BYTES = 100_000_000_000_000L; @@ -77,8 +75,8 @@ public DataSourceCompactionConfig( @JsonProperty("metricsSpec") @Nullable AggregatorFactory[] metricsSpec, @JsonProperty("transformSpec") @Nullable UserCompactionTaskTransformConfig transformSpec, @JsonProperty("ioConfig") @Nullable UserCompactionTaskIOConfig ioConfig, - @JsonProperty("taskContext") @Nullable Map taskContext, - @JsonProperty("engine") @Nullable Engine engine + @JsonProperty("engine") @Nullable Engine engine, + @JsonProperty("taskContext") @Nullable Map taskContext ) { this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource"); @@ -250,7 +248,7 @@ public static DataSourceCompactionConfig from(Engine defaultEngine, DataSourceCo || partitionsSpec instanceof DynamicPartitionsSpec)) { throw InvalidInput.exception( "Invalid partition spec type[%s] for MSQ compaction engine %s." - + " Type must be either DynamicPartitionsSpec or DynamicRangePartitionsSpec .", + + " Type must be either DynamicPartitionsSpec or DynamicRangePartitionsSpec.", partitionsSpec.getClass(), engineSourceLog ); @@ -276,13 +274,13 @@ public static DataSourceCompactionConfig from(Engine defaultEngine, DataSourceCo currentConfig.getMetricsSpec(), currentConfig.getTransformSpec(), currentConfig.getIoConfig(), - currentConfig.getTaskContext(), - newEngine + newEngine, currentConfig.getTaskContext() ); } /** * Engine to be used for a compaction task. + * Should be synchronized with {@link org.apache.druid.indexing.common.task.Engine}. */ public enum Engine { diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index 76f071672edb..ed2f3eda26a4 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java @@ -466,18 +466,20 @@ private int submitCompactionTasks( } DataSourceCompactionConfig.Engine engine = config.getEngine(); - - Map autoCompactionContext = config.getTaskContext(); + Map autoCompactionContext = newAutoCompactionContext(config.getTaskContext()); int numCurrentCompactionTasksAndSubtasks; - if (engine == DataSourceCompactionConfig.Engine.MSQ && autoCompactionContext!= null && !autoCompactionContext.containsKey("maxNumTasks")) { - autoCompactionContext = newAutoCompactionContext(autoCompactionContext); - numCurrentCompactionTasksAndSubtasks = numAvailableCompactionTaskSlots; - // todo (vishesh): there was a plan to use auto strategy for task slots calculation. How to introduce that - // and then how to retrieve that number to manage underneath calculation - // also check if there is a need to clone the map here - autoCompactionContext.putIfAbsent("maxNumTasks", numCurrentCompactionTasksAndSubtasks); + final String maxNumTasksContextParam = "maxNumTasks"; + + if (engine == DataSourceCompactionConfig.Engine.MSQ) { + if (!autoCompactionContext.containsKey(maxNumTasksContextParam)) { + // Since MSQ needs all task slots for the calculated #tasks to be available upfront, allot all available + // compaction slots to current compaction task to avoid stalling. + numCurrentCompactionTasksAndSubtasks = numAvailableCompactionTaskSlots; + autoCompactionContext.put(maxNumTasksContextParam, numCurrentCompactionTasksAndSubtasks); + } else { + numCurrentCompactionTasksAndSubtasks = (int) autoCompactionContext.get(maxNumTasksContextParam); } - else { + } else { numCurrentCompactionTasksAndSubtasks = findMaxNumTaskSlotsUsedByOneNativeCompactionTask(config.getTuningConfig()); } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionStrategy.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionStrategy.java deleted file mode 100644 index b3b5923c6c65..000000000000 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactionStrategy.java +++ /dev/null @@ -1,15 +0,0 @@ -package org.apache.druid.server.coordinator.duty; - -import com.google.inject.BindingAnnotation; - -import java.lang.annotation.ElementType; -import java.lang.annotation.Retention; -import java.lang.annotation.RetentionPolicy; -import java.lang.annotation.Target; - -@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD}) -@Retention(RetentionPolicy.RUNTIME) -@BindingAnnotation -public @interface CompactionStrategy -{ -} diff --git a/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java index c2a26cc18811..f6cb1ae84ccd 100644 --- a/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java +++ b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java @@ -101,32 +101,11 @@ public Response setCompactionTaskLimit( current, compactionTaskSlotRatio, maxCompactionTaskSlots, - useAutoScaleSlots, - null + useAutoScaleSlots ); return updateConfigHelper(operator, AuthorizationUtils.buildAuditInfo(req)); } - @POST - @Path("/engine") - @Consumes(MediaType.APPLICATION_JSON) - public Response setCompactionEngine( - @QueryParam("engine") DataSourceCompactionConfig.Engine engine, - @Context HttpServletRequest req - ) - { - UnaryOperator operator = - current -> CoordinatorCompactionConfig.from( - current, - null, - null, - null, - engine - ); - // TODO (vishesh): add the check to loop over all configs and validate if this update is compatible. - return updateConfigHelper(operator, AuthorizationUtils.buildAuditInfo(req)); - } - @POST @Consumes(MediaType.APPLICATION_JSON) public Response addOrUpdateCompactionConfig( @@ -140,8 +119,8 @@ public Response addOrUpdateCompactionConfig( .getCompactionConfigs() .stream() .collect(Collectors.toMap(DataSourceCompactionConfig::getDataSource, Function.identity())); - DataSourceCompactionConfig updateConfig = DataSourceCompactionConfig.from(current.getEngine(), newConfig); - newConfigs.put(updateConfig.getDataSource(), updateConfig); + DataSourceCompactionConfig updatedConfig = DataSourceCompactionConfig.from(current.getEngine(), newConfig); + newConfigs.put(updatedConfig.getDataSource(), updatedConfig); newCompactionConfig = CoordinatorCompactionConfig.from(current, ImmutableList.copyOf(newConfigs.values())); return newCompactionConfig; diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigTest.java index 6eb4e91db306..4df2bb519f3b 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigTest.java @@ -69,8 +69,8 @@ public void testSerdeBasic() throws IOException null, null, null, - ImmutableMap.of("key", "val"), - null + null, + ImmutableMap.of("key", "val") ); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -100,8 +100,8 @@ public void testSerdeWithMaxRowsPerSegment() throws IOException null, null, null, - ImmutableMap.of("key", "val"), - null + null, + ImmutableMap.of("key", "val") ); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -150,8 +150,8 @@ public void testSerdeWithMaxTotalRows() throws IOException null, null, null, - ImmutableMap.of("key", "val"), - null + null, + ImmutableMap.of("key", "val") ); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -200,8 +200,8 @@ public void testSerdeMaxTotalRowsWithMaxRowsPerSegment() throws IOException null, null, null, - ImmutableMap.of("key", "val"), - null + null, + ImmutableMap.of("key", "val") ); final String json = OBJECT_MAPPER.writeValueAsString(config); @@ -309,8 +309,8 @@ public void testSerdeGranularitySpec() throws IOException null, null, null, - ImmutableMap.of("key", "val"), - null + null, + ImmutableMap.of("key", "val") ); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -340,8 +340,8 @@ public void testSerdeGranularitySpecWithQueryGranularity() throws Exception null, null, null, - ImmutableMap.of("key", "val"), - null + null, + ImmutableMap.of("key", "val") ); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -374,8 +374,8 @@ public void testSerdeWithNullGranularitySpec() throws IOException null, null, null, - ImmutableMap.of("key", "val"), - null + null, + ImmutableMap.of("key", "val") ); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -405,8 +405,8 @@ public void testSerdeGranularitySpecWithNullValues() throws IOException null, null, null, - ImmutableMap.of("key", "val"), - null + null, + ImmutableMap.of("key", "val") ); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -436,8 +436,8 @@ public void testSerdeGranularitySpecWithRollup() throws IOException null, null, null, - ImmutableMap.of("key", "val"), - null + null, + ImmutableMap.of("key", "val") ); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -470,8 +470,8 @@ public void testSerdeIOConfigWithNonNullDropExisting() throws IOException null, null, new UserCompactionTaskIOConfig(true), - ImmutableMap.of("key", "val"), - null + null, + ImmutableMap.of("key", "val") ); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -502,8 +502,8 @@ public void testSerdeIOConfigWithNullDropExisting() throws IOException null, null, new UserCompactionTaskIOConfig(null), - ImmutableMap.of("key", "val"), - null + null, + ImmutableMap.of("key", "val") ); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -534,8 +534,8 @@ public void testSerdeDimensionsSpec() throws IOException null, null, null, - ImmutableMap.of("key", "val"), - null + null, + ImmutableMap.of("key", "val") ); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -566,8 +566,8 @@ public void testSerdeTransformSpec() throws IOException null, new UserCompactionTaskTransformConfig(new SelectorDimFilter("dim1", "foo", null)), null, - ImmutableMap.of("key", "val"), - null + null, + ImmutableMap.of("key", "val") ); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -597,8 +597,8 @@ public void testSerdeMetricsSpec() throws IOException new AggregatorFactory[] {new CountAggregatorFactory("cnt")}, null, null, - ImmutableMap.of("key", "val"), - null + null, + ImmutableMap.of("key", "val") ); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java index 76021c1b3b2f..db8fdce20add 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java @@ -1797,8 +1797,7 @@ private CoordinatorRunStats doCompactSegments( compactionConfigs, numCompactionTaskSlots == null ? null : 1.0, // 100% when numCompactionTaskSlots is not null numCompactionTaskSlots, - useAutoScaleSlots, - null + useAutoScaleSlots ) ) .build(); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/KillCompactionConfigTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/KillCompactionConfigTest.java index 03809389baec..1bb5d7d74b08 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/KillCompactionConfigTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/KillCompactionConfigTest.java @@ -192,8 +192,8 @@ public void testRunRemoveInactiveDatasourceCompactionConfig() null, null, null, - ImmutableMap.of("key", "val"), - null + null, + ImmutableMap.of("key", "val") ); DataSourceCompactionConfig activeDatasourceConfig = new DataSourceCompactionConfig( @@ -208,8 +208,8 @@ public void testRunRemoveInactiveDatasourceCompactionConfig() null, null, null, - ImmutableMap.of("key", "val"), - null + null, + ImmutableMap.of("key", "val") ); CoordinatorCompactionConfig originalCurrentConfig = CoordinatorCompactionConfig.from(ImmutableList.of(inactiveDatasourceConfig, activeDatasourceConfig)); byte[] originalCurrentConfigBytes = {1, 2, 3}; @@ -295,8 +295,8 @@ public void testRunRetryForRetryableException() null, null, null, - ImmutableMap.of("key", "val"), - null + null, + ImmutableMap.of("key", "val") ); CoordinatorCompactionConfig originalCurrentConfig = CoordinatorCompactionConfig.from(ImmutableList.of(inactiveDatasourceConfig)); diff --git a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java index 6a5cf9b1cc26..1dfda54c25c6 100644 --- a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java @@ -62,8 +62,8 @@ public class CoordinatorCompactionConfigsResourceTest null, null, null, - ImmutableMap.of("key", "val"), - null + null, + ImmutableMap.of("key", "val") ); private static final DataSourceCompactionConfig NEW_CONFIG = new DataSourceCompactionConfig( "newDataSource", @@ -77,8 +77,8 @@ public class CoordinatorCompactionConfigsResourceTest null, null, null, - ImmutableMap.of("key", "val"), - null + null, + ImmutableMap.of("key", "val") ); private static final byte[] OLD_CONFIG_IN_BYTES = {1, 2, 3}; @@ -191,8 +191,8 @@ public void testAddOrUpdateCompactionConfigWithExistingConfig() null, null, null, - ImmutableMap.of("key", "val"), - null + null, + ImmutableMap.of("key", "val") ); Response result = coordinatorCompactionConfigsResource.addOrUpdateCompactionConfig( newConfig, @@ -233,8 +233,8 @@ public void testDeleteCompactionConfigWithExistingConfig() null, null, null, - ImmutableMap.of("key", "val"), - null + null, + ImmutableMap.of("key", "val") ); final CoordinatorCompactionConfig originalConfig = CoordinatorCompactionConfig.from(ImmutableList.of(toDelete)); Mockito.when(mockJacksonConfigManager.convertByteToConfig( @@ -392,8 +392,8 @@ public void testAddOrUpdateCompactionConfigWithoutExistingConfig() null, null, null, - ImmutableMap.of("key", "val"), - null + null, + ImmutableMap.of("key", "val") ); String author = "maytas"; String comment = "hello"; From 177bea9e71909a34b0798680e3b88cca0f1acb79 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Mon, 22 Apr 2024 12:48:08 +0530 Subject: [PATCH 10/54] Precondition checks and updates to match lastCompactionState with compaction config --- .../compaction/CompactionToMSQTaskImpl.java | 138 ++++++++++++++---- .../apache/druid/msq/exec/ControllerImpl.java | 15 +- .../common/task/CompactionToMSQTask.java | 19 +++ .../DataSourceCompactionConfig.java | 40 ++--- .../sql/calcite/parser/DruidSqlInsert.java | 1 + 5 files changed, 167 insertions(+), 46 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQTaskImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQTaskImpl.java index 1146abde4756..a16aa7ad21e9 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQTaskImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQTaskImpl.java @@ -34,13 +34,14 @@ import org.apache.druid.indexing.common.task.CompactionTask; import org.apache.druid.indexing.common.task.CompactionToMSQTask; import org.apache.druid.indexing.common.task.CurrentSubTaskHolder; -import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.java.util.common.NonnullPair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.AllGranularity; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.java.util.common.granularity.PeriodGranularity; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.msq.indexing.MSQControllerTask; import org.apache.druid.msq.indexing.MSQSpec; import org.apache.druid.msq.indexing.MSQTuningConfig; @@ -52,17 +53,22 @@ import org.apache.druid.query.QueryContext; import org.apache.druid.query.TableDataSource; import org.apache.druid.query.dimension.DefaultDimensionSpec; +import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.query.expression.TimestampFloorExprMacro; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.rpc.indexing.OverlordClient; +import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.sql.calcite.parser.DruidSqlInsert; -import org.apache.druid.sql.calcite.parser.DruidSqlReplace; +import org.apache.druid.sql.calcite.planner.ColumnMapping; import org.apache.druid.sql.calcite.planner.ColumnMappings; import org.joda.time.Interval; @@ -81,6 +87,10 @@ public class CompactionToMSQTaskImpl implements CompactionToMSQTask final OverlordClient overlordClient; final ObjectMapper jsonMapper; + private static final String TIME_VIRTUAL_COLUMN = "v0"; + private static final String TIME_COLUMN = ColumnHolder.TIME_COLUMN_NAME; + + public CompactionToMSQTaskImpl(final OverlordClient overlordClient, final ObjectMapper jsonMapper) { this.overlordClient = overlordClient; @@ -97,6 +107,51 @@ private static RowSignature getRowSignature(DataSchema dataSchema) return rowSignatureBuilder.build(); } + private static List getAggregateDimensions(DataSchema ds) + { + List dimensionSpecs = ds.getDimensionsSpec().getDimensions().stream() + .map(dim -> new DefaultDimensionSpec( + dim.getName(), + dim.getName(), + dim.getColumnType() + )) + .collect(Collectors.toList()); + + + + // Dimensions in group-by aren't allowed to have time column as the output name. + if (isQueryGranularityEmpty(ds)){ + dimensionSpecs.add(new DefaultDimensionSpec(TIME_COLUMN, TIME_VIRTUAL_COLUMN, ColumnType.LONG)); + } else { + // The changed granularity would result in a new virtual column that needs to be aggregated upon. + dimensionSpecs.add(new DefaultDimensionSpec(TIME_VIRTUAL_COLUMN, TIME_VIRTUAL_COLUMN, ColumnType.LONG)); + } + return dimensionSpecs; + } + + private static ColumnMappings getColumnMappings(DataSchema dataSchema) + { + List columnMappings = dataSchema.getDimensionsSpec() + .getDimensions() + .stream() + .map(dim -> new ColumnMapping( + dim.getName(), dim.getName())) + .collect(Collectors.toList()); + columnMappings.addAll(Arrays.stream(dataSchema.getAggregators()) + .map(agg -> new ColumnMapping(agg.getName(), agg.getName())) + .collect( + Collectors.toList())); + if (isGroupBy(dataSchema)) { + // For group-by queries, time will always be one of the dimension. Since dimensions in groupby aren't allowed to + // have time column as the output name, we map time dimension to a fixed column name in dimensions, and map it + // back to the time column here. + columnMappings.add(new ColumnMapping(TIME_VIRTUAL_COLUMN, TIME_COLUMN)); + } else { + columnMappings.add(new ColumnMapping(TIME_COLUMN, TIME_COLUMN)); + } + return new ColumnMappings(columnMappings); + } + private static MultipleIntervalSegmentSpec createMultipleIntervalSpec(Interval interval) { return new MultipleIntervalSegmentSpec(Collections.singletonList(interval)); @@ -126,26 +181,50 @@ private static Query buildScanQuery(CompactionTask compactionTask, Interval i .build(); } - private static Query buildGroupByQuery(CompactionTask compactionTask, Interval interval, DataSchema dataSchema) + private static boolean isGroupBy(DataSchema ds) + { + return ds.getAggregators().length > 0; + } + + private static boolean isQueryGranularityEmpty(DataSchema ds) { + return ds.getGranularitySpec() == null + || ds.getGranularitySpec().getQueryGranularity() == null; + } + private static VirtualColumns getVirtualColumns(DataSchema ds) { + VirtualColumns virtualColumns = VirtualColumns.EMPTY; + + if (!isQueryGranularityEmpty(ds) && !ds.getGranularitySpec().getQueryGranularity().equals(Granularities.ALL)) { + PeriodGranularity periodQueryGranularity = (PeriodGranularity) ds.getGranularitySpec() + .getQueryGranularity(); + VirtualColumn virtualColumn = new ExpressionVirtualColumn( + TIME_VIRTUAL_COLUMN, + StringUtils.format( + "timestamp_floor(\"%s\", '%s')", + TIME_COLUMN, + periodQueryGranularity.getPeriod() + .toString() + ), + ColumnType.LONG, + new ExprMacroTable(Collections.singletonList(new TimestampFloorExprMacro())) + ); + virtualColumns = VirtualColumns.create(virtualColumn); + } + return virtualColumns; + } + private static Query buildGroupByQuery( + CompactionTask compactionTask, Interval interval, DataSchema dataSchema + ) + { DimFilter dimFilter = dataSchema.getTransformSpec().getFilter(); GroupByQuery.Builder builder = new GroupByQuery.Builder() .setDataSource(new TableDataSource(compactionTask.getDataSource())) - .setVirtualColumns(VirtualColumns.EMPTY) + .setVirtualColumns(getVirtualColumns(dataSchema)) .setDimFilter(dimFilter) - .setGranularity(dataSchema.getGranularitySpec().getQueryGranularity() != null - ? dataSchema.getGranularitySpec() - .getQueryGranularity() - : new AllGranularity() - ) - .setDimensions(dataSchema.getDimensionsSpec() - .getDimensions() - .stream() - .map(d -> new DefaultDimensionSpec(d.getName(), d.getName())) - .collect(Collectors.toList()) - ) + .setGranularity(new AllGranularity()) + .setDimensions(getAggregateDimensions(dataSchema)) .setAggregatorSpecs(Arrays.asList(dataSchema.getAggregators())) .setContext(compactionTask.getContext()) .setInterval(interval); @@ -164,9 +243,15 @@ public TaskStatus createAndRunMSQTasks( ) throws Exception { + List msqControllerTasks = new ArrayList<>(); QueryContext compactionTaskContext = new QueryContext(compactionTask.getContext()); + if (!MultiStageQueryContext.isFinalizeAggregations(compactionTaskContext)) { + throw InvalidInput.exception( + "finalizeAggregations=false currently not supported for auto-compaction with MSQ engine."); + } + for (NonnullPair intervalDataSchema : intervalDataSchemas) { Query query; Interval interval = intervalDataSchema.lhs; @@ -189,7 +274,7 @@ public TaskStatus createAndRunMSQTasks( if (maxNumTasks < 2) { throw InvalidInput.exception( "MSQ context maxNumTasks [%,d] cannot be less than 2, " - + "since at least 1 controller and 1 worker is necessary", + + "since at least 1 controller and 1 worker is necessary.", maxNumTasks ); } @@ -197,8 +282,7 @@ public TaskStatus createAndRunMSQTasks( final int maxNumWorkers = maxNumTasks - 1; final int maxRowsInMemory = MultiStageQueryContext.getRowsInMemory(compactionTaskContext); - boolean isGroupBy = ds.getAggregators().length > 0; - if (!isGroupBy) { + if (!isGroupBy(ds)) { query = buildScanQuery(compactionTask, interval, ds); } else { query = buildGroupByQuery(compactionTask, interval, ds); @@ -212,7 +296,7 @@ public TaskStatus createAndRunMSQTasks( MSQSpec msqSpec = MSQSpec.builder() .query(query) - .columnMappings(ColumnMappings.identity(getRowSignature(ds))) + .columnMappings(getColumnMappings(ds)) .destination(msqDestination) .assignmentStrategy(MultiStageQueryContext.getAssignmentStrategy(compactionTaskContext)) .tuningConfig(msqTuningConfig) @@ -228,7 +312,7 @@ public TaskStatus createAndRunMSQTasks( null, null, null, - null + msqControllerTaskContext ); // Doing a serde roundtrip for MSQControllerTask as the "injector" field of this class is supposed to be injected @@ -258,7 +342,7 @@ private static MSQTuningConfig getMsqTuningConfig( int maxRowsInMemory ) { - Integer rowsPerSegment = null; + Integer rowsPerSegment = PartitionsSpec.DEFAULT_MAX_ROWS_PER_SEGMENT; if (compactionTask.getTuningConfig() != null) { PartitionsSpec partitionsSpec = compactionTask.getTuningConfig().getPartitionsSpec(); if (partitionsSpec instanceof DynamicPartitionsSpec) { @@ -290,6 +374,12 @@ private Map createMSQTaskContext(CompactionTask compactionTask, .getSegmentGranularity() : DEFAULT_SEGMENT_GRANULARITY) ); + if (!isQueryGranularityEmpty(ds)) { + context.put( + DruidSqlInsert.SQL_INSERT_QUERY_GRANULARITY, + jsonMapper.writeValueAsString(ds.getGranularitySpec().getQueryGranularity()) + ); + } } catch (JsonProcessingException e) { throw DruidException @@ -301,8 +391,6 @@ private Map createMSQTaskContext(CompactionTask compactionTask, + "serializable. Please raise an issue in case you are seeing this message while executing a query." ); } - context.put(DruidSqlReplace.SQL_REPLACE_TIME_CHUNKS, interval.toString()); - context.put(Tasks.STORE_COMPACTION_STATE_KEY, true); return context; } @@ -328,10 +416,6 @@ private static TaskStatus runSubtasks( try { if (eachTask.isReady(toolbox.getTaskActionClient())) { log.info("Running MSQControllerTask: " + json); - // Currently multiple MSQControllerTasks, if created, are started serially in-place, just as multiple - // ParallelIndexSupervisorTasks for native compaction. A better strategy may be to trigger them via the - // overlord. In that case, the subtask statuses would need to be tracked to determine the final status of the - // compaction task. final TaskStatus eachResult = eachTask.run(toolbox); if (!eachResult.isSuccess()) { failCnt++; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 918a4fd29691..0864c96f2be0 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -213,6 +213,7 @@ import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.DruidServerMetadata; +import org.apache.druid.sql.calcite.parser.DruidSqlInsert; import org.apache.druid.sql.calcite.planner.ColumnMapping; import org.apache.druid.sql.calcite.planner.ColumnMappings; import org.apache.druid.sql.calcite.rel.DruidQuery; @@ -1765,6 +1766,16 @@ private void publishSegmentsIfNeeded( ); if (!segments.isEmpty() && storeCompactionState) { + if (!(task.getQuerySpec().getDestination() instanceof DataSourceMSQDestination)) { + throw new MSQException( + UnknownFault.forMessage( + StringUtils.format( + "Query[%s] cannot store compaction state in segments as destination[%s] not a datasource.", + queryDef.getQueryId(), + task.getQuerySpec().getDestination().getClass() + ))); + } + DataSourceMSQDestination destination = (DataSourceMSQDestination) task.getQuerySpec().getDestination(); if (!destination.isReplaceTimeChunks()) { // Store compaction state only for replace queries. @@ -1831,9 +1842,9 @@ private static Function, Set> addCompactionStateTo GranularitySpec granularitySpec = new UniformGranularitySpec( segmentGranularity, - dataSchema.getGranularitySpec().getQueryGranularity(), + QueryContext.of(task.getContext()).getGranularity(DruidSqlInsert.SQL_INSERT_QUERY_GRANULARITY, jsonMapper), dataSchema.getGranularitySpec().isRollup(), - dataSchema.getGranularitySpec().inputIntervals() + ((DataSourceMSQDestination) task.getQuerySpec().getDestination()).getReplaceTimeChunks() ); DimensionsSpec dimensionsSpec = dataSchema.getDimensionsSpec(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionToMSQTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionToMSQTask.java index 29c999495cdb..cff17e1a3b70 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionToMSQTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionToMSQTask.java @@ -1,3 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.druid.indexing.common.task; import org.apache.druid.indexer.TaskStatus; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java index b08d56d78481..387386b5460b 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java @@ -232,22 +232,22 @@ public int hashCode() return result; } - public static DataSourceCompactionConfig from(Engine defaultEngine, DataSourceCompactionConfig currentConfig) + public static DataSourceCompactionConfig from(Engine defaultEngine, DataSourceCompactionConfig newConfig) { - Engine newEngine = currentConfig.getEngine(); + Engine newEngine = newConfig.getEngine(); String engineSourceLog = "specified in spec"; if (newEngine == null) { newEngine = defaultEngine; engineSourceLog = "set as default"; } if (newEngine == Engine.MSQ) { - if (currentConfig.getTuningConfig() != null) { - PartitionsSpec partitionsSpec = currentConfig.getTuningConfig().getPartitionsSpec(); + if (newConfig.getTuningConfig() != null) { + PartitionsSpec partitionsSpec = newConfig.getTuningConfig().getPartitionsSpec(); if (partitionsSpec != null && !(partitionsSpec instanceof DimensionRangePartitionsSpec || partitionsSpec instanceof DynamicPartitionsSpec)) { throw InvalidInput.exception( - "Invalid partition spec type[%s] for MSQ compaction engine %s." + "Invalid partition spec type[%s] for MSQ compaction engine[%s]." + " Type must be either DynamicPartitionsSpec or DynamicRangePartitionsSpec.", partitionsSpec.getClass(), engineSourceLog @@ -261,20 +261,26 @@ public static DataSourceCompactionConfig from(Engine defaultEngine, DataSourceCo ); } } + if (newConfig.getMetricsSpec() != null + && newConfig.getGranularitySpec() != null + && !newConfig.getGranularitySpec() + .isRollup()) { + throw InvalidInput.exception("rollup in granularitySpec must be set to True if metricsSpec is specifed."); + } } return new DataSourceCompactionConfig( - currentConfig.getDataSource(), - currentConfig.getTaskPriority(), - currentConfig.getInputSegmentSizeBytes(), - currentConfig.getMaxRowsPerSegment(), - currentConfig.getSkipOffsetFromLatest(), - currentConfig.getTuningConfig(), - currentConfig.getGranularitySpec(), - currentConfig.getDimensionsSpec(), - currentConfig.getMetricsSpec(), - currentConfig.getTransformSpec(), - currentConfig.getIoConfig(), - newEngine, currentConfig.getTaskContext() + newConfig.getDataSource(), + newConfig.getTaskPriority(), + newConfig.getInputSegmentSizeBytes(), + newConfig.getMaxRowsPerSegment(), + newConfig.getSkipOffsetFromLatest(), + newConfig.getTuningConfig(), + newConfig.getGranularitySpec(), + newConfig.getDimensionsSpec(), + newConfig.getMetricsSpec(), + newConfig.getTransformSpec(), + newConfig.getIoConfig(), + newEngine, newConfig.getTaskContext() ); } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlInsert.java b/sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlInsert.java index 1dc77885263c..1877a212343f 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlInsert.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlInsert.java @@ -39,6 +39,7 @@ public class DruidSqlInsert extends DruidSqlIngest { public static final String SQL_INSERT_SEGMENT_GRANULARITY = "sqlInsertSegmentGranularity"; + public static final String SQL_INSERT_QUERY_GRANULARITY = "sqlInsertQueryGranularity"; // This allows reusing super.unparse public static final SqlOperator OPERATOR = DruidSqlIngestOperator.INSERT_OPERATOR; From e6222e8f2ee52ecf97b1e05cec6b98d434235ea7 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Tue, 23 Apr 2024 11:09:52 +0530 Subject: [PATCH 11/54] Simplify guice module and refactor CompactionToMSQTaskImpl --- .../compaction/CompactionToMSQTaskImpl.java | 315 +++++++++--------- .../msq/guice/MSQCompactionDruidModule.java | 27 +- .../common/task/CompactionToMSQTask.java | 4 +- 3 files changed, 157 insertions(+), 189 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQTaskImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQTaskImpl.java index a16aa7ad21e9..6ff03d9d100b 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQTaskImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQTaskImpl.java @@ -22,8 +22,8 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; +import com.google.inject.Inject; import org.apache.druid.data.input.impl.DimensionSchema; -import org.apache.druid.error.DruidException; import org.apache.druid.error.InvalidInput; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; @@ -46,7 +46,6 @@ import org.apache.druid.msq.indexing.MSQSpec; import org.apache.druid.msq.indexing.MSQTuningConfig; import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination; -import org.apache.druid.msq.indexing.destination.MSQDestination; import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.Druids; import org.apache.druid.query.Query; @@ -90,13 +89,144 @@ public class CompactionToMSQTaskImpl implements CompactionToMSQTask private static final String TIME_VIRTUAL_COLUMN = "v0"; private static final String TIME_COLUMN = ColumnHolder.TIME_COLUMN_NAME; - + @Inject public CompactionToMSQTaskImpl(final OverlordClient overlordClient, final ObjectMapper jsonMapper) { this.overlordClient = overlordClient; this.jsonMapper = jsonMapper; } + @Override + public TaskStatus createAndRunMSQTasks( + CompactionTask compactionTask, + TaskToolbox taskToolbox, + List> intervalDataSchemas + ) throws JsonProcessingException + { + List msqControllerTasks = new ArrayList<>(); + QueryContext compactionTaskContext = new QueryContext(compactionTask.getContext()); + + if (!MultiStageQueryContext.isFinalizeAggregations(compactionTaskContext)) { + throw InvalidInput.exception( + "finalizeAggregations=false currently not supported for auto-compaction with MSQ engine."); + } + + for (NonnullPair intervalDataSchema : intervalDataSchemas) { + Query query; + Interval interval = intervalDataSchema.lhs; + DataSchema ds = intervalDataSchema.rhs; + + if (!isGroupBy(ds)) { + query = buildScanQuery(compactionTask, interval, ds); + } else { + query = buildGroupByQuery(compactionTask, interval, ds); + } + + MSQSpec msqSpec = MSQSpec.builder() + .query(query) + .columnMappings(getColumnMappings(ds)) + .destination(buildMSQDestination(compactionTask, ds, compactionTaskContext)) + .assignmentStrategy(MultiStageQueryContext.getAssignmentStrategy(compactionTaskContext)) + .tuningConfig(buildMSQTuningConfig(compactionTask, compactionTaskContext)) + .build(); + + Map MSQControllerTaskContext = createMSQTaskContext(compactionTask, ds); + + MSQControllerTask controllerTask = new MSQControllerTask( + compactionTask.getId(), + msqSpec.withOverriddenContext(MSQControllerTaskContext), + null, + MSQControllerTaskContext, + null, + null, + null, + MSQControllerTaskContext + ); + + // Doing a serde roundtrip for MSQControllerTask as the "injector" field of this class is supposed to be injected + // by the mapper. + MSQControllerTask serdedMSQControllerTask = jsonMapper.readerFor(MSQControllerTask.class) + .readValue(jsonMapper.writeValueAsString(controllerTask)); + msqControllerTasks.add(serdedMSQControllerTask); + } + + if (msqControllerTasks.isEmpty()) { + log.warn( + "Can't find segments from inputSpec[%s], nothing to do.", + compactionTask.getIoConfig().getInputSpec() + ); + } + return runSubtasks( + msqControllerTasks, + taskToolbox, + compactionTask.getCurrentSubTaskHolder(), + compactionTask.getId() + ); + } + + private static DataSourceMSQDestination buildMSQDestination( + CompactionTask compactionTask, + DataSchema ds, + QueryContext compactionTaskContext + ) + { + final Interval replaceInterval = compactionTask.getIoConfig() + .getInputSpec() + .findInterval(compactionTask.getDataSource()); + + final List segmentSortOrder = MultiStageQueryContext.getSortOrder(compactionTaskContext); + + return new DataSourceMSQDestination( + ds.getDataSource(), + ds.getGranularitySpec().getSegmentGranularity(), + segmentSortOrder, + ImmutableList.of(replaceInterval) + ); + } + + private static MSQTuningConfig buildMSQTuningConfig(CompactionTask compactionTask, QueryContext compactionTaskContext) + { + + // Transfer MSQ-related context params, if any, from the compaction context itself. + final int maxNumTasks = MultiStageQueryContext.getMaxNumTasks(compactionTaskContext); + if (maxNumTasks < 2) { + throw InvalidInput.exception( + "MSQ context maxNumTasks [%,d] cannot be less than 2, " + + "since at least 1 controller and 1 worker is necessary.", + maxNumTasks + ); + } + // This parameter is used internally for the number of worker tasks only, so we subtract 1 + final int maxNumWorkers = maxNumTasks - 1; + final int maxRowsInMemory = MultiStageQueryContext.getRowsInMemory(compactionTaskContext); + + Integer rowsPerSegment = getRowsPerSegment(compactionTask); + + return new MSQTuningConfig( + maxNumWorkers, + maxRowsInMemory, + rowsPerSegment, + compactionTask.getTuningConfig() != null ? compactionTask.getTuningConfig().getIndexSpec() : null + ); + } + + private static Integer getRowsPerSegment(CompactionTask compactionTask) + { + Integer rowsPerSegment = PartitionsSpec.DEFAULT_MAX_ROWS_PER_SEGMENT; + if (compactionTask.getTuningConfig() != null) { + PartitionsSpec partitionsSpec = compactionTask.getTuningConfig().getPartitionsSpec(); + if (partitionsSpec instanceof DynamicPartitionsSpec) { + rowsPerSegment = partitionsSpec.getMaxRowsPerSegment(); + } else if (partitionsSpec instanceof DimensionRangePartitionsSpec) { + DimensionRangePartitionsSpec dimensionRangePartitionsSpec = (DimensionRangePartitionsSpec) partitionsSpec; + rowsPerSegment = dimensionRangePartitionsSpec.getTargetRowsPerSegment() != null + ? dimensionRangePartitionsSpec.getTargetRowsPerSegment() + : dimensionRangePartitionsSpec.getMaxRowsPerSegment(); + } + } + return rowsPerSegment; + } + private static RowSignature getRowSignature(DataSchema dataSchema) { RowSignature.Builder rowSignatureBuilder = RowSignature.builder(); @@ -118,9 +248,8 @@ private static List getAggregateDimensions(DataSchema ds) .collect(Collectors.toList()); - // Dimensions in group-by aren't allowed to have time column as the output name. - if (isQueryGranularityEmpty(ds)){ + if (isQueryGranularityEmpty(ds)) { dimensionSpecs.add(new DefaultDimensionSpec(TIME_COLUMN, TIME_VIRTUAL_COLUMN, ColumnType.LONG)); } else { // The changed granularity would result in a new virtual column that needs to be aggregated upon. @@ -188,22 +317,21 @@ private static boolean isGroupBy(DataSchema ds) private static boolean isQueryGranularityEmpty(DataSchema ds) { - return ds.getGranularitySpec() == null - || ds.getGranularitySpec().getQueryGranularity() == null; + return ds.getGranularitySpec() == null || ds.getGranularitySpec().getQueryGranularity() == null; } - private static VirtualColumns getVirtualColumns(DataSchema ds) { + + private static VirtualColumns getVirtualColumns(DataSchema ds) + { VirtualColumns virtualColumns = VirtualColumns.EMPTY; if (!isQueryGranularityEmpty(ds) && !ds.getGranularitySpec().getQueryGranularity().equals(Granularities.ALL)) { - PeriodGranularity periodQueryGranularity = (PeriodGranularity) ds.getGranularitySpec() - .getQueryGranularity(); + PeriodGranularity periodQueryGranularity = (PeriodGranularity) ds.getGranularitySpec().getQueryGranularity(); VirtualColumn virtualColumn = new ExpressionVirtualColumn( TIME_VIRTUAL_COLUMN, StringUtils.format( "timestamp_floor(\"%s\", '%s')", TIME_COLUMN, - periodQueryGranularity.getPeriod() - .toString() + periodQueryGranularity.getPeriod().toString() ), ColumnType.LONG, new ExprMacroTable(Collections.singletonList(new TimestampFloorExprMacro())) @@ -235,161 +363,22 @@ private static Query buildGroupByQuery( return builder.build(); } - @Override - public TaskStatus createAndRunMSQTasks( - CompactionTask compactionTask, - TaskToolbox taskToolbox, - List> intervalDataSchemas - ) throws Exception - { - - - List msqControllerTasks = new ArrayList<>(); - QueryContext compactionTaskContext = new QueryContext(compactionTask.getContext()); - - if (!MultiStageQueryContext.isFinalizeAggregations(compactionTaskContext)) { - throw InvalidInput.exception( - "finalizeAggregations=false currently not supported for auto-compaction with MSQ engine."); - } - - for (NonnullPair intervalDataSchema : intervalDataSchemas) { - Query query; - Interval interval = intervalDataSchema.lhs; - DataSchema ds = intervalDataSchema.rhs; - final Interval replaceInterval = compactionTask.getIoConfig() - .getInputSpec() - .findInterval(compactionTask.getDataSource()); - - final List segmentSortOrder = MultiStageQueryContext.getSortOrder(compactionTaskContext); - - MSQDestination msqDestination = new DataSourceMSQDestination( - ds.getDataSource(), - ds.getGranularitySpec().getSegmentGranularity(), - segmentSortOrder, - ImmutableList.of(replaceInterval) - ); - - // Transfer MSQ-related context params, if any, from the compaction context itself. - final int maxNumTasks = MultiStageQueryContext.getMaxNumTasks(compactionTaskContext); - if (maxNumTasks < 2) { - throw InvalidInput.exception( - "MSQ context maxNumTasks [%,d] cannot be less than 2, " - + "since at least 1 controller and 1 worker is necessary.", - maxNumTasks - ); - } - // This parameter is used internally for the number of worker tasks only, so we subtract 1 - final int maxNumWorkers = maxNumTasks - 1; - final int maxRowsInMemory = MultiStageQueryContext.getRowsInMemory(compactionTaskContext); - - if (!isGroupBy(ds)) { - query = buildScanQuery(compactionTask, interval, ds); - } else { - query = buildGroupByQuery(compactionTask, interval, ds); - } - - MSQTuningConfig msqTuningConfig = getMsqTuningConfig( - compactionTask, - maxNumWorkers, - maxRowsInMemory - ); - - MSQSpec msqSpec = MSQSpec.builder() - .query(query) - .columnMappings(getColumnMappings(ds)) - .destination(msqDestination) - .assignmentStrategy(MultiStageQueryContext.getAssignmentStrategy(compactionTaskContext)) - .tuningConfig(msqTuningConfig) - .build(); - - Map msqControllerTaskContext = createMSQTaskContext(compactionTask, interval, ds); - - MSQControllerTask controllerTask = new MSQControllerTask( - compactionTask.getId(), - msqSpec.withOverriddenContext(msqControllerTaskContext), - null, - msqControllerTaskContext, - null, - null, - null, - msqControllerTaskContext - ); - - // Doing a serde roundtrip for MSQControllerTask as the "injector" field of this class is supposed to be injected - // by the mapper. - MSQControllerTask serdedMSQControllerTask = jsonMapper.readerFor(MSQControllerTask.class) - .readValue(jsonMapper.writeValueAsString(controllerTask)); - msqControllerTasks.add(serdedMSQControllerTask); - } - - if (msqControllerTasks.isEmpty()) { - log.warn( - "Can't find segments from inputSpec[%s], nothing to do.", - compactionTask.getIoConfig().getInputSpec() - ); - } - return runSubtasks( - msqControllerTasks, - taskToolbox, - compactionTask.getCurrentSubTaskHolder(), - compactionTask.getId() - ); - } - - private static MSQTuningConfig getMsqTuningConfig( - CompactionTask compactionTask, - int maxNumWorkers, - int maxRowsInMemory - ) - { - Integer rowsPerSegment = PartitionsSpec.DEFAULT_MAX_ROWS_PER_SEGMENT; - if (compactionTask.getTuningConfig() != null) { - PartitionsSpec partitionsSpec = compactionTask.getTuningConfig().getPartitionsSpec(); - if (partitionsSpec instanceof DynamicPartitionsSpec) { - rowsPerSegment = partitionsSpec.getMaxRowsPerSegment(); - } else if (partitionsSpec instanceof DimensionRangePartitionsSpec) { - DimensionRangePartitionsSpec dimensionRangePartitionsSpec = (DimensionRangePartitionsSpec) partitionsSpec; - rowsPerSegment = dimensionRangePartitionsSpec.getTargetRowsPerSegment() != null - ? dimensionRangePartitionsSpec.getTargetRowsPerSegment() - : dimensionRangePartitionsSpec.getMaxRowsPerSegment(); - } - } - - return new MSQTuningConfig( - maxNumWorkers, - maxRowsInMemory, - rowsPerSegment, - compactionTask.getTuningConfig().getIndexSpec() - ); - } - - private Map createMSQTaskContext(CompactionTask compactionTask, Interval interval, DataSchema ds) + private Map createMSQTaskContext(CompactionTask compactionTask, DataSchema ds) + throws JsonProcessingException { Map context = new HashMap<>(compactionTask.getContext()); - try { + context.put( + DruidSqlInsert.SQL_INSERT_SEGMENT_GRANULARITY, + jsonMapper.writeValueAsString(ds.getGranularitySpec() != null + ? ds.getGranularitySpec() + .getSegmentGranularity() + : DEFAULT_SEGMENT_GRANULARITY) + ); + if (!isQueryGranularityEmpty(ds)) { context.put( - DruidSqlInsert.SQL_INSERT_SEGMENT_GRANULARITY, - jsonMapper.writeValueAsString(ds.getGranularitySpec() != null - ? ds.getGranularitySpec() - .getSegmentGranularity() - : DEFAULT_SEGMENT_GRANULARITY) + DruidSqlInsert.SQL_INSERT_QUERY_GRANULARITY, + jsonMapper.writeValueAsString(ds.getGranularitySpec().getQueryGranularity()) ); - if (!isQueryGranularityEmpty(ds)) { - context.put( - DruidSqlInsert.SQL_INSERT_QUERY_GRANULARITY, - jsonMapper.writeValueAsString(ds.getGranularitySpec().getQueryGranularity()) - ); - } - } - catch (JsonProcessingException e) { - throw DruidException - .defensive() - .build( - e, - "Unable to deserialize the DEFAULT_SEGMENT_GRANULARITY in CompactionToMSQTaskImpl. " - + "This shouldn't have happened since the DEFAULT_SEGMENT_GRANULARITY object is guaranteed to be " - + "serializable. Please raise an issue in case you are seeing this message while executing a query." - ); } return context; } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQCompactionDruidModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQCompactionDruidModule.java index 1cfeac77b5c1..53d4abfca23a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQCompactionDruidModule.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQCompactionDruidModule.java @@ -19,41 +19,20 @@ package org.apache.druid.msq.guice; -import com.fasterxml.jackson.databind.Module; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Binder; -import com.google.inject.Provides; import org.apache.druid.guice.LazySingleton; import org.apache.druid.indexing.common.task.CompactionToMSQTask; import org.apache.druid.initialization.DruidModule; import org.apache.druid.msq.compaction.CompactionToMSQTaskImpl; -import org.apache.druid.rpc.indexing.OverlordClient; - -import java.util.Collections; -import java.util.List; +/** + * Module for running compaction on MSQ + */ public class MSQCompactionDruidModule implements DruidModule { - @Override public void configure(Binder binder) { binder.bind(CompactionToMSQTask.class).to(CompactionToMSQTaskImpl.class).in(LazySingleton.class); } - - @Override - public List getJacksonModules() - { - return Collections.emptyList(); - } - - @Provides - @LazySingleton - CompactionToMSQTaskImpl getCompactionToMSQImpl( - final OverlordClient overlordClient, - final ObjectMapper jsonMapper - ) - { - return new CompactionToMSQTaskImpl(overlordClient, jsonMapper); - } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionToMSQTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionToMSQTask.java index cff17e1a3b70..98df31917a30 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionToMSQTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionToMSQTask.java @@ -19,6 +19,7 @@ package org.apache.druid.indexing.common.task; +import com.fasterxml.jackson.core.JsonProcessingException; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.java.util.common.NonnullPair; @@ -33,6 +34,5 @@ TaskStatus createAndRunMSQTasks( CompactionTask compactionTask, TaskToolbox taskToolbox, List> dataSchemas - ) throws Exception; - + ) throws JsonProcessingException; } From 8923fbdcc9a0a866192468e2aede4ce6624402a4 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Tue, 23 Apr 2024 12:39:40 +0530 Subject: [PATCH 12/54] Fix checkstyle --- .../compaction/CompactionToMSQTaskImpl.java | 21 ++--- .../druid/indexing/common/TaskToolbox.java | 3 +- .../indexing/common/task/CompactionTask.java | 90 +++++++++++-------- .../common/task/CompactionToMSQTask.java | 10 +-- .../overlord/TestTaskToolboxFactory.java | 3 +- .../indexing/ClientCompactionTaskQuery.java | 2 +- 6 files changed, 68 insertions(+), 61 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQTaskImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQTaskImpl.java index 6ff03d9d100b..4cde6b422752 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQTaskImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQTaskImpl.java @@ -86,7 +86,7 @@ public class CompactionToMSQTaskImpl implements CompactionToMSQTask final OverlordClient overlordClient; final ObjectMapper jsonMapper; - private static final String TIME_VIRTUAL_COLUMN = "v0"; + private static final String TIME_VIRTUAL_COLUMN = "vTime"; private static final String TIME_COLUMN = ColumnHolder.TIME_COLUMN_NAME; @Inject @@ -130,17 +130,17 @@ public TaskStatus createAndRunMSQTasks( .tuningConfig(buildMSQTuningConfig(compactionTask, compactionTaskContext)) .build(); - Map MSQControllerTaskContext = createMSQTaskContext(compactionTask, ds); + Map msqControllerTaskContext = createMSQTaskContext(compactionTask, ds); MSQControllerTask controllerTask = new MSQControllerTask( compactionTask.getId(), - msqSpec.withOverriddenContext(MSQControllerTaskContext), + msqSpec.withOverriddenContext(msqControllerTaskContext), null, - MSQControllerTaskContext, + msqControllerTaskContext, null, null, null, - MSQControllerTaskContext + msqControllerTaskContext ); // Doing a serde roundtrip for MSQControllerTask as the "injector" field of this class is supposed to be injected @@ -281,11 +281,6 @@ private static ColumnMappings getColumnMappings(DataSchema dataSchema) return new ColumnMappings(columnMappings); } - private static MultipleIntervalSegmentSpec createMultipleIntervalSpec(Interval interval) - { - return new MultipleIntervalSegmentSpec(Collections.singletonList(interval)); - } - private static List getOrderBySpec(PartitionsSpec partitionSpec) { if (partitionSpec.getType() == SecondaryPartitionType.RANGE) { @@ -303,7 +298,7 @@ private static Query buildScanQuery(CompactionTask compactionTask, Interval i return new Druids.ScanQueryBuilder().dataSource(dataSchema.getDataSource()) .columns(rowSignature.getColumnNames()) .columnTypes(rowSignature.getColumnTypes()) - .intervals(createMultipleIntervalSpec(interval)) + .intervals(new MultipleIntervalSegmentSpec(Collections.singletonList(interval))) .legacy(false) .filters(dataSchema.getTransformSpec().getFilter()) .context(compactionTask.getContext()) @@ -341,9 +336,7 @@ private static VirtualColumns getVirtualColumns(DataSchema ds) return virtualColumns; } - private static Query buildGroupByQuery( - CompactionTask compactionTask, Interval interval, DataSchema dataSchema - ) + private static Query buildGroupByQuery(CompactionTask compactionTask, Interval interval, DataSchema dataSchema) { DimFilter dimFilter = dataSchema.getTransformSpec().getFilter(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java index b8ec6d401357..e716a2488728 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java @@ -870,7 +870,8 @@ public Builder centralizedTableSchemaConfig(final CentralizedDatasourceSchemaCon return this; } - public Builder compactionToMSQ(final CompactionToMSQTask compactionToMSQTask){ + public Builder compactionToMSQ(final CompactionToMSQTask compactionToMSQTask) + { this.compactionToMSQTask = compactionToMSQTask; return this; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 0720462e8675..5012054a8e12 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -44,6 +44,7 @@ import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.error.DruidException; import org.apache.druid.indexer.Checks; import org.apache.druid.indexer.Property; import org.apache.druid.indexer.TaskStatus; @@ -484,9 +485,16 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception ); if (engine == Engine.MSQ) { - registerResourceCloserOnAbnormalExit(currentSubTaskHolder); - return toolbox.getCompactionToMSQ() - .createAndRunMSQTasks(this, toolbox, intervalDataSchemas); + if (toolbox.getCompactionToMSQ() == null) { + throw DruidException.forPersona(DruidException.Persona.ADMIN) + .ofCategory(DruidException.Category.NOT_FOUND) + .build( + "Extension[druid-multi-stage-query] required for running compaction on MSQ " + + "not found on the Indexer"); + } + registerResourceCloserOnAbnormalExit(currentSubTaskHolder); + return toolbox.getCompactionToMSQ() + .createAndRunMSQTasks(this, toolbox, intervalDataSchemas); } else { final List ingestionSpecs = createIngestionSpecs( intervalDataSchemas, @@ -510,8 +518,9 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception .collect(Collectors.toList()); if (subtasks.isEmpty()) { - String msg = StringUtils.format("Can't find segments from inputSpec[%s], nothing to do.", - ioConfig.getInputSpec() + String msg = StringUtils.format( + "Can't find segments from inputSpec[%s], nothing to do.", + ioConfig.getInputSpec() ); log.warn(msg); return TaskStatus.failure(getId(), msg); @@ -527,47 +536,50 @@ private TaskStatus runParallelIndexSubtasks(List ta final int totalNumSpecs = tasks.size(); log.info("Generated [%d] compaction task specs", totalNumSpecs); - int failCnt = 0; - final TaskReport.ReportMap completionReports = new TaskReport.ReportMap(); - for (int i = 0; i < tasks.size(); i++) { - ParallelIndexSupervisorTask eachSpec = tasks.get(i); - final String json = toolbox.getJsonMapper().writerWithDefaultPrettyPrinter().writeValueAsString(eachSpec); - if (!currentSubTaskHolder.setTask(eachSpec)) { - String errMsg = "Task was asked to stop. Finish as failed."; - log.info(errMsg); - return TaskStatus.failure(getId(), errMsg); - } - try { - if (eachSpec.isReady(toolbox.getTaskActionClient())) { - log.info("Running indexSpec: " + json); - final TaskStatus eachResult = eachSpec.run(toolbox); - if (!eachResult.isSuccess()) { - failCnt++; - log.warn("Failed to run indexSpec: [%s].\nTrying the next indexSpec.", json); - } - - String reportKeySuffix = "_" + i; - Optional.ofNullable(eachSpec.getCompletionReports()) - .ifPresent(reports -> completionReports.putAll( - CollectionUtils.mapKeys(reports, key -> key + reportKeySuffix))); - } else { + int failCnt = 0; + final TaskReport.ReportMap completionReports = new TaskReport.ReportMap(); + for (int i = 0; i < tasks.size(); i++) { + ParallelIndexSupervisorTask eachSpec = tasks.get(i); + final String json = toolbox.getJsonMapper().writerWithDefaultPrettyPrinter().writeValueAsString(eachSpec); + if (!currentSubTaskHolder.setTask(eachSpec)) { + String errMsg = "Task was asked to stop. Finish as failed."; + log.info(errMsg); + return TaskStatus.failure(getId(), errMsg); + } + try { + if (eachSpec.isReady(toolbox.getTaskActionClient())) { + log.info("Running indexSpec: " + json); + final TaskStatus eachResult = eachSpec.run(toolbox); + if (!eachResult.isSuccess()) { failCnt++; - log.warn("indexSpec is not ready: [%s].\nTrying the next indexSpec.", json); + log.warn("Failed to run indexSpec: [%s].\nTrying the next indexSpec.", json); } - } - catch (Exception e) { + + String reportKeySuffix = "_" + i; + Optional.ofNullable(eachSpec.getCompletionReports()) + .ifPresent(reports -> completionReports.putAll( + CollectionUtils.mapKeys(reports, key -> key + reportKeySuffix))); + } else { failCnt++; - log.warn(e, "Failed to run indexSpec: [%s].\nTrying the next indexSpec.", json); + log.warn("indexSpec is not ready: [%s].\nTrying the next indexSpec.", json); } } + catch (Exception e) { + failCnt++; + log.warn(e, "Failed to run indexSpec: [%s].\nTrying the next indexSpec.", json); + } + } - String msg = StringUtils.format("Ran [%d] specs, [%d] succeeded, [%d] failed", - totalNumSpecs, totalNumSpecs - failCnt, failCnt - ); + String msg = StringUtils.format( + "Ran [%d] specs, [%d] succeeded, [%d] failed", + totalNumSpecs, + totalNumSpecs - failCnt, + failCnt + ); - toolbox.getTaskReportFileWriter().write(getId(), completionReports); - log.info(msg); - return failCnt == 0 ? TaskStatus.success(getId()) : TaskStatus.failure(getId(), msg); + toolbox.getTaskReportFileWriter().write(getId(), completionReports); + log.info(msg); + return failCnt == 0 ? TaskStatus.success(getId()) : TaskStatus.failure(getId(), msg); } @VisibleForTesting diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionToMSQTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionToMSQTask.java index 98df31917a30..2f0785b690d6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionToMSQTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionToMSQTask.java @@ -30,9 +30,9 @@ public interface CompactionToMSQTask { - TaskStatus createAndRunMSQTasks( - CompactionTask compactionTask, - TaskToolbox taskToolbox, - List> dataSchemas - ) throws JsonProcessingException; + TaskStatus createAndRunMSQTasks( + CompactionTask compactionTask, + TaskToolbox taskToolbox, + List> dataSchemas + ) throws JsonProcessingException; } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskToolboxFactory.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskToolboxFactory.java index 2995f83f6838..35b1a16bb453 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskToolboxFactory.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskToolboxFactory.java @@ -400,7 +400,8 @@ public Builder setCentralizedTableSchemaConfig(CentralizedDatasourceSchemaConfig return this; } - public Builder setCompactionToMSQ(CompactionToMSQTask compactionToMSQTask){ + public Builder setCompactionToMSQ(CompactionToMSQTask compactionToMSQTask) + { this.compactionToMSQTask = compactionToMSQTask; return this; } diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQuery.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQuery.java index b6beac37a708..0a5872dc0e62 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQuery.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQuery.java @@ -61,7 +61,7 @@ public ClientCompactionTaskQuery( @JsonProperty("transformSpec") ClientCompactionTaskTransformSpec transformSpec, @JsonProperty("context") Map context, @JsonProperty("engine") DataSourceCompactionConfig.Engine engine - ) + ) { this.id = Preconditions.checkNotNull(id, "id"); this.dataSource = dataSource; From 279c619d7578e71adffebf4f501b5bcdfdf8dc48 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Wed, 24 Apr 2024 12:34:59 +0530 Subject: [PATCH 13/54] Move CompactionToMSQTaskImpl to msq indexing. --- .../msq/guice/MSQCompactionDruidModule.java | 38 ------------------- .../druid/msq/guice/MSQIndexingModule.java | 3 ++ .../CompactionToMSQTaskImpl.java | 5 +-- ...rg.apache.druid.initialization.DruidModule | 1 - .../indexing/common/TaskToolboxFactory.java | 3 +- 5 files changed, 6 insertions(+), 44 deletions(-) delete mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQCompactionDruidModule.java rename extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/{compaction => indexing}/CompactionToMSQTaskImpl.java (98%) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQCompactionDruidModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQCompactionDruidModule.java deleted file mode 100644 index 53d4abfca23a..000000000000 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQCompactionDruidModule.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.msq.guice; - -import com.google.inject.Binder; -import org.apache.druid.guice.LazySingleton; -import org.apache.druid.indexing.common.task.CompactionToMSQTask; -import org.apache.druid.initialization.DruidModule; -import org.apache.druid.msq.compaction.CompactionToMSQTaskImpl; - -/** - * Module for running compaction on MSQ - */ -public class MSQCompactionDruidModule implements DruidModule -{ - @Override - public void configure(Binder binder) - { - binder.bind(CompactionToMSQTask.class).to(CompactionToMSQTaskImpl.class).in(LazySingleton.class); - } -} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java index 125a66331e60..27f96d68c17c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java @@ -28,12 +28,14 @@ import org.apache.druid.frame.processor.Bouncer; import org.apache.druid.guice.LazySingleton; import org.apache.druid.guice.annotations.Self; +import org.apache.druid.indexing.common.task.CompactionToMSQTask; import org.apache.druid.initialization.DruidModule; import org.apache.druid.msq.counters.ChannelCounters; import org.apache.druid.msq.counters.CounterSnapshotsSerializer; import org.apache.druid.msq.counters.SegmentGenerationProgressCounter; import org.apache.druid.msq.counters.SuperSorterProgressTrackerCounter; import org.apache.druid.msq.counters.WarningCounters; +import org.apache.druid.msq.indexing.CompactionToMSQTaskImpl; import org.apache.druid.msq.indexing.MSQControllerTask; import org.apache.druid.msq.indexing.MSQWorkerTask; import org.apache.druid.msq.indexing.error.BroadcastTablesTooLargeFault; @@ -205,6 +207,7 @@ public List getJacksonModules() @Override public void configure(Binder binder) { + binder.bind(CompactionToMSQTask.class).to(CompactionToMSQTaskImpl.class).in(LazySingleton.class); } @Provides diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQTaskImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CompactionToMSQTaskImpl.java similarity index 98% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQTaskImpl.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CompactionToMSQTaskImpl.java index 4cde6b422752..745d79a16a10 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/compaction/CompactionToMSQTaskImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CompactionToMSQTaskImpl.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.msq.compaction; +package org.apache.druid.msq.indexing; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; @@ -42,9 +42,6 @@ import org.apache.druid.java.util.common.granularity.PeriodGranularity; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.math.expr.ExprMacroTable; -import org.apache.druid.msq.indexing.MSQControllerTask; -import org.apache.druid.msq.indexing.MSQSpec; -import org.apache.druid.msq.indexing.MSQTuningConfig; import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination; import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.Druids; diff --git a/extensions-core/multi-stage-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-core/multi-stage-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule index 5c7962e2e69d..cabd131fb758 100644 --- a/extensions-core/multi-stage-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule +++ b/extensions-core/multi-stage-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule @@ -18,4 +18,3 @@ org.apache.druid.msq.guice.MSQIndexingModule org.apache.druid.msq.guice.MSQDurableStorageModule org.apache.druid.msq.guice.MSQSqlModule org.apache.druid.msq.guice.SqlTaskModule -org.apache.druid.msq.guice.MSQCompactionDruidModule \ No newline at end of file diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java index 8205250f1309..ca2666bbb37c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java @@ -68,6 +68,7 @@ import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.tasklogs.TaskLogPusher; +import javax.annotation.Nullable; import java.io.File; import java.util.function.Function; @@ -162,7 +163,7 @@ public TaskToolboxFactory( TaskLogPusher taskLogPusher, @AttemptId String attemptId, CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig, - CompactionToMSQTask compactionToMSQTask + @Nullable CompactionToMSQTask compactionToMSQTask ) { this.segmentLoaderConfig = segmentLoadConfig; From 0a7532141f5dbe2473fdbb4f215a287c8d6e143d Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Wed, 24 Apr 2024 12:53:59 +0530 Subject: [PATCH 14/54] Fix tests --- .../druid/server/coordinator/DataSourceCompactionConfig.java | 3 +++ .../server/http/CoordinatorCompactionConfigsResourceTest.java | 4 ++-- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java index 387386b5460b..f55054b0f3c6 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java @@ -268,6 +268,9 @@ public static DataSourceCompactionConfig from(Engine defaultEngine, DataSourceCo throw InvalidInput.exception("rollup in granularitySpec must be set to True if metricsSpec is specifed."); } } + if (newEngine == newConfig.getEngine()) { + return newConfig; + } return new DataSourceCompactionConfig( newConfig.getDataSource(), newConfig.getTaskPriority(), diff --git a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java index 1dfda54c25c6..40f0f8787323 100644 --- a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java @@ -191,7 +191,7 @@ public void testAddOrUpdateCompactionConfigWithExistingConfig() null, null, null, - null, + DataSourceCompactionConfig.Engine.NATIVE, ImmutableMap.of("key", "val") ); Response result = coordinatorCompactionConfigsResource.addOrUpdateCompactionConfig( @@ -392,7 +392,7 @@ public void testAddOrUpdateCompactionConfigWithoutExistingConfig() null, null, null, - null, + DataSourceCompactionConfig.Engine.MSQ, ImmutableMap.of("key", "val") ); String author = "maytas"; From 6e2d3f4c4f54505e3629cecd758831464c88cad8 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Sat, 27 Apr 2024 17:17:38 +0530 Subject: [PATCH 15/54] Move to json deserialization of compaction strategy. --- .../druid/msq/guice/MSQIndexingModule.java | 7 +- ...skImpl.java => MSQCompactionStrategy.java} | 31 ++- .../druid/indexing/common/TaskToolbox.java | 23 +- .../indexing/common/TaskToolboxFactory.java | 8 +- ...ToMSQTask.java => CompactionStrategy.java} | 14 +- .../indexing/common/task/CompactionTask.java | 238 ++-------------- .../druid/indexing/common/task/Engine.java | 3 +- .../common/task/NativeCompactionStrategy.java | 262 ++++++++++++++++++ .../overlord/http/OverlordResource.java | 1 + .../indexing/common/TaskToolboxTest.java | 3 +- ...penderatorDriverRealtimeIndexTaskTest.java | 3 +- .../ClientCompactionTaskQuerySerdeTest.java | 8 +- .../common/task/CompactionTaskTest.java | 32 +-- .../common/task/RealtimeIndexTaskTest.java | 3 +- .../SingleTaskBackgroundRunnerTest.java | 3 +- .../indexing/overlord/TaskLifecycleTest.java | 3 +- .../overlord/TestTaskToolboxFactory.java | 11 +- .../SeekableStreamIndexTaskTestBase.java | 3 +- .../worker/WorkerTaskManagerTest.java | 3 +- .../worker/WorkerTaskMonitorTest.java | 3 +- .../indexing/ClientCompactionTaskQuery.java | 18 +- .../CoordinatorCompactionConfig.java | 19 +- .../DataSourceCompactionConfig.java | 50 ++-- .../coordinator/UserCompactionStrategy.java | 48 ++++ .../coordinator/duty/CompactSegments.java | 13 +- ...rdinatorCompactionConfigsResourceTest.java | 5 +- 26 files changed, 458 insertions(+), 357 deletions(-) rename extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/{CompactionToMSQTaskImpl.java => MSQCompactionStrategy.java} (96%) rename indexing-service/src/main/java/org/apache/druid/indexing/common/task/{CompactionToMSQTask.java => CompactionStrategy.java} (75%) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionStrategy.java create mode 100644 server/src/main/java/org/apache/druid/server/coordinator/UserCompactionStrategy.java diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java index 27f96d68c17c..354a7f461a20 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java @@ -20,6 +20,7 @@ package org.apache.druid.msq.guice; import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.jsontype.NamedType; import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.common.collect.ImmutableList; import com.google.inject.Binder; @@ -28,14 +29,13 @@ import org.apache.druid.frame.processor.Bouncer; import org.apache.druid.guice.LazySingleton; import org.apache.druid.guice.annotations.Self; -import org.apache.druid.indexing.common.task.CompactionToMSQTask; import org.apache.druid.initialization.DruidModule; import org.apache.druid.msq.counters.ChannelCounters; import org.apache.druid.msq.counters.CounterSnapshotsSerializer; import org.apache.druid.msq.counters.SegmentGenerationProgressCounter; import org.apache.druid.msq.counters.SuperSorterProgressTrackerCounter; import org.apache.druid.msq.counters.WarningCounters; -import org.apache.druid.msq.indexing.CompactionToMSQTaskImpl; +import org.apache.druid.msq.indexing.MSQCompactionStrategy; import org.apache.druid.msq.indexing.MSQControllerTask; import org.apache.druid.msq.indexing.MSQWorkerTask; import org.apache.druid.msq.indexing.error.BroadcastTablesTooLargeFault; @@ -199,6 +199,8 @@ public List getJacksonModules() NilInputSource.class ); + module.registerSubtypes(new NamedType(MSQCompactionStrategy.class, MSQCompactionStrategy.type)); + FAULT_CLASSES.forEach(module::registerSubtypes); module.addSerializer(new CounterSnapshotsSerializer()); return Collections.singletonList(module); @@ -207,7 +209,6 @@ public List getJacksonModules() @Override public void configure(Binder binder) { - binder.bind(CompactionToMSQTask.class).to(CompactionToMSQTaskImpl.class).in(LazySingleton.class); } @Provides diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CompactionToMSQTaskImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionStrategy.java similarity index 96% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CompactionToMSQTaskImpl.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionStrategy.java index 745d79a16a10..5e4360b9de71 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CompactionToMSQTaskImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionStrategy.java @@ -19,10 +19,11 @@ package org.apache.druid.msq.indexing; +import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.json.JsonMapper; import com.google.common.collect.ImmutableList; -import com.google.inject.Inject; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.error.InvalidInput; import org.apache.druid.indexer.TaskStatus; @@ -32,7 +33,7 @@ import org.apache.druid.indexer.partitions.SecondaryPartitionType; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.task.CompactionTask; -import org.apache.druid.indexing.common.task.CompactionToMSQTask; +import org.apache.druid.indexing.common.task.CompactionStrategy; import org.apache.druid.indexing.common.task.CurrentSubTaskHolder; import org.apache.druid.java.util.common.NonnullPair; import org.apache.druid.java.util.common.StringUtils; @@ -76,25 +77,37 @@ import java.util.Map; import java.util.stream.Collectors; -public class CompactionToMSQTaskImpl implements CompactionToMSQTask +public class MSQCompactionStrategy implements CompactionStrategy { - private static final Logger log = new Logger(CompactionToMSQTaskImpl.class); + private static final Logger log = new Logger(MSQCompactionStrategy.class); + public static final String type = "MSQ"; private static final Granularity DEFAULT_SEGMENT_GRANULARITY = Granularities.ALL; - final OverlordClient overlordClient; - final ObjectMapper jsonMapper; + + OverlordClient overlordClient; + ObjectMapper jsonMapper; private static final String TIME_VIRTUAL_COLUMN = "vTime"; private static final String TIME_COLUMN = ColumnHolder.TIME_COLUMN_NAME; - @Inject - public CompactionToMSQTaskImpl(final OverlordClient overlordClient, final ObjectMapper jsonMapper) + public MSQCompactionStrategy(@JacksonInject OverlordClient overlordClient, @JacksonInject JsonMapper jsonMapper) { this.overlordClient = overlordClient; this.jsonMapper = jsonMapper; } @Override - public TaskStatus createAndRunMSQTasks( + public String getType() + { + return type; + } + + public void setJsonMapper(ObjectMapper jsonMapper) + { + this.jsonMapper = jsonMapper; + } + + @Override + public TaskStatus runCompactionTasks( CompactionTask compactionTask, TaskToolbox taskToolbox, List> intervalDataSchemas diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java index e716a2488728..62d649894f87 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java @@ -36,7 +36,6 @@ import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.config.TaskConfig; -import org.apache.druid.indexing.common.task.CompactionToMSQTask; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTaskClientProvider; import org.apache.druid.indexing.common.task.batch.parallel.ShuffleClient; import org.apache.druid.indexing.worker.shuffle.IntermediaryDataManager; @@ -135,8 +134,6 @@ public class TaskToolbox private final String attemptId; private final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; - private final CompactionToMSQTask compactionToMSQTask; - public TaskToolbox( SegmentLoaderConfig segmentLoaderConfig, TaskConfig config, @@ -178,8 +175,7 @@ public TaskToolbox( ShuffleClient shuffleClient, TaskLogPusher taskLogPusher, String attemptId, - CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig, - CompactionToMSQTask compactionToMSQTask + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { this.segmentLoaderConfig = segmentLoaderConfig; @@ -224,7 +220,6 @@ public TaskToolbox( this.taskLogPusher = taskLogPusher; this.attemptId = attemptId; this.centralizedDatasourceSchemaConfig = centralizedDatasourceSchemaConfig; - this.compactionToMSQTask = compactionToMSQTask; } public SegmentLoaderConfig getSegmentLoaderConfig() @@ -502,11 +497,6 @@ public CentralizedDatasourceSchemaConfig getCentralizedTableSchemaConfig() return centralizedDatasourceSchemaConfig; } - public CompactionToMSQTask getCompactionToMSQ() - { - return compactionToMSQTask; - } - /** * Create {@link AdjustedRuntimeInfo} based on the given {@link RuntimeInfo} and {@link AppenderatorsManager}. This * is a way to allow code to properly apportion the amount of processors and heap available to the entire JVM. @@ -574,7 +564,6 @@ public static class Builder private TaskLogPusher taskLogPusher; private String attemptId; private CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; - private CompactionToMSQTask compactionToMSQTask; public Builder() { @@ -621,7 +610,6 @@ public Builder(TaskToolbox other) this.supervisorTaskClientProvider = other.supervisorTaskClientProvider; this.shuffleClient = other.shuffleClient; this.centralizedDatasourceSchemaConfig = other.centralizedDatasourceSchemaConfig; - this.compactionToMSQTask = other.compactionToMSQTask; } public Builder config(final SegmentLoaderConfig segmentLoaderConfig) @@ -870,12 +858,6 @@ public Builder centralizedTableSchemaConfig(final CentralizedDatasourceSchemaCon return this; } - public Builder compactionToMSQ(final CompactionToMSQTask compactionToMSQTask) - { - this.compactionToMSQTask = compactionToMSQTask; - return this; - } - public TaskToolbox build() { return new TaskToolbox( @@ -919,8 +901,7 @@ public TaskToolbox build() shuffleClient, taskLogPusher, attemptId, - centralizedDatasourceSchemaConfig, - compactionToMSQTask + centralizedDatasourceSchemaConfig ); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java index ca2666bbb37c..669a30dc5de9 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java @@ -37,7 +37,6 @@ import org.apache.druid.indexer.report.TaskReportFileWriter; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; import org.apache.druid.indexing.common.config.TaskConfig; -import org.apache.druid.indexing.common.task.CompactionToMSQTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTaskClientProvider; @@ -68,7 +67,6 @@ import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.tasklogs.TaskLogPusher; -import javax.annotation.Nullable; import java.io.File; import java.util.function.Function; @@ -119,7 +117,6 @@ public class TaskToolboxFactory private final TaskLogPusher taskLogPusher; private final String attemptId; private final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; - private final CompactionToMSQTask compactionToMSQTask; @Inject public TaskToolboxFactory( @@ -162,8 +159,7 @@ public TaskToolboxFactory( ShuffleClient shuffleClient, TaskLogPusher taskLogPusher, @AttemptId String attemptId, - CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig, - @Nullable CompactionToMSQTask compactionToMSQTask + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { this.segmentLoaderConfig = segmentLoadConfig; @@ -206,7 +202,6 @@ public TaskToolboxFactory( this.taskLogPusher = taskLogPusher; this.attemptId = attemptId; this.centralizedDatasourceSchemaConfig = centralizedDatasourceSchemaConfig; - this.compactionToMSQTask = compactionToMSQTask; } public TaskToolbox build(Task task) @@ -271,7 +266,6 @@ public TaskToolbox build(TaskConfig config, Task task) .taskLogPusher(taskLogPusher) .attemptId(attemptId) .centralizedTableSchemaConfig(centralizedDatasourceSchemaConfig) - .compactionToMSQ(compactionToMSQTask) .build(); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionToMSQTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionStrategy.java similarity index 75% rename from indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionToMSQTask.java rename to indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionStrategy.java index 2f0785b690d6..f3395a27f8e6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionToMSQTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionStrategy.java @@ -19,6 +19,8 @@ package org.apache.druid.indexing.common.task; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.core.JsonProcessingException; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskToolbox; @@ -27,12 +29,18 @@ import org.joda.time.Interval; import java.util.List; - -public interface CompactionToMSQTask +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = CompactionStrategy.TYPE_PROPERTY) +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = NativeCompactionStrategy.type, value = NativeCompactionStrategy.class) +}) +public interface CompactionStrategy { - TaskStatus createAndRunMSQTasks( + String TYPE_PROPERTY = "type"; + TaskStatus runCompactionTasks( CompactionTask compactionTask, TaskToolbox taskToolbox, List> dataSchemas ) throws JsonProcessingException; + + String getType(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 35e3aa827e5d..405ac690d2a8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -25,7 +25,6 @@ import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonInclude.Include; import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.core.JsonProcessingException; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.collect.BiMap; @@ -35,7 +34,6 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import org.apache.curator.shaded.com.google.common.base.Verify; -import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; import org.apache.druid.client.indexing.ClientCompactionTaskQuery; import org.apache.druid.client.indexing.ClientCompactionTaskTransformSpec; @@ -44,13 +42,11 @@ import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.error.DruidException; import org.apache.druid.indexer.Checks; import org.apache.druid.indexer.Property; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; -import org.apache.druid.indexer.report.TaskReport; import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.RetryPolicyFactory; import org.apache.druid.indexing.common.SegmentCacheManagerFactory; @@ -58,11 +54,8 @@ import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig; -import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexIOConfig; -import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexIngestionSpec; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTask; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; -import org.apache.druid.indexing.input.DruidInputSource; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; @@ -99,7 +92,6 @@ import org.apache.druid.timeline.SegmentTimeline; import org.apache.druid.timeline.TimelineObjectHolder; import org.apache.druid.timeline.VersionedIntervalTimeline; -import org.apache.druid.utils.CollectionUtils; import org.joda.time.Duration; import org.joda.time.Interval; @@ -117,7 +109,6 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; -import java.util.Optional; import java.util.Set; import java.util.TreeMap; import java.util.function.Supplier; @@ -149,7 +140,6 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg private static final String TYPE = "compact"; - private static final boolean STORE_COMPACTION_STATE = true; static { Verify.verify(TYPE.equals(CompactSegments.COMPACTION_TASK_TYPE)); @@ -171,7 +161,7 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg @JsonIgnore private final PartitionConfigurationManager partitionConfigurationManager; @Nullable - private final Engine engine; + private final CompactionStrategy compactionStrategy; @JsonIgnore private final SegmentCacheManagerFactory segmentCacheManagerFactory; @@ -199,7 +189,7 @@ public CompactionTask( @JsonProperty("granularitySpec") @Nullable final ClientCompactionTaskGranularitySpec granularitySpec, @JsonProperty("tuningConfig") @Nullable final TuningConfig tuningConfig, @JsonProperty("context") @Nullable final Map context, - @JsonProperty("engine") @Nullable final Engine engine, + @JsonProperty("compactionStrategy") final CompactionStrategy compactionStrategy, @JacksonInject SegmentCacheManagerFactory segmentCacheManagerFactory ) { @@ -253,7 +243,7 @@ public CompactionTask( this.tuningConfig = tuningConfig != null ? getTuningConfig(tuningConfig) : null; this.segmentProvider = new SegmentProvider(dataSource, this.ioConfig.getInputSpec()); this.partitionConfigurationManager = new PartitionConfigurationManager(this.tuningConfig); - this.engine = engine; + this.compactionStrategy = compactionStrategy; this.segmentCacheManagerFactory = segmentCacheManagerFactory; } @@ -362,6 +352,16 @@ public DimensionsSpec getDimensionsSpec() return dimensionsSpec; } + public PartitionConfigurationManager getPartitionConfigurationManager() + { + return partitionConfigurationManager; + } + + public SegmentCacheManagerFactory getSegmentCacheManagerFactory() + { + return segmentCacheManagerFactory; + } + @JsonProperty @Nullable public ClientCompactionTaskTransformSpec getTransformSpec() @@ -400,9 +400,9 @@ public ParallelIndexTuningConfig getTuningConfig() } @JsonProperty - public Engine getEngine() + public CompactionStrategy getCompactionStrategy() { - return engine; + return compactionStrategy; } @Override @@ -490,7 +490,7 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception getMetricBuilder() ); - if (engine == Engine.MSQ) { + /*if (engine == Engine.MSQ) { if (toolbox.getCompactionToMSQ() == null) { throw DruidException.forPersona(DruidException.Persona.ADMIN) .ofCategory(DruidException.Category.NOT_FOUND) @@ -500,125 +500,14 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception } registerResourceCloserOnAbnormalExit(currentSubTaskHolder); return toolbox.getCompactionToMSQ() - .createAndRunMSQTasks(this, toolbox, intervalDataSchemas); + .runCompactionTasks(this, toolbox, intervalDataSchemas); } else { - final List ingestionSpecs = createIngestionSpecs( - intervalDataSchemas, - toolbox, - ioConfig, - partitionConfigurationManager, - toolbox.getCoordinatorClient(), - segmentCacheManagerFactory - ); - - List subtasks = IntStream - .range(0, ingestionSpecs.size()) - .mapToObj(i -> { - // The ID of SubtaskSpecs is used as the base sequenceName in segment allocation protocol. - // The indexing tasks generated by the compaction task should use different sequenceNames - // so that they can allocate valid segment IDs with no duplication. - ParallelIndexIngestionSpec ingestionSpec = ingestionSpecs.get(i); - final String baseSequenceName = createIndexTaskSpecId(i); - return newTask(baseSequenceName, ingestionSpec); - }) - .collect(Collectors.toList()); - - if (subtasks.isEmpty()) { - String msg = StringUtils.format( - "Can't find segments from inputSpec[%s], nothing to do.", - ioConfig.getInputSpec() - ); - log.warn(msg); - return TaskStatus.failure(getId(), msg); - } - registerResourceCloserOnAbnormalExit(currentSubTaskHolder); - return runParallelIndexSubtasks(subtasks, toolbox); - } - } - - private TaskStatus runParallelIndexSubtasks(List tasks, TaskToolbox toolbox) - throws JsonProcessingException - { - final int totalNumSpecs = tasks.size(); - log.info("Generated [%d] compaction task specs", totalNumSpecs); - - int failCnt = 0; - final TaskReport.ReportMap completionReports = new TaskReport.ReportMap(); - for (int i = 0; i < tasks.size(); i++) { - ParallelIndexSupervisorTask eachSpec = tasks.get(i); - final String json = toolbox.getJsonMapper().writerWithDefaultPrettyPrinter().writeValueAsString(eachSpec); - if (!currentSubTaskHolder.setTask(eachSpec)) { - String errMsg = "Task was asked to stop. Finish as failed."; - log.info(errMsg); - return TaskStatus.failure(getId(), errMsg); - } - try { - if (eachSpec.isReady(toolbox.getTaskActionClient())) { - log.info("Running indexSpec: " + json); - final TaskStatus eachResult = eachSpec.run(toolbox); - if (!eachResult.isSuccess()) { - failCnt++; - log.warn("Failed to run indexSpec: [%s].\nTrying the next indexSpec.", json); - } - - String reportKeySuffix = "_" + i; - Optional.ofNullable(eachSpec.getCompletionReports()) - .ifPresent(reports -> completionReports.putAll( - CollectionUtils.mapKeys(reports, key -> key + reportKeySuffix))); - } else { - failCnt++; - log.warn("indexSpec is not ready: [%s].\nTrying the next indexSpec.", json); - } - } - catch (Exception e) { - failCnt++; - log.warn(e, "Failed to run indexSpec: [%s].\nTrying the next indexSpec.", json); - } - } - - String msg = StringUtils.format( - "Ran [%d] specs, [%d] succeeded, [%d] failed", - totalNumSpecs, - totalNumSpecs - failCnt, - failCnt - ); - - toolbox.getTaskReportFileWriter().write(getId(), completionReports); - log.info(msg); - return failCnt == 0 ? TaskStatus.success(getId()) : TaskStatus.failure(getId(), msg); - } - - @VisibleForTesting - ParallelIndexSupervisorTask newTask(String baseSequenceName, ParallelIndexIngestionSpec ingestionSpec) - { - return new ParallelIndexSupervisorTask( - getId(), - getGroupId(), - getTaskResource(), - ingestionSpec, - baseSequenceName, - createContextForSubtask(), - true - ); - } - @VisibleForTesting - Map createContextForSubtask() - { - final Map newContext = new HashMap<>(getContext()); - newContext.put(CTX_KEY_APPENDERATOR_TRACKING_TASK_ID, getId()); - newContext.putIfAbsent(CompactSegments.STORE_COMPACTION_STATE_KEY, STORE_COMPACTION_STATE); - // Set the priority of the compaction task. - newContext.put(Tasks.PRIORITY_KEY, getPriority()); - return newContext; + }*/ + registerResourceCloserOnAbnormalExit(currentSubTaskHolder); + return compactionStrategy.runCompactionTasks(this, toolbox, intervalDataSchemas); } - private String createIndexTaskSpecId(int i) - { - return StringUtils.format("%s_%d", getId(), i); - } - - /** * Generate dataschema for segments in each interval * @return @@ -731,84 +620,6 @@ static List> createDataSchemasForIntervals( } } - /** - * Generate {@link ParallelIndexIngestionSpec} from input dataschemas. - * - * @return an empty list if input segments don't exist. Otherwise, a generated ingestionSpec. - */ - @VisibleForTesting - static List createIngestionSpecs( - List> dataschemas, - final TaskToolbox toolbox, - final CompactionIOConfig ioConfig, - final PartitionConfigurationManager partitionConfigurationManager, - final CoordinatorClient coordinatorClient, - final SegmentCacheManagerFactory segmentCacheManagerFactory - ) - { - final CompactionTuningConfig compactionTuningConfig = partitionConfigurationManager.computeTuningConfig(); - - return dataschemas.stream().map((dataSchema) -> new ParallelIndexIngestionSpec( - dataSchema.rhs, - createIoConfig(toolbox, - dataSchema.rhs, - dataSchema.lhs, - coordinatorClient, - segmentCacheManagerFactory, - ioConfig - ), - compactionTuningConfig - ) - - ).collect(Collectors.toList()); - } - - private static ParallelIndexIOConfig createIoConfig( - TaskToolbox toolbox, - DataSchema dataSchema, - Interval interval, - CoordinatorClient coordinatorClient, - SegmentCacheManagerFactory segmentCacheManagerFactory, - CompactionIOConfig compactionIOConfig - ) - { - if (!compactionIOConfig.isAllowNonAlignedInterval()) { - // Validate interval alignment. - final Granularity segmentGranularity = dataSchema.getGranularitySpec().getSegmentGranularity(); - final Interval widenedInterval = Intervals.utc( - segmentGranularity.bucketStart(interval.getStart()).getMillis(), - segmentGranularity.bucketEnd(interval.getEnd().minus(1)).getMillis() - ); - - if (!interval.equals(widenedInterval)) { - throw new IAE( - "Interval[%s] to compact is not aligned with segmentGranularity[%s]", - interval, - segmentGranularity - ); - } - } - - return new ParallelIndexIOConfig( - null, - new DruidInputSource( - dataSchema.getDataSource(), - interval, - null, - null, - null, - null, - toolbox.getIndexIO(), - coordinatorClient, - segmentCacheManagerFactory, - toolbox.getConfig() - ).withTaskToolbox(toolbox), - null, - false, - compactionIOConfig.isDropExisting() - ); - } - private static List> retrieveRelevantTimelineHolders( TaskToolbox toolbox, SegmentProvider segmentProvider, @@ -1288,7 +1099,7 @@ public static class Builder private TuningConfig tuningConfig; @Nullable private Map context; - private Engine engine; + private CompactionStrategy compactionStrategy; public Builder( String dataSource, @@ -1371,9 +1182,9 @@ public Builder context(Map context) return this; } - public Builder engine(Engine engine) + public Builder engine(CompactionStrategy engine) { - this.engine = engine; + this.compactionStrategy = engine; return this; } @@ -1396,7 +1207,7 @@ public CompactionTask build() granularitySpec, tuningConfig, context, - engine, + compactionStrategy, segmentCacheManagerFactory ); } @@ -1566,4 +1377,5 @@ public CompactionTuningConfig withPartitionsSpec(PartitionsSpec partitionsSpec) ); } } + } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Engine.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Engine.java index f91ea1434d49..79036e2765cc 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Engine.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Engine.java @@ -21,10 +21,11 @@ import com.fasterxml.jackson.annotation.JsonCreator; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.server.coordinator.UserCompactionStrategy; /** * Engine to be used for a compaction task. - * Should be synchronized with {@link org.apache.druid.server.coordinator.DataSourceCompactionConfig.Engine}. + * Should be synchronized with {@link UserCompactionStrategy.CompactionEngine}. */ public enum Engine { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionStrategy.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionStrategy.java new file mode 100644 index 000000000000..1f71fa3d255f --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionStrategy.java @@ -0,0 +1,262 @@ +package org.apache.druid.indexing.common.task; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.google.common.annotations.VisibleForTesting; +import org.apache.druid.client.coordinator.CoordinatorClient; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.report.TaskReport; +import org.apache.druid.indexing.common.SegmentCacheManagerFactory; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexIOConfig; +import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexIngestionSpec; +import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTask; +import org.apache.druid.indexing.input.DruidInputSource; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.NonnullPair; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.server.coordinator.duty.CompactSegments; +import org.apache.druid.utils.CollectionUtils; +import org.codehaus.jackson.annotate.JsonCreator; +import org.joda.time.Interval; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.apache.druid.indexing.common.task.CompactionTask.CTX_KEY_APPENDERATOR_TRACKING_TASK_ID; + +public class NativeCompactionStrategy implements CompactionStrategy +{ + + private static final Logger log = new Logger(NativeCompactionStrategy.class); + public static final String type = "NATIVE"; + private static final boolean STORE_COMPACTION_STATE = true; + + @JsonCreator + public NativeCompactionStrategy() + { + } + + /** + * Generate {@link ParallelIndexIngestionSpec} from input dataschemas. + * + * @return an empty list if input segments don't exist. Otherwise, a generated ingestionSpec. + */ + @VisibleForTesting + static List createIngestionSpecs( + List> dataschemas, + final TaskToolbox toolbox, + final CompactionIOConfig ioConfig, + final CompactionTask.PartitionConfigurationManager partitionConfigurationManager, + final CoordinatorClient coordinatorClient, + final SegmentCacheManagerFactory segmentCacheManagerFactory + ) + { + final CompactionTask.CompactionTuningConfig compactionTuningConfig = partitionConfigurationManager.computeTuningConfig(); + + return dataschemas.stream().map((dataSchema) -> new ParallelIndexIngestionSpec( + dataSchema.rhs, + createIoConfig( + toolbox, + dataSchema.rhs, + dataSchema.lhs, + coordinatorClient, + segmentCacheManagerFactory, + ioConfig + ), + compactionTuningConfig + ) + + ).collect(Collectors.toList()); + } + + private String createIndexTaskSpecId(String taskId, int i) + { + return StringUtils.format("%s_%d", taskId, i); + } + + + private static ParallelIndexIOConfig createIoConfig( + TaskToolbox toolbox, + DataSchema dataSchema, + Interval interval, + CoordinatorClient coordinatorClient, + SegmentCacheManagerFactory segmentCacheManagerFactory, + CompactionIOConfig compactionIOConfig + ) + { + if (!compactionIOConfig.isAllowNonAlignedInterval()) { + // Validate interval alignment. + final Granularity segmentGranularity = dataSchema.getGranularitySpec().getSegmentGranularity(); + final Interval widenedInterval = Intervals.utc( + segmentGranularity.bucketStart(interval.getStart()).getMillis(), + segmentGranularity.bucketEnd(interval.getEnd().minus(1)).getMillis() + ); + + if (!interval.equals(widenedInterval)) { + throw new IAE( + "Interval[%s] to compact is not aligned with segmentGranularity[%s]", + interval, + segmentGranularity + ); + } + } + + return new ParallelIndexIOConfig( + null, + new DruidInputSource( + dataSchema.getDataSource(), + interval, + null, + null, + null, + null, + toolbox.getIndexIO(), + coordinatorClient, + segmentCacheManagerFactory, + toolbox.getConfig() + ).withTaskToolbox(toolbox), + null, + false, + compactionIOConfig.isDropExisting() + ); + } + + @Override + public TaskStatus runCompactionTasks( + CompactionTask compactionTask, + TaskToolbox taskToolbox, + List> dataSchemas + ) throws JsonProcessingException + { + final List ingestionSpecs = createIngestionSpecs( + dataSchemas, + taskToolbox, + compactionTask.getIoConfig(), + compactionTask.getPartitionConfigurationManager(), + taskToolbox.getCoordinatorClient(), + compactionTask.getSegmentCacheManagerFactory() + ); + + List subtasks = IntStream + .range(0, ingestionSpecs.size()) + .mapToObj(i -> { + // The ID of SubtaskSpecs is used as the base sequenceName in segment allocation protocol. + // The indexing tasks generated by the compaction task should use different sequenceNames + // so that they can allocate valid segment IDs with no duplication. + ParallelIndexIngestionSpec ingestionSpec = ingestionSpecs.get(i); + final String baseSequenceName = createIndexTaskSpecId(compactionTask.getId(), i); + return newTask(compactionTask, baseSequenceName, ingestionSpec); + }) + .collect(Collectors.toList()); + + if (subtasks.isEmpty()) { + String msg = StringUtils.format( + "Can't find segments from inputSpec[%s], nothing to do.", + compactionTask.getIoConfig().getInputSpec() + ); + log.warn(msg); + return TaskStatus.failure(compactionTask.getId(), msg); + } + return runParallelIndexSubtasks( + subtasks, + taskToolbox, + compactionTask.getCurrentSubTaskHolder(), + compactionTask.getId() + ); + } + + private TaskStatus runParallelIndexSubtasks( + List tasks, + TaskToolbox toolbox, + CurrentSubTaskHolder currentSubTaskHolder, + String compactionTaskId + ) + throws JsonProcessingException + { + final int totalNumSpecs = tasks.size(); + log.info("Generated [%d] compaction task specs", totalNumSpecs); + + int failCnt = 0; + final TaskReport.ReportMap completionReports = new TaskReport.ReportMap(); + for (int i = 0; i < tasks.size(); i++) { + ParallelIndexSupervisorTask eachSpec = tasks.get(i); + final String json = toolbox.getJsonMapper().writerWithDefaultPrettyPrinter().writeValueAsString(eachSpec); + if (!currentSubTaskHolder.setTask(eachSpec)) { + String errMsg = "Task was asked to stop. Finish as failed."; + log.info(errMsg); + return TaskStatus.failure(compactionTaskId, errMsg); + } + try { + if (eachSpec.isReady(toolbox.getTaskActionClient())) { + log.info("Running indexSpec: " + json); + final TaskStatus eachResult = eachSpec.run(toolbox); + if (!eachResult.isSuccess()) { + failCnt++; + log.warn("Failed to run indexSpec: [%s].\nTrying the next indexSpec.", json); + } + + String reportKeySuffix = "_" + i; + Optional.ofNullable(eachSpec.getCompletionReports()) + .ifPresent(reports -> completionReports.putAll( + CollectionUtils.mapKeys(reports, key -> key + reportKeySuffix))); + } else { + failCnt++; + log.warn("indexSpec is not ready: [%s].\nTrying the next indexSpec.", json); + } + } + catch (Exception e) { + failCnt++; + log.warn(e, "Failed to run indexSpec: [%s].\nTrying the next indexSpec.", json); + } + } + + String msg = StringUtils.format( + "Ran [%d] specs, [%d] succeeded, [%d] failed", + totalNumSpecs, + totalNumSpecs - failCnt, + failCnt + ); + + toolbox.getTaskReportFileWriter().write(compactionTaskId, completionReports); + log.info(msg); + return failCnt == 0 ? TaskStatus.success(compactionTaskId) : TaskStatus.failure(compactionTaskId, msg); + } + + @VisibleForTesting + ParallelIndexSupervisorTask newTask(CompactionTask compactionTask, String baseSequenceName, ParallelIndexIngestionSpec ingestionSpec) + { + return new ParallelIndexSupervisorTask( + compactionTask.getId(), + compactionTask.getGroupId(), + compactionTask.getTaskResource(), + ingestionSpec, + baseSequenceName, + createContextForSubtask(compactionTask), + true + ); + } + + Map createContextForSubtask(CompactionTask compactionTask) + { + final Map newContext = new HashMap<>(compactionTask.getContext()); + newContext.put(CTX_KEY_APPENDERATOR_TRACKING_TASK_ID, compactionTask.getId()); + newContext.putIfAbsent(CompactSegments.STORE_COMPACTION_STATE_KEY, STORE_COMPACTION_STATE); + // Set the priority of the compaction task. + newContext.put(Tasks.PRIORITY_KEY, compactionTask.getPriority()); + return newContext; + } + + @Override + public String getType() + { + return type; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java index 56123a561f6b..49e430cb3ea5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java @@ -210,6 +210,7 @@ public Response taskPost( .build(); } + Access authResult = AuthorizationUtils.authorizeAllResourceActions( req, resourceActions, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java index e63ce24d41bc..e1ac9482436b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java @@ -157,8 +157,7 @@ public void setUp() throws IOException null, null, "1", - CentralizedDatasourceSchemaConfig.create(), - null + CentralizedDatasourceSchemaConfig.create() ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java index 547176238b66..79c4ef86ec6c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java @@ -1647,8 +1647,7 @@ public void close() null, null, "1", - CentralizedDatasourceSchemaConfig.create(), - null + CentralizedDatasourceSchemaConfig.create() ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java index 898735244f29..3e3adbecd4f5 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java @@ -63,7 +63,7 @@ import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; -import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.UserCompactionStrategy; import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.server.security.AuthorizerMapper; import org.joda.time.Duration; @@ -129,7 +129,7 @@ public void testClientCompactionTaskQueryToCompactionTask() throws IOException new AggregatorFactory[] {new CountAggregatorFactory("cnt")}, new ClientCompactionTaskTransformSpec(new SelectorDimFilter("dim1", "foo", null)), ImmutableMap.of("key", "value"), - DataSourceCompactionConfig.Engine.MSQ + new UserCompactionStrategy(UserCompactionStrategy.CompactionEngine.NATIVE) ); final byte[] json = mapper.writeValueAsBytes(query); @@ -236,8 +236,8 @@ public void testClientCompactionTaskQueryToCompactionTask() throws IOException task.getMetricsSpec() ); Assert.assertEquals( - query.getEngine().toString(), - task.getEngine().toString() + query.getCompactionStrategy().getType().toString(), + task.getCompactionStrategy().getType().toString() ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index a42d30938ca6..476f89f4d09e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -965,7 +965,7 @@ public void testCreateIngestionSchema() throws IOException METRIC_BUILDER ); - final List ingestionSpecs = CompactionTask.createIngestionSpecs( + final List ingestionSpecs = NativeCompactionStrategy.createIngestionSpecs( dataSchemasForIntervals, toolbox, new CompactionIOConfig(null, false, null), @@ -1047,7 +1047,7 @@ public void testCreateIngestionSchemaWithTargetPartitionSize() throws IOExceptio METRIC_BUILDER ); - final List ingestionSpecs = CompactionTask.createIngestionSpecs( + final List ingestionSpecs = NativeCompactionStrategy.createIngestionSpecs( dataSchemasForIntervals, toolbox, new CompactionIOConfig(null, false, null), @@ -1130,7 +1130,7 @@ public void testCreateIngestionSchemaWithMaxTotalRows() throws IOException METRIC_BUILDER ); - final List ingestionSpecs = CompactionTask.createIngestionSpecs( + final List ingestionSpecs = NativeCompactionStrategy.createIngestionSpecs( dataSchemasForIntervals, toolbox, new CompactionIOConfig(null, false, null), @@ -1213,7 +1213,7 @@ public void testCreateIngestionSchemaWithNumShards() throws IOException METRIC_BUILDER ); - final List ingestionSpecs = CompactionTask.createIngestionSpecs( + final List ingestionSpecs = NativeCompactionStrategy.createIngestionSpecs( dataSchemasForIntervals, toolbox, new CompactionIOConfig(null, false, null), @@ -1284,7 +1284,7 @@ public void testCreateIngestionSchemaWithCustomDimensionsSpec() throws IOExcepti METRIC_BUILDER ); - final List ingestionSpecs = CompactionTask.createIngestionSpecs( + final List ingestionSpecs = NativeCompactionStrategy.createIngestionSpecs( dataSchemasForIntervals, toolbox, new CompactionIOConfig(null, false, null), @@ -1335,7 +1335,7 @@ public void testCreateIngestionSchemaWithCustomMetricsSpec() throws IOException METRIC_BUILDER ); - final List ingestionSpecs = CompactionTask.createIngestionSpecs( + final List ingestionSpecs = NativeCompactionStrategy.createIngestionSpecs( dataSchemasForIntervals, toolbox, new CompactionIOConfig(null, false, null), @@ -1379,7 +1379,7 @@ public void testCreateIngestionSchemaWithCustomSegments() throws IOException METRIC_BUILDER ); - final List ingestionSpecs = CompactionTask.createIngestionSpecs( + final List ingestionSpecs = NativeCompactionStrategy.createIngestionSpecs( dataSchemasForIntervals, toolbox, new CompactionIOConfig(null, false, null), @@ -1430,7 +1430,7 @@ public void testCreateIngestionSchemaWithDifferentSegmentSet() throws IOExceptio METRIC_BUILDER ); - CompactionTask.createIngestionSpecs( + NativeCompactionStrategy.createIngestionSpecs( dataSchemasForIntervals, toolbox, new CompactionIOConfig(null, false, null), @@ -1461,7 +1461,7 @@ public void testMissingMetadata() throws IOException METRIC_BUILDER ); - CompactionTask.createIngestionSpecs( + NativeCompactionStrategy.createIngestionSpecs( dataSchemasForIntervals, toolbox, new CompactionIOConfig(null, false, null), @@ -1504,7 +1504,7 @@ public void testSegmentGranularityAndNullQueryGranularity() throws IOException METRIC_BUILDER ); - final List ingestionSpecs = CompactionTask.createIngestionSpecs( + final List ingestionSpecs = NativeCompactionStrategy.createIngestionSpecs( dataSchemasForIntervals, toolbox, new CompactionIOConfig(null, false, null), @@ -1548,7 +1548,7 @@ public void testQueryGranularityAndNullSegmentGranularity() throws IOException new ClientCompactionTaskGranularitySpec(null, new PeriodGranularity(Period.months(3), null, null), null), METRIC_BUILDER ); - final List ingestionSpecs = CompactionTask.createIngestionSpecs( + final List ingestionSpecs = NativeCompactionStrategy.createIngestionSpecs( dataSchemasForIntervals, toolbox, new CompactionIOConfig(null, false, null), @@ -1595,7 +1595,7 @@ public void testQueryGranularityAndSegmentGranularityNonNull() throws IOExceptio METRIC_BUILDER ); - final List ingestionSpecs = CompactionTask.createIngestionSpecs( + final List ingestionSpecs = NativeCompactionStrategy.createIngestionSpecs( dataSchemasForIntervals, toolbox, new CompactionIOConfig(null, false, null), @@ -1643,7 +1643,7 @@ public void testNullGranularitySpec() throws IOException METRIC_BUILDER ); - final List ingestionSpecs = CompactionTask.createIngestionSpecs( + final List ingestionSpecs = NativeCompactionStrategy.createIngestionSpecs( dataSchemasForIntervals, toolbox, new CompactionIOConfig(null, false, null), @@ -1689,7 +1689,7 @@ public void testGranularitySpecWithNullQueryGranularityAndNullSegmentGranularity METRIC_BUILDER ); - final List ingestionSpecs = CompactionTask.createIngestionSpecs( + final List ingestionSpecs = NativeCompactionStrategy.createIngestionSpecs( dataSchemasForIntervals, toolbox, new CompactionIOConfig(null, false, null), @@ -1735,7 +1735,7 @@ public void testGranularitySpecWithNotNullRollup() METRIC_BUILDER ); - final List ingestionSpecs = CompactionTask.createIngestionSpecs( + final List ingestionSpecs = NativeCompactionStrategy.createIngestionSpecs( dataSchemasForIntervals, toolbox, new CompactionIOConfig(null, false, null), @@ -1766,7 +1766,7 @@ public void testGranularitySpecWithNullRollup() METRIC_BUILDER ); - final List ingestionSpecs = CompactionTask.createIngestionSpecs( + final List ingestionSpecs = NativeCompactionStrategy.createIngestionSpecs( dataSchemasForIntervals, toolbox, new CompactionIOConfig(null, false, null), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java index 693b21459666..43253a10bccc 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -1020,8 +1020,7 @@ public void close() null, null, "1", - CentralizedDatasourceSchemaConfig.create(), - null + CentralizedDatasourceSchemaConfig.create() ); return toolboxFactory.build(task); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java index 8da6f58bd4d2..30d2d289abac 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java @@ -137,8 +137,7 @@ public void setup() throws IOException null, null, "1", - CentralizedDatasourceSchemaConfig.create(), - null + CentralizedDatasourceSchemaConfig.create() ); runner = new SingleTaskBackgroundRunner( toolboxFactory, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java index 4cd236e270f6..0046645106ca 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java @@ -667,8 +667,7 @@ public void announceSegment(DataSegment segment) null, null, "1", - CentralizedDatasourceSchemaConfig.create(), - null + CentralizedDatasourceSchemaConfig.create() ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskToolboxFactory.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskToolboxFactory.java index 35b1a16bb453..afc1945ee096 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskToolboxFactory.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskToolboxFactory.java @@ -36,7 +36,7 @@ import org.apache.druid.indexing.common.actions.TaskActionClientFactory; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.config.TaskConfigBuilder; -import org.apache.druid.indexing.common.task.CompactionToMSQTask; +import org.apache.druid.indexing.common.task.CompactionStrategy; import org.apache.druid.indexing.common.task.NoopTestTaskReportFileWriter; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTaskClientProvider; import org.apache.druid.indexing.common.task.batch.parallel.ShuffleClient; @@ -118,8 +118,7 @@ public TestTaskToolboxFactory( bob.shuffleClient, bob.taskLogPusher, bob.attemptId, - bob.centralizedDatasourceSchemaConfig, - bob.compactionToMSQTask + bob.centralizedDatasourceSchemaConfig ); } @@ -164,7 +163,7 @@ public static class Builder private TaskLogPusher taskLogPusher; private String attemptId; private CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; - private CompactionToMSQTask compactionToMSQTask; + private CompactionStrategy compactionStrategy; public Builder setConfig(TaskConfig config) { @@ -400,9 +399,9 @@ public Builder setCentralizedTableSchemaConfig(CentralizedDatasourceSchemaConfig return this; } - public Builder setCompactionToMSQ(CompactionToMSQTask compactionToMSQTask) + public Builder setCompactionToMSQ(CompactionStrategy compactionStrategy) { - this.compactionToMSQTask = compactionToMSQTask; + this.compactionStrategy = compactionStrategy; return this; } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java index 4b80f2c5af3c..eabb640c133e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java @@ -701,8 +701,7 @@ public void close() null, null, "1", - CentralizedDatasourceSchemaConfig.create(), - null + CentralizedDatasourceSchemaConfig.create() ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java index 00d4bab8f8a6..1aeb67d5a404 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java @@ -168,8 +168,7 @@ private WorkerTaskManager createWorkerTaskManager() null, null, "1", - CentralizedDatasourceSchemaConfig.create(), - null + CentralizedDatasourceSchemaConfig.create() ), taskConfig, location diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java index 3ac280fec20e..4e1a801979c5 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java @@ -210,8 +210,7 @@ private WorkerTaskMonitor createTaskMonitor() null, null, "1", - CentralizedDatasourceSchemaConfig.create(), - null + CentralizedDatasourceSchemaConfig.create() ), taskConfig, new NoopServiceEmitter(), diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQuery.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQuery.java index 0a5872dc0e62..61562dd4fe5b 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQuery.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQuery.java @@ -23,7 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.UserCompactionStrategy; import javax.annotation.Nullable; import java.util.Arrays; @@ -47,7 +47,7 @@ public class ClientCompactionTaskQuery implements ClientTaskQuery private final AggregatorFactory[] metricsSpec; private final ClientCompactionTaskTransformSpec transformSpec; private final Map context; - private final DataSourceCompactionConfig.Engine engine; + private final UserCompactionStrategy compactionStrategy; @JsonCreator public ClientCompactionTaskQuery( @@ -60,7 +60,7 @@ public ClientCompactionTaskQuery( @JsonProperty("metricsSpec") AggregatorFactory[] metrics, @JsonProperty("transformSpec") ClientCompactionTaskTransformSpec transformSpec, @JsonProperty("context") Map context, - @JsonProperty("engine") DataSourceCompactionConfig.Engine engine + @JsonProperty("compactionStrategy") UserCompactionStrategy compactionStrategy ) { this.id = Preconditions.checkNotNull(id, "id"); @@ -72,7 +72,7 @@ public ClientCompactionTaskQuery( this.metricsSpec = metrics; this.transformSpec = transformSpec; this.context = context; - this.engine = engine; + this.compactionStrategy = compactionStrategy; } @JsonProperty @@ -140,9 +140,9 @@ public Map getContext() } @JsonProperty - public DataSourceCompactionConfig.Engine getEngine() + public UserCompactionStrategy getCompactionStrategy() { - return engine; + return compactionStrategy; } @Override @@ -164,7 +164,7 @@ public boolean equals(Object o) Arrays.equals(metricsSpec, that.metricsSpec) && Objects.equals(transformSpec, that.transformSpec) && Objects.equals(context, that.context) && - Objects.equals(engine, that.engine); + Objects.equals(compactionStrategy, that.compactionStrategy); } @Override @@ -179,7 +179,7 @@ public int hashCode() dimensionsSpec, transformSpec, context, - engine + compactionStrategy ); result = 31 * result + Arrays.hashCode(metricsSpec); return result; @@ -198,7 +198,7 @@ public String toString() ", metricsSpec=" + Arrays.toString(metricsSpec) + ", transformSpec=" + transformSpec + ", context=" + context + - ", engine=" + engine + + ", compactionStrategy=" + compactionStrategy + '}'; } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfig.java index e03772c534dc..d4c679033539 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfig.java @@ -34,13 +34,12 @@ public class CoordinatorCompactionConfig private static final double DEFAULT_COMPACTION_TASK_RATIO = 0.1; private static final int DEFAULT_MAX_COMPACTION_TASK_SLOTS = Integer.MAX_VALUE; private static final boolean DEFAULT_USE_AUTO_SCALE_SLOTS = false; - private static final DataSourceCompactionConfig.Engine DEFAULT_COMPACTION_ENGINE = DataSourceCompactionConfig.Engine.NATIVE; - + private static final UserCompactionStrategy.CompactionEngine DEFAULT_COMPACTION_ENGINE = UserCompactionStrategy.CompactionEngine.NATIVE; private final List compactionConfigs; private final double compactionTaskSlotRatio; private final int maxCompactionTaskSlots; private final boolean useAutoScaleSlots; - private final DataSourceCompactionConfig.Engine engine; + private final UserCompactionStrategy.CompactionEngine compactionEngine; public static CoordinatorCompactionConfig from( CoordinatorCompactionConfig baseConfig, @@ -96,7 +95,7 @@ public CoordinatorCompactionConfig( DEFAULT_MAX_COMPACTION_TASK_SLOTS : maxCompactionTaskSlots; this.useAutoScaleSlots = useAutoScaleSlots == null ? DEFAULT_USE_AUTO_SCALE_SLOTS : useAutoScaleSlots; - this.engine = DEFAULT_COMPACTION_ENGINE; + this.compactionEngine = DEFAULT_COMPACTION_ENGINE; } @JsonProperty @@ -124,9 +123,9 @@ public boolean isUseAutoScaleSlots() } @JsonProperty - public DataSourceCompactionConfig.Engine getEngine() + public UserCompactionStrategy.CompactionEngine getEngine() { - return engine; + return compactionEngine; } @Override @@ -142,14 +141,16 @@ public boolean equals(Object o) return Double.compare(that.compactionTaskSlotRatio, compactionTaskSlotRatio) == 0 && maxCompactionTaskSlots == that.maxCompactionTaskSlots && useAutoScaleSlots == that.useAutoScaleSlots && - engine == that.engine && + compactionEngine == that.compactionEngine && Objects.equals(compactionConfigs, that.compactionConfigs); } @Override public int hashCode() { - return Objects.hash(compactionConfigs, compactionTaskSlotRatio, maxCompactionTaskSlots, useAutoScaleSlots, engine); + return Objects.hash(compactionConfigs, compactionTaskSlotRatio, maxCompactionTaskSlots, useAutoScaleSlots, + compactionEngine + ); } @Override @@ -160,7 +161,7 @@ public String toString() ", compactionTaskSlotRatio=" + compactionTaskSlotRatio + ", maxCompactionTaskSlots=" + maxCompactionTaskSlots + ", useAutoScaleSlots=" + useAutoScaleSlots + - ", engine=" + engine + + ", engine=" + compactionEngine + '}'; } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java index f55054b0f3c6..e3b90e2160a9 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java @@ -26,7 +26,6 @@ import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.aggregation.AggregatorFactory; import org.joda.time.Period; @@ -60,7 +59,7 @@ public class DataSourceCompactionConfig private final UserCompactionTaskTransformConfig transformSpec; private final UserCompactionTaskIOConfig ioConfig; private final Map taskContext; - private final Engine engine; + private final UserCompactionStrategy.CompactionEngine compactionEngine; @JsonCreator public DataSourceCompactionConfig( @@ -75,7 +74,7 @@ public DataSourceCompactionConfig( @JsonProperty("metricsSpec") @Nullable AggregatorFactory[] metricsSpec, @JsonProperty("transformSpec") @Nullable UserCompactionTaskTransformConfig transformSpec, @JsonProperty("ioConfig") @Nullable UserCompactionTaskIOConfig ioConfig, - @JsonProperty("engine") @Nullable Engine engine, + @JsonProperty("engine") @Nullable UserCompactionStrategy.CompactionEngine compactionEngine, @JsonProperty("taskContext") @Nullable Map taskContext ) { @@ -95,7 +94,7 @@ public DataSourceCompactionConfig( this.dimensionsSpec = dimensionsSpec; this.transformSpec = transformSpec; this.taskContext = taskContext; - this.engine = engine; + this.compactionEngine = compactionEngine; } @JsonProperty @@ -181,9 +180,9 @@ public Map getTaskContext() @JsonProperty @Nullable - public Engine getEngine() + public UserCompactionStrategy.CompactionEngine getEngine() { - return engine; + return compactionEngine; } @Override @@ -207,7 +206,7 @@ public boolean equals(Object o) Arrays.equals(metricsSpec, that.metricsSpec) && Objects.equals(transformSpec, that.transformSpec) && Objects.equals(ioConfig, that.ioConfig) && - Objects.equals(engine, that.engine) && + Objects.equals(compactionEngine, that.compactionEngine) && Objects.equals(taskContext, that.taskContext); } @@ -226,21 +225,24 @@ public int hashCode() transformSpec, ioConfig, taskContext, - engine + compactionEngine ); result = 31 * result + Arrays.hashCode(metricsSpec); return result; } - public static DataSourceCompactionConfig from(Engine defaultEngine, DataSourceCompactionConfig newConfig) + public static DataSourceCompactionConfig from( + UserCompactionStrategy.CompactionEngine defaultCompactionEngine, + DataSourceCompactionConfig newConfig + ) { - Engine newEngine = newConfig.getEngine(); + UserCompactionStrategy.CompactionEngine newCompactionEngine = newConfig.getEngine(); String engineSourceLog = "specified in spec"; - if (newEngine == null) { - newEngine = defaultEngine; + if (newCompactionEngine == null) { + newCompactionEngine = defaultCompactionEngine; engineSourceLog = "set as default"; } - if (newEngine == Engine.MSQ) { + if (newCompactionEngine == UserCompactionStrategy.CompactionEngine.MSQ) { if (newConfig.getTuningConfig() != null) { PartitionsSpec partitionsSpec = newConfig.getTuningConfig().getPartitionsSpec(); @@ -268,7 +270,7 @@ public static DataSourceCompactionConfig from(Engine defaultEngine, DataSourceCo throw InvalidInput.exception("rollup in granularitySpec must be set to True if metricsSpec is specifed."); } } - if (newEngine == newConfig.getEngine()) { + if (newCompactionEngine == newConfig.getEngine()) { return newConfig; } return new DataSourceCompactionConfig( @@ -283,26 +285,8 @@ public static DataSourceCompactionConfig from(Engine defaultEngine, DataSourceCo newConfig.getMetricsSpec(), newConfig.getTransformSpec(), newConfig.getIoConfig(), - newEngine, newConfig.getTaskContext() + newCompactionEngine, newConfig.getTaskContext() ); } - /** - * Engine to be used for a compaction task. - * Should be synchronized with {@link org.apache.druid.indexing.common.task.Engine}. - */ - public enum Engine - { - NATIVE, - MSQ; - - @JsonCreator - public static Engine fromString(String name) - { - if (name == null) { - return null; - } - return valueOf(StringUtils.toUpperCase(name)); - } - } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionStrategy.java b/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionStrategy.java new file mode 100644 index 000000000000..b3b7032c9e27 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionStrategy.java @@ -0,0 +1,48 @@ +package org.apache.druid.server.coordinator; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.java.util.common.StringUtils; + +/** + * Encapsulates the Engine to be used for a compaction task. + * Should be synchronized with the types for {@link org.apache.druid.indexing.common.task.CompactionTasksLauncher}. + */ +public class UserCompactionStrategy +{ + private final CompactionEngine type; + + public UserCompactionStrategy(CompactionEngine type) + { + this.type = type; + } + + @JsonProperty + public CompactionEngine getType() + { + return type; + } + + public enum CompactionEngine + { + NATIVE, + MSQ; + + @JsonCreator + public static CompactionEngine fromString(String name) + { + if (name == null) { + return null; + } + return valueOf(StringUtils.toUpperCase(name)); + } + } + + @Override + public String toString() + { + return "UserCompactionStrategy{" + + "type=" + type + + '}'; + } +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index ed2f3eda26a4..36c7c920a7ae 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java @@ -21,6 +21,8 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; @@ -49,6 +51,7 @@ import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; +import org.apache.druid.server.coordinator.UserCompactionStrategy; import org.apache.druid.server.coordinator.compact.CompactionSegmentIterator; import org.apache.druid.server.coordinator.compact.CompactionSegmentSearchPolicy; import org.apache.druid.server.coordinator.compact.CompactionStatistics; @@ -465,12 +468,12 @@ private int submitCompactionTasks( } } - DataSourceCompactionConfig.Engine engine = config.getEngine(); + UserCompactionStrategy.CompactionEngine compactionEngine = config.getEngine(); Map autoCompactionContext = newAutoCompactionContext(config.getTaskContext()); int numCurrentCompactionTasksAndSubtasks; final String maxNumTasksContextParam = "maxNumTasks"; - if (engine == DataSourceCompactionConfig.Engine.MSQ) { + if (compactionEngine == UserCompactionStrategy.CompactionEngine.MSQ) { if (!autoCompactionContext.containsKey(maxNumTasksContextParam)) { // Since MSQ needs all task slots for the calculated #tasks to be available upfront, allot all available // compaction slots to current compaction task to avoid stalling. @@ -498,7 +501,7 @@ private int submitCompactionTasks( transformSpec, dropExisting, autoCompactionContext, - engine + new UserCompactionStrategy(compactionEngine) ); LOG.info( @@ -652,7 +655,7 @@ private String compactSegments( @Nullable ClientCompactionTaskTransformSpec transformSpec, @Nullable Boolean dropExisting, @Nullable Map context, - @Nullable DataSourceCompactionConfig.Engine engine + @Nullable UserCompactionStrategy compactionStrategy ) { Preconditions.checkArgument(!segments.isEmpty(), "Expect non-empty segments to compact"); @@ -681,7 +684,7 @@ private String compactSegments( metricsSpec, transformSpec, context, - engine + compactionStrategy ); FutureUtils.getUnchecked(overlordClient.runTask(taskId, taskPayload), true); return taskId; diff --git a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java index 40f0f8787323..5478a66b467c 100644 --- a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java @@ -31,6 +31,7 @@ import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.UserCompactionStrategy; import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; import org.joda.time.Period; import org.junit.Assert; @@ -191,7 +192,7 @@ public void testAddOrUpdateCompactionConfigWithExistingConfig() null, null, null, - DataSourceCompactionConfig.Engine.NATIVE, + UserCompactionStrategy.CompactionEngine.NATIVE, ImmutableMap.of("key", "val") ); Response result = coordinatorCompactionConfigsResource.addOrUpdateCompactionConfig( @@ -392,7 +393,7 @@ public void testAddOrUpdateCompactionConfigWithoutExistingConfig() null, null, null, - DataSourceCompactionConfig.Engine.MSQ, + UserCompactionStrategy.CompactionEngine.MSQ, ImmutableMap.of("key", "val") ); String author = "maytas"; From c600fa36abf0c4533c5b2499cb23baa91f3bf1e8 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Mon, 29 Apr 2024 13:32:27 +0530 Subject: [PATCH 16/54] Refactorings --- .../apache/druid/msq/exec/ControllerImpl.java | 4 +- .../common/task/AbstractBatchIndexTask.java | 4 +- .../common/task/CompactionStrategy.java | 7 ++++ .../indexing/common/task/CompactionTask.java | 24 +++++------ .../ClientCompactionTaskQuerySerdeTest.java | 3 +- .../druid/indexer/CompactionEngine.java | 14 ++++--- .../druid/timeline/CompactionState.java | 35 +++++++++++++--- .../druid/timeline/DataSegmentTest.java | 18 +++++--- .../CoordinatorCompactionConfig.java | 8 ++-- .../DataSourceCompactionConfig.java | 15 +++---- .../coordinator/UserCompactionStrategy.java | 42 +++++++++---------- .../coordinator/duty/CompactSegments.java | 7 ++-- .../CoordinatorCompactionConfigsResource.java | 2 +- ...rdinatorCompactionConfigsResourceTest.java | 6 +-- 14 files changed, 116 insertions(+), 73 deletions(-) rename indexing-service/src/main/java/org/apache/druid/indexing/common/task/Engine.java => processing/src/main/java/org/apache/druid/indexer/CompactionEngine.java (74%) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 57f7a4ace19d..caf3c01eeece 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -63,6 +63,7 @@ import org.apache.druid.frame.util.DurableStorageUtils; import org.apache.druid.frame.write.InvalidFieldException; import org.apache.druid.frame.write.InvalidNullByteException; +import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; @@ -1893,7 +1894,8 @@ private static Function, Set> addCompactionStateTo metricsSpec, transformSpec, indexSpec.asMap(jsonMapper), - granularitySpec.asMap(jsonMapper) + granularitySpec.asMap(jsonMapper), + CompactionEngine.MSQ ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java index bfe110a33e0c..3253a70cad8d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java @@ -30,6 +30,7 @@ import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.IngestionState; import org.apache.druid.indexer.report.IngestionStatsAndErrors; import org.apache.druid.indexer.report.IngestionStatsAndErrorsTaskReport; @@ -635,7 +636,8 @@ public static Function, Set> addCompactionStateToS metricsSpec, transformSpec, tuningConfig.getIndexSpec().asMap(toolbox.getJsonMapper()), - granularitySpec.asMap(toolbox.getJsonMapper()) + granularitySpec.asMap(toolbox.getJsonMapper()), + CompactionEngine.NATIVE ); } else { return Function.identity(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionStrategy.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionStrategy.java index f3395a27f8e6..a5f4b10f1111 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionStrategy.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionStrategy.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.core.JsonProcessingException; +import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.java.util.common.NonnullPair; @@ -29,6 +30,11 @@ import org.joda.time.Interval; import java.util.List; + +/** + * Strategy to be used for executing a compaction task. + * All subtypes should be synchronized with {@link CompactionEngine}. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = CompactionStrategy.TYPE_PROPERTY) @JsonSubTypes(value = { @JsonSubTypes.Type(name = NativeCompactionStrategy.type, value = NativeCompactionStrategy.class) @@ -36,6 +42,7 @@ public interface CompactionStrategy { String TYPE_PROPERTY = "type"; + TaskStatus runCompactionTasks( CompactionTask compactionTask, TaskToolbox taskToolbox, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 405ac690d2a8..a084f48e8db6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -42,6 +42,7 @@ import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.error.DruidException; import org.apache.druid.indexer.Checks; import org.apache.druid.indexer.Property; import org.apache.druid.indexer.TaskStatus; @@ -490,20 +491,15 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception getMetricBuilder() ); - /*if (engine == Engine.MSQ) { - if (toolbox.getCompactionToMSQ() == null) { - throw DruidException.forPersona(DruidException.Persona.ADMIN) - .ofCategory(DruidException.Category.NOT_FOUND) - .build( - "Extension[druid-multi-stage-query] required for running compaction on MSQ " - + "not found on the Indexer"); - } - registerResourceCloserOnAbnormalExit(currentSubTaskHolder); - return toolbox.getCompactionToMSQ() - .runCompactionTasks(this, toolbox, intervalDataSchemas); - } else { - - }*/ + if (compactionStrategy == null) { + // Can only happen for MSQ engine, when the json subtype reqd for deserialization isn't available due to + // missing extn. + throw DruidException.forPersona(DruidException.Persona.ADMIN) + .ofCategory(DruidException.Category.NOT_FOUND) + .build( + "Extension[druid-multi-stage-query] required for running compaction on MSQ " + + "not found on the Indexer."); + } registerResourceCloserOnAbnormalExit(currentSubTaskHolder); return compactionStrategy.runCompactionTasks(this, toolbox, intervalDataSchemas); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java index 3e3adbecd4f5..e3d0db916dc0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java @@ -40,6 +40,7 @@ import org.apache.druid.guice.GuiceAnnotationIntrospector; import org.apache.druid.guice.GuiceInjectableValues; import org.apache.druid.guice.GuiceInjectors; +import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexing.common.RetryPolicyConfig; import org.apache.druid.indexing.common.RetryPolicyFactory; @@ -129,7 +130,7 @@ public void testClientCompactionTaskQueryToCompactionTask() throws IOException new AggregatorFactory[] {new CountAggregatorFactory("cnt")}, new ClientCompactionTaskTransformSpec(new SelectorDimFilter("dim1", "foo", null)), ImmutableMap.of("key", "value"), - new UserCompactionStrategy(UserCompactionStrategy.CompactionEngine.NATIVE) + new UserCompactionStrategy(CompactionEngine.NATIVE) ); final byte[] json = mapper.writeValueAsBytes(query); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Engine.java b/processing/src/main/java/org/apache/druid/indexer/CompactionEngine.java similarity index 74% rename from indexing-service/src/main/java/org/apache/druid/indexing/common/task/Engine.java rename to processing/src/main/java/org/apache/druid/indexer/CompactionEngine.java index 79036e2765cc..5cc39bf4bf0b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Engine.java +++ b/processing/src/main/java/org/apache/druid/indexer/CompactionEngine.java @@ -17,24 +17,26 @@ * under the License. */ -package org.apache.druid.indexing.common.task; +package org.apache.druid.indexer; import com.fasterxml.jackson.annotation.JsonCreator; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.server.coordinator.UserCompactionStrategy; /** - * Engine to be used for a compaction task. - * Should be synchronized with {@link UserCompactionStrategy.CompactionEngine}. + * Encapsulates the Engine to be used for a compaction task. + * Should be synchronized with the subtypes for {@link org.apache.druid.indexing.common.task.CompactionStrategy}. */ -public enum Engine +public enum CompactionEngine { NATIVE, MSQ; @JsonCreator - public static Engine fromString(String name) + public static CompactionEngine fromString(String name) { + if (name == null) { + return null; + } return valueOf(StringUtils.toUpperCase(name)); } } diff --git a/processing/src/main/java/org/apache/druid/timeline/CompactionState.java b/processing/src/main/java/org/apache/druid/timeline/CompactionState.java index 2c6e0d96c397..a2eadfdedad4 100644 --- a/processing/src/main/java/org/apache/druid/timeline/CompactionState.java +++ b/processing/src/main/java/org/apache/druid/timeline/CompactionState.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.partitions.PartitionsSpec; import java.util.List; @@ -44,6 +45,8 @@ */ public class CompactionState { + private static final CompactionEngine DEFAULT_COMPACTION_ENGINE = CompactionEngine.NATIVE; + private final PartitionsSpec partitionsSpec; private final DimensionsSpec dimensionsSpec; // org.apache.druid.segment.transform.TransformSpec cannot be used here because it's in the 'processing' module which @@ -59,6 +62,8 @@ public class CompactionState // has a dependency on the 'core' module where this class is. private final List metricsSpec; + private final CompactionEngine compactionEngine; + @JsonCreator public CompactionState( @JsonProperty("partitionsSpec") PartitionsSpec partitionsSpec, @@ -66,7 +71,8 @@ public CompactionState( @JsonProperty("metricsSpec") List metricsSpec, @JsonProperty("transformSpec") Map transformSpec, @JsonProperty("indexSpec") Map indexSpec, - @JsonProperty("granularitySpec") Map granularitySpec + @JsonProperty("granularitySpec") Map granularitySpec, + @JsonProperty("compactionEngine") CompactionEngine compactionEngine ) { this.partitionsSpec = partitionsSpec; @@ -75,6 +81,7 @@ public CompactionState( this.transformSpec = transformSpec; this.indexSpec = indexSpec; this.granularitySpec = granularitySpec; + this.compactionEngine = compactionEngine == null ? DEFAULT_COMPACTION_ENGINE : compactionEngine; } @JsonProperty @@ -113,6 +120,12 @@ public Map getGranularitySpec() return granularitySpec; } + @JsonProperty + public CompactionEngine getCompactionEngine() + { + return compactionEngine; + } + @Override public boolean equals(Object o) { @@ -128,13 +141,22 @@ public boolean equals(Object o) Objects.equals(transformSpec, that.transformSpec) && Objects.equals(indexSpec, that.indexSpec) && Objects.equals(granularitySpec, that.granularitySpec) && - Objects.equals(metricsSpec, that.metricsSpec); + Objects.equals(metricsSpec, that.metricsSpec) && + Objects.equals(compactionEngine, that.compactionEngine); } @Override public int hashCode() { - return Objects.hash(partitionsSpec, dimensionsSpec, transformSpec, indexSpec, granularitySpec, metricsSpec); + return Objects.hash( + partitionsSpec, + dimensionsSpec, + transformSpec, + indexSpec, + granularitySpec, + metricsSpec, + compactionEngine + ); } @Override @@ -147,6 +169,7 @@ public String toString() ", indexSpec=" + indexSpec + ", granularitySpec=" + granularitySpec + ", metricsSpec=" + metricsSpec + + ", compactionEngine=" + compactionEngine + '}'; } @@ -156,7 +179,8 @@ public static Function, Set> addCompactionStateToS List metricsSpec, Map transformSpec, Map indexSpec, - Map granularitySpec + Map granularitySpec, + CompactionEngine compactionEngine ) { CompactionState compactionState = new CompactionState( @@ -165,7 +189,8 @@ public static Function, Set> addCompactionStateToS metricsSpec, transformSpec, indexSpec, - granularitySpec + granularitySpec, + compactionEngine ); return segments -> segments diff --git a/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java b/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java index 3f0667b870c9..fe5745ecea52 100644 --- a/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java +++ b/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java @@ -26,6 +26,7 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.RangeSet; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.jackson.DefaultObjectMapper; @@ -132,7 +133,8 @@ public void testV1Serialization() throws Exception ImmutableList.of(ImmutableMap.of("type", "count", "name", "count")), ImmutableMap.of("filter", ImmutableMap.of("type", "selector", "dimension", "dim1", "value", "foo")), ImmutableMap.of(), - ImmutableMap.of() + ImmutableMap.of(), + CompactionEngine.NATIVE ), TEST_VERSION, 1 @@ -153,7 +155,7 @@ public void testV1Serialization() throws Exception Assert.assertEquals(ImmutableMap.of("type", "numbered", "partitionNum", 3, "partitions", 0), objectMap.get("shardSpec")); Assert.assertEquals(TEST_VERSION, objectMap.get("binaryVersion")); Assert.assertEquals(1, objectMap.get("size")); - Assert.assertEquals(6, ((Map) objectMap.get("lastCompactionState")).size()); + Assert.assertEquals(7, ((Map) objectMap.get("lastCompactionState")).size()); DataSegment deserializedSegment = MAPPER.readValue(MAPPER.writeValueAsString(segment), DataSegment.class); @@ -197,7 +199,8 @@ public void testDeserializationDataSegmentLastCompactionStateWithNullSpecs() thr null, null, ImmutableMap.of(), - ImmutableMap.of() + ImmutableMap.of(), + CompactionEngine.NATIVE ), TEST_VERSION, 1 @@ -348,7 +351,8 @@ public void testWithLastCompactionState() ImmutableList.of(ImmutableMap.of("type", "count", "name", "count")), ImmutableMap.of("filter", ImmutableMap.of("type", "selector", "dimension", "dim1", "value", "foo")), Collections.singletonMap("test", "map"), - Collections.singletonMap("test2", "map2") + Collections.singletonMap("test2", "map2"), + CompactionEngine.NATIVE ); final DataSegment segment1 = DataSegment.builder() .dataSource("foo") @@ -390,7 +394,8 @@ public void testAnnotateWithLastCompactionState() metricsSpec, transformSpec, indexSpec, - granularitySpec + granularitySpec, + CompactionEngine.NATIVE ); final Function, Set> addCompactionStateFunction = @@ -400,7 +405,8 @@ public void testAnnotateWithLastCompactionState() metricsSpec, transformSpec, indexSpec, - granularitySpec + granularitySpec, + CompactionEngine.NATIVE ); final DataSegment segment1 = DataSegment.builder() diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfig.java index d4c679033539..8c9214ca9264 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfig.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableList; +import org.apache.druid.indexer.CompactionEngine; import javax.annotation.Nullable; import java.util.List; @@ -34,12 +35,13 @@ public class CoordinatorCompactionConfig private static final double DEFAULT_COMPACTION_TASK_RATIO = 0.1; private static final int DEFAULT_MAX_COMPACTION_TASK_SLOTS = Integer.MAX_VALUE; private static final boolean DEFAULT_USE_AUTO_SCALE_SLOTS = false; - private static final UserCompactionStrategy.CompactionEngine DEFAULT_COMPACTION_ENGINE = UserCompactionStrategy.CompactionEngine.NATIVE; + private static final CompactionEngine DEFAULT_COMPACTION_ENGINE = CompactionEngine.NATIVE; + private final List compactionConfigs; private final double compactionTaskSlotRatio; private final int maxCompactionTaskSlots; private final boolean useAutoScaleSlots; - private final UserCompactionStrategy.CompactionEngine compactionEngine; + private final CompactionEngine compactionEngine; public static CoordinatorCompactionConfig from( CoordinatorCompactionConfig baseConfig, @@ -123,7 +125,7 @@ public boolean isUseAutoScaleSlots() } @JsonProperty - public UserCompactionStrategy.CompactionEngine getEngine() + public CompactionEngine getEngine() { return compactionEngine; } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java index e3b90e2160a9..6b13f4b236df 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import org.apache.druid.error.InvalidInput; +import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; @@ -59,7 +60,7 @@ public class DataSourceCompactionConfig private final UserCompactionTaskTransformConfig transformSpec; private final UserCompactionTaskIOConfig ioConfig; private final Map taskContext; - private final UserCompactionStrategy.CompactionEngine compactionEngine; + private final CompactionEngine compactionEngine; @JsonCreator public DataSourceCompactionConfig( @@ -74,7 +75,7 @@ public DataSourceCompactionConfig( @JsonProperty("metricsSpec") @Nullable AggregatorFactory[] metricsSpec, @JsonProperty("transformSpec") @Nullable UserCompactionTaskTransformConfig transformSpec, @JsonProperty("ioConfig") @Nullable UserCompactionTaskIOConfig ioConfig, - @JsonProperty("engine") @Nullable UserCompactionStrategy.CompactionEngine compactionEngine, + @JsonProperty("engine") @Nullable CompactionEngine compactionEngine, @JsonProperty("taskContext") @Nullable Map taskContext ) { @@ -180,7 +181,7 @@ public Map getTaskContext() @JsonProperty @Nullable - public UserCompactionStrategy.CompactionEngine getEngine() + public CompactionEngine getEngine() { return compactionEngine; } @@ -232,17 +233,17 @@ public int hashCode() } public static DataSourceCompactionConfig from( - UserCompactionStrategy.CompactionEngine defaultCompactionEngine, - DataSourceCompactionConfig newConfig + DataSourceCompactionConfig newConfig, + CompactionEngine defaultCompactionEngine ) { - UserCompactionStrategy.CompactionEngine newCompactionEngine = newConfig.getEngine(); + CompactionEngine newCompactionEngine = newConfig.getEngine(); String engineSourceLog = "specified in spec"; if (newCompactionEngine == null) { newCompactionEngine = defaultCompactionEngine; engineSourceLog = "set as default"; } - if (newCompactionEngine == UserCompactionStrategy.CompactionEngine.MSQ) { + if (newCompactionEngine == CompactionEngine.MSQ) { if (newConfig.getTuningConfig() != null) { PartitionsSpec partitionsSpec = newConfig.getTuningConfig().getPartitionsSpec(); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionStrategy.java b/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionStrategy.java index b3b7032c9e27..94fa42b72ce2 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionStrategy.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionStrategy.java @@ -1,13 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.druid.server.coordinator; -import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.indexer.CompactionEngine; + -/** - * Encapsulates the Engine to be used for a compaction task. - * Should be synchronized with the types for {@link org.apache.druid.indexing.common.task.CompactionTasksLauncher}. - */ public class UserCompactionStrategy { private final CompactionEngine type; @@ -23,21 +38,6 @@ public CompactionEngine getType() return type; } - public enum CompactionEngine - { - NATIVE, - MSQ; - - @JsonCreator - public static CompactionEngine fromString(String name) - { - if (name == null) { - return null; - } - return valueOf(StringUtils.toUpperCase(name)); - } - } - @Override public String toString() { diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index 36c7c920a7ae..0248b94a7b02 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java @@ -21,8 +21,6 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; @@ -38,6 +36,7 @@ import org.apache.druid.client.indexing.TaskPayloadResponse; import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.common.utils.IdUtils; +import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.java.util.common.ISE; @@ -468,12 +467,12 @@ private int submitCompactionTasks( } } - UserCompactionStrategy.CompactionEngine compactionEngine = config.getEngine(); + CompactionEngine compactionEngine = config.getEngine(); Map autoCompactionContext = newAutoCompactionContext(config.getTaskContext()); int numCurrentCompactionTasksAndSubtasks; final String maxNumTasksContextParam = "maxNumTasks"; - if (compactionEngine == UserCompactionStrategy.CompactionEngine.MSQ) { + if (compactionEngine == CompactionEngine.MSQ) { if (!autoCompactionContext.containsKey(maxNumTasksContextParam)) { // Since MSQ needs all task slots for the calculated #tasks to be available upfront, allot all available // compaction slots to current compaction task to avoid stalling. diff --git a/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java index f6cb1ae84ccd..e29aee26f2b4 100644 --- a/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java +++ b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java @@ -119,7 +119,7 @@ public Response addOrUpdateCompactionConfig( .getCompactionConfigs() .stream() .collect(Collectors.toMap(DataSourceCompactionConfig::getDataSource, Function.identity())); - DataSourceCompactionConfig updatedConfig = DataSourceCompactionConfig.from(current.getEngine(), newConfig); + DataSourceCompactionConfig updatedConfig = DataSourceCompactionConfig.from(newConfig, current.getEngine()); newConfigs.put(updatedConfig.getDataSource(), updatedConfig); newCompactionConfig = CoordinatorCompactionConfig.from(current, ImmutableList.copyOf(newConfigs.values())); diff --git a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java index 5478a66b467c..3d7063e7b958 100644 --- a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java @@ -24,6 +24,7 @@ import org.apache.druid.audit.AuditManager; import org.apache.druid.common.config.ConfigManager; import org.apache.druid.common.config.JacksonConfigManager; +import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.metadata.MetadataStorageConnector; @@ -31,7 +32,6 @@ import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; -import org.apache.druid.server.coordinator.UserCompactionStrategy; import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; import org.joda.time.Period; import org.junit.Assert; @@ -192,7 +192,7 @@ public void testAddOrUpdateCompactionConfigWithExistingConfig() null, null, null, - UserCompactionStrategy.CompactionEngine.NATIVE, + CompactionEngine.NATIVE, ImmutableMap.of("key", "val") ); Response result = coordinatorCompactionConfigsResource.addOrUpdateCompactionConfig( @@ -393,7 +393,7 @@ public void testAddOrUpdateCompactionConfigWithoutExistingConfig() null, null, null, - UserCompactionStrategy.CompactionEngine.MSQ, + CompactionEngine.MSQ, ImmutableMap.of("key", "val") ); String author = "maytas"; From 2f0203e2e35b92cbdbfdf278cae5a0ad76405484 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Mon, 29 Apr 2024 13:42:27 +0530 Subject: [PATCH 17/54] Revert compaction engine field addition in CompactionState for now --- .../apache/druid/msq/exec/ControllerImpl.java | 4 +-- .../common/task/AbstractBatchIndexTask.java | 4 +-- .../druid/timeline/CompactionState.java | 31 ++++--------------- .../druid/timeline/DataSegmentTest.java | 16 +++------- 4 files changed, 13 insertions(+), 42 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index caf3c01eeece..57f7a4ace19d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -63,7 +63,6 @@ import org.apache.druid.frame.util.DurableStorageUtils; import org.apache.druid.frame.write.InvalidFieldException; import org.apache.druid.frame.write.InvalidNullByteException; -import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; @@ -1894,8 +1893,7 @@ private static Function, Set> addCompactionStateTo metricsSpec, transformSpec, indexSpec.asMap(jsonMapper), - granularitySpec.asMap(jsonMapper), - CompactionEngine.MSQ + granularitySpec.asMap(jsonMapper) ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java index 3253a70cad8d..bfe110a33e0c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java @@ -30,7 +30,6 @@ import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.IngestionState; import org.apache.druid.indexer.report.IngestionStatsAndErrors; import org.apache.druid.indexer.report.IngestionStatsAndErrorsTaskReport; @@ -636,8 +635,7 @@ public static Function, Set> addCompactionStateToS metricsSpec, transformSpec, tuningConfig.getIndexSpec().asMap(toolbox.getJsonMapper()), - granularitySpec.asMap(toolbox.getJsonMapper()), - CompactionEngine.NATIVE + granularitySpec.asMap(toolbox.getJsonMapper()) ); } else { return Function.identity(); diff --git a/processing/src/main/java/org/apache/druid/timeline/CompactionState.java b/processing/src/main/java/org/apache/druid/timeline/CompactionState.java index a2eadfdedad4..24344d9cd822 100644 --- a/processing/src/main/java/org/apache/druid/timeline/CompactionState.java +++ b/processing/src/main/java/org/apache/druid/timeline/CompactionState.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.partitions.PartitionsSpec; import java.util.List; @@ -37,7 +36,7 @@ * The compaction task is a task that reads Druid segments and overwrites them with new ones. Since this task always * reads segments in the same order, the same task spec will always create the same set of segments * (not same segment ID, but same content). - * + *

* Note that this class doesn't include all fields in the compaction task spec. Only the configurations that can * affect the content of segment should be included. * @@ -45,8 +44,6 @@ */ public class CompactionState { - private static final CompactionEngine DEFAULT_COMPACTION_ENGINE = CompactionEngine.NATIVE; - private final PartitionsSpec partitionsSpec; private final DimensionsSpec dimensionsSpec; // org.apache.druid.segment.transform.TransformSpec cannot be used here because it's in the 'processing' module which @@ -62,8 +59,6 @@ public class CompactionState // has a dependency on the 'core' module where this class is. private final List metricsSpec; - private final CompactionEngine compactionEngine; - @JsonCreator public CompactionState( @JsonProperty("partitionsSpec") PartitionsSpec partitionsSpec, @@ -71,8 +66,7 @@ public CompactionState( @JsonProperty("metricsSpec") List metricsSpec, @JsonProperty("transformSpec") Map transformSpec, @JsonProperty("indexSpec") Map indexSpec, - @JsonProperty("granularitySpec") Map granularitySpec, - @JsonProperty("compactionEngine") CompactionEngine compactionEngine + @JsonProperty("granularitySpec") Map granularitySpec ) { this.partitionsSpec = partitionsSpec; @@ -81,7 +75,6 @@ public CompactionState( this.transformSpec = transformSpec; this.indexSpec = indexSpec; this.granularitySpec = granularitySpec; - this.compactionEngine = compactionEngine == null ? DEFAULT_COMPACTION_ENGINE : compactionEngine; } @JsonProperty @@ -120,12 +113,6 @@ public Map getGranularitySpec() return granularitySpec; } - @JsonProperty - public CompactionEngine getCompactionEngine() - { - return compactionEngine; - } - @Override public boolean equals(Object o) { @@ -141,8 +128,7 @@ public boolean equals(Object o) Objects.equals(transformSpec, that.transformSpec) && Objects.equals(indexSpec, that.indexSpec) && Objects.equals(granularitySpec, that.granularitySpec) && - Objects.equals(metricsSpec, that.metricsSpec) && - Objects.equals(compactionEngine, that.compactionEngine); + Objects.equals(metricsSpec, that.metricsSpec); } @Override @@ -154,8 +140,7 @@ public int hashCode() transformSpec, indexSpec, granularitySpec, - metricsSpec, - compactionEngine + metricsSpec ); } @@ -169,7 +154,6 @@ public String toString() ", indexSpec=" + indexSpec + ", granularitySpec=" + granularitySpec + ", metricsSpec=" + metricsSpec + - ", compactionEngine=" + compactionEngine + '}'; } @@ -179,8 +163,7 @@ public static Function, Set> addCompactionStateToS List metricsSpec, Map transformSpec, Map indexSpec, - Map granularitySpec, - CompactionEngine compactionEngine + Map granularitySpec ) { CompactionState compactionState = new CompactionState( @@ -189,8 +172,7 @@ public static Function, Set> addCompactionStateToS metricsSpec, transformSpec, indexSpec, - granularitySpec, - compactionEngine + granularitySpec ); return segments -> segments @@ -198,5 +180,4 @@ public static Function, Set> addCompactionStateToS .map(s -> s.withLastCompactionState(compactionState)) .collect(Collectors.toSet()); } - } diff --git a/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java b/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java index fe5745ecea52..370a16591270 100644 --- a/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java +++ b/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java @@ -26,7 +26,6 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.RangeSet; import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.jackson.DefaultObjectMapper; @@ -133,8 +132,7 @@ public void testV1Serialization() throws Exception ImmutableList.of(ImmutableMap.of("type", "count", "name", "count")), ImmutableMap.of("filter", ImmutableMap.of("type", "selector", "dimension", "dim1", "value", "foo")), ImmutableMap.of(), - ImmutableMap.of(), - CompactionEngine.NATIVE + ImmutableMap.of() ), TEST_VERSION, 1 @@ -199,8 +197,7 @@ public void testDeserializationDataSegmentLastCompactionStateWithNullSpecs() thr null, null, ImmutableMap.of(), - ImmutableMap.of(), - CompactionEngine.NATIVE + ImmutableMap.of() ), TEST_VERSION, 1 @@ -351,8 +348,7 @@ public void testWithLastCompactionState() ImmutableList.of(ImmutableMap.of("type", "count", "name", "count")), ImmutableMap.of("filter", ImmutableMap.of("type", "selector", "dimension", "dim1", "value", "foo")), Collections.singletonMap("test", "map"), - Collections.singletonMap("test2", "map2"), - CompactionEngine.NATIVE + Collections.singletonMap("test2", "map2") ); final DataSegment segment1 = DataSegment.builder() .dataSource("foo") @@ -394,8 +390,7 @@ public void testAnnotateWithLastCompactionState() metricsSpec, transformSpec, indexSpec, - granularitySpec, - CompactionEngine.NATIVE + granularitySpec ); final Function, Set> addCompactionStateFunction = @@ -405,8 +400,7 @@ public void testAnnotateWithLastCompactionState() metricsSpec, transformSpec, indexSpec, - granularitySpec, - CompactionEngine.NATIVE + granularitySpec ); final DataSegment segment1 = DataSegment.builder() From a33e55d33e0ef9697fd1e5227568f73885aa437c Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Mon, 29 Apr 2024 16:09:15 +0530 Subject: [PATCH 18/54] Update serde test. --- .../msq/indexing/MSQCompactionStrategy.java | 4 ++- .../common/task/CompactionStrategy.java | 8 +++++- .../indexing/common/task/CompactionTask.java | 6 ++--- .../common/task/NativeCompactionStrategy.java | 2 ++ .../ClientCompactionTaskQuerySerdeTest.java | 3 ++- .../coordinator/UserCompactionStrategy.java | 25 ++++++++++++++++++- 6 files changed, 40 insertions(+), 8 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionStrategy.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionStrategy.java index 5e4360b9de71..60d399106e81 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionStrategy.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionStrategy.java @@ -20,6 +20,7 @@ package org.apache.druid.msq.indexing; import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.json.JsonMapper; @@ -32,8 +33,8 @@ import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexer.partitions.SecondaryPartitionType; import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.task.CompactionTask; import org.apache.druid.indexing.common.task.CompactionStrategy; +import org.apache.druid.indexing.common.task.CompactionTask; import org.apache.druid.indexing.common.task.CurrentSubTaskHolder; import org.apache.druid.java.util.common.NonnullPair; import org.apache.druid.java.util.common.StringUtils; @@ -96,6 +97,7 @@ public MSQCompactionStrategy(@JacksonInject OverlordClient overlordClient, @Jack } @Override + @JsonProperty public String getType() { return type; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionStrategy.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionStrategy.java index a5f4b10f1111..1c9a52eb0a81 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionStrategy.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionStrategy.java @@ -19,6 +19,7 @@ package org.apache.druid.indexing.common.task; +import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.core.JsonProcessingException; @@ -35,7 +36,11 @@ * Strategy to be used for executing a compaction task. * All subtypes should be synchronized with {@link CompactionEngine}. */ -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = CompactionStrategy.TYPE_PROPERTY) +@JsonTypeInfo( + use = JsonTypeInfo.Id.NAME, + include = JsonTypeInfo.As.EXISTING_PROPERTY, + property = CompactionStrategy.TYPE_PROPERTY, + visible = true) @JsonSubTypes(value = { @JsonSubTypes.Type(name = NativeCompactionStrategy.type, value = NativeCompactionStrategy.class) }) @@ -49,5 +54,6 @@ TaskStatus runCompactionTasks( List> dataSchemas ) throws JsonProcessingException; + @JsonProperty String getType(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index a084f48e8db6..ec851c1869ac 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -1178,14 +1178,12 @@ public Builder context(Map context) return this; } - public Builder engine(CompactionStrategy engine) + public Builder compactionStrategy(CompactionStrategy compactionStrategy) { - this.compactionStrategy = engine; + this.compactionStrategy = compactionStrategy; return this; } - - public CompactionTask build() { return new CompactionTask( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionStrategy.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionStrategy.java index 1f71fa3d255f..84bd012b076c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionStrategy.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionStrategy.java @@ -1,5 +1,6 @@ package org.apache.druid.indexing.common.task; +import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; import com.google.common.annotations.VisibleForTesting; import org.apache.druid.client.coordinator.CoordinatorClient; @@ -255,6 +256,7 @@ Map createContextForSubtask(CompactionTask compactionTask) } @Override + @JsonProperty public String getType() { return type; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java index e3d0db916dc0..a4506ffaa6b6 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java @@ -306,6 +306,7 @@ public void testCompactionTaskToClientCompactionTaskQuery() throws IOException ) .metricsSpec(new AggregatorFactory[] {new CountAggregatorFactory("cnt")}) .transformSpec(new ClientCompactionTaskTransformSpec(new SelectorDimFilter("dim1", "foo", null))) + .compactionStrategy(new NativeCompactionStrategy()) .build(); final ClientCompactionTaskQuery expected = new ClientCompactionTaskQuery( @@ -353,7 +354,7 @@ public void testCompactionTaskToClientCompactionTaskQuery() throws IOException new AggregatorFactory[] {new CountAggregatorFactory("cnt")}, new ClientCompactionTaskTransformSpec(new SelectorDimFilter("dim1", "foo", null)), new HashMap<>(), - null + new UserCompactionStrategy(CompactionEngine.NATIVE) ); final byte[] json = mapper.writeValueAsBytes(task); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionStrategy.java b/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionStrategy.java index 94fa42b72ce2..fbc90db95f41 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionStrategy.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionStrategy.java @@ -19,15 +19,19 @@ package org.apache.druid.server.coordinator; +import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.indexer.CompactionEngine; +import java.util.Objects; + public class UserCompactionStrategy { private final CompactionEngine type; - public UserCompactionStrategy(CompactionEngine type) + @JsonCreator + public UserCompactionStrategy(@JsonProperty("type") CompactionEngine type) { this.type = type; } @@ -45,4 +49,23 @@ public String toString() "type=" + type + '}'; } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + UserCompactionStrategy that = (UserCompactionStrategy) o; + return type == that.type; + } + + @Override + public int hashCode() + { + return Objects.hash(type); + } } From f3a86c1013fe0173dbe8395be223b750473d5333 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Thu, 2 May 2024 15:59:02 +0530 Subject: [PATCH 19/54] Remove serde step for MSQControllerTask, update tests, add context checks. --- .../druid/msq/guice/MSQIndexingModule.java | 2 +- .../msq/indexing/MSQCompactionStrategy.java | 60 +++++++++++-------- .../druid/msq/indexing/MSQControllerTask.java | 16 +++++ .../apache/druid/msq/exec/MSQReplaceTest.java | 56 +++++++++++------ .../common/task/CompactionStrategy.java | 4 +- .../indexing/common/task/CompactionTask.java | 4 +- .../common/task/NativeCompactionStrategy.java | 28 +++++++-- .../overlord/http/OverlordResource.java | 1 - .../coordinator/duty/CompactSegments.java | 3 +- 9 files changed, 118 insertions(+), 56 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java index 354a7f461a20..137e08046ce1 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java @@ -199,7 +199,7 @@ public List getJacksonModules() NilInputSource.class ); - module.registerSubtypes(new NamedType(MSQCompactionStrategy.class, MSQCompactionStrategy.type)); + module.registerSubtypes(new NamedType(MSQCompactionStrategy.class, MSQCompactionStrategy.TYPE)); FAULT_CLASSES.forEach(module::registerSubtypes); module.addSerializer(new CounterSnapshotsSerializer()); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionStrategy.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionStrategy.java index 60d399106e81..4a9ef55e0794 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionStrategy.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionStrategy.java @@ -23,8 +23,8 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.json.JsonMapper; import com.google.common.collect.ImmutableList; +import com.google.inject.Injector; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.error.InvalidInput; import org.apache.druid.indexer.TaskStatus; @@ -45,6 +45,7 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination; +import org.apache.druid.msq.kernel.WorkerAssignmentStrategy; import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.Druids; import org.apache.druid.query.Query; @@ -57,7 +58,6 @@ import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; -import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnHolder; @@ -81,46 +81,50 @@ public class MSQCompactionStrategy implements CompactionStrategy { private static final Logger log = new Logger(MSQCompactionStrategy.class); - public static final String type = "MSQ"; + public static final String TYPE = "MSQ"; private static final Granularity DEFAULT_SEGMENT_GRANULARITY = Granularities.ALL; - OverlordClient overlordClient; - ObjectMapper jsonMapper; + private final ObjectMapper jsonMapper; + private final Injector injector; private static final String TIME_VIRTUAL_COLUMN = "vTime"; private static final String TIME_COLUMN = ColumnHolder.TIME_COLUMN_NAME; - public MSQCompactionStrategy(@JacksonInject OverlordClient overlordClient, @JacksonInject JsonMapper jsonMapper) + public MSQCompactionStrategy(@JacksonInject ObjectMapper jsonMapper, @JacksonInject Injector injector) { - this.overlordClient = overlordClient; this.jsonMapper = jsonMapper; + this.injector = injector; } @Override @JsonProperty public String getType() { - return type; + return TYPE; } - public void setJsonMapper(ObjectMapper jsonMapper) - { - this.jsonMapper = jsonMapper; - } - - @Override - public TaskStatus runCompactionTasks( + public List compactionToMSQTasks( CompactionTask compactionTask, - TaskToolbox taskToolbox, List> intervalDataSchemas ) throws JsonProcessingException { List msqControllerTasks = new ArrayList<>(); QueryContext compactionTaskContext = new QueryContext(compactionTask.getContext()); + // These checks cannot be included in compaction config validation as context param keys are unknown outside MSQ. if (!MultiStageQueryContext.isFinalizeAggregations(compactionTaskContext)) { throw InvalidInput.exception( - "finalizeAggregations=false currently not supported for auto-compaction with MSQ engine."); + "Config[%s] cannot be set to false for auto-compaction with MSQ engine.", + MultiStageQueryContext.CTX_FINALIZE_AGGREGATIONS, + MultiStageQueryContext.isFinalizeAggregations(compactionTaskContext) + ); + } + if (MultiStageQueryContext.getAssignmentStrategy(compactionTaskContext) == WorkerAssignmentStrategy.AUTO) { + throw InvalidInput.exception( + "Config[%s] cannot be set to value[%s] for auto-compaction with MSQ engine.", + MultiStageQueryContext.CTX_TASK_ASSIGNMENT_STRATEGY, + WorkerAssignmentStrategy.AUTO + ); } for (NonnullPair intervalDataSchema : intervalDataSchemas) { @@ -152,15 +156,21 @@ public TaskStatus runCompactionTasks( null, null, null, - msqControllerTaskContext + msqControllerTaskContext, + injector ); - - // Doing a serde roundtrip for MSQControllerTask as the "injector" field of this class is supposed to be injected - // by the mapper. - MSQControllerTask serdedMSQControllerTask = jsonMapper.readerFor(MSQControllerTask.class) - .readValue(jsonMapper.writeValueAsString(controllerTask)); - msqControllerTasks.add(serdedMSQControllerTask); + msqControllerTasks.add(controllerTask); } + return msqControllerTasks; + } + @Override + public TaskStatus runCompactionTasks( + CompactionTask compactionTask, + TaskToolbox taskToolbox, + List> intervalDataSchemas + ) throws JsonProcessingException + { + List msqControllerTasks = compactionToMSQTasks(compactionTask, intervalDataSchemas); if (msqControllerTasks.isEmpty()) { log.warn( @@ -210,6 +220,8 @@ private static MSQTuningConfig buildMSQTuningConfig(CompactionTask compactionTas } // This parameter is used internally for the number of worker tasks only, so we subtract 1 final int maxNumWorkers = maxNumTasks - 1; + // We don't consider maxRowsInMemory coming via CompactionTuningConfig since it always sets a default value if no + // value specified by user. final int maxRowsInMemory = MultiStageQueryContext.getRowsInMemory(compactionTaskContext); Integer rowsPerSegment = getRowsPerSegment(compactionTask); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java index 9cc9e4dae745..3e6db40ff293 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java @@ -141,6 +141,22 @@ public MSQControllerTask( addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, true); } + public MSQControllerTask( + @Nullable String id, + MSQSpec querySpec, + @Nullable String sqlQuery, + @Nullable Map sqlQueryContext, + @Nullable SqlResults.Context sqlResultsContext, + @Nullable List sqlTypeNames, + @Nullable List nativeTypeNames, + @Nullable Map context, + Injector injector + ) + { + this(id, querySpec, sqlQuery, sqlQueryContext, sqlResultsContext, sqlTypeNames, nativeTypeNames, context); + this.injector = injector; + } + @Override public String getType() { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java index 9a4fb98666b3..86ef7fc8c722 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java @@ -179,7 +179,8 @@ public void testReplaceOnFooWithAll(String contextName, Map cont context, Collections.emptyList(), Collections.singletonList(new FloatDimensionSchema("m1")), - GranularityType.DAY + GranularityType.DAY, + Intervals.ONLY_ETERNITY ) ) .verifyResults(); @@ -237,7 +238,8 @@ public void testReplaceOnFooWithWhere(String contextName, Map co context, Collections.emptyList(), Collections.singletonList(new FloatDimensionSchema("m1")), - GranularityType.DAY + GranularityType.DAY, + ImmutableList.of(Intervals.of("2000-01-02T/P1D")) ) ) .verifyResults(); @@ -327,7 +329,8 @@ public void testReplaceOnFoo1WithAllExtern(String contextName, Map c context, Collections.emptyList(), Collections.singletonList(new FloatDimensionSchema("m1")), - GranularityType.MONTH + GranularityType.MONTH, + ImmutableList.of(Intervals.of("2000-01-01T/2000-03-01T")) ) ) .verifyResults(); @@ -742,7 +749,8 @@ public void testReplaceWhereClauseLargerThanData(String contextName, Map contex context, Collections.emptyList(), Collections.singletonList(new FloatDimensionSchema("m1")), - GranularityType.MONTH + GranularityType.MONTH, + Collections.singletonList(Intervals.of("2000-01-01T/2000-03-01T")) ) ) .verifyResults(); @@ -895,7 +904,8 @@ public void testReplaceTimeChunksLargerThanData(String contextName, Map conte new StringDimensionSchema("dim1"), new LongDimensionSchema("cnt") ), - GranularityType.DAY + GranularityType.DAY, + Intervals.ONLY_ETERNITY ) ) .verifyResults(); @@ -1076,7 +1088,8 @@ public void testReplaceSegmentsInsertIntoNewTable(String contextName, Map context, List partitionDimensions, List dimensions, - GranularityType segmentGranularity + GranularityType segmentGranularity, + List intervals ) { if (!context.containsKey(Tasks.STORE_COMPACTION_STATE_KEY) @@ -1892,7 +1910,7 @@ private CompactionState expectedCompactionState( segmentGranularity.getDefaultGranularity(), GranularityType.NONE.getDefaultGranularity(), false, - Intervals.ONLY_ETERNITY + intervals ); List metricsSpec = Collections.emptyList(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionStrategy.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionStrategy.java index 1c9a52eb0a81..20fded29337e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionStrategy.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionStrategy.java @@ -42,11 +42,11 @@ property = CompactionStrategy.TYPE_PROPERTY, visible = true) @JsonSubTypes(value = { - @JsonSubTypes.Type(name = NativeCompactionStrategy.type, value = NativeCompactionStrategy.class) + @JsonSubTypes.Type(name = NativeCompactionStrategy.TYPE, value = NativeCompactionStrategy.class) }) public interface CompactionStrategy { - String TYPE_PROPERTY = "type"; + String TYPE_PROPERTY = "TYPE"; TaskStatus runCompactionTasks( CompactionTask compactionTask, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index ec851c1869ac..2d0f6a2ec5c0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -157,12 +157,12 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg private final ClientCompactionTaskGranularitySpec granularitySpec; @Nullable private final CompactionTuningConfig tuningConfig; + @Nullable + private final CompactionStrategy compactionStrategy; @JsonIgnore private final SegmentProvider segmentProvider; @JsonIgnore private final PartitionConfigurationManager partitionConfigurationManager; - @Nullable - private final CompactionStrategy compactionStrategy; @JsonIgnore private final SegmentCacheManagerFactory segmentCacheManagerFactory; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionStrategy.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionStrategy.java index 84bd012b076c..dccd04ac8202 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionStrategy.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionStrategy.java @@ -1,3 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.druid.indexing.common.task; import com.fasterxml.jackson.annotation.JsonProperty; @@ -31,13 +50,10 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; -import static org.apache.druid.indexing.common.task.CompactionTask.CTX_KEY_APPENDERATOR_TRACKING_TASK_ID; - public class NativeCompactionStrategy implements CompactionStrategy { - private static final Logger log = new Logger(NativeCompactionStrategy.class); - public static final String type = "NATIVE"; + public static final String TYPE = "NATIVE"; private static final boolean STORE_COMPACTION_STATE = true; @JsonCreator @@ -248,7 +264,7 @@ ParallelIndexSupervisorTask newTask(CompactionTask compactionTask, String baseSe Map createContextForSubtask(CompactionTask compactionTask) { final Map newContext = new HashMap<>(compactionTask.getContext()); - newContext.put(CTX_KEY_APPENDERATOR_TRACKING_TASK_ID, compactionTask.getId()); + newContext.put(CompactionTask.CTX_KEY_APPENDERATOR_TRACKING_TASK_ID, compactionTask.getId()); newContext.putIfAbsent(CompactSegments.STORE_COMPACTION_STATE_KEY, STORE_COMPACTION_STATE); // Set the priority of the compaction task. newContext.put(Tasks.PRIORITY_KEY, compactionTask.getPriority()); @@ -259,6 +275,6 @@ Map createContextForSubtask(CompactionTask compactionTask) @JsonProperty public String getType() { - return type; + return TYPE; } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java index 49e430cb3ea5..56123a561f6b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java @@ -210,7 +210,6 @@ public Response taskPost( .build(); } - Access authResult = AuthorizationUtils.authorizeAllResourceActions( req, resourceActions, diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index 0248b94a7b02..5e24a37bc78e 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java @@ -475,7 +475,8 @@ private int submitCompactionTasks( if (compactionEngine == CompactionEngine.MSQ) { if (!autoCompactionContext.containsKey(maxNumTasksContextParam)) { // Since MSQ needs all task slots for the calculated #tasks to be available upfront, allot all available - // compaction slots to current compaction task to avoid stalling. + // compaction slots to current compaction task to avoid stalling. Setting "taskAssignment" to "auto" has the + // problem of not being able to determine the actual count, which is required for subsequent tasks. numCurrentCompactionTasksAndSubtasks = numAvailableCompactionTaskSlots; autoCompactionContext.put(maxNumTasksContextParam, numCurrentCompactionTasksAndSubtasks); } else { From 1a9af3cac338e53f4f360f0762e958b874c82971 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Tue, 7 May 2024 12:09:49 +0530 Subject: [PATCH 20/54] Fields scope changes --- .../msq/indexing/MSQCompactionStrategy.java | 24 +- .../msq/util/MultiStageQueryContext.java | 4 +- .../indexing/MSQCompactionStrategyTest.java | 238 ++++++++++++++++++ 3 files changed, 254 insertions(+), 12 deletions(-) create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionStrategyTest.java diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionStrategy.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionStrategy.java index 4a9ef55e0794..ed08e0b757b1 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionStrategy.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionStrategy.java @@ -87,8 +87,8 @@ public class MSQCompactionStrategy implements CompactionStrategy private final ObjectMapper jsonMapper; private final Injector injector; - private static final String TIME_VIRTUAL_COLUMN = "vTime"; - private static final String TIME_COLUMN = ColumnHolder.TIME_COLUMN_NAME; + public static final String TIME_VIRTUAL_COLUMN = "vTime"; + public static final String TIME_COLUMN = ColumnHolder.TIME_COLUMN_NAME; public MSQCompactionStrategy(@JacksonInject ObjectMapper jsonMapper, @JacksonInject Injector injector) { @@ -263,22 +263,26 @@ private static RowSignature getRowSignature(DataSchema dataSchema) private static List getAggregateDimensions(DataSchema ds) { - List dimensionSpecs = ds.getDimensionsSpec().getDimensions().stream() + List dimensionSpecs = new ArrayList<>(); + + if (isQueryGranularityEmpty(ds)) { + dimensionSpecs.add(new DefaultDimensionSpec(TIME_COLUMN, TIME_VIRTUAL_COLUMN, ColumnType.LONG)); + } else { + // The changed granularity would result in a new virtual column that needs to be aggregated upon. + dimensionSpecs.add(new DefaultDimensionSpec(TIME_VIRTUAL_COLUMN, TIME_VIRTUAL_COLUMN, ColumnType.LONG)); + } + + dimensionSpecs.addAll(ds.getDimensionsSpec().getDimensions().stream() .map(dim -> new DefaultDimensionSpec( dim.getName(), dim.getName(), dim.getColumnType() )) - .collect(Collectors.toList()); + .collect(Collectors.toList())); // Dimensions in group-by aren't allowed to have time column as the output name. - if (isQueryGranularityEmpty(ds)) { - dimensionSpecs.add(new DefaultDimensionSpec(TIME_COLUMN, TIME_VIRTUAL_COLUMN, ColumnType.LONG)); - } else { - // The changed granularity would result in a new virtual column that needs to be aggregated upon. - dimensionSpecs.add(new DefaultDimensionSpec(TIME_VIRTUAL_COLUMN, TIME_VIRTUAL_COLUMN, ColumnType.LONG)); - } + return dimensionSpecs; } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java index 60734b5b1dad..17906a8116ed 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java @@ -101,7 +101,7 @@ public class MultiStageQueryContext public static final String CTX_MAX_NUM_TASKS = "maxNumTasks"; @VisibleForTesting - static final int DEFAULT_MAX_NUM_TASKS = 2; + public static final int DEFAULT_MAX_NUM_TASKS = 2; public static final String CTX_TASK_ASSIGNMENT_STRATEGY = "taskAssignment"; private static final String DEFAULT_TASK_ASSIGNMENT_STRATEGY = WorkerAssignmentStrategy.MAX.toString(); @@ -139,7 +139,7 @@ public class MultiStageQueryContext public static final String CTX_ROWS_IN_MEMORY = "rowsInMemory"; // Lower than the default to minimize the impact of per-row overheads that are not accounted for by // OnheapIncrementalIndex. For example: overheads related to creating bitmaps during persist. - static final int DEFAULT_ROWS_IN_MEMORY = 100000; + public static final int DEFAULT_ROWS_IN_MEMORY = 100000; public static final String CTX_IS_REINDEX = "isReindex"; diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionStrategyTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionStrategyTest.java new file mode 100644 index 000000000000..e865425b68d2 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionStrategyTest.java @@ -0,0 +1,238 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.indexing; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.InjectableValues; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import org.apache.druid.client.indexing.ClientCompactionTaskTransformSpec; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.data.input.impl.DimensionSchema; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.LongDimensionSchema; +import org.apache.druid.data.input.impl.StringDimensionSchema; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; +import org.apache.druid.indexing.common.task.CompactionIntervalSpec; +import org.apache.druid.indexing.common.task.CompactionTask; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.NonnullPair; +import org.apache.druid.java.util.common.granularity.GranularityType; +import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination; +import org.apache.druid.msq.kernel.WorkerAssignmentStrategy; +import org.apache.druid.msq.util.MultiStageQueryContext; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.query.expression.LookupEnabledTestExprMacroTable; +import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.data.CompressionFactory; +import org.apache.druid.segment.data.CompressionStrategy; +import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; +import org.apache.druid.segment.transform.TransformSpec; +import org.apache.druid.sql.calcite.parser.DruidSqlInsert; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class MSQCompactionStrategyTest +{ + private static final String DATA_SOURCE = "dataSource"; + private static final Interval COMPACTION_INTERVAL = Intervals.of("2017-01-01/2017-07-01"); + + private static final String TIMESTAMP_COLUMN = "timestamp"; + private static final int TARGET_ROWS_PER_SEGMENT = 100000; + private static final GranularityType SEGMENT_GRANULARITY = GranularityType.HOUR; + private static final GranularityType QUERY_GRANULARITY = GranularityType.HOUR; + private static List PARTITION_DIMENSIONS; + private static List SORT_ORDER_DIMENSIONS; + + private static final StringDimensionSchema DIM1 = new StringDimensionSchema( + "string_dim", + null, + null + ); + private static final LongDimensionSchema DIM2 = new LongDimensionSchema("long_dim"); + private static final List DIMENSIONS = ImmutableList.of(DIM1, DIM2); + private static final ObjectMapper JSON_MAPPER = new DefaultObjectMapper(); + private static final AggregatorFactory AGG1 = new CountAggregatorFactory("agg_0"); + private static final AggregatorFactory AGG2 = new LongSumAggregatorFactory("agg_1", "long_dim_1"); + private static List AGGREGATORS = ImmutableList.of(AGG1, AGG2); + + @BeforeClass + public static void setupClass() + { + NullHandling.initializeForTests(); + + final StringDimensionSchema stringDimensionSchema = new StringDimensionSchema( + "string_dim", + null, + null + ); + + PARTITION_DIMENSIONS = Collections.singletonList(stringDimensionSchema.getName()); + + final LongDimensionSchema longDimensionSchema = new LongDimensionSchema("long_dim"); + SORT_ORDER_DIMENSIONS = Collections.singletonList(longDimensionSchema.getName()); + + + JSON_MAPPER.setInjectableValues(new InjectableValues.Std().addValue( + ExprMacroTable.class, + LookupEnabledTestExprMacroTable.INSTANCE + )); + } + + @Test + public void testCompactionToMSQTasks() throws JsonProcessingException + { + DimFilter dimFilter = new SelectorDimFilter("dim1", "foo", null); + ClientCompactionTaskTransformSpec transformSpec = + new ClientCompactionTaskTransformSpec(dimFilter); + final CompactionTask.Builder builder = new CompactionTask.Builder( + DATA_SOURCE, + null, + null + ); + IndexSpec indexSpec = IndexSpec.builder() + .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) + .withDimensionCompression(CompressionStrategy.LZ4) + .withMetricCompression(CompressionStrategy.LZF) + .withLongEncoding(CompressionFactory.LongEncodingStrategy.LONGS) + .build(); + + Map context = new HashMap<>(); + context.put(MultiStageQueryContext.CTX_SORT_ORDER, JSON_MAPPER.writeValueAsString(SORT_ORDER_DIMENSIONS)); + + builder + .inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, null)) + .tuningConfig(createTuningConfig(indexSpec)) + .transformSpec(transformSpec) + .context(context); + + final CompactionTask taskCreatedWithTransformSpec = builder.build(); + Assert.assertEquals( + transformSpec, + taskCreatedWithTransformSpec.getTransformSpec() + ); + + DataSchema dataSchema = new DataSchema( + DATA_SOURCE, + new TimestampSpec(TIMESTAMP_COLUMN, null, null), + new DimensionsSpec(DIMENSIONS), + AGGREGATORS.toArray(new AggregatorFactory[0]), + new UniformGranularitySpec( + SEGMENT_GRANULARITY.getDefaultGranularity(), + QUERY_GRANULARITY.getDefaultGranularity(), + Collections.singletonList(COMPACTION_INTERVAL) + ), + new TransformSpec(dimFilter, Collections.emptyList()) + ); + + List msqControllerTasks = new MSQCompactionStrategy(JSON_MAPPER, null) + .compactionToMSQTasks( + taskCreatedWithTransformSpec, + Collections.singletonList(new NonnullPair<>( + COMPACTION_INTERVAL, + dataSchema + )) + ); + + Assert.assertEquals(1, msqControllerTasks.size()); + + MSQControllerTask msqControllerTask = msqControllerTasks.get(0); + MSQSpec actualMSQSpec = msqControllerTask.getQuerySpec(); + Assert.assertEquals( + new MSQTuningConfig( + MultiStageQueryContext.DEFAULT_MAX_NUM_TASKS, + MultiStageQueryContext.DEFAULT_ROWS_IN_MEMORY, + TARGET_ROWS_PER_SEGMENT, + indexSpec + ), + actualMSQSpec.getTuningConfig() + ); + Assert.assertEquals(new DataSourceMSQDestination( + DATA_SOURCE, + SEGMENT_GRANULARITY.getDefaultGranularity(), + SORT_ORDER_DIMENSIONS, + Collections.singletonList(COMPACTION_INTERVAL) + ), actualMSQSpec.getDestination()); + + Assert.assertEquals(dimFilter, actualMSQSpec.getQuery().getFilter()); + Assert.assertEquals( + JSON_MAPPER.writeValueAsString(SEGMENT_GRANULARITY.toString()), + msqControllerTask.getContext().get(DruidSqlInsert.SQL_INSERT_SEGMENT_GRANULARITY) + ); + Assert.assertEquals( + JSON_MAPPER.writeValueAsString(QUERY_GRANULARITY.toString()), + msqControllerTask.getContext().get(DruidSqlInsert.SQL_INSERT_QUERY_GRANULARITY) + ); + Assert.assertEquals(WorkerAssignmentStrategy.MAX, actualMSQSpec.getAssignmentStrategy()); + } + + private static CompactionTask.CompactionTuningConfig createTuningConfig(IndexSpec indexSpec) + { + return new CompactionTask.CompactionTuningConfig( + null, + null, // null to compute maxRowsPerSegment automatically + null, + 500000, + 1000000L, + null, + null, + null, + null, + new DimensionRangePartitionsSpec(TARGET_ROWS_PER_SEGMENT, null, PARTITION_DIMENSIONS, false), + indexSpec, + null, + null, + true, + false, + 5000L, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + ); + } +} From b7913102bb7ec243389a4e9274600236a74423e3 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Thu, 9 May 2024 15:37:42 +0530 Subject: [PATCH 21/54] Fix TYPE field and currentSubtaskHolder --- .../msq/indexing/MSQCompactionStrategy.java | 93 ++++++++++++------- .../common/task/CompactionStrategy.java | 7 +- .../indexing/common/task/CompactionTask.java | 8 +- .../common/task/NativeCompactionStrategy.java | 25 +++-- .../coordinator/UserCompactionStrategy.java | 16 ++-- 5 files changed, 88 insertions(+), 61 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionStrategy.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionStrategy.java index ed08e0b757b1..a46c99ef993f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionStrategy.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionStrategy.java @@ -76,6 +76,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.stream.Collectors; public class MSQCompactionStrategy implements CompactionStrategy @@ -97,12 +98,23 @@ public MSQCompactionStrategy(@JacksonInject ObjectMapper jsonMapper, @JacksonInj } @Override - @JsonProperty + @JsonProperty("TYPE") public String getType() { return TYPE; } + @Override + public CurrentSubTaskHolder getCurrentSubTaskHolder() + { + return new CurrentSubTaskHolder( + (taskObject, config) -> { + final MSQControllerTask msqControllerTask = (MSQControllerTask) taskObject; + msqControllerTask.stopGracefully(config); + } + ); + } + public List compactionToMSQTasks( CompactionTask compactionTask, List> intervalDataSchemas @@ -130,23 +142,23 @@ public List compactionToMSQTasks( for (NonnullPair intervalDataSchema : intervalDataSchemas) { Query query; Interval interval = intervalDataSchema.lhs; - DataSchema ds = intervalDataSchema.rhs; + DataSchema dataSchema = intervalDataSchema.rhs; - if (!isGroupBy(ds)) { - query = buildScanQuery(compactionTask, interval, ds); + if (!isGroupBy(dataSchema)) { + query = buildScanQuery(compactionTask, interval, dataSchema); } else { - query = buildGroupByQuery(compactionTask, interval, ds); + query = buildGroupByQuery(compactionTask, interval, dataSchema); } MSQSpec msqSpec = MSQSpec.builder() .query(query) - .columnMappings(getColumnMappings(ds)) - .destination(buildMSQDestination(compactionTask, ds, compactionTaskContext)) + .columnMappings(getColumnMappings(dataSchema)) + .destination(buildMSQDestination(compactionTask, dataSchema, compactionTaskContext)) .assignmentStrategy(MultiStageQueryContext.getAssignmentStrategy(compactionTaskContext)) .tuningConfig(buildMSQTuningConfig(compactionTask, compactionTaskContext)) .build(); - Map msqControllerTaskContext = createMSQTaskContext(compactionTask, ds); + Map msqControllerTaskContext = createMSQTaskContext(compactionTask, dataSchema); MSQControllerTask controllerTask = new MSQControllerTask( compactionTask.getId(), @@ -163,6 +175,7 @@ public List compactionToMSQTasks( } return msqControllerTasks; } + @Override public TaskStatus runCompactionTasks( CompactionTask compactionTask, @@ -188,7 +201,7 @@ public TaskStatus runCompactionTasks( private static DataSourceMSQDestination buildMSQDestination( CompactionTask compactionTask, - DataSchema ds, + DataSchema dataSchema, QueryContext compactionTaskContext ) { @@ -199,8 +212,8 @@ private static DataSourceMSQDestination buildMSQDestination( final List segmentSortOrder = MultiStageQueryContext.getSortOrder(compactionTaskContext); return new DataSourceMSQDestination( - ds.getDataSource(), - ds.getGranularitySpec().getSegmentGranularity(), + dataSchema.getDataSource(), + dataSchema.getGranularitySpec().getSegmentGranularity(), segmentSortOrder, ImmutableList.of(replaceInterval) ); @@ -255,30 +268,30 @@ private static RowSignature getRowSignature(DataSchema dataSchema) { RowSignature.Builder rowSignatureBuilder = RowSignature.builder(); rowSignatureBuilder.add(dataSchema.getTimestampSpec().getTimestampColumn(), ColumnType.LONG); - for (DimensionSchema ds : dataSchema.getDimensionsSpec().getDimensions()) { - rowSignatureBuilder.add(ds.getName(), ColumnType.fromString(ds.getTypeName())); + for (DimensionSchema dimensionSchema : dataSchema.getDimensionsSpec().getDimensions()) { + rowSignatureBuilder.add(dimensionSchema.getName(), ColumnType.fromString(dimensionSchema.getTypeName())); } return rowSignatureBuilder.build(); } - private static List getAggregateDimensions(DataSchema ds) + private static List getAggregateDimensions(DataSchema dataSchema) { List dimensionSpecs = new ArrayList<>(); - if (isQueryGranularityEmpty(ds)) { + if (isQueryGranularityEmptyOrNone(dataSchema)) { dimensionSpecs.add(new DefaultDimensionSpec(TIME_COLUMN, TIME_VIRTUAL_COLUMN, ColumnType.LONG)); } else { // The changed granularity would result in a new virtual column that needs to be aggregated upon. dimensionSpecs.add(new DefaultDimensionSpec(TIME_VIRTUAL_COLUMN, TIME_VIRTUAL_COLUMN, ColumnType.LONG)); } - dimensionSpecs.addAll(ds.getDimensionsSpec().getDimensions().stream() - .map(dim -> new DefaultDimensionSpec( - dim.getName(), - dim.getName(), - dim.getColumnType() - )) - .collect(Collectors.toList())); + dimensionSpecs.addAll(dataSchema.getDimensionsSpec().getDimensions().stream() + .map(dim -> new DefaultDimensionSpec( + dim.getName(), + dim.getName(), + dim.getColumnType() + )) + .collect(Collectors.toList())); // Dimensions in group-by aren't allowed to have time column as the output name. @@ -333,22 +346,30 @@ private static Query buildScanQuery(CompactionTask compactionTask, Interval i .build(); } - private static boolean isGroupBy(DataSchema ds) + private static boolean isGroupBy(DataSchema dataSchema) { - return ds.getAggregators().length > 0; + return dataSchema.getAggregators().length > 0; } - private static boolean isQueryGranularityEmpty(DataSchema ds) + private static boolean isQueryGranularityEmptyOrNone(DataSchema dataSchema) { - return ds.getGranularitySpec() == null || ds.getGranularitySpec().getQueryGranularity() == null; + return dataSchema.getGranularitySpec() == null + || dataSchema.getGranularitySpec().getQueryGranularity() == null + || Objects.equals( + dataSchema.getGranularitySpec().getQueryGranularity(), + Granularities.NONE + ); } - private static VirtualColumns getVirtualColumns(DataSchema ds) + private static VirtualColumns getVirtualTimeColumn(DataSchema dataSchema) { VirtualColumns virtualColumns = VirtualColumns.EMPTY; - if (!isQueryGranularityEmpty(ds) && !ds.getGranularitySpec().getQueryGranularity().equals(Granularities.ALL)) { - PeriodGranularity periodQueryGranularity = (PeriodGranularity) ds.getGranularitySpec().getQueryGranularity(); + if (!isQueryGranularityEmptyOrNone(dataSchema) && !dataSchema.getGranularitySpec() + .getQueryGranularity() + .equals(Granularities.ALL)) { + PeriodGranularity periodQueryGranularity = (PeriodGranularity) dataSchema.getGranularitySpec() + .getQueryGranularity(); VirtualColumn virtualColumn = new ExpressionVirtualColumn( TIME_VIRTUAL_COLUMN, StringUtils.format( @@ -370,7 +391,7 @@ private static Query buildGroupByQuery(CompactionTask compactionTask, Interva GroupByQuery.Builder builder = new GroupByQuery.Builder() .setDataSource(new TableDataSource(compactionTask.getDataSource())) - .setVirtualColumns(getVirtualColumns(dataSchema)) + .setVirtualColumns(getVirtualTimeColumn(dataSchema)) .setDimFilter(dimFilter) .setGranularity(new AllGranularity()) .setDimensions(getAggregateDimensions(dataSchema)) @@ -384,21 +405,21 @@ private static Query buildGroupByQuery(CompactionTask compactionTask, Interva return builder.build(); } - private Map createMSQTaskContext(CompactionTask compactionTask, DataSchema ds) + private Map createMSQTaskContext(CompactionTask compactionTask, DataSchema dataSchema) throws JsonProcessingException { Map context = new HashMap<>(compactionTask.getContext()); context.put( DruidSqlInsert.SQL_INSERT_SEGMENT_GRANULARITY, - jsonMapper.writeValueAsString(ds.getGranularitySpec() != null - ? ds.getGranularitySpec() - .getSegmentGranularity() + jsonMapper.writeValueAsString(dataSchema.getGranularitySpec() != null + ? dataSchema.getGranularitySpec() + .getSegmentGranularity() : DEFAULT_SEGMENT_GRANULARITY) ); - if (!isQueryGranularityEmpty(ds)) { + if (!isQueryGranularityEmptyOrNone(dataSchema)) { context.put( DruidSqlInsert.SQL_INSERT_QUERY_GRANULARITY, - jsonMapper.writeValueAsString(ds.getGranularitySpec().getQueryGranularity()) + jsonMapper.writeValueAsString(dataSchema.getGranularitySpec().getQueryGranularity()) ); } return context; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionStrategy.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionStrategy.java index 20fded29337e..cc2608c1a889 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionStrategy.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionStrategy.java @@ -19,11 +19,9 @@ package org.apache.druid.indexing.common.task; -import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.core.JsonProcessingException; -import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.java.util.common.NonnullPair; @@ -34,7 +32,7 @@ /** * Strategy to be used for executing a compaction task. - * All subtypes should be synchronized with {@link CompactionEngine}. + * Should be synchronized with {@link org.apache.druid.server.coordinator.UserCompactionStrategy} */ @JsonTypeInfo( use = JsonTypeInfo.Id.NAME, @@ -54,6 +52,7 @@ TaskStatus runCompactionTasks( List> dataSchemas ) throws JsonProcessingException; - @JsonProperty + CurrentSubTaskHolder getCurrentSubTaskHolder(); + String getType(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 2d0f6a2ec5c0..8b1054278248 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -167,12 +167,7 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg private final SegmentCacheManagerFactory segmentCacheManagerFactory; @JsonIgnore - private final CurrentSubTaskHolder currentSubTaskHolder = new CurrentSubTaskHolder( - (taskObject, config) -> { - final ParallelIndexSupervisorTask indexTask = (ParallelIndexSupervisorTask) taskObject; - indexTask.stopGracefully(config); - } - ); + private final CurrentSubTaskHolder currentSubTaskHolder; @JsonCreator public CompactionTask( @@ -246,6 +241,7 @@ public CompactionTask( this.partitionConfigurationManager = new PartitionConfigurationManager(this.tuningConfig); this.compactionStrategy = compactionStrategy; this.segmentCacheManagerFactory = segmentCacheManagerFactory; + this.currentSubTaskHolder = compactionStrategy.getCurrentSubTaskHolder(); } @VisibleForTesting diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionStrategy.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionStrategy.java index dccd04ac8202..ec330ab17a8b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionStrategy.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionStrategy.java @@ -61,6 +61,24 @@ public NativeCompactionStrategy() { } + @Override + public CurrentSubTaskHolder getCurrentSubTaskHolder() + { + return new CurrentSubTaskHolder( + (taskObject, config) -> { + final ParallelIndexSupervisorTask indexTask = (ParallelIndexSupervisorTask) taskObject; + indexTask.stopGracefully(config); + } + ); + } + + @Override + @JsonProperty("TYPE") + public String getType() + { + return TYPE; + } + /** * Generate {@link ParallelIndexIngestionSpec} from input dataschemas. * @@ -270,11 +288,4 @@ Map createContextForSubtask(CompactionTask compactionTask) newContext.put(Tasks.PRIORITY_KEY, compactionTask.getPriority()); return newContext; } - - @Override - @JsonProperty - public String getType() - { - return TYPE; - } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionStrategy.java b/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionStrategy.java index fbc90db95f41..738f29fecaf3 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionStrategy.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionStrategy.java @@ -28,25 +28,25 @@ public class UserCompactionStrategy { - private final CompactionEngine type; + private final CompactionEngine TYPE; @JsonCreator - public UserCompactionStrategy(@JsonProperty("type") CompactionEngine type) + public UserCompactionStrategy(@JsonProperty("TYPE") CompactionEngine TYPE) { - this.type = type; + this.TYPE = TYPE; } - @JsonProperty + @JsonProperty("TYPE") public CompactionEngine getType() { - return type; + return TYPE; } @Override public String toString() { return "UserCompactionStrategy{" + - "type=" + type + + "TYPE=" + TYPE + '}'; } @@ -60,12 +60,12 @@ public boolean equals(Object o) return false; } UserCompactionStrategy that = (UserCompactionStrategy) o; - return type == that.type; + return TYPE == that.TYPE; } @Override public int hashCode() { - return Objects.hash(type); + return Objects.hash(TYPE); } } From bd7ffb7d3aa3b9baec9abc326ec38946546d6a72 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Fri, 17 May 2024 20:04:43 +0530 Subject: [PATCH 22/54] Add segment load wait --- .../msq/indexing/MSQCompactionStrategy.java | 56 +++++++++---------- 1 file changed, 28 insertions(+), 28 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionStrategy.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionStrategy.java index a46c99ef993f..af0e1c12ef58 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionStrategy.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionStrategy.java @@ -115,6 +115,29 @@ public CurrentSubTaskHolder getCurrentSubTaskHolder() ); } + @Override + public TaskStatus runCompactionTasks( + CompactionTask compactionTask, + TaskToolbox taskToolbox, + List> intervalDataSchemas + ) throws JsonProcessingException + { + List msqControllerTasks = compactionToMSQTasks(compactionTask, intervalDataSchemas); + + if (msqControllerTasks.isEmpty()) { + log.warn( + "Can't find segments from inputSpec[%s], nothing to do.", + compactionTask.getIoConfig().getInputSpec() + ); + } + return runSubtasks( + msqControllerTasks, + taskToolbox, + compactionTask.getCurrentSubTaskHolder(), + compactionTask.getId() + ); + } + public List compactionToMSQTasks( CompactionTask compactionTask, List> intervalDataSchemas @@ -176,29 +199,6 @@ public List compactionToMSQTasks( return msqControllerTasks; } - @Override - public TaskStatus runCompactionTasks( - CompactionTask compactionTask, - TaskToolbox taskToolbox, - List> intervalDataSchemas - ) throws JsonProcessingException - { - List msqControllerTasks = compactionToMSQTasks(compactionTask, intervalDataSchemas); - - if (msqControllerTasks.isEmpty()) { - log.warn( - "Can't find segments from inputSpec[%s], nothing to do.", - compactionTask.getIoConfig().getInputSpec() - ); - } - return runSubtasks( - msqControllerTasks, - taskToolbox, - compactionTask.getCurrentSubTaskHolder(), - compactionTask.getId() - ); - } - private static DataSourceMSQDestination buildMSQDestination( CompactionTask compactionTask, DataSchema dataSchema, @@ -279,6 +279,7 @@ private static List getAggregateDimensions(DataSchema dataSchema) List dimensionSpecs = new ArrayList<>(); if (isQueryGranularityEmptyOrNone(dataSchema)) { + // Dimensions in group-by aren't allowed to have time column name as the output name. dimensionSpecs.add(new DefaultDimensionSpec(TIME_COLUMN, TIME_VIRTUAL_COLUMN, ColumnType.LONG)); } else { // The changed granularity would result in a new virtual column that needs to be aggregated upon. @@ -293,9 +294,6 @@ private static List getAggregateDimensions(DataSchema dataSchema) )) .collect(Collectors.toList())); - - // Dimensions in group-by aren't allowed to have time column as the output name. - return dimensionSpecs; } @@ -361,7 +359,7 @@ private static boolean isQueryGranularityEmptyOrNone(DataSchema dataSchema) ); } - private static VirtualColumns getVirtualTimeColumn(DataSchema dataSchema) + private static VirtualColumns getVirtualColumns(DataSchema dataSchema) { VirtualColumns virtualColumns = VirtualColumns.EMPTY; @@ -370,6 +368,7 @@ private static VirtualColumns getVirtualTimeColumn(DataSchema dataSchema) .equals(Granularities.ALL)) { PeriodGranularity periodQueryGranularity = (PeriodGranularity) dataSchema.getGranularitySpec() .getQueryGranularity(); + // Need to create a virtual column for time as that's the only way to support query granularity VirtualColumn virtualColumn = new ExpressionVirtualColumn( TIME_VIRTUAL_COLUMN, StringUtils.format( @@ -391,7 +390,7 @@ private static Query buildGroupByQuery(CompactionTask compactionTask, Interva GroupByQuery.Builder builder = new GroupByQuery.Builder() .setDataSource(new TableDataSource(compactionTask.getDataSource())) - .setVirtualColumns(getVirtualTimeColumn(dataSchema)) + .setVirtualColumns(getVirtualColumns(dataSchema)) .setDimFilter(dimFilter) .setGranularity(new AllGranularity()) .setDimensions(getAggregateDimensions(dataSchema)) @@ -422,6 +421,7 @@ private Map createMSQTaskContext(CompactionTask compactionTask, jsonMapper.writeValueAsString(dataSchema.getGranularitySpec().getQueryGranularity()) ); } + context.put(MultiStageQueryContext.CTX_SEGMENT_LOAD_WAIT, true); return context; } From 299c25d2788bf9abbd2388e98072233de2e472e5 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Mon, 20 May 2024 09:58:00 +0530 Subject: [PATCH 23/54] Extract dim and metrics from query for lastCompactionState --- .../apache/druid/msq/exec/ControllerImpl.java | 31 ++++++++++++++----- 1 file changed, 24 insertions(+), 7 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 91feeb3715a6..c857ef721313 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -1869,20 +1869,37 @@ private static Function, Set> addCompactionStateTo ((DataSourceMSQDestination) task.getQuerySpec().getDestination()).getReplaceTimeChunks() ); - DimensionsSpec dimensionsSpec = dataSchema.getDimensionsSpec(); Map transformSpec = TransformSpec.NONE.equals(dataSchema.getTransformSpec()) ? null : new ClientCompactionTaskTransformSpec( dataSchema.getTransformSpec().getFilter() ).asMap(jsonMapper); - List metricsSpec = dataSchema.getAggregators() == null - ? null - : jsonMapper.convertValue( - dataSchema.getAggregators(), new TypeReference>() - { - }); + DimensionsSpec dimensionsSpec; + List metricsSpec; + + if (task.getQuerySpec().getQuery() instanceof GroupByQuery) { + GroupByQuery groupByQuery = (GroupByQuery) task.getQuerySpec().getQuery(); + + + dimensionsSpec = new DimensionsSpec(groupByQuery.getDimensions() + .stream() + .map(dimensionSpec -> DimensionSchema.getDefaultSchemaForBuiltInType( + dimensionSpec.getOutputName(), + dimensionSpec.getOutputType() + )) + .collect( + Collectors.toList())); + metricsSpec = jsonMapper.convertValue( + groupByQuery.getAggregatorSpecs(), new TypeReference>() + { + }); + } else { + dimensionsSpec = new DimensionsSpec(dataSchema.getDimensionsSpec().getDimensions()); + metricsSpec = Collections.emptyList(); + } + IndexSpec indexSpec = tuningConfig.getIndexSpec(); log.info("Query[%s] storing compaction state in segments.", queryId); From 642952c01a3f45d975c010a1fc70cb4a89369a2f Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Fri, 24 May 2024 22:46:50 +0530 Subject: [PATCH 24/54] Address review comments --- .../apache/druid/msq/exec/ControllerImpl.java | 145 ++++++++--------- .../druid/msq/guice/MSQIndexingModule.java | 4 +- ...Strategy.java => MSQCompactionRunner.java} | 87 +++++++---- ...SegmentGeneratorFrameProcessorFactory.java | 14 +- .../apache/druid/msq/exec/MSQReplaceTest.java | 6 +- ...Test.java => MSQCompactionRunnerTest.java} | 5 +- .../common/task/AbstractBatchIndexTask.java | 5 +- ...ionStrategy.java => CompactionRunner.java} | 27 ++-- .../indexing/common/task/CompactionTask.java | 98 +++--------- ...ategy.java => NativeCompactionRunner.java} | 89 ++++++++--- .../ClientCompactionTaskQuerySerdeTest.java | 15 +- .../task/CompactionTaskParallelRunTest.java | 60 ++++--- .../common/task/CompactionTaskRunTest.java | 42 ++--- .../common/task/CompactionTaskTest.java | 48 ++---- .../batch/parallel/PartialCompactionTest.java | 1 - .../druid/indexer/CompactionEngine.java | 7 +- .../druid/timeline/CompactionState.java | 32 +++- .../druid/timeline/DataSegmentTest.java | 25 +-- .../indexing/ClientCompactionTaskQuery.java | 20 +-- .../ClientCompactionRunnerInfo.java | 146 ++++++++++++++++++ .../CoordinatorCompactionConfig.java | 36 +++-- .../DataSourceCompactionConfig.java | 68 +++----- .../coordinator/UserCompactionStrategy.java | 71 --------- .../coordinator/duty/CompactSegments.java | 8 +- .../compact/NewestSegmentFirstPolicyTest.java | 136 +++++++++++----- .../coordinator/duty/CompactSegmentsTest.java | 27 +++- .../server/http/DataSegmentPlusTest.java | 7 +- .../sql/calcite/schema/SystemSchemaTest.java | 6 +- 28 files changed, 708 insertions(+), 527 deletions(-) rename extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/{MSQCompactionStrategy.java => MSQCompactionRunner.java} (91%) rename extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/{MSQCompactionStrategyTest.java => MSQCompactionRunnerTest.java} (99%) rename indexing-service/src/main/java/org/apache/druid/indexing/common/task/{CompactionStrategy.java => CompactionRunner.java} (66%) rename indexing-service/src/main/java/org/apache/druid/indexing/common/task/{NativeCompactionStrategy.java => NativeCompactionRunner.java} (77%) create mode 100644 server/src/main/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfo.java delete mode 100644 server/src/main/java/org/apache/druid/server/coordinator/UserCompactionStrategy.java diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index c857ef721313..a0361f83cbad 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -63,6 +63,7 @@ import org.apache.druid.frame.util.DurableStorageUtils; import org.apache.druid.frame.write.InvalidFieldException; import org.apache.druid.frame.write.InvalidNullByteException; +import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; @@ -1769,37 +1770,40 @@ private void handleQueryResults( ); if (!segments.isEmpty() && storeCompactionState) { - if (!(task.getQuerySpec().getDestination() instanceof DataSourceMSQDestination)) { - throw new MSQException( - UnknownFault.forMessage( - StringUtils.format( - "Query[%s] cannot store compaction state in segments as destination[%s] not a datasource.", - queryDef.getQueryId(), - task.getQuerySpec().getDestination().getClass() - ))); - } - - DataSourceMSQDestination destination = (DataSourceMSQDestination) task.getQuerySpec().getDestination(); - if (!destination.isReplaceTimeChunks()) { - // Store compaction state only for replace queries. + if (!MSQControllerTask.isIngestion(task.getQuerySpec())) { log.warn( - "storeCompactionState flag set for a non-REPLACE query [%s]. Ignoring the flag for now.", - queryDef.getQueryId() + "Query[%s] cannot store compaction state in segments as destination[%s] not a datasource.", + queryDef.getQueryId(), + task.getQuerySpec().getDestination().getClass() ); } else { - DataSchema dataSchema = ((SegmentGeneratorFrameProcessorFactory) queryKernel - .getStageDefinition(finalStageId).getProcessorFactory()).getDataSchema(); - - ShardSpec shardSpec = segments.stream().findFirst().get().getShardSpec(); - - Function, Set> compactionStateAnnotateFunction = addCompactionStateToSegments( - task(), - context.jsonMapper(), - dataSchema, - shardSpec, - queryDef.getQueryId() - ); - segments = compactionStateAnnotateFunction.apply(segments); + DataSourceMSQDestination destination = (DataSourceMSQDestination) task.getQuerySpec().getDestination(); + if (!destination.isReplaceTimeChunks()) { + // Store compaction state only for replace queries. + log.warn( + "storeCompactionState flag set for a non-REPLACE query [%s]. Ignoring the flag for now.", + queryDef.getQueryId() + ); + } else { + DataSchema dataSchema = ((SegmentGeneratorFrameProcessorFactory) queryKernel + .getStageDefinition(finalStageId).getProcessorFactory()).getDataSchema(); + + ShardSpec shardSpec = segments.stream().findFirst().get().getShardSpec(); + + Map dimensionToAggregatoryFactoryMap = + ((SegmentGeneratorFrameProcessorFactory) queryKernel + .getStageDefinition(finalStageId).getProcessorFactory()).getDimensionToAggregatorFactoryMap(); + + Function, Set> compactionStateAnnotateFunction = addCompactionStateToSegments( + task(), + context.jsonMapper(), + dataSchema, + dimensionToAggregatoryFactoryMap, + shardSpec, + queryDef.getQueryId() + ); + segments = compactionStateAnnotateFunction.apply(segments); + } } } log.info("Query [%s] publishing %d segments.", queryDef.getQueryId(), segments.size()); @@ -1830,6 +1834,7 @@ private static Function, Set> addCompactionStateTo MSQControllerTask task, ObjectMapper jsonMapper, DataSchema dataSchema, + Map dimensionToAggregatorFactoryMap, ShardSpec shardSpec, String queryId ) @@ -1869,48 +1874,31 @@ private static Function, Set> addCompactionStateTo ((DataSourceMSQDestination) task.getQuerySpec().getDestination()).getReplaceTimeChunks() ); + DimensionsSpec dimensionsSpec = dataSchema.getDimensionsSpec(); Map transformSpec = TransformSpec.NONE.equals(dataSchema.getTransformSpec()) ? null : new ClientCompactionTaskTransformSpec( dataSchema.getTransformSpec().getFilter() ).asMap(jsonMapper); + List metricsSpec = dataSchema.getAggregators() == null + ? null + : jsonMapper.convertValue( + dataSchema.getAggregators(), new TypeReference>() + { + }); - DimensionsSpec dimensionsSpec; - List metricsSpec; - - if (task.getQuerySpec().getQuery() instanceof GroupByQuery) { - GroupByQuery groupByQuery = (GroupByQuery) task.getQuerySpec().getQuery(); - - - dimensionsSpec = new DimensionsSpec(groupByQuery.getDimensions() - .stream() - .map(dimensionSpec -> DimensionSchema.getDefaultSchemaForBuiltInType( - dimensionSpec.getOutputName(), - dimensionSpec.getOutputType() - )) - .collect( - Collectors.toList())); - metricsSpec = jsonMapper.convertValue( - groupByQuery.getAggregatorSpecs(), new TypeReference>() - { - }); - } else { - dimensionsSpec = new DimensionsSpec(dataSchema.getDimensionsSpec().getDimensions()); - metricsSpec = Collections.emptyList(); - } - IndexSpec indexSpec = tuningConfig.getIndexSpec(); - log.info("Query[%s] storing compaction state in segments.", queryId); - return CompactionState.addCompactionStateToSegments( partitionSpec, dimensionsSpec, + dimensionToAggregatorFactoryMap, metricsSpec, transformSpec, indexSpec.asMap(jsonMapper), - granularitySpec.asMap(jsonMapper) + granularitySpec.asMap(jsonMapper), + CompactionEngine.MSQ ); } @@ -2030,9 +2018,20 @@ private static QueryDefinition makeQueryDefinition( } } + // Map to track the aggregator factories of dimensions that have been created from metrics due to context flag + // finalizeAggregations=true. + final Map dimensionToAggregatoryFactoryMap = new HashMap<>(); + // Then, add a segment-generation stage. final DataSchema dataSchema = - generateDataSchema(querySpec, querySignature, queryClusterBy, columnMappings, jsonMapper); + generateDataSchema( + querySpec, + querySignature, + queryClusterBy, + columnMappings, + jsonMapper, + dimensionToAggregatoryFactoryMap + ); builder.add( StageDefinition.builder(queryDef.getNextStageNumber()) @@ -2042,13 +2041,14 @@ private static QueryDefinition makeQueryDefinition( new SegmentGeneratorFrameProcessorFactory( dataSchema, columnMappings, - tuningConfig + tuningConfig, + dimensionToAggregatoryFactoryMap ) ) ); return builder.build(); - } else if (querySpec.getDestination() instanceof TaskReportMSQDestination) { + } else if (querySpec.getDestination() instanceof TaskReportMSQDestination) { return queryDef; } else if (querySpec.getDestination() instanceof DurableStorageMSQDestination) { @@ -2118,7 +2118,8 @@ private static DataSchema generateDataSchema( RowSignature querySignature, ClusterBy queryClusterBy, ColumnMappings columnMappings, - ObjectMapper jsonMapper + ObjectMapper jsonMapper, + Map dimensionToAggregatorFactoryMap ) { final DataSourceMSQDestination destination = (DataSourceMSQDestination) querySpec.getDestination(); @@ -2131,7 +2132,8 @@ private static DataSchema generateDataSchema( destination.getSegmentSortOrder(), columnMappings, isRollupQuery, - querySpec.getQuery() + querySpec.getQuery(), + dimensionToAggregatorFactoryMap ); return new DataSchema( @@ -2322,7 +2324,9 @@ private static Pair, List> makeDimensio final List segmentSortOrder, final ColumnMappings columnMappings, final boolean isRollupQuery, - final Query query + final Query query, + final Map dimensionToAggregatoryFactoryMap + ) { // Log a warning unconditionally if arrayIngestMode is MVD, since the behaviour is incorrect, and is subject to @@ -2366,23 +2370,26 @@ private static Pair, List> makeDimensio Map outputColumnAggregatorFactories = new HashMap<>(); - if (isRollupQuery) { - // Populate aggregators from the native query when doing an ingest in rollup mode. - for (AggregatorFactory aggregatorFactory : ((GroupByQuery) query).getAggregatorSpecs()) { - for (final int outputColumn : columnMappings.getOutputColumnsForQueryColumn(aggregatorFactory.getName())) { - final String outputColumnName = columnMappings.getOutputColumnName(outputColumn); - if (outputColumnAggregatorFactories.containsKey(outputColumnName)) { - throw new ISE("There can only be one aggregation for column [%s].", outputColumn); - } else { + // Populate aggregators from the native query when doing an ingest in rollup mode. + for (AggregatorFactory aggregatorFactory : ((GroupByQuery) query).getAggregatorSpecs()) { + for (final int outputColumn : columnMappings.getOutputColumnsForQueryColumn(aggregatorFactory.getName())) { + final String outputColumnName = columnMappings.getOutputColumnName(outputColumn); + if (outputColumnAggregatorFactories.containsKey(outputColumnName)) { + throw new ISE("There can only be one aggregation for column [%s].", outputColumn); + } else { + if (isRollupQuery) { outputColumnAggregatorFactories.put( outputColumnName, aggregatorFactory.withName(outputColumnName).getCombiningFactory() ); + } else { + dimensionToAggregatoryFactoryMap.put(outputColumnName, aggregatorFactory); } } } } + // Each column can be of either time, dimension, aggregator. For this method. we can ignore the time column. // For non-complex columns, If the aggregator factory of the column is not available, we treat the column as // a dimension. For complex columns, certains hacks are in place. diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java index 137e08046ce1..49417c92ee9d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java @@ -35,7 +35,7 @@ import org.apache.druid.msq.counters.SegmentGenerationProgressCounter; import org.apache.druid.msq.counters.SuperSorterProgressTrackerCounter; import org.apache.druid.msq.counters.WarningCounters; -import org.apache.druid.msq.indexing.MSQCompactionStrategy; +import org.apache.druid.msq.indexing.MSQCompactionRunner; import org.apache.druid.msq.indexing.MSQControllerTask; import org.apache.druid.msq.indexing.MSQWorkerTask; import org.apache.druid.msq.indexing.error.BroadcastTablesTooLargeFault; @@ -199,7 +199,7 @@ public List getJacksonModules() NilInputSource.class ); - module.registerSubtypes(new NamedType(MSQCompactionStrategy.class, MSQCompactionStrategy.TYPE)); + module.registerSubtypes(new NamedType(MSQCompactionRunner.class, MSQCompactionRunner.type)); FAULT_CLASSES.forEach(module::registerSubtypes); module.addSerializer(new CounterSnapshotsSerializer()); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionStrategy.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java similarity index 91% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionStrategy.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java index af0e1c12ef58..b2387b0049cc 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionStrategy.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java @@ -20,6 +20,7 @@ package org.apache.druid.msq.indexing; import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; @@ -33,10 +34,11 @@ import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexer.partitions.SecondaryPartitionType; import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.task.CompactionStrategy; +import org.apache.druid.indexing.common.task.CompactionRunner; import org.apache.druid.indexing.common.task.CompactionTask; import org.apache.druid.indexing.common.task.CurrentSubTaskHolder; import org.apache.druid.java.util.common.NonnullPair; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.AllGranularity; import org.apache.druid.java.util.common.granularity.Granularities; @@ -79,10 +81,10 @@ import java.util.Objects; import java.util.stream.Collectors; -public class MSQCompactionStrategy implements CompactionStrategy +public class MSQCompactionRunner implements CompactionRunner { - private static final Logger log = new Logger(MSQCompactionStrategy.class); - public static final String TYPE = "MSQ"; + private static final Logger log = new Logger(MSQCompactionRunner.class); + public static final String type = "MSQ"; private static final Granularity DEFAULT_SEGMENT_GRANULARITY = Granularities.ALL; private final ObjectMapper jsonMapper; @@ -91,36 +93,72 @@ public class MSQCompactionStrategy implements CompactionStrategy public static final String TIME_VIRTUAL_COLUMN = "vTime"; public static final String TIME_COLUMN = ColumnHolder.TIME_COLUMN_NAME; - public MSQCompactionStrategy(@JacksonInject ObjectMapper jsonMapper, @JacksonInject Injector injector) + @JsonIgnore + private final CurrentSubTaskHolder currentSubTaskHolder = new CurrentSubTaskHolder( + (taskObject, config) -> { + final MSQControllerTask msqControllerTask = (MSQControllerTask) taskObject; + msqControllerTask.stopGracefully(config); + }); + + + public MSQCompactionRunner(@JacksonInject ObjectMapper jsonMapper, @JacksonInject Injector injector) { this.jsonMapper = jsonMapper; this.injector = injector; } + @Override + public Pair supportsCompactionConfig( + CompactionTask compactionTask + ) + { + QueryContext compactionTaskContext = new QueryContext(compactionTask.getContext()); + + // These checks cannot be included in compaction config validation as context param keys are unknown outside MSQ. + if (!MultiStageQueryContext.isFinalizeAggregations(compactionTaskContext)) { + return Pair.of(false, StringUtils.format( + "Config[%s] cannot be set to false for auto-compaction with MSQ engine.", + MultiStageQueryContext.CTX_FINALIZE_AGGREGATIONS + )); + } + if (MultiStageQueryContext.getAssignmentStrategy(compactionTaskContext) == WorkerAssignmentStrategy.AUTO) { + return Pair.of( + false, + StringUtils.format("Config[%s] cannot be set to value[%s] for auto-compaction with MSQ engine.", + MultiStageQueryContext.CTX_TASK_ASSIGNMENT_STRATEGY, + WorkerAssignmentStrategy.AUTO + ) + ); + } + if (compactionTask.getTuningConfig() != null) { + PartitionsSpec partitionsSpec = compactionTask.getTuningConfig().getPartitionsSpec(); + if (!(partitionsSpec instanceof DynamicPartitionsSpec + || partitionsSpec instanceof DimensionRangePartitionsSpec)) { + return Pair.of(false, "PartitionsSpec not among DynamicPartitionSpec or DimensionRangePartitionsSpec"); + } + } + return Pair.of(true, null); + } + @Override @JsonProperty("TYPE") public String getType() { - return TYPE; + return type; } @Override public CurrentSubTaskHolder getCurrentSubTaskHolder() { - return new CurrentSubTaskHolder( - (taskObject, config) -> { - final MSQControllerTask msqControllerTask = (MSQControllerTask) taskObject; - msqControllerTask.stopGracefully(config); - } - ); + return currentSubTaskHolder; } @Override public TaskStatus runCompactionTasks( CompactionTask compactionTask, - TaskToolbox taskToolbox, - List> intervalDataSchemas - ) throws JsonProcessingException + List> intervalDataSchemas, + TaskToolbox taskToolbox + ) throws Exception { List msqControllerTasks = compactionToMSQTasks(compactionTask, intervalDataSchemas); @@ -133,7 +171,7 @@ public TaskStatus runCompactionTasks( return runSubtasks( msqControllerTasks, taskToolbox, - compactionTask.getCurrentSubTaskHolder(), + currentSubTaskHolder, compactionTask.getId() ); } @@ -144,23 +182,7 @@ public List compactionToMSQTasks( ) throws JsonProcessingException { List msqControllerTasks = new ArrayList<>(); - QueryContext compactionTaskContext = new QueryContext(compactionTask.getContext()); - // These checks cannot be included in compaction config validation as context param keys are unknown outside MSQ. - if (!MultiStageQueryContext.isFinalizeAggregations(compactionTaskContext)) { - throw InvalidInput.exception( - "Config[%s] cannot be set to false for auto-compaction with MSQ engine.", - MultiStageQueryContext.CTX_FINALIZE_AGGREGATIONS, - MultiStageQueryContext.isFinalizeAggregations(compactionTaskContext) - ); - } - if (MultiStageQueryContext.getAssignmentStrategy(compactionTaskContext) == WorkerAssignmentStrategy.AUTO) { - throw InvalidInput.exception( - "Config[%s] cannot be set to value[%s] for auto-compaction with MSQ engine.", - MultiStageQueryContext.CTX_TASK_ASSIGNMENT_STRATEGY, - WorkerAssignmentStrategy.AUTO - ); - } for (NonnullPair intervalDataSchema : intervalDataSchemas) { Query query; @@ -172,6 +194,7 @@ public List compactionToMSQTasks( } else { query = buildGroupByQuery(compactionTask, interval, dataSchema); } + QueryContext compactionTaskContext = new QueryContext(compactionTask.getContext()); MSQSpec msqSpec = MSQSpec.builder() .query(query) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessorFactory.java index e925e1a1c028..941a6cab670c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessorFactory.java @@ -49,6 +49,7 @@ import org.apache.druid.msq.kernel.ProcessorsAndChannels; import org.apache.druid.msq.kernel.StageDefinition; import org.apache.druid.msq.kernel.StagePartition; +import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.data.CompressionFactory; import org.apache.druid.segment.data.CompressionStrategy; @@ -71,6 +72,7 @@ import java.io.File; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Set; import java.util.function.Consumer; @@ -82,17 +84,21 @@ public class SegmentGeneratorFrameProcessorFactory private final DataSchema dataSchema; private final ColumnMappings columnMappings; private final MSQTuningConfig tuningConfig; + private final Map dimensionToAggregatorFactoryMap; @JsonCreator public SegmentGeneratorFrameProcessorFactory( @JsonProperty("dataSchema") final DataSchema dataSchema, @JsonProperty("columnMappings") final ColumnMappings columnMappings, - @JsonProperty("tuningConfig") final MSQTuningConfig tuningConfig + @JsonProperty("tuningConfig") final MSQTuningConfig tuningConfig, + @JsonProperty("dimensionToAggregatorFactoryMap") + final Map dimensionToAggregatorFactoryMap ) { this.dataSchema = Preconditions.checkNotNull(dataSchema, "dataSchema"); this.columnMappings = Preconditions.checkNotNull(columnMappings, "columnMappings"); this.tuningConfig = Preconditions.checkNotNull(tuningConfig, "tuningConfig"); + this.dimensionToAggregatorFactoryMap = dimensionToAggregatorFactoryMap; } @JsonProperty @@ -113,6 +119,12 @@ public MSQTuningConfig getTuningConfig() return tuningConfig; } + @JsonProperty + public Map getDimensionToAggregatorFactoryMap() + { + return dimensionToAggregatorFactoryMap; + } + @Override public ProcessorsAndChannels> makeProcessors( StageDefinition stageDefinition, diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java index 86ef7fc8c722..c0b6a4bb425f 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java @@ -29,6 +29,7 @@ import org.apache.druid.data.input.impl.FloatDimensionSchema; import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; +import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; @@ -1917,10 +1918,11 @@ private CompactionState expectedCompactionState( return new CompactionState( partitionsSpec, dimensionsSpec, - metricsSpec, + Collections.emptyMap(), metricsSpec, null, indexSpec.asMap(objectMapper), - granularitySpec.asMap(objectMapper) + granularitySpec.asMap(objectMapper), + CompactionEngine.NATIVE ); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionStrategyTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java similarity index 99% rename from extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionStrategyTest.java rename to extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java index e865425b68d2..f5dce5c50b49 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionStrategyTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java @@ -65,7 +65,7 @@ import java.util.List; import java.util.Map; -public class MSQCompactionStrategyTest +public class MSQCompactionRunnerTest { private static final String DATA_SOURCE = "dataSource"; private static final Interval COMPACTION_INTERVAL = Intervals.of("2017-01-01/2017-07-01"); @@ -120,7 +120,6 @@ public void testCompactionToMSQTasks() throws JsonProcessingException new ClientCompactionTaskTransformSpec(dimFilter); final CompactionTask.Builder builder = new CompactionTask.Builder( DATA_SOURCE, - null, null ); IndexSpec indexSpec = IndexSpec.builder() @@ -158,7 +157,7 @@ public void testCompactionToMSQTasks() throws JsonProcessingException new TransformSpec(dimFilter, Collections.emptyList()) ); - List msqControllerTasks = new MSQCompactionStrategy(JSON_MAPPER, null) + List msqControllerTasks = new MSQCompactionRunner(JSON_MAPPER, null) .compactionToMSQTasks( taskCreatedWithTransformSpec, Collections.singletonList(new NonnullPair<>( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java index 9fe3b78ee2d2..164154df1e4e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java @@ -30,6 +30,7 @@ import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.IngestionState; import org.apache.druid.indexer.report.IngestionStatsAndErrors; import org.apache.druid.indexer.report.IngestionStatsAndErrorsTaskReport; @@ -633,10 +634,12 @@ public static Function, Set> addCompactionStateToS return CompactionState.addCompactionStateToSegments( tuningConfig.getPartitionsSpec(), dimensionsSpec, + Collections.emptyMap(), metricsSpec, transformSpec, tuningConfig.getIndexSpec().asMap(toolbox.getJsonMapper()), - granularitySpec.asMap(toolbox.getJsonMapper()) + granularitySpec.asMap(toolbox.getJsonMapper()), + CompactionEngine.NATIVE ); } else { return Function.identity(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionStrategy.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java similarity index 66% rename from indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionStrategy.java rename to indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java index cc2608c1a889..df5a0e6f56de 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionStrategy.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java @@ -21,38 +21,47 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.fasterxml.jackson.core.JsonProcessingException; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.java.util.common.NonnullPair; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.server.coordinator.ClientCompactionRunnerInfo; import org.joda.time.Interval; import java.util.List; /** * Strategy to be used for executing a compaction task. - * Should be synchronized with {@link org.apache.druid.server.coordinator.UserCompactionStrategy} + * Should be synchronized with {@link ClientCompactionRunnerInfo} */ @JsonTypeInfo( use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.EXISTING_PROPERTY, - property = CompactionStrategy.TYPE_PROPERTY, + property = CompactionRunner.TYPE_PROPERTY, visible = true) @JsonSubTypes(value = { - @JsonSubTypes.Type(name = NativeCompactionStrategy.TYPE, value = NativeCompactionStrategy.class) + @JsonSubTypes.Type(name = NativeCompactionRunner.type, value = NativeCompactionRunner.class) }) -public interface CompactionStrategy +public interface CompactionRunner { - String TYPE_PROPERTY = "TYPE"; + String TYPE_PROPERTY = "type"; TaskStatus runCompactionTasks( CompactionTask compactionTask, - TaskToolbox taskToolbox, - List> dataSchemas - ) throws JsonProcessingException; + List> dataSchemas, + TaskToolbox taskToolbox + ) throws Exception; CurrentSubTaskHolder getCurrentSubTaskHolder(); + /** + * Checks if the provided compaction config is supported by the runner. + * @param compactionTask + * @return Pair of (supported) boolean and a reason string. Reason string is null if supported is true. + * + */ + Pair supportsCompactionConfig(CompactionTask compactionTask); + String getType(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 8b1054278248..4eb2b2f4f326 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -19,7 +19,6 @@ package org.apache.druid.indexing.common.task; -import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonInclude; @@ -42,20 +41,17 @@ import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.error.DruidException; +import org.apache.druid.error.InvalidInput; import org.apache.druid.indexer.Checks; import org.apache.druid.indexer.Property; import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.RetryPolicyFactory; -import org.apache.druid.indexing.common.SegmentCacheManagerFactory; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig; -import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTask; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; @@ -158,14 +154,9 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg @Nullable private final CompactionTuningConfig tuningConfig; @Nullable - private final CompactionStrategy compactionStrategy; + private final CompactionRunner compactionRunner; @JsonIgnore private final SegmentProvider segmentProvider; - @JsonIgnore - private final PartitionConfigurationManager partitionConfigurationManager; - @JsonIgnore - private final SegmentCacheManagerFactory segmentCacheManagerFactory; - @JsonIgnore private final CurrentSubTaskHolder currentSubTaskHolder; @@ -185,8 +176,7 @@ public CompactionTask( @JsonProperty("granularitySpec") @Nullable final ClientCompactionTaskGranularitySpec granularitySpec, @JsonProperty("tuningConfig") @Nullable final TuningConfig tuningConfig, @JsonProperty("context") @Nullable final Map context, - @JsonProperty("compactionStrategy") final CompactionStrategy compactionStrategy, - @JacksonInject SegmentCacheManagerFactory segmentCacheManagerFactory + @JsonProperty("compactionRunner") final CompactionRunner compactionRunner ) { super( @@ -238,10 +228,8 @@ public CompactionTask( } this.tuningConfig = tuningConfig != null ? getTuningConfig(tuningConfig) : null; this.segmentProvider = new SegmentProvider(dataSource, this.ioConfig.getInputSpec()); - this.partitionConfigurationManager = new PartitionConfigurationManager(this.tuningConfig); - this.compactionStrategy = compactionStrategy; - this.segmentCacheManagerFactory = segmentCacheManagerFactory; - this.currentSubTaskHolder = compactionStrategy.getCurrentSubTaskHolder(); + this.compactionRunner = compactionRunner; + this.currentSubTaskHolder = compactionRunner.getCurrentSubTaskHolder(); } @VisibleForTesting @@ -349,16 +337,6 @@ public DimensionsSpec getDimensionsSpec() return dimensionsSpec; } - public PartitionConfigurationManager getPartitionConfigurationManager() - { - return partitionConfigurationManager; - } - - public SegmentCacheManagerFactory getSegmentCacheManagerFactory() - { - return segmentCacheManagerFactory; - } - @JsonProperty @Nullable public ClientCompactionTaskTransformSpec getTransformSpec() @@ -391,15 +369,15 @@ public ClientCompactionTaskGranularitySpec getGranularitySpec() @Nullable @JsonProperty - public ParallelIndexTuningConfig getTuningConfig() + public CompactionTuningConfig getTuningConfig() { return tuningConfig; } @JsonProperty - public CompactionStrategy getCompactionStrategy() + public CompactionRunner getCompactionRunner() { - return compactionStrategy; + return compactionRunner; } @Override @@ -487,17 +465,16 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception getMetricBuilder() ); - if (compactionStrategy == null) { - // Can only happen for MSQ engine, when the json subtype reqd for deserialization isn't available due to - // missing extn. - throw DruidException.forPersona(DruidException.Persona.ADMIN) - .ofCategory(DruidException.Category.NOT_FOUND) - .build( - "Extension[druid-multi-stage-query] required for running compaction on MSQ " - + "not found on the Indexer."); + registerResourceCloserOnAbnormalExit(compactionRunner.getCurrentSubTaskHolder()); + Pair supportsCompactionConfig = compactionRunner.supportsCompactionConfig(this); + if (!supportsCompactionConfig.lhs) { + throw InvalidInput.exception( + "Compaction config for engine[%s] not supported[%s]", + compactionRunner.getType(), + supportsCompactionConfig.rhs + ); } - registerResourceCloserOnAbnormalExit(currentSubTaskHolder); - return compactionStrategy.runCompactionTasks(this, toolbox, intervalDataSchemas); + return compactionRunner.runCompactionTasks(this, intervalDataSchemas, toolbox); } /** @@ -1041,39 +1018,9 @@ void checkSegments(LockGranularity lockGranularityInUse, List lates } } - @VisibleForTesting - static class PartitionConfigurationManager - { - @Nullable - private final CompactionTuningConfig tuningConfig; - - PartitionConfigurationManager(@Nullable CompactionTuningConfig tuningConfig) - { - this.tuningConfig = tuningConfig; - } - - @Nullable - CompactionTuningConfig computeTuningConfig() - { - CompactionTuningConfig newTuningConfig = tuningConfig == null - ? CompactionTuningConfig.defaultConfig() - : tuningConfig; - PartitionsSpec partitionsSpec = newTuningConfig.getGivenOrDefaultPartitionsSpec(); - if (partitionsSpec instanceof DynamicPartitionsSpec) { - final DynamicPartitionsSpec dynamicPartitionsSpec = (DynamicPartitionsSpec) partitionsSpec; - partitionsSpec = new DynamicPartitionsSpec( - dynamicPartitionsSpec.getMaxRowsPerSegment(), - dynamicPartitionsSpec.getMaxTotalRowsOr(DynamicPartitionsSpec.DEFAULT_COMPACTION_MAX_TOTAL_ROWS) - ); - } - return newTuningConfig.withPartitionsSpec(partitionsSpec); - } - } - public static class Builder { private final String dataSource; - private final SegmentCacheManagerFactory segmentCacheManagerFactory; private final RetryPolicyFactory retryPolicyFactory; private CompactionIOConfig ioConfig; @@ -1091,16 +1038,14 @@ public static class Builder private TuningConfig tuningConfig; @Nullable private Map context; - private CompactionStrategy compactionStrategy; + private CompactionRunner compactionRunner; public Builder( String dataSource, - SegmentCacheManagerFactory segmentCacheManagerFactory, RetryPolicyFactory retryPolicyFactory ) { this.dataSource = dataSource; - this.segmentCacheManagerFactory = segmentCacheManagerFactory; this.retryPolicyFactory = retryPolicyFactory; } @@ -1174,9 +1119,9 @@ public Builder context(Map context) return this; } - public Builder compactionStrategy(CompactionStrategy compactionStrategy) + public Builder compactionRunner(CompactionRunner compactionRunner) { - this.compactionStrategy = compactionStrategy; + this.compactionRunner = compactionRunner; return this; } @@ -1197,8 +1142,7 @@ public CompactionTask build() granularitySpec, tuningConfig, context, - compactionStrategy, - segmentCacheManagerFactory + compactionRunner ); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionStrategy.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java similarity index 77% rename from indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionStrategy.java rename to indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java index ec330ab17a8b..75a6e8c2e6f9 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionStrategy.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java @@ -19,11 +19,14 @@ package org.apache.druid.indexing.common.task; -import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.core.JsonProcessingException; import com.google.common.annotations.VisibleForTesting; import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; +import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexer.report.TaskReport; import org.apache.druid.indexing.common.SegmentCacheManagerFactory; import org.apache.druid.indexing.common.TaskToolbox; @@ -34,6 +37,7 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.NonnullPair; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.logger.Logger; @@ -43,6 +47,7 @@ import org.codehaus.jackson.annotate.JsonCreator; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -50,33 +55,44 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; -public class NativeCompactionStrategy implements CompactionStrategy +public class NativeCompactionRunner implements CompactionRunner { - private static final Logger log = new Logger(NativeCompactionStrategy.class); - public static final String TYPE = "NATIVE"; + private static final Logger log = new Logger(NativeCompactionRunner.class); + public static final String type = "NATIVE"; private static final boolean STORE_COMPACTION_STATE = true; + @JsonIgnore + private final SegmentCacheManagerFactory segmentCacheManagerFactory; + @JsonIgnore + private final CurrentSubTaskHolder currentSubTaskHolder = new CurrentSubTaskHolder( + (taskObject, config) -> { + final ParallelIndexSupervisorTask indexTask = (ParallelIndexSupervisorTask) taskObject; + indexTask.stopGracefully(config); + }); @JsonCreator - public NativeCompactionStrategy() + public NativeCompactionRunner(@JacksonInject SegmentCacheManagerFactory segmentCacheManagerFactory) { + this.segmentCacheManagerFactory = segmentCacheManagerFactory; } @Override public CurrentSubTaskHolder getCurrentSubTaskHolder() { - return new CurrentSubTaskHolder( - (taskObject, config) -> { - final ParallelIndexSupervisorTask indexTask = (ParallelIndexSupervisorTask) taskObject; - indexTask.stopGracefully(config); - } - ); + return currentSubTaskHolder; + } + + @Override + public Pair supportsCompactionConfig( + CompactionTask compactionTask + ) + { + return Pair.of(true, null); } @Override - @JsonProperty("TYPE") public String getType() { - return TYPE; + return type; } /** @@ -89,7 +105,7 @@ static List createIngestionSpecs( List> dataschemas, final TaskToolbox toolbox, final CompactionIOConfig ioConfig, - final CompactionTask.PartitionConfigurationManager partitionConfigurationManager, + final PartitionConfigurationManager partitionConfigurationManager, final CoordinatorClient coordinatorClient, final SegmentCacheManagerFactory segmentCacheManagerFactory ) @@ -167,17 +183,21 @@ private static ParallelIndexIOConfig createIoConfig( @Override public TaskStatus runCompactionTasks( CompactionTask compactionTask, - TaskToolbox taskToolbox, - List> dataSchemas - ) throws JsonProcessingException + List> dataSchemas, + TaskToolbox taskToolbox + ) throws Exception { + final PartitionConfigurationManager partitionConfigurationManager = + new NativeCompactionRunner.PartitionConfigurationManager(compactionTask.getTuningConfig()); + + final List ingestionSpecs = createIngestionSpecs( dataSchemas, taskToolbox, compactionTask.getIoConfig(), - compactionTask.getPartitionConfigurationManager(), + partitionConfigurationManager, taskToolbox.getCoordinatorClient(), - compactionTask.getSegmentCacheManagerFactory() + segmentCacheManagerFactory ); List subtasks = IntStream @@ -203,7 +223,7 @@ public TaskStatus runCompactionTasks( return runParallelIndexSubtasks( subtasks, taskToolbox, - compactionTask.getCurrentSubTaskHolder(), + currentSubTaskHolder, compactionTask.getId() ); } @@ -288,4 +308,33 @@ Map createContextForSubtask(CompactionTask compactionTask) newContext.put(Tasks.PRIORITY_KEY, compactionTask.getPriority()); return newContext; } + + @VisibleForTesting + static class PartitionConfigurationManager + { + @Nullable + private final CompactionTask.CompactionTuningConfig tuningConfig; + + PartitionConfigurationManager(@Nullable CompactionTask.CompactionTuningConfig tuningConfig) + { + this.tuningConfig = tuningConfig; + } + + @Nullable + CompactionTask.CompactionTuningConfig computeTuningConfig() + { + CompactionTask.CompactionTuningConfig newTuningConfig = tuningConfig == null + ? CompactionTask.CompactionTuningConfig.defaultConfig() + : tuningConfig; + PartitionsSpec partitionsSpec = newTuningConfig.getGivenOrDefaultPartitionsSpec(); + if (partitionsSpec instanceof DynamicPartitionsSpec) { + final DynamicPartitionsSpec dynamicPartitionsSpec = (DynamicPartitionsSpec) partitionsSpec; + partitionsSpec = new DynamicPartitionsSpec( + dynamicPartitionsSpec.getMaxRowsPerSegment(), + dynamicPartitionsSpec.getMaxTotalRowsOr(DynamicPartitionsSpec.DEFAULT_COMPACTION_MAX_TOTAL_ROWS) + ); + } + return newTuningConfig.withPartitionsSpec(partitionsSpec); + } + } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java index a4506ffaa6b6..ba83d88ed865 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java @@ -64,7 +64,7 @@ import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; -import org.apache.druid.server.coordinator.UserCompactionStrategy; +import org.apache.druid.server.coordinator.ClientCompactionRunnerInfo; import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.server.security.AuthorizerMapper; import org.joda.time.Duration; @@ -130,7 +130,7 @@ public void testClientCompactionTaskQueryToCompactionTask() throws IOException new AggregatorFactory[] {new CountAggregatorFactory("cnt")}, new ClientCompactionTaskTransformSpec(new SelectorDimFilter("dim1", "foo", null)), ImmutableMap.of("key", "value"), - new UserCompactionStrategy(CompactionEngine.NATIVE) + new ClientCompactionRunnerInfo(CompactionEngine.NATIVE) ); final byte[] json = mapper.writeValueAsBytes(query); @@ -237,8 +237,8 @@ public void testClientCompactionTaskQueryToCompactionTask() throws IOException task.getMetricsSpec() ); Assert.assertEquals( - query.getCompactionStrategy().getType().toString(), - task.getCompactionStrategy().getType().toString() + query.getClientCompactionRunnerInfo().getType().toString(), + task.getCompactionRunner().getType().toString() ); } @@ -248,7 +248,6 @@ public void testCompactionTaskToClientCompactionTaskQuery() throws IOException final ObjectMapper mapper = setupInjectablesInObjectMapper(new DefaultObjectMapper()); final CompactionTask.Builder builder = new CompactionTask.Builder( "datasource", - new SegmentCacheManagerFactory(mapper), new RetryPolicyFactory(new RetryPolicyConfig()) ); final CompactionTask task = builder @@ -306,7 +305,9 @@ public void testCompactionTaskToClientCompactionTaskQuery() throws IOException ) .metricsSpec(new AggregatorFactory[] {new CountAggregatorFactory("cnt")}) .transformSpec(new ClientCompactionTaskTransformSpec(new SelectorDimFilter("dim1", "foo", null))) - .compactionStrategy(new NativeCompactionStrategy()) + .compactionRunner(new NativeCompactionRunner( + new SegmentCacheManagerFactory(new TestUtils().getTestObjectMapper()) + )) .build(); final ClientCompactionTaskQuery expected = new ClientCompactionTaskQuery( @@ -354,7 +355,7 @@ public void testCompactionTaskToClientCompactionTaskQuery() throws IOException new AggregatorFactory[] {new CountAggregatorFactory("cnt")}, new ClientCompactionTaskTransformSpec(new SelectorDimFilter("dim1", "foo", null)), new HashMap<>(), - new UserCompactionStrategy(CompactionEngine.NATIVE) + new ClientCompactionRunnerInfo(CompactionEngine.NATIVE) ); final byte[] json = mapper.writeValueAsBytes(task); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java index ba9a6e3e2be2..54d448485001 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java @@ -32,6 +32,7 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; @@ -160,7 +161,6 @@ public void testRunParallelWithDynamicPartitioningMatchCompactionState() throws final Builder builder = new Builder( DATA_SOURCE, - getSegmentCacheManagerFactory(), RETRY_POLICY_FACTORY ); final CompactionTask compactionTask = builder @@ -185,7 +185,7 @@ public void testRunParallelWithDynamicPartitioningMatchCompactionState() throws CompactionState expectedState = new CompactionState( new DynamicPartitionsSpec(null, Long.MAX_VALUE), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), - ImmutableList.of(expectedLongSumMetric), + Collections.emptyMap(), ImmutableList.of(expectedLongSumMetric), null, compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), getObjectMapper().readValue( @@ -198,7 +198,8 @@ public void testRunParallelWithDynamicPartitioningMatchCompactionState() throws ) ), Map.class - ) + ), + CompactionEngine.NATIVE ); Assert.assertEquals("Compaction state for " + segment.getId(), expectedState, segment.getLastCompactionState()); } @@ -215,7 +216,6 @@ public void testRunParallelWithHashPartitioningMatchCompactionState() throws Exc final Builder builder = new Builder( DATA_SOURCE, - getSegmentCacheManagerFactory(), RETRY_POLICY_FACTORY ); final CompactionTask compactionTask = builder @@ -236,7 +236,7 @@ public void testRunParallelWithHashPartitioningMatchCompactionState() throws Exc CompactionState expectedState = new CompactionState( new HashedPartitionsSpec(null, 3, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), - ImmutableList.of(expectedLongSumMetric), + Collections.emptyMap(), ImmutableList.of(expectedLongSumMetric), null, compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), getObjectMapper().readValue( @@ -249,7 +249,8 @@ public void testRunParallelWithHashPartitioningMatchCompactionState() throws Exc ) ), Map.class - ) + ), + CompactionEngine.NATIVE ); Assert.assertEquals("Compaction state for " + segment.getId(), expectedState, segment.getLastCompactionState()); } @@ -281,7 +282,6 @@ public void testRunParallelWithRangePartitioning() throws Exception final Builder builder = new Builder( DATA_SOURCE, - getSegmentCacheManagerFactory(), RETRY_POLICY_FACTORY ); final CompactionTask compactionTask = builder @@ -302,7 +302,7 @@ public void testRunParallelWithRangePartitioning() throws Exception CompactionState expectedState = new CompactionState( new SingleDimensionPartitionsSpec(7, null, "dim", false), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), - ImmutableList.of(expectedLongSumMetric), + Collections.emptyMap(), ImmutableList.of(expectedLongSumMetric), null, compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), getObjectMapper().readValue( @@ -315,7 +315,8 @@ public void testRunParallelWithRangePartitioning() throws Exception ) ), Map.class - ) + ), + CompactionEngine.NATIVE ); Assert.assertEquals("Compaction state for " + segment.getId(), expectedState, segment.getLastCompactionState()); } @@ -332,7 +333,6 @@ public void testRunParallelWithRangePartitioningAndNoUpfrontSegmentFetching() th final Builder builder = new Builder( DATA_SOURCE, - getSegmentCacheManagerFactory(), RETRY_POLICY_FACTORY ); @@ -363,7 +363,7 @@ public void testRunParallelWithRangePartitioningAndNoUpfrontSegmentFetching() th CompactionState expectedState = new CompactionState( new SingleDimensionPartitionsSpec(7, null, "dim", false), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), - ImmutableList.of(expectedLongSumMetric), + Collections.emptyMap(), ImmutableList.of(expectedLongSumMetric), null, compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), getObjectMapper().readValue( @@ -378,7 +378,8 @@ public void testRunParallelWithRangePartitioningAndNoUpfrontSegmentFetching() th ) ), Map.class - ) + ), + CompactionEngine.NATIVE ); Assert.assertEquals("Compaction state for " + segment.getId(), expectedState, segment.getLastCompactionState()); } @@ -395,7 +396,6 @@ public void testRunParallelWithMultiDimensionRangePartitioning() throws Exceptio final Builder builder = new Builder( DATA_SOURCE, - getSegmentCacheManagerFactory(), RETRY_POLICY_FACTORY ); final CompactionTask compactionTask = builder @@ -419,7 +419,7 @@ public void testRunParallelWithMultiDimensionRangePartitioning() throws Exceptio CompactionState expectedState = new CompactionState( new DimensionRangePartitionsSpec(7, null, Arrays.asList("dim1", "dim2"), false), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), - ImmutableList.of(expectedLongSumMetric), + Collections.emptyMap(), ImmutableList.of(expectedLongSumMetric), null, compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), getObjectMapper().readValue( @@ -432,7 +432,8 @@ public void testRunParallelWithMultiDimensionRangePartitioning() throws Exceptio ) ), Map.class - ) + ), + CompactionEngine.NATIVE ); Assert.assertEquals("Compaction state for " + segment.getId(), expectedState, segment.getLastCompactionState()); } @@ -449,7 +450,6 @@ public void testRunParallelWithRangePartitioningWithSingleTask() throws Exceptio final Builder builder = new Builder( DATA_SOURCE, - getSegmentCacheManagerFactory(), RETRY_POLICY_FACTORY ); final CompactionTask compactionTask = builder @@ -470,7 +470,7 @@ public void testRunParallelWithRangePartitioningWithSingleTask() throws Exceptio CompactionState expectedState = new CompactionState( new SingleDimensionPartitionsSpec(7, null, "dim", false), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), - ImmutableList.of(expectedLongSumMetric), + Collections.emptyMap(), ImmutableList.of(expectedLongSumMetric), null, compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), getObjectMapper().readValue( @@ -483,7 +483,8 @@ public void testRunParallelWithRangePartitioningWithSingleTask() throws Exceptio ) ), Map.class - ) + ), + CompactionEngine.NATIVE ); Assert.assertEquals("Compaction state for " + segment.getId(), expectedState, segment.getLastCompactionState()); } @@ -500,7 +501,6 @@ public void testRunParallelWithMultiDimensionRangePartitioningWithSingleTask() t final Builder builder = new Builder( DATA_SOURCE, - getSegmentCacheManagerFactory(), RETRY_POLICY_FACTORY ); final CompactionTask compactionTask = builder @@ -524,7 +524,7 @@ public void testRunParallelWithMultiDimensionRangePartitioningWithSingleTask() t CompactionState expectedState = new CompactionState( new DimensionRangePartitionsSpec(7, null, Arrays.asList("dim1", "dim2"), false), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), - ImmutableList.of(expectedLongSumMetric), + Collections.emptyMap(), ImmutableList.of(expectedLongSumMetric), null, compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), getObjectMapper().readValue( @@ -537,7 +537,8 @@ public void testRunParallelWithMultiDimensionRangePartitioningWithSingleTask() t ) ), Map.class - ) + ), + CompactionEngine.NATIVE ); Assert.assertEquals("Compaction state for " + segment.getId(), expectedState, segment.getLastCompactionState()); } @@ -551,7 +552,6 @@ public void testRunCompactionStateNotStoreIfContextSetToFalse() final Builder builder = new Builder( DATA_SOURCE, - getSegmentCacheManagerFactory(), RETRY_POLICY_FACTORY ); final CompactionTask compactionTask = builder @@ -582,7 +582,6 @@ public void testRunCompactionWithFilterShouldStoreInState() throws Exception final Builder builder = new Builder( DATA_SOURCE, - getSegmentCacheManagerFactory(), RETRY_POLICY_FACTORY ); final CompactionTask compactionTask = builder @@ -609,7 +608,7 @@ public void testRunCompactionWithFilterShouldStoreInState() throws Exception CompactionState expectedState = new CompactionState( new DynamicPartitionsSpec(null, Long.MAX_VALUE), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), - ImmutableList.of(expectedLongSumMetric), + Collections.emptyMap(), ImmutableList.of(expectedLongSumMetric), getObjectMapper().readValue( getObjectMapper().writeValueAsString(compactionTask.getTransformSpec()), Map.class @@ -625,7 +624,8 @@ public void testRunCompactionWithFilterShouldStoreInState() throws Exception ) ), Map.class - ) + ), + CompactionEngine.NATIVE ); Assert.assertEquals("Compaction state for " + segment.getId(), expectedState, segment.getLastCompactionState()); } @@ -639,7 +639,6 @@ public void testRunCompactionWithNewMetricsShouldStoreInState() throws Exception final Builder builder = new Builder( DATA_SOURCE, - getSegmentCacheManagerFactory(), RETRY_POLICY_FACTORY ); final CompactionTask compactionTask = builder @@ -672,7 +671,7 @@ public void testRunCompactionWithNewMetricsShouldStoreInState() throws Exception CompactionState expectedState = new CompactionState( new DynamicPartitionsSpec(null, Long.MAX_VALUE), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), - ImmutableList.of(expectedCountMetric, expectedLongSumMetric), + Collections.emptyMap(), ImmutableList.of(expectedCountMetric, expectedLongSumMetric), getObjectMapper().readValue( getObjectMapper().writeValueAsString(compactionTask.getTransformSpec()), Map.class @@ -688,7 +687,8 @@ public void testRunCompactionWithNewMetricsShouldStoreInState() throws Exception ) ), Map.class - ) + ), + CompactionEngine.NATIVE ); Assert.assertEquals("Compaction state for " + segment.getId(), expectedState, segment.getLastCompactionState()); } @@ -702,7 +702,6 @@ public void testCompactHashAndDynamicPartitionedSegments() runIndexTask(null, true); final Builder builder = new Builder( DATA_SOURCE, - getSegmentCacheManagerFactory(), RETRY_POLICY_FACTORY ); final CompactionTask compactionTask = builder @@ -752,7 +751,6 @@ public void testCompactRangeAndDynamicPartitionedSegments() runIndexTask(null, true); final Builder builder = new Builder( DATA_SOURCE, - getSegmentCacheManagerFactory(), RETRY_POLICY_FACTORY ); final CompactionTask compactionTask = builder @@ -844,7 +842,6 @@ public void testCompactionDropSegmentsOfInputIntervalIfDropFlagIsSet() throws Ex final Builder builder = new Builder( DATA_SOURCE, - getSegmentCacheManagerFactory(), RETRY_POLICY_FACTORY ); final CompactionTask compactionTask = builder @@ -891,7 +888,6 @@ public void testCompactionDoesNotDropSegmentsIfDropFlagNotSet() throws Exception final Builder builder = new Builder( DATA_SOURCE, - getSegmentCacheManagerFactory(), RETRY_POLICY_FACTORY ); final CompactionTask compactionTask = builder diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index 2893ef476a63..62b86de12824 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -38,6 +38,7 @@ import org.apache.druid.data.input.impl.ParseSpec; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; @@ -242,7 +243,7 @@ public static CompactionState getDefaultCompactionState( return new CompactionState( new DynamicPartitionsSpec(5000000, Long.MAX_VALUE), expectedDims, - ImmutableList.of(expectedMetric), + Collections.emptyMap(), ImmutableList.of(expectedMetric), null, IndexSpec.DEFAULT.asMap(mapper), mapper.readValue( @@ -255,7 +256,8 @@ public static CompactionState getDefaultCompactionState( ) ), Map.class - ) + ), + CompactionEngine.NATIVE ); } @@ -281,7 +283,6 @@ public void testRunWithDynamicPartitioning() throws Exception final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); @@ -349,7 +350,6 @@ public void testRunWithHashPartitioning() throws Exception final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); @@ -416,7 +416,7 @@ public void testRunWithHashPartitioning() throws Exception CompactionState expectedState = new CompactionState( new HashedPartitionsSpec(null, 3, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), - ImmutableList.of(expectedLongSumMetric), + Collections.emptyMap(), ImmutableList.of(expectedLongSumMetric), null, compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), getObjectMapper().readValue( @@ -429,7 +429,8 @@ public void testRunWithHashPartitioning() throws Exception ) ), Map.class - ) + ), + CompactionEngine.NATIVE ); Assert.assertEquals(expectedState, segments.get(segmentIdx).getLastCompactionState()); Assert.assertSame(HashBasedNumberedShardSpec.class, segments.get(segmentIdx).getShardSpec().getClass()); @@ -449,7 +450,6 @@ public void testRunCompactionTwice() throws Exception final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); @@ -544,7 +544,6 @@ public void testRunIndexAndCompactAtTheSameTimeForDifferentInterval() throws Exc final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); @@ -656,7 +655,6 @@ public void testWithSegmentGranularity() throws Exception final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); @@ -726,7 +724,6 @@ public void testWithSegmentGranularityMisalignedInterval() throws Exception final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); @@ -761,7 +758,6 @@ public void testWithSegmentGranularityMisalignedIntervalAllowed() throws Excepti final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); @@ -806,7 +802,6 @@ public void testCompactionWithFilterInTransformSpec() throws Exception final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); @@ -837,7 +832,7 @@ public void testCompactionWithFilterInTransformSpec() throws Exception CompactionState expectedCompactionState = new CompactionState( new DynamicPartitionsSpec(5000000, Long.MAX_VALUE), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), - ImmutableList.of(expectedLongSumMetric), + Collections.emptyMap(), ImmutableList.of(expectedLongSumMetric), getObjectMapper().readValue(getObjectMapper().writeValueAsString(compactionTask.getTransformSpec()), Map.class), IndexSpec.DEFAULT.asMap(mapper), mapper.readValue( @@ -850,7 +845,8 @@ public void testCompactionWithFilterInTransformSpec() throws Exception ) ), Map.class - ) + ), + CompactionEngine.NATIVE ); Assert.assertEquals( expectedCompactionState, @@ -866,7 +862,6 @@ public void testCompactionWithNewMetricInMetricsSpec() throws Exception final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); @@ -903,7 +898,7 @@ public void testCompactionWithNewMetricInMetricsSpec() throws Exception CompactionState expectedCompactionState = new CompactionState( new DynamicPartitionsSpec(5000000, Long.MAX_VALUE), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), - ImmutableList.of(expectedCountMetric, expectedLongSumMetric), + Collections.emptyMap(), ImmutableList.of(expectedCountMetric, expectedLongSumMetric), getObjectMapper().readValue(getObjectMapper().writeValueAsString(compactionTask.getTransformSpec()), Map.class), IndexSpec.DEFAULT.asMap(mapper), mapper.readValue( @@ -916,7 +911,8 @@ public void testCompactionWithNewMetricInMetricsSpec() throws Exception ) ), Map.class - ) + ), + CompactionEngine.NATIVE ); Assert.assertEquals( expectedCompactionState, @@ -932,7 +928,6 @@ public void testWithGranularitySpecNonNullSegmentGranularityAndNullQueryGranular final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); @@ -1001,7 +996,6 @@ public void testWithGranularitySpecNonNullQueryGranularityAndNullSegmentGranular final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); @@ -1055,7 +1049,6 @@ public void testWithGranularitySpecNonNullQueryGranularityAndNonNullSegmentGranu final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); @@ -1094,7 +1087,6 @@ public void testWithGranularitySpecNullQueryGranularityAndNullSegmentGranularity final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); @@ -1147,7 +1139,6 @@ public void testCompactThenAppend() throws Exception final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); @@ -1209,7 +1200,6 @@ public void testPartialIntervalCompactWithFinerSegmentGranularityThenFullInterva final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); @@ -1365,7 +1355,6 @@ public void testCompactDatasourceOverIntervalWithOnlyTombstones() throws Excepti final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); @@ -1473,7 +1462,6 @@ public void testPartialIntervalCompactWithFinerSegmentGranularityThenFullInterva final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); @@ -1545,7 +1533,6 @@ public void testRunIndexAndCompactForSameSegmentAtTheSameTime() throws Exception final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); @@ -1600,7 +1587,6 @@ public void testRunIndexAndCompactForSameSegmentAtTheSameTime2() throws Exceptio final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); @@ -1691,7 +1677,6 @@ public void testRunWithSpatialDimensions() throws Exception final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); @@ -1823,7 +1808,6 @@ public void testRunWithAutoCastDimensions() throws Exception final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index e976d2a88210..aa0164898f53 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -69,7 +69,7 @@ import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.config.TaskConfigBuilder; import org.apache.druid.indexing.common.task.CompactionTask.Builder; -import org.apache.druid.indexing.common.task.CompactionTask.PartitionConfigurationManager; +import org.apache.druid.indexing.common.task.NativeCompactionRunner.PartitionConfigurationManager; import org.apache.druid.indexing.common.task.CompactionTask.SegmentProvider; import org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexIOConfig; @@ -399,7 +399,6 @@ public void testCreateCompactionTaskWithGranularitySpec() { final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); builder.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); @@ -409,7 +408,6 @@ public void testCreateCompactionTaskWithGranularitySpec() final Builder builder2 = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); builder2.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); @@ -427,7 +425,6 @@ public void testCompactionTaskEmitter() { final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); builder.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); @@ -448,7 +445,6 @@ public void testCreateCompactionTaskWithConflictingGranularitySpecAndSegmentGran { final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); builder.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); @@ -479,7 +475,6 @@ public void testCreateCompactionTaskWithTransformSpec() new ClientCompactionTaskTransformSpec(new SelectorDimFilter("dim1", "foo", null)); final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); builder.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); @@ -498,7 +493,6 @@ public void testCreateCompactionTaskWithMetricsSpec() AggregatorFactory[] aggregatorFactories = new AggregatorFactory[]{new CountAggregatorFactory("cnt")}; final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); builder.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); @@ -516,7 +510,6 @@ public void testCreateCompactionTaskWithNullSegmentGranularityInGranularitySpecA { final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); builder.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); @@ -545,7 +538,6 @@ public void testCreateCompactionTaskWithSameGranularitySpecAndSegmentGranularity { final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); builder.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); @@ -561,7 +553,6 @@ public void testSerdeWithInterval() throws IOException { final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); final CompactionTask task = builder @@ -582,7 +573,6 @@ public void testSerdeWithSegments() throws IOException { final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); final CompactionTask task = builder @@ -601,7 +591,6 @@ public void testSerdeWithDimensions() throws IOException { final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); @@ -684,7 +673,6 @@ public void testSerdeWithOldTuningConfigSuccessfullyDeserializeToNewOne() throws final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); @@ -705,7 +693,6 @@ public void testInputSourceResources() { final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); final CompactionTask task = builder @@ -966,7 +953,7 @@ public void testCreateIngestionSchema() throws IOException METRIC_BUILDER ); - final List ingestionSpecs = NativeCompactionStrategy.createIngestionSpecs( + final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( dataSchemasForIntervals, toolbox, new CompactionIOConfig(null, false, null), @@ -1048,7 +1035,7 @@ public void testCreateIngestionSchemaWithTargetPartitionSize() throws IOExceptio METRIC_BUILDER ); - final List ingestionSpecs = NativeCompactionStrategy.createIngestionSpecs( + final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( dataSchemasForIntervals, toolbox, new CompactionIOConfig(null, false, null), @@ -1131,7 +1118,7 @@ public void testCreateIngestionSchemaWithMaxTotalRows() throws IOException METRIC_BUILDER ); - final List ingestionSpecs = NativeCompactionStrategy.createIngestionSpecs( + final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( dataSchemasForIntervals, toolbox, new CompactionIOConfig(null, false, null), @@ -1214,7 +1201,7 @@ public void testCreateIngestionSchemaWithNumShards() throws IOException METRIC_BUILDER ); - final List ingestionSpecs = NativeCompactionStrategy.createIngestionSpecs( + final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( dataSchemasForIntervals, toolbox, new CompactionIOConfig(null, false, null), @@ -1285,7 +1272,7 @@ public void testCreateIngestionSchemaWithCustomDimensionsSpec() throws IOExcepti METRIC_BUILDER ); - final List ingestionSpecs = NativeCompactionStrategy.createIngestionSpecs( + final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( dataSchemasForIntervals, toolbox, new CompactionIOConfig(null, false, null), @@ -1336,7 +1323,7 @@ public void testCreateIngestionSchemaWithCustomMetricsSpec() throws IOException METRIC_BUILDER ); - final List ingestionSpecs = NativeCompactionStrategy.createIngestionSpecs( + final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( dataSchemasForIntervals, toolbox, new CompactionIOConfig(null, false, null), @@ -1380,7 +1367,7 @@ public void testCreateIngestionSchemaWithCustomSegments() throws IOException METRIC_BUILDER ); - final List ingestionSpecs = NativeCompactionStrategy.createIngestionSpecs( + final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( dataSchemasForIntervals, toolbox, new CompactionIOConfig(null, false, null), @@ -1431,7 +1418,7 @@ public void testCreateIngestionSchemaWithDifferentSegmentSet() throws IOExceptio METRIC_BUILDER ); - NativeCompactionStrategy.createIngestionSpecs( + NativeCompactionRunner.createIngestionSpecs( dataSchemasForIntervals, toolbox, new CompactionIOConfig(null, false, null), @@ -1462,7 +1449,7 @@ public void testMissingMetadata() throws IOException METRIC_BUILDER ); - NativeCompactionStrategy.createIngestionSpecs( + NativeCompactionRunner.createIngestionSpecs( dataSchemasForIntervals, toolbox, new CompactionIOConfig(null, false, null), @@ -1480,7 +1467,6 @@ public void testEmptyInterval() final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); @@ -1505,7 +1491,7 @@ public void testSegmentGranularityAndNullQueryGranularity() throws IOException METRIC_BUILDER ); - final List ingestionSpecs = NativeCompactionStrategy.createIngestionSpecs( + final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( dataSchemasForIntervals, toolbox, new CompactionIOConfig(null, false, null), @@ -1549,7 +1535,7 @@ public void testQueryGranularityAndNullSegmentGranularity() throws IOException new ClientCompactionTaskGranularitySpec(null, new PeriodGranularity(Period.months(3), null, null), null), METRIC_BUILDER ); - final List ingestionSpecs = NativeCompactionStrategy.createIngestionSpecs( + final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( dataSchemasForIntervals, toolbox, new CompactionIOConfig(null, false, null), @@ -1596,7 +1582,7 @@ public void testQueryGranularityAndSegmentGranularityNonNull() throws IOExceptio METRIC_BUILDER ); - final List ingestionSpecs = NativeCompactionStrategy.createIngestionSpecs( + final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( dataSchemasForIntervals, toolbox, new CompactionIOConfig(null, false, null), @@ -1644,7 +1630,7 @@ public void testNullGranularitySpec() throws IOException METRIC_BUILDER ); - final List ingestionSpecs = NativeCompactionStrategy.createIngestionSpecs( + final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( dataSchemasForIntervals, toolbox, new CompactionIOConfig(null, false, null), @@ -1690,7 +1676,7 @@ public void testGranularitySpecWithNullQueryGranularityAndNullSegmentGranularity METRIC_BUILDER ); - final List ingestionSpecs = NativeCompactionStrategy.createIngestionSpecs( + final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( dataSchemasForIntervals, toolbox, new CompactionIOConfig(null, false, null), @@ -1736,7 +1722,7 @@ public void testGranularitySpecWithNotNullRollup() METRIC_BUILDER ); - final List ingestionSpecs = NativeCompactionStrategy.createIngestionSpecs( + final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( dataSchemasForIntervals, toolbox, new CompactionIOConfig(null, false, null), @@ -1767,7 +1753,7 @@ public void testGranularitySpecWithNullRollup() METRIC_BUILDER ); - final List ingestionSpecs = NativeCompactionStrategy.createIngestionSpecs( + final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( dataSchemasForIntervals, toolbox, new CompactionIOConfig(null, false, null), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java index a14d11d6f784..241e5f2de081 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java @@ -241,7 +241,6 @@ private Builder newCompactionTaskBuilder() { return new Builder( DATASOURCE, - getSegmentCacheManagerFactory(), RETRY_POLICY_FACTORY ); } diff --git a/processing/src/main/java/org/apache/druid/indexer/CompactionEngine.java b/processing/src/main/java/org/apache/druid/indexer/CompactionEngine.java index 5cc39bf4bf0b..4713542c7b28 100644 --- a/processing/src/main/java/org/apache/druid/indexer/CompactionEngine.java +++ b/processing/src/main/java/org/apache/druid/indexer/CompactionEngine.java @@ -24,7 +24,7 @@ /** * Encapsulates the Engine to be used for a compaction task. - * Should be synchronized with the subtypes for {@link org.apache.druid.indexing.common.task.CompactionStrategy}. + * Should be kept in sync with the subtypes for {@link org.apache.druid.indexing.common.task.CompactionRunner}. */ public enum CompactionEngine { @@ -34,9 +34,6 @@ public enum CompactionEngine @JsonCreator public static CompactionEngine fromString(String name) { - if (name == null) { - return null; - } - return valueOf(StringUtils.toUpperCase(name)); + return name == null ? null : valueOf(StringUtils.toUpperCase(name)); } } diff --git a/processing/src/main/java/org/apache/druid/timeline/CompactionState.java b/processing/src/main/java/org/apache/druid/timeline/CompactionState.java index 24344d9cd822..3fb21a4a7289 100644 --- a/processing/src/main/java/org/apache/druid/timeline/CompactionState.java +++ b/processing/src/main/java/org/apache/druid/timeline/CompactionState.java @@ -22,8 +22,11 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.query.aggregation.AggregatorFactory; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Objects; @@ -58,15 +61,20 @@ public class CompactionState // org.apache.druid.query.aggregation.AggregatorFactory cannot be used here because it's in the 'processing' module which // has a dependency on the 'core' module where this class is. private final List metricsSpec; + private final CompactionEngine engine; + + private final Map dimensionToAggregatoryFactoryMap; @JsonCreator public CompactionState( @JsonProperty("partitionsSpec") PartitionsSpec partitionsSpec, @JsonProperty("dimensionsSpec") DimensionsSpec dimensionsSpec, + @JsonProperty("dimensionToAggregatoryFactoryMap") Map dimensionToAggregatoryFactoryMap, @JsonProperty("metricsSpec") List metricsSpec, @JsonProperty("transformSpec") Map transformSpec, @JsonProperty("indexSpec") Map indexSpec, - @JsonProperty("granularitySpec") Map granularitySpec + @JsonProperty("granularitySpec") Map granularitySpec, + @JsonProperty("engine") CompactionEngine engine ) { this.partitionsSpec = partitionsSpec; @@ -75,6 +83,8 @@ public CompactionState( this.transformSpec = transformSpec; this.indexSpec = indexSpec; this.granularitySpec = granularitySpec; + this.engine = engine; + this.dimensionToAggregatoryFactoryMap = dimensionToAggregatoryFactoryMap; } @JsonProperty @@ -113,6 +123,18 @@ public Map getGranularitySpec() return granularitySpec; } + @JsonProperty + public CompactionEngine getEngine() + { + return engine; + } + + @JsonProperty + public Map getDimensionToAggregatoryFactoryMap() + { + return dimensionToAggregatoryFactoryMap; + } + @Override public boolean equals(Object o) { @@ -160,19 +182,23 @@ public String toString() public static Function, Set> addCompactionStateToSegments( PartitionsSpec partitionsSpec, DimensionsSpec dimensionsSpec, + Map dimensionToAggregatoryFactoryMap, List metricsSpec, Map transformSpec, Map indexSpec, - Map granularitySpec + Map granularitySpec, + CompactionEngine engine ) { CompactionState compactionState = new CompactionState( partitionsSpec, dimensionsSpec, + dimensionToAggregatoryFactoryMap, metricsSpec, transformSpec, indexSpec, - granularitySpec + granularitySpec, + engine ); return segments -> segments diff --git a/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java b/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java index 370a16591270..36e0637afe1f 100644 --- a/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java +++ b/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java @@ -26,6 +26,7 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.RangeSet; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.jackson.DefaultObjectMapper; @@ -129,10 +130,11 @@ public void testV1Serialization() throws Exception new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "bar", "foo")) ), - ImmutableList.of(ImmutableMap.of("type", "count", "name", "count")), + Collections.emptyMap(), ImmutableList.of(ImmutableMap.of("type", "count", "name", "count")), ImmutableMap.of("filter", ImmutableMap.of("type", "selector", "dimension", "dim1", "value", "foo")), ImmutableMap.of(), - ImmutableMap.of() + ImmutableMap.of(), + CompactionEngine.NATIVE ), TEST_VERSION, 1 @@ -194,10 +196,11 @@ public void testDeserializationDataSegmentLastCompactionStateWithNullSpecs() thr new CompactionState( new HashedPartitionsSpec(100000, null, ImmutableList.of("dim1")), null, + Collections.emptyMap(), null, null, - null, ImmutableMap.of(), - ImmutableMap.of() + ImmutableMap.of(), + CompactionEngine.NATIVE ), TEST_VERSION, 1 @@ -345,10 +348,11 @@ public void testWithLastCompactionState() final CompactionState compactionState = new CompactionState( new DynamicPartitionsSpec(null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar", "foo"))), - ImmutableList.of(ImmutableMap.of("type", "count", "name", "count")), + Collections.emptyMap(), ImmutableList.of(ImmutableMap.of("type", "count", "name", "count")), ImmutableMap.of("filter", ImmutableMap.of("type", "selector", "dimension", "dim1", "value", "foo")), Collections.singletonMap("test", "map"), - Collections.singletonMap("test2", "map2") + Collections.singletonMap("test2", "map2"), + CompactionEngine.NATIVE ); final DataSegment segment1 = DataSegment.builder() .dataSource("foo") @@ -387,20 +391,23 @@ public void testAnnotateWithLastCompactionState() final CompactionState compactionState = new CompactionState( dynamicPartitionsSpec, dimensionsSpec, - metricsSpec, + Collections.emptyMap(), metricsSpec, transformSpec, indexSpec, - granularitySpec + granularitySpec, + CompactionEngine.NATIVE ); final Function, Set> addCompactionStateFunction = CompactionState.addCompactionStateToSegments( dynamicPartitionsSpec, dimensionsSpec, + Collections.emptyMap(), metricsSpec, transformSpec, indexSpec, - granularitySpec + granularitySpec, + CompactionEngine.NATIVE ); final DataSegment segment1 = DataSegment.builder() diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQuery.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQuery.java index 61562dd4fe5b..dd4ebafd3919 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQuery.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQuery.java @@ -23,7 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.server.coordinator.UserCompactionStrategy; +import org.apache.druid.server.coordinator.ClientCompactionRunnerInfo; import javax.annotation.Nullable; import java.util.Arrays; @@ -47,7 +47,7 @@ public class ClientCompactionTaskQuery implements ClientTaskQuery private final AggregatorFactory[] metricsSpec; private final ClientCompactionTaskTransformSpec transformSpec; private final Map context; - private final UserCompactionStrategy compactionStrategy; + private final ClientCompactionRunnerInfo compactionRunner; @JsonCreator public ClientCompactionTaskQuery( @@ -60,7 +60,7 @@ public ClientCompactionTaskQuery( @JsonProperty("metricsSpec") AggregatorFactory[] metrics, @JsonProperty("transformSpec") ClientCompactionTaskTransformSpec transformSpec, @JsonProperty("context") Map context, - @JsonProperty("compactionStrategy") UserCompactionStrategy compactionStrategy + @JsonProperty("compactionRunner") ClientCompactionRunnerInfo compactionRunner ) { this.id = Preconditions.checkNotNull(id, "id"); @@ -72,7 +72,7 @@ public ClientCompactionTaskQuery( this.metricsSpec = metrics; this.transformSpec = transformSpec; this.context = context; - this.compactionStrategy = compactionStrategy; + this.compactionRunner = compactionRunner; } @JsonProperty @@ -139,10 +139,10 @@ public Map getContext() return context; } - @JsonProperty - public UserCompactionStrategy getCompactionStrategy() + @JsonProperty("compactionRunner") + public ClientCompactionRunnerInfo getClientCompactionRunnerInfo() { - return compactionStrategy; + return compactionRunner; } @Override @@ -164,7 +164,7 @@ public boolean equals(Object o) Arrays.equals(metricsSpec, that.metricsSpec) && Objects.equals(transformSpec, that.transformSpec) && Objects.equals(context, that.context) && - Objects.equals(compactionStrategy, that.compactionStrategy); + Objects.equals(compactionRunner, that.compactionRunner); } @Override @@ -179,7 +179,7 @@ public int hashCode() dimensionsSpec, transformSpec, context, - compactionStrategy + compactionRunner ); result = 31 * result + Arrays.hashCode(metricsSpec); return result; @@ -198,7 +198,7 @@ public String toString() ", metricsSpec=" + Arrays.toString(metricsSpec) + ", transformSpec=" + transformSpec + ", context=" + context + - ", compactionStrategy=" + compactionStrategy + + ", compactionRunner=" + compactionRunner + '}'; } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfo.java b/server/src/main/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfo.java new file mode 100644 index 000000000000..59db36751752 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfo.java @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.coordinator; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.error.InvalidInput; +import org.apache.druid.indexer.CompactionEngine; +import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; +import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; +import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.query.QueryContext; + +import java.util.Objects; + + +/** + * This class is just used to pass the strategy type via the "type" parameter for deserilization to appropriate + * {@link org.apache.druid.indexing.common.task.CompactionRunner} subtype at the overlod. + */ +public class ClientCompactionRunnerInfo +{ + private final CompactionEngine type; + + @JsonCreator + public ClientCompactionRunnerInfo(@JsonProperty("type") CompactionEngine type) + { + this.type = type; + } + +// @JsonProperty("type") + @JsonProperty + public CompactionEngine getType() + { + return type; + } + + @Override + public String toString() + { + return "ClientCompactionRunnerInfo{" + + "type=" + type + + '}'; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ClientCompactionRunnerInfo that = (ClientCompactionRunnerInfo) o; + return type == that.type; + } + + @Override + public int hashCode() + { + return Objects.hash(type); + } + + public static void supportsCompactionConfig(DataSourceCompactionConfig newConfig, String engineSource) + { + CompactionEngine compactionEngine = newConfig.getEngine(); + if (compactionEngine == CompactionEngine.MSQ) { + if (newConfig.getTuningConfig() != null) { + PartitionsSpec partitionsSpec = newConfig.getTuningConfig().getPartitionsSpec(); + + if (partitionsSpec != null && !(partitionsSpec instanceof DimensionRangePartitionsSpec + || partitionsSpec instanceof DynamicPartitionsSpec)) { + throw InvalidInput.exception( + "Invalid partition spec type[%s] for MSQ compaction engine[%s]." + + " Type must be either DynamicPartitionsSpec or DynamicRangePartitionsSpec.", + partitionsSpec.getClass(), + engineSource + ); + } + if (partitionsSpec instanceof DynamicPartitionsSpec + && ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows() != null) { + throw InvalidInput.exception( + "maxTotalRows[%d] in DynamicPartitionsSpec not supported for MSQ compaction engine[%s].", + ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows(), engineSource + ); + } + } + if (newConfig.getMetricsSpec() != null + && newConfig.getGranularitySpec() != null + && !newConfig.getGranularitySpec() + .isRollup()) { + throw InvalidInput.exception( + "rollup in granularitySpec must be set to True if metricsSpec is specifed " + + "for MSQ compaction engine[%s]."); + } + + QueryContext queryContext = QueryContext.of(newConfig.getTaskContext()); + if (!queryContext.getBoolean(MSQContext.CTX_FINALIZE_AGGREGATIONS, true)) { + throw InvalidInput.exception( + "Config[%s] cannot be set to false for auto-compaction with MSQ engine.", + MSQContext.CTX_FINALIZE_AGGREGATIONS + ); + + } + if (queryContext.getString(MSQContext.CTX_TASK_ASSIGNMENT_STRATEGY, MSQContext.TASK_ASSIGNMENT_STRATEGY_MAX) + .equals(MSQContext.TASK_ASSIGNMENT_STRATEGY_AUTO)) { + throw InvalidInput.exception( + "Config[%s] cannot be set to value[%s] for auto-compaction with MSQ engine.", + MSQContext.CTX_TASK_ASSIGNMENT_STRATEGY, + MSQContext.TASK_ASSIGNMENT_STRATEGY_AUTO + ); + } + } + } + + /** + * This class copies over MSQ context parameters from the MSQ extension. This is required to validate the submitted + * compaction config at the coordinator. The values used here should be kept in sync with those in + * {@link org.apache.druid.msq.util.MultiStageQueryContext} + */ + + private static class MSQContext { + public static final String CTX_FINALIZE_AGGREGATIONS = "finalizeAggregations"; + public static final String CTX_TASK_ASSIGNMENT_STRATEGY = "taskAssignment"; + public static final String TASK_ASSIGNMENT_STRATEGY_MAX = "MAX"; + public static final String TASK_ASSIGNMENT_STRATEGY_AUTO = "AUTO"; + } +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfig.java index 8c9214ca9264..036c53121e91 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfig.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableList; +import org.apache.druid.common.config.Configs; import org.apache.druid.indexer.CompactionEngine; import javax.annotation.Nullable; @@ -52,7 +53,8 @@ public static CoordinatorCompactionConfig from( compactionConfigs, baseConfig.compactionTaskSlotRatio, baseConfig.maxCompactionTaskSlots, - baseConfig.useAutoScaleSlots + baseConfig.useAutoScaleSlots, + null ); } @@ -67,18 +69,19 @@ public static CoordinatorCompactionConfig from( baseConfig.compactionConfigs, compactionTaskSlotRatio == null ? baseConfig.compactionTaskSlotRatio : compactionTaskSlotRatio, maxCompactionTaskSlots == null ? baseConfig.maxCompactionTaskSlots : maxCompactionTaskSlots, - useAutoScaleSlots == null ? baseConfig.useAutoScaleSlots : useAutoScaleSlots + useAutoScaleSlots == null ? baseConfig.useAutoScaleSlots : useAutoScaleSlots, + null ); } public static CoordinatorCompactionConfig from(List compactionConfigs) { - return new CoordinatorCompactionConfig(compactionConfigs, null, null, null); + return new CoordinatorCompactionConfig(compactionConfigs, null, null, null, null); } public static CoordinatorCompactionConfig empty() { - return new CoordinatorCompactionConfig(ImmutableList.of(), null, null, null); + return new CoordinatorCompactionConfig(ImmutableList.of(), null, null, null, null); } @JsonCreator @@ -86,18 +89,15 @@ public CoordinatorCompactionConfig( @JsonProperty("compactionConfigs") List compactionConfigs, @JsonProperty("compactionTaskSlotRatio") @Nullable Double compactionTaskSlotRatio, @JsonProperty("maxCompactionTaskSlots") @Nullable Integer maxCompactionTaskSlots, - @JsonProperty("useAutoScaleSlots") @Nullable Boolean useAutoScaleSlots + @JsonProperty("useAutoScaleSlots") @Nullable Boolean useAutoScaleSlots, + @JsonProperty("compactionEngine") @Nullable CompactionEngine compactionEngine ) { this.compactionConfigs = compactionConfigs; - this.compactionTaskSlotRatio = compactionTaskSlotRatio == null ? - DEFAULT_COMPACTION_TASK_RATIO : - compactionTaskSlotRatio; - this.maxCompactionTaskSlots = maxCompactionTaskSlots == null ? - DEFAULT_MAX_COMPACTION_TASK_SLOTS : - maxCompactionTaskSlots; - this.useAutoScaleSlots = useAutoScaleSlots == null ? DEFAULT_USE_AUTO_SCALE_SLOTS : useAutoScaleSlots; - this.compactionEngine = DEFAULT_COMPACTION_ENGINE; + this.compactionTaskSlotRatio = Configs.valueOrDefault(compactionTaskSlotRatio, DEFAULT_COMPACTION_TASK_RATIO); + this.maxCompactionTaskSlots = Configs.valueOrDefault(maxCompactionTaskSlots, DEFAULT_MAX_COMPACTION_TASK_SLOTS); + this.useAutoScaleSlots = Configs.valueOrDefault(useAutoScaleSlots, DEFAULT_USE_AUTO_SCALE_SLOTS); + this.compactionEngine = Configs.valueOrDefault(compactionEngine, DEFAULT_COMPACTION_ENGINE); } @JsonProperty @@ -150,8 +150,12 @@ public boolean equals(Object o) @Override public int hashCode() { - return Objects.hash(compactionConfigs, compactionTaskSlotRatio, maxCompactionTaskSlots, useAutoScaleSlots, - compactionEngine + return Objects.hash( + compactionConfigs, + compactionTaskSlotRatio, + maxCompactionTaskSlots, + useAutoScaleSlots, + compactionEngine ); } @@ -163,7 +167,7 @@ public String toString() ", compactionTaskSlotRatio=" + compactionTaskSlotRatio + ", maxCompactionTaskSlots=" + maxCompactionTaskSlots + ", useAutoScaleSlots=" + useAutoScaleSlots + - ", engine=" + compactionEngine + + ", compactionEngine=" + compactionEngine + '}'; } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java index 6b13f4b236df..3d11819aa88c 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java @@ -207,7 +207,7 @@ public boolean equals(Object o) Arrays.equals(metricsSpec, that.metricsSpec) && Objects.equals(transformSpec, that.transformSpec) && Objects.equals(ioConfig, that.ioConfig) && - Objects.equals(compactionEngine, that.compactionEngine) && + this.compactionEngine == that.compactionEngine && Objects.equals(taskContext, that.taskContext); } @@ -238,56 +238,28 @@ public static DataSourceCompactionConfig from( ) { CompactionEngine newCompactionEngine = newConfig.getEngine(); - String engineSourceLog = "specified in spec"; + String engineSource = "specified in spec"; if (newCompactionEngine == null) { newCompactionEngine = defaultCompactionEngine; - engineSourceLog = "set as default"; + engineSource = "set as default"; + newConfig = new DataSourceCompactionConfig( + newConfig.getDataSource(), + newConfig.getTaskPriority(), + newConfig.getInputSegmentSizeBytes(), + newConfig.getMaxRowsPerSegment(), + newConfig.getSkipOffsetFromLatest(), + newConfig.getTuningConfig(), + newConfig.getGranularitySpec(), + newConfig.getDimensionsSpec(), + newConfig.getMetricsSpec(), + newConfig.getTransformSpec(), + newConfig.getIoConfig(), + newCompactionEngine, + newConfig.getTaskContext() + ); } - if (newCompactionEngine == CompactionEngine.MSQ) { - if (newConfig.getTuningConfig() != null) { - PartitionsSpec partitionsSpec = newConfig.getTuningConfig().getPartitionsSpec(); - if (partitionsSpec != null && !(partitionsSpec instanceof DimensionRangePartitionsSpec - || partitionsSpec instanceof DynamicPartitionsSpec)) { - throw InvalidInput.exception( - "Invalid partition spec type[%s] for MSQ compaction engine[%s]." - + " Type must be either DynamicPartitionsSpec or DynamicRangePartitionsSpec.", - partitionsSpec.getClass(), - engineSourceLog - ); - } - if (partitionsSpec instanceof DynamicPartitionsSpec - && ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows() != null) { - throw InvalidInput.exception( - "maxTotalRows[%d] in DynamicPartitionsSpec not supported for MSQ compaction engine %s.", - ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows(), engineSourceLog - ); - } - } - if (newConfig.getMetricsSpec() != null - && newConfig.getGranularitySpec() != null - && !newConfig.getGranularitySpec() - .isRollup()) { - throw InvalidInput.exception("rollup in granularitySpec must be set to True if metricsSpec is specifed."); - } - } - if (newCompactionEngine == newConfig.getEngine()) { - return newConfig; - } - return new DataSourceCompactionConfig( - newConfig.getDataSource(), - newConfig.getTaskPriority(), - newConfig.getInputSegmentSizeBytes(), - newConfig.getMaxRowsPerSegment(), - newConfig.getSkipOffsetFromLatest(), - newConfig.getTuningConfig(), - newConfig.getGranularitySpec(), - newConfig.getDimensionsSpec(), - newConfig.getMetricsSpec(), - newConfig.getTransformSpec(), - newConfig.getIoConfig(), - newCompactionEngine, newConfig.getTaskContext() - ); + ClientCompactionRunnerInfo.supportsCompactionConfig(newConfig, engineSource); + return newConfig; } - } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionStrategy.java b/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionStrategy.java deleted file mode 100644 index 738f29fecaf3..000000000000 --- a/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionStrategy.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.server.coordinator; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.indexer.CompactionEngine; - -import java.util.Objects; - - -public class UserCompactionStrategy -{ - private final CompactionEngine TYPE; - - @JsonCreator - public UserCompactionStrategy(@JsonProperty("TYPE") CompactionEngine TYPE) - { - this.TYPE = TYPE; - } - - @JsonProperty("TYPE") - public CompactionEngine getType() - { - return TYPE; - } - - @Override - public String toString() - { - return "UserCompactionStrategy{" + - "TYPE=" + TYPE + - '}'; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - UserCompactionStrategy that = (UserCompactionStrategy) o; - return TYPE == that.TYPE; - } - - @Override - public int hashCode() - { - return Objects.hash(TYPE); - } -} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index 5e24a37bc78e..0c07c3b689c6 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java @@ -50,7 +50,7 @@ import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; -import org.apache.druid.server.coordinator.UserCompactionStrategy; +import org.apache.druid.server.coordinator.ClientCompactionRunnerInfo; import org.apache.druid.server.coordinator.compact.CompactionSegmentIterator; import org.apache.druid.server.coordinator.compact.CompactionSegmentSearchPolicy; import org.apache.druid.server.coordinator.compact.CompactionStatistics; @@ -501,7 +501,7 @@ private int submitCompactionTasks( transformSpec, dropExisting, autoCompactionContext, - new UserCompactionStrategy(compactionEngine) + new ClientCompactionRunnerInfo(compactionEngine) ); LOG.info( @@ -655,7 +655,7 @@ private String compactSegments( @Nullable ClientCompactionTaskTransformSpec transformSpec, @Nullable Boolean dropExisting, @Nullable Map context, - @Nullable UserCompactionStrategy compactionStrategy + @Nullable ClientCompactionRunnerInfo compactionRunner ) { Preconditions.checkArgument(!segments.isEmpty(), "Expect non-empty segments to compact"); @@ -684,7 +684,7 @@ private String compactSegments( metricsSpec, transformSpec, context, - compactionStrategy + compactionRunner ); FutureUtils.getUnchecked(overlordClient.runTask(taskId, taskPayload), true); return taskId; diff --git a/server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicyTest.java b/server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicyTest.java index 31e269f50f41..83525eddbe5a 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicyTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicyTest.java @@ -29,6 +29,7 @@ import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; @@ -729,13 +730,15 @@ public void testIteratorReturnsNothingAsSegmentsWasCompactedAndHaveSameSegmentGr Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, null) + new CompactionState(partitionsSpec, null, Collections.emptyMap(), null, null, indexSpec, null, + CompactionEngine.NATIVE) ), new SegmentGenerateSpec( Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, null) + new CompactionState(partitionsSpec, null, Collections.emptyMap(), null, null, indexSpec, null, + CompactionEngine.NATIVE) ) ); @@ -762,13 +765,21 @@ public void testIteratorReturnsNothingAsSegmentsWasCompactedAndHaveSameSegmentGr Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of("segmentGranularity", "day")) + new CompactionState(partitionsSpec, null, + Collections.emptyMap(), + null, null, indexSpec, ImmutableMap.of("segmentGranularity", "day"), + CompactionEngine.NATIVE + ) ), new SegmentGenerateSpec( Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of("segmentGranularity", "day")) + new CompactionState(partitionsSpec, null, + Collections.emptyMap(), + null, null, indexSpec, ImmutableMap.of("segmentGranularity", "day"), + CompactionEngine.NATIVE + ) ) ); @@ -795,13 +806,15 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentSeg Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, null) + new CompactionState(partitionsSpec, null, Collections.emptyMap(), null, null, indexSpec, null, + CompactionEngine.NATIVE) ), new SegmentGenerateSpec( Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, null) + new CompactionState(partitionsSpec, null, Collections.emptyMap(), null, null, indexSpec, null, + CompactionEngine.NATIVE) ) ); @@ -838,13 +851,21 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentSeg Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of("segmentGranularity", "day")) + new CompactionState(partitionsSpec, null, + Collections.emptyMap(), + null, null, indexSpec, ImmutableMap.of("segmentGranularity", "day"), + CompactionEngine.NATIVE + ) ), new SegmentGenerateSpec( Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of("segmentGranularity", "day")) + new CompactionState(partitionsSpec, null, + Collections.emptyMap(), + null, null, indexSpec, ImmutableMap.of("segmentGranularity", "day"), + CompactionEngine.NATIVE + ) ) ); @@ -881,7 +902,8 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentTim Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, null) + new CompactionState(partitionsSpec, null, Collections.emptyMap(), null, null, indexSpec, null, + CompactionEngine.NATIVE) ) ); @@ -933,7 +955,8 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentOri Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, null) + new CompactionState(partitionsSpec, null, Collections.emptyMap(), null, null, indexSpec, null, + CompactionEngine.NATIVE) ) ); @@ -987,19 +1010,28 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentRol Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of("rollup", "false")) + new CompactionState(partitionsSpec, null, + Collections.emptyMap(), + null, null, indexSpec, ImmutableMap.of("rollup", "false"), + CompactionEngine.NATIVE + ) ), new SegmentGenerateSpec( Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of("rollup", "true")) + new CompactionState(partitionsSpec, null, + Collections.emptyMap(), + null, null, indexSpec, ImmutableMap.of("rollup", "true"), + CompactionEngine.NATIVE + ) ), new SegmentGenerateSpec( Intervals.of("2017-10-03T00:00:00/2017-10-04T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of()) + new CompactionState(partitionsSpec, null, Collections.emptyMap(), null, null, indexSpec, ImmutableMap.of(), + CompactionEngine.NATIVE) ) ); @@ -1047,19 +1079,28 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentQue Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of("queryGranularity", "day")) + new CompactionState(partitionsSpec, null, + Collections.emptyMap(), + null, null, indexSpec, ImmutableMap.of("queryGranularity", "day"), + CompactionEngine.NATIVE + ) ), new SegmentGenerateSpec( Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of("queryGranularity", "minute")) + new CompactionState(partitionsSpec, null, + Collections.emptyMap(), + null, null, indexSpec, ImmutableMap.of("queryGranularity", "minute"), + CompactionEngine.NATIVE + ) ), new SegmentGenerateSpec( Intervals.of("2017-10-03T00:00:00/2017-10-04T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of()) + new CompactionState(partitionsSpec, null, Collections.emptyMap(), null, null, indexSpec, ImmutableMap.of(), + CompactionEngine.NATIVE) ) ); @@ -1108,25 +1149,38 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentDim Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar", "foo"))), null, null, indexSpec, null) + new CompactionState(partitionsSpec, new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar", "foo"))), + Collections.emptyMap(), + null, null, indexSpec, null, + CompactionEngine.NATIVE + ) ), new SegmentGenerateSpec( Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("foo"))), null, null, indexSpec, null) + new CompactionState(partitionsSpec, new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("foo"))), + Collections.emptyMap(), + null, null, indexSpec, null, + CompactionEngine.NATIVE + ) ), new SegmentGenerateSpec( Intervals.of("2017-10-03T00:00:00/2017-10-04T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, DimensionsSpec.EMPTY, null, null, indexSpec, null) + new CompactionState(partitionsSpec, DimensionsSpec.EMPTY, + Collections.emptyMap(), + null, null, indexSpec, null, + CompactionEngine.NATIVE + ) ), new SegmentGenerateSpec( Intervals.of("2017-10-04T00:00:00/2017-10-05T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, null) + new CompactionState(partitionsSpec, null, Collections.emptyMap(), null, null, indexSpec, null, + CompactionEngine.NATIVE) ) ); @@ -1210,10 +1264,11 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentFil new CompactionState( partitionsSpec, null, - null, + Collections.emptyMap(), null, mapper.readValue(mapper.writeValueAsString(new TransformSpec(new SelectorDimFilter("dim1", "foo", null), null)), new TypeReference>() {}), indexSpec, - null + null, + CompactionEngine.NATIVE ) ), new SegmentGenerateSpec( @@ -1223,10 +1278,11 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentFil new CompactionState( partitionsSpec, null, - null, + Collections.emptyMap(), null, mapper.readValue(mapper.writeValueAsString(new TransformSpec(new SelectorDimFilter("dim1", "bar", null), null)), new TypeReference>() {}), indexSpec, - null + null, + CompactionEngine.NATIVE ) ), new SegmentGenerateSpec( @@ -1236,17 +1292,19 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentFil new CompactionState( partitionsSpec, null, - null, + Collections.emptyMap(), null, mapper.readValue(mapper.writeValueAsString(new TransformSpec(null, null)), new TypeReference>() {}), indexSpec, - null + null, + CompactionEngine.NATIVE ) ), new SegmentGenerateSpec( Intervals.of("2017-10-04T00:00:00/2017-10-05T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, null) + new CompactionState(partitionsSpec, null, Collections.emptyMap(), null, null, indexSpec, null, + CompactionEngine.NATIVE) ) ); @@ -1334,10 +1392,12 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentMet new CompactionState( partitionsSpec, null, + Collections.emptyMap(), mapper.convertValue(new AggregatorFactory[] {new CountAggregatorFactory("cnt")}, new TypeReference>() {}), null, indexSpec, - null + null, + CompactionEngine.NATIVE ) ), new SegmentGenerateSpec( @@ -1347,10 +1407,12 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentMet new CompactionState( partitionsSpec, null, + Collections.emptyMap(), mapper.convertValue(new AggregatorFactory[] {new CountAggregatorFactory("cnt"), new LongSumAggregatorFactory("val", "val")}, new TypeReference>() {}), null, indexSpec, - null + null, + CompactionEngine.NATIVE ) ), new SegmentGenerateSpec( @@ -1360,17 +1422,19 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentMet new CompactionState( partitionsSpec, null, - mapper.convertValue(new AggregatorFactory[] {}, new TypeReference>() {}), + Collections.emptyMap(), mapper.convertValue(new AggregatorFactory[] {}, new TypeReference>() {}), null, indexSpec, - null + null, + CompactionEngine.NATIVE ) ), new SegmentGenerateSpec( Intervals.of("2017-10-04T00:00:00/2017-10-05T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, null) + new CompactionState(partitionsSpec, null, Collections.emptyMap(), null, null, indexSpec, null, + CompactionEngine.NATIVE) ) ); @@ -1476,7 +1540,8 @@ public void testIteratorReturnsSegmentsAsCompactionStateChangedWithCompactedStat Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, null, null, newIndexSpecMap, null) + new CompactionState(partitionsSpec, null, Collections.emptyMap(), null, null, newIndexSpecMap, null, + CompactionEngine.NATIVE) ) ); @@ -1526,10 +1591,11 @@ public void testIteratorDoesNotReturnSegmentWithChangingAppendableIndexSpec() new CompactionState( partitionsSpec, null, - null, + Collections.emptyMap(), null, null, IndexSpec.DEFAULT.asMap(mapper), - null + null, + CompactionEngine.NATIVE ) ) ); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java index db8fdce20add..a81f9c5c792c 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java @@ -42,6 +42,7 @@ import org.apache.druid.client.indexing.TaskPayloadResponse; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.RunnerTaskState; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskState; @@ -407,12 +408,24 @@ public void testMakeStatsForDataSourceWithCompactedIntervalBetweenNonCompactedIn DataSegment afterNoon = createSegment(dataSourceName, j, false, k); if (j == 3) { // Make two intervals on this day compacted (two compacted intervals back-to-back) - beforeNoon = beforeNoon.withLastCompactionState(new CompactionState(partitionsSpec, null, null, null, ImmutableMap.of(), ImmutableMap.of())); - afterNoon = afterNoon.withLastCompactionState(new CompactionState(partitionsSpec, null, null, null, ImmutableMap.of(), ImmutableMap.of())); + beforeNoon = beforeNoon.withLastCompactionState(new CompactionState(partitionsSpec, null, + Collections.emptyMap(), + null, null, ImmutableMap.of(), ImmutableMap.of(), + CompactionEngine.NATIVE + )); + afterNoon = afterNoon.withLastCompactionState(new CompactionState(partitionsSpec, null, + Collections.emptyMap(), + null, null, ImmutableMap.of(), ImmutableMap.of(), + CompactionEngine.NATIVE + )); } if (j == 1) { // Make one interval on this day compacted - afterNoon = afterNoon.withLastCompactionState(new CompactionState(partitionsSpec, null, null, null, ImmutableMap.of(), ImmutableMap.of())); + afterNoon = afterNoon.withLastCompactionState(new CompactionState(partitionsSpec, null, + Collections.emptyMap(), + null, null, ImmutableMap.of(), ImmutableMap.of(), + CompactionEngine.NATIVE + )); } segments.add(beforeNoon); segments.add(afterNoon); @@ -1797,7 +1810,8 @@ private CoordinatorRunStats doCompactSegments( compactionConfigs, numCompactionTaskSlots == null ? null : 1.0, // 100% when numCompactionTaskSlots is not null numCompactionTaskSlots, - useAutoScaleSlots + useAutoScaleSlots, + null ) ) .build(); @@ -2108,7 +2122,7 @@ private void compactSegments( clientCompactionTaskQuery.getDimensionsSpec() == null ? null : new DimensionsSpec( clientCompactionTaskQuery.getDimensionsSpec().getDimensions() ), - metricsSpec, + Collections.emptyMap(), metricsSpec, transformSpec, ImmutableMap.of( "bitmap", @@ -2120,7 +2134,8 @@ private void compactSegments( "longEncoding", "longs" ), - ImmutableMap.of() + ImmutableMap.of(), + CompactionEngine.NATIVE ), 1, segmentSize diff --git a/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java b/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java index 0f20fc96bdcc..a18d8a4f4879 100644 --- a/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java +++ b/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java @@ -26,6 +26,7 @@ import com.google.common.collect.ImmutableMap; import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; @@ -41,6 +42,7 @@ import org.junit.Test; import java.util.Arrays; +import java.util.Collections; import java.util.Map; public class DataSegmentPlusTest @@ -88,10 +90,11 @@ public void testSerde() throws JsonProcessingException new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "bar", "foo")) ), - ImmutableList.of(ImmutableMap.of("type", "count", "name", "count")), + Collections.emptyMap(), ImmutableList.of(ImmutableMap.of("type", "count", "name", "count")), ImmutableMap.of("filter", ImmutableMap.of("type", "selector", "dimension", "dim1", "value", "foo")), ImmutableMap.of(), - ImmutableMap.of() + ImmutableMap.of(), + CompactionEngine.NATIVE ), TEST_VERSION, 1 diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java index fb7c75353384..e404be64715c 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java @@ -51,6 +51,7 @@ import org.apache.druid.discovery.DruidNodeDiscovery; import org.apache.druid.discovery.DruidNodeDiscoveryProvider; import org.apache.druid.discovery.NodeRole; +import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexing.overlord.supervisor.SupervisorStatus; @@ -283,10 +284,11 @@ public void setUp(@TempDir File tmpDir) throws Exception private final CompactionState expectedCompactionState = new CompactionState( new DynamicPartitionsSpec(null, null), null, - null, + Collections.emptyMap(), null, null, Collections.singletonMap("test", "map"), - Collections.singletonMap("test2", "map2") + Collections.singletonMap("test2", "map2"), + CompactionEngine.NATIVE ); private final DataSegment publishedCompactedSegment1 = new DataSegment( From 02df1b5cf3cc6866e4f088d151baf6b15bac0981 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Sat, 25 May 2024 23:25:29 +0530 Subject: [PATCH 25/54] Post merge fixes --- .../org/apache/druid/msq/exec/ControllerImpl.java | 3 ++- .../indexing/common/task/CompactionTask.java | 15 +++------------ 2 files changed, 5 insertions(+), 13 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index e3eae856e91f..e23b2b9cd244 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -1621,7 +1621,8 @@ private static Function, Set> addCompactionStateTo GranularitySpec granularitySpec = new UniformGranularitySpec( segmentGranularity, - QueryContext.of(task.getContext()).getGranularity(DruidSqlInsert.SQL_INSERT_QUERY_GRANULARITY, jsonMapper), + QueryContext.of(querySpec.getQuery().getContext()) + .getGranularity(DruidSqlInsert.SQL_INSERT_QUERY_GRANULARITY, jsonMapper), dataSchema.getGranularitySpec().isRollup(), ((DataSourceMSQDestination) querySpec.getDestination()).getReplaceTimeChunks() ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 776136330e2a..7a36446b71f4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -157,18 +157,7 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg @JsonIgnore private final SegmentProvider segmentProvider; @JsonIgnore - private final PartitionConfigurationManager partitionConfigurationManager; - - @JsonIgnore - private final SegmentCacheManagerFactory segmentCacheManagerFactory; - - @JsonIgnore - private final CurrentSubTaskHolder currentSubTaskHolder = new CurrentSubTaskHolder( - (taskObject, config) -> { - final ParallelIndexSupervisorTask indexTask = (ParallelIndexSupervisorTask) taskObject; - indexTask.stopGracefully(config); - } - ); + private final CurrentSubTaskHolder currentSubTaskHolder; @JsonCreator public CompactionTask( @@ -238,6 +227,8 @@ public CompactionTask( } this.tuningConfig = tuningConfig != null ? getTuningConfig(tuningConfig) : null; this.segmentProvider = new SegmentProvider(dataSource, this.ioConfig.getInputSpec()); + this.compactionRunner = compactionRunner; + this.currentSubTaskHolder = compactionRunner.getCurrentSubTaskHolder(); // Do not load any lookups in sub-tasks launched by compaction task, unless transformSpec is present. // If transformSpec is present, we will not modify the context so that the sub-tasks can make the From fad84d14f9cd0437f6ac3a92d475c12efc3f10c7 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Sun, 26 May 2024 17:37:11 +0530 Subject: [PATCH 26/54] Address review comments and fix compaction state comparison for MSQ. --- docs/multi-stage-query/known-issues.md | 5 +++ .../apache/druid/msq/exec/ControllerImpl.java | 33 ++++++++++-------- .../msq/indexing/MSQCompactionRunner.java | 2 +- .../common/task/CompactionRunner.java | 7 ++-- .../common/task/CompactionTaskTest.java | 2 -- .../druid/timeline/CompactionState.java | 1 - .../coordinator/compact/CompactionStatus.java | 34 ++++++++++++++++--- .../coordinator/duty/CompactSegments.java | 2 +- 8 files changed, 56 insertions(+), 30 deletions(-) diff --git a/docs/multi-stage-query/known-issues.md b/docs/multi-stage-query/known-issues.md index 39286edfcdd7..540dc2f08eb9 100644 --- a/docs/multi-stage-query/known-issues.md +++ b/docs/multi-stage-query/known-issues.md @@ -68,3 +68,8 @@ properties, and the `indexSpec` [`tuningConfig`](../ingestion/ingestion-spec.md# - The maximum number of elements in a window cannot exceed a value of 100,000. - To avoid `leafOperators` in MSQ engine, window functions have an extra scan stage after the window stage for cases where native engine has a non-empty `leafOperator`. + +## Compaction + +- Compaction configured with MSQ engine currently doesn't support context flags `finalizeAggregations=false` and + `taskAssignment=auto`. diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index e23b2b9cd244..5cb87ece4205 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -220,6 +220,7 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; +import java.util.LinkedHashMap; import java.util.LinkedHashSet; import java.util.List; import java.util.Map; @@ -1773,8 +1774,9 @@ private static QueryDefinition makeQueryDefinition( } // Map to track the aggregator factories of dimensions that have been created from metrics due to context flag - // finalizeAggregations=true. - final Map dimensionToAggregatoryFactoryMap = new HashMap<>(); + // finalizeAggregations=true. Using LinkedHashMap to preserve order as metrics are compared as arrays + // in CompactionStatus. + final Map dimensionToAggregatoryFactoryMap = new LinkedHashMap<>(); // Then, add a segment-generation stage. final DataSchema dataSchema = @@ -2116,25 +2118,26 @@ private static Pair, List> makeDimensio Map outputColumnAggregatorFactories = new HashMap<>(); // Populate aggregators from the native query when doing an ingest in rollup mode. - for (AggregatorFactory aggregatorFactory : ((GroupByQuery) query).getAggregatorSpecs()) { - for (final int outputColumn : columnMappings.getOutputColumnsForQueryColumn(aggregatorFactory.getName())) { - final String outputColumnName = columnMappings.getOutputColumnName(outputColumn); - if (outputColumnAggregatorFactories.containsKey(outputColumnName)) { - throw new ISE("There can only be one aggregation for column [%s].", outputColumn); - } else { - if (isRollupQuery) { - outputColumnAggregatorFactories.put( - outputColumnName, - aggregatorFactory.withName(outputColumnName).getCombiningFactory() - ); + if (query instanceof GroupByQuery){ + for (AggregatorFactory aggregatorFactory : ((GroupByQuery) query).getAggregatorSpecs()) { + for (final int outputColumn : columnMappings.getOutputColumnsForQueryColumn(aggregatorFactory.getName())) { + final String outputColumnName = columnMappings.getOutputColumnName(outputColumn); + if (outputColumnAggregatorFactories.containsKey(outputColumnName)) { + throw new ISE("There can only be one aggregation for column [%s].", outputColumn); } else { - dimensionToAggregatoryFactoryMap.put(outputColumnName, aggregatorFactory); + if (isRollupQuery) { + outputColumnAggregatorFactories.put( + outputColumnName, + aggregatorFactory.withName(outputColumnName).getCombiningFactory() + ); + } else { + dimensionToAggregatoryFactoryMap.put(outputColumnName, aggregatorFactory); + } } } } } - // Each column can be of either time, dimension, aggregator. For this method. we can ignore the time column. // For non-complex columns, If the aggregator factory of the column is not available, we treat the column as // a dimension. For complex columns, certains hacks are in place. diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java index b2387b0049cc..c0f232712ff4 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java @@ -90,7 +90,7 @@ public class MSQCompactionRunner implements CompactionRunner private final ObjectMapper jsonMapper; private final Injector injector; - public static final String TIME_VIRTUAL_COLUMN = "vTime"; + public static final String TIME_VIRTUAL_COLUMN = "__vTime"; public static final String TIME_COLUMN = ColumnHolder.TIME_COLUMN_NAME; @JsonIgnore diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java index df5a0e6f56de..90ece4b01059 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java @@ -19,6 +19,7 @@ package org.apache.druid.indexing.common.task; +import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import org.apache.druid.indexer.TaskStatus; @@ -35,11 +36,7 @@ * Strategy to be used for executing a compaction task. * Should be synchronized with {@link ClientCompactionRunnerInfo} */ -@JsonTypeInfo( - use = JsonTypeInfo.Id.NAME, - include = JsonTypeInfo.As.EXISTING_PROPERTY, - property = CompactionRunner.TYPE_PROPERTY, - visible = true) +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = CompactionRunner.TYPE_PROPERTY) @JsonSubTypes(value = { @JsonSubTypes.Type(name = NativeCompactionRunner.type, value = NativeCompactionRunner.class) }) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index 05017c5fe500..359aa4ca71e8 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -1822,7 +1822,6 @@ public void testGetDefaultLookupLoadingSpec() { final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); final CompactionTask task = builder @@ -1836,7 +1835,6 @@ public void testGetDefaultLookupLoadingSpecWithTransformSpec() { final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); final CompactionTask task = builder diff --git a/processing/src/main/java/org/apache/druid/timeline/CompactionState.java b/processing/src/main/java/org/apache/druid/timeline/CompactionState.java index 3fb21a4a7289..07dfe2f7ea15 100644 --- a/processing/src/main/java/org/apache/druid/timeline/CompactionState.java +++ b/processing/src/main/java/org/apache/druid/timeline/CompactionState.java @@ -62,7 +62,6 @@ public class CompactionState // has a dependency on the 'core' module where this class is. private final List metricsSpec; private final CompactionEngine engine; - private final Map dimensionToAggregatoryFactoryMap; @JsonCreator diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java index 862f2e7c5b4c..76e274518283 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java @@ -26,7 +26,9 @@ import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; import org.apache.druid.client.indexing.ClientCompactionTaskTransformSpec; import org.apache.druid.common.config.Configs; +import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.java.util.common.StringUtils; @@ -41,7 +43,9 @@ import javax.annotation.Nullable; import java.util.Arrays; import java.util.List; +import java.util.Map; import java.util.function.Function; +import java.util.stream.Collectors; /** * Represents the status of compaction for a given list of candidate segments. @@ -261,7 +265,9 @@ private CompactionStatus segmentGranularityIsUpToDate() private CompactionStatus rollupIsUpToDate() { - if (configuredGranularitySpec == null) { + // MSQ considers a query as rollup only if finalizeAggregations=false. Compaction can have that set to true. + if (configuredGranularitySpec == null || (configuredGranularitySpec.isRollup() + && lastCompactionState.getEngine() == CompactionEngine.MSQ)) { return COMPLETE; } else { return CompactionStatus.completeIfEqual( @@ -291,10 +297,23 @@ private CompactionStatus dimensionsSpecIsUpToDate() return COMPLETE; } else { final DimensionsSpec existingDimensionsSpec = lastCompactionState.getDimensionsSpec(); + final Map dimensionToAggregatorFactoryMap = + lastCompactionState.getDimensionToAggregatoryFactoryMap(); + // Remove dimensions which were converted from metrics due to finalizeAggregations=true + final List originalDimensions = + existingDimensionsSpec == null + ? null + : dimensionToAggregatorFactoryMap == null + ? existingDimensionsSpec.getDimensions() + : existingDimensionsSpec.getDimensions() + .stream() + .filter(dimensionSchema -> !dimensionToAggregatorFactoryMap.containsKey( + dimensionSchema.getName())) + .collect(Collectors.toList()); return CompactionStatus.completeIfEqual( "dimensionsSpec", compactionConfig.getDimensionsSpec().getDimensions(), - existingDimensionsSpec == null ? null : existingDimensionsSpec.getDimensions() + originalDimensions ); } } @@ -307,9 +326,14 @@ private CompactionStatus metricsSpecIsUpToDate() } final List metricSpecList = lastCompactionState.getMetricsSpec(); - final AggregatorFactory[] existingMetricsSpec - = CollectionUtils.isNullOrEmpty(metricSpecList) - ? null : objectMapper.convertValue(metricSpecList, AggregatorFactory[].class); + final AggregatorFactory[] existingMetricsSpec; + if (CollectionUtils.isNullOrEmpty(metricSpecList)) { + final Map dimensionToAggregatorFactoryMap = + lastCompactionState.getDimensionToAggregatoryFactoryMap(); + existingMetricsSpec = dimensionToAggregatorFactoryMap == null ? null: dimensionToAggregatorFactoryMap.values().toArray(new AggregatorFactory[0]); + } else { + existingMetricsSpec = objectMapper.convertValue(metricSpecList, AggregatorFactory[].class); + } if (existingMetricsSpec == null || !Arrays.deepEquals(configuredMetricsSpec, existingMetricsSpec)) { return CompactionStatus.configChanged( diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index 0c07c3b689c6..d4f48ceaccb4 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java @@ -236,7 +236,7 @@ private boolean cancelTaskIfGranularityChanged( Granularity configuredSegmentGranularity = dataSourceCompactionConfig.getGranularitySpec() .getSegmentGranularity(); Granularity taskSegmentGranularity = compactionTaskQuery.getGranularitySpec().getSegmentGranularity(); - if (configuredSegmentGranularity.equals(taskSegmentGranularity)) { + if (configuredSegmentGranularity != null && configuredSegmentGranularity.equals(taskSegmentGranularity)) { return false; } From 9c7ee58f10aae89ce8311b3b1233b67a7b8141a6 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Mon, 27 May 2024 13:01:20 +0530 Subject: [PATCH 27/54] Fix checkstyle issues --- .../org/apache/druid/msq/exec/ControllerImpl.java | 2 +- .../apache/druid/msq/guice/MSQIndexingModule.java | 2 +- .../druid/msq/indexing/MSQCompactionRunner.java | 12 ++---------- .../org/apache/druid/msq/exec/MSQReplaceTest.java | 2 +- .../druid/indexing/common/task/CompactionRunner.java | 4 +--- .../druid/indexing/common/task/CompactionTask.java | 3 +-- .../indexing/common/task/NativeCompactionRunner.java | 7 ------- .../task/ClientCompactionTaskQuerySerdeTest.java | 4 ---- .../indexing/common/task/CompactionTaskTest.java | 2 +- .../org/apache/druid/timeline/CompactionState.java | 1 - .../coordinator/ClientCompactionRunnerInfo.java | 5 ++--- .../coordinator/DataSourceCompactionConfig.java | 4 ---- .../server/coordinator/compact/CompactionStatus.java | 4 +++- .../server/coordinator/duty/CompactSegments.java | 2 +- 14 files changed, 14 insertions(+), 40 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 5cb87ece4205..151c0e72d8e1 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -2118,7 +2118,7 @@ private static Pair, List> makeDimensio Map outputColumnAggregatorFactories = new HashMap<>(); // Populate aggregators from the native query when doing an ingest in rollup mode. - if (query instanceof GroupByQuery){ + if (query instanceof GroupByQuery) { for (AggregatorFactory aggregatorFactory : ((GroupByQuery) query).getAggregatorSpecs()) { for (final int outputColumn : columnMappings.getOutputColumnsForQueryColumn(aggregatorFactory.getName())) { final String outputColumnName = columnMappings.getOutputColumnName(outputColumn); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java index 7222ca0d83fd..c3f684713c39 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java @@ -192,7 +192,7 @@ public List getJacksonModules() NilInputSource.class ); - module.registerSubtypes(new NamedType(MSQCompactionRunner.class, MSQCompactionRunner.type)); + module.registerSubtypes(new NamedType(MSQCompactionRunner.class, "MSQ")); FAULT_CLASSES.forEach(module::registerSubtypes); module.addSerializer(new CounterSnapshotsSerializer()); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java index c0f232712ff4..47bcbf2726d6 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonIgnore; -import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; @@ -84,7 +83,6 @@ public class MSQCompactionRunner implements CompactionRunner { private static final Logger log = new Logger(MSQCompactionRunner.class); - public static final String type = "MSQ"; private static final Granularity DEFAULT_SEGMENT_GRANULARITY = Granularities.ALL; private final ObjectMapper jsonMapper; @@ -134,19 +132,13 @@ public Pair supportsCompactionConfig( PartitionsSpec partitionsSpec = compactionTask.getTuningConfig().getPartitionsSpec(); if (!(partitionsSpec instanceof DynamicPartitionsSpec || partitionsSpec instanceof DimensionRangePartitionsSpec)) { - return Pair.of(false, "PartitionsSpec not among DynamicPartitionSpec or DimensionRangePartitionsSpec"); + return Pair.of(false, "PartitionsSpec not among DynamicPartitionSpec or DimensionRangePartitionsSpec " + + "required with MSQ engine"); } } return Pair.of(true, null); } - @Override - @JsonProperty("TYPE") - public String getType() - { - return type; - } - @Override public CurrentSubTaskHolder getCurrentSubTaskHolder() { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java index 50dfcd0abc1b..f00c37738fd9 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java @@ -29,9 +29,9 @@ import org.apache.druid.data.input.impl.FloatDimensionSchema; import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; -import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.error.DruidException; import org.apache.druid.error.DruidExceptionMatcher; +import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java index 90ece4b01059..02e0e142379e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java @@ -19,7 +19,6 @@ package org.apache.druid.indexing.common.task; -import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import org.apache.druid.indexer.TaskStatus; @@ -38,7 +37,7 @@ */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = CompactionRunner.TYPE_PROPERTY) @JsonSubTypes(value = { - @JsonSubTypes.Type(name = NativeCompactionRunner.type, value = NativeCompactionRunner.class) + @JsonSubTypes.Type(name = "NATIVE", value = NativeCompactionRunner.class) }) public interface CompactionRunner { @@ -60,5 +59,4 @@ TaskStatus runCompactionTasks( */ Pair supportsCompactionConfig(CompactionTask compactionTask); - String getType(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 7a36446b71f4..b8780c0887f7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -476,8 +476,7 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception Pair supportsCompactionConfig = compactionRunner.supportsCompactionConfig(this); if (!supportsCompactionConfig.lhs) { throw InvalidInput.exception( - "Compaction config for engine[%s] not supported[%s]", - compactionRunner.getType(), + "Compaction config not supported[%s].", supportsCompactionConfig.rhs ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java index 75a6e8c2e6f9..73fa29aef1fc 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java @@ -58,7 +58,6 @@ public class NativeCompactionRunner implements CompactionRunner { private static final Logger log = new Logger(NativeCompactionRunner.class); - public static final String type = "NATIVE"; private static final boolean STORE_COMPACTION_STATE = true; @JsonIgnore private final SegmentCacheManagerFactory segmentCacheManagerFactory; @@ -89,12 +88,6 @@ public Pair supportsCompactionConfig( return Pair.of(true, null); } - @Override - public String getType() - { - return type; - } - /** * Generate {@link ParallelIndexIngestionSpec} from input dataschemas. * diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java index f66c02931d23..458fb08ae766 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java @@ -292,10 +292,6 @@ private void assertQueryToTask(ClientCompactionTaskQuery query, CompactionTask t query.getMetricsSpec(), task.getMetricsSpec() ); - Assert.assertEquals( - query.getClientCompactionRunnerInfo().getType().toString(), - task.getCompactionRunner().getType().toString() - ); } private ClientCompactionTaskQuery createCompactionTaskQuery(String id, ClientCompactionTaskTransformSpec transformSpec) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index 359aa4ca71e8..ec8502c41c71 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -69,9 +69,9 @@ import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.config.TaskConfigBuilder; import org.apache.druid.indexing.common.task.CompactionTask.Builder; -import org.apache.druid.indexing.common.task.NativeCompactionRunner.PartitionConfigurationManager; import org.apache.druid.indexing.common.task.CompactionTask.SegmentProvider; import org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig; +import org.apache.druid.indexing.common.task.NativeCompactionRunner.PartitionConfigurationManager; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexIOConfig; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexIngestionSpec; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; diff --git a/processing/src/main/java/org/apache/druid/timeline/CompactionState.java b/processing/src/main/java/org/apache/druid/timeline/CompactionState.java index 07dfe2f7ea15..bb8eb3b1ca0b 100644 --- a/processing/src/main/java/org/apache/druid/timeline/CompactionState.java +++ b/processing/src/main/java/org/apache/druid/timeline/CompactionState.java @@ -26,7 +26,6 @@ import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.query.aggregation.AggregatorFactory; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Objects; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfo.java b/server/src/main/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfo.java index 59db36751752..2e3dd4fa84ad 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfo.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfo.java @@ -45,7 +45,6 @@ public ClientCompactionRunnerInfo(@JsonProperty("type") CompactionEngine type) this.type = type; } -// @JsonProperty("type") @JsonProperty public CompactionEngine getType() { @@ -136,8 +135,8 @@ public static void supportsCompactionConfig(DataSourceCompactionConfig newConfig * compaction config at the coordinator. The values used here should be kept in sync with those in * {@link org.apache.druid.msq.util.MultiStageQueryContext} */ - - private static class MSQContext { + private static class MSQContext + { public static final String CTX_FINALIZE_AGGREGATIONS = "finalizeAggregations"; public static final String CTX_TASK_ASSIGNMENT_STRATEGY = "taskAssignment"; public static final String TASK_ASSIGNMENT_STRATEGY_MAX = "MAX"; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java index 3d11819aa88c..a1c0050d991c 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java @@ -22,11 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import org.apache.druid.error.InvalidInput; import org.apache.druid.indexer.CompactionEngine; -import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; -import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; -import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.query.aggregation.AggregatorFactory; import org.joda.time.Period; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java index 76e274518283..60fa7cb957c0 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java @@ -330,7 +330,9 @@ private CompactionStatus metricsSpecIsUpToDate() if (CollectionUtils.isNullOrEmpty(metricSpecList)) { final Map dimensionToAggregatorFactoryMap = lastCompactionState.getDimensionToAggregatoryFactoryMap(); - existingMetricsSpec = dimensionToAggregatorFactoryMap == null ? null: dimensionToAggregatorFactoryMap.values().toArray(new AggregatorFactory[0]); + existingMetricsSpec = dimensionToAggregatorFactoryMap == null + ? null + : dimensionToAggregatorFactoryMap.values().toArray(new AggregatorFactory[0]); } else { existingMetricsSpec = objectMapper.convertValue(metricSpecList, AggregatorFactory[].class); } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index d4f48ceaccb4..6fde055cf0d6 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java @@ -47,10 +47,10 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; +import org.apache.druid.server.coordinator.ClientCompactionRunnerInfo; import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; -import org.apache.druid.server.coordinator.ClientCompactionRunnerInfo; import org.apache.druid.server.coordinator.compact.CompactionSegmentIterator; import org.apache.druid.server.coordinator.compact.CompactionSegmentSearchPolicy; import org.apache.druid.server.coordinator.compact.CompactionStatistics; From 4f5d3562ed6d038d683cbd172fbac66f4736de6d Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Mon, 27 May 2024 15:13:22 +0530 Subject: [PATCH 28/54] Fix checkstyle issues and tests --- .../msq/indexing/MSQCompactionRunnerTest.java | 7 ++- .../indexing/common/task/CompactionTask.java | 5 +- .../ClientCompactionTaskQuerySerdeTest.java | 9 +-- .../task/CompactionTaskParallelRunTest.java | 42 ++++++++----- .../common/task/CompactionTaskRunTest.java | 63 ++++++++++++------- .../common/task/CompactionTaskTest.java | 48 +++++++++----- .../batch/parallel/PartialCompactionTest.java | 4 +- 7 files changed, 117 insertions(+), 61 deletions(-) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java index f5dce5c50b49..e19e12ca7e01 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java @@ -33,6 +33,7 @@ import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexing.common.task.CompactionIntervalSpec; import org.apache.druid.indexing.common.task.CompactionTask; +import org.apache.druid.indexing.common.task.NativeCompactionRunner; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.NonnullPair; @@ -120,7 +121,8 @@ public void testCompactionToMSQTasks() throws JsonProcessingException new ClientCompactionTaskTransformSpec(dimFilter); final CompactionTask.Builder builder = new CompactionTask.Builder( DATA_SOURCE, - null + null, + new MSQCompactionRunner(JSON_MAPPER, null) ); IndexSpec indexSpec = IndexSpec.builder() .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) @@ -131,6 +133,7 @@ public void testCompactionToMSQTasks() throws JsonProcessingException Map context = new HashMap<>(); context.put(MultiStageQueryContext.CTX_SORT_ORDER, JSON_MAPPER.writeValueAsString(SORT_ORDER_DIMENSIONS)); + context.put(MultiStageQueryContext.CTX_MAX_NUM_TASKS, 2); builder .inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, null)) @@ -172,7 +175,7 @@ public void testCompactionToMSQTasks() throws JsonProcessingException MSQSpec actualMSQSpec = msqControllerTask.getQuerySpec(); Assert.assertEquals( new MSQTuningConfig( - MultiStageQueryContext.DEFAULT_MAX_NUM_TASKS, + 1, MultiStageQueryContext.DEFAULT_ROWS_IN_MEMORY, TARGET_ROWS_PER_SEGMENT, indexSpec diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index b8780c0887f7..df4cdea4ccbc 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -194,6 +194,7 @@ public CompactionTask( new Property<>("segments", segments) ) ); + Preconditions.checkNotNull(compactionRunner, "compactionRunner cannot be null"); if (ioConfig != null) { this.ioConfig = ioConfig; } else if (interval != null) { @@ -1048,11 +1049,13 @@ public static class Builder public Builder( String dataSource, - RetryPolicyFactory retryPolicyFactory + RetryPolicyFactory retryPolicyFactory, + CompactionRunner compactionRunner ) { this.dataSource = dataSource; this.retryPolicyFactory = retryPolicyFactory; + this.compactionRunner = compactionRunner; } public Builder interval(Interval interval) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java index 458fb08ae766..89ca3609553e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java @@ -339,8 +339,8 @@ private CompactionTask createCompactionTask(ClientCompactionTaskTransformSpec tr { CompactionTask.Builder compactionTaskBuilder = new CompactionTask.Builder( "datasource", - new RetryPolicyFactory(new RetryPolicyConfig()) - ) + new RetryPolicyFactory(new RetryPolicyConfig()), + new NativeCompactionRunner(new SegmentCacheManagerFactory(new TestUtils().getTestObjectMapper()))) .inputSpec(new CompactionIntervalSpec(Intervals.of("2019/2020"), "testSha256OfSortedSegmentIds"), true) .tuningConfig( new ParallelIndexTuningConfig( @@ -387,10 +387,7 @@ private CompactionTask createCompactionTask(ClientCompactionTaskTransformSpec tr ) .metricsSpec(METRICS_SPEC) .transformSpec(transformSpec) - .context(ImmutableMap.of("key", "value")) - .compactionRunner(new NativeCompactionRunner( - new SegmentCacheManagerFactory(new TestUtils().getTestObjectMapper()) - )); + .context(ImmutableMap.of("key", "value")); return compactionTaskBuilder.build(); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java index 54d448485001..f41a6f9ca563 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java @@ -161,7 +161,8 @@ public void testRunParallelWithDynamicPartitioningMatchCompactionState() throws final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(getSegmentCacheManagerFactory()) ); final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) @@ -216,7 +217,8 @@ public void testRunParallelWithHashPartitioningMatchCompactionState() throws Exc final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(getSegmentCacheManagerFactory()) ); final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) @@ -282,7 +284,8 @@ public void testRunParallelWithRangePartitioning() throws Exception final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(getSegmentCacheManagerFactory()) ); final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) @@ -333,7 +336,8 @@ public void testRunParallelWithRangePartitioningAndNoUpfrontSegmentFetching() th final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(getSegmentCacheManagerFactory()) ); final CompactionTask compactionTask = builder @@ -396,7 +400,8 @@ public void testRunParallelWithMultiDimensionRangePartitioning() throws Exceptio final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(getSegmentCacheManagerFactory()) ); final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) @@ -450,7 +455,8 @@ public void testRunParallelWithRangePartitioningWithSingleTask() throws Exceptio final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(getSegmentCacheManagerFactory()) ); final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) @@ -501,7 +507,8 @@ public void testRunParallelWithMultiDimensionRangePartitioningWithSingleTask() t final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(getSegmentCacheManagerFactory()) ); final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) @@ -552,7 +559,8 @@ public void testRunCompactionStateNotStoreIfContextSetToFalse() final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(getSegmentCacheManagerFactory()) ); final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) @@ -582,7 +590,8 @@ public void testRunCompactionWithFilterShouldStoreInState() throws Exception final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(getSegmentCacheManagerFactory()) ); final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) @@ -639,7 +648,8 @@ public void testRunCompactionWithNewMetricsShouldStoreInState() throws Exception final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(getSegmentCacheManagerFactory()) ); final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) @@ -702,7 +712,8 @@ public void testCompactHashAndDynamicPartitionedSegments() runIndexTask(null, true); final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(getSegmentCacheManagerFactory()) ); final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) @@ -751,7 +762,8 @@ public void testCompactRangeAndDynamicPartitionedSegments() runIndexTask(null, true); final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(getSegmentCacheManagerFactory()) ); final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) @@ -842,7 +854,8 @@ public void testCompactionDropSegmentsOfInputIntervalIfDropFlagIsSet() throws Ex final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(getSegmentCacheManagerFactory()) ); final CompactionTask compactionTask = builder // Set the dropExisting flag to true in the IOConfig of the compaction task @@ -888,7 +901,8 @@ public void testCompactionDoesNotDropSegmentsIfDropFlagNotSet() throws Exception final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(getSegmentCacheManagerFactory()) ); final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index 62b86de12824..d2313d81b919 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -283,7 +283,8 @@ public void testRunWithDynamicPartitioning() throws Exception final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(getSegmentCacheManagerFactory()) ); final CompactionTask compactionTask = builder @@ -350,7 +351,8 @@ public void testRunWithHashPartitioning() throws Exception final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(getSegmentCacheManagerFactory()) ); final CompactionTask compactionTask = builder @@ -450,7 +452,8 @@ public void testRunCompactionTwice() throws Exception final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(getSegmentCacheManagerFactory()) ); final CompactionTask compactionTask1 = builder @@ -544,7 +547,8 @@ public void testRunIndexAndCompactAtTheSameTimeForDifferentInterval() throws Exc final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(getSegmentCacheManagerFactory()) ); final CompactionTask compactionTask = builder @@ -655,7 +659,8 @@ public void testWithSegmentGranularity() throws Exception final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(getSegmentCacheManagerFactory()) ); // day segmentGranularity @@ -724,7 +729,8 @@ public void testWithSegmentGranularityMisalignedInterval() throws Exception final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(getSegmentCacheManagerFactory()) ); final CompactionTask compactionTask1 = builder @@ -758,7 +764,8 @@ public void testWithSegmentGranularityMisalignedIntervalAllowed() throws Excepti final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(getSegmentCacheManagerFactory()) ); // day segmentGranularity @@ -802,7 +809,8 @@ public void testCompactionWithFilterInTransformSpec() throws Exception final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(getSegmentCacheManagerFactory()) ); // day segmentGranularity @@ -862,7 +870,8 @@ public void testCompactionWithNewMetricInMetricsSpec() throws Exception final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(getSegmentCacheManagerFactory()) ); // day segmentGranularity @@ -928,7 +937,8 @@ public void testWithGranularitySpecNonNullSegmentGranularityAndNullQueryGranular final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(getSegmentCacheManagerFactory()) ); // day segmentGranularity @@ -996,7 +1006,8 @@ public void testWithGranularitySpecNonNullQueryGranularityAndNullSegmentGranular final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(getSegmentCacheManagerFactory()) ); // day queryGranularity @@ -1049,7 +1060,8 @@ public void testWithGranularitySpecNonNullQueryGranularityAndNonNullSegmentGranu final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(getSegmentCacheManagerFactory()) ); // day segmentGranularity and day queryGranularity @@ -1087,7 +1099,8 @@ public void testWithGranularitySpecNullQueryGranularityAndNullSegmentGranularity final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(getSegmentCacheManagerFactory()) ); final CompactionTask compactionTask1 = builder @@ -1139,7 +1152,8 @@ public void testCompactThenAppend() throws Exception final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(getSegmentCacheManagerFactory()) ); final CompactionTask compactionTask = builder @@ -1200,7 +1214,8 @@ public void testPartialIntervalCompactWithFinerSegmentGranularityThenFullInterva final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(getSegmentCacheManagerFactory()) ); // Setup partial compaction: @@ -1355,7 +1370,8 @@ public void testCompactDatasourceOverIntervalWithOnlyTombstones() throws Excepti final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(getSegmentCacheManagerFactory()) ); // Setup partial interval compaction: @@ -1462,7 +1478,8 @@ public void testPartialIntervalCompactWithFinerSegmentGranularityThenFullInterva final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(getSegmentCacheManagerFactory()) ); final Interval partialInterval = Intervals.of("2014-01-01T01:00:00/2014-01-01T02:00:00"); @@ -1533,7 +1550,8 @@ public void testRunIndexAndCompactForSameSegmentAtTheSameTime() throws Exception final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(getSegmentCacheManagerFactory()) ); final CompactionTask compactionTask = builder @@ -1587,7 +1605,8 @@ public void testRunIndexAndCompactForSameSegmentAtTheSameTime2() throws Exceptio final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(getSegmentCacheManagerFactory()) ); final CompactionTask compactionTask = builder @@ -1677,7 +1696,8 @@ public void testRunWithSpatialDimensions() throws Exception final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(getSegmentCacheManagerFactory()) ); final CompactionTask compactionTask = builder @@ -1808,7 +1828,8 @@ public void testRunWithAutoCastDimensions() throws Exception final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(getSegmentCacheManagerFactory()) ); final CompactionTask compactionTask = builder diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index ec8502c41c71..4501ea0121d9 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -400,7 +400,8 @@ public void testCreateCompactionTaskWithGranularitySpec() { final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(segmentCacheManagerFactory) ); builder.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); builder.tuningConfig(createTuningConfig()); @@ -409,7 +410,8 @@ public void testCreateCompactionTaskWithGranularitySpec() final Builder builder2 = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(segmentCacheManagerFactory) ); builder2.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); builder2.tuningConfig(createTuningConfig()); @@ -426,7 +428,8 @@ public void testCompactionTaskEmitter() { final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(segmentCacheManagerFactory) ); builder.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); builder.tuningConfig(createTuningConfig()); @@ -446,7 +449,8 @@ public void testCreateCompactionTaskWithConflictingGranularitySpecAndSegmentGran { final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(segmentCacheManagerFactory) ); builder.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); builder.tuningConfig(createTuningConfig()); @@ -476,7 +480,8 @@ public void testCreateCompactionTaskWithTransformSpec() new ClientCompactionTaskTransformSpec(new SelectorDimFilter("dim1", "foo", null)); final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(segmentCacheManagerFactory) ); builder.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); builder.tuningConfig(createTuningConfig()); @@ -494,7 +499,8 @@ public void testCreateCompactionTaskWithMetricsSpec() AggregatorFactory[] aggregatorFactories = new AggregatorFactory[]{new CountAggregatorFactory("cnt")}; final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(segmentCacheManagerFactory) ); builder.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); builder.tuningConfig(createTuningConfig()); @@ -511,7 +517,8 @@ public void testCreateCompactionTaskWithNullSegmentGranularityInGranularitySpecA { final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(segmentCacheManagerFactory) ); builder.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); builder.tuningConfig(createTuningConfig()); @@ -539,7 +546,8 @@ public void testCreateCompactionTaskWithSameGranularitySpecAndSegmentGranularity { final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(segmentCacheManagerFactory) ); builder.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); builder.tuningConfig(createTuningConfig()); @@ -554,7 +562,8 @@ public void testSerdeWithInterval() throws IOException { final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(segmentCacheManagerFactory) ); final CompactionTask task = builder .inputSpec( @@ -574,7 +583,8 @@ public void testSerdeWithSegments() throws IOException { final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(segmentCacheManagerFactory) ); final CompactionTask task = builder .segments(SEGMENTS) @@ -592,7 +602,8 @@ public void testSerdeWithDimensions() throws IOException { final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(segmentCacheManagerFactory) ); final CompactionTask task = builder @@ -674,7 +685,8 @@ public void testSerdeWithOldTuningConfigSuccessfullyDeserializeToNewOne() throws final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(segmentCacheManagerFactory) ); final CompactionTask expectedFromJson = builder @@ -694,7 +706,8 @@ public void testInputSourceResources() { final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(segmentCacheManagerFactory) ); final CompactionTask task = builder .inputSpec( @@ -1468,7 +1481,8 @@ public void testEmptyInterval() final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(segmentCacheManagerFactory) ); @SuppressWarnings("unused") @@ -1822,7 +1836,8 @@ public void testGetDefaultLookupLoadingSpec() { final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(segmentCacheManagerFactory) ); final CompactionTask task = builder .interval(Intervals.of("2000-01-01/2000-01-02")) @@ -1835,7 +1850,8 @@ public void testGetDefaultLookupLoadingSpecWithTransformSpec() { final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(segmentCacheManagerFactory) ); final CompactionTask task = builder .interval(Intervals.of("2000-01-01/2000-01-02")) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java index 241e5f2de081..88707206236f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java @@ -33,6 +33,7 @@ import org.apache.druid.indexing.common.RetryPolicyFactory; import org.apache.druid.indexing.common.task.CompactionTask; import org.apache.druid.indexing.common.task.CompactionTask.Builder; +import org.apache.druid.indexing.common.task.NativeCompactionRunner; import org.apache.druid.indexing.common.task.SpecificSegmentsSpec; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; @@ -241,7 +242,8 @@ private Builder newCompactionTaskBuilder() { return new Builder( DATASOURCE, - RETRY_POLICY_FACTORY + RETRY_POLICY_FACTORY, + new NativeCompactionRunner(getSegmentCacheManagerFactory()) ); } } From 7df2dc87b8834a71cd065a8ff2a312172be03125 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Tue, 28 May 2024 11:49:47 +0530 Subject: [PATCH 29/54] Fix more tests --- .../org/apache/druid/indexing/common/task/CompactionTask.java | 2 +- .../test/java/org/apache/druid/timeline/DataSegmentTest.java | 2 +- .../druid/server/coordinator/compact/CompactionStatus.java | 3 ++- .../java/org/apache/druid/server/http/DataSegmentPlusTest.java | 2 +- 4 files changed, 5 insertions(+), 4 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index df4cdea4ccbc..e103b7fce985 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -477,7 +477,7 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception Pair supportsCompactionConfig = compactionRunner.supportsCompactionConfig(this); if (!supportsCompactionConfig.lhs) { throw InvalidInput.exception( - "Compaction config not supported[%s].", + "Compaction config not supported. Reason[%s].", supportsCompactionConfig.rhs ); } diff --git a/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java b/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java index 36e0637afe1f..86033011a515 100644 --- a/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java +++ b/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java @@ -155,7 +155,7 @@ public void testV1Serialization() throws Exception Assert.assertEquals(ImmutableMap.of("type", "numbered", "partitionNum", 3, "partitions", 0), objectMap.get("shardSpec")); Assert.assertEquals(TEST_VERSION, objectMap.get("binaryVersion")); Assert.assertEquals(1, objectMap.get("size")); - Assert.assertEquals(7, ((Map) objectMap.get("lastCompactionState")).size()); + Assert.assertEquals(8, ((Map) objectMap.get("lastCompactionState")).size()); DataSegment deserializedSegment = MAPPER.readValue(MAPPER.writeValueAsString(segment), DataSegment.class); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java index 60fa7cb957c0..8388e8bf4bc9 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java @@ -266,7 +266,8 @@ private CompactionStatus segmentGranularityIsUpToDate() private CompactionStatus rollupIsUpToDate() { // MSQ considers a query as rollup only if finalizeAggregations=false. Compaction can have that set to true. - if (configuredGranularitySpec == null || (configuredGranularitySpec.isRollup() + if (configuredGranularitySpec == null || (configuredGranularitySpec.isRollup() != null + && configuredGranularitySpec.isRollup() && lastCompactionState.getEngine() == CompactionEngine.MSQ)) { return COMPLETE; } else { diff --git a/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java b/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java index a18d8a4f4879..2e489630100c 100644 --- a/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java +++ b/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java @@ -128,7 +128,7 @@ public void testSerde() throws JsonProcessingException Assert.assertEquals(ImmutableMap.of("type", "numbered", "partitionNum", 3, "partitions", 0), segmentObjectMap.get("shardSpec")); Assert.assertEquals(TEST_VERSION, segmentObjectMap.get("binaryVersion")); Assert.assertEquals(1, segmentObjectMap.get("size")); - Assert.assertEquals(6, ((Map) segmentObjectMap.get("lastCompactionState")).size()); + Assert.assertEquals(8, ((Map) segmentObjectMap.get("lastCompactionState")).size()); // verify extra metadata Assert.assertEquals(createdDateStr, objectMap.get("createdDate")); From 6d893fcd0f56b2d87fc6fbefd77c75c2eeec7e5b Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Tue, 28 May 2024 13:50:10 +0530 Subject: [PATCH 30/54] Add segmentCacheManagerFactory in CompactionTask and update tests --- .../msq/indexing/MSQCompactionRunnerTest.java | 2 +- .../indexing/common/task/CompactionTask.java | 16 +++-- .../ClientCompactionTaskQuerySerdeTest.java | 3 +- .../task/CompactionTaskParallelRunTest.java | 42 ++++++++----- .../common/task/CompactionTaskRunTest.java | 63 ++++++++++++------- .../common/task/CompactionTaskTest.java | 53 +++++++++------- .../batch/parallel/PartialCompactionTest.java | 4 +- .../IncrementalIndexStorageAdapterTest.java | 4 +- 8 files changed, 121 insertions(+), 66 deletions(-) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java index e19e12ca7e01..fbd7b4b98dff 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java @@ -33,7 +33,6 @@ import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexing.common.task.CompactionIntervalSpec; import org.apache.druid.indexing.common.task.CompactionTask; -import org.apache.druid.indexing.common.task.NativeCompactionRunner; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.NonnullPair; @@ -122,6 +121,7 @@ public void testCompactionToMSQTasks() throws JsonProcessingException final CompactionTask.Builder builder = new CompactionTask.Builder( DATA_SOURCE, null, + null, new MSQCompactionRunner(JSON_MAPPER, null) ); IndexSpec indexSpec = IndexSpec.builder() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index e103b7fce985..3d7ea42ec3dd 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -19,6 +19,7 @@ package org.apache.druid.indexing.common.task; +import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonInclude; @@ -48,6 +49,7 @@ import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.RetryPolicyFactory; +import org.apache.druid.indexing.common.SegmentCacheManagerFactory; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction; import org.apache.druid.indexing.common.actions.TaskActionClient; @@ -175,7 +177,8 @@ public CompactionTask( @JsonProperty("granularitySpec") @Nullable final ClientCompactionTaskGranularitySpec granularitySpec, @JsonProperty("tuningConfig") @Nullable final TuningConfig tuningConfig, @JsonProperty("context") @Nullable final Map context, - @JsonProperty("compactionRunner") final CompactionRunner compactionRunner + @JsonProperty("compactionRunner") final CompactionRunner compactionRunner, + @JacksonInject SegmentCacheManagerFactory segmentCacheManagerFactory ) { super( @@ -194,7 +197,6 @@ public CompactionTask( new Property<>("segments", segments) ) ); - Preconditions.checkNotNull(compactionRunner, "compactionRunner cannot be null"); if (ioConfig != null) { this.ioConfig = ioConfig; } else if (interval != null) { @@ -228,7 +230,9 @@ public CompactionTask( } this.tuningConfig = tuningConfig != null ? getTuningConfig(tuningConfig) : null; this.segmentProvider = new SegmentProvider(dataSource, this.ioConfig.getInputSpec()); - this.compactionRunner = compactionRunner; + this.compactionRunner = compactionRunner == null + ? new NativeCompactionRunner(segmentCacheManagerFactory) + : compactionRunner; this.currentSubTaskHolder = compactionRunner.getCurrentSubTaskHolder(); // Do not load any lookups in sub-tasks launched by compaction task, unless transformSpec is present. @@ -1028,6 +1032,7 @@ void checkSegments(LockGranularity lockGranularityInUse, List lates public static class Builder { private final String dataSource; + private final SegmentCacheManagerFactory segmentCacheManagerFactory; private final RetryPolicyFactory retryPolicyFactory; private CompactionIOConfig ioConfig; @@ -1049,11 +1054,13 @@ public static class Builder public Builder( String dataSource, + SegmentCacheManagerFactory segmentCacheManagerFactory, RetryPolicyFactory retryPolicyFactory, CompactionRunner compactionRunner ) { this.dataSource = dataSource; + this.segmentCacheManagerFactory = segmentCacheManagerFactory; this.retryPolicyFactory = retryPolicyFactory; this.compactionRunner = compactionRunner; } @@ -1151,7 +1158,8 @@ public CompactionTask build() granularitySpec, tuningConfig, context, - compactionRunner + compactionRunner, + segmentCacheManagerFactory ); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java index 89ca3609553e..3f19da85ab16 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java @@ -339,8 +339,9 @@ private CompactionTask createCompactionTask(ClientCompactionTaskTransformSpec tr { CompactionTask.Builder compactionTaskBuilder = new CompactionTask.Builder( "datasource", + new SegmentCacheManagerFactory(new TestUtils().getTestObjectMapper()), new RetryPolicyFactory(new RetryPolicyConfig()), - new NativeCompactionRunner(new SegmentCacheManagerFactory(new TestUtils().getTestObjectMapper()))) + null) .inputSpec(new CompactionIntervalSpec(Intervals.of("2019/2020"), "testSha256OfSortedSegmentIds"), true) .tuningConfig( new ParallelIndexTuningConfig( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java index f41a6f9ca563..dd17365ab668 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java @@ -161,8 +161,9 @@ public void testRunParallelWithDynamicPartitioningMatchCompactionState() throws final Builder builder = new Builder( DATA_SOURCE, + getSegmentCacheManagerFactory(), RETRY_POLICY_FACTORY, - new NativeCompactionRunner(getSegmentCacheManagerFactory()) + null ); final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) @@ -217,8 +218,9 @@ public void testRunParallelWithHashPartitioningMatchCompactionState() throws Exc final Builder builder = new Builder( DATA_SOURCE, + getSegmentCacheManagerFactory(), RETRY_POLICY_FACTORY, - new NativeCompactionRunner(getSegmentCacheManagerFactory()) + null ); final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) @@ -284,8 +286,9 @@ public void testRunParallelWithRangePartitioning() throws Exception final Builder builder = new Builder( DATA_SOURCE, + getSegmentCacheManagerFactory(), RETRY_POLICY_FACTORY, - new NativeCompactionRunner(getSegmentCacheManagerFactory()) + null ); final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) @@ -336,8 +339,9 @@ public void testRunParallelWithRangePartitioningAndNoUpfrontSegmentFetching() th final Builder builder = new Builder( DATA_SOURCE, + getSegmentCacheManagerFactory(), RETRY_POLICY_FACTORY, - new NativeCompactionRunner(getSegmentCacheManagerFactory()) + null ); final CompactionTask compactionTask = builder @@ -400,8 +404,9 @@ public void testRunParallelWithMultiDimensionRangePartitioning() throws Exceptio final Builder builder = new Builder( DATA_SOURCE, + getSegmentCacheManagerFactory(), RETRY_POLICY_FACTORY, - new NativeCompactionRunner(getSegmentCacheManagerFactory()) + null ); final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) @@ -455,8 +460,9 @@ public void testRunParallelWithRangePartitioningWithSingleTask() throws Exceptio final Builder builder = new Builder( DATA_SOURCE, + getSegmentCacheManagerFactory(), RETRY_POLICY_FACTORY, - new NativeCompactionRunner(getSegmentCacheManagerFactory()) + null ); final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) @@ -507,8 +513,9 @@ public void testRunParallelWithMultiDimensionRangePartitioningWithSingleTask() t final Builder builder = new Builder( DATA_SOURCE, + getSegmentCacheManagerFactory(), RETRY_POLICY_FACTORY, - new NativeCompactionRunner(getSegmentCacheManagerFactory()) + null ); final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) @@ -559,8 +566,9 @@ public void testRunCompactionStateNotStoreIfContextSetToFalse() final Builder builder = new Builder( DATA_SOURCE, + getSegmentCacheManagerFactory(), RETRY_POLICY_FACTORY, - new NativeCompactionRunner(getSegmentCacheManagerFactory()) + null ); final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) @@ -590,8 +598,9 @@ public void testRunCompactionWithFilterShouldStoreInState() throws Exception final Builder builder = new Builder( DATA_SOURCE, + getSegmentCacheManagerFactory(), RETRY_POLICY_FACTORY, - new NativeCompactionRunner(getSegmentCacheManagerFactory()) + null ); final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) @@ -648,8 +657,9 @@ public void testRunCompactionWithNewMetricsShouldStoreInState() throws Exception final Builder builder = new Builder( DATA_SOURCE, + getSegmentCacheManagerFactory(), RETRY_POLICY_FACTORY, - new NativeCompactionRunner(getSegmentCacheManagerFactory()) + null ); final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) @@ -712,8 +722,9 @@ public void testCompactHashAndDynamicPartitionedSegments() runIndexTask(null, true); final Builder builder = new Builder( DATA_SOURCE, + getSegmentCacheManagerFactory(), RETRY_POLICY_FACTORY, - new NativeCompactionRunner(getSegmentCacheManagerFactory()) + null ); final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) @@ -762,8 +773,9 @@ public void testCompactRangeAndDynamicPartitionedSegments() runIndexTask(null, true); final Builder builder = new Builder( DATA_SOURCE, + getSegmentCacheManagerFactory(), RETRY_POLICY_FACTORY, - new NativeCompactionRunner(getSegmentCacheManagerFactory()) + null ); final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) @@ -854,8 +866,9 @@ public void testCompactionDropSegmentsOfInputIntervalIfDropFlagIsSet() throws Ex final Builder builder = new Builder( DATA_SOURCE, + getSegmentCacheManagerFactory(), RETRY_POLICY_FACTORY, - new NativeCompactionRunner(getSegmentCacheManagerFactory()) + null ); final CompactionTask compactionTask = builder // Set the dropExisting flag to true in the IOConfig of the compaction task @@ -901,8 +914,9 @@ public void testCompactionDoesNotDropSegmentsIfDropFlagNotSet() throws Exception final Builder builder = new Builder( DATA_SOURCE, + getSegmentCacheManagerFactory(), RETRY_POLICY_FACTORY, - new NativeCompactionRunner(getSegmentCacheManagerFactory()) + null ); final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index d2313d81b919..b2e6494f62fc 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -283,8 +283,9 @@ public void testRunWithDynamicPartitioning() throws Exception final Builder builder = new Builder( DATA_SOURCE, + segmentCacheManagerFactory, RETRY_POLICY_FACTORY, - new NativeCompactionRunner(getSegmentCacheManagerFactory()) + null ); final CompactionTask compactionTask = builder @@ -351,8 +352,9 @@ public void testRunWithHashPartitioning() throws Exception final Builder builder = new Builder( DATA_SOURCE, + segmentCacheManagerFactory, RETRY_POLICY_FACTORY, - new NativeCompactionRunner(getSegmentCacheManagerFactory()) + null ); final CompactionTask compactionTask = builder @@ -452,8 +454,9 @@ public void testRunCompactionTwice() throws Exception final Builder builder = new Builder( DATA_SOURCE, + segmentCacheManagerFactory, RETRY_POLICY_FACTORY, - new NativeCompactionRunner(getSegmentCacheManagerFactory()) + null ); final CompactionTask compactionTask1 = builder @@ -547,8 +550,9 @@ public void testRunIndexAndCompactAtTheSameTimeForDifferentInterval() throws Exc final Builder builder = new Builder( DATA_SOURCE, + segmentCacheManagerFactory, RETRY_POLICY_FACTORY, - new NativeCompactionRunner(getSegmentCacheManagerFactory()) + null ); final CompactionTask compactionTask = builder @@ -659,8 +663,9 @@ public void testWithSegmentGranularity() throws Exception final Builder builder = new Builder( DATA_SOURCE, + segmentCacheManagerFactory, RETRY_POLICY_FACTORY, - new NativeCompactionRunner(getSegmentCacheManagerFactory()) + null ); // day segmentGranularity @@ -729,8 +734,9 @@ public void testWithSegmentGranularityMisalignedInterval() throws Exception final Builder builder = new Builder( DATA_SOURCE, + segmentCacheManagerFactory, RETRY_POLICY_FACTORY, - new NativeCompactionRunner(getSegmentCacheManagerFactory()) + null ); final CompactionTask compactionTask1 = builder @@ -764,8 +770,9 @@ public void testWithSegmentGranularityMisalignedIntervalAllowed() throws Excepti final Builder builder = new Builder( DATA_SOURCE, + segmentCacheManagerFactory, RETRY_POLICY_FACTORY, - new NativeCompactionRunner(getSegmentCacheManagerFactory()) + null ); // day segmentGranularity @@ -809,8 +816,9 @@ public void testCompactionWithFilterInTransformSpec() throws Exception final Builder builder = new Builder( DATA_SOURCE, + segmentCacheManagerFactory, RETRY_POLICY_FACTORY, - new NativeCompactionRunner(getSegmentCacheManagerFactory()) + null ); // day segmentGranularity @@ -870,8 +878,9 @@ public void testCompactionWithNewMetricInMetricsSpec() throws Exception final Builder builder = new Builder( DATA_SOURCE, + segmentCacheManagerFactory, RETRY_POLICY_FACTORY, - new NativeCompactionRunner(getSegmentCacheManagerFactory()) + null ); // day segmentGranularity @@ -937,8 +946,9 @@ public void testWithGranularitySpecNonNullSegmentGranularityAndNullQueryGranular final Builder builder = new Builder( DATA_SOURCE, + segmentCacheManagerFactory, RETRY_POLICY_FACTORY, - new NativeCompactionRunner(getSegmentCacheManagerFactory()) + null ); // day segmentGranularity @@ -1006,8 +1016,9 @@ public void testWithGranularitySpecNonNullQueryGranularityAndNullSegmentGranular final Builder builder = new Builder( DATA_SOURCE, + segmentCacheManagerFactory, RETRY_POLICY_FACTORY, - new NativeCompactionRunner(getSegmentCacheManagerFactory()) + null ); // day queryGranularity @@ -1060,8 +1071,9 @@ public void testWithGranularitySpecNonNullQueryGranularityAndNonNullSegmentGranu final Builder builder = new Builder( DATA_SOURCE, + segmentCacheManagerFactory, RETRY_POLICY_FACTORY, - new NativeCompactionRunner(getSegmentCacheManagerFactory()) + null ); // day segmentGranularity and day queryGranularity @@ -1099,8 +1111,9 @@ public void testWithGranularitySpecNullQueryGranularityAndNullSegmentGranularity final Builder builder = new Builder( DATA_SOURCE, + segmentCacheManagerFactory, RETRY_POLICY_FACTORY, - new NativeCompactionRunner(getSegmentCacheManagerFactory()) + null ); final CompactionTask compactionTask1 = builder @@ -1152,8 +1165,9 @@ public void testCompactThenAppend() throws Exception final Builder builder = new Builder( DATA_SOURCE, + segmentCacheManagerFactory, RETRY_POLICY_FACTORY, - new NativeCompactionRunner(getSegmentCacheManagerFactory()) + null ); final CompactionTask compactionTask = builder @@ -1214,8 +1228,9 @@ public void testPartialIntervalCompactWithFinerSegmentGranularityThenFullInterva final Builder builder = new Builder( DATA_SOURCE, + segmentCacheManagerFactory, RETRY_POLICY_FACTORY, - new NativeCompactionRunner(getSegmentCacheManagerFactory()) + null ); // Setup partial compaction: @@ -1370,8 +1385,9 @@ public void testCompactDatasourceOverIntervalWithOnlyTombstones() throws Excepti final Builder builder = new Builder( DATA_SOURCE, + segmentCacheManagerFactory, RETRY_POLICY_FACTORY, - new NativeCompactionRunner(getSegmentCacheManagerFactory()) + null ); // Setup partial interval compaction: @@ -1478,8 +1494,9 @@ public void testPartialIntervalCompactWithFinerSegmentGranularityThenFullInterva final Builder builder = new Builder( DATA_SOURCE, + segmentCacheManagerFactory, RETRY_POLICY_FACTORY, - new NativeCompactionRunner(getSegmentCacheManagerFactory()) + null ); final Interval partialInterval = Intervals.of("2014-01-01T01:00:00/2014-01-01T02:00:00"); @@ -1550,8 +1567,9 @@ public void testRunIndexAndCompactForSameSegmentAtTheSameTime() throws Exception final Builder builder = new Builder( DATA_SOURCE, + segmentCacheManagerFactory, RETRY_POLICY_FACTORY, - new NativeCompactionRunner(getSegmentCacheManagerFactory()) + null ); final CompactionTask compactionTask = builder @@ -1605,8 +1623,9 @@ public void testRunIndexAndCompactForSameSegmentAtTheSameTime2() throws Exceptio final Builder builder = new Builder( DATA_SOURCE, + segmentCacheManagerFactory, RETRY_POLICY_FACTORY, - new NativeCompactionRunner(getSegmentCacheManagerFactory()) + null ); final CompactionTask compactionTask = builder @@ -1696,8 +1715,9 @@ public void testRunWithSpatialDimensions() throws Exception final Builder builder = new Builder( DATA_SOURCE, + segmentCacheManagerFactory, RETRY_POLICY_FACTORY, - new NativeCompactionRunner(getSegmentCacheManagerFactory()) + null ); final CompactionTask compactionTask = builder @@ -1828,8 +1848,9 @@ public void testRunWithAutoCastDimensions() throws Exception final Builder builder = new Builder( DATA_SOURCE, + segmentCacheManagerFactory, RETRY_POLICY_FACTORY, - new NativeCompactionRunner(getSegmentCacheManagerFactory()) + null ); final CompactionTask compactionTask = builder diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index 4501ea0121d9..7ecccb28253b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -400,8 +400,9 @@ public void testCreateCompactionTaskWithGranularitySpec() { final Builder builder = new Builder( DATA_SOURCE, + segmentCacheManagerFactory, RETRY_POLICY_FACTORY, - new NativeCompactionRunner(segmentCacheManagerFactory) + null ); builder.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); builder.tuningConfig(createTuningConfig()); @@ -410,8 +411,9 @@ public void testCreateCompactionTaskWithGranularitySpec() final Builder builder2 = new Builder( DATA_SOURCE, + segmentCacheManagerFactory, RETRY_POLICY_FACTORY, - new NativeCompactionRunner(segmentCacheManagerFactory) + null ); builder2.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); builder2.tuningConfig(createTuningConfig()); @@ -428,8 +430,9 @@ public void testCompactionTaskEmitter() { final Builder builder = new Builder( DATA_SOURCE, + segmentCacheManagerFactory, RETRY_POLICY_FACTORY, - new NativeCompactionRunner(segmentCacheManagerFactory) + null ); builder.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); builder.tuningConfig(createTuningConfig()); @@ -449,8 +452,8 @@ public void testCreateCompactionTaskWithConflictingGranularitySpecAndSegmentGran { final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY, - new NativeCompactionRunner(segmentCacheManagerFactory) + segmentCacheManagerFactory, RETRY_POLICY_FACTORY, + null ); builder.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); builder.tuningConfig(createTuningConfig()); @@ -480,8 +483,8 @@ public void testCreateCompactionTaskWithTransformSpec() new ClientCompactionTaskTransformSpec(new SelectorDimFilter("dim1", "foo", null)); final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY, - new NativeCompactionRunner(segmentCacheManagerFactory) + segmentCacheManagerFactory, RETRY_POLICY_FACTORY, + null ); builder.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); builder.tuningConfig(createTuningConfig()); @@ -499,8 +502,9 @@ public void testCreateCompactionTaskWithMetricsSpec() AggregatorFactory[] aggregatorFactories = new AggregatorFactory[]{new CountAggregatorFactory("cnt")}; final Builder builder = new Builder( DATA_SOURCE, + segmentCacheManagerFactory, RETRY_POLICY_FACTORY, - new NativeCompactionRunner(segmentCacheManagerFactory) + null ); builder.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); builder.tuningConfig(createTuningConfig()); @@ -517,8 +521,9 @@ public void testCreateCompactionTaskWithNullSegmentGranularityInGranularitySpecA { final Builder builder = new Builder( DATA_SOURCE, + segmentCacheManagerFactory, RETRY_POLICY_FACTORY, - new NativeCompactionRunner(segmentCacheManagerFactory) + null ); builder.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); builder.tuningConfig(createTuningConfig()); @@ -546,8 +551,9 @@ public void testCreateCompactionTaskWithSameGranularitySpecAndSegmentGranularity { final Builder builder = new Builder( DATA_SOURCE, + segmentCacheManagerFactory, RETRY_POLICY_FACTORY, - new NativeCompactionRunner(segmentCacheManagerFactory) + null ); builder.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); builder.tuningConfig(createTuningConfig()); @@ -562,8 +568,9 @@ public void testSerdeWithInterval() throws IOException { final Builder builder = new Builder( DATA_SOURCE, + segmentCacheManagerFactory, RETRY_POLICY_FACTORY, - new NativeCompactionRunner(segmentCacheManagerFactory) + null ); final CompactionTask task = builder .inputSpec( @@ -583,8 +590,9 @@ public void testSerdeWithSegments() throws IOException { final Builder builder = new Builder( DATA_SOURCE, + segmentCacheManagerFactory, RETRY_POLICY_FACTORY, - new NativeCompactionRunner(segmentCacheManagerFactory) + null ); final CompactionTask task = builder .segments(SEGMENTS) @@ -602,8 +610,9 @@ public void testSerdeWithDimensions() throws IOException { final Builder builder = new Builder( DATA_SOURCE, + segmentCacheManagerFactory, RETRY_POLICY_FACTORY, - new NativeCompactionRunner(segmentCacheManagerFactory) + null ); final CompactionTask task = builder @@ -685,8 +694,9 @@ public void testSerdeWithOldTuningConfigSuccessfullyDeserializeToNewOne() throws final Builder builder = new Builder( DATA_SOURCE, + segmentCacheManagerFactory, RETRY_POLICY_FACTORY, - new NativeCompactionRunner(segmentCacheManagerFactory) + null ); final CompactionTask expectedFromJson = builder @@ -706,8 +716,9 @@ public void testInputSourceResources() { final Builder builder = new Builder( DATA_SOURCE, + segmentCacheManagerFactory, RETRY_POLICY_FACTORY, - new NativeCompactionRunner(segmentCacheManagerFactory) + null ); final CompactionTask task = builder .inputSpec( @@ -1481,8 +1492,8 @@ public void testEmptyInterval() final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY, - new NativeCompactionRunner(segmentCacheManagerFactory) + segmentCacheManagerFactory, RETRY_POLICY_FACTORY, + null ); @SuppressWarnings("unused") @@ -1836,8 +1847,8 @@ public void testGetDefaultLookupLoadingSpec() { final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY, - new NativeCompactionRunner(segmentCacheManagerFactory) + segmentCacheManagerFactory, RETRY_POLICY_FACTORY, + null ); final CompactionTask task = builder .interval(Intervals.of("2000-01-01/2000-01-02")) @@ -1850,8 +1861,8 @@ public void testGetDefaultLookupLoadingSpecWithTransformSpec() { final Builder builder = new Builder( DATA_SOURCE, - RETRY_POLICY_FACTORY, - new NativeCompactionRunner(segmentCacheManagerFactory) + segmentCacheManagerFactory, RETRY_POLICY_FACTORY, + null ); final CompactionTask task = builder .interval(Intervals.of("2000-01-01/2000-01-02")) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java index 88707206236f..a09739049155 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java @@ -242,8 +242,8 @@ private Builder newCompactionTaskBuilder() { return new Builder( DATASOURCE, - RETRY_POLICY_FACTORY, - new NativeCompactionRunner(getSegmentCacheManagerFactory()) + getSegmentCacheManagerFactory(), RETRY_POLICY_FACTORY, + null ); } } diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java index 0d0f979f715b..b200f4f5955a 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java @@ -165,10 +165,10 @@ public void testSanity() throws Exception Assert.assertEquals(2, results.size()); - ResultRow row = results.get(0); + ResultRow row = results.get(1); Assert.assertArrayEquals(new Object[]{NullHandling.defaultStringValue(), "bo", 1L}, row.getArray()); - row = results.get(1); + row = results.get(0); Assert.assertArrayEquals(new Object[]{"hi", NullHandling.defaultStringValue(), 1L}, row.getArray()); } } From 30770aca12acbb1fed90c85ff3cd1819effb448a Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Tue, 28 May 2024 16:59:37 +0530 Subject: [PATCH 31/54] Minor fix --- .../org/apache/druid/indexing/common/task/CompactionTask.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 3d7ea42ec3dd..0418a4425990 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -233,7 +233,7 @@ public CompactionTask( this.compactionRunner = compactionRunner == null ? new NativeCompactionRunner(segmentCacheManagerFactory) : compactionRunner; - this.currentSubTaskHolder = compactionRunner.getCurrentSubTaskHolder(); + this.currentSubTaskHolder = this.compactionRunner.getCurrentSubTaskHolder(); // Do not load any lookups in sub-tasks launched by compaction task, unless transformSpec is present. // If transformSpec is present, we will not modify the context so that the sub-tasks can make the From 0ce405b16f55951012cebcc83b9efdaed8bb6737 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Wed, 29 May 2024 11:24:58 +0530 Subject: [PATCH 32/54] Fix style --- .../indexing/common/task/CompactionTask.java | 2 +- .../task/CompactionTaskParallelRunTest.java | 24 +- .../common/task/CompactionTaskRunTest.java | 12 +- .../batch/parallel/PartialCompactionTest.java | 4 +- .../IncrementalIndexStorageAdapterTest.java | 4 +- .../druid/timeline/DataSegmentTest.java | 12 +- .../compact/NewestSegmentFirstPolicyTest.java | 317 ++++++++++++++---- .../coordinator/duty/CompactSegmentsTest.java | 48 ++- .../server/http/DataSegmentPlusTest.java | 3 +- .../sql/calcite/schema/SystemSchemaTest.java | 3 +- 10 files changed, 317 insertions(+), 112 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 0418a4425990..f378753d899e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -1060,7 +1060,7 @@ public Builder( ) { this.dataSource = dataSource; - this.segmentCacheManagerFactory = segmentCacheManagerFactory; + this.segmentCacheManagerFactory = segmentCacheManagerFactory; this.retryPolicyFactory = retryPolicyFactory; this.compactionRunner = compactionRunner; } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java index dd17365ab668..08e7ba809bad 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java @@ -187,7 +187,8 @@ public void testRunParallelWithDynamicPartitioningMatchCompactionState() throws CompactionState expectedState = new CompactionState( new DynamicPartitionsSpec(null, Long.MAX_VALUE), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), - Collections.emptyMap(), ImmutableList.of(expectedLongSumMetric), + Collections.emptyMap(), + ImmutableList.of(expectedLongSumMetric), null, compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), getObjectMapper().readValue( @@ -240,7 +241,8 @@ public void testRunParallelWithHashPartitioningMatchCompactionState() throws Exc CompactionState expectedState = new CompactionState( new HashedPartitionsSpec(null, 3, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), - Collections.emptyMap(), ImmutableList.of(expectedLongSumMetric), + Collections.emptyMap(), + ImmutableList.of(expectedLongSumMetric), null, compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), getObjectMapper().readValue( @@ -308,7 +310,8 @@ public void testRunParallelWithRangePartitioning() throws Exception CompactionState expectedState = new CompactionState( new SingleDimensionPartitionsSpec(7, null, "dim", false), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), - Collections.emptyMap(), ImmutableList.of(expectedLongSumMetric), + Collections.emptyMap(), + ImmutableList.of(expectedLongSumMetric), null, compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), getObjectMapper().readValue( @@ -371,7 +374,8 @@ public void testRunParallelWithRangePartitioningAndNoUpfrontSegmentFetching() th CompactionState expectedState = new CompactionState( new SingleDimensionPartitionsSpec(7, null, "dim", false), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), - Collections.emptyMap(), ImmutableList.of(expectedLongSumMetric), + Collections.emptyMap(), + ImmutableList.of(expectedLongSumMetric), null, compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), getObjectMapper().readValue( @@ -429,7 +433,8 @@ public void testRunParallelWithMultiDimensionRangePartitioning() throws Exceptio CompactionState expectedState = new CompactionState( new DimensionRangePartitionsSpec(7, null, Arrays.asList("dim1", "dim2"), false), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), - Collections.emptyMap(), ImmutableList.of(expectedLongSumMetric), + Collections.emptyMap(), + ImmutableList.of(expectedLongSumMetric), null, compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), getObjectMapper().readValue( @@ -482,7 +487,8 @@ public void testRunParallelWithRangePartitioningWithSingleTask() throws Exceptio CompactionState expectedState = new CompactionState( new SingleDimensionPartitionsSpec(7, null, "dim", false), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), - Collections.emptyMap(), ImmutableList.of(expectedLongSumMetric), + Collections.emptyMap(), + ImmutableList.of(expectedLongSumMetric), null, compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), getObjectMapper().readValue( @@ -626,7 +632,8 @@ public void testRunCompactionWithFilterShouldStoreInState() throws Exception CompactionState expectedState = new CompactionState( new DynamicPartitionsSpec(null, Long.MAX_VALUE), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), - Collections.emptyMap(), ImmutableList.of(expectedLongSumMetric), + Collections.emptyMap(), + ImmutableList.of(expectedLongSumMetric), getObjectMapper().readValue( getObjectMapper().writeValueAsString(compactionTask.getTransformSpec()), Map.class @@ -691,7 +698,8 @@ public void testRunCompactionWithNewMetricsShouldStoreInState() throws Exception CompactionState expectedState = new CompactionState( new DynamicPartitionsSpec(null, Long.MAX_VALUE), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), - Collections.emptyMap(), ImmutableList.of(expectedCountMetric, expectedLongSumMetric), + Collections.emptyMap(), + ImmutableList.of(expectedCountMetric, expectedLongSumMetric), getObjectMapper().readValue( getObjectMapper().writeValueAsString(compactionTask.getTransformSpec()), Map.class diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index b2e6494f62fc..660ebe13f51f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -243,7 +243,8 @@ public static CompactionState getDefaultCompactionState( return new CompactionState( new DynamicPartitionsSpec(5000000, Long.MAX_VALUE), expectedDims, - Collections.emptyMap(), ImmutableList.of(expectedMetric), + Collections.emptyMap(), + ImmutableList.of(expectedMetric), null, IndexSpec.DEFAULT.asMap(mapper), mapper.readValue( @@ -420,7 +421,8 @@ public void testRunWithHashPartitioning() throws Exception CompactionState expectedState = new CompactionState( new HashedPartitionsSpec(null, 3, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), - Collections.emptyMap(), ImmutableList.of(expectedLongSumMetric), + Collections.emptyMap(), + ImmutableList.of(expectedLongSumMetric), null, compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), getObjectMapper().readValue( @@ -848,7 +850,8 @@ public void testCompactionWithFilterInTransformSpec() throws Exception CompactionState expectedCompactionState = new CompactionState( new DynamicPartitionsSpec(5000000, Long.MAX_VALUE), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), - Collections.emptyMap(), ImmutableList.of(expectedLongSumMetric), + Collections.emptyMap(), + ImmutableList.of(expectedLongSumMetric), getObjectMapper().readValue(getObjectMapper().writeValueAsString(compactionTask.getTransformSpec()), Map.class), IndexSpec.DEFAULT.asMap(mapper), mapper.readValue( @@ -916,7 +919,8 @@ public void testCompactionWithNewMetricInMetricsSpec() throws Exception CompactionState expectedCompactionState = new CompactionState( new DynamicPartitionsSpec(5000000, Long.MAX_VALUE), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), - Collections.emptyMap(), ImmutableList.of(expectedCountMetric, expectedLongSumMetric), + Collections.emptyMap(), + ImmutableList.of(expectedCountMetric, expectedLongSumMetric), getObjectMapper().readValue(getObjectMapper().writeValueAsString(compactionTask.getTransformSpec()), Map.class), IndexSpec.DEFAULT.asMap(mapper), mapper.readValue( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java index a09739049155..6f591e5ee1f7 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java @@ -33,7 +33,6 @@ import org.apache.druid.indexing.common.RetryPolicyFactory; import org.apache.druid.indexing.common.task.CompactionTask; import org.apache.druid.indexing.common.task.CompactionTask.Builder; -import org.apache.druid.indexing.common.task.NativeCompactionRunner; import org.apache.druid.indexing.common.task.SpecificSegmentsSpec; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; @@ -242,7 +241,8 @@ private Builder newCompactionTaskBuilder() { return new Builder( DATASOURCE, - getSegmentCacheManagerFactory(), RETRY_POLICY_FACTORY, + getSegmentCacheManagerFactory(), + RETRY_POLICY_FACTORY, null ); } diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java index b200f4f5955a..0d0f979f715b 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java @@ -165,10 +165,10 @@ public void testSanity() throws Exception Assert.assertEquals(2, results.size()); - ResultRow row = results.get(1); + ResultRow row = results.get(0); Assert.assertArrayEquals(new Object[]{NullHandling.defaultStringValue(), "bo", 1L}, row.getArray()); - row = results.get(0); + row = results.get(1); Assert.assertArrayEquals(new Object[]{"hi", NullHandling.defaultStringValue(), 1L}, row.getArray()); } } diff --git a/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java b/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java index 86033011a515..b232930a74c1 100644 --- a/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java +++ b/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java @@ -130,7 +130,8 @@ public void testV1Serialization() throws Exception new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "bar", "foo")) ), - Collections.emptyMap(), ImmutableList.of(ImmutableMap.of("type", "count", "name", "count")), + Collections.emptyMap(), + ImmutableList.of(ImmutableMap.of("type", "count", "name", "count")), ImmutableMap.of("filter", ImmutableMap.of("type", "selector", "dimension", "dim1", "value", "foo")), ImmutableMap.of(), ImmutableMap.of(), @@ -196,7 +197,8 @@ public void testDeserializationDataSegmentLastCompactionStateWithNullSpecs() thr new CompactionState( new HashedPartitionsSpec(100000, null, ImmutableList.of("dim1")), null, - Collections.emptyMap(), null, + Collections.emptyMap(), + null, null, ImmutableMap.of(), ImmutableMap.of(), @@ -348,7 +350,8 @@ public void testWithLastCompactionState() final CompactionState compactionState = new CompactionState( new DynamicPartitionsSpec(null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar", "foo"))), - Collections.emptyMap(), ImmutableList.of(ImmutableMap.of("type", "count", "name", "count")), + Collections.emptyMap(), + ImmutableList.of(ImmutableMap.of("type", "count", "name", "count")), ImmutableMap.of("filter", ImmutableMap.of("type", "selector", "dimension", "dim1", "value", "foo")), Collections.singletonMap("test", "map"), Collections.singletonMap("test2", "map2"), @@ -391,7 +394,8 @@ public void testAnnotateWithLastCompactionState() final CompactionState compactionState = new CompactionState( dynamicPartitionsSpec, dimensionsSpec, - Collections.emptyMap(), metricsSpec, + Collections.emptyMap(), + metricsSpec, transformSpec, indexSpec, granularitySpec, diff --git a/server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicyTest.java b/server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicyTest.java index 83525eddbe5a..19a53810e513 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicyTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicyTest.java @@ -730,15 +730,31 @@ public void testIteratorReturnsNothingAsSegmentsWasCompactedAndHaveSameSegmentGr Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, Collections.emptyMap(), null, null, indexSpec, null, - CompactionEngine.NATIVE) + new CompactionState( + partitionsSpec, + null, + Collections.emptyMap(), + null, + null, + indexSpec, + null, + CompactionEngine.NATIVE + ) ), new SegmentGenerateSpec( Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, Collections.emptyMap(), null, null, indexSpec, null, - CompactionEngine.NATIVE) + new CompactionState( + partitionsSpec, + null, + Collections.emptyMap(), + null, + null, + indexSpec, + null, + CompactionEngine.NATIVE + ) ) ); @@ -765,20 +781,30 @@ public void testIteratorReturnsNothingAsSegmentsWasCompactedAndHaveSameSegmentGr Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, - Collections.emptyMap(), - null, null, indexSpec, ImmutableMap.of("segmentGranularity", "day"), - CompactionEngine.NATIVE + new CompactionState( + partitionsSpec, + null, + Collections.emptyMap(), + null, + null, + indexSpec, + ImmutableMap.of("segmentGranularity", "day"), + CompactionEngine.NATIVE ) ), new SegmentGenerateSpec( Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, - Collections.emptyMap(), - null, null, indexSpec, ImmutableMap.of("segmentGranularity", "day"), - CompactionEngine.NATIVE + new CompactionState( + partitionsSpec, + null, + Collections.emptyMap(), + null, + null, + indexSpec, + ImmutableMap.of("segmentGranularity", "day"), + CompactionEngine.NATIVE ) ) ); @@ -806,15 +832,31 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentSeg Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, Collections.emptyMap(), null, null, indexSpec, null, - CompactionEngine.NATIVE) + new CompactionState( + partitionsSpec, + null, + Collections.emptyMap(), + null, + null, + indexSpec, + null, + CompactionEngine.NATIVE + ) ), new SegmentGenerateSpec( Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, Collections.emptyMap(), null, null, indexSpec, null, - CompactionEngine.NATIVE) + new CompactionState( + partitionsSpec, + null, + Collections.emptyMap(), + null, + null, + indexSpec, + null, + CompactionEngine.NATIVE + ) ) ); @@ -851,20 +893,30 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentSeg Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, - Collections.emptyMap(), - null, null, indexSpec, ImmutableMap.of("segmentGranularity", "day"), - CompactionEngine.NATIVE + new CompactionState( + partitionsSpec, + null, + Collections.emptyMap(), + null, + null, + indexSpec, + ImmutableMap.of("segmentGranularity", "day"), + CompactionEngine.NATIVE ) ), new SegmentGenerateSpec( Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, - Collections.emptyMap(), - null, null, indexSpec, ImmutableMap.of("segmentGranularity", "day"), - CompactionEngine.NATIVE + new CompactionState( + partitionsSpec, + null, + Collections.emptyMap(), + null, + null, + indexSpec, + ImmutableMap.of("segmentGranularity", "day"), + CompactionEngine.NATIVE ) ) ); @@ -902,8 +954,16 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentTim Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, Collections.emptyMap(), null, null, indexSpec, null, - CompactionEngine.NATIVE) + new CompactionState( + partitionsSpec, + null, + Collections.emptyMap(), + null, + null, + indexSpec, + null, + CompactionEngine.NATIVE + ) ) ); @@ -955,8 +1015,16 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentOri Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, Collections.emptyMap(), null, null, indexSpec, null, - CompactionEngine.NATIVE) + new CompactionState( + partitionsSpec, + null, + Collections.emptyMap(), + null, + null, + indexSpec, + null, + CompactionEngine.NATIVE + ) ) ); @@ -1010,28 +1078,46 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentRol Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, - Collections.emptyMap(), - null, null, indexSpec, ImmutableMap.of("rollup", "false"), - CompactionEngine.NATIVE + new CompactionState( + partitionsSpec, + null, + Collections.emptyMap(), + null, + null, + indexSpec, + ImmutableMap.of("rollup", "false"), + CompactionEngine.NATIVE ) ), new SegmentGenerateSpec( Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, - Collections.emptyMap(), - null, null, indexSpec, ImmutableMap.of("rollup", "true"), - CompactionEngine.NATIVE + new CompactionState( + partitionsSpec, + null, + Collections.emptyMap(), + null, + null, + indexSpec, + ImmutableMap.of("rollup", "true"), + CompactionEngine.NATIVE ) ), new SegmentGenerateSpec( Intervals.of("2017-10-03T00:00:00/2017-10-04T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, Collections.emptyMap(), null, null, indexSpec, ImmutableMap.of(), - CompactionEngine.NATIVE) + new CompactionState( + partitionsSpec, + null, + Collections.emptyMap(), + null, + null, + indexSpec, + ImmutableMap.of(), + CompactionEngine.NATIVE + ) ) ); @@ -1079,28 +1165,46 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentQue Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, - Collections.emptyMap(), - null, null, indexSpec, ImmutableMap.of("queryGranularity", "day"), - CompactionEngine.NATIVE + new CompactionState( + partitionsSpec, + null, + Collections.emptyMap(), + null, + null, + indexSpec, + ImmutableMap.of("queryGranularity", "day"), + CompactionEngine.NATIVE ) ), new SegmentGenerateSpec( Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, - Collections.emptyMap(), - null, null, indexSpec, ImmutableMap.of("queryGranularity", "minute"), - CompactionEngine.NATIVE + new CompactionState( + partitionsSpec, + null, + Collections.emptyMap(), + null, + null, + indexSpec, + ImmutableMap.of("queryGranularity", "minute"), + CompactionEngine.NATIVE ) ), new SegmentGenerateSpec( Intervals.of("2017-10-03T00:00:00/2017-10-04T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, Collections.emptyMap(), null, null, indexSpec, ImmutableMap.of(), - CompactionEngine.NATIVE) + new CompactionState( + partitionsSpec, + null, + Collections.emptyMap(), + null, + null, + indexSpec, + ImmutableMap.of(), + CompactionEngine.NATIVE + ) ) ); @@ -1149,38 +1253,61 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentDim Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar", "foo"))), - Collections.emptyMap(), - null, null, indexSpec, null, - CompactionEngine.NATIVE + new CompactionState( + partitionsSpec, + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar", "foo"))), + Collections.emptyMap(), + null, + null, + indexSpec, + null, + CompactionEngine.NATIVE ) ), new SegmentGenerateSpec( Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("foo"))), - Collections.emptyMap(), - null, null, indexSpec, null, - CompactionEngine.NATIVE + new CompactionState( + partitionsSpec, + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("foo"))), + Collections.emptyMap(), + null, + null, + indexSpec, + null, + CompactionEngine.NATIVE ) ), new SegmentGenerateSpec( Intervals.of("2017-10-03T00:00:00/2017-10-04T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, DimensionsSpec.EMPTY, - Collections.emptyMap(), - null, null, indexSpec, null, - CompactionEngine.NATIVE + new CompactionState( + partitionsSpec, + DimensionsSpec.EMPTY, + Collections.emptyMap(), + null, + null, + indexSpec, + null, + CompactionEngine.NATIVE ) ), new SegmentGenerateSpec( Intervals.of("2017-10-04T00:00:00/2017-10-05T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, Collections.emptyMap(), null, null, indexSpec, null, - CompactionEngine.NATIVE) + new CompactionState( + partitionsSpec, + null, + Collections.emptyMap(), + null, + null, + indexSpec, + null, + CompactionEngine.NATIVE + ) ) ); @@ -1264,8 +1391,16 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentFil new CompactionState( partitionsSpec, null, - Collections.emptyMap(), null, - mapper.readValue(mapper.writeValueAsString(new TransformSpec(new SelectorDimFilter("dim1", "foo", null), null)), new TypeReference>() {}), + Collections.emptyMap(), + null, + mapper.readValue( + mapper.writeValueAsString( + new TransformSpec( + new SelectorDimFilter("dim1", "foo", null), + null + ) + ), + new TypeReference>() {}), indexSpec, null, CompactionEngine.NATIVE @@ -1278,8 +1413,16 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentFil new CompactionState( partitionsSpec, null, - Collections.emptyMap(), null, - mapper.readValue(mapper.writeValueAsString(new TransformSpec(new SelectorDimFilter("dim1", "bar", null), null)), new TypeReference>() {}), + Collections.emptyMap(), + null, + mapper.readValue( + mapper.writeValueAsString( + new TransformSpec( + new SelectorDimFilter("dim1", "bar", null), + null + ) + ), + new TypeReference>() {}), indexSpec, null, CompactionEngine.NATIVE @@ -1303,8 +1446,16 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentFil Intervals.of("2017-10-04T00:00:00/2017-10-05T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, Collections.emptyMap(), null, null, indexSpec, null, - CompactionEngine.NATIVE) + new CompactionState( + partitionsSpec, + null, + Collections.emptyMap(), + null, + null, + indexSpec, + null, + CompactionEngine.NATIVE + ) ) ); @@ -1422,7 +1573,8 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentMet new CompactionState( partitionsSpec, null, - Collections.emptyMap(), mapper.convertValue(new AggregatorFactory[] {}, new TypeReference>() {}), + Collections.emptyMap(), + mapper.convertValue(new AggregatorFactory[] {}, new TypeReference>() {}), null, indexSpec, null, @@ -1433,8 +1585,16 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentMet Intervals.of("2017-10-04T00:00:00/2017-10-05T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, Collections.emptyMap(), null, null, indexSpec, null, - CompactionEngine.NATIVE) + new CompactionState( + partitionsSpec, + null, + Collections.emptyMap(), + null, + null, + indexSpec, + null, + CompactionEngine.NATIVE + ) ) ); @@ -1540,8 +1700,16 @@ public void testIteratorReturnsSegmentsAsCompactionStateChangedWithCompactedStat Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), new Period("P1D"), null, - new CompactionState(partitionsSpec, null, Collections.emptyMap(), null, null, newIndexSpecMap, null, - CompactionEngine.NATIVE) + new CompactionState( + partitionsSpec, + null, + Collections.emptyMap(), + null, + null, + newIndexSpecMap, + null, + CompactionEngine.NATIVE + ) ) ); @@ -1591,7 +1759,8 @@ public void testIteratorDoesNotReturnSegmentWithChangingAppendableIndexSpec() new CompactionState( partitionsSpec, null, - Collections.emptyMap(), null, + Collections.emptyMap(), + null, null, IndexSpec.DEFAULT.asMap(mapper), null, diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java index 285f53ae3a0e..f2b102dac43d 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java @@ -408,24 +408,42 @@ public void testMakeStatsForDataSourceWithCompactedIntervalBetweenNonCompactedIn DataSegment afterNoon = createSegment(dataSourceName, j, false, k); if (j == 3) { // Make two intervals on this day compacted (two compacted intervals back-to-back) - beforeNoon = beforeNoon.withLastCompactionState(new CompactionState(partitionsSpec, null, - Collections.emptyMap(), - null, null, ImmutableMap.of(), ImmutableMap.of(), - CompactionEngine.NATIVE - )); - afterNoon = afterNoon.withLastCompactionState(new CompactionState(partitionsSpec, null, - Collections.emptyMap(), - null, null, ImmutableMap.of(), ImmutableMap.of(), - CompactionEngine.NATIVE - )); + beforeNoon = beforeNoon.withLastCompactionState( + new CompactionState( + partitionsSpec, + null, + Collections.emptyMap(), + null, + null, + ImmutableMap.of(), + ImmutableMap.of(), + CompactionEngine.NATIVE + )); + afterNoon = afterNoon.withLastCompactionState( + new CompactionState( + partitionsSpec, + null, + Collections.emptyMap(), + null, + null, + ImmutableMap.of(), + ImmutableMap.of(), + CompactionEngine.NATIVE + )); } if (j == 1) { // Make one interval on this day compacted - afterNoon = afterNoon.withLastCompactionState(new CompactionState(partitionsSpec, null, - Collections.emptyMap(), - null, null, ImmutableMap.of(), ImmutableMap.of(), - CompactionEngine.NATIVE - )); + afterNoon = afterNoon.withLastCompactionState( + new CompactionState( + partitionsSpec, + null, + Collections.emptyMap(), + null, + null, + ImmutableMap.of(), + ImmutableMap.of(), + CompactionEngine.NATIVE + )); } segments.add(beforeNoon); segments.add(afterNoon); diff --git a/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java b/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java index 2e489630100c..8c7030c86739 100644 --- a/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java +++ b/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java @@ -90,7 +90,8 @@ public void testSerde() throws JsonProcessingException new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "bar", "foo")) ), - Collections.emptyMap(), ImmutableList.of(ImmutableMap.of("type", "count", "name", "count")), + Collections.emptyMap(), + ImmutableList.of(ImmutableMap.of("type", "count", "name", "count")), ImmutableMap.of("filter", ImmutableMap.of("type", "selector", "dimension", "dim1", "value", "foo")), ImmutableMap.of(), ImmutableMap.of(), diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java index e404be64715c..211d64a1d032 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java @@ -284,7 +284,8 @@ public void setUp(@TempDir File tmpDir) throws Exception private final CompactionState expectedCompactionState = new CompactionState( new DynamicPartitionsSpec(null, null), null, - Collections.emptyMap(), null, + Collections.emptyMap(), + null, null, Collections.singletonMap("test", "map"), Collections.singletonMap("test2", "map2"), From b7261c2dc384cc8b312bebb385071e276cea5ce7 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Wed, 29 May 2024 20:28:50 +0530 Subject: [PATCH 33/54] Add tests for coverage and minor refactoring --- .../msq/indexing/MSQCompactionRunner.java | 59 ++-- .../msq/indexing/MSQCompactionRunnerTest.java | 252 +++++++++++++++--- .../common/task/CompactionRunner.java | 7 +- .../indexing/common/task/CompactionTask.java | 7 +- .../common/task/NativeCompactionRunner.java | 5 +- .../ClientCompactionRunnerInfo.java | 54 ++-- .../DataSourceCompactionConfig.java | 10 +- .../ClientCompactionRunnerInfoTest.java | 167 ++++++++++++ 8 files changed, 463 insertions(+), 98 deletions(-) create mode 100644 server/src/test/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfoTest.java diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java index 47bcbf2726d6..b632f306fabc 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java @@ -37,7 +37,6 @@ import org.apache.druid.indexing.common.task.CompactionTask; import org.apache.druid.indexing.common.task.CurrentSubTaskHolder; import org.apache.druid.java.util.common.NonnullPair; -import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.AllGranularity; import org.apache.druid.java.util.common.granularity.Granularities; @@ -106,37 +105,57 @@ public MSQCompactionRunner(@JacksonInject ObjectMapper jsonMapper, @JacksonInjec } @Override - public Pair supportsCompactionConfig( + public NonnullPair supportsCompactionSpec( CompactionTask compactionTask ) { - QueryContext compactionTaskContext = new QueryContext(compactionTask.getContext()); + if (compactionTask.getTuningConfig() != null) { + PartitionsSpec partitionsSpec = compactionTask.getTuningConfig().getPartitionsSpec(); + if (!(partitionsSpec instanceof DynamicPartitionsSpec + || partitionsSpec instanceof DimensionRangePartitionsSpec)) { + return new NonnullPair<>(false, StringUtils.format( + "Invalid partition spec type[%s] for MSQ compaction engine." + + " Type must be either DynamicPartitionsSpec or DynamicRangePartitionsSpec.", + partitionsSpec.getClass() + )); + } + if (partitionsSpec instanceof DynamicPartitionsSpec + && ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows() != null) { + return new NonnullPair<>(false, StringUtils.format( + "maxTotalRows[%d] in DynamicPartitionsSpec not supported for MSQ compaction engine.", + ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows() + )); + } + } + if (compactionTask.getMetricsSpec() != null + && compactionTask.getGranularitySpec() != null + && !compactionTask.getGranularitySpec() + .isRollup()) { + return new NonnullPair<>( + false, + "rollup in granularitySpec must be set to True if metricsSpec is specifed " + + "for MSQ compaction engine." + ); + } - // These checks cannot be included in compaction config validation as context param keys are unknown outside MSQ. + QueryContext compactionTaskContext = new QueryContext(compactionTask.getContext()); if (!MultiStageQueryContext.isFinalizeAggregations(compactionTaskContext)) { - return Pair.of(false, StringUtils.format( + return new NonnullPair<>(false, StringUtils.format( "Config[%s] cannot be set to false for auto-compaction with MSQ engine.", MultiStageQueryContext.CTX_FINALIZE_AGGREGATIONS )); } if (MultiStageQueryContext.getAssignmentStrategy(compactionTaskContext) == WorkerAssignmentStrategy.AUTO) { - return Pair.of( + return new NonnullPair<>( false, - StringUtils.format("Config[%s] cannot be set to value[%s] for auto-compaction with MSQ engine.", - MultiStageQueryContext.CTX_TASK_ASSIGNMENT_STRATEGY, - WorkerAssignmentStrategy.AUTO + StringUtils.format( + "Config[%s] cannot be set to value[%s] for auto-compaction with MSQ engine.", + MultiStageQueryContext.CTX_TASK_ASSIGNMENT_STRATEGY, + WorkerAssignmentStrategy.AUTO ) ); } - if (compactionTask.getTuningConfig() != null) { - PartitionsSpec partitionsSpec = compactionTask.getTuningConfig().getPartitionsSpec(); - if (!(partitionsSpec instanceof DynamicPartitionsSpec - || partitionsSpec instanceof DimensionRangePartitionsSpec)) { - return Pair.of(false, "PartitionsSpec not among DynamicPartitionSpec or DimensionRangePartitionsSpec " - + "required with MSQ engine"); - } - } - return Pair.of(true, null); + return new NonnullPair<>(true, ""); } @Override @@ -155,10 +174,11 @@ public TaskStatus runCompactionTasks( List msqControllerTasks = compactionToMSQTasks(compactionTask, intervalDataSchemas); if (msqControllerTasks.isEmpty()) { - log.warn( + String msg = StringUtils.format( "Can't find segments from inputSpec[%s], nothing to do.", compactionTask.getIoConfig().getInputSpec() ); + return TaskStatus.failure(compactionTask.getId(), msg); } return runSubtasks( msqControllerTasks, @@ -175,7 +195,6 @@ public List compactionToMSQTasks( { List msqControllerTasks = new ArrayList<>(); - for (NonnullPair intervalDataSchema : intervalDataSchemas) { Query query; Interval interval = intervalDataSchema.lhs; diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java index fbd7b4b98dff..ca354d23bce3 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java @@ -23,6 +23,8 @@ import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; import org.apache.druid.client.indexing.ClientCompactionTaskTransformSpec; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.impl.DimensionSchema; @@ -30,7 +32,11 @@ import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; +import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; +import org.apache.druid.indexer.partitions.HashedPartitionsSpec; +import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexing.common.task.CompactionIntervalSpec; import org.apache.druid.indexing.common.task.CompactionTask; import org.apache.druid.jackson.DefaultObjectMapper; @@ -60,6 +66,7 @@ import org.junit.BeforeClass; import org.junit.Test; +import javax.annotation.Nullable; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -88,6 +95,7 @@ public class MSQCompactionRunnerTest private static final AggregatorFactory AGG1 = new CountAggregatorFactory("agg_0"); private static final AggregatorFactory AGG2 = new LongSumAggregatorFactory("agg_1", "long_dim_1"); private static List AGGREGATORS = ImmutableList.of(AGG1, AGG2); + private static final MSQCompactionRunner MSQ_COMPACTION_RUNNER = new MSQCompactionRunner(JSON_MAPPER, null); @BeforeClass public static void setupClass() @@ -113,38 +121,157 @@ public static void setupClass() } @Test - public void testCompactionToMSQTasks() throws JsonProcessingException + public void testHashPartitionsSpec() throws JsonProcessingException { - DimFilter dimFilter = new SelectorDimFilter("dim1", "foo", null); - ClientCompactionTaskTransformSpec transformSpec = - new ClientCompactionTaskTransformSpec(dimFilter); - final CompactionTask.Builder builder = new CompactionTask.Builder( - DATA_SOURCE, + CompactionTask compactionTask = createCompactionTask( + new HashedPartitionsSpec(3, null, ImmutableList.of("dummy")), null, + Collections.emptyMap() + ); + Assert.assertFalse(MSQ_COMPACTION_RUNNER.supportsCompactionSpec(compactionTask).lhs); + } + + @Test + public void testDimensionRangePartitionsSpec() throws JsonProcessingException + { + CompactionTask compactionTask = createCompactionTask( + new DimensionRangePartitionsSpec(TARGET_ROWS_PER_SEGMENT, null, PARTITION_DIMENSIONS, false), null, - new MSQCompactionRunner(JSON_MAPPER, null) + Collections.emptyMap() ); - IndexSpec indexSpec = IndexSpec.builder() - .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) - .withDimensionCompression(CompressionStrategy.LZ4) - .withMetricCompression(CompressionStrategy.LZF) - .withLongEncoding(CompressionFactory.LongEncodingStrategy.LONGS) - .build(); + Assert.assertTrue(MSQ_COMPACTION_RUNNER.supportsCompactionSpec(compactionTask).lhs); + } - Map context = new HashMap<>(); - context.put(MultiStageQueryContext.CTX_SORT_ORDER, JSON_MAPPER.writeValueAsString(SORT_ORDER_DIMENSIONS)); - context.put(MultiStageQueryContext.CTX_MAX_NUM_TASKS, 2); + @Test + public void testInvalidDynamicPartitionsSpec() throws JsonProcessingException + { + CompactionTask compactionTask = createCompactionTask( + new DynamicPartitionsSpec(3, 3L), + null, + Collections.emptyMap() + ); + Assert.assertFalse(MSQ_COMPACTION_RUNNER.supportsCompactionSpec(compactionTask).lhs); + } - builder - .inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, null)) - .tuningConfig(createTuningConfig(indexSpec)) - .transformSpec(transformSpec) - .context(context); + @Test + public void testDynamicPartitionsSpec() throws JsonProcessingException + { + CompactionTask compactionTask = createCompactionTask( + new DynamicPartitionsSpec(3, null), + null, + Collections.emptyMap() + ); + Assert.assertTrue(MSQ_COMPACTION_RUNNER.supportsCompactionSpec(compactionTask).lhs); + } + + @Test + public void testWithWorkerStrategyAuto() throws JsonProcessingException + { + CompactionTask compactionTask = createCompactionTask( + new DynamicPartitionsSpec(3, null), + null, + ImmutableMap.of(MultiStageQueryContext.CTX_TASK_ASSIGNMENT_STRATEGY, WorkerAssignmentStrategy.AUTO.toString()) + ); + Assert.assertFalse(MSQ_COMPACTION_RUNNER.supportsCompactionSpec(compactionTask).lhs); + } + + @Test + public void testWithFinalizeAggregationsFalse() throws JsonProcessingException + { + CompactionTask compactionTask = createCompactionTask( + new DynamicPartitionsSpec(3, null), + null, + ImmutableMap.of(MultiStageQueryContext.CTX_FINALIZE_AGGREGATIONS, false) + ); + Assert.assertFalse(MSQ_COMPACTION_RUNNER.supportsCompactionSpec(compactionTask).lhs); + } + + @Test + public void testRunCompactionTasksWithEmptyTaskList() throws Exception + { + CompactionTask compactionTask = createCompactionTask(null, null, Collections.emptyMap()); + TaskStatus taskStatus = MSQ_COMPACTION_RUNNER.runCompactionTasks(compactionTask, Collections.emptyList(), null); + Assert.assertTrue(taskStatus.isFailure()); + } + + @Test + public void testMSQControllerTaskSpecWithScan() throws JsonProcessingException + { + DimFilter dimFilter = new SelectorDimFilter("dim1", "foo", null); + + CompactionTask taskCreatedWithTransformSpec = createCompactionTask( + new DimensionRangePartitionsSpec(TARGET_ROWS_PER_SEGMENT, null, PARTITION_DIMENSIONS, false), + dimFilter, + Collections.emptyMap() + ); + + DataSchema dataSchema = new DataSchema( + DATA_SOURCE, + new TimestampSpec(TIMESTAMP_COLUMN, null, null), + new DimensionsSpec(DIMENSIONS), + new AggregatorFactory[]{}, + new UniformGranularitySpec( + SEGMENT_GRANULARITY.getDefaultGranularity(), + QUERY_GRANULARITY.getDefaultGranularity(), + Collections.singletonList(COMPACTION_INTERVAL) + ), + new TransformSpec(dimFilter, Collections.emptyList()) + ); + + + List msqControllerTasks = MSQ_COMPACTION_RUNNER + .compactionToMSQTasks( + taskCreatedWithTransformSpec, + Collections.singletonList(new NonnullPair<>( + COMPACTION_INTERVAL, + dataSchema + )) + ); + + MSQControllerTask msqControllerTask = Iterables.getOnlyElement(msqControllerTasks); - final CompactionTask taskCreatedWithTransformSpec = builder.build(); + MSQSpec actualMSQSpec = msqControllerTask.getQuerySpec(); + + Assert.assertEquals( + new MSQTuningConfig( + 1, + MultiStageQueryContext.DEFAULT_ROWS_IN_MEMORY, + TARGET_ROWS_PER_SEGMENT, + createIndexSpec() + ), + actualMSQSpec.getTuningConfig() + ); Assert.assertEquals( - transformSpec, - taskCreatedWithTransformSpec.getTransformSpec() + new DataSourceMSQDestination( + DATA_SOURCE, + SEGMENT_GRANULARITY.getDefaultGranularity(), + SORT_ORDER_DIMENSIONS, + Collections.singletonList(COMPACTION_INTERVAL) + ), + actualMSQSpec.getDestination() + ); + + Assert.assertEquals(dimFilter, actualMSQSpec.getQuery().getFilter()); + Assert.assertEquals( + JSON_MAPPER.writeValueAsString(SEGMENT_GRANULARITY.toString()), + msqControllerTask.getContext().get(DruidSqlInsert.SQL_INSERT_SEGMENT_GRANULARITY) + ); + Assert.assertEquals( + JSON_MAPPER.writeValueAsString(QUERY_GRANULARITY.toString()), + msqControllerTask.getContext().get(DruidSqlInsert.SQL_INSERT_QUERY_GRANULARITY) + ); + Assert.assertEquals(WorkerAssignmentStrategy.MAX, actualMSQSpec.getAssignmentStrategy()); + } + + @Test + public void testMSQControllerTaskSpecWithAggregators() throws JsonProcessingException + { + DimFilter dimFilter = new SelectorDimFilter("dim1", "foo", null); + + CompactionTask taskCreatedWithTransformSpec = createCompactionTask( + new DimensionRangePartitionsSpec(TARGET_ROWS_PER_SEGMENT, null, PARTITION_DIMENSIONS, false), + dimFilter, + Collections.emptyMap() ); DataSchema dataSchema = new DataSchema( @@ -160,7 +287,8 @@ public void testCompactionToMSQTasks() throws JsonProcessingException new TransformSpec(dimFilter, Collections.emptyList()) ); - List msqControllerTasks = new MSQCompactionRunner(JSON_MAPPER, null) + + List msqControllerTasks = MSQ_COMPACTION_RUNNER .compactionToMSQTasks( taskCreatedWithTransformSpec, Collections.singletonList(new NonnullPair<>( @@ -169,25 +297,28 @@ public void testCompactionToMSQTasks() throws JsonProcessingException )) ); - Assert.assertEquals(1, msqControllerTasks.size()); + MSQControllerTask msqControllerTask = Iterables.getOnlyElement(msqControllerTasks); - MSQControllerTask msqControllerTask = msqControllerTasks.get(0); MSQSpec actualMSQSpec = msqControllerTask.getQuerySpec(); + Assert.assertEquals( new MSQTuningConfig( 1, MultiStageQueryContext.DEFAULT_ROWS_IN_MEMORY, TARGET_ROWS_PER_SEGMENT, - indexSpec + createIndexSpec() ), actualMSQSpec.getTuningConfig() ); - Assert.assertEquals(new DataSourceMSQDestination( - DATA_SOURCE, - SEGMENT_GRANULARITY.getDefaultGranularity(), - SORT_ORDER_DIMENSIONS, - Collections.singletonList(COMPACTION_INTERVAL) - ), actualMSQSpec.getDestination()); + Assert.assertEquals( + new DataSourceMSQDestination( + DATA_SOURCE, + SEGMENT_GRANULARITY.getDefaultGranularity(), + SORT_ORDER_DIMENSIONS, + Collections.singletonList(COMPACTION_INTERVAL) + ), + actualMSQSpec.getDestination() + ); Assert.assertEquals(dimFilter, actualMSQSpec.getQuery().getFilter()); Assert.assertEquals( @@ -201,7 +332,44 @@ public void testCompactionToMSQTasks() throws JsonProcessingException Assert.assertEquals(WorkerAssignmentStrategy.MAX, actualMSQSpec.getAssignmentStrategy()); } - private static CompactionTask.CompactionTuningConfig createTuningConfig(IndexSpec indexSpec) + private CompactionTask createCompactionTask( + @Nullable PartitionsSpec partitionsSpec, + DimFilter dimFilter, + Map contextParams + ) + throws JsonProcessingException + { + ClientCompactionTaskTransformSpec transformSpec = + new ClientCompactionTaskTransformSpec(dimFilter); + final CompactionTask.Builder builder = new CompactionTask.Builder( + DATA_SOURCE, + null, + null, + new MSQCompactionRunner(JSON_MAPPER, null) + ); + IndexSpec indexSpec = createIndexSpec(); + + Map context = new HashMap<>(); + context.put(MultiStageQueryContext.CTX_SORT_ORDER, JSON_MAPPER.writeValueAsString(SORT_ORDER_DIMENSIONS)); + context.put(MultiStageQueryContext.CTX_MAX_NUM_TASKS, 2); + context.putAll(contextParams); + + builder + .inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, null)) + .tuningConfig(createTuningConfig( + indexSpec, + partitionsSpec == null ? new DynamicPartitionsSpec(100, null) : partitionsSpec + )) + .transformSpec(transformSpec) + .context(context); + + return builder.build(); + } + + private static CompactionTask.CompactionTuningConfig createTuningConfig( + IndexSpec indexSpec, + PartitionsSpec partitionsSpec + ) { return new CompactionTask.CompactionTuningConfig( null, @@ -213,11 +381,11 @@ private static CompactionTask.CompactionTuningConfig createTuningConfig(IndexSpe null, null, null, - new DimensionRangePartitionsSpec(TARGET_ROWS_PER_SEGMENT, null, PARTITION_DIMENSIONS, false), + partitionsSpec, indexSpec, null, null, - true, + !(partitionsSpec instanceof DynamicPartitionsSpec), false, 5000L, null, @@ -237,4 +405,14 @@ private static CompactionTask.CompactionTuningConfig createTuningConfig(IndexSpe null ); } + + private static IndexSpec createIndexSpec() + { + return IndexSpec.builder() + .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) + .withDimensionCompression(CompressionStrategy.LZ4) + .withMetricCompression(CompressionStrategy.LZF) + .withLongEncoding(CompressionFactory.LongEncodingStrategy.LONGS) + .build(); + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java index 02e0e142379e..c2db9b92653e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java @@ -24,7 +24,6 @@ import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.java.util.common.NonnullPair; -import org.apache.druid.java.util.common.Pair; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.server.coordinator.ClientCompactionRunnerInfo; import org.joda.time.Interval; @@ -53,10 +52,10 @@ TaskStatus runCompactionTasks( /** * Checks if the provided compaction config is supported by the runner. - * @param compactionTask - * @return Pair of (supported) boolean and a reason string. Reason string is null if supported is true. * + * @param compactionTask + * @return Pair of (supported) boolean and a reason string. Reason string is empty if supported is true. */ - Pair supportsCompactionConfig(CompactionTask compactionTask); + NonnullPair supportsCompactionSpec(CompactionTask compactionTask); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index f378753d899e..4176c7e73f6a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -478,12 +478,9 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception ); registerResourceCloserOnAbnormalExit(compactionRunner.getCurrentSubTaskHolder()); - Pair supportsCompactionConfig = compactionRunner.supportsCompactionConfig(this); + NonnullPair supportsCompactionConfig = compactionRunner.supportsCompactionSpec(this); if (!supportsCompactionConfig.lhs) { - throw InvalidInput.exception( - "Compaction config not supported. Reason[%s].", - supportsCompactionConfig.rhs - ); + throw InvalidInput.exception("Compaction spec not supported. Reason[%s].", supportsCompactionConfig.rhs); } return compactionRunner.runCompactionTasks(this, intervalDataSchemas, toolbox); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java index 73fa29aef1fc..c3e470c35a6e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java @@ -37,7 +37,6 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.NonnullPair; -import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.logger.Logger; @@ -81,11 +80,11 @@ public CurrentSubTaskHolder getCurrentSubTaskHolder() } @Override - public Pair supportsCompactionConfig( + public NonnullPair supportsCompactionSpec( CompactionTask compactionTask ) { - return Pair.of(true, null); + return new NonnullPair<>(true, null); } /** diff --git a/server/src/main/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfo.java b/server/src/main/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfo.java index 2e3dd4fa84ad..119487e8c9c2 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfo.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfo.java @@ -21,11 +21,12 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.error.InvalidInput; import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.java.util.common.NonnullPair; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.QueryContext; import java.util.Objects; @@ -51,14 +52,6 @@ public CompactionEngine getType() return type; } - @Override - public String toString() - { - return "ClientCompactionRunnerInfo{" + - "type=" + type + - '}'; - } - @Override public boolean equals(Object o) { @@ -78,56 +71,61 @@ public int hashCode() return Objects.hash(type); } - public static void supportsCompactionConfig(DataSourceCompactionConfig newConfig, String engineSource) + public static NonnullPair supportsCompactionConfig(DataSourceCompactionConfig newConfig, String engineSource) { CompactionEngine compactionEngine = newConfig.getEngine(); if (compactionEngine == CompactionEngine.MSQ) { if (newConfig.getTuningConfig() != null) { PartitionsSpec partitionsSpec = newConfig.getTuningConfig().getPartitionsSpec(); - - if (partitionsSpec != null && !(partitionsSpec instanceof DimensionRangePartitionsSpec - || partitionsSpec instanceof DynamicPartitionsSpec)) { - throw InvalidInput.exception( + if (!(partitionsSpec instanceof DimensionRangePartitionsSpec + || partitionsSpec instanceof DynamicPartitionsSpec)) { + return new NonnullPair<>(false, StringUtils.format( "Invalid partition spec type[%s] for MSQ compaction engine[%s]." + " Type must be either DynamicPartitionsSpec or DynamicRangePartitionsSpec.", partitionsSpec.getClass(), engineSource + ) ); } if (partitionsSpec instanceof DynamicPartitionsSpec && ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows() != null) { - throw InvalidInput.exception( + return new NonnullPair<>(false, StringUtils.format( "maxTotalRows[%d] in DynamicPartitionsSpec not supported for MSQ compaction engine[%s].", ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows(), engineSource - ); + )); } } + if (newConfig.getMetricsSpec() != null && newConfig.getGranularitySpec() != null && !newConfig.getGranularitySpec() .isRollup()) { - throw InvalidInput.exception( + return new NonnullPair<>(false, StringUtils.format( "rollup in granularitySpec must be set to True if metricsSpec is specifed " - + "for MSQ compaction engine[%s]."); + + "for MSQ compaction engine[%s].", engineSource)); } QueryContext queryContext = QueryContext.of(newConfig.getTaskContext()); - if (!queryContext.getBoolean(MSQContext.CTX_FINALIZE_AGGREGATIONS, true)) { - throw InvalidInput.exception( - "Config[%s] cannot be set to false for auto-compaction with MSQ engine.", - MSQContext.CTX_FINALIZE_AGGREGATIONS - ); + if (!queryContext.getBoolean(MSQContext.CTX_FINALIZE_AGGREGATIONS, true)) { + return new NonnullPair<>(false, StringUtils.format( + "Config[%s] cannot be set to false for auto-compaction with MSQ engine[%s].", + MSQContext.CTX_FINALIZE_AGGREGATIONS, + engineSource + )); } + if (queryContext.getString(MSQContext.CTX_TASK_ASSIGNMENT_STRATEGY, MSQContext.TASK_ASSIGNMENT_STRATEGY_MAX) .equals(MSQContext.TASK_ASSIGNMENT_STRATEGY_AUTO)) { - throw InvalidInput.exception( - "Config[%s] cannot be set to value[%s] for auto-compaction with MSQ engine.", + return new NonnullPair<>(false, StringUtils.format( + "Config[%s] cannot be set to value[%s] for auto-compaction with MSQ engine[%s].", MSQContext.CTX_TASK_ASSIGNMENT_STRATEGY, - MSQContext.TASK_ASSIGNMENT_STRATEGY_AUTO - ); + MSQContext.TASK_ASSIGNMENT_STRATEGY_AUTO, + engineSource + )); } } + return new NonnullPair<>(true, ""); } /** @@ -135,7 +133,7 @@ public static void supportsCompactionConfig(DataSourceCompactionConfig newConfig * compaction config at the coordinator. The values used here should be kept in sync with those in * {@link org.apache.druid.msq.util.MultiStageQueryContext} */ - private static class MSQContext + public static class MSQContext { public static final String CTX_FINALIZE_AGGREGATIONS = "finalizeAggregations"; public static final String CTX_TASK_ASSIGNMENT_STRATEGY = "taskAssignment"; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java index a1c0050d991c..91c22d0f5ede 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java @@ -22,7 +22,9 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import org.apache.druid.error.InvalidInput; import org.apache.druid.indexer.CompactionEngine; +import org.apache.druid.java.util.common.NonnullPair; import org.apache.druid.query.aggregation.AggregatorFactory; import org.joda.time.Period; @@ -255,7 +257,13 @@ public static DataSourceCompactionConfig from( ); } - ClientCompactionRunnerInfo.supportsCompactionConfig(newConfig, engineSource); + NonnullPair supportsCompactionConfig = ClientCompactionRunnerInfo.supportsCompactionConfig( + newConfig, + engineSource + ); + if (!supportsCompactionConfig.lhs) { + throw InvalidInput.exception("Compaction config not supported. Reason[%s].", supportsCompactionConfig.rhs); + } return newConfig; } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfoTest.java b/server/src/test/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfoTest.java new file mode 100644 index 000000000000..0a4286d749de --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfoTest.java @@ -0,0 +1,167 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.coordinator; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.data.input.SegmentsSplitHintSpec; +import org.apache.druid.indexer.CompactionEngine; +import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; +import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; +import org.apache.druid.indexer.partitions.HashedPartitionsSpec; +import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.java.util.common.HumanReadableBytes; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.data.CompressionFactory; +import org.apache.druid.segment.data.CompressionStrategy; +import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; +import org.joda.time.Duration; +import org.joda.time.Period; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Collections; +import java.util.Map; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class ClientCompactionRunnerInfoTest +{ + @Test + public void testHashedPartitionsSpecs() + { + assertFalse( + ClientCompactionRunnerInfo.supportsCompactionConfig( + createCompactionConfig(new HashedPartitionsSpec(100, null, null), Collections.emptyMap()), + "dummy" + ).lhs + ); + } + + @Test + public void testrInvalidDynamicPartitionsSpecs() + { + assertFalse( + ClientCompactionRunnerInfo.supportsCompactionConfig( + createCompactionConfig(new DynamicPartitionsSpec(100, 100L), Collections.emptyMap()), + "dummy" + ).lhs + ); + } + + @Test + public void testDynamicPartitionsSpecs() + { + assertTrue(ClientCompactionRunnerInfo.supportsCompactionConfig( + createCompactionConfig(new DynamicPartitionsSpec(100, null), Collections.emptyMap()), + "dummy" + ).lhs); + } + + @Test + public void testDimensionRangePartitionsSpecs() + { + assertTrue(ClientCompactionRunnerInfo.supportsCompactionConfig( + createCompactionConfig( + new DimensionRangePartitionsSpec(100, null, ImmutableList.of("partitionDim"), false), + Collections.emptyMap() + ), + "engine" + ).lhs); + } + + @Test + public void testWithWorkerStrategyAuto() + { + DataSourceCompactionConfig compactionConfig = createCompactionConfig( + new DynamicPartitionsSpec(3, null), + ImmutableMap.of(ClientCompactionRunnerInfo.MSQContext.CTX_TASK_ASSIGNMENT_STRATEGY, "AUTO") + ); + Assert.assertFalse(ClientCompactionRunnerInfo.supportsCompactionConfig(compactionConfig, "").lhs); + } + + @Test + public void testWithFinalizeAggregationsFalse() + { + DataSourceCompactionConfig compactionConfig = createCompactionConfig( + new DynamicPartitionsSpec(3, null), + ImmutableMap.of(ClientCompactionRunnerInfo.MSQContext.CTX_FINALIZE_AGGREGATIONS, false) + ); + Assert.assertFalse(ClientCompactionRunnerInfo.supportsCompactionConfig(compactionConfig, "").lhs); + } + + private static DataSourceCompactionConfig createCompactionConfig( + PartitionsSpec partitionsSpec, + Map context + ) + { + final DataSourceCompactionConfig config = new DataSourceCompactionConfig( + "dataSource", + null, + 500L, + 10000, + new Period(3600), + createTuningConfig(partitionsSpec), + null, + null, + null, + null, + null, + CompactionEngine.MSQ, + context + ); + return config; + } + + private static UserCompactionTaskQueryTuningConfig createTuningConfig(PartitionsSpec partitionsSpec) + { + final UserCompactionTaskQueryTuningConfig tuningConfig = new UserCompactionTaskQueryTuningConfig( + 40000, + null, + 2000L, + null, + new SegmentsSplitHintSpec(new HumanReadableBytes(100000L), null), + partitionsSpec, + IndexSpec.builder() + .withDimensionCompression(CompressionStrategy.LZ4) + .withMetricCompression(CompressionStrategy.LZF) + .withLongEncoding(CompressionFactory.LongEncodingStrategy.LONGS) + .build(), + IndexSpec.builder() + .withDimensionCompression(CompressionStrategy.LZ4) + .withMetricCompression(CompressionStrategy.UNCOMPRESSED) + .withLongEncoding(CompressionFactory.LongEncodingStrategy.AUTO) + .build(), + 2, + 1000L, + TmpFileSegmentWriteOutMediumFactory.instance(), + 100, + 5, + 1000L, + new Duration(3000L), + 7, + 1000, + 100, + 2 + ); + return tuningConfig; + } +} From 987504be775aef6910d0dcf8ce4c8db5b10cdfc1 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Thu, 30 May 2024 10:50:00 +0530 Subject: [PATCH 34/54] Fix test failures --- .../druid/indexing/common/task/NativeCompactionRunner.java | 2 +- .../server/coordinator/ClientCompactionRunnerInfo.java | 6 ++++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java index c3e470c35a6e..28031a69154a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java @@ -84,7 +84,7 @@ public NonnullPair supportsCompactionSpec( CompactionTask compactionTask ) { - return new NonnullPair<>(true, null); + return new NonnullPair<>(true, ""); } /** diff --git a/server/src/main/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfo.java b/server/src/main/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfo.java index 119487e8c9c2..6fc91af443ac 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfo.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfo.java @@ -71,6 +71,12 @@ public int hashCode() return Objects.hash(type); } + /** + * Checks if the provided compaction config is supported by the runner + * @param newConfig The updated compaction config + * @param engineSource String indicating the source of compaction engine. + * @return Pair of support boolean and reason string. The reason string is empty if support boolean is True. + */ public static NonnullPair supportsCompactionConfig(DataSourceCompactionConfig newConfig, String engineSource) { CompactionEngine compactionEngine = newConfig.getEngine(); From dfe40afc3178578a0576c9d989d744264d130df2 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Fri, 31 May 2024 11:58:04 +0530 Subject: [PATCH 35/54] Address review comments --- .../org/apache/druid/msq/exec/ControllerImpl.java | 2 +- .../apache/druid/msq/guice/MSQIndexingModule.java | 2 +- .../druid/msq/indexing/MSQCompactionRunner.java | 2 +- .../indexing/common/task/CompactionRunner.java | 2 +- .../org/apache/druid/indexer/CompactionEngine.java | 13 ++++++++++++- .../org/apache/druid/timeline/CompactionState.java | 3 ++- .../coordinator/ClientCompactionRunnerInfo.java | 8 ++++++++ 7 files changed, 26 insertions(+), 6 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 151c0e72d8e1..19cd4fa2a3c9 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -1516,7 +1516,7 @@ private void handleQueryResults( if (!destination.isReplaceTimeChunks()) { // Store compaction state only for replace queries. log.warn( - "storeCompactionState flag set for a non-REPLACE query[%s]. Ignoring the flag for now.", + "Ignoring storeCompactionState flag since it is set for a non-REPLACE query[%s].", queryDef.getQueryId() ); } else { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java index c3f684713c39..421e949f6c52 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java @@ -192,7 +192,7 @@ public List getJacksonModules() NilInputSource.class ); - module.registerSubtypes(new NamedType(MSQCompactionRunner.class, "MSQ")); + module.registerSubtypes(new NamedType(MSQCompactionRunner.class, "msq")); FAULT_CLASSES.forEach(module::registerSubtypes); module.addSerializer(new CounterSnapshotsSerializer()); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java index b632f306fabc..7c813e1f3d40 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java @@ -133,7 +133,7 @@ public NonnullPair supportsCompactionSpec( .isRollup()) { return new NonnullPair<>( false, - "rollup in granularitySpec must be set to True if metricsSpec is specifed " + "rollup in granularitySpec must be set to true if metricsSpec is specifed " + "for MSQ compaction engine." ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java index c2db9b92653e..e7d82e154096 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java @@ -36,7 +36,7 @@ */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = CompactionRunner.TYPE_PROPERTY) @JsonSubTypes(value = { - @JsonSubTypes.Type(name = "NATIVE", value = NativeCompactionRunner.class) + @JsonSubTypes.Type(name = "native", value = NativeCompactionRunner.class) }) public interface CompactionRunner { diff --git a/processing/src/main/java/org/apache/druid/indexer/CompactionEngine.java b/processing/src/main/java/org/apache/druid/indexer/CompactionEngine.java index 4713542c7b28..1758a9183b69 100644 --- a/processing/src/main/java/org/apache/druid/indexer/CompactionEngine.java +++ b/processing/src/main/java/org/apache/druid/indexer/CompactionEngine.java @@ -20,19 +20,30 @@ package org.apache.druid.indexer; import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; import org.apache.druid.java.util.common.StringUtils; +import javax.annotation.Nullable; + /** * Encapsulates the Engine to be used for a compaction task. * Should be kept in sync with the subtypes for {@link org.apache.druid.indexing.common.task.CompactionRunner}. */ public enum CompactionEngine { + // Using uppercase constants since "native" is a reserved keyword. NATIVE, MSQ; + @Override + @JsonValue + public String toString() + { + return this.name().toLowerCase(); + } + @JsonCreator - public static CompactionEngine fromString(String name) + public static CompactionEngine fromString(@Nullable String name) { return name == null ? null : valueOf(StringUtils.toUpperCase(name)); } diff --git a/processing/src/main/java/org/apache/druid/timeline/CompactionState.java b/processing/src/main/java/org/apache/druid/timeline/CompactionState.java index bb8eb3b1ca0b..545e63282c6c 100644 --- a/processing/src/main/java/org/apache/druid/timeline/CompactionState.java +++ b/processing/src/main/java/org/apache/druid/timeline/CompactionState.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.common.config.Configs; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.partitions.PartitionsSpec; @@ -81,7 +82,7 @@ public CompactionState( this.transformSpec = transformSpec; this.indexSpec = indexSpec; this.granularitySpec = granularitySpec; - this.engine = engine; + this.engine = Configs.valueOrDefault(engine, CompactionEngine.NATIVE); this.dimensionToAggregatoryFactoryMap = dimensionToAggregatoryFactoryMap; } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfo.java b/server/src/main/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfo.java index 6fc91af443ac..53d316d522e7 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfo.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfo.java @@ -52,6 +52,14 @@ public CompactionEngine getType() return type; } + @Override + public String toString() + { + return "ClientCompactionRunnerInfo{" + + "type=" + type + + '}'; + } + @Override public boolean equals(Object o) { From 98aafb0f5c6b17a6fdd390b75c89df3cb24816ef Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Tue, 18 Jun 2024 17:05:40 +0530 Subject: [PATCH 36/54] Address review comments --- docs/multi-stage-query/known-issues.md | 5 - .../druid/msq/guice/MSQIndexingModule.java | 2 +- .../msq/indexing/MSQCompactionRunner.java | 113 ++++++------ .../msq/indexing/MSQCompactionRunnerTest.java | 41 ++--- .../common/task/CompactionRunner.java | 17 +- .../indexing/common/task/CompactionTask.java | 21 +-- .../common/task/NativeCompactionRunner.java | 20 ++- .../common/task/CompactionTaskTest.java | 32 ++-- .../druid/indexer/CompactionEngine.java | 3 +- .../druid/timeline/CompactionState.java | 8 +- .../ClientCompactionRunnerInfo.java | 168 ++++++++++++------ .../DataSourceCompactionConfig.java | 12 +- .../coordinator/compact/CompactionStatus.java | 2 + .../ClientCompactionRunnerInfoTest.java | 32 +--- 14 files changed, 252 insertions(+), 224 deletions(-) diff --git a/docs/multi-stage-query/known-issues.md b/docs/multi-stage-query/known-issues.md index 540dc2f08eb9..39286edfcdd7 100644 --- a/docs/multi-stage-query/known-issues.md +++ b/docs/multi-stage-query/known-issues.md @@ -68,8 +68,3 @@ properties, and the `indexSpec` [`tuningConfig`](../ingestion/ingestion-spec.md# - The maximum number of elements in a window cannot exceed a value of 100,000. - To avoid `leafOperators` in MSQ engine, window functions have an extra scan stage after the window stage for cases where native engine has a non-empty `leafOperator`. - -## Compaction - -- Compaction configured with MSQ engine currently doesn't support context flags `finalizeAggregations=false` and - `taskAssignment=auto`. diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java index 421e949f6c52..de722215a822 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java @@ -192,7 +192,7 @@ public List getJacksonModules() NilInputSource.class ); - module.registerSubtypes(new NamedType(MSQCompactionRunner.class, "msq")); + module.registerSubtypes(new NamedType(MSQCompactionRunner.class, MSQCompactionRunner.TYPE)); FAULT_CLASSES.forEach(module::registerSubtypes); module.addSerializer(new CounterSnapshotsSerializer()); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java index 7c813e1f3d40..2ab00714795a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java @@ -20,6 +20,7 @@ package org.apache.druid.msq.indexing; import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; @@ -36,7 +37,6 @@ import org.apache.druid.indexing.common.task.CompactionRunner; import org.apache.druid.indexing.common.task.CompactionTask; import org.apache.druid.indexing.common.task.CurrentSubTaskHolder; -import org.apache.druid.java.util.common.NonnullPair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.AllGranularity; import org.apache.druid.java.util.common.granularity.Granularities; @@ -45,7 +45,6 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination; -import org.apache.druid.msq.kernel.WorkerAssignmentStrategy; import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.Druids; import org.apache.druid.query.Query; @@ -65,6 +64,7 @@ import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; +import org.apache.druid.server.coordinator.ClientCompactionRunnerInfo; import org.apache.druid.sql.calcite.parser.DruidSqlInsert; import org.apache.druid.sql.calcite.planner.ColumnMapping; import org.apache.druid.sql.calcite.planner.ColumnMappings; @@ -82,13 +82,13 @@ public class MSQCompactionRunner implements CompactionRunner { private static final Logger log = new Logger(MSQCompactionRunner.class); + public static final String TYPE = "msq"; private static final Granularity DEFAULT_SEGMENT_GRANULARITY = Granularities.ALL; private final ObjectMapper jsonMapper; private final Injector injector; public static final String TIME_VIRTUAL_COLUMN = "__vTime"; - public static final String TIME_COLUMN = ColumnHolder.TIME_COLUMN_NAME; @JsonIgnore private final CurrentSubTaskHolder currentSubTaskHolder = new CurrentSubTaskHolder( @@ -98,64 +98,57 @@ public class MSQCompactionRunner implements CompactionRunner }); + @JsonCreator public MSQCompactionRunner(@JacksonInject ObjectMapper jsonMapper, @JacksonInject Injector injector) { this.jsonMapper = jsonMapper; this.injector = injector; } + /** + Checks if the provided compaction config is supported by MSQ. + The same validation is done at + {@link ClientCompactionRunnerInfo#msqEngineSupportsCompactionConfig} + The following configs aren't supported: + *
    + *
  • finalizeAggregations set to false in context.
  • + * + *
  • taskAssignment set to auto in context.
  • + * + *
  • partitionsSpec of type HashedParititionsSpec.
  • + * + *
  • maxTotalRows in DynamicPartitionsSpec.
  • + * + *
  • rollup set to false in granularitySpec when metricsSpec is specified.
  • + *
+ */ @Override - public NonnullPair supportsCompactionSpec( + public ClientCompactionRunnerInfo.ValidationResult validateCompactionTask( CompactionTask compactionTask ) { if (compactionTask.getTuningConfig() != null) { - PartitionsSpec partitionsSpec = compactionTask.getTuningConfig().getPartitionsSpec(); - if (!(partitionsSpec instanceof DynamicPartitionsSpec - || partitionsSpec instanceof DimensionRangePartitionsSpec)) { - return new NonnullPair<>(false, StringUtils.format( - "Invalid partition spec type[%s] for MSQ compaction engine." - + " Type must be either DynamicPartitionsSpec or DynamicRangePartitionsSpec.", - partitionsSpec.getClass() - )); - } - if (partitionsSpec instanceof DynamicPartitionsSpec - && ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows() != null) { - return new NonnullPair<>(false, StringUtils.format( - "maxTotalRows[%d] in DynamicPartitionsSpec not supported for MSQ compaction engine.", - ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows() - )); + ClientCompactionRunnerInfo.ValidationResult partitionSpecValidationResult = + ClientCompactionRunnerInfo.validatePartitionsSpec(compactionTask.getTuningConfig().getPartitionsSpec()); + if (!partitionSpecValidationResult.isValid()) { + return partitionSpecValidationResult; } } - if (compactionTask.getMetricsSpec() != null - && compactionTask.getGranularitySpec() != null - && !compactionTask.getGranularitySpec() - .isRollup()) { - return new NonnullPair<>( - false, - "rollup in granularitySpec must be set to true if metricsSpec is specifed " - + "for MSQ compaction engine." + if (compactionTask.getGranularitySpec() != null) { + ClientCompactionRunnerInfo.ValidationResult rollupValidationResult = ClientCompactionRunnerInfo.validateRollup( + compactionTask.getMetricsSpec(), + compactionTask.getGranularitySpec().isRollup() ); + if (!rollupValidationResult.isValid()) { + return rollupValidationResult; + } } - - QueryContext compactionTaskContext = new QueryContext(compactionTask.getContext()); - if (!MultiStageQueryContext.isFinalizeAggregations(compactionTaskContext)) { - return new NonnullPair<>(false, StringUtils.format( - "Config[%s] cannot be set to false for auto-compaction with MSQ engine.", - MultiStageQueryContext.CTX_FINALIZE_AGGREGATIONS - )); + ClientCompactionRunnerInfo.ValidationResult finalizeAggregationValidationResult = + ClientCompactionRunnerInfo.validateFinalizeAggregations(compactionTask.getContext()); + if (!finalizeAggregationValidationResult.isValid()) { + return finalizeAggregationValidationResult; } - if (MultiStageQueryContext.getAssignmentStrategy(compactionTaskContext) == WorkerAssignmentStrategy.AUTO) { - return new NonnullPair<>( - false, - StringUtils.format( - "Config[%s] cannot be set to value[%s] for auto-compaction with MSQ engine.", - MultiStageQueryContext.CTX_TASK_ASSIGNMENT_STRATEGY, - WorkerAssignmentStrategy.AUTO - ) - ); - } - return new NonnullPair<>(true, ""); + return new ClientCompactionRunnerInfo.ValidationResult(true, null); } @Override @@ -167,11 +160,11 @@ public CurrentSubTaskHolder getCurrentSubTaskHolder() @Override public TaskStatus runCompactionTasks( CompactionTask compactionTask, - List> intervalDataSchemas, + Map intervalDataSchemas, TaskToolbox taskToolbox ) throws Exception { - List msqControllerTasks = compactionToMSQTasks(compactionTask, intervalDataSchemas); + List msqControllerTasks = createMsqControllerTasks(compactionTask, intervalDataSchemas); if (msqControllerTasks.isEmpty()) { String msg = StringUtils.format( @@ -188,22 +181,22 @@ public TaskStatus runCompactionTasks( ); } - public List compactionToMSQTasks( + public List createMsqControllerTasks( CompactionTask compactionTask, - List> intervalDataSchemas + Map intervalDataSchemas ) throws JsonProcessingException { - List msqControllerTasks = new ArrayList<>(); + final List msqControllerTasks = new ArrayList<>(); - for (NonnullPair intervalDataSchema : intervalDataSchemas) { + for (Map.Entry intervalDataSchema : intervalDataSchemas.entrySet()) { Query query; - Interval interval = intervalDataSchema.lhs; - DataSchema dataSchema = intervalDataSchema.rhs; + Interval interval = intervalDataSchema.getKey(); + DataSchema dataSchema = intervalDataSchema.getValue(); - if (!isGroupBy(dataSchema)) { - query = buildScanQuery(compactionTask, interval, dataSchema); - } else { + if (isGroupBy(dataSchema)) { query = buildGroupByQuery(compactionTask, interval, dataSchema); + } else { + query = buildScanQuery(compactionTask, interval, dataSchema); } QueryContext compactionTaskContext = new QueryContext(compactionTask.getContext()); @@ -314,7 +307,7 @@ private static List getAggregateDimensions(DataSchema dataSchema) if (isQueryGranularityEmptyOrNone(dataSchema)) { // Dimensions in group-by aren't allowed to have time column name as the output name. - dimensionSpecs.add(new DefaultDimensionSpec(TIME_COLUMN, TIME_VIRTUAL_COLUMN, ColumnType.LONG)); + dimensionSpecs.add(new DefaultDimensionSpec(ColumnHolder.TIME_COLUMN_NAME, TIME_VIRTUAL_COLUMN, ColumnType.LONG)); } else { // The changed granularity would result in a new virtual column that needs to be aggregated upon. dimensionSpecs.add(new DefaultDimensionSpec(TIME_VIRTUAL_COLUMN, TIME_VIRTUAL_COLUMN, ColumnType.LONG)); @@ -347,9 +340,9 @@ private static ColumnMappings getColumnMappings(DataSchema dataSchema) // For group-by queries, time will always be one of the dimension. Since dimensions in groupby aren't allowed to // have time column as the output name, we map time dimension to a fixed column name in dimensions, and map it // back to the time column here. - columnMappings.add(new ColumnMapping(TIME_VIRTUAL_COLUMN, TIME_COLUMN)); + columnMappings.add(new ColumnMapping(TIME_VIRTUAL_COLUMN, ColumnHolder.TIME_COLUMN_NAME)); } else { - columnMappings.add(new ColumnMapping(TIME_COLUMN, TIME_COLUMN)); + columnMappings.add(new ColumnMapping(ColumnHolder.TIME_COLUMN_NAME, ColumnHolder.TIME_COLUMN_NAME)); } return new ColumnMappings(columnMappings); } @@ -407,7 +400,7 @@ private static VirtualColumns getVirtualColumns(DataSchema dataSchema) TIME_VIRTUAL_COLUMN, StringUtils.format( "timestamp_floor(\"%s\", '%s')", - TIME_COLUMN, + ColumnHolder.TIME_COLUMN_NAME, periodQueryGranularity.getPeriod().toString() ), ColumnType.LONG, diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java index ca354d23bce3..5e2f4ec99b4f 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java @@ -41,7 +41,6 @@ import org.apache.druid.indexing.common.task.CompactionTask; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.NonnullPair; import org.apache.druid.java.util.common.granularity.GranularityType; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination; @@ -94,7 +93,7 @@ public class MSQCompactionRunnerTest private static final ObjectMapper JSON_MAPPER = new DefaultObjectMapper(); private static final AggregatorFactory AGG1 = new CountAggregatorFactory("agg_0"); private static final AggregatorFactory AGG2 = new LongSumAggregatorFactory("agg_1", "long_dim_1"); - private static List AGGREGATORS = ImmutableList.of(AGG1, AGG2); + private static final List AGGREGATORS = ImmutableList.of(AGG1, AGG2); private static final MSQCompactionRunner MSQ_COMPACTION_RUNNER = new MSQCompactionRunner(JSON_MAPPER, null); @BeforeClass @@ -128,7 +127,7 @@ public void testHashPartitionsSpec() throws JsonProcessingException null, Collections.emptyMap() ); - Assert.assertFalse(MSQ_COMPACTION_RUNNER.supportsCompactionSpec(compactionTask).lhs); + Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); } @Test @@ -139,7 +138,7 @@ public void testDimensionRangePartitionsSpec() throws JsonProcessingException null, Collections.emptyMap() ); - Assert.assertTrue(MSQ_COMPACTION_RUNNER.supportsCompactionSpec(compactionTask).lhs); + Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); } @Test @@ -150,7 +149,7 @@ public void testInvalidDynamicPartitionsSpec() throws JsonProcessingException null, Collections.emptyMap() ); - Assert.assertFalse(MSQ_COMPACTION_RUNNER.supportsCompactionSpec(compactionTask).lhs); + Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); } @Test @@ -161,7 +160,7 @@ public void testDynamicPartitionsSpec() throws JsonProcessingException null, Collections.emptyMap() ); - Assert.assertTrue(MSQ_COMPACTION_RUNNER.supportsCompactionSpec(compactionTask).lhs); + Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); } @Test @@ -172,7 +171,7 @@ public void testWithWorkerStrategyAuto() throws JsonProcessingException null, ImmutableMap.of(MultiStageQueryContext.CTX_TASK_ASSIGNMENT_STRATEGY, WorkerAssignmentStrategy.AUTO.toString()) ); - Assert.assertFalse(MSQ_COMPACTION_RUNNER.supportsCompactionSpec(compactionTask).lhs); + Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); } @Test @@ -183,14 +182,14 @@ public void testWithFinalizeAggregationsFalse() throws JsonProcessingException null, ImmutableMap.of(MultiStageQueryContext.CTX_FINALIZE_AGGREGATIONS, false) ); - Assert.assertFalse(MSQ_COMPACTION_RUNNER.supportsCompactionSpec(compactionTask).lhs); + Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); } @Test public void testRunCompactionTasksWithEmptyTaskList() throws Exception { CompactionTask compactionTask = createCompactionTask(null, null, Collections.emptyMap()); - TaskStatus taskStatus = MSQ_COMPACTION_RUNNER.runCompactionTasks(compactionTask, Collections.emptyList(), null); + TaskStatus taskStatus = MSQ_COMPACTION_RUNNER.runCompactionTasks(compactionTask, Collections.emptyMap(), null); Assert.assertTrue(taskStatus.isFailure()); } @@ -219,14 +218,10 @@ public void testMSQControllerTaskSpecWithScan() throws JsonProcessingException ); - List msqControllerTasks = MSQ_COMPACTION_RUNNER - .compactionToMSQTasks( - taskCreatedWithTransformSpec, - Collections.singletonList(new NonnullPair<>( - COMPACTION_INTERVAL, - dataSchema - )) - ); + List msqControllerTasks = MSQ_COMPACTION_RUNNER.createMsqControllerTasks( + taskCreatedWithTransformSpec, + Collections.singletonMap(COMPACTION_INTERVAL, dataSchema) + ); MSQControllerTask msqControllerTask = Iterables.getOnlyElement(msqControllerTasks); @@ -288,14 +283,10 @@ public void testMSQControllerTaskSpecWithAggregators() throws JsonProcessingExce ); - List msqControllerTasks = MSQ_COMPACTION_RUNNER - .compactionToMSQTasks( - taskCreatedWithTransformSpec, - Collections.singletonList(new NonnullPair<>( - COMPACTION_INTERVAL, - dataSchema - )) - ); + List msqControllerTasks = MSQ_COMPACTION_RUNNER.createMsqControllerTasks( + taskCreatedWithTransformSpec, + Collections.singletonMap(COMPACTION_INTERVAL, dataSchema) + ); MSQControllerTask msqControllerTask = Iterables.getOnlyElement(msqControllerTasks); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java index e7d82e154096..5b4d1dd4aea7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java @@ -23,12 +23,11 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.java.util.common.NonnullPair; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.server.coordinator.ClientCompactionRunnerInfo; import org.joda.time.Interval; -import java.util.List; +import java.util.Map; /** * Strategy to be used for executing a compaction task. @@ -36,15 +35,18 @@ */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = CompactionRunner.TYPE_PROPERTY) @JsonSubTypes(value = { - @JsonSubTypes.Type(name = "native", value = NativeCompactionRunner.class) + @JsonSubTypes.Type(name = NativeCompactionRunner.TYPE, value = NativeCompactionRunner.class) }) public interface CompactionRunner { String TYPE_PROPERTY = "type"; + /** + * Converts compaction tasks to the runner-type tasks and runs them serially for each interval. + */ TaskStatus runCompactionTasks( CompactionTask compactionTask, - List> dataSchemas, + Map intervalDataSchemaMap, TaskToolbox taskToolbox ) throws Exception; @@ -52,10 +54,9 @@ TaskStatus runCompactionTasks( /** * Checks if the provided compaction config is supported by the runner. - * - * @param compactionTask - * @return Pair of (supported) boolean and a reason string. Reason string is empty if supported is true. + * The same validation is done at {@link org.apache.druid.msq.indexing.MSQCompactionRunner#validateCompactionTask} + * @return ValidationResult. The reason string is null if isValid() is True. */ - NonnullPair supportsCompactionSpec(CompactionTask compactionTask); + ClientCompactionRunnerInfo.ValidationResult validateCompactionTask(CompactionTask compactionTask); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 4176c7e73f6a..d8791d1eeddd 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -85,6 +85,7 @@ import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; +import org.apache.druid.server.coordinator.ClientCompactionRunnerInfo; import org.apache.druid.server.coordinator.duty.CompactSegments; import org.apache.druid.server.lookup.cache.LookupLoadingSpec; import org.apache.druid.server.security.ResourceAction; @@ -465,7 +466,7 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception // emit metric for compact ingestion mode: emitCompactIngestionModeMetrics(toolbox.getEmitter(), ioConfig.isDropExisting()); - final List> intervalDataSchemas = createDataSchemasForIntervals( + final Map intervalDataSchemas = createDataSchemasForIntervals( UTC_CLOCK, toolbox, getTaskLockHelper().getLockGranularityToUse(), @@ -478,9 +479,9 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception ); registerResourceCloserOnAbnormalExit(compactionRunner.getCurrentSubTaskHolder()); - NonnullPair supportsCompactionConfig = compactionRunner.supportsCompactionSpec(this); - if (!supportsCompactionConfig.lhs) { - throw InvalidInput.exception("Compaction spec not supported. Reason[%s].", supportsCompactionConfig.rhs); + ClientCompactionRunnerInfo.ValidationResult supportsCompactionConfig = compactionRunner.validateCompactionTask(this); + if (!supportsCompactionConfig.isValid()) { + throw InvalidInput.exception("Compaction spec not supported. Reason[%s].", supportsCompactionConfig.getReason()); } return compactionRunner.runCompactionTasks(this, intervalDataSchemas, toolbox); } @@ -491,7 +492,7 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception * @throws IOException */ @VisibleForTesting - static List> createDataSchemasForIntervals( + static Map createDataSchemasForIntervals( final Clock clock, final TaskToolbox toolbox, final LockGranularity lockGranularityInUse, @@ -510,11 +511,11 @@ static List> createDataSchemasForIntervals( ); if (timelineSegments.isEmpty()) { - return Collections.emptyList(); + return Collections.emptyMap(); } if (granularitySpec == null || granularitySpec.getSegmentGranularity() == null) { - List> intervalToDataSchemaList = new ArrayList<>(); + Map intervalDataSchemaMap = new HashMap<>(); // original granularity final Map> intervalToSegments = new TreeMap<>( @@ -567,9 +568,9 @@ static List> createDataSchemasForIntervals( ? new ClientCompactionTaskGranularitySpec(segmentGranularityToUse, null, null) : granularitySpec.withSegmentGranularity(segmentGranularityToUse) ); - intervalToDataSchemaList.add(new NonnullPair<>(interval, dataSchema)); + intervalDataSchemaMap.put(interval, dataSchema); } - return intervalToDataSchemaList; + return intervalDataSchemaMap; } else { // given segment granularity final DataSchema dataSchema = createDataSchema( @@ -593,7 +594,7 @@ static List> createDataSchemasForIntervals( metricsSpec, granularitySpec ); - return Collections.singletonList(new NonnullPair<>(segmentProvider.interval, dataSchema)); + return Collections.singletonMap(segmentProvider.interval, dataSchema); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java index 28031a69154a..0f096cbf45a1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java @@ -41,6 +41,7 @@ import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.server.coordinator.ClientCompactionRunnerInfo; import org.apache.druid.server.coordinator.duty.CompactSegments; import org.apache.druid.utils.CollectionUtils; import org.codehaus.jackson.annotate.JsonCreator; @@ -57,6 +58,7 @@ public class NativeCompactionRunner implements CompactionRunner { private static final Logger log = new Logger(NativeCompactionRunner.class); + public static final String TYPE = "native"; private static final boolean STORE_COMPACTION_STATE = true; @JsonIgnore private final SegmentCacheManagerFactory segmentCacheManagerFactory; @@ -80,11 +82,11 @@ public CurrentSubTaskHolder getCurrentSubTaskHolder() } @Override - public NonnullPair supportsCompactionSpec( + public ClientCompactionRunnerInfo.ValidationResult validateCompactionTask( CompactionTask compactionTask ) { - return new NonnullPair<>(true, ""); + return new ClientCompactionRunnerInfo.ValidationResult(true, null); } /** @@ -94,7 +96,7 @@ public NonnullPair supportsCompactionSpec( */ @VisibleForTesting static List createIngestionSpecs( - List> dataschemas, + Map intervalDataSchemaMap, final TaskToolbox toolbox, final CompactionIOConfig ioConfig, final PartitionConfigurationManager partitionConfigurationManager, @@ -104,12 +106,12 @@ static List createIngestionSpecs( { final CompactionTask.CompactionTuningConfig compactionTuningConfig = partitionConfigurationManager.computeTuningConfig(); - return dataschemas.stream().map((dataSchema) -> new ParallelIndexIngestionSpec( - dataSchema.rhs, + return intervalDataSchemaMap.entrySet().stream().map((dataSchema) -> new ParallelIndexIngestionSpec( + dataSchema.getValue(), createIoConfig( toolbox, - dataSchema.rhs, - dataSchema.lhs, + dataSchema.getValue(), + dataSchema.getKey(), coordinatorClient, segmentCacheManagerFactory, ioConfig @@ -175,7 +177,7 @@ private static ParallelIndexIOConfig createIoConfig( @Override public TaskStatus runCompactionTasks( CompactionTask compactionTask, - List> dataSchemas, + Map intervalDataSchemaMap, TaskToolbox taskToolbox ) throws Exception { @@ -184,7 +186,7 @@ public TaskStatus runCompactionTasks( final List ingestionSpecs = createIngestionSpecs( - dataSchemas, + intervalDataSchemaMap, taskToolbox, compactionTask.getIoConfig(), partitionConfigurationManager, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index 7ecccb28253b..db7ca8ef80e0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -966,7 +966,7 @@ public void testSegmentProviderFindSegmentsWithEmptySegmentsThrowException() @Test public void testCreateIngestionSchema() throws IOException { - final List> dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( clock, toolbox, LockGranularity.TIME_CHUNK, @@ -1048,7 +1048,7 @@ public void testCreateIngestionSchemaWithTargetPartitionSize() throws IOExceptio null, null ); - final List> dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( clock, toolbox, LockGranularity.TIME_CHUNK, @@ -1131,7 +1131,7 @@ public void testCreateIngestionSchemaWithMaxTotalRows() throws IOException null, null ); - final List> dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( clock, toolbox, LockGranularity.TIME_CHUNK, @@ -1214,7 +1214,7 @@ public void testCreateIngestionSchemaWithNumShards() throws IOException null, null ); - final List> dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( clock, toolbox, LockGranularity.TIME_CHUNK, @@ -1285,7 +1285,7 @@ public void testCreateIngestionSchemaWithCustomDimensionsSpec() throws IOExcepti ) ); - final List> dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( clock, toolbox, LockGranularity.TIME_CHUNK, @@ -1336,7 +1336,7 @@ public void testCreateIngestionSchemaWithCustomMetricsSpec() throws IOException new DoubleMaxAggregatorFactory("custom_double_max", "agg_4") }; - final List> dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( clock, toolbox, LockGranularity.TIME_CHUNK, @@ -1380,7 +1380,7 @@ public void testCreateIngestionSchemaWithCustomMetricsSpec() throws IOException @Test public void testCreateIngestionSchemaWithCustomSegments() throws IOException { - final List> dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( clock, toolbox, LockGranularity.TIME_CHUNK, @@ -1431,7 +1431,7 @@ public void testCreateIngestionSchemaWithDifferentSegmentSet() throws IOExceptio Collections.sort(segments); // Remove one segment in the middle segments.remove(segments.size() / 2); - final List> dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( clock, toolbox, LockGranularity.TIME_CHUNK, @@ -1462,7 +1462,7 @@ public void testMissingMetadata() throws IOException final TestIndexIO indexIO = (TestIndexIO) toolbox.getIndexIO(); indexIO.removeMetadata(Iterables.getFirst(indexIO.getQueryableIndexMap().keySet(), null)); final List segments = new ArrayList<>(SEGMENTS); - final List> dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( clock, toolbox, LockGranularity.TIME_CHUNK, @@ -1505,7 +1505,7 @@ public void testEmptyInterval() @Test public void testSegmentGranularityAndNullQueryGranularity() throws IOException { - final List> dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( clock, toolbox, LockGranularity.TIME_CHUNK, @@ -1550,7 +1550,7 @@ public void testSegmentGranularityAndNullQueryGranularity() throws IOException @Test public void testQueryGranularityAndNullSegmentGranularity() throws IOException { - final List> dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( clock, toolbox, LockGranularity.TIME_CHUNK, @@ -1592,7 +1592,7 @@ public void testQueryGranularityAndNullSegmentGranularity() throws IOException @Test public void testQueryGranularityAndSegmentGranularityNonNull() throws IOException { - final List> dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( clock, toolbox, LockGranularity.TIME_CHUNK, @@ -1644,7 +1644,7 @@ public void testQueryGranularityAndSegmentGranularityNonNull() throws IOExceptio @Test public void testNullGranularitySpec() throws IOException { - final List> dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( clock, toolbox, LockGranularity.TIME_CHUNK, @@ -1690,7 +1690,7 @@ public void testNullGranularitySpec() throws IOException public void testGranularitySpecWithNullQueryGranularityAndNullSegmentGranularity() throws IOException { - final List> dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( clock, toolbox, LockGranularity.TIME_CHUNK, @@ -1736,7 +1736,7 @@ public void testGranularitySpecWithNullQueryGranularityAndNullSegmentGranularity public void testGranularitySpecWithNotNullRollup() throws IOException { - final List> dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( clock, toolbox, LockGranularity.TIME_CHUNK, @@ -1767,7 +1767,7 @@ public void testGranularitySpecWithNotNullRollup() public void testGranularitySpecWithNullRollup() throws IOException { - final List> dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( clock, toolbox, LockGranularity.TIME_CHUNK, diff --git a/processing/src/main/java/org/apache/druid/indexer/CompactionEngine.java b/processing/src/main/java/org/apache/druid/indexer/CompactionEngine.java index 1758a9183b69..b14b0c453271 100644 --- a/processing/src/main/java/org/apache/druid/indexer/CompactionEngine.java +++ b/processing/src/main/java/org/apache/druid/indexer/CompactionEngine.java @@ -31,7 +31,6 @@ */ public enum CompactionEngine { - // Using uppercase constants since "native" is a reserved keyword. NATIVE, MSQ; @@ -39,7 +38,7 @@ public enum CompactionEngine @JsonValue public String toString() { - return this.name().toLowerCase(); + return StringUtils.toLowerCase(this.name().toLowerCase()); } @JsonCreator diff --git a/processing/src/main/java/org/apache/druid/timeline/CompactionState.java b/processing/src/main/java/org/apache/druid/timeline/CompactionState.java index 545e63282c6c..ecc4c279b922 100644 --- a/processing/src/main/java/org/apache/druid/timeline/CompactionState.java +++ b/processing/src/main/java/org/apache/druid/timeline/CompactionState.java @@ -27,6 +27,7 @@ import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.query.aggregation.AggregatorFactory; +import javax.annotation.Nullable; import java.util.List; import java.util.Map; import java.util.Objects; @@ -68,12 +69,13 @@ public class CompactionState public CompactionState( @JsonProperty("partitionsSpec") PartitionsSpec partitionsSpec, @JsonProperty("dimensionsSpec") DimensionsSpec dimensionsSpec, - @JsonProperty("dimensionToAggregatoryFactoryMap") Map dimensionToAggregatoryFactoryMap, + @Nullable @JsonProperty("dimensionToAggregatoryFactoryMap") + Map dimensionToAggregatoryFactoryMap, @JsonProperty("metricsSpec") List metricsSpec, @JsonProperty("transformSpec") Map transformSpec, @JsonProperty("indexSpec") Map indexSpec, @JsonProperty("granularitySpec") Map granularitySpec, - @JsonProperty("engine") CompactionEngine engine + @Nullable @JsonProperty("engine") CompactionEngine engine ) { this.partitionsSpec = partitionsSpec; @@ -123,12 +125,14 @@ public Map getGranularitySpec() } @JsonProperty + @Nullable public CompactionEngine getEngine() { return engine; } @JsonProperty + @Nullable public Map getDimensionToAggregatoryFactoryMap() { return dimensionToAggregatoryFactoryMap; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfo.java b/server/src/main/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfo.java index 53d316d522e7..ffca7780320f 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfo.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfo.java @@ -25,10 +25,11 @@ import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; -import org.apache.druid.java.util.common.NonnullPair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.QueryContext; +import org.apache.druid.query.aggregation.AggregatorFactory; +import java.util.Map; import java.util.Objects; @@ -73,73 +74,133 @@ public boolean equals(Object o) return type == that.type; } + public static class ValidationResult{ + private final boolean valid; + private final String reason; + + public ValidationResult(boolean valid, String reason) + { + this.valid = valid; + this.reason = reason; + } + + public boolean isValid() + { + return valid; + } + + public String getReason() + { + return reason; + } + } + @Override public int hashCode() { return Objects.hash(type); } + public static ValidationResult supportsCompactionConfig(DataSourceCompactionConfig newConfig) + { + CompactionEngine compactionEngine = newConfig.getEngine(); + if (compactionEngine == CompactionEngine.NATIVE) { + return new ValidationResult(true, null); + } else { + return msqEngineSupportsCompactionConfig(newConfig); + } + } + /** - * Checks if the provided compaction config is supported by the runner + * Checks if the provided compaction config is supported by MSQ. The following configs aren't supported: + *
    + *
  • finalizeAggregations set to false in context.
  • + * + *
  • partitionsSpec of type HashedParititionsSpec.
  • + * + *
  • maxTotalRows in DynamicPartitionsSpec.
  • + * + *
  • rollup set to false in granularitySpec when metricsSpec is specified.
  • + *
+ * * @param newConfig The updated compaction config - * @param engineSource String indicating the source of compaction engine. - * @return Pair of support boolean and reason string. The reason string is empty if support boolean is True. + * @return ValidationResult. The reason string is null if isValid() is True. */ - public static NonnullPair supportsCompactionConfig(DataSourceCompactionConfig newConfig, String engineSource) + private static ValidationResult msqEngineSupportsCompactionConfig(DataSourceCompactionConfig newConfig) { - CompactionEngine compactionEngine = newConfig.getEngine(); - if (compactionEngine == CompactionEngine.MSQ) { - if (newConfig.getTuningConfig() != null) { - PartitionsSpec partitionsSpec = newConfig.getTuningConfig().getPartitionsSpec(); - if (!(partitionsSpec instanceof DimensionRangePartitionsSpec - || partitionsSpec instanceof DynamicPartitionsSpec)) { - return new NonnullPair<>(false, StringUtils.format( - "Invalid partition spec type[%s] for MSQ compaction engine[%s]." - + " Type must be either DynamicPartitionsSpec or DynamicRangePartitionsSpec.", - partitionsSpec.getClass(), - engineSource - ) - ); - } - if (partitionsSpec instanceof DynamicPartitionsSpec - && ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows() != null) { - return new NonnullPair<>(false, StringUtils.format( - "maxTotalRows[%d] in DynamicPartitionsSpec not supported for MSQ compaction engine[%s].", - ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows(), engineSource - )); - } + if (newConfig.getTuningConfig() != null) { + ValidationResult partitionSpecValidationResult = + validatePartitionsSpec(newConfig.getTuningConfig().getPartitionsSpec()); + if (!partitionSpecValidationResult.isValid()) { + return partitionSpecValidationResult; } - - if (newConfig.getMetricsSpec() != null - && newConfig.getGranularitySpec() != null - && !newConfig.getGranularitySpec() - .isRollup()) { - return new NonnullPair<>(false, StringUtils.format( - "rollup in granularitySpec must be set to True if metricsSpec is specifed " - + "for MSQ compaction engine[%s].", engineSource)); + } + if (newConfig.getGranularitySpec() != null) { + ValidationResult rollupValidationResult = validateRollup( + newConfig.getMetricsSpec(), + newConfig.getGranularitySpec().isRollup() + ); + if (!rollupValidationResult.isValid()) { + return rollupValidationResult; } + } + ValidationResult finalizeAggregationValidationResult = validateFinalizeAggregations(newConfig.getTaskContext()); + if (!finalizeAggregationValidationResult.isValid()) { + return finalizeAggregationValidationResult; + } + return new ValidationResult(true, null); + } - QueryContext queryContext = QueryContext.of(newConfig.getTaskContext()); + /** + * Validte that partitionSpec is either 'dynamic` or 'range', and if 'dynamic', ensure maxTotalRows is null. + */ + public static ValidationResult validatePartitionsSpec(PartitionsSpec partitionsSpec) + { + if (!(partitionsSpec instanceof DimensionRangePartitionsSpec + || partitionsSpec instanceof DynamicPartitionsSpec)) { + return new ValidationResult(false, StringUtils.format( + "Invalid partition spec type[%s] for MSQ engine." + + " Type must be either DynamicPartitionsSpec or DynamicRangePartitionsSpec.", + partitionsSpec.getClass() + ) + ); + } + if (partitionsSpec instanceof DynamicPartitionsSpec + && ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows() != null) { + return new ValidationResult(false, StringUtils.format( + "maxTotalRows[%d] in DynamicPartitionsSpec not supported for MSQ engine.", + ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows() + )); + } + return new ValidationResult(true, null); + } - if (!queryContext.getBoolean(MSQContext.CTX_FINALIZE_AGGREGATIONS, true)) { - return new NonnullPair<>(false, StringUtils.format( - "Config[%s] cannot be set to false for auto-compaction with MSQ engine[%s].", - MSQContext.CTX_FINALIZE_AGGREGATIONS, - engineSource - )); - } + /** + * Validate rollup is set to false in granularitySpec when metricsSpec is specified. + */ + public static ValidationResult validateRollup(AggregatorFactory[] metricsSpec, boolean isRollup) { + if (metricsSpec != null + ) { + return new ValidationResult(false, StringUtils.format( + "rollup in granularitySpec must be set to True if metricsSpec is specifed " + + "for MSQ engine.")); + } + return new ValidationResult(true, null); + } - if (queryContext.getString(MSQContext.CTX_TASK_ASSIGNMENT_STRATEGY, MSQContext.TASK_ASSIGNMENT_STRATEGY_MAX) - .equals(MSQContext.TASK_ASSIGNMENT_STRATEGY_AUTO)) { - return new NonnullPair<>(false, StringUtils.format( - "Config[%s] cannot be set to value[%s] for auto-compaction with MSQ engine[%s].", - MSQContext.CTX_TASK_ASSIGNMENT_STRATEGY, - MSQContext.TASK_ASSIGNMENT_STRATEGY_AUTO, - engineSource - )); - } + /** + * Validate finalizeAggregations is not set to false + */ + public static ValidationResult validateFinalizeAggregations(Map context){ + QueryContext queryContext = QueryContext.of(context); + + if (!queryContext.getBoolean(MSQContext.CTX_FINALIZE_AGGREGATIONS, true)) { + return new ValidationResult(false, StringUtils.format( + "Config[%s] cannot be set to false for auto-compaction with MSQ engine.", + MSQContext.CTX_FINALIZE_AGGREGATIONS + )); } - return new NonnullPair<>(true, ""); + return new ValidationResult(true, null); } /** @@ -150,8 +211,5 @@ public static NonnullPair supportsCompactionConfig(DataSourceCo public static class MSQContext { public static final String CTX_FINALIZE_AGGREGATIONS = "finalizeAggregations"; - public static final String CTX_TASK_ASSIGNMENT_STRATEGY = "taskAssignment"; - public static final String TASK_ASSIGNMENT_STRATEGY_MAX = "MAX"; - public static final String TASK_ASSIGNMENT_STRATEGY_AUTO = "AUTO"; } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java index 91c22d0f5ede..0895427d24dd 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java @@ -24,7 +24,6 @@ import com.google.common.base.Preconditions; import org.apache.druid.error.InvalidInput; import org.apache.druid.indexer.CompactionEngine; -import org.apache.druid.java.util.common.NonnullPair; import org.apache.druid.query.aggregation.AggregatorFactory; import org.joda.time.Period; @@ -236,10 +235,8 @@ public static DataSourceCompactionConfig from( ) { CompactionEngine newCompactionEngine = newConfig.getEngine(); - String engineSource = "specified in spec"; if (newCompactionEngine == null) { newCompactionEngine = defaultCompactionEngine; - engineSource = "set as default"; newConfig = new DataSourceCompactionConfig( newConfig.getDataSource(), newConfig.getTaskPriority(), @@ -257,12 +254,11 @@ public static DataSourceCompactionConfig from( ); } - NonnullPair supportsCompactionConfig = ClientCompactionRunnerInfo.supportsCompactionConfig( - newConfig, - engineSource + ClientCompactionRunnerInfo.ValidationResult validationResult = ClientCompactionRunnerInfo.supportsCompactionConfig( + newConfig ); - if (!supportsCompactionConfig.lhs) { - throw InvalidInput.exception("Compaction config not supported. Reason[%s].", supportsCompactionConfig.rhs); + if (!validationResult.isValid()) { + throw InvalidInput.exception("Compaction config not supported. Reason[%s].", validationResult.getReason()); } return newConfig; } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java index 8388e8bf4bc9..f67585864820 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java @@ -329,6 +329,8 @@ private CompactionStatus metricsSpecIsUpToDate() final List metricSpecList = lastCompactionState.getMetricsSpec(); final AggregatorFactory[] existingMetricsSpec; if (CollectionUtils.isNullOrEmpty(metricSpecList)) { + // Use the dimensionToAggregatorFactoryMap to check metrics that were converted to dimensions due to + // finalizeAggregations=true final Map dimensionToAggregatorFactoryMap = lastCompactionState.getDimensionToAggregatoryFactoryMap(); existingMetricsSpec = dimensionToAggregatorFactoryMap == null diff --git a/server/src/test/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfoTest.java b/server/src/test/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfoTest.java index 0a4286d749de..c863a16c7283 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfoTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfoTest.java @@ -50,9 +50,8 @@ public void testHashedPartitionsSpecs() { assertFalse( ClientCompactionRunnerInfo.supportsCompactionConfig( - createCompactionConfig(new HashedPartitionsSpec(100, null, null), Collections.emptyMap()), - "dummy" - ).lhs + createCompactionConfig(new HashedPartitionsSpec(100, null, null), Collections.emptyMap()) + ).isValid() ); } @@ -61,9 +60,8 @@ public void testrInvalidDynamicPartitionsSpecs() { assertFalse( ClientCompactionRunnerInfo.supportsCompactionConfig( - createCompactionConfig(new DynamicPartitionsSpec(100, 100L), Collections.emptyMap()), - "dummy" - ).lhs + createCompactionConfig(new DynamicPartitionsSpec(100, 100L), Collections.emptyMap()) + ).isValid() ); } @@ -71,9 +69,8 @@ public void testrInvalidDynamicPartitionsSpecs() public void testDynamicPartitionsSpecs() { assertTrue(ClientCompactionRunnerInfo.supportsCompactionConfig( - createCompactionConfig(new DynamicPartitionsSpec(100, null), Collections.emptyMap()), - "dummy" - ).lhs); + createCompactionConfig(new DynamicPartitionsSpec(100, null), Collections.emptyMap()) + ).isValid()); } @Test @@ -83,19 +80,8 @@ public void testDimensionRangePartitionsSpecs() createCompactionConfig( new DimensionRangePartitionsSpec(100, null, ImmutableList.of("partitionDim"), false), Collections.emptyMap() - ), - "engine" - ).lhs); - } - - @Test - public void testWithWorkerStrategyAuto() - { - DataSourceCompactionConfig compactionConfig = createCompactionConfig( - new DynamicPartitionsSpec(3, null), - ImmutableMap.of(ClientCompactionRunnerInfo.MSQContext.CTX_TASK_ASSIGNMENT_STRATEGY, "AUTO") - ); - Assert.assertFalse(ClientCompactionRunnerInfo.supportsCompactionConfig(compactionConfig, "").lhs); + ) + ).isValid()); } @Test @@ -105,7 +91,7 @@ public void testWithFinalizeAggregationsFalse() new DynamicPartitionsSpec(3, null), ImmutableMap.of(ClientCompactionRunnerInfo.MSQContext.CTX_FINALIZE_AGGREGATIONS, false) ); - Assert.assertFalse(ClientCompactionRunnerInfo.supportsCompactionConfig(compactionConfig, "").lhs); + Assert.assertFalse(ClientCompactionRunnerInfo.supportsCompactionConfig(compactionConfig).isValid()); } private static DataSourceCompactionConfig createCompactionConfig( From bcfab9b7739d67bba16ea657e01da6571b8bbdc3 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Thu, 20 Jun 2024 09:26:35 +0530 Subject: [PATCH 37/54] Remove finalizeAggregations check --- .../msq/indexing/MSQCompactionRunner.java | 5 ---- .../common/task/NativeCompactionRunner.java | 1 - .../ClientCompactionRunnerInfo.java | 24 ++----------------- .../DataSourceCompactionConfig.java | 2 +- .../ClientCompactionRunnerInfoTest.java | 10 ++++---- 5 files changed, 8 insertions(+), 34 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java index 2ab00714795a..84ffad31f541 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java @@ -143,11 +143,6 @@ public ClientCompactionRunnerInfo.ValidationResult validateCompactionTask( return rollupValidationResult; } } - ClientCompactionRunnerInfo.ValidationResult finalizeAggregationValidationResult = - ClientCompactionRunnerInfo.validateFinalizeAggregations(compactionTask.getContext()); - if (!finalizeAggregationValidationResult.isValid()) { - return finalizeAggregationValidationResult; - } return new ClientCompactionRunnerInfo.ValidationResult(true, null); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java index 0f096cbf45a1..28be9b958875 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java @@ -36,7 +36,6 @@ import org.apache.druid.indexing.input.DruidInputSource; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.NonnullPair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.logger.Logger; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfo.java b/server/src/main/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfo.java index ffca7780320f..15079a6ba749 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfo.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfo.java @@ -101,7 +101,7 @@ public int hashCode() return Objects.hash(type); } - public static ValidationResult supportsCompactionConfig(DataSourceCompactionConfig newConfig) + public static ValidationResult validateCompactionConfig(DataSourceCompactionConfig newConfig) { CompactionEngine compactionEngine = newConfig.getEngine(); if (compactionEngine == CompactionEngine.NATIVE) { @@ -144,10 +144,6 @@ private static ValidationResult msqEngineSupportsCompactionConfig(DataSourceComp return rollupValidationResult; } } - ValidationResult finalizeAggregationValidationResult = validateFinalizeAggregations(newConfig.getTaskContext()); - if (!finalizeAggregationValidationResult.isValid()) { - return finalizeAggregationValidationResult; - } return new ValidationResult(true, null); } @@ -179,8 +175,7 @@ public static ValidationResult validatePartitionsSpec(PartitionsSpec partitionsS * Validate rollup is set to false in granularitySpec when metricsSpec is specified. */ public static ValidationResult validateRollup(AggregatorFactory[] metricsSpec, boolean isRollup) { - if (metricsSpec != null - ) { + if (metricsSpec != null && !isRollup) { return new ValidationResult(false, StringUtils.format( "rollup in granularitySpec must be set to True if metricsSpec is specifed " + "for MSQ engine.")); @@ -188,21 +183,6 @@ public static ValidationResult validateRollup(AggregatorFactory[] metricsSpec, b return new ValidationResult(true, null); } - /** - * Validate finalizeAggregations is not set to false - */ - public static ValidationResult validateFinalizeAggregations(Map context){ - QueryContext queryContext = QueryContext.of(context); - - if (!queryContext.getBoolean(MSQContext.CTX_FINALIZE_AGGREGATIONS, true)) { - return new ValidationResult(false, StringUtils.format( - "Config[%s] cannot be set to false for auto-compaction with MSQ engine.", - MSQContext.CTX_FINALIZE_AGGREGATIONS - )); - } - return new ValidationResult(true, null); - } - /** * This class copies over MSQ context parameters from the MSQ extension. This is required to validate the submitted * compaction config at the coordinator. The values used here should be kept in sync with those in diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java index 0895427d24dd..155a2834d882 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java @@ -254,7 +254,7 @@ public static DataSourceCompactionConfig from( ); } - ClientCompactionRunnerInfo.ValidationResult validationResult = ClientCompactionRunnerInfo.supportsCompactionConfig( + ClientCompactionRunnerInfo.ValidationResult validationResult = ClientCompactionRunnerInfo.validateCompactionConfig( newConfig ); if (!validationResult.isValid()) { diff --git a/server/src/test/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfoTest.java b/server/src/test/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfoTest.java index c863a16c7283..53787bc7e2cc 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfoTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfoTest.java @@ -49,7 +49,7 @@ public class ClientCompactionRunnerInfoTest public void testHashedPartitionsSpecs() { assertFalse( - ClientCompactionRunnerInfo.supportsCompactionConfig( + ClientCompactionRunnerInfo.validateCompactionConfig( createCompactionConfig(new HashedPartitionsSpec(100, null, null), Collections.emptyMap()) ).isValid() ); @@ -59,7 +59,7 @@ public void testHashedPartitionsSpecs() public void testrInvalidDynamicPartitionsSpecs() { assertFalse( - ClientCompactionRunnerInfo.supportsCompactionConfig( + ClientCompactionRunnerInfo.validateCompactionConfig( createCompactionConfig(new DynamicPartitionsSpec(100, 100L), Collections.emptyMap()) ).isValid() ); @@ -68,7 +68,7 @@ public void testrInvalidDynamicPartitionsSpecs() @Test public void testDynamicPartitionsSpecs() { - assertTrue(ClientCompactionRunnerInfo.supportsCompactionConfig( + assertTrue(ClientCompactionRunnerInfo.validateCompactionConfig( createCompactionConfig(new DynamicPartitionsSpec(100, null), Collections.emptyMap()) ).isValid()); } @@ -76,7 +76,7 @@ public void testDynamicPartitionsSpecs() @Test public void testDimensionRangePartitionsSpecs() { - assertTrue(ClientCompactionRunnerInfo.supportsCompactionConfig( + assertTrue(ClientCompactionRunnerInfo.validateCompactionConfig( createCompactionConfig( new DimensionRangePartitionsSpec(100, null, ImmutableList.of("partitionDim"), false), Collections.emptyMap() @@ -91,7 +91,7 @@ public void testWithFinalizeAggregationsFalse() new DynamicPartitionsSpec(3, null), ImmutableMap.of(ClientCompactionRunnerInfo.MSQContext.CTX_FINALIZE_AGGREGATIONS, false) ); - Assert.assertFalse(ClientCompactionRunnerInfo.supportsCompactionConfig(compactionConfig).isValid()); + Assert.assertFalse(ClientCompactionRunnerInfo.validateCompactionConfig(compactionConfig).isValid()); } private static DataSourceCompactionConfig createCompactionConfig( From 2043bebb0b264e75c97afdaa144b37a79b0fa99a Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Thu, 20 Jun 2024 18:28:30 +0530 Subject: [PATCH 38/54] Remove dimensionToAggregatorFactory map and change source of metricsSpec to query definition --- .../apache/druid/msq/exec/ControllerImpl.java | 83 ++--- .../msq/indexing/MSQCompactionRunner.java | 8 +- ...SegmentGeneratorFrameProcessorFactory.java | 14 +- .../msq/util/MultiStageQueryContext.java | 2 +- .../apache/druid/msq/exec/MSQReplaceTest.java | 5 +- .../common/task/AbstractBatchIndexTask.java | 5 +- .../task/CompactionTaskParallelRunTest.java | 38 +-- .../common/task/CompactionTaskRunTest.java | 17 +- .../common/task/CompactionTaskTest.java | 1 - .../druid/timeline/CompactionState.java | 47 +-- .../druid/timeline/DataSegmentTest.java | 23 +- .../ClientCompactionRunnerInfo.java | 7 +- .../DataSourceCompactionConfig.java | 35 -- .../coordinator/compact/CompactionStatus.java | 39 +-- .../coordinator/duty/CompactSegments.java | 4 +- .../CoordinatorCompactionConfigsResource.java | 10 +- .../ClientCompactionRunnerInfoTest.java | 10 +- .../compact/NewestSegmentFirstPolicyTest.java | 301 ++---------------- .../coordinator/duty/CompactSegmentsTest.java | 41 +-- .../server/http/DataSegmentPlusTest.java | 8 +- .../sql/calcite/schema/SystemSchemaTest.java | 5 +- 21 files changed, 133 insertions(+), 570 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 19cd4fa2a3c9..84f7f6564711 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -57,7 +57,6 @@ import org.apache.druid.frame.util.DurableStorageUtils; import org.apache.druid.frame.write.InvalidFieldException; import org.apache.druid.frame.write.InvalidNullByteException; -import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; @@ -220,7 +219,6 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; -import java.util.LinkedHashMap; import java.util.LinkedHashSet; import java.util.List; import java.util.Map; @@ -1525,15 +1523,10 @@ private void handleQueryResults( ShardSpec shardSpec = segments.stream().findFirst().get().getShardSpec(); - Map dimensionToAggregatoryFactoryMap = - ((SegmentGeneratorFrameProcessorFactory) queryKernel - .getStageDefinition(finalStageId).getProcessorFactory()).getDimensionToAggregatorFactoryMap(); - Function, Set> compactionStateAnnotateFunction = addCompactionStateToSegments( querySpec, context.jsonMapper(), dataSchema, - dimensionToAggregatoryFactoryMap, shardSpec, queryDef.getQueryId() ); @@ -1587,7 +1580,6 @@ private static Function, Set> addCompactionStateTo MSQSpec querySpec, ObjectMapper jsonMapper, DataSchema dataSchema, - Map dimensionToAggregatorFactoryMap, ShardSpec shardSpec, String queryId ) @@ -1634,25 +1626,37 @@ private static Function, Set> addCompactionStateTo : new ClientCompactionTaskTransformSpec( dataSchema.getTransformSpec().getFilter() ).asMap(jsonMapper); - List metricsSpec = dataSchema.getAggregators() == null - ? null - : jsonMapper.convertValue( - dataSchema.getAggregators(), new TypeReference>() - { - }); - + List metricsSpec = null; + + if (querySpec.getQuery() instanceof GroupByQuery){ + GroupByQuery groupByQuery = (GroupByQuery) querySpec.getQuery(); + // Need to fetch this from the querySpec since the dataSchema uses the AggregatorFactory's combining factory + // version which updates field_name to name. For e.g. + + // LongSumAggregatorFactory{fieldName='added', expression='null', name='added_sum'} + // gets updated to + // LongSumAggregatorFactory{fieldName='added_sum', expression='null', name='added_sum'} + + // Also converting to metricsSpec from list to array as direct serialization from list doesn't capture the type. + metricsSpec = jsonMapper.convertValue( + groupByQuery.getAggregatorSpecs().toArray(new AggregatorFactory[0]), + new TypeReference>() + { + } + ); + } IndexSpec indexSpec = tuningConfig.getIndexSpec(); + log.info("Query[%s] storing compaction state in segments.", queryId); + return CompactionState.addCompactionStateToSegments( partitionSpec, dimensionsSpec, - dimensionToAggregatorFactoryMap, metricsSpec, transformSpec, indexSpec.asMap(jsonMapper), - granularitySpec.asMap(jsonMapper), - CompactionEngine.MSQ + granularitySpec.asMap(jsonMapper) ); } @@ -1773,21 +1777,9 @@ private static QueryDefinition makeQueryDefinition( } } - // Map to track the aggregator factories of dimensions that have been created from metrics due to context flag - // finalizeAggregations=true. Using LinkedHashMap to preserve order as metrics are compared as arrays - // in CompactionStatus. - final Map dimensionToAggregatoryFactoryMap = new LinkedHashMap<>(); - // Then, add a segment-generation stage. final DataSchema dataSchema = - makeDataSchemaForIngestion( - querySpec, - querySignature, - queryClusterBy, - columnMappings, - jsonMapper, - dimensionToAggregatoryFactoryMap - ); + makeDataSchemaForIngestion(querySpec, querySignature, queryClusterBy, columnMappings, jsonMapper); builder.add( StageDefinition.builder(queryDef.getNextStageNumber()) @@ -1797,8 +1789,7 @@ private static QueryDefinition makeQueryDefinition( new SegmentGeneratorFrameProcessorFactory( dataSchema, columnMappings, - tuningConfig, - dimensionToAggregatoryFactoryMap + tuningConfig ) ) ); @@ -1878,8 +1869,7 @@ private static DataSchema makeDataSchemaForIngestion( RowSignature querySignature, ClusterBy queryClusterBy, ColumnMappings columnMappings, - ObjectMapper jsonMapper, - Map dimensionToAggregatorFactoryMap + ObjectMapper jsonMapper ) { final DataSourceMSQDestination destination = (DataSourceMSQDestination) querySpec.getDestination(); @@ -1892,8 +1882,7 @@ private static DataSchema makeDataSchemaForIngestion( destination.getSegmentSortOrder(), columnMappings, isRollupQuery, - querySpec.getQuery(), - dimensionToAggregatorFactoryMap + querySpec.getQuery() ); return new DataSchema( @@ -2071,9 +2060,7 @@ private static Pair, List> makeDimensio final List segmentSortOrder, final ColumnMappings columnMappings, final boolean isRollupQuery, - final Query query, - final Map dimensionToAggregatoryFactoryMap - + final Query query ) { // Log a warning unconditionally if arrayIngestMode is MVD, since the behaviour is incorrect, and is subject to @@ -2117,22 +2104,18 @@ private static Pair, List> makeDimensio Map outputColumnAggregatorFactories = new HashMap<>(); - // Populate aggregators from the native query when doing an ingest in rollup mode. - if (query instanceof GroupByQuery) { + if (isRollupQuery) { + // Populate aggregators from the native query when doing an ingest in rollup mode. for (AggregatorFactory aggregatorFactory : ((GroupByQuery) query).getAggregatorSpecs()) { for (final int outputColumn : columnMappings.getOutputColumnsForQueryColumn(aggregatorFactory.getName())) { final String outputColumnName = columnMappings.getOutputColumnName(outputColumn); if (outputColumnAggregatorFactories.containsKey(outputColumnName)) { throw new ISE("There can only be one aggregation for column [%s].", outputColumn); } else { - if (isRollupQuery) { - outputColumnAggregatorFactories.put( - outputColumnName, - aggregatorFactory.withName(outputColumnName).getCombiningFactory() - ); - } else { - dimensionToAggregatoryFactoryMap.put(outputColumnName, aggregatorFactory); - } + outputColumnAggregatorFactories.put( + outputColumnName, + aggregatorFactory.withName(outputColumnName).getCombiningFactory() + ); } } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java index 84ffad31f541..529af5361e5e 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java @@ -55,6 +55,7 @@ import org.apache.druid.query.expression.TimestampFloorExprMacro; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.segment.VirtualColumn; @@ -111,10 +112,6 @@ public MSQCompactionRunner(@JacksonInject ObjectMapper jsonMapper, @JacksonInjec {@link ClientCompactionRunnerInfo#msqEngineSupportsCompactionConfig} The following configs aren't supported: *
    - *
  • finalizeAggregations set to false in context.
  • - * - *
  • taskAssignment set to auto in context.
  • - * *
  • partitionsSpec of type HashedParititionsSpec.
  • * *
  • maxTotalRows in DynamicPartitionsSpec.
  • @@ -444,6 +441,9 @@ private Map createMSQTaskContext(CompactionTask compactionTask, ); } context.put(MultiStageQueryContext.CTX_SEGMENT_LOAD_WAIT, true); + // Similar to compaction using the native engine, don't finalize aggregations. + context.put(MultiStageQueryContext.CTX_FINALIZE_AGGREGATIONS, false); + context.put(GroupByQueryConfig.CTX_KEY_ENABLE_MULTI_VALUE_UNNESTING, false); return context; } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessorFactory.java index 941a6cab670c..e925e1a1c028 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessorFactory.java @@ -49,7 +49,6 @@ import org.apache.druid.msq.kernel.ProcessorsAndChannels; import org.apache.druid.msq.kernel.StageDefinition; import org.apache.druid.msq.kernel.StagePartition; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.data.CompressionFactory; import org.apache.druid.segment.data.CompressionStrategy; @@ -72,7 +71,6 @@ import java.io.File; import java.util.HashSet; import java.util.List; -import java.util.Map; import java.util.Objects; import java.util.Set; import java.util.function.Consumer; @@ -84,21 +82,17 @@ public class SegmentGeneratorFrameProcessorFactory private final DataSchema dataSchema; private final ColumnMappings columnMappings; private final MSQTuningConfig tuningConfig; - private final Map dimensionToAggregatorFactoryMap; @JsonCreator public SegmentGeneratorFrameProcessorFactory( @JsonProperty("dataSchema") final DataSchema dataSchema, @JsonProperty("columnMappings") final ColumnMappings columnMappings, - @JsonProperty("tuningConfig") final MSQTuningConfig tuningConfig, - @JsonProperty("dimensionToAggregatorFactoryMap") - final Map dimensionToAggregatorFactoryMap + @JsonProperty("tuningConfig") final MSQTuningConfig tuningConfig ) { this.dataSchema = Preconditions.checkNotNull(dataSchema, "dataSchema"); this.columnMappings = Preconditions.checkNotNull(columnMappings, "columnMappings"); this.tuningConfig = Preconditions.checkNotNull(tuningConfig, "tuningConfig"); - this.dimensionToAggregatorFactoryMap = dimensionToAggregatorFactoryMap; } @JsonProperty @@ -119,12 +113,6 @@ public MSQTuningConfig getTuningConfig() return tuningConfig; } - @JsonProperty - public Map getDimensionToAggregatorFactoryMap() - { - return dimensionToAggregatorFactoryMap; - } - @Override public ProcessorsAndChannels> makeProcessors( StageDefinition stageDefinition, diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java index 21a98456ef65..b616a3f11898 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java @@ -102,7 +102,7 @@ public class MultiStageQueryContext public static final String CTX_MAX_NUM_TASKS = "maxNumTasks"; @VisibleForTesting - public static final int DEFAULT_MAX_NUM_TASKS = 2; + static final int DEFAULT_MAX_NUM_TASKS = 2; public static final String CTX_TASK_ASSIGNMENT_STRATEGY = "taskAssignment"; private static final String DEFAULT_TASK_ASSIGNMENT_STRATEGY = WorkerAssignmentStrategy.MAX.toString(); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java index f00c37738fd9..db92dd9b7d83 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java @@ -31,7 +31,6 @@ import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.error.DruidException; import org.apache.druid.error.DruidExceptionMatcher; -import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; @@ -1966,12 +1965,10 @@ private CompactionState expectedCompactionState( return new CompactionState( partitionsSpec, dimensionsSpec, - Collections.emptyMap(), metricsSpec, null, indexSpec.asMap(objectMapper), - granularitySpec.asMap(objectMapper), - CompactionEngine.NATIVE + granularitySpec.asMap(objectMapper) ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java index 30252434dcaa..5a17c4379f18 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java @@ -30,7 +30,6 @@ import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.IngestionState; import org.apache.druid.indexer.report.IngestionStatsAndErrors; import org.apache.druid.indexer.report.IngestionStatsAndErrorsTaskReport; @@ -636,12 +635,10 @@ public static Function, Set> addCompactionStateToS return CompactionState.addCompactionStateToSegments( tuningConfig.getPartitionsSpec(), dimensionsSpec, - Collections.emptyMap(), metricsSpec, transformSpec, tuningConfig.getIndexSpec().asMap(toolbox.getJsonMapper()), - granularitySpec.asMap(toolbox.getJsonMapper()), - CompactionEngine.NATIVE + granularitySpec.asMap(toolbox.getJsonMapper()) ); } else { return Function.identity(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java index 08e7ba809bad..077cb6c86c43 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java @@ -32,7 +32,6 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; @@ -187,7 +186,6 @@ public void testRunParallelWithDynamicPartitioningMatchCompactionState() throws CompactionState expectedState = new CompactionState( new DynamicPartitionsSpec(null, Long.MAX_VALUE), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), - Collections.emptyMap(), ImmutableList.of(expectedLongSumMetric), null, compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), @@ -201,8 +199,7 @@ public void testRunParallelWithDynamicPartitioningMatchCompactionState() throws ) ), Map.class - ), - CompactionEngine.NATIVE + ) ); Assert.assertEquals("Compaction state for " + segment.getId(), expectedState, segment.getLastCompactionState()); } @@ -241,7 +238,6 @@ public void testRunParallelWithHashPartitioningMatchCompactionState() throws Exc CompactionState expectedState = new CompactionState( new HashedPartitionsSpec(null, 3, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), - Collections.emptyMap(), ImmutableList.of(expectedLongSumMetric), null, compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), @@ -255,8 +251,7 @@ public void testRunParallelWithHashPartitioningMatchCompactionState() throws Exc ) ), Map.class - ), - CompactionEngine.NATIVE + ) ); Assert.assertEquals("Compaction state for " + segment.getId(), expectedState, segment.getLastCompactionState()); } @@ -310,7 +305,6 @@ public void testRunParallelWithRangePartitioning() throws Exception CompactionState expectedState = new CompactionState( new SingleDimensionPartitionsSpec(7, null, "dim", false), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), - Collections.emptyMap(), ImmutableList.of(expectedLongSumMetric), null, compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), @@ -324,8 +318,7 @@ public void testRunParallelWithRangePartitioning() throws Exception ) ), Map.class - ), - CompactionEngine.NATIVE + ) ); Assert.assertEquals("Compaction state for " + segment.getId(), expectedState, segment.getLastCompactionState()); } @@ -374,7 +367,6 @@ public void testRunParallelWithRangePartitioningAndNoUpfrontSegmentFetching() th CompactionState expectedState = new CompactionState( new SingleDimensionPartitionsSpec(7, null, "dim", false), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), - Collections.emptyMap(), ImmutableList.of(expectedLongSumMetric), null, compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), @@ -390,8 +382,7 @@ public void testRunParallelWithRangePartitioningAndNoUpfrontSegmentFetching() th ) ), Map.class - ), - CompactionEngine.NATIVE + ) ); Assert.assertEquals("Compaction state for " + segment.getId(), expectedState, segment.getLastCompactionState()); } @@ -433,7 +424,6 @@ public void testRunParallelWithMultiDimensionRangePartitioning() throws Exceptio CompactionState expectedState = new CompactionState( new DimensionRangePartitionsSpec(7, null, Arrays.asList("dim1", "dim2"), false), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), - Collections.emptyMap(), ImmutableList.of(expectedLongSumMetric), null, compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), @@ -447,8 +437,7 @@ public void testRunParallelWithMultiDimensionRangePartitioning() throws Exceptio ) ), Map.class - ), - CompactionEngine.NATIVE + ) ); Assert.assertEquals("Compaction state for " + segment.getId(), expectedState, segment.getLastCompactionState()); } @@ -487,7 +476,6 @@ public void testRunParallelWithRangePartitioningWithSingleTask() throws Exceptio CompactionState expectedState = new CompactionState( new SingleDimensionPartitionsSpec(7, null, "dim", false), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), - Collections.emptyMap(), ImmutableList.of(expectedLongSumMetric), null, compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), @@ -501,8 +489,7 @@ public void testRunParallelWithRangePartitioningWithSingleTask() throws Exceptio ) ), Map.class - ), - CompactionEngine.NATIVE + ) ); Assert.assertEquals("Compaction state for " + segment.getId(), expectedState, segment.getLastCompactionState()); } @@ -544,7 +531,7 @@ public void testRunParallelWithMultiDimensionRangePartitioningWithSingleTask() t CompactionState expectedState = new CompactionState( new DimensionRangePartitionsSpec(7, null, Arrays.asList("dim1", "dim2"), false), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), - Collections.emptyMap(), ImmutableList.of(expectedLongSumMetric), + ImmutableList.of(expectedLongSumMetric), null, compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), getObjectMapper().readValue( @@ -557,8 +544,7 @@ public void testRunParallelWithMultiDimensionRangePartitioningWithSingleTask() t ) ), Map.class - ), - CompactionEngine.NATIVE + ) ); Assert.assertEquals("Compaction state for " + segment.getId(), expectedState, segment.getLastCompactionState()); } @@ -632,7 +618,6 @@ public void testRunCompactionWithFilterShouldStoreInState() throws Exception CompactionState expectedState = new CompactionState( new DynamicPartitionsSpec(null, Long.MAX_VALUE), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), - Collections.emptyMap(), ImmutableList.of(expectedLongSumMetric), getObjectMapper().readValue( getObjectMapper().writeValueAsString(compactionTask.getTransformSpec()), @@ -649,8 +634,7 @@ public void testRunCompactionWithFilterShouldStoreInState() throws Exception ) ), Map.class - ), - CompactionEngine.NATIVE + ) ); Assert.assertEquals("Compaction state for " + segment.getId(), expectedState, segment.getLastCompactionState()); } @@ -698,7 +682,6 @@ public void testRunCompactionWithNewMetricsShouldStoreInState() throws Exception CompactionState expectedState = new CompactionState( new DynamicPartitionsSpec(null, Long.MAX_VALUE), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), - Collections.emptyMap(), ImmutableList.of(expectedCountMetric, expectedLongSumMetric), getObjectMapper().readValue( getObjectMapper().writeValueAsString(compactionTask.getTransformSpec()), @@ -715,8 +698,7 @@ public void testRunCompactionWithNewMetricsShouldStoreInState() throws Exception ) ), Map.class - ), - CompactionEngine.NATIVE + ) ); Assert.assertEquals("Compaction state for " + segment.getId(), expectedState, segment.getLastCompactionState()); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index 4e39cde95596..471932ccfe2e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -38,7 +38,6 @@ import org.apache.druid.data.input.impl.ParseSpec; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; @@ -246,7 +245,6 @@ public static CompactionState getDefaultCompactionState( return new CompactionState( new DynamicPartitionsSpec(5000000, Long.MAX_VALUE), expectedDims, - Collections.emptyMap(), ImmutableList.of(expectedMetric), null, IndexSpec.DEFAULT.asMap(mapper), @@ -260,8 +258,7 @@ public static CompactionState getDefaultCompactionState( ) ), Map.class - ), - CompactionEngine.NATIVE + ) ); } @@ -424,7 +421,6 @@ public void testRunWithHashPartitioning() throws Exception CompactionState expectedState = new CompactionState( new HashedPartitionsSpec(null, 3, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), - Collections.emptyMap(), ImmutableList.of(expectedLongSumMetric), null, compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), @@ -438,8 +434,7 @@ public void testRunWithHashPartitioning() throws Exception ) ), Map.class - ), - CompactionEngine.NATIVE + ) ); Assert.assertEquals(expectedState, segments.get(segmentIdx).getLastCompactionState()); Assert.assertSame(HashBasedNumberedShardSpec.class, segments.get(segmentIdx).getShardSpec().getClass()); @@ -853,7 +848,6 @@ public void testCompactionWithFilterInTransformSpec() throws Exception CompactionState expectedCompactionState = new CompactionState( new DynamicPartitionsSpec(5000000, Long.MAX_VALUE), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), - Collections.emptyMap(), ImmutableList.of(expectedLongSumMetric), getObjectMapper().readValue(getObjectMapper().writeValueAsString(compactionTask.getTransformSpec()), Map.class), IndexSpec.DEFAULT.asMap(mapper), @@ -867,8 +861,7 @@ public void testCompactionWithFilterInTransformSpec() throws Exception ) ), Map.class - ), - CompactionEngine.NATIVE + ) ); Assert.assertEquals( expectedCompactionState, @@ -922,7 +915,6 @@ public void testCompactionWithNewMetricInMetricsSpec() throws Exception CompactionState expectedCompactionState = new CompactionState( new DynamicPartitionsSpec(5000000, Long.MAX_VALUE), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), - Collections.emptyMap(), ImmutableList.of(expectedCountMetric, expectedLongSumMetric), getObjectMapper().readValue(getObjectMapper().writeValueAsString(compactionTask.getTransformSpec()), Map.class), IndexSpec.DEFAULT.asMap(mapper), @@ -936,8 +928,7 @@ public void testCompactionWithNewMetricInMetricsSpec() throws Exception ) ), Map.class - ), - CompactionEngine.NATIVE + ) ); Assert.assertEquals( expectedCompactionState, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index c405a7d0a33a..1ccc00017357 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -80,7 +80,6 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.NonnullPair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; diff --git a/processing/src/main/java/org/apache/druid/timeline/CompactionState.java b/processing/src/main/java/org/apache/druid/timeline/CompactionState.java index ecc4c279b922..2c6e0d96c397 100644 --- a/processing/src/main/java/org/apache/druid/timeline/CompactionState.java +++ b/processing/src/main/java/org/apache/druid/timeline/CompactionState.java @@ -21,13 +21,9 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.common.config.Configs; import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.partitions.PartitionsSpec; -import org.apache.druid.query.aggregation.AggregatorFactory; -import javax.annotation.Nullable; import java.util.List; import java.util.Map; import java.util.Objects; @@ -40,7 +36,7 @@ * The compaction task is a task that reads Druid segments and overwrites them with new ones. Since this task always * reads segments in the same order, the same task spec will always create the same set of segments * (not same segment ID, but same content). - *

    + * * Note that this class doesn't include all fields in the compaction task spec. Only the configurations that can * affect the content of segment should be included. * @@ -62,20 +58,15 @@ public class CompactionState // org.apache.druid.query.aggregation.AggregatorFactory cannot be used here because it's in the 'processing' module which // has a dependency on the 'core' module where this class is. private final List metricsSpec; - private final CompactionEngine engine; - private final Map dimensionToAggregatoryFactoryMap; @JsonCreator public CompactionState( @JsonProperty("partitionsSpec") PartitionsSpec partitionsSpec, @JsonProperty("dimensionsSpec") DimensionsSpec dimensionsSpec, - @Nullable @JsonProperty("dimensionToAggregatoryFactoryMap") - Map dimensionToAggregatoryFactoryMap, @JsonProperty("metricsSpec") List metricsSpec, @JsonProperty("transformSpec") Map transformSpec, @JsonProperty("indexSpec") Map indexSpec, - @JsonProperty("granularitySpec") Map granularitySpec, - @Nullable @JsonProperty("engine") CompactionEngine engine + @JsonProperty("granularitySpec") Map granularitySpec ) { this.partitionsSpec = partitionsSpec; @@ -84,8 +75,6 @@ public CompactionState( this.transformSpec = transformSpec; this.indexSpec = indexSpec; this.granularitySpec = granularitySpec; - this.engine = Configs.valueOrDefault(engine, CompactionEngine.NATIVE); - this.dimensionToAggregatoryFactoryMap = dimensionToAggregatoryFactoryMap; } @JsonProperty @@ -124,20 +113,6 @@ public Map getGranularitySpec() return granularitySpec; } - @JsonProperty - @Nullable - public CompactionEngine getEngine() - { - return engine; - } - - @JsonProperty - @Nullable - public Map getDimensionToAggregatoryFactoryMap() - { - return dimensionToAggregatoryFactoryMap; - } - @Override public boolean equals(Object o) { @@ -159,14 +134,7 @@ public boolean equals(Object o) @Override public int hashCode() { - return Objects.hash( - partitionsSpec, - dimensionsSpec, - transformSpec, - indexSpec, - granularitySpec, - metricsSpec - ); + return Objects.hash(partitionsSpec, dimensionsSpec, transformSpec, indexSpec, granularitySpec, metricsSpec); } @Override @@ -185,23 +153,19 @@ public String toString() public static Function, Set> addCompactionStateToSegments( PartitionsSpec partitionsSpec, DimensionsSpec dimensionsSpec, - Map dimensionToAggregatoryFactoryMap, List metricsSpec, Map transformSpec, Map indexSpec, - Map granularitySpec, - CompactionEngine engine + Map granularitySpec ) { CompactionState compactionState = new CompactionState( partitionsSpec, dimensionsSpec, - dimensionToAggregatoryFactoryMap, metricsSpec, transformSpec, indexSpec, - granularitySpec, - engine + granularitySpec ); return segments -> segments @@ -209,4 +173,5 @@ public static Function, Set> addCompactionStateToS .map(s -> s.withLastCompactionState(compactionState)) .collect(Collectors.toSet()); } + } diff --git a/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java b/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java index b232930a74c1..3f0667b870c9 100644 --- a/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java +++ b/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java @@ -26,7 +26,6 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.RangeSet; import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.jackson.DefaultObjectMapper; @@ -130,12 +129,10 @@ public void testV1Serialization() throws Exception new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "bar", "foo")) ), - Collections.emptyMap(), ImmutableList.of(ImmutableMap.of("type", "count", "name", "count")), ImmutableMap.of("filter", ImmutableMap.of("type", "selector", "dimension", "dim1", "value", "foo")), ImmutableMap.of(), - ImmutableMap.of(), - CompactionEngine.NATIVE + ImmutableMap.of() ), TEST_VERSION, 1 @@ -156,7 +153,7 @@ public void testV1Serialization() throws Exception Assert.assertEquals(ImmutableMap.of("type", "numbered", "partitionNum", 3, "partitions", 0), objectMap.get("shardSpec")); Assert.assertEquals(TEST_VERSION, objectMap.get("binaryVersion")); Assert.assertEquals(1, objectMap.get("size")); - Assert.assertEquals(8, ((Map) objectMap.get("lastCompactionState")).size()); + Assert.assertEquals(6, ((Map) objectMap.get("lastCompactionState")).size()); DataSegment deserializedSegment = MAPPER.readValue(MAPPER.writeValueAsString(segment), DataSegment.class); @@ -197,12 +194,10 @@ public void testDeserializationDataSegmentLastCompactionStateWithNullSpecs() thr new CompactionState( new HashedPartitionsSpec(100000, null, ImmutableList.of("dim1")), null, - Collections.emptyMap(), null, null, ImmutableMap.of(), - ImmutableMap.of(), - CompactionEngine.NATIVE + ImmutableMap.of() ), TEST_VERSION, 1 @@ -350,12 +345,10 @@ public void testWithLastCompactionState() final CompactionState compactionState = new CompactionState( new DynamicPartitionsSpec(null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar", "foo"))), - Collections.emptyMap(), ImmutableList.of(ImmutableMap.of("type", "count", "name", "count")), ImmutableMap.of("filter", ImmutableMap.of("type", "selector", "dimension", "dim1", "value", "foo")), Collections.singletonMap("test", "map"), - Collections.singletonMap("test2", "map2"), - CompactionEngine.NATIVE + Collections.singletonMap("test2", "map2") ); final DataSegment segment1 = DataSegment.builder() .dataSource("foo") @@ -394,24 +387,20 @@ public void testAnnotateWithLastCompactionState() final CompactionState compactionState = new CompactionState( dynamicPartitionsSpec, dimensionsSpec, - Collections.emptyMap(), metricsSpec, transformSpec, indexSpec, - granularitySpec, - CompactionEngine.NATIVE + granularitySpec ); final Function, Set> addCompactionStateFunction = CompactionState.addCompactionStateToSegments( dynamicPartitionsSpec, dimensionsSpec, - Collections.emptyMap(), metricsSpec, transformSpec, indexSpec, - granularitySpec, - CompactionEngine.NATIVE + granularitySpec ); final DataSegment segment1 = DataSegment.builder() diff --git a/server/src/main/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfo.java b/server/src/main/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfo.java index 15079a6ba749..1171695ad77d 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfo.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfo.java @@ -101,9 +101,12 @@ public int hashCode() return Objects.hash(type); } - public static ValidationResult validateCompactionConfig(DataSourceCompactionConfig newConfig) + public static ValidationResult validateCompactionConfig( + DataSourceCompactionConfig newConfig, + CompactionEngine defaultCompactionEngine + ) { - CompactionEngine compactionEngine = newConfig.getEngine(); + CompactionEngine compactionEngine = newConfig.getEngine() == null ? defaultCompactionEngine : newConfig.getEngine(); if (compactionEngine == CompactionEngine.NATIVE) { return new ValidationResult(true, null); } else { diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java index 155a2834d882..064d89c2c4eb 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import org.apache.druid.error.InvalidInput; import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.query.aggregation.AggregatorFactory; import org.joda.time.Period; @@ -228,38 +227,4 @@ public int hashCode() result = 31 * result + Arrays.hashCode(metricsSpec); return result; } - - public static DataSourceCompactionConfig from( - DataSourceCompactionConfig newConfig, - CompactionEngine defaultCompactionEngine - ) - { - CompactionEngine newCompactionEngine = newConfig.getEngine(); - if (newCompactionEngine == null) { - newCompactionEngine = defaultCompactionEngine; - newConfig = new DataSourceCompactionConfig( - newConfig.getDataSource(), - newConfig.getTaskPriority(), - newConfig.getInputSegmentSizeBytes(), - newConfig.getMaxRowsPerSegment(), - newConfig.getSkipOffsetFromLatest(), - newConfig.getTuningConfig(), - newConfig.getGranularitySpec(), - newConfig.getDimensionsSpec(), - newConfig.getMetricsSpec(), - newConfig.getTransformSpec(), - newConfig.getIoConfig(), - newCompactionEngine, - newConfig.getTaskContext() - ); - } - - ClientCompactionRunnerInfo.ValidationResult validationResult = ClientCompactionRunnerInfo.validateCompactionConfig( - newConfig - ); - if (!validationResult.isValid()) { - throw InvalidInput.exception("Compaction config not supported. Reason[%s].", validationResult.getReason()); - } - return newConfig; - } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java index f67585864820..862f2e7c5b4c 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java @@ -26,9 +26,7 @@ import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; import org.apache.druid.client.indexing.ClientCompactionTaskTransformSpec; import org.apache.druid.common.config.Configs; -import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.java.util.common.StringUtils; @@ -43,9 +41,7 @@ import javax.annotation.Nullable; import java.util.Arrays; import java.util.List; -import java.util.Map; import java.util.function.Function; -import java.util.stream.Collectors; /** * Represents the status of compaction for a given list of candidate segments. @@ -265,10 +261,7 @@ private CompactionStatus segmentGranularityIsUpToDate() private CompactionStatus rollupIsUpToDate() { - // MSQ considers a query as rollup only if finalizeAggregations=false. Compaction can have that set to true. - if (configuredGranularitySpec == null || (configuredGranularitySpec.isRollup() != null - && configuredGranularitySpec.isRollup() - && lastCompactionState.getEngine() == CompactionEngine.MSQ)) { + if (configuredGranularitySpec == null) { return COMPLETE; } else { return CompactionStatus.completeIfEqual( @@ -298,23 +291,10 @@ private CompactionStatus dimensionsSpecIsUpToDate() return COMPLETE; } else { final DimensionsSpec existingDimensionsSpec = lastCompactionState.getDimensionsSpec(); - final Map dimensionToAggregatorFactoryMap = - lastCompactionState.getDimensionToAggregatoryFactoryMap(); - // Remove dimensions which were converted from metrics due to finalizeAggregations=true - final List originalDimensions = - existingDimensionsSpec == null - ? null - : dimensionToAggregatorFactoryMap == null - ? existingDimensionsSpec.getDimensions() - : existingDimensionsSpec.getDimensions() - .stream() - .filter(dimensionSchema -> !dimensionToAggregatorFactoryMap.containsKey( - dimensionSchema.getName())) - .collect(Collectors.toList()); return CompactionStatus.completeIfEqual( "dimensionsSpec", compactionConfig.getDimensionsSpec().getDimensions(), - originalDimensions + existingDimensionsSpec == null ? null : existingDimensionsSpec.getDimensions() ); } } @@ -327,18 +307,9 @@ private CompactionStatus metricsSpecIsUpToDate() } final List metricSpecList = lastCompactionState.getMetricsSpec(); - final AggregatorFactory[] existingMetricsSpec; - if (CollectionUtils.isNullOrEmpty(metricSpecList)) { - // Use the dimensionToAggregatorFactoryMap to check metrics that were converted to dimensions due to - // finalizeAggregations=true - final Map dimensionToAggregatorFactoryMap = - lastCompactionState.getDimensionToAggregatoryFactoryMap(); - existingMetricsSpec = dimensionToAggregatorFactoryMap == null - ? null - : dimensionToAggregatorFactoryMap.values().toArray(new AggregatorFactory[0]); - } else { - existingMetricsSpec = objectMapper.convertValue(metricSpecList, AggregatorFactory[].class); - } + final AggregatorFactory[] existingMetricsSpec + = CollectionUtils.isNullOrEmpty(metricSpecList) + ? null : objectMapper.convertValue(metricSpecList, AggregatorFactory[].class); if (existingMetricsSpec == null || !Arrays.deepEquals(configuredMetricsSpec, existingMetricsSpec)) { return CompactionStatus.configChanged( diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index c14b57c239de..698a9bdf4a9f 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java @@ -236,11 +236,11 @@ private boolean cancelTaskIfGranularityChanged( Granularity configuredSegmentGranularity = dataSourceCompactionConfig.getGranularitySpec() .getSegmentGranularity(); Granularity taskSegmentGranularity = compactionTaskQuery.getGranularitySpec().getSegmentGranularity(); - if (configuredSegmentGranularity != null && configuredSegmentGranularity.equals(taskSegmentGranularity)) { + if (configuredSegmentGranularity == null || configuredSegmentGranularity.equals(taskSegmentGranularity)) { return false; } - LOG.info( + LOG.warn( "Cancelling task [%s] as task segmentGranularity is [%s] but compaction config segmentGranularity is [%s]", compactionTaskQuery.getId(), taskSegmentGranularity, configuredSegmentGranularity ); diff --git a/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java index e29aee26f2b4..6b4ba22f1a72 100644 --- a/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java +++ b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java @@ -27,8 +27,10 @@ import org.apache.druid.audit.AuditInfo; import org.apache.druid.audit.AuditManager; import org.apache.druid.common.config.ConfigManager.SetResult; +import org.apache.druid.error.InvalidInput; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.server.coordinator.ClientCompactionRunnerInfo; import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; @@ -119,8 +121,12 @@ public Response addOrUpdateCompactionConfig( .getCompactionConfigs() .stream() .collect(Collectors.toMap(DataSourceCompactionConfig::getDataSource, Function.identity())); - DataSourceCompactionConfig updatedConfig = DataSourceCompactionConfig.from(newConfig, current.getEngine()); - newConfigs.put(updatedConfig.getDataSource(), updatedConfig); + ClientCompactionRunnerInfo.ValidationResult validationResult = + ClientCompactionRunnerInfo.validateCompactionConfig(newConfig, current.getEngine()); + if (!validationResult.isValid()) { + throw InvalidInput.exception("Compaction config not supported. Reason[%s].", validationResult.getReason()); + } + newConfigs.put(newConfig.getDataSource(), newConfig); newCompactionConfig = CoordinatorCompactionConfig.from(current, ImmutableList.copyOf(newConfigs.values())); return newCompactionConfig; diff --git a/server/src/test/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfoTest.java b/server/src/test/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfoTest.java index 53787bc7e2cc..89d7bfc690b0 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfoTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfoTest.java @@ -50,7 +50,7 @@ public void testHashedPartitionsSpecs() { assertFalse( ClientCompactionRunnerInfo.validateCompactionConfig( - createCompactionConfig(new HashedPartitionsSpec(100, null, null), Collections.emptyMap()) + createCompactionConfig(new HashedPartitionsSpec(100, null, null), Collections.emptyMap()), CompactionEngine.NATIVE ).isValid() ); } @@ -60,7 +60,7 @@ public void testrInvalidDynamicPartitionsSpecs() { assertFalse( ClientCompactionRunnerInfo.validateCompactionConfig( - createCompactionConfig(new DynamicPartitionsSpec(100, 100L), Collections.emptyMap()) + createCompactionConfig(new DynamicPartitionsSpec(100, 100L), Collections.emptyMap()), CompactionEngine.NATIVE ).isValid() ); } @@ -69,7 +69,7 @@ public void testrInvalidDynamicPartitionsSpecs() public void testDynamicPartitionsSpecs() { assertTrue(ClientCompactionRunnerInfo.validateCompactionConfig( - createCompactionConfig(new DynamicPartitionsSpec(100, null), Collections.emptyMap()) + createCompactionConfig(new DynamicPartitionsSpec(100, null), Collections.emptyMap()), CompactionEngine.NATIVE ).isValid()); } @@ -80,7 +80,7 @@ public void testDimensionRangePartitionsSpecs() createCompactionConfig( new DimensionRangePartitionsSpec(100, null, ImmutableList.of("partitionDim"), false), Collections.emptyMap() - ) + ), CompactionEngine.NATIVE ).isValid()); } @@ -91,7 +91,7 @@ public void testWithFinalizeAggregationsFalse() new DynamicPartitionsSpec(3, null), ImmutableMap.of(ClientCompactionRunnerInfo.MSQContext.CTX_FINALIZE_AGGREGATIONS, false) ); - Assert.assertFalse(ClientCompactionRunnerInfo.validateCompactionConfig(compactionConfig).isValid()); + Assert.assertFalse(ClientCompactionRunnerInfo.validateCompactionConfig(compactionConfig, CompactionEngine.NATIVE).isValid()); } private static DataSourceCompactionConfig createCompactionConfig( diff --git a/server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicyTest.java b/server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicyTest.java index 19a53810e513..31e269f50f41 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicyTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicyTest.java @@ -29,7 +29,6 @@ import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; @@ -730,31 +729,13 @@ public void testIteratorReturnsNothingAsSegmentsWasCompactedAndHaveSameSegmentGr Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), new Period("P1D"), null, - new CompactionState( - partitionsSpec, - null, - Collections.emptyMap(), - null, - null, - indexSpec, - null, - CompactionEngine.NATIVE - ) + new CompactionState(partitionsSpec, null, null, null, indexSpec, null) ), new SegmentGenerateSpec( Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), new Period("P1D"), null, - new CompactionState( - partitionsSpec, - null, - Collections.emptyMap(), - null, - null, - indexSpec, - null, - CompactionEngine.NATIVE - ) + new CompactionState(partitionsSpec, null, null, null, indexSpec, null) ) ); @@ -781,31 +762,13 @@ public void testIteratorReturnsNothingAsSegmentsWasCompactedAndHaveSameSegmentGr Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), new Period("P1D"), null, - new CompactionState( - partitionsSpec, - null, - Collections.emptyMap(), - null, - null, - indexSpec, - ImmutableMap.of("segmentGranularity", "day"), - CompactionEngine.NATIVE - ) + new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of("segmentGranularity", "day")) ), new SegmentGenerateSpec( Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), new Period("P1D"), null, - new CompactionState( - partitionsSpec, - null, - Collections.emptyMap(), - null, - null, - indexSpec, - ImmutableMap.of("segmentGranularity", "day"), - CompactionEngine.NATIVE - ) + new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of("segmentGranularity", "day")) ) ); @@ -832,31 +795,13 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentSeg Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), new Period("P1D"), null, - new CompactionState( - partitionsSpec, - null, - Collections.emptyMap(), - null, - null, - indexSpec, - null, - CompactionEngine.NATIVE - ) + new CompactionState(partitionsSpec, null, null, null, indexSpec, null) ), new SegmentGenerateSpec( Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), new Period("P1D"), null, - new CompactionState( - partitionsSpec, - null, - Collections.emptyMap(), - null, - null, - indexSpec, - null, - CompactionEngine.NATIVE - ) + new CompactionState(partitionsSpec, null, null, null, indexSpec, null) ) ); @@ -893,31 +838,13 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentSeg Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), new Period("P1D"), null, - new CompactionState( - partitionsSpec, - null, - Collections.emptyMap(), - null, - null, - indexSpec, - ImmutableMap.of("segmentGranularity", "day"), - CompactionEngine.NATIVE - ) + new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of("segmentGranularity", "day")) ), new SegmentGenerateSpec( Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), new Period("P1D"), null, - new CompactionState( - partitionsSpec, - null, - Collections.emptyMap(), - null, - null, - indexSpec, - ImmutableMap.of("segmentGranularity", "day"), - CompactionEngine.NATIVE - ) + new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of("segmentGranularity", "day")) ) ); @@ -954,16 +881,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentTim Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), new Period("P1D"), null, - new CompactionState( - partitionsSpec, - null, - Collections.emptyMap(), - null, - null, - indexSpec, - null, - CompactionEngine.NATIVE - ) + new CompactionState(partitionsSpec, null, null, null, indexSpec, null) ) ); @@ -1015,16 +933,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentOri Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), new Period("P1D"), null, - new CompactionState( - partitionsSpec, - null, - Collections.emptyMap(), - null, - null, - indexSpec, - null, - CompactionEngine.NATIVE - ) + new CompactionState(partitionsSpec, null, null, null, indexSpec, null) ) ); @@ -1078,46 +987,19 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentRol Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), new Period("P1D"), null, - new CompactionState( - partitionsSpec, - null, - Collections.emptyMap(), - null, - null, - indexSpec, - ImmutableMap.of("rollup", "false"), - CompactionEngine.NATIVE - ) + new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of("rollup", "false")) ), new SegmentGenerateSpec( Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), new Period("P1D"), null, - new CompactionState( - partitionsSpec, - null, - Collections.emptyMap(), - null, - null, - indexSpec, - ImmutableMap.of("rollup", "true"), - CompactionEngine.NATIVE - ) + new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of("rollup", "true")) ), new SegmentGenerateSpec( Intervals.of("2017-10-03T00:00:00/2017-10-04T00:00:00"), new Period("P1D"), null, - new CompactionState( - partitionsSpec, - null, - Collections.emptyMap(), - null, - null, - indexSpec, - ImmutableMap.of(), - CompactionEngine.NATIVE - ) + new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of()) ) ); @@ -1165,46 +1047,19 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentQue Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), new Period("P1D"), null, - new CompactionState( - partitionsSpec, - null, - Collections.emptyMap(), - null, - null, - indexSpec, - ImmutableMap.of("queryGranularity", "day"), - CompactionEngine.NATIVE - ) + new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of("queryGranularity", "day")) ), new SegmentGenerateSpec( Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), new Period("P1D"), null, - new CompactionState( - partitionsSpec, - null, - Collections.emptyMap(), - null, - null, - indexSpec, - ImmutableMap.of("queryGranularity", "minute"), - CompactionEngine.NATIVE - ) + new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of("queryGranularity", "minute")) ), new SegmentGenerateSpec( Intervals.of("2017-10-03T00:00:00/2017-10-04T00:00:00"), new Period("P1D"), null, - new CompactionState( - partitionsSpec, - null, - Collections.emptyMap(), - null, - null, - indexSpec, - ImmutableMap.of(), - CompactionEngine.NATIVE - ) + new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of()) ) ); @@ -1253,61 +1108,25 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentDim Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), new Period("P1D"), null, - new CompactionState( - partitionsSpec, - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar", "foo"))), - Collections.emptyMap(), - null, - null, - indexSpec, - null, - CompactionEngine.NATIVE - ) + new CompactionState(partitionsSpec, new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar", "foo"))), null, null, indexSpec, null) ), new SegmentGenerateSpec( Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), new Period("P1D"), null, - new CompactionState( - partitionsSpec, - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("foo"))), - Collections.emptyMap(), - null, - null, - indexSpec, - null, - CompactionEngine.NATIVE - ) + new CompactionState(partitionsSpec, new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("foo"))), null, null, indexSpec, null) ), new SegmentGenerateSpec( Intervals.of("2017-10-03T00:00:00/2017-10-04T00:00:00"), new Period("P1D"), null, - new CompactionState( - partitionsSpec, - DimensionsSpec.EMPTY, - Collections.emptyMap(), - null, - null, - indexSpec, - null, - CompactionEngine.NATIVE - ) + new CompactionState(partitionsSpec, DimensionsSpec.EMPTY, null, null, indexSpec, null) ), new SegmentGenerateSpec( Intervals.of("2017-10-04T00:00:00/2017-10-05T00:00:00"), new Period("P1D"), null, - new CompactionState( - partitionsSpec, - null, - Collections.emptyMap(), - null, - null, - indexSpec, - null, - CompactionEngine.NATIVE - ) + new CompactionState(partitionsSpec, null, null, null, indexSpec, null) ) ); @@ -1391,19 +1210,10 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentFil new CompactionState( partitionsSpec, null, - Collections.emptyMap(), null, - mapper.readValue( - mapper.writeValueAsString( - new TransformSpec( - new SelectorDimFilter("dim1", "foo", null), - null - ) - ), - new TypeReference>() {}), + mapper.readValue(mapper.writeValueAsString(new TransformSpec(new SelectorDimFilter("dim1", "foo", null), null)), new TypeReference>() {}), indexSpec, - null, - CompactionEngine.NATIVE + null ) ), new SegmentGenerateSpec( @@ -1413,19 +1223,10 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentFil new CompactionState( partitionsSpec, null, - Collections.emptyMap(), null, - mapper.readValue( - mapper.writeValueAsString( - new TransformSpec( - new SelectorDimFilter("dim1", "bar", null), - null - ) - ), - new TypeReference>() {}), + mapper.readValue(mapper.writeValueAsString(new TransformSpec(new SelectorDimFilter("dim1", "bar", null), null)), new TypeReference>() {}), indexSpec, - null, - CompactionEngine.NATIVE + null ) ), new SegmentGenerateSpec( @@ -1435,27 +1236,17 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentFil new CompactionState( partitionsSpec, null, - Collections.emptyMap(), null, + null, mapper.readValue(mapper.writeValueAsString(new TransformSpec(null, null)), new TypeReference>() {}), indexSpec, - null, - CompactionEngine.NATIVE + null ) ), new SegmentGenerateSpec( Intervals.of("2017-10-04T00:00:00/2017-10-05T00:00:00"), new Period("P1D"), null, - new CompactionState( - partitionsSpec, - null, - Collections.emptyMap(), - null, - null, - indexSpec, - null, - CompactionEngine.NATIVE - ) + new CompactionState(partitionsSpec, null, null, null, indexSpec, null) ) ); @@ -1543,12 +1334,10 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentMet new CompactionState( partitionsSpec, null, - Collections.emptyMap(), mapper.convertValue(new AggregatorFactory[] {new CountAggregatorFactory("cnt")}, new TypeReference>() {}), null, indexSpec, - null, - CompactionEngine.NATIVE + null ) ), new SegmentGenerateSpec( @@ -1558,12 +1347,10 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentMet new CompactionState( partitionsSpec, null, - Collections.emptyMap(), mapper.convertValue(new AggregatorFactory[] {new CountAggregatorFactory("cnt"), new LongSumAggregatorFactory("val", "val")}, new TypeReference>() {}), null, indexSpec, - null, - CompactionEngine.NATIVE + null ) ), new SegmentGenerateSpec( @@ -1573,28 +1360,17 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentMet new CompactionState( partitionsSpec, null, - Collections.emptyMap(), mapper.convertValue(new AggregatorFactory[] {}, new TypeReference>() {}), null, indexSpec, - null, - CompactionEngine.NATIVE + null ) ), new SegmentGenerateSpec( Intervals.of("2017-10-04T00:00:00/2017-10-05T00:00:00"), new Period("P1D"), null, - new CompactionState( - partitionsSpec, - null, - Collections.emptyMap(), - null, - null, - indexSpec, - null, - CompactionEngine.NATIVE - ) + new CompactionState(partitionsSpec, null, null, null, indexSpec, null) ) ); @@ -1700,16 +1476,7 @@ public void testIteratorReturnsSegmentsAsCompactionStateChangedWithCompactedStat Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), new Period("P1D"), null, - new CompactionState( - partitionsSpec, - null, - Collections.emptyMap(), - null, - null, - newIndexSpecMap, - null, - CompactionEngine.NATIVE - ) + new CompactionState(partitionsSpec, null, null, null, newIndexSpecMap, null) ) ); @@ -1759,12 +1526,10 @@ public void testIteratorDoesNotReturnSegmentWithChangingAppendableIndexSpec() new CompactionState( partitionsSpec, null, - Collections.emptyMap(), null, null, IndexSpec.DEFAULT.asMap(mapper), - null, - CompactionEngine.NATIVE + null ) ) ); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java index f2b102dac43d..048bd3231b7b 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java @@ -408,42 +408,12 @@ public void testMakeStatsForDataSourceWithCompactedIntervalBetweenNonCompactedIn DataSegment afterNoon = createSegment(dataSourceName, j, false, k); if (j == 3) { // Make two intervals on this day compacted (two compacted intervals back-to-back) - beforeNoon = beforeNoon.withLastCompactionState( - new CompactionState( - partitionsSpec, - null, - Collections.emptyMap(), - null, - null, - ImmutableMap.of(), - ImmutableMap.of(), - CompactionEngine.NATIVE - )); - afterNoon = afterNoon.withLastCompactionState( - new CompactionState( - partitionsSpec, - null, - Collections.emptyMap(), - null, - null, - ImmutableMap.of(), - ImmutableMap.of(), - CompactionEngine.NATIVE - )); + beforeNoon = beforeNoon.withLastCompactionState(new CompactionState(partitionsSpec, null, null, null, ImmutableMap.of(), ImmutableMap.of())); + afterNoon = afterNoon.withLastCompactionState(new CompactionState(partitionsSpec, null, null, null, ImmutableMap.of(), ImmutableMap.of())); } if (j == 1) { // Make one interval on this day compacted - afterNoon = afterNoon.withLastCompactionState( - new CompactionState( - partitionsSpec, - null, - Collections.emptyMap(), - null, - null, - ImmutableMap.of(), - ImmutableMap.of(), - CompactionEngine.NATIVE - )); + afterNoon = afterNoon.withLastCompactionState(new CompactionState(partitionsSpec, null, null, null, ImmutableMap.of(), ImmutableMap.of())); } segments.add(beforeNoon); segments.add(afterNoon); @@ -2140,7 +2110,7 @@ private void compactSegments( clientCompactionTaskQuery.getDimensionsSpec() == null ? null : new DimensionsSpec( clientCompactionTaskQuery.getDimensionsSpec().getDimensions() ), - Collections.emptyMap(), metricsSpec, + metricsSpec, transformSpec, ImmutableMap.of( "bitmap", @@ -2152,8 +2122,7 @@ private void compactSegments( "longEncoding", "longs" ), - ImmutableMap.of(), - CompactionEngine.NATIVE + ImmutableMap.of() ), 1, segmentSize diff --git a/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java b/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java index 8c7030c86739..0f20fc96bdcc 100644 --- a/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java +++ b/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java @@ -26,7 +26,6 @@ import com.google.common.collect.ImmutableMap; import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; @@ -42,7 +41,6 @@ import org.junit.Test; import java.util.Arrays; -import java.util.Collections; import java.util.Map; public class DataSegmentPlusTest @@ -90,12 +88,10 @@ public void testSerde() throws JsonProcessingException new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "bar", "foo")) ), - Collections.emptyMap(), ImmutableList.of(ImmutableMap.of("type", "count", "name", "count")), ImmutableMap.of("filter", ImmutableMap.of("type", "selector", "dimension", "dim1", "value", "foo")), ImmutableMap.of(), - ImmutableMap.of(), - CompactionEngine.NATIVE + ImmutableMap.of() ), TEST_VERSION, 1 @@ -129,7 +125,7 @@ public void testSerde() throws JsonProcessingException Assert.assertEquals(ImmutableMap.of("type", "numbered", "partitionNum", 3, "partitions", 0), segmentObjectMap.get("shardSpec")); Assert.assertEquals(TEST_VERSION, segmentObjectMap.get("binaryVersion")); Assert.assertEquals(1, segmentObjectMap.get("size")); - Assert.assertEquals(8, ((Map) segmentObjectMap.get("lastCompactionState")).size()); + Assert.assertEquals(6, ((Map) segmentObjectMap.get("lastCompactionState")).size()); // verify extra metadata Assert.assertEquals(createdDateStr, objectMap.get("createdDate")); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java index 5330305503c8..c2321d52250e 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java @@ -51,7 +51,6 @@ import org.apache.druid.discovery.DruidNodeDiscovery; import org.apache.druid.discovery.DruidNodeDiscoveryProvider; import org.apache.druid.discovery.NodeRole; -import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexing.overlord.supervisor.SupervisorStatus; @@ -284,12 +283,10 @@ public void setUp(@TempDir File tmpDir) throws Exception private final CompactionState expectedCompactionState = new CompactionState( new DynamicPartitionsSpec(null, null), null, - Collections.emptyMap(), null, null, Collections.singletonMap("test", "map"), - Collections.singletonMap("test2", "map2"), - CompactionEngine.NATIVE + Collections.singletonMap("test2", "map2") ); private final DataSegment publishedCompactedSegment1 = new DataSegment( From 6eb58883ccdb276fbbb8617bb2859439d49c2db6 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Mon, 24 Jun 2024 10:39:41 +0530 Subject: [PATCH 39/54] Change metrics comparison to use combining factory --- .../apache/druid/msq/exec/ControllerImpl.java | 27 ++++++------------- .../msq/indexing/MSQCompactionRunner.java | 27 +++++++++---------- .../coordinator/compact/CompactionStatus.java | 21 ++++++++++++--- 3 files changed, 38 insertions(+), 37 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 84f7f6564711..742e51a6064a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -1616,6 +1616,7 @@ private static Function, Set> addCompactionStateTo segmentGranularity, QueryContext.of(querySpec.getQuery().getContext()) .getGranularity(DruidSqlInsert.SQL_INSERT_QUERY_GRANULARITY, jsonMapper), + // MSQ has a custom definition of rollup where it needs to be a group-by query dataSchema.getGranularitySpec().isRollup(), ((DataSourceMSQDestination) querySpec.getDestination()).getReplaceTimeChunks() ); @@ -1626,25 +1627,13 @@ private static Function, Set> addCompactionStateTo : new ClientCompactionTaskTransformSpec( dataSchema.getTransformSpec().getFilter() ).asMap(jsonMapper); - List metricsSpec = null; - - if (querySpec.getQuery() instanceof GroupByQuery){ - GroupByQuery groupByQuery = (GroupByQuery) querySpec.getQuery(); - // Need to fetch this from the querySpec since the dataSchema uses the AggregatorFactory's combining factory - // version which updates field_name to name. For e.g. - - // LongSumAggregatorFactory{fieldName='added', expression='null', name='added_sum'} - // gets updated to - // LongSumAggregatorFactory{fieldName='added_sum', expression='null', name='added_sum'} - - // Also converting to metricsSpec from list to array as direct serialization from list doesn't capture the type. - metricsSpec = jsonMapper.convertValue( - groupByQuery.getAggregatorSpecs().toArray(new AggregatorFactory[0]), - new TypeReference>() - { - } - ); - } + List metricsSpec = dataSchema.getAggregators() == null + ? null + : jsonMapper.convertValue( + dataSchema.getAggregators(), + new TypeReference>() {} + ); + IndexSpec indexSpec = tuningConfig.getIndexSpec(); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java index 529af5361e5e..45f2e56003f8 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java @@ -88,7 +88,10 @@ public class MSQCompactionRunner implements CompactionRunner private final ObjectMapper jsonMapper; private final Injector injector; - + // Needed as output column name while grouping in the scenario of: + // a) no query granularity -- to specify an output name for the time dimension column since __time is a reserved name. + // b) custom query granularity -- to create a virtual column containing the rounded-off row timestamp. + // In both cases, the new column is converted back to __time later using columnMappings. public static final String TIME_VIRTUAL_COLUMN = "__vTime"; @JsonIgnore @@ -107,10 +110,10 @@ public MSQCompactionRunner(@JacksonInject ObjectMapper jsonMapper, @JacksonInjec } /** - Checks if the provided compaction config is supported by MSQ. - The same validation is done at - {@link ClientCompactionRunnerInfo#msqEngineSupportsCompactionConfig} - The following configs aren't supported: + * Checks if the provided compaction config is supported by MSQ. + * The same validation is done at + * {@link ClientCompactionRunnerInfo#msqEngineSupportsCompactionConfig} + * The following configs aren't supported: *
      *
    • partitionsSpec of type HashedParititionsSpec.
    • * @@ -195,7 +198,7 @@ public List createMsqControllerTasks( MSQSpec msqSpec = MSQSpec.builder() .query(query) .columnMappings(getColumnMappings(dataSchema)) - .destination(buildMSQDestination(compactionTask, dataSchema, compactionTaskContext)) + .destination(buildMSQDestination(compactionTask, dataSchema)) .assignmentStrategy(MultiStageQueryContext.getAssignmentStrategy(compactionTaskContext)) .tuningConfig(buildMSQTuningConfig(compactionTask, compactionTaskContext)) .build(); @@ -220,20 +223,17 @@ public List createMsqControllerTasks( private static DataSourceMSQDestination buildMSQDestination( CompactionTask compactionTask, - DataSchema dataSchema, - QueryContext compactionTaskContext + DataSchema dataSchema ) { final Interval replaceInterval = compactionTask.getIoConfig() .getInputSpec() .findInterval(compactionTask.getDataSource()); - final List segmentSortOrder = MultiStageQueryContext.getSortOrder(compactionTaskContext); - return new DataSourceMSQDestination( dataSchema.getDataSource(), dataSchema.getGranularitySpec().getSegmentGranularity(), - segmentSortOrder, + null, ImmutableList.of(replaceInterval) ); } @@ -440,10 +440,9 @@ private Map createMSQTaskContext(CompactionTask compactionTask, jsonMapper.writeValueAsString(dataSchema.getGranularitySpec().getQueryGranularity()) ); } - context.put(MultiStageQueryContext.CTX_SEGMENT_LOAD_WAIT, true); // Similar to compaction using the native engine, don't finalize aggregations. - context.put(MultiStageQueryContext.CTX_FINALIZE_AGGREGATIONS, false); - context.put(GroupByQueryConfig.CTX_KEY_ENABLE_MULTI_VALUE_UNNESTING, false); + context.putIfAbsent(MultiStageQueryContext.CTX_FINALIZE_AGGREGATIONS, false); + context.putIfAbsent(GroupByQueryConfig.CTX_KEY_ENABLE_MULTI_VALUE_UNNESTING, false); return context; } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java index 862f2e7c5b4c..9a26c98ee515 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java @@ -305,17 +305,30 @@ private CompactionStatus metricsSpecIsUpToDate() if (ArrayUtils.isEmpty(configuredMetricsSpec)) { return COMPLETE; } + final AggregatorFactory[] configuredMetricsCombiningFactorySpec = + Arrays.stream(configuredMetricsSpec) + .map(AggregatorFactory::getCombiningFactory) + .toArray(AggregatorFactory[]::new); final List metricSpecList = lastCompactionState.getMetricsSpec(); final AggregatorFactory[] existingMetricsSpec = CollectionUtils.isNullOrEmpty(metricSpecList) ? null : objectMapper.convertValue(metricSpecList, AggregatorFactory[].class); - - if (existingMetricsSpec == null || !Arrays.deepEquals(configuredMetricsSpec, existingMetricsSpec)) { + final AggregatorFactory[] existingMetricsCombiningFactorySpec = + existingMetricsSpec == null + ? null + : Arrays.stream(existingMetricsSpec) + .map(AggregatorFactory::getCombiningFactory) + .toArray(AggregatorFactory[]::new); + + if (existingMetricsSpec == null || !Arrays.deepEquals( + configuredMetricsCombiningFactorySpec, + existingMetricsCombiningFactorySpec + )) { return CompactionStatus.configChanged( "metricsSpec", - Arrays.toString(configuredMetricsSpec), - Arrays.toString(existingMetricsSpec) + Arrays.toString(configuredMetricsCombiningFactorySpec), + Arrays.toString(existingMetricsCombiningFactorySpec) ); } else { return COMPLETE; From 950f05d06264963281b6a6fb21633237998acaac Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Tue, 25 Jun 2024 18:35:28 +0530 Subject: [PATCH 40/54] Address latest review comments - part 1 --- .../apache/druid/msq/exec/ControllerImpl.java | 2 +- .../msq/indexing/MSQCompactionRunner.java | 18 ++-- .../apache/druid/msq/exec/MSQReplaceTest.java | 38 ++++----- .../common/task/CompactionRunner.java | 8 +- .../indexing/common/task/CompactionTask.java | 4 +- .../common/task/NativeCompactionRunner.java | 6 +- .../ClientCompactionTaskQuerySerdeTest.java | 2 +- .../druid/indexer/CompactionEngine.java | 2 +- .../indexing}/ClientCompactionRunnerInfo.java | 85 +++++++------------ .../ClientCompactionTaskGranularitySpec.java | 10 ++- .../indexing/ClientCompactionTaskQuery.java | 6 +- .../CompactionConfigValidationResult.java | 44 ++++++++++ .../DataSourceCompactionConfig.java | 12 +-- .../coordinator/duty/CompactSegments.java | 34 ++++---- .../CoordinatorCompactionConfigsResource.java | 5 +- .../ClientCompactionRunnerInfoTest.java | 1 + 16 files changed, 154 insertions(+), 123 deletions(-) rename server/src/main/java/org/apache/druid/{server/coordinator => client/indexing}/ClientCompactionRunnerInfo.java (70%) create mode 100644 server/src/main/java/org/apache/druid/server/coordinator/CompactionConfigValidationResult.java diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 742e51a6064a..33833f905e0a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -1616,8 +1616,8 @@ private static Function, Set> addCompactionStateTo segmentGranularity, QueryContext.of(querySpec.getQuery().getContext()) .getGranularity(DruidSqlInsert.SQL_INSERT_QUERY_GRANULARITY, jsonMapper), - // MSQ has a custom definition of rollup where it needs to be a group-by query dataSchema.getGranularitySpec().isRollup(), + // Not using dataSchema.getGranularitySpec().inputIntervals() as that always has ETERNITY ((DataSourceMSQDestination) querySpec.getDestination()).getReplaceTimeChunks() ); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java index 45f2e56003f8..99dc64872724 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java @@ -65,7 +65,8 @@ import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; -import org.apache.druid.server.coordinator.ClientCompactionRunnerInfo; +import org.apache.druid.client.indexing.ClientCompactionRunnerInfo; +import org.apache.druid.server.coordinator.CompactionConfigValidationResult; import org.apache.druid.sql.calcite.parser.DruidSqlInsert; import org.apache.druid.sql.calcite.planner.ColumnMapping; import org.apache.druid.sql.calcite.planner.ColumnMappings; @@ -110,32 +111,29 @@ public MSQCompactionRunner(@JacksonInject ObjectMapper jsonMapper, @JacksonInjec } /** - * Checks if the provided compaction config is supported by MSQ. - * The same validation is done at + * Checks if the provided compaction config is supported by MSQ. The same validation is done at * {@link ClientCompactionRunnerInfo#msqEngineSupportsCompactionConfig} * The following configs aren't supported: *
        *
      • partitionsSpec of type HashedParititionsSpec.
      • - * *
      • maxTotalRows in DynamicPartitionsSpec.
      • - * *
      • rollup set to false in granularitySpec when metricsSpec is specified.
      • *
      */ @Override - public ClientCompactionRunnerInfo.ValidationResult validateCompactionTask( + public CompactionConfigValidationResult validateCompactionTask( CompactionTask compactionTask ) { if (compactionTask.getTuningConfig() != null) { - ClientCompactionRunnerInfo.ValidationResult partitionSpecValidationResult = - ClientCompactionRunnerInfo.validatePartitionsSpec(compactionTask.getTuningConfig().getPartitionsSpec()); + CompactionConfigValidationResult partitionSpecValidationResult = + ClientCompactionRunnerInfo.validatePartitionsSpecForMsq(compactionTask.getTuningConfig().getPartitionsSpec()); if (!partitionSpecValidationResult.isValid()) { return partitionSpecValidationResult; } } if (compactionTask.getGranularitySpec() != null) { - ClientCompactionRunnerInfo.ValidationResult rollupValidationResult = ClientCompactionRunnerInfo.validateRollup( + CompactionConfigValidationResult rollupValidationResult = ClientCompactionRunnerInfo.validateRollupForMsq( compactionTask.getMetricsSpec(), compactionTask.getGranularitySpec().isRollup() ); @@ -143,7 +141,7 @@ public ClientCompactionRunnerInfo.ValidationResult validateCompactionTask( return rollupValidationResult; } } - return new ClientCompactionRunnerInfo.ValidationResult(true, null); + return new CompactionConfigValidationResult(true, null); } @Override diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java index db92dd9b7d83..6181f4261d44 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java @@ -185,7 +185,7 @@ public void testReplaceOnFooWithAll(String contextName, Map cont Collections.emptyList(), Collections.singletonList(new FloatDimensionSchema("m1")), GranularityType.DAY, - Intervals.ONLY_ETERNITY + Intervals.ETERNITY ) ) .verifyResults(); @@ -244,7 +244,7 @@ public void testReplaceOnFooWithWhere(String contextName, Map co Collections.emptyList(), Collections.singletonList(new FloatDimensionSchema("m1")), GranularityType.DAY, - ImmutableList.of(Intervals.of("2000-01-02T/P1D")) + Intervals.of("2000-01-02T/P1D") ) ) .verifyResults(); @@ -335,7 +335,7 @@ public void testReplaceOnFoo1WithAllExtern(String contextName, Map c Collections.emptyList(), Collections.singletonList(new FloatDimensionSchema("m1")), GranularityType.MONTH, - ImmutableList.of(Intervals.of("2000-01-01T/2000-03-01T")) + Intervals.of("2000-01-01T/2000-03-01T") ) ) .verifyResults(); @@ -755,7 +755,7 @@ public void testReplaceWhereClauseLargerThanData(String contextName, Map contex Collections.emptyList(), Collections.singletonList(new FloatDimensionSchema("m1")), GranularityType.MONTH, - Collections.singletonList(Intervals.of("2000-01-01T/2000-03-01T")) + Intervals.of("2000-01-01T/2000-03-01T") ) ) .verifyResults(); @@ -910,7 +910,7 @@ public void testReplaceTimeChunksLargerThanData(String contextName, Map conte new LongDimensionSchema("cnt") ), GranularityType.DAY, - Intervals.ONLY_ETERNITY + Intervals.ETERNITY ) ) .verifyResults(); @@ -1094,7 +1094,7 @@ public void testReplaceSegmentsInsertIntoNewTable(String contextName, Map partitionDimensions, List dimensions, GranularityType segmentGranularity, - List intervals + Interval interval ) { if (!context.containsKey(Tasks.STORE_COMPACTION_STATE_KEY) @@ -1958,7 +1958,7 @@ private CompactionState expectedCompactionState( segmentGranularity.getDefaultGranularity(), GranularityType.NONE.getDefaultGranularity(), false, - intervals + Collections.singletonList(interval) ); List metricsSpec = Collections.emptyList(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java index 5b4d1dd4aea7..736223cf73b2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java @@ -24,7 +24,8 @@ import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.server.coordinator.ClientCompactionRunnerInfo; +import org.apache.druid.client.indexing.ClientCompactionRunnerInfo; +import org.apache.druid.server.coordinator.CompactionConfigValidationResult; import org.joda.time.Interval; import java.util.Map; @@ -42,7 +43,7 @@ public interface CompactionRunner String TYPE_PROPERTY = "type"; /** - * Converts compaction tasks to the runner-type tasks and runs them serially for each interval. + * Creates and runs sub-tasks for the given CompactionTask, one interval at a time. */ TaskStatus runCompactionTasks( CompactionTask compactionTask, @@ -55,8 +56,7 @@ TaskStatus runCompactionTasks( /** * Checks if the provided compaction config is supported by the runner. * The same validation is done at {@link org.apache.druid.msq.indexing.MSQCompactionRunner#validateCompactionTask} - * @return ValidationResult. The reason string is null if isValid() is True. */ - ClientCompactionRunnerInfo.ValidationResult validateCompactionTask(CompactionTask compactionTask); + CompactionConfigValidationResult validateCompactionTask(CompactionTask compactionTask); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index d8791d1eeddd..eb3861ca824d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -85,7 +85,7 @@ import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; -import org.apache.druid.server.coordinator.ClientCompactionRunnerInfo; +import org.apache.druid.server.coordinator.CompactionConfigValidationResult; import org.apache.druid.server.coordinator.duty.CompactSegments; import org.apache.druid.server.lookup.cache.LookupLoadingSpec; import org.apache.druid.server.security.ResourceAction; @@ -479,7 +479,7 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception ); registerResourceCloserOnAbnormalExit(compactionRunner.getCurrentSubTaskHolder()); - ClientCompactionRunnerInfo.ValidationResult supportsCompactionConfig = compactionRunner.validateCompactionTask(this); + CompactionConfigValidationResult supportsCompactionConfig = compactionRunner.validateCompactionTask(this); if (!supportsCompactionConfig.isValid()) { throw InvalidInput.exception("Compaction spec not supported. Reason[%s].", supportsCompactionConfig.getReason()); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java index 28be9b958875..722c6010b206 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java @@ -40,7 +40,7 @@ import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.server.coordinator.ClientCompactionRunnerInfo; +import org.apache.druid.server.coordinator.CompactionConfigValidationResult; import org.apache.druid.server.coordinator.duty.CompactSegments; import org.apache.druid.utils.CollectionUtils; import org.codehaus.jackson.annotate.JsonCreator; @@ -81,11 +81,11 @@ public CurrentSubTaskHolder getCurrentSubTaskHolder() } @Override - public ClientCompactionRunnerInfo.ValidationResult validateCompactionTask( + public CompactionConfigValidationResult validateCompactionTask( CompactionTask compactionTask ) { - return new ClientCompactionRunnerInfo.ValidationResult(true, null); + return new CompactionConfigValidationResult(true, null); } /** diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java index d580e55483dc..db3e381f1465 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java @@ -65,7 +65,7 @@ import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; -import org.apache.druid.server.coordinator.ClientCompactionRunnerInfo; +import org.apache.druid.client.indexing.ClientCompactionRunnerInfo; import org.apache.druid.server.lookup.cache.LookupLoadingSpec; import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.server.security.AuthorizerMapper; diff --git a/processing/src/main/java/org/apache/druid/indexer/CompactionEngine.java b/processing/src/main/java/org/apache/druid/indexer/CompactionEngine.java index b14b0c453271..a95f64b896a7 100644 --- a/processing/src/main/java/org/apache/druid/indexer/CompactionEngine.java +++ b/processing/src/main/java/org/apache/druid/indexer/CompactionEngine.java @@ -38,7 +38,7 @@ public enum CompactionEngine @JsonValue public String toString() { - return StringUtils.toLowerCase(this.name().toLowerCase()); + return StringUtils.toLowerCase(this.name()); } @JsonCreator diff --git a/server/src/main/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfo.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java similarity index 70% rename from server/src/main/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfo.java rename to server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java index 1171695ad77d..e2b18fc05bfb 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfo.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.server.coordinator; +package org.apache.druid.client.indexing; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; @@ -25,11 +25,11 @@ import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.query.QueryContext; import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.server.coordinator.CompactionConfigValidationResult; +import org.apache.druid.server.coordinator.DataSourceCompactionConfig; -import java.util.Map; +import javax.annotation.Nullable; import java.util.Objects; @@ -74,41 +74,20 @@ public boolean equals(Object o) return type == that.type; } - public static class ValidationResult{ - private final boolean valid; - private final String reason; - - public ValidationResult(boolean valid, String reason) - { - this.valid = valid; - this.reason = reason; - } - - public boolean isValid() - { - return valid; - } - - public String getReason() - { - return reason; - } - } - @Override public int hashCode() { return Objects.hash(type); } - public static ValidationResult validateCompactionConfig( + public static CompactionConfigValidationResult validateCompactionConfig( DataSourceCompactionConfig newConfig, CompactionEngine defaultCompactionEngine ) { CompactionEngine compactionEngine = newConfig.getEngine() == null ? defaultCompactionEngine : newConfig.getEngine(); if (compactionEngine == CompactionEngine.NATIVE) { - return new ValidationResult(true, null); + return new CompactionConfigValidationResult(true, null); } else { return msqEngineSupportsCompactionConfig(newConfig); } @@ -118,28 +97,22 @@ public static ValidationResult validateCompactionConfig( * Checks if the provided compaction config is supported by MSQ. The following configs aren't supported: *
        *
      • finalizeAggregations set to false in context.
      • - * *
      • partitionsSpec of type HashedParititionsSpec.
      • - * *
      • maxTotalRows in DynamicPartitionsSpec.
      • - * *
      • rollup set to false in granularitySpec when metricsSpec is specified.
      • *
      - * - * @param newConfig The updated compaction config - * @return ValidationResult. The reason string is null if isValid() is True. */ - private static ValidationResult msqEngineSupportsCompactionConfig(DataSourceCompactionConfig newConfig) + private static CompactionConfigValidationResult msqEngineSupportsCompactionConfig(DataSourceCompactionConfig newConfig) { if (newConfig.getTuningConfig() != null) { - ValidationResult partitionSpecValidationResult = - validatePartitionsSpec(newConfig.getTuningConfig().getPartitionsSpec()); + CompactionConfigValidationResult partitionSpecValidationResult = + validatePartitionsSpecForMsq(newConfig.getTuningConfig().getPartitionsSpec()); if (!partitionSpecValidationResult.isValid()) { return partitionSpecValidationResult; } } if (newConfig.getGranularitySpec() != null) { - ValidationResult rollupValidationResult = validateRollup( + CompactionConfigValidationResult rollupValidationResult = validateRollupForMsq( newConfig.getMetricsSpec(), newConfig.getGranularitySpec().isRollup() ); @@ -147,43 +120,49 @@ private static ValidationResult msqEngineSupportsCompactionConfig(DataSourceComp return rollupValidationResult; } } - return new ValidationResult(true, null); + return new CompactionConfigValidationResult(true, null); } /** * Validte that partitionSpec is either 'dynamic` or 'range', and if 'dynamic', ensure maxTotalRows is null. */ - public static ValidationResult validatePartitionsSpec(PartitionsSpec partitionsSpec) + public static CompactionConfigValidationResult validatePartitionsSpecForMsq(PartitionsSpec partitionsSpec) { if (!(partitionsSpec instanceof DimensionRangePartitionsSpec || partitionsSpec instanceof DynamicPartitionsSpec)) { - return new ValidationResult(false, StringUtils.format( - "Invalid partition spec type[%s] for MSQ engine." - + " Type must be either DynamicPartitionsSpec or DynamicRangePartitionsSpec.", - partitionsSpec.getClass() - ) + return new CompactionConfigValidationResult( + false, + "Invalid partitionsSpec type[%s] for MSQ engine. Type must be either 'dynamic' or 'range'.", + partitionsSpec.getClass().getSimpleName() + ); } if (partitionsSpec instanceof DynamicPartitionsSpec && ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows() != null) { - return new ValidationResult(false, StringUtils.format( + return new CompactionConfigValidationResult( + false, "maxTotalRows[%d] in DynamicPartitionsSpec not supported for MSQ engine.", ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows() - )); + ); } - return new ValidationResult(true, null); + return new CompactionConfigValidationResult(true, null); } /** * Validate rollup is set to false in granularitySpec when metricsSpec is specified. */ - public static ValidationResult validateRollup(AggregatorFactory[] metricsSpec, boolean isRollup) { - if (metricsSpec != null && !isRollup) { - return new ValidationResult(false, StringUtils.format( - "rollup in granularitySpec must be set to True if metricsSpec is specifed " - + "for MSQ engine.")); + public static CompactionConfigValidationResult validateRollupForMsq( + AggregatorFactory[] metricsSpec, + @Nullable Boolean isRollup + ) + { + if (metricsSpec != null && isRollup != null && !isRollup) { + return new CompactionConfigValidationResult( + false, + "rollup in granularitySpec must be set to True if metricsSpec is specifed for MSQ engine." + ); } - return new ValidationResult(true, null); + return new CompactionConfigValidationResult(true, null); } /** diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskGranularitySpec.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskGranularitySpec.java index 3ba732cfbf74..27d5b21f43dc 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskGranularitySpec.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskGranularitySpec.java @@ -24,6 +24,7 @@ import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.segment.indexing.granularity.GranularitySpec; +import javax.annotation.Nullable; import java.util.Objects; /** @@ -44,9 +45,9 @@ public class ClientCompactionTaskGranularitySpec @JsonCreator public ClientCompactionTaskGranularitySpec( - @JsonProperty("segmentGranularity") Granularity segmentGranularity, - @JsonProperty("queryGranularity") Granularity queryGranularity, - @JsonProperty("rollup") Boolean rollup + @JsonProperty("segmentGranularity") @Nullable Granularity segmentGranularity, + @JsonProperty("queryGranularity") @Nullable Granularity queryGranularity, + @JsonProperty("rollup") @Nullable Boolean rollup ) { this.queryGranularity = queryGranularity; @@ -55,18 +56,21 @@ public ClientCompactionTaskGranularitySpec( } @JsonProperty + @Nullable public Granularity getSegmentGranularity() { return segmentGranularity; } @JsonProperty + @Nullable public Granularity getQueryGranularity() { return queryGranularity; } @JsonProperty + @Nullable public Boolean isRollup() { return rollup; diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQuery.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQuery.java index dd4ebafd3919..d8efe4397ac5 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQuery.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQuery.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.server.coordinator.ClientCompactionRunnerInfo; import javax.annotation.Nullable; import java.util.Arrays; @@ -60,7 +59,7 @@ public ClientCompactionTaskQuery( @JsonProperty("metricsSpec") AggregatorFactory[] metrics, @JsonProperty("transformSpec") ClientCompactionTaskTransformSpec transformSpec, @JsonProperty("context") Map context, - @JsonProperty("compactionRunner") ClientCompactionRunnerInfo compactionRunner + @JsonProperty("compactionRunner") @Nullable ClientCompactionRunnerInfo compactionRunner ) { this.id = Preconditions.checkNotNull(id, "id"); @@ -140,7 +139,8 @@ public Map getContext() } @JsonProperty("compactionRunner") - public ClientCompactionRunnerInfo getClientCompactionRunnerInfo() + @Nullable + public ClientCompactionRunnerInfo getCompactionRunner() { return compactionRunner; } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CompactionConfigValidationResult.java b/server/src/main/java/org/apache/druid/server/coordinator/CompactionConfigValidationResult.java new file mode 100644 index 000000000000..e412fa40c36e --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/CompactionConfigValidationResult.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.coordinator; + +import org.apache.druid.java.util.common.StringUtils; + +public class CompactionConfigValidationResult +{ + private final boolean valid; + private final String reason; + + public CompactionConfigValidationResult(boolean valid, String format, Object... args) + { + this.valid = valid; + this.reason = StringUtils.format(format, args); + } + + public boolean isValid() + { + return valid; + } + + public String getReason() + { + return reason; + } +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java index 064d89c2c4eb..767e8218f319 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java @@ -56,7 +56,7 @@ public class DataSourceCompactionConfig private final UserCompactionTaskTransformConfig transformSpec; private final UserCompactionTaskIOConfig ioConfig; private final Map taskContext; - private final CompactionEngine compactionEngine; + private final CompactionEngine engine; @JsonCreator public DataSourceCompactionConfig( @@ -71,7 +71,7 @@ public DataSourceCompactionConfig( @JsonProperty("metricsSpec") @Nullable AggregatorFactory[] metricsSpec, @JsonProperty("transformSpec") @Nullable UserCompactionTaskTransformConfig transformSpec, @JsonProperty("ioConfig") @Nullable UserCompactionTaskIOConfig ioConfig, - @JsonProperty("engine") @Nullable CompactionEngine compactionEngine, + @JsonProperty("engine") @Nullable CompactionEngine engine, @JsonProperty("taskContext") @Nullable Map taskContext ) { @@ -91,7 +91,7 @@ public DataSourceCompactionConfig( this.dimensionsSpec = dimensionsSpec; this.transformSpec = transformSpec; this.taskContext = taskContext; - this.compactionEngine = compactionEngine; + this.engine = engine; } @JsonProperty @@ -179,7 +179,7 @@ public Map getTaskContext() @Nullable public CompactionEngine getEngine() { - return compactionEngine; + return engine; } @Override @@ -203,7 +203,7 @@ public boolean equals(Object o) Arrays.equals(metricsSpec, that.metricsSpec) && Objects.equals(transformSpec, that.transformSpec) && Objects.equals(ioConfig, that.ioConfig) && - this.compactionEngine == that.compactionEngine && + this.engine == that.engine && Objects.equals(taskContext, that.taskContext); } @@ -222,7 +222,7 @@ public int hashCode() transformSpec, ioConfig, taskContext, - compactionEngine + engine ); result = 31 * result + Arrays.hashCode(metricsSpec); return result; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index 698a9bdf4a9f..1587df351d46 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java @@ -47,7 +47,7 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; -import org.apache.druid.server.coordinator.ClientCompactionRunnerInfo; +import org.apache.druid.client.indexing.ClientCompactionRunnerInfo; import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; @@ -89,6 +89,8 @@ public class CompactSegments implements CoordinatorCustomDuty private static final Predicate IS_COMPACTION_TASK = status -> null != status && COMPACTION_TASK_TYPE.equals(status.getType()); + private static final int MAX_TASK_SLOTS_FOR_MSQ_COMPACTION = 8; + private final CompactionSegmentSearchPolicy policy; private final OverlordClient overlordClient; @@ -240,7 +242,7 @@ private boolean cancelTaskIfGranularityChanged( return false; } - LOG.warn( + LOG.info( "Cancelling task [%s] as task segmentGranularity is [%s] but compaction config segmentGranularity is [%s]", compactionTaskQuery.getId(), taskSegmentGranularity, configuredSegmentGranularity ); @@ -364,9 +366,9 @@ private int submitCompactionTasks( } int numSubmittedTasks = 0; - int numCompactionTasksAndSubtasks = 0; + int totalTaskSlotsAssigned = 0; - while (iterator.hasNext() && numCompactionTasksAndSubtasks < numAvailableCompactionTaskSlots) { + while (iterator.hasNext() && totalTaskSlotsAssigned < numAvailableCompactionTaskSlots) { final SegmentsToCompact entry = iterator.next(); final List segmentsToCompact = entry.getSegments(); if (segmentsToCompact.isEmpty()) { @@ -466,23 +468,25 @@ private int submitCompactionTasks( } } - CompactionEngine compactionEngine = config.getEngine(); - Map autoCompactionContext = newAutoCompactionContext(config.getTaskContext()); - int numCurrentCompactionTasksAndSubtasks; + final CompactionEngine compactionEngine = config.getEngine(); + final Map autoCompactionContext = newAutoCompactionContext(config.getTaskContext()); + int slotsRequiredForCurrentTask; final String maxNumTasksContextParam = "maxNumTasks"; if (compactionEngine == CompactionEngine.MSQ) { - if (!autoCompactionContext.containsKey(maxNumTasksContextParam)) { + if (autoCompactionContext.containsKey(maxNumTasksContextParam)) { + slotsRequiredForCurrentTask = (int) autoCompactionContext.get(maxNumTasksContextParam); + } else { // Since MSQ needs all task slots for the calculated #tasks to be available upfront, allot all available // compaction slots to current compaction task to avoid stalling. Setting "taskAssignment" to "auto" has the // problem of not being able to determine the actual count, which is required for subsequent tasks. - numCurrentCompactionTasksAndSubtasks = numAvailableCompactionTaskSlots; - autoCompactionContext.put(maxNumTasksContextParam, numCurrentCompactionTasksAndSubtasks); - } else { - numCurrentCompactionTasksAndSubtasks = (int) autoCompactionContext.get(maxNumTasksContextParam); + slotsRequiredForCurrentTask = numAvailableCompactionTaskSlots > MAX_TASK_SLOTS_FOR_MSQ_COMPACTION + ? MAX_TASK_SLOTS_FOR_MSQ_COMPACTION + : numAvailableCompactionTaskSlots; + autoCompactionContext.put(maxNumTasksContextParam, slotsRequiredForCurrentTask); } } else { - numCurrentCompactionTasksAndSubtasks = findMaxNumTaskSlotsUsedByOneNativeCompactionTask(config.getTuningConfig()); + slotsRequiredForCurrentTask = findMaxNumTaskSlotsUsedByOneNativeCompactionTask(config.getTuningConfig()); } final String taskId = compactSegments( @@ -510,7 +514,7 @@ private int submitCompactionTasks( LOG.debugSegments(segmentsToCompact, "Compacting segments"); // Count the compaction task itself + its sub tasks numSubmittedTasks++; - numCompactionTasksAndSubtasks += numCurrentCompactionTasksAndSubtasks; + totalTaskSlotsAssigned += slotsRequiredForCurrentTask; } LOG.info("Submitted a total of [%d] compaction tasks.", numSubmittedTasks); @@ -650,7 +654,7 @@ private String compactSegments( @Nullable ClientCompactionTaskTransformSpec transformSpec, @Nullable Boolean dropExisting, @Nullable Map context, - @Nullable ClientCompactionRunnerInfo compactionRunner + ClientCompactionRunnerInfo compactionRunner ) { Preconditions.checkArgument(!segments.isEmpty(), "Expect non-empty segments to compact"); diff --git a/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java index 6b4ba22f1a72..564567cd9ac7 100644 --- a/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java +++ b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java @@ -30,11 +30,12 @@ import org.apache.druid.error.InvalidInput; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.server.coordinator.ClientCompactionRunnerInfo; +import org.apache.druid.client.indexing.ClientCompactionRunnerInfo; import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfigHistory; +import org.apache.druid.server.coordinator.CompactionConfigValidationResult; import org.apache.druid.server.http.security.ConfigResourceFilter; import org.apache.druid.server.security.AuthorizationUtils; import org.joda.time.Interval; @@ -121,7 +122,7 @@ public Response addOrUpdateCompactionConfig( .getCompactionConfigs() .stream() .collect(Collectors.toMap(DataSourceCompactionConfig::getDataSource, Function.identity())); - ClientCompactionRunnerInfo.ValidationResult validationResult = + CompactionConfigValidationResult validationResult = ClientCompactionRunnerInfo.validateCompactionConfig(newConfig, current.getEngine()); if (!validationResult.isValid()) { throw InvalidInput.exception("Compaction config not supported. Reason[%s].", validationResult.getReason()); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfoTest.java b/server/src/test/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfoTest.java index 89d7bfc690b0..e8e38167ca10 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfoTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfoTest.java @@ -21,6 +21,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import org.apache.druid.client.indexing.ClientCompactionRunnerInfo; import org.apache.druid.data.input.SegmentsSplitHintSpec; import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; From cc5b07069afbcb36d3f81c9cba3f322c38d6b79b Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Wed, 26 Jun 2024 15:57:55 +0530 Subject: [PATCH 41/54] Address latest review comments - part 2 --- .../msq/indexing/MSQCompactionRunner.java | 27 ++--- .../msq/util/MultiStageQueryContext.java | 2 + .../msq/indexing/MSQCompactionRunnerTest.java | 4 +- .../indexing/common/task/CompactionTask.java | 6 +- .../ClientCompactionTaskQuerySerdeTest.java | 3 +- .../task/CompactionTaskParallelRunTest.java | 42 +++----- .../common/task/CompactionTaskRunTest.java | 63 ++++------- .../common/task/CompactionTaskTest.java | 48 +++------ .../batch/parallel/PartialCompactionTest.java | 3 +- .../indexing/ClientCompactionRunnerInfo.java | 45 ++++---- .../client/indexing/ClientMsqContext.java | 31 ++++++ .../CompactionConfigValidationResult.java | 2 +- .../coordinator/duty/CompactSegments.java | 67 ++++++++---- .../CoordinatorCompactionConfigsResource.java | 4 +- .../ClientCompactionRunnerInfoTest.java | 102 +++++++++++++----- .../coordinator/duty/CompactSegmentsTest.java | 1 - 16 files changed, 255 insertions(+), 195 deletions(-) create mode 100644 server/src/main/java/org/apache/druid/client/indexing/ClientMsqContext.java rename server/src/test/java/org/apache/druid/{server/coordinator => client/indexing}/ClientCompactionRunnerInfoTest.java (53%) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java index 99dc64872724..0ec04fcc09dd 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java @@ -117,7 +117,8 @@ public MSQCompactionRunner(@JacksonInject ObjectMapper jsonMapper, @JacksonInjec *
        *
      • partitionsSpec of type HashedParititionsSpec.
      • *
      • maxTotalRows in DynamicPartitionsSpec.
      • - *
      • rollup set to false in granularitySpec when metricsSpec is specified.
      • + *
      • rollup set to false in granularitySpec when metricsSpec is specified. Null is treated as true.
      • + *
      • queryGranularity set to ALL in granularitySpec.
      • *
      */ @Override @@ -125,23 +126,25 @@ public CompactionConfigValidationResult validateCompactionTask( CompactionTask compactionTask ) { + List validationResults = new ArrayList<>(); if (compactionTask.getTuningConfig() != null) { - CompactionConfigValidationResult partitionSpecValidationResult = - ClientCompactionRunnerInfo.validatePartitionsSpecForMsq(compactionTask.getTuningConfig().getPartitionsSpec()); - if (!partitionSpecValidationResult.isValid()) { - return partitionSpecValidationResult; - } + validationResults.add(ClientCompactionRunnerInfo.validatePartitionsSpecForMsq( + compactionTask.getTuningConfig().getPartitionsSpec()) + ); } if (compactionTask.getGranularitySpec() != null) { - CompactionConfigValidationResult rollupValidationResult = ClientCompactionRunnerInfo.validateRollupForMsq( + validationResults.add(ClientCompactionRunnerInfo.validateRollupForMsq( compactionTask.getMetricsSpec(), compactionTask.getGranularitySpec().isRollup() - ); - if (!rollupValidationResult.isValid()) { - return rollupValidationResult; - } + )); + validationResults.add(ClientCompactionRunnerInfo.validateQueryGranularityForMsq( + compactionTask.getGranularitySpec().getQueryGranularity() + )); } - return new CompactionConfigValidationResult(true, null); + return validationResults.stream() + .filter(result -> !result.isValid()) + .findFirst() + .orElse(new CompactionConfigValidationResult(true, null)); } @Override diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java index b616a3f11898..66f0833caffb 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java @@ -100,6 +100,8 @@ public class MultiStageQueryContext public static final String CTX_MSQ_MODE = "mode"; public static final String DEFAULT_MSQ_MODE = MSQMode.STRICT_MODE.toString(); + // Note: CTX_MAX_NUM_TASKS and DEFAULT_MAX_NUM_TASKS values used here should be kept in sync with those in + // org.apache.druid.client.indexing.ClientMsqContext public static final String CTX_MAX_NUM_TASKS = "maxNumTasks"; @VisibleForTesting static final int DEFAULT_MAX_NUM_TASKS = 2; diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java index 5e2f4ec99b4f..190b0602f821 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java @@ -335,8 +335,7 @@ private CompactionTask createCompactionTask( final CompactionTask.Builder builder = new CompactionTask.Builder( DATA_SOURCE, null, - null, - new MSQCompactionRunner(JSON_MAPPER, null) + null ); IndexSpec indexSpec = createIndexSpec(); @@ -352,6 +351,7 @@ private CompactionTask createCompactionTask( partitionsSpec == null ? new DynamicPartitionsSpec(100, null) : partitionsSpec )) .transformSpec(transformSpec) + .compactionRunner( new MSQCompactionRunner(JSON_MAPPER, null)) .context(context); return builder.build(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index eb3861ca824d..fe4d09d8481a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -231,6 +231,8 @@ public CompactionTask( } this.tuningConfig = tuningConfig != null ? getTuningConfig(tuningConfig) : null; this.segmentProvider = new SegmentProvider(dataSource, this.ioConfig.getInputSpec()); + // Note: The default compactionRunnerType used here should match the default runner used in CompactSegments#run + // when no runner is detected in the returned compactionTaskQuery. this.compactionRunner = compactionRunner == null ? new NativeCompactionRunner(segmentCacheManagerFactory) : compactionRunner; @@ -1053,14 +1055,12 @@ public static class Builder public Builder( String dataSource, SegmentCacheManagerFactory segmentCacheManagerFactory, - RetryPolicyFactory retryPolicyFactory, - CompactionRunner compactionRunner + RetryPolicyFactory retryPolicyFactory ) { this.dataSource = dataSource; this.segmentCacheManagerFactory = segmentCacheManagerFactory; this.retryPolicyFactory = retryPolicyFactory; - this.compactionRunner = compactionRunner; } public Builder interval(Interval interval) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java index db3e381f1465..a031b05d3643 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java @@ -341,8 +341,7 @@ private CompactionTask createCompactionTask(ClientCompactionTaskTransformSpec tr CompactionTask.Builder compactionTaskBuilder = new CompactionTask.Builder( "datasource", new SegmentCacheManagerFactory(TestIndex.INDEX_IO, MAPPER), - new RetryPolicyFactory(new RetryPolicyConfig()), - null + new RetryPolicyFactory(new RetryPolicyConfig()) ) .inputSpec(new CompactionIntervalSpec(Intervals.of("2019/2020"), "testSha256OfSortedSegmentIds"), true) .tuningConfig( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java index 077cb6c86c43..ba9a6e3e2be2 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java @@ -161,8 +161,7 @@ public void testRunParallelWithDynamicPartitioningMatchCompactionState() throws final Builder builder = new Builder( DATA_SOURCE, getSegmentCacheManagerFactory(), - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) @@ -217,8 +216,7 @@ public void testRunParallelWithHashPartitioningMatchCompactionState() throws Exc final Builder builder = new Builder( DATA_SOURCE, getSegmentCacheManagerFactory(), - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) @@ -284,8 +282,7 @@ public void testRunParallelWithRangePartitioning() throws Exception final Builder builder = new Builder( DATA_SOURCE, getSegmentCacheManagerFactory(), - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) @@ -336,8 +333,7 @@ public void testRunParallelWithRangePartitioningAndNoUpfrontSegmentFetching() th final Builder builder = new Builder( DATA_SOURCE, getSegmentCacheManagerFactory(), - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); final CompactionTask compactionTask = builder @@ -400,8 +396,7 @@ public void testRunParallelWithMultiDimensionRangePartitioning() throws Exceptio final Builder builder = new Builder( DATA_SOURCE, getSegmentCacheManagerFactory(), - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) @@ -455,8 +450,7 @@ public void testRunParallelWithRangePartitioningWithSingleTask() throws Exceptio final Builder builder = new Builder( DATA_SOURCE, getSegmentCacheManagerFactory(), - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) @@ -507,8 +501,7 @@ public void testRunParallelWithMultiDimensionRangePartitioningWithSingleTask() t final Builder builder = new Builder( DATA_SOURCE, getSegmentCacheManagerFactory(), - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) @@ -559,8 +552,7 @@ public void testRunCompactionStateNotStoreIfContextSetToFalse() final Builder builder = new Builder( DATA_SOURCE, getSegmentCacheManagerFactory(), - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) @@ -591,8 +583,7 @@ public void testRunCompactionWithFilterShouldStoreInState() throws Exception final Builder builder = new Builder( DATA_SOURCE, getSegmentCacheManagerFactory(), - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) @@ -649,8 +640,7 @@ public void testRunCompactionWithNewMetricsShouldStoreInState() throws Exception final Builder builder = new Builder( DATA_SOURCE, getSegmentCacheManagerFactory(), - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) @@ -713,8 +703,7 @@ public void testCompactHashAndDynamicPartitionedSegments() final Builder builder = new Builder( DATA_SOURCE, getSegmentCacheManagerFactory(), - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) @@ -764,8 +753,7 @@ public void testCompactRangeAndDynamicPartitionedSegments() final Builder builder = new Builder( DATA_SOURCE, getSegmentCacheManagerFactory(), - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) @@ -857,8 +845,7 @@ public void testCompactionDropSegmentsOfInputIntervalIfDropFlagIsSet() throws Ex final Builder builder = new Builder( DATA_SOURCE, getSegmentCacheManagerFactory(), - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); final CompactionTask compactionTask = builder // Set the dropExisting flag to true in the IOConfig of the compaction task @@ -905,8 +892,7 @@ public void testCompactionDoesNotDropSegmentsIfDropFlagNotSet() throws Exception final Builder builder = new Builder( DATA_SOURCE, getSegmentCacheManagerFactory(), - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index 471932ccfe2e..599a24fac802 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -285,8 +285,7 @@ public void testRunWithDynamicPartitioning() throws Exception final Builder builder = new Builder( DATA_SOURCE, segmentCacheManagerFactory, - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); final CompactionTask compactionTask = builder @@ -354,8 +353,7 @@ public void testRunWithHashPartitioning() throws Exception final Builder builder = new Builder( DATA_SOURCE, segmentCacheManagerFactory, - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); final CompactionTask compactionTask = builder @@ -455,8 +453,7 @@ public void testRunCompactionTwice() throws Exception final Builder builder = new Builder( DATA_SOURCE, segmentCacheManagerFactory, - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); final CompactionTask compactionTask1 = builder @@ -551,8 +548,7 @@ public void testRunIndexAndCompactAtTheSameTimeForDifferentInterval() throws Exc final Builder builder = new Builder( DATA_SOURCE, segmentCacheManagerFactory, - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); final CompactionTask compactionTask = builder @@ -664,8 +660,7 @@ public void testWithSegmentGranularity() throws Exception final Builder builder = new Builder( DATA_SOURCE, segmentCacheManagerFactory, - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); // day segmentGranularity @@ -735,8 +730,7 @@ public void testWithSegmentGranularityMisalignedInterval() throws Exception final Builder builder = new Builder( DATA_SOURCE, segmentCacheManagerFactory, - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); final CompactionTask compactionTask1 = builder @@ -771,8 +765,7 @@ public void testWithSegmentGranularityMisalignedIntervalAllowed() throws Excepti final Builder builder = new Builder( DATA_SOURCE, segmentCacheManagerFactory, - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); // day segmentGranularity @@ -817,8 +810,7 @@ public void testCompactionWithFilterInTransformSpec() throws Exception final Builder builder = new Builder( DATA_SOURCE, segmentCacheManagerFactory, - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); // day segmentGranularity @@ -878,8 +870,7 @@ public void testCompactionWithNewMetricInMetricsSpec() throws Exception final Builder builder = new Builder( DATA_SOURCE, segmentCacheManagerFactory, - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); // day segmentGranularity @@ -945,8 +936,7 @@ public void testWithGranularitySpecNonNullSegmentGranularityAndNullQueryGranular final Builder builder = new Builder( DATA_SOURCE, segmentCacheManagerFactory, - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); // day segmentGranularity @@ -1015,8 +1005,7 @@ public void testWithGranularitySpecNonNullQueryGranularityAndNullSegmentGranular final Builder builder = new Builder( DATA_SOURCE, segmentCacheManagerFactory, - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); // day queryGranularity @@ -1070,8 +1059,7 @@ public void testWithGranularitySpecNonNullQueryGranularityAndNonNullSegmentGranu final Builder builder = new Builder( DATA_SOURCE, segmentCacheManagerFactory, - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); // day segmentGranularity and day queryGranularity @@ -1110,8 +1098,7 @@ public void testWithGranularitySpecNullQueryGranularityAndNullSegmentGranularity final Builder builder = new Builder( DATA_SOURCE, segmentCacheManagerFactory, - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); final CompactionTask compactionTask1 = builder @@ -1164,8 +1151,7 @@ public void testCompactThenAppend() throws Exception final Builder builder = new Builder( DATA_SOURCE, segmentCacheManagerFactory, - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); final CompactionTask compactionTask = builder @@ -1227,8 +1213,7 @@ public void testPartialIntervalCompactWithFinerSegmentGranularityThenFullInterva final Builder builder = new Builder( DATA_SOURCE, segmentCacheManagerFactory, - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); // Setup partial compaction: @@ -1384,8 +1369,7 @@ public void testCompactDatasourceOverIntervalWithOnlyTombstones() throws Excepti final Builder builder = new Builder( DATA_SOURCE, segmentCacheManagerFactory, - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); // Setup partial interval compaction: @@ -1493,8 +1477,7 @@ public void testPartialIntervalCompactWithFinerSegmentGranularityThenFullInterva final Builder builder = new Builder( DATA_SOURCE, segmentCacheManagerFactory, - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); final Interval partialInterval = Intervals.of("2014-01-01T01:00:00/2014-01-01T02:00:00"); @@ -1566,8 +1549,7 @@ public void testRunIndexAndCompactForSameSegmentAtTheSameTime() throws Exception final Builder builder = new Builder( DATA_SOURCE, segmentCacheManagerFactory, - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); final CompactionTask compactionTask = builder @@ -1622,8 +1604,7 @@ public void testRunIndexAndCompactForSameSegmentAtTheSameTime2() throws Exceptio final Builder builder = new Builder( DATA_SOURCE, segmentCacheManagerFactory, - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); final CompactionTask compactionTask = builder @@ -1714,8 +1695,7 @@ public void testRunWithSpatialDimensions() throws Exception final Builder builder = new Builder( DATA_SOURCE, segmentCacheManagerFactory, - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); final CompactionTask compactionTask = builder @@ -1847,8 +1827,7 @@ public void testRunWithAutoCastDimensions() throws Exception final Builder builder = new Builder( DATA_SOURCE, segmentCacheManagerFactory, - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); final CompactionTask compactionTask = builder diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index 1ccc00017357..4b0e17e33709 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -401,8 +401,7 @@ public void testCreateCompactionTaskWithGranularitySpec() final Builder builder = new Builder( DATA_SOURCE, segmentCacheManagerFactory, - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); builder.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); builder.tuningConfig(createTuningConfig()); @@ -412,8 +411,7 @@ public void testCreateCompactionTaskWithGranularitySpec() final Builder builder2 = new Builder( DATA_SOURCE, segmentCacheManagerFactory, - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); builder2.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); builder2.tuningConfig(createTuningConfig()); @@ -431,8 +429,7 @@ public void testCompactionTaskEmitter() final Builder builder = new Builder( DATA_SOURCE, segmentCacheManagerFactory, - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); builder.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); builder.tuningConfig(createTuningConfig()); @@ -452,8 +449,7 @@ public void testCreateCompactionTaskWithConflictingGranularitySpecAndSegmentGran { final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY, - null + segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); builder.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); builder.tuningConfig(createTuningConfig()); @@ -483,8 +479,7 @@ public void testCreateCompactionTaskWithTransformSpec() new ClientCompactionTaskTransformSpec(new SelectorDimFilter("dim1", "foo", null)); final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY, - null + segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); builder.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); builder.tuningConfig(createTuningConfig()); @@ -503,8 +498,7 @@ public void testCreateCompactionTaskWithMetricsSpec() final Builder builder = new Builder( DATA_SOURCE, segmentCacheManagerFactory, - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); builder.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); builder.tuningConfig(createTuningConfig()); @@ -522,8 +516,7 @@ public void testCreateCompactionTaskWithNullSegmentGranularityInGranularitySpecA final Builder builder = new Builder( DATA_SOURCE, segmentCacheManagerFactory, - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); builder.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); builder.tuningConfig(createTuningConfig()); @@ -552,8 +545,7 @@ public void testCreateCompactionTaskWithSameGranularitySpecAndSegmentGranularity final Builder builder = new Builder( DATA_SOURCE, segmentCacheManagerFactory, - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); builder.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); builder.tuningConfig(createTuningConfig()); @@ -569,8 +561,7 @@ public void testSerdeWithInterval() throws IOException final Builder builder = new Builder( DATA_SOURCE, segmentCacheManagerFactory, - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); final CompactionTask task = builder .inputSpec( @@ -591,8 +582,7 @@ public void testSerdeWithSegments() throws IOException final Builder builder = new Builder( DATA_SOURCE, segmentCacheManagerFactory, - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); final CompactionTask task = builder .segments(SEGMENTS) @@ -611,8 +601,7 @@ public void testSerdeWithDimensions() throws IOException final Builder builder = new Builder( DATA_SOURCE, segmentCacheManagerFactory, - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); final CompactionTask task = builder @@ -695,8 +684,7 @@ public void testSerdeWithOldTuningConfigSuccessfullyDeserializeToNewOne() throws final Builder builder = new Builder( DATA_SOURCE, segmentCacheManagerFactory, - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); final CompactionTask expectedFromJson = builder @@ -717,8 +705,7 @@ public void testInputSourceResources() final Builder builder = new Builder( DATA_SOURCE, segmentCacheManagerFactory, - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); final CompactionTask task = builder .inputSpec( @@ -1492,8 +1479,7 @@ public void testEmptyInterval() final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY, - null + segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); @SuppressWarnings("unused") @@ -1847,8 +1833,7 @@ public void testGetDefaultLookupLoadingSpec() { final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY, - null + segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); final CompactionTask task = builder .interval(Intervals.of("2000-01-01/2000-01-02")) @@ -1861,8 +1846,7 @@ public void testGetDefaultLookupLoadingSpecWithTransformSpec() { final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, RETRY_POLICY_FACTORY, - null + segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); final CompactionTask task = builder .interval(Intervals.of("2000-01-01/2000-01-02")) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java index 6f591e5ee1f7..a14d11d6f784 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java @@ -242,8 +242,7 @@ private Builder newCompactionTaskBuilder() return new Builder( DATASOURCE, getSegmentCacheManagerFactory(), - RETRY_POLICY_FACTORY, - null + RETRY_POLICY_FACTORY ); } } diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java index e2b18fc05bfb..5e5f0f527526 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java @@ -25,11 +25,15 @@ import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.server.coordinator.CompactionConfigValidationResult; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.List; import java.util.Objects; @@ -96,35 +100,33 @@ public static CompactionConfigValidationResult validateCompactionConfig( /** * Checks if the provided compaction config is supported by MSQ. The following configs aren't supported: *
        - *
      • finalizeAggregations set to false in context.
      • *
      • partitionsSpec of type HashedParititionsSpec.
      • *
      • maxTotalRows in DynamicPartitionsSpec.
      • - *
      • rollup set to false in granularitySpec when metricsSpec is specified.
      • + *
      • rollup set to false in granularitySpec when metricsSpec is specified. Null is treated as true.
      • + *
      • queryGranularity set to ALL in granularitySpec.
      • *
      */ private static CompactionConfigValidationResult msqEngineSupportsCompactionConfig(DataSourceCompactionConfig newConfig) { + List validationResults = new ArrayList<>(); if (newConfig.getTuningConfig() != null) { - CompactionConfigValidationResult partitionSpecValidationResult = - validatePartitionsSpecForMsq(newConfig.getTuningConfig().getPartitionsSpec()); - if (!partitionSpecValidationResult.isValid()) { - return partitionSpecValidationResult; - } + validationResults.add(validatePartitionsSpecForMsq(newConfig.getTuningConfig().getPartitionsSpec())); } if (newConfig.getGranularitySpec() != null) { - CompactionConfigValidationResult rollupValidationResult = validateRollupForMsq( + validationResults.add(validateRollupForMsq( newConfig.getMetricsSpec(), newConfig.getGranularitySpec().isRollup() - ); - if (!rollupValidationResult.isValid()) { - return rollupValidationResult; - } + )); + validationResults.add(validateQueryGranularityForMsq(newConfig.getGranularitySpec().getQueryGranularity())); } - return new CompactionConfigValidationResult(true, null); + return validationResults.stream() + .filter(result -> !result.isValid()) + .findFirst() + .orElse(new CompactionConfigValidationResult(true, null)); } /** - * Validte that partitionSpec is either 'dynamic` or 'range', and if 'dynamic', ensure maxTotalRows is null. + * Validate that partitionSpec is either 'dynamic` or 'range', and if 'dynamic', ensure 'maxTotalRows' is null. */ public static CompactionConfigValidationResult validatePartitionsSpecForMsq(PartitionsSpec partitionsSpec) { @@ -166,12 +168,15 @@ public static CompactionConfigValidationResult validateRollupForMsq( } /** - * This class copies over MSQ context parameters from the MSQ extension. This is required to validate the submitted - * compaction config at the coordinator. The values used here should be kept in sync with those in - * {@link org.apache.druid.msq.util.MultiStageQueryContext} + * Validate query granularity is not set to ALL. */ - public static class MSQContext - { - public static final String CTX_FINALIZE_AGGREGATIONS = "finalizeAggregations"; + public static CompactionConfigValidationResult validateQueryGranularityForMsq(Granularity queryGranularity){ + if(queryGranularity!=null && queryGranularity.equals(Granularities.ALL)) { + return new CompactionConfigValidationResult( + false, + "queryGranularity[ALL] in granularitySpec not supported for MSQ engine" + ); + } + return new CompactionConfigValidationResult(true, null); } } diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientMsqContext.java b/server/src/main/java/org/apache/druid/client/indexing/ClientMsqContext.java new file mode 100644 index 000000000000..eee6fa7fa71b --- /dev/null +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientMsqContext.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.client.indexing; + +/** + * This class copies over MSQ context parameters from the MSQ extension. This is required to validate the submitted + * compaction config at the coordinator. The values used here should be kept in sync with those in + * {@link org.apache.druid.msq.util.MultiStageQueryContext} + */ +public class ClientMsqContext +{ + public static final String CTX_MAX_NUM_TASKS = "maxNumTasks"; + public static final int DEFAULT_MAX_NUM_TASKS = 2; +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CompactionConfigValidationResult.java b/server/src/main/java/org/apache/druid/server/coordinator/CompactionConfigValidationResult.java index e412fa40c36e..88eaa3e923a3 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CompactionConfigValidationResult.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CompactionConfigValidationResult.java @@ -29,7 +29,7 @@ public class CompactionConfigValidationResult public CompactionConfigValidationResult(boolean valid, String format, Object... args) { this.valid = valid; - this.reason = StringUtils.format(format, args); + this.reason = format == null ? null : StringUtils.format(format, args); } public boolean isValid() diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index 1587df351d46..bba76814af57 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java @@ -27,11 +27,13 @@ import com.google.inject.Inject; import org.apache.druid.client.indexing.ClientCompactionIOConfig; import org.apache.druid.client.indexing.ClientCompactionIntervalSpec; +import org.apache.druid.client.indexing.ClientCompactionRunnerInfo; import org.apache.druid.client.indexing.ClientCompactionTaskDimensionsSpec; import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; import org.apache.druid.client.indexing.ClientCompactionTaskQuery; import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; import org.apache.druid.client.indexing.ClientCompactionTaskTransformSpec; +import org.apache.druid.client.indexing.ClientMsqContext; import org.apache.druid.client.indexing.ClientTaskQuery; import org.apache.druid.client.indexing.TaskPayloadResponse; import org.apache.druid.common.guava.FutureUtils; @@ -44,10 +46,10 @@ import org.apache.druid.java.util.common.granularity.GranularityType; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.metadata.LockFilterPolicy; +import org.apache.druid.query.QueryContext; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; -import org.apache.druid.client.indexing.ClientCompactionRunnerInfo; import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; @@ -89,7 +91,9 @@ public class CompactSegments implements CoordinatorCustomDuty private static final Predicate IS_COMPACTION_TASK = status -> null != status && COMPACTION_TASK_TYPE.equals(status.getType()); - private static final int MAX_TASK_SLOTS_FOR_MSQ_COMPACTION = 8; + // An artificial limit imposed just to avoid taking up too many compaction task slots for a single MSQ compaction + // task. Can be updated if needed. + private static final int MAX_TASK_SLOTS_FOR_MSQ_COMPACTION_TASK = 8; private final CompactionSegmentSearchPolicy policy; private final OverlordClient overlordClient; @@ -171,10 +175,17 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) final Interval interval = compactionTaskQuery.getIoConfig().getInputSpec().getInterval(); intervalsToSkipCompaction.computeIfAbsent(status.getDataSource(), k -> new ArrayList<>()) .add(interval); - - busyCompactionTaskSlots += findMaxNumTaskSlotsUsedByOneNativeCompactionTask( - compactionTaskQuery.getTuningConfig() - ); + // Note: The default compactionRunnerType used here should match the default runner used in CompactionTask when + // no runner is provided there. + CompactionEngine compactionRunnerType = compactionTaskQuery.getCompactionRunner() == null + ? CompactionEngine.NATIVE + : compactionTaskQuery.getCompactionRunner().getType(); + if (compactionRunnerType == CompactionEngine.NATIVE) { + busyCompactionTaskSlots += + findMaxNumTaskSlotsUsedByOneNativeCompactionTask(compactionTaskQuery.getTuningConfig()); + } else { + busyCompactionTaskSlots += findMaxNumTaskSlotsUsedByOneMsqCompactionTask(compactionTaskQuery.getContext()); + } } // Skip all the intervals locked by higher priority tasks for each datasource @@ -201,7 +212,8 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) compactionConfigs, currentRunAutoCompactionSnapshotBuilders, availableCompactionTaskSlots, - iterator + iterator, + dynamicConfig.getEngine() ); final CoordinatorRunStats stats = params.getCoordinatorStats(); @@ -284,11 +296,13 @@ private Map> getLockedIntervals( } /** - * Returns the maximum number of task slots used by one compaction task at any time when the task is issued with - * the given tuningConfig. + * Returns the maximum number of task slots used by one native compaction task at any time when the task is + * issued with the given tuningConfig. */ @VisibleForTesting - static int findMaxNumTaskSlotsUsedByOneNativeCompactionTask(@Nullable ClientCompactionTaskQueryTuningConfig tuningConfig) + static int findMaxNumTaskSlotsUsedByOneNativeCompactionTask( + @Nullable ClientCompactionTaskQueryTuningConfig tuningConfig + ) { if (isParallelMode(tuningConfig)) { @Nullable @@ -300,6 +314,16 @@ static int findMaxNumTaskSlotsUsedByOneNativeCompactionTask(@Nullable ClientComp } } + /** + * Returns the maximum number of task slots used by one MSQ compaction task at any time when the task is + * issued with the given context. + */ + static int findMaxNumTaskSlotsUsedByOneMsqCompactionTask(@Nullable Map context) + { + return QueryContext.of(context).getInt(ClientMsqContext.CTX_MAX_NUM_TASKS, ClientMsqContext.DEFAULT_MAX_NUM_TASKS); + } + + /** * Returns true if the compaction task can run in the parallel mode with the given tuningConfig. * This method should be synchronized with ParallelIndexSupervisorTask.isParallelMode(InputSource, ParallelIndexTuningConfig). @@ -358,7 +382,8 @@ private int submitCompactionTasks( Map compactionConfigs, Map currentRunAutoCompactionSnapshotBuilders, int numAvailableCompactionTaskSlots, - CompactionSegmentIterator iterator + CompactionSegmentIterator iterator, + CompactionEngine defaultEngine ) { if (numAvailableCompactionTaskSlots <= 0) { @@ -468,22 +493,22 @@ private int submitCompactionTasks( } } - final CompactionEngine compactionEngine = config.getEngine(); + final CompactionEngine compactionEngine = config.getEngine() == null? defaultEngine: config.getEngine(); final Map autoCompactionContext = newAutoCompactionContext(config.getTaskContext()); int slotsRequiredForCurrentTask; - final String maxNumTasksContextParam = "maxNumTasks"; if (compactionEngine == CompactionEngine.MSQ) { - if (autoCompactionContext.containsKey(maxNumTasksContextParam)) { - slotsRequiredForCurrentTask = (int) autoCompactionContext.get(maxNumTasksContextParam); + if (autoCompactionContext.containsKey(ClientMsqContext.CTX_MAX_NUM_TASKS)) { + slotsRequiredForCurrentTask = (int) autoCompactionContext.get(ClientMsqContext.CTX_MAX_NUM_TASKS); } else { // Since MSQ needs all task slots for the calculated #tasks to be available upfront, allot all available - // compaction slots to current compaction task to avoid stalling. Setting "taskAssignment" to "auto" has the - // problem of not being able to determine the actual count, which is required for subsequent tasks. - slotsRequiredForCurrentTask = numAvailableCompactionTaskSlots > MAX_TASK_SLOTS_FOR_MSQ_COMPACTION - ? MAX_TASK_SLOTS_FOR_MSQ_COMPACTION - : numAvailableCompactionTaskSlots; - autoCompactionContext.put(maxNumTasksContextParam, slotsRequiredForCurrentTask); + // compaction slots (upto a max of MAX_TASK_SLOTS_FOR_MSQ_COMPACTION) to current compaction task to avoid + // stalling. Setting "taskAssignment" to "auto" has the problem of not being able to determine the actual + // count, which is required for subsequent tasks. + slotsRequiredForCurrentTask = Math.min(numAvailableCompactionTaskSlots, + MAX_TASK_SLOTS_FOR_MSQ_COMPACTION_TASK + ); + autoCompactionContext.put(ClientMsqContext.CTX_MAX_NUM_TASKS, slotsRequiredForCurrentTask); } } else { slotsRequiredForCurrentTask = findMaxNumTaskSlotsUsedByOneNativeCompactionTask(config.getTuningConfig()); diff --git a/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java index 564567cd9ac7..e54ede41baf4 100644 --- a/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java +++ b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java @@ -26,16 +26,16 @@ import org.apache.druid.audit.AuditEntry; import org.apache.druid.audit.AuditInfo; import org.apache.druid.audit.AuditManager; +import org.apache.druid.client.indexing.ClientCompactionRunnerInfo; import org.apache.druid.common.config.ConfigManager.SetResult; import org.apache.druid.error.InvalidInput; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.client.indexing.ClientCompactionRunnerInfo; +import org.apache.druid.server.coordinator.CompactionConfigValidationResult; import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfigHistory; -import org.apache.druid.server.coordinator.CompactionConfigValidationResult; import org.apache.druid.server.http.security.ConfigResourceFilter; import org.apache.druid.server.security.AuthorizationUtils; import org.joda.time.Interval; diff --git a/server/src/test/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfoTest.java b/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfoTest.java similarity index 53% rename from server/src/test/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfoTest.java rename to server/src/test/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfoTest.java index e8e38167ca10..43f8ee4c0a12 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/ClientCompactionRunnerInfoTest.java +++ b/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfoTest.java @@ -17,11 +17,9 @@ * under the License. */ -package org.apache.druid.server.coordinator; +package org.apache.druid.client.indexing; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import org.apache.druid.client.indexing.ClientCompactionRunnerInfo; import org.apache.druid.data.input.SegmentsSplitHintSpec; import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; @@ -29,15 +27,21 @@ import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.java.util.common.HumanReadableBytes; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.data.CompressionFactory; import org.apache.druid.segment.data.CompressionStrategy; import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; +import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; +import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; import org.joda.time.Duration; import org.joda.time.Period; -import org.junit.Assert; import org.junit.Test; +import javax.annotation.Nullable; import java.util.Collections; import java.util.Map; @@ -49,55 +53,99 @@ public class ClientCompactionRunnerInfoTest @Test public void testHashedPartitionsSpecs() { - assertFalse( - ClientCompactionRunnerInfo.validateCompactionConfig( - createCompactionConfig(new HashedPartitionsSpec(100, null, null), Collections.emptyMap()), CompactionEngine.NATIVE - ).isValid() + DataSourceCompactionConfig compactionConfig = createCompactionConfig( + new HashedPartitionsSpec(100, null, null), + Collections.emptyMap(), + null, + null ); + assertFalse(ClientCompactionRunnerInfo.validateCompactionConfig(compactionConfig, CompactionEngine.NATIVE) + .isValid()); } @Test public void testrInvalidDynamicPartitionsSpecs() { - assertFalse( - ClientCompactionRunnerInfo.validateCompactionConfig( - createCompactionConfig(new DynamicPartitionsSpec(100, 100L), Collections.emptyMap()), CompactionEngine.NATIVE - ).isValid() + DataSourceCompactionConfig compactionConfig = createCompactionConfig( + new DynamicPartitionsSpec(100, 100L), + Collections.emptyMap(), + null, + null ); + assertFalse(ClientCompactionRunnerInfo.validateCompactionConfig(compactionConfig, CompactionEngine.NATIVE) + .isValid()); } @Test public void testDynamicPartitionsSpecs() { - assertTrue(ClientCompactionRunnerInfo.validateCompactionConfig( - createCompactionConfig(new DynamicPartitionsSpec(100, null), Collections.emptyMap()), CompactionEngine.NATIVE - ).isValid()); + DataSourceCompactionConfig compactionConfig = createCompactionConfig( + new DynamicPartitionsSpec(100, null), + Collections.emptyMap(), + null, + null + ); + assertTrue(ClientCompactionRunnerInfo.validateCompactionConfig(compactionConfig, CompactionEngine.NATIVE) + .isValid()); } @Test public void testDimensionRangePartitionsSpecs() { - assertTrue(ClientCompactionRunnerInfo.validateCompactionConfig( - createCompactionConfig( - new DimensionRangePartitionsSpec(100, null, ImmutableList.of("partitionDim"), false), - Collections.emptyMap() - ), CompactionEngine.NATIVE - ).isValid()); + DataSourceCompactionConfig compactionConfig = createCompactionConfig( + new DimensionRangePartitionsSpec(100, null, ImmutableList.of("partitionDim"), false), + Collections.emptyMap(), + null, + null + ); + assertTrue(ClientCompactionRunnerInfo.validateCompactionConfig(compactionConfig, CompactionEngine.NATIVE) + .isValid()); + } + + @Test + public void testQueryGranularityAll() + { + DataSourceCompactionConfig compactionConfig = createCompactionConfig( + new DynamicPartitionsSpec(3, null), + Collections.emptyMap(), + new UserCompactionTaskGranularityConfig(Granularities.ALL, Granularities.ALL, false), + null + ); + assertFalse(ClientCompactionRunnerInfo.validateCompactionConfig(compactionConfig, CompactionEngine.NATIVE) + .isValid()); } @Test - public void testWithFinalizeAggregationsFalse() + public void testRollupFalseWithMetricsSpec() { DataSourceCompactionConfig compactionConfig = createCompactionConfig( new DynamicPartitionsSpec(3, null), - ImmutableMap.of(ClientCompactionRunnerInfo.MSQContext.CTX_FINALIZE_AGGREGATIONS, false) + Collections.emptyMap(), + new UserCompactionTaskGranularityConfig(null, null, false), + new AggregatorFactory[]{new LongSumAggregatorFactory("sum", "sum")} ); - Assert.assertFalse(ClientCompactionRunnerInfo.validateCompactionConfig(compactionConfig, CompactionEngine.NATIVE).isValid()); + assertFalse(ClientCompactionRunnerInfo.validateCompactionConfig(compactionConfig, CompactionEngine.NATIVE) + .isValid()); + } + + @Test + public void testRollupNullWithMetricsSpec() + { + DataSourceCompactionConfig compactionConfig = createCompactionConfig( + new DynamicPartitionsSpec(3, null), + Collections.emptyMap(), + new UserCompactionTaskGranularityConfig(null, null, null), + new AggregatorFactory[]{new LongSumAggregatorFactory("sum", "sum")} + ); + assertTrue(ClientCompactionRunnerInfo.validateCompactionConfig(compactionConfig, CompactionEngine.NATIVE) + .isValid()); } private static DataSourceCompactionConfig createCompactionConfig( PartitionsSpec partitionsSpec, - Map context + Map context, + @Nullable UserCompactionTaskGranularityConfig granularitySpec, + @Nullable AggregatorFactory[] metricsSpec ) { final DataSourceCompactionConfig config = new DataSourceCompactionConfig( @@ -107,9 +155,9 @@ private static DataSourceCompactionConfig createCompactionConfig( 10000, new Period(3600), createTuningConfig(partitionsSpec), + granularitySpec, null, - null, - null, + metricsSpec, null, null, CompactionEngine.MSQ, diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java index 048bd3231b7b..62c2130b2a2c 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java @@ -42,7 +42,6 @@ import org.apache.druid.client.indexing.TaskPayloadResponse; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.RunnerTaskState; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskState; From 3f81cf135192c993166e66c3243cdff450003396 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Thu, 27 Jun 2024 08:44:57 +0530 Subject: [PATCH 42/54] Fix style, coverage and tests. --- .../msq/indexing/MSQCompactionRunnerTest.java | 74 +++++++++++-------- .../druid/indexer/CompactionEngineTest.java | 44 +++++++++++ .../indexing/ClientCompactionRunnerInfo.java | 5 +- .../coordinator/duty/CompactSegments.java | 2 +- 4 files changed, 92 insertions(+), 33 deletions(-) create mode 100644 processing/src/test/java/org/apache/druid/indexer/CompactionEngineTest.java diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java index 190b0602f821..17c30f2aec4a 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java @@ -23,8 +23,8 @@ import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; +import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; import org.apache.druid.client.indexing.ClientCompactionTaskTransformSpec; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.impl.DimensionSchema; @@ -41,6 +41,7 @@ import org.apache.druid.indexing.common.task.CompactionTask; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.GranularityType; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination; @@ -81,15 +82,14 @@ public class MSQCompactionRunnerTest private static final GranularityType SEGMENT_GRANULARITY = GranularityType.HOUR; private static final GranularityType QUERY_GRANULARITY = GranularityType.HOUR; private static List PARTITION_DIMENSIONS; - private static List SORT_ORDER_DIMENSIONS; private static final StringDimensionSchema DIM1 = new StringDimensionSchema( "string_dim", null, null ); - private static final LongDimensionSchema DIM2 = new LongDimensionSchema("long_dim"); - private static final List DIMENSIONS = ImmutableList.of(DIM1, DIM2); + private static final LongDimensionSchema longDimensionSchema = new LongDimensionSchema("long_dim"); + private static final List DIMENSIONS = ImmutableList.of(DIM1, longDimensionSchema); private static final ObjectMapper JSON_MAPPER = new DefaultObjectMapper(); private static final AggregatorFactory AGG1 = new CountAggregatorFactory("agg_0"); private static final AggregatorFactory AGG2 = new LongSumAggregatorFactory("agg_1", "long_dim_1"); @@ -109,10 +109,6 @@ public static void setupClass() PARTITION_DIMENSIONS = Collections.singletonList(stringDimensionSchema.getName()); - final LongDimensionSchema longDimensionSchema = new LongDimensionSchema("long_dim"); - SORT_ORDER_DIMENSIONS = Collections.singletonList(longDimensionSchema.getName()); - - JSON_MAPPER.setInjectableValues(new InjectableValues.Std().addValue( ExprMacroTable.class, LookupEnabledTestExprMacroTable.INSTANCE @@ -120,67 +116,79 @@ public static void setupClass() } @Test - public void testHashPartitionsSpec() throws JsonProcessingException + public void testHashPartitionsSpec() { CompactionTask compactionTask = createCompactionTask( new HashedPartitionsSpec(3, null, ImmutableList.of("dummy")), null, - Collections.emptyMap() + Collections.emptyMap(), + null, + null ); Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); } @Test - public void testDimensionRangePartitionsSpec() throws JsonProcessingException + public void testDimensionRangePartitionsSpec() { CompactionTask compactionTask = createCompactionTask( new DimensionRangePartitionsSpec(TARGET_ROWS_PER_SEGMENT, null, PARTITION_DIMENSIONS, false), null, - Collections.emptyMap() + Collections.emptyMap(), + null, + null ); Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); } @Test - public void testInvalidDynamicPartitionsSpec() throws JsonProcessingException + public void testInvalidDynamicPartitionsSpec() { CompactionTask compactionTask = createCompactionTask( new DynamicPartitionsSpec(3, 3L), null, - Collections.emptyMap() + Collections.emptyMap(), + null, + null ); Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); } @Test - public void testDynamicPartitionsSpec() throws JsonProcessingException + public void testDynamicPartitionsSpec() { CompactionTask compactionTask = createCompactionTask( new DynamicPartitionsSpec(3, null), null, - Collections.emptyMap() + Collections.emptyMap(), + null, + null ); Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); } @Test - public void testWithWorkerStrategyAuto() throws JsonProcessingException + public void testQueryGranularityAll() { CompactionTask compactionTask = createCompactionTask( new DynamicPartitionsSpec(3, null), null, - ImmutableMap.of(MultiStageQueryContext.CTX_TASK_ASSIGNMENT_STRATEGY, WorkerAssignmentStrategy.AUTO.toString()) + Collections.emptyMap(), + new ClientCompactionTaskGranularitySpec(null, Granularities.ALL, null), + null ); Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); } @Test - public void testWithFinalizeAggregationsFalse() throws JsonProcessingException + public void testRollupFalseWithMetricsSpec() { CompactionTask compactionTask = createCompactionTask( new DynamicPartitionsSpec(3, null), null, - ImmutableMap.of(MultiStageQueryContext.CTX_FINALIZE_AGGREGATIONS, false) + Collections.emptyMap(), + new ClientCompactionTaskGranularitySpec(null, null, false), + AGGREGATORS.toArray(new AggregatorFactory[0]) ); Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); } @@ -188,7 +196,7 @@ public void testWithFinalizeAggregationsFalse() throws JsonProcessingException @Test public void testRunCompactionTasksWithEmptyTaskList() throws Exception { - CompactionTask compactionTask = createCompactionTask(null, null, Collections.emptyMap()); + CompactionTask compactionTask = createCompactionTask(null, null, Collections.emptyMap(), null, null); TaskStatus taskStatus = MSQ_COMPACTION_RUNNER.runCompactionTasks(compactionTask, Collections.emptyMap(), null); Assert.assertTrue(taskStatus.isFailure()); } @@ -201,7 +209,9 @@ public void testMSQControllerTaskSpecWithScan() throws JsonProcessingException CompactionTask taskCreatedWithTransformSpec = createCompactionTask( new DimensionRangePartitionsSpec(TARGET_ROWS_PER_SEGMENT, null, PARTITION_DIMENSIONS, false), dimFilter, - Collections.emptyMap() + Collections.emptyMap(), + null, + null ); DataSchema dataSchema = new DataSchema( @@ -240,7 +250,7 @@ public void testMSQControllerTaskSpecWithScan() throws JsonProcessingException new DataSourceMSQDestination( DATA_SOURCE, SEGMENT_GRANULARITY.getDefaultGranularity(), - SORT_ORDER_DIMENSIONS, + null, Collections.singletonList(COMPACTION_INTERVAL) ), actualMSQSpec.getDestination() @@ -266,7 +276,9 @@ public void testMSQControllerTaskSpecWithAggregators() throws JsonProcessingExce CompactionTask taskCreatedWithTransformSpec = createCompactionTask( new DimensionRangePartitionsSpec(TARGET_ROWS_PER_SEGMENT, null, PARTITION_DIMENSIONS, false), dimFilter, - Collections.emptyMap() + Collections.emptyMap(), + null, + null ); DataSchema dataSchema = new DataSchema( @@ -305,7 +317,7 @@ public void testMSQControllerTaskSpecWithAggregators() throws JsonProcessingExce new DataSourceMSQDestination( DATA_SOURCE, SEGMENT_GRANULARITY.getDefaultGranularity(), - SORT_ORDER_DIMENSIONS, + null, Collections.singletonList(COMPACTION_INTERVAL) ), actualMSQSpec.getDestination() @@ -325,10 +337,11 @@ public void testMSQControllerTaskSpecWithAggregators() throws JsonProcessingExce private CompactionTask createCompactionTask( @Nullable PartitionsSpec partitionsSpec, - DimFilter dimFilter, - Map contextParams + @Nullable DimFilter dimFilter, + Map contextParams, + @Nullable ClientCompactionTaskGranularitySpec granularitySpec, + @Nullable AggregatorFactory[] metricsSpec ) - throws JsonProcessingException { ClientCompactionTaskTransformSpec transformSpec = new ClientCompactionTaskTransformSpec(dimFilter); @@ -340,7 +353,6 @@ private CompactionTask createCompactionTask( IndexSpec indexSpec = createIndexSpec(); Map context = new HashMap<>(); - context.put(MultiStageQueryContext.CTX_SORT_ORDER, JSON_MAPPER.writeValueAsString(SORT_ORDER_DIMENSIONS)); context.put(MultiStageQueryContext.CTX_MAX_NUM_TASKS, 2); context.putAll(contextParams); @@ -351,7 +363,9 @@ private CompactionTask createCompactionTask( partitionsSpec == null ? new DynamicPartitionsSpec(100, null) : partitionsSpec )) .transformSpec(transformSpec) - .compactionRunner( new MSQCompactionRunner(JSON_MAPPER, null)) + .granularitySpec(granularitySpec) + .metricsSpec(metricsSpec) + .compactionRunner(new MSQCompactionRunner(JSON_MAPPER, null)) .context(context); return builder.build(); diff --git a/processing/src/test/java/org/apache/druid/indexer/CompactionEngineTest.java b/processing/src/test/java/org/apache/druid/indexer/CompactionEngineTest.java new file mode 100644 index 000000000000..01e9cb8886f6 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/indexer/CompactionEngineTest.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexer; + +import org.junit.Test; + +import static org.junit.Assert.*; + +public class CompactionEngineTest +{ + + @Test + public void testFromString() + { + assertEquals(CompactionEngine.NATIVE, CompactionEngine.fromString("native")); + assertEquals(CompactionEngine.MSQ, CompactionEngine.fromString("msq")); + assertNull(CompactionEngine.fromString(null)); + assertThrows(IllegalArgumentException.class, () -> CompactionEngine.fromString("random")); + } + + @Test + public void testToString() + { + assertEquals("native", CompactionEngine.NATIVE.toString()); + assertEquals("msq", CompactionEngine.MSQ.toString()); + } +} \ No newline at end of file diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java index 5e5f0f527526..e955fba0e2e2 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java @@ -170,8 +170,9 @@ public static CompactionConfigValidationResult validateRollupForMsq( /** * Validate query granularity is not set to ALL. */ - public static CompactionConfigValidationResult validateQueryGranularityForMsq(Granularity queryGranularity){ - if(queryGranularity!=null && queryGranularity.equals(Granularities.ALL)) { + public static CompactionConfigValidationResult validateQueryGranularityForMsq(Granularity queryGranularity) + { + if (queryGranularity != null && queryGranularity.equals(Granularities.ALL)) { return new CompactionConfigValidationResult( false, "queryGranularity[ALL] in granularitySpec not supported for MSQ engine" diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index bba76814af57..a499e608a5a1 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java @@ -493,7 +493,7 @@ private int submitCompactionTasks( } } - final CompactionEngine compactionEngine = config.getEngine() == null? defaultEngine: config.getEngine(); + final CompactionEngine compactionEngine = config.getEngine() == null ? defaultEngine : config.getEngine(); final Map autoCompactionContext = newAutoCompactionContext(config.getTaskContext()); int slotsRequiredForCurrentTask; From 053f2e9269afac64ded01e89df219d718508a0e3 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Thu, 27 Jun 2024 09:03:19 +0530 Subject: [PATCH 43/54] Fix build --- .../java/org/apache/druid/msq/indexing/MSQCompactionRunner.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java index 0ec04fcc09dd..aab55a52dab2 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java @@ -256,6 +256,7 @@ private static MSQTuningConfig buildMSQTuningConfig(CompactionTask compactionTas // We don't consider maxRowsInMemory coming via CompactionTuningConfig since it always sets a default value if no // value specified by user. final int maxRowsInMemory = MultiStageQueryContext.getRowsInMemory(compactionTaskContext); + final Integer maxNumSegments = MultiStageQueryContext.getMaxNumSegments(compactionTaskContext); Integer rowsPerSegment = getRowsPerSegment(compactionTask); @@ -263,6 +264,7 @@ private static MSQTuningConfig buildMSQTuningConfig(CompactionTask compactionTas maxNumWorkers, maxRowsInMemory, rowsPerSegment, + maxNumSegments, compactionTask.getTuningConfig() != null ? compactionTask.getTuningConfig().getIndexSpec() : null ); } From 74273ea0a0eca738f2216196e9ab7fb3ea1bb558 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Thu, 27 Jun 2024 09:16:18 +0530 Subject: [PATCH 44/54] try again --- .../org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java index 17c30f2aec4a..5709b7dbdf37 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java @@ -242,6 +242,7 @@ public void testMSQControllerTaskSpecWithScan() throws JsonProcessingException 1, MultiStageQueryContext.DEFAULT_ROWS_IN_MEMORY, TARGET_ROWS_PER_SEGMENT, + null, createIndexSpec() ), actualMSQSpec.getTuningConfig() @@ -309,6 +310,7 @@ public void testMSQControllerTaskSpecWithAggregators() throws JsonProcessingExce 1, MultiStageQueryContext.DEFAULT_ROWS_IN_MEMORY, TARGET_ROWS_PER_SEGMENT, + null, createIndexSpec() ), actualMSQSpec.getTuningConfig() From af5a3447ed1f3ed52d0c04c6c1d4e7ac7044f2a7 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Thu, 27 Jun 2024 19:18:24 +0530 Subject: [PATCH 45/54] fix style --- .../druid/msq/indexing/MSQCompactionRunner.java | 2 +- .../msq/indexing/MSQCompactionRunnerTest.java | 4 ++-- .../indexing/common/task/CompactionRunner.java | 2 +- .../ClientCompactionTaskQuerySerdeTest.java | 2 +- .../druid/indexer/CompactionEngineTest.java | 17 ++++++++--------- 5 files changed, 13 insertions(+), 14 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java index aab55a52dab2..d275bb6c6c1a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java @@ -26,6 +26,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.inject.Injector; +import org.apache.druid.client.indexing.ClientCompactionRunnerInfo; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.error.InvalidInput; import org.apache.druid.indexer.TaskStatus; @@ -65,7 +66,6 @@ import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; -import org.apache.druid.client.indexing.ClientCompactionRunnerInfo; import org.apache.druid.server.coordinator.CompactionConfigValidationResult; import org.apache.druid.sql.calcite.parser.DruidSqlInsert; import org.apache.druid.sql.calcite.planner.ColumnMapping; diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java index 5709b7dbdf37..157490b2fd8c 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java @@ -88,8 +88,8 @@ public class MSQCompactionRunnerTest null, null ); - private static final LongDimensionSchema longDimensionSchema = new LongDimensionSchema("long_dim"); - private static final List DIMENSIONS = ImmutableList.of(DIM1, longDimensionSchema); + private static final LongDimensionSchema LONG_DIMENSION_SCHEMA = new LongDimensionSchema("long_dim"); + private static final List DIMENSIONS = ImmutableList.of(DIM1, LONG_DIMENSION_SCHEMA); private static final ObjectMapper JSON_MAPPER = new DefaultObjectMapper(); private static final AggregatorFactory AGG1 = new CountAggregatorFactory("agg_0"); private static final AggregatorFactory AGG2 = new LongSumAggregatorFactory("agg_1", "long_dim_1"); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java index 736223cf73b2..8d30a60d04e6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java @@ -21,10 +21,10 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; +import org.apache.druid.client.indexing.ClientCompactionRunnerInfo; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.client.indexing.ClientCompactionRunnerInfo; import org.apache.druid.server.coordinator.CompactionConfigValidationResult; import org.joda.time.Interval; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java index a031b05d3643..3d6c8085c98b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java @@ -28,6 +28,7 @@ import org.apache.druid.client.coordinator.NoopCoordinatorClient; import org.apache.druid.client.indexing.ClientCompactionIOConfig; import org.apache.druid.client.indexing.ClientCompactionIntervalSpec; +import org.apache.druid.client.indexing.ClientCompactionRunnerInfo; import org.apache.druid.client.indexing.ClientCompactionTaskDimensionsSpec; import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; import org.apache.druid.client.indexing.ClientCompactionTaskQuery; @@ -65,7 +66,6 @@ import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; -import org.apache.druid.client.indexing.ClientCompactionRunnerInfo; import org.apache.druid.server.lookup.cache.LookupLoadingSpec; import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.server.security.AuthorizerMapper; diff --git a/processing/src/test/java/org/apache/druid/indexer/CompactionEngineTest.java b/processing/src/test/java/org/apache/druid/indexer/CompactionEngineTest.java index 01e9cb8886f6..b97a6c6a5eab 100644 --- a/processing/src/test/java/org/apache/druid/indexer/CompactionEngineTest.java +++ b/processing/src/test/java/org/apache/druid/indexer/CompactionEngineTest.java @@ -19,26 +19,25 @@ package org.apache.druid.indexer; +import org.junit.Assert; import org.junit.Test; -import static org.junit.Assert.*; - public class CompactionEngineTest { @Test public void testFromString() { - assertEquals(CompactionEngine.NATIVE, CompactionEngine.fromString("native")); - assertEquals(CompactionEngine.MSQ, CompactionEngine.fromString("msq")); - assertNull(CompactionEngine.fromString(null)); - assertThrows(IllegalArgumentException.class, () -> CompactionEngine.fromString("random")); + Assert.assertEquals(CompactionEngine.NATIVE, CompactionEngine.fromString("native")); + Assert.assertEquals(CompactionEngine.MSQ, CompactionEngine.fromString("msq")); + Assert.assertNull(CompactionEngine.fromString(null)); + Assert.assertThrows(IllegalArgumentException.class, () -> CompactionEngine.fromString("random")); } @Test public void testToString() { - assertEquals("native", CompactionEngine.NATIVE.toString()); - assertEquals("msq", CompactionEngine.MSQ.toString()); + Assert.assertEquals("native", CompactionEngine.NATIVE.toString()); + Assert.assertEquals("msq", CompactionEngine.MSQ.toString()); } -} \ No newline at end of file +} From 569ca490c93587359f78b4e859003060ffc3ef5c Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Tue, 2 Jul 2024 08:35:11 +0530 Subject: [PATCH 46/54] Support query granularity for scan queries, support query granularity ALL, validate maxNumTasks. --- .../msq/indexing/MSQCompactionRunner.java | 67 ++++++++++--------- .../indexing/ClientCompactionRunnerInfo.java | 28 +++++--- 2 files changed, 52 insertions(+), 43 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java index d275bb6c6c1a..8b85d73061b2 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java @@ -28,7 +28,6 @@ import com.google.inject.Injector; import org.apache.druid.client.indexing.ClientCompactionRunnerInfo; import org.apache.druid.data.input.impl.DimensionSchema; -import org.apache.druid.error.InvalidInput; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; @@ -54,12 +53,12 @@ import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.expression.TimestampFloorExprMacro; +import org.apache.druid.query.expression.TimestampParseExprMacro; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; -import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; @@ -137,10 +136,8 @@ public CompactionConfigValidationResult validateCompactionTask( compactionTask.getMetricsSpec(), compactionTask.getGranularitySpec().isRollup() )); - validationResults.add(ClientCompactionRunnerInfo.validateQueryGranularityForMsq( - compactionTask.getGranularitySpec().getQueryGranularity() - )); } + validationResults.add(ClientCompactionRunnerInfo.validateMaxNumTasksForMsq(compactionTask.getContext())); return validationResults.stream() .filter(result -> !result.isValid()) .findFirst() @@ -241,18 +238,13 @@ private static DataSourceMSQDestination buildMSQDestination( private static MSQTuningConfig buildMSQTuningConfig(CompactionTask compactionTask, QueryContext compactionTaskContext) { - // Transfer MSQ-related context params, if any, from the compaction context itself. + final int maxNumTasks = MultiStageQueryContext.getMaxNumTasks(compactionTaskContext); - if (maxNumTasks < 2) { - throw InvalidInput.exception( - "MSQ context maxNumTasks [%,d] cannot be less than 2, " - + "since at least 1 controller and 1 worker is necessary.", - maxNumTasks - ); - } + // This parameter is used internally for the number of worker tasks only, so we subtract 1 final int maxNumWorkers = maxNumTasks - 1; + // We don't consider maxRowsInMemory coming via CompactionTuningConfig since it always sets a default value if no // value specified by user. final int maxRowsInMemory = MultiStageQueryContext.getRowsInMemory(compactionTaskContext); @@ -290,6 +282,10 @@ private static RowSignature getRowSignature(DataSchema dataSchema) { RowSignature.Builder rowSignatureBuilder = RowSignature.builder(); rowSignatureBuilder.add(dataSchema.getTimestampSpec().getTimestampColumn(), ColumnType.LONG); + if (!isQueryGranularityEmptyOrNone(dataSchema)) { + // A virtual column for query granularity would have been added. Add corresponding column type. + rowSignatureBuilder.add(TIME_VIRTUAL_COLUMN, ColumnType.LONG); + } for (DimensionSchema dimensionSchema : dataSchema.getDimensionsSpec().getDimensions()) { rowSignatureBuilder.add(dimensionSchema.getName(), ColumnType.fromString(dimensionSchema.getTypeName())); } @@ -315,7 +311,6 @@ private static List getAggregateDimensions(DataSchema dataSchema) dim.getColumnType() )) .collect(Collectors.toList())); - return dimensionSpecs; } @@ -331,10 +326,11 @@ private static ColumnMappings getColumnMappings(DataSchema dataSchema) .map(agg -> new ColumnMapping(agg.getName(), agg.getName())) .collect( Collectors.toList())); - if (isGroupBy(dataSchema)) { - // For group-by queries, time will always be one of the dimension. Since dimensions in groupby aren't allowed to - // have time column as the output name, we map time dimension to a fixed column name in dimensions, and map it - // back to the time column here. + if (isGroupBy(dataSchema) || !isQueryGranularityEmptyOrNone(dataSchema)) { + // For scan queries, a virtual column is created from __time if a custom query granularity is provided. For + // group-by queries, as insert needs __time, it will always be one of the dimensions. Since dimensions in groupby + // aren't allowed to have time column as the output name, we map time dimension to TIME_VIRTUAL_COLUMN in + // dimensions, and map it back to the time column here. columnMappings.add(new ColumnMapping(TIME_VIRTUAL_COLUMN, ColumnHolder.TIME_COLUMN_NAME)); } else { columnMappings.add(new ColumnMapping(ColumnHolder.TIME_COLUMN_NAME, ColumnHolder.TIME_COLUMN_NAME)); @@ -358,6 +354,7 @@ private static Query buildScanQuery(CompactionTask compactionTask, Interval i RowSignature rowSignature = getRowSignature(dataSchema); return new Druids.ScanQueryBuilder().dataSource(dataSchema.getDataSource()) .columns(rowSignature.getColumnNames()) + .virtualColumns(getVirtualColumns(dataSchema, interval)) .columnTypes(rowSignature.getColumnTypes()) .intervals(new MultipleIntervalSegmentSpec(Collections.singletonList(interval))) .legacy(false) @@ -368,6 +365,7 @@ private static Query buildScanQuery(CompactionTask compactionTask, Interval i private static boolean isGroupBy(DataSchema dataSchema) { + // rollup=true with no metrics is a no-op. return dataSchema.getAggregators().length > 0; } @@ -381,29 +379,34 @@ private static boolean isQueryGranularityEmptyOrNone(DataSchema dataSchema) ); } - private static VirtualColumns getVirtualColumns(DataSchema dataSchema) + private static VirtualColumns getVirtualColumns(DataSchema dataSchema, Interval interval) { - VirtualColumns virtualColumns = VirtualColumns.EMPTY; + if (isQueryGranularityEmptyOrNone(dataSchema)) { + return VirtualColumns.EMPTY; + } + String virtualColumnExpr; + if (dataSchema.getGranularitySpec() + .getQueryGranularity() + .equals(Granularities.ALL)) { + virtualColumnExpr = StringUtils.format("timestamp_parse('%s')", interval.getStart()); - if (!isQueryGranularityEmptyOrNone(dataSchema) && !dataSchema.getGranularitySpec() - .getQueryGranularity() - .equals(Granularities.ALL)) { + } else { PeriodGranularity periodQueryGranularity = (PeriodGranularity) dataSchema.getGranularitySpec() .getQueryGranularity(); // Need to create a virtual column for time as that's the only way to support query granularity - VirtualColumn virtualColumn = new ExpressionVirtualColumn( - TIME_VIRTUAL_COLUMN, + virtualColumnExpr = StringUtils.format( "timestamp_floor(\"%s\", '%s')", ColumnHolder.TIME_COLUMN_NAME, periodQueryGranularity.getPeriod().toString() - ), - ColumnType.LONG, - new ExprMacroTable(Collections.singletonList(new TimestampFloorExprMacro())) - ); - virtualColumns = VirtualColumns.create(virtualColumn); + ); } - return virtualColumns; + return VirtualColumns.create(new ExpressionVirtualColumn( + TIME_VIRTUAL_COLUMN, + virtualColumnExpr, + ColumnType.LONG, + new ExprMacroTable(ImmutableList.of(new TimestampFloorExprMacro(), new TimestampParseExprMacro())) + )); } private static Query buildGroupByQuery(CompactionTask compactionTask, Interval interval, DataSchema dataSchema) @@ -412,7 +415,7 @@ private static Query buildGroupByQuery(CompactionTask compactionTask, Interva GroupByQuery.Builder builder = new GroupByQuery.Builder() .setDataSource(new TableDataSource(compactionTask.getDataSource())) - .setVirtualColumns(getVirtualColumns(dataSchema)) + .setVirtualColumns(getVirtualColumns(dataSchema, interval)) .setDimFilter(dimFilter) .setGranularity(new AllGranularity()) .setDimensions(getAggregateDimensions(dataSchema)) diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java index e955fba0e2e2..08655fe520a9 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java @@ -25,8 +25,7 @@ import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; -import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.query.QueryContext; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.server.coordinator.CompactionConfigValidationResult; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; @@ -34,6 +33,7 @@ import javax.annotation.Nullable; import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.Objects; @@ -117,8 +117,8 @@ private static CompactionConfigValidationResult msqEngineSupportsCompactionConfi newConfig.getMetricsSpec(), newConfig.getGranularitySpec().isRollup() )); - validationResults.add(validateQueryGranularityForMsq(newConfig.getGranularitySpec().getQueryGranularity())); } + validationResults.add(validateMaxNumTasksForMsq(newConfig.getTaskContext())); return validationResults.stream() .filter(result -> !result.isValid()) .findFirst() @@ -168,15 +168,21 @@ public static CompactionConfigValidationResult validateRollupForMsq( } /** - * Validate query granularity is not set to ALL. + * Validate maxNumTasks >= 2 in context. + * @param context + * @return */ - public static CompactionConfigValidationResult validateQueryGranularityForMsq(Granularity queryGranularity) - { - if (queryGranularity != null && queryGranularity.equals(Granularities.ALL)) { - return new CompactionConfigValidationResult( - false, - "queryGranularity[ALL] in granularitySpec not supported for MSQ engine" - ); + public static CompactionConfigValidationResult validateMaxNumTasksForMsq(Map context){ + if (context != null) { + int maxNumTasks = QueryContext.of(context) + .getInt(ClientMsqContext.CTX_MAX_NUM_TASKS, ClientMsqContext.DEFAULT_MAX_NUM_TASKS); + if (maxNumTasks < 2) { + return new CompactionConfigValidationResult(false, + "MSQ context maxNumTasks [%,d] cannot be less than 2, " + + "since at least 1 controller and 1 worker is necessary.", + maxNumTasks + ); + } } return new CompactionConfigValidationResult(true, null); } From 01cdabc60a1298c78ccd8136f0588623000593d7 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Tue, 2 Jul 2024 08:37:14 +0530 Subject: [PATCH 47/54] fix style --- .../druid/client/indexing/ClientCompactionRunnerInfo.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java index 08655fe520a9..6faefe4c7690 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java @@ -172,7 +172,8 @@ public static CompactionConfigValidationResult validateRollupForMsq( * @param context * @return */ - public static CompactionConfigValidationResult validateMaxNumTasksForMsq(Map context){ + public static CompactionConfigValidationResult validateMaxNumTasksForMsq(Map context) + { if (context != null) { int maxNumTasks = QueryContext.of(context) .getInt(ClientMsqContext.CTX_MAX_NUM_TASKS, ClientMsqContext.DEFAULT_MAX_NUM_TASKS); From b122cdf6089da2b2776fe60594028a50799958d4 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Tue, 2 Jul 2024 11:08:43 +0530 Subject: [PATCH 48/54] Fix "ALL" granularity deserialization and support dedup when rollup=true without metrics --- .../msq/indexing/MSQCompactionRunner.java | 37 +++++++++++++++---- 1 file changed, 29 insertions(+), 8 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java index 8b85d73061b2..3abcba694885 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java @@ -365,7 +365,12 @@ private static Query buildScanQuery(CompactionTask compactionTask, Interval i private static boolean isGroupBy(DataSchema dataSchema) { - // rollup=true with no metrics is a no-op. + if (dataSchema.getGranularitySpec() != null) { + // If rollup is true without any metrics, all columns are treated as dimensions and + // duplicate rows are removed in line with native compaction. + return dataSchema.getGranularitySpec().isRollup(); + } + // If no rollup specified, decide based on whether metrics are present. return dataSchema.getAggregators().length > 0; } @@ -379,6 +384,10 @@ private static boolean isQueryGranularityEmptyOrNone(DataSchema dataSchema) ); } + /** + * Creates a virtual timestamp column to create a new __time field according to the provided queryGranularity, as + * queryGranularity field itself is mandated to be ALL in MSQControllerTask. + */ private static VirtualColumns getVirtualColumns(DataSchema dataSchema, Interval interval) { if (isQueryGranularityEmptyOrNone(dataSchema)) { @@ -388,12 +397,13 @@ private static VirtualColumns getVirtualColumns(DataSchema dataSchema, Interval if (dataSchema.getGranularitySpec() .getQueryGranularity() .equals(Granularities.ALL)) { + // For ALL query granularity, all records in a segment are assigned the interval start timestamp of the segment. + // It's the same behaviour in native compaction. virtualColumnExpr = StringUtils.format("timestamp_parse('%s')", interval.getStart()); - } else { PeriodGranularity periodQueryGranularity = (PeriodGranularity) dataSchema.getGranularitySpec() .getQueryGranularity(); - // Need to create a virtual column for time as that's the only way to support query granularity + // Round of the __time column according to the required granularity. virtualColumnExpr = StringUtils.format( "timestamp_floor(\"%s\", '%s')", @@ -429,25 +439,36 @@ private static Query buildGroupByQuery(CompactionTask compactionTask, Interva return builder.build(); } + private String serializeGranularity(Granularity granularity, ObjectMapper jsonMapper) throws JsonProcessingException + { + if (granularity != null) { + // AllGranularity by default gets deserialized into {"type": "all"} since there is no custom serialize impl -- as + // is there for PeriodGranularity. Not implementing the serializer itself to avoid things breaking elsewhere. + return granularity.equals(Granularities.ALL) ? "ALL" : jsonMapper.writeValueAsString(granularity); + } + return null; + } + private Map createMSQTaskContext(CompactionTask compactionTask, DataSchema dataSchema) throws JsonProcessingException { Map context = new HashMap<>(compactionTask.getContext()); context.put( DruidSqlInsert.SQL_INSERT_SEGMENT_GRANULARITY, - jsonMapper.writeValueAsString(dataSchema.getGranularitySpec() != null - ? dataSchema.getGranularitySpec() - .getSegmentGranularity() - : DEFAULT_SEGMENT_GRANULARITY) + serializeGranularity(dataSchema.getGranularitySpec() != null + ? dataSchema.getGranularitySpec() + .getSegmentGranularity() + : DEFAULT_SEGMENT_GRANULARITY, jsonMapper) ); if (!isQueryGranularityEmptyOrNone(dataSchema)) { context.put( DruidSqlInsert.SQL_INSERT_QUERY_GRANULARITY, - jsonMapper.writeValueAsString(dataSchema.getGranularitySpec().getQueryGranularity()) + serializeGranularity(dataSchema.getGranularitySpec().getQueryGranularity(), jsonMapper) ); } // Similar to compaction using the native engine, don't finalize aggregations. context.putIfAbsent(MultiStageQueryContext.CTX_FINALIZE_AGGREGATIONS, false); + // Only scalar or array-type dimensions are allowed as grouping keys. context.putIfAbsent(GroupByQueryConfig.CTX_KEY_ENABLE_MULTI_VALUE_UNNESTING, false); return context; } From fa6353da9bca94447b4d94ca7043baaad8d389d4 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Tue, 2 Jul 2024 11:19:41 +0530 Subject: [PATCH 49/54] fix tests --- .../org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java | 2 +- .../druid/client/indexing/ClientCompactionRunnerInfoTest.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java index 157490b2fd8c..a2ceeff3fb30 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java @@ -177,7 +177,7 @@ public void testQueryGranularityAll() new ClientCompactionTaskGranularitySpec(null, Granularities.ALL, null), null ); - Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); + Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); } @Test diff --git a/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfoTest.java b/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfoTest.java index 43f8ee4c0a12..8adbb847d21e 100644 --- a/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfoTest.java +++ b/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfoTest.java @@ -111,7 +111,7 @@ public void testQueryGranularityAll() new UserCompactionTaskGranularityConfig(Granularities.ALL, Granularities.ALL, false), null ); - assertFalse(ClientCompactionRunnerInfo.validateCompactionConfig(compactionConfig, CompactionEngine.NATIVE) + assertTrue(ClientCompactionRunnerInfo.validateCompactionConfig(compactionConfig, CompactionEngine.NATIVE) .isValid()); } From 832c08f544bafe11cbf34c1320f99173bec7b437 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Tue, 9 Jul 2024 23:27:10 +0530 Subject: [PATCH 50/54] Address review comments and add ITs --- .../msq/indexing/MSQCompactionRunner.java | 10 +- .../docker/environment-configs/common | 2 +- .../duty/ITAutoCompactionTest.java | 134 ++++++++++++---- .../indexing/ClientCompactionRunnerInfo.java | 48 ++++-- ...tMsqContext.java => ClientMSQContext.java} | 2 +- .../coordinator/compact/CompactionStatus.java | 7 + .../coordinator/duty/CompactSegments.java | 26 ++-- .../CoordinatorCompactionConfigsResource.java | 10 +- .../ClientCompactionRunnerInfoTest.java | 89 ++++++++--- .../DataSourceCompactionConfigTest.java | 8 +- .../coordinator/duty/CompactSegmentsTest.java | 147 +++++++++++------- ...rdinatorCompactionConfigsResourceTest.java | 114 ++++++++++++++ 12 files changed, 461 insertions(+), 136 deletions(-) rename server/src/main/java/org/apache/druid/client/indexing/{ClientMsqContext.java => ClientMSQContext.java} (97%) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java index 3abcba694885..47f901dce00b 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java @@ -111,13 +111,14 @@ public MSQCompactionRunner(@JacksonInject ObjectMapper jsonMapper, @JacksonInjec /** * Checks if the provided compaction config is supported by MSQ. The same validation is done at - * {@link ClientCompactionRunnerInfo#msqEngineSupportsCompactionConfig} + * {@link ClientCompactionRunnerInfo#MSQEngineSupportsCompactionConfig} * The following configs aren't supported: *
        *
      • partitionsSpec of type HashedParititionsSpec.
      • *
      • maxTotalRows in DynamicPartitionsSpec.
      • *
      • rollup set to false in granularitySpec when metricsSpec is specified. Null is treated as true.
      • *
      • queryGranularity set to ALL in granularitySpec.
      • + *
      • Each metric has output column name same as the input name.
      • *
      */ @Override @@ -127,17 +128,18 @@ public CompactionConfigValidationResult validateCompactionTask( { List validationResults = new ArrayList<>(); if (compactionTask.getTuningConfig() != null) { - validationResults.add(ClientCompactionRunnerInfo.validatePartitionsSpecForMsq( + validationResults.add(ClientCompactionRunnerInfo.validatePartitionsSpecForMSQ( compactionTask.getTuningConfig().getPartitionsSpec()) ); } if (compactionTask.getGranularitySpec() != null) { - validationResults.add(ClientCompactionRunnerInfo.validateRollupForMsq( + validationResults.add(ClientCompactionRunnerInfo.validateRollupForMSQ( compactionTask.getMetricsSpec(), compactionTask.getGranularitySpec().isRollup() )); } - validationResults.add(ClientCompactionRunnerInfo.validateMaxNumTasksForMsq(compactionTask.getContext())); + validationResults.add(ClientCompactionRunnerInfo.validateMaxNumTasksForMSQ(compactionTask.getContext())); + validationResults.add(ClientCompactionRunnerInfo.validateMetricsSpecForMSQ(compactionTask.getMetricsSpec())); return validationResults.stream() .filter(result -> !result.isValid()) .findFirst() diff --git a/integration-tests/docker/environment-configs/common b/integration-tests/docker/environment-configs/common index 3ce06d90ea26..e99ee2acd43c 100644 --- a/integration-tests/docker/environment-configs/common +++ b/integration-tests/docker/environment-configs/common @@ -28,7 +28,7 @@ DRUID_DEP_LIB_DIR=/shared/hadoop_xml:/shared/docker/lib/*:/usr/local/druid/lib/m # Druid configs # If you are making a change in load list below, make the necessary changes in github actions too -druid_extensions_loadList=["mysql-metadata-storage","druid-basic-security","simple-client-sslcontext","druid-testing-tools","druid-lookups-cached-global","druid-histogram","druid-datasketches","druid-parquet-extensions","druid-avro-extensions","druid-protobuf-extensions","druid-orc-extensions","druid-kafka-indexing-service","druid-s3-extensions"] +druid_extensions_loadList=["mysql-metadata-storage","druid-basic-security","simple-client-sslcontext","druid-testing-tools","druid-lookups-cached-global","druid-histogram","druid-datasketches","druid-parquet-extensions","druid-avro-extensions","druid-protobuf-extensions","druid-orc-extensions","druid-kafka-indexing-service","druid-s3-extensions","druid-multi-stage-query"] druid_startup_logging_logProperties=true druid_extensions_directory=/shared/docker/extensions druid_auth_authenticator_basic_authorizerName=basic diff --git a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java index 3abcb4af9822..09988338a023 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java @@ -26,6 +26,7 @@ import org.apache.datasketches.hll.TgtHllType; import org.apache.druid.data.input.MaxSizeSplitHintSpec; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; @@ -75,9 +76,11 @@ import org.joda.time.chrono.ISOChronology; import org.testng.Assert; import org.testng.annotations.BeforeMethod; +import org.testng.annotations.DataProvider; import org.testng.annotations.Guice; import org.testng.annotations.Test; +import javax.annotation.Nullable; import java.io.Closeable; import java.io.IOException; import java.io.InputStream; @@ -106,6 +109,12 @@ public class ITAutoCompactionTest extends AbstractIndexerTest private static final int MAX_ROWS_PER_SEGMENT_COMPACTED = 10000; private static final Period NO_SKIP_OFFSET = Period.seconds(0); + @DataProvider(name = "engine") + public static Object[][] engine() + { + return new Object[][]{{CompactionEngine.NATIVE}, {CompactionEngine.MSQ}}; + } + @Inject protected CompactionResourceTestClient compactionResource; @@ -319,8 +328,8 @@ public void testAutoCompactionRowWithMetricAndRowWithoutMetricShouldPreserveExis } } - @Test - public void testAutoCompactionOnlyRowsWithoutMetricShouldAddNewMetrics() throws Exception + @Test(dataProvider = "engine") + public void testAutoCompactionOnlyRowsWithoutMetricShouldAddNewMetrics(CompactionEngine engine) throws Exception { // added = 31, count = null, sum_added = null loadData(INDEX_TASK_WITHOUT_ROLLUP_FOR_PRESERVE_METRICS); @@ -347,7 +356,8 @@ public void testAutoCompactionOnlyRowsWithoutMetricShouldAddNewMetrics() throws new UserCompactionTaskDimensionsConfig(DimensionsSpec.getDefaultSchemas(ImmutableList.of("language"))), null, new AggregatorFactory[] {new CountAggregatorFactory("count"), new LongSumAggregatorFactory("sum_added", "added")}, - false + false, + engine ); // should now only have 1 row after compaction // added = null, count = 2, sum_added = 62 @@ -456,7 +466,7 @@ public void testAutoCompactionDutySubmitAndVerifyCompaction() throws Exception verifySegmentsCount(4); verifyQuery(INDEX_QUERIES_RESOURCE); - submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, Period.days(1)); + submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, Period.days(1), null); //...compacted into 1 new segment for 1 day. 1 day compacted and 1 day skipped/remains uncompacted. (3 total) forceTriggerAutoCompaction(3); verifyQuery(INDEX_QUERIES_RESOURCE); @@ -474,7 +484,7 @@ public void testAutoCompactionDutySubmitAndVerifyCompaction() throws Exception 0, 1, 1); - submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET); + submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, null); //...compacted into 1 new segment for the remaining one day. 2 day compacted and 0 day uncompacted. (2 total) forceTriggerAutoCompaction(2); verifyQuery(INDEX_QUERIES_RESOURCE); @@ -507,9 +517,9 @@ public void testAutoCompactionDutyCanUpdateCompactionConfig() throws Exception verifyQuery(INDEX_QUERIES_RESOURCE); // Dummy compaction config which will be overwritten - submitCompactionConfig(10000, NO_SKIP_OFFSET); + submitCompactionConfig(10000, NO_SKIP_OFFSET, null); // New compaction config should overwrites the existing compaction config - submitCompactionConfig(1, NO_SKIP_OFFSET); + submitCompactionConfig(1, NO_SKIP_OFFSET, null); LOG.info("Auto compaction test with dynamic partitioning"); @@ -523,7 +533,7 @@ public void testAutoCompactionDutyCanUpdateCompactionConfig() throws Exception LOG.info("Auto compaction test with hash partitioning"); final HashedPartitionsSpec hashedPartitionsSpec = new HashedPartitionsSpec(null, 3, null); - submitCompactionConfig(hashedPartitionsSpec, NO_SKIP_OFFSET, 1, null, null, null, null, false); + submitCompactionConfig(hashedPartitionsSpec, NO_SKIP_OFFSET, 1, null, null, null, null, false, null); // 2 segments published per day after compaction. forceTriggerAutoCompaction(4); verifyQuery(INDEX_QUERIES_RESOURCE); @@ -538,7 +548,7 @@ public void testAutoCompactionDutyCanUpdateCompactionConfig() throws Exception "city", false ); - submitCompactionConfig(rangePartitionsSpec, NO_SKIP_OFFSET, 1, null, null, null, null, false); + submitCompactionConfig(rangePartitionsSpec, NO_SKIP_OFFSET, 1, null, null, null, null, false, null); forceTriggerAutoCompaction(2); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(rangePartitionsSpec, 2); @@ -546,8 +556,8 @@ public void testAutoCompactionDutyCanUpdateCompactionConfig() throws Exception } } - @Test - public void testAutoCompactionDutyCanDeleteCompactionConfig() throws Exception + @Test(dataProvider = "engine") + public void testAutoCompactionDutyCanDeleteCompactionConfig(CompactionEngine engine) throws Exception { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { @@ -557,7 +567,7 @@ public void testAutoCompactionDutyCanDeleteCompactionConfig() throws Exception verifySegmentsCount(4); verifyQuery(INDEX_QUERIES_RESOURCE); - submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET); + submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, engine); deleteCompactionConfig(); // ...should remains unchanged (4 total) @@ -583,7 +593,7 @@ public void testAutoCompactionDutyCanUpdateTaskSlots() throws Exception verifySegmentsCount(4); verifyQuery(INDEX_QUERIES_RESOURCE); - submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET); + submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, null); // ...should remains unchanged (4 total) forceTriggerAutoCompaction(4); verifyQuery(INDEX_QUERIES_RESOURCE); @@ -910,14 +920,14 @@ public void testAutoCompactionDutyWithSegmentGranularityAndMixedVersion() throws verifySegmentsCount(4); verifyQuery(INDEX_QUERIES_RESOURCE); - submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, Period.days(1)); + submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, Period.days(1), null); //...compacted into 1 new segment for 1 day. 1 day compacted and 1 day skipped/remains uncompacted. (3 total) forceTriggerAutoCompaction(3); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(1, MAX_ROWS_PER_SEGMENT_COMPACTED); Granularity newGranularity = Granularities.YEAR; - submitCompactionConfig(1000, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null)); + submitCompactionConfig(1000, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), null); LOG.info("Auto compaction test with YEAR segment granularity"); @@ -949,7 +959,7 @@ public void testAutoCompactionDutyWithSegmentGranularityAndExistingCompactedSegm verifyQuery(INDEX_QUERIES_RESOURCE); // Compacted without SegmentGranularity in auto compaction config - submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET); + submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, null); forceTriggerAutoCompaction(2); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(2, MAX_ROWS_PER_SEGMENT_COMPACTED); @@ -959,7 +969,7 @@ public void testAutoCompactionDutyWithSegmentGranularityAndExistingCompactedSegm // Segments were compacted and already has DAY granularity since it was initially ingested with DAY granularity. // Now set auto compaction with DAY granularity in the granularitySpec Granularity newGranularity = Granularities.DAY; - submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null)); + submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), null); forceTriggerAutoCompaction(2); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(2, MAX_ROWS_PER_SEGMENT_COMPACTED); @@ -981,7 +991,7 @@ public void testAutoCompactionDutyWithSegmentGranularityAndExistingCompactedSegm verifyQuery(INDEX_QUERIES_RESOURCE); // Compacted without SegmentGranularity in auto compaction config - submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET); + submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, null); forceTriggerAutoCompaction(2); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(2, MAX_ROWS_PER_SEGMENT_COMPACTED); @@ -991,7 +1001,7 @@ public void testAutoCompactionDutyWithSegmentGranularityAndExistingCompactedSegm // Segments were compacted and already has DAY granularity since it was initially ingested with DAY granularity. // Now set auto compaction with DAY granularity in the granularitySpec Granularity newGranularity = Granularities.YEAR; - submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null)); + submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), null); forceTriggerAutoCompaction(1); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(1, MAX_ROWS_PER_SEGMENT_COMPACTED); @@ -1585,24 +1595,89 @@ private void verifyQuery(String queryResource, Map keyValueToRep queryHelper.testQueriesFromString(queryResponseTemplate); } - private void submitCompactionConfig(Integer maxRowsPerSegment, Period skipOffsetFromLatest) throws Exception + private void submitCompactionConfig(Integer maxRowsPerSegment, Period skipOffsetFromLatest, CompactionEngine engine) + throws Exception + { + submitCompactionConfig(maxRowsPerSegment, skipOffsetFromLatest, null, engine); + } + + private void submitCompactionConfig( + Integer maxRowsPerSegment, + Period skipOffsetFromLatest, + UserCompactionTaskGranularityConfig granularitySpec, + CompactionEngine engine + ) throws Exception + { + submitCompactionConfig(maxRowsPerSegment, skipOffsetFromLatest, granularitySpec, false, engine); + } + + private void submitCompactionConfig( + Integer maxRowsPerSegment, + Period skipOffsetFromLatest, + UserCompactionTaskGranularityConfig granularitySpec, + boolean dropExisting + ) throws Exception { - submitCompactionConfig(maxRowsPerSegment, skipOffsetFromLatest, null); + submitCompactionConfig(maxRowsPerSegment, skipOffsetFromLatest, granularitySpec, dropExisting, null); } - private void submitCompactionConfig(Integer maxRowsPerSegment, Period skipOffsetFromLatest, UserCompactionTaskGranularityConfig granularitySpec) throws Exception + private void submitCompactionConfig( + Integer maxRowsPerSegment, + Period skipOffsetFromLatest, + UserCompactionTaskGranularityConfig granularitySpec, + boolean dropExisting, + @Nullable CompactionEngine engine + ) throws Exception { - submitCompactionConfig(maxRowsPerSegment, skipOffsetFromLatest, granularitySpec, false); + submitCompactionConfig(maxRowsPerSegment, skipOffsetFromLatest, granularitySpec, null, null, null, dropExisting, + engine + ); } - private void submitCompactionConfig(Integer maxRowsPerSegment, Period skipOffsetFromLatest, UserCompactionTaskGranularityConfig granularitySpec, boolean dropExisting) throws Exception + private void submitCompactionConfig( + Integer maxRowsPerSegment, + Period skipOffsetFromLatest, + UserCompactionTaskGranularityConfig granularitySpec, + UserCompactionTaskDimensionsConfig dimensionsSpec, + UserCompactionTaskTransformConfig transformSpec, + AggregatorFactory[] metricsSpec, + boolean dropExisting + ) throws Exception { - submitCompactionConfig(maxRowsPerSegment, skipOffsetFromLatest, granularitySpec, null, null, null, dropExisting); + submitCompactionConfig( + maxRowsPerSegment, + skipOffsetFromLatest, + granularitySpec, + dimensionsSpec, + transformSpec, + metricsSpec, + dropExisting, + null + ); } - private void submitCompactionConfig(Integer maxRowsPerSegment, Period skipOffsetFromLatest, UserCompactionTaskGranularityConfig granularitySpec, UserCompactionTaskDimensionsConfig dimensionsSpec, UserCompactionTaskTransformConfig transformSpec, AggregatorFactory[] metricsSpec, boolean dropExisting) throws Exception + private void submitCompactionConfig( + Integer maxRowsPerSegment, + Period skipOffsetFromLatest, + UserCompactionTaskGranularityConfig granularitySpec, + UserCompactionTaskDimensionsConfig dimensionsSpec, + UserCompactionTaskTransformConfig transformSpec, + AggregatorFactory[] metricsSpec, + boolean dropExisting, + @Nullable CompactionEngine engine + ) throws Exception { - submitCompactionConfig(new DynamicPartitionsSpec(maxRowsPerSegment, null), skipOffsetFromLatest, 1, granularitySpec, dimensionsSpec, transformSpec, metricsSpec, dropExisting); + submitCompactionConfig( + new DynamicPartitionsSpec(maxRowsPerSegment, null), + skipOffsetFromLatest, + 1, + granularitySpec, + dimensionsSpec, + transformSpec, + metricsSpec, + dropExisting, + engine + ); } private void submitCompactionConfig( @@ -1613,7 +1688,8 @@ private void submitCompactionConfig( UserCompactionTaskDimensionsConfig dimensionsSpec, UserCompactionTaskTransformConfig transformSpec, AggregatorFactory[] metricsSpec, - boolean dropExisting + boolean dropExisting, + @Nullable CompactionEngine engine ) throws Exception { DataSourceCompactionConfig compactionConfig = new DataSourceCompactionConfig( @@ -1648,7 +1724,7 @@ private void submitCompactionConfig( metricsSpec, transformSpec, !dropExisting ? null : new UserCompactionTaskIOConfig(true), - null, + engine, null ); compactionResource.submitCompactionConfig(compactionConfig); diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java index 6faefe4c7690..ed9e22dfaa29 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java @@ -32,6 +32,7 @@ import javax.annotation.Nullable; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Objects; @@ -93,7 +94,7 @@ public static CompactionConfigValidationResult validateCompactionConfig( if (compactionEngine == CompactionEngine.NATIVE) { return new CompactionConfigValidationResult(true, null); } else { - return msqEngineSupportsCompactionConfig(newConfig); + return compactionConfigSupportedByMSQEngine(newConfig); } } @@ -104,21 +105,23 @@ public static CompactionConfigValidationResult validateCompactionConfig( *
    • maxTotalRows in DynamicPartitionsSpec.
    • *
    • rollup set to false in granularitySpec when metricsSpec is specified. Null is treated as true.
    • *
    • queryGranularity set to ALL in granularitySpec.
    • + *
    • Each metric has output column name same as the input name.
    • * */ - private static CompactionConfigValidationResult msqEngineSupportsCompactionConfig(DataSourceCompactionConfig newConfig) + private static CompactionConfigValidationResult compactionConfigSupportedByMSQEngine(DataSourceCompactionConfig newConfig) { List validationResults = new ArrayList<>(); if (newConfig.getTuningConfig() != null) { - validationResults.add(validatePartitionsSpecForMsq(newConfig.getTuningConfig().getPartitionsSpec())); + validationResults.add(validatePartitionsSpecForMSQ(newConfig.getTuningConfig().getPartitionsSpec())); } if (newConfig.getGranularitySpec() != null) { - validationResults.add(validateRollupForMsq( + validationResults.add(validateRollupForMSQ( newConfig.getMetricsSpec(), newConfig.getGranularitySpec().isRollup() )); } - validationResults.add(validateMaxNumTasksForMsq(newConfig.getTaskContext())); + validationResults.add(validateMaxNumTasksForMSQ(newConfig.getTaskContext())); + validationResults.add(validateMetricsSpecForMSQ(newConfig.getMetricsSpec())); return validationResults.stream() .filter(result -> !result.isValid()) .findFirst() @@ -128,7 +131,7 @@ private static CompactionConfigValidationResult msqEngineSupportsCompactionConfi /** * Validate that partitionSpec is either 'dynamic` or 'range', and if 'dynamic', ensure 'maxTotalRows' is null. */ - public static CompactionConfigValidationResult validatePartitionsSpecForMsq(PartitionsSpec partitionsSpec) + public static CompactionConfigValidationResult validatePartitionsSpecForMSQ(PartitionsSpec partitionsSpec) { if (!(partitionsSpec instanceof DimensionRangePartitionsSpec || partitionsSpec instanceof DynamicPartitionsSpec)) { @@ -153,7 +156,7 @@ public static CompactionConfigValidationResult validatePartitionsSpecForMsq(Part /** * Validate rollup is set to false in granularitySpec when metricsSpec is specified. */ - public static CompactionConfigValidationResult validateRollupForMsq( + public static CompactionConfigValidationResult validateRollupForMSQ( AggregatorFactory[] metricsSpec, @Nullable Boolean isRollup ) @@ -169,14 +172,12 @@ public static CompactionConfigValidationResult validateRollupForMsq( /** * Validate maxNumTasks >= 2 in context. - * @param context - * @return */ - public static CompactionConfigValidationResult validateMaxNumTasksForMsq(Map context) + public static CompactionConfigValidationResult validateMaxNumTasksForMSQ(Map context) { if (context != null) { int maxNumTasks = QueryContext.of(context) - .getInt(ClientMsqContext.CTX_MAX_NUM_TASKS, ClientMsqContext.DEFAULT_MAX_NUM_TASKS); + .getInt(ClientMSQContext.CTX_MAX_NUM_TASKS, ClientMSQContext.DEFAULT_MAX_NUM_TASKS); if (maxNumTasks < 2) { return new CompactionConfigValidationResult(false, "MSQ context maxNumTasks [%,d] cannot be less than 2, " @@ -187,4 +188,29 @@ public static CompactionConfigValidationResult validateMaxNumTasksForMsq(Map + !(aggregatorFactory.requiredFields().isEmpty() + || aggregatorFactory.requiredFields().size() == 1 + && aggregatorFactory.requiredFields() + .get(0) + .equals(aggregatorFactory.getName()))) + .findFirst() + .map(aggregatorFactory -> + new CompactionConfigValidationResult( + false, + "Different name[%s] and fieldName(s)[%s] for aggregator unsupported for MSQ engine.", + aggregatorFactory.getName(), + aggregatorFactory.requiredFields() + )).orElse(new CompactionConfigValidationResult(true, null)); + } } diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientMsqContext.java b/server/src/main/java/org/apache/druid/client/indexing/ClientMSQContext.java similarity index 97% rename from server/src/main/java/org/apache/druid/client/indexing/ClientMsqContext.java rename to server/src/main/java/org/apache/druid/client/indexing/ClientMSQContext.java index eee6fa7fa71b..e80b86f65b3f 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientMsqContext.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientMSQContext.java @@ -24,7 +24,7 @@ * compaction config at the coordinator. The values used here should be kept in sync with those in * {@link org.apache.druid.msq.util.MultiStageQueryContext} */ -public class ClientMsqContext +public class ClientMSQContext { public static final String CTX_MAX_NUM_TASKS = "maxNumTasks"; public static final int DEFAULT_MAX_NUM_TASKS = 2; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java index 9a26c98ee515..0a76acc870b5 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java @@ -299,6 +299,13 @@ private CompactionStatus dimensionsSpecIsUpToDate() } } + /** + * Compares combining factories of configured and actual metricsSpec as MSQ engine in particular persists + * the combining factory in the dataschema, and combining factory of combining factory is effectively a no-op. + * Conversion to combining factory is a lossy conversion since combining factories typically + * use aggregator factory's output col name as both its input and output col names -- leading to false + * positives -- but it is acceptable for compaction. + */ private CompactionStatus metricsSpecIsUpToDate() { final AggregatorFactory[] configuredMetricsSpec = compactionConfig.getMetricsSpec(); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index a499e608a5a1..5f733db739f0 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java @@ -33,7 +33,7 @@ import org.apache.druid.client.indexing.ClientCompactionTaskQuery; import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; import org.apache.druid.client.indexing.ClientCompactionTaskTransformSpec; -import org.apache.druid.client.indexing.ClientMsqContext; +import org.apache.druid.client.indexing.ClientMSQContext; import org.apache.druid.client.indexing.ClientTaskQuery; import org.apache.druid.client.indexing.TaskPayloadResponse; import org.apache.druid.common.guava.FutureUtils; @@ -46,7 +46,6 @@ import org.apache.druid.java.util.common.granularity.GranularityType; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.metadata.LockFilterPolicy; -import org.apache.druid.query.QueryContext; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; @@ -91,9 +90,10 @@ public class CompactSegments implements CoordinatorCustomDuty private static final Predicate IS_COMPACTION_TASK = status -> null != status && COMPACTION_TASK_TYPE.equals(status.getType()); - // An artificial limit imposed just to avoid taking up too many compaction task slots for a single MSQ compaction - // task. Can be updated if needed. - private static final int MAX_TASK_SLOTS_FOR_MSQ_COMPACTION_TASK = 8; + /** + * Limit to ensure that an MSQ compaction task doesn't take up all task slots in a cluster. + */ + static final int MAX_TASK_SLOTS_FOR_MSQ_COMPACTION_TASK = 5; private final CompactionSegmentSearchPolicy policy; private final OverlordClient overlordClient; @@ -320,7 +320,9 @@ static int findMaxNumTaskSlotsUsedByOneNativeCompactionTask( */ static int findMaxNumTaskSlotsUsedByOneMsqCompactionTask(@Nullable Map context) { - return QueryContext.of(context).getInt(ClientMsqContext.CTX_MAX_NUM_TASKS, ClientMsqContext.DEFAULT_MAX_NUM_TASKS); + return context == null + ? ClientMSQContext.DEFAULT_MAX_NUM_TASKS + : (int) context.getOrDefault(ClientMSQContext.CTX_MAX_NUM_TASKS, ClientMSQContext.DEFAULT_MAX_NUM_TASKS); } @@ -498,17 +500,19 @@ private int submitCompactionTasks( int slotsRequiredForCurrentTask; if (compactionEngine == CompactionEngine.MSQ) { - if (autoCompactionContext.containsKey(ClientMsqContext.CTX_MAX_NUM_TASKS)) { - slotsRequiredForCurrentTask = (int) autoCompactionContext.get(ClientMsqContext.CTX_MAX_NUM_TASKS); + if (autoCompactionContext.containsKey(ClientMSQContext.CTX_MAX_NUM_TASKS)) { + slotsRequiredForCurrentTask = (int) autoCompactionContext.get(ClientMSQContext.CTX_MAX_NUM_TASKS); } else { // Since MSQ needs all task slots for the calculated #tasks to be available upfront, allot all available // compaction slots (upto a max of MAX_TASK_SLOTS_FOR_MSQ_COMPACTION) to current compaction task to avoid // stalling. Setting "taskAssignment" to "auto" has the problem of not being able to determine the actual // count, which is required for subsequent tasks. - slotsRequiredForCurrentTask = Math.min(numAvailableCompactionTaskSlots, - MAX_TASK_SLOTS_FOR_MSQ_COMPACTION_TASK + slotsRequiredForCurrentTask = Math.min( + // Update the slots to 2 (min required for MSQ) if only 1 slot is available. + numAvailableCompactionTaskSlots == 1 ? 2 : numAvailableCompactionTaskSlots, + MAX_TASK_SLOTS_FOR_MSQ_COMPACTION_TASK ); - autoCompactionContext.put(ClientMsqContext.CTX_MAX_NUM_TASKS, slotsRequiredForCurrentTask); + autoCompactionContext.put(ClientMSQContext.CTX_MAX_NUM_TASKS, slotsRequiredForCurrentTask); } } else { slotsRequiredForCurrentTask = findMaxNumTaskSlotsUsedByOneNativeCompactionTask(config.getTuningConfig()); diff --git a/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java index e54ede41baf4..0bba5cf63fa1 100644 --- a/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java +++ b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java @@ -28,7 +28,9 @@ import org.apache.druid.audit.AuditManager; import org.apache.druid.client.indexing.ClientCompactionRunnerInfo; import org.apache.druid.common.config.ConfigManager.SetResult; +import org.apache.druid.error.DruidException; import org.apache.druid.error.InvalidInput; +import org.apache.druid.error.NotFound; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.server.coordinator.CompactionConfigValidationResult; @@ -127,6 +129,7 @@ public Response addOrUpdateCompactionConfig( if (!validationResult.isValid()) { throw InvalidInput.exception("Compaction config not supported. Reason[%s].", validationResult.getReason()); } + // Don't persist config with the default engine if engine not specified, to enable update of the default. newConfigs.put(newConfig.getDataSource(), newConfig); newCompactionConfig = CoordinatorCompactionConfig.from(current, ImmutableList.copyOf(newConfigs.values())); @@ -214,7 +217,7 @@ public Response deleteCompactionConfig( final DataSourceCompactionConfig config = configs.remove(dataSource); if (config == null) { - throw new NoSuchElementException("datasource not found"); + throw NotFound.exception("datasource not found"); } return CoordinatorCompactionConfig.from(current, ImmutableList.copyOf(configs.values())); @@ -239,9 +242,8 @@ private Response updateConfigHelper( updateRetryDelay(); } } - catch (NoSuchElementException e) { - LOG.warn(e, "Update compaction config failed"); - return Response.status(Response.Status.NOT_FOUND).build(); + catch (DruidException e) { + return ServletResourceUtils.buildErrorResponseFrom(e); } catch (Exception e) { LOG.warn(e, "Update compaction config failed"); diff --git a/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfoTest.java b/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfoTest.java index 8adbb847d21e..22beae90cb2e 100644 --- a/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfoTest.java +++ b/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfoTest.java @@ -27,6 +27,7 @@ import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.java.util.common.HumanReadableBytes; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; @@ -34,24 +35,23 @@ import org.apache.druid.segment.data.CompressionFactory; import org.apache.druid.segment.data.CompressionStrategy; import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; +import org.apache.druid.server.coordinator.CompactionConfigValidationResult; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; import org.joda.time.Duration; import org.joda.time.Period; +import org.junit.Assert; import org.junit.Test; import javax.annotation.Nullable; import java.util.Collections; import java.util.Map; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - public class ClientCompactionRunnerInfoTest { @Test - public void testHashedPartitionsSpecs() + public void testMSQEngineWithHashedPartitionsSpecIsInvalid() { DataSourceCompactionConfig compactionConfig = createCompactionConfig( new HashedPartitionsSpec(100, null, null), @@ -59,12 +59,23 @@ public void testHashedPartitionsSpecs() null, null ); - assertFalse(ClientCompactionRunnerInfo.validateCompactionConfig(compactionConfig, CompactionEngine.NATIVE) - .isValid()); + CompactionConfigValidationResult validationResult = ClientCompactionRunnerInfo.validateCompactionConfig( + compactionConfig, + CompactionEngine.NATIVE + ); + Assert.assertFalse(validationResult.isValid()); + Assert.assertEquals( + StringUtils.format( + "Invalid partitionsSpec type[%s] for MSQ engine." + + " Type must be either 'dynamic' or 'range'.", + HashedPartitionsSpec.class.getSimpleName() + ), + validationResult.getReason() + ); } @Test - public void testrInvalidDynamicPartitionsSpecs() + public void testMSQEngineWithMaxTotalRowsIsInvalid() { DataSourceCompactionConfig compactionConfig = createCompactionConfig( new DynamicPartitionsSpec(100, 100L), @@ -72,12 +83,19 @@ public void testrInvalidDynamicPartitionsSpecs() null, null ); - assertFalse(ClientCompactionRunnerInfo.validateCompactionConfig(compactionConfig, CompactionEngine.NATIVE) - .isValid()); + CompactionConfigValidationResult validationResult = ClientCompactionRunnerInfo.validateCompactionConfig( + compactionConfig, + CompactionEngine.NATIVE + ); + Assert.assertFalse(validationResult.isValid()); + Assert.assertEquals(StringUtils.format( + "maxTotalRows[%d] in DynamicPartitionsSpec not supported for MSQ engine.", + 100 + ), validationResult.getReason()); } @Test - public void testDynamicPartitionsSpecs() + public void testMSQEngineWithDynamicPartitionsSpecIsValid() { DataSourceCompactionConfig compactionConfig = createCompactionConfig( new DynamicPartitionsSpec(100, null), @@ -85,12 +103,12 @@ public void testDynamicPartitionsSpecs() null, null ); - assertTrue(ClientCompactionRunnerInfo.validateCompactionConfig(compactionConfig, CompactionEngine.NATIVE) + Assert.assertTrue(ClientCompactionRunnerInfo.validateCompactionConfig(compactionConfig, CompactionEngine.NATIVE) .isValid()); } @Test - public void testDimensionRangePartitionsSpecs() + public void testMSQEngineWithDimensionRangePartitionsSpecIsValid() { DataSourceCompactionConfig compactionConfig = createCompactionConfig( new DimensionRangePartitionsSpec(100, null, ImmutableList.of("partitionDim"), false), @@ -98,12 +116,12 @@ public void testDimensionRangePartitionsSpecs() null, null ); - assertTrue(ClientCompactionRunnerInfo.validateCompactionConfig(compactionConfig, CompactionEngine.NATIVE) + Assert.assertTrue(ClientCompactionRunnerInfo.validateCompactionConfig(compactionConfig, CompactionEngine.NATIVE) .isValid()); } @Test - public void testQueryGranularityAll() + public void testMSQEngineWithQueryGranularityAllIsValid() { DataSourceCompactionConfig compactionConfig = createCompactionConfig( new DynamicPartitionsSpec(3, null), @@ -111,12 +129,12 @@ public void testQueryGranularityAll() new UserCompactionTaskGranularityConfig(Granularities.ALL, Granularities.ALL, false), null ); - assertTrue(ClientCompactionRunnerInfo.validateCompactionConfig(compactionConfig, CompactionEngine.NATIVE) + Assert.assertTrue(ClientCompactionRunnerInfo.validateCompactionConfig(compactionConfig, CompactionEngine.NATIVE) .isValid()); } @Test - public void testRollupFalseWithMetricsSpec() + public void testMSQEngineWithRollupFalseWithMetricsSpecIsInValid() { DataSourceCompactionConfig compactionConfig = createCompactionConfig( new DynamicPartitionsSpec(3, null), @@ -124,12 +142,43 @@ public void testRollupFalseWithMetricsSpec() new UserCompactionTaskGranularityConfig(null, null, false), new AggregatorFactory[]{new LongSumAggregatorFactory("sum", "sum")} ); - assertFalse(ClientCompactionRunnerInfo.validateCompactionConfig(compactionConfig, CompactionEngine.NATIVE) - .isValid()); + CompactionConfigValidationResult validationResult = ClientCompactionRunnerInfo.validateCompactionConfig( + compactionConfig, + CompactionEngine.NATIVE + ); + Assert.assertFalse(validationResult.isValid()); + Assert.assertEquals( + "rollup in granularitySpec must be set to True if metricsSpec is specifed for MSQ engine.", + validationResult.getReason() + ); + } + + @Test + public void testMSQEngineWithUnsupportedMetricsSpecIsInValid() + { + final String inputColName = "added"; + final String outputColName = "sum_added"; + DataSourceCompactionConfig compactionConfig = createCompactionConfig( + new DynamicPartitionsSpec(3, null), + Collections.emptyMap(), + new UserCompactionTaskGranularityConfig(null, null, null), + new AggregatorFactory[]{new LongSumAggregatorFactory(outputColName, inputColName)} + ); + CompactionConfigValidationResult validationResult = ClientCompactionRunnerInfo.validateCompactionConfig( + compactionConfig, + CompactionEngine.NATIVE + ); + Assert.assertFalse(validationResult.isValid()); + Assert.assertEquals( + StringUtils.format("Different name[%s] and fieldName(s)[%s] for aggregator unsupported for MSQ engine.", + outputColName, + Collections.singletonList(inputColName)), + validationResult.getReason() + ); } @Test - public void testRollupNullWithMetricsSpec() + public void testMSQEngineWithRollupNullWithMetricsSpecIsValid() { DataSourceCompactionConfig compactionConfig = createCompactionConfig( new DynamicPartitionsSpec(3, null), @@ -137,7 +186,7 @@ public void testRollupNullWithMetricsSpec() new UserCompactionTaskGranularityConfig(null, null, null), new AggregatorFactory[]{new LongSumAggregatorFactory("sum", "sum")} ); - assertTrue(ClientCompactionRunnerInfo.validateCompactionConfig(compactionConfig, CompactionEngine.NATIVE) + Assert.assertTrue(ClientCompactionRunnerInfo.validateCompactionConfig(compactionConfig, CompactionEngine.NATIVE) .isValid()); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigTest.java index 4df2bb519f3b..a9334f077a47 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigTest.java @@ -25,6 +25,7 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.SegmentsSplitHintSpec; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.HumanReadableBytes; @@ -83,6 +84,7 @@ public void testSerdeBasic() throws IOException Assert.assertEquals(config.getTuningConfig(), fromJson.getTuningConfig()); Assert.assertEquals(config.getTaskContext(), fromJson.getTaskContext()); Assert.assertEquals(config.getGranularitySpec(), fromJson.getGranularitySpec()); + Assert.assertEquals(config.getEngine(), fromJson.getEngine()); } @Test @@ -100,7 +102,7 @@ public void testSerdeWithMaxRowsPerSegment() throws IOException null, null, null, - null, + CompactionEngine.MSQ, ImmutableMap.of("key", "val") ); final String json = OBJECT_MAPPER.writeValueAsString(config); @@ -113,6 +115,7 @@ public void testSerdeWithMaxRowsPerSegment() throws IOException Assert.assertEquals(config.getSkipOffsetFromLatest(), fromJson.getSkipOffsetFromLatest()); Assert.assertEquals(config.getTuningConfig(), fromJson.getTuningConfig()); Assert.assertEquals(config.getTaskContext(), fromJson.getTaskContext()); + Assert.assertEquals(config.getEngine(), fromJson.getEngine()); } @Test @@ -150,7 +153,7 @@ public void testSerdeWithMaxTotalRows() throws IOException null, null, null, - null, + CompactionEngine.NATIVE, ImmutableMap.of("key", "val") ); final String json = OBJECT_MAPPER.writeValueAsString(config); @@ -163,6 +166,7 @@ public void testSerdeWithMaxTotalRows() throws IOException Assert.assertEquals(config.getSkipOffsetFromLatest(), fromJson.getSkipOffsetFromLatest()); Assert.assertEquals(config.getTuningConfig(), fromJson.getTuningConfig()); Assert.assertEquals(config.getTaskContext(), fromJson.getTaskContext()); + Assert.assertEquals(config.getEngine(), fromJson.getEngine()); } @Test diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java index 62c2130b2a2c..da90f5ee4ea3 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java @@ -36,12 +36,14 @@ import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; import org.apache.druid.client.indexing.ClientCompactionTaskQuery; import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; +import org.apache.druid.client.indexing.ClientMSQContext; import org.apache.druid.client.indexing.ClientTaskQuery; import org.apache.druid.client.indexing.IndexingTotalWorkerCapacityInfo; import org.apache.druid.client.indexing.NoopOverlordClient; import org.apache.druid.client.indexing.TaskPayloadResponse; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.RunnerTaskState; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskState; @@ -130,50 +132,63 @@ public class CompactSegmentsTest private static final int MAXIMUM_CAPACITY_WITH_AUTO_SCALE = 10; private static final NewestSegmentFirstPolicy SEARCH_POLICY = new NewestSegmentFirstPolicy(JSON_MAPPER); - @Parameterized.Parameters(name = "{0}") + @Parameterized.Parameters(name = "scenario={0}, engine={2}") public static Collection constructorFeeder() { final MutableInt nextRangePartitionBoundary = new MutableInt(0); + + final DynamicPartitionsSpec dynamicPartitionsSpec = new DynamicPartitionsSpec(300000, Long.MAX_VALUE); + final BiFunction numberedShardSpecCreator = NumberedShardSpec::new; + + final HashedPartitionsSpec hashedPartitionsSpec = new HashedPartitionsSpec(null, 2, ImmutableList.of("dim")); + final BiFunction hashBasedNumberedShardSpecCreator = + (bucketId, numBuckets) -> new HashBasedNumberedShardSpec( + bucketId, + numBuckets, + bucketId, + numBuckets, + ImmutableList.of("dim"), + null, + JSON_MAPPER + ); + + final SingleDimensionPartitionsSpec singleDimensionPartitionsSpec = + new SingleDimensionPartitionsSpec(300000, null, "dim", false); + final BiFunction singleDimensionShardSpeCreator = + (bucketId, numBuckets) -> new SingleDimensionShardSpec( + "dim", + bucketId == 0 ? null : String.valueOf(nextRangePartitionBoundary.getAndIncrement()), + bucketId.equals(numBuckets) ? null : String.valueOf(nextRangePartitionBoundary.getAndIncrement()), + bucketId, + numBuckets + ); + + // Hash partition spec is not supported by MSQ engine. return ImmutableList.of( - new Object[]{ - new DynamicPartitionsSpec(300000, Long.MAX_VALUE), - (BiFunction) NumberedShardSpec::new - }, - new Object[]{ - new HashedPartitionsSpec(null, 2, ImmutableList.of("dim")), - (BiFunction) (bucketId, numBuckets) -> new HashBasedNumberedShardSpec( - bucketId, - numBuckets, - bucketId, - numBuckets, - ImmutableList.of("dim"), - null, - JSON_MAPPER - ) - }, - new Object[]{ - new SingleDimensionPartitionsSpec(300000, null, "dim", false), - (BiFunction) (bucketId, numBuckets) -> new SingleDimensionShardSpec( - "dim", - bucketId == 0 ? null : String.valueOf(nextRangePartitionBoundary.getAndIncrement()), - bucketId.equals(numBuckets) ? null : String.valueOf(nextRangePartitionBoundary.getAndIncrement()), - bucketId, - numBuckets - ) - } + new Object[]{dynamicPartitionsSpec, numberedShardSpecCreator, CompactionEngine.NATIVE}, + new Object[]{hashedPartitionsSpec, hashBasedNumberedShardSpecCreator, CompactionEngine.NATIVE}, + new Object[]{singleDimensionPartitionsSpec, singleDimensionShardSpeCreator, CompactionEngine.NATIVE}, + new Object[]{dynamicPartitionsSpec, numberedShardSpecCreator, CompactionEngine.MSQ}, + new Object[]{singleDimensionPartitionsSpec, singleDimensionShardSpeCreator, CompactionEngine.MSQ} ); } private final PartitionsSpec partitionsSpec; private final BiFunction shardSpecFactory; + private final CompactionEngine engine; private DataSourcesSnapshot dataSources; Map> datasourceToSegments = new HashMap<>(); - public CompactSegmentsTest(PartitionsSpec partitionsSpec, BiFunction shardSpecFactory) + public CompactSegmentsTest( + PartitionsSpec partitionsSpec, + BiFunction shardSpecFactory, + CompactionEngine engine + ) { this.partitionsSpec = partitionsSpec; this.shardSpecFactory = shardSpecFactory; + this.engine = engine; } @Before @@ -640,7 +655,12 @@ public void testRunMultipleCompactionTaskSlots() final CoordinatorRunStats stats = doCompactSegments(compactSegments, 3); Assert.assertEquals(3, stats.get(Stats.Compaction.AVAILABLE_SLOTS)); Assert.assertEquals(3, stats.get(Stats.Compaction.MAX_SLOTS)); - Assert.assertEquals(3, stats.get(Stats.Compaction.SUBMITTED_TASKS)); + // Native takes up 1 task slot by default whereas MSQ takes up all available upto 5. + if (engine == CompactionEngine.NATIVE) { + Assert.assertEquals(3, stats.get(Stats.Compaction.SUBMITTED_TASKS)); + } else { + Assert.assertEquals(1, stats.get(Stats.Compaction.SUBMITTED_TASKS)); + } } @Test @@ -654,7 +674,12 @@ public void testRunMultipleCompactionTaskSlotsWithUseAutoScaleSlotsOverMaxSlot() doCompactSegments(compactSegments, createCompactionConfigs(), maxCompactionSlot, true); Assert.assertEquals(maxCompactionSlot, stats.get(Stats.Compaction.AVAILABLE_SLOTS)); Assert.assertEquals(maxCompactionSlot, stats.get(Stats.Compaction.MAX_SLOTS)); - Assert.assertEquals(maxCompactionSlot, stats.get(Stats.Compaction.SUBMITTED_TASKS)); + // Native takes up 1 task slot by default whereas MSQ takes up all available upto 5. + if (engine == CompactionEngine.NATIVE) { + Assert.assertEquals(maxCompactionSlot, stats.get(Stats.Compaction.SUBMITTED_TASKS)); + } else { + Assert.assertEquals(1, stats.get(Stats.Compaction.SUBMITTED_TASKS)); + } } @Test @@ -668,7 +693,15 @@ public void testRunMultipleCompactionTaskSlotsWithUseAutoScaleSlotsUnderMaxSlot( doCompactSegments(compactSegments, createCompactionConfigs(), maxCompactionSlot, true); Assert.assertEquals(MAXIMUM_CAPACITY_WITH_AUTO_SCALE, stats.get(Stats.Compaction.AVAILABLE_SLOTS)); Assert.assertEquals(MAXIMUM_CAPACITY_WITH_AUTO_SCALE, stats.get(Stats.Compaction.MAX_SLOTS)); - Assert.assertEquals(MAXIMUM_CAPACITY_WITH_AUTO_SCALE, stats.get(Stats.Compaction.SUBMITTED_TASKS)); + // Native takes up 1 task slot by default whereas MSQ takes up all available upto 5. + if (engine == CompactionEngine.NATIVE) { + Assert.assertEquals(MAXIMUM_CAPACITY_WITH_AUTO_SCALE, stats.get(Stats.Compaction.SUBMITTED_TASKS)); + } else { + Assert.assertEquals( + MAXIMUM_CAPACITY_WITH_AUTO_SCALE / CompactSegments.MAX_TASK_SLOTS_FOR_MSQ_COMPACTION_TASK, + stats.get(Stats.Compaction.SUBMITTED_TASKS) + ); + } } @Test @@ -712,7 +745,7 @@ public void testCompactWithoutGranularitySpec() null, null, null, - null, + engine, null ) ); @@ -770,7 +803,7 @@ public void testCompactWithNotNullIOConfig() null, null, new UserCompactionTaskIOConfig(true), - null, + engine, null ) ); @@ -820,7 +853,7 @@ public void testCompactWithNullIOConfig() null, null, null, - null, + engine, null ) ); @@ -870,7 +903,7 @@ public void testCompactWithGranularitySpec() null, null, null, - null, + engine, null ) ); @@ -931,7 +964,7 @@ public void testCompactWithDimensionSpec() null, null, null, - null, + engine, null ) ); @@ -984,7 +1017,7 @@ public void testCompactWithoutDimensionSpec() null, null, null, - null, + engine, null ) ); @@ -1034,7 +1067,7 @@ public void testCompactWithRollupInGranularitySpec() null, null, null, - null, + engine, null ) ); @@ -1145,7 +1178,7 @@ public void testCompactWithGranularitySpecConflictWithActiveCompactionTask() null, null, null, - null, + engine, null ) ); @@ -1174,8 +1207,13 @@ public void testRunParallelCompactionMultipleCompactionTaskSlots() { final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); final CompactSegments compactSegments = new CompactSegments(SEARCH_POLICY, overlordClient); - - final CoordinatorRunStats stats = doCompactSegments(compactSegments, createCompactionConfigs(2), 4); + final CoordinatorRunStats stats; + // Native uses maxNumConcurrentSubTasks for task slots whereas MSQ uses maxNumTasks. + if (engine == CompactionEngine.NATIVE) { + stats = doCompactSegments(compactSegments, createCompactionConfigs(2, null), 4); + } else { + stats = doCompactSegments(compactSegments, createCompactionConfigs(null, 2), 4); + } Assert.assertEquals(4, stats.get(Stats.Compaction.AVAILABLE_SLOTS)); Assert.assertEquals(4, stats.get(Stats.Compaction.MAX_SLOTS)); Assert.assertEquals(2, stats.get(Stats.Compaction.SUBMITTED_TASKS)); @@ -1207,7 +1245,7 @@ public void testRunWithLockedIntervals() // is submitted for dataSource_0 CompactSegments compactSegments = new CompactSegments(SEARCH_POLICY, overlordClient); final CoordinatorRunStats stats = - doCompactSegments(compactSegments, createCompactionConfigs(2), 4); + doCompactSegments(compactSegments, createCompactionConfigs(2, null), 4); Assert.assertEquals(1, stats.get(Stats.Compaction.SUBMITTED_TASKS)); Assert.assertEquals(1, overlordClient.submittedCompactionTasks.size()); @@ -1261,7 +1299,7 @@ public void testCompactWithTransformSpec() null, new UserCompactionTaskTransformConfig(new SelectorDimFilter("dim1", "foo", null)), null, - null, + engine, null ) ); @@ -1312,7 +1350,7 @@ public void testCompactWithoutCustomSpecs() null, null, null, - null, + engine, null ) ); @@ -1365,7 +1403,7 @@ public void testCompactWithMetricsSpec() aggregatorFactories, null, null, - null, + engine, null ) ); @@ -1446,7 +1484,7 @@ public void testDetermineSegmentGranularityFromSegmentsToCompact() null, null, null, - null, + engine, null ) ); @@ -1533,7 +1571,7 @@ public void testDetermineSegmentGranularityFromSegmentGranularityInCompactionCon null, null, null, - null, + engine, null ) ); @@ -1591,7 +1629,7 @@ public void testCompactWithMetricsSpecShouldSetPreserveExistingMetricsTrue() new AggregatorFactory[] {new CountAggregatorFactory("cnt")}, null, null, - null, + engine, null ) ); @@ -1644,7 +1682,7 @@ public void testCompactWithoutMetricsSpecShouldSetPreserveExistingMetricsFalse() null, null, null, - null, + engine, null ) ); @@ -1918,10 +1956,13 @@ private void addMoreData(String dataSource, int day) private List createCompactionConfigs() { - return createCompactionConfigs(null); + return createCompactionConfigs(null, null); } - private List createCompactionConfigs(@Nullable Integer maxNumConcurrentSubTasks) + private List createCompactionConfigs( + @Nullable Integer maxNumConcurrentSubTasks, + @Nullable Integer maxNumTasksForMSQ + ) { final List compactionConfigs = new ArrayList<>(); for (int i = 0; i < 3; i++) { @@ -1959,8 +2000,8 @@ private List createCompactionConfigs(@Nullable Integ null, null, null, - null, - null + engine, + maxNumTasksForMSQ == null ? null : ImmutableMap.of(ClientMSQContext.CTX_MAX_NUM_TASKS, maxNumTasksForMSQ) ) ); } diff --git a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java index 3d7063e7b958..2ec14003314d 100644 --- a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java @@ -22,10 +22,14 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.druid.audit.AuditManager; +import org.apache.druid.client.indexing.ClientMSQContext; import org.apache.druid.common.config.ConfigManager; import org.apache.druid.common.config.JacksonConfigManager; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.ErrorResponse; import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.metadata.MetadataStorageConnector; import org.apache.druid.metadata.MetadataStorageTablesConfig; @@ -206,6 +210,7 @@ public void testAddOrUpdateCompactionConfigWithExistingConfig() Assert.assertEquals(2, newConfigCaptor.getValue().getCompactionConfigs().size()); Assert.assertEquals(OLD_CONFIG, newConfigCaptor.getValue().getCompactionConfigs().get(0)); Assert.assertEquals(newConfig, newConfigCaptor.getValue().getCompactionConfigs().get(1)); + Assert.assertEquals(newConfig.getEngine(), newConfigCaptor.getValue().getEngine()); } @Test @@ -407,6 +412,115 @@ public void testAddOrUpdateCompactionConfigWithoutExistingConfig() Assert.assertNotNull(newConfigCaptor.getValue()); Assert.assertEquals(1, newConfigCaptor.getValue().getCompactionConfigs().size()); Assert.assertEquals(newConfig, newConfigCaptor.getValue().getCompactionConfigs().get(0)); + Assert.assertEquals(newConfig.getEngine(), newConfigCaptor.getValue().getCompactionConfigs().get(0).getEngine()); + } + + @Test + public void testAddOrUpdateCompactionConfigWithoutExistingConfigAndEngineAsNull() + { + Mockito.when(mockConnector.lookup( + ArgumentMatchers.anyString(), + ArgumentMatchers.eq("name"), + ArgumentMatchers.eq("payload"), + ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY) + ) + ).thenReturn(null); + Mockito.when(mockJacksonConfigManager.convertByteToConfig( + ArgumentMatchers.eq(null), + ArgumentMatchers.eq(CoordinatorCompactionConfig.class), + ArgumentMatchers.eq(CoordinatorCompactionConfig.empty()) + ) + ).thenReturn(CoordinatorCompactionConfig.empty()); + final ArgumentCaptor oldConfigCaptor = ArgumentCaptor.forClass(byte[].class); + final ArgumentCaptor newConfigCaptor = ArgumentCaptor.forClass( + CoordinatorCompactionConfig.class); + Mockito.when(mockJacksonConfigManager.set( + ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY), + oldConfigCaptor.capture(), + newConfigCaptor.capture(), + ArgumentMatchers.any() + ) + ).thenReturn(ConfigManager.SetResult.ok()); + + final DataSourceCompactionConfig newConfig = new DataSourceCompactionConfig( + "dataSource", + null, + 500L, + null, + new Period(3600), + null, + new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null), + null, + null, + null, + null, + null, + ImmutableMap.of("key", "val") + ); + Response ignore = coordinatorCompactionConfigsResource.addOrUpdateCompactionConfig( + newConfig, + mockHttpServletRequest + ); + Assert.assertEquals(null, newConfigCaptor.getValue().getCompactionConfigs().get(0).getEngine()); + } + + @Test + public void testAddOrUpdateCompactionConfigWithInvalidMaxNumTasksForMSQEngine() + { + Mockito.when(mockConnector.lookup( + ArgumentMatchers.anyString(), + ArgumentMatchers.eq("name"), + ArgumentMatchers.eq("payload"), + ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY) + ) + ).thenReturn(null); + Mockito.when(mockJacksonConfigManager.convertByteToConfig( + ArgumentMatchers.eq(null), + ArgumentMatchers.eq(CoordinatorCompactionConfig.class), + ArgumentMatchers.eq(CoordinatorCompactionConfig.empty()) + ) + ).thenReturn(CoordinatorCompactionConfig.empty()); + final ArgumentCaptor oldConfigCaptor = ArgumentCaptor.forClass(byte[].class); + final ArgumentCaptor newConfigCaptor = ArgumentCaptor.forClass( + CoordinatorCompactionConfig.class); + Mockito.when(mockJacksonConfigManager.set( + ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY), + oldConfigCaptor.capture(), + newConfigCaptor.capture(), + ArgumentMatchers.any() + ) + ).thenReturn(ConfigManager.SetResult.ok()); + + int maxNumTasks = 1; + + final DataSourceCompactionConfig newConfig = new DataSourceCompactionConfig( + "dataSource", + null, + 500L, + null, + new Period(3600), + null, + new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null), + null, + null, + null, + null, + CompactionEngine.MSQ, + ImmutableMap.of(ClientMSQContext.CTX_MAX_NUM_TASKS, maxNumTasks) + ); + Response response = coordinatorCompactionConfigsResource.addOrUpdateCompactionConfig( + newConfig, + mockHttpServletRequest + ); + Assert.assertEquals(DruidException.Category.INVALID_INPUT.getExpectedStatus(), response.getStatus()); + Assert.assertEquals( + StringUtils.format( + "Compaction config not supported. Reason[MSQ context maxNumTasks [%,d] cannot be less than 2, " + + "since at least 1 controller and 1 worker is necessary.].", + maxNumTasks + ), + ((ErrorResponse) response.getEntity()).getUnderlyingException().getMessage() + ); } @Test From fea7f94e7b6aae592e023d4aea3e4e3e70735b70 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Thu, 11 Jul 2024 11:22:05 +0530 Subject: [PATCH 51/54] Address review comments --- .../msq/indexing/MSQCompactionRunner.java | 2 +- .../duty/ITAutoCompactionTest.java | 81 +++++++++++++++---- .../client/indexing/ClientMSQContext.java | 4 + .../coordinator/compact/CompactionStatus.java | 28 +------ .../coordinator/duty/CompactSegments.java | 7 +- .../ClientCompactionRunnerInfoTest.java | 21 ++--- .../coordinator/duty/CompactSegmentsTest.java | 39 ++++++--- ...rdinatorCompactionConfigsResourceTest.java | 20 +---- 8 files changed, 111 insertions(+), 91 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java index 47f901dce00b..ac43e7c864b8 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java @@ -111,7 +111,7 @@ public MSQCompactionRunner(@JacksonInject ObjectMapper jsonMapper, @JacksonInjec /** * Checks if the provided compaction config is supported by MSQ. The same validation is done at - * {@link ClientCompactionRunnerInfo#MSQEngineSupportsCompactionConfig} + * {@link ClientCompactionRunnerInfo#compactionConfigSupportedByMSQEngine} * The following configs aren't supported: *
        *
      • partitionsSpec of type HashedParititionsSpec.
      • diff --git a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java index 09988338a023..9a6f9cabed79 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java @@ -466,7 +466,7 @@ public void testAutoCompactionDutySubmitAndVerifyCompaction() throws Exception verifySegmentsCount(4); verifyQuery(INDEX_QUERIES_RESOURCE); - submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, Period.days(1), null); + submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, Period.days(1)); //...compacted into 1 new segment for 1 day. 1 day compacted and 1 day skipped/remains uncompacted. (3 total) forceTriggerAutoCompaction(3); verifyQuery(INDEX_QUERIES_RESOURCE); @@ -484,7 +484,7 @@ public void testAutoCompactionDutySubmitAndVerifyCompaction() throws Exception 0, 1, 1); - submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, null); + submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET); //...compacted into 1 new segment for the remaining one day. 2 day compacted and 0 day uncompacted. (2 total) forceTriggerAutoCompaction(2); verifyQuery(INDEX_QUERIES_RESOURCE); @@ -517,9 +517,9 @@ public void testAutoCompactionDutyCanUpdateCompactionConfig() throws Exception verifyQuery(INDEX_QUERIES_RESOURCE); // Dummy compaction config which will be overwritten - submitCompactionConfig(10000, NO_SKIP_OFFSET, null); + submitCompactionConfig(10000, NO_SKIP_OFFSET); // New compaction config should overwrites the existing compaction config - submitCompactionConfig(1, NO_SKIP_OFFSET, null); + submitCompactionConfig(1, NO_SKIP_OFFSET); LOG.info("Auto compaction test with dynamic partitioning"); @@ -533,7 +533,7 @@ public void testAutoCompactionDutyCanUpdateCompactionConfig() throws Exception LOG.info("Auto compaction test with hash partitioning"); final HashedPartitionsSpec hashedPartitionsSpec = new HashedPartitionsSpec(null, 3, null); - submitCompactionConfig(hashedPartitionsSpec, NO_SKIP_OFFSET, 1, null, null, null, null, false, null); + submitCompactionConfig(hashedPartitionsSpec, NO_SKIP_OFFSET, 1, null, null, null, null, false); // 2 segments published per day after compaction. forceTriggerAutoCompaction(4); verifyQuery(INDEX_QUERIES_RESOURCE); @@ -548,7 +548,7 @@ public void testAutoCompactionDutyCanUpdateCompactionConfig() throws Exception "city", false ); - submitCompactionConfig(rangePartitionsSpec, NO_SKIP_OFFSET, 1, null, null, null, null, false, null); + submitCompactionConfig(rangePartitionsSpec, NO_SKIP_OFFSET, 1, null, null, null, null, false); forceTriggerAutoCompaction(2); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(rangePartitionsSpec, 2); @@ -593,7 +593,7 @@ public void testAutoCompactionDutyCanUpdateTaskSlots() throws Exception verifySegmentsCount(4); verifyQuery(INDEX_QUERIES_RESOURCE); - submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, null); + submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET); // ...should remains unchanged (4 total) forceTriggerAutoCompaction(4); verifyQuery(INDEX_QUERIES_RESOURCE); @@ -920,14 +920,14 @@ public void testAutoCompactionDutyWithSegmentGranularityAndMixedVersion() throws verifySegmentsCount(4); verifyQuery(INDEX_QUERIES_RESOURCE); - submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, Period.days(1), null); + submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, Period.days(1)); //...compacted into 1 new segment for 1 day. 1 day compacted and 1 day skipped/remains uncompacted. (3 total) forceTriggerAutoCompaction(3); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(1, MAX_ROWS_PER_SEGMENT_COMPACTED); Granularity newGranularity = Granularities.YEAR; - submitCompactionConfig(1000, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), null); + submitCompactionConfig(1000, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null)); LOG.info("Auto compaction test with YEAR segment granularity"); @@ -959,7 +959,7 @@ public void testAutoCompactionDutyWithSegmentGranularityAndExistingCompactedSegm verifyQuery(INDEX_QUERIES_RESOURCE); // Compacted without SegmentGranularity in auto compaction config - submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, null); + submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET); forceTriggerAutoCompaction(2); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(2, MAX_ROWS_PER_SEGMENT_COMPACTED); @@ -969,7 +969,7 @@ public void testAutoCompactionDutyWithSegmentGranularityAndExistingCompactedSegm // Segments were compacted and already has DAY granularity since it was initially ingested with DAY granularity. // Now set auto compaction with DAY granularity in the granularitySpec Granularity newGranularity = Granularities.DAY; - submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), null); + submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null)); forceTriggerAutoCompaction(2); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(2, MAX_ROWS_PER_SEGMENT_COMPACTED); @@ -991,7 +991,7 @@ public void testAutoCompactionDutyWithSegmentGranularityAndExistingCompactedSegm verifyQuery(INDEX_QUERIES_RESOURCE); // Compacted without SegmentGranularity in auto compaction config - submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, null); + submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET); forceTriggerAutoCompaction(2); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(2, MAX_ROWS_PER_SEGMENT_COMPACTED); @@ -1001,7 +1001,7 @@ public void testAutoCompactionDutyWithSegmentGranularityAndExistingCompactedSegm // Segments were compacted and already has DAY granularity since it was initially ingested with DAY granularity. // Now set auto compaction with DAY granularity in the granularitySpec Granularity newGranularity = Granularities.YEAR; - submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), null); + submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null)); forceTriggerAutoCompaction(1); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(1, MAX_ROWS_PER_SEGMENT_COMPACTED); @@ -1595,17 +1595,33 @@ private void verifyQuery(String queryResource, Map keyValueToRep queryHelper.testQueriesFromString(queryResponseTemplate); } - private void submitCompactionConfig(Integer maxRowsPerSegment, Period skipOffsetFromLatest, CompactionEngine engine) + private void submitCompactionConfig(Integer maxRowsPerSegment, Period skipOffsetFromLatest) + throws Exception + { + submitCompactionConfig(maxRowsPerSegment, skipOffsetFromLatest, null, null); + } + + private void submitCompactionConfig(Integer maxRowsPerSegment, Period skipOffsetFromLatest, @Nullable CompactionEngine engine) throws Exception { submitCompactionConfig(maxRowsPerSegment, skipOffsetFromLatest, null, engine); } + private void submitCompactionConfig( + Integer maxRowsPerSegment, + Period skipOffsetFromLatest, + UserCompactionTaskGranularityConfig granularitySpec + ) throws Exception + { + submitCompactionConfig(maxRowsPerSegment, skipOffsetFromLatest, granularitySpec, false, null); + } + + private void submitCompactionConfig( Integer maxRowsPerSegment, Period skipOffsetFromLatest, UserCompactionTaskGranularityConfig granularitySpec, - CompactionEngine engine + @Nullable CompactionEngine engine ) throws Exception { submitCompactionConfig(maxRowsPerSegment, skipOffsetFromLatest, granularitySpec, false, engine); @@ -1629,8 +1645,15 @@ private void submitCompactionConfig( @Nullable CompactionEngine engine ) throws Exception { - submitCompactionConfig(maxRowsPerSegment, skipOffsetFromLatest, granularitySpec, null, null, null, dropExisting, - engine + submitCompactionConfig( + maxRowsPerSegment, + skipOffsetFromLatest, + granularitySpec, + null, + null, + null, + dropExisting, + engine ); } @@ -1680,6 +1703,30 @@ private void submitCompactionConfig( ); } + private void submitCompactionConfig( + PartitionsSpec partitionsSpec, + Period skipOffsetFromLatest, + int maxNumConcurrentSubTasks, + UserCompactionTaskGranularityConfig granularitySpec, + UserCompactionTaskDimensionsConfig dimensionsSpec, + UserCompactionTaskTransformConfig transformSpec, + AggregatorFactory[] metricsSpec, + boolean dropExisting + ) throws Exception + { + submitCompactionConfig( + partitionsSpec, + skipOffsetFromLatest, + maxNumConcurrentSubTasks, + granularitySpec, + dimensionsSpec, + transformSpec, + metricsSpec, + dropExisting, + null + ); + } + private void submitCompactionConfig( PartitionsSpec partitionsSpec, Period skipOffsetFromLatest, diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientMSQContext.java b/server/src/main/java/org/apache/druid/client/indexing/ClientMSQContext.java index e80b86f65b3f..45279bda3ed3 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientMSQContext.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientMSQContext.java @@ -28,4 +28,8 @@ public class ClientMSQContext { public static final String CTX_MAX_NUM_TASKS = "maxNumTasks"; public static final int DEFAULT_MAX_NUM_TASKS = 2; + /** + * Limit to ensure that an MSQ compaction task doesn't take up all task slots in a cluster. + */ + public static final int MAX_TASK_SLOTS_FOR_MSQ_COMPACTION_TASK = 5; } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java index 0a76acc870b5..862f2e7c5b4c 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java @@ -299,43 +299,23 @@ private CompactionStatus dimensionsSpecIsUpToDate() } } - /** - * Compares combining factories of configured and actual metricsSpec as MSQ engine in particular persists - * the combining factory in the dataschema, and combining factory of combining factory is effectively a no-op. - * Conversion to combining factory is a lossy conversion since combining factories typically - * use aggregator factory's output col name as both its input and output col names -- leading to false - * positives -- but it is acceptable for compaction. - */ private CompactionStatus metricsSpecIsUpToDate() { final AggregatorFactory[] configuredMetricsSpec = compactionConfig.getMetricsSpec(); if (ArrayUtils.isEmpty(configuredMetricsSpec)) { return COMPLETE; } - final AggregatorFactory[] configuredMetricsCombiningFactorySpec = - Arrays.stream(configuredMetricsSpec) - .map(AggregatorFactory::getCombiningFactory) - .toArray(AggregatorFactory[]::new); final List metricSpecList = lastCompactionState.getMetricsSpec(); final AggregatorFactory[] existingMetricsSpec = CollectionUtils.isNullOrEmpty(metricSpecList) ? null : objectMapper.convertValue(metricSpecList, AggregatorFactory[].class); - final AggregatorFactory[] existingMetricsCombiningFactorySpec = - existingMetricsSpec == null - ? null - : Arrays.stream(existingMetricsSpec) - .map(AggregatorFactory::getCombiningFactory) - .toArray(AggregatorFactory[]::new); - - if (existingMetricsSpec == null || !Arrays.deepEquals( - configuredMetricsCombiningFactorySpec, - existingMetricsCombiningFactorySpec - )) { + + if (existingMetricsSpec == null || !Arrays.deepEquals(configuredMetricsSpec, existingMetricsSpec)) { return CompactionStatus.configChanged( "metricsSpec", - Arrays.toString(configuredMetricsCombiningFactorySpec), - Arrays.toString(existingMetricsCombiningFactorySpec) + Arrays.toString(configuredMetricsSpec), + Arrays.toString(existingMetricsSpec) ); } else { return COMPLETE; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index 5f733db739f0..1a227b06fa97 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java @@ -90,11 +90,6 @@ public class CompactSegments implements CoordinatorCustomDuty private static final Predicate IS_COMPACTION_TASK = status -> null != status && COMPACTION_TASK_TYPE.equals(status.getType()); - /** - * Limit to ensure that an MSQ compaction task doesn't take up all task slots in a cluster. - */ - static final int MAX_TASK_SLOTS_FOR_MSQ_COMPACTION_TASK = 5; - private final CompactionSegmentSearchPolicy policy; private final OverlordClient overlordClient; @@ -510,7 +505,7 @@ private int submitCompactionTasks( slotsRequiredForCurrentTask = Math.min( // Update the slots to 2 (min required for MSQ) if only 1 slot is available. numAvailableCompactionTaskSlots == 1 ? 2 : numAvailableCompactionTaskSlots, - MAX_TASK_SLOTS_FOR_MSQ_COMPACTION_TASK + ClientMSQContext.MAX_TASK_SLOTS_FOR_MSQ_COMPACTION_TASK ); autoCompactionContext.put(ClientMSQContext.CTX_MAX_NUM_TASKS, slotsRequiredForCurrentTask); } diff --git a/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfoTest.java b/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfoTest.java index 22beae90cb2e..f6d4a2b6e581 100644 --- a/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfoTest.java +++ b/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfoTest.java @@ -27,7 +27,6 @@ import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.java.util.common.HumanReadableBytes; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; @@ -65,11 +64,8 @@ public void testMSQEngineWithHashedPartitionsSpecIsInvalid() ); Assert.assertFalse(validationResult.isValid()); Assert.assertEquals( - StringUtils.format( - "Invalid partitionsSpec type[%s] for MSQ engine." - + " Type must be either 'dynamic' or 'range'.", - HashedPartitionsSpec.class.getSimpleName() - ), + "Invalid partitionsSpec type[HashedPartitionsSpec] for MSQ engine." + + " Type must be either 'dynamic' or 'range'.", validationResult.getReason() ); } @@ -88,10 +84,10 @@ public void testMSQEngineWithMaxTotalRowsIsInvalid() CompactionEngine.NATIVE ); Assert.assertFalse(validationResult.isValid()); - Assert.assertEquals(StringUtils.format( - "maxTotalRows[%d] in DynamicPartitionsSpec not supported for MSQ engine.", - 100 - ), validationResult.getReason()); + Assert.assertEquals( + "maxTotalRows[100] in DynamicPartitionsSpec not supported for MSQ engine.", + validationResult.getReason() + ); } @Test @@ -156,6 +152,7 @@ public void testMSQEngineWithRollupFalseWithMetricsSpecIsInValid() @Test public void testMSQEngineWithUnsupportedMetricsSpecIsInValid() { + // Aggregators having different input and ouput column names are unsupported. final String inputColName = "added"; final String outputColName = "sum_added"; DataSourceCompactionConfig compactionConfig = createCompactionConfig( @@ -170,9 +167,7 @@ public void testMSQEngineWithUnsupportedMetricsSpecIsInValid() ); Assert.assertFalse(validationResult.isValid()); Assert.assertEquals( - StringUtils.format("Different name[%s] and fieldName(s)[%s] for aggregator unsupported for MSQ engine.", - outputColName, - Collections.singletonList(inputColName)), + "Different name[sum_added] and fieldName(s)[[added]] for aggregator unsupported for MSQ engine.", validationResult.getReason() ); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java index da90f5ee4ea3..236cfaf7da54 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java @@ -132,7 +132,7 @@ public class CompactSegmentsTest private static final int MAXIMUM_CAPACITY_WITH_AUTO_SCALE = 10; private static final NewestSegmentFirstPolicy SEARCH_POLICY = new NewestSegmentFirstPolicy(JSON_MAPPER); - @Parameterized.Parameters(name = "scenario={0}, engine={2}") + @Parameterized.Parameters(name = "scenario: {0}, engine: {2}") public static Collection constructorFeeder() { final MutableInt nextRangePartitionBoundary = new MutableInt(0); @@ -154,7 +154,7 @@ public static Collection constructorFeeder() final SingleDimensionPartitionsSpec singleDimensionPartitionsSpec = new SingleDimensionPartitionsSpec(300000, null, "dim", false); - final BiFunction singleDimensionShardSpeCreator = + final BiFunction singleDimensionShardSpecCreator = (bucketId, numBuckets) -> new SingleDimensionShardSpec( "dim", bucketId == 0 ? null : String.valueOf(nextRangePartitionBoundary.getAndIncrement()), @@ -167,9 +167,9 @@ public static Collection constructorFeeder() return ImmutableList.of( new Object[]{dynamicPartitionsSpec, numberedShardSpecCreator, CompactionEngine.NATIVE}, new Object[]{hashedPartitionsSpec, hashBasedNumberedShardSpecCreator, CompactionEngine.NATIVE}, - new Object[]{singleDimensionPartitionsSpec, singleDimensionShardSpeCreator, CompactionEngine.NATIVE}, + new Object[]{singleDimensionPartitionsSpec, singleDimensionShardSpecCreator, CompactionEngine.NATIVE}, new Object[]{dynamicPartitionsSpec, numberedShardSpecCreator, CompactionEngine.MSQ}, - new Object[]{singleDimensionPartitionsSpec, singleDimensionShardSpeCreator, CompactionEngine.MSQ} + new Object[]{singleDimensionPartitionsSpec, singleDimensionShardSpecCreator, CompactionEngine.MSQ} ); } @@ -655,7 +655,8 @@ public void testRunMultipleCompactionTaskSlots() final CoordinatorRunStats stats = doCompactSegments(compactSegments, 3); Assert.assertEquals(3, stats.get(Stats.Compaction.AVAILABLE_SLOTS)); Assert.assertEquals(3, stats.get(Stats.Compaction.MAX_SLOTS)); - // Native takes up 1 task slot by default whereas MSQ takes up all available upto 5. + // Native takes up 1 task slot by default whereas MSQ takes up all available upto 5. Since there are 3 available + // slots, there are 3 submitted tasks for native whereas 1 for MSQ. if (engine == CompactionEngine.NATIVE) { Assert.assertEquals(3, stats.get(Stats.Compaction.SUBMITTED_TASKS)); } else { @@ -674,7 +675,8 @@ public void testRunMultipleCompactionTaskSlotsWithUseAutoScaleSlotsOverMaxSlot() doCompactSegments(compactSegments, createCompactionConfigs(), maxCompactionSlot, true); Assert.assertEquals(maxCompactionSlot, stats.get(Stats.Compaction.AVAILABLE_SLOTS)); Assert.assertEquals(maxCompactionSlot, stats.get(Stats.Compaction.MAX_SLOTS)); - // Native takes up 1 task slot by default whereas MSQ takes up all available upto 5. + // Native takes up 1 task slot by default whereas MSQ takes up all available upto 5. Since there are 3 available + // slots, there are 3 submitted tasks for native whereas 1 for MSQ. if (engine == CompactionEngine.NATIVE) { Assert.assertEquals(maxCompactionSlot, stats.get(Stats.Compaction.SUBMITTED_TASKS)); } else { @@ -693,12 +695,13 @@ public void testRunMultipleCompactionTaskSlotsWithUseAutoScaleSlotsUnderMaxSlot( doCompactSegments(compactSegments, createCompactionConfigs(), maxCompactionSlot, true); Assert.assertEquals(MAXIMUM_CAPACITY_WITH_AUTO_SCALE, stats.get(Stats.Compaction.AVAILABLE_SLOTS)); Assert.assertEquals(MAXIMUM_CAPACITY_WITH_AUTO_SCALE, stats.get(Stats.Compaction.MAX_SLOTS)); - // Native takes up 1 task slot by default whereas MSQ takes up all available upto 5. + // Native takes up 1 task slot by default whereas MSQ takes up all available upto 5. Since there are 10 available + // slots, there are 10 submitted tasks for native whereas 2 for MSQ. if (engine == CompactionEngine.NATIVE) { Assert.assertEquals(MAXIMUM_CAPACITY_WITH_AUTO_SCALE, stats.get(Stats.Compaction.SUBMITTED_TASKS)); } else { Assert.assertEquals( - MAXIMUM_CAPACITY_WITH_AUTO_SCALE / CompactSegments.MAX_TASK_SLOTS_FOR_MSQ_COMPACTION_TASK, + MAXIMUM_CAPACITY_WITH_AUTO_SCALE / ClientMSQContext.MAX_TASK_SLOTS_FOR_MSQ_COMPACTION_TASK, stats.get(Stats.Compaction.SUBMITTED_TASKS) ); } @@ -1210,9 +1213,9 @@ public void testRunParallelCompactionMultipleCompactionTaskSlots() final CoordinatorRunStats stats; // Native uses maxNumConcurrentSubTasks for task slots whereas MSQ uses maxNumTasks. if (engine == CompactionEngine.NATIVE) { - stats = doCompactSegments(compactSegments, createCompactionConfigs(2, null), 4); + stats = doCompactSegments(compactSegments, createcompactionConfigsForNative(2), 4); } else { - stats = doCompactSegments(compactSegments, createCompactionConfigs(null, 2), 4); + stats = doCompactSegments(compactSegments, createcompactionConfigsForMSQ(2), 4); } Assert.assertEquals(4, stats.get(Stats.Compaction.AVAILABLE_SLOTS)); Assert.assertEquals(4, stats.get(Stats.Compaction.MAX_SLOTS)); @@ -1245,7 +1248,7 @@ public void testRunWithLockedIntervals() // is submitted for dataSource_0 CompactSegments compactSegments = new CompactSegments(SEARCH_POLICY, overlordClient); final CoordinatorRunStats stats = - doCompactSegments(compactSegments, createCompactionConfigs(2, null), 4); + doCompactSegments(compactSegments, createcompactionConfigsForNative(2), 4); Assert.assertEquals(1, stats.get(Stats.Compaction.SUBMITTED_TASKS)); Assert.assertEquals(1, overlordClient.submittedCompactionTasks.size()); @@ -1959,8 +1962,18 @@ private List createCompactionConfigs() return createCompactionConfigs(null, null); } + private List createcompactionConfigsForNative(@Nullable Integer maxNumConcurrentSubTasks) + { + return createCompactionConfigs(maxNumConcurrentSubTasks, null); + } + + private List createcompactionConfigsForMSQ(Integer maxNumTasks) + { + return createCompactionConfigs(null, maxNumTasks); + } + private List createCompactionConfigs( - @Nullable Integer maxNumConcurrentSubTasks, + @Nullable Integer maxNumConcurrentSubTasksForNative, @Nullable Integer maxNumTasksForMSQ ) { @@ -1986,7 +1999,7 @@ private List createCompactionConfigs( null, null, null, - maxNumConcurrentSubTasks, + maxNumConcurrentSubTasksForNative, null, null, null, diff --git a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java index 2ec14003314d..17db22854779 100644 --- a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java @@ -29,7 +29,6 @@ import org.apache.druid.error.ErrorResponse; import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.metadata.MetadataStorageConnector; import org.apache.druid.metadata.MetadataStorageTablesConfig; @@ -457,7 +456,7 @@ public void testAddOrUpdateCompactionConfigWithoutExistingConfigAndEngineAsNull( null, ImmutableMap.of("key", "val") ); - Response ignore = coordinatorCompactionConfigsResource.addOrUpdateCompactionConfig( + coordinatorCompactionConfigsResource.addOrUpdateCompactionConfig( newConfig, mockHttpServletRequest ); @@ -480,16 +479,6 @@ public void testAddOrUpdateCompactionConfigWithInvalidMaxNumTasksForMSQEngine() ArgumentMatchers.eq(CoordinatorCompactionConfig.empty()) ) ).thenReturn(CoordinatorCompactionConfig.empty()); - final ArgumentCaptor oldConfigCaptor = ArgumentCaptor.forClass(byte[].class); - final ArgumentCaptor newConfigCaptor = ArgumentCaptor.forClass( - CoordinatorCompactionConfig.class); - Mockito.when(mockJacksonConfigManager.set( - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY), - oldConfigCaptor.capture(), - newConfigCaptor.capture(), - ArgumentMatchers.any() - ) - ).thenReturn(ConfigManager.SetResult.ok()); int maxNumTasks = 1; @@ -514,11 +503,8 @@ public void testAddOrUpdateCompactionConfigWithInvalidMaxNumTasksForMSQEngine() ); Assert.assertEquals(DruidException.Category.INVALID_INPUT.getExpectedStatus(), response.getStatus()); Assert.assertEquals( - StringUtils.format( - "Compaction config not supported. Reason[MSQ context maxNumTasks [%,d] cannot be less than 2, " - + "since at least 1 controller and 1 worker is necessary.].", - maxNumTasks - ), + "Compaction config not supported. Reason[MSQ context maxNumTasks [1] cannot be less than 2, " + + "since at least 1 controller and 1 worker is necessary.].", ((ErrorResponse) response.getEntity()).getUnderlyingException().getMessage() ); } From eb816b3a92073ee86b2e5c215228fc2f79e486f9 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Thu, 11 Jul 2024 13:17:16 +0530 Subject: [PATCH 52/54] Fix code coverage --- .../msq/indexing/MSQCompactionRunnerTest.java | 50 +++++++++++++------ 1 file changed, 35 insertions(+), 15 deletions(-) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java index a2ceeff3fb30..35eca8cfcb4f 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java @@ -60,6 +60,7 @@ import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.transform.TransformSpec; +import org.apache.druid.server.coordinator.CompactionConfigValidationResult; import org.apache.druid.sql.calcite.parser.DruidSqlInsert; import org.joda.time.Interval; import org.junit.Assert; @@ -92,7 +93,7 @@ public class MSQCompactionRunnerTest private static final List DIMENSIONS = ImmutableList.of(DIM1, LONG_DIMENSION_SCHEMA); private static final ObjectMapper JSON_MAPPER = new DefaultObjectMapper(); private static final AggregatorFactory AGG1 = new CountAggregatorFactory("agg_0"); - private static final AggregatorFactory AGG2 = new LongSumAggregatorFactory("agg_1", "long_dim_1"); + private static final AggregatorFactory AGG2 = new LongSumAggregatorFactory("sum_added", "sum_added"); private static final List AGGREGATORS = ImmutableList.of(AGG1, AGG2); private static final MSQCompactionRunner MSQ_COMPACTION_RUNNER = new MSQCompactionRunner(JSON_MAPPER, null); @@ -116,7 +117,7 @@ public static void setupClass() } @Test - public void testHashPartitionsSpec() + public void testHashedPartitionsSpecIsInvalid() { CompactionTask compactionTask = createCompactionTask( new HashedPartitionsSpec(3, null, ImmutableList.of("dummy")), @@ -129,7 +130,7 @@ public void testHashPartitionsSpec() } @Test - public void testDimensionRangePartitionsSpec() + public void testDimensionRangePartitionsSpecIsValid() { CompactionTask compactionTask = createCompactionTask( new DimensionRangePartitionsSpec(TARGET_ROWS_PER_SEGMENT, null, PARTITION_DIMENSIONS, false), @@ -142,7 +143,7 @@ public void testDimensionRangePartitionsSpec() } @Test - public void testInvalidDynamicPartitionsSpec() + public void testMaxTotalRowsIsInvalid() { CompactionTask compactionTask = createCompactionTask( new DynamicPartitionsSpec(3, 3L), @@ -155,7 +156,7 @@ public void testInvalidDynamicPartitionsSpec() } @Test - public void testDynamicPartitionsSpec() + public void testDynamicPartitionsSpecIsValid() { CompactionTask compactionTask = createCompactionTask( new DynamicPartitionsSpec(3, null), @@ -168,7 +169,7 @@ public void testDynamicPartitionsSpec() } @Test - public void testQueryGranularityAll() + public void testQueryGranularityAllIsValid() { CompactionTask compactionTask = createCompactionTask( new DynamicPartitionsSpec(3, null), @@ -181,7 +182,7 @@ public void testQueryGranularityAll() } @Test - public void testRollupFalseWithMetricsSpec() + public void testRollupFalseWithMetricsSpecIsInValid() { CompactionTask compactionTask = createCompactionTask( new DynamicPartitionsSpec(3, null), @@ -194,7 +195,28 @@ public void testRollupFalseWithMetricsSpec() } @Test - public void testRunCompactionTasksWithEmptyTaskList() throws Exception + public void testMSQEngineWithUnsupportedMetricsSpecIsInValid() + { + // Aggregators having different input and ouput column names are unsupported. + final String inputColName = "added"; + final String outputColName = "sum_added"; + CompactionTask compactionTask = createCompactionTask( + new DynamicPartitionsSpec(3, null), + null, + Collections.emptyMap(), + new ClientCompactionTaskGranularitySpec(null, null, null), + new AggregatorFactory[]{new LongSumAggregatorFactory(outputColName, inputColName)} + ); + CompactionConfigValidationResult validationResult = MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask); + Assert.assertFalse(validationResult.isValid()); + Assert.assertEquals( + "Different name[sum_added] and fieldName(s)[[added]] for aggregator unsupported for MSQ engine.", + validationResult.getReason() + ); + } + + @Test + public void testRunCompactionTasksWithEmptyTaskListFails() throws Exception { CompactionTask compactionTask = createCompactionTask(null, null, Collections.emptyMap(), null, null); TaskStatus taskStatus = MSQ_COMPACTION_RUNNER.runCompactionTasks(compactionTask, Collections.emptyMap(), null); @@ -202,7 +224,7 @@ public void testRunCompactionTasksWithEmptyTaskList() throws Exception } @Test - public void testMSQControllerTaskSpecWithScan() throws JsonProcessingException + public void testMSQControllerTaskSpecWithScanIsValid() throws JsonProcessingException { DimFilter dimFilter = new SelectorDimFilter("dim1", "foo", null); @@ -221,7 +243,8 @@ public void testMSQControllerTaskSpecWithScan() throws JsonProcessingException new AggregatorFactory[]{}, new UniformGranularitySpec( SEGMENT_GRANULARITY.getDefaultGranularity(), - QUERY_GRANULARITY.getDefaultGranularity(), + null, + false, Collections.singletonList(COMPACTION_INTERVAL) ), new TransformSpec(dimFilter, Collections.emptyList()) @@ -262,15 +285,12 @@ public void testMSQControllerTaskSpecWithScan() throws JsonProcessingException JSON_MAPPER.writeValueAsString(SEGMENT_GRANULARITY.toString()), msqControllerTask.getContext().get(DruidSqlInsert.SQL_INSERT_SEGMENT_GRANULARITY) ); - Assert.assertEquals( - JSON_MAPPER.writeValueAsString(QUERY_GRANULARITY.toString()), - msqControllerTask.getContext().get(DruidSqlInsert.SQL_INSERT_QUERY_GRANULARITY) - ); + Assert.assertNull(msqControllerTask.getContext().get(DruidSqlInsert.SQL_INSERT_QUERY_GRANULARITY)); Assert.assertEquals(WorkerAssignmentStrategy.MAX, actualMSQSpec.getAssignmentStrategy()); } @Test - public void testMSQControllerTaskSpecWithAggregators() throws JsonProcessingException + public void testMSQControllerTaskSpecWithAggregatorsIsValid() throws JsonProcessingException { DimFilter dimFilter = new SelectorDimFilter("dim1", "foo", null); From 607ae3e1b0c7759fad31a116848cc611cf3a7a4a Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Thu, 11 Jul 2024 15:13:18 +0530 Subject: [PATCH 53/54] Fix IT failure --- .../duty/ITAutoCompactionTest.java | 77 +++++++++++++++++-- 1 file changed, 70 insertions(+), 7 deletions(-) diff --git a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java index 9a6f9cabed79..db35569be54d 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java @@ -328,8 +328,8 @@ public void testAutoCompactionRowWithMetricAndRowWithoutMetricShouldPreserveExis } } - @Test(dataProvider = "engine") - public void testAutoCompactionOnlyRowsWithoutMetricShouldAddNewMetrics(CompactionEngine engine) throws Exception + @Test + public void testAutoCompactionOnlyRowsWithoutMetricShouldAddNewMetrics() throws Exception { // added = 31, count = null, sum_added = null loadData(INDEX_TASK_WITHOUT_ROLLUP_FOR_PRESERVE_METRICS); @@ -356,8 +356,7 @@ public void testAutoCompactionOnlyRowsWithoutMetricShouldAddNewMetrics(Compactio new UserCompactionTaskDimensionsConfig(DimensionsSpec.getDefaultSchemas(ImmutableList.of("language"))), null, new AggregatorFactory[] {new CountAggregatorFactory("count"), new LongSumAggregatorFactory("sum_added", "added")}, - false, - engine + false ); // should now only have 1 row after compaction // added = null, count = 2, sum_added = 62 @@ -393,6 +392,67 @@ public void testAutoCompactionOnlyRowsWithoutMetricShouldAddNewMetrics(Compactio } } + @Test(dataProvider = "engine") + public void testAutoCompactionWithMetricColumnSameAsInputColShouldOverwriteInputWithMetrics(CompactionEngine engine) + throws Exception + { + // added = 31, count = null, sum_added = null + loadData(INDEX_TASK_WITHOUT_ROLLUP_FOR_PRESERVE_METRICS); + // added = 31, count = null, sum_added = null + loadData(INDEX_TASK_WITHOUT_ROLLUP_FOR_PRESERVE_METRICS); + if (engine == CompactionEngine.MSQ) { + updateCompactionTaskSlot(0.1, 2, false); + } + try (final Closeable ignored = unloader(fullDatasourceName)) { + final List intervalsBeforeCompaction = coordinator.getSegmentIntervals(fullDatasourceName); + intervalsBeforeCompaction.sort(null); + // 2 segments across 1 days... + verifySegmentsCount(2); + Map queryAndResultFields = ImmutableMap.of( + "%%FIELD_TO_QUERY%%", "added", + "%%EXPECTED_COUNT_RESULT%%", 2, + "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(31))), ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(31)))) + ); + verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); + + submitCompactionConfig( + MAX_ROWS_PER_SEGMENT_COMPACTED, + NO_SKIP_OFFSET, + new UserCompactionTaskGranularityConfig(null, null, true), + new UserCompactionTaskDimensionsConfig(DimensionsSpec.getDefaultSchemas(ImmutableList.of("language"))), + null, + new AggregatorFactory[] {new CountAggregatorFactory("count"), new LongSumAggregatorFactory("added", "added")}, + false, + engine + ); + // should now only have 1 row after compaction + // count = 2, added = 62 + forceTriggerAutoCompaction(1); + + queryAndResultFields = ImmutableMap.of( + "%%FIELD_TO_QUERY%%", "count", + "%%EXPECTED_COUNT_RESULT%%", 1, + "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(2)))) + ); + verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); + queryAndResultFields = ImmutableMap.of( + "%%FIELD_TO_QUERY%%", "added", + "%%EXPECTED_COUNT_RESULT%%", 1, + "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(62)))) + ); + verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); + + verifySegmentsCompacted(1, MAX_ROWS_PER_SEGMENT_COMPACTED); + checkCompactionIntervals(intervalsBeforeCompaction); + + List compactTasksBefore = indexer.getCompleteTasksForDataSource(fullDatasourceName); + // Verify rollup segments does not get compacted again + forceTriggerAutoCompaction(1); + List compactTasksAfter = indexer.getCompleteTasksForDataSource(fullDatasourceName); + Assert.assertEquals(compactTasksAfter.size(), compactTasksBefore.size()); + } + } + @Test public void testAutoCompactionOnlyRowsWithMetricShouldPreserveExistingMetrics() throws Exception { @@ -1408,7 +1468,7 @@ public void testAutoCompactionDutyWithFilter() throws Exception } @Test - public void testAutoCompactionDutyWithMetricsSpec() throws Exception + public void testAutoCompationDutyWithMetricsSpec() throws Exception { loadData(INDEX_TASK_WITH_DIMENSION_SPEC); try (final Closeable ignored = unloader(fullDatasourceName)) { @@ -1601,8 +1661,11 @@ private void submitCompactionConfig(Integer maxRowsPerSegment, Period skipOffset submitCompactionConfig(maxRowsPerSegment, skipOffsetFromLatest, null, null); } - private void submitCompactionConfig(Integer maxRowsPerSegment, Period skipOffsetFromLatest, @Nullable CompactionEngine engine) - throws Exception + private void submitCompactionConfig( + Integer maxRowsPerSegment, + Period skipOffsetFromLatest, + @Nullable CompactionEngine engine + ) throws Exception { submitCompactionConfig(maxRowsPerSegment, skipOffsetFromLatest, null, engine); } From 9c820231d832ad888a43cf7c16ec69a0d4a6f2a0 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Fri, 12 Jul 2024 11:11:58 +0530 Subject: [PATCH 54/54] Fix IT failure --- .../coordinator/duty/ITAutoCompactionTest.java | 14 ++++---------- 1 file changed, 4 insertions(+), 10 deletions(-) diff --git a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java index db35569be54d..230a19236c16 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java @@ -396,9 +396,9 @@ public void testAutoCompactionOnlyRowsWithoutMetricShouldAddNewMetrics() throws public void testAutoCompactionWithMetricColumnSameAsInputColShouldOverwriteInputWithMetrics(CompactionEngine engine) throws Exception { - // added = 31, count = null, sum_added = null + // added = 31 loadData(INDEX_TASK_WITHOUT_ROLLUP_FOR_PRESERVE_METRICS); - // added = 31, count = null, sum_added = null + // added = 31 loadData(INDEX_TASK_WITHOUT_ROLLUP_FOR_PRESERVE_METRICS); if (engine == CompactionEngine.MSQ) { updateCompactionTaskSlot(0.1, 2, false); @@ -421,20 +421,14 @@ public void testAutoCompactionWithMetricColumnSameAsInputColShouldOverwriteInput new UserCompactionTaskGranularityConfig(null, null, true), new UserCompactionTaskDimensionsConfig(DimensionsSpec.getDefaultSchemas(ImmutableList.of("language"))), null, - new AggregatorFactory[] {new CountAggregatorFactory("count"), new LongSumAggregatorFactory("added", "added")}, + new AggregatorFactory[] {new LongSumAggregatorFactory("added", "added")}, false, engine ); // should now only have 1 row after compaction - // count = 2, added = 62 + // added = 62 forceTriggerAutoCompaction(1); - queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "count", - "%%EXPECTED_COUNT_RESULT%%", 1, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(2)))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); queryAndResultFields = ImmutableMap.of( "%%FIELD_TO_QUERY%%", "added", "%%EXPECTED_COUNT_RESULT%%", 1,