mirror of https://github.com/apache/druid.git
Auto-Compaction using Multi-Stage Query Engine (#16291)
Description: Compaction operations issued by the Coordinator currently run using the native query engine. As majority of the advancements that we are making in batch ingestion are in MSQ, it is imperative that we support compaction on MSQ to make Compaction more robust and possibly faster. For instance, we have seen OOM errors in native compaction that MSQ could have handled by its auto-calculation of tuning parameters. This commit enables compaction on MSQ to remove the dependency on native engine. Main changes: * `DataSourceCompactionConfig` now has an additional field `engine` that can be one of `[native, msq]` with `native` being the default. * if engine is MSQ, `CompactSegments` duty assigns all available compaction task slots to the launched `CompactionTask` to ensure full capacity is available to MSQ. This is to avoid stalling which could happen in case a fraction of the tasks were allotted and they eventually fell short of the number of tasks required by the MSQ engine to run the compaction. * `ClientCompactionTaskQuery` has a new field `compactionRunner` with just one `engine` field. * `CompactionTask` now has `CompactionRunner` interface instance with its implementations `NativeCompactinRunner` and `MSQCompactionRunner` in the `druid-multi-stage-query` extension. The objectmapper deserializes `ClientCompactionRunnerInfo` in `ClientCompactionTaskQuery` to the `CompactionRunner` instance that is mapped to the specified type [`native`, `msq`]. * `CompactTask` uses the `CompactionRunner` instance it receives to create the indexing tasks. * `CompactionTask` to `MSQControllerTask` conversion logic checks whether metrics are present in the segment schema. If present, the task is created with a native group-by query; if not, the task is issued with a scan query. The `storeCompactionState` flag is set in the context. * Each created `MSQControllerTask` is launched in-place and its `TaskStatus` tracked to determine the final status of the `CompactionTask`. The id of each of these tasks is the same as that of `CompactionTask` since otherwise, the workers will be unable to determine the controller task's location for communication (as they haven't been launched via the overlord).
This commit is contained in:
parent
eb981d855f
commit
197c54f673
|
@ -104,6 +104,7 @@ public class NewestSegmentFirstPolicyBenchmark
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
)
|
||||
);
|
||||
|
|
|
@ -193,6 +193,7 @@ import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
|||
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 @@ public class ControllerImpl implements Controller
|
|||
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 @@ public class ControllerImpl implements Controller
|
|||
|
||||
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 @@ public class ControllerImpl implements Controller
|
|||
List<Object> metricsSpec = dataSchema.getAggregators() == null
|
||||
? null
|
||||
: jsonMapper.convertValue(
|
||||
dataSchema.getAggregators(), new TypeReference<List<Object>>()
|
||||
{
|
||||
});
|
||||
dataSchema.getAggregators(),
|
||||
new TypeReference<List<Object>>() {}
|
||||
);
|
||||
|
||||
|
||||
IndexSpec indexSpec = tuningConfig.getIndexSpec();
|
||||
|
|
|
@ -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.CounterSnapshotsSerializer;
|
|||
import org.apache.druid.msq.counters.SegmentGenerationProgressCounter;
|
||||
import org.apache.druid.msq.counters.SuperSorterProgressTrackerCounter;
|
||||
import org.apache.druid.msq.counters.WarningCounters;
|
||||
import org.apache.druid.msq.indexing.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 class MSQIndexingModule implements DruidModule
|
|||
NilInputSource.class
|
||||
);
|
||||
|
||||
module.registerSubtypes(new NamedType(MSQCompactionRunner.class, MSQCompactionRunner.TYPE));
|
||||
|
||||
FAULT_CLASSES.forEach(module::registerSubtypes);
|
||||
module.addSerializer(new CounterSnapshotsSerializer());
|
||||
return Collections.singletonList(module);
|
||||
|
|
|
@ -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:
|
||||
* <ul>
|
||||
* <li>partitionsSpec of type HashedParititionsSpec.</li>
|
||||
* <li>maxTotalRows in DynamicPartitionsSpec.</li>
|
||||
* <li>rollup set to false in granularitySpec when metricsSpec is specified. Null is treated as true.</li>
|
||||
* <li>queryGranularity set to ALL in granularitySpec.</li>
|
||||
* <li>Each metric has output column name same as the input name.</li>
|
||||
* </ul>
|
||||
*/
|
||||
@Override
|
||||
public CompactionConfigValidationResult validateCompactionTask(
|
||||
CompactionTask compactionTask
|
||||
)
|
||||
{
|
||||
List<CompactionConfigValidationResult> 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<Interval, DataSchema> intervalDataSchemas,
|
||||
TaskToolbox taskToolbox
|
||||
) throws Exception
|
||||
{
|
||||
List<MSQControllerTask> 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<MSQControllerTask> createMsqControllerTasks(
|
||||
CompactionTask compactionTask,
|
||||
Map<Interval, DataSchema> intervalDataSchemas
|
||||
) throws JsonProcessingException
|
||||
{
|
||||
final List<MSQControllerTask> msqControllerTasks = new ArrayList<>();
|
||||
|
||||
for (Map.Entry<Interval, DataSchema> 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<String, Object> 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<DimensionSpec> getAggregateDimensions(DataSchema dataSchema)
|
||||
{
|
||||
List<DimensionSpec> 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<ColumnMapping> 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<OrderByColumnSpec> getOrderBySpec(PartitionsSpec partitionSpec)
|
||||
{
|
||||
if (partitionSpec.getType() == SecondaryPartitionType.RANGE) {
|
||||
List<String> 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<String, Object> createMSQTaskContext(CompactionTask compactionTask, DataSchema dataSchema)
|
||||
throws JsonProcessingException
|
||||
{
|
||||
Map<String, Object> 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<MSQControllerTask> 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);
|
||||
}
|
||||
}
|
|
@ -144,6 +144,22 @@ public class MSQControllerTask extends AbstractTask implements ClientTaskQuery,
|
|||
addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, true);
|
||||
}
|
||||
|
||||
public MSQControllerTask(
|
||||
@Nullable String id,
|
||||
MSQSpec querySpec,
|
||||
@Nullable String sqlQuery,
|
||||
@Nullable Map<String, Object> sqlQueryContext,
|
||||
@Nullable SqlResults.Context sqlResultsContext,
|
||||
@Nullable List<SqlTypeName> sqlTypeNames,
|
||||
@Nullable List<ColumnType> nativeTypeNames,
|
||||
@Nullable Map<String, Object> context,
|
||||
Injector injector
|
||||
)
|
||||
{
|
||||
this(id, querySpec, sqlQuery, sqlQueryContext, sqlResultsContext, sqlTypeNames, nativeTypeNames, context);
|
||||
this.injector = injector;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getType()
|
||||
{
|
||||
|
|
|
@ -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";
|
||||
|
||||
|
|
|
@ -186,7 +186,8 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
context,
|
||||
Collections.emptyList(),
|
||||
Collections.singletonList(new FloatDimensionSchema("m1")),
|
||||
GranularityType.DAY
|
||||
GranularityType.DAY,
|
||||
Intervals.ETERNITY
|
||||
)
|
||||
)
|
||||
.verifyResults();
|
||||
|
@ -244,7 +245,8 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
context,
|
||||
Collections.emptyList(),
|
||||
Collections.singletonList(new FloatDimensionSchema("m1")),
|
||||
GranularityType.DAY
|
||||
GranularityType.DAY,
|
||||
Intervals.of("2000-01-02T/P1D")
|
||||
)
|
||||
)
|
||||
.verifyResults();
|
||||
|
@ -334,7 +336,8 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
context,
|
||||
Collections.emptyList(),
|
||||
Collections.singletonList(new LongDimensionSchema("cnt")),
|
||||
GranularityType.HOUR
|
||||
GranularityType.HOUR,
|
||||
Intervals.ETERNITY
|
||||
)
|
||||
)
|
||||
.verifyResults();
|
||||
|
@ -409,7 +412,8 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
context,
|
||||
Collections.emptyList(),
|
||||
Collections.singletonList(new StringDimensionSchema("user")),
|
||||
GranularityType.HOUR
|
||||
GranularityType.HOUR,
|
||||
Intervals.of("2016-06-27T01:00:00.000Z/2016-06-27T02:00:00.000Z")
|
||||
)
|
||||
)
|
||||
.verifyResults();
|
||||
|
@ -490,7 +494,8 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
context,
|
||||
Collections.emptyList(),
|
||||
Collections.singletonList(new FloatDimensionSchema("m1")),
|
||||
GranularityType.ALL
|
||||
GranularityType.ALL,
|
||||
Intervals.ETERNITY
|
||||
)
|
||||
)
|
||||
.verifyResults();
|
||||
|
@ -582,7 +587,8 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
context,
|
||||
Collections.emptyList(),
|
||||
Collections.singletonList(new FloatDimensionSchema("m1")),
|
||||
GranularityType.MONTH
|
||||
GranularityType.MONTH,
|
||||
Intervals.ETERNITY
|
||||
)
|
||||
)
|
||||
.verifyResults();
|
||||
|
@ -664,7 +670,8 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
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 class MSQReplaceTest extends MSQTestBase
|
|||
context,
|
||||
Collections.emptyList(),
|
||||
Collections.singletonList(new FloatDimensionSchema("m1")),
|
||||
GranularityType.MONTH
|
||||
GranularityType.MONTH,
|
||||
Intervals.of("2000-01-01T/2002-01-01T")
|
||||
)
|
||||
)
|
||||
.verifyResults();
|
||||
|
@ -900,7 +908,8 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
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 class MSQReplaceTest extends MSQTestBase
|
|||
context,
|
||||
Collections.emptyList(),
|
||||
Collections.singletonList(new FloatDimensionSchema("m1")),
|
||||
GranularityType.MONTH
|
||||
GranularityType.MONTH,
|
||||
Intervals.of("2000-01-01T/2002-01-01T")
|
||||
)
|
||||
)
|
||||
.verifyResults();
|
||||
|
@ -1070,7 +1080,8 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
context,
|
||||
Collections.emptyList(),
|
||||
Collections.singletonList(new FloatDimensionSchema("m1")),
|
||||
GranularityType.MONTH
|
||||
GranularityType.MONTH,
|
||||
Intervals.ETERNITY
|
||||
)
|
||||
)
|
||||
.verifyResults();
|
||||
|
@ -1110,7 +1121,8 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
new StringDimensionSchema("dim1"),
|
||||
new LongDimensionSchema("cnt")
|
||||
),
|
||||
GranularityType.DAY
|
||||
GranularityType.DAY,
|
||||
Intervals.ETERNITY
|
||||
)
|
||||
)
|
||||
.verifyResults();
|
||||
|
@ -1190,7 +1202,8 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
context,
|
||||
Collections.emptyList(),
|
||||
Collections.singletonList(new FloatDimensionSchema("m1")),
|
||||
GranularityType.ALL
|
||||
GranularityType.ALL,
|
||||
Intervals.ETERNITY
|
||||
)
|
||||
)
|
||||
.verifyResults();
|
||||
|
@ -1248,7 +1261,8 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
context,
|
||||
Collections.emptyList(),
|
||||
Arrays.asList(new FloatDimensionSchema("m1"), new DoubleDimensionSchema("m2")),
|
||||
GranularityType.QUARTER
|
||||
GranularityType.QUARTER,
|
||||
Intervals.ETERNITY
|
||||
)
|
||||
)
|
||||
.verifyResults();
|
||||
|
@ -1340,7 +1354,8 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
context,
|
||||
Collections.emptyList(),
|
||||
Collections.singletonList(new StringDimensionSchema("d")),
|
||||
GranularityType.ALL
|
||||
GranularityType.ALL,
|
||||
Intervals.ETERNITY
|
||||
)
|
||||
)
|
||||
.verifyResults();
|
||||
|
@ -1412,7 +1427,8 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
context,
|
||||
Collections.emptyList(),
|
||||
Collections.singletonList(new FloatDimensionSchema("d")),
|
||||
GranularityType.ALL
|
||||
GranularityType.ALL,
|
||||
Intervals.ETERNITY
|
||||
)
|
||||
)
|
||||
.verifyResults();
|
||||
|
@ -1495,7 +1511,8 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
context,
|
||||
Collections.singletonList("d"),
|
||||
Collections.singletonList(new StringDimensionSchema("d")),
|
||||
GranularityType.DAY
|
||||
GranularityType.DAY,
|
||||
Intervals.of("1999-01-01T00:00:00.000Z/2002-01-01T00:00:00.000Z")
|
||||
)
|
||||
)
|
||||
.verifyResults();
|
||||
|
@ -2022,7 +2039,8 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
Map<String, Object> context,
|
||||
List<String> partitionDimensions,
|
||||
List<DimensionSchema> dimensions,
|
||||
GranularityType segmentGranularity
|
||||
GranularityType segmentGranularity,
|
||||
Interval interval
|
||||
)
|
||||
{
|
||||
if (!context.containsKey(Tasks.STORE_COMPACTION_STATE_KEY)
|
||||
|
@ -2049,7 +2067,7 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
segmentGranularity.getDefaultGranularity(),
|
||||
GranularityType.NONE.getDefaultGranularity(),
|
||||
false,
|
||||
Intervals.ONLY_ETERNITY
|
||||
Collections.singletonList(interval)
|
||||
);
|
||||
List<Object> metricsSpec = Collections.emptyList();
|
||||
|
||||
|
|
|
@ -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<String> 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<DimensionSchema> 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<AggregatorFactory> 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<MSQControllerTask> 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<MSQControllerTask> 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<String, Object> 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<String, Object> 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();
|
||||
}
|
||||
}
|
|
@ -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<Interval, DataSchema> 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);
|
||||
|
||||
}
|
|
@ -34,7 +34,6 @@ import com.google.common.collect.ImmutableSet;
|
|||
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.SplitHintSpec;
|
|||
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.TaskToolbox;
|
|||
import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction;
|
||||
import org.apache.druid.indexing.common.actions.TaskActionClient;
|
||||
import org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig;
|
||||
import org.apache.druid.indexing.common.task.batch.parallel.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.loading.SegmentCacheManager;
|
|||
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.DataSegment;
|
|||
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.HashSet;
|
|||
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 class CompactionTask extends AbstractBatchIndexTask implements PendingSeg
|
|||
@JsonProperty("granularitySpec") @Nullable final ClientCompactionTaskGranularitySpec granularitySpec,
|
||||
@JsonProperty("tuningConfig") @Nullable final TuningConfig tuningConfig,
|
||||
@JsonProperty("context") @Nullable final Map<String, Object> context,
|
||||
@JsonProperty("compactionRunner") final CompactionRunner compactionRunner,
|
||||
@JacksonInject SegmentCacheManagerFactory segmentCacheManagerFactory
|
||||
)
|
||||
{
|
||||
|
@ -248,8 +231,12 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg
|
|||
}
|
||||
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 class CompactionTask extends AbstractBatchIndexTask implements PendingSeg
|
|||
|
||||
@Nullable
|
||||
@JsonProperty
|
||||
public ParallelIndexTuningConfig getTuningConfig()
|
||||
public CompactionTuningConfig getTuningConfig()
|
||||
{
|
||||
return tuningConfig;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public CompactionRunner getCompactionRunner()
|
||||
{
|
||||
return compactionRunner;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getType()
|
||||
{
|
||||
|
@ -472,144 +465,44 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg
|
|||
@Override
|
||||
public TaskStatus runTask(TaskToolbox toolbox) throws Exception
|
||||
{
|
||||
|
||||
// emit metric for compact ingestion mode:
|
||||
emitCompactIngestionModeMetrics(toolbox.getEmitter(), ioConfig.isDropExisting());
|
||||
|
||||
final List<ParallelIndexIngestionSpec> ingestionSpecs = createIngestionSchema(
|
||||
final Map<Interval, DataSchema> intervalDataSchemas = createDataSchemasForIntervals(
|
||||
UTC_CLOCK,
|
||||
toolbox,
|
||||
getTaskLockHelper().getLockGranularityToUse(),
|
||||
ioConfig,
|
||||
segmentProvider,
|
||||
partitionConfigurationManager,
|
||||
dimensionsSpec,
|
||||
transformSpec,
|
||||
metricsSpec,
|
||||
granularitySpec,
|
||||
toolbox.getCoordinatorClient(),
|
||||
segmentCacheManagerFactory,
|
||||
getMetricBuilder()
|
||||
);
|
||||
final List<ParallelIndexSupervisorTask> 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<String, Object> createContextForSubtask()
|
||||
{
|
||||
final Map<String, Object> 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<ParallelIndexIngestionSpec> createIngestionSchema(
|
||||
static Map<Interval, DataSchema> 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 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg
|
|||
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<ParallelIndexIngestionSpec> specs = new ArrayList<>();
|
||||
Map<Interval, DataSchema> intervalDataSchemaMap = new HashMap<>();
|
||||
|
||||
// original granularity
|
||||
final Map<Interval, List<DataSegment>> intervalToSegments = new TreeMap<>(
|
||||
|
@ -679,24 +570,9 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg
|
|||
? 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,70 +596,10 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg
|
|||
metricsSpec,
|
||||
granularitySpec
|
||||
);
|
||||
|
||||
return Collections.singletonList(
|
||||
new ParallelIndexIngestionSpec(
|
||||
dataSchema,
|
||||
createIoConfig(
|
||||
toolbox,
|
||||
dataSchema,
|
||||
segmentProvider.interval,
|
||||
coordinatorClient,
|
||||
segmentCacheManagerFactory,
|
||||
ioConfig
|
||||
),
|
||||
compactionTuningConfig
|
||||
)
|
||||
);
|
||||
return Collections.singletonMap(segmentProvider.interval, dataSchema);
|
||||
}
|
||||
}
|
||||
|
||||
private static ParallelIndexIOConfig createIoConfig(
|
||||
TaskToolbox toolbox,
|
||||
DataSchema dataSchema,
|
||||
Interval interval,
|
||||
CoordinatorClient coordinatorClient,
|
||||
SegmentCacheManagerFactory segmentCacheManagerFactory,
|
||||
CompactionIOConfig compactionIOConfig
|
||||
)
|
||||
{
|
||||
if (!compactionIOConfig.isAllowNonAlignedInterval()) {
|
||||
// Validate interval alignment.
|
||||
final Granularity segmentGranularity = dataSchema.getGranularitySpec().getSegmentGranularity();
|
||||
final Interval widenedInterval = Intervals.utc(
|
||||
segmentGranularity.bucketStart(interval.getStart()).getMillis(),
|
||||
segmentGranularity.bucketEnd(interval.getEnd().minus(1)).getMillis()
|
||||
);
|
||||
|
||||
if (!interval.equals(widenedInterval)) {
|
||||
throw new IAE(
|
||||
"Interval[%s] to compact is not aligned with segmentGranularity[%s]",
|
||||
interval,
|
||||
segmentGranularity
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
return new ParallelIndexIOConfig(
|
||||
null,
|
||||
new DruidInputSource(
|
||||
dataSchema.getDataSource(),
|
||||
interval,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
toolbox.getIndexIO(),
|
||||
coordinatorClient,
|
||||
segmentCacheManagerFactory,
|
||||
toolbox.getConfig()
|
||||
).withTaskToolbox(toolbox),
|
||||
null,
|
||||
false,
|
||||
compactionIOConfig.isDropExisting()
|
||||
);
|
||||
}
|
||||
|
||||
private static List<TimelineObjectHolder<String, DataSegment>> retrieveRelevantTimelineHolders(
|
||||
TaskToolbox toolbox,
|
||||
SegmentProvider segmentProvider,
|
||||
|
@ -1213,35 +1029,6 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg
|
|||
}
|
||||
}
|
||||
|
||||
@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 class CompactionTask extends AbstractBatchIndexTask implements PendingSeg
|
|||
private TuningConfig tuningConfig;
|
||||
@Nullable
|
||||
private Map<String, Object> context;
|
||||
private CompactionRunner compactionRunner;
|
||||
|
||||
public Builder(
|
||||
String dataSource,
|
||||
|
@ -1345,6 +1133,12 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg
|
|||
return this;
|
||||
}
|
||||
|
||||
public Builder compactionRunner(CompactionRunner compactionRunner)
|
||||
{
|
||||
this.compactionRunner = compactionRunner;
|
||||
return this;
|
||||
}
|
||||
|
||||
public CompactionTask build()
|
||||
{
|
||||
return new CompactionTask(
|
||||
|
@ -1362,6 +1156,7 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg
|
|||
granularitySpec,
|
||||
tuningConfig,
|
||||
context,
|
||||
compactionRunner,
|
||||
segmentCacheManagerFactory
|
||||
);
|
||||
}
|
||||
|
|
|
@ -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<ParallelIndexIngestionSpec> createIngestionSpecs(
|
||||
Map<Interval, DataSchema> 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<Interval, DataSchema> intervalDataSchemaMap,
|
||||
TaskToolbox taskToolbox
|
||||
) throws Exception
|
||||
{
|
||||
final PartitionConfigurationManager partitionConfigurationManager =
|
||||
new NativeCompactionRunner.PartitionConfigurationManager(compactionTask.getTuningConfig());
|
||||
|
||||
|
||||
final List<ParallelIndexIngestionSpec> ingestionSpecs = createIngestionSpecs(
|
||||
intervalDataSchemaMap,
|
||||
taskToolbox,
|
||||
compactionTask.getIoConfig(),
|
||||
partitionConfigurationManager,
|
||||
taskToolbox.getCoordinatorClient(),
|
||||
segmentCacheManagerFactory
|
||||
);
|
||||
|
||||
List<ParallelIndexSupervisorTask> 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<ParallelIndexSupervisorTask> 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<String, Object> createContextForSubtask(CompactionTask compactionTask)
|
||||
{
|
||||
final Map<String, Object> 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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -28,6 +28,7 @@ import org.apache.druid.client.coordinator.CoordinatorClient;
|
|||
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.data.input.impl.DimensionsSpec;
|
|||
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 @@ public class ClientCompactionTaskQuerySerdeTest
|
|||
new ClientCompactionTaskDimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))),
|
||||
METRICS_SPEC,
|
||||
transformSpec,
|
||||
context
|
||||
context,
|
||||
new ClientCompactionRunnerInfo(CompactionEngine.NATIVE)
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -68,9 +68,9 @@ import org.apache.druid.indexing.common.actions.TaskActionClient;
|
|||
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 class CompactionTaskTest
|
|||
{
|
||||
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 class CompactionTaskTest
|
|||
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 class CompactionTaskTest
|
|||
@Test
|
||||
public void testCreateIngestionSchema() throws IOException
|
||||
{
|
||||
final List<ParallelIndexIngestionSpec> ingestionSpecs = CompactionTask.createIngestionSchema(
|
||||
final Map<Interval, DataSchema> 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<ParallelIndexIngestionSpec> ingestionSpecs = NativeCompactionRunner.createIngestionSpecs(
|
||||
dataSchemasForIntervals,
|
||||
toolbox,
|
||||
new CompactionIOConfig(null, false, null),
|
||||
new PartitionConfigurationManager(TUNING_CONFIG),
|
||||
COORDINATOR_CLIENT,
|
||||
segmentCacheManagerFactory
|
||||
);
|
||||
|
||||
final List<DimensionsSpec> expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration();
|
||||
|
||||
ingestionSpecs.sort(
|
||||
|
@ -987,21 +991,27 @@ public class CompactionTaskTest
|
|||
null,
|
||||
null
|
||||
);
|
||||
final List<ParallelIndexIngestionSpec> ingestionSpecs = CompactionTask.createIngestionSchema(
|
||||
final Map<Interval, DataSchema> 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<ParallelIndexIngestionSpec> ingestionSpecs = NativeCompactionRunner.createIngestionSpecs(
|
||||
dataSchemasForIntervals,
|
||||
toolbox,
|
||||
new CompactionIOConfig(null, false, null),
|
||||
new PartitionConfigurationManager(tuningConfig),
|
||||
COORDINATOR_CLIENT,
|
||||
segmentCacheManagerFactory
|
||||
);
|
||||
|
||||
final List<DimensionsSpec> expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration();
|
||||
|
||||
ingestionSpecs.sort(
|
||||
|
@ -1064,21 +1074,27 @@ public class CompactionTaskTest
|
|||
null,
|
||||
null
|
||||
);
|
||||
final List<ParallelIndexIngestionSpec> ingestionSpecs = CompactionTask.createIngestionSchema(
|
||||
final Map<Interval, DataSchema> 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<ParallelIndexIngestionSpec> ingestionSpecs = NativeCompactionRunner.createIngestionSpecs(
|
||||
dataSchemasForIntervals,
|
||||
toolbox,
|
||||
new CompactionIOConfig(null, false, null),
|
||||
new PartitionConfigurationManager(tuningConfig),
|
||||
COORDINATOR_CLIENT,
|
||||
segmentCacheManagerFactory
|
||||
);
|
||||
|
||||
final List<DimensionsSpec> expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration();
|
||||
|
||||
ingestionSpecs.sort(
|
||||
|
@ -1141,21 +1157,26 @@ public class CompactionTaskTest
|
|||
null,
|
||||
null
|
||||
);
|
||||
final List<ParallelIndexIngestionSpec> ingestionSpecs = CompactionTask.createIngestionSchema(
|
||||
final Map<Interval, DataSchema> 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<ParallelIndexIngestionSpec> ingestionSpecs = NativeCompactionRunner.createIngestionSpecs(
|
||||
dataSchemasForIntervals,
|
||||
toolbox,
|
||||
new CompactionIOConfig(null, false, null),
|
||||
new PartitionConfigurationManager(tuningConfig),
|
||||
COORDINATOR_CLIENT,
|
||||
segmentCacheManagerFactory
|
||||
);
|
||||
final List<DimensionsSpec> expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration();
|
||||
|
||||
ingestionSpecs.sort(
|
||||
|
@ -1207,22 +1228,27 @@ public class CompactionTaskTest
|
|||
)
|
||||
);
|
||||
|
||||
final List<ParallelIndexIngestionSpec> ingestionSpecs = CompactionTask.createIngestionSchema(
|
||||
final Map<Interval, DataSchema> 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<ParallelIndexIngestionSpec> 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 class CompactionTaskTest
|
|||
new DoubleMaxAggregatorFactory("custom_double_max", "agg_4")
|
||||
};
|
||||
|
||||
final List<ParallelIndexIngestionSpec> ingestionSpecs = CompactionTask.createIngestionSchema(
|
||||
final Map<Interval, DataSchema> 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<ParallelIndexIngestionSpec> ingestionSpecs = NativeCompactionRunner.createIngestionSpecs(
|
||||
dataSchemasForIntervals,
|
||||
toolbox,
|
||||
new CompactionIOConfig(null, false, null),
|
||||
new PartitionConfigurationManager(TUNING_CONFIG),
|
||||
COORDINATOR_CLIENT,
|
||||
segmentCacheManagerFactory
|
||||
);
|
||||
|
||||
final List<DimensionsSpec> expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration();
|
||||
|
||||
ingestionSpecs.sort(
|
||||
|
@ -1292,21 +1323,27 @@ public class CompactionTaskTest
|
|||
@Test
|
||||
public void testCreateIngestionSchemaWithCustomSegments() throws IOException
|
||||
{
|
||||
final List<ParallelIndexIngestionSpec> ingestionSpecs = CompactionTask.createIngestionSchema(
|
||||
final Map<Interval, DataSchema> 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<ParallelIndexIngestionSpec> ingestionSpecs = NativeCompactionRunner.createIngestionSpecs(
|
||||
dataSchemasForIntervals,
|
||||
toolbox,
|
||||
new CompactionIOConfig(null, false, null),
|
||||
new PartitionConfigurationManager(TUNING_CONFIG),
|
||||
COORDINATOR_CLIENT,
|
||||
segmentCacheManagerFactory
|
||||
);
|
||||
|
||||
final List<DimensionsSpec> expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration();
|
||||
|
||||
ingestionSpecs.sort(
|
||||
|
@ -1337,21 +1374,26 @@ public class CompactionTaskTest
|
|||
Collections.sort(segments);
|
||||
// Remove one segment in the middle
|
||||
segments.remove(segments.size() / 2);
|
||||
CompactionTask.createIngestionSchema(
|
||||
final Map<Interval, DataSchema> 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 class CompactionTaskTest
|
|||
final TestIndexIO indexIO = (TestIndexIO) toolbox.getIndexIO();
|
||||
indexIO.removeMetadata(Iterables.getFirst(indexIO.getQueryableIndexMap().keySet(), null));
|
||||
final List<DataSegment> segments = new ArrayList<>(SEGMENTS);
|
||||
CompactionTask.createIngestionSchema(
|
||||
final Map<Interval, DataSchema> 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 class CompactionTaskTest
|
|||
|
||||
final Builder builder = new Builder(
|
||||
DATA_SOURCE,
|
||||
segmentCacheManagerFactory,
|
||||
RETRY_POLICY_FACTORY
|
||||
segmentCacheManagerFactory, RETRY_POLICY_FACTORY
|
||||
);
|
||||
|
||||
@SuppressWarnings("unused")
|
||||
|
@ -1401,21 +1447,26 @@ public class CompactionTaskTest
|
|||
@Test
|
||||
public void testSegmentGranularityAndNullQueryGranularity() throws IOException
|
||||
{
|
||||
final List<ParallelIndexIngestionSpec> ingestionSpecs = CompactionTask.createIngestionSchema(
|
||||
final Map<Interval, DataSchema> 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<ParallelIndexIngestionSpec> ingestionSpecs = NativeCompactionRunner.createIngestionSpecs(
|
||||
dataSchemasForIntervals,
|
||||
toolbox,
|
||||
new CompactionIOConfig(null, false, null),
|
||||
new PartitionConfigurationManager(TUNING_CONFIG),
|
||||
COORDINATOR_CLIENT,
|
||||
segmentCacheManagerFactory
|
||||
);
|
||||
final List<DimensionsSpec> expectedDimensionsSpec = ImmutableList.of(
|
||||
new DimensionsSpec(getDimensionSchema(new DoubleDimensionSchema("string_to_double")))
|
||||
);
|
||||
|
@ -1441,21 +1492,25 @@ public class CompactionTaskTest
|
|||
@Test
|
||||
public void testQueryGranularityAndNullSegmentGranularity() throws IOException
|
||||
{
|
||||
final List<ParallelIndexIngestionSpec> ingestionSpecs = CompactionTask.createIngestionSchema(
|
||||
final Map<Interval, DataSchema> 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<ParallelIndexIngestionSpec> ingestionSpecs = NativeCompactionRunner.createIngestionSpecs(
|
||||
dataSchemasForIntervals,
|
||||
toolbox,
|
||||
new CompactionIOConfig(null, false, null),
|
||||
new PartitionConfigurationManager(TUNING_CONFIG),
|
||||
COORDINATOR_CLIENT,
|
||||
segmentCacheManagerFactory
|
||||
);
|
||||
final List<DimensionsSpec> expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration();
|
||||
|
||||
ingestionSpecs.sort(
|
||||
|
@ -1479,13 +1534,11 @@ public class CompactionTaskTest
|
|||
@Test
|
||||
public void testQueryGranularityAndSegmentGranularityNonNull() throws IOException
|
||||
{
|
||||
final List<ParallelIndexIngestionSpec> ingestionSpecs = CompactionTask.createIngestionSchema(
|
||||
final Map<Interval, DataSchema> 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 class CompactionTaskTest
|
|||
new PeriodGranularity(Period.months(3), null, null),
|
||||
null
|
||||
),
|
||||
COORDINATOR_CLIENT,
|
||||
segmentCacheManagerFactory,
|
||||
METRIC_BUILDER
|
||||
);
|
||||
|
||||
final List<ParallelIndexIngestionSpec> ingestionSpecs = NativeCompactionRunner.createIngestionSpecs(
|
||||
dataSchemasForIntervals,
|
||||
toolbox,
|
||||
new CompactionIOConfig(null, false, null),
|
||||
new PartitionConfigurationManager(TUNING_CONFIG),
|
||||
COORDINATOR_CLIENT,
|
||||
segmentCacheManagerFactory
|
||||
);
|
||||
|
||||
|
||||
final List<DimensionsSpec> expectedDimensionsSpec = ImmutableList.of(
|
||||
new DimensionsSpec(getDimensionSchema(new DoubleDimensionSchema("string_to_double")))
|
||||
);
|
||||
|
@ -1524,21 +1586,28 @@ public class CompactionTaskTest
|
|||
@Test
|
||||
public void testNullGranularitySpec() throws IOException
|
||||
{
|
||||
final List<ParallelIndexIngestionSpec> ingestionSpecs = CompactionTask.createIngestionSchema(
|
||||
final Map<Interval, DataSchema> 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<ParallelIndexIngestionSpec> ingestionSpecs = NativeCompactionRunner.createIngestionSpecs(
|
||||
dataSchemasForIntervals,
|
||||
toolbox,
|
||||
new CompactionIOConfig(null, false, null),
|
||||
new PartitionConfigurationManager(TUNING_CONFIG),
|
||||
COORDINATOR_CLIENT,
|
||||
segmentCacheManagerFactory
|
||||
);
|
||||
|
||||
|
||||
final List<DimensionsSpec> expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration();
|
||||
|
||||
ingestionSpecs.sort(
|
||||
|
@ -1563,21 +1632,28 @@ public class CompactionTaskTest
|
|||
public void testGranularitySpecWithNullQueryGranularityAndNullSegmentGranularity()
|
||||
throws IOException
|
||||
{
|
||||
final List<ParallelIndexIngestionSpec> ingestionSpecs = CompactionTask.createIngestionSchema(
|
||||
final Map<Interval, DataSchema> 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<ParallelIndexIngestionSpec> ingestionSpecs = NativeCompactionRunner.createIngestionSpecs(
|
||||
dataSchemasForIntervals,
|
||||
toolbox,
|
||||
new CompactionIOConfig(null, false, null),
|
||||
new PartitionConfigurationManager(TUNING_CONFIG),
|
||||
COORDINATOR_CLIENT,
|
||||
segmentCacheManagerFactory
|
||||
);
|
||||
|
||||
|
||||
final List<DimensionsSpec> expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration();
|
||||
|
||||
ingestionSpecs.sort(
|
||||
|
@ -1602,22 +1678,27 @@ public class CompactionTaskTest
|
|||
public void testGranularitySpecWithNotNullRollup()
|
||||
throws IOException
|
||||
{
|
||||
final List<ParallelIndexIngestionSpec> ingestionSpecs = CompactionTask.createIngestionSchema(
|
||||
final Map<Interval, DataSchema> 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<ParallelIndexIngestionSpec> 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 class CompactionTaskTest
|
|||
public void testGranularitySpecWithNullRollup()
|
||||
throws IOException
|
||||
{
|
||||
final List<ParallelIndexIngestionSpec> ingestionSpecs = CompactionTask.createIngestionSchema(
|
||||
final Map<Interval, DataSchema> 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<ParallelIndexIngestionSpec> 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 class CompactionTaskTest
|
|||
{
|
||||
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 class CompactionTaskTest
|
|||
{
|
||||
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"))
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -75,6 +75,7 @@ public class CompactionUtil
|
|||
null,
|
||||
null,
|
||||
new UserCompactionTaskIOConfig(true),
|
||||
null,
|
||||
null
|
||||
);
|
||||
}
|
||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.commons.io.IOUtils;
|
|||
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.Period;
|
|||
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 class ITAutoCompactionTest extends AbstractIndexerTest
|
|||
}
|
||||
}
|
||||
|
||||
@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<String> intervalsBeforeCompaction = coordinator.getSegmentIntervals(fullDatasourceName);
|
||||
intervalsBeforeCompaction.sort(null);
|
||||
// 2 segments across 1 days...
|
||||
verifySegmentsCount(2);
|
||||
Map<String, Object> 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<TaskResponseObject> compactTasksBefore = indexer.getCompleteTasksForDataSource(fullDatasourceName);
|
||||
// Verify rollup segments does not get compacted again
|
||||
forceTriggerAutoCompaction(1);
|
||||
List<TaskResponseObject> compactTasksAfter = indexer.getCompleteTasksForDataSource(fullDatasourceName);
|
||||
Assert.assertEquals(compactTasksAfter.size(), compactTasksBefore.size());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAutoCompactionOnlyRowsWithMetricShouldPreserveExistingMetrics() throws Exception
|
||||
{
|
||||
|
@ -546,8 +610,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
|
|||
}
|
||||
}
|
||||
|
||||
@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 class ITAutoCompactionTest extends AbstractIndexerTest
|
|||
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 class ITAutoCompactionTest extends AbstractIndexerTest
|
|||
}
|
||||
|
||||
@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 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
|
|||
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);
|
||||
submitCompactionConfig(maxRowsPerSegment, skipOffsetFromLatest, null, null);
|
||||
}
|
||||
|
||||
private void submitCompactionConfig(Integer maxRowsPerSegment, Period skipOffsetFromLatest, UserCompactionTaskGranularityConfig granularitySpec) throws Exception
|
||||
private void submitCompactionConfig(
|
||||
Integer maxRowsPerSegment,
|
||||
Period skipOffsetFromLatest,
|
||||
@Nullable CompactionEngine engine
|
||||
) throws Exception
|
||||
{
|
||||
submitCompactionConfig(maxRowsPerSegment, skipOffsetFromLatest, granularitySpec, false);
|
||||
submitCompactionConfig(maxRowsPerSegment, skipOffsetFromLatest, null, engine);
|
||||
}
|
||||
|
||||
private void submitCompactionConfig(Integer maxRowsPerSegment, Period skipOffsetFromLatest, UserCompactionTaskGranularityConfig granularitySpec, boolean dropExisting) throws Exception
|
||||
private void submitCompactionConfig(
|
||||
Integer maxRowsPerSegment,
|
||||
Period skipOffsetFromLatest,
|
||||
UserCompactionTaskGranularityConfig granularitySpec
|
||||
) throws Exception
|
||||
{
|
||||
submitCompactionConfig(maxRowsPerSegment, skipOffsetFromLatest, granularitySpec, null, null, null, dropExisting);
|
||||
submitCompactionConfig(maxRowsPerSegment, skipOffsetFromLatest, granularitySpec, false, 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,
|
||||
@Nullable CompactionEngine engine
|
||||
) throws Exception
|
||||
{
|
||||
submitCompactionConfig(new DynamicPartitionsSpec(maxRowsPerSegment, null), skipOffsetFromLatest, 1, granularitySpec, dimensionsSpec, transformSpec, metricsSpec, dropExisting);
|
||||
submitCompactionConfig(maxRowsPerSegment, skipOffsetFromLatest, granularitySpec, false, engine);
|
||||
}
|
||||
|
||||
private void submitCompactionConfig(
|
||||
Integer maxRowsPerSegment,
|
||||
Period skipOffsetFromLatest,
|
||||
UserCompactionTaskGranularityConfig granularitySpec,
|
||||
boolean dropExisting
|
||||
) throws Exception
|
||||
{
|
||||
submitCompactionConfig(maxRowsPerSegment, skipOffsetFromLatest, granularitySpec, dropExisting, null);
|
||||
}
|
||||
|
||||
private void submitCompactionConfig(
|
||||
Integer maxRowsPerSegment,
|
||||
Period skipOffsetFromLatest,
|
||||
UserCompactionTaskGranularityConfig granularitySpec,
|
||||
boolean dropExisting,
|
||||
@Nullable CompactionEngine engine
|
||||
) throws Exception
|
||||
{
|
||||
submitCompactionConfig(
|
||||
maxRowsPerSegment,
|
||||
skipOffsetFromLatest,
|
||||
granularitySpec,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
dropExisting,
|
||||
engine
|
||||
);
|
||||
}
|
||||
|
||||
private void submitCompactionConfig(
|
||||
Integer maxRowsPerSegment,
|
||||
Period skipOffsetFromLatest,
|
||||
UserCompactionTaskGranularityConfig granularitySpec,
|
||||
UserCompactionTaskDimensionsConfig dimensionsSpec,
|
||||
UserCompactionTaskTransformConfig transformSpec,
|
||||
AggregatorFactory[] metricsSpec,
|
||||
boolean dropExisting
|
||||
) throws Exception
|
||||
{
|
||||
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,
|
||||
@Nullable CompactionEngine engine
|
||||
) throws Exception
|
||||
{
|
||||
submitCompactionConfig(
|
||||
new DynamicPartitionsSpec(maxRowsPerSegment, null),
|
||||
skipOffsetFromLatest,
|
||||
1,
|
||||
granularitySpec,
|
||||
dimensionsSpec,
|
||||
transformSpec,
|
||||
metricsSpec,
|
||||
dropExisting,
|
||||
engine
|
||||
);
|
||||
}
|
||||
|
||||
private void submitCompactionConfig(
|
||||
|
@ -1615,6 +1770,31 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
|
|||
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 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
|
|||
metricsSpec,
|
||||
transformSpec,
|
||||
!dropExisting ? null : new UserCompactionTaskIOConfig(true),
|
||||
engine,
|
||||
null
|
||||
);
|
||||
compactionResource.submitCompactionConfig(compactionConfig);
|
||||
|
|
|
@ -103,6 +103,7 @@ public class ITAutoCompactionUpgradeTest extends AbstractIndexerTest
|
|||
null,
|
||||
null,
|
||||
new UserCompactionTaskIOConfig(true),
|
||||
null,
|
||||
null
|
||||
);
|
||||
compactionResource.submitCompactionConfig(compactionConfig);
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
}
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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:
|
||||
* <ul>
|
||||
* <li>partitionsSpec of type HashedParititionsSpec.</li>
|
||||
* <li>maxTotalRows in DynamicPartitionsSpec.</li>
|
||||
* <li>rollup set to false in granularitySpec when metricsSpec is specified. Null is treated as true.</li>
|
||||
* <li>queryGranularity set to ALL in granularitySpec.</li>
|
||||
* <li>Each metric has output column name same as the input name.</li>
|
||||
* </ul>
|
||||
*/
|
||||
private static CompactionConfigValidationResult compactionConfigSupportedByMSQEngine(DataSourceCompactionConfig newConfig)
|
||||
{
|
||||
List<CompactionConfigValidationResult> 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<String, Object> 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));
|
||||
}
|
||||
}
|
|
@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||
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 class ClientCompactionTaskGranularitySpec
|
|||
}
|
||||
|
||||
@JsonProperty
|
||||
@Nullable
|
||||
public Granularity getSegmentGranularity()
|
||||
{
|
||||
return segmentGranularity;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
@Nullable
|
||||
public Granularity getQueryGranularity()
|
||||
{
|
||||
return queryGranularity;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
@Nullable
|
||||
public Boolean isRollup()
|
||||
{
|
||||
return rollup;
|
||||
|
|
|
@ -46,6 +46,7 @@ public class ClientCompactionTaskQuery implements ClientTaskQuery
|
|||
private final AggregatorFactory[] metricsSpec;
|
||||
private final ClientCompactionTaskTransformSpec transformSpec;
|
||||
private final Map<String, Object> context;
|
||||
private final ClientCompactionRunnerInfo compactionRunner;
|
||||
|
||||
@JsonCreator
|
||||
public ClientCompactionTaskQuery(
|
||||
|
@ -57,7 +58,8 @@ public class ClientCompactionTaskQuery implements ClientTaskQuery
|
|||
@JsonProperty("dimensionsSpec") ClientCompactionTaskDimensionsSpec dimensionsSpec,
|
||||
@JsonProperty("metricsSpec") AggregatorFactory[] metrics,
|
||||
@JsonProperty("transformSpec") ClientCompactionTaskTransformSpec transformSpec,
|
||||
@JsonProperty("context") Map<String, Object> context
|
||||
@JsonProperty("context") Map<String, Object> context,
|
||||
@JsonProperty("compactionRunner") @Nullable ClientCompactionRunnerInfo compactionRunner
|
||||
)
|
||||
{
|
||||
this.id = Preconditions.checkNotNull(id, "id");
|
||||
|
@ -69,6 +71,7 @@ public class ClientCompactionTaskQuery implements ClientTaskQuery
|
|||
this.metricsSpec = metrics;
|
||||
this.transformSpec = transformSpec;
|
||||
this.context = context;
|
||||
this.compactionRunner = compactionRunner;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
|
@ -135,6 +138,13 @@ public class ClientCompactionTaskQuery implements ClientTaskQuery
|
|||
return context;
|
||||
}
|
||||
|
||||
@JsonProperty("compactionRunner")
|
||||
@Nullable
|
||||
public ClientCompactionRunnerInfo getCompactionRunner()
|
||||
{
|
||||
return compactionRunner;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
|
@ -153,7 +163,8 @@ public class ClientCompactionTaskQuery implements ClientTaskQuery
|
|||
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 class ClientCompactionTaskQuery implements ClientTaskQuery
|
|||
granularitySpec,
|
||||
dimensionsSpec,
|
||||
transformSpec,
|
||||
context
|
||||
context,
|
||||
compactionRunner
|
||||
);
|
||||
result = 31 * result + Arrays.hashCode(metricsSpec);
|
||||
return result;
|
||||
|
@ -186,6 +198,7 @@ public class ClientCompactionTaskQuery implements ClientTaskQuery
|
|||
", metricsSpec=" + Arrays.toString(metricsSpec) +
|
||||
", transformSpec=" + transformSpec +
|
||||
", context=" + context +
|
||||
", compactionRunner=" + compactionRunner +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -22,6 +22,8 @@ package org.apache.druid.server.coordinator;
|
|||
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<DataSourceCompactionConfig> 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 class CoordinatorCompactionConfig
|
|||
compactionConfigs,
|
||||
baseConfig.compactionTaskSlotRatio,
|
||||
baseConfig.maxCompactionTaskSlots,
|
||||
baseConfig.useAutoScaleSlots
|
||||
baseConfig.useAutoScaleSlots,
|
||||
null
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -64,18 +69,19 @@ public class CoordinatorCompactionConfig
|
|||
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<DataSourceCompactionConfig> 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 class CoordinatorCompactionConfig
|
|||
@JsonProperty("compactionConfigs") List<DataSourceCompactionConfig> 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 class CoordinatorCompactionConfig
|
|||
return useAutoScaleSlots;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public CompactionEngine getEngine()
|
||||
{
|
||||
return compactionEngine;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
|
@ -135,13 +143,20 @@ public class CoordinatorCompactionConfig
|
|||
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 class CoordinatorCompactionConfig
|
|||
", compactionTaskSlotRatio=" + compactionTaskSlotRatio +
|
||||
", maxCompactionTaskSlots=" + maxCompactionTaskSlots +
|
||||
", useAutoScaleSlots=" + useAutoScaleSlots +
|
||||
", compactionEngine=" + compactionEngine +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,6 +22,7 @@ package org.apache.druid.server.coordinator;
|
|||
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<String, Object> taskContext;
|
||||
private final CompactionEngine engine;
|
||||
|
||||
@JsonCreator
|
||||
public DataSourceCompactionConfig(
|
||||
|
@ -69,6 +71,7 @@ public class 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<String, Object> taskContext
|
||||
)
|
||||
{
|
||||
|
@ -88,6 +91,7 @@ public class DataSourceCompactionConfig
|
|||
this.dimensionsSpec = dimensionsSpec;
|
||||
this.transformSpec = transformSpec;
|
||||
this.taskContext = taskContext;
|
||||
this.engine = engine;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
|
@ -171,6 +175,13 @@ public class DataSourceCompactionConfig
|
|||
return taskContext;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
@Nullable
|
||||
public CompactionEngine getEngine()
|
||||
{
|
||||
return engine;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
|
@ -192,6 +203,7 @@ public class DataSourceCompactionConfig
|
|||
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 class DataSourceCompactionConfig
|
|||
dimensionsSpec,
|
||||
transformSpec,
|
||||
ioConfig,
|
||||
taskContext
|
||||
taskContext,
|
||||
engine
|
||||
);
|
||||
result = 31 * result + Arrays.hashCode(metricsSpec);
|
||||
return result;
|
||||
|
|
|
@ -27,15 +27,18 @@ import com.google.common.base.Predicate;
|
|||
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 class CompactSegments implements CoordinatorCustomDuty
|
|||
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 class CompactSegments implements CoordinatorCustomDuty
|
|||
compactionConfigs,
|
||||
currentRunAutoCompactionSnapshotBuilders,
|
||||
availableCompactionTaskSlots,
|
||||
iterator
|
||||
iterator,
|
||||
dynamicConfig.getEngine()
|
||||
);
|
||||
|
||||
final CoordinatorRunStats stats = params.getCoordinatorStats();
|
||||
|
@ -234,8 +245,7 @@ public class CompactSegments implements CoordinatorCustomDuty
|
|||
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 @@ public class CompactSegments implements CoordinatorCustomDuty
|
|||
}
|
||||
|
||||
/**
|
||||
* 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 @@ public class CompactSegments implements CoordinatorCustomDuty
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* 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<String, Object> 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 @@ public class CompactSegments implements CoordinatorCustomDuty
|
|||
Map<String, DataSourceCompactionConfig> compactionConfigs,
|
||||
Map<String, AutoCompactionSnapshot.Builder> currentRunAutoCompactionSnapshotBuilders,
|
||||
int numAvailableCompactionTaskSlots,
|
||||
CompactionSegmentIterator iterator
|
||||
CompactionSegmentIterator iterator,
|
||||
CompactionEngine defaultEngine
|
||||
)
|
||||
{
|
||||
if (numAvailableCompactionTaskSlots <= 0) {
|
||||
|
@ -363,9 +388,9 @@ public class CompactSegments implements CoordinatorCustomDuty
|
|||
}
|
||||
|
||||
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<DataSegment> segmentsToCompact = entry.getSegments();
|
||||
if (segmentsToCompact.isEmpty()) {
|
||||
|
@ -465,6 +490,29 @@ public class CompactSegments implements CoordinatorCustomDuty
|
|||
}
|
||||
}
|
||||
|
||||
final CompactionEngine compactionEngine = config.getEngine() == null ? defaultEngine : config.getEngine();
|
||||
final Map<String, Object> 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 @@ public class CompactSegments implements CoordinatorCustomDuty
|
|||
config.getMetricsSpec(),
|
||||
transformSpec,
|
||||
dropExisting,
|
||||
newAutoCompactionContext(config.getTaskContext())
|
||||
autoCompactionContext,
|
||||
new ClientCompactionRunnerInfo(compactionEngine)
|
||||
);
|
||||
|
||||
LOG.info(
|
||||
|
@ -489,7 +538,7 @@ public class CompactSegments implements CoordinatorCustomDuty
|
|||
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 @@ public class CompactSegments implements CoordinatorCustomDuty
|
|||
@Nullable AggregatorFactory[] metricsSpec,
|
||||
@Nullable ClientCompactionTaskTransformSpec transformSpec,
|
||||
@Nullable Boolean dropExisting,
|
||||
@Nullable Map<String, Object> context
|
||||
@Nullable Map<String, Object> context,
|
||||
ClientCompactionRunnerInfo compactionRunner
|
||||
)
|
||||
{
|
||||
Preconditions.checkArgument(!segments.isEmpty(), "Expect non-empty segments to compact");
|
||||
|
@ -656,7 +706,8 @@ public class CompactSegments implements CoordinatorCustomDuty
|
|||
dimensionsSpec,
|
||||
metricsSpec,
|
||||
transformSpec,
|
||||
context
|
||||
context,
|
||||
compactionRunner
|
||||
);
|
||||
FutureUtils.getUnchecked(overlordClient.runTask(taskId, taskPayload), true);
|
||||
return taskId;
|
||||
|
|
|
@ -26,9 +26,14 @@ import com.sun.jersey.spi.container.ResourceFilters;
|
|||
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 class CoordinatorCompactionConfigsResource
|
|||
.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 class CoordinatorCompactionConfigsResource
|
|||
|
||||
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 @@ public class CoordinatorCompactionConfigsResource
|
|||
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");
|
||||
|
|
|
@ -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<String, Object> 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;
|
||||
}
|
||||
}
|
|
@ -25,6 +25,7 @@ import com.google.common.collect.ImmutableMap;
|
|||
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 class DataSourceCompactionConfigTest extends InitializedNullHandlingTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
ImmutableMap.of("key", "val")
|
||||
);
|
||||
final String json = OBJECT_MAPPER.writeValueAsString(config);
|
||||
|
@ -82,6 +84,7 @@ public class DataSourceCompactionConfigTest extends InitializedNullHandlingTest
|
|||
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 class DataSourceCompactionConfigTest extends InitializedNullHandlingTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
CompactionEngine.MSQ,
|
||||
ImmutableMap.of("key", "val")
|
||||
);
|
||||
final String json = OBJECT_MAPPER.writeValueAsString(config);
|
||||
|
@ -111,6 +115,7 @@ public class DataSourceCompactionConfigTest extends InitializedNullHandlingTest
|
|||
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 class DataSourceCompactionConfigTest extends InitializedNullHandlingTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
CompactionEngine.NATIVE,
|
||||
ImmutableMap.of("key", "val")
|
||||
);
|
||||
final String json = OBJECT_MAPPER.writeValueAsString(config);
|
||||
|
@ -160,6 +166,7 @@ public class DataSourceCompactionConfigTest extends InitializedNullHandlingTest
|
|||
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 class DataSourceCompactionConfigTest extends InitializedNullHandlingTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
ImmutableMap.of("key", "val")
|
||||
);
|
||||
|
||||
|
@ -305,6 +313,7 @@ public class DataSourceCompactionConfigTest extends InitializedNullHandlingTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
ImmutableMap.of("key", "val")
|
||||
);
|
||||
final String json = OBJECT_MAPPER.writeValueAsString(config);
|
||||
|
@ -335,6 +344,7 @@ public class DataSourceCompactionConfigTest extends InitializedNullHandlingTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
ImmutableMap.of("key", "val")
|
||||
);
|
||||
final String json = OBJECT_MAPPER.writeValueAsString(config);
|
||||
|
@ -368,6 +378,7 @@ public class DataSourceCompactionConfigTest extends InitializedNullHandlingTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
ImmutableMap.of("key", "val")
|
||||
);
|
||||
final String json = OBJECT_MAPPER.writeValueAsString(config);
|
||||
|
@ -398,6 +409,7 @@ public class DataSourceCompactionConfigTest extends InitializedNullHandlingTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
ImmutableMap.of("key", "val")
|
||||
);
|
||||
final String json = OBJECT_MAPPER.writeValueAsString(config);
|
||||
|
@ -428,6 +440,7 @@ public class DataSourceCompactionConfigTest extends InitializedNullHandlingTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
ImmutableMap.of("key", "val")
|
||||
);
|
||||
final String json = OBJECT_MAPPER.writeValueAsString(config);
|
||||
|
@ -461,6 +474,7 @@ public class DataSourceCompactionConfigTest extends InitializedNullHandlingTest
|
|||
null,
|
||||
null,
|
||||
new UserCompactionTaskIOConfig(true),
|
||||
null,
|
||||
ImmutableMap.of("key", "val")
|
||||
);
|
||||
final String json = OBJECT_MAPPER.writeValueAsString(config);
|
||||
|
@ -492,6 +506,7 @@ public class DataSourceCompactionConfigTest extends InitializedNullHandlingTest
|
|||
null,
|
||||
null,
|
||||
new UserCompactionTaskIOConfig(null),
|
||||
null,
|
||||
ImmutableMap.of("key", "val")
|
||||
);
|
||||
final String json = OBJECT_MAPPER.writeValueAsString(config);
|
||||
|
@ -523,6 +538,7 @@ public class DataSourceCompactionConfigTest extends InitializedNullHandlingTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
ImmutableMap.of("key", "val")
|
||||
);
|
||||
final String json = OBJECT_MAPPER.writeValueAsString(config);
|
||||
|
@ -554,6 +570,7 @@ public class DataSourceCompactionConfigTest extends InitializedNullHandlingTest
|
|||
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 class DataSourceCompactionConfigTest extends InitializedNullHandlingTest
|
|||
new AggregatorFactory[] {new CountAggregatorFactory("cnt")},
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
ImmutableMap.of("key", "val")
|
||||
);
|
||||
final String json = OBJECT_MAPPER.writeValueAsString(config);
|
||||
|
|
|
@ -96,6 +96,7 @@ public class NewestSegmentFirstIteratorTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
);
|
||||
Assert.assertEquals(
|
||||
|
@ -141,6 +142,7 @@ public class NewestSegmentFirstIteratorTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
);
|
||||
Assert.assertEquals(
|
||||
|
@ -186,6 +188,7 @@ public class NewestSegmentFirstIteratorTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
);
|
||||
Assert.assertEquals(
|
||||
|
@ -231,6 +234,7 @@ public class NewestSegmentFirstIteratorTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
);
|
||||
Assert.assertEquals(
|
||||
|
@ -276,6 +280,7 @@ public class NewestSegmentFirstIteratorTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
);
|
||||
Assert.assertEquals(
|
||||
|
@ -321,6 +326,7 @@ public class NewestSegmentFirstIteratorTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
);
|
||||
Assert.assertEquals(
|
||||
|
@ -366,6 +372,7 @@ public class NewestSegmentFirstIteratorTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
);
|
||||
Assert.assertEquals(
|
||||
|
@ -411,6 +418,7 @@ public class NewestSegmentFirstIteratorTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
);
|
||||
Assert.assertEquals(
|
||||
|
@ -456,6 +464,7 @@ public class NewestSegmentFirstIteratorTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
);
|
||||
Assert.assertEquals(
|
||||
|
|
|
@ -2023,6 +2023,7 @@ public class NewestSegmentFirstPolicyTest
|
|||
metricsSpec,
|
||||
transformSpec,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
);
|
||||
}
|
||||
|
|
|
@ -36,12 +36,14 @@ import org.apache.druid.client.indexing.ClientCompactionIntervalSpec;
|
|||
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<Object[]> constructorFeeder()
|
||||
{
|
||||
final MutableInt nextRangePartitionBoundary = new MutableInt(0);
|
||||
|
||||
final DynamicPartitionsSpec dynamicPartitionsSpec = new DynamicPartitionsSpec(300000, Long.MAX_VALUE);
|
||||
final BiFunction<Integer, Integer, ShardSpec> numberedShardSpecCreator = NumberedShardSpec::new;
|
||||
|
||||
final HashedPartitionsSpec hashedPartitionsSpec = new HashedPartitionsSpec(null, 2, ImmutableList.of("dim"));
|
||||
final BiFunction<Integer, Integer, ShardSpec> 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<Integer, Integer, ShardSpec> 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<Integer, Integer, ShardSpec>) NumberedShardSpec::new
|
||||
},
|
||||
new Object[]{
|
||||
new HashedPartitionsSpec(null, 2, ImmutableList.of("dim")),
|
||||
(BiFunction<Integer, Integer, ShardSpec>) (bucketId, numBuckets) -> new HashBasedNumberedShardSpec(
|
||||
bucketId,
|
||||
numBuckets,
|
||||
bucketId,
|
||||
numBuckets,
|
||||
ImmutableList.of("dim"),
|
||||
null,
|
||||
JSON_MAPPER
|
||||
)
|
||||
},
|
||||
new Object[]{
|
||||
new SingleDimensionPartitionsSpec(300000, null, "dim", false),
|
||||
(BiFunction<Integer, Integer, ShardSpec>) (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<Integer, Integer, ShardSpec> shardSpecFactory;
|
||||
private final CompactionEngine engine;
|
||||
|
||||
private DataSourcesSnapshot dataSources;
|
||||
Map<String, List<DataSegment>> datasourceToSegments = new HashMap<>();
|
||||
|
||||
public CompactSegmentsTest(PartitionsSpec partitionsSpec, BiFunction<Integer, Integer, ShardSpec> shardSpecFactory)
|
||||
public CompactSegmentsTest(
|
||||
PartitionsSpec partitionsSpec,
|
||||
BiFunction<Integer, Integer, ShardSpec> shardSpecFactory,
|
||||
CompactionEngine engine
|
||||
)
|
||||
{
|
||||
this.partitionsSpec = partitionsSpec;
|
||||
this.shardSpecFactory = shardSpecFactory;
|
||||
this.engine = engine;
|
||||
}
|
||||
|
||||
@Before
|
||||
|
@ -640,7 +655,13 @@ public class CompactSegmentsTest
|
|||
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 class CompactSegmentsTest
|
|||
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 class CompactSegmentsTest
|
|||
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 class CompactSegmentsTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
engine,
|
||||
null
|
||||
)
|
||||
);
|
||||
|
@ -769,6 +806,7 @@ public class CompactSegmentsTest
|
|||
null,
|
||||
null,
|
||||
new UserCompactionTaskIOConfig(true),
|
||||
engine,
|
||||
null
|
||||
)
|
||||
);
|
||||
|
@ -818,6 +856,7 @@ public class CompactSegmentsTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
engine,
|
||||
null
|
||||
)
|
||||
);
|
||||
|
@ -867,6 +906,7 @@ public class CompactSegmentsTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
engine,
|
||||
null
|
||||
)
|
||||
);
|
||||
|
@ -927,6 +967,7 @@ public class CompactSegmentsTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
engine,
|
||||
null
|
||||
)
|
||||
);
|
||||
|
@ -979,6 +1020,7 @@ public class CompactSegmentsTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
engine,
|
||||
null
|
||||
)
|
||||
);
|
||||
|
@ -1028,6 +1070,7 @@ public class CompactSegmentsTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
engine,
|
||||
null
|
||||
)
|
||||
);
|
||||
|
@ -1081,6 +1124,7 @@ public class CompactSegmentsTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
)
|
||||
);
|
||||
|
@ -1137,6 +1181,7 @@ public class CompactSegmentsTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
engine,
|
||||
null
|
||||
)
|
||||
);
|
||||
|
@ -1165,8 +1210,13 @@ public class CompactSegmentsTest
|
|||
{
|
||||
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 class CompactSegmentsTest
|
|||
// 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 class CompactSegmentsTest
|
|||
null,
|
||||
new UserCompactionTaskTransformConfig(new SelectorDimFilter("dim1", "foo", null)),
|
||||
null,
|
||||
engine,
|
||||
null
|
||||
)
|
||||
);
|
||||
|
@ -1302,6 +1353,7 @@ public class CompactSegmentsTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
engine,
|
||||
null
|
||||
)
|
||||
);
|
||||
|
@ -1354,6 +1406,7 @@ public class CompactSegmentsTest
|
|||
aggregatorFactories,
|
||||
null,
|
||||
null,
|
||||
engine,
|
||||
null
|
||||
)
|
||||
);
|
||||
|
@ -1434,6 +1487,7 @@ public class CompactSegmentsTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
engine,
|
||||
null
|
||||
)
|
||||
);
|
||||
|
@ -1520,6 +1574,7 @@ public class CompactSegmentsTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
engine,
|
||||
null
|
||||
)
|
||||
);
|
||||
|
@ -1577,6 +1632,7 @@ public class CompactSegmentsTest
|
|||
new AggregatorFactory[] {new CountAggregatorFactory("cnt")},
|
||||
null,
|
||||
null,
|
||||
engine,
|
||||
null
|
||||
)
|
||||
);
|
||||
|
@ -1629,6 +1685,7 @@ public class CompactSegmentsTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
engine,
|
||||
null
|
||||
)
|
||||
);
|
||||
|
@ -1781,7 +1838,8 @@ public class CompactSegmentsTest
|
|||
compactionConfigs,
|
||||
numCompactionTaskSlots == null ? null : 1.0, // 100% when numCompactionTaskSlots is not null
|
||||
numCompactionTaskSlots,
|
||||
useAutoScaleSlots
|
||||
useAutoScaleSlots,
|
||||
null
|
||||
)
|
||||
)
|
||||
.build();
|
||||
|
@ -1901,10 +1959,23 @@ public class CompactSegmentsTest
|
|||
|
||||
private List<DataSourceCompactionConfig> createCompactionConfigs()
|
||||
{
|
||||
return createCompactionConfigs(null);
|
||||
return createCompactionConfigs(null, null);
|
||||
}
|
||||
|
||||
private List<DataSourceCompactionConfig> createCompactionConfigs(@Nullable Integer maxNumConcurrentSubTasks)
|
||||
private List<DataSourceCompactionConfig> createcompactionConfigsForNative(@Nullable Integer maxNumConcurrentSubTasks)
|
||||
{
|
||||
return createCompactionConfigs(maxNumConcurrentSubTasks, null);
|
||||
}
|
||||
|
||||
private List<DataSourceCompactionConfig> createcompactionConfigsForMSQ(Integer maxNumTasks)
|
||||
{
|
||||
return createCompactionConfigs(null, maxNumTasks);
|
||||
}
|
||||
|
||||
private List<DataSourceCompactionConfig> createCompactionConfigs(
|
||||
@Nullable Integer maxNumConcurrentSubTasksForNative,
|
||||
@Nullable Integer maxNumTasksForMSQ
|
||||
)
|
||||
{
|
||||
final List<DataSourceCompactionConfig> compactionConfigs = new ArrayList<>();
|
||||
for (int i = 0; i < 3; i++) {
|
||||
|
@ -1928,7 +1999,7 @@ public class CompactSegmentsTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
maxNumConcurrentSubTasks,
|
||||
maxNumConcurrentSubTasksForNative,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
|
@ -1942,7 +2013,8 @@ public class CompactSegmentsTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
engine,
|
||||
maxNumTasksForMSQ == null ? null : ImmutableMap.of(ClientMSQContext.CTX_MAX_NUM_TASKS, maxNumTasksForMSQ)
|
||||
)
|
||||
);
|
||||
}
|
||||
|
@ -2172,7 +2244,7 @@ public class CompactSegmentsTest
|
|||
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 class CompactSegmentsTest
|
|||
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));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -158,6 +158,7 @@ public class KillCompactionConfigTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
ImmutableMap.of("key", "val")
|
||||
);
|
||||
|
||||
|
@ -173,6 +174,7 @@ public class KillCompactionConfigTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
ImmutableMap.of("key", "val")
|
||||
);
|
||||
CoordinatorCompactionConfig originalCurrentConfig = CoordinatorCompactionConfig.from(ImmutableList.of(inactiveDatasourceConfig, activeDatasourceConfig));
|
||||
|
@ -255,6 +257,7 @@ public class KillCompactionConfigTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
ImmutableMap.of("key", "val")
|
||||
);
|
||||
|
||||
|
|
|
@ -22,8 +22,12 @@ package org.apache.druid.server.http;
|
|||
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 class CoordinatorCompactionConfigsResourceTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
CompactionEngine.NATIVE,
|
||||
ImmutableMap.of("key", "val")
|
||||
);
|
||||
Response result = coordinatorCompactionConfigsResource.addOrUpdateCompactionConfig(
|
||||
|
@ -202,6 +209,7 @@ public class CoordinatorCompactionConfigsResourceTest
|
|||
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 class CoordinatorCompactionConfigsResourceTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
ImmutableMap.of("key", "val")
|
||||
);
|
||||
final CoordinatorCompactionConfig originalConfig = CoordinatorCompactionConfig.from(ImmutableList.of(toDelete));
|
||||
|
@ -388,6 +397,7 @@ public class CoordinatorCompactionConfigsResourceTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
CompactionEngine.MSQ,
|
||||
ImmutableMap.of("key", "val")
|
||||
);
|
||||
String author = "maytas";
|
||||
|
@ -401,6 +411,102 @@ public class CoordinatorCompactionConfigsResourceTest
|
|||
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<byte[]> oldConfigCaptor = ArgumentCaptor.forClass(byte[].class);
|
||||
final ArgumentCaptor<CoordinatorCompactionConfig> 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
|
||||
|
|
|
@ -39,6 +39,7 @@ import javax.annotation.Nullable;
|
|||
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;
|
||||
|
|
Loading…
Reference in New Issue