diff --git a/codestyle/spotbugs-exclude.xml b/codestyle/spotbugs-exclude.xml
index f4efc658902..607d4da423a 100644
--- a/codestyle/spotbugs-exclude.xml
+++ b/codestyle/spotbugs-exclude.xml
@@ -71,6 +71,13 @@
+
+
+
+
+
+
+
diff --git a/core/src/main/java/org/apache/druid/common/utils/IdUtils.java b/core/src/main/java/org/apache/druid/common/utils/IdUtils.java
index 1dcac02b2e3..2d3f3301016 100644
--- a/core/src/main/java/org/apache/druid/common/utils/IdUtils.java
+++ b/core/src/main/java/org/apache/druid/common/utils/IdUtils.java
@@ -41,7 +41,7 @@ public class IdUtils
private static final Joiner UNDERSCORE_JOINER = Joiner.on("_");
- public static void validateId(String thingToValidate, String stringToValidate)
+ public static String validateId(String thingToValidate, String stringToValidate)
{
Preconditions.checkArgument(
!Strings.isNullOrEmpty(stringToValidate),
@@ -71,6 +71,8 @@ public class IdUtils
throw new IAE("%s cannot contain character #%d (at position %d).", thingToValidate, (int) c, i);
}
}
+
+ return stringToValidate;
}
public static String getRandomId()
diff --git a/distribution/pom.xml b/distribution/pom.xml
index 84d3c218fb8..17e6b5381d0 100644
--- a/distribution/pom.xml
+++ b/distribution/pom.xml
@@ -216,6 +216,8 @@
-c
org.apache.druid.extensions:druid-lookups-cached-single
-c
+ org.apache.druid.extensions:druid-multi-stage-query
+ -c
org.apache.druid.extensions:druid-protobuf-extensions
-c
org.apache.druid.extensions:mysql-metadata-storage
@@ -396,6 +398,8 @@
-c
org.apache.druid.extensions:druid-lookups-cached-single
-c
+ org.apache.druid.extensions:druid-multi-stage-query
+ -c
org.apache.druid.extensions:druid-protobuf-extensions
-c
org.apache.druid.extensions:mysql-metadata-storage
@@ -679,6 +683,8 @@
-c
org.apache.druid.extensions:druid-lookups-cached-global
-c
+ org.apache.druid.extensions:druid-multi-stage-query
+ -c
org.apache.druid.extensions:druid-protobuf-extensions
-c
org.apache.druid.extensions:mysql-metadata-storage
diff --git a/examples/conf/druid/cluster/_common/common.runtime.properties b/examples/conf/druid/cluster/_common/common.runtime.properties
index edcef960297..eafa11742d5 100644
--- a/examples/conf/druid/cluster/_common/common.runtime.properties
+++ b/examples/conf/druid/cluster/_common/common.runtime.properties
@@ -24,7 +24,7 @@
# If you specify `druid.extensions.loadList=[]`, Druid won't load any extension from file system.
# If you don't specify `druid.extensions.loadList`, Druid will load all the extensions under root extension directory.
# More info: https://druid.apache.org/docs/latest/operations/including-extensions.html
-druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches"]
+druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches", "druid-multi-stage-query"]
# If you have a different version of Hadoop, place your Hadoop client jar files in your hadoop-dependencies directory
# and uncomment the line below to point to your directory.
diff --git a/examples/conf/druid/cluster/master/coordinator-overlord/runtime.properties b/examples/conf/druid/cluster/master/coordinator-overlord/runtime.properties
index 8928cc9f8ed..00071a83d7d 100644
--- a/examples/conf/druid/cluster/master/coordinator-overlord/runtime.properties
+++ b/examples/conf/druid/cluster/master/coordinator-overlord/runtime.properties
@@ -22,6 +22,7 @@ druid.plaintextPort=8081
druid.coordinator.startDelay=PT10S
druid.coordinator.period=PT5S
+druid.manager.segments.pollDuration=PT5S
# Run the overlord service in the coordinator process
druid.coordinator.asOverlord.enabled=true
diff --git a/examples/conf/druid/single-server/large/_common/common.runtime.properties b/examples/conf/druid/single-server/large/_common/common.runtime.properties
index edcef960297..eafa11742d5 100644
--- a/examples/conf/druid/single-server/large/_common/common.runtime.properties
+++ b/examples/conf/druid/single-server/large/_common/common.runtime.properties
@@ -24,7 +24,7 @@
# If you specify `druid.extensions.loadList=[]`, Druid won't load any extension from file system.
# If you don't specify `druid.extensions.loadList`, Druid will load all the extensions under root extension directory.
# More info: https://druid.apache.org/docs/latest/operations/including-extensions.html
-druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches"]
+druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches", "druid-multi-stage-query"]
# If you have a different version of Hadoop, place your Hadoop client jar files in your hadoop-dependencies directory
# and uncomment the line below to point to your directory.
diff --git a/examples/conf/druid/single-server/large/coordinator-overlord/runtime.properties b/examples/conf/druid/single-server/large/coordinator-overlord/runtime.properties
index 8928cc9f8ed..00071a83d7d 100644
--- a/examples/conf/druid/single-server/large/coordinator-overlord/runtime.properties
+++ b/examples/conf/druid/single-server/large/coordinator-overlord/runtime.properties
@@ -22,6 +22,7 @@ druid.plaintextPort=8081
druid.coordinator.startDelay=PT10S
druid.coordinator.period=PT5S
+druid.manager.segments.pollDuration=PT5S
# Run the overlord service in the coordinator process
druid.coordinator.asOverlord.enabled=true
diff --git a/examples/conf/druid/single-server/medium/_common/common.runtime.properties b/examples/conf/druid/single-server/medium/_common/common.runtime.properties
index edcef960297..eafa11742d5 100644
--- a/examples/conf/druid/single-server/medium/_common/common.runtime.properties
+++ b/examples/conf/druid/single-server/medium/_common/common.runtime.properties
@@ -24,7 +24,7 @@
# If you specify `druid.extensions.loadList=[]`, Druid won't load any extension from file system.
# If you don't specify `druid.extensions.loadList`, Druid will load all the extensions under root extension directory.
# More info: https://druid.apache.org/docs/latest/operations/including-extensions.html
-druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches"]
+druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches", "druid-multi-stage-query"]
# If you have a different version of Hadoop, place your Hadoop client jar files in your hadoop-dependencies directory
# and uncomment the line below to point to your directory.
diff --git a/examples/conf/druid/single-server/medium/coordinator-overlord/runtime.properties b/examples/conf/druid/single-server/medium/coordinator-overlord/runtime.properties
index 8928cc9f8ed..00071a83d7d 100644
--- a/examples/conf/druid/single-server/medium/coordinator-overlord/runtime.properties
+++ b/examples/conf/druid/single-server/medium/coordinator-overlord/runtime.properties
@@ -22,6 +22,7 @@ druid.plaintextPort=8081
druid.coordinator.startDelay=PT10S
druid.coordinator.period=PT5S
+druid.manager.segments.pollDuration=PT5S
# Run the overlord service in the coordinator process
druid.coordinator.asOverlord.enabled=true
diff --git a/examples/conf/druid/single-server/micro-quickstart/_common/common.runtime.properties b/examples/conf/druid/single-server/micro-quickstart/_common/common.runtime.properties
index edcef960297..eafa11742d5 100644
--- a/examples/conf/druid/single-server/micro-quickstart/_common/common.runtime.properties
+++ b/examples/conf/druid/single-server/micro-quickstart/_common/common.runtime.properties
@@ -24,7 +24,7 @@
# If you specify `druid.extensions.loadList=[]`, Druid won't load any extension from file system.
# If you don't specify `druid.extensions.loadList`, Druid will load all the extensions under root extension directory.
# More info: https://druid.apache.org/docs/latest/operations/including-extensions.html
-druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches"]
+druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches", "druid-multi-stage-query"]
# If you have a different version of Hadoop, place your Hadoop client jar files in your hadoop-dependencies directory
# and uncomment the line below to point to your directory.
diff --git a/examples/conf/druid/single-server/micro-quickstart/coordinator-overlord/runtime.properties b/examples/conf/druid/single-server/micro-quickstart/coordinator-overlord/runtime.properties
index 8928cc9f8ed..00071a83d7d 100644
--- a/examples/conf/druid/single-server/micro-quickstart/coordinator-overlord/runtime.properties
+++ b/examples/conf/druid/single-server/micro-quickstart/coordinator-overlord/runtime.properties
@@ -22,6 +22,7 @@ druid.plaintextPort=8081
druid.coordinator.startDelay=PT10S
druid.coordinator.period=PT5S
+druid.manager.segments.pollDuration=PT5S
# Run the overlord service in the coordinator process
druid.coordinator.asOverlord.enabled=true
diff --git a/examples/conf/druid/single-server/nano-quickstart/_common/common.runtime.properties b/examples/conf/druid/single-server/nano-quickstart/_common/common.runtime.properties
index edcef960297..eafa11742d5 100644
--- a/examples/conf/druid/single-server/nano-quickstart/_common/common.runtime.properties
+++ b/examples/conf/druid/single-server/nano-quickstart/_common/common.runtime.properties
@@ -24,7 +24,7 @@
# If you specify `druid.extensions.loadList=[]`, Druid won't load any extension from file system.
# If you don't specify `druid.extensions.loadList`, Druid will load all the extensions under root extension directory.
# More info: https://druid.apache.org/docs/latest/operations/including-extensions.html
-druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches"]
+druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches", "druid-multi-stage-query"]
# If you have a different version of Hadoop, place your Hadoop client jar files in your hadoop-dependencies directory
# and uncomment the line below to point to your directory.
diff --git a/examples/conf/druid/single-server/nano-quickstart/coordinator-overlord/runtime.properties b/examples/conf/druid/single-server/nano-quickstart/coordinator-overlord/runtime.properties
index 8928cc9f8ed..00071a83d7d 100644
--- a/examples/conf/druid/single-server/nano-quickstart/coordinator-overlord/runtime.properties
+++ b/examples/conf/druid/single-server/nano-quickstart/coordinator-overlord/runtime.properties
@@ -22,6 +22,7 @@ druid.plaintextPort=8081
druid.coordinator.startDelay=PT10S
druid.coordinator.period=PT5S
+druid.manager.segments.pollDuration=PT5S
# Run the overlord service in the coordinator process
druid.coordinator.asOverlord.enabled=true
diff --git a/examples/conf/druid/single-server/small/_common/common.runtime.properties b/examples/conf/druid/single-server/small/_common/common.runtime.properties
index edcef960297..eafa11742d5 100644
--- a/examples/conf/druid/single-server/small/_common/common.runtime.properties
+++ b/examples/conf/druid/single-server/small/_common/common.runtime.properties
@@ -24,7 +24,7 @@
# If you specify `druid.extensions.loadList=[]`, Druid won't load any extension from file system.
# If you don't specify `druid.extensions.loadList`, Druid will load all the extensions under root extension directory.
# More info: https://druid.apache.org/docs/latest/operations/including-extensions.html
-druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches"]
+druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches", "druid-multi-stage-query"]
# If you have a different version of Hadoop, place your Hadoop client jar files in your hadoop-dependencies directory
# and uncomment the line below to point to your directory.
diff --git a/examples/conf/druid/single-server/small/coordinator-overlord/runtime.properties b/examples/conf/druid/single-server/small/coordinator-overlord/runtime.properties
index 8928cc9f8ed..00071a83d7d 100644
--- a/examples/conf/druid/single-server/small/coordinator-overlord/runtime.properties
+++ b/examples/conf/druid/single-server/small/coordinator-overlord/runtime.properties
@@ -22,6 +22,7 @@ druid.plaintextPort=8081
druid.coordinator.startDelay=PT10S
druid.coordinator.period=PT5S
+druid.manager.segments.pollDuration=PT5S
# Run the overlord service in the coordinator process
druid.coordinator.asOverlord.enabled=true
diff --git a/examples/conf/druid/single-server/xlarge/_common/common.runtime.properties b/examples/conf/druid/single-server/xlarge/_common/common.runtime.properties
index edcef960297..eafa11742d5 100644
--- a/examples/conf/druid/single-server/xlarge/_common/common.runtime.properties
+++ b/examples/conf/druid/single-server/xlarge/_common/common.runtime.properties
@@ -24,7 +24,7 @@
# If you specify `druid.extensions.loadList=[]`, Druid won't load any extension from file system.
# If you don't specify `druid.extensions.loadList`, Druid will load all the extensions under root extension directory.
# More info: https://druid.apache.org/docs/latest/operations/including-extensions.html
-druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches"]
+druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches", "druid-multi-stage-query"]
# If you have a different version of Hadoop, place your Hadoop client jar files in your hadoop-dependencies directory
# and uncomment the line below to point to your directory.
diff --git a/examples/conf/druid/single-server/xlarge/coordinator-overlord/runtime.properties b/examples/conf/druid/single-server/xlarge/coordinator-overlord/runtime.properties
index 8928cc9f8ed..00071a83d7d 100644
--- a/examples/conf/druid/single-server/xlarge/coordinator-overlord/runtime.properties
+++ b/examples/conf/druid/single-server/xlarge/coordinator-overlord/runtime.properties
@@ -22,6 +22,7 @@ druid.plaintextPort=8081
druid.coordinator.startDelay=PT10S
druid.coordinator.period=PT5S
+druid.manager.segments.pollDuration=PT5S
# Run the overlord service in the coordinator process
druid.coordinator.asOverlord.enabled=true
diff --git a/extensions-core/multi-stage-query/README.md b/extensions-core/multi-stage-query/README.md
new file mode 100644
index 00000000000..491be586d8b
--- /dev/null
+++ b/extensions-core/multi-stage-query/README.md
@@ -0,0 +1,141 @@
+
+
+# `druid-multi-stage-query` developer notes
+
+This document provides developer notes for the major packages of the `druid-multi-stage-query` extension. It does not
+discuss future plans; these are discussed on the list or in GitHub issues.
+
+## Model
+
+Multi-stage queries are modeled as a directed acyclic graph (DAG) of stages. Each stage has some inputs (or, possibly,
+zero inputs, if the stage generates data out of thin air). Those inputs can be Druid tables, external data, or
+the outputs of other stages of the same query. There is one final stage that produces the query result. Stage outputs,
+whether they are inputs to other stages or results of the query itself, are optionally shuffled.
+
+SQL-based ingestion jobs run as multi-stage query tasks. In this case, the result of the query is inserted into the
+target table.
+
+Package `org.apache.druid.msq.kernel` and `org.apache.druid.msq.input` contain the model classes for multi-stage
+queries.
+
+Main classes:
+
+- [QueryDefinition](src/main/java/org/apache/druid/msq/kernel/QueryDefinition.java) represents a multi-stage query.
+- [StageDefinition](src/main/java/org/apache/druid/msq/kernel/StageDefinition.java) represents an individual stage of
+ a multi-stage query.
+- [InputSpec](src/main/java/org/apache/druid/msq/input/InputSpec.java) represents an input to a stage. Links between
+ stages are represented by [StageInputSpec](src/main/java/org/apache/druid/msq/input/stage/StageInputSpec.java).
+- [ShuffleSpec](src/main/java/org/apache/druid/msq/input/ShuffleSpec.java) represents the shuffle that happens as part
+ of stage output.
+
+## Indexing service
+
+Package `org.apache.druid.msq.indexing` contains code related to integrating with the indexing service. This allows
+multi-stage queries to run as indexing service tasks.
+
+Main classes:
+
+- [MSQControllerTask](src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java) is a `query_controller` task.
+ Each query has one controller task. The controller task launches worker tasks to do the actual work.
+- [MSQWorkerTask](src/main/java/org/apache/druid/msq/indexing/MSQWorkerTask.java) is a `query_worker` task. These stick
+ around for the lifetime of the query. Each task may do work for multiple stages. It has a specific worker number
+ that is retained across all stages that the task may work on.
+
+## Planning
+
+Multi-stage queries, when run as SQL via query tasks, are planned in three phases:
+
+1. The SQL planner generates a native query corresponding to the user's SQL query.
+2. The `query_controller` task generates a multi-stage QueryDefinition corresponding to the native query, using
+ QueryKit.
+3. The `query_controller` task determines how many workers will run and generates WorkOrders for each worker.
+
+Once all three of these phases are complete, `query_worker` tasks are launched, sent their WorkOrders, and query
+execution begins.
+
+Packages `org.apache.druid.msq.querykit`, `org.apache.druid.msq.input`, and `org.apache.druid.msq.kernel` contain code
+related to query planning.
+
+Main classes:
+
+- [QueryKit](src/main/java/org/apache/druid/msq/querykit/QueryKit.java) implementations produce QueryDefinition
+ instances from native Druid queries.
+- [InputSlice](src/main/java/org/apache/druid/msq/input/InputSlice.java) represents a slice of stage input assigned to
+ a particular worker.
+- [WorkerAssignmentStrategy](src/main/java/org/apache/druid/msq/kernel/WorkerAssignmentStrategy.java) drives the splitting
+ of input specs into input slices, and is therefore responsible for assigning work to workers.
+- [WorkOrder](src/main/java/org/apache/druid/msq/kernel/WorkOrder.java) represents the work assigned to a particular
+ worker in a particular stage.
+
+## Execution
+
+Package `org.apache.druid.msq.exec` and `org.apache.druid.msq.kernel` contain code related to driving query execution.
+
+Main classes:
+
+- [ControllerQueryKernel](src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java) is the state
+ machine that drives execution on the controller.
+- [WorkerStageKernel](src/main/java/org/apache/druid/msq/kernel/worker/WorkerStageKernel.java) is the state machine
+ that drives execution on workers.
+- [ControllerImpl](src/main/java/org/apache/druid/msq/kernel/exec/ControllerImpl.java) embeds a ControllerQueryKernel
+ and handles controller-side execution beyond the state machine, including query planning, RPC, counters, and so on.
+- [WorkerImpl](src/main/java/org/apache/druid/msq/kernel/exec/WorkerImpl.java) embeds a WorkerStageKernel and handles
+ worker-side execution beyond the state machine, including setup of processors, channels, counters, and so on.
+
+## Statistics
+
+Package `org.apache.druid.msq.statistics` contains code related to determining partition boundaries as part of
+doing a range-based shuffle. During a stage that intends to do range-based shuffle, workers gather statistics
+using a ClusterByStatisticsCollector, which are then merged on the controller and used to generate partition
+boundaries.
+
+- [ClusterByStatisticsCollector](src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollector.java)
+ is the interface to statistics collection.
+- [ClusterByStatisticsCollectorImpl](src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollectorImpl.java)
+ is the main implementation of the statistics collection interface.
+
+## Counters
+
+Package `org.apache.druid.msq.counters` contains code related to tracking and reporting query execution metrics.
+
+Main classes:
+
+- [CounterTracker](src/main/java/org/apache/druid/msq/counters/CounterTracker.java) is used by workers to keep track of
+ named counters of various types.
+- [CounterSnapshots](src/main/java/org/apache/druid/msq/counters/CounterSnapshots.java) are periodically reported from
+ workers to the controller.
+- [CounterSnapshotsTree](src/main/java/org/apache/druid/msq/counters/CounterSnapshotsTree.java) is used by the
+ controller to store worker snapshots. It is also included in task reports, which enables live metrics, and also
+ allows query counters to be reviewed after the query has been completed.
+
+## SQL
+
+Package `org.apache.druid.msq.sql` contains code related to integration with Druid SQL APIs.
+
+Main classes:
+
+- [SqlTaskResource](src/main/java/org/apache/druid/msq/counters/CounterTracker.java) offers the endpoint
+ `/druid/v2/sql/task`, where SQL queries are executed as multi-stage query tasks.
+- [MSQTaskSqlEngine](src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java) is a SqlEngine implementation that
+ executes SQL queries as multi-stage query tasks. It is injected into the SqlTaskResource.
+
+## References
+
+- Multi-stage distributed query proposal: https://github.com/apache/druid/issues/12262
diff --git a/extensions-core/multi-stage-query/pom.xml b/extensions-core/multi-stage-query/pom.xml
new file mode 100644
index 00000000000..a8a2513bab5
--- /dev/null
+++ b/extensions-core/multi-stage-query/pom.xml
@@ -0,0 +1,299 @@
+
+
+
+
+
+ 4.0.0
+
+ org.apache.druid.extensions
+ druid-multi-stage-query
+ druid-multi-stage-query
+ druid-multi-stage-query
+
+
+ org.apache.druid
+ druid
+ 0.24.0-SNAPSHOT
+ ../../pom.xml
+
+
+
+
+ org.apache.druid
+ druid-core
+ ${project.parent.version}
+ provided
+
+
+ org.apache.druid
+ druid-processing
+ ${project.parent.version}
+ provided
+
+
+ org.apache.druid
+ druid-server
+ ${project.parent.version}
+ provided
+
+
+ org.apache.druid
+ druid-indexing-service
+ ${project.parent.version}
+ provided
+
+
+ org.apache.druid
+ druid-sql
+ ${project.parent.version}
+ provided
+
+
+ org.apache.druid
+ druid-services
+ ${project.parent.version}
+ provided
+
+
+ com.google.inject
+ guice
+ provided
+
+
+ com.google.inject.extensions
+ guice-multibindings
+ provided
+
+
+ com.google.guava
+ guava
+ provided
+
+
+ com.opencsv
+ opencsv
+ provided
+
+
+ com.fasterxml.jackson.core
+ jackson-databind
+ provided
+
+
+ com.fasterxml.jackson.core
+ jackson-core
+ provided
+
+
+ com.fasterxml.jackson.core
+ jackson-annotations
+ provided
+
+
+ javax.validation
+ validation-api
+ provided
+
+
+ com.google.code.findbugs
+ jsr305
+ provided
+
+
+ jakarta.inject
+ jakarta.inject-api
+ provided
+
+
+ joda-time
+ joda-time
+ provided
+
+
+ io.netty
+ netty
+ provided
+
+
+ org.apache.calcite
+ calcite-core
+ provided
+
+
+ com.fasterxml.jackson.dataformat
+ jackson-dataformat-smile
+ provided
+
+
+ org.jdbi
+ jdbi
+ provided
+
+
+ com.fasterxml.jackson.jaxrs
+ jackson-jaxrs-smile-provider
+ provided
+
+
+ javax.ws.rs
+ jsr311-api
+ provided
+
+
+ org.apache.commons
+ commons-lang3
+ provided
+
+
+ javax.servlet
+ javax.servlet-api
+ provided
+
+
+ com.sun.jersey
+ jersey-server
+ provided
+
+
+ com.google.errorprone
+ error_prone_annotations
+ provided
+
+
+ org.lz4
+ lz4-java
+ provided
+
+
+ org.apache.datasketches
+ datasketches-java
+ provided
+
+
+ org.apache.datasketches
+ datasketches-memory
+ provided
+
+
+ it.unimi.dsi
+ fastutil-core
+ provided
+
+
+ commons-lang
+ commons-lang
+ provided
+
+
+ commons-io
+ commons-io
+ provided
+
+
+
+
+ org.easymock
+ easymock
+ test
+
+
+ org.hamcrest
+ hamcrest-all
+ test
+
+
+ org.hamcrest
+ hamcrest-core
+ test
+
+
+ junit
+ junit
+ test
+
+
+ org.mockito
+ mockito-core
+ test
+
+
+ nl.jqno.equalsverifier
+ equalsverifier
+ test
+
+
+ org.apache.druid
+ druid-core
+ ${project.parent.version}
+ test
+ test-jar
+
+
+ org.apache.druid
+ druid-processing
+ ${project.parent.version}
+ test
+ test-jar
+
+
+ org.apache.druid
+ druid-server
+ ${project.parent.version}
+ test-jar
+ test
+
+
+ org.apache.druid
+ druid-sql
+ ${project.parent.version}
+ test-jar
+ test
+
+
+ org.apache.druid
+ druid-hll
+ ${project.parent.version}
+
+ provided
+
+
+
+
+
+
+ org.jacoco
+ jacoco-maven-plugin
+
+
+
+ org/apache/druid/msq/guice/*
+ org/apache/druid/msq/indexing/IndexerControllerClient*
+ org/apache/druid/msq/indexing/IndexerControllerContext*
+ org/apache/druid/msq/indexing/IndexerWorkerClient*
+ org/apache/druid/msq/indexing/IndexerWorkerContext*
+ org/apache/druid/msq/indexing/IndexerWorkerManagerClient*
+
+
+
+
+
+
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/ChannelCounters.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/ChannelCounters.java
new file mode 100644
index 00000000000..461b066ce9d
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/ChannelCounters.java
@@ -0,0 +1,274 @@
+/*
+ * 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.counters;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.google.errorprone.annotations.concurrent.GuardedBy;
+import it.unimi.dsi.fastutil.longs.LongArrayList;
+import it.unimi.dsi.fastutil.longs.LongList;
+import org.apache.druid.frame.Frame;
+
+import javax.annotation.Nullable;
+import java.util.Arrays;
+
+/**
+ * Counters for inputs and outputs. Created by {@link CounterTracker#channel}.
+ */
+public class ChannelCounters implements QueryCounter
+{
+ private static final int NO_PARTITION = 0;
+
+ @GuardedBy("this")
+ private final LongList rows = new LongArrayList();
+
+ @GuardedBy("this")
+ private final LongList bytes = new LongArrayList();
+
+ @GuardedBy("this")
+ private final LongList frames = new LongArrayList();
+
+ @GuardedBy("this")
+ private final LongList files = new LongArrayList();
+
+ @GuardedBy("this")
+ private final LongList totalFiles = new LongArrayList();
+
+ public void incrementRowCount()
+ {
+ add(NO_PARTITION, 1, 0, 0, 0);
+ }
+
+ public void incrementFileCount()
+ {
+ add(NO_PARTITION, 0, 0, 0, 1);
+ }
+
+ public void addFile(final long nRows, final long nBytes)
+ {
+ add(NO_PARTITION, nRows, nBytes, 0, 1);
+ }
+
+ public void addFrame(final int partitionNumber, final Frame frame)
+ {
+ add(partitionNumber, frame.numRows(), frame.numBytes(), 1, 0);
+ }
+
+ public ChannelCounters setTotalFiles(final long nFiles)
+ {
+ synchronized (this) {
+ ensureCapacityForPartition(NO_PARTITION);
+ totalFiles.set(NO_PARTITION, nFiles);
+ return this;
+ }
+ }
+
+ private void add(
+ final int partitionNumber,
+ final long nRows,
+ final long nBytes,
+ final long nFrames,
+ final long nFiles
+ )
+ {
+ synchronized (this) {
+ ensureCapacityForPartition(partitionNumber);
+ rows.set(partitionNumber, rows.getLong(partitionNumber) + nRows);
+ bytes.set(partitionNumber, bytes.getLong(partitionNumber) + nBytes);
+ frames.set(partitionNumber, frames.getLong(partitionNumber) + nFrames);
+ files.set(partitionNumber, files.getLong(partitionNumber) + nFiles);
+ }
+ }
+
+ @GuardedBy("this")
+ private void ensureCapacityForPartition(final int partitionNumber)
+ {
+ while (partitionNumber >= rows.size()) {
+ rows.add(0);
+ }
+
+ while (partitionNumber >= bytes.size()) {
+ bytes.add(0);
+ }
+
+ while (partitionNumber >= frames.size()) {
+ frames.add(0);
+ }
+
+ while (partitionNumber >= files.size()) {
+ files.add(0);
+ }
+
+ while (partitionNumber >= totalFiles.size()) {
+ totalFiles.add(0);
+ }
+ }
+
+ @Override
+ @Nullable
+ public Snapshot snapshot()
+ {
+ final long[] rowsArray;
+ final long[] bytesArray;
+ final long[] framesArray;
+ final long[] filesArray;
+ final long[] totalFilesArray;
+
+ synchronized (this) {
+ rowsArray = listToArray(rows);
+ bytesArray = listToArray(bytes);
+ framesArray = listToArray(frames);
+ filesArray = listToArray(files);
+ totalFilesArray = listToArray(totalFiles);
+ }
+
+ if (rowsArray == null
+ && bytesArray == null
+ && framesArray == null
+ && filesArray == null
+ && totalFilesArray == null) {
+ return null;
+ } else {
+ return new Snapshot(rowsArray, bytesArray, framesArray, filesArray, totalFilesArray);
+ }
+ }
+
+ @Nullable
+ private static long[] listToArray(final LongList longList)
+ {
+ boolean allZero = true;
+
+ for (int i = 0; i < longList.size(); i++) {
+ if (longList.getLong(i) != 0) {
+ allZero = false;
+ break;
+ }
+ }
+
+ if (allZero) {
+ return null;
+ } else {
+ return longList.toArray(new long[0]);
+ }
+ }
+
+ @JsonTypeName("channel")
+ public static class Snapshot implements QueryCounterSnapshot
+ {
+ private final long[] rows;
+ private final long[] bytes;
+ private final long[] frames;
+ private final long[] files;
+ private final long[] totalFiles;
+
+ @JsonCreator
+ public Snapshot(
+ @Nullable @JsonProperty("rows") final long[] rows,
+ @Nullable @JsonProperty("bytes") final long[] bytes,
+ @Nullable @JsonProperty("frames") final long[] frames,
+ @Nullable @JsonProperty("files") final long[] files,
+ @Nullable @JsonProperty("totalFiles") final long[] totalFiles
+ )
+ {
+ this.rows = rows;
+ this.bytes = bytes;
+ this.frames = frames;
+ this.files = files;
+ this.totalFiles = totalFiles;
+ }
+
+ @JsonProperty
+ @JsonInclude(JsonInclude.Include.NON_NULL)
+ public long[] getRows()
+ {
+ return rows;
+ }
+
+ @JsonProperty
+ @JsonInclude(JsonInclude.Include.NON_NULL)
+ public long[] getBytes()
+ {
+ return bytes;
+ }
+
+ @JsonProperty
+ @JsonInclude(JsonInclude.Include.NON_NULL)
+ public long[] getFrames()
+ {
+ return frames;
+ }
+
+ @JsonProperty
+ @JsonInclude(JsonInclude.Include.NON_NULL)
+ public long[] getFiles()
+ {
+ return files;
+ }
+
+ @JsonProperty
+ @JsonInclude(JsonInclude.Include.NON_NULL)
+ public long[] getTotalFiles()
+ {
+ return totalFiles;
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ Snapshot snapshot = (Snapshot) o;
+ return Arrays.equals(rows, snapshot.rows)
+ && Arrays.equals(bytes, snapshot.bytes)
+ && Arrays.equals(frames, snapshot.frames)
+ && Arrays.equals(files, snapshot.files)
+ && Arrays.equals(totalFiles, snapshot.totalFiles);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ int result = Arrays.hashCode(rows);
+ result = 31 * result + Arrays.hashCode(bytes);
+ result = 31 * result + Arrays.hashCode(frames);
+ result = 31 * result + Arrays.hashCode(files);
+ result = 31 * result + Arrays.hashCode(totalFiles);
+ return result;
+ }
+
+ @Override
+ public String toString()
+ {
+ return "ChannelCounters.Snapshot{" +
+ "rows=" + Arrays.toString(rows) +
+ ", bytes=" + Arrays.toString(bytes) +
+ ", frames=" + Arrays.toString(frames) +
+ ", files=" + Arrays.toString(files) +
+ ", totalFiles=" + Arrays.toString(totalFiles) +
+ '}';
+ }
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterNames.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterNames.java
new file mode 100644
index 00000000000..3c1d58ec59b
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterNames.java
@@ -0,0 +1,142 @@
+/*
+ * 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.counters;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.ordering.StringComparators;
+
+import java.util.Comparator;
+import java.util.Map;
+
+/**
+ * Standard names for counters.
+ */
+public class CounterNames
+{
+ private static final String INPUT = "input";
+ private static final String OUTPUT = "output";
+ private static final String SHUFFLE = "shuffle";
+ private static final String SORT_PROGRESS = "sortProgress";
+ private static final String WARNINGS = "warnings";
+ private static final Comparator COMPARATOR = new NameComparator();
+
+ private CounterNames()
+ {
+ // No construction: statics only.
+ }
+
+ /**
+ * Standard name for an input channel counter created by {@link CounterTracker#channel}.
+ */
+ public static String inputChannel(final int inputNumber)
+ {
+ return StringUtils.format("%s%d", INPUT, inputNumber);
+ }
+
+ /**
+ * Standard name for an output channel counter created by {@link CounterTracker#channel}.
+ */
+ public static String outputChannel()
+ {
+ return OUTPUT;
+ }
+
+ /**
+ * Standard name for a shuffle channel counter created by {@link CounterTracker#channel}.
+ */
+ public static String shuffleChannel()
+ {
+ return SHUFFLE;
+ }
+
+ /**
+ * Standard name for a sort progress counter created by {@link CounterTracker#sortProgress()}.
+ */
+ public static String sortProgress()
+ {
+ return SORT_PROGRESS;
+ }
+
+ /**
+ * Standard name for a warnings counter created by {@link CounterTracker#warnings()}.
+ */
+ public static String warnings()
+ {
+ return WARNINGS;
+ }
+
+ /**
+ * Standard comparator for counter names. Not necessary for functionality, but helps with human-readability.
+ */
+ public static Comparator comparator()
+ {
+ return COMPARATOR;
+ }
+
+ /**
+ * Comparator that ensures counters are sorted in a nice order when serialized to JSON. Not necessary for
+ * functionality, but helps with human-readability.
+ */
+ private static class NameComparator implements Comparator
+ {
+ private static final Map ORDER =
+ ImmutableMap.builder()
+ .put(OUTPUT, 0)
+ .put(SHUFFLE, 1)
+ .put(SORT_PROGRESS, 2)
+ .put(WARNINGS, 3)
+ .build();
+
+ @Override
+ public int compare(final String name1, final String name2)
+ {
+ final boolean isInput1 = name1.startsWith(INPUT);
+ final boolean isInput2 = name2.startsWith(INPUT);
+
+ if (isInput1 && isInput2) {
+ // Compare INPUT alphanumerically, so e.g. "input2" is before "input10"
+ return StringComparators.ALPHANUMERIC.compare(name1, name2);
+ } else if (isInput1 != isInput2) {
+ // INPUT goes first
+ return isInput1 ? -1 : 1;
+ }
+
+ assert !isInput1 && !isInput2;
+
+ final Integer order1 = ORDER.get(name1);
+ final Integer order2 = ORDER.get(name2);
+
+ if (order1 != null && order2 != null) {
+ // Respect ordering from ORDER
+ return Integer.compare(order1, order2);
+ } else if (order1 != null) {
+ // Names from ORDER go before names that are not in ORDER
+ return -1;
+ } else if (order2 != null) {
+ // Names from ORDER go before names that are not in ORDER
+ return 1;
+ } else {
+ assert order1 == null && order2 == null;
+ return name1.compareTo(name2);
+ }
+ }
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterSnapshots.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterSnapshots.java
new file mode 100644
index 00000000000..d75eb1ce8cb
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterSnapshots.java
@@ -0,0 +1,75 @@
+/*
+ * 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.counters;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.google.common.collect.ImmutableSortedMap;
+
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Named counter snapshots. Immutable. Often part of a {@link CounterSnapshotsTree}.
+ */
+public class CounterSnapshots
+{
+ private final Map snapshotMap;
+
+ @JsonCreator
+ public CounterSnapshots(final Map snapshotMap)
+ {
+ this.snapshotMap = ImmutableSortedMap.copyOf(snapshotMap, CounterNames.comparator());
+ }
+
+ public Map getMap()
+ {
+ return snapshotMap;
+ }
+
+ public boolean isEmpty()
+ {
+ return snapshotMap.isEmpty();
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ CounterSnapshots that = (CounterSnapshots) o;
+ return Objects.equals(snapshotMap, that.snapshotMap);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(snapshotMap);
+ }
+
+ @Override
+ public String toString()
+ {
+ return snapshotMap.toString();
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterSnapshotsSerializer.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterSnapshotsSerializer.java
new file mode 100644
index 00000000000..7cbb6bc6c42
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterSnapshotsSerializer.java
@@ -0,0 +1,55 @@
+/*
+ * 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.counters;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.SerializerProvider;
+import com.fasterxml.jackson.databind.ser.std.StdSerializer;
+
+import java.io.IOException;
+import java.util.Map;
+
+/**
+ * Serializer for {@link CounterSnapshots}. Necessary because otherwise the map structure causes Jackson
+ * to miss including type codes.
+ */
+public class CounterSnapshotsSerializer extends StdSerializer
+{
+ public CounterSnapshotsSerializer()
+ {
+ super(CounterSnapshots.class);
+ }
+
+ @Override
+ public void serialize(
+ final CounterSnapshots value,
+ final JsonGenerator jg,
+ final SerializerProvider serializers
+ ) throws IOException
+ {
+ jg.writeStartObject();
+
+ for (final Map.Entry entry : value.getMap().entrySet()) {
+ jg.writeObjectField(entry.getKey(), entry.getValue());
+ }
+
+ jg.writeEndObject();
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterSnapshotsTree.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterSnapshotsTree.java
new file mode 100644
index 00000000000..953f8b718de
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterSnapshotsTree.java
@@ -0,0 +1,104 @@
+/*
+ * 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.counters;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonValue;
+import com.google.errorprone.annotations.concurrent.GuardedBy;
+import it.unimi.dsi.fastutil.ints.Int2ObjectAVLTreeMap;
+import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
+import org.apache.druid.msq.exec.ControllerClient;
+import org.apache.druid.msq.indexing.report.MSQTaskReportPayload;
+
+import java.util.Map;
+
+/**
+ * Tree of {@link CounterSnapshots} (named counter snapshots) organized by stage and worker.
+ *
+ * These are used for worker-to-controller counters propagation with
+ * {@link ControllerClient#postCounters} and reporting to end users with
+ * {@link MSQTaskReportPayload#getCounters}).
+ *
+ * The map is mutable, but thread-safe. The individual snapshot objects are immutable.
+ */
+public class CounterSnapshotsTree
+{
+ // stage -> worker -> counters
+ @GuardedBy("snapshotsMap")
+ private final Int2ObjectMap> snapshotsMap;
+
+ public CounterSnapshotsTree()
+ {
+ this.snapshotsMap = new Int2ObjectAVLTreeMap<>();
+ }
+
+ @JsonCreator
+ public static CounterSnapshotsTree fromMap(final Map> map)
+ {
+ final CounterSnapshotsTree retVal = new CounterSnapshotsTree();
+ retVal.putAll(map);
+ return retVal;
+ }
+
+ public void put(final int stageNumber, final int workerNumber, final CounterSnapshots snapshots)
+ {
+ synchronized (snapshotsMap) {
+ snapshotsMap.computeIfAbsent(stageNumber, ignored -> new Int2ObjectAVLTreeMap<>())
+ .put(workerNumber, snapshots);
+ }
+ }
+
+ public void putAll(final CounterSnapshotsTree other)
+ {
+ putAll(other.copyMap());
+ }
+
+ public boolean isEmpty()
+ {
+ synchronized (snapshotsMap) {
+ return snapshotsMap.isEmpty();
+ }
+ }
+
+ @JsonValue
+ public Map> copyMap()
+ {
+ final Map> retVal = new Int2ObjectAVLTreeMap<>();
+
+ synchronized (snapshotsMap) {
+ for (Int2ObjectMap.Entry> entry : snapshotsMap.int2ObjectEntrySet()) {
+ retVal.put(entry.getIntKey(), new Int2ObjectAVLTreeMap<>(entry.getValue()));
+ }
+ }
+
+ return retVal;
+ }
+
+ private void putAll(final Map> otherMap)
+ {
+ synchronized (snapshotsMap) {
+ for (Map.Entry> stageEntry : otherMap.entrySet()) {
+ for (Map.Entry workerEntry : stageEntry.getValue().entrySet()) {
+ put(stageEntry.getKey(), workerEntry.getKey(), workerEntry.getValue());
+ }
+ }
+ }
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterTracker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterTracker.java
new file mode 100644
index 00000000000..36af372a56c
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterTracker.java
@@ -0,0 +1,74 @@
+/*
+ * 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.counters;
+
+import org.apache.druid.frame.processor.SuperSorterProgressTracker;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Supplier;
+
+/**
+ * Class that tracks query counters for a particular worker in a particular stage.
+ *
+ * Counters are all tracked on a (stage, worker, counter) basis by the {@link #countersMap} object.
+ *
+ * Immutable {@link CounterSnapshots} snapshots can be created by {@link #snapshot()}.
+ */
+public class CounterTracker
+{
+ private final ConcurrentHashMap countersMap = new ConcurrentHashMap<>();
+
+ public ChannelCounters channel(final String name)
+ {
+ return counter(name, ChannelCounters::new);
+ }
+
+ public SuperSorterProgressTracker sortProgress()
+ {
+ return counter(CounterNames.sortProgress(), SuperSorterProgressTrackerCounter::new).tracker();
+ }
+
+ public WarningCounters warnings()
+ {
+ return counter(CounterNames.warnings(), WarningCounters::new);
+ }
+
+ @SuppressWarnings("unchecked")
+ public T counter(final String counterName, final Supplier newCounterFn)
+ {
+ return (T) countersMap.computeIfAbsent(counterName, ignored -> newCounterFn.get());
+ }
+
+ public CounterSnapshots snapshot()
+ {
+ final Map m = new HashMap<>();
+
+ for (final Map.Entry entry : countersMap.entrySet()) {
+ final QueryCounterSnapshot counterSnapshot = entry.getValue().snapshot();
+ if (counterSnapshot != null) {
+ m.put(entry.getKey(), counterSnapshot);
+ }
+ }
+
+ return new CounterSnapshots(m);
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/QueryCounter.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/QueryCounter.java
new file mode 100644
index 00000000000..c23022c30f4
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/QueryCounter.java
@@ -0,0 +1,32 @@
+/*
+ * 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.counters;
+
+import javax.annotation.Nullable;
+
+/**
+ * A kind of query counter. Counters are flexible in what they can track and how they work, so this interface
+ * does not specify anything beyond requiring the ability to take a snapshot.
+ */
+public interface QueryCounter
+{
+ @Nullable
+ QueryCounterSnapshot snapshot();
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/QueryCounterSnapshot.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/QueryCounterSnapshot.java
new file mode 100644
index 00000000000..c065f7f8252
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/QueryCounterSnapshot.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.msq.counters;
+
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
+
+/**
+ * Marker interface for the results of {@link QueryCounter#snapshot()}. No methods, because the only purpose of these
+ * snapshots is to pass things along from worker -> controller -> report.
+ */
+@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
+public interface QueryCounterSnapshot
+{
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/SuperSorterProgressTrackerCounter.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/SuperSorterProgressTrackerCounter.java
new file mode 100644
index 00000000000..b10a2b4ddbc
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/SuperSorterProgressTrackerCounter.java
@@ -0,0 +1,71 @@
+/*
+ * 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.counters;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.fasterxml.jackson.annotation.JsonValue;
+import org.apache.druid.frame.processor.SuperSorterProgressSnapshot;
+import org.apache.druid.frame.processor.SuperSorterProgressTracker;
+
+import javax.annotation.Nullable;
+
+public class SuperSorterProgressTrackerCounter implements QueryCounter
+{
+ private final SuperSorterProgressTracker tracker;
+
+ public SuperSorterProgressTrackerCounter()
+ {
+ this.tracker = new SuperSorterProgressTracker();
+ }
+
+ public SuperSorterProgressTracker tracker()
+ {
+ return tracker;
+ }
+
+ @Nullable
+ @Override
+ public QueryCounterSnapshot snapshot()
+ {
+ return new Snapshot(tracker.snapshot());
+ }
+
+ /**
+ * Wrapper class that exists for JSON serde.
+ */
+ @JsonTypeName("sortProgress")
+ public static class Snapshot implements QueryCounterSnapshot
+ {
+ private final SuperSorterProgressSnapshot snapshot;
+
+ @JsonCreator
+ public Snapshot(SuperSorterProgressSnapshot snapshot)
+ {
+ this.snapshot = snapshot;
+ }
+
+ @JsonValue
+ public SuperSorterProgressSnapshot getSnapshot()
+ {
+ return snapshot;
+ }
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/WarningCounters.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/WarningCounters.java
new file mode 100644
index 00000000000..7c56dfe02c6
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/WarningCounters.java
@@ -0,0 +1,94 @@
+/*
+ * 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.counters;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.fasterxml.jackson.annotation.JsonValue;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+
+import javax.annotation.Nullable;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Counters for warnings. Created by {@link CounterTracker#warnings()}.
+ */
+@JsonTypeName("warnings")
+public class WarningCounters implements QueryCounter
+{
+ private final ConcurrentHashMap warningCodeCounter = new ConcurrentHashMap<>();
+
+ public void incrementWarningCount(String errorCode)
+ {
+ warningCodeCounter.compute(errorCode, (ignored, oldCount) -> oldCount == null ? 1 : oldCount + 1);
+ }
+
+ @Override
+ @Nullable
+ public Snapshot snapshot()
+ {
+ if (warningCodeCounter.isEmpty()) {
+ return null;
+ }
+
+ final Map countCopy = ImmutableMap.copyOf(warningCodeCounter);
+ return new Snapshot(countCopy);
+ }
+
+ @JsonTypeName("warnings")
+ public static class Snapshot implements QueryCounterSnapshot
+ {
+ private final Map warningCountMap;
+
+ @JsonCreator
+ public Snapshot(Map warningCountMap)
+ {
+ this.warningCountMap = Preconditions.checkNotNull(warningCountMap, "warningCountMap");
+ }
+
+ @JsonValue
+ public Map getWarningCountMap()
+ {
+ return warningCountMap;
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ Snapshot snapshot = (Snapshot) o;
+ return Objects.equals(warningCountMap, snapshot.warningCountMap);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(warningCountMap);
+ }
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Controller.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Controller.java
new file mode 100644
index 00000000000..07730de45e7
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Controller.java
@@ -0,0 +1,125 @@
+/*
+ * 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.exec;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.druid.indexer.TaskStatus;
+import org.apache.druid.indexing.common.TaskReport;
+import org.apache.druid.msq.counters.CounterSnapshots;
+import org.apache.druid.msq.counters.CounterSnapshotsTree;
+import org.apache.druid.msq.indexing.MSQControllerTask;
+import org.apache.druid.msq.indexing.error.MSQErrorReport;
+import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
+
+import javax.annotation.Nullable;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Interface for the controller of a multi-stage query.
+ */
+public interface Controller
+{
+ /**
+ * POJO for capturing the status of a controller task that is currently running.
+ */
+ class RunningControllerStatus
+ {
+ private final String id;
+
+ @JsonCreator
+ public RunningControllerStatus(String id)
+ {
+ this.id = id;
+ }
+
+ @JsonProperty("id")
+ public String getId()
+ {
+ return id;
+ }
+ }
+
+ /**
+ * Unique task/query ID for the batch query run by this controller.
+ */
+ String id();
+
+ /**
+ * The task which this controller runs.
+ */
+ MSQControllerTask task();
+
+ /**
+ * Runs the controller logic in the current thread. Surrounding classes provide the execution thread.
+ */
+ TaskStatus run() throws Exception;
+
+ /**
+ * Terminate the query DAG upon a cancellation request.
+ */
+ void stopGracefully();
+
+ // Worker-to-controller messages
+
+ /**
+ * Provide a {@link ClusterByStatisticsSnapshot} for shuffling stages.
+ */
+ void updateStatus(int stageNumber, int workerNumber, Object keyStatisticsObject);
+
+ /**
+ * System error reported by a subtask. Note that the errors are organized by
+ * taskId, not by query/stage/worker, because system errors are associated
+ * with a task rather than a specific query/stage/worker execution context.
+ */
+ void workerError(MSQErrorReport errorReport);
+
+ /**
+ * System warning reported by a subtask. Indicates that the worker has encountered a non-lethal error. Worker should
+ * continue its execution in such a case. If the worker wants to report an error and stop its execution,
+ * please use {@link Controller#workerError}
+ */
+ void workerWarning(List errorReports);
+
+ /**
+ * Periodic update of {@link CounterSnapshots} from subtasks.
+ */
+ void updateCounters(CounterSnapshotsTree snapshotsTree);
+
+ /**
+ * Reports that results are ready for a subtask.
+ */
+ void resultsComplete(
+ String queryId,
+ int stageNumber,
+ int workerNumber,
+ Object resultObject
+ );
+
+ /**
+ * Returns the current list of task ids, ordered by worker number. The Nth task has worker number N.
+ */
+ List getTaskIds();
+
+ @Nullable
+ Map liveReports();
+
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerClient.java
new file mode 100644
index 00000000000..f621133586c
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerClient.java
@@ -0,0 +1,81 @@
+/*
+ * 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.exec;
+
+import org.apache.druid.msq.counters.CounterSnapshotsTree;
+import org.apache.druid.msq.indexing.error.MSQErrorReport;
+import org.apache.druid.msq.kernel.StageId;
+import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * Client for the multi-stage query controller. Used by a Worker task.
+ */
+public interface ControllerClient extends AutoCloseable
+{
+ /**
+ * Client side method to update the controller with key statistics for a particular stage and worker.
+ * Controller's implementation collates all the key statistics for a stage to generate the partition boundaries.
+ */
+ void postKeyStatistics(
+ StageId stageId,
+ int workerNumber,
+ ClusterByStatisticsSnapshot keyStatistics
+ ) throws IOException;
+
+ /**
+ * Client-side method to update the controller with counters for a particular stage and worker. The controller uses
+ * this to compile live reports, track warnings generated etc.
+ */
+ void postCounters(CounterSnapshotsTree snapshotsTree) throws IOException;
+
+ /**
+ * Client side method to update the controller with the result object for a particular stage and worker. This also
+ * informs the controller that the computation for that particular stage has been done by the worker.
+ */
+ void postResultsComplete(
+ StageId stageId,
+ int workerNumber,
+ @Nullable Object resultObject
+ ) throws IOException;
+
+ /**
+ * Client side method to inform the controller that the error has occured in the given worker.
+ */
+ void postWorkerError(
+ String workerId,
+ MSQErrorReport errorWrapper
+ ) throws IOException;
+
+ /**
+ * Client side method to inform the controller about the warnings generated by the given worker.
+ */
+ void postWorkerWarning(
+ String workerId,
+ List MSQErrorReports
+ ) throws IOException;
+ List getTaskList() throws IOException;
+
+ @Override
+ void close();
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerContext.java
new file mode 100644
index 00000000000..f77290c0921
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerContext.java
@@ -0,0 +1,82 @@
+/*
+ * 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.exec;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.inject.Injector;
+import org.apache.druid.client.coordinator.CoordinatorClient;
+import org.apache.druid.indexing.common.TaskReport;
+import org.apache.druid.indexing.common.actions.TaskActionClient;
+import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.server.DruidNode;
+
+import java.util.Map;
+
+/**
+ * Context used by multi-stage query controllers.
+ *
+ * Useful because it allows test fixtures to provide their own implementations.
+ */
+public interface ControllerContext
+{
+ ObjectMapper jsonMapper();
+
+ /**
+ * Provides a way for tasks to request injectable objects. Useful because tasks are not able to request injection
+ * at the time of server startup, because the server doesn't know what tasks it will be running.
+ */
+ Injector injector();
+
+ /**
+ * Fetch node info about self.
+ */
+ DruidNode selfNode();
+
+ /**
+ * Provide access to the Coordinator service.
+ */
+ CoordinatorClient coordinatorClient();
+
+ /**
+ * Provide access to segment actions in the Overlord.
+ */
+ TaskActionClient taskActionClient();
+
+ /**
+ * Provides services about workers: starting, canceling, obtaining status.
+ */
+ WorkerManagerClient workerManager();
+
+ /**
+ * Callback from the controller implementation to "register" the controller. Used in the indexing task implementation
+ * to set up the task chat web service.
+ */
+ void registerController(Controller controller, Closer closer);
+
+ /**
+ * Client for communicating with workers.
+ */
+ WorkerClient taskClientFor(Controller controller);
+
+ /**
+ * Writes controller task report.
+ */
+ void writeReports(String controllerTaskId, Map reports);
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java
new file mode 100644
index 00000000000..3aa346c3846
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java
@@ -0,0 +1,2171 @@
+/*
+ * 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.exec;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Iterators;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import it.unimi.dsi.fastutil.ints.Int2ObjectAVLTreeMap;
+import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
+import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
+import it.unimi.dsi.fastutil.ints.IntSet;
+import org.apache.druid.common.guava.FutureUtils;
+import org.apache.druid.data.input.StringTuple;
+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.frame.allocation.ArenaMemoryAllocator;
+import org.apache.druid.frame.channel.FrameChannelSequence;
+import org.apache.druid.frame.key.ClusterBy;
+import org.apache.druid.frame.key.ClusterByPartition;
+import org.apache.druid.frame.key.ClusterByPartitions;
+import org.apache.druid.frame.key.RowKey;
+import org.apache.druid.frame.key.RowKeyReader;
+import org.apache.druid.frame.key.SortColumn;
+import org.apache.druid.frame.processor.FrameProcessorExecutor;
+import org.apache.druid.frame.processor.FrameProcessors;
+import org.apache.druid.indexer.TaskState;
+import org.apache.druid.indexer.TaskStatus;
+import org.apache.druid.indexing.common.LockGranularity;
+import org.apache.druid.indexing.common.TaskLock;
+import org.apache.druid.indexing.common.TaskLockType;
+import org.apache.druid.indexing.common.TaskReport;
+import org.apache.druid.indexing.common.actions.LockListAction;
+import org.apache.druid.indexing.common.actions.MarkSegmentsAsUnusedAction;
+import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction;
+import org.apache.druid.indexing.common.actions.SegmentAllocateAction;
+import org.apache.druid.indexing.common.actions.SegmentInsertAction;
+import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction;
+import org.apache.druid.indexing.overlord.Segments;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.JodaUtils;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.concurrent.Execs;
+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.guava.Sequences;
+import org.apache.druid.java.util.common.guava.Yielder;
+import org.apache.druid.java.util.common.guava.Yielders;
+import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.msq.counters.CounterSnapshots;
+import org.apache.druid.msq.counters.CounterSnapshotsTree;
+import org.apache.druid.msq.indexing.ColumnMapping;
+import org.apache.druid.msq.indexing.ColumnMappings;
+import org.apache.druid.msq.indexing.DataSourceMSQDestination;
+import org.apache.druid.msq.indexing.InputChannelFactory;
+import org.apache.druid.msq.indexing.InputChannelsImpl;
+import org.apache.druid.msq.indexing.MSQControllerTask;
+import org.apache.druid.msq.indexing.MSQSpec;
+import org.apache.druid.msq.indexing.MSQTuningConfig;
+import org.apache.druid.msq.indexing.MSQWorkerTaskLauncher;
+import org.apache.druid.msq.indexing.SegmentGeneratorFrameProcessorFactory;
+import org.apache.druid.msq.indexing.TaskReportMSQDestination;
+import org.apache.druid.msq.indexing.error.CanceledFault;
+import org.apache.druid.msq.indexing.error.CannotParseExternalDataFault;
+import org.apache.druid.msq.indexing.error.FaultsExceededChecker;
+import org.apache.druid.msq.indexing.error.InsertCannotAllocateSegmentFault;
+import org.apache.druid.msq.indexing.error.InsertCannotBeEmptyFault;
+import org.apache.druid.msq.indexing.error.InsertCannotOrderByDescendingFault;
+import org.apache.druid.msq.indexing.error.InsertCannotReplaceExistingSegmentFault;
+import org.apache.druid.msq.indexing.error.InsertLockPreemptedFault;
+import org.apache.druid.msq.indexing.error.InsertTimeOutOfBoundsFault;
+import org.apache.druid.msq.indexing.error.MSQErrorReport;
+import org.apache.druid.msq.indexing.error.MSQException;
+import org.apache.druid.msq.indexing.error.MSQFault;
+import org.apache.druid.msq.indexing.error.MSQWarningReportLimiterPublisher;
+import org.apache.druid.msq.indexing.error.MSQWarnings;
+import org.apache.druid.msq.indexing.error.QueryNotSupportedFault;
+import org.apache.druid.msq.indexing.error.TooManyWarningsFault;
+import org.apache.druid.msq.indexing.error.UnknownFault;
+import org.apache.druid.msq.indexing.report.MSQResultsReport;
+import org.apache.druid.msq.indexing.report.MSQStagesReport;
+import org.apache.druid.msq.indexing.report.MSQStatusReport;
+import org.apache.druid.msq.indexing.report.MSQTaskReport;
+import org.apache.druid.msq.indexing.report.MSQTaskReportPayload;
+import org.apache.druid.msq.input.InputSpec;
+import org.apache.druid.msq.input.InputSpecSlicer;
+import org.apache.druid.msq.input.InputSpecSlicerFactory;
+import org.apache.druid.msq.input.InputSpecs;
+import org.apache.druid.msq.input.MapInputSpecSlicer;
+import org.apache.druid.msq.input.external.ExternalInputSpec;
+import org.apache.druid.msq.input.external.ExternalInputSpecSlicer;
+import org.apache.druid.msq.input.stage.InputChannels;
+import org.apache.druid.msq.input.stage.ReadablePartition;
+import org.apache.druid.msq.input.stage.StageInputSlice;
+import org.apache.druid.msq.input.stage.StageInputSpec;
+import org.apache.druid.msq.input.stage.StageInputSpecSlicer;
+import org.apache.druid.msq.input.table.TableInputSpec;
+import org.apache.druid.msq.input.table.TableInputSpecSlicer;
+import org.apache.druid.msq.kernel.QueryDefinition;
+import org.apache.druid.msq.kernel.QueryDefinitionBuilder;
+import org.apache.druid.msq.kernel.StageDefinition;
+import org.apache.druid.msq.kernel.StageId;
+import org.apache.druid.msq.kernel.StagePartition;
+import org.apache.druid.msq.kernel.TargetSizeShuffleSpec;
+import org.apache.druid.msq.kernel.WorkOrder;
+import org.apache.druid.msq.kernel.controller.ControllerQueryKernel;
+import org.apache.druid.msq.kernel.controller.ControllerStagePhase;
+import org.apache.druid.msq.kernel.controller.WorkerInputs;
+import org.apache.druid.msq.querykit.DataSegmentTimelineView;
+import org.apache.druid.msq.querykit.MultiQueryKit;
+import org.apache.druid.msq.querykit.QueryKit;
+import org.apache.druid.msq.querykit.QueryKitUtils;
+import org.apache.druid.msq.querykit.ShuffleSpecFactories;
+import org.apache.druid.msq.querykit.ShuffleSpecFactory;
+import org.apache.druid.msq.querykit.groupby.GroupByQueryKit;
+import org.apache.druid.msq.querykit.scan.ScanQueryKit;
+import org.apache.druid.msq.shuffle.DurableStorageInputChannelFactory;
+import org.apache.druid.msq.shuffle.DurableStorageOutputChannelFactory;
+import org.apache.druid.msq.shuffle.WorkerInputChannelFactory;
+import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
+import org.apache.druid.msq.util.DimensionSchemaUtils;
+import org.apache.druid.msq.util.IntervalUtils;
+import org.apache.druid.msq.util.MSQFutureUtils;
+import org.apache.druid.msq.util.MultiStageQueryContext;
+import org.apache.druid.msq.util.PassthroughAggregatorFactory;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.GroupByQueryConfig;
+import org.apache.druid.query.scan.ScanQuery;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.Cursor;
+import org.apache.druid.segment.DimensionHandlerUtils;
+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.column.ValueType;
+import org.apache.druid.segment.indexing.DataSchema;
+import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec;
+import org.apache.druid.segment.indexing.granularity.GranularitySpec;
+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.rel.DruidQuery;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.VersionedIntervalTimeline;
+import org.apache.druid.timeline.partition.DimensionRangeShardSpec;
+import org.apache.druid.timeline.partition.NumberedPartialShardSpec;
+import org.apache.druid.timeline.partition.NumberedShardSpec;
+import org.apache.druid.timeline.partition.ShardSpec;
+import org.joda.time.DateTime;
+import org.joda.time.Interval;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import java.util.stream.StreamSupport;
+
+public class ControllerImpl implements Controller
+{
+ private static final Logger log = new Logger(ControllerImpl.class);
+
+ private final MSQControllerTask task;
+ private final ControllerContext context;
+
+ /**
+ * Queue of "commands" to run on the {@link ControllerQueryKernel}. Various threads insert into the queue
+ * using {@link #addToKernelManipulationQueue}. The main thread running {@link RunQueryUntilDone#run()} reads
+ * from the queue and executes the commands.
+ *
+ * This ensures that all manipulations on {@link ControllerQueryKernel}, and all core logic, are run in
+ * a single-threaded manner.
+ */
+ private final BlockingQueue> kernelManipulationQueue =
+ new ArrayBlockingQueue<>(Limits.MAX_KERNEL_MANIPULATION_QUEUE_SIZE);
+
+ // For system error reporting. This is the very first error we got from a worker. (We only report that one.)
+ private final AtomicReference workerErrorRef = new AtomicReference<>();
+
+ // For system warning reporting
+ private final ConcurrentLinkedQueue workerWarnings = new ConcurrentLinkedQueue<>();
+
+ // Query definition.
+ // For live reports. Written by the main controller thread, read by HTTP threads.
+ private final AtomicReference queryDefRef = new AtomicReference<>();
+
+ // Last reported CounterSnapshots per stage per worker
+ // For live reports. Written by the main controller thread, read by HTTP threads.
+ private final CounterSnapshotsTree taskCountersForLiveReports = new CounterSnapshotsTree();
+
+ // Stage number -> stage phase
+ // For live reports. Written by the main controller thread, read by HTTP threads.
+ private final ConcurrentHashMap stagePhasesForLiveReports = new ConcurrentHashMap<>();
+
+ // Stage number -> runtime interval. Endpoint is eternity's end if the stage is still running.
+ // For live reports. Written by the main controller thread, read by HTTP threads.
+ private final ConcurrentHashMap stageRuntimesForLiveReports = new ConcurrentHashMap<>();
+
+ // Stage number -> worker count. Only set for stages that have started.
+ // For live reports. Written by the main controller thread, read by HTTP threads.
+ private final ConcurrentHashMap stageWorkerCountsForLiveReports = new ConcurrentHashMap<>();
+
+ // Stage number -> partition count. Only set for stages that have started.
+ // For live reports. Written by the main controller thread, read by HTTP threads.
+ private final ConcurrentHashMap stagePartitionCountsForLiveReports = new ConcurrentHashMap<>();
+
+ // Time at which the query started.
+ // For live reports. Written by the main controller thread, read by HTTP threads.
+ private volatile DateTime queryStartTime = null;
+
+ private volatile DruidNode selfDruidNode;
+ private volatile MSQWorkerTaskLauncher workerTaskLauncher;
+ private volatile WorkerClient netClient;
+
+ private volatile FaultsExceededChecker faultsExceededChecker = null;
+
+ public ControllerImpl(
+ final MSQControllerTask task,
+ final ControllerContext context
+ )
+ {
+ this.task = task;
+ this.context = context;
+ }
+
+ @Override
+ public String id()
+ {
+ return task.getId();
+ }
+
+ @Override
+ public MSQControllerTask task()
+ {
+ return task;
+ }
+
+ @Override
+ public TaskStatus run() throws Exception
+ {
+ final Closer closer = Closer.create();
+
+ try {
+ return runTask(closer);
+ }
+ catch (Throwable e) {
+ try {
+ closer.close();
+ }
+ catch (Throwable e2) {
+ e.addSuppressed(e2);
+ }
+
+ // We really don't expect this to error out. runTask should handle everything nicely. If it doesn't, something
+ // strange happened, so log it.
+ log.warn(e, "Encountered unhandled controller exception.");
+ return TaskStatus.failure(id(), e.toString());
+ }
+ finally {
+ closer.close();
+ }
+ }
+
+ @Override
+ public void stopGracefully()
+ {
+ final QueryDefinition queryDef = queryDefRef.get();
+
+ // stopGracefully() is called when the containing process is terminated, or when the task is canceled.
+ log.info("Query [%s] canceled.", queryDef != null ? queryDef.getQueryId() : "");
+
+ addToKernelManipulationQueue(
+ kernel -> {
+ throw new MSQException(CanceledFault.INSTANCE);
+ }
+ );
+ }
+
+ public TaskStatus runTask(final Closer closer)
+ {
+ QueryDefinition queryDef = null;
+ ControllerQueryKernel queryKernel = null;
+ ListenableFuture> workerTaskRunnerFuture = null;
+ CounterSnapshotsTree countersSnapshot = null;
+ Yielder resultsYielder = null;
+ Throwable exceptionEncountered = null;
+
+ final TaskState taskStateForReport;
+ final MSQErrorReport errorForReport;
+
+ try {
+ // Planning-related: convert the native query from MSQSpec into a multi-stage QueryDefinition.
+ this.queryStartTime = DateTimes.nowUtc();
+ queryDef = initializeQueryDefAndState(closer);
+
+ final InputSpecSlicerFactory inputSpecSlicerFactory = makeInputSpecSlicerFactory(makeDataSegmentTimelineView());
+
+ // Execution-related: run the multi-stage QueryDefinition.
+ final Pair> queryRunResult =
+ new RunQueryUntilDone(queryDef, inputSpecSlicerFactory, closer).run();
+
+ queryKernel = Preconditions.checkNotNull(queryRunResult.lhs);
+ workerTaskRunnerFuture = Preconditions.checkNotNull(queryRunResult.rhs);
+ resultsYielder = getFinalResultsYielder(queryDef, queryKernel);
+ publishSegmentsIfNeeded(queryDef, queryKernel);
+ }
+ catch (Throwable e) {
+ exceptionEncountered = e;
+ }
+
+ // Fetch final counters in separate try, in case runQueryUntilDone threw an exception.
+ try {
+ countersSnapshot = getFinalCountersSnapshot(queryKernel);
+ }
+ catch (Throwable e) {
+ if (exceptionEncountered != null) {
+ exceptionEncountered.addSuppressed(e);
+ } else {
+ exceptionEncountered = e;
+ }
+ }
+
+ if (queryKernel != null && queryKernel.isSuccess() && exceptionEncountered == null) {
+ taskStateForReport = TaskState.SUCCESS;
+ errorForReport = null;
+ } else {
+ // Query failure. Generate an error report and log the error(s) we encountered.
+ final String selfHost = MSQTasks.getHostFromSelfNode(selfDruidNode);
+ final MSQErrorReport controllerError =
+ exceptionEncountered != null
+ ? MSQErrorReport.fromException(id(), selfHost, null, exceptionEncountered)
+ : null;
+ final MSQErrorReport workerError = workerErrorRef.get();
+
+ taskStateForReport = TaskState.FAILED;
+ errorForReport = MSQTasks.makeErrorReport(id(), selfHost, controllerError, workerError);
+
+ // Log the errors we encountered.
+ if (controllerError != null) {
+ log.warn("Controller: %s", MSQTasks.errorReportToLogMessage(controllerError));
+ }
+
+ if (workerError != null) {
+ log.warn("Worker: %s", MSQTasks.errorReportToLogMessage(workerError));
+ }
+ }
+
+ try {
+ // Write report even if something went wrong.
+ final MSQStagesReport stagesReport;
+ final MSQResultsReport resultsReport;
+
+ if (queryDef != null) {
+ final Map stagePhaseMap;
+
+ if (queryKernel != null) {
+ // Once the query finishes, cleanup would have happened for all the stages that were successful
+ // Therefore we mark it as done to make the reports prettier and more accurate
+ queryKernel.markSuccessfulTerminalStagesAsFinished();
+ stagePhaseMap = queryKernel.getActiveStages()
+ .stream()
+ .collect(
+ Collectors.toMap(StageId::getStageNumber, queryKernel::getStagePhase)
+ );
+ } else {
+ stagePhaseMap = Collections.emptyMap();
+ }
+
+ stagesReport = makeStageReport(
+ queryDef,
+ stagePhaseMap,
+ stageRuntimesForLiveReports,
+ stageWorkerCountsForLiveReports,
+ stagePartitionCountsForLiveReports
+ );
+ } else {
+ stagesReport = null;
+ }
+
+ if (resultsYielder != null) {
+ resultsReport = makeResultsTaskReport(
+ queryDef,
+ resultsYielder,
+ task.getQuerySpec().getColumnMappings(),
+ task.getSqlTypeNames()
+ );
+ } else {
+ resultsReport = null;
+ }
+
+ final MSQTaskReportPayload taskReportPayload = new MSQTaskReportPayload(
+ makeStatusReport(
+ taskStateForReport,
+ errorForReport,
+ workerWarnings,
+ queryStartTime,
+ new Interval(queryStartTime, DateTimes.nowUtc()).toDurationMillis()
+ ),
+ stagesReport,
+ countersSnapshot,
+ resultsReport
+ );
+
+ context.writeReports(
+ id(),
+ TaskReport.buildTaskReports(new MSQTaskReport(id(), taskReportPayload))
+ );
+ }
+ catch (Throwable e) {
+ log.warn(e, "Error encountered while writing task report. Skipping.");
+ }
+
+ if (queryKernel != null && queryKernel.isSuccess()) {
+ // If successful, encourage the tasks to exit successfully.
+ postFinishToAllTasks();
+ workerTaskLauncher.stop(false);
+ } else {
+ // If not successful, cancel running tasks.
+ if (workerTaskLauncher != null) {
+ workerTaskLauncher.stop(true);
+ }
+ }
+
+ // Wait for worker tasks to exit. Ignore their return status. At this point, we've done everything we need to do,
+ // so we don't care about the task exit status.
+ if (workerTaskRunnerFuture != null) {
+ try {
+ workerTaskRunnerFuture.get();
+ }
+ catch (Exception ignored) {
+ // Suppress.
+ }
+ }
+
+ cleanUpDurableStorageIfNeeded();
+
+ if (taskStateForReport == TaskState.SUCCESS) {
+ return TaskStatus.success(id());
+ } else {
+ // errorForReport is nonnull when taskStateForReport != SUCCESS. Use that message.
+ return TaskStatus.failure(id(), errorForReport.getFault().getCodeWithMessage());
+ }
+ }
+
+ /**
+ * Adds some logic to {@link #kernelManipulationQueue}, where it will, in due time, be executed by the main
+ * controller loop in {@link RunQueryUntilDone#run()}.
+ *
+ * If the consumer throws an exception, the query fails.
+ */
+ private void addToKernelManipulationQueue(Consumer kernelConsumer)
+ {
+ if (!kernelManipulationQueue.offer(kernelConsumer)) {
+ final String message = "Controller kernel queue is full. Main controller loop may be delayed or stuck.";
+ log.warn(message);
+ throw new IllegalStateException(message);
+ }
+ }
+
+ private QueryDefinition initializeQueryDefAndState(final Closer closer)
+ {
+ this.selfDruidNode = context.selfNode();
+ context.registerController(this, closer);
+
+ this.netClient = new ExceptionWrappingWorkerClient(context.taskClientFor(this));
+ closer.register(netClient::close);
+
+ final boolean isDurableStorageEnabled =
+ MultiStageQueryContext.isDurableStorageEnabled(task.getQuerySpec().getQuery().getContext());
+
+ final QueryDefinition queryDef = makeQueryDefinition(
+ id(),
+ makeQueryControllerToolKit(),
+ task.getQuerySpec()
+ );
+
+ QueryValidator.validateQueryDef(queryDef);
+ queryDefRef.set(queryDef);
+
+ log.debug("Query [%s] durable storage mode is set to %s.", queryDef.getQueryId(), isDurableStorageEnabled);
+
+ this.workerTaskLauncher = new MSQWorkerTaskLauncher(
+ id(),
+ task.getDataSource(),
+ context,
+ isDurableStorageEnabled,
+
+ // 10 minutes +- 2 minutes jitter
+ TimeUnit.SECONDS.toMillis(600 + ThreadLocalRandom.current().nextInt(-4, 5) * 30L)
+ );
+
+ long maxParseExceptions = -1;
+
+ if (task.getSqlQueryContext() != null) {
+ maxParseExceptions = Optional.ofNullable(
+ task.getSqlQueryContext().get(MSQWarnings.CTX_MAX_PARSE_EXCEPTIONS_ALLOWED))
+ .map(DimensionHandlerUtils::convertObjectToLong)
+ .orElse(MSQWarnings.DEFAULT_MAX_PARSE_EXCEPTIONS_ALLOWED);
+ }
+
+ this.faultsExceededChecker = new FaultsExceededChecker(
+ ImmutableMap.of(CannotParseExternalDataFault.CODE, maxParseExceptions)
+ );
+
+ return queryDef;
+ }
+
+ /**
+ * Provide a {@link ClusterByStatisticsSnapshot} for shuffling stages.
+ */
+ @Override
+ public void updateStatus(int stageNumber, int workerNumber, Object keyStatisticsObject)
+ {
+ addToKernelManipulationQueue(
+ queryKernel -> {
+ final StageId stageId = queryKernel.getStageId(stageNumber);
+
+ // We need a specially-decorated ObjectMapper to deserialize key statistics.
+ final StageDefinition stageDef = queryKernel.getStageDefinition(stageId);
+ final ObjectMapper mapper = MSQTasks.decorateObjectMapperForKeyCollectorSnapshot(
+ context.jsonMapper(),
+ stageDef.getShuffleSpec().get().getClusterBy(),
+ stageDef.getShuffleSpec().get().doesAggregateByClusterKey()
+ );
+
+ final ClusterByStatisticsSnapshot keyStatistics;
+ try {
+ keyStatistics = mapper.convertValue(keyStatisticsObject, ClusterByStatisticsSnapshot.class);
+ }
+ catch (IllegalArgumentException e) {
+ throw new IAE(
+ e,
+ "Unable to deserialize the key statistic for stage [%s] received from the worker [%d]",
+ stageId,
+ workerNumber
+ );
+ }
+
+ queryKernel.addResultKeyStatisticsForStageAndWorker(stageId, workerNumber, keyStatistics);
+ }
+ );
+ }
+
+ @Override
+ public void workerError(MSQErrorReport errorReport)
+ {
+ if (!workerTaskLauncher.isTaskCanceledByController(errorReport.getTaskId())) {
+ workerErrorRef.compareAndSet(null, errorReport);
+ }
+ }
+
+ /**
+ * This method intakes all the warnings that are generated by the worker. It is the responsibility of the
+ * worker node to ensure that it doesn't spam the controller with unneseccary warning stack traces. Currently, that
+ * limiting is implemented in {@link MSQWarningReportLimiterPublisher}
+ */
+ @Override
+ public void workerWarning(List errorReports)
+ {
+ // This check safeguards that the controller doesn't run out of memory. Workers apply their own limiting to
+ // protect their own memory, and to conserve worker -> controller bandwidth.
+ long numReportsToAddCheck = Math.min(
+ errorReports.size(),
+ Limits.MAX_WORKERS * Limits.MAX_VERBOSE_WARNINGS - workerWarnings.size()
+ );
+ if (numReportsToAddCheck > 0) {
+ synchronized (workerWarnings) {
+ long numReportsToAdd = Math.min(
+ errorReports.size(),
+ Limits.MAX_WORKERS * Limits.MAX_VERBOSE_WARNINGS - workerWarnings.size()
+ );
+ for (int i = 0; i < numReportsToAdd; ++i) {
+ workerWarnings.add(errorReports.get(i));
+ }
+ }
+ }
+ }
+
+ /**
+ * Periodic update of {@link CounterSnapshots} from subtasks.
+ */
+ @Override
+ public void updateCounters(CounterSnapshotsTree snapshotsTree)
+ {
+ taskCountersForLiveReports.putAll(snapshotsTree);
+ Optional> warningsExceeded =
+ faultsExceededChecker.addFaultsAndCheckIfExceeded(taskCountersForLiveReports);
+
+ if (warningsExceeded.isPresent()) {
+ // Present means the warning limit was exceeded, and warnings have therefore turned into an error.
+ String errorCode = warningsExceeded.get().lhs;
+ Long limit = warningsExceeded.get().rhs;
+ workerError(MSQErrorReport.fromFault(
+ id(),
+ selfDruidNode.getHost(),
+ null,
+ new TooManyWarningsFault(limit.intValue(), errorCode)
+ ));
+ addToKernelManipulationQueue(
+ queryKernel ->
+ queryKernel.getActiveStages().forEach(queryKernel::failStage)
+ );
+ }
+ }
+
+ /**
+ * Reports that results are ready for a subtask.
+ */
+ @SuppressWarnings("unchecked")
+ @Override
+ public void resultsComplete(
+ final String queryId,
+ final int stageNumber,
+ final int workerNumber,
+ Object resultObject
+ )
+ {
+ addToKernelManipulationQueue(
+ queryKernel -> {
+ final StageId stageId = new StageId(queryId, stageNumber);
+ final Object convertedResultObject;
+ try {
+ convertedResultObject = context.jsonMapper().convertValue(
+ resultObject,
+ queryKernel.getStageDefinition(stageId).getProcessorFactory().getAccumulatedResultTypeReference()
+ );
+ }
+ catch (IllegalArgumentException e) {
+ throw new IAE(
+ e,
+ "Unable to deserialize the result object for stage [%s] received from the worker [%d]",
+ stageId,
+ workerNumber
+ );
+ }
+
+
+ queryKernel.setResultsCompleteForStageAndWorker(stageId, workerNumber, convertedResultObject);
+ }
+ );
+ }
+
+ @Override
+ @Nullable
+ public Map liveReports()
+ {
+ final QueryDefinition queryDef = queryDefRef.get();
+
+ if (queryDef == null) {
+ return null;
+ }
+
+ return TaskReport.buildTaskReports(
+ new MSQTaskReport(
+ id(),
+ new MSQTaskReportPayload(
+ makeStatusReport(
+ TaskState.RUNNING,
+ null,
+ workerWarnings,
+ queryStartTime,
+ queryStartTime == null ? -1L : new Interval(queryStartTime, DateTimes.nowUtc()).toDurationMillis()
+ ),
+ makeStageReport(
+ queryDef,
+ stagePhasesForLiveReports,
+ stageRuntimesForLiveReports,
+ stageWorkerCountsForLiveReports,
+ stagePartitionCountsForLiveReports
+ ),
+ makeCountersSnapshotForLiveReports(),
+ null
+ )
+ )
+ );
+ }
+
+ /**
+ * Returns the segments that will be generated by this job. Delegates to
+ * {@link #generateSegmentIdsWithShardSpecsForAppend} or {@link #generateSegmentIdsWithShardSpecsForReplace} as
+ * appropriate. This is a potentially expensive call, since it requires calling Overlord APIs.
+ *
+ * @throws MSQException with {@link InsertCannotAllocateSegmentFault} if an allocation cannot be made
+ */
+ private List generateSegmentIdsWithShardSpecs(
+ final DataSourceMSQDestination destination,
+ final RowSignature signature,
+ final ClusterBy clusterBy,
+ final ClusterByPartitions partitionBoundaries,
+ final boolean mayHaveMultiValuedClusterByFields
+ ) throws IOException
+ {
+ if (destination.isReplaceTimeChunks()) {
+ return generateSegmentIdsWithShardSpecsForReplace(
+ destination,
+ signature,
+ clusterBy,
+ partitionBoundaries,
+ mayHaveMultiValuedClusterByFields
+ );
+ } else {
+ final RowKeyReader keyReader = clusterBy.keyReader(signature);
+ return generateSegmentIdsWithShardSpecsForAppend(destination, partitionBoundaries, keyReader);
+ }
+ }
+
+ /**
+ * Used by {@link #generateSegmentIdsWithShardSpecs}.
+ */
+ private List generateSegmentIdsWithShardSpecsForAppend(
+ final DataSourceMSQDestination destination,
+ final ClusterByPartitions partitionBoundaries,
+ final RowKeyReader keyReader
+ ) throws IOException
+ {
+ final Granularity segmentGranularity = destination.getSegmentGranularity();
+
+ String previousSegmentId = null;
+
+ final List retVal = new ArrayList<>(partitionBoundaries.size());
+
+ for (ClusterByPartition partitionBoundary : partitionBoundaries) {
+ final DateTime timestamp = getBucketDateTime(partitionBoundary, segmentGranularity, keyReader);
+ final SegmentIdWithShardSpec allocation;
+ try {
+ allocation = context.taskActionClient().submit(
+ new SegmentAllocateAction(
+ task.getDataSource(),
+ timestamp,
+ // Same granularity for queryGranularity, segmentGranularity because we don't have insight here
+ // into what queryGranularity "actually" is. (It depends on what time floor function was used.)
+ segmentGranularity,
+ segmentGranularity,
+ id(),
+ previousSegmentId,
+ false,
+ NumberedPartialShardSpec.instance(),
+ LockGranularity.TIME_CHUNK,
+ TaskLockType.SHARED
+ )
+ );
+ }
+ catch (ISE e) {
+ if (isTaskLockPreemptedException(e)) {
+ throw new MSQException(e, InsertLockPreemptedFault.instance());
+ } else {
+ throw e;
+ }
+ }
+
+ if (allocation == null) {
+ throw new MSQException(
+ new InsertCannotAllocateSegmentFault(
+ task.getDataSource(),
+ segmentGranularity.bucket(timestamp)
+ )
+ );
+ }
+
+ retVal.add(allocation);
+ previousSegmentId = allocation.asSegmentId().toString();
+ }
+
+ return retVal;
+ }
+
+ /**
+ * Used by {@link #generateSegmentIdsWithShardSpecs}.
+ */
+ private List generateSegmentIdsWithShardSpecsForReplace(
+ final DataSourceMSQDestination destination,
+ final RowSignature signature,
+ final ClusterBy clusterBy,
+ final ClusterByPartitions partitionBoundaries,
+ final boolean mayHaveMultiValuedClusterByFields
+ ) throws IOException
+ {
+ final RowKeyReader keyReader = clusterBy.keyReader(signature);
+ final SegmentIdWithShardSpec[] retVal = new SegmentIdWithShardSpec[partitionBoundaries.size()];
+ final Granularity segmentGranularity = destination.getSegmentGranularity();
+ final List shardColumns;
+
+ if (mayHaveMultiValuedClusterByFields) {
+ // DimensionRangeShardSpec cannot handle multi-valued fields.
+ shardColumns = Collections.emptyList();
+ } else {
+ shardColumns = computeShardColumns(signature, clusterBy, task.getQuerySpec().getColumnMappings());
+ }
+
+ // Group partition ranges by bucket (time chunk), so we can generate shardSpecs for each bucket independently.
+ final Map>> partitionsByBucket = new HashMap<>();
+ for (int i = 0; i < partitionBoundaries.ranges().size(); i++) {
+ ClusterByPartition partitionBoundary = partitionBoundaries.ranges().get(i);
+ final DateTime bucketDateTime = getBucketDateTime(partitionBoundary, segmentGranularity, keyReader);
+ partitionsByBucket.computeIfAbsent(bucketDateTime, ignored -> new ArrayList<>())
+ .add(Pair.of(i, partitionBoundary));
+ }
+
+ // Process buckets (time chunks) one at a time.
+ for (final Map.Entry>> bucketEntry : partitionsByBucket.entrySet()) {
+ final Interval interval = segmentGranularity.bucket(bucketEntry.getKey());
+
+ // Validate interval against the replaceTimeChunks set of intervals.
+ if (destination.getReplaceTimeChunks().stream().noneMatch(chunk -> chunk.contains(interval))) {
+ throw new MSQException(new InsertTimeOutOfBoundsFault(interval));
+ }
+
+ final List> ranges = bucketEntry.getValue();
+ String version = null;
+
+ final List locks = context.taskActionClient().submit(new LockListAction());
+ for (final TaskLock lock : locks) {
+ if (lock.getInterval().contains(interval)) {
+ version = lock.getVersion();
+ }
+ }
+
+ if (version == null) {
+ // Lock was revoked, probably, because we should have originally acquired it in isReady.
+ throw new MSQException(InsertLockPreemptedFault.INSTANCE);
+ }
+
+ for (int segmentNumber = 0; segmentNumber < ranges.size(); segmentNumber++) {
+ final int partitionNumber = ranges.get(segmentNumber).lhs;
+ final ShardSpec shardSpec;
+
+ if (shardColumns.isEmpty()) {
+ shardSpec = new NumberedShardSpec(segmentNumber, ranges.size());
+ } else {
+ final ClusterByPartition range = ranges.get(segmentNumber).rhs;
+ final StringTuple start =
+ segmentNumber == 0 ? null : makeStringTuple(clusterBy, keyReader, range.getStart());
+ final StringTuple end =
+ segmentNumber == ranges.size() - 1 ? null : makeStringTuple(clusterBy, keyReader, range.getEnd());
+
+ shardSpec = new DimensionRangeShardSpec(shardColumns, start, end, segmentNumber, ranges.size());
+ }
+
+ retVal[partitionNumber] = new SegmentIdWithShardSpec(task.getDataSource(), interval, version, shardSpec);
+ }
+ }
+
+ return Arrays.asList(retVal);
+ }
+
+ /**
+ * Returns a complete list of task ids, ordered by worker number. The Nth task has worker number N.
+ *
+ * If the currently-running set of tasks is incomplete, returns an absent Optional.
+ */
+ @Override
+ public List getTaskIds()
+ {
+ if (workerTaskLauncher == null) {
+ return Collections.emptyList();
+ }
+
+ return workerTaskLauncher.getTaskList();
+ }
+
+ @SuppressWarnings({"unchecked", "rawtypes"})
+ @Nullable
+ private Int2ObjectMap makeWorkerFactoryInfosForStage(
+ final QueryDefinition queryDef,
+ final int stageNumber,
+ final WorkerInputs workerInputs,
+ @Nullable final List segmentsToGenerate
+ )
+ {
+ if (MSQControllerTask.isIngestion(task.getQuerySpec()) &&
+ stageNumber == queryDef.getFinalStageDefinition().getStageNumber()) {
+ // noinspection unchecked,rawtypes
+ return (Int2ObjectMap) makeSegmentGeneratorWorkerFactoryInfos(workerInputs, segmentsToGenerate);
+ } else {
+ return null;
+ }
+ }
+
+ @SuppressWarnings("rawtypes")
+ private QueryKit makeQueryControllerToolKit()
+ {
+ final Map, QueryKit> kitMap =
+ ImmutableMap., QueryKit>builder()
+ .put(ScanQuery.class, new ScanQueryKit(context.jsonMapper()))
+ .put(GroupByQuery.class, new GroupByQueryKit())
+ .build();
+
+ return new MultiQueryKit(kitMap);
+ }
+
+ private DataSegmentTimelineView makeDataSegmentTimelineView()
+ {
+ return (dataSource, intervals) -> {
+ final Collection dataSegments =
+ context.coordinatorClient().fetchUsedSegmentsInDataSourceForIntervals(dataSource, intervals);
+
+ if (dataSegments.isEmpty()) {
+ return Optional.empty();
+ } else {
+ return Optional.of(VersionedIntervalTimeline.forSegments(dataSegments));
+ }
+ };
+ }
+
+ private Int2ObjectMap> makeSegmentGeneratorWorkerFactoryInfos(
+ final WorkerInputs workerInputs,
+ final List segmentsToGenerate
+ )
+ {
+ final Int2ObjectMap> retVal = new Int2ObjectAVLTreeMap<>();
+
+ for (final int workerNumber : workerInputs.workers()) {
+ // SegmentGenerator stage has a single input from another stage.
+ final StageInputSlice stageInputSlice =
+ (StageInputSlice) Iterables.getOnlyElement(workerInputs.inputsForWorker(workerNumber));
+
+ final List workerSegments = new ArrayList<>();
+ retVal.put(workerNumber, workerSegments);
+
+ for (final ReadablePartition partition : stageInputSlice.getPartitions()) {
+ workerSegments.add(segmentsToGenerate.get(partition.getPartitionNumber()));
+ }
+ }
+
+ return retVal;
+ }
+
+ private void contactWorkersForStage(final TaskContactFn contactFn, final IntSet workers)
+ {
+ final List taskIds = getTaskIds();
+ final List> taskFutures = new ArrayList<>(workers.size());
+
+ for (int workerNumber : workers) {
+ final String taskId = taskIds.get(workerNumber);
+ taskFutures.add(contactFn.contactTask(netClient, taskId, workerNumber));
+ }
+
+ FutureUtils.getUnchecked(MSQFutureUtils.allAsList(taskFutures, true), true);
+ }
+
+ private void startWorkForStage(
+ final QueryDefinition queryDef,
+ final ControllerQueryKernel queryKernel,
+ final int stageNumber,
+ @Nullable final List segmentsToGenerate
+ )
+ {
+ final Int2ObjectMap extraInfos = makeWorkerFactoryInfosForStage(
+ queryDef,
+ stageNumber,
+ queryKernel.getWorkerInputsForStage(queryKernel.getStageId(stageNumber)),
+ segmentsToGenerate
+ );
+
+ final Int2ObjectMap workOrders = queryKernel.createWorkOrders(stageNumber, extraInfos);
+
+ contactWorkersForStage(
+ (netClient, taskId, workerNumber) -> netClient.postWorkOrder(taskId, workOrders.get(workerNumber)),
+ workOrders.keySet()
+ );
+ }
+
+ private void postResultPartitionBoundariesForStage(
+ final QueryDefinition queryDef,
+ final int stageNumber,
+ final ClusterByPartitions resultPartitionBoundaries,
+ final IntSet workers
+ )
+ {
+ contactWorkersForStage(
+ (netClient, taskId, workerNumber) ->
+ netClient.postResultPartitionBoundaries(
+ taskId,
+ new StageId(queryDef.getQueryId(), stageNumber),
+ resultPartitionBoundaries
+ ),
+ workers
+ );
+ }
+
+ /**
+ * Publish the list of segments. Additionally, if {@link DataSourceMSQDestination#isReplaceTimeChunks()},
+ * also drop all other segments within the replacement intervals.
+ *
+ * If any existing segments cannot be dropped because their intervals are not wholly contained within the
+ * replacement parameter, throws a {@link MSQException} with {@link InsertCannotReplaceExistingSegmentFault}.
+ */
+ private void publishAllSegments(final Set segments) throws IOException
+ {
+ final DataSourceMSQDestination destination =
+ (DataSourceMSQDestination) task.getQuerySpec().getDestination();
+ final Set segmentsToDrop;
+
+ if (destination.isReplaceTimeChunks()) {
+ final List intervalsToDrop = findIntervalsToDrop(Preconditions.checkNotNull(segments, "segments"));
+
+ if (intervalsToDrop.isEmpty()) {
+ segmentsToDrop = null;
+ } else {
+ // Determine which segments to drop as part of the replace operation. This is safe because, in the case where we
+ // are doing a replace, the isReady method (which runs prior to the task starting) acquires an exclusive lock.
+ segmentsToDrop =
+ ImmutableSet.copyOf(
+ context.taskActionClient().submit(
+ new RetrieveUsedSegmentsAction(
+ task.getDataSource(),
+ null,
+ intervalsToDrop,
+ Segments.ONLY_VISIBLE
+ )
+ )
+ );
+
+ // Validate that there are no segments that partially overlap the intervals-to-drop. Otherwise, the replace
+ // may be incomplete.
+ for (final DataSegment segmentToDrop : segmentsToDrop) {
+ if (destination.getReplaceTimeChunks()
+ .stream()
+ .noneMatch(interval -> interval.contains(segmentToDrop.getInterval()))) {
+ throw new MSQException(new InsertCannotReplaceExistingSegmentFault(segmentToDrop.getId()));
+ }
+ }
+ }
+
+ if (segments.isEmpty()) {
+ // Nothing to publish, only drop. We already validated that the intervalsToDrop do not have any
+ // partially-overlapping segments, so it's safe to drop them as intervals instead of as specific segments.
+ for (final Interval interval : intervalsToDrop) {
+ context.taskActionClient()
+ .submit(new MarkSegmentsAsUnusedAction(task.getDataSource(), interval));
+ }
+ } else {
+ try {
+ context.taskActionClient()
+ .submit(SegmentTransactionalInsertAction.overwriteAction(null, segmentsToDrop, segments));
+ }
+ catch (Exception e) {
+ if (isTaskLockPreemptedException(e)) {
+ throw new MSQException(e, InsertLockPreemptedFault.instance());
+ } else {
+ throw e;
+ }
+ }
+ }
+ } else if (!segments.isEmpty()) {
+ // Append mode.
+ try {
+ context.taskActionClient().submit(new SegmentInsertAction(segments));
+ }
+ catch (Exception e) {
+ if (isTaskLockPreemptedException(e)) {
+ throw new MSQException(e, InsertLockPreemptedFault.instance());
+ } else {
+ throw e;
+ }
+ }
+ }
+ }
+
+ /**
+ * When doing an ingestion with {@link DataSourceMSQDestination#isReplaceTimeChunks()}, finds intervals
+ * containing data that should be dropped.
+ */
+ private List findIntervalsToDrop(final Set publishedSegments)
+ {
+ // Safe to cast because publishAllSegments is only called for dataSource destinations.
+ final DataSourceMSQDestination destination =
+ (DataSourceMSQDestination) task.getQuerySpec().getDestination();
+ final List replaceIntervals =
+ new ArrayList<>(JodaUtils.condenseIntervals(destination.getReplaceTimeChunks()));
+ final List publishIntervals =
+ JodaUtils.condenseIntervals(Iterables.transform(publishedSegments, DataSegment::getInterval));
+ return IntervalUtils.difference(replaceIntervals, publishIntervals);
+ }
+
+ private CounterSnapshotsTree getCountersFromAllTasks()
+ {
+ final CounterSnapshotsTree retVal = new CounterSnapshotsTree();
+ final List taskList = workerTaskLauncher.getTaskList();
+
+ final List> futures = new ArrayList<>();
+
+ for (String taskId : taskList) {
+ futures.add(netClient.getCounters(taskId));
+ }
+
+ final List snapshotsTrees =
+ FutureUtils.getUnchecked(MSQFutureUtils.allAsList(futures, true), true);
+
+ for (CounterSnapshotsTree snapshotsTree : snapshotsTrees) {
+ retVal.putAll(snapshotsTree);
+ }
+
+ return retVal;
+ }
+
+ private void postFinishToAllTasks()
+ {
+ final List taskList = workerTaskLauncher.getTaskList();
+
+ final List> futures = new ArrayList<>();
+
+ for (String taskId : taskList) {
+ futures.add(netClient.postFinish(taskId));
+ }
+
+ FutureUtils.getUnchecked(MSQFutureUtils.allAsList(futures, true), true);
+ }
+
+ private CounterSnapshotsTree makeCountersSnapshotForLiveReports()
+ {
+ // taskCountersForLiveReports is mutable: Copy so we get a point-in-time snapshot.
+ return CounterSnapshotsTree.fromMap(taskCountersForLiveReports.copyMap());
+ }
+
+ private CounterSnapshotsTree getFinalCountersSnapshot(@Nullable final ControllerQueryKernel queryKernel)
+ {
+ if (queryKernel != null && queryKernel.isSuccess()) {
+ return getCountersFromAllTasks();
+ } else {
+ return makeCountersSnapshotForLiveReports();
+ }
+ }
+
+ @Nullable
+ private Yielder getFinalResultsYielder(
+ final QueryDefinition queryDef,
+ final ControllerQueryKernel queryKernel
+ )
+ {
+ if (queryKernel.isSuccess() && isInlineResults(task.getQuerySpec())) {
+ final StageId finalStageId = queryKernel.getStageId(queryDef.getFinalStageDefinition().getStageNumber());
+ final List taskIds = getTaskIds();
+ final Closer closer = Closer.create();
+
+ final ListeningExecutorService resultReaderExec =
+ MoreExecutors.listeningDecorator(Execs.singleThreaded("result-reader-%d"));
+ closer.register(resultReaderExec::shutdownNow);
+
+ final InputChannelFactory inputChannelFactory;
+
+ if (MultiStageQueryContext.isDurableStorageEnabled(task.getQuerySpec().getQuery().getContext())) {
+ inputChannelFactory = DurableStorageInputChannelFactory.createStandardImplementation(
+ id(),
+ () -> taskIds,
+ MSQTasks.makeStorageConnector(context.injector()),
+ closer
+ );
+ } else {
+ inputChannelFactory = new WorkerInputChannelFactory(netClient, () -> taskIds);
+ }
+
+ final InputChannels inputChannels = new InputChannelsImpl(
+ queryDef,
+ queryKernel.getResultPartitionsForStage(finalStageId),
+ inputChannelFactory,
+ () -> ArenaMemoryAllocator.createOnHeap(5_000_000),
+ new FrameProcessorExecutor(resultReaderExec),
+ null
+ );
+
+ return Yielders.each(
+ Sequences.concat(
+ StreamSupport.stream(queryKernel.getResultPartitionsForStage(finalStageId).spliterator(), false)
+ .map(
+ readablePartition -> {
+ try {
+ return new FrameChannelSequence(
+ inputChannels.openChannel(
+ new StagePartition(
+ queryKernel.getStageDefinition(finalStageId).getId(),
+ readablePartition.getPartitionNumber()
+ )
+ )
+ );
+ }
+ catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+ ).collect(Collectors.toList())
+ ).flatMap(
+ frame -> {
+ final Cursor cursor = FrameProcessors.makeCursor(
+ frame,
+ queryKernel.getStageDefinition(finalStageId).getFrameReader()
+ );
+
+ final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory();
+ final ColumnMappings columnMappings = task.getQuerySpec().getColumnMappings();
+ @SuppressWarnings("rawtypes")
+ final List selectors =
+ columnMappings.getMappings()
+ .stream()
+ .map(
+ mapping ->
+ columnSelectorFactory.makeColumnValueSelector(
+ mapping.getQueryColumn())
+ ).collect(Collectors.toList());
+
+ final List retVal = new ArrayList<>();
+ while (!cursor.isDone()) {
+ final Object[] row = new Object[columnMappings.getMappings().size()];
+ for (int i = 0; i < row.length; i++) {
+ row[i] = selectors.get(i).getObject();
+ }
+ retVal.add(row);
+ cursor.advance();
+ }
+
+ return Sequences.simple(retVal);
+ }
+ ).withBaggage(resultReaderExec::shutdownNow)
+ );
+ } else {
+ return null;
+ }
+ }
+
+ private void publishSegmentsIfNeeded(
+ final QueryDefinition queryDef,
+ final ControllerQueryKernel queryKernel
+ ) throws IOException
+ {
+ if (queryKernel.isSuccess() && MSQControllerTask.isIngestion(task.getQuerySpec())) {
+ final StageId finalStageId = queryKernel.getStageId(queryDef.getFinalStageDefinition().getStageNumber());
+
+ //noinspection unchecked
+ @SuppressWarnings("unchecked")
+ final Set segments = (Set) queryKernel.getResultObjectForStage(finalStageId);
+ log.info("Query [%s] publishing %d segments.", queryDef.getQueryId(), segments.size());
+ publishAllSegments(segments);
+ }
+ }
+
+ /**
+ * Clean up durable storage, if used for stage output.
+ *
+ * Note that this is only called by the controller task itself. It isn't called automatically by anything in
+ * particular if the controller fails early without being able to run its cleanup routines. This can cause files
+ * to be left in durable storage beyond their useful life.
+ */
+ private void cleanUpDurableStorageIfNeeded()
+ {
+ if (MultiStageQueryContext.isDurableStorageEnabled(task.getQuerySpec().getQuery().getContext())) {
+ final String controllerDirName = DurableStorageOutputChannelFactory.getControllerDirectory(task.getId());
+ try {
+ // Delete all temporary files as a failsafe
+ MSQTasks.makeStorageConnector(context.injector()).deleteRecursively(controllerDirName);
+ }
+ catch (Exception e) {
+ // If an error is thrown while cleaning up a file, log it and try to continue with the cleanup
+ log.warn(e, "Error while cleaning up temporary files at path %s", controllerDirName);
+ }
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ private static QueryDefinition makeQueryDefinition(
+ final String queryId,
+ @SuppressWarnings("rawtypes") final QueryKit toolKit,
+ final MSQSpec querySpec
+ )
+ {
+ final MSQTuningConfig tuningConfig = querySpec.getTuningConfig();
+ final ShuffleSpecFactory shuffleSpecFactory;
+
+ if (MSQControllerTask.isIngestion(querySpec)) {
+ shuffleSpecFactory = (clusterBy, aggregate) ->
+ new TargetSizeShuffleSpec(
+ clusterBy,
+ tuningConfig.getRowsPerSegment(),
+ aggregate
+ );
+ } else if (querySpec.getDestination() instanceof TaskReportMSQDestination) {
+ shuffleSpecFactory = ShuffleSpecFactories.singlePartition();
+ } else {
+ throw new ISE("Unsupported destination [%s]", querySpec.getDestination());
+ }
+
+ final Query> queryToPlan;
+
+ if (querySpec.getColumnMappings().hasOutputColumn(ColumnHolder.TIME_COLUMN_NAME)) {
+ queryToPlan = querySpec.getQuery().withOverriddenContext(
+ ImmutableMap.of(
+ QueryKitUtils.CTX_TIME_COLUMN_NAME,
+ querySpec.getColumnMappings().getQueryColumnForOutputColumn(ColumnHolder.TIME_COLUMN_NAME)
+ )
+ );
+ } else {
+ queryToPlan = querySpec.getQuery();
+ }
+
+ final QueryDefinition queryDef;
+
+ try {
+ queryDef = toolKit.makeQueryDefinition(
+ queryId,
+ queryToPlan,
+ toolKit,
+ shuffleSpecFactory,
+ tuningConfig.getMaxNumWorkers(),
+ 0
+ );
+ }
+ catch (MSQException e) {
+ // If the toolkit throws a MSQFault, don't wrap it in a more generic QueryNotSupportedFault
+ throw e;
+ }
+ catch (Exception e) {
+ throw new MSQException(e, QueryNotSupportedFault.INSTANCE);
+ }
+
+ if (MSQControllerTask.isIngestion(querySpec)) {
+ final RowSignature querySignature = queryDef.getFinalStageDefinition().getSignature();
+ final ClusterBy queryClusterBy = queryDef.getFinalStageDefinition().getClusterBy();
+ final ColumnMappings columnMappings = querySpec.getColumnMappings();
+
+ // Find the stage that provides shuffled input to the final segment-generation stage.
+ StageDefinition finalShuffleStageDef = queryDef.getFinalStageDefinition();
+
+ while (!finalShuffleStageDef.doesShuffle()
+ && InputSpecs.getStageNumbers(finalShuffleStageDef.getInputSpecs()).size() == 1) {
+ finalShuffleStageDef = queryDef.getStageDefinition(
+ Iterables.getOnlyElement(InputSpecs.getStageNumbers(finalShuffleStageDef.getInputSpecs()))
+ );
+ }
+
+ if (!finalShuffleStageDef.doesShuffle()) {
+ finalShuffleStageDef = null;
+ }
+
+ // Add all query stages.
+ // Set shuffleCheckHasMultipleValues on the stage that serves as input to the final segment-generation stage.
+ final QueryDefinitionBuilder builder = QueryDefinition.builder();
+
+ for (final StageDefinition stageDef : queryDef.getStageDefinitions()) {
+ if (stageDef.equals(finalShuffleStageDef)) {
+ builder.add(StageDefinition.builder(stageDef).shuffleCheckHasMultipleValues(true));
+ } else {
+ builder.add(StageDefinition.builder(stageDef));
+ }
+ }
+
+ // Then, add a segment-generation stage.
+ final DataSchema dataSchema = generateDataSchema(querySpec, querySignature, queryClusterBy, columnMappings);
+ builder.add(
+ StageDefinition.builder(queryDef.getNextStageNumber())
+ .inputs(new StageInputSpec(queryDef.getFinalStageDefinition().getStageNumber()))
+ .maxWorkerCount(tuningConfig.getMaxNumWorkers())
+ .processorFactory(
+ new SegmentGeneratorFrameProcessorFactory(
+ dataSchema,
+ columnMappings,
+ tuningConfig
+ )
+ )
+ );
+
+ return builder.build();
+ } else if (querySpec.getDestination() instanceof TaskReportMSQDestination) {
+ return queryDef;
+ } else {
+ throw new ISE("Unsupported destination [%s]", querySpec.getDestination());
+ }
+ }
+
+ private static DataSchema generateDataSchema(
+ MSQSpec querySpec,
+ RowSignature querySignature,
+ ClusterBy queryClusterBy,
+ ColumnMappings columnMappings
+ )
+ {
+ final DataSourceMSQDestination destination = (DataSourceMSQDestination) querySpec.getDestination();
+ final boolean isRollupQuery = isRollupQuery(querySpec.getQuery());
+
+ final Pair, List> dimensionsAndAggregators =
+ makeDimensionsAndAggregatorsForIngestion(
+ querySignature,
+ queryClusterBy,
+ destination.getSegmentSortOrder(),
+ columnMappings,
+ isRollupQuery,
+ querySpec.getQuery()
+ );
+
+ return new DataSchema(
+ destination.getDataSource(),
+ new TimestampSpec(ColumnHolder.TIME_COLUMN_NAME, "millis", null),
+ new DimensionsSpec(dimensionsAndAggregators.lhs),
+ dimensionsAndAggregators.rhs.toArray(new AggregatorFactory[0]),
+ makeGranularitySpecForIngestion(querySpec.getQuery(), querySpec.getColumnMappings(), isRollupQuery),
+ new TransformSpec(null, Collections.emptyList())
+ );
+ }
+
+ private static GranularitySpec makeGranularitySpecForIngestion(
+ final Query> query,
+ final ColumnMappings columnMappings,
+ final boolean isRollupQuery
+ )
+ {
+ if (isRollupQuery) {
+ final String queryGranularity = query.getContextValue(GroupByQuery.CTX_TIMESTAMP_RESULT_FIELD_GRANULARITY, "");
+
+ if (timeIsGroupByDimension((GroupByQuery) query, columnMappings) && !queryGranularity.isEmpty()) {
+ return new ArbitraryGranularitySpec(
+ Granularity.fromString(queryGranularity),
+ true,
+ Intervals.ONLY_ETERNITY
+ );
+ }
+ return new ArbitraryGranularitySpec(Granularities.NONE, true, Intervals.ONLY_ETERNITY);
+ } else {
+ return new ArbitraryGranularitySpec(Granularities.NONE, false, Intervals.ONLY_ETERNITY);
+ }
+ }
+
+ /**
+ * Checks that a {@link GroupByQuery} is grouping on the primary time column.
+ *
+ * The logic here is roundabout. First, we check which column in the {@link GroupByQuery} corresponds to the
+ * output column {@link ColumnHolder#TIME_COLUMN_NAME}, using our {@link ColumnMappings}. Then, we check for the
+ * presence of an optimization done in {@link DruidQuery#toGroupByQuery()}, where the context parameter
+ * {@link GroupByQuery#CTX_TIMESTAMP_RESULT_FIELD} and various related parameters are set when one of the dimensions
+ * is detected to be a time-floor. Finally, we check that the name of that dimension, and the name of our time field
+ * from {@link ColumnMappings}, are the same.
+ */
+ private static boolean timeIsGroupByDimension(GroupByQuery groupByQuery, ColumnMappings columnMappings)
+ {
+ if (columnMappings.hasOutputColumn(ColumnHolder.TIME_COLUMN_NAME)) {
+ final String queryTimeColumn = columnMappings.getQueryColumnForOutputColumn(ColumnHolder.TIME_COLUMN_NAME);
+ return queryTimeColumn.equals(groupByQuery.getContextValue(GroupByQuery.CTX_TIMESTAMP_RESULT_FIELD));
+ } else {
+ return false;
+ }
+ }
+
+ /**
+ * Whether a native query represents an ingestion with rollup.
+ *
+ * Checks for three things:
+ *
+ * - The query must be a {@link GroupByQuery}, because rollup requires columns to be split into dimensions and
+ * aggregations.
+ * - The query must not finalize aggregations, because rollup requires inserting the intermediate type of
+ * complex aggregations, not the finalized type. (So further rollup is possible.)
+ * - The query must explicitly disable {@link GroupByQueryConfig#CTX_KEY_ENABLE_MULTI_VALUE_UNNESTING}, because
+ * groupBy on multi-value dimensions implicitly unnests, which is not desired behavior for rollup at ingestion time
+ * (rollup expects multi-value dimensions to be treated as arrays).
+ */
+ private static boolean isRollupQuery(Query> query)
+ {
+ return query instanceof GroupByQuery
+ && !MultiStageQueryContext.isFinalizeAggregations(query.getQueryContext())
+ && !query.getContextBoolean(GroupByQueryConfig.CTX_KEY_ENABLE_MULTI_VALUE_UNNESTING, true);
+ }
+
+ private static boolean isInlineResults(final MSQSpec querySpec)
+ {
+ return querySpec.getDestination() instanceof TaskReportMSQDestination;
+ }
+
+ private static boolean isTimeBucketedIngestion(final MSQSpec querySpec)
+ {
+ return MSQControllerTask.isIngestion(querySpec)
+ && !((DataSourceMSQDestination) querySpec.getDestination()).getSegmentGranularity()
+ .equals(Granularities.ALL);
+ }
+
+ /**
+ * Compute shard columns for {@link DimensionRangeShardSpec}. Returns an empty list if range-based sharding
+ * is not applicable.
+ */
+ private static List computeShardColumns(
+ final RowSignature signature,
+ final ClusterBy clusterBy,
+ final ColumnMappings columnMappings
+ )
+ {
+ final List clusterByColumns = clusterBy.getColumns();
+ final List shardColumns = new ArrayList<>();
+ final boolean boosted = isClusterByBoosted(clusterBy);
+ final int numShardColumns = clusterByColumns.size() - clusterBy.getBucketByCount() - (boosted ? 1 : 0);
+
+ if (numShardColumns == 0) {
+ return Collections.emptyList();
+ }
+
+ for (int i = clusterBy.getBucketByCount(); i < clusterBy.getBucketByCount() + numShardColumns; i++) {
+ final SortColumn column = clusterByColumns.get(i);
+ final List outputColumns = columnMappings.getOutputColumnsForQueryColumn(column.columnName());
+
+ // DimensionRangeShardSpec only handles ascending order.
+ if (column.descending()) {
+ return Collections.emptyList();
+ }
+
+ ColumnType columnType = signature.getColumnType(column.columnName()).orElse(null);
+
+ // DimensionRangeShardSpec only handles strings.
+ if (!(ColumnType.STRING.equals(columnType))) {
+ return Collections.emptyList();
+ }
+
+ // DimensionRangeShardSpec only handles columns that appear as-is in the output.
+ if (outputColumns.isEmpty()) {
+ return Collections.emptyList();
+ }
+
+ shardColumns.add(outputColumns.get(0));
+ }
+
+ return shardColumns;
+ }
+
+ /**
+ * Checks if the {@link ClusterBy} has a {@link QueryKitUtils#PARTITION_BOOST_COLUMN}. See javadocs for that
+ * constant for more details about what it does.
+ */
+ private static boolean isClusterByBoosted(final ClusterBy clusterBy)
+ {
+ return !clusterBy.getColumns().isEmpty()
+ && clusterBy.getColumns()
+ .get(clusterBy.getColumns().size() - 1)
+ .columnName()
+ .equals(QueryKitUtils.PARTITION_BOOST_COLUMN);
+ }
+
+ private static StringTuple makeStringTuple(
+ final ClusterBy clusterBy,
+ final RowKeyReader keyReader,
+ final RowKey key
+ )
+ {
+ final String[] array = new String[clusterBy.getColumns().size() - clusterBy.getBucketByCount()];
+ final boolean boosted = isClusterByBoosted(clusterBy);
+
+ for (int i = 0; i < array.length; i++) {
+ final Object val = keyReader.read(key, clusterBy.getBucketByCount() + i);
+
+ if (i == array.length - 1 && boosted) {
+ // Boost column
+ //noinspection RedundantCast: false alarm; the cast is necessary
+ array[i] = StringUtils.format("%016d", (long) val);
+ } else {
+ array[i] = (String) val;
+ }
+ }
+
+ return new StringTuple(array);
+ }
+
+ private static Pair, List> makeDimensionsAndAggregatorsForIngestion(
+ final RowSignature querySignature,
+ final ClusterBy queryClusterBy,
+ final List segmentSortOrder,
+ final ColumnMappings columnMappings,
+ final boolean isRollupQuery,
+ final Query> query
+ )
+ {
+ final List dimensions = new ArrayList<>();
+ final List aggregators = new ArrayList<>();
+
+ // During ingestion, segment sort order is determined by the order of fields in the DimensionsSchema. We want
+ // this to match user intent as dictated by the declared segment sort order and CLUSTERED BY, so add things in
+ // that order.
+
+ // Start with segmentSortOrder.
+ final Set outputColumnsInOrder = new LinkedHashSet<>(segmentSortOrder);
+
+ // Then the query-level CLUSTERED BY.
+ // Note: this doesn't work when CLUSTERED BY specifies an expression that is not being selected.
+ // Such fields in CLUSTERED BY still control partitioning as expected, but do not affect sort order of rows
+ // within an individual segment.
+ for (final SortColumn clusterByColumn : queryClusterBy.getColumns()) {
+ if (clusterByColumn.descending()) {
+ throw new MSQException(new InsertCannotOrderByDescendingFault(clusterByColumn.columnName()));
+ }
+
+ outputColumnsInOrder.addAll(columnMappings.getOutputColumnsForQueryColumn(clusterByColumn.columnName()));
+ }
+
+ // Then all other columns.
+ outputColumnsInOrder.addAll(columnMappings.getOutputColumnNames());
+
+ Map outputColumnAggregatorFactories = new HashMap<>();
+
+ if (isRollupQuery) {
+ // Populate aggregators from the native query when doing an ingest in rollup mode.
+ for (AggregatorFactory aggregatorFactory : ((GroupByQuery) query).getAggregatorSpecs()) {
+ String outputColumn = Iterables.getOnlyElement(columnMappings.getOutputColumnsForQueryColumn(aggregatorFactory.getName()));
+ if (outputColumnAggregatorFactories.containsKey(outputColumn)) {
+ throw new ISE("There can only be one aggregator factory for column [%s].", outputColumn);
+ } else {
+ outputColumnAggregatorFactories.put(
+ outputColumn,
+ aggregatorFactory.withName(outputColumn).getCombiningFactory()
+ );
+ }
+ }
+ }
+
+ // Each column can be of either time, dimension, aggregator. For this method. we can ignore the time column.
+ // For non-complex columns, If the aggregator factory of the column is not available, we treat the column as
+ // a dimension. For complex columns, certains hacks are in place.
+ for (final String outputColumn : outputColumnsInOrder) {
+ final String queryColumn = columnMappings.getQueryColumnForOutputColumn(outputColumn);
+ final ColumnType type =
+ querySignature.getColumnType(queryColumn)
+ .orElseThrow(() -> new ISE("No type for column [%s]", outputColumn));
+
+ if (!outputColumn.equals(ColumnHolder.TIME_COLUMN_NAME)) {
+
+ if (!type.is(ValueType.COMPLEX)) {
+ // non complex columns
+ populateDimensionsAndAggregators(
+ dimensions,
+ aggregators,
+ outputColumnAggregatorFactories,
+ outputColumn,
+ type
+ );
+ } else {
+ // complex columns only
+ if (DimensionHandlerUtils.DIMENSION_HANDLER_PROVIDERS.containsKey(type.getComplexTypeName())) {
+ dimensions.add(DimensionSchemaUtils.createDimensionSchema(outputColumn, type));
+ } else if (!isRollupQuery) {
+ aggregators.add(new PassthroughAggregatorFactory(outputColumn, type.getComplexTypeName()));
+ } else {
+ populateDimensionsAndAggregators(
+ dimensions,
+ aggregators,
+ outputColumnAggregatorFactories,
+ outputColumn,
+ type
+ );
+ }
+ }
+ }
+ }
+
+ return Pair.of(dimensions, aggregators);
+ }
+
+
+ /**
+ * If the output column is present in the outputColumnAggregatorFactories that means we already have the aggregator information for this column.
+ * else treat this column as a dimension.
+ *
+ * @param dimensions list is poulated if the output col is deemed to be a dimension
+ * @param aggregators list is populated with the aggregator if the output col is deemed to be a aggregation column.
+ * @param outputColumnAggregatorFactories output col -> AggregatorFactory map
+ * @param outputColumn column name
+ * @param type columnType
+ */
+ private static void populateDimensionsAndAggregators(
+ List dimensions,
+ List aggregators,
+ Map outputColumnAggregatorFactories,
+ String outputColumn,
+ ColumnType type
+ )
+ {
+ if (outputColumnAggregatorFactories.containsKey(outputColumn)) {
+ aggregators.add(outputColumnAggregatorFactories.get(outputColumn));
+ } else {
+ dimensions.add(DimensionSchemaUtils.createDimensionSchema(outputColumn, type));
+ }
+ }
+
+ private static DateTime getBucketDateTime(
+ final ClusterByPartition partitionBoundary,
+ final Granularity segmentGranularity,
+ final RowKeyReader keyReader
+ )
+ {
+ if (Granularities.ALL.equals(segmentGranularity)) {
+ return DateTimes.utc(0);
+ } else {
+ final RowKey startKey = partitionBoundary.getStart();
+ final DateTime timestamp =
+ DateTimes.utc(MSQTasks.primaryTimestampFromObjectForInsert(keyReader.read(startKey, 0)));
+
+ if (segmentGranularity.bucketStart(timestamp.getMillis()) != timestamp.getMillis()) {
+ // It's a bug in... something? if this happens.
+ throw new ISE(
+ "Received boundary value [%s] misaligned with segmentGranularity [%s]",
+ timestamp,
+ segmentGranularity
+ );
+ }
+
+ return timestamp;
+ }
+ }
+
+ private static MSQStagesReport makeStageReport(
+ final QueryDefinition queryDef,
+ final Map stagePhaseMap,
+ final Map stageRuntimeMap,
+ final Map stageWorkerCountMap,
+ final Map stagePartitionCountMap
+ )
+ {
+ return MSQStagesReport.create(
+ queryDef,
+ ImmutableMap.copyOf(stagePhaseMap),
+ copyOfStageRuntimesEndingAtCurrentTime(stageRuntimeMap),
+ stageWorkerCountMap,
+ stagePartitionCountMap
+ );
+ }
+
+ private static MSQResultsReport makeResultsTaskReport(
+ final QueryDefinition queryDef,
+ final Yielder resultsYielder,
+ final ColumnMappings columnMappings,
+ @Nullable final List sqlTypeNames
+ )
+ {
+ final RowSignature querySignature = queryDef.getFinalStageDefinition().getSignature();
+ final RowSignature.Builder mappedSignature = RowSignature.builder();
+
+ for (final ColumnMapping mapping : columnMappings.getMappings()) {
+ mappedSignature.add(
+ mapping.getOutputColumn(),
+ querySignature.getColumnType(mapping.getQueryColumn()).orElse(null)
+ );
+ }
+
+ return new MSQResultsReport(mappedSignature.build(), sqlTypeNames, resultsYielder);
+ }
+
+ private static MSQStatusReport makeStatusReport(
+ final TaskState taskState,
+ @Nullable final MSQErrorReport errorReport,
+ final Queue errorReports,
+ @Nullable final DateTime queryStartTime,
+ final long queryDuration
+ )
+ {
+ return new MSQStatusReport(taskState, errorReport, errorReports, queryStartTime, queryDuration);
+ }
+
+ private static InputSpecSlicerFactory makeInputSpecSlicerFactory(final DataSegmentTimelineView timelineView)
+ {
+ return stagePartitionsMap -> new MapInputSpecSlicer(
+ ImmutableMap., InputSpecSlicer>builder()
+ .put(StageInputSpec.class, new StageInputSpecSlicer(stagePartitionsMap))
+ .put(ExternalInputSpec.class, new ExternalInputSpecSlicer())
+ .put(TableInputSpec.class, new TableInputSpecSlicer(timelineView))
+ .build()
+ );
+ }
+
+ private static Map copyOfStageRuntimesEndingAtCurrentTime(
+ final Map stageRuntimesMap
+ )
+ {
+ final Int2ObjectMap retVal = new Int2ObjectOpenHashMap<>(stageRuntimesMap.size());
+ final DateTime now = DateTimes.nowUtc();
+
+ for (Map.Entry entry : stageRuntimesMap.entrySet()) {
+ final int stageNumber = entry.getKey();
+ final Interval interval = entry.getValue();
+
+ retVal.put(
+ stageNumber,
+ interval.getEnd().equals(DateTimes.MAX) ? new Interval(interval.getStart(), now) : interval
+ );
+ }
+
+ return retVal;
+ }
+
+ /**
+ * Method that determines whether an exception was raised due to the task lock for the controller task being
+ * preempted. Uses string comparison, because the relevant Overlord APIs do not have a more reliable way of
+ * discerning the cause of errors.
+ *
+ * Error strings are taken from {@link org.apache.druid.indexing.common.actions.TaskLocks}
+ * and {@link SegmentAllocateAction}.
+ */
+ private static boolean isTaskLockPreemptedException(Exception e)
+ {
+ final String exceptionMsg = e.getMessage();
+ final List validExceptionExcerpts = ImmutableList.of(
+ "are not covered by locks" /* From TaskLocks */,
+ "is preempted and no longer valid" /* From SegmentAllocateAction */
+ );
+ return validExceptionExcerpts.stream().anyMatch(exceptionMsg::contains);
+ }
+
+ private static void logKernelStatus(final String queryId, final ControllerQueryKernel queryKernel)
+ {
+ if (log.isDebugEnabled()) {
+ log.debug(
+ "Query [%s] kernel state: %s",
+ queryId,
+ queryKernel.getActiveStages()
+ .stream()
+ .sorted(Comparator.comparing(id -> queryKernel.getStageDefinition(id).getStageNumber()))
+ .map(id -> StringUtils.format(
+ "%d:%d[%s:%s]>%s",
+ queryKernel.getStageDefinition(id).getStageNumber(),
+ queryKernel.getWorkerInputsForStage(id).workerCount(),
+ queryKernel.getStageDefinition(id).doesShuffle() ? "SHUFFLE" : "RETAIN",
+ queryKernel.getStagePhase(id),
+ queryKernel.doesStageHaveResultPartitions(id)
+ ? Iterators.size(queryKernel.getResultPartitionsForStage(id).iterator())
+ : "?"
+ )
+ )
+ .collect(Collectors.joining("; "))
+ );
+ }
+ }
+
+ /**
+ * Main controller logic for running a multi-stage query.
+ */
+ private class RunQueryUntilDone
+ {
+ private final QueryDefinition queryDef;
+ private final InputSpecSlicerFactory inputSpecSlicerFactory;
+ private final Closer closer;
+ private final ControllerQueryKernel queryKernel;
+
+ /**
+ * Set of stages that have got their partition boundaries sent out.
+ */
+ private final Set stageResultPartitionBoundariesSent = new HashSet<>();
+
+ /**
+ * Return value of {@link MSQWorkerTaskLauncher#start()}. Set by {@link #startTaskLauncher()}.
+ */
+ private ListenableFuture> workerTaskLauncherFuture;
+
+ /**
+ * Segments to generate. Populated prior to launching the final stage of a query with destination
+ * {@link DataSourceMSQDestination} (which originate from SQL INSERT or REPLACE). The final stage of such a query
+ * uses {@link SegmentGeneratorFrameProcessorFactory}, which requires a list of segment IDs to generate.
+ */
+ private List segmentsToGenerate;
+
+ public RunQueryUntilDone(
+ final QueryDefinition queryDef,
+ final InputSpecSlicerFactory inputSpecSlicerFactory,
+ final Closer closer
+ )
+ {
+ this.queryDef = queryDef;
+ this.inputSpecSlicerFactory = inputSpecSlicerFactory;
+ this.closer = closer;
+ this.queryKernel = new ControllerQueryKernel(queryDef);
+ }
+
+ /**
+ * Primary 'run' method.
+ */
+ private Pair> run() throws IOException, InterruptedException
+ {
+ startTaskLauncher();
+
+ while (!queryKernel.isDone()) {
+ startStages();
+ sendPartitionBoundaries();
+ updateLiveReportMaps();
+ cleanUpEffectivelyFinishedStages();
+ runKernelCommands();
+ }
+
+ if (!queryKernel.isSuccess()) {
+ throwKernelExceptionIfNotUnknown();
+ }
+
+ cleanUpEffectivelyFinishedStages();
+ return Pair.of(queryKernel, workerTaskLauncherFuture);
+ }
+
+ /**
+ * Run at least one command from {@link #kernelManipulationQueue}, waiting for it if necessary.
+ */
+ private void runKernelCommands() throws InterruptedException
+ {
+ if (!queryKernel.isDone()) {
+ // Run the next command, waiting for it if necessary.
+ Consumer command = kernelManipulationQueue.take();
+ command.accept(queryKernel);
+
+ // Run all pending commands after that one. Helps avoid deep queues.
+ // After draining the command queue, move on to the next iteration of the controller loop.
+ while ((command = kernelManipulationQueue.poll()) != null) {
+ command.accept(queryKernel);
+ }
+ }
+ }
+
+ /**
+ * Start up the {@link MSQWorkerTaskLauncher}, such that later on it can be used to launch new tasks
+ * via {@link MSQWorkerTaskLauncher#launchTasksIfNeeded}.
+ */
+ private void startTaskLauncher()
+ {
+ // Start tasks.
+ log.debug("Query [%s] starting task launcher.", queryDef.getQueryId());
+
+ workerTaskLauncherFuture = workerTaskLauncher.start();
+ closer.register(() -> workerTaskLauncher.stop(true));
+
+ workerTaskLauncherFuture.addListener(
+ () ->
+ addToKernelManipulationQueue(queryKernel -> {
+ // Throw an exception in the main loop, if anything went wrong.
+ FutureUtils.getUncheckedImmediately(workerTaskLauncherFuture);
+ }),
+ Execs.directExecutor()
+ );
+ }
+
+ /**
+ * Start up any stages that are ready to start.
+ */
+ private void startStages() throws IOException, InterruptedException
+ {
+ logKernelStatus(queryDef.getQueryId(), queryKernel);
+ final List newStageIds = queryKernel.createAndGetNewStageIds(
+ inputSpecSlicerFactory,
+ task.getQuerySpec().getAssignmentStrategy()
+ );
+
+ for (final StageId stageId : newStageIds) {
+ queryKernel.startStage(stageId);
+
+ // Allocate segments, if this is the final stage of an ingestion.
+ if (MSQControllerTask.isIngestion(task.getQuerySpec())
+ && stageId.getStageNumber() == queryDef.getFinalStageDefinition().getStageNumber()) {
+ // We need to find the shuffle details (like partition ranges) to generate segments. Generally this is
+ // going to correspond to the stage immediately prior to the final segment-generator stage.
+ int shuffleStageNumber = Iterables.getOnlyElement(queryDef.getFinalStageDefinition().getInputStageNumbers());
+
+ // The following logic assumes that output of all the stages without a shuffle retain the partition boundaries
+ // of the input to that stage. This may not always be the case. For example: GROUP BY queries without an
+ // ORDER BY clause. This works for QueryKit generated queries up until now, but it should be reworked as it
+ // might not always be the case.
+ while (!queryDef.getStageDefinition(shuffleStageNumber).doesShuffle()) {
+ shuffleStageNumber =
+ Iterables.getOnlyElement(queryDef.getStageDefinition(shuffleStageNumber).getInputStageNumbers());
+ }
+
+ final StageId shuffleStageId = new StageId(queryDef.getQueryId(), shuffleStageNumber);
+ final boolean isTimeBucketed = isTimeBucketedIngestion(task.getQuerySpec());
+ final ClusterByPartitions partitionBoundaries =
+ queryKernel.getResultPartitionBoundariesForStage(shuffleStageId);
+
+ // We require some data to be inserted in case it is partitioned by anything other than all and we are
+ // inserting everything into a single bucket. This can be handled more gracefully instead of throwing an exception
+ // Note: This can also be the case when we have limit queries but validation in Broker SQL layer prevents such
+ // queries
+ if (isTimeBucketed && partitionBoundaries.equals(ClusterByPartitions.oneUniversalPartition())) {
+ throw new MSQException(new InsertCannotBeEmptyFault(task.getDataSource()));
+ } else {
+ log.info("Query [%s] generating %d segments.", queryDef.getQueryId(), partitionBoundaries.size());
+ }
+
+ final boolean mayHaveMultiValuedClusterByFields =
+ !queryKernel.getStageDefinition(shuffleStageId).mustGatherResultKeyStatistics()
+ || queryKernel.hasStageCollectorEncounteredAnyMultiValueField(shuffleStageId);
+
+ segmentsToGenerate = generateSegmentIdsWithShardSpecs(
+ (DataSourceMSQDestination) task.getQuerySpec().getDestination(),
+ queryKernel.getStageDefinition(shuffleStageId).getSignature(),
+ queryKernel.getStageDefinition(shuffleStageId).getShuffleSpec().get().getClusterBy(),
+ partitionBoundaries,
+ mayHaveMultiValuedClusterByFields
+ );
+ }
+
+ final int workerCount = queryKernel.getWorkerInputsForStage(stageId).workerCount();
+ log.info(
+ "Query [%s] starting %d workers for stage %d.",
+ stageId.getQueryId(),
+ workerCount,
+ stageId.getStageNumber()
+ );
+
+ workerTaskLauncher.launchTasksIfNeeded(workerCount);
+ stageRuntimesForLiveReports.put(stageId.getStageNumber(), new Interval(DateTimes.nowUtc(), DateTimes.MAX));
+ startWorkForStage(queryDef, queryKernel, stageId.getStageNumber(), segmentsToGenerate);
+ }
+ }
+
+ /**
+ * Send partition boundaries to any stages that are ready to receive partition boundaries.
+ */
+ private void sendPartitionBoundaries()
+ {
+ logKernelStatus(queryDef.getQueryId(), queryKernel);
+ for (final StageId stageId : queryKernel.getActiveStages()) {
+
+ if (queryKernel.getStageDefinition(stageId).mustGatherResultKeyStatistics()
+ && queryKernel.doesStageHaveResultPartitions(stageId)
+ && stageResultPartitionBoundariesSent.add(stageId)) {
+ if (log.isDebugEnabled()) {
+ final ClusterByPartitions partitions = queryKernel.getResultPartitionBoundariesForStage(stageId);
+ log.debug(
+ "Query [%s] sending out partition boundaries for stage %d: %s",
+ stageId.getQueryId(),
+ stageId.getStageNumber(),
+ IntStream.range(0, partitions.size())
+ .mapToObj(i -> StringUtils.format("%s:%s", i, partitions.get(i)))
+ .collect(Collectors.joining(", "))
+ );
+ } else {
+ log.info(
+ "Query [%s] sending out partition boundaries for stage %d.",
+ stageId.getQueryId(),
+ stageId.getStageNumber()
+ );
+ }
+
+ postResultPartitionBoundariesForStage(
+ queryDef,
+ stageId.getStageNumber(),
+ queryKernel.getResultPartitionBoundariesForStage(stageId),
+ queryKernel.getWorkerInputsForStage(stageId).workers()
+ );
+ }
+ }
+ }
+
+ /**
+ * Update the various maps used for live reports.
+ */
+ private void updateLiveReportMaps()
+ {
+ logKernelStatus(queryDef.getQueryId(), queryKernel);
+
+ // Live reports: update stage phases, worker counts, partition counts.
+ for (StageId stageId : queryKernel.getActiveStages()) {
+ final int stageNumber = stageId.getStageNumber();
+ stagePhasesForLiveReports.put(stageNumber, queryKernel.getStagePhase(stageId));
+
+ if (queryKernel.doesStageHaveResultPartitions(stageId)) {
+ stagePartitionCountsForLiveReports.computeIfAbsent(
+ stageNumber,
+ k -> Iterators.size(queryKernel.getResultPartitionsForStage(stageId).iterator())
+ );
+ }
+
+ stageWorkerCountsForLiveReports.putIfAbsent(
+ stageNumber,
+ queryKernel.getWorkerInputsForStage(stageId).workerCount()
+ );
+ }
+
+ // Live reports: update stage end times for any stages that just ended.
+ for (StageId stageId : queryKernel.getActiveStages()) {
+ if (ControllerStagePhase.isSuccessfulTerminalPhase(queryKernel.getStagePhase(stageId))) {
+ stageRuntimesForLiveReports.compute(
+ queryKernel.getStageDefinition(stageId).getStageNumber(),
+ (k, currentValue) -> {
+ if (currentValue.getEnd().equals(DateTimes.MAX)) {
+ return new Interval(currentValue.getStart(), DateTimes.nowUtc());
+ } else {
+ return currentValue;
+ }
+ }
+ );
+ }
+ }
+ }
+
+ /**
+ * Issue cleanup commands to any stages that are effectivley finished, allowing them to delete their outputs.
+ */
+ private void cleanUpEffectivelyFinishedStages()
+ {
+ for (final StageId stageId : queryKernel.getEffectivelyFinishedStageIds()) {
+ log.info("Query [%s] issuing cleanup order for stage %d.", queryDef.getQueryId(), stageId.getStageNumber());
+ contactWorkersForStage(
+ (netClient, taskId, workerNumber) -> netClient.postCleanupStage(taskId, stageId),
+ queryKernel.getWorkerInputsForStage(stageId).workers()
+ );
+ queryKernel.finishStage(stageId, true);
+ }
+ }
+
+ /**
+ * Throw {@link MSQException} if the kernel method {@link ControllerQueryKernel#getFailureReasonForStage}
+ * has any failure reason other than {@link UnknownFault}.
+ */
+ private void throwKernelExceptionIfNotUnknown()
+ {
+ for (final StageId stageId : queryKernel.getActiveStages()) {
+ if (queryKernel.getStagePhase(stageId) == ControllerStagePhase.FAILED) {
+ final MSQFault fault = queryKernel.getFailureReasonForStage(stageId);
+
+ // Fall through (without throwing an exception) in case of UnknownFault; we may be able to generate
+ // a better exception later in query teardown.
+ if (!UnknownFault.CODE.equals(fault.getErrorCode())) {
+ throw new MSQException(fault);
+ }
+ }
+ }
+ }
+ }
+
+ /**
+ * Interface used by {@link #contactWorkersForStage}.
+ */
+ private interface TaskContactFn
+ {
+ ListenableFuture contactTask(WorkerClient client, String taskId, int workerNumber);
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExceptionWrappingWorkerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExceptionWrappingWorkerClient.java
new file mode 100644
index 00000000000..1c3cc39987c
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExceptionWrappingWorkerClient.java
@@ -0,0 +1,145 @@
+/*
+ * 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.exec;
+
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.SettableFuture;
+import org.apache.druid.frame.channel.ReadableByteChunksFrameChannel;
+import org.apache.druid.frame.key.ClusterByPartitions;
+import org.apache.druid.msq.counters.CounterSnapshotsTree;
+import org.apache.druid.msq.indexing.error.MSQException;
+import org.apache.druid.msq.indexing.error.WorkerRpcFailedFault;
+import org.apache.druid.msq.kernel.StageId;
+import org.apache.druid.msq.kernel.WorkOrder;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+
+/**
+ * Wrapper around any {@link WorkerClient} that converts exceptions into {@link MSQException}
+ * with {@link WorkerRpcFailedFault}. Useful so each implementation of WorkerClient does not need to do this on
+ * its own.
+ */
+public class ExceptionWrappingWorkerClient implements WorkerClient
+{
+ private final WorkerClient client;
+
+ public ExceptionWrappingWorkerClient(final WorkerClient client)
+ {
+ this.client = Preconditions.checkNotNull(client, "client");
+ }
+
+ @Override
+ public ListenableFuture postWorkOrder(String workerTaskId, WorkOrder workOrder)
+ {
+ return wrap(workerTaskId, client, c -> c.postWorkOrder(workerTaskId, workOrder));
+ }
+
+ @Override
+ public ListenableFuture postResultPartitionBoundaries(
+ final String workerTaskId,
+ final StageId stageId,
+ final ClusterByPartitions partitionBoundaries
+ )
+ {
+ return wrap(workerTaskId, client, c -> c.postResultPartitionBoundaries(workerTaskId, stageId, partitionBoundaries));
+ }
+
+ @Override
+ public ListenableFuture postCleanupStage(String workerTaskId, StageId stageId)
+ {
+ return wrap(workerTaskId, client, c -> c.postCleanupStage(workerTaskId, stageId));
+ }
+
+ @Override
+ public ListenableFuture postFinish(String workerTaskId)
+ {
+ return wrap(workerTaskId, client, c -> c.postFinish(workerTaskId));
+ }
+
+ @Override
+ public ListenableFuture getCounters(String workerTaskId)
+ {
+ return wrap(workerTaskId, client, c -> c.getCounters(workerTaskId));
+ }
+
+ @Override
+ public ListenableFuture fetchChannelData(
+ String workerTaskId,
+ StageId stageId,
+ int partitionNumber,
+ long offset,
+ ReadableByteChunksFrameChannel channel
+ )
+ {
+ return wrap(workerTaskId, client, c -> c.fetchChannelData(workerTaskId, stageId, partitionNumber, offset, channel));
+ }
+
+ @Override
+ public void close() throws IOException
+ {
+ client.close();
+ }
+
+ private static ListenableFuture wrap(
+ final String workerTaskId,
+ final WorkerClient client,
+ final ClientFn clientFn
+ )
+ {
+ final SettableFuture retVal = SettableFuture.create();
+ final ListenableFuture clientFuture;
+
+ try {
+ clientFuture = clientFn.apply(client);
+ }
+ catch (Exception e) {
+ throw new MSQException(e, new WorkerRpcFailedFault(workerTaskId));
+ }
+
+ Futures.addCallback(
+ clientFuture,
+ new FutureCallback()
+ {
+ @Override
+ public void onSuccess(@Nullable T result)
+ {
+ retVal.set(result);
+ }
+
+ @Override
+ public void onFailure(Throwable t)
+ {
+ retVal.setException(new MSQException(t, new WorkerRpcFailedFault(workerTaskId)));
+ }
+ }
+ );
+
+ return retVal;
+ }
+
+ private interface ClientFn
+ {
+ ListenableFuture apply(WorkerClient client);
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java
new file mode 100644
index 00000000000..ac5cf95d1dd
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java
@@ -0,0 +1,61 @@
+/*
+ * 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.exec;
+
+public class Limits
+{
+ /**
+ * Maximum number of columns that can appear in a frame signature.
+ *
+ * Somewhat less than {@link WorkerMemoryParameters#STANDARD_FRAME_SIZE} divided by typical minimum column size:
+ * {@link org.apache.druid.frame.allocation.AppendableMemory#DEFAULT_INITIAL_ALLOCATION_SIZE}.
+ */
+ public static final int MAX_FRAME_COLUMNS = 2000;
+
+ /**
+ * Maximum number of workers that can be used in a stage, regardless of available memory.
+ */
+ public static final int MAX_WORKERS = 1000;
+
+ /**
+ * Maximum number of input files per worker
+ */
+ public static final int MAX_INPUT_FILES_PER_WORKER = 10_000;
+
+ /**
+ * Maximum number of parse exceptions with their stack traces a worker can send to the controller.
+ */
+ public static final long MAX_VERBOSE_PARSE_EXCEPTIONS = 5;
+
+ /**
+ * Maximum number of warnings with their stack traces a worker can send to the controller.
+ */
+ public static final long MAX_VERBOSE_WARNINGS = 10;
+
+ /**
+ * Maximum number of input bytes per worker in case number of tasks is determined automatically.
+ */
+ public static final long MAX_INPUT_BYTES_PER_WORKER = 10 * 1024 * 1024 * 1024L;
+
+ /**
+ * Maximum size of the kernel manipulation queue in {@link org.apache.druid.msq.indexing.MSQControllerTask}.
+ */
+ public static final int MAX_KERNEL_MANIPULATION_QUEUE_SIZE = 100_000;
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/MSQTasks.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/MSQTasks.java
new file mode 100644
index 00000000000..fa1371f132e
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/MSQTasks.java
@@ -0,0 +1,216 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.msq.exec;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.inject.Injector;
+import com.google.inject.Key;
+import org.apache.druid.frame.key.ClusterBy;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.msq.guice.MultiStageQuery;
+import org.apache.druid.msq.indexing.error.CanceledFault;
+import org.apache.druid.msq.indexing.error.DurableStorageConfigurationFault;
+import org.apache.druid.msq.indexing.error.InsertTimeNullFault;
+import org.apache.druid.msq.indexing.error.MSQErrorReport;
+import org.apache.druid.msq.indexing.error.MSQException;
+import org.apache.druid.msq.indexing.error.MSQFault;
+import org.apache.druid.msq.indexing.error.UnknownFault;
+import org.apache.druid.msq.indexing.error.WorkerFailedFault;
+import org.apache.druid.msq.indexing.error.WorkerRpcFailedFault;
+import org.apache.druid.msq.statistics.KeyCollectorFactory;
+import org.apache.druid.msq.statistics.KeyCollectorSnapshot;
+import org.apache.druid.msq.statistics.KeyCollectorSnapshotDeserializerModule;
+import org.apache.druid.msq.statistics.KeyCollectors;
+import org.apache.druid.segment.column.ColumnHolder;
+import org.apache.druid.server.DruidNode;
+import org.apache.druid.storage.StorageConnector;
+
+import javax.annotation.Nullable;
+import java.util.UUID;
+
+public class MSQTasks
+{
+ /**
+ * Message used by {@link #makeErrorReport} when no other message is known.
+ */
+ static final String GENERIC_QUERY_FAILED_MESSAGE = "Query failed";
+
+ private static final String TASK_ID_PREFIX = "query-";
+
+ /**
+ * Returns a controller task ID given a SQL query id.
+ */
+ public static String controllerTaskId(@Nullable final String queryId)
+ {
+ return TASK_ID_PREFIX + (queryId == null ? UUID.randomUUID().toString() : queryId);
+ }
+
+ /**
+ * Returns a worker task ID given a SQL query id.
+ */
+ public static String workerTaskId(final String controllerTaskId, final int workerNumber)
+ {
+ return StringUtils.format("%s-worker%d", controllerTaskId, workerNumber);
+ }
+
+ /**
+ * If "Object" is a Long, returns it. Otherwise, throws an appropriate exception assuming this operation is
+ * being done to read the primary timestamp (__time) as part of an INSERT.
+ */
+ public static long primaryTimestampFromObjectForInsert(final Object timestamp)
+ {
+ if (timestamp instanceof Long) {
+ return (long) timestamp;
+ } else if (timestamp == null) {
+ throw new MSQException(InsertTimeNullFault.INSTANCE);
+ } else {
+ // Normally we expect the SQL layer to validate that __time for INSERT is a TIMESTAMP type, which would
+ // be a long at execution time. So a nice user-friendly message isn't needed here: it would only happen
+ // if the SQL layer is bypassed. Nice, friendly users wouldn't do that :)
+ final UnknownFault fault =
+ UnknownFault.forMessage(StringUtils.format("Incorrect type for [%s]", ColumnHolder.TIME_COLUMN_NAME));
+ throw new MSQException(fault);
+ }
+ }
+
+ /**
+ * Returns a decorated copy of an ObjectMapper that knows how to deserialize the appropriate kind of
+ * {@link KeyCollectorSnapshot}.
+ */
+ static ObjectMapper decorateObjectMapperForKeyCollectorSnapshot(
+ final ObjectMapper mapper,
+ final ClusterBy clusterBy,
+ final boolean aggregate
+ )
+ {
+ final KeyCollectorFactory, ?> keyCollectorFactory =
+ KeyCollectors.makeStandardFactory(clusterBy, aggregate);
+
+ final ObjectMapper mapperCopy = mapper.copy();
+ mapperCopy.registerModule(new KeyCollectorSnapshotDeserializerModule(keyCollectorFactory));
+ return mapperCopy;
+ }
+
+ /**
+ * Returns the host:port from a {@link DruidNode}. Convenience method to make it easier to construct
+ * {@link MSQErrorReport} instances.
+ */
+ @Nullable
+ static String getHostFromSelfNode(@Nullable final DruidNode selfNode)
+ {
+ return selfNode != null ? selfNode.getHostAndPortToUse() : null;
+ }
+
+ static StorageConnector makeStorageConnector(final Injector injector)
+ {
+ try {
+ return injector.getInstance(Key.get(StorageConnector.class, MultiStageQuery.class));
+ }
+ catch (Exception e) {
+ throw new MSQException(new DurableStorageConfigurationFault(e.toString()));
+ }
+ }
+
+ /**
+ * Builds an error report from a possible controller error report and a possible worker error report. Both may be
+ * null, in which case this function will return a report with {@link UnknownFault}.
+ *
+ * We only include a single {@link MSQErrorReport} in the task report, because it's important that a query have
+ * a single {@link MSQFault} explaining why it failed. To aid debugging
+ * in cases where we choose the controller error over the worker error, we'll log the worker error too, even though
+ * it doesn't appear in the report.
+ *
+ * Logic: we prefer the controller exception unless it's {@link WorkerFailedFault}, {@link WorkerRpcFailedFault},
+ * or {@link CanceledFault}. In these cases we prefer the worker error report. This ensures we get the best, most
+ * useful exception even when the controller cancels worker tasks after a failure. (As tasks are canceled one by
+ * one, worker -> worker and controller -> worker RPCs to the canceled tasks will fail. We want to ignore these
+ * failed RPCs and get to the "true" error that started it all.)
+ */
+ static MSQErrorReport makeErrorReport(
+ final String controllerTaskId,
+ final String controllerHost,
+ @Nullable MSQErrorReport controllerErrorReport,
+ @Nullable MSQErrorReport workerErrorReport
+ )
+ {
+ if (controllerErrorReport == null && workerErrorReport == null) {
+ // Something went wrong, but we have no idea what.
+ return MSQErrorReport.fromFault(
+ controllerTaskId,
+ controllerHost,
+ null,
+ UnknownFault.forMessage(GENERIC_QUERY_FAILED_MESSAGE)
+ );
+ } else if (controllerErrorReport == null) {
+ // workerErrorReport is nonnull.
+ return workerErrorReport;
+ } else {
+ // controllerErrorReport is nonnull.
+
+ // Pick the "best" error if both are set. See the javadoc for the logic we use. In these situations, we
+ // expect the caller to also log the other one. (There is no logging in _this_ method, because it's a helper
+ // function, and it's best if helper functions run quietly.)
+ if (workerErrorReport != null && (controllerErrorReport.getFault() instanceof WorkerFailedFault
+ || controllerErrorReport.getFault() instanceof WorkerRpcFailedFault
+ || controllerErrorReport.getFault() instanceof CanceledFault)) {
+ return workerErrorReport;
+ } else {
+ return controllerErrorReport;
+ }
+ }
+ }
+
+ /**
+ * Returns a string form of a {@link MSQErrorReport} suitable for logging.
+ */
+ static String errorReportToLogMessage(final MSQErrorReport errorReport)
+ {
+ final StringBuilder logMessage = new StringBuilder("Work failed");
+
+ if (errorReport.getStageNumber() != null) {
+ logMessage.append("; stage ").append(errorReport.getStageNumber());
+ }
+
+ logMessage.append("; task ").append(errorReport.getTaskId());
+
+ if (errorReport.getHost() != null) {
+ logMessage.append("; host ").append(errorReport.getHost());
+ }
+
+ logMessage.append(": ").append(errorReport.getFault().getCodeWithMessage());
+
+ if (errorReport.getExceptionStackTrace() != null) {
+ if (errorReport.getFault() instanceof UnknownFault) {
+ // Log full stack trace for unknown faults.
+ logMessage.append('\n').append(errorReport.getExceptionStackTrace());
+ } else {
+ // Log first line only (error class, message) for known faults, to avoid polluting logs.
+ final String stackTrace = errorReport.getExceptionStackTrace();
+ final int firstNewLine = stackTrace.indexOf('\n');
+
+ logMessage.append(" (")
+ .append(firstNewLine > 0 ? stackTrace.substring(0, firstNewLine) : stackTrace)
+ .append(")");
+ }
+ }
+
+ return logMessage.toString();
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/QueryValidator.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/QueryValidator.java
new file mode 100644
index 00000000000..87980130d23
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/QueryValidator.java
@@ -0,0 +1,77 @@
+/*
+ * 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.exec;
+
+import com.google.common.math.IntMath;
+import com.google.common.primitives.Ints;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.msq.indexing.error.MSQException;
+import org.apache.druid.msq.indexing.error.TooManyColumnsFault;
+import org.apache.druid.msq.indexing.error.TooManyInputFilesFault;
+import org.apache.druid.msq.indexing.error.TooManyWorkersFault;
+import org.apache.druid.msq.input.InputSlice;
+import org.apache.druid.msq.kernel.QueryDefinition;
+import org.apache.druid.msq.kernel.StageDefinition;
+import org.apache.druid.msq.kernel.WorkOrder;
+
+import java.math.RoundingMode;
+
+public class QueryValidator
+{
+ /**
+ * Validate that a {@link QueryDefinition} falls within the {@link Limits#MAX_FRAME_COLUMNS} and
+ * {@link Limits#MAX_WORKERS} limits.
+ */
+ public static void validateQueryDef(final QueryDefinition queryDef)
+ {
+ for (final StageDefinition stageDef : queryDef.getStageDefinitions()) {
+ final int numColumns = stageDef.getSignature().size();
+
+ if (numColumns > Limits.MAX_FRAME_COLUMNS) {
+ throw new MSQException(new TooManyColumnsFault(numColumns, Limits.MAX_FRAME_COLUMNS));
+ }
+
+ final int numWorkers = stageDef.getMaxWorkerCount();
+ if (numWorkers > Limits.MAX_WORKERS) {
+ throw new MSQException(new TooManyWorkersFault(numWorkers, Limits.MAX_WORKERS));
+ } else if (numWorkers <= 0) {
+ throw new ISE("Number of workers must be greater than 0");
+ }
+ }
+ }
+
+ /**
+ * Validate that a {@link WorkOrder} falls within the {@link Limits#MAX_INPUT_FILES_PER_WORKER} limit.
+ */
+ public static void validateWorkOrder(final WorkOrder order)
+ {
+ final int numInputFiles = Ints.checkedCast(order.getInputs().stream().mapToLong(InputSlice::fileCount).sum());
+
+ if (numInputFiles > Limits.MAX_INPUT_FILES_PER_WORKER) {
+ throw new MSQException(
+ new TooManyInputFilesFault(
+ numInputFiles,
+ Limits.MAX_INPUT_FILES_PER_WORKER,
+ IntMath.divide(numInputFiles, Limits.MAX_INPUT_FILES_PER_WORKER, RoundingMode.CEILING)
+ )
+ );
+ }
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/TaskDataSegmentProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/TaskDataSegmentProvider.java
new file mode 100644
index 00000000000..7c4dc2bb09a
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/TaskDataSegmentProvider.java
@@ -0,0 +1,115 @@
+/*
+ * 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.exec;
+
+import org.apache.druid.common.guava.FutureUtils;
+import org.apache.druid.java.util.common.FileUtils;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.RE;
+import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.msq.counters.ChannelCounters;
+import org.apache.druid.msq.querykit.DataSegmentProvider;
+import org.apache.druid.msq.querykit.LazyResourceHolder;
+import org.apache.druid.msq.rpc.CoordinatorServiceClient;
+import org.apache.druid.segment.IndexIO;
+import org.apache.druid.segment.QueryableIndex;
+import org.apache.druid.segment.QueryableIndexSegment;
+import org.apache.druid.segment.Segment;
+import org.apache.druid.segment.loading.SegmentCacheManager;
+import org.apache.druid.segment.loading.SegmentLoadingException;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
+import org.apache.druid.utils.CloseableUtils;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.concurrent.ExecutionException;
+
+/**
+ * Production implementation of {@link DataSegmentProvider} using Coordinator APIs.
+ */
+public class TaskDataSegmentProvider implements DataSegmentProvider
+{
+ private final CoordinatorServiceClient coordinatorClient;
+ private final SegmentCacheManager segmentCacheManager;
+ private final IndexIO indexIO;
+
+ public TaskDataSegmentProvider(
+ CoordinatorServiceClient coordinatorClient,
+ SegmentCacheManager segmentCacheManager,
+ IndexIO indexIO
+ )
+ {
+ this.coordinatorClient = coordinatorClient;
+ this.segmentCacheManager = segmentCacheManager;
+ this.indexIO = indexIO;
+ }
+
+ @Override
+ public LazyResourceHolder fetchSegment(
+ final SegmentId segmentId,
+ final ChannelCounters channelCounters
+ )
+ {
+ try {
+ // Use LazyResourceHolder so Coordinator call and segment downloads happen in processing threads,
+ // rather than the main thread.
+ return new LazyResourceHolder<>(
+ () -> {
+ final DataSegment dataSegment;
+ try {
+ dataSegment = FutureUtils.get(
+ coordinatorClient.fetchUsedSegment(
+ segmentId.getDataSource(),
+ segmentId.toString()
+ ),
+ true
+ );
+ }
+ catch (InterruptedException | ExecutionException e) {
+ throw new RE(e, "Failed to fetch segment details from Coordinator for [%s]", segmentId);
+ }
+
+ final Closer closer = Closer.create();
+ try {
+ final File segmentDir = segmentCacheManager.getSegmentFiles(dataSegment);
+ closer.register(() -> FileUtils.deleteDirectory(segmentDir));
+
+ final QueryableIndex index = indexIO.loadIndex(segmentDir);
+ final int numRows = index.getNumRows();
+ final long size = dataSegment.getSize();
+ closer.register(() -> channelCounters.addFile(numRows, size));
+ closer.register(index);
+ return Pair.of(new QueryableIndexSegment(index, dataSegment.getId()), closer);
+ }
+ catch (IOException | SegmentLoadingException e) {
+ throw CloseableUtils.closeInCatch(
+ new RE(e, "Failed to download segment [%s]", segmentId),
+ closer
+ );
+ }
+ }
+ );
+ }
+ catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java
new file mode 100644
index 00000000000..f069c91e145
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java
@@ -0,0 +1,108 @@
+/*
+ * 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.exec;
+
+import org.apache.druid.frame.key.ClusterByPartitions;
+import org.apache.druid.indexer.TaskStatus;
+import org.apache.druid.msq.counters.CounterSnapshotsTree;
+import org.apache.druid.msq.indexing.MSQWorkerTask;
+import org.apache.druid.msq.kernel.StageId;
+import org.apache.druid.msq.kernel.WorkOrder;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.io.InputStream;
+
+public interface Worker
+{
+ /**
+ * Unique ID for this worker.
+ */
+ String id();
+
+ /**
+ * The task which this worker runs.
+ */
+ MSQWorkerTask task();
+
+ /**
+ * Runs the worker in the current thread. Surrounding classes provide
+ * the execution thread.
+ */
+ TaskStatus run() throws Exception;
+
+ /**
+ * Terminate the worker upon a cancellation request.
+ */
+ void stopGracefully();
+
+ /**
+ * Report that the controller has failed. The worker must cease work immediately. Cleanup then exit.
+ * Do not send final messages to the controller: there will be no one home at the other end.
+ */
+ void controllerFailed();
+
+ // Controller-to-worker, and worker-to-worker messages
+
+ /**
+ * Called when the worker chat handler receives a request for a work order. Accepts the work order and schedules it for
+ * execution
+ */
+ void postWorkOrder(WorkOrder workOrder);
+
+ /**
+ * Called when the worker chat handler recieves the result partition boundaries for a particular stageNumber
+ * and queryId
+ */
+ boolean postResultPartitionBoundaries(
+ ClusterByPartitions stagePartitionBoundaries,
+ String queryId,
+ int stageNumber
+ );
+
+ /**
+ * Returns an InputStream of the worker output for a particular queryId, stageNumber and partitionNumber.
+ * Offset indicates the number of bytes to skip the channel data, and is used to prevent re-reading the same data
+ * during retry in case of a connection error
+ *
+ * Returns a null if the workerOutput for a particular queryId, stageNumber, and partitionNumber is not found.
+ *
+ * @throws IOException when the worker output is found but there is an error while reading it.
+ */
+ @Nullable
+ InputStream readChannel(String queryId, int stageNumber, int partitionNumber, long offset) throws IOException;
+
+ /**
+ * Returns the snapshot of the worker counters
+ */
+ CounterSnapshotsTree getCounters();
+
+ /**
+ * Called when the worker receives a POST request to clean up the stage with stageId, and is no longer required.
+ * This marks the stage as FINISHED in its stage kernel, cleans up the worker output for the stage and optionally
+ * frees any resources held on by the worker for the particular stage
+ */
+ void postCleanupStage(StageId stageId);
+
+ /**
+ * Called when the work required for the query has been finished
+ */
+ void postFinish();
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java
new file mode 100644
index 00000000000..72a0a81604d
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java
@@ -0,0 +1,84 @@
+/*
+ * 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.exec;
+
+import com.google.common.util.concurrent.ListenableFuture;
+import org.apache.druid.frame.channel.ReadableByteChunksFrameChannel;
+import org.apache.druid.frame.key.ClusterByPartitions;
+import org.apache.druid.msq.counters.CounterSnapshotsTree;
+import org.apache.druid.msq.kernel.StageId;
+import org.apache.druid.msq.kernel.WorkOrder;
+
+import java.io.IOException;
+
+/**
+ * Client for multi-stage query workers. Used by the controller task.
+ */
+public interface WorkerClient extends AutoCloseable
+{
+ /**
+ * Worker's client method to add a {@link WorkOrder} to the worker to work on
+ */
+ ListenableFuture postWorkOrder(String workerId, WorkOrder workOrder);
+
+ /**
+ * Worker's client method to inform it of the partition boundaries for the given stage. This is usually invoked by the
+ * controller after collating the result statistics from all the workers processing the query
+ */
+ ListenableFuture postResultPartitionBoundaries(
+ String workerTaskId,
+ StageId stageId,
+ ClusterByPartitions partitionBoundaries
+ );
+
+ /**
+ * Worker's client method to inform that the work has been done, and it can initiate cleanup and shutdown
+ */
+ ListenableFuture postFinish(String workerId);
+
+ /**
+ * Fetches all the counters gathered by that worker
+ */
+ ListenableFuture getCounters(String workerId);
+
+ /**
+ * Worker's client method that informs it that the results and resources for the given stage are no longer required
+ * and that they can be cleaned up
+ */
+ ListenableFuture postCleanupStage(String workerTaskId, StageId stageId);
+
+ /**
+ * Fetch some data from a worker and add it to the provided channel. The exact amount of data is determined
+ * by the server.
+ *
+ * Returns a future that resolves to true (no more data left), false (there is more data left), or exception (some
+ * kind of unrecoverable exception).
+ */
+ ListenableFuture fetchChannelData(
+ String workerTaskId,
+ StageId stageId,
+ int partitionNumber,
+ long offset,
+ ReadableByteChunksFrameChannel channel
+ );
+
+ @Override
+ void close() throws IOException;
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerContext.java
new file mode 100644
index 00000000000..6b4a387b8da
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerContext.java
@@ -0,0 +1,76 @@
+/*
+ * 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.exec;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.inject.Injector;
+import org.apache.druid.frame.processor.Bouncer;
+import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.msq.kernel.FrameContext;
+import org.apache.druid.msq.kernel.QueryDefinition;
+import org.apache.druid.server.DruidNode;
+
+import java.io.File;
+
+/**
+ * Context used by multi-stage query workers.
+ *
+ * Useful because it allows test fixtures to provide their own implementations.
+ */
+public interface WorkerContext
+{
+ ObjectMapper jsonMapper();
+
+ // Using an Injector directly because tasks do not have a way to provide their own Guice modules.
+ Injector injector();
+
+ /**
+ * Callback from the worker implementation to "register" the worker. Used in
+ * the indexer to set up the task chat services.
+ */
+ void registerWorker(Worker worker, Closer closer);
+
+ /**
+ * Creates and fetches the controller client for the provided controller ID.
+ */
+ ControllerClient makeControllerClient(String controllerId);
+
+ /**
+ * Creates and fetches a {@link WorkerClient}. It is independent of the workerId because the workerId is passed
+ * in to every method of the client.
+ */
+ WorkerClient makeWorkerClient();
+
+ /**
+ * Fetch a directory for temporary outputs
+ */
+ File tempDir();
+
+ FrameContext frameContext(QueryDefinition queryDef, int stageNumber);
+
+ int threadCount();
+
+ /**
+ * Fetch node info about self
+ */
+ DruidNode selfNode();
+
+ Bouncer processorBouncer();
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java
new file mode 100644
index 00000000000..275965abbe4
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java
@@ -0,0 +1,1249 @@
+/*
+ * 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.exec;
+
+import com.google.common.base.Function;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Suppliers;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.SettableFuture;
+import it.unimi.dsi.fastutil.bytes.ByteArrays;
+import org.apache.druid.frame.allocation.ArenaMemoryAllocator;
+import org.apache.druid.frame.channel.BlockingQueueFrameChannel;
+import org.apache.druid.frame.channel.ReadableFileFrameChannel;
+import org.apache.druid.frame.channel.ReadableFrameChannel;
+import org.apache.druid.frame.channel.ReadableNilFrameChannel;
+import org.apache.druid.frame.file.FrameFile;
+import org.apache.druid.frame.file.FrameFileWriter;
+import org.apache.druid.frame.key.ClusterBy;
+import org.apache.druid.frame.key.ClusterByPartitions;
+import org.apache.druid.frame.processor.BlockingQueueOutputChannelFactory;
+import org.apache.druid.frame.processor.Bouncer;
+import org.apache.druid.frame.processor.FileOutputChannelFactory;
+import org.apache.druid.frame.processor.FrameChannelMuxer;
+import org.apache.druid.frame.processor.FrameProcessor;
+import org.apache.druid.frame.processor.FrameProcessorExecutor;
+import org.apache.druid.frame.processor.OutputChannel;
+import org.apache.druid.frame.processor.OutputChannelFactory;
+import org.apache.druid.frame.processor.OutputChannels;
+import org.apache.druid.frame.processor.SuperSorter;
+import org.apache.druid.frame.processor.SuperSorterProgressTracker;
+import org.apache.druid.indexer.TaskStatus;
+import org.apache.druid.java.util.common.FileUtils;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.msq.counters.CounterNames;
+import org.apache.druid.msq.counters.CounterSnapshotsTree;
+import org.apache.druid.msq.counters.CounterTracker;
+import org.apache.druid.msq.indexing.CountingOutputChannelFactory;
+import org.apache.druid.msq.indexing.InputChannelFactory;
+import org.apache.druid.msq.indexing.InputChannelsImpl;
+import org.apache.druid.msq.indexing.KeyStatisticsCollectionProcessor;
+import org.apache.druid.msq.indexing.MSQWorkerTask;
+import org.apache.druid.msq.indexing.error.CanceledFault;
+import org.apache.druid.msq.indexing.error.MSQErrorReport;
+import org.apache.druid.msq.indexing.error.MSQException;
+import org.apache.druid.msq.indexing.error.MSQWarningReportLimiterPublisher;
+import org.apache.druid.msq.indexing.error.MSQWarningReportPublisher;
+import org.apache.druid.msq.indexing.error.MSQWarningReportSimplePublisher;
+import org.apache.druid.msq.input.InputSlice;
+import org.apache.druid.msq.input.InputSliceReader;
+import org.apache.druid.msq.input.InputSlices;
+import org.apache.druid.msq.input.MapInputSliceReader;
+import org.apache.druid.msq.input.NilInputSlice;
+import org.apache.druid.msq.input.NilInputSliceReader;
+import org.apache.druid.msq.input.external.ExternalInputSlice;
+import org.apache.druid.msq.input.external.ExternalInputSliceReader;
+import org.apache.druid.msq.input.stage.InputChannels;
+import org.apache.druid.msq.input.stage.ReadablePartition;
+import org.apache.druid.msq.input.stage.StageInputSlice;
+import org.apache.druid.msq.input.stage.StageInputSliceReader;
+import org.apache.druid.msq.input.table.SegmentsInputSlice;
+import org.apache.druid.msq.input.table.SegmentsInputSliceReader;
+import org.apache.druid.msq.kernel.FrameContext;
+import org.apache.druid.msq.kernel.FrameProcessorFactory;
+import org.apache.druid.msq.kernel.ProcessorsAndChannels;
+import org.apache.druid.msq.kernel.QueryDefinition;
+import org.apache.druid.msq.kernel.StageDefinition;
+import org.apache.druid.msq.kernel.StageId;
+import org.apache.druid.msq.kernel.StagePartition;
+import org.apache.druid.msq.kernel.WorkOrder;
+import org.apache.druid.msq.kernel.worker.WorkerStageKernel;
+import org.apache.druid.msq.kernel.worker.WorkerStagePhase;
+import org.apache.druid.msq.querykit.DataSegmentProvider;
+import org.apache.druid.msq.shuffle.DurableStorageInputChannelFactory;
+import org.apache.druid.msq.shuffle.DurableStorageOutputChannelFactory;
+import org.apache.druid.msq.shuffle.WorkerInputChannelFactory;
+import org.apache.druid.msq.statistics.ClusterByStatisticsCollector;
+import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
+import org.apache.druid.msq.util.DecoratedExecutorService;
+import org.apache.druid.msq.util.MultiStageQueryContext;
+import org.apache.druid.query.PrioritizedCallable;
+import org.apache.druid.query.PrioritizedRunnable;
+import org.apache.druid.query.QueryProcessingPool;
+import org.apache.druid.server.DruidNode;
+
+import javax.annotation.Nullable;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.RandomAccessFile;
+import java.nio.channels.Channels;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+
+/**
+ * Interface for a worker of a multi-stage query.
+ */
+public class WorkerImpl implements Worker
+{
+ private static final Logger log = new Logger(WorkerImpl.class);
+
+ private final MSQWorkerTask task;
+ private final WorkerContext context;
+ private final DruidNode selfDruidNode;
+ private final Bouncer processorBouncer;
+
+ private final BlockingQueue> kernelManipulationQueue = new LinkedBlockingDeque<>();
+ private final ConcurrentHashMap> stageOutputs = new ConcurrentHashMap<>();
+ private final ConcurrentHashMap stageCounters = new ConcurrentHashMap<>();
+ private final boolean durableStageStorageEnabled;
+
+ /**
+ * Set once in {@link #runTask} and never reassigned.
+ */
+ private volatile ControllerClient controllerClient;
+
+ /**
+ * Set once in {@link #runTask} and never reassigned. Used by processing threads so we can contact other workers
+ * during a shuffle.
+ */
+ private volatile WorkerClient workerClient;
+
+ /**
+ * Set to false by {@link #controllerFailed()} as a way of enticing the {@link #runTask} method to exit promptly.
+ */
+ private volatile boolean controllerAlive = true;
+
+ public WorkerImpl(MSQWorkerTask task, WorkerContext context)
+ {
+ this.task = task;
+ this.context = context;
+ this.selfDruidNode = context.selfNode();
+ this.processorBouncer = context.processorBouncer();
+ this.durableStageStorageEnabled = MultiStageQueryContext.isDurableStorageEnabled(task.getContext());
+ }
+
+ @Override
+ public String id()
+ {
+ return task.getId();
+ }
+
+ @Override
+ public MSQWorkerTask task()
+ {
+ return task;
+ }
+
+ @Override
+ public TaskStatus run() throws Exception
+ {
+ try (final Closer closer = Closer.create()) {
+ Optional maybeErrorReport;
+
+ try {
+ maybeErrorReport = runTask(closer);
+ }
+ catch (Throwable e) {
+ maybeErrorReport = Optional.of(
+ MSQErrorReport.fromException(id(), MSQTasks.getHostFromSelfNode(selfDruidNode), null, e)
+ );
+ }
+
+ if (maybeErrorReport.isPresent()) {
+ final MSQErrorReport errorReport = maybeErrorReport.get();
+ final String errorLogMessage = MSQTasks.errorReportToLogMessage(errorReport);
+ log.warn(errorLogMessage);
+
+ closer.register(() -> {
+ if (controllerAlive && controllerClient != null && selfDruidNode != null) {
+ controllerClient.postWorkerError(id(), errorReport);
+ }
+ });
+
+ return TaskStatus.failure(id(), errorReport.getFault().getCodeWithMessage());
+ } else {
+ return TaskStatus.success(id());
+ }
+ }
+ }
+
+ /**
+ * Runs worker logic. Returns an empty Optional on success. On failure, returns an error report for errors that
+ * happened in other threads; throws exceptions for errors that happened in the main worker loop.
+ */
+ public Optional runTask(final Closer closer) throws Exception
+ {
+ this.controllerClient = context.makeControllerClient(task.getControllerTaskId());
+ closer.register(controllerClient::close);
+ context.registerWorker(this, closer); // Uses controllerClient, so must be called after that is initialized
+ this.workerClient = new ExceptionWrappingWorkerClient(context.makeWorkerClient());
+ closer.register(workerClient::close);
+
+ final KernelHolder kernelHolder = new KernelHolder();
+ final String cancellationId = id();
+
+ final FrameProcessorExecutor workerExec = new FrameProcessorExecutor(makeProcessingPool());
+
+ // Delete all the stage outputs
+ closer.register(() -> {
+ for (final StageId stageId : stageOutputs.keySet()) {
+ cleanStageOutput(stageId);
+ }
+ });
+
+ // Close stage output processors and running futures (if present)
+ closer.register(() -> {
+ try {
+ workerExec.cancel(cancellationId);
+ }
+ catch (InterruptedException e) {
+ // Strange that cancellation would itself be interrupted. Throw an exception, since this is unexpected.
+ throw new RuntimeException(e);
+ }
+ });
+
+ final MSQWarningReportPublisher msqWarningReportPublisher = new MSQWarningReportLimiterPublisher(
+ new MSQWarningReportSimplePublisher(
+ id(),
+ controllerClient,
+ id(),
+ MSQTasks.getHostFromSelfNode(selfDruidNode)
+ )
+ );
+
+ closer.register(msqWarningReportPublisher);
+
+ final Map> partitionBoundariesFutureMap = new HashMap<>();
+
+ final Map stageFrameContexts = new HashMap<>();
+
+ while (!kernelHolder.isDone()) {
+ boolean didSomething = false;
+
+ for (final WorkerStageKernel kernel : kernelHolder.getStageKernelMap().values()) {
+ final StageDefinition stageDefinition = kernel.getStageDefinition();
+
+ if (kernel.getPhase() == WorkerStagePhase.NEW) {
+ log.debug("New work order: %s", context.jsonMapper().writeValueAsString(kernel.getWorkOrder()));
+
+ // Create separate inputChannelFactory per stage, because the list of tasks can grow between stages, and
+ // so we need to avoid the memoization in baseInputChannelFactory.
+ final InputChannelFactory inputChannelFactory = makeBaseInputChannelFactory(closer);
+
+ // Compute memory parameters for all stages, even ones that haven't been assigned yet, so we can fail-fast
+ // if some won't work. (We expect that all stages will get assigned to the same pool of workers.)
+ for (final StageDefinition stageDef : kernel.getWorkOrder().getQueryDefinition().getStageDefinitions()) {
+ stageFrameContexts.computeIfAbsent(
+ stageDef.getId(),
+ stageId -> context.frameContext(
+ kernel.getWorkOrder().getQueryDefinition(),
+ stageId.getStageNumber()
+ )
+ );
+ }
+
+ // Start working on this stage immediately.
+ kernel.startReading();
+ final SettableFuture partitionBoundariesFuture =
+ startWorkOrder(
+ kernel,
+ inputChannelFactory,
+ stageCounters.computeIfAbsent(stageDefinition.getId(), ignored -> new CounterTracker()),
+ workerExec,
+ cancellationId,
+ context.threadCount(),
+ stageFrameContexts.get(stageDefinition.getId()),
+ msqWarningReportPublisher
+ );
+
+ if (partitionBoundariesFuture != null) {
+ if (partitionBoundariesFutureMap.put(stageDefinition.getId(), partitionBoundariesFuture) != null) {
+ throw new ISE("Work order collision for stage [%s]", stageDefinition.getId());
+ }
+ }
+
+ didSomething = true;
+ logKernelStatus(kernelHolder.getStageKernelMap().values());
+ }
+
+ if (kernel.getPhase() == WorkerStagePhase.READING_INPUT && kernel.hasResultKeyStatisticsSnapshot()) {
+ if (controllerAlive) {
+ controllerClient.postKeyStatistics(
+ stageDefinition.getId(),
+ kernel.getWorkOrder().getWorkerNumber(),
+ kernel.getResultKeyStatisticsSnapshot()
+ );
+ }
+ kernel.startPreshuffleWaitingForResultPartitionBoundaries();
+
+ didSomething = true;
+ logKernelStatus(kernelHolder.getStageKernelMap().values());
+ }
+
+ logKernelStatus(kernelHolder.getStageKernelMap().values());
+ if (kernel.getPhase() == WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES
+ && kernel.hasResultPartitionBoundaries()) {
+ partitionBoundariesFutureMap.get(stageDefinition.getId()).set(kernel.getResultPartitionBoundaries());
+ kernel.startPreshuffleWritingOutput();
+
+ didSomething = true;
+ logKernelStatus(kernelHolder.getStageKernelMap().values());
+ }
+
+ if (kernel.getPhase() == WorkerStagePhase.RESULTS_READY
+ && kernel.addPostedResultsComplete(Pair.of(stageDefinition.getId(), kernel.getWorkOrder().getWorkerNumber()))) {
+ if (controllerAlive) {
+ controllerClient.postResultsComplete(
+ stageDefinition.getId(),
+ kernel.getWorkOrder().getWorkerNumber(),
+ kernel.getResultObject()
+ );
+ }
+ }
+
+ if (kernel.getPhase() == WorkerStagePhase.FAILED) {
+ // Better than throwing an exception, because we can include the stage number.
+ return Optional.of(
+ MSQErrorReport.fromException(
+ id(),
+ MSQTasks.getHostFromSelfNode(selfDruidNode),
+ stageDefinition.getId().getStageNumber(),
+ kernel.getException()
+ )
+ );
+ }
+ }
+
+ if (!didSomething && !kernelHolder.isDone()) {
+ Consumer nextCommand;
+
+ do {
+ postCountersToController();
+ } while ((nextCommand = kernelManipulationQueue.poll(5, TimeUnit.SECONDS)) == null);
+
+ nextCommand.accept(kernelHolder);
+ logKernelStatus(kernelHolder.getStageKernelMap().values());
+ }
+ }
+
+ // Empty means success.
+ return Optional.empty();
+ }
+
+ @Override
+ public void stopGracefully()
+ {
+ kernelManipulationQueue.add(
+ kernel -> {
+ // stopGracefully() is called when the containing process is terminated, or when the task is canceled.
+ throw new MSQException(CanceledFault.INSTANCE);
+ }
+ );
+ }
+
+ @Override
+ public void controllerFailed()
+ {
+ controllerAlive = false;
+ stopGracefully();
+ }
+
+ @Override
+ public InputStream readChannel(
+ final String queryId,
+ final int stageNumber,
+ final int partitionNumber,
+ final long offset
+ ) throws IOException
+ {
+ final StageId stageId = new StageId(queryId, stageNumber);
+ final StagePartition stagePartition = new StagePartition(stageId, partitionNumber);
+ final ConcurrentHashMap partitionOutputsForStage = stageOutputs.get(stageId);
+
+ if (partitionOutputsForStage == null) {
+ return null;
+ }
+ final ReadableFrameChannel channel = partitionOutputsForStage.get(partitionNumber);
+
+ if (channel == null) {
+ return null;
+ }
+
+ if (channel instanceof ReadableNilFrameChannel) {
+ // Build an empty frame file.
+ final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ FrameFileWriter.open(Channels.newChannel(baos), null).close();
+
+ final ByteArrayInputStream in = new ByteArrayInputStream(baos.toByteArray());
+
+ //noinspection ResultOfMethodCallIgnored: OK to ignore since "skip" always works for ByteArrayInputStream.
+ in.skip(offset);
+
+ return in;
+ } else if (channel instanceof ReadableFileFrameChannel) {
+ // Close frameFile once we've returned an input stream: no need to retain a reference to the mmap after that,
+ // since we aren't using it.
+ try (final FrameFile frameFile = ((ReadableFileFrameChannel) channel).newFrameFileReference()) {
+ final RandomAccessFile randomAccessFile = new RandomAccessFile(frameFile.file(), "r");
+
+ if (offset >= randomAccessFile.length()) {
+ randomAccessFile.close();
+ return new ByteArrayInputStream(ByteArrays.EMPTY_ARRAY);
+ } else {
+ randomAccessFile.seek(offset);
+ return Channels.newInputStream(randomAccessFile.getChannel());
+ }
+ }
+ } else {
+ String errorMsg = StringUtils.format(
+ "Returned server error to client because channel for [%s] is not nil or file-based (class = %s)",
+ stagePartition,
+ channel.getClass().getName()
+ );
+ log.error(StringUtils.encodeForFormat(errorMsg));
+
+ throw new IOException(errorMsg);
+ }
+ }
+
+ @Override
+ public void postWorkOrder(final WorkOrder workOrder)
+ {
+ if (task.getWorkerNumber() != workOrder.getWorkerNumber()) {
+ throw new ISE("Worker number mismatch: expected [%d]", task.getWorkerNumber());
+ }
+
+ kernelManipulationQueue.add(
+ kernelHolder ->
+ kernelHolder.getStageKernelMap().computeIfAbsent(
+ workOrder.getStageDefinition().getId(),
+ ignored -> WorkerStageKernel.create(workOrder)
+ )
+ );
+ }
+
+ @Override
+ public boolean postResultPartitionBoundaries(
+ final ClusterByPartitions stagePartitionBoundaries,
+ final String queryId,
+ final int stageNumber
+ )
+ {
+ final StageId stageId = new StageId(queryId, stageNumber);
+
+ kernelManipulationQueue.add(
+ kernelHolder -> {
+ final WorkerStageKernel stageKernel = kernelHolder.getStageKernelMap().get(stageId);
+
+ // Ignore the update if we don't have a kernel for this stage.
+ if (stageKernel != null) {
+ stageKernel.setResultPartitionBoundaries(stagePartitionBoundaries);
+ } else {
+ log.warn("Ignored result partition boundaries call for unknown stage [%s]", stageId);
+ }
+ }
+ );
+ return true;
+ }
+
+ @Override
+ public void postCleanupStage(final StageId stageId)
+ {
+ log.info("Cleanup order for stage: [%s] received", stageId);
+ kernelManipulationQueue.add(
+ holder -> {
+ cleanStageOutput(stageId);
+ // Mark the stage as FINISHED
+ holder.getStageKernelMap().get(stageId).setStageFinished();
+ }
+ );
+ }
+
+ @Override
+ public void postFinish()
+ {
+ kernelManipulationQueue.add(KernelHolder::setDone);
+ }
+
+ @Override
+ public CounterSnapshotsTree getCounters()
+ {
+ final CounterSnapshotsTree retVal = new CounterSnapshotsTree();
+
+ for (final Map.Entry entry : stageCounters.entrySet()) {
+ retVal.put(entry.getKey().getStageNumber(), task().getWorkerNumber(), entry.getValue().snapshot());
+ }
+
+ return retVal;
+ }
+
+ private InputChannelFactory makeBaseInputChannelFactory(final Closer closer)
+ {
+ final Supplier> workerTaskList = Suppliers.memoize(
+ () -> {
+ try {
+ return controllerClient.getTaskList();
+ }
+ catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+ )::get;
+
+ if (durableStageStorageEnabled) {
+ return DurableStorageInputChannelFactory.createStandardImplementation(
+ task.getControllerTaskId(),
+ workerTaskList,
+ MSQTasks.makeStorageConnector(context.injector()),
+ closer
+ );
+ } else {
+ return new WorkerOrLocalInputChannelFactory(workerTaskList);
+ }
+ }
+
+ private OutputChannelFactory makeStageOutputChannelFactory(final FrameContext frameContext, final int stageNumber)
+ {
+ // Use the standard frame size, since we assume this size when computing how much is needed to merge output
+ // files from different workers.
+ final int frameSize = frameContext.memoryParameters().getStandardFrameSize();
+
+ if (durableStageStorageEnabled) {
+ return DurableStorageOutputChannelFactory.createStandardImplementation(
+ task.getControllerTaskId(),
+ id(),
+ stageNumber,
+ frameSize,
+ MSQTasks.makeStorageConnector(context.injector())
+ );
+ } else {
+ final File fileChannelDirectory =
+ new File(context.tempDir(), StringUtils.format("output_stage_%06d", stageNumber));
+
+ return new FileOutputChannelFactory(fileChannelDirectory, frameSize);
+ }
+ }
+
+ /**
+ * Decorates the server-wide {@link QueryProcessingPool} such that any Callables and Runnables, not just
+ * {@link PrioritizedCallable} and {@link PrioritizedRunnable}, may be added to it.
+ *
+ * In production, the underlying {@link QueryProcessingPool} pool is set up by
+ * {@link org.apache.druid.guice.DruidProcessingModule}.
+ */
+ private ListeningExecutorService makeProcessingPool()
+ {
+ final QueryProcessingPool queryProcessingPool = context.injector().getInstance(QueryProcessingPool.class);
+ final int priority = 0;
+
+ return new DecoratedExecutorService(
+ queryProcessingPool,
+ new DecoratedExecutorService.Decorator()
+ {
+ @Override
+ public Callable decorateCallable(Callable callable)
+ {
+ return new PrioritizedCallable()
+ {
+ @Override
+ public int getPriority()
+ {
+ return priority;
+ }
+
+ @Override
+ public T call() throws Exception
+ {
+ return callable.call();
+ }
+ };
+ }
+
+ @Override
+ public Runnable decorateRunnable(Runnable runnable)
+ {
+ return new PrioritizedRunnable()
+ {
+ @Override
+ public int getPriority()
+ {
+ return priority;
+ }
+
+ @Override
+ public void run()
+ {
+ runnable.run();
+ }
+ };
+ }
+ }
+ );
+ }
+
+ /**
+ * Posts all counters for this worker to the controller.
+ */
+ private void postCountersToController() throws IOException
+ {
+ final CounterSnapshotsTree snapshotsTree = getCounters();
+
+ if (controllerAlive && !snapshotsTree.isEmpty()) {
+ controllerClient.postCounters(snapshotsTree);
+ }
+ }
+
+ /**
+ * Cleans up the stage outputs corresponding to the provided stage id. It essentially calls {@code doneReading()} on
+ * the readable channels corresponding to all the partitions for that stage, and removes it from the {@code stageOutputs}
+ * map
+ */
+ private void cleanStageOutput(final StageId stageId)
+ {
+ // This code is thread-safe because remove() on ConcurrentHashMap will remove and return the removed channel only for
+ // one thread. For the other threads it will return null, therefore we will call doneReading for a channel only once
+ final ConcurrentHashMap partitionOutputsForStage = stageOutputs.remove(stageId);
+ // Check for null, this can be the case if this method is called simultaneously from multiple threads.
+ if (partitionOutputsForStage == null) {
+ return;
+ }
+ for (final int partition : partitionOutputsForStage.keySet()) {
+ final ReadableFrameChannel output = partitionOutputsForStage.remove(partition);
+ if (output == null) {
+ continue;
+ }
+ output.close();
+
+ // One caveat with this approach is that in case of a worker crash, while the MM/Indexer systems will delete their
+ // temp directories where intermediate results were stored, it won't be the case for the external storage.
+ // Therefore, the logic for cleaning the stage output in case of a worker/machine crash has to be external.
+ // We currently take care of this in the controller.
+ if (durableStageStorageEnabled) {
+ final String fileName = DurableStorageOutputChannelFactory.getPartitionFileName(
+ task.getControllerTaskId(),
+ task.getId(),
+ stageId.getStageNumber(),
+ partition
+ );
+ try {
+ MSQTasks.makeStorageConnector(context.injector()).deleteFile(fileName);
+ }
+ catch (Exception e) {
+ // If an error is thrown while cleaning up a file, log it and try to continue with the cleanup
+ log.warn(e, "Error while cleaning up temporary files at path " + fileName);
+ }
+ }
+ }
+ }
+
+ @SuppressWarnings({"rawtypes", "unchecked"})
+ @Nullable
+ private SettableFuture startWorkOrder(
+ final WorkerStageKernel kernel,
+ final InputChannelFactory inputChannelFactory,
+ final CounterTracker counters,
+ final FrameProcessorExecutor exec,
+ final String cancellationId,
+ final int parallelism,
+ final FrameContext frameContext,
+ final MSQWarningReportPublisher MSQWarningReportPublisher
+ ) throws IOException
+ {
+ final WorkOrder workOrder = kernel.getWorkOrder();
+ final int workerNumber = workOrder.getWorkerNumber();
+ final StageDefinition stageDef = workOrder.getStageDefinition();
+
+ final InputChannels inputChannels =
+ new InputChannelsImpl(
+ workOrder.getQueryDefinition(),
+ InputSlices.allReadablePartitions(workOrder.getInputs()),
+ inputChannelFactory,
+ () -> ArenaMemoryAllocator.createOnHeap(frameContext.memoryParameters().getStandardFrameSize()),
+ exec,
+ cancellationId
+ );
+
+ final InputSliceReader inputSliceReader = makeInputSliceReader(
+ workOrder.getQueryDefinition(),
+ inputChannels,
+ frameContext.tempDir(),
+ frameContext.dataSegmentProvider()
+ );
+
+ final OutputChannelFactory workerOutputChannelFactory;
+
+ if (stageDef.doesShuffle()) {
+ // Writing to a consumer in the same JVM (which will be set up later on in this method). Use the large frame
+ // size, since we may be writing to a SuperSorter, and we'll generate fewer temp files if we use larger frames.
+ // Note: it's not *guaranteed* that we're writing to a SuperSorter, but it's harmless to use large frames
+ // even if not.
+ workerOutputChannelFactory =
+ new BlockingQueueOutputChannelFactory(frameContext.memoryParameters().getLargeFrameSize());
+ } else {
+ // Writing stage output.
+ workerOutputChannelFactory = makeStageOutputChannelFactory(frameContext, stageDef.getStageNumber());
+ }
+
+ final ResultAndChannels> workerResultAndOutputChannels =
+ makeAndRunWorkers(
+ workerNumber,
+ workOrder.getStageDefinition().getProcessorFactory(),
+ workOrder.getExtraInfo(),
+ new CountingOutputChannelFactory(
+ workerOutputChannelFactory,
+ counters.channel(CounterNames.outputChannel())
+ ),
+ stageDef,
+ workOrder.getInputs(),
+ inputSliceReader,
+ frameContext,
+ exec,
+ cancellationId,
+ parallelism,
+ processorBouncer,
+ counters,
+ MSQWarningReportPublisher
+ );
+
+ final ListenableFuture stagePartitionBoundariesFuture;
+ final ListenableFuture outputChannelsFuture;
+
+ if (stageDef.doesShuffle()) {
+ final ClusterBy clusterBy = workOrder.getStageDefinition().getShuffleSpec().get().getClusterBy();
+
+ final CountingOutputChannelFactory shuffleOutputChannelFactory =
+ new CountingOutputChannelFactory(
+ makeStageOutputChannelFactory(frameContext, stageDef.getStageNumber()),
+ counters.channel(CounterNames.shuffleChannel())
+ );
+
+ if (stageDef.doesSortDuringShuffle()) {
+ if (stageDef.mustGatherResultKeyStatistics()) {
+ stagePartitionBoundariesFuture = SettableFuture.create();
+ } else {
+ stagePartitionBoundariesFuture = Futures.immediateFuture(kernel.getResultPartitionBoundaries());
+ }
+
+ outputChannelsFuture = superSortOutputChannels(
+ workOrder.getStageDefinition(),
+ clusterBy,
+ workerResultAndOutputChannels.getOutputChannels(),
+ stagePartitionBoundariesFuture,
+ shuffleOutputChannelFactory,
+ exec,
+ cancellationId,
+ frameContext.memoryParameters(),
+ context,
+ kernelManipulationQueue,
+ counters.sortProgress()
+ );
+ } else {
+ // No sorting, just combining all outputs into one big partition. Use a muxer to get everything into one file.
+ // Note: even if there is only one output channel, we'll run it through the muxer anyway, to ensure the data
+ // gets written to a file. (httpGetChannelData requires files.)
+ final OutputChannel outputChannel = shuffleOutputChannelFactory.openChannel(0);
+
+ final FrameChannelMuxer muxer =
+ new FrameChannelMuxer(
+ workerResultAndOutputChannels.getOutputChannels()
+ .getAllChannels()
+ .stream()
+ .map(OutputChannel::getReadableChannel)
+ .collect(Collectors.toList()),
+ outputChannel.getWritableChannel()
+ );
+
+ //noinspection unchecked, rawtypes
+ outputChannelsFuture = Futures.transform(
+ exec.runFully(muxer, cancellationId),
+ (Function) ignored -> OutputChannels.wrap(Collections.singletonList(outputChannel.readOnly()))
+ );
+
+ stagePartitionBoundariesFuture = null;
+ }
+ } else {
+ stagePartitionBoundariesFuture = null;
+
+ // Retain read-only versions to reduce memory footprint.
+ outputChannelsFuture = Futures.immediateFuture(workerResultAndOutputChannels.getOutputChannels().readOnly());
+ }
+
+ // Output channels and future are all constructed. Sanity check, record them, and set up callbacks.
+ Futures.addCallback(
+ Futures.allAsList(
+ Arrays.asList(
+ workerResultAndOutputChannels.getResultFuture(),
+ Futures.transform(
+ outputChannelsFuture,
+ new Function()
+ {
+ @Override
+ public OutputChannels apply(final OutputChannels channels)
+ {
+ sanityCheckOutputChannels(channels);
+ return channels;
+ }
+ }
+ )
+ )
+ ),
+ new FutureCallback>()
+ {
+ @Override
+ public void onSuccess(final List workerResultAndOutputChannelsResolved)
+ {
+ Object resultObject = workerResultAndOutputChannelsResolved.get(0);
+ final OutputChannels outputChannels = (OutputChannels) workerResultAndOutputChannelsResolved.get(1);
+
+ for (OutputChannel channel : outputChannels.getAllChannels()) {
+ stageOutputs.computeIfAbsent(stageDef.getId(), ignored1 -> new ConcurrentHashMap<>())
+ .computeIfAbsent(channel.getPartitionNumber(), ignored2 -> channel.getReadableChannel());
+ }
+ kernelManipulationQueue.add(holder -> holder.getStageKernelMap()
+ .get(stageDef.getId())
+ .setResultsComplete(resultObject));
+ }
+
+ @Override
+ public void onFailure(final Throwable t)
+ {
+ kernelManipulationQueue.add(
+ kernelHolder ->
+ kernelHolder.getStageKernelMap().get(stageDef.getId()).fail(t)
+ );
+ }
+ }
+ );
+
+ // Return settable result-key-statistics future, so callers can set it and unblock the supersorter if needed.
+ return stageDef.mustGatherResultKeyStatistics()
+ ? (SettableFuture) stagePartitionBoundariesFuture
+ : null;
+ }
+
+ private static , I, WorkerClass extends FrameProcessor, T, R> ResultAndChannels makeAndRunWorkers(
+ final int workerNumber,
+ final FactoryType processorFactory,
+ final I processorFactoryExtraInfo,
+ final OutputChannelFactory outputChannelFactory,
+ final StageDefinition stageDefinition,
+ final List inputSlices,
+ final InputSliceReader inputSliceReader,
+ final FrameContext frameContext,
+ final FrameProcessorExecutor exec,
+ final String cancellationId,
+ final int parallelism,
+ final Bouncer processorBouncer,
+ final CounterTracker counters,
+ final MSQWarningReportPublisher warningPublisher
+ ) throws IOException
+ {
+ final ProcessorsAndChannels processors =
+ processorFactory.makeProcessors(
+ stageDefinition,
+ workerNumber,
+ inputSlices,
+ inputSliceReader,
+ processorFactoryExtraInfo,
+ outputChannelFactory,
+ frameContext,
+ parallelism,
+ counters,
+ e -> warningPublisher.publishException(stageDefinition.getStageNumber(), e)
+ );
+
+ final Sequence processorSequence = processors.processors();
+
+ final int maxOutstandingProcessors;
+
+ if (processors.getOutputChannels().getAllChannels().isEmpty()) {
+ // No output channels: run up to "parallelism" processors at once.
+ maxOutstandingProcessors = Math.max(1, parallelism);
+ } else {
+ // If there are output channels, that acts as a ceiling on the number of processors that can run at once.
+ maxOutstandingProcessors =
+ Math.max(1, Math.min(parallelism, processors.getOutputChannels().getAllChannels().size()));
+ }
+
+ final ListenableFuture workResultFuture = exec.runAllFully(
+ processorSequence,
+ processorFactory.newAccumulatedResult(),
+ processorFactory::accumulateResult,
+ maxOutstandingProcessors,
+ processorBouncer,
+ cancellationId
+ );
+
+ return new ResultAndChannels<>(workResultFuture, processors.getOutputChannels());
+ }
+
+ private static InputSliceReader makeInputSliceReader(
+ final QueryDefinition queryDef,
+ final InputChannels inputChannels,
+ final File temporaryDirectory,
+ final DataSegmentProvider segmentProvider
+ )
+ {
+ return new MapInputSliceReader(
+ ImmutableMap., InputSliceReader>builder()
+ .put(NilInputSlice.class, NilInputSliceReader.INSTANCE)
+ .put(StageInputSlice.class, new StageInputSliceReader(queryDef.getQueryId(), inputChannels))
+ .put(ExternalInputSlice.class, new ExternalInputSliceReader(temporaryDirectory))
+ .put(SegmentsInputSlice.class, new SegmentsInputSliceReader(segmentProvider))
+ .build()
+ );
+ }
+
+ private static ListenableFuture superSortOutputChannels(
+ final StageDefinition stageDefinition,
+ final ClusterBy clusterBy,
+ final OutputChannels processorOutputChannels,
+ final ListenableFuture stagePartitionBoundariesFuture,
+ final OutputChannelFactory outputChannelFactory,
+ final FrameProcessorExecutor exec,
+ final String cancellationId,
+ final WorkerMemoryParameters memoryParameters,
+ final WorkerContext context,
+ final BlockingQueue> kernelManipulationQueue,
+ final SuperSorterProgressTracker superSorterProgressTracker
+ ) throws IOException
+ {
+ if (!stageDefinition.doesShuffle()) {
+ throw new ISE("Output channels do not need shuffling");
+ }
+
+ final List channelsToSuperSort;
+
+ if (processorOutputChannels.getAllChannels().isEmpty()) {
+ // No data coming out of this processor. Report empty statistics, if the kernel is expecting statistics.
+ if (stageDefinition.mustGatherResultKeyStatistics()) {
+ kernelManipulationQueue.add(
+ holder ->
+ holder.getStageKernelMap().get(stageDefinition.getId())
+ .setResultKeyStatisticsSnapshot(ClusterByStatisticsSnapshot.empty())
+ );
+ }
+
+ // Process one empty channel so the SuperSorter has something to do.
+ final BlockingQueueFrameChannel channel = BlockingQueueFrameChannel.minimal();
+ channel.writable().close();
+ channelsToSuperSort = Collections.singletonList(channel.readable());
+ } else if (stageDefinition.mustGatherResultKeyStatistics()) {
+ channelsToSuperSort = collectKeyStatistics(
+ stageDefinition,
+ clusterBy,
+ processorOutputChannels,
+ exec,
+ cancellationId,
+ kernelManipulationQueue
+ );
+ } else {
+ channelsToSuperSort = processorOutputChannels.getAllChannels()
+ .stream()
+ .map(OutputChannel::getReadableChannel)
+ .collect(Collectors.toList());
+ }
+
+ final File sorterTmpDir = new File(context.tempDir(), "super-sort-" + UUID.randomUUID());
+ FileUtils.mkdirp(sorterTmpDir);
+ if (!sorterTmpDir.isDirectory()) {
+ throw new IOException("Cannot create directory: " + sorterTmpDir);
+ }
+
+ final SuperSorter sorter = new SuperSorter(
+ channelsToSuperSort,
+ stageDefinition.getFrameReader(),
+ clusterBy,
+ stagePartitionBoundariesFuture,
+ exec,
+ sorterTmpDir,
+ outputChannelFactory,
+ () -> ArenaMemoryAllocator.createOnHeap(memoryParameters.getLargeFrameSize()),
+ memoryParameters.getSuperSorterMaxActiveProcessors(),
+ memoryParameters.getSuperSorterMaxChannelsPerProcessor(),
+ -1,
+ cancellationId,
+ superSorterProgressTracker
+ );
+
+ return sorter.run();
+ }
+
+ private static List collectKeyStatistics(
+ final StageDefinition stageDefinition,
+ final ClusterBy clusterBy,
+ final OutputChannels processorOutputChannels,
+ final FrameProcessorExecutor exec,
+ final String cancellationId,
+ final BlockingQueue> kernelManipulationQueue
+ )
+ {
+ final List retVal = new ArrayList<>();
+ final List processors = new ArrayList<>();
+
+ for (final OutputChannel outputChannel : processorOutputChannels.getAllChannels()) {
+ final BlockingQueueFrameChannel channel = BlockingQueueFrameChannel.minimal();
+ retVal.add(channel.readable());
+
+ processors.add(
+ new KeyStatisticsCollectionProcessor(
+ outputChannel.getReadableChannel(),
+ channel.writable(),
+ stageDefinition.getFrameReader(),
+ clusterBy,
+ stageDefinition.createResultKeyStatisticsCollector()
+ )
+ );
+ }
+
+ final ListenableFuture clusterByStatisticsCollectorFuture =
+ exec.runAllFully(
+ Sequences.simple(processors),
+ stageDefinition.createResultKeyStatisticsCollector(),
+ ClusterByStatisticsCollector::addAll,
+ // Run all processors simultaneously. They are lightweight and this keeps things moving.
+ processors.size(),
+ Bouncer.unlimited(),
+ cancellationId
+ );
+
+ Futures.addCallback(
+ clusterByStatisticsCollectorFuture,
+ new FutureCallback()
+ {
+ @Override
+ public void onSuccess(final ClusterByStatisticsCollector result)
+ {
+ kernelManipulationQueue.add(
+ holder ->
+ holder.getStageKernelMap().get(stageDefinition.getId())
+ .setResultKeyStatisticsSnapshot(result.snapshot())
+ );
+ }
+
+ @Override
+ public void onFailure(Throwable t)
+ {
+ kernelManipulationQueue.add(
+ holder -> {
+ log.noStackTrace()
+ .warn(t, "Failed to gather clusterBy statistics for stage [%s]", stageDefinition.getId());
+ holder.getStageKernelMap().get(stageDefinition.getId()).fail(t);
+ }
+ );
+ }
+ }
+ );
+
+ return retVal;
+ }
+
+ private static void sanityCheckOutputChannels(final OutputChannels outputChannels)
+ {
+ // Verify there is exactly one channel per partition.
+ for (int partitionNumber : outputChannels.getPartitionNumbers()) {
+ final List outputChannelsForPartition =
+ outputChannels.getChannelsForPartition(partitionNumber);
+
+ Preconditions.checkState(partitionNumber >= 0, "Expected partitionNumber >= 0, but got [%s]", partitionNumber);
+ Preconditions.checkState(
+ outputChannelsForPartition.size() == 1,
+ "Expected one channel for partition [%s], but got [%s]",
+ partitionNumber,
+ outputChannelsForPartition.size()
+ );
+ }
+ }
+
+ /**
+ * Log (at DEBUG level) a string explaining the status of all work assigned to this worker.
+ */
+ private static void logKernelStatus(final Collection kernels)
+ {
+ if (log.isDebugEnabled()) {
+ log.debug(
+ "Stages: %s",
+ kernels.stream()
+ .sorted(Comparator.comparing(k -> k.getStageDefinition().getStageNumber()))
+ .map(WorkerImpl::makeKernelStageStatusString)
+ .collect(Collectors.joining("; "))
+ );
+ }
+ }
+
+ /**
+ * Helper used by {@link #logKernelStatus}.
+ */
+ private static String makeKernelStageStatusString(final WorkerStageKernel kernel)
+ {
+ final String inputPartitionNumbers =
+ StreamSupport.stream(InputSlices.allReadablePartitions(kernel.getWorkOrder().getInputs()).spliterator(), false)
+ .map(ReadablePartition::getPartitionNumber)
+ .sorted()
+ .map(String::valueOf)
+ .collect(Collectors.joining(","));
+
+ // String like ">50" if shuffling to 50 partitions, ">?" if shuffling to unknown number of partitions.
+ final String shuffleStatus =
+ kernel.getStageDefinition().doesShuffle()
+ ? ">" + (kernel.hasResultPartitionBoundaries() ? kernel.getResultPartitionBoundaries().size() : "?")
+ : "";
+
+ return StringUtils.format(
+ "S%d:W%d:P[%s]%s:%s:%s",
+ kernel.getStageDefinition().getStageNumber(),
+ kernel.getWorkOrder().getWorkerNumber(),
+ inputPartitionNumbers,
+ shuffleStatus,
+ kernel.getStageDefinition().doesShuffle() ? "SHUFFLE" : "RETAIN",
+ kernel.getPhase()
+ );
+ }
+
+ /**
+ * An {@link InputChannelFactory} that loads data locally when possible, and otherwise connects directly to other
+ * workers. Used when durable shuffle storage is off.
+ */
+ private class WorkerOrLocalInputChannelFactory implements InputChannelFactory
+ {
+ private final Supplier> taskList;
+ private final WorkerInputChannelFactory workerInputChannelFactory;
+
+ public WorkerOrLocalInputChannelFactory(final Supplier> taskList)
+ {
+ this.workerInputChannelFactory = new WorkerInputChannelFactory(workerClient, taskList);
+ this.taskList = taskList;
+ }
+
+ @Override
+ public ReadableFrameChannel openChannel(StageId stageId, int workerNumber, int partitionNumber)
+ {
+ final String taskId = taskList.get().get(workerNumber);
+ if (taskId.equals(id())) {
+ final ConcurrentMap partitionOutputsForStage = stageOutputs.get(stageId);
+ if (partitionOutputsForStage == null) {
+ throw new ISE("Unable to find outputs for stage: [%s]", stageId);
+ }
+
+ final ReadableFrameChannel myChannel = partitionOutputsForStage.get(partitionNumber);
+
+ if (myChannel instanceof ReadableFileFrameChannel) {
+ // Must duplicate the channel to avoid double-closure upon task cleanup.
+ final FrameFile frameFile = ((ReadableFileFrameChannel) myChannel).newFrameFileReference();
+ return new ReadableFileFrameChannel(frameFile);
+ } else if (myChannel instanceof ReadableNilFrameChannel) {
+ return myChannel;
+ } else {
+ throw new ISE("Output for stage: [%s] are stored in an instance of %s which is not "
+ + "supported", stageId, myChannel.getClass());
+ }
+ } else {
+ return workerInputChannelFactory.openChannel(stageId, workerNumber, partitionNumber);
+ }
+ }
+ }
+
+ private static class KernelHolder
+ {
+ private final Map stageKernelMap = new HashMap<>();
+ private boolean done = false;
+
+ public Map getStageKernelMap()
+ {
+ return stageKernelMap;
+ }
+
+ public boolean isDone()
+ {
+ return done;
+ }
+
+ public void setDone()
+ {
+ this.done = true;
+ }
+ }
+
+ private static class ResultAndChannels
+ {
+ private final ListenableFuture resultFuture;
+ private final OutputChannels outputChannels;
+
+ public ResultAndChannels(
+ ListenableFuture resultFuture,
+ OutputChannels outputChannels
+ )
+ {
+ this.resultFuture = resultFuture;
+ this.outputChannels = outputChannels;
+ }
+
+ public ListenableFuture getResultFuture()
+ {
+ return resultFuture;
+ }
+
+ public OutputChannels getOutputChannels()
+ {
+ return outputChannels;
+ }
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerManagerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerManagerClient.java
new file mode 100644
index 00000000000..7d320b111ff
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerManagerClient.java
@@ -0,0 +1,57 @@
+/*
+ * 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.exec;
+
+import org.apache.druid.indexer.TaskLocation;
+import org.apache.druid.indexer.TaskStatus;
+import org.apache.druid.msq.indexing.MSQWorkerTask;
+
+import java.io.Closeable;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Generic interface to the "worker manager" mechanism which starts, cancels and monitors worker tasks.
+ */
+public interface WorkerManagerClient extends Closeable
+{
+ String run(String controllerId, MSQWorkerTask task);
+
+ /**
+ * @param workerId the task ID
+ *
+ * @return a {@code TaskLocation} associated with the task or
+ * {@code TaskLocation.unknown()} if no associated entry could be found
+ */
+ TaskLocation location(String workerId);
+
+ /**
+ * Fetches status map corresponding to a group of task ids
+ */
+ Map statuses(Set taskIds);
+
+ /**
+ * Cancel the task corresponding to the provided workerId
+ */
+ void cancel(String workerId);
+
+ @Override
+ void close();
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerMemoryParameters.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerMemoryParameters.java
new file mode 100644
index 00000000000..9f003311917
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerMemoryParameters.java
@@ -0,0 +1,294 @@
+/*
+ * 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.exec;
+
+import com.google.common.primitives.Ints;
+import org.apache.druid.msq.indexing.error.MSQException;
+import org.apache.druid.msq.indexing.error.NotEnoughMemoryFault;
+import org.apache.druid.msq.indexing.error.TooManyWorkersFault;
+
+import java.util.Objects;
+
+/**
+ * Class for determining how much JVM heap to allocate to various purposes.
+ *
+ * First, we take {@link #USABLE_MEMORY_FRACTION} out of the total JVM heap and split it into "bundles" of
+ * equal size. The number of bundles is based entirely on server configuration; this makes the calculation
+ * robust to different queries running simultaneously in the same JVM.
+ *
+ * Then, we split up the resources for each bundle in two different ways: one assuming it'll be used for a
+ * {@link org.apache.druid.frame.processor.SuperSorter}, and one assuming it'll be used for a regular
+ * processor. Callers can then use whichever set of allocations makes sense. (We assume no single bundle
+ * will be used for both purposes.)
+ */
+public class WorkerMemoryParameters
+{
+ /**
+ * Percent of memory that we allocate to bundles. It is less than 100% because we need to leave some space
+ * left over for miscellaneous other stuff, and to ensure that GC pressure does not get too high.
+ */
+ private static final double USABLE_MEMORY_FRACTION = 0.75;
+
+ /**
+ * Percent of each bundle's memory that we allocate to appenderators. It is less than 100% because appenderators
+ * unfortunately have a variety of unaccounted-for memory usage.
+ */
+ static final double APPENDERATOR_MEMORY_FRACTION = 0.67;
+
+ /**
+ * Size for "standard frames", which are used for most purposes, except inputs to super-sorters.
+ *
+ * In particular, frames that travel between workers are always the minimum size. This is helpful because it makes
+ * it easier to compute the amount of memory needed to merge input streams.
+ */
+ private static final int STANDARD_FRAME_SIZE = 1_000_000;
+
+ /**
+ * Size for "large frames", which are used for inputs and inner channels in to super-sorters.
+ *
+ * This is helpful because it minimizes the number of temporary files needed during super-sorting.
+ */
+ private static final int LARGE_FRAME_SIZE = 8_000_000;
+
+ /**
+ * Minimum amount of bundle memory available for processing (i.e., total bundle size minus the amount
+ * needed for input channels). This memory is guaranteed to be available for things like segment generation
+ * and broadcast data.
+ */
+ public static final long PROCESSING_MINIMUM_BYTES = 25_000_000;
+
+ /**
+ * Maximum amount of parallelism for the super-sorter. Higher amounts of concurrency tend to be wasteful.
+ */
+ private static final int MAX_SUPER_SORTER_PROCESSORS = 4;
+
+ /**
+ * Each super-sorter must have at least 1 processor with 2 input frames and 1 output frame. That's 3 total.
+ */
+ private static final int MIN_SUPER_SORTER_FRAMES = 3;
+
+ /**
+ * (Very) rough estimate of the on-heap overhead of reading a column.
+ */
+ private static final int APPENDERATOR_MERGE_ROUGH_MEMORY_PER_COLUMN = 3_000;
+
+ /**
+ * Fraction of free memory per bundle that can be used by {@link org.apache.druid.msq.querykit.BroadcastJoinHelper}
+ * to store broadcast data on-heap. This is used to limit the total size of input frames, which we expect to
+ * expand on-heap. Expansion can potentially be somewhat over 2x: for example, strings are UTF-8 in frames, but are
+ * UTF-16 on-heap, which is a 2x expansion, and object and index overhead must be considered on top of that. So
+ * we use a value somewhat lower than 0.5.
+ */
+ static final double BROADCAST_JOIN_MEMORY_FRACTION = 0.3;
+
+ private final int superSorterMaxActiveProcessors;
+ private final int superSorterMaxChannelsPerProcessor;
+ private final long appenderatorMemory;
+ private final long broadcastJoinMemory;
+
+ WorkerMemoryParameters(
+ final int superSorterMaxActiveProcessors,
+ final int superSorterMaxChannelsPerProcessor,
+ final long appenderatorMemory,
+ final long broadcastJoinMemory
+ )
+ {
+ this.superSorterMaxActiveProcessors = superSorterMaxActiveProcessors;
+ this.superSorterMaxChannelsPerProcessor = superSorterMaxChannelsPerProcessor;
+ this.appenderatorMemory = appenderatorMemory;
+ this.broadcastJoinMemory = broadcastJoinMemory;
+ }
+
+ /**
+ * Returns an object specifying memory-usage parameters for a stage in a worker.
+ *
+ * Throws a {@link MSQException} with an appropriate fault if the provided combination of parameters cannot
+ * yield a workable memory situation.
+ *
+ * @param maxMemoryInJvm memory available in the entire JVM. This will be divided amongst processors.
+ * @param numWorkersInJvm number of workers that can run concurrently in this JVM. Generally equal to
+ * the task capacity.
+ * @param numProcessingThreadsInJvm size of the processing thread pool in the JVM.
+ * @param numInputWorkers number of workers across input stages that need to be merged together.
+ */
+ public static WorkerMemoryParameters compute(
+ final long maxMemoryInJvm,
+ final int numWorkersInJvm,
+ final int numProcessingThreadsInJvm,
+ final int numInputWorkers
+ )
+ {
+ final long bundleMemory = memoryPerBundle(maxMemoryInJvm, numWorkersInJvm, numProcessingThreadsInJvm);
+ final long bundleMemoryForInputChannels = memoryNeededForInputChannels(numInputWorkers);
+ final long bundleMemoryForProcessing = bundleMemory - bundleMemoryForInputChannels;
+
+ if (bundleMemoryForProcessing < PROCESSING_MINIMUM_BYTES) {
+ final int maxWorkers = computeMaxWorkers(maxMemoryInJvm, numWorkersInJvm, numProcessingThreadsInJvm);
+
+ if (maxWorkers > 0) {
+ throw new MSQException(new TooManyWorkersFault(numInputWorkers, Math.min(Limits.MAX_WORKERS, maxWorkers)));
+ } else {
+ // Not enough memory for even one worker. More of a NotEnoughMemory situation than a TooManyWorkers situation.
+ throw new MSQException(
+ new NotEnoughMemoryFault(
+ maxMemoryInJvm,
+ numWorkersInJvm,
+ numProcessingThreadsInJvm
+ )
+ );
+ }
+ }
+
+ // Compute memory breakdown for super-sorting bundles.
+ final int maxNumFramesForSuperSorter = Ints.checkedCast(bundleMemory / WorkerMemoryParameters.LARGE_FRAME_SIZE);
+
+ if (maxNumFramesForSuperSorter < MIN_SUPER_SORTER_FRAMES) {
+ throw new MSQException(new NotEnoughMemoryFault(maxMemoryInJvm, numWorkersInJvm, numProcessingThreadsInJvm));
+ }
+
+ final int superSorterMaxActiveProcessors = Math.min(
+ numProcessingThreadsInJvm,
+ Math.min(
+ maxNumFramesForSuperSorter / MIN_SUPER_SORTER_FRAMES,
+ MAX_SUPER_SORTER_PROCESSORS
+ )
+ );
+
+ // Apportion max frames to all processors equally, then subtract one to account for an output frame.
+ final int superSorterMaxChannelsPerProcessor = maxNumFramesForSuperSorter / superSorterMaxActiveProcessors - 1;
+
+ return new WorkerMemoryParameters(
+ superSorterMaxActiveProcessors,
+ superSorterMaxChannelsPerProcessor,
+ (long) (bundleMemoryForProcessing * APPENDERATOR_MEMORY_FRACTION),
+ (long) (bundleMemoryForProcessing * BROADCAST_JOIN_MEMORY_FRACTION)
+ );
+ }
+
+ public int getSuperSorterMaxActiveProcessors()
+ {
+ return superSorterMaxActiveProcessors;
+ }
+
+ public int getSuperSorterMaxChannelsPerProcessor()
+ {
+ return superSorterMaxChannelsPerProcessor;
+ }
+
+ public long getAppenderatorMaxBytesInMemory()
+ {
+ // Half for indexing, half for merging.
+ return Math.max(1, appenderatorMemory / 2);
+ }
+
+ public int getAppenderatorMaxColumnsToMerge()
+ {
+ // Half for indexing, half for merging.
+ return Ints.checkedCast(Math.max(2, appenderatorMemory / 2 / APPENDERATOR_MERGE_ROUGH_MEMORY_PER_COLUMN));
+ }
+
+ public int getStandardFrameSize()
+ {
+ return STANDARD_FRAME_SIZE;
+ }
+
+ public int getLargeFrameSize()
+ {
+ return LARGE_FRAME_SIZE;
+ }
+
+ public long getBroadcastJoinMemory()
+ {
+ return broadcastJoinMemory;
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ WorkerMemoryParameters that = (WorkerMemoryParameters) o;
+ return superSorterMaxActiveProcessors == that.superSorterMaxActiveProcessors
+ && superSorterMaxChannelsPerProcessor == that.superSorterMaxChannelsPerProcessor
+ && appenderatorMemory == that.appenderatorMemory
+ && broadcastJoinMemory == that.broadcastJoinMemory;
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(
+ superSorterMaxActiveProcessors,
+ superSorterMaxChannelsPerProcessor,
+ appenderatorMemory,
+ broadcastJoinMemory
+ );
+ }
+
+ @Override
+ public String toString()
+ {
+ return "WorkerMemoryParameters{" +
+ "superSorterMaxActiveProcessors=" + superSorterMaxActiveProcessors +
+ ", superSorterMaxChannelsPerProcessor=" + superSorterMaxChannelsPerProcessor +
+ ", appenderatorMemory=" + appenderatorMemory +
+ ", broadcastJoinMemory=" + broadcastJoinMemory +
+ '}';
+ }
+
+ /**
+ * Computes the highest value of numInputWorkers, for the given parameters, that can be passed to
+ * {@link #compute} without resulting in a {@link TooManyWorkersFault}.
+ *
+ * Returns 0 if no number of workers would be OK.
+ */
+ static int computeMaxWorkers(
+ final long maxMemoryInJvm,
+ final int numWorkersInJvm,
+ final int numProcessingThreadsInJvm
+ )
+ {
+ final long bundleMemory = memoryPerBundle(maxMemoryInJvm, numWorkersInJvm, numProcessingThreadsInJvm);
+
+ // Inverse of memoryNeededForInputChannels.
+ return Ints.checkedCast((bundleMemory - PROCESSING_MINIMUM_BYTES) / STANDARD_FRAME_SIZE - 1);
+ }
+
+ private static long memoryPerBundle(
+ final long maxMemoryInJvm,
+ final int numWorkersInJvm,
+ final int numProcessingThreadsInJvm
+ )
+ {
+ final int bundleCount = numWorkersInJvm + numProcessingThreadsInJvm;
+ return (long) (maxMemoryInJvm * USABLE_MEMORY_FRACTION) / bundleCount;
+ }
+
+ private static long memoryNeededForInputChannels(final int numInputWorkers)
+ {
+ // Regular processors require input-channel-merging for their inputs. Calculate how much that is.
+ // Requirement: inputChannelsPerProcessor number of input frames, one output frame.
+ return (long) STANDARD_FRAME_SIZE * (numInputWorkers + 1);
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQDurableStorageModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQDurableStorageModule.java
new file mode 100644
index 00000000000..67818532162
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQDurableStorageModule.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.msq.guice;
+
+import com.fasterxml.jackson.databind.Module;
+import com.google.common.collect.ImmutableList;
+import com.google.inject.Binder;
+import com.google.inject.Inject;
+import com.google.inject.Key;
+import org.apache.druid.guice.JsonConfigProvider;
+import org.apache.druid.guice.LazySingleton;
+import org.apache.druid.initialization.DruidModule;
+import org.apache.druid.storage.StorageConnector;
+import org.apache.druid.storage.StorageConnectorProvider;
+
+import java.util.List;
+import java.util.Properties;
+
+/**
+ * Module for functionality related to durable storage for stage output data.
+ */
+public class MSQDurableStorageModule implements DruidModule
+{
+ public static final String MSQ_INTERMEDIATE_STORAGE_PREFIX =
+ String.join(".", MSQIndexingModule.BASE_MSQ_KEY, "intermediate.storage");
+
+ public static final String MSQ_INTERMEDIATE_STORAGE_ENABLED =
+ String.join(".", MSQ_INTERMEDIATE_STORAGE_PREFIX, "enable");
+
+ @Inject
+ private Properties properties;
+
+ public MSQDurableStorageModule()
+ {
+ }
+
+ public MSQDurableStorageModule(Properties properties)
+ {
+ this.properties = properties;
+ }
+
+ @Override
+ public List extends Module> getJacksonModules()
+ {
+ return ImmutableList.of();
+ }
+
+ @Override
+ public void configure(Binder binder)
+ {
+ if (isDurableShuffleStorageEnabled()) {
+ JsonConfigProvider.bind(
+ binder,
+ MSQ_INTERMEDIATE_STORAGE_PREFIX,
+ StorageConnectorProvider.class,
+ MultiStageQuery.class
+ );
+
+ binder.bind(Key.get(StorageConnector.class, MultiStageQuery.class))
+ .toProvider(Key.get(StorageConnectorProvider.class, MultiStageQuery.class))
+ .in(LazySingleton.class);
+ }
+ }
+
+ private boolean isDurableShuffleStorageEnabled()
+ {
+ return Boolean.parseBoolean((String) properties.getOrDefault(MSQ_INTERMEDIATE_STORAGE_ENABLED, "false"));
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQExternalDataSourceModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQExternalDataSourceModule.java
new file mode 100644
index 00000000000..c2ed663acfa
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQExternalDataSourceModule.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.msq.guice;
+
+import com.fasterxml.jackson.databind.Module;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import com.google.inject.Binder;
+import org.apache.druid.initialization.DruidModule;
+import org.apache.druid.sql.calcite.external.ExternalDataSource;
+
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Module that adds {@link ExternalDataSource}, a class that is in core but is not bound by any core modules.
+ *
+ * It is only available if this extension is loaded.
+ */
+public class MSQExternalDataSourceModule implements DruidModule
+{
+ @Override
+ public List extends Module> getJacksonModules()
+ {
+ return Collections.singletonList(
+ new SimpleModule(getClass().getSimpleName())
+ .registerSubtypes(ExternalDataSource.class)
+ );
+ }
+
+ @Override
+ public void configure(Binder binder)
+ {
+ // Nothing to do.
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java
new file mode 100644
index 00000000000..313de7cd111
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java
@@ -0,0 +1,201 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.msq.guice;
+
+import com.fasterxml.jackson.databind.Module;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import com.google.common.collect.ImmutableList;
+import com.google.inject.Binder;
+import com.google.inject.Provides;
+import org.apache.druid.discovery.NodeRole;
+import org.apache.druid.frame.processor.Bouncer;
+import org.apache.druid.guice.LazySingleton;
+import org.apache.druid.guice.annotations.Self;
+import org.apache.druid.initialization.DruidModule;
+import org.apache.druid.msq.counters.ChannelCounters;
+import org.apache.druid.msq.counters.CounterSnapshotsSerializer;
+import org.apache.druid.msq.counters.SuperSorterProgressTrackerCounter;
+import org.apache.druid.msq.counters.WarningCounters;
+import org.apache.druid.msq.indexing.MSQControllerTask;
+import org.apache.druid.msq.indexing.MSQWorkerTask;
+import org.apache.druid.msq.indexing.SegmentGeneratorFrameProcessorFactory;
+import org.apache.druid.msq.indexing.error.BroadcastTablesTooLargeFault;
+import org.apache.druid.msq.indexing.error.CanceledFault;
+import org.apache.druid.msq.indexing.error.CannotParseExternalDataFault;
+import org.apache.druid.msq.indexing.error.ColumnNameRestrictedFault;
+import org.apache.druid.msq.indexing.error.ColumnTypeNotSupportedFault;
+import org.apache.druid.msq.indexing.error.DurableStorageConfigurationFault;
+import org.apache.druid.msq.indexing.error.InsertCannotAllocateSegmentFault;
+import org.apache.druid.msq.indexing.error.InsertCannotBeEmptyFault;
+import org.apache.druid.msq.indexing.error.InsertCannotOrderByDescendingFault;
+import org.apache.druid.msq.indexing.error.InsertCannotReplaceExistingSegmentFault;
+import org.apache.druid.msq.indexing.error.InsertLockPreemptedFault;
+import org.apache.druid.msq.indexing.error.InsertTimeNullFault;
+import org.apache.druid.msq.indexing.error.InsertTimeOutOfBoundsFault;
+import org.apache.druid.msq.indexing.error.InvalidNullByteFault;
+import org.apache.druid.msq.indexing.error.MSQFault;
+import org.apache.druid.msq.indexing.error.NotEnoughMemoryFault;
+import org.apache.druid.msq.indexing.error.QueryNotSupportedFault;
+import org.apache.druid.msq.indexing.error.RowTooLargeFault;
+import org.apache.druid.msq.indexing.error.TaskStartTimeoutFault;
+import org.apache.druid.msq.indexing.error.TooManyBucketsFault;
+import org.apache.druid.msq.indexing.error.TooManyColumnsFault;
+import org.apache.druid.msq.indexing.error.TooManyInputFilesFault;
+import org.apache.druid.msq.indexing.error.TooManyPartitionsFault;
+import org.apache.druid.msq.indexing.error.TooManyWarningsFault;
+import org.apache.druid.msq.indexing.error.TooManyWorkersFault;
+import org.apache.druid.msq.indexing.error.UnknownFault;
+import org.apache.druid.msq.indexing.error.WorkerFailedFault;
+import org.apache.druid.msq.indexing.error.WorkerRpcFailedFault;
+import org.apache.druid.msq.indexing.report.MSQTaskReport;
+import org.apache.druid.msq.input.NilInputSlice;
+import org.apache.druid.msq.input.NilInputSource;
+import org.apache.druid.msq.input.external.ExternalInputSlice;
+import org.apache.druid.msq.input.external.ExternalInputSpec;
+import org.apache.druid.msq.input.stage.StageInputSlice;
+import org.apache.druid.msq.input.stage.StageInputSpec;
+import org.apache.druid.msq.input.table.SegmentsInputSlice;
+import org.apache.druid.msq.input.table.TableInputSpec;
+import org.apache.druid.msq.kernel.NilExtraInfoHolder;
+import org.apache.druid.msq.querykit.InputNumberDataSource;
+import org.apache.druid.msq.querykit.common.OffsetLimitFrameProcessorFactory;
+import org.apache.druid.msq.querykit.groupby.GroupByPostShuffleFrameProcessorFactory;
+import org.apache.druid.msq.querykit.groupby.GroupByPreShuffleFrameProcessorFactory;
+import org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessorFactory;
+import org.apache.druid.msq.util.PassthroughAggregatorFactory;
+import org.apache.druid.query.DruidProcessingConfig;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Module that adds {@link MSQControllerTask}, {@link MSQWorkerTask}, and dependencies.
+ */
+public class MSQIndexingModule implements DruidModule
+{
+ static final String BASE_MSQ_KEY = "druid.msq";
+
+ public static final List> FAULT_CLASSES = ImmutableList.of(
+ BroadcastTablesTooLargeFault.class,
+ CanceledFault.class,
+ CannotParseExternalDataFault.class,
+ ColumnTypeNotSupportedFault.class,
+ ColumnNameRestrictedFault.class,
+ DurableStorageConfigurationFault.class,
+ InsertCannotAllocateSegmentFault.class,
+ InsertCannotBeEmptyFault.class,
+ InsertCannotOrderByDescendingFault.class,
+ InsertCannotReplaceExistingSegmentFault.class,
+ InsertLockPreemptedFault.class,
+ InsertTimeNullFault.class,
+ InsertTimeOutOfBoundsFault.class,
+ InvalidNullByteFault.class,
+ NotEnoughMemoryFault.class,
+ QueryNotSupportedFault.class,
+ RowTooLargeFault.class,
+ TaskStartTimeoutFault.class,
+ TooManyBucketsFault.class,
+ TooManyColumnsFault.class,
+ TooManyInputFilesFault.class,
+ TooManyPartitionsFault.class,
+ TooManyWarningsFault.class,
+ TooManyWorkersFault.class,
+ UnknownFault.class,
+ WorkerFailedFault.class,
+ WorkerRpcFailedFault.class
+ );
+
+ @Override
+ public List extends Module> getJacksonModules()
+ {
+ final SimpleModule module = new SimpleModule(getClass().getSimpleName());
+
+ module.registerSubtypes(
+ // Task classes
+ MSQControllerTask.class,
+ MSQWorkerTask.class,
+
+ // FrameChannelWorkerFactory and FrameChannelWorkerFactoryExtraInfoHolder classes
+ SegmentGeneratorFrameProcessorFactory.class,
+ SegmentGeneratorFrameProcessorFactory.SegmentGeneratorExtraInfoHolder.class,
+ ScanQueryFrameProcessorFactory.class,
+ GroupByPreShuffleFrameProcessorFactory.class,
+ GroupByPostShuffleFrameProcessorFactory.class,
+ OffsetLimitFrameProcessorFactory.class,
+ NilExtraInfoHolder.class,
+
+ // FrameChannelWorkerFactory and FrameChannelWorkerFactoryExtraInfoHolder classes
+ ScanQueryFrameProcessorFactory.class,
+ GroupByPreShuffleFrameProcessorFactory.class,
+ GroupByPostShuffleFrameProcessorFactory.class,
+ OffsetLimitFrameProcessorFactory.class,
+ NilExtraInfoHolder.class,
+
+ // DataSource classes (note: ExternalDataSource is in MSQSqlModule)
+ InputNumberDataSource.class,
+
+ // TaskReport classes
+ MSQTaskReport.class,
+
+ // QueryCounter.Snapshot classes
+ ChannelCounters.Snapshot.class,
+ SuperSorterProgressTrackerCounter.Snapshot.class,
+ WarningCounters.Snapshot.class,
+
+ // InputSpec classes
+ ExternalInputSpec.class,
+ StageInputSpec.class,
+ TableInputSpec.class,
+
+ // InputSlice classes
+ ExternalInputSlice.class,
+ NilInputSlice.class,
+ SegmentsInputSlice.class,
+ StageInputSlice.class,
+
+ // Other
+ PassthroughAggregatorFactory.class,
+ NilInputSource.class
+ );
+
+ FAULT_CLASSES.forEach(module::registerSubtypes);
+ module.addSerializer(new CounterSnapshotsSerializer());
+ return Collections.singletonList(module);
+ }
+
+ @Override
+ public void configure(Binder binder)
+ {
+ }
+
+ @Provides
+ @LazySingleton
+ public Bouncer makeBouncer(final DruidProcessingConfig processingConfig, @Self final Set nodeRoles)
+ {
+ if (nodeRoles.contains(NodeRole.PEON) && !nodeRoles.contains(NodeRole.INDEXER)) {
+ // CliPeon -> use only one thread regardless of configured # of processing threads. This matches the expected
+ // resource usage pattern for CliPeon-based tasks (one task / one working thread per JVM).
+ return new Bouncer(1);
+ } else {
+ return new Bouncer(processingConfig.getNumThreads());
+ }
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQServiceClientModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQServiceClientModule.java
new file mode 100644
index 00000000000..436fa7bebbe
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQServiceClientModule.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.msq.guice;
+
+import com.fasterxml.jackson.databind.Module;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.inject.Binder;
+import com.google.inject.Provides;
+import org.apache.druid.client.coordinator.Coordinator;
+import org.apache.druid.discovery.DruidNodeDiscoveryProvider;
+import org.apache.druid.discovery.NodeRole;
+import org.apache.druid.guice.ManageLifecycle;
+import org.apache.druid.guice.annotations.EscalatedGlobal;
+import org.apache.druid.guice.annotations.Json;
+import org.apache.druid.initialization.DruidModule;
+import org.apache.druid.msq.rpc.CoordinatorServiceClient;
+import org.apache.druid.msq.rpc.CoordinatorServiceClientImpl;
+import org.apache.druid.rpc.DiscoveryServiceLocator;
+import org.apache.druid.rpc.ServiceClientFactory;
+import org.apache.druid.rpc.ServiceLocator;
+import org.apache.druid.rpc.StandardRetryPolicy;
+
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Module for providing {@link CoordinatorServiceClient}.
+ */
+public class MSQServiceClientModule implements DruidModule
+{
+ private static final int COORDINATOR_ATTEMPTS = 6;
+
+ @Override
+ public List extends Module> getJacksonModules()
+ {
+ return Collections.emptyList();
+ }
+
+ @Override
+ public void configure(Binder binder)
+ {
+ // Nothing to do.
+ }
+
+ @Provides
+ @ManageLifecycle
+ @Coordinator
+ public ServiceLocator makeCoordinatorServiceLocator(final DruidNodeDiscoveryProvider discoveryProvider)
+ {
+ return new DiscoveryServiceLocator(discoveryProvider, NodeRole.COORDINATOR);
+ }
+
+ @Provides
+ public CoordinatorServiceClient makeCoordinatorServiceClient(
+ @Json final ObjectMapper jsonMapper,
+ @EscalatedGlobal final ServiceClientFactory clientFactory,
+ @Coordinator final ServiceLocator serviceLocator
+ )
+ {
+ return new CoordinatorServiceClientImpl(
+ clientFactory.makeClient(
+ NodeRole.COORDINATOR.getJsonName(),
+ serviceLocator,
+ StandardRetryPolicy.builder().maxAttempts(COORDINATOR_ATTEMPTS).build()
+ ),
+ jsonMapper
+ );
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQSqlModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQSqlModule.java
new file mode 100644
index 00000000000..9aea33f4072
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQSqlModule.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.msq.guice;
+
+import com.fasterxml.jackson.databind.Module;
+import com.google.inject.Binder;
+import com.google.inject.Inject;
+import org.apache.druid.discovery.NodeRole;
+import org.apache.druid.guice.annotations.LoadScope;
+import org.apache.druid.initialization.DruidModule;
+import org.apache.druid.metadata.input.InputSourceModule;
+import org.apache.druid.sql.calcite.external.ExternalOperatorConversion;
+import org.apache.druid.sql.guice.SqlBindings;
+
+import java.util.List;
+import java.util.Properties;
+
+/**
+ * Module for providing the {@code EXTERN} operator.
+ */
+@LoadScope(roles = NodeRole.BROKER_JSON_NAME)
+public class MSQSqlModule implements DruidModule
+{
+ @Inject
+ Properties properties = null;
+
+ @Override
+ public List extends Module> getJacksonModules()
+ {
+ // We want this module to bring input sources along for the ride.
+ return new InputSourceModule().getJacksonModules();
+ }
+
+ @Override
+ public void configure(Binder binder)
+ {
+ // We want this module to bring InputSourceModule along for the ride.
+ binder.install(new InputSourceModule());
+
+ // Set up the EXTERN macro.
+ SqlBindings.addOperatorConversion(binder, ExternalOperatorConversion.class);
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MultiStageQuery.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MultiStageQuery.java
new file mode 100644
index 00000000000..954b5fdd14b
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MultiStageQuery.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.msq.guice;
+
+import com.google.inject.BindingAnnotation;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+@Target({ElementType.METHOD})
+@Retention(RetentionPolicy.RUNTIME)
+@BindingAnnotation
+public @interface MultiStageQuery
+{
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/SqlTaskModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/SqlTaskModule.java
new file mode 100644
index 00000000000..d8f8b06f9b9
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/SqlTaskModule.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.msq.guice;
+
+import com.fasterxml.jackson.databind.Module;
+import com.google.inject.Binder;
+import org.apache.druid.discovery.NodeRole;
+import org.apache.druid.guice.Jerseys;
+import org.apache.druid.guice.LifecycleModule;
+import org.apache.druid.guice.annotations.LoadScope;
+import org.apache.druid.initialization.DruidModule;
+import org.apache.druid.msq.sql.SqlTaskResource;
+
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Module for adding the {@link SqlTaskResource} endpoint to the Broker.
+ */
+@LoadScope(roles = NodeRole.BROKER_JSON_NAME)
+public class SqlTaskModule implements DruidModule
+{
+ @Override
+ public void configure(Binder binder)
+ {
+ // Force eager initialization.
+ LifecycleModule.register(binder, SqlTaskResource.class);
+ Jerseys.addResource(binder, SqlTaskResource.class);
+ }
+
+ @Override
+ public List extends Module> getJacksonModules()
+ {
+ return Collections.emptyList();
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ColumnMapping.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ColumnMapping.java
new file mode 100644
index 00000000000..6e55270d6c2
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ColumnMapping.java
@@ -0,0 +1,82 @@
+/*
+ * 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.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+
+import java.util.Objects;
+
+public class ColumnMapping
+{
+ private final String queryColumn;
+ private final String outputColumn;
+
+ @JsonCreator
+ public ColumnMapping(
+ @JsonProperty("queryColumn") String queryColumn,
+ @JsonProperty("outputColumn") String outputColumn
+ )
+ {
+ this.queryColumn = Preconditions.checkNotNull(queryColumn, "queryColumn");
+ this.outputColumn = Preconditions.checkNotNull(outputColumn, "outputColumn");
+ }
+
+ @JsonProperty
+ public String getQueryColumn()
+ {
+ return queryColumn;
+ }
+
+ @JsonProperty
+ public String getOutputColumn()
+ {
+ return outputColumn;
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ ColumnMapping that = (ColumnMapping) o;
+ return Objects.equals(queryColumn, that.queryColumn) && Objects.equals(outputColumn, that.outputColumn);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(queryColumn, outputColumn);
+ }
+
+ @Override
+ public String toString()
+ {
+ return "ColumnMapping{" +
+ "queryColumn='" + queryColumn + '\'' +
+ ", outputColumn='" + outputColumn + '\'' +
+ '}';
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ColumnMappings.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ColumnMappings.java
new file mode 100644
index 00000000000..fddfbeefc76
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ColumnMappings.java
@@ -0,0 +1,144 @@
+/*
+ * 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.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonValue;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.segment.column.RowSignature;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class ColumnMappings
+{
+ private final List mappings;
+ private final Map outputToQueryColumnMap;
+ private final Map> queryToOutputColumnsMap;
+
+ @JsonCreator
+ public ColumnMappings(final List mappings)
+ {
+ this.mappings = validateNoDuplicateOutputColumns(Preconditions.checkNotNull(mappings, "mappings"));
+ this.outputToQueryColumnMap = new HashMap<>();
+ this.queryToOutputColumnsMap = new HashMap<>();
+
+ for (final ColumnMapping mapping : mappings) {
+ outputToQueryColumnMap.put(mapping.getOutputColumn(), mapping.getQueryColumn());
+ queryToOutputColumnsMap.computeIfAbsent(mapping.getQueryColumn(), k -> new ArrayList<>())
+ .add(mapping.getOutputColumn());
+ }
+ }
+
+ public static ColumnMappings identity(final RowSignature signature)
+ {
+ return new ColumnMappings(
+ signature.getColumnNames()
+ .stream()
+ .map(column -> new ColumnMapping(column, column))
+ .collect(Collectors.toList())
+ );
+ }
+
+ public List getOutputColumnNames()
+ {
+ return mappings.stream().map(ColumnMapping::getOutputColumn).collect(Collectors.toList());
+ }
+
+ public boolean hasOutputColumn(final String columnName)
+ {
+ return outputToQueryColumnMap.containsKey(columnName);
+ }
+
+ public String getQueryColumnForOutputColumn(final String outputColumn)
+ {
+ final String queryColumn = outputToQueryColumnMap.get(outputColumn);
+ if (queryColumn != null) {
+ return queryColumn;
+ } else {
+ throw new IAE("No such output column [%s]", outputColumn);
+ }
+ }
+
+ public List getOutputColumnsForQueryColumn(final String queryColumn)
+ {
+ final List outputColumns = queryToOutputColumnsMap.get(queryColumn);
+ if (outputColumns != null) {
+ return outputColumns;
+ } else {
+ return Collections.emptyList();
+ }
+ }
+
+ @JsonValue
+ public List getMappings()
+ {
+ return mappings;
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ ColumnMappings that = (ColumnMappings) o;
+ return Objects.equals(mappings, that.mappings);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(mappings);
+ }
+
+ @Override
+ public String toString()
+ {
+ return "ColumnMappings{" +
+ "mappings=" + mappings +
+ '}';
+ }
+
+ private static List validateNoDuplicateOutputColumns(final List mappings)
+ {
+ final Set encountered = new HashSet<>();
+
+ for (final ColumnMapping mapping : mappings) {
+ if (!encountered.add(mapping.getOutputColumn())) {
+ throw new ISE("Duplicate output column [%s]", mapping.getOutputColumn());
+ }
+ }
+
+ return mappings;
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ControllerChatHandler.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ControllerChatHandler.java
new file mode 100644
index 00000000000..5561a98a07b
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ControllerChatHandler.java
@@ -0,0 +1,194 @@
+/*
+ * 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 org.apache.druid.indexing.common.TaskReport;
+import org.apache.druid.indexing.common.TaskToolbox;
+import org.apache.druid.msq.counters.CounterSnapshots;
+import org.apache.druid.msq.counters.CounterSnapshotsTree;
+import org.apache.druid.msq.exec.Controller;
+import org.apache.druid.msq.exec.ControllerClient;
+import org.apache.druid.msq.indexing.error.MSQErrorReport;
+import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
+import org.apache.druid.segment.realtime.firehose.ChatHandler;
+import org.apache.druid.segment.realtime.firehose.ChatHandlers;
+import org.apache.druid.server.security.Action;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.util.List;
+import java.util.Map;
+
+public class ControllerChatHandler implements ChatHandler
+{
+ private final Controller controller;
+ private final MSQControllerTask task;
+ private final TaskToolbox toolbox;
+
+ public ControllerChatHandler(TaskToolbox toolbox, Controller controller)
+ {
+ this.controller = controller;
+ this.task = controller.task();
+ this.toolbox = toolbox;
+ }
+
+ /**
+ * Used by subtasks to post {@link ClusterByStatisticsSnapshot} for shuffling stages.
+ *
+ * See {@link ControllerClient#postKeyStatistics} for the client-side code that calls this API.
+ */
+ @POST
+ @Path("/keyStatistics/{queryId}/{stageNumber}/{workerNumber}")
+ @Produces(MediaType.APPLICATION_JSON)
+ @Consumes(MediaType.APPLICATION_JSON)
+ public Response httpPostKeyStatistics(
+ final Object keyStatisticsObject,
+ @PathParam("queryId") final String queryId,
+ @PathParam("stageNumber") final int stageNumber,
+ @PathParam("workerNumber") final int workerNumber,
+ @Context final HttpServletRequest req
+ )
+ {
+ ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper());
+ controller.updateStatus(stageNumber, workerNumber, keyStatisticsObject);
+ return Response.status(Response.Status.ACCEPTED).build();
+ }
+
+ /**
+ * Used by subtasks to post system errors. Note that the errors are organized by taskId, not by query/stage/worker,
+ * because system errors are associated with a task rather than a specific query/stage/worker execution context.
+ *
+ * See {@link ControllerClient#postWorkerError} for the client-side code that calls this API.
+ */
+ @POST
+ @Path("/workerError/{taskId}")
+ @Produces(MediaType.APPLICATION_JSON)
+ @Consumes(MediaType.APPLICATION_JSON)
+ public Response httpPostWorkerError(
+ final MSQErrorReport errorReport,
+ @PathParam("taskId") final String taskId,
+ @Context final HttpServletRequest req
+ )
+ {
+ ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper());
+ controller.workerError(errorReport);
+ return Response.status(Response.Status.ACCEPTED).build();
+ }
+
+ /**
+ * Used by subtasks to post system warnings.
+ *
+ * See {@link ControllerClient#postWorkerWarning} for the client-side code that calls this API.
+ */
+ @POST
+ @Path("/workerWarning/{taskId}")
+ @Produces(MediaType.APPLICATION_JSON)
+ @Consumes(MediaType.APPLICATION_JSON)
+ public Response httpPostWorkerWarning(
+ final List errorReport,
+ @PathParam("taskId") final String taskId,
+ @Context final HttpServletRequest req
+ )
+ {
+ ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper());
+ controller.workerWarning(errorReport);
+ return Response.status(Response.Status.ACCEPTED).build();
+ }
+
+
+ /**
+ * Used by subtasks to post {@link CounterSnapshots} periodically.
+ *
+ * See {@link ControllerClient#postCounters} for the client-side code that calls this API.
+ */
+ @POST
+ @Path("/counters")
+ @Produces(MediaType.APPLICATION_JSON)
+ @Consumes(MediaType.APPLICATION_JSON)
+ public Response httpPostCounters(
+ final CounterSnapshotsTree snapshotsTree,
+ @Context final HttpServletRequest req
+ )
+ {
+ ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper());
+ controller.updateCounters(snapshotsTree);
+ return Response.status(Response.Status.OK).build();
+ }
+
+ /**
+ * Used by subtasks to post notifications that their results are ready.
+ *
+ * See {@link ControllerClient#postResultsComplete} for the client-side code that calls this API.
+ */
+ @POST
+ @Path("/resultsComplete/{queryId}/{stageNumber}/{workerNumber}")
+ @Produces(MediaType.APPLICATION_JSON)
+ @Consumes(MediaType.APPLICATION_JSON)
+ public Response httpPostResultsComplete(
+ final Object resultObject,
+ @PathParam("queryId") final String queryId,
+ @PathParam("stageNumber") final int stageNumber,
+ @PathParam("workerNumber") final int workerNumber,
+ @Context final HttpServletRequest req
+ )
+ {
+ ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper());
+ controller.resultsComplete(queryId, stageNumber, workerNumber, resultObject);
+ return Response.status(Response.Status.ACCEPTED).build();
+ }
+
+ /**
+ * See {@link ControllerClient#getTaskList} for the client-side code that calls this API.
+ */
+ @GET
+ @Path("/taskList")
+ @Produces(MediaType.APPLICATION_JSON)
+ public Response httpGetTaskList(@Context final HttpServletRequest req)
+ {
+ ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper());
+
+ return Response.ok(new MSQTaskList(controller.getTaskIds())).build();
+ }
+
+ /**
+ * See {@link org.apache.druid.indexing.overlord.RemoteTaskRunner#streamTaskReports} for the client-side code that
+ * calls this API.
+ */
+ @GET
+ @Path("/liveReports")
+ @Produces(MediaType.APPLICATION_JSON)
+ public Response httpGetLiveReports(@Context final HttpServletRequest req)
+ {
+ ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper());
+ final Map reports = controller.liveReports();
+ if (reports == null) {
+ return Response.status(Response.Status.NOT_FOUND).build();
+ }
+ return Response.ok(reports).build();
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountableInputSourceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountableInputSourceReader.java
new file mode 100644
index 00000000000..54b8ac9a8cd
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountableInputSourceReader.java
@@ -0,0 +1,58 @@
+/*
+ * 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 org.apache.druid.data.input.InputRow;
+import org.apache.druid.data.input.InputRowListPlusRawValues;
+import org.apache.druid.data.input.InputSourceReader;
+import org.apache.druid.java.util.common.parsers.CloseableIterator;
+import org.apache.druid.msq.counters.ChannelCounters;
+
+import java.io.IOException;
+
+public class CountableInputSourceReader implements InputSourceReader
+{
+ private final InputSourceReader inputSourceReader;
+ private final ChannelCounters channelCounters;
+
+ public CountableInputSourceReader(
+ final InputSourceReader inputSourceReader,
+ final ChannelCounters channelCounters
+ )
+ {
+ this.inputSourceReader = inputSourceReader;
+ this.channelCounters = channelCounters;
+ }
+
+ @Override
+ public CloseableIterator read() throws IOException
+ {
+ return inputSourceReader.read().map(inputRow -> {
+ channelCounters.incrementRowCount();
+ return inputRow;
+ });
+ }
+
+ @Override
+ public CloseableIterator sample() throws IOException
+ {
+ return inputSourceReader.sample();
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingOutputChannelFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingOutputChannelFactory.java
new file mode 100644
index 00000000000..0521389d667
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingOutputChannelFactory.java
@@ -0,0 +1,64 @@
+/*
+ * 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.google.common.base.Preconditions;
+import org.apache.druid.frame.processor.OutputChannel;
+import org.apache.druid.frame.processor.OutputChannelFactory;
+import org.apache.druid.msq.counters.ChannelCounters;
+
+import java.io.IOException;
+
+public class CountingOutputChannelFactory implements OutputChannelFactory
+{
+ private final OutputChannelFactory baseFactory;
+ private final ChannelCounters channelCounters;
+
+ public CountingOutputChannelFactory(
+ final OutputChannelFactory baseFactory,
+ final ChannelCounters channelCounters
+ )
+ {
+ this.baseFactory = Preconditions.checkNotNull(baseFactory, "baseFactory");
+ this.channelCounters = Preconditions.checkNotNull(channelCounters, "channelCounter");
+ }
+
+ @Override
+ public OutputChannel openChannel(int partitionNumber) throws IOException
+ {
+ final OutputChannel baseChannel = baseFactory.openChannel(partitionNumber);
+
+ return baseChannel.mapWritableChannel(
+ baseWritableChannel ->
+ new CountingWritableFrameChannel(
+ baseChannel.getWritableChannel(),
+ channelCounters,
+ baseChannel.getPartitionNumber()
+ )
+ );
+ }
+
+ @Override
+ public OutputChannel openNilChannel(final int partitionNumber)
+ {
+ // No need for counters on nil channels: they never receive input.
+ return baseFactory.openNilChannel(partitionNumber);
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingReadableFrameChannel.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingReadableFrameChannel.java
new file mode 100644
index 00000000000..6462832acbf
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingReadableFrameChannel.java
@@ -0,0 +1,75 @@
+/*
+ * 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.google.common.util.concurrent.ListenableFuture;
+import org.apache.druid.frame.Frame;
+import org.apache.druid.frame.channel.ReadableFrameChannel;
+import org.apache.druid.msq.counters.ChannelCounters;
+
+public class CountingReadableFrameChannel implements ReadableFrameChannel
+{
+ private final ReadableFrameChannel baseChannel;
+ private final ChannelCounters channelCounters;
+ private final int partitionNumber;
+
+ public CountingReadableFrameChannel(
+ ReadableFrameChannel baseChannel,
+ ChannelCounters channelCounters,
+ int partitionNumber
+ )
+ {
+ this.baseChannel = baseChannel;
+ this.channelCounters = channelCounters;
+ this.partitionNumber = partitionNumber;
+ }
+
+ @Override
+ public boolean isFinished()
+ {
+ return baseChannel.isFinished();
+ }
+
+ @Override
+ public boolean canRead()
+ {
+ return baseChannel.canRead();
+ }
+
+ @Override
+ public Frame read()
+ {
+ final Frame frame = baseChannel.read();
+ channelCounters.addFrame(partitionNumber, frame);
+ return frame;
+ }
+
+ @Override
+ public ListenableFuture> readabilityFuture()
+ {
+ return baseChannel.readabilityFuture();
+ }
+
+ @Override
+ public void close()
+ {
+ baseChannel.close();
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingWritableFrameChannel.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingWritableFrameChannel.java
new file mode 100644
index 00000000000..1daf6ef32ef
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingWritableFrameChannel.java
@@ -0,0 +1,71 @@
+/*
+ * 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.google.common.util.concurrent.ListenableFuture;
+import org.apache.druid.frame.channel.FrameWithPartition;
+import org.apache.druid.frame.channel.WritableFrameChannel;
+import org.apache.druid.msq.counters.ChannelCounters;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+
+public class CountingWritableFrameChannel implements WritableFrameChannel
+{
+ private final WritableFrameChannel baseChannel;
+ private final ChannelCounters channelCounters;
+ private final int partitionNumber;
+
+ public CountingWritableFrameChannel(
+ final WritableFrameChannel baseChannel,
+ final ChannelCounters channelCounters,
+ final int partitionNumber
+ )
+ {
+ this.baseChannel = baseChannel;
+ this.channelCounters = channelCounters;
+ this.partitionNumber = partitionNumber;
+ }
+
+ @Override
+ public void write(FrameWithPartition frame) throws IOException
+ {
+ baseChannel.write(frame);
+ channelCounters.addFrame(partitionNumber, frame.frame());
+ }
+
+ @Override
+ public void fail(@Nullable Throwable cause) throws IOException
+ {
+ baseChannel.fail(cause);
+ }
+
+ @Override
+ public void close() throws IOException
+ {
+ baseChannel.close();
+ }
+
+ @Override
+ public ListenableFuture> writabilityFuture()
+ {
+ return baseChannel.writabilityFuture();
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/DataSourceMSQDestination.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/DataSourceMSQDestination.java
new file mode 100644
index 00000000000..6b1a683ff0e
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/DataSourceMSQDestination.java
@@ -0,0 +1,163 @@
+/*
+ * 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.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.granularity.Granularity;
+import org.joda.time.Interval;
+
+import javax.annotation.Nullable;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+public class DataSourceMSQDestination implements MSQDestination
+{
+ static final String TYPE = "dataSource";
+
+ private final String dataSource;
+ private final Granularity segmentGranularity;
+ private final List segmentSortOrder;
+
+ @Nullable
+ private final List replaceTimeChunks;
+
+ @JsonCreator
+ public DataSourceMSQDestination(
+ @JsonProperty("dataSource") String dataSource,
+ @JsonProperty("segmentGranularity") Granularity segmentGranularity,
+ @JsonProperty("segmentSortOrder") @Nullable List segmentSortOrder,
+ @JsonProperty("replaceTimeChunks") @Nullable List replaceTimeChunks
+ )
+ {
+ this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource");
+ this.segmentGranularity = Preconditions.checkNotNull(segmentGranularity, "segmentGranularity");
+ this.segmentSortOrder = segmentSortOrder != null ? segmentSortOrder : Collections.emptyList();
+ this.replaceTimeChunks = replaceTimeChunks;
+
+ if (replaceTimeChunks != null) {
+ // Verify that if replaceTimeChunks is provided, it is nonempty.
+ if (replaceTimeChunks.isEmpty()) {
+ throw new IAE("replaceTimeChunks must be null or nonempty; cannot be empty");
+ }
+
+ // Verify all provided time chunks are aligned with segmentGranularity.
+ for (final Interval interval : replaceTimeChunks) {
+ // ETERNITY gets a free pass.
+ if (!Intervals.ETERNITY.equals(interval)) {
+ final boolean startIsAligned =
+ segmentGranularity.bucketStart(interval.getStart()).equals(interval.getStart());
+
+ final boolean endIsAligned =
+ segmentGranularity.bucketStart(interval.getEnd()).equals(interval.getEnd())
+ || segmentGranularity.increment(segmentGranularity.bucketStart(interval.getEnd()))
+ .equals(interval.getEnd());
+
+ if (!startIsAligned || !endIsAligned) {
+ throw new IAE(
+ "Time chunk [%s] provided in replaceTimeChunks is not aligned with segmentGranularity [%s]",
+ interval,
+ segmentGranularity
+ );
+ }
+ }
+ }
+ }
+ }
+
+ @JsonProperty
+ public String getDataSource()
+ {
+ return dataSource;
+ }
+
+ @JsonProperty
+ public Granularity getSegmentGranularity()
+ {
+ return segmentGranularity;
+ }
+
+ @JsonProperty
+ @JsonInclude(JsonInclude.Include.NON_EMPTY)
+ public List getSegmentSortOrder()
+ {
+ return segmentSortOrder;
+ }
+
+ /**
+ * Returns the list of time chunks to replace, or null if {@link #isReplaceTimeChunks()} is false.
+ *
+ * Invariants: if nonnull, then this will *also* be nonempty, and all intervals will be aligned
+ * with {@link #getSegmentGranularity()}. Each interval may comprise multiple time chunks.
+ */
+ @Nullable
+ @JsonProperty
+ @JsonInclude(JsonInclude.Include.NON_NULL)
+ public List getReplaceTimeChunks()
+ {
+ return replaceTimeChunks;
+ }
+
+ /**
+ * Whether this object is in replace-existing-time-chunks mode.
+ */
+ public boolean isReplaceTimeChunks()
+ {
+ return replaceTimeChunks != null;
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ DataSourceMSQDestination that = (DataSourceMSQDestination) o;
+ return Objects.equals(dataSource, that.dataSource)
+ && Objects.equals(segmentGranularity, that.segmentGranularity)
+ && Objects.equals(segmentSortOrder, that.segmentSortOrder)
+ && Objects.equals(replaceTimeChunks, that.replaceTimeChunks);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(dataSource, segmentGranularity, segmentSortOrder, replaceTimeChunks);
+ }
+
+ @Override
+ public String toString()
+ {
+ return "DataSourceMSQDestination{" +
+ "dataSource='" + dataSource + '\'' +
+ ", segmentGranularity=" + segmentGranularity +
+ ", segmentSortOrder=" + segmentSortOrder +
+ ", replaceTimeChunks=" + replaceTimeChunks +
+ '}';
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerClient.java
new file mode 100644
index 00000000000..2c249e26398
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerClient.java
@@ -0,0 +1,182 @@
+/*
+ * 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.databind.ObjectMapper;
+import org.apache.druid.common.guava.FutureUtils;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.http.client.response.BytesFullResponseHandler;
+import org.apache.druid.java.util.http.client.response.BytesFullResponseHolder;
+import org.apache.druid.java.util.http.client.response.HttpResponseHandler;
+import org.apache.druid.msq.counters.CounterSnapshotsTree;
+import org.apache.druid.msq.exec.ControllerClient;
+import org.apache.druid.msq.indexing.error.MSQErrorReport;
+import org.apache.druid.msq.kernel.StageId;
+import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
+import org.apache.druid.rpc.IgnoreHttpResponseHandler;
+import org.apache.druid.rpc.RequestBuilder;
+import org.apache.druid.rpc.ServiceClient;
+import org.jboss.netty.handler.codec.http.HttpMethod;
+
+import javax.annotation.Nullable;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+
+public class IndexerControllerClient implements ControllerClient
+{
+ private final ServiceClient serviceClient;
+ private final ObjectMapper jsonMapper;
+ private final Closeable baggage;
+
+ public IndexerControllerClient(
+ final ServiceClient serviceClient,
+ final ObjectMapper jsonMapper,
+ final Closeable baggage
+ )
+ {
+ this.serviceClient = serviceClient;
+ this.jsonMapper = jsonMapper;
+ this.baggage = baggage;
+ }
+
+ @Override
+ public void postKeyStatistics(
+ StageId stageId,
+ int workerNumber,
+ ClusterByStatisticsSnapshot keyStatistics
+ ) throws IOException
+ {
+ final String path = StringUtils.format(
+ "/keyStatistics/%s/%s/%d",
+ StringUtils.urlEncode(stageId.getQueryId()),
+ stageId.getStageNumber(),
+ workerNumber
+ );
+
+ doRequest(
+ new RequestBuilder(HttpMethod.POST, path)
+ .jsonContent(jsonMapper, keyStatistics),
+ IgnoreHttpResponseHandler.INSTANCE
+ );
+ }
+
+ @Override
+ public void postCounters(CounterSnapshotsTree snapshotsTree) throws IOException
+ {
+ doRequest(
+ new RequestBuilder(HttpMethod.POST, "/counters")
+ .jsonContent(jsonMapper, snapshotsTree),
+ IgnoreHttpResponseHandler.INSTANCE
+ );
+ }
+
+ @Override
+ public void postResultsComplete(StageId stageId, int workerNumber, @Nullable Object resultObject) throws IOException
+ {
+ final String path = StringUtils.format(
+ "/resultsComplete/%s/%s/%d",
+ StringUtils.urlEncode(stageId.getQueryId()),
+ stageId.getStageNumber(),
+ workerNumber
+ );
+
+ doRequest(
+ new RequestBuilder(HttpMethod.POST, path)
+ .jsonContent(jsonMapper, resultObject),
+ IgnoreHttpResponseHandler.INSTANCE
+ );
+ }
+
+ @Override
+ public void postWorkerError(String workerId, MSQErrorReport errorWrapper) throws IOException
+ {
+ final String path = StringUtils.format(
+ "/workerError/%s",
+ StringUtils.urlEncode(workerId)
+ );
+
+ doRequest(
+ new RequestBuilder(HttpMethod.POST, path)
+ .jsonContent(jsonMapper, errorWrapper),
+ IgnoreHttpResponseHandler.INSTANCE
+ );
+ }
+
+ @Override
+ public void postWorkerWarning(String workerId, List MSQErrorReports) throws IOException
+ {
+ final String path = StringUtils.format(
+ "/workerWarning/%s",
+ StringUtils.urlEncode(workerId)
+ );
+
+ doRequest(
+ new RequestBuilder(HttpMethod.POST, path)
+ .jsonContent(jsonMapper, MSQErrorReports),
+ IgnoreHttpResponseHandler.INSTANCE
+ );
+ }
+
+ @Override
+ public List getTaskList() throws IOException
+ {
+ final BytesFullResponseHolder retVal = doRequest(
+ new RequestBuilder(HttpMethod.GET, "/taskList"),
+ new BytesFullResponseHandler()
+ );
+
+ final MSQTaskList taskList = jsonMapper.readValue(retVal.getContent(), MSQTaskList.class);
+ return taskList.getTaskIds();
+ }
+
+ @Override
+ public void close()
+ {
+ try {
+ baggage.close();
+ }
+ catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ /**
+ * Similar to {@link ServiceClient#request}, but preserves IOExceptions rather than wrapping them in
+ * {@link ExecutionException}.
+ */
+ private FinalType doRequest(
+ RequestBuilder requestBuilder,
+ HttpResponseHandler handler
+ ) throws IOException
+ {
+ try {
+ return FutureUtils.get(serviceClient.asyncRequest(requestBuilder, handler), true);
+ }
+ catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ throw new RuntimeException(e);
+ }
+ catch (ExecutionException e) {
+ throw new IOException(e.getCause());
+ }
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerContext.java
new file mode 100644
index 00000000000..cc767626ecb
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerContext.java
@@ -0,0 +1,123 @@
+/*
+ * 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.databind.ObjectMapper;
+import com.google.inject.Injector;
+import com.google.inject.Key;
+import org.apache.druid.client.coordinator.CoordinatorClient;
+import org.apache.druid.guice.annotations.Self;
+import org.apache.druid.indexing.common.TaskReport;
+import org.apache.druid.indexing.common.TaskToolbox;
+import org.apache.druid.indexing.common.actions.TaskActionClient;
+import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.msq.exec.Controller;
+import org.apache.druid.msq.exec.ControllerContext;
+import org.apache.druid.msq.exec.WorkerClient;
+import org.apache.druid.msq.exec.WorkerManagerClient;
+import org.apache.druid.rpc.ServiceClientFactory;
+import org.apache.druid.rpc.indexing.OverlordClient;
+import org.apache.druid.segment.realtime.firehose.ChatHandler;
+import org.apache.druid.server.DruidNode;
+
+import java.util.Map;
+
+/**
+ * Implementation for {@link ControllerContext} required to run multi-stage queries as indexing tasks.
+ */
+public class IndexerControllerContext implements ControllerContext
+{
+ private final TaskToolbox toolbox;
+ private final Injector injector;
+ private final ServiceClientFactory clientFactory;
+ private final OverlordClient overlordClient;
+ private final WorkerManagerClient workerManager;
+
+ public IndexerControllerContext(
+ final TaskToolbox toolbox,
+ final Injector injector,
+ final ServiceClientFactory clientFactory,
+ final OverlordClient overlordClient
+ )
+ {
+ this.toolbox = toolbox;
+ this.injector = injector;
+ this.clientFactory = clientFactory;
+ this.overlordClient = overlordClient;
+ this.workerManager = new IndexerWorkerManagerClient(overlordClient);
+ }
+
+ @Override
+ public ObjectMapper jsonMapper()
+ {
+ return toolbox.getJsonMapper();
+ }
+
+ @Override
+ public Injector injector()
+ {
+ return injector;
+ }
+
+ @Override
+ public DruidNode selfNode()
+ {
+ return injector.getInstance(Key.get(DruidNode.class, Self.class));
+ }
+
+ @Override
+ public CoordinatorClient coordinatorClient()
+ {
+ return toolbox.getCoordinatorClient();
+ }
+
+ @Override
+ public TaskActionClient taskActionClient()
+ {
+ return toolbox.getTaskActionClient();
+ }
+
+ @Override
+ public WorkerClient taskClientFor(Controller controller)
+ {
+ // Ignore controller parameter.
+ return new IndexerWorkerClient(clientFactory, overlordClient, jsonMapper());
+ }
+
+ @Override
+ public void registerController(Controller controller, final Closer closer)
+ {
+ ChatHandler chatHandler = new ControllerChatHandler(toolbox, controller);
+ toolbox.getChatHandlerProvider().register(controller.id(), chatHandler, false);
+ closer.register(() -> toolbox.getChatHandlerProvider().unregister(controller.id()));
+ }
+
+ @Override
+ public WorkerManagerClient workerManager()
+ {
+ return workerManager;
+ }
+
+ @Override
+ public void writeReports(String controllerTaskId, Map reports)
+ {
+ toolbox.getTaskReportFileWriter().write(controllerTaskId, reports);
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java
new file mode 100644
index 00000000000..65e55545026
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java
@@ -0,0 +1,120 @@
+/*
+ * 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.databind.ObjectMapper;
+import org.apache.druid.msq.exec.WorkerMemoryParameters;
+import org.apache.druid.msq.kernel.FrameContext;
+import org.apache.druid.msq.querykit.DataSegmentProvider;
+import org.apache.druid.query.groupby.strategy.GroupByStrategySelector;
+import org.apache.druid.segment.IndexIO;
+import org.apache.druid.segment.IndexMergerV9;
+import org.apache.druid.segment.incremental.RowIngestionMeters;
+import org.apache.druid.segment.join.JoinableFactory;
+import org.apache.druid.segment.loading.DataSegmentPusher;
+
+import java.io.File;
+
+public class IndexerFrameContext implements FrameContext
+{
+ private final IndexerWorkerContext context;
+ private final IndexIO indexIO;
+ private final DataSegmentProvider dataSegmentProvider;
+ private final WorkerMemoryParameters memoryParameters;
+
+ public IndexerFrameContext(
+ IndexerWorkerContext context,
+ IndexIO indexIO,
+ DataSegmentProvider dataSegmentProvider,
+ WorkerMemoryParameters memoryParameters
+ )
+ {
+ this.context = context;
+ this.indexIO = indexIO;
+ this.dataSegmentProvider = dataSegmentProvider;
+ this.memoryParameters = memoryParameters;
+ }
+
+ @Override
+ public JoinableFactory joinableFactory()
+ {
+ return context.injector().getInstance(JoinableFactory.class);
+ }
+
+ @Override
+ public GroupByStrategySelector groupByStrategySelector()
+ {
+ return context.injector().getInstance(GroupByStrategySelector.class);
+ }
+
+ @Override
+ public RowIngestionMeters rowIngestionMeters()
+ {
+ return context.toolbox().getRowIngestionMetersFactory().createRowIngestionMeters();
+ }
+
+ @Override
+ public DataSegmentProvider dataSegmentProvider()
+ {
+ return dataSegmentProvider;
+ }
+
+ @Override
+ public File tempDir()
+ {
+ return context.tempDir();
+ }
+
+ @Override
+ public ObjectMapper jsonMapper()
+ {
+ return context.jsonMapper();
+ }
+
+ @Override
+ public IndexIO indexIO()
+ {
+ return indexIO;
+ }
+
+ @Override
+ public File persistDir()
+ {
+ return context.toolbox().getPersistDir();
+ }
+
+ @Override
+ public DataSegmentPusher segmentPusher()
+ {
+ return context.toolbox().getSegmentPusher();
+ }
+
+ @Override
+ public IndexMergerV9 indexMerger()
+ {
+ return context.toolbox().getIndexMergerV9();
+ }
+
+ @Override
+ public WorkerMemoryParameters memoryParameters()
+ {
+ return memoryParameters;
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerClient.java
new file mode 100644
index 00000000000..430c264291f
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerClient.java
@@ -0,0 +1,271 @@
+/*
+ * 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.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.SettableFuture;
+import com.google.errorprone.annotations.concurrent.GuardedBy;
+import org.apache.druid.common.guava.FutureUtils;
+import org.apache.druid.frame.channel.ReadableByteChunksFrameChannel;
+import org.apache.druid.frame.file.FrameFileHttpResponseHandler;
+import org.apache.druid.frame.file.FrameFilePartialFetch;
+import org.apache.druid.frame.key.ClusterByPartitions;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.http.client.response.BytesFullResponseHandler;
+import org.apache.druid.java.util.http.client.response.BytesFullResponseHolder;
+import org.apache.druid.msq.counters.CounterSnapshotsTree;
+import org.apache.druid.msq.exec.WorkerClient;
+import org.apache.druid.msq.kernel.StageId;
+import org.apache.druid.msq.kernel.WorkOrder;
+import org.apache.druid.rpc.IgnoreHttpResponseHandler;
+import org.apache.druid.rpc.RequestBuilder;
+import org.apache.druid.rpc.ServiceClient;
+import org.apache.druid.rpc.ServiceClientFactory;
+import org.apache.druid.rpc.StandardRetryPolicy;
+import org.apache.druid.rpc.indexing.OverlordClient;
+import org.apache.druid.rpc.indexing.SpecificTaskRetryPolicy;
+import org.apache.druid.rpc.indexing.SpecificTaskServiceLocator;
+import org.apache.druid.utils.CloseableUtils;
+import org.jboss.netty.handler.codec.http.HttpMethod;
+
+import javax.annotation.Nonnull;
+import javax.ws.rs.core.HttpHeaders;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+public class IndexerWorkerClient implements WorkerClient
+{
+ private final ServiceClientFactory clientFactory;
+ private final OverlordClient overlordClient;
+ private final ObjectMapper jsonMapper;
+
+ @GuardedBy("clientMap")
+ private final Map> clientMap = new HashMap<>();
+
+ public IndexerWorkerClient(
+ final ServiceClientFactory clientFactory,
+ final OverlordClient overlordClient,
+ final ObjectMapper jsonMapper
+ )
+ {
+ this.clientFactory = clientFactory;
+ this.overlordClient = overlordClient;
+ this.jsonMapper = jsonMapper;
+ }
+
+
+ @Nonnull
+ public static String getStagePartitionPath(StageId stageId, int partitionNumber)
+ {
+ return StringUtils.format(
+ "/channels/%s/%d/%d",
+ StringUtils.urlEncode(stageId.getQueryId()),
+ stageId.getStageNumber(),
+ partitionNumber
+ );
+ }
+
+ @Override
+ public ListenableFuture postWorkOrder(String workerTaskId, WorkOrder workOrder)
+ {
+ return getClient(workerTaskId).asyncRequest(
+ new RequestBuilder(HttpMethod.POST, "/workOrder")
+ .jsonContent(jsonMapper, workOrder),
+ IgnoreHttpResponseHandler.INSTANCE
+ );
+ }
+
+ @Override
+ public ListenableFuture postResultPartitionBoundaries(
+ String workerTaskId,
+ StageId stageId,
+ ClusterByPartitions partitionBoundaries
+ )
+ {
+ final String path = StringUtils.format(
+ "/resultPartitionBoundaries/%s/%d",
+ StringUtils.urlEncode(stageId.getQueryId()),
+ stageId.getStageNumber()
+ );
+
+ return getClient(workerTaskId).asyncRequest(
+ new RequestBuilder(HttpMethod.POST, path)
+ .jsonContent(jsonMapper, partitionBoundaries),
+ IgnoreHttpResponseHandler.INSTANCE
+ );
+ }
+
+ /**
+ * Client-side method for {@link WorkerChatHandler#httpPostCleanupStage}.
+ */
+ @Override
+ public ListenableFuture postCleanupStage(
+ final String workerTaskId,
+ final StageId stageId
+ )
+ {
+ final String path = StringUtils.format(
+ "/cleanupStage/%s/%d",
+ StringUtils.urlEncode(stageId.getQueryId()),
+ stageId.getStageNumber()
+ );
+
+ return getClient(workerTaskId).asyncRequest(
+ new RequestBuilder(HttpMethod.POST, path),
+ IgnoreHttpResponseHandler.INSTANCE
+ );
+ }
+
+ @Override
+ public ListenableFuture postFinish(String workerTaskId)
+ {
+ return getClient(workerTaskId).asyncRequest(
+ new RequestBuilder(HttpMethod.POST, "/finish"),
+ IgnoreHttpResponseHandler.INSTANCE
+ );
+ }
+
+ @Override
+ public ListenableFuture getCounters(String workerTaskId)
+ {
+ return FutureUtils.transform(
+ getClient(workerTaskId).asyncRequest(
+ new RequestBuilder(HttpMethod.GET, "/counters"),
+ new BytesFullResponseHandler()
+ ),
+ holder -> deserialize(holder, new TypeReference() {})
+ );
+ }
+
+ private static final Logger log = new Logger(IndexerWorkerClient.class);
+
+ @Override
+ public ListenableFuture fetchChannelData(
+ String workerTaskId,
+ StageId stageId,
+ int partitionNumber,
+ long offset,
+ ReadableByteChunksFrameChannel channel
+ )
+ {
+ final ServiceClient client = getClient(workerTaskId);
+ final String path = getStagePartitionPath(stageId, partitionNumber);
+
+ final SettableFuture retVal = SettableFuture.create();
+ final ListenableFuture clientFuture =
+ client.asyncRequest(
+ new RequestBuilder(HttpMethod.GET, StringUtils.format("%s?offset=%d", path, offset))
+ .header(HttpHeaders.ACCEPT_ENCODING, "identity"), // Data is compressed at app level
+ new FrameFileHttpResponseHandler(channel)
+ );
+
+ Futures.addCallback(
+ clientFuture,
+ new FutureCallback()
+ {
+ @Override
+ public void onSuccess(FrameFilePartialFetch partialFetch)
+ {
+ if (partialFetch.isExceptionCaught()) {
+ // Exception while reading channel. Recoverable.
+ log.noStackTrace().info(
+ partialFetch.getExceptionCaught(),
+ "Encountered exception while reading channel [%s]",
+ channel.getId()
+ );
+ }
+
+ // Empty fetch means this is the last fetch for the channel.
+ partialFetch.backpressureFuture().addListener(
+ () -> retVal.set(partialFetch.isLastFetch()),
+ Execs.directExecutor()
+ );
+ }
+
+ @Override
+ public void onFailure(Throwable t)
+ {
+ retVal.setException(t);
+ }
+ }
+ );
+
+ return retVal;
+ }
+
+ @Override
+ public void close() throws IOException
+ {
+ synchronized (clientMap) {
+ try {
+ final List closeables =
+ clientMap.values().stream().map(pair -> pair.rhs).collect(Collectors.toList());
+ CloseableUtils.closeAll(closeables);
+ }
+ finally {
+ clientMap.clear();
+ }
+ }
+ }
+
+ private ServiceClient getClient(final String workerTaskId)
+ {
+ synchronized (clientMap) {
+ return clientMap.computeIfAbsent(
+ workerTaskId,
+ id -> {
+ final SpecificTaskServiceLocator locator = new SpecificTaskServiceLocator(id, overlordClient);
+ final ServiceClient client = clientFactory.makeClient(
+ id,
+ locator,
+ new SpecificTaskRetryPolicy(workerTaskId, StandardRetryPolicy.unlimited())
+ );
+ return Pair.of(client, locator);
+ }
+ ).lhs;
+ }
+ }
+
+ /**
+ * Deserialize a {@link BytesFullResponseHolder} as JSON.
+ *
+ * It would be reasonable to move this to {@link BytesFullResponseHolder} itself, or some shared utility class.
+ */
+ private T deserialize(final BytesFullResponseHolder bytesHolder, final TypeReference typeReference)
+ {
+ try {
+ return jsonMapper.readValue(bytesHolder.getContent(), typeReference);
+ }
+ catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java
new file mode 100644
index 00000000000..0536785562a
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java
@@ -0,0 +1,296 @@
+/*
+ * 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.databind.ObjectMapper;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.errorprone.annotations.concurrent.GuardedBy;
+import com.google.inject.Injector;
+import com.google.inject.Key;
+import it.unimi.dsi.fastutil.ints.IntSet;
+import org.apache.druid.frame.processor.Bouncer;
+import org.apache.druid.guice.annotations.EscalatedGlobal;
+import org.apache.druid.guice.annotations.Self;
+import org.apache.druid.indexing.common.SegmentCacheManagerFactory;
+import org.apache.druid.indexing.common.TaskToolbox;
+import org.apache.druid.indexing.worker.config.WorkerConfig;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.msq.exec.ControllerClient;
+import org.apache.druid.msq.exec.TaskDataSegmentProvider;
+import org.apache.druid.msq.exec.Worker;
+import org.apache.druid.msq.exec.WorkerClient;
+import org.apache.druid.msq.exec.WorkerContext;
+import org.apache.druid.msq.exec.WorkerMemoryParameters;
+import org.apache.druid.msq.input.InputSpecs;
+import org.apache.druid.msq.kernel.FrameContext;
+import org.apache.druid.msq.kernel.QueryDefinition;
+import org.apache.druid.msq.rpc.CoordinatorServiceClient;
+import org.apache.druid.rpc.ServiceClientFactory;
+import org.apache.druid.rpc.ServiceLocations;
+import org.apache.druid.rpc.ServiceLocator;
+import org.apache.druid.rpc.StandardRetryPolicy;
+import org.apache.druid.rpc.indexing.OverlordClient;
+import org.apache.druid.rpc.indexing.SpecificTaskRetryPolicy;
+import org.apache.druid.rpc.indexing.SpecificTaskServiceLocator;
+import org.apache.druid.segment.IndexIO;
+import org.apache.druid.segment.loading.SegmentCacheManager;
+import org.apache.druid.segment.realtime.appenderator.UnifiedIndexerAppenderatorsManager;
+import org.apache.druid.server.DruidNode;
+
+import java.io.File;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ThreadLocalRandom;
+
+public class IndexerWorkerContext implements WorkerContext
+{
+ private static final Logger log = new Logger(IndexerWorkerContext.class);
+ private static final long FREQUENCY_CHECK_MILLIS = 1000;
+ private static final long FREQUENCY_CHECK_JITTER = 30;
+
+ private final TaskToolbox toolbox;
+ private final Injector injector;
+ private final IndexIO indexIO;
+ private final TaskDataSegmentProvider dataSegmentProvider;
+ private final ServiceClientFactory clientFactory;
+
+ @GuardedBy("this")
+ private OverlordClient overlordClient;
+
+ @GuardedBy("this")
+ private ServiceLocator controllerLocator;
+
+ public IndexerWorkerContext(
+ final TaskToolbox toolbox,
+ final Injector injector,
+ final IndexIO indexIO,
+ final TaskDataSegmentProvider dataSegmentProvider,
+ final ServiceClientFactory clientFactory
+ )
+ {
+ this.toolbox = toolbox;
+ this.injector = injector;
+ this.indexIO = indexIO;
+ this.dataSegmentProvider = dataSegmentProvider;
+ this.clientFactory = clientFactory;
+ }
+
+ public static IndexerWorkerContext createProductionInstance(final TaskToolbox toolbox, final Injector injector)
+ {
+ final IndexIO indexIO = injector.getInstance(IndexIO.class);
+ final CoordinatorServiceClient coordinatorServiceClient =
+ injector.getInstance(CoordinatorServiceClient.class).withRetryPolicy(StandardRetryPolicy.unlimited());
+ final SegmentCacheManager segmentCacheManager =
+ injector.getInstance(SegmentCacheManagerFactory.class)
+ .manufacturate(new File(toolbox.getIndexingTmpDir(), "segment-fetch"));
+ final ServiceClientFactory serviceClientFactory =
+ injector.getInstance(Key.get(ServiceClientFactory.class, EscalatedGlobal.class));
+
+ return new IndexerWorkerContext(
+ toolbox,
+ injector,
+ indexIO,
+ new TaskDataSegmentProvider(coordinatorServiceClient, segmentCacheManager, indexIO),
+ serviceClientFactory
+ );
+ }
+
+ public TaskToolbox toolbox()
+ {
+ return toolbox;
+ }
+
+ @Override
+ public ObjectMapper jsonMapper()
+ {
+ return toolbox.getJsonMapper();
+ }
+
+ @Override
+ public Injector injector()
+ {
+ return injector;
+ }
+
+ @Override
+ public void registerWorker(Worker worker, Closer closer)
+ {
+ WorkerChatHandler chatHandler = new WorkerChatHandler(toolbox, worker);
+ toolbox.getChatHandlerProvider().register(worker.id(), chatHandler, false);
+ closer.register(() -> toolbox.getChatHandlerProvider().unregister(worker.id()));
+ closer.register(() -> {
+ synchronized (this) {
+ if (controllerLocator != null) {
+ controllerLocator.close();
+ }
+ }
+ });
+
+ // Register the periodic controller checker
+ final ExecutorService periodicControllerCheckerExec = Execs.singleThreaded("controller-status-checker-%s");
+ closer.register(periodicControllerCheckerExec::shutdownNow);
+ final ServiceLocator controllerLocator = makeControllerLocator(worker.task().getControllerTaskId());
+ periodicControllerCheckerExec.submit(() -> controllerCheckerRunnable(controllerLocator, worker));
+ }
+
+ @VisibleForTesting
+ void controllerCheckerRunnable(final ServiceLocator controllerLocator, final Worker worker)
+ {
+ while (true) {
+ // Add some randomness to the frequency of the loop to avoid requests from simultaneously spun up tasks bunching
+ // up and stagger them randomly
+ long sleepTimeMillis = FREQUENCY_CHECK_MILLIS + ThreadLocalRandom.current().nextLong(
+ -FREQUENCY_CHECK_JITTER,
+ 2 * FREQUENCY_CHECK_JITTER
+ );
+ final ServiceLocations controllerLocations;
+ try {
+ controllerLocations = controllerLocator.locate().get();
+ }
+ catch (Throwable e) {
+ // Service locator exceptions are not recoverable.
+ log.noStackTrace().warn(
+ e,
+ "Periodic fetch of controller location encountered an exception. Worker task [%s] will exit.",
+ worker.id()
+ );
+ worker.controllerFailed();
+ break;
+ }
+
+ if (controllerLocations.isClosed() || controllerLocations.getLocations().isEmpty()) {
+ log.warn(
+ "Periodic fetch of controller location returned [%s]. Worker task [%s] will exit.",
+ controllerLocations,
+ worker.id()
+ );
+ worker.controllerFailed();
+ break;
+ }
+
+ try {
+ Thread.sleep(sleepTimeMillis);
+ }
+ catch (InterruptedException ignored) {
+ // Do nothing: an interrupt means we were shut down. Status checker should exit quietly.
+ }
+ }
+ }
+
+ @Override
+ public File tempDir()
+ {
+ return toolbox.getIndexingTmpDir();
+ }
+
+ @Override
+ public ControllerClient makeControllerClient(String controllerId)
+ {
+ final ServiceLocator locator = makeControllerLocator(controllerId);
+
+ return new IndexerControllerClient(
+ clientFactory.makeClient(
+ controllerId,
+ locator,
+ new SpecificTaskRetryPolicy(controllerId, StandardRetryPolicy.unlimited())
+ ),
+ jsonMapper(),
+ locator
+ );
+ }
+
+ @Override
+ public WorkerClient makeWorkerClient()
+ {
+ // Ignore workerId parameter. The workerId is passed into each method of WorkerClient individually.
+ return new IndexerWorkerClient(clientFactory, makeOverlordClient(), jsonMapper());
+ }
+
+ @Override
+ public FrameContext frameContext(QueryDefinition queryDef, int stageNumber)
+ {
+ final int numWorkersInJvm;
+
+ // Determine the max number of workers in JVM for memory allocations.
+ if (toolbox.getAppenderatorsManager() instanceof UnifiedIndexerAppenderatorsManager) {
+ // CliIndexer
+ numWorkersInJvm = injector.getInstance(WorkerConfig.class).getCapacity();
+ } else {
+ // CliPeon
+ numWorkersInJvm = 1;
+ }
+
+ final IntSet inputStageNumbers =
+ InputSpecs.getStageNumbers(queryDef.getStageDefinition(stageNumber).getInputSpecs());
+ final int numInputWorkers =
+ inputStageNumbers.intStream()
+ .map(inputStageNumber -> queryDef.getStageDefinition(inputStageNumber).getMaxWorkerCount())
+ .sum();
+
+ return new IndexerFrameContext(
+ this,
+ indexIO,
+ dataSegmentProvider,
+ WorkerMemoryParameters.compute(
+ Runtime.getRuntime().maxMemory(),
+ numWorkersInJvm,
+ processorBouncer().getMaxCount(),
+ numInputWorkers
+ )
+ );
+ }
+
+ @Override
+ public int threadCount()
+ {
+ return processorBouncer().getMaxCount();
+ }
+
+ @Override
+ public DruidNode selfNode()
+ {
+ return injector.getInstance(Key.get(DruidNode.class, Self.class));
+ }
+
+ @Override
+ public Bouncer processorBouncer()
+ {
+ return injector.getInstance(Bouncer.class);
+ }
+
+ private synchronized OverlordClient makeOverlordClient()
+ {
+ if (overlordClient == null) {
+ overlordClient = injector.getInstance(OverlordClient.class)
+ .withRetryPolicy(StandardRetryPolicy.unlimited());
+ }
+ return overlordClient;
+ }
+
+ private synchronized ServiceLocator makeControllerLocator(final String controllerId)
+ {
+ if (controllerLocator == null) {
+ controllerLocator = new SpecificTaskServiceLocator(controllerId, makeOverlordClient());
+ }
+
+ return controllerLocator;
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerManagerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerManagerClient.java
new file mode 100644
index 00000000000..ac1e71a5525
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerManagerClient.java
@@ -0,0 +1,81 @@
+/*
+ * 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 org.apache.druid.client.indexing.TaskStatusResponse;
+import org.apache.druid.common.guava.FutureUtils;
+import org.apache.druid.indexer.TaskLocation;
+import org.apache.druid.indexer.TaskStatus;
+import org.apache.druid.msq.exec.WorkerManagerClient;
+import org.apache.druid.rpc.indexing.OverlordClient;
+
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Worker manager client backed by the Indexer service. Glues together
+ * three different mechanisms to provide the single multi-stage query interface.
+ */
+public class IndexerWorkerManagerClient implements WorkerManagerClient
+{
+ private final OverlordClient overlordClient;
+
+ public IndexerWorkerManagerClient(final OverlordClient overlordClient)
+ {
+ this.overlordClient = overlordClient;
+ }
+
+ @Override
+ public String run(String controllerId, MSQWorkerTask task)
+ {
+ FutureUtils.getUnchecked(overlordClient.runTask(controllerId, task), true);
+ return controllerId;
+ }
+
+ @Override
+ public void cancel(String taskId)
+ {
+ FutureUtils.getUnchecked(overlordClient.cancelTask(taskId), true);
+ }
+
+ @Override
+ public Map statuses(Set taskIds)
+ {
+ return FutureUtils.getUnchecked(overlordClient.taskStatuses(taskIds), true);
+ }
+
+ @Override
+ public TaskLocation location(String workerId)
+ {
+ final TaskStatusResponse response = FutureUtils.getUnchecked(overlordClient.taskStatus(workerId), true);
+
+ if (response.getStatus() != null) {
+ return response.getStatus().getLocation();
+ } else {
+ return TaskLocation.unknown();
+ }
+ }
+
+ @Override
+ public void close()
+ {
+ // Nothing to do. The OverlordServiceClient is closed by the JVM lifecycle.
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/InputChannelFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/InputChannelFactory.java
new file mode 100644
index 00000000000..d6b02810c2c
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/InputChannelFactory.java
@@ -0,0 +1,38 @@
+/*
+ * 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 org.apache.druid.frame.channel.ReadableFrameChannel;
+import org.apache.druid.msq.kernel.StageId;
+
+import java.io.IOException;
+
+/**
+ * Creates {@link ReadableFrameChannel} to fetch frames corresponding to a particular stage and partition from the
+ * provided worker id
+ */
+public interface InputChannelFactory
+{
+ /**
+ * Given stageId, partitionNumber and workerNumber, this method opens the ReadableFrameChannel to fetch the
+ * corresponding frames
+ */
+ ReadableFrameChannel openChannel(StageId stageId, int workerNumber, int partitionNumber) throws IOException;
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/InputChannelsImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/InputChannelsImpl.java
new file mode 100644
index 00000000000..1c01d480e33
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/InputChannelsImpl.java
@@ -0,0 +1,211 @@
+/*
+ * 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.google.common.collect.Iterables;
+import org.apache.druid.frame.FrameType;
+import org.apache.druid.frame.allocation.MemoryAllocator;
+import org.apache.druid.frame.channel.BlockingQueueFrameChannel;
+import org.apache.druid.frame.channel.ReadableFrameChannel;
+import org.apache.druid.frame.key.ClusterBy;
+import org.apache.druid.frame.processor.FrameChannelMerger;
+import org.apache.druid.frame.processor.FrameChannelMuxer;
+import org.apache.druid.frame.processor.FrameProcessorExecutor;
+import org.apache.druid.frame.read.FrameReader;
+import org.apache.druid.frame.write.FrameWriters;
+import org.apache.druid.msq.input.stage.InputChannels;
+import org.apache.druid.msq.input.stage.ReadablePartition;
+import org.apache.druid.msq.input.stage.ReadablePartitions;
+import org.apache.druid.msq.kernel.QueryDefinition;
+import org.apache.druid.msq.kernel.StageDefinition;
+import org.apache.druid.msq.kernel.StageId;
+import org.apache.druid.msq.kernel.StagePartition;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Supplier;
+
+/**
+ * Implementation of {@link InputChannels}.
+ */
+public class InputChannelsImpl implements InputChannels
+{
+ private final QueryDefinition queryDefinition;
+ private final InputChannelFactory channelFactory;
+ private final Supplier allocatorMaker;
+ private final FrameProcessorExecutor exec;
+ private final String cancellationId;
+ private final Map readablePartitionMap;
+
+ public InputChannelsImpl(
+ final QueryDefinition queryDefinition,
+ final ReadablePartitions readablePartitions,
+ final InputChannelFactory channelFactory,
+ final Supplier allocatorMaker,
+ final FrameProcessorExecutor exec,
+ final String cancellationId
+ )
+ {
+ this.queryDefinition = queryDefinition;
+ this.readablePartitionMap = new HashMap<>();
+ this.channelFactory = channelFactory;
+ this.allocatorMaker = allocatorMaker;
+ this.exec = exec;
+ this.cancellationId = cancellationId;
+
+ for (final ReadablePartition readablePartition : readablePartitions) {
+ readablePartitionMap.put(
+ new StagePartition(
+ new StageId(queryDefinition.getQueryId(), readablePartition.getStageNumber()),
+ readablePartition.getPartitionNumber()
+ ),
+ readablePartition
+ );
+ }
+ }
+
+ @Override
+ public ReadableFrameChannel openChannel(final StagePartition stagePartition) throws IOException
+ {
+ final StageDefinition stageDef = queryDefinition.getStageDefinition(stagePartition.getStageId());
+ final ReadablePartition readablePartition = readablePartitionMap.get(stagePartition);
+ final ClusterBy inputClusterBy = stageDef.getClusterBy();
+ final boolean isSorted = inputClusterBy.getBucketByCount() != inputClusterBy.getColumns().size();
+
+ if (isSorted) {
+ return openSorted(stageDef, readablePartition);
+ } else {
+ return openUnsorted(stageDef, readablePartition);
+ }
+ }
+
+ @Override
+ public FrameReader frameReader(final int stageNumber)
+ {
+ return queryDefinition.getStageDefinition(stageNumber).getFrameReader();
+ }
+
+ private ReadableFrameChannel openSorted(
+ final StageDefinition stageDefinition,
+ final ReadablePartition readablePartition
+ ) throws IOException
+ {
+ // Note: this method uses a single FrameChannelMerger, not a SuperSorter, for efficiency. (Currently, SuperSorter
+ // is always multi-level and always uses disk.)
+ final BlockingQueueFrameChannel queueChannel = BlockingQueueFrameChannel.minimal();
+
+ final List channels = openChannels(
+ stageDefinition.getId(),
+ readablePartition
+ );
+
+ if (channels.size() == 1) {
+ return Iterables.getOnlyElement(channels);
+ } else {
+ final FrameChannelMerger merger = new FrameChannelMerger(
+ channels,
+ stageDefinition.getFrameReader(),
+ queueChannel.writable(),
+ FrameWriters.makeFrameWriterFactory(
+ FrameType.ROW_BASED,
+ allocatorMaker.get(),
+ stageDefinition.getFrameReader().signature(),
+
+ // No sortColumns, because FrameChannelMerger generates frames that are sorted all on its own
+ Collections.emptyList()
+ ),
+ stageDefinition.getClusterBy(),
+ null,
+ -1
+ );
+
+ // Discard future, since there is no need to keep it. We aren't interested in its return value. If it fails,
+ // downstream processors are notified through fail(e) on in-memory channels. If we need to cancel it, we use
+ // the cancellationId.
+ exec.runFully(merger, cancellationId);
+
+ return queueChannel.readable();
+ }
+ }
+
+ private ReadableFrameChannel openUnsorted(
+ final StageDefinition stageDefinition,
+ final ReadablePartition readablePartition
+ ) throws IOException
+ {
+ final List channels = openChannels(
+ stageDefinition.getId(),
+ readablePartition
+ );
+
+ if (channels.size() == 1) {
+ return Iterables.getOnlyElement(channels);
+ } else {
+ final BlockingQueueFrameChannel queueChannel = BlockingQueueFrameChannel.minimal();
+ final FrameChannelMuxer muxer = new FrameChannelMuxer(channels, queueChannel.writable());
+
+ // Discard future, since there is no need to keep it. We aren't interested in its return value. If it fails,
+ // downstream processors are notified through fail(e) on in-memory channels. If we need to cancel it, we use
+ // the cancellationId.
+ exec.runFully(muxer, cancellationId);
+
+ return queueChannel.readable();
+ }
+ }
+
+ private List openChannels(
+ final StageId stageId,
+ final ReadablePartition readablePartition
+ ) throws IOException
+ {
+ final List channels = new ArrayList<>();
+
+ try {
+ for (final int workerNumber : readablePartition.getWorkerNumbers()) {
+ channels.add(
+ channelFactory.openChannel(
+ stageId,
+ workerNumber,
+ readablePartition.getPartitionNumber()
+ )
+ );
+ }
+
+ return channels;
+ }
+ catch (Exception e) {
+ // Close all channels opened so far before throwing the exception.
+ for (final ReadableFrameChannel channel : channels) {
+ try {
+ channel.close();
+ }
+ catch (Exception e2) {
+ e.addSuppressed(e2);
+ }
+ }
+
+ throw e;
+ }
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/KeyStatisticsCollectionProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/KeyStatisticsCollectionProcessor.java
new file mode 100644
index 00000000000..af528f6ecbe
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/KeyStatisticsCollectionProcessor.java
@@ -0,0 +1,183 @@
+/*
+ * 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.google.common.primitives.Ints;
+import it.unimi.dsi.fastutil.ints.IntSet;
+import org.apache.datasketches.memory.Memory;
+import org.apache.druid.frame.Frame;
+import org.apache.druid.frame.allocation.MemoryRange;
+import org.apache.druid.frame.channel.FrameWithPartition;
+import org.apache.druid.frame.channel.ReadableFrameChannel;
+import org.apache.druid.frame.channel.WritableFrameChannel;
+import org.apache.druid.frame.key.ClusterBy;
+import org.apache.druid.frame.key.FrameComparisonWidget;
+import org.apache.druid.frame.key.RowKey;
+import org.apache.druid.frame.processor.FrameProcessor;
+import org.apache.druid.frame.processor.FrameProcessors;
+import org.apache.druid.frame.processor.ReturnOrAwait;
+import org.apache.druid.frame.read.FrameReader;
+import org.apache.druid.frame.read.FrameReaderUtils;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.msq.sql.MSQTaskQueryMaker;
+import org.apache.druid.msq.statistics.ClusterByStatisticsCollector;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.Cursor;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.function.IntSupplier;
+import java.util.function.Supplier;
+
+public class KeyStatisticsCollectionProcessor implements FrameProcessor