LakshSingla commented on code in PR #16291:
URL: https://github.com/apache/druid/pull/16291#discussion_r1637687114


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java:
##########
@@ -0,0 +1,508 @@
+/*
+ * 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.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.data.input.impl.DimensionSchema;
+import org.apache.druid.error.InvalidInput;
+import org.apache.druid.indexer.TaskStatus;
+import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec;
+import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
+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.NonnullPair;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.granularity.AllGranularity;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.granularity.Granularity;
+import org.apache.druid.java.util.common.granularity.PeriodGranularity;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.math.expr.ExprMacroTable;
+import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination;
+import org.apache.druid.msq.kernel.WorkerAssignmentStrategy;
+import org.apache.druid.msq.util.MultiStageQueryContext;
+import org.apache.druid.query.Druids;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContext;
+import org.apache.druid.query.TableDataSource;
+import org.apache.druid.query.dimension.DefaultDimensionSpec;
+import org.apache.druid.query.dimension.DimensionSpec;
+import org.apache.druid.query.expression.TimestampFloorExprMacro;
+import org.apache.druid.query.filter.DimFilter;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
+import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.VirtualColumns;
+import org.apache.druid.segment.column.ColumnHolder;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.column.RowSignature;
+import org.apache.druid.segment.indexing.DataSchema;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+import org.apache.druid.sql.calcite.parser.DruidSqlInsert;
+import org.apache.druid.sql.calcite.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);
+  private static final Granularity DEFAULT_SEGMENT_GRANULARITY = 
Granularities.ALL;
+
+  private final ObjectMapper jsonMapper;
+  private final Injector injector;
+
+  public static final String TIME_VIRTUAL_COLUMN = "__vTime";
+  public static final String TIME_COLUMN = ColumnHolder.TIME_COLUMN_NAME;
+
+  @JsonIgnore
+  private final CurrentSubTaskHolder currentSubTaskHolder = new 
CurrentSubTaskHolder(
+      (taskObject, config) -> {
+        final MSQControllerTask msqControllerTask = (MSQControllerTask) 
taskObject;
+        msqControllerTask.stopGracefully(config);
+      });
+
+
+  public MSQCompactionRunner(@JacksonInject ObjectMapper jsonMapper, 
@JacksonInject Injector injector)
+  {
+    this.jsonMapper = jsonMapper;
+    this.injector = injector;
+  }
+
+  @Override
+  public NonnullPair<Boolean, String> supportsCompactionSpec(
+      CompactionTask compactionTask
+  )
+  {
+    if (compactionTask.getTuningConfig() != null) {
+      PartitionsSpec partitionsSpec = 
compactionTask.getTuningConfig().getPartitionsSpec();
+      if (!(partitionsSpec instanceof DynamicPartitionsSpec
+            || partitionsSpec instanceof DimensionRangePartitionsSpec)) {
+        return new NonnullPair<>(false, StringUtils.format(
+            "Invalid partition spec type[%s] for MSQ compaction engine."
+            + " Type must be either DynamicPartitionsSpec or 
DynamicRangePartitionsSpec.",
+            partitionsSpec.getClass()
+        ));
+      }
+      if (partitionsSpec instanceof DynamicPartitionsSpec
+          && ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows() != 
null) {
+        return new NonnullPair<>(false, StringUtils.format(
+            "maxTotalRows[%d] in DynamicPartitionsSpec not supported for MSQ 
compaction engine.",
+            ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows()
+        ));
+      }
+    }
+    if (compactionTask.getMetricsSpec() != null
+        && compactionTask.getGranularitySpec() != null
+        && !compactionTask.getGranularitySpec()
+                          .isRollup()) {
+      return new NonnullPair<>(
+          false,
+          "rollup in granularitySpec must be set to true if metricsSpec is 
specifed "
+          + "for MSQ compaction engine."

Review Comment:
   nit: MSQ compaction engine can be renamed as MSQ engine for consistency with 
other validation messages. 
   



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java:
##########
@@ -0,0 +1,508 @@
+/*
+ * 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.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.data.input.impl.DimensionSchema;
+import org.apache.druid.error.InvalidInput;
+import org.apache.druid.indexer.TaskStatus;
+import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec;
+import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
+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.NonnullPair;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.granularity.AllGranularity;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.granularity.Granularity;
+import org.apache.druid.java.util.common.granularity.PeriodGranularity;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.math.expr.ExprMacroTable;
+import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination;
+import org.apache.druid.msq.kernel.WorkerAssignmentStrategy;
+import org.apache.druid.msq.util.MultiStageQueryContext;
+import org.apache.druid.query.Druids;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContext;
+import org.apache.druid.query.TableDataSource;
+import org.apache.druid.query.dimension.DefaultDimensionSpec;
+import org.apache.druid.query.dimension.DimensionSpec;
+import org.apache.druid.query.expression.TimestampFloorExprMacro;
+import org.apache.druid.query.filter.DimFilter;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
+import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.VirtualColumns;
+import org.apache.druid.segment.column.ColumnHolder;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.column.RowSignature;
+import org.apache.druid.segment.indexing.DataSchema;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+import org.apache.druid.sql.calcite.parser.DruidSqlInsert;
+import org.apache.druid.sql.calcite.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);
+  private static final Granularity DEFAULT_SEGMENT_GRANULARITY = 
Granularities.ALL;
+
+  private final ObjectMapper jsonMapper;
+  private final Injector injector;
+
+  public static final String TIME_VIRTUAL_COLUMN = "__vTime";
+  public static final String TIME_COLUMN = ColumnHolder.TIME_COLUMN_NAME;

Review Comment:
   We shouldn't need this variable. The callers can use 
`ColumnHolder.TIME_COLUMN_NAME` instead. 



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java:
##########
@@ -0,0 +1,508 @@
+/*
+ * 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.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.data.input.impl.DimensionSchema;
+import org.apache.druid.error.InvalidInput;
+import org.apache.druid.indexer.TaskStatus;
+import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec;
+import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
+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.NonnullPair;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.granularity.AllGranularity;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.granularity.Granularity;
+import org.apache.druid.java.util.common.granularity.PeriodGranularity;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.math.expr.ExprMacroTable;
+import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination;
+import org.apache.druid.msq.kernel.WorkerAssignmentStrategy;
+import org.apache.druid.msq.util.MultiStageQueryContext;
+import org.apache.druid.query.Druids;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContext;
+import org.apache.druid.query.TableDataSource;
+import org.apache.druid.query.dimension.DefaultDimensionSpec;
+import org.apache.druid.query.dimension.DimensionSpec;
+import org.apache.druid.query.expression.TimestampFloorExprMacro;
+import org.apache.druid.query.filter.DimFilter;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
+import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.VirtualColumns;
+import org.apache.druid.segment.column.ColumnHolder;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.column.RowSignature;
+import org.apache.druid.segment.indexing.DataSchema;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+import org.apache.druid.sql.calcite.parser.DruidSqlInsert;
+import org.apache.druid.sql.calcite.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);
+  private static final Granularity DEFAULT_SEGMENT_GRANULARITY = 
Granularities.ALL;
+
+  private final ObjectMapper jsonMapper;
+  private final Injector injector;
+
+  public static final String TIME_VIRTUAL_COLUMN = "__vTime";
+  public static final String TIME_COLUMN = ColumnHolder.TIME_COLUMN_NAME;
+
+  @JsonIgnore
+  private final CurrentSubTaskHolder currentSubTaskHolder = new 
CurrentSubTaskHolder(
+      (taskObject, config) -> {
+        final MSQControllerTask msqControllerTask = (MSQControllerTask) 
taskObject;
+        msqControllerTask.stopGracefully(config);
+      });
+
+
+  public MSQCompactionRunner(@JacksonInject ObjectMapper jsonMapper, 
@JacksonInject Injector injector)
+  {
+    this.jsonMapper = jsonMapper;
+    this.injector = injector;
+  }
+
+  @Override
+  public NonnullPair<Boolean, String> supportsCompactionSpec(
+      CompactionTask compactionTask
+  )
+  {
+    if (compactionTask.getTuningConfig() != null) {
+      PartitionsSpec partitionsSpec = 
compactionTask.getTuningConfig().getPartitionsSpec();
+      if (!(partitionsSpec instanceof DynamicPartitionsSpec
+            || partitionsSpec instanceof DimensionRangePartitionsSpec)) {
+        return new NonnullPair<>(false, StringUtils.format(
+            "Invalid partition spec type[%s] for MSQ compaction engine."
+            + " Type must be either DynamicPartitionsSpec or 
DynamicRangePartitionsSpec.",
+            partitionsSpec.getClass()
+        ));
+      }
+      if (partitionsSpec instanceof DynamicPartitionsSpec
+          && ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows() != 
null) {
+        return new NonnullPair<>(false, StringUtils.format(
+            "maxTotalRows[%d] in DynamicPartitionsSpec not supported for MSQ 
compaction engine.",
+            ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows()
+        ));
+      }
+    }
+    if (compactionTask.getMetricsSpec() != null
+        && compactionTask.getGranularitySpec() != null
+        && !compactionTask.getGranularitySpec()
+                          .isRollup()) {
+      return new NonnullPair<>(
+          false,
+          "rollup in granularitySpec must be set to true if metricsSpec is 
specifed "
+          + "for MSQ compaction engine."
+      );
+    }
+
+    QueryContext compactionTaskContext = new 
QueryContext(compactionTask.getContext());
+    if (!MultiStageQueryContext.isFinalizeAggregations(compactionTaskContext)) 
{
+      return new NonnullPair<>(false, StringUtils.format(
+          "Config[%s] cannot be set to false for auto-compaction with MSQ 
engine.",
+          MultiStageQueryContext.CTX_FINALIZE_AGGREGATIONS
+      ));
+    }
+    if (MultiStageQueryContext.getAssignmentStrategy(compactionTaskContext) == 
WorkerAssignmentStrategy.AUTO) {
+      return new NonnullPair<>(
+          false,
+          StringUtils.format(
+              "Config[%s] cannot be set to value[%s] for auto-compaction with 
MSQ engine.",
+              MultiStageQueryContext.CTX_TASK_ASSIGNMENT_STRATEGY,
+              WorkerAssignmentStrategy.AUTO
+          )
+      );
+    }
+    return new NonnullPair<>(true, "");
+  }
+
+  @Override
+  public CurrentSubTaskHolder getCurrentSubTaskHolder()
+  {
+    return currentSubTaskHolder;
+  }
+
+  @Override
+  public TaskStatus runCompactionTasks(
+      CompactionTask compactionTask,
+      List<NonnullPair<Interval, DataSchema>> intervalDataSchemas,
+      TaskToolbox taskToolbox
+  ) throws Exception
+  {
+    List<MSQControllerTask> msqControllerTasks = 
compactionToMSQTasks(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> compactionToMSQTasks(

Review Comment:
   nit: Consider creating a separate utils class for this, so that different 
configurations can be unit tested independently. 



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java:
##########
@@ -0,0 +1,508 @@
+/*
+ * 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.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.data.input.impl.DimensionSchema;
+import org.apache.druid.error.InvalidInput;
+import org.apache.druid.indexer.TaskStatus;
+import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec;
+import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
+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.NonnullPair;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.granularity.AllGranularity;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.granularity.Granularity;
+import org.apache.druid.java.util.common.granularity.PeriodGranularity;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.math.expr.ExprMacroTable;
+import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination;
+import org.apache.druid.msq.kernel.WorkerAssignmentStrategy;
+import org.apache.druid.msq.util.MultiStageQueryContext;
+import org.apache.druid.query.Druids;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContext;
+import org.apache.druid.query.TableDataSource;
+import org.apache.druid.query.dimension.DefaultDimensionSpec;
+import org.apache.druid.query.dimension.DimensionSpec;
+import org.apache.druid.query.expression.TimestampFloorExprMacro;
+import org.apache.druid.query.filter.DimFilter;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
+import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.VirtualColumns;
+import org.apache.druid.segment.column.ColumnHolder;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.column.RowSignature;
+import org.apache.druid.segment.indexing.DataSchema;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+import org.apache.druid.sql.calcite.parser.DruidSqlInsert;
+import org.apache.druid.sql.calcite.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);
+  private static final Granularity DEFAULT_SEGMENT_GRANULARITY = 
Granularities.ALL;
+
+  private final ObjectMapper jsonMapper;
+  private final Injector injector;
+
+  public static final String TIME_VIRTUAL_COLUMN = "__vTime";
+  public static final String TIME_COLUMN = ColumnHolder.TIME_COLUMN_NAME;
+
+  @JsonIgnore
+  private final CurrentSubTaskHolder currentSubTaskHolder = new 
CurrentSubTaskHolder(
+      (taskObject, config) -> {
+        final MSQControllerTask msqControllerTask = (MSQControllerTask) 
taskObject;
+        msqControllerTask.stopGracefully(config);
+      });
+
+
+  public MSQCompactionRunner(@JacksonInject ObjectMapper jsonMapper, 
@JacksonInject Injector injector)
+  {
+    this.jsonMapper = jsonMapper;
+    this.injector = injector;
+  }
+
+  @Override
+  public NonnullPair<Boolean, String> supportsCompactionSpec(
+      CompactionTask compactionTask
+  )
+  {
+    if (compactionTask.getTuningConfig() != null) {
+      PartitionsSpec partitionsSpec = 
compactionTask.getTuningConfig().getPartitionsSpec();
+      if (!(partitionsSpec instanceof DynamicPartitionsSpec
+            || partitionsSpec instanceof DimensionRangePartitionsSpec)) {
+        return new NonnullPair<>(false, StringUtils.format(
+            "Invalid partition spec type[%s] for MSQ compaction engine."
+            + " Type must be either DynamicPartitionsSpec or 
DynamicRangePartitionsSpec.",
+            partitionsSpec.getClass()
+        ));
+      }
+      if (partitionsSpec instanceof DynamicPartitionsSpec
+          && ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows() != 
null) {
+        return new NonnullPair<>(false, StringUtils.format(
+            "maxTotalRows[%d] in DynamicPartitionsSpec not supported for MSQ 
compaction engine.",
+            ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows()
+        ));
+      }
+    }
+    if (compactionTask.getMetricsSpec() != null
+        && compactionTask.getGranularitySpec() != null
+        && !compactionTask.getGranularitySpec()
+                          .isRollup()) {
+      return new NonnullPair<>(
+          false,
+          "rollup in granularitySpec must be set to true if metricsSpec is 
specifed "
+          + "for MSQ compaction engine."
+      );
+    }
+
+    QueryContext compactionTaskContext = new 
QueryContext(compactionTask.getContext());
+    if (!MultiStageQueryContext.isFinalizeAggregations(compactionTaskContext)) 
{
+      return new NonnullPair<>(false, StringUtils.format(
+          "Config[%s] cannot be set to false for auto-compaction with MSQ 
engine.",
+          MultiStageQueryContext.CTX_FINALIZE_AGGREGATIONS
+      ));
+    }
+    if (MultiStageQueryContext.getAssignmentStrategy(compactionTaskContext) == 
WorkerAssignmentStrategy.AUTO) {
+      return new NonnullPair<>(
+          false,
+          StringUtils.format(
+              "Config[%s] cannot be set to value[%s] for auto-compaction with 
MSQ engine.",
+              MultiStageQueryContext.CTX_TASK_ASSIGNMENT_STRATEGY,
+              WorkerAssignmentStrategy.AUTO
+          )
+      );
+    }
+    return new NonnullPair<>(true, "");
+  }
+
+  @Override
+  public CurrentSubTaskHolder getCurrentSubTaskHolder()
+  {
+    return currentSubTaskHolder;
+  }
+
+  @Override
+  public TaskStatus runCompactionTasks(
+      CompactionTask compactionTask,
+      List<NonnullPair<Interval, DataSchema>> intervalDataSchemas,
+      TaskToolbox taskToolbox
+  ) throws Exception
+  {
+    List<MSQControllerTask> msqControllerTasks = 
compactionToMSQTasks(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> compactionToMSQTasks(
+      CompactionTask compactionTask,
+      List<NonnullPair<Interval, DataSchema>> intervalDataSchemas

Review Comment:
   I am not very familiar with the traditional auto-compaction configuration. 
How many elements are expected to be in this list, and will we be running that 
many MSQ jobs at once? For example, if we have 10 elements, will be running 10 
jobs at once, and is this how we do it for traditional auto-compaction as well? 



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java:
##########
@@ -0,0 +1,508 @@
+/*
+ * 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.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.data.input.impl.DimensionSchema;
+import org.apache.druid.error.InvalidInput;
+import org.apache.druid.indexer.TaskStatus;
+import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec;
+import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
+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.NonnullPair;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.granularity.AllGranularity;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.granularity.Granularity;
+import org.apache.druid.java.util.common.granularity.PeriodGranularity;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.math.expr.ExprMacroTable;
+import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination;
+import org.apache.druid.msq.kernel.WorkerAssignmentStrategy;
+import org.apache.druid.msq.util.MultiStageQueryContext;
+import org.apache.druid.query.Druids;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContext;
+import org.apache.druid.query.TableDataSource;
+import org.apache.druid.query.dimension.DefaultDimensionSpec;
+import org.apache.druid.query.dimension.DimensionSpec;
+import org.apache.druid.query.expression.TimestampFloorExprMacro;
+import org.apache.druid.query.filter.DimFilter;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
+import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.VirtualColumns;
+import org.apache.druid.segment.column.ColumnHolder;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.column.RowSignature;
+import org.apache.druid.segment.indexing.DataSchema;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+import org.apache.druid.sql.calcite.parser.DruidSqlInsert;
+import org.apache.druid.sql.calcite.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);
+  private static final Granularity DEFAULT_SEGMENT_GRANULARITY = 
Granularities.ALL;
+
+  private final ObjectMapper jsonMapper;
+  private final Injector injector;
+
+  public static final String TIME_VIRTUAL_COLUMN = "__vTime";

Review Comment:
   Can you please add Javadoc for this field stating why this column is 
required, and who populates it. 



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java:
##########


Review Comment:
   Note on the MSQControllerTask launched via autocompaction - Do we assign it 
the same priority as a normal controller task, or do we assign it the priority 
of an autocompaction task? 



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java:
##########
@@ -143,6 +143,22 @@ public MSQControllerTask(
     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

Review Comment:
   When do we require a custom injector to be supplied? I see it is called in 
the `MSQCompactionRunner` and we are passing it the same injector there 
(annotated with `@JacksonInject`).



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java:
##########
@@ -0,0 +1,508 @@
+/*
+ * 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.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.data.input.impl.DimensionSchema;
+import org.apache.druid.error.InvalidInput;
+import org.apache.druid.indexer.TaskStatus;
+import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec;
+import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
+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.NonnullPair;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.granularity.AllGranularity;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.granularity.Granularity;
+import org.apache.druid.java.util.common.granularity.PeriodGranularity;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.math.expr.ExprMacroTable;
+import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination;
+import org.apache.druid.msq.kernel.WorkerAssignmentStrategy;
+import org.apache.druid.msq.util.MultiStageQueryContext;
+import org.apache.druid.query.Druids;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContext;
+import org.apache.druid.query.TableDataSource;
+import org.apache.druid.query.dimension.DefaultDimensionSpec;
+import org.apache.druid.query.dimension.DimensionSpec;
+import org.apache.druid.query.expression.TimestampFloorExprMacro;
+import org.apache.druid.query.filter.DimFilter;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
+import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.VirtualColumns;
+import org.apache.druid.segment.column.ColumnHolder;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.column.RowSignature;
+import org.apache.druid.segment.indexing.DataSchema;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+import org.apache.druid.sql.calcite.parser.DruidSqlInsert;
+import org.apache.druid.sql.calcite.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);
+  private static final Granularity DEFAULT_SEGMENT_GRANULARITY = 
Granularities.ALL;
+
+  private final ObjectMapper jsonMapper;
+  private final Injector injector;
+
+  public static final String TIME_VIRTUAL_COLUMN = "__vTime";
+  public static final String TIME_COLUMN = ColumnHolder.TIME_COLUMN_NAME;
+
+  @JsonIgnore
+  private final CurrentSubTaskHolder currentSubTaskHolder = new 
CurrentSubTaskHolder(
+      (taskObject, config) -> {
+        final MSQControllerTask msqControllerTask = (MSQControllerTask) 
taskObject;
+        msqControllerTask.stopGracefully(config);
+      });
+
+
+  public MSQCompactionRunner(@JacksonInject ObjectMapper jsonMapper, 
@JacksonInject Injector injector)
+  {
+    this.jsonMapper = jsonMapper;
+    this.injector = injector;
+  }
+
+  @Override
+  public NonnullPair<Boolean, String> supportsCompactionSpec(
+      CompactionTask compactionTask
+  )
+  {
+    if (compactionTask.getTuningConfig() != null) {
+      PartitionsSpec partitionsSpec = 
compactionTask.getTuningConfig().getPartitionsSpec();
+      if (!(partitionsSpec instanceof DynamicPartitionsSpec
+            || partitionsSpec instanceof DimensionRangePartitionsSpec)) {
+        return new NonnullPair<>(false, StringUtils.format(
+            "Invalid partition spec type[%s] for MSQ compaction engine."
+            + " Type must be either DynamicPartitionsSpec or 
DynamicRangePartitionsSpec.",
+            partitionsSpec.getClass()
+        ));
+      }
+      if (partitionsSpec instanceof DynamicPartitionsSpec
+          && ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows() != 
null) {
+        return new NonnullPair<>(false, StringUtils.format(
+            "maxTotalRows[%d] in DynamicPartitionsSpec not supported for MSQ 
compaction engine.",

Review Comment:
   nit: `MSQ compaction engine` can be renamed as `MSQ engine` 



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java:
##########
@@ -0,0 +1,508 @@
+/*
+ * 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.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.data.input.impl.DimensionSchema;
+import org.apache.druid.error.InvalidInput;
+import org.apache.druid.indexer.TaskStatus;
+import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec;
+import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
+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.NonnullPair;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.granularity.AllGranularity;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.granularity.Granularity;
+import org.apache.druid.java.util.common.granularity.PeriodGranularity;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.math.expr.ExprMacroTable;
+import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination;
+import org.apache.druid.msq.kernel.WorkerAssignmentStrategy;
+import org.apache.druid.msq.util.MultiStageQueryContext;
+import org.apache.druid.query.Druids;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContext;
+import org.apache.druid.query.TableDataSource;
+import org.apache.druid.query.dimension.DefaultDimensionSpec;
+import org.apache.druid.query.dimension.DimensionSpec;
+import org.apache.druid.query.expression.TimestampFloorExprMacro;
+import org.apache.druid.query.filter.DimFilter;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
+import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.VirtualColumns;
+import org.apache.druid.segment.column.ColumnHolder;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.column.RowSignature;
+import org.apache.druid.segment.indexing.DataSchema;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+import org.apache.druid.sql.calcite.parser.DruidSqlInsert;
+import org.apache.druid.sql.calcite.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);
+  private static final Granularity DEFAULT_SEGMENT_GRANULARITY = 
Granularities.ALL;
+
+  private final ObjectMapper jsonMapper;
+  private final Injector injector;
+
+  public static final String TIME_VIRTUAL_COLUMN = "__vTime";
+  public static final String TIME_COLUMN = ColumnHolder.TIME_COLUMN_NAME;
+
+  @JsonIgnore
+  private final CurrentSubTaskHolder currentSubTaskHolder = new 
CurrentSubTaskHolder(
+      (taskObject, config) -> {
+        final MSQControllerTask msqControllerTask = (MSQControllerTask) 
taskObject;
+        msqControllerTask.stopGracefully(config);
+      });
+
+
+  public MSQCompactionRunner(@JacksonInject ObjectMapper jsonMapper, 
@JacksonInject Injector injector)
+  {
+    this.jsonMapper = jsonMapper;
+    this.injector = injector;
+  }
+
+  @Override
+  public NonnullPair<Boolean, String> supportsCompactionSpec(
+      CompactionTask compactionTask
+  )
+  {
+    if (compactionTask.getTuningConfig() != null) {
+      PartitionsSpec partitionsSpec = 
compactionTask.getTuningConfig().getPartitionsSpec();
+      if (!(partitionsSpec instanceof DynamicPartitionsSpec
+            || partitionsSpec instanceof DimensionRangePartitionsSpec)) {
+        return new NonnullPair<>(false, StringUtils.format(
+            "Invalid partition spec type[%s] for MSQ compaction engine."
+            + " Type must be either DynamicPartitionsSpec or 
DynamicRangePartitionsSpec.",
+            partitionsSpec.getClass()
+        ));
+      }
+      if (partitionsSpec instanceof DynamicPartitionsSpec
+          && ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows() != 
null) {
+        return new NonnullPair<>(false, StringUtils.format(
+            "maxTotalRows[%d] in DynamicPartitionsSpec not supported for MSQ 
compaction engine.",
+            ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows()
+        ));
+      }
+    }
+    if (compactionTask.getMetricsSpec() != null
+        && compactionTask.getGranularitySpec() != null
+        && !compactionTask.getGranularitySpec()
+                          .isRollup()) {
+      return new NonnullPair<>(
+          false,
+          "rollup in granularitySpec must be set to true if metricsSpec is 
specifed "
+          + "for MSQ compaction engine."
+      );
+    }
+
+    QueryContext compactionTaskContext = new 
QueryContext(compactionTask.getContext());
+    if (!MultiStageQueryContext.isFinalizeAggregations(compactionTaskContext)) 
{
+      return new NonnullPair<>(false, StringUtils.format(
+          "Config[%s] cannot be set to false for auto-compaction with MSQ 
engine.",
+          MultiStageQueryContext.CTX_FINALIZE_AGGREGATIONS
+      ));
+    }
+    if (MultiStageQueryContext.getAssignmentStrategy(compactionTaskContext) == 
WorkerAssignmentStrategy.AUTO) {
+      return new NonnullPair<>(
+          false,
+          StringUtils.format(
+              "Config[%s] cannot be set to value[%s] for auto-compaction with 
MSQ engine.",
+              MultiStageQueryContext.CTX_TASK_ASSIGNMENT_STRATEGY,
+              WorkerAssignmentStrategy.AUTO
+          )
+      );
+    }
+    return new NonnullPair<>(true, "");
+  }
+
+  @Override
+  public CurrentSubTaskHolder getCurrentSubTaskHolder()
+  {
+    return currentSubTaskHolder;
+  }
+
+  @Override
+  public TaskStatus runCompactionTasks(
+      CompactionTask compactionTask,
+      List<NonnullPair<Interval, DataSchema>> intervalDataSchemas,
+      TaskToolbox taskToolbox
+  ) throws Exception
+  {
+    List<MSQControllerTask> msqControllerTasks = 
compactionToMSQTasks(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> compactionToMSQTasks(
+      CompactionTask compactionTask,
+      List<NonnullPair<Interval, DataSchema>> intervalDataSchemas
+  ) throws JsonProcessingException
+  {
+    List<MSQControllerTask> msqControllerTasks = new ArrayList<>();
+
+    for (NonnullPair<Interval, DataSchema> intervalDataSchema : 
intervalDataSchemas) {
+      Query<?> query;
+      Interval interval = intervalDataSchema.lhs;
+      DataSchema dataSchema = intervalDataSchema.rhs;
+
+      if (!isGroupBy(dataSchema)) {
+        query = buildScanQuery(compactionTask, interval, dataSchema);
+      } else {
+        query = buildGroupByQuery(compactionTask, interval, dataSchema);
+      }
+      QueryContext compactionTaskContext = new 
QueryContext(compactionTask.getContext());
+
+      MSQSpec msqSpec = MSQSpec.builder()
+                               .query(query)
+                               .columnMappings(getColumnMappings(dataSchema))
+                               
.destination(buildMSQDestination(compactionTask, dataSchema, 
compactionTaskContext))
+                               
.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,
+      QueryContext compactionTaskContext
+  )
+  {
+    final Interval replaceInterval = compactionTask.getIoConfig()
+                                                   .getInputSpec()
+                                                   
.findInterval(compactionTask.getDataSource());
+
+    final List<String> segmentSortOrder = 
MultiStageQueryContext.getSortOrder(compactionTaskContext);

Review Comment:
   What's populating the sort order originally? Should we even be allowing it 
for auto-compaction, since I think the native auto-compaction doesn't support 
it. 



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java:
##########
@@ -0,0 +1,508 @@
+/*
+ * 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.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.data.input.impl.DimensionSchema;
+import org.apache.druid.error.InvalidInput;
+import org.apache.druid.indexer.TaskStatus;
+import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec;
+import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
+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.NonnullPair;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.granularity.AllGranularity;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.granularity.Granularity;
+import org.apache.druid.java.util.common.granularity.PeriodGranularity;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.math.expr.ExprMacroTable;
+import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination;
+import org.apache.druid.msq.kernel.WorkerAssignmentStrategy;
+import org.apache.druid.msq.util.MultiStageQueryContext;
+import org.apache.druid.query.Druids;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContext;
+import org.apache.druid.query.TableDataSource;
+import org.apache.druid.query.dimension.DefaultDimensionSpec;
+import org.apache.druid.query.dimension.DimensionSpec;
+import org.apache.druid.query.expression.TimestampFloorExprMacro;
+import org.apache.druid.query.filter.DimFilter;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
+import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.VirtualColumns;
+import org.apache.druid.segment.column.ColumnHolder;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.column.RowSignature;
+import org.apache.druid.segment.indexing.DataSchema;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+import org.apache.druid.sql.calcite.parser.DruidSqlInsert;
+import org.apache.druid.sql.calcite.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);
+  private static final Granularity DEFAULT_SEGMENT_GRANULARITY = 
Granularities.ALL;
+
+  private final ObjectMapper jsonMapper;
+  private final Injector injector;
+
+  public static final String TIME_VIRTUAL_COLUMN = "__vTime";
+  public static final String TIME_COLUMN = ColumnHolder.TIME_COLUMN_NAME;
+
+  @JsonIgnore
+  private final CurrentSubTaskHolder currentSubTaskHolder = new 
CurrentSubTaskHolder(
+      (taskObject, config) -> {
+        final MSQControllerTask msqControllerTask = (MSQControllerTask) 
taskObject;
+        msqControllerTask.stopGracefully(config);
+      });
+
+
+  public MSQCompactionRunner(@JacksonInject ObjectMapper jsonMapper, 
@JacksonInject Injector injector)
+  {
+    this.jsonMapper = jsonMapper;
+    this.injector = injector;
+  }
+
+  @Override
+  public NonnullPair<Boolean, String> supportsCompactionSpec(
+      CompactionTask compactionTask
+  )
+  {
+    if (compactionTask.getTuningConfig() != null) {
+      PartitionsSpec partitionsSpec = 
compactionTask.getTuningConfig().getPartitionsSpec();
+      if (!(partitionsSpec instanceof DynamicPartitionsSpec
+            || partitionsSpec instanceof DimensionRangePartitionsSpec)) {
+        return new NonnullPair<>(false, StringUtils.format(
+            "Invalid partition spec type[%s] for MSQ compaction engine."
+            + " Type must be either DynamicPartitionsSpec or 
DynamicRangePartitionsSpec.",
+            partitionsSpec.getClass()
+        ));
+      }
+      if (partitionsSpec instanceof DynamicPartitionsSpec
+          && ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows() != 
null) {
+        return new NonnullPair<>(false, StringUtils.format(
+            "maxTotalRows[%d] in DynamicPartitionsSpec not supported for MSQ 
compaction engine.",
+            ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows()
+        ));
+      }
+    }
+    if (compactionTask.getMetricsSpec() != null
+        && compactionTask.getGranularitySpec() != null
+        && !compactionTask.getGranularitySpec()
+                          .isRollup()) {
+      return new NonnullPair<>(
+          false,
+          "rollup in granularitySpec must be set to true if metricsSpec is 
specifed "
+          + "for MSQ compaction engine."
+      );
+    }
+
+    QueryContext compactionTaskContext = new 
QueryContext(compactionTask.getContext());
+    if (!MultiStageQueryContext.isFinalizeAggregations(compactionTaskContext)) 
{
+      return new NonnullPair<>(false, StringUtils.format(
+          "Config[%s] cannot be set to false for auto-compaction with MSQ 
engine.",
+          MultiStageQueryContext.CTX_FINALIZE_AGGREGATIONS
+      ));
+    }
+    if (MultiStageQueryContext.getAssignmentStrategy(compactionTaskContext) == 
WorkerAssignmentStrategy.AUTO) {

Review Comment:
   Can the user specify the MAX compaction strategy with MSQ autocompaction? If 
so, how many tasks will actually run if the value specified is different from 
the 
[auto-compaction-configuration](https://druid.apache.org/docs/latest/api-reference/automatic-compaction-api/#view-automatic-compaction-configuration)?
 



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java:
##########
@@ -0,0 +1,508 @@
+/*
+ * 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.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.data.input.impl.DimensionSchema;
+import org.apache.druid.error.InvalidInput;
+import org.apache.druid.indexer.TaskStatus;
+import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec;
+import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
+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.NonnullPair;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.granularity.AllGranularity;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.granularity.Granularity;
+import org.apache.druid.java.util.common.granularity.PeriodGranularity;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.math.expr.ExprMacroTable;
+import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination;
+import org.apache.druid.msq.kernel.WorkerAssignmentStrategy;
+import org.apache.druid.msq.util.MultiStageQueryContext;
+import org.apache.druid.query.Druids;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContext;
+import org.apache.druid.query.TableDataSource;
+import org.apache.druid.query.dimension.DefaultDimensionSpec;
+import org.apache.druid.query.dimension.DimensionSpec;
+import org.apache.druid.query.expression.TimestampFloorExprMacro;
+import org.apache.druid.query.filter.DimFilter;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
+import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.VirtualColumns;
+import org.apache.druid.segment.column.ColumnHolder;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.column.RowSignature;
+import org.apache.druid.segment.indexing.DataSchema;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+import org.apache.druid.sql.calcite.parser.DruidSqlInsert;
+import org.apache.druid.sql.calcite.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);
+  private static final Granularity DEFAULT_SEGMENT_GRANULARITY = 
Granularities.ALL;
+
+  private final ObjectMapper jsonMapper;
+  private final Injector injector;
+
+  public static final String TIME_VIRTUAL_COLUMN = "__vTime";
+  public static final String TIME_COLUMN = ColumnHolder.TIME_COLUMN_NAME;
+
+  @JsonIgnore
+  private final CurrentSubTaskHolder currentSubTaskHolder = new 
CurrentSubTaskHolder(
+      (taskObject, config) -> {
+        final MSQControllerTask msqControllerTask = (MSQControllerTask) 
taskObject;
+        msqControllerTask.stopGracefully(config);
+      });
+
+
+  public MSQCompactionRunner(@JacksonInject ObjectMapper jsonMapper, 
@JacksonInject Injector injector)
+  {
+    this.jsonMapper = jsonMapper;
+    this.injector = injector;
+  }
+
+  @Override
+  public NonnullPair<Boolean, String> supportsCompactionSpec(
+      CompactionTask compactionTask
+  )
+  {
+    if (compactionTask.getTuningConfig() != null) {
+      PartitionsSpec partitionsSpec = 
compactionTask.getTuningConfig().getPartitionsSpec();
+      if (!(partitionsSpec instanceof DynamicPartitionsSpec
+            || partitionsSpec instanceof DimensionRangePartitionsSpec)) {
+        return new NonnullPair<>(false, StringUtils.format(
+            "Invalid partition spec type[%s] for MSQ compaction engine."
+            + " Type must be either DynamicPartitionsSpec or 
DynamicRangePartitionsSpec.",
+            partitionsSpec.getClass()
+        ));
+      }
+      if (partitionsSpec instanceof DynamicPartitionsSpec
+          && ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows() != 
null) {
+        return new NonnullPair<>(false, StringUtils.format(
+            "maxTotalRows[%d] in DynamicPartitionsSpec not supported for MSQ 
compaction engine.",
+            ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows()
+        ));
+      }
+    }
+    if (compactionTask.getMetricsSpec() != null
+        && compactionTask.getGranularitySpec() != null
+        && !compactionTask.getGranularitySpec()
+                          .isRollup()) {
+      return new NonnullPair<>(
+          false,
+          "rollup in granularitySpec must be set to true if metricsSpec is 
specifed "
+          + "for MSQ compaction engine."
+      );
+    }
+
+    QueryContext compactionTaskContext = new 
QueryContext(compactionTask.getContext());
+    if (!MultiStageQueryContext.isFinalizeAggregations(compactionTaskContext)) 
{
+      return new NonnullPair<>(false, StringUtils.format(
+          "Config[%s] cannot be set to false for auto-compaction with MSQ 
engine.",
+          MultiStageQueryContext.CTX_FINALIZE_AGGREGATIONS
+      ));

Review Comment:
   I fear that it is deviating from the normal behavior of the compaction. 
Finalize aggregations will finalize the aggregation, and throw away the 
intermediate data. For example, in the case of sketches, finalizing will save 
the estimated value in the segment and eliminate the sketch itself. 
   Compare this with `rollup = true` in the native engine, where we will also 
store the sketch data.
   
   This will produce a discrepancy when trying to roll up the data from the 
segments that have been compacted.
   
   Moreover, w.r.t dimensionToAggregationMap - I think its an unnecessary 
restriction that can cause trouble. Consider a case where there are two 
segments with a column - one is created with sketches and `finalizeAggregations 
= true`, while the other is created where the columns is a dimension. The 
compaction in native will treat the column as a dimension in both the places, 
however it will get confusing as to how the column will get treated as in case 
of MSQ compaction.
   We shouldn't store the aggregator used to create a dimension because the 
intermediate state that the aggregated value stores can get lost when 
finalizeAggregations = true (depends on the aggregation). 



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java:
##########
@@ -0,0 +1,508 @@
+/*
+ * 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.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.data.input.impl.DimensionSchema;
+import org.apache.druid.error.InvalidInput;
+import org.apache.druid.indexer.TaskStatus;
+import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec;
+import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
+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.NonnullPair;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.granularity.AllGranularity;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.granularity.Granularity;
+import org.apache.druid.java.util.common.granularity.PeriodGranularity;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.math.expr.ExprMacroTable;
+import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination;
+import org.apache.druid.msq.kernel.WorkerAssignmentStrategy;
+import org.apache.druid.msq.util.MultiStageQueryContext;
+import org.apache.druid.query.Druids;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContext;
+import org.apache.druid.query.TableDataSource;
+import org.apache.druid.query.dimension.DefaultDimensionSpec;
+import org.apache.druid.query.dimension.DimensionSpec;
+import org.apache.druid.query.expression.TimestampFloorExprMacro;
+import org.apache.druid.query.filter.DimFilter;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
+import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.VirtualColumns;
+import org.apache.druid.segment.column.ColumnHolder;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.column.RowSignature;
+import org.apache.druid.segment.indexing.DataSchema;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+import org.apache.druid.sql.calcite.parser.DruidSqlInsert;
+import org.apache.druid.sql.calcite.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);
+  private static final Granularity DEFAULT_SEGMENT_GRANULARITY = 
Granularities.ALL;
+
+  private final ObjectMapper jsonMapper;
+  private final Injector injector;
+
+  public static final String TIME_VIRTUAL_COLUMN = "__vTime";
+  public static final String TIME_COLUMN = ColumnHolder.TIME_COLUMN_NAME;
+
+  @JsonIgnore
+  private final CurrentSubTaskHolder currentSubTaskHolder = new 
CurrentSubTaskHolder(
+      (taskObject, config) -> {
+        final MSQControllerTask msqControllerTask = (MSQControllerTask) 
taskObject;
+        msqControllerTask.stopGracefully(config);
+      });
+
+
+  public MSQCompactionRunner(@JacksonInject ObjectMapper jsonMapper, 
@JacksonInject Injector injector)
+  {
+    this.jsonMapper = jsonMapper;
+    this.injector = injector;
+  }
+
+  @Override
+  public NonnullPair<Boolean, String> supportsCompactionSpec(
+      CompactionTask compactionTask
+  )
+  {
+    if (compactionTask.getTuningConfig() != null) {
+      PartitionsSpec partitionsSpec = 
compactionTask.getTuningConfig().getPartitionsSpec();
+      if (!(partitionsSpec instanceof DynamicPartitionsSpec
+            || partitionsSpec instanceof DimensionRangePartitionsSpec)) {
+        return new NonnullPair<>(false, StringUtils.format(
+            "Invalid partition spec type[%s] for MSQ compaction engine."
+            + " Type must be either DynamicPartitionsSpec or 
DynamicRangePartitionsSpec.",
+            partitionsSpec.getClass()
+        ));
+      }
+      if (partitionsSpec instanceof DynamicPartitionsSpec
+          && ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows() != 
null) {
+        return new NonnullPair<>(false, StringUtils.format(
+            "maxTotalRows[%d] in DynamicPartitionsSpec not supported for MSQ 
compaction engine.",
+            ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows()
+        ));
+      }
+    }
+    if (compactionTask.getMetricsSpec() != null
+        && compactionTask.getGranularitySpec() != null
+        && !compactionTask.getGranularitySpec()
+                          .isRollup()) {
+      return new NonnullPair<>(
+          false,
+          "rollup in granularitySpec must be set to true if metricsSpec is 
specifed "
+          + "for MSQ compaction engine."
+      );
+    }
+
+    QueryContext compactionTaskContext = new 
QueryContext(compactionTask.getContext());
+    if (!MultiStageQueryContext.isFinalizeAggregations(compactionTaskContext)) 
{
+      return new NonnullPair<>(false, StringUtils.format(
+          "Config[%s] cannot be set to false for auto-compaction with MSQ 
engine.",
+          MultiStageQueryContext.CTX_FINALIZE_AGGREGATIONS
+      ));

Review Comment:
   I am of the opinion that `finalizeAggregations` shouldn't be set by the 
users at all. We should figure out the value from the given native compaction 
spec, and IMO rollup = true means finalizeAggregations = false. We are not 
allowing rollup = false in MSQ, so we don't have to worry about that.  



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to