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/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 3ca6d5780de1..a30e96860875 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 @@ -193,6 +193,7 @@ import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.server.DruidNode; +import org.apache.druid.sql.calcite.parser.DruidSqlInsert; import org.apache.druid.sql.calcite.planner.ColumnMappings; import org.apache.druid.sql.calcite.rel.DruidQuery; import org.apache.druid.sql.http.ResultFormat; @@ -1557,7 +1558,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 { @@ -1657,9 +1658,11 @@ private static Function, Set> addCompactionStateTo GranularitySpec granularitySpec = new UniformGranularitySpec( segmentGranularity, - dataSchema.getGranularitySpec().getQueryGranularity(), + QueryContext.of(querySpec.getQuery().getContext()) + .getGranularity(DruidSqlInsert.SQL_INSERT_QUERY_GRANULARITY, jsonMapper), dataSchema.getGranularitySpec().isRollup(), - dataSchema.getGranularitySpec().inputIntervals() + // Not using dataSchema.getGranularitySpec().inputIntervals() as that always has ETERNITY + ((DataSourceMSQDestination) querySpec.getDestination()).getReplaceTimeChunks() ); DimensionsSpec dimensionsSpec = dataSchema.getDimensionsSpec(); @@ -1671,9 +1674,9 @@ private static Function, Set> addCompactionStateTo List metricsSpec = dataSchema.getAggregators() == null ? null : jsonMapper.convertValue( - dataSchema.getAggregators(), new TypeReference>() - { - }); + dataSchema.getAggregators(), + new TypeReference>() {} + ); IndexSpec indexSpec = tuningConfig.getIndexSpec(); 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 c64c893e5cd8..01a8743dd865 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; @@ -29,6 +30,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.MSQCompactionRunner; import org.apache.druid.msq.indexing.MSQControllerTask; import org.apache.druid.msq.indexing.MSQWorkerTask; import org.apache.druid.msq.indexing.error.BroadcastTablesTooLargeFault; @@ -192,6 +194,8 @@ public List getJacksonModules() NilInputSource.class ); + module.registerSubtypes(new NamedType(MSQCompactionRunner.class, MSQCompactionRunner.TYPE)); + FAULT_CLASSES.forEach(module::registerSubtypes); module.addSerializer(new CounterSnapshotsSerializer()); return Collections.singletonList(module); 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 new file mode 100644 index 000000000000..ac43e7c864b8 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java @@ -0,0 +1,524 @@ +/* + * 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.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; +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.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.CompactionRunner; +import org.apache.druid.indexing.common.task.CompactionTask; +import org.apache.druid.indexing.common.task.CurrentSubTaskHolder; +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.destination.DataSourceMSQDestination; +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.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.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.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; +import org.joda.time.Interval; + +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.Objects; +import java.util.stream.Collectors; + +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; + // 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 + private final CurrentSubTaskHolder currentSubTaskHolder = new CurrentSubTaskHolder( + (taskObject, config) -> { + final MSQControllerTask msqControllerTask = (MSQControllerTask) taskObject; + msqControllerTask.stopGracefully(config); + }); + + + @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#compactionConfigSupportedByMSQEngine} + * 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 + public CompactionConfigValidationResult validateCompactionTask( + CompactionTask compactionTask + ) + { + List validationResults = new ArrayList<>(); + if (compactionTask.getTuningConfig() != null) { + validationResults.add(ClientCompactionRunnerInfo.validatePartitionsSpecForMSQ( + compactionTask.getTuningConfig().getPartitionsSpec()) + ); + } + if (compactionTask.getGranularitySpec() != null) { + validationResults.add(ClientCompactionRunnerInfo.validateRollupForMSQ( + compactionTask.getMetricsSpec(), + compactionTask.getGranularitySpec().isRollup() + )); + } + validationResults.add(ClientCompactionRunnerInfo.validateMaxNumTasksForMSQ(compactionTask.getContext())); + validationResults.add(ClientCompactionRunnerInfo.validateMetricsSpecForMSQ(compactionTask.getMetricsSpec())); + return validationResults.stream() + .filter(result -> !result.isValid()) + .findFirst() + .orElse(new CompactionConfigValidationResult(true, null)); + } + + @Override + public CurrentSubTaskHolder getCurrentSubTaskHolder() + { + return currentSubTaskHolder; + } + + @Override + public TaskStatus runCompactionTasks( + CompactionTask compactionTask, + Map intervalDataSchemas, + TaskToolbox taskToolbox + ) throws Exception + { + List msqControllerTasks = createMsqControllerTasks(compactionTask, intervalDataSchemas); + + if (msqControllerTasks.isEmpty()) { + 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, + taskToolbox, + currentSubTaskHolder, + compactionTask.getId() + ); + } + + public List createMsqControllerTasks( + CompactionTask compactionTask, + Map intervalDataSchemas + ) throws JsonProcessingException + { + final List msqControllerTasks = new ArrayList<>(); + + for (Map.Entry intervalDataSchema : intervalDataSchemas.entrySet()) { + Query query; + Interval interval = intervalDataSchema.getKey(); + DataSchema dataSchema = intervalDataSchema.getValue(); + + if (isGroupBy(dataSchema)) { + query = buildGroupByQuery(compactionTask, interval, dataSchema); + } else { + query = buildScanQuery(compactionTask, interval, dataSchema); + } + QueryContext compactionTaskContext = new QueryContext(compactionTask.getContext()); + + MSQSpec msqSpec = MSQSpec.builder() + .query(query) + .columnMappings(getColumnMappings(dataSchema)) + .destination(buildMSQDestination(compactionTask, dataSchema)) + .assignmentStrategy(MultiStageQueryContext.getAssignmentStrategy(compactionTaskContext)) + .tuningConfig(buildMSQTuningConfig(compactionTask, compactionTaskContext)) + .build(); + + Map msqControllerTaskContext = createMSQTaskContext(compactionTask, dataSchema); + + MSQControllerTask controllerTask = new MSQControllerTask( + compactionTask.getId(), + msqSpec.withOverriddenContext(msqControllerTaskContext), + null, + msqControllerTaskContext, + null, + null, + null, + msqControllerTaskContext, + injector + ); + msqControllerTasks.add(controllerTask); + } + return msqControllerTasks; + } + + private static DataSourceMSQDestination buildMSQDestination( + CompactionTask compactionTask, + DataSchema dataSchema + ) + { + final Interval replaceInterval = compactionTask.getIoConfig() + .getInputSpec() + .findInterval(compactionTask.getDataSource()); + + return new DataSourceMSQDestination( + dataSchema.getDataSource(), + dataSchema.getGranularitySpec().getSegmentGranularity(), + null, + 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); + + // 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); + final Integer maxNumSegments = MultiStageQueryContext.getMaxNumSegments(compactionTaskContext); + + Integer rowsPerSegment = getRowsPerSegment(compactionTask); + + return new MSQTuningConfig( + maxNumWorkers, + maxRowsInMemory, + rowsPerSegment, + maxNumSegments, + 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(); + 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())); + } + return rowSignatureBuilder.build(); + } + + 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(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)); + } + + dimensionSpecs.addAll(dataSchema.getDimensionsSpec().getDimensions().stream() + .map(dim -> new DefaultDimensionSpec( + dim.getName(), + dim.getName(), + dim.getColumnType() + )) + .collect(Collectors.toList())); + 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) || !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)); + } + return new ColumnMappings(columnMappings); + } + + private static 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(); + } + + private static Query buildScanQuery(CompactionTask compactionTask, Interval interval, DataSchema dataSchema) + { + 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) + .filters(dataSchema.getTransformSpec().getFilter()) + .context(compactionTask.getContext()) + .build(); + } + + private static boolean isGroupBy(DataSchema dataSchema) + { + 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; + } + + private static boolean isQueryGranularityEmptyOrNone(DataSchema dataSchema) + { + return dataSchema.getGranularitySpec() == null + || dataSchema.getGranularitySpec().getQueryGranularity() == null + || Objects.equals( + dataSchema.getGranularitySpec().getQueryGranularity(), + Granularities.NONE + ); + } + + /** + * 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)) { + return VirtualColumns.EMPTY; + } + String virtualColumnExpr; + 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(); + // Round of the __time column according to the required granularity. + virtualColumnExpr = + StringUtils.format( + "timestamp_floor(\"%s\", '%s')", + ColumnHolder.TIME_COLUMN_NAME, + periodQueryGranularity.getPeriod().toString() + ); + } + 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) + { + DimFilter dimFilter = dataSchema.getTransformSpec().getFilter(); + + GroupByQuery.Builder builder = new GroupByQuery.Builder() + .setDataSource(new TableDataSource(compactionTask.getDataSource())) + .setVirtualColumns(getVirtualColumns(dataSchema, interval)) + .setDimFilter(dimFilter) + .setGranularity(new AllGranularity()) + .setDimensions(getAggregateDimensions(dataSchema)) + .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(); + } + + 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, + serializeGranularity(dataSchema.getGranularitySpec() != null + ? dataSchema.getGranularitySpec() + .getSegmentGranularity() + : DEFAULT_SEGMENT_GRANULARITY, jsonMapper) + ); + if (!isQueryGranularityEmptyOrNone(dataSchema)) { + context.put( + DruidSqlInsert.SQL_INSERT_QUERY_GRANULARITY, + 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; + } + + private static TaskStatus runSubtasks( + List tasks, + TaskToolbox toolbox, + CurrentSubTaskHolder currentSubTaskHolder, + String compactionTaskId + ) throws JsonProcessingException + { + final int totalNumSpecs = tasks.size(); + log.info("Generated [%d] MSQControllerTask specs", totalNumSpecs); + + int failCnt = 0; + + 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 + ); + 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/indexing/MSQControllerTask.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java index b9c8ebe3b806..c3f6feaab245 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 @@ -144,6 +144,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/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 f8bdb36bfaea..cbf9a1a905f0 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; @@ -145,7 +147,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/exec/MSQReplaceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java index 2e05d4479103..f6eb80b32826 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 @@ -186,7 +186,8 @@ public void testReplaceOnFooWithAll(String contextName, Map cont context, Collections.emptyList(), Collections.singletonList(new FloatDimensionSchema("m1")), - GranularityType.DAY + GranularityType.DAY, + Intervals.ETERNITY ) ) .verifyResults(); @@ -244,7 +245,8 @@ public void testReplaceOnFooWithWhere(String contextName, Map co context, Collections.emptyList(), Collections.singletonList(new FloatDimensionSchema("m1")), - GranularityType.DAY + GranularityType.DAY, + Intervals.of("2000-01-02T/P1D") ) ) .verifyResults(); @@ -334,7 +336,8 @@ public void testReplaceOnFoo1WithAllExtern(String contextName, Map c context, Collections.emptyList(), Collections.singletonList(new FloatDimensionSchema("m1")), - GranularityType.MONTH + GranularityType.MONTH, + Intervals.of("2000-01-01T/2000-03-01T") ) ) .verifyResults(); @@ -749,7 +756,8 @@ public void testReplaceWhereClauseLargerThanData(String contextName, Map contex context, Collections.emptyList(), Collections.singletonList(new FloatDimensionSchema("m1")), - GranularityType.MONTH + GranularityType.MONTH, + Intervals.of("2000-01-01T/2000-03-01T") ) ) .verifyResults(); @@ -1009,7 +1018,8 @@ public void testReplaceTimeChunksLargerThanData(String contextName, Map conte new StringDimensionSchema("dim1"), new LongDimensionSchema("cnt") ), - GranularityType.DAY + GranularityType.DAY, + Intervals.ETERNITY ) ) .verifyResults(); @@ -1190,7 +1202,8 @@ public void testReplaceSegmentsInsertIntoNewTable(String contextName, Map context, List partitionDimensions, List dimensions, - GranularityType segmentGranularity + GranularityType segmentGranularity, + Interval interval ) { if (!context.containsKey(Tasks.STORE_COMPACTION_STATE_KEY) @@ -2049,7 +2067,7 @@ private CompactionState expectedCompactionState( segmentGranularity.getDefaultGranularity(), GranularityType.NONE.getDefaultGranularity(), false, - Intervals.ONLY_ETERNITY + Collections.singletonList(interval) ); List metricsSpec = Collections.emptyList(); 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 new file mode 100644 index 000000000000..35eca8cfcb4f --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java @@ -0,0 +1,445 @@ +/* + * 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 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; +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.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; +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; +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.server.coordinator.CompactionConfigValidationResult; +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 javax.annotation.Nullable; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class MSQCompactionRunnerTest +{ + 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 final StringDimensionSchema DIM1 = new StringDimensionSchema( + "string_dim", + null, + null + ); + 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("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); + + @BeforeClass + public static void setupClass() + { + NullHandling.initializeForTests(); + + final StringDimensionSchema stringDimensionSchema = new StringDimensionSchema( + "string_dim", + null, + null + ); + + PARTITION_DIMENSIONS = Collections.singletonList(stringDimensionSchema.getName()); + + JSON_MAPPER.setInjectableValues(new InjectableValues.Std().addValue( + ExprMacroTable.class, + LookupEnabledTestExprMacroTable.INSTANCE + )); + } + + @Test + public void testHashedPartitionsSpecIsInvalid() + { + CompactionTask compactionTask = createCompactionTask( + new HashedPartitionsSpec(3, null, ImmutableList.of("dummy")), + null, + Collections.emptyMap(), + null, + null + ); + Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); + } + + @Test + public void testDimensionRangePartitionsSpecIsValid() + { + CompactionTask compactionTask = createCompactionTask( + new DimensionRangePartitionsSpec(TARGET_ROWS_PER_SEGMENT, null, PARTITION_DIMENSIONS, false), + null, + Collections.emptyMap(), + null, + null + ); + Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); + } + + @Test + public void testMaxTotalRowsIsInvalid() + { + CompactionTask compactionTask = createCompactionTask( + new DynamicPartitionsSpec(3, 3L), + null, + Collections.emptyMap(), + null, + null + ); + Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); + } + + @Test + public void testDynamicPartitionsSpecIsValid() + { + CompactionTask compactionTask = createCompactionTask( + new DynamicPartitionsSpec(3, null), + null, + Collections.emptyMap(), + null, + null + ); + Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); + } + + @Test + public void testQueryGranularityAllIsValid() + { + CompactionTask compactionTask = createCompactionTask( + new DynamicPartitionsSpec(3, null), + null, + Collections.emptyMap(), + new ClientCompactionTaskGranularitySpec(null, Granularities.ALL, null), + null + ); + Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); + } + + @Test + public void testRollupFalseWithMetricsSpecIsInValid() + { + CompactionTask compactionTask = createCompactionTask( + new DynamicPartitionsSpec(3, null), + null, + Collections.emptyMap(), + new ClientCompactionTaskGranularitySpec(null, null, false), + AGGREGATORS.toArray(new AggregatorFactory[0]) + ); + Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); + } + + @Test + 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); + Assert.assertTrue(taskStatus.isFailure()); + } + + @Test + public void testMSQControllerTaskSpecWithScanIsValid() 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(), + null, + null + ); + + DataSchema dataSchema = new DataSchema( + DATA_SOURCE, + new TimestampSpec(TIMESTAMP_COLUMN, null, null), + new DimensionsSpec(DIMENSIONS), + new AggregatorFactory[]{}, + new UniformGranularitySpec( + SEGMENT_GRANULARITY.getDefaultGranularity(), + null, + false, + Collections.singletonList(COMPACTION_INTERVAL) + ), + new TransformSpec(dimFilter, Collections.emptyList()) + ); + + + List msqControllerTasks = MSQ_COMPACTION_RUNNER.createMsqControllerTasks( + taskCreatedWithTransformSpec, + Collections.singletonMap(COMPACTION_INTERVAL, dataSchema) + ); + + MSQControllerTask msqControllerTask = Iterables.getOnlyElement(msqControllerTasks); + + MSQSpec actualMSQSpec = msqControllerTask.getQuerySpec(); + + Assert.assertEquals( + new MSQTuningConfig( + 1, + MultiStageQueryContext.DEFAULT_ROWS_IN_MEMORY, + TARGET_ROWS_PER_SEGMENT, + null, + createIndexSpec() + ), + actualMSQSpec.getTuningConfig() + ); + Assert.assertEquals( + new DataSourceMSQDestination( + DATA_SOURCE, + SEGMENT_GRANULARITY.getDefaultGranularity(), + null, + 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.assertNull(msqControllerTask.getContext().get(DruidSqlInsert.SQL_INSERT_QUERY_GRANULARITY)); + Assert.assertEquals(WorkerAssignmentStrategy.MAX, actualMSQSpec.getAssignmentStrategy()); + } + + @Test + public void testMSQControllerTaskSpecWithAggregatorsIsValid() 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(), + null, + null + ); + + 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 = MSQ_COMPACTION_RUNNER.createMsqControllerTasks( + taskCreatedWithTransformSpec, + Collections.singletonMap(COMPACTION_INTERVAL, dataSchema) + ); + + MSQControllerTask msqControllerTask = Iterables.getOnlyElement(msqControllerTasks); + + MSQSpec actualMSQSpec = msqControllerTask.getQuerySpec(); + + Assert.assertEquals( + new MSQTuningConfig( + 1, + MultiStageQueryContext.DEFAULT_ROWS_IN_MEMORY, + TARGET_ROWS_PER_SEGMENT, + null, + createIndexSpec() + ), + actualMSQSpec.getTuningConfig() + ); + Assert.assertEquals( + new DataSourceMSQDestination( + DATA_SOURCE, + SEGMENT_GRANULARITY.getDefaultGranularity(), + null, + 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 CompactionTask createCompactionTask( + @Nullable PartitionsSpec partitionsSpec, + @Nullable DimFilter dimFilter, + Map contextParams, + @Nullable ClientCompactionTaskGranularitySpec granularitySpec, + @Nullable AggregatorFactory[] metricsSpec + ) + { + ClientCompactionTaskTransformSpec transformSpec = + new ClientCompactionTaskTransformSpec(dimFilter); + final CompactionTask.Builder builder = new CompactionTask.Builder( + DATA_SOURCE, + null, + null + ); + IndexSpec indexSpec = createIndexSpec(); + + Map context = new HashMap<>(); + 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) + .granularitySpec(granularitySpec) + .metricsSpec(metricsSpec) + .compactionRunner(new MSQCompactionRunner(JSON_MAPPER, null)) + .context(context); + + return builder.build(); + } + + private static CompactionTask.CompactionTuningConfig createTuningConfig( + IndexSpec indexSpec, + PartitionsSpec partitionsSpec + ) + { + return new CompactionTask.CompactionTuningConfig( + null, + null, // null to compute maxRowsPerSegment automatically + null, + 500000, + 1000000L, + null, + null, + null, + null, + partitionsSpec, + indexSpec, + null, + null, + !(partitionsSpec instanceof DynamicPartitionsSpec), + false, + 5000L, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + 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 new file mode 100644 index 000000000000..8d30a60d04e6 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java @@ -0,0 +1,62 @@ +/* + * 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.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.server.coordinator.CompactionConfigValidationResult; +import org.joda.time.Interval; + +import java.util.Map; + +/** + * Strategy to be used for executing a compaction task. + * Should be synchronized with {@link ClientCompactionRunnerInfo} + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = CompactionRunner.TYPE_PROPERTY) +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = NativeCompactionRunner.TYPE, value = NativeCompactionRunner.class) +}) +public interface CompactionRunner +{ + String TYPE_PROPERTY = "type"; + + /** + * Creates and runs sub-tasks for the given CompactionTask, one interval at a time. + */ + TaskStatus runCompactionTasks( + CompactionTask compactionTask, + Map intervalDataSchemaMap, + TaskToolbox taskToolbox + ) throws Exception; + + CurrentSubTaskHolder getCurrentSubTaskHolder(); + + /** + * 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} + */ + 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 81447f3fd5eb..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 @@ -34,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; @@ -43,12 +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.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.indexer.report.TaskReport; import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.RetryPolicyFactory; import org.apache.druid.indexing.common.SegmentCacheManagerFactory; @@ -56,11 +54,7 @@ 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; @@ -91,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.CompactionConfigValidationResult; import org.apache.druid.server.coordinator.duty.CompactSegments; import org.apache.druid.server.lookup.cache.LookupLoadingSpec; import org.apache.druid.server.security.ResourceAction; @@ -98,7 +93,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; @@ -116,7 +110,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; @@ -147,8 +140,6 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg */ public static final String CTX_KEY_APPENDERATOR_TRACKING_TASK_ID = "appenderatorTrackingTaskId"; - private static final boolean STORE_COMPACTION_STATE = true; - static { Verify.verify(TYPE.equals(CompactSegments.COMPACTION_TASK_TYPE)); } @@ -164,21 +155,12 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg private final ClientCompactionTaskGranularitySpec granularitySpec; @Nullable private final CompactionTuningConfig tuningConfig; + @Nullable + private final CompactionRunner compactionRunner; @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( @@ -196,6 +178,7 @@ 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, @JacksonInject SegmentCacheManagerFactory segmentCacheManagerFactory ) { @@ -248,8 +231,12 @@ public CompactionTask( } this.tuningConfig = tuningConfig != null ? getTuningConfig(tuningConfig) : null; this.segmentProvider = new SegmentProvider(dataSource, this.ioConfig.getInputSpec()); - this.partitionConfigurationManager = new PartitionConfigurationManager(this.tuningConfig); - this.segmentCacheManagerFactory = segmentCacheManagerFactory; + // 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; + 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 @@ -397,11 +384,17 @@ public ClientCompactionTaskGranularitySpec getGranularitySpec() @Nullable @JsonProperty - public ParallelIndexTuningConfig getTuningConfig() + public CompactionTuningConfig getTuningConfig() { return tuningConfig; } + @JsonProperty + public CompactionRunner getCompactionRunner() + { + return compactionRunner; + } + @Override public String getType() { @@ -472,144 +465,44 @@ 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 Map 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); - } else { - registerResourceCloserOnAbnormalExit(currentSubTaskHolder); - final int totalNumSpecs = indexTaskSpecs.size(); - log.info("Generated [%d] compaction task specs", totalNumSpecs); - - int failCnt = 0; - final TaskReport.ReportMap completionReports = new TaskReport.ReportMap(); - for (int i = 0; i < indexTaskSpecs.size(); i++) { - ParallelIndexSupervisorTask eachSpec = indexTaskSpecs.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); + registerResourceCloserOnAbnormalExit(compactionRunner.getCurrentSubTaskHolder()); + CompactionConfigValidationResult supportsCompactionConfig = compactionRunner.validateCompactionTask(this); + if (!supportsCompactionConfig.isValid()) { + throw InvalidInput.exception("Compaction spec not supported. Reason[%s].", supportsCompactionConfig.getReason()); } - } - - @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; - } - - private String createIndexTaskSpecId(int i) - { - return StringUtils.format("%s_%d", getId(), i); + return compactionRunner.runCompactionTasks(this, intervalDataSchemas, toolbox); } /** - * 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 createIngestionSchema( + static Map 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 { @@ -619,14 +512,12 @@ static List createIngestionSchema( lockGranularityInUse ); - if (timelineSegments.size() == 0) { - return Collections.emptyList(); + if (timelineSegments.isEmpty()) { + return Collections.emptyMap(); } - final CompactionTuningConfig compactionTuningConfig = partitionConfigurationManager.computeTuningConfig(); - if (granularitySpec == null || granularitySpec.getSegmentGranularity() == null) { - final List specs = new ArrayList<>(); + Map intervalDataSchemaMap = new HashMap<>(); // original granularity final Map> intervalToSegments = new TreeMap<>( @@ -679,24 +570,9 @@ static List createIngestionSchema( ? new ClientCompactionTaskGranularitySpec(segmentGranularityToUse, null, null) : granularitySpec.withSegmentGranularity(segmentGranularityToUse) ); - - specs.add( - new ParallelIndexIngestionSpec( - dataSchema, - createIoConfig( - toolbox, - dataSchema, - interval, - coordinatorClient, - segmentCacheManagerFactory, - ioConfig - ), - compactionTuningConfig - ) - ); + intervalDataSchemaMap.put(interval, dataSchema); } - - return specs; + return intervalDataSchemaMap; } else { // given segment granularity final DataSchema dataSchema = createDataSchema( @@ -720,68 +596,8 @@ static List createIngestionSchema( metricsSpec, granularitySpec ); - - return Collections.singletonList( - new ParallelIndexIngestionSpec( - dataSchema, - createIoConfig( - toolbox, - dataSchema, - segmentProvider.interval, - coordinatorClient, - segmentCacheManagerFactory, - ioConfig - ), - compactionTuningConfig - ) - ); - } - } - - 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 Collections.singletonMap(segmentProvider.interval, dataSchema); } - - 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( @@ -1213,35 +1029,6 @@ 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; @@ -1263,6 +1050,7 @@ public static class Builder private TuningConfig tuningConfig; @Nullable private Map context; + private CompactionRunner compactionRunner; public Builder( String dataSource, @@ -1345,6 +1133,12 @@ public Builder context(Map context) return this; } + public Builder compactionRunner(CompactionRunner compactionRunner) + { + this.compactionRunner = compactionRunner; + return this; + } + public CompactionTask build() { return new CompactionTask( @@ -1362,6 +1156,7 @@ public CompactionTask build() granularitySpec, tuningConfig, context, + compactionRunner, segmentCacheManagerFactory ); } 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 new file mode 100644 index 000000000000..722c6010b206 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java @@ -0,0 +1,333 @@ +/* + * 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.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; +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.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.CompactionConfigValidationResult; +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 javax.annotation.Nullable; +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; + +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; + @JsonIgnore + private final CurrentSubTaskHolder currentSubTaskHolder = new CurrentSubTaskHolder( + (taskObject, config) -> { + final ParallelIndexSupervisorTask indexTask = (ParallelIndexSupervisorTask) taskObject; + indexTask.stopGracefully(config); + }); + + @JsonCreator + public NativeCompactionRunner(@JacksonInject SegmentCacheManagerFactory segmentCacheManagerFactory) + { + this.segmentCacheManagerFactory = segmentCacheManagerFactory; + } + + @Override + public CurrentSubTaskHolder getCurrentSubTaskHolder() + { + return currentSubTaskHolder; + } + + @Override + public CompactionConfigValidationResult validateCompactionTask( + CompactionTask compactionTask + ) + { + return new CompactionConfigValidationResult(true, null); + } + + /** + * Generate {@link ParallelIndexIngestionSpec} from input dataschemas. + * + * @return an empty list if input segments don't exist. Otherwise, a generated ingestionSpec. + */ + @VisibleForTesting + static List createIngestionSpecs( + Map intervalDataSchemaMap, + final TaskToolbox toolbox, + final CompactionIOConfig ioConfig, + final PartitionConfigurationManager partitionConfigurationManager, + final CoordinatorClient coordinatorClient, + final SegmentCacheManagerFactory segmentCacheManagerFactory + ) + { + final CompactionTask.CompactionTuningConfig compactionTuningConfig = partitionConfigurationManager.computeTuningConfig(); + + return intervalDataSchemaMap.entrySet().stream().map((dataSchema) -> new ParallelIndexIngestionSpec( + dataSchema.getValue(), + createIoConfig( + toolbox, + dataSchema.getValue(), + dataSchema.getKey(), + 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, + Map intervalDataSchemaMap, + TaskToolbox taskToolbox + ) throws Exception + { + final PartitionConfigurationManager partitionConfigurationManager = + new NativeCompactionRunner.PartitionConfigurationManager(compactionTask.getTuningConfig()); + + + final List ingestionSpecs = createIngestionSpecs( + intervalDataSchemaMap, + taskToolbox, + compactionTask.getIoConfig(), + partitionConfigurationManager, + taskToolbox.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(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, + currentSubTaskHolder, + 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(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()); + 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 71514af17b42..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; @@ -40,6 +41,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; @@ -329,7 +331,8 @@ private ClientCompactionTaskQuery createCompactionTaskQuery(String id, ClientCom new ClientCompactionTaskDimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), METRICS_SPEC, transformSpec, - context + context, + new ClientCompactionRunnerInfo(CompactionEngine.NATIVE) ); } 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 134f5305169d..48a7932a2414 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 @@ -68,9 +68,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.CompactionTask.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; @@ -447,8 +447,7 @@ public void testCreateCompactionTaskWithConflictingGranularitySpecAndSegmentGran { final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, - RETRY_POLICY_FACTORY + segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); builder.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); builder.tuningConfig(createTuningConfig()); @@ -478,8 +477,7 @@ public void testCreateCompactionTaskWithTransformSpec() new ClientCompactionTaskTransformSpec(new SelectorDimFilter("dim1", "foo", null)); final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, - RETRY_POLICY_FACTORY + segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); builder.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); builder.tuningConfig(createTuningConfig()); @@ -911,21 +909,27 @@ public void testSegmentProviderFindSegmentsWithEmptySegmentsThrowException() @Test public void testCreateIngestionSchema() throws IOException { - final List ingestionSpecs = CompactionTask.createIngestionSchema( + final Map 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 = NativeCompactionRunner.createIngestionSpecs( + dataSchemasForIntervals, + toolbox, + new CompactionIOConfig(null, false, null), + new PartitionConfigurationManager(TUNING_CONFIG), + COORDINATOR_CLIENT, + segmentCacheManagerFactory + ); + final List expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(); ingestionSpecs.sort( @@ -987,21 +991,27 @@ public void testCreateIngestionSchemaWithTargetPartitionSize() throws IOExceptio null, null ); - final List ingestionSpecs = CompactionTask.createIngestionSchema( + final Map 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 = NativeCompactionRunner.createIngestionSpecs( + dataSchemasForIntervals, + toolbox, + new CompactionIOConfig(null, false, null), + new PartitionConfigurationManager(tuningConfig), + COORDINATOR_CLIENT, + segmentCacheManagerFactory + ); + final List expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(); ingestionSpecs.sort( @@ -1064,21 +1074,27 @@ public void testCreateIngestionSchemaWithMaxTotalRows() throws IOException null, null ); - final List ingestionSpecs = CompactionTask.createIngestionSchema( + final Map 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 = NativeCompactionRunner.createIngestionSpecs( + dataSchemasForIntervals, + toolbox, + new CompactionIOConfig(null, false, null), + new PartitionConfigurationManager(tuningConfig), + COORDINATOR_CLIENT, + segmentCacheManagerFactory + ); + final List expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(); ingestionSpecs.sort( @@ -1141,21 +1157,26 @@ public void testCreateIngestionSchemaWithNumShards() throws IOException null, null ); - final List ingestionSpecs = CompactionTask.createIngestionSchema( + final Map 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 = NativeCompactionRunner.createIngestionSpecs( + dataSchemasForIntervals, + toolbox, + new CompactionIOConfig(null, false, null), + new PartitionConfigurationManager(tuningConfig), + COORDINATOR_CLIENT, + segmentCacheManagerFactory + ); final List expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(); ingestionSpecs.sort( @@ -1207,22 +1228,27 @@ public void testCreateIngestionSchemaWithCustomDimensionsSpec() throws IOExcepti ) ); - final List ingestionSpecs = CompactionTask.createIngestionSchema( + final Map 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 = NativeCompactionRunner.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), @@ -1253,22 +1279,27 @@ public void testCreateIngestionSchemaWithCustomMetricsSpec() throws IOException new DoubleMaxAggregatorFactory("custom_double_max", "agg_4") }; - final List ingestionSpecs = CompactionTask.createIngestionSchema( + final Map 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 = NativeCompactionRunner.createIngestionSpecs( + dataSchemasForIntervals, + toolbox, + new CompactionIOConfig(null, false, null), + new PartitionConfigurationManager(TUNING_CONFIG), + COORDINATOR_CLIENT, + segmentCacheManagerFactory + ); + final List expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(); ingestionSpecs.sort( @@ -1292,21 +1323,27 @@ public void testCreateIngestionSchemaWithCustomMetricsSpec() throws IOException @Test public void testCreateIngestionSchemaWithCustomSegments() throws IOException { - final List ingestionSpecs = CompactionTask.createIngestionSchema( + final Map 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 = NativeCompactionRunner.createIngestionSpecs( + dataSchemasForIntervals, + toolbox, + new CompactionIOConfig(null, false, null), + new PartitionConfigurationManager(TUNING_CONFIG), + COORDINATOR_CLIENT, + segmentCacheManagerFactory + ); + final List expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(); ingestionSpecs.sort( @@ -1337,21 +1374,26 @@ public void testCreateIngestionSchemaWithDifferentSegmentSet() throws IOExceptio Collections.sort(segments); // Remove one segment in the middle segments.remove(segments.size() / 2); - CompactionTask.createIngestionSchema( + final Map 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 ); + + NativeCompactionRunner.createIngestionSpecs( + dataSchemasForIntervals, + toolbox, + new CompactionIOConfig(null, false, null), + new PartitionConfigurationManager(TUNING_CONFIG), + COORDINATOR_CLIENT, + segmentCacheManagerFactory + ); } @Test @@ -1363,21 +1405,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.createIngestionSchema( + final Map 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 ); + + NativeCompactionRunner.createIngestionSpecs( + dataSchemasForIntervals, + toolbox, + new CompactionIOConfig(null, false, null), + new PartitionConfigurationManager(TUNING_CONFIG), + COORDINATOR_CLIENT, + segmentCacheManagerFactory + ); } @Test @@ -1388,8 +1435,7 @@ public void testEmptyInterval() final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, - RETRY_POLICY_FACTORY + segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); @SuppressWarnings("unused") @@ -1401,21 +1447,26 @@ public void testEmptyInterval() @Test public void testSegmentGranularityAndNullQueryGranularity() throws IOException { - final List ingestionSpecs = CompactionTask.createIngestionSchema( + final Map 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 = NativeCompactionRunner.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"))) ); @@ -1441,21 +1492,25 @@ public void testSegmentGranularityAndNullQueryGranularity() throws IOException @Test public void testQueryGranularityAndNullSegmentGranularity() throws IOException { - final List ingestionSpecs = CompactionTask.createIngestionSchema( + final Map 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 = NativeCompactionRunner.createIngestionSpecs( + dataSchemasForIntervals, + toolbox, + new CompactionIOConfig(null, false, null), + new PartitionConfigurationManager(TUNING_CONFIG), + COORDINATOR_CLIENT, + segmentCacheManagerFactory + ); final List expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(); ingestionSpecs.sort( @@ -1479,13 +1534,11 @@ public void testQueryGranularityAndNullSegmentGranularity() throws IOException @Test public void testQueryGranularityAndSegmentGranularityNonNull() throws IOException { - final List ingestionSpecs = CompactionTask.createIngestionSchema( + final Map 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, @@ -1494,10 +1547,19 @@ public void testQueryGranularityAndSegmentGranularityNonNull() throws IOExceptio new PeriodGranularity(Period.months(3), null, null), null ), - COORDINATOR_CLIENT, - segmentCacheManagerFactory, METRIC_BUILDER ); + + final List ingestionSpecs = NativeCompactionRunner.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"))) ); @@ -1524,21 +1586,28 @@ public void testQueryGranularityAndSegmentGranularityNonNull() throws IOExceptio @Test public void testNullGranularitySpec() throws IOException { - final List ingestionSpecs = CompactionTask.createIngestionSchema( + final Map 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 = NativeCompactionRunner.createIngestionSpecs( + dataSchemasForIntervals, + toolbox, + new CompactionIOConfig(null, false, null), + new PartitionConfigurationManager(TUNING_CONFIG), + COORDINATOR_CLIENT, + segmentCacheManagerFactory + ); + + final List expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(); ingestionSpecs.sort( @@ -1563,21 +1632,28 @@ public void testNullGranularitySpec() throws IOException public void testGranularitySpecWithNullQueryGranularityAndNullSegmentGranularity() throws IOException { - final List ingestionSpecs = CompactionTask.createIngestionSchema( + final Map 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 = NativeCompactionRunner.createIngestionSpecs( + dataSchemasForIntervals, + toolbox, + new CompactionIOConfig(null, false, null), + new PartitionConfigurationManager(TUNING_CONFIG), + COORDINATOR_CLIENT, + segmentCacheManagerFactory + ); + + final List expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(); ingestionSpecs.sort( @@ -1602,22 +1678,27 @@ public void testGranularitySpecWithNullQueryGranularityAndNullSegmentGranularity public void testGranularitySpecWithNotNullRollup() throws IOException { - final List ingestionSpecs = CompactionTask.createIngestionSchema( + final Map 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 = NativeCompactionRunner.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()); @@ -1628,21 +1709,28 @@ public void testGranularitySpecWithNotNullRollup() public void testGranularitySpecWithNullRollup() throws IOException { - final List ingestionSpecs = CompactionTask.createIngestionSchema( + final Map 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 = NativeCompactionRunner.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 @@ -1701,8 +1789,7 @@ public void testGetDefaultLookupLoadingSpec() { final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, - RETRY_POLICY_FACTORY + segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); final CompactionTask task = builder .interval(Intervals.of("2000-01-01/2000-01-02")) @@ -1715,8 +1802,7 @@ public void testGetDefaultLookupLoadingSpecWithTransformSpec() { final Builder builder = new Builder( DATA_SOURCE, - segmentCacheManagerFactory, - RETRY_POLICY_FACTORY + segmentCacheManagerFactory, RETRY_POLICY_FACTORY ); final CompactionTask task = builder .interval(Intervals.of("2000-01-01/2000-01-02")) 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/main/java/org/apache/druid/testing/utils/CompactionUtil.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/CompactionUtil.java index a57a404b61ea..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 @@ -75,6 +75,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..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 @@ -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; @@ -383,6 +392,61 @@ public void testAutoCompactionOnlyRowsWithoutMetricShouldAddNewMetrics() throws } } + @Test(dataProvider = "engine") + public void testAutoCompactionWithMetricColumnSameAsInputColShouldOverwriteInputWithMetrics(CompactionEngine engine) + throws Exception + { + // added = 31 + loadData(INDEX_TASK_WITHOUT_ROLLUP_FOR_PRESERVE_METRICS); + // added = 31 + 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 LongSumAggregatorFactory("added", "added")}, + false, + engine + ); + // should now only have 1 row after compaction + // added = 62 + forceTriggerAutoCompaction(1); + + 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 { @@ -546,8 +610,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 +621,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) @@ -1398,7 +1462,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)) { @@ -1585,24 +1649,115 @@ 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) + 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, + @Nullable 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( @@ -1615,6 +1770,31 @@ private void submitCompactionConfig( AggregatorFactory[] metricsSpec, boolean dropExisting ) throws Exception + { + submitCompactionConfig( + partitionsSpec, + skipOffsetFromLatest, + maxNumConcurrentSubTasks, + granularitySpec, + dimensionsSpec, + transformSpec, + metricsSpec, + dropExisting, + null + ); + } + + private void submitCompactionConfig( + PartitionsSpec partitionsSpec, + Period skipOffsetFromLatest, + int maxNumConcurrentSubTasks, + UserCompactionTaskGranularityConfig granularitySpec, + UserCompactionTaskDimensionsConfig dimensionsSpec, + UserCompactionTaskTransformConfig transformSpec, + AggregatorFactory[] metricsSpec, + boolean dropExisting, + @Nullable CompactionEngine engine + ) throws Exception { DataSourceCompactionConfig compactionConfig = new DataSourceCompactionConfig( fullDatasourceName, @@ -1648,6 +1828,7 @@ private void submitCompactionConfig( metricsSpec, transformSpec, !dropExisting ? null : new UserCompactionTaskIOConfig(true), + engine, 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/processing/src/main/java/org/apache/druid/indexer/CompactionEngine.java b/processing/src/main/java/org/apache/druid/indexer/CompactionEngine.java new file mode 100644 index 000000000000..a95f64b896a7 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/indexer/CompactionEngine.java @@ -0,0 +1,49 @@ +/* + * 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 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 +{ + NATIVE, + MSQ; + + @Override + @JsonValue + public String toString() + { + return StringUtils.toLowerCase(this.name()); + } + + @JsonCreator + 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/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/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..b97a6c6a5eab --- /dev/null +++ b/processing/src/test/java/org/apache/druid/indexer/CompactionEngineTest.java @@ -0,0 +1,43 @@ +/* + * 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.Assert; +import org.junit.Test; + +public class CompactionEngineTest +{ + + @Test + public void testFromString() + { + 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() + { + Assert.assertEquals("native", CompactionEngine.NATIVE.toString()); + Assert.assertEquals("msq", CompactionEngine.MSQ.toString()); + } +} 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 new file mode 100644 index 000000000000..ed9e22dfaa29 --- /dev/null +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java @@ -0,0 +1,216 @@ +/* + * 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; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +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 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.Arrays; +import java.util.List; +import java.util.Map; +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 + 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 CompactionConfigValidationResult validateCompactionConfig( + DataSourceCompactionConfig newConfig, + CompactionEngine defaultCompactionEngine + ) + { + CompactionEngine compactionEngine = newConfig.getEngine() == null ? defaultCompactionEngine : newConfig.getEngine(); + if (compactionEngine == CompactionEngine.NATIVE) { + return new CompactionConfigValidationResult(true, null); + } else { + return compactionConfigSupportedByMSQEngine(newConfig); + } + } + + /** + * Checks if the provided compaction config is supported by MSQ. 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.
  • + *
+ */ + private static CompactionConfigValidationResult compactionConfigSupportedByMSQEngine(DataSourceCompactionConfig newConfig) + { + List validationResults = new ArrayList<>(); + if (newConfig.getTuningConfig() != null) { + validationResults.add(validatePartitionsSpecForMSQ(newConfig.getTuningConfig().getPartitionsSpec())); + } + if (newConfig.getGranularitySpec() != null) { + validationResults.add(validateRollupForMSQ( + newConfig.getMetricsSpec(), + newConfig.getGranularitySpec().isRollup() + )); + } + validationResults.add(validateMaxNumTasksForMSQ(newConfig.getTaskContext())); + validationResults.add(validateMetricsSpecForMSQ(newConfig.getMetricsSpec())); + return validationResults.stream() + .filter(result -> !result.isValid()) + .findFirst() + .orElse(new CompactionConfigValidationResult(true, null)); + } + + /** + * Validate that partitionSpec is either 'dynamic` or 'range', and if 'dynamic', ensure 'maxTotalRows' is null. + */ + public static CompactionConfigValidationResult validatePartitionsSpecForMSQ(PartitionsSpec partitionsSpec) + { + if (!(partitionsSpec instanceof DimensionRangePartitionsSpec + || partitionsSpec instanceof DynamicPartitionsSpec)) { + 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 CompactionConfigValidationResult( + false, + "maxTotalRows[%d] in DynamicPartitionsSpec not supported for MSQ engine.", + ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows() + ); + } + return new CompactionConfigValidationResult(true, null); + } + + /** + * Validate rollup is set to false in granularitySpec when metricsSpec is specified. + */ + 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 CompactionConfigValidationResult(true, null); + } + + /** + * Validate maxNumTasks >= 2 in 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); + 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); + } + + /** + * Validate each metric has output column name same as the input name. + */ + public static CompactionConfigValidationResult validateMetricsSpecForMSQ(AggregatorFactory[] metricsSpec) + { + if (metricsSpec == null) { + return new CompactionConfigValidationResult(true, null); + } + return Arrays.stream(metricsSpec) + .filter(aggregatorFactory -> + !(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/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 5873bd229dbe..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 @@ -46,6 +46,7 @@ public class ClientCompactionTaskQuery implements ClientTaskQuery private final AggregatorFactory[] metricsSpec; private final ClientCompactionTaskTransformSpec transformSpec; private final Map context; + private final ClientCompactionRunnerInfo compactionRunner; @JsonCreator public ClientCompactionTaskQuery( @@ -57,7 +58,8 @@ public ClientCompactionTaskQuery( @JsonProperty("dimensionsSpec") ClientCompactionTaskDimensionsSpec dimensionsSpec, @JsonProperty("metricsSpec") AggregatorFactory[] metrics, @JsonProperty("transformSpec") ClientCompactionTaskTransformSpec transformSpec, - @JsonProperty("context") Map context + @JsonProperty("context") Map context, + @JsonProperty("compactionRunner") @Nullable ClientCompactionRunnerInfo compactionRunner ) { this.id = Preconditions.checkNotNull(id, "id"); @@ -69,6 +71,7 @@ public ClientCompactionTaskQuery( this.metricsSpec = metrics; this.transformSpec = transformSpec; this.context = context; + this.compactionRunner = compactionRunner; } @JsonProperty @@ -135,6 +138,13 @@ public Map getContext() return context; } + @JsonProperty("compactionRunner") + @Nullable + public ClientCompactionRunnerInfo getCompactionRunner() + { + return compactionRunner; + } + @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(compactionRunner, that.compactionRunner); } @Override @@ -167,7 +178,8 @@ public int hashCode() granularitySpec, dimensionsSpec, transformSpec, - context + context, + compactionRunner ); result = 31 * result + Arrays.hashCode(metricsSpec); return result; @@ -186,6 +198,7 @@ public String toString() ", metricsSpec=" + Arrays.toString(metricsSpec) + ", transformSpec=" + transformSpec + ", context=" + context + + ", compactionRunner=" + compactionRunner + '}'; } } 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..45279bda3ed3 --- /dev/null +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientMSQContext.java @@ -0,0 +1,35 @@ +/* + * 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; + /** + * 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/CompactionConfigValidationResult.java b/server/src/main/java/org/apache/druid/server/coordinator/CompactionConfigValidationResult.java new file mode 100644 index 000000000000..88eaa3e923a3 --- /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 = format == null ? null : 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/CoordinatorCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfig.java index 2d7b6c0100ac..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,8 @@ 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; import java.util.List; @@ -34,11 +36,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 CompactionEngine DEFAULT_COMPACTION_ENGINE = CompactionEngine.NATIVE; private final List compactionConfigs; private final double compactionTaskSlotRatio; private final int maxCompactionTaskSlots; private final boolean useAutoScaleSlots; + private final CompactionEngine compactionEngine; public static CoordinatorCompactionConfig from( CoordinatorCompactionConfig baseConfig, @@ -49,7 +53,8 @@ public static CoordinatorCompactionConfig from( compactionConfigs, baseConfig.compactionTaskSlotRatio, baseConfig.maxCompactionTaskSlots, - baseConfig.useAutoScaleSlots + baseConfig.useAutoScaleSlots, + null ); } @@ -64,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 @@ -83,19 +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.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 @@ -122,6 +124,12 @@ public boolean isUseAutoScaleSlots() return useAutoScaleSlots; } + @JsonProperty + public CompactionEngine getEngine() + { + return compactionEngine; + } + @Override public boolean equals(Object o) { @@ -135,13 +143,20 @@ public boolean equals(Object o) return Double.compare(that.compactionTaskSlotRatio, compactionTaskSlotRatio) == 0 && maxCompactionTaskSlots == that.maxCompactionTaskSlots && useAutoScaleSlots == that.useAutoScaleSlots && + compactionEngine == that.compactionEngine && Objects.equals(compactionConfigs, that.compactionConfigs); } @Override public int hashCode() { - return Objects.hash(compactionConfigs, compactionTaskSlotRatio, maxCompactionTaskSlots, useAutoScaleSlots); + return Objects.hash( + compactionConfigs, + compactionTaskSlotRatio, + maxCompactionTaskSlots, + useAutoScaleSlots, + compactionEngine + ); } @Override @@ -152,6 +167,7 @@ public String toString() ", compactionTaskSlotRatio=" + compactionTaskSlotRatio + ", maxCompactionTaskSlots=" + maxCompactionTaskSlots + ", useAutoScaleSlots=" + useAutoScaleSlots + + ", 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 da89040f50a4..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 @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.query.aggregation.AggregatorFactory; import org.joda.time.Period; @@ -55,6 +56,7 @@ public class DataSourceCompactionConfig private final UserCompactionTaskTransformConfig transformSpec; private final UserCompactionTaskIOConfig ioConfig; private final Map taskContext; + private final CompactionEngine engine; @JsonCreator public DataSourceCompactionConfig( @@ -69,6 +71,7 @@ public DataSourceCompactionConfig( @JsonProperty("metricsSpec") @Nullable AggregatorFactory[] metricsSpec, @JsonProperty("transformSpec") @Nullable UserCompactionTaskTransformConfig transformSpec, @JsonProperty("ioConfig") @Nullable UserCompactionTaskIOConfig ioConfig, + @JsonProperty("engine") @Nullable CompactionEngine engine, @JsonProperty("taskContext") @Nullable Map taskContext ) { @@ -88,6 +91,7 @@ public DataSourceCompactionConfig( this.dimensionsSpec = dimensionsSpec; this.transformSpec = transformSpec; this.taskContext = taskContext; + this.engine = engine; } @JsonProperty @@ -171,6 +175,13 @@ public Map getTaskContext() return taskContext; } + @JsonProperty + @Nullable + public CompactionEngine getEngine() + { + return engine; + } + @Override public boolean equals(Object o) { @@ -192,6 +203,7 @@ public boolean equals(Object o) Arrays.equals(metricsSpec, that.metricsSpec) && Objects.equals(transformSpec, that.transformSpec) && Objects.equals(ioConfig, that.ioConfig) && + this.engine == that.engine && Objects.equals(taskContext, that.taskContext); } @@ -209,7 +221,8 @@ public int hashCode() dimensionsSpec, transformSpec, ioConfig, - taskContext + taskContext, + 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 335912fe23ed..90fb684db6c7 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,15 +27,18 @@ 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; 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; @@ -167,10 +170,17 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) final Interval interval = compactionTaskQuery.getIoConfig().getInputSpec().getInterval(); intervalsToSkipCompaction.computeIfAbsent(status.getDataSource(), k -> new ArrayList<>()) .add(interval); - - busyCompactionTaskSlots += findMaxNumTaskSlotsUsedByOneCompactionTask( - 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 @@ -197,7 +207,8 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) compactionConfigs, currentRunAutoCompactionSnapshotBuilders, availableCompactionTaskSlots, - iterator + iterator, + dynamicConfig.getEngine() ); final CoordinatorRunStats stats = params.getCoordinatorStats(); @@ -234,8 +245,7 @@ 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; } @@ -281,11 +291,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 findMaxNumTaskSlotsUsedByOneCompactionTask(@Nullable ClientCompactionTaskQueryTuningConfig tuningConfig) + static int findMaxNumTaskSlotsUsedByOneNativeCompactionTask( + @Nullable ClientCompactionTaskQueryTuningConfig tuningConfig + ) { if (isParallelMode(tuningConfig)) { @Nullable @@ -297,6 +309,18 @@ static int findMaxNumTaskSlotsUsedByOneCompactionTask(@Nullable ClientCompaction } } + /** + * 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 context == null + ? ClientMSQContext.DEFAULT_MAX_NUM_TASKS + : (int) context.getOrDefault(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). @@ -355,7 +379,8 @@ private int submitCompactionTasks( Map compactionConfigs, Map currentRunAutoCompactionSnapshotBuilders, int numAvailableCompactionTaskSlots, - CompactionSegmentIterator iterator + CompactionSegmentIterator iterator, + CompactionEngine defaultEngine ) { if (numAvailableCompactionTaskSlots <= 0) { @@ -363,9 +388,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()) { @@ -465,6 +490,29 @@ private int submitCompactionTasks( } } + final CompactionEngine compactionEngine = config.getEngine() == null ? defaultEngine : config.getEngine(); + final Map autoCompactionContext = newAutoCompactionContext(config.getTaskContext()); + int slotsRequiredForCurrentTask; + + if (compactionEngine == CompactionEngine.MSQ) { + 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( + // Update the slots to 2 (min required for MSQ) if only 1 slot is available. + numAvailableCompactionTaskSlots == 1 ? 2 : numAvailableCompactionTaskSlots, + ClientMSQContext.MAX_TASK_SLOTS_FOR_MSQ_COMPACTION_TASK + ); + autoCompactionContext.put(ClientMSQContext.CTX_MAX_NUM_TASKS, slotsRequiredForCurrentTask); + } + } else { + slotsRequiredForCurrentTask = findMaxNumTaskSlotsUsedByOneNativeCompactionTask(config.getTuningConfig()); + } + final String taskId = compactSegments( "coordinator-issued", segmentsToCompact, @@ -479,7 +527,8 @@ private int submitCompactionTasks( config.getMetricsSpec(), transformSpec, dropExisting, - newAutoCompactionContext(config.getTaskContext()) + autoCompactionContext, + new ClientCompactionRunnerInfo(compactionEngine) ); LOG.info( @@ -489,7 +538,7 @@ private int submitCompactionTasks( LOG.debugSegments(segmentsToCompact, "Compacting segments"); // Count the compaction task itself + its sub tasks numSubmittedTasks++; - numCompactionTasksAndSubtasks += findMaxNumTaskSlotsUsedByOneCompactionTask(config.getTuningConfig()); + totalTaskSlotsAssigned += slotsRequiredForCurrentTask; } LOG.info("Submitted a total of [%d] compaction tasks.", numSubmittedTasks); @@ -628,7 +677,8 @@ private String compactSegments( @Nullable AggregatorFactory[] metricsSpec, @Nullable ClientCompactionTaskTransformSpec transformSpec, @Nullable Boolean dropExisting, - @Nullable Map context + @Nullable Map context, + ClientCompactionRunnerInfo compactionRunner ) { Preconditions.checkArgument(!segments.isEmpty(), "Expect non-empty segments to compact"); @@ -656,7 +706,8 @@ private String compactSegments( dimensionsSpec, metricsSpec, transformSpec, - context + context, + compactionRunner ); 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 57833506f093..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 @@ -26,9 +26,14 @@ 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.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; import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; @@ -119,6 +124,12 @@ public Response addOrUpdateCompactionConfig( .getCompactionConfigs() .stream() .collect(Collectors.toMap(DataSourceCompactionConfig::getDataSource, Function.identity())); + CompactionConfigValidationResult validationResult = + ClientCompactionRunnerInfo.validateCompactionConfig(newConfig, current.getEngine()); + 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())); @@ -206,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())); @@ -231,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 new file mode 100644 index 000000000000..f6d4a2b6e581 --- /dev/null +++ b/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfoTest.java @@ -0,0 +1,246 @@ +/* + * 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; + +import com.google.common.collect.ImmutableList; +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.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.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; + +public class ClientCompactionRunnerInfoTest +{ + @Test + public void testMSQEngineWithHashedPartitionsSpecIsInvalid() + { + DataSourceCompactionConfig compactionConfig = createCompactionConfig( + new HashedPartitionsSpec(100, null, null), + Collections.emptyMap(), + null, + null + ); + CompactionConfigValidationResult validationResult = ClientCompactionRunnerInfo.validateCompactionConfig( + compactionConfig, + CompactionEngine.NATIVE + ); + Assert.assertFalse(validationResult.isValid()); + Assert.assertEquals( + "Invalid partitionsSpec type[HashedPartitionsSpec] for MSQ engine." + + " Type must be either 'dynamic' or 'range'.", + validationResult.getReason() + ); + } + + @Test + public void testMSQEngineWithMaxTotalRowsIsInvalid() + { + DataSourceCompactionConfig compactionConfig = createCompactionConfig( + new DynamicPartitionsSpec(100, 100L), + Collections.emptyMap(), + null, + null + ); + CompactionConfigValidationResult validationResult = ClientCompactionRunnerInfo.validateCompactionConfig( + compactionConfig, + CompactionEngine.NATIVE + ); + Assert.assertFalse(validationResult.isValid()); + Assert.assertEquals( + "maxTotalRows[100] in DynamicPartitionsSpec not supported for MSQ engine.", + validationResult.getReason() + ); + } + + @Test + public void testMSQEngineWithDynamicPartitionsSpecIsValid() + { + DataSourceCompactionConfig compactionConfig = createCompactionConfig( + new DynamicPartitionsSpec(100, null), + Collections.emptyMap(), + null, + null + ); + Assert.assertTrue(ClientCompactionRunnerInfo.validateCompactionConfig(compactionConfig, CompactionEngine.NATIVE) + .isValid()); + } + + @Test + public void testMSQEngineWithDimensionRangePartitionsSpecIsValid() + { + DataSourceCompactionConfig compactionConfig = createCompactionConfig( + new DimensionRangePartitionsSpec(100, null, ImmutableList.of("partitionDim"), false), + Collections.emptyMap(), + null, + null + ); + Assert.assertTrue(ClientCompactionRunnerInfo.validateCompactionConfig(compactionConfig, CompactionEngine.NATIVE) + .isValid()); + } + + @Test + public void testMSQEngineWithQueryGranularityAllIsValid() + { + DataSourceCompactionConfig compactionConfig = createCompactionConfig( + new DynamicPartitionsSpec(3, null), + Collections.emptyMap(), + new UserCompactionTaskGranularityConfig(Granularities.ALL, Granularities.ALL, false), + null + ); + Assert.assertTrue(ClientCompactionRunnerInfo.validateCompactionConfig(compactionConfig, CompactionEngine.NATIVE) + .isValid()); + } + + @Test + public void testMSQEngineWithRollupFalseWithMetricsSpecIsInValid() + { + DataSourceCompactionConfig compactionConfig = createCompactionConfig( + new DynamicPartitionsSpec(3, null), + Collections.emptyMap(), + new UserCompactionTaskGranularityConfig(null, null, false), + new AggregatorFactory[]{new LongSumAggregatorFactory("sum", "sum")} + ); + 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() + { + // Aggregators having different input and ouput column names are unsupported. + 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( + "Different name[sum_added] and fieldName(s)[[added]] for aggregator unsupported for MSQ engine.", + validationResult.getReason() + ); + } + + @Test + public void testMSQEngineWithRollupNullWithMetricsSpecIsValid() + { + DataSourceCompactionConfig compactionConfig = createCompactionConfig( + new DynamicPartitionsSpec(3, null), + Collections.emptyMap(), + new UserCompactionTaskGranularityConfig(null, null, null), + new AggregatorFactory[]{new LongSumAggregatorFactory("sum", "sum")} + ); + Assert.assertTrue(ClientCompactionRunnerInfo.validateCompactionConfig(compactionConfig, CompactionEngine.NATIVE) + .isValid()); + } + + private static DataSourceCompactionConfig createCompactionConfig( + PartitionsSpec partitionsSpec, + Map context, + @Nullable UserCompactionTaskGranularityConfig granularitySpec, + @Nullable AggregatorFactory[] metricsSpec + ) + { + final DataSourceCompactionConfig config = new DataSourceCompactionConfig( + "dataSource", + null, + 500L, + 10000, + new Period(3600), + createTuningConfig(partitionsSpec), + granularitySpec, + null, + metricsSpec, + 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; + } +} 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..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; @@ -69,6 +70,7 @@ public void testSerdeBasic() throws IOException null, null, null, + null, ImmutableMap.of("key", "val") ); final String json = OBJECT_MAPPER.writeValueAsString(config); @@ -82,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 @@ -99,6 +102,7 @@ public void testSerdeWithMaxRowsPerSegment() throws IOException null, null, null, + CompactionEngine.MSQ, ImmutableMap.of("key", "val") ); final String json = OBJECT_MAPPER.writeValueAsString(config); @@ -111,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 @@ -148,6 +153,7 @@ public void testSerdeWithMaxTotalRows() throws IOException null, null, null, + CompactionEngine.NATIVE, ImmutableMap.of("key", "val") ); final String json = OBJECT_MAPPER.writeValueAsString(config); @@ -160,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 @@ -197,6 +204,7 @@ public void testSerdeMaxTotalRowsWithMaxRowsPerSegment() throws IOException null, null, null, + null, ImmutableMap.of("key", "val") ); @@ -305,6 +313,7 @@ public void testSerdeGranularitySpec() throws IOException null, null, null, + null, ImmutableMap.of("key", "val") ); final String json = OBJECT_MAPPER.writeValueAsString(config); @@ -335,6 +344,7 @@ public void testSerdeGranularitySpecWithQueryGranularity() throws Exception null, null, null, + null, ImmutableMap.of("key", "val") ); final String json = OBJECT_MAPPER.writeValueAsString(config); @@ -368,6 +378,7 @@ public void testSerdeWithNullGranularitySpec() throws IOException null, null, null, + null, ImmutableMap.of("key", "val") ); final String json = OBJECT_MAPPER.writeValueAsString(config); @@ -398,6 +409,7 @@ public void testSerdeGranularitySpecWithNullValues() throws IOException null, null, null, + null, ImmutableMap.of("key", "val") ); final String json = OBJECT_MAPPER.writeValueAsString(config); @@ -428,6 +440,7 @@ public void testSerdeGranularitySpecWithRollup() throws IOException null, null, null, + null, ImmutableMap.of("key", "val") ); final String json = OBJECT_MAPPER.writeValueAsString(config); @@ -461,6 +474,7 @@ public void testSerdeIOConfigWithNonNullDropExisting() throws IOException null, null, new UserCompactionTaskIOConfig(true), + null, ImmutableMap.of("key", "val") ); final String json = OBJECT_MAPPER.writeValueAsString(config); @@ -492,6 +506,7 @@ public void testSerdeIOConfigWithNullDropExisting() throws IOException null, null, new UserCompactionTaskIOConfig(null), + null, ImmutableMap.of("key", "val") ); final String json = OBJECT_MAPPER.writeValueAsString(config); @@ -523,6 +538,7 @@ public void testSerdeDimensionsSpec() throws IOException null, null, null, + null, ImmutableMap.of("key", "val") ); final String json = OBJECT_MAPPER.writeValueAsString(config); @@ -554,6 +570,7 @@ public void testSerdeTransformSpec() throws IOException null, new UserCompactionTaskTransformConfig(new SelectorDimFilter("dim1", "foo", null)), null, + null, ImmutableMap.of("key", "val") ); final String json = OBJECT_MAPPER.writeValueAsString(config); @@ -584,6 +601,7 @@ public void testSerdeMetricsSpec() throws IOException new AggregatorFactory[] {new CountAggregatorFactory("cnt")}, null, null, + null, ImmutableMap.of("key", "val") ); final String json = OBJECT_MAPPER.writeValueAsString(config); 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 173e4537ccad..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 @@ -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 singleDimensionShardSpecCreator = + (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, singleDimensionShardSpecCreator, CompactionEngine.NATIVE}, + new Object[]{dynamicPartitionsSpec, numberedShardSpecCreator, CompactionEngine.MSQ}, + new Object[]{singleDimensionPartitionsSpec, singleDimensionShardSpecCreator, 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,13 @@ 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. 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 { + Assert.assertEquals(1, stats.get(Stats.Compaction.SUBMITTED_TASKS)); + } } @Test @@ -654,7 +675,13 @@ 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. 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 { + Assert.assertEquals(1, stats.get(Stats.Compaction.SUBMITTED_TASKS)); + } } @Test @@ -668,7 +695,16 @@ 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. 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 / ClientMSQContext.MAX_TASK_SLOTS_FOR_MSQ_COMPACTION_TASK, + stats.get(Stats.Compaction.SUBMITTED_TASKS) + ); + } } @Test @@ -712,6 +748,7 @@ public void testCompactWithoutGranularitySpec() null, null, null, + engine, null ) ); @@ -769,6 +806,7 @@ public void testCompactWithNotNullIOConfig() null, null, new UserCompactionTaskIOConfig(true), + engine, null ) ); @@ -818,6 +856,7 @@ public void testCompactWithNullIOConfig() null, null, null, + engine, null ) ); @@ -867,6 +906,7 @@ public void testCompactWithGranularitySpec() null, null, null, + engine, null ) ); @@ -927,6 +967,7 @@ public void testCompactWithDimensionSpec() null, null, null, + engine, null ) ); @@ -979,6 +1020,7 @@ public void testCompactWithoutDimensionSpec() null, null, null, + engine, null ) ); @@ -1028,6 +1070,7 @@ public void testCompactWithRollupInGranularitySpec() null, null, null, + engine, null ) ); @@ -1081,6 +1124,7 @@ public void testCompactWithGranularitySpecConflictWithActiveCompactionTask() null, null, null, + null, null ) ); @@ -1137,6 +1181,7 @@ public void testCompactWithGranularitySpecConflictWithActiveCompactionTask() null, null, null, + engine, null ) ); @@ -1165,8 +1210,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, createcompactionConfigsForNative(2), 4); + } else { + stats = doCompactSegments(compactSegments, createcompactionConfigsForMSQ(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)); @@ -1198,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), 4); + doCompactSegments(compactSegments, createcompactionConfigsForNative(2), 4); Assert.assertEquals(1, stats.get(Stats.Compaction.SUBMITTED_TASKS)); Assert.assertEquals(1, overlordClient.submittedCompactionTasks.size()); @@ -1252,6 +1302,7 @@ public void testCompactWithTransformSpec() null, new UserCompactionTaskTransformConfig(new SelectorDimFilter("dim1", "foo", null)), null, + engine, null ) ); @@ -1302,6 +1353,7 @@ public void testCompactWithoutCustomSpecs() null, null, null, + engine, null ) ); @@ -1354,6 +1406,7 @@ public void testCompactWithMetricsSpec() aggregatorFactories, null, null, + engine, null ) ); @@ -1434,6 +1487,7 @@ public void testDetermineSegmentGranularityFromSegmentsToCompact() null, null, null, + engine, null ) ); @@ -1520,6 +1574,7 @@ public void testDetermineSegmentGranularityFromSegmentGranularityInCompactionCon null, null, null, + engine, null ) ); @@ -1577,6 +1632,7 @@ public void testCompactWithMetricsSpecShouldSetPreserveExistingMetricsTrue() new AggregatorFactory[] {new CountAggregatorFactory("cnt")}, null, null, + engine, null ) ); @@ -1629,6 +1685,7 @@ public void testCompactWithoutMetricsSpecShouldSetPreserveExistingMetricsFalse() null, null, null, + engine, null ) ); @@ -1781,7 +1838,8 @@ private CoordinatorRunStats doCompactSegments( compactionConfigs, numCompactionTaskSlots == null ? null : 1.0, // 100% when numCompactionTaskSlots is not null numCompactionTaskSlots, - useAutoScaleSlots + useAutoScaleSlots, + null ) ) .build(); @@ -1901,10 +1959,23 @@ 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 createcompactionConfigsForNative(@Nullable Integer maxNumConcurrentSubTasks) + { + return createCompactionConfigs(maxNumConcurrentSubTasks, null); + } + + private List createcompactionConfigsForMSQ(Integer maxNumTasks) + { + return createCompactionConfigs(null, maxNumTasks); + } + + private List createCompactionConfigs( + @Nullable Integer maxNumConcurrentSubTasksForNative, + @Nullable Integer maxNumTasksForMSQ + ) { final List compactionConfigs = new ArrayList<>(); for (int i = 0; i < 3; i++) { @@ -1928,7 +1999,7 @@ private List createCompactionConfigs(@Nullable Integ null, null, null, - maxNumConcurrentSubTasks, + maxNumConcurrentSubTasksForNative, null, null, null, @@ -1942,7 +2013,8 @@ private List createCompactionConfigs(@Nullable Integ null, null, null, - null + engine, + maxNumTasksForMSQ == null ? null : ImmutableMap.of(ClientMSQContext.CTX_MAX_NUM_TASKS, maxNumTasksForMSQ) ) ); } @@ -2172,7 +2244,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 @@ -2181,7 +2253,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)); } } 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 ec4547c940c4..3d441d9b06d9 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 @@ -158,6 +158,7 @@ public void testRunRemoveInactiveDatasourceCompactionConfig() null, null, null, + null, ImmutableMap.of("key", "val") ); @@ -173,6 +174,7 @@ public void testRunRemoveInactiveDatasourceCompactionConfig() null, null, null, + null, ImmutableMap.of("key", "val") ); CoordinatorCompactionConfig originalCurrentConfig = CoordinatorCompactionConfig.from(ImmutableList.of(inactiveDatasourceConfig, activeDatasourceConfig)); @@ -255,6 +257,7 @@ public void testRunRetryForRetryableException() null, null, null, + null, ImmutableMap.of("key", "val") ); 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..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 @@ -22,8 +22,12 @@ 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.granularity.Granularities; import org.apache.druid.metadata.MetadataStorageConnector; @@ -62,6 +66,7 @@ public class CoordinatorCompactionConfigsResourceTest null, null, null, + null, ImmutableMap.of("key", "val") ); private static final DataSourceCompactionConfig NEW_CONFIG = new DataSourceCompactionConfig( @@ -76,6 +81,7 @@ public class CoordinatorCompactionConfigsResourceTest null, null, null, + null, ImmutableMap.of("key", "val") ); private static final byte[] OLD_CONFIG_IN_BYTES = {1, 2, 3}; @@ -189,6 +195,7 @@ public void testAddOrUpdateCompactionConfigWithExistingConfig() null, null, null, + CompactionEngine.NATIVE, ImmutableMap.of("key", "val") ); Response result = coordinatorCompactionConfigsResource.addOrUpdateCompactionConfig( @@ -202,6 +209,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 @@ -230,6 +238,7 @@ public void testDeleteCompactionConfigWithExistingConfig() null, null, null, + null, ImmutableMap.of("key", "val") ); final CoordinatorCompactionConfig originalConfig = CoordinatorCompactionConfig.from(ImmutableList.of(toDelete)); @@ -388,6 +397,7 @@ public void testAddOrUpdateCompactionConfigWithoutExistingConfig() null, null, null, + CompactionEngine.MSQ, ImmutableMap.of("key", "val") ); String author = "maytas"; @@ -401,6 +411,102 @@ 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") + ); + 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()); + + 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( + "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() + ); } @Test 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;