mirror of https://github.com/apache/druid.git
Add druid-multi-stage-query extension. (#12918)
* Add druid-multi-stage-query extension. * Adjustments from CI. * Task ID validation. * Various changes from code review. * Remove unnecessary code. * LGTM-related.
This commit is contained in:
parent
f0fc45028e
commit
d7d15ba51f
|
@ -71,6 +71,13 @@
|
|||
<Class name="org.apache.druid.frame.file.FrameFileWriter" />
|
||||
</And>
|
||||
</Match>
|
||||
<Match>
|
||||
<!-- Spotbugs doesn't like awaiting a latch without checking return value -->
|
||||
<And>
|
||||
<Bug pattern="RV_RETURN_VALUE_IGNORED" />
|
||||
<Class name="org.apache.druid.msq.indexing.MSQWorkerTaskLauncher" />
|
||||
</And>
|
||||
</Match>
|
||||
|
||||
<Bug pattern="AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION"/>
|
||||
<Bug pattern="BC_UNCONFIRMED_CAST"/>
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -216,6 +216,8 @@
|
|||
<argument>-c</argument>
|
||||
<argument>org.apache.druid.extensions:druid-lookups-cached-single</argument>
|
||||
<argument>-c</argument>
|
||||
<argument>org.apache.druid.extensions:druid-multi-stage-query</argument>
|
||||
<argument>-c</argument>
|
||||
<argument>org.apache.druid.extensions:druid-protobuf-extensions</argument>
|
||||
<argument>-c</argument>
|
||||
<argument>org.apache.druid.extensions:mysql-metadata-storage</argument>
|
||||
|
@ -396,6 +398,8 @@
|
|||
<argument>-c</argument>
|
||||
<argument>org.apache.druid.extensions:druid-lookups-cached-single</argument>
|
||||
<argument>-c</argument>
|
||||
<argument>org.apache.druid.extensions:druid-multi-stage-query</argument>
|
||||
<argument>-c</argument>
|
||||
<argument>org.apache.druid.extensions:druid-protobuf-extensions</argument>
|
||||
<argument>-c</argument>
|
||||
<argument>org.apache.druid.extensions:mysql-metadata-storage</argument>
|
||||
|
@ -679,6 +683,8 @@
|
|||
<argument>-c</argument>
|
||||
<argument>org.apache.druid.extensions:druid-lookups-cached-global</argument>
|
||||
<argument>-c</argument>
|
||||
<argument>org.apache.druid.extensions:druid-multi-stage-query</argument>
|
||||
<argument>-c</argument>
|
||||
<argument>org.apache.druid.extensions:druid-protobuf-extensions</argument>
|
||||
<argument>-c</argument>
|
||||
<argument>org.apache.druid.extensions:mysql-metadata-storage</argument>
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -0,0 +1,141 @@
|
|||
<!--
|
||||
~ 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.
|
||||
-->
|
||||
|
||||
# `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
|
|
@ -0,0 +1,299 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
|
||||
<!--
|
||||
~ 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.
|
||||
-->
|
||||
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
|
||||
<groupId>org.apache.druid.extensions</groupId>
|
||||
<artifactId>druid-multi-stage-query</artifactId>
|
||||
<name>druid-multi-stage-query</name>
|
||||
<description>druid-multi-stage-query</description>
|
||||
|
||||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.24.0-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-core</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-processing</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-server</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-indexing-service</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-sql</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-services</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.inject</groupId>
|
||||
<artifactId>guice</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.inject.extensions</groupId>
|
||||
<artifactId>guice-multibindings</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.guava</groupId>
|
||||
<artifactId>guava</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.opencsv</groupId>
|
||||
<artifactId>opencsv</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-databind</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-core</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-annotations</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>javax.validation</groupId>
|
||||
<artifactId>validation-api</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.code.findbugs</groupId>
|
||||
<artifactId>jsr305</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>jakarta.inject</groupId>
|
||||
<artifactId>jakarta.inject-api</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>joda-time</groupId>
|
||||
<artifactId>joda-time</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>io.netty</groupId>
|
||||
<artifactId>netty</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.calcite</groupId>
|
||||
<artifactId>calcite-core</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.dataformat</groupId>
|
||||
<artifactId>jackson-dataformat-smile</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.jdbi</groupId>
|
||||
<artifactId>jdbi</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.jaxrs</groupId>
|
||||
<artifactId>jackson-jaxrs-smile-provider</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>javax.ws.rs</groupId>
|
||||
<artifactId>jsr311-api</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.commons</groupId>
|
||||
<artifactId>commons-lang3</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>javax.servlet</groupId>
|
||||
<artifactId>javax.servlet-api</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.sun.jersey</groupId>
|
||||
<artifactId>jersey-server</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.errorprone</groupId>
|
||||
<artifactId>error_prone_annotations</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.lz4</groupId>
|
||||
<artifactId>lz4-java</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.datasketches</groupId>
|
||||
<artifactId>datasketches-java</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.datasketches</groupId>
|
||||
<artifactId>datasketches-memory</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>it.unimi.dsi</groupId>
|
||||
<artifactId>fastutil-core</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>commons-lang</groupId>
|
||||
<artifactId>commons-lang</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>commons-io</groupId>
|
||||
<artifactId>commons-io</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
|
||||
<!-- Tests -->
|
||||
<dependency>
|
||||
<groupId>org.easymock</groupId>
|
||||
<artifactId>easymock</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.hamcrest</groupId>
|
||||
<artifactId>hamcrest-all</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.hamcrest</groupId>
|
||||
<artifactId>hamcrest-core</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>junit</groupId>
|
||||
<artifactId>junit</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.mockito</groupId>
|
||||
<artifactId>mockito-core</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>nl.jqno.equalsverifier</groupId>
|
||||
<artifactId>equalsverifier</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-core</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<scope>test</scope>
|
||||
<type>test-jar</type>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-processing</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<scope>test</scope>
|
||||
<type>test-jar</type>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-server</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-sql</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-hll</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<!-- Used in tests. Marked as scope "provided" instead of "test", because scope "test" prevents
|
||||
running using this extension's classpath in an IDE. -->
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
|
||||
<build>
|
||||
<plugins>
|
||||
<plugin>
|
||||
<groupId>org.jacoco</groupId>
|
||||
<artifactId>jacoco-maven-plugin</artifactId>
|
||||
<configuration>
|
||||
<excludes>
|
||||
<!-- Production implementation classes, to be tested in integration tests -->
|
||||
<exclude>org/apache/druid/msq/guice/*</exclude>
|
||||
<exclude>org/apache/druid/msq/indexing/IndexerControllerClient*</exclude>
|
||||
<exclude>org/apache/druid/msq/indexing/IndexerControllerContext*</exclude>
|
||||
<exclude>org/apache/druid/msq/indexing/IndexerWorkerClient*</exclude>
|
||||
<exclude>org/apache/druid/msq/indexing/IndexerWorkerContext*</exclude>
|
||||
<exclude>org/apache/druid/msq/indexing/IndexerWorkerManagerClient*</exclude>
|
||||
</excludes>
|
||||
</configuration>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</build>
|
||||
</project>
|
|
@ -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) +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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<String> 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<String> 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<String>
|
||||
{
|
||||
private static final Map<String, Integer> ORDER =
|
||||
ImmutableMap.<String, Integer>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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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<String, QueryCounterSnapshot> snapshotMap;
|
||||
|
||||
@JsonCreator
|
||||
public CounterSnapshots(final Map<String, QueryCounterSnapshot> snapshotMap)
|
||||
{
|
||||
this.snapshotMap = ImmutableSortedMap.copyOf(snapshotMap, CounterNames.comparator());
|
||||
}
|
||||
|
||||
public Map<String, QueryCounterSnapshot> 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();
|
||||
}
|
||||
}
|
|
@ -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<CounterSnapshots>
|
||||
{
|
||||
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<String, QueryCounterSnapshot> entry : value.getMap().entrySet()) {
|
||||
jg.writeObjectField(entry.getKey(), entry.getValue());
|
||||
}
|
||||
|
||||
jg.writeEndObject();
|
||||
}
|
||||
}
|
|
@ -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<Int2ObjectMap<CounterSnapshots>> snapshotsMap;
|
||||
|
||||
public CounterSnapshotsTree()
|
||||
{
|
||||
this.snapshotsMap = new Int2ObjectAVLTreeMap<>();
|
||||
}
|
||||
|
||||
@JsonCreator
|
||||
public static CounterSnapshotsTree fromMap(final Map<Integer, Map<Integer, CounterSnapshots>> 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<Integer, Map<Integer, CounterSnapshots>> copyMap()
|
||||
{
|
||||
final Map<Integer, Map<Integer, CounterSnapshots>> retVal = new Int2ObjectAVLTreeMap<>();
|
||||
|
||||
synchronized (snapshotsMap) {
|
||||
for (Int2ObjectMap.Entry<Int2ObjectMap<CounterSnapshots>> entry : snapshotsMap.int2ObjectEntrySet()) {
|
||||
retVal.put(entry.getIntKey(), new Int2ObjectAVLTreeMap<>(entry.getValue()));
|
||||
}
|
||||
}
|
||||
|
||||
return retVal;
|
||||
}
|
||||
|
||||
private void putAll(final Map<Integer, Map<Integer, CounterSnapshots>> otherMap)
|
||||
{
|
||||
synchronized (snapshotsMap) {
|
||||
for (Map.Entry<Integer, Map<Integer, CounterSnapshots>> stageEntry : otherMap.entrySet()) {
|
||||
for (Map.Entry<Integer, CounterSnapshots> workerEntry : stageEntry.getValue().entrySet()) {
|
||||
put(stageEntry.getKey(), workerEntry.getKey(), workerEntry.getValue());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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<String, QueryCounter> 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 extends QueryCounter> T counter(final String counterName, final Supplier<T> newCounterFn)
|
||||
{
|
||||
return (T) countersMap.computeIfAbsent(counterName, ignored -> newCounterFn.get());
|
||||
}
|
||||
|
||||
public CounterSnapshots snapshot()
|
||||
{
|
||||
final Map<String, QueryCounterSnapshot> m = new HashMap<>();
|
||||
|
||||
for (final Map.Entry<String, QueryCounter> entry : countersMap.entrySet()) {
|
||||
final QueryCounterSnapshot counterSnapshot = entry.getValue().snapshot();
|
||||
if (counterSnapshot != null) {
|
||||
m.put(entry.getKey(), counterSnapshot);
|
||||
}
|
||||
}
|
||||
|
||||
return new CounterSnapshots(m);
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
|
@ -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
|
||||
{
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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<String, Long> 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<String, Long> countCopy = ImmutableMap.copyOf(warningCodeCounter);
|
||||
return new Snapshot(countCopy);
|
||||
}
|
||||
|
||||
@JsonTypeName("warnings")
|
||||
public static class Snapshot implements QueryCounterSnapshot
|
||||
{
|
||||
private final Map<String, Long> warningCountMap;
|
||||
|
||||
@JsonCreator
|
||||
public Snapshot(Map<String, Long> warningCountMap)
|
||||
{
|
||||
this.warningCountMap = Preconditions.checkNotNull(warningCountMap, "warningCountMap");
|
||||
}
|
||||
|
||||
@JsonValue
|
||||
public Map<String, Long> 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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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<MSQErrorReport> 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<String> getTaskIds();
|
||||
|
||||
@Nullable
|
||||
Map<String, TaskReport> liveReports();
|
||||
|
||||
}
|
|
@ -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<MSQErrorReport> MSQErrorReports
|
||||
) throws IOException;
|
||||
List<String> getTaskList() throws IOException;
|
||||
|
||||
@Override
|
||||
void close();
|
||||
}
|
|
@ -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<String, TaskReport> reports);
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -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<Void> postWorkOrder(String workerTaskId, WorkOrder workOrder)
|
||||
{
|
||||
return wrap(workerTaskId, client, c -> c.postWorkOrder(workerTaskId, workOrder));
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListenableFuture<Void> postResultPartitionBoundaries(
|
||||
final String workerTaskId,
|
||||
final StageId stageId,
|
||||
final ClusterByPartitions partitionBoundaries
|
||||
)
|
||||
{
|
||||
return wrap(workerTaskId, client, c -> c.postResultPartitionBoundaries(workerTaskId, stageId, partitionBoundaries));
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListenableFuture<Void> postCleanupStage(String workerTaskId, StageId stageId)
|
||||
{
|
||||
return wrap(workerTaskId, client, c -> c.postCleanupStage(workerTaskId, stageId));
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListenableFuture<Void> postFinish(String workerTaskId)
|
||||
{
|
||||
return wrap(workerTaskId, client, c -> c.postFinish(workerTaskId));
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListenableFuture<CounterSnapshotsTree> getCounters(String workerTaskId)
|
||||
{
|
||||
return wrap(workerTaskId, client, c -> c.getCounters(workerTaskId));
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListenableFuture<Boolean> 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 <T> ListenableFuture<T> wrap(
|
||||
final String workerTaskId,
|
||||
final WorkerClient client,
|
||||
final ClientFn<T> clientFn
|
||||
)
|
||||
{
|
||||
final SettableFuture<T> retVal = SettableFuture.create();
|
||||
final ListenableFuture<T> clientFuture;
|
||||
|
||||
try {
|
||||
clientFuture = clientFn.apply(client);
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw new MSQException(e, new WorkerRpcFailedFault(workerTaskId));
|
||||
}
|
||||
|
||||
Futures.addCallback(
|
||||
clientFuture,
|
||||
new FutureCallback<T>()
|
||||
{
|
||||
@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<T>
|
||||
{
|
||||
ListenableFuture<T> apply(WorkerClient client);
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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)
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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<Segment> 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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
|
@ -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<Void> 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<Void> 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<Void> postFinish(String workerId);
|
||||
|
||||
/**
|
||||
* Fetches all the counters gathered by that worker
|
||||
*/
|
||||
ListenableFuture<CounterSnapshotsTree> 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<Void> 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<Boolean> fetchChannelData(
|
||||
String workerTaskId,
|
||||
StageId stageId,
|
||||
int partitionNumber,
|
||||
long offset,
|
||||
ReadableByteChunksFrameChannel channel
|
||||
);
|
||||
|
||||
@Override
|
||||
void close() throws IOException;
|
||||
}
|
|
@ -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();
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -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<String, TaskStatus> statuses(Set<String> taskIds);
|
||||
|
||||
/**
|
||||
* Cancel the task corresponding to the provided workerId
|
||||
*/
|
||||
void cancel(String workerId);
|
||||
|
||||
@Override
|
||||
void close();
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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"));
|
||||
}
|
||||
}
|
|
@ -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.
|
||||
}
|
||||
}
|
|
@ -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<Class<? extends MSQFault>> 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<NodeRole> 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());
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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
|
||||
);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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
|
||||
{
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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 + '\'' +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -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<ColumnMapping> mappings;
|
||||
private final Map<String, String> outputToQueryColumnMap;
|
||||
private final Map<String, List<String>> queryToOutputColumnsMap;
|
||||
|
||||
@JsonCreator
|
||||
public ColumnMappings(final List<ColumnMapping> 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<String> 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<String> getOutputColumnsForQueryColumn(final String queryColumn)
|
||||
{
|
||||
final List<String> outputColumns = queryToOutputColumnsMap.get(queryColumn);
|
||||
if (outputColumns != null) {
|
||||
return outputColumns;
|
||||
} else {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
}
|
||||
|
||||
@JsonValue
|
||||
public List<ColumnMapping> 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<ColumnMapping> validateNoDuplicateOutputColumns(final List<ColumnMapping> mappings)
|
||||
{
|
||||
final Set<String> encountered = new HashSet<>();
|
||||
|
||||
for (final ColumnMapping mapping : mappings) {
|
||||
if (!encountered.add(mapping.getOutputColumn())) {
|
||||
throw new ISE("Duplicate output column [%s]", mapping.getOutputColumn());
|
||||
}
|
||||
}
|
||||
|
||||
return mappings;
|
||||
}
|
||||
}
|
|
@ -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<MSQErrorReport> 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<String, TaskReport> reports = controller.liveReports();
|
||||
if (reports == null) {
|
||||
return Response.status(Response.Status.NOT_FOUND).build();
|
||||
}
|
||||
return Response.ok(reports).build();
|
||||
}
|
||||
}
|
|
@ -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<InputRow> read() throws IOException
|
||||
{
|
||||
return inputSourceReader.read().map(inputRow -> {
|
||||
channelCounters.incrementRowCount();
|
||||
return inputRow;
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public CloseableIterator<InputRowListPlusRawValues> sample() throws IOException
|
||||
{
|
||||
return inputSourceReader.sample();
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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<String> segmentSortOrder;
|
||||
|
||||
@Nullable
|
||||
private final List<Interval> replaceTimeChunks;
|
||||
|
||||
@JsonCreator
|
||||
public DataSourceMSQDestination(
|
||||
@JsonProperty("dataSource") String dataSource,
|
||||
@JsonProperty("segmentGranularity") Granularity segmentGranularity,
|
||||
@JsonProperty("segmentSortOrder") @Nullable List<String> segmentSortOrder,
|
||||
@JsonProperty("replaceTimeChunks") @Nullable List<Interval> 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<String> 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<Interval> 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 +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -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<MSQErrorReport> 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<String> 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 <IntermediateType, FinalType> FinalType doRequest(
|
||||
RequestBuilder requestBuilder,
|
||||
HttpResponseHandler<IntermediateType, FinalType> 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());
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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<String, TaskReport> reports)
|
||||
{
|
||||
toolbox.getTaskReportFileWriter().write(controllerTaskId, reports);
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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<String, Pair<ServiceClient, Closeable>> 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<Void> postWorkOrder(String workerTaskId, WorkOrder workOrder)
|
||||
{
|
||||
return getClient(workerTaskId).asyncRequest(
|
||||
new RequestBuilder(HttpMethod.POST, "/workOrder")
|
||||
.jsonContent(jsonMapper, workOrder),
|
||||
IgnoreHttpResponseHandler.INSTANCE
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListenableFuture<Void> 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<Void> 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<Void> postFinish(String workerTaskId)
|
||||
{
|
||||
return getClient(workerTaskId).asyncRequest(
|
||||
new RequestBuilder(HttpMethod.POST, "/finish"),
|
||||
IgnoreHttpResponseHandler.INSTANCE
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListenableFuture<CounterSnapshotsTree> getCounters(String workerTaskId)
|
||||
{
|
||||
return FutureUtils.transform(
|
||||
getClient(workerTaskId).asyncRequest(
|
||||
new RequestBuilder(HttpMethod.GET, "/counters"),
|
||||
new BytesFullResponseHandler()
|
||||
),
|
||||
holder -> deserialize(holder, new TypeReference<CounterSnapshotsTree>() {})
|
||||
);
|
||||
}
|
||||
|
||||
private static final Logger log = new Logger(IndexerWorkerClient.class);
|
||||
|
||||
@Override
|
||||
public ListenableFuture<Boolean> fetchChannelData(
|
||||
String workerTaskId,
|
||||
StageId stageId,
|
||||
int partitionNumber,
|
||||
long offset,
|
||||
ReadableByteChunksFrameChannel channel
|
||||
)
|
||||
{
|
||||
final ServiceClient client = getClient(workerTaskId);
|
||||
final String path = getStagePartitionPath(stageId, partitionNumber);
|
||||
|
||||
final SettableFuture<Boolean> retVal = SettableFuture.create();
|
||||
final ListenableFuture<FrameFilePartialFetch> 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<FrameFilePartialFetch>()
|
||||
{
|
||||
@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<Closeable> 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> T deserialize(final BytesFullResponseHolder bytesHolder, final TypeReference<T> typeReference)
|
||||
{
|
||||
try {
|
||||
return jsonMapper.readValue(bytesHolder.getContent(), typeReference);
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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<String, TaskStatus> statuses(Set<String> 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.
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
|
@ -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<MemoryAllocator> allocatorMaker;
|
||||
private final FrameProcessorExecutor exec;
|
||||
private final String cancellationId;
|
||||
private final Map<StagePartition, ReadablePartition> readablePartitionMap;
|
||||
|
||||
public InputChannelsImpl(
|
||||
final QueryDefinition queryDefinition,
|
||||
final ReadablePartitions readablePartitions,
|
||||
final InputChannelFactory channelFactory,
|
||||
final Supplier<MemoryAllocator> 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<ReadableFrameChannel> 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<ReadableFrameChannel> 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<ReadableFrameChannel> openChannels(
|
||||
final StageId stageId,
|
||||
final ReadablePartition readablePartition
|
||||
) throws IOException
|
||||
{
|
||||
final List<ReadableFrameChannel> 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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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<ClusterByStatisticsCollector>
|
||||
{
|
||||
/**
|
||||
* Constant chosen such that a column full of "standard" values, with row count
|
||||
* {@link MSQTaskQueryMaker#DEFAULT_ROWS_PER_SEGMENT}, and *some* redundancy between
|
||||
* rows (therefore: some "reasonable" compression) will not have any columns greater than 2GB in size.
|
||||
*/
|
||||
private static final int STANDARD_VALUE_SIZE = 1000;
|
||||
|
||||
/**
|
||||
* Constant chosen such that a segment full of "standard" rows, with row count
|
||||
* {@link MSQTaskQueryMaker#DEFAULT_ROWS_PER_SEGMENT}, and *some* redundancy between
|
||||
* rows (therefore: some "reasonable" compression) will not be larger than 5GB in size.
|
||||
*/
|
||||
private static final int STANDARD_ROW_SIZE = 2000;
|
||||
|
||||
private final ReadableFrameChannel inputChannel;
|
||||
private final WritableFrameChannel outputChannel;
|
||||
private final FrameReader frameReader;
|
||||
private final ClusterBy clusterBy;
|
||||
|
||||
private ClusterByStatisticsCollector clusterByStatisticsCollector;
|
||||
|
||||
public KeyStatisticsCollectionProcessor(
|
||||
final ReadableFrameChannel inputChannel,
|
||||
final WritableFrameChannel outputChannel,
|
||||
final FrameReader frameReader,
|
||||
final ClusterBy clusterBy,
|
||||
final ClusterByStatisticsCollector clusterByStatisticsCollector
|
||||
)
|
||||
{
|
||||
this.inputChannel = inputChannel;
|
||||
this.outputChannel = outputChannel;
|
||||
this.frameReader = frameReader;
|
||||
this.clusterBy = clusterBy;
|
||||
this.clusterByStatisticsCollector = clusterByStatisticsCollector;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<ReadableFrameChannel> inputChannels()
|
||||
{
|
||||
return Collections.singletonList(inputChannel);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<WritableFrameChannel> outputChannels()
|
||||
{
|
||||
return Collections.singletonList(outputChannel);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReturnOrAwait<ClusterByStatisticsCollector> runIncrementally(final IntSet readableInputs) throws IOException
|
||||
{
|
||||
if (readableInputs.isEmpty()) {
|
||||
return ReturnOrAwait.awaitAll(1);
|
||||
}
|
||||
|
||||
if (inputChannel.isFinished()) {
|
||||
return ReturnOrAwait.returnObject(clusterByStatisticsCollector);
|
||||
}
|
||||
|
||||
final Frame frame = inputChannel.read();
|
||||
final Cursor cursor = FrameProcessors.makeCursor(frame, frameReader);
|
||||
final IntSupplier rowWeightSupplier = makeRowWeightSupplier(frameReader, cursor.getColumnSelectorFactory());
|
||||
final FrameComparisonWidget comparisonWidget = frameReader.makeComparisonWidget(frame, clusterBy.getColumns());
|
||||
|
||||
for (int i = 0; i < frame.numRows(); i++, cursor.advance()) {
|
||||
final RowKey key = comparisonWidget.readKey(i);
|
||||
clusterByStatisticsCollector.add(key, rowWeightSupplier.getAsInt());
|
||||
}
|
||||
|
||||
// Clears partition info (uses NO_PARTITION), but that's OK, because it isn't needed downstream of this processor.
|
||||
outputChannel.write(new FrameWithPartition(frame, FrameWithPartition.NO_PARTITION));
|
||||
return ReturnOrAwait.awaitAll(1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void cleanup() throws IOException
|
||||
{
|
||||
FrameProcessors.closeAll(
|
||||
inputChannels(),
|
||||
outputChannels(),
|
||||
() -> clusterByStatisticsCollector = null
|
||||
);
|
||||
}
|
||||
|
||||
private IntSupplier makeRowWeightSupplier(
|
||||
final FrameReader frameReader,
|
||||
final ColumnSelectorFactory columnSelectorFactory
|
||||
)
|
||||
{
|
||||
final Supplier<MemoryRange<Memory>> rowMemorySupplier =
|
||||
FrameReaderUtils.makeRowMemorySupplier(columnSelectorFactory, frameReader.signature());
|
||||
|
||||
final int numFields = frameReader.signature().size();
|
||||
|
||||
if (rowMemorySupplier == null) {
|
||||
// Can't access row memory.
|
||||
throw new ISE("Can't read row memory from frame. Wrong frame type or signature?");
|
||||
}
|
||||
|
||||
return () -> {
|
||||
final MemoryRange<Memory> rowMemory = rowMemorySupplier.get();
|
||||
|
||||
if (rowMemory == null) {
|
||||
// Can't access row memory.
|
||||
throw new ISE("Can't read row memory from frame. Wrong type or signature?");
|
||||
}
|
||||
|
||||
long maxValueLength = 0;
|
||||
long totalLength = 0;
|
||||
long currentValueStartPosition = (long) Integer.BYTES * numFields;
|
||||
|
||||
for (int i = 0; i < numFields; i++) {
|
||||
final long currentValueEndPosition = rowMemory.memory().getInt(rowMemory.start() + (long) Integer.BYTES * i);
|
||||
final long valueLength = currentValueEndPosition - currentValueStartPosition;
|
||||
|
||||
if (valueLength > maxValueLength) {
|
||||
maxValueLength = valueLength;
|
||||
}
|
||||
|
||||
totalLength += valueLength;
|
||||
currentValueStartPosition = currentValueEndPosition;
|
||||
}
|
||||
|
||||
return 1 + Ints.checkedCast(
|
||||
Math.max(
|
||||
maxValueLength / STANDARD_VALUE_SIZE,
|
||||
totalLength / STANDARD_ROW_SIZE
|
||||
)
|
||||
);
|
||||
};
|
||||
}
|
||||
}
|
|
@ -0,0 +1,222 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.msq.indexing;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JacksonInject;
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||
import com.google.inject.Injector;
|
||||
import com.google.inject.Key;
|
||||
import org.apache.druid.guice.annotations.EscalatedGlobal;
|
||||
import org.apache.druid.indexer.TaskStatus;
|
||||
import org.apache.druid.indexing.common.TaskLock;
|
||||
import org.apache.druid.indexing.common.TaskLockType;
|
||||
import org.apache.druid.indexing.common.TaskToolbox;
|
||||
import org.apache.druid.indexing.common.actions.TaskActionClient;
|
||||
import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction;
|
||||
import org.apache.druid.indexing.common.config.TaskConfig;
|
||||
import org.apache.druid.indexing.common.task.AbstractTask;
|
||||
import org.apache.druid.indexing.common.task.Tasks;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.concurrent.Execs;
|
||||
import org.apache.druid.msq.exec.Controller;
|
||||
import org.apache.druid.msq.exec.ControllerContext;
|
||||
import org.apache.druid.msq.exec.ControllerImpl;
|
||||
import org.apache.druid.msq.exec.MSQTasks;
|
||||
import org.apache.druid.msq.util.MultiStageQueryContext;
|
||||
import org.apache.druid.rpc.ServiceClientFactory;
|
||||
import org.apache.druid.rpc.StandardRetryPolicy;
|
||||
import org.apache.druid.rpc.indexing.OverlordClient;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
|
||||
@JsonTypeName(MSQControllerTask.TYPE)
|
||||
public class MSQControllerTask extends AbstractTask
|
||||
{
|
||||
public static final String TYPE = "query_controller";
|
||||
public static final String DUMMY_DATASOURCE_FOR_SELECT = "__query_select";
|
||||
|
||||
private final MSQSpec querySpec;
|
||||
|
||||
// Enables users, and the web console, to see the original SQL query (if any). Not used by anything else in Druid.
|
||||
@Nullable
|
||||
private final String sqlQuery;
|
||||
|
||||
// Enables users, and the web console, to see the original SQL context (if any). Not used by any other Druid logic.
|
||||
@Nullable
|
||||
private final Map<String, Object> sqlQueryContext;
|
||||
|
||||
// Enables users, and the web console, to see the original SQL type names (if any). Not used by any other Druid logic.
|
||||
@Nullable
|
||||
private final List<String> sqlTypeNames;
|
||||
@Nullable
|
||||
private final ExecutorService remoteFetchExecutorService;
|
||||
|
||||
// Using an Injector directly because tasks do not have a way to provide their own Guice modules.
|
||||
@JacksonInject
|
||||
private Injector injector;
|
||||
|
||||
private volatile Controller controller;
|
||||
|
||||
@JsonCreator
|
||||
public MSQControllerTask(
|
||||
@JsonProperty("id") @Nullable String id,
|
||||
@JsonProperty("spec") MSQSpec querySpec,
|
||||
@JsonProperty("sqlQuery") @Nullable String sqlQuery,
|
||||
@JsonProperty("sqlQueryContext") @Nullable Map<String, Object> sqlQueryContext,
|
||||
@JsonProperty("sqlTypeNames") @Nullable List<String> sqlTypeNames,
|
||||
@JsonProperty("context") @Nullable Map<String, Object> context
|
||||
)
|
||||
{
|
||||
super(
|
||||
id != null ? id : MSQTasks.controllerTaskId(null),
|
||||
id,
|
||||
null,
|
||||
getDataSourceForTaskMetadata(querySpec),
|
||||
context
|
||||
);
|
||||
|
||||
this.querySpec = querySpec;
|
||||
this.sqlQuery = sqlQuery;
|
||||
this.sqlQueryContext = sqlQueryContext;
|
||||
this.sqlTypeNames = sqlTypeNames;
|
||||
|
||||
if (MultiStageQueryContext.isDurableStorageEnabled(querySpec.getQuery().getContext())) {
|
||||
this.remoteFetchExecutorService =
|
||||
Executors.newCachedThreadPool(Execs.makeThreadFactory(getId() + "-remote-fetcher-%d"));
|
||||
} else {
|
||||
this.remoteFetchExecutorService = null;
|
||||
}
|
||||
|
||||
addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, true);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getType()
|
||||
{
|
||||
return TYPE;
|
||||
}
|
||||
|
||||
@JsonProperty("spec")
|
||||
public MSQSpec getQuerySpec()
|
||||
{
|
||||
return querySpec;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@JsonProperty
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public String getSqlQuery()
|
||||
{
|
||||
return sqlQuery;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@JsonProperty
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public Map<String, Object> getSqlQueryContext()
|
||||
{
|
||||
return sqlQueryContext;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@JsonProperty
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public List<String> getSqlTypeNames()
|
||||
{
|
||||
return sqlTypeNames;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isReady(TaskActionClient taskActionClient) throws Exception
|
||||
{
|
||||
// If we're in replace mode, acquire locks for all intervals before declaring the task ready.
|
||||
if (isIngestion(querySpec) && ((DataSourceMSQDestination) querySpec.getDestination()).isReplaceTimeChunks()) {
|
||||
final List<Interval> intervals =
|
||||
((DataSourceMSQDestination) querySpec.getDestination()).getReplaceTimeChunks();
|
||||
|
||||
for (final Interval interval : intervals) {
|
||||
final TaskLock taskLock =
|
||||
taskActionClient.submit(new TimeChunkLockTryAcquireAction(TaskLockType.EXCLUSIVE, interval));
|
||||
|
||||
if (taskLock == null) {
|
||||
return false;
|
||||
} else if (taskLock.isRevoked()) {
|
||||
throw new ISE(StringUtils.format("Lock for interval [%s] was revoked", interval));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskStatus run(final TaskToolbox toolbox) throws Exception
|
||||
{
|
||||
final ServiceClientFactory clientFactory =
|
||||
injector.getInstance(Key.get(ServiceClientFactory.class, EscalatedGlobal.class));
|
||||
final OverlordClient overlordClient = injector.getInstance(OverlordClient.class)
|
||||
.withRetryPolicy(StandardRetryPolicy.unlimited());
|
||||
final ControllerContext context = new IndexerControllerContext(
|
||||
toolbox,
|
||||
injector,
|
||||
clientFactory,
|
||||
overlordClient
|
||||
);
|
||||
controller = new ControllerImpl(this, context);
|
||||
return controller.run();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stopGracefully(final TaskConfig taskConfig)
|
||||
{
|
||||
if (controller != null) {
|
||||
controller.stopGracefully();
|
||||
}
|
||||
if (remoteFetchExecutorService != null) {
|
||||
// This is to make sure we don't leak connections.
|
||||
remoteFetchExecutorService.shutdownNow();
|
||||
}
|
||||
}
|
||||
|
||||
private static String getDataSourceForTaskMetadata(final MSQSpec querySpec)
|
||||
{
|
||||
final MSQDestination destination = querySpec.getDestination();
|
||||
|
||||
if (destination instanceof DataSourceMSQDestination) {
|
||||
return ((DataSourceMSQDestination) destination).getDataSource();
|
||||
} else {
|
||||
return DUMMY_DATASOURCE_FOR_SELECT;
|
||||
}
|
||||
}
|
||||
|
||||
public static boolean isIngestion(final MSQSpec querySpec)
|
||||
{
|
||||
return querySpec.getDestination() instanceof DataSourceMSQDestination;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,33 @@
|
|||
/*
|
||||
* 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.JsonSubTypes;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||
|
||||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
|
||||
@JsonSubTypes(value = {
|
||||
@JsonSubTypes.Type(name = DataSourceMSQDestination.TYPE, value = DataSourceMSQDestination.class),
|
||||
@JsonSubTypes.Type(name = TaskReportMSQDestination.TYPE, value = TaskReportMSQDestination.class)
|
||||
})
|
||||
public interface MSQDestination
|
||||
{
|
||||
// No methods. Just a marker interface for deserialization.
|
||||
}
|
|
@ -0,0 +1,96 @@
|
|||
/*
|
||||
* 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 org.apache.druid.frame.key.ClusterByPartitions;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
public class MSQPartitionAssignment
|
||||
{
|
||||
private final ClusterByPartitions partitions;
|
||||
private final Map<Integer, SegmentIdWithShardSpec> allocations;
|
||||
|
||||
@JsonCreator
|
||||
public MSQPartitionAssignment(
|
||||
@JsonProperty("partitions") ClusterByPartitions partitions,
|
||||
@JsonProperty("allocations") Map<Integer, SegmentIdWithShardSpec> allocations
|
||||
)
|
||||
{
|
||||
this.partitions = Preconditions.checkNotNull(partitions, "partitions");
|
||||
this.allocations = Preconditions.checkNotNull(allocations, "allocations");
|
||||
|
||||
// Sanity checks.
|
||||
for (final int partitionNumber : allocations.keySet()) {
|
||||
if (partitionNumber < 0 || partitionNumber >= partitions.size()) {
|
||||
throw new IAE("Partition [%s] out of bounds", partitionNumber);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public ClusterByPartitions getPartitions()
|
||||
{
|
||||
return partitions;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Map<Integer, SegmentIdWithShardSpec> getAllocations()
|
||||
{
|
||||
return allocations;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
MSQPartitionAssignment that = (MSQPartitionAssignment) o;
|
||||
return Objects.equals(partitions, that.partitions) && Objects.equals(
|
||||
allocations,
|
||||
that.allocations
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(partitions, allocations);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "MSQPartitionAssignment{" +
|
||||
"partitions=" + partitions +
|
||||
", allocations=" + allocations +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -0,0 +1,160 @@
|
|||
/*
|
||||
* 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 org.apache.druid.msq.kernel.WorkerAssignmentStrategy;
|
||||
import org.apache.druid.query.Query;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Objects;
|
||||
|
||||
public class MSQSpec
|
||||
{
|
||||
private final Query<?> query;
|
||||
private final ColumnMappings columnMappings;
|
||||
private final MSQDestination destination;
|
||||
private final WorkerAssignmentStrategy assignmentStrategy;
|
||||
private final MSQTuningConfig tuningConfig;
|
||||
|
||||
@JsonCreator
|
||||
public MSQSpec(
|
||||
@JsonProperty("query") Query<?> query,
|
||||
@JsonProperty("columnMappings") @Nullable ColumnMappings columnMappings,
|
||||
@JsonProperty("destination") MSQDestination destination,
|
||||
@JsonProperty("assignmentStrategy") WorkerAssignmentStrategy assignmentStrategy,
|
||||
@JsonProperty("tuningConfig") MSQTuningConfig tuningConfig
|
||||
)
|
||||
{
|
||||
this.query = Preconditions.checkNotNull(query, "query");
|
||||
this.columnMappings = Preconditions.checkNotNull(columnMappings, "columnMappings");
|
||||
this.destination = Preconditions.checkNotNull(destination, "destination");
|
||||
this.assignmentStrategy = Preconditions.checkNotNull(assignmentStrategy, "assignmentStrategy");
|
||||
this.tuningConfig = Preconditions.checkNotNull(tuningConfig, "tuningConfig");
|
||||
}
|
||||
|
||||
public static Builder builder()
|
||||
{
|
||||
return new Builder();
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Query<?> getQuery()
|
||||
{
|
||||
return query;
|
||||
}
|
||||
|
||||
@JsonProperty("columnMappings")
|
||||
public ColumnMappings getColumnMappings()
|
||||
{
|
||||
return columnMappings;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public MSQDestination getDestination()
|
||||
{
|
||||
return destination;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public WorkerAssignmentStrategy getAssignmentStrategy()
|
||||
{
|
||||
return assignmentStrategy;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public MSQTuningConfig getTuningConfig()
|
||||
{
|
||||
return tuningConfig;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
MSQSpec that = (MSQSpec) o;
|
||||
return Objects.equals(query, that.query)
|
||||
&& Objects.equals(columnMappings, that.columnMappings)
|
||||
&& Objects.equals(destination, that.destination)
|
||||
&& Objects.equals(assignmentStrategy, that.assignmentStrategy)
|
||||
&& Objects.equals(tuningConfig, that.tuningConfig);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(query, columnMappings, destination, assignmentStrategy, tuningConfig);
|
||||
}
|
||||
|
||||
public static class Builder
|
||||
{
|
||||
private Query<?> query;
|
||||
private ColumnMappings columnMappings;
|
||||
private MSQDestination destination = TaskReportMSQDestination.instance();
|
||||
private WorkerAssignmentStrategy assignmentStrategy = WorkerAssignmentStrategy.MAX;
|
||||
private MSQTuningConfig tuningConfig;
|
||||
|
||||
public Builder query(Query<?> query)
|
||||
{
|
||||
this.query = query;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder columnMappings(final ColumnMappings columnMappings)
|
||||
{
|
||||
this.columnMappings = columnMappings;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder destination(final MSQDestination destination)
|
||||
{
|
||||
this.destination = destination;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder assignmentStrategy(final WorkerAssignmentStrategy assignmentStrategy)
|
||||
{
|
||||
this.assignmentStrategy = assignmentStrategy;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder tuningConfig(final MSQTuningConfig tuningConfig)
|
||||
{
|
||||
this.tuningConfig = tuningConfig;
|
||||
return this;
|
||||
}
|
||||
|
||||
public MSQSpec build()
|
||||
{
|
||||
if (destination == null) {
|
||||
destination = TaskReportMSQDestination.instance();
|
||||
}
|
||||
|
||||
return new MSQSpec(query, columnMappings, destination, assignmentStrategy, tuningConfig);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
|
||||
public class MSQTaskList
|
||||
{
|
||||
private final List<String> taskIds;
|
||||
|
||||
@JsonCreator
|
||||
public MSQTaskList(@JsonProperty("taskIds") List<String> taskIds)
|
||||
{
|
||||
this.taskIds = Preconditions.checkNotNull(taskIds, "taskIds");
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public List<String> getTaskIds()
|
||||
{
|
||||
return taskIds;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
MSQTaskList that = (MSQTaskList) o;
|
||||
return Objects.equals(taskIds, that.taskIds);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(taskIds);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "MSQTaskList{" +
|
||||
"taskIds=" + taskIds +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -0,0 +1,140 @@
|
|||
/*
|
||||
* 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.JsonInclude;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.apache.druid.indexer.partitions.PartitionsSpec;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* Tuning parameters for multi-stage query tasks: {@link MSQControllerTask} and {@link MSQWorkerTask}.
|
||||
*/
|
||||
public class MSQTuningConfig
|
||||
{
|
||||
/**
|
||||
* Lower than {@link org.apache.druid.segment.indexing.TuningConfig#DEFAULT_MAX_ROWS_IN_MEMORY} to minimize the
|
||||
* impact of per-row overheads that are not accounted for by OnheapIncrementalIndex. For example: overheads
|
||||
* related to creating bitmaps during persist.
|
||||
*
|
||||
* If this value proves to work well, we'll update {@link org.apache.druid.segment.indexing.TuningConfig} to bring
|
||||
* the two values in line with each other.
|
||||
*/
|
||||
private static final int DEFAULT_MAX_ROWS_IN_MEMORY = 100000;
|
||||
|
||||
/**
|
||||
* One worker task.
|
||||
*/
|
||||
private static final int DEFAULT_MAX_NUM_TASKS = 1;
|
||||
|
||||
@Nullable
|
||||
private final Integer maxNumWorkers;
|
||||
|
||||
@Nullable
|
||||
private final Integer maxRowsInMemory;
|
||||
|
||||
@Nullable
|
||||
private final Integer rowsPerSegment;
|
||||
|
||||
public MSQTuningConfig(
|
||||
@JsonProperty("maxNumWorkers") @Nullable final Integer maxNumWorkers,
|
||||
@JsonProperty("maxRowsInMemory") @Nullable final Integer maxRowsInMemory,
|
||||
@JsonProperty("rowsPerSegment") @Nullable final Integer rowsPerSegment
|
||||
)
|
||||
{
|
||||
this.maxNumWorkers = maxNumWorkers;
|
||||
this.maxRowsInMemory = maxRowsInMemory;
|
||||
this.rowsPerSegment = rowsPerSegment;
|
||||
}
|
||||
|
||||
public static MSQTuningConfig defaultConfig()
|
||||
{
|
||||
return new MSQTuningConfig(null, null, null);
|
||||
}
|
||||
|
||||
@JsonProperty("maxNumWorkers")
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
Integer getMaxNumWorkersForSerialization()
|
||||
{
|
||||
return maxNumWorkers;
|
||||
}
|
||||
|
||||
@JsonProperty("maxRowsInMemory")
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
Integer getMaxRowsInMemoryForSerialization()
|
||||
{
|
||||
return maxRowsInMemory;
|
||||
}
|
||||
|
||||
@JsonProperty("rowsPerSegment")
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
Integer getRowsPerSegmentForSerialization()
|
||||
{
|
||||
return rowsPerSegment;
|
||||
}
|
||||
|
||||
public int getMaxNumWorkers()
|
||||
{
|
||||
return maxNumWorkers != null ? maxNumWorkers : DEFAULT_MAX_NUM_TASKS;
|
||||
}
|
||||
|
||||
public int getMaxRowsInMemory()
|
||||
{
|
||||
return maxRowsInMemory != null ? maxRowsInMemory : DEFAULT_MAX_ROWS_IN_MEMORY;
|
||||
}
|
||||
|
||||
public int getRowsPerSegment()
|
||||
{
|
||||
return rowsPerSegment != null ? rowsPerSegment : PartitionsSpec.DEFAULT_MAX_ROWS_PER_SEGMENT;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
MSQTuningConfig that = (MSQTuningConfig) o;
|
||||
return Objects.equals(maxNumWorkers, that.maxNumWorkers)
|
||||
&& Objects.equals(maxRowsInMemory, that.maxRowsInMemory)
|
||||
&& Objects.equals(rowsPerSegment, that.rowsPerSegment);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(maxNumWorkers, maxRowsInMemory, rowsPerSegment);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "MSQTuningConfig{" +
|
||||
"maxNumWorkers=" + maxNumWorkers +
|
||||
", maxRowsInMemory=" + maxRowsInMemory +
|
||||
", rowsPerSegment=" + rowsPerSegment +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -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.indexing;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JacksonInject;
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.inject.Injector;
|
||||
import org.apache.druid.indexer.TaskStatus;
|
||||
import org.apache.druid.indexing.common.TaskToolbox;
|
||||
import org.apache.druid.indexing.common.actions.TaskActionClient;
|
||||
import org.apache.druid.indexing.common.config.TaskConfig;
|
||||
import org.apache.druid.indexing.common.task.AbstractTask;
|
||||
import org.apache.druid.msq.exec.MSQTasks;
|
||||
import org.apache.druid.msq.exec.Worker;
|
||||
import org.apache.druid.msq.exec.WorkerContext;
|
||||
import org.apache.druid.msq.exec.WorkerImpl;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
@JsonTypeName(MSQWorkerTask.TYPE)
|
||||
public class MSQWorkerTask extends AbstractTask
|
||||
{
|
||||
public static final String TYPE = "query_worker";
|
||||
|
||||
private final String controllerTaskId;
|
||||
private final int workerNumber;
|
||||
|
||||
// Using an Injector directly because tasks do not have a way to provide their own Guice modules.
|
||||
@JacksonInject
|
||||
private Injector injector;
|
||||
|
||||
private volatile Worker worker;
|
||||
|
||||
@JsonCreator
|
||||
@VisibleForTesting
|
||||
public MSQWorkerTask(
|
||||
@JsonProperty("controllerTaskId") final String controllerTaskId,
|
||||
@JsonProperty("dataSource") final String dataSource,
|
||||
@JsonProperty("workerNumber") final int workerNumber,
|
||||
@JsonProperty("context") final Map<String, Object> context
|
||||
)
|
||||
{
|
||||
super(
|
||||
MSQTasks.workerTaskId(controllerTaskId, workerNumber),
|
||||
controllerTaskId,
|
||||
null,
|
||||
dataSource,
|
||||
context
|
||||
);
|
||||
|
||||
this.controllerTaskId = controllerTaskId;
|
||||
this.workerNumber = workerNumber;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getControllerTaskId()
|
||||
{
|
||||
return controllerTaskId;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public int getWorkerNumber()
|
||||
{
|
||||
return workerNumber;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getType()
|
||||
{
|
||||
return TYPE;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public boolean isReady(final TaskActionClient taskActionClient)
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskStatus run(final TaskToolbox toolbox) throws Exception
|
||||
{
|
||||
final WorkerContext context = IndexerWorkerContext.createProductionInstance(toolbox, injector);
|
||||
worker = new WorkerImpl(this, context);
|
||||
return worker.run();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stopGracefully(TaskConfig taskConfig)
|
||||
{
|
||||
if (worker != null) {
|
||||
worker.stopGracefully();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,487 @@
|
|||
/*
|
||||
* 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.ImmutableList;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import com.google.common.util.concurrent.SettableFuture;
|
||||
import com.google.errorprone.annotations.concurrent.GuardedBy;
|
||||
import it.unimi.dsi.fastutil.ints.IntOpenHashSet;
|
||||
import it.unimi.dsi.fastutil.ints.IntSet;
|
||||
import org.apache.druid.common.guava.FutureUtils;
|
||||
import org.apache.druid.indexer.TaskLocation;
|
||||
import org.apache.druid.indexer.TaskState;
|
||||
import org.apache.druid.indexer.TaskStatus;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
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.msq.exec.ControllerContext;
|
||||
import org.apache.druid.msq.exec.ControllerImpl;
|
||||
import org.apache.druid.msq.exec.WorkerManagerClient;
|
||||
import org.apache.druid.msq.indexing.error.MSQException;
|
||||
import org.apache.druid.msq.indexing.error.TaskStartTimeoutFault;
|
||||
import org.apache.druid.msq.indexing.error.UnknownFault;
|
||||
import org.apache.druid.msq.indexing.error.WorkerFailedFault;
|
||||
import org.apache.druid.msq.util.MultiStageQueryContext;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.OptionalLong;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
/**
|
||||
* Like {@link org.apache.druid.indexing.common.task.batch.parallel.TaskMonitor}, but different.
|
||||
*/
|
||||
public class MSQWorkerTaskLauncher
|
||||
{
|
||||
private static final Logger log = new Logger(MSQWorkerTaskLauncher.class);
|
||||
private static final long HIGH_FREQUENCY_CHECK_MILLIS = 100;
|
||||
private static final long LOW_FREQUENCY_CHECK_MILLIS = 2000;
|
||||
private static final long SWITCH_TO_LOW_FREQUENCY_CHECK_AFTER_MILLIS = 10000;
|
||||
private static final long SHUTDOWN_TIMEOUT_MS = Duration.ofMinutes(1).toMillis();
|
||||
|
||||
// States for "state" variable.
|
||||
private enum State
|
||||
{
|
||||
NEW,
|
||||
STARTED,
|
||||
STOPPED
|
||||
}
|
||||
|
||||
private final String controllerTaskId;
|
||||
private final String dataSource;
|
||||
private final ControllerContext context;
|
||||
private final ExecutorService exec;
|
||||
private final long maxTaskStartDelayMillis;
|
||||
private final boolean durableStageStorageEnabled;
|
||||
|
||||
// Mutable state meant to be accessible by threads outside the main loop.
|
||||
private final SettableFuture<?> stopFuture = SettableFuture.create();
|
||||
private final AtomicReference<State> state = new AtomicReference<>(State.NEW);
|
||||
private final AtomicBoolean cancelTasksOnStop = new AtomicBoolean();
|
||||
|
||||
@GuardedBy("taskIds")
|
||||
private int desiredTaskCount = 0;
|
||||
|
||||
// Worker number -> task ID.
|
||||
@GuardedBy("taskIds")
|
||||
private final List<String> taskIds = new ArrayList<>();
|
||||
|
||||
// Worker number -> whether the task has fully started up or not.
|
||||
@GuardedBy("taskIds")
|
||||
private final IntSet fullyStartedTasks = new IntOpenHashSet();
|
||||
|
||||
// Mutable state accessible only to the main loop. LinkedHashMap since order of key set matters. Tasks are added
|
||||
// here once they are submitted for running, but before they are fully started up.
|
||||
private final Map<String, TaskTracker> taskTrackers = new LinkedHashMap<>();
|
||||
|
||||
// Set of tasks which are issued a cancel request by the controller.
|
||||
private final Set<String> canceledWorkerTasks = ConcurrentHashMap.newKeySet();
|
||||
|
||||
public MSQWorkerTaskLauncher(
|
||||
final String controllerTaskId,
|
||||
final String dataSource,
|
||||
final ControllerContext context,
|
||||
final boolean durableStageStorageEnabled,
|
||||
final long maxTaskStartDelayMillis
|
||||
)
|
||||
{
|
||||
this.controllerTaskId = controllerTaskId;
|
||||
this.dataSource = dataSource;
|
||||
this.context = context;
|
||||
this.exec = Execs.singleThreaded(
|
||||
"multi-stage-query-task-launcher[" + StringUtils.encodeForFormat(controllerTaskId) + "]-%s"
|
||||
);
|
||||
this.durableStageStorageEnabled = durableStageStorageEnabled;
|
||||
this.maxTaskStartDelayMillis = maxTaskStartDelayMillis;
|
||||
}
|
||||
|
||||
/**
|
||||
* Launches tasks, blocking until they are all in RUNNING state. Returns a future that resolves successfully when
|
||||
* all tasks end successfully or are canceled. The returned future resolves to an exception if one of the tasks fails
|
||||
* without being explicitly canceled, or if something else goes wrong.
|
||||
*/
|
||||
public ListenableFuture<?> start()
|
||||
{
|
||||
if (state.compareAndSet(State.NEW, State.STARTED)) {
|
||||
exec.submit(() -> {
|
||||
try {
|
||||
mainLoop();
|
||||
}
|
||||
catch (Throwable e) {
|
||||
log.warn(e, "Error encountered in main loop. Abandoning worker tasks.");
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
// Return an "everything is done" future that callers can wait for.
|
||||
return stopFuture;
|
||||
}
|
||||
|
||||
/**
|
||||
* Stops all tasks, blocking until they exit. Returns quietly, no matter whether there was an exception
|
||||
* associated with the future from {@link #start()} or not.
|
||||
*/
|
||||
public void stop(final boolean interrupt)
|
||||
{
|
||||
if (state.compareAndSet(State.STARTED, State.STOPPED)) {
|
||||
if (interrupt) {
|
||||
cancelTasksOnStop.set(true);
|
||||
}
|
||||
|
||||
synchronized (taskIds) {
|
||||
// Wake up sleeping mainLoop.
|
||||
taskIds.notifyAll();
|
||||
}
|
||||
|
||||
// Only shutdown the executor when transitioning from STARTED.
|
||||
exec.shutdown();
|
||||
} else if (state.get() == State.STOPPED) {
|
||||
// interrupt = true is sticky: don't reset on interrupt = false.
|
||||
if (interrupt) {
|
||||
cancelTasksOnStop.set(true);
|
||||
}
|
||||
} else {
|
||||
throw new ISE("Cannot stop(%s) from state [%s]", interrupt, state.get());
|
||||
}
|
||||
|
||||
// Block until stopped.
|
||||
try {
|
||||
FutureUtils.getUnchecked(stopFuture, false);
|
||||
}
|
||||
catch (Throwable ignored) {
|
||||
// Suppress.
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the list of currently-active tasks.
|
||||
*/
|
||||
public List<String> getTaskList()
|
||||
{
|
||||
synchronized (taskIds) {
|
||||
return ImmutableList.copyOf(taskIds);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Launch additional tasks, if needed, to bring the size of {@link #taskIds} up to {@code taskCount}. If enough
|
||||
* tasks are already running, this method does nothing.
|
||||
*/
|
||||
public void launchTasksIfNeeded(final int taskCount) throws InterruptedException
|
||||
{
|
||||
synchronized (taskIds) {
|
||||
if (taskCount > desiredTaskCount) {
|
||||
desiredTaskCount = taskCount;
|
||||
}
|
||||
|
||||
while (taskIds.size() < taskCount || !IntStream.range(0, taskCount).allMatch(fullyStartedTasks::contains)) {
|
||||
if (stopFuture.isDone() || stopFuture.isCancelled()) {
|
||||
FutureUtils.getUnchecked(stopFuture, false);
|
||||
throw new ISE("Stopped");
|
||||
}
|
||||
|
||||
taskIds.wait();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks if the controller has canceled the input taskId. This method is used in {@link ControllerImpl}
|
||||
* to figure out if the worker taskId is canceled by the controller. If yes, the errors from that worker taskId
|
||||
* are ignored for the error reports.
|
||||
*
|
||||
* @return true if task is canceled by the controller, else false
|
||||
*/
|
||||
public boolean isTaskCanceledByController(String taskId)
|
||||
{
|
||||
return canceledWorkerTasks.contains(taskId);
|
||||
}
|
||||
|
||||
private void mainLoop()
|
||||
{
|
||||
try {
|
||||
Throwable caught = null;
|
||||
|
||||
while (state.get() == State.STARTED) {
|
||||
final long loopStartTime = System.currentTimeMillis();
|
||||
|
||||
try {
|
||||
runNewTasks();
|
||||
updateTaskTrackersAndTaskIds();
|
||||
checkForErroneousTasks();
|
||||
}
|
||||
catch (Throwable e) {
|
||||
state.set(State.STOPPED);
|
||||
cancelTasksOnStop.set(true);
|
||||
caught = e;
|
||||
break;
|
||||
}
|
||||
|
||||
// Sleep for a bit, maybe.
|
||||
sleep(computeSleepTime(System.currentTimeMillis() - loopStartTime), false);
|
||||
}
|
||||
|
||||
// Only valid transition out of STARTED.
|
||||
assert state.get() == State.STOPPED;
|
||||
|
||||
final long stopStartTime = System.currentTimeMillis();
|
||||
|
||||
while (taskTrackers.values().stream().anyMatch(tracker -> !tracker.isComplete())) {
|
||||
final long loopStartTime = System.currentTimeMillis();
|
||||
|
||||
if (cancelTasksOnStop.get()) {
|
||||
shutDownTasks();
|
||||
}
|
||||
|
||||
updateTaskTrackersAndTaskIds();
|
||||
|
||||
// Sleep for a bit, maybe.
|
||||
final long now = System.currentTimeMillis();
|
||||
|
||||
if (now > stopStartTime + SHUTDOWN_TIMEOUT_MS) {
|
||||
if (caught != null) {
|
||||
throw caught;
|
||||
} else {
|
||||
throw new ISE("Task shutdown timed out (limit = %,dms)", SHUTDOWN_TIMEOUT_MS);
|
||||
}
|
||||
}
|
||||
|
||||
sleep(computeSleepTime(now - loopStartTime), true);
|
||||
}
|
||||
|
||||
if (caught != null) {
|
||||
throw caught;
|
||||
}
|
||||
|
||||
stopFuture.set(null);
|
||||
}
|
||||
catch (Throwable e) {
|
||||
if (!stopFuture.isDone()) {
|
||||
stopFuture.setException(e);
|
||||
}
|
||||
}
|
||||
|
||||
synchronized (taskIds) {
|
||||
// notify taskIds so launchWorkersIfNeeded can wake up, if it is sleeping, and notice stopFuture is done.
|
||||
taskIds.notifyAll();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Used by the main loop to launch new tasks up to {@link #desiredTaskCount}. Adds trackers to {@link #taskTrackers}
|
||||
* for newly launched tasks.
|
||||
*/
|
||||
private void runNewTasks()
|
||||
{
|
||||
final Map<String, Object> taskContext = new HashMap<>();
|
||||
|
||||
if (durableStageStorageEnabled) {
|
||||
taskContext.put(MultiStageQueryContext.CTX_ENABLE_DURABLE_SHUFFLE_STORAGE, true);
|
||||
}
|
||||
|
||||
final int firstTask;
|
||||
final int taskCount;
|
||||
|
||||
synchronized (taskIds) {
|
||||
firstTask = taskIds.size();
|
||||
taskCount = desiredTaskCount;
|
||||
}
|
||||
|
||||
for (int i = firstTask; i < taskCount; i++) {
|
||||
final MSQWorkerTask task = new MSQWorkerTask(
|
||||
controllerTaskId,
|
||||
dataSource,
|
||||
i,
|
||||
taskContext
|
||||
);
|
||||
|
||||
taskTrackers.put(task.getId(), new TaskTracker(i));
|
||||
context.workerManager().run(task.getId(), task);
|
||||
|
||||
synchronized (taskIds) {
|
||||
taskIds.add(task.getId());
|
||||
taskIds.notifyAll();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Used by the main loop to update {@link #taskTrackers} and {@link #fullyStartedTasks}.
|
||||
*/
|
||||
private void updateTaskTrackersAndTaskIds()
|
||||
{
|
||||
final Set<String> taskStatusesNeeded = new HashSet<>();
|
||||
for (final Map.Entry<String, TaskTracker> taskEntry : taskTrackers.entrySet()) {
|
||||
if (!taskEntry.getValue().isComplete()) {
|
||||
taskStatusesNeeded.add(taskEntry.getKey());
|
||||
}
|
||||
}
|
||||
|
||||
if (!taskStatusesNeeded.isEmpty()) {
|
||||
final WorkerManagerClient workerManager = context.workerManager();
|
||||
final Map<String, TaskStatus> statuses = workerManager.statuses(taskStatusesNeeded);
|
||||
|
||||
for (Map.Entry<String, TaskStatus> statusEntry : statuses.entrySet()) {
|
||||
final String taskId = statusEntry.getKey();
|
||||
final TaskTracker tracker = taskTrackers.get(taskId);
|
||||
tracker.status = statusEntry.getValue();
|
||||
|
||||
if (!tracker.status.getStatusCode().isComplete() && tracker.unknownLocation()) {
|
||||
// Look up location if not known. Note: this location is not used to actually contact the task. For that,
|
||||
// we have SpecificTaskServiceLocator. This location is only used to determine if a task has started up.
|
||||
tracker.initialLocation = workerManager.location(taskId);
|
||||
}
|
||||
|
||||
if (tracker.status.getStatusCode() == TaskState.RUNNING && !tracker.unknownLocation()) {
|
||||
synchronized (taskIds) {
|
||||
fullyStartedTasks.add(tracker.workerNumber);
|
||||
taskIds.notifyAll();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Used by the main loop to generate exceptions if any tasks have failed, have taken too long to start up, or
|
||||
* have gone inexplicably missing.
|
||||
*
|
||||
* Throws an exception if some task is erroneous.
|
||||
*/
|
||||
private void checkForErroneousTasks()
|
||||
{
|
||||
final int numTasks = taskTrackers.size();
|
||||
|
||||
for (final Map.Entry<String, TaskTracker> taskEntry : taskTrackers.entrySet()) {
|
||||
final String taskId = taskEntry.getKey();
|
||||
final TaskTracker tracker = taskEntry.getValue();
|
||||
|
||||
if (tracker.status == null) {
|
||||
throw new MSQException(UnknownFault.forMessage(StringUtils.format("Task [%s] status missing", taskId)));
|
||||
}
|
||||
|
||||
if (tracker.didRunTimeOut(maxTaskStartDelayMillis) && !canceledWorkerTasks.contains(taskId)) {
|
||||
throw new MSQException(new TaskStartTimeoutFault(numTasks + 1));
|
||||
}
|
||||
|
||||
if (tracker.didFail() && !canceledWorkerTasks.contains(taskId)) {
|
||||
throw new MSQException(new WorkerFailedFault(taskId, tracker.status.getErrorMsg()));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void shutDownTasks()
|
||||
{
|
||||
for (final Map.Entry<String, TaskTracker> taskEntry : taskTrackers.entrySet()) {
|
||||
final String taskId = taskEntry.getKey();
|
||||
final TaskTracker tracker = taskEntry.getValue();
|
||||
if (!canceledWorkerTasks.contains(taskId)
|
||||
&& (tracker.status == null || !tracker.status.getStatusCode().isComplete())) {
|
||||
canceledWorkerTasks.add(taskId);
|
||||
context.workerManager().cancel(taskId);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Used by the main loop to decide how often to check task status.
|
||||
*/
|
||||
private long computeSleepTime(final long loopDurationMs)
|
||||
{
|
||||
final OptionalLong maxTaskStartTime =
|
||||
taskTrackers.values().stream().mapToLong(tracker -> tracker.startTimeMs).max();
|
||||
|
||||
if (maxTaskStartTime.isPresent() &&
|
||||
System.currentTimeMillis() - maxTaskStartTime.getAsLong() < SWITCH_TO_LOW_FREQUENCY_CHECK_AFTER_MILLIS) {
|
||||
return HIGH_FREQUENCY_CHECK_MILLIS - loopDurationMs;
|
||||
} else {
|
||||
return LOW_FREQUENCY_CHECK_MILLIS - loopDurationMs;
|
||||
}
|
||||
}
|
||||
|
||||
private void sleep(final long sleepMillis, final boolean shuttingDown) throws InterruptedException
|
||||
{
|
||||
if (sleepMillis > 0) {
|
||||
if (shuttingDown) {
|
||||
Thread.sleep(sleepMillis);
|
||||
} else {
|
||||
// wait on taskIds so we can wake up early if needed.
|
||||
synchronized (taskIds) {
|
||||
taskIds.wait(sleepMillis);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// No wait, but check interrupted status anyway.
|
||||
if (Thread.interrupted()) {
|
||||
throw new InterruptedException();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Tracker for information about a worker. Mutable.
|
||||
*/
|
||||
private static class TaskTracker
|
||||
{
|
||||
private final int workerNumber;
|
||||
private final long startTimeMs = System.currentTimeMillis();
|
||||
private TaskStatus status;
|
||||
private TaskLocation initialLocation;
|
||||
|
||||
public TaskTracker(int workerNumber)
|
||||
{
|
||||
this.workerNumber = workerNumber;
|
||||
}
|
||||
|
||||
public boolean unknownLocation()
|
||||
{
|
||||
return initialLocation == null || TaskLocation.unknown().equals(initialLocation);
|
||||
}
|
||||
|
||||
public boolean isComplete()
|
||||
{
|
||||
return status != null && status.getStatusCode().isComplete();
|
||||
}
|
||||
|
||||
public boolean didFail()
|
||||
{
|
||||
return status != null && status.getStatusCode().isFailure();
|
||||
}
|
||||
|
||||
public boolean didRunTimeOut(final long maxTaskStartDelayMillis)
|
||||
{
|
||||
return (status == null || status.getStatusCode() == TaskState.RUNNING)
|
||||
&& unknownLocation()
|
||||
&& System.currentTimeMillis() - startTimeMs > maxTaskStartDelayMillis;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,295 @@
|
|||
/*
|
||||
* 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 com.google.common.util.concurrent.ListenableFuture;
|
||||
import it.unimi.dsi.fastutil.ints.IntSet;
|
||||
import it.unimi.dsi.fastutil.objects.Object2IntMap;
|
||||
import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap;
|
||||
import org.apache.druid.common.guava.FutureUtils;
|
||||
import org.apache.druid.data.input.InputRow;
|
||||
import org.apache.druid.data.input.Row;
|
||||
import org.apache.druid.data.input.Rows;
|
||||
import org.apache.druid.frame.Frame;
|
||||
import org.apache.druid.frame.channel.ReadableFrameChannel;
|
||||
import org.apache.druid.frame.channel.WritableFrameChannel;
|
||||
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.segment.FrameStorageAdapter;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.java.util.common.Intervals;
|
||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||
import org.apache.druid.java.util.common.guava.Sequence;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
import org.apache.druid.msq.exec.MSQTasks;
|
||||
import org.apache.druid.msq.input.ReadableInput;
|
||||
import org.apache.druid.msq.util.SequenceUtils;
|
||||
import org.apache.druid.segment.BaseObjectColumnValueSelector;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.Cursor;
|
||||
import org.apache.druid.segment.VirtualColumns;
|
||||
import org.apache.druid.segment.column.ColumnHolder;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.apache.druid.segment.realtime.appenderator.Appenderator;
|
||||
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
|
||||
import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class SegmentGeneratorFrameProcessor implements FrameProcessor<DataSegment>
|
||||
{
|
||||
private static final Logger log = new Logger(SegmentGeneratorFrameProcessor.class);
|
||||
|
||||
private final ReadableFrameChannel inChannel;
|
||||
private final FrameReader frameReader;
|
||||
private final Appenderator appenderator;
|
||||
private final SegmentIdWithShardSpec segmentIdWithShardSpec;
|
||||
private final List<String> dimensionsForInputRows;
|
||||
private final Object2IntMap<String> outputColumnNameToFrameColumnNumberMap;
|
||||
|
||||
private boolean firstRun = true;
|
||||
private long rowsWritten = 0L;
|
||||
|
||||
SegmentGeneratorFrameProcessor(
|
||||
final ReadableInput readableInput,
|
||||
final ColumnMappings columnMappings,
|
||||
final List<String> dimensionsForInputRows,
|
||||
final Appenderator appenderator,
|
||||
final SegmentIdWithShardSpec segmentIdWithShardSpec
|
||||
)
|
||||
{
|
||||
this.inChannel = readableInput.getChannel();
|
||||
this.frameReader = readableInput.getChannelFrameReader();
|
||||
this.appenderator = appenderator;
|
||||
this.segmentIdWithShardSpec = segmentIdWithShardSpec;
|
||||
this.dimensionsForInputRows = dimensionsForInputRows;
|
||||
|
||||
outputColumnNameToFrameColumnNumberMap = new Object2IntOpenHashMap<>();
|
||||
outputColumnNameToFrameColumnNumberMap.defaultReturnValue(-1);
|
||||
|
||||
for (final ColumnMapping columnMapping : columnMappings.getMappings()) {
|
||||
outputColumnNameToFrameColumnNumberMap.put(
|
||||
columnMapping.getOutputColumn(),
|
||||
frameReader.signature().indexOf(columnMapping.getQueryColumn())
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<ReadableFrameChannel> inputChannels()
|
||||
{
|
||||
return Collections.singletonList(inChannel);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<WritableFrameChannel> outputChannels()
|
||||
{
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReturnOrAwait<DataSegment> runIncrementally(final IntSet readableInputs) throws InterruptedException
|
||||
{
|
||||
if (firstRun) {
|
||||
log.debug("Starting job for segment [%s].", segmentIdWithShardSpec.asSegmentId());
|
||||
appenderator.startJob();
|
||||
firstRun = false;
|
||||
}
|
||||
|
||||
if (readableInputs.isEmpty()) {
|
||||
return ReturnOrAwait.awaitAll(1);
|
||||
}
|
||||
|
||||
if (inChannel.isFinished()) {
|
||||
if (rowsWritten == 0) {
|
||||
log.debug("Finished reading. No data for segment [%s], skipping.", segmentIdWithShardSpec.asSegmentId());
|
||||
return ReturnOrAwait.returnObject(null);
|
||||
} else {
|
||||
log.debug("Finished reading. Pushing segment [%s].", segmentIdWithShardSpec.asSegmentId());
|
||||
|
||||
// This is a blocking action which violates the FrameProcessor's contract.
|
||||
// useUniquePath = false because this class is meant to be used by batch jobs.
|
||||
final ListenableFuture<SegmentsAndCommitMetadata> pushFuture =
|
||||
appenderator.push(Collections.singletonList(segmentIdWithShardSpec), null, false);
|
||||
|
||||
final SegmentsAndCommitMetadata metadata;
|
||||
|
||||
try {
|
||||
metadata = FutureUtils.get(pushFuture, true);
|
||||
}
|
||||
catch (ExecutionException e) {
|
||||
throw new RuntimeException(e.getCause());
|
||||
}
|
||||
|
||||
appenderator.clear();
|
||||
|
||||
log.debug("Finished work for segment [%s].", segmentIdWithShardSpec.asSegmentId());
|
||||
return ReturnOrAwait.returnObject(Iterables.getOnlyElement(metadata.getSegments()));
|
||||
}
|
||||
} else {
|
||||
if (appenderator.getSegments().isEmpty()) {
|
||||
log.debug("Received first frame for segment [%s].", segmentIdWithShardSpec.asSegmentId());
|
||||
}
|
||||
|
||||
addFrame(inChannel.read());
|
||||
return ReturnOrAwait.awaitAll(1);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void cleanup() throws IOException
|
||||
{
|
||||
FrameProcessors.closeAll(inputChannels(), outputChannels(), appenderator::close);
|
||||
}
|
||||
|
||||
private void addFrame(final Frame frame)
|
||||
{
|
||||
final RowSignature signature = frameReader.signature();
|
||||
|
||||
// Reuse input row to avoid redoing allocations.
|
||||
final MSQInputRow inputRow = new MSQInputRow();
|
||||
|
||||
final Sequence<Cursor> cursorSequence =
|
||||
new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY)
|
||||
.makeCursors(null, Intervals.ETERNITY, VirtualColumns.EMPTY, Granularities.ALL, false, null);
|
||||
|
||||
SequenceUtils.forEach(
|
||||
cursorSequence,
|
||||
cursor -> {
|
||||
final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory();
|
||||
|
||||
//noinspection rawtypes
|
||||
@SuppressWarnings("rawtypes")
|
||||
final List<BaseObjectColumnValueSelector> selectors =
|
||||
frameReader.signature()
|
||||
.getColumnNames()
|
||||
.stream()
|
||||
.map(columnSelectorFactory::makeColumnValueSelector)
|
||||
.collect(Collectors.toList());
|
||||
|
||||
while (!cursor.isDone()) {
|
||||
for (int j = 0; j < signature.size(); j++) {
|
||||
inputRow.getBackingArray()[j] = selectors.get(j).getObject();
|
||||
}
|
||||
|
||||
try {
|
||||
rowsWritten++;
|
||||
appenderator.add(segmentIdWithShardSpec, inputRow, null);
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
|
||||
cursor.advance();
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
private class MSQInputRow implements InputRow
|
||||
{
|
||||
private final Object[] backingArray;
|
||||
private final int timeColumnNumber = outputColumnNameToFrameColumnNumberMap.getInt(ColumnHolder.TIME_COLUMN_NAME);
|
||||
|
||||
public MSQInputRow()
|
||||
{
|
||||
this.backingArray = new Object[frameReader.signature().size()];
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getTimestampFromEpoch()
|
||||
{
|
||||
if (timeColumnNumber < 0) {
|
||||
return 0;
|
||||
} else {
|
||||
return MSQTasks.primaryTimestampFromObjectForInsert(backingArray[timeColumnNumber]);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public DateTime getTimestamp()
|
||||
{
|
||||
return DateTimes.utc(getTimestampFromEpoch());
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getDimensions()
|
||||
{
|
||||
return dimensionsForInputRows;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public Object getRaw(String columnName)
|
||||
{
|
||||
final int columnNumber = outputColumnNameToFrameColumnNumberMap.getInt(columnName);
|
||||
if (columnNumber < 0) {
|
||||
return null;
|
||||
} else {
|
||||
return backingArray[columnNumber];
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getDimension(String columnName)
|
||||
{
|
||||
return Rows.objectToStrings(getRaw(columnName));
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public Number getMetric(String columnName)
|
||||
{
|
||||
return Rows.objectToNumber(columnName, getRaw(columnName), true);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(Row other)
|
||||
{
|
||||
// Not used during indexing.
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
private Object[] getBackingArray()
|
||||
{
|
||||
return backingArray;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "MSQInputRow{" +
|
||||
"backingArray=" + Arrays.toString(backingArray) +
|
||||
", timeColumnNumber=" + timeColumnNumber +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,373 @@
|
|||
/*
|
||||
* 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.fasterxml.jackson.annotation.JsonTypeName;
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Iterables;
|
||||
import org.apache.druid.frame.processor.OutputChannelFactory;
|
||||
import org.apache.druid.frame.processor.OutputChannels;
|
||||
import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
|
||||
import org.apache.druid.indexer.partitions.PartitionsSpec;
|
||||
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.msq.counters.CounterTracker;
|
||||
import org.apache.druid.msq.exec.WorkerMemoryParameters;
|
||||
import org.apache.druid.msq.input.InputSlice;
|
||||
import org.apache.druid.msq.input.InputSliceReader;
|
||||
import org.apache.druid.msq.input.ReadableInput;
|
||||
import org.apache.druid.msq.kernel.ExtraInfoHolder;
|
||||
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.StageDefinition;
|
||||
import org.apache.druid.msq.kernel.StagePartition;
|
||||
import org.apache.druid.segment.IndexSpec;
|
||||
import org.apache.druid.segment.data.CompressionFactory;
|
||||
import org.apache.druid.segment.data.CompressionStrategy;
|
||||
import org.apache.druid.segment.incremental.AppendableIndexSpec;
|
||||
import org.apache.druid.segment.incremental.ParseExceptionHandler;
|
||||
import org.apache.druid.segment.incremental.RowIngestionMeters;
|
||||
import org.apache.druid.segment.indexing.DataSchema;
|
||||
import org.apache.druid.segment.indexing.TuningConfig;
|
||||
import org.apache.druid.segment.realtime.FireDepartmentMetrics;
|
||||
import org.apache.druid.segment.realtime.appenderator.Appenderator;
|
||||
import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig;
|
||||
import org.apache.druid.segment.realtime.appenderator.Appenderators;
|
||||
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
|
||||
import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.joda.time.Period;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.File;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
import java.util.function.Consumer;
|
||||
|
||||
@JsonTypeName("segmentGenerator")
|
||||
public class SegmentGeneratorFrameProcessorFactory
|
||||
implements FrameProcessorFactory<List<SegmentIdWithShardSpec>, SegmentGeneratorFrameProcessor, DataSegment, Set<DataSegment>>
|
||||
{
|
||||
private final DataSchema dataSchema;
|
||||
private final ColumnMappings columnMappings;
|
||||
private final MSQTuningConfig tuningConfig;
|
||||
|
||||
@JsonCreator
|
||||
public SegmentGeneratorFrameProcessorFactory(
|
||||
@JsonProperty("dataSchema") final DataSchema dataSchema,
|
||||
@JsonProperty("columnMappings") final ColumnMappings columnMappings,
|
||||
@JsonProperty("tuningConfig") final MSQTuningConfig tuningConfig
|
||||
)
|
||||
{
|
||||
this.dataSchema = Preconditions.checkNotNull(dataSchema, "dataSchema");
|
||||
this.columnMappings = Preconditions.checkNotNull(columnMappings, "columnMappings");
|
||||
this.tuningConfig = Preconditions.checkNotNull(tuningConfig, "tuningConfig");
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public DataSchema getDataSchema()
|
||||
{
|
||||
return dataSchema;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public ColumnMappings getColumnMappings()
|
||||
{
|
||||
return columnMappings;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public MSQTuningConfig getTuningConfig()
|
||||
{
|
||||
return tuningConfig;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ProcessorsAndChannels<SegmentGeneratorFrameProcessor, DataSegment> makeProcessors(
|
||||
StageDefinition stageDefinition,
|
||||
int workerNumber,
|
||||
List<InputSlice> inputSlices,
|
||||
InputSliceReader inputSliceReader,
|
||||
@Nullable List<SegmentIdWithShardSpec> extra,
|
||||
OutputChannelFactory outputChannelFactory,
|
||||
FrameContext frameContext,
|
||||
int maxOutstandingProcessors,
|
||||
CounterTracker counters,
|
||||
Consumer<Throwable> warningPublisher
|
||||
)
|
||||
{
|
||||
final RowIngestionMeters meters = frameContext.rowIngestionMeters();
|
||||
|
||||
final ParseExceptionHandler parseExceptionHandler = new ParseExceptionHandler(
|
||||
meters,
|
||||
TuningConfig.DEFAULT_LOG_PARSE_EXCEPTIONS,
|
||||
TuningConfig.DEFAULT_MAX_PARSE_EXCEPTIONS,
|
||||
TuningConfig.DEFAULT_MAX_SAVED_PARSE_EXCEPTIONS
|
||||
);
|
||||
|
||||
// Expect a single input slice.
|
||||
final InputSlice slice = Iterables.getOnlyElement(inputSlices);
|
||||
final Sequence<Pair<Integer, ReadableInput>> inputSequence =
|
||||
Sequences.simple(Iterables.transform(
|
||||
inputSliceReader.attach(0, slice, counters, warningPublisher),
|
||||
new Function<ReadableInput, Pair<Integer, ReadableInput>>()
|
||||
{
|
||||
int i = 0;
|
||||
|
||||
@Override
|
||||
public Pair<Integer, ReadableInput> apply(ReadableInput readableInput)
|
||||
{
|
||||
return Pair.of(i++, readableInput);
|
||||
}
|
||||
}
|
||||
));
|
||||
|
||||
final Sequence<SegmentGeneratorFrameProcessor> workers = inputSequence.map(
|
||||
readableInputPair -> {
|
||||
final StagePartition stagePartition = Preconditions.checkNotNull(readableInputPair.rhs.getStagePartition());
|
||||
final SegmentIdWithShardSpec segmentIdWithShardSpec = extra.get(readableInputPair.lhs);
|
||||
final String idString = StringUtils.format("%s:%s", stagePartition, workerNumber);
|
||||
final File persistDirectory = new File(
|
||||
frameContext.persistDir(),
|
||||
segmentIdWithShardSpec.asSegmentId().toString()
|
||||
);
|
||||
|
||||
// Create directly, without using AppenderatorsManager, because we need different memory overrides due to
|
||||
// using one Appenderator per processing thread instead of per task.
|
||||
// Note: "createOffline" ignores the batchProcessingMode and always acts like CLOSED_SEGMENTS_SINKS.
|
||||
final Appenderator appenderator =
|
||||
Appenderators.createOffline(
|
||||
idString,
|
||||
dataSchema,
|
||||
makeAppenderatorConfig(
|
||||
tuningConfig,
|
||||
persistDirectory,
|
||||
frameContext.memoryParameters()
|
||||
),
|
||||
new FireDepartmentMetrics(), // We should eventually expose the FireDepartmentMetrics
|
||||
frameContext.segmentPusher(),
|
||||
frameContext.jsonMapper(),
|
||||
frameContext.indexIO(),
|
||||
frameContext.indexMerger(),
|
||||
meters,
|
||||
parseExceptionHandler,
|
||||
true
|
||||
);
|
||||
|
||||
return new SegmentGeneratorFrameProcessor(
|
||||
readableInputPair.rhs,
|
||||
columnMappings,
|
||||
dataSchema.getDimensionsSpec().getDimensionNames(),
|
||||
appenderator,
|
||||
segmentIdWithShardSpec
|
||||
);
|
||||
}
|
||||
);
|
||||
|
||||
return new ProcessorsAndChannels<>(workers, OutputChannels.none());
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeReference<Set<DataSegment>> getAccumulatedResultTypeReference()
|
||||
{
|
||||
return new TypeReference<Set<DataSegment>>() {};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<DataSegment> newAccumulatedResult()
|
||||
{
|
||||
return new HashSet<>();
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public Set<DataSegment> accumulateResult(Set<DataSegment> accumulated, DataSegment current)
|
||||
{
|
||||
if (current != null) {
|
||||
accumulated.add(current);
|
||||
}
|
||||
|
||||
return accumulated;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public Set<DataSegment> mergeAccumulatedResult(Set<DataSegment> accumulated, Set<DataSegment> otherAccumulated)
|
||||
{
|
||||
accumulated.addAll(otherAccumulated);
|
||||
return accumulated;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
SegmentGeneratorFrameProcessorFactory that = (SegmentGeneratorFrameProcessorFactory) o;
|
||||
return Objects.equals(dataSchema, that.dataSchema)
|
||||
&& Objects.equals(columnMappings, that.columnMappings)
|
||||
&& Objects.equals(tuningConfig, that.tuningConfig);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(dataSchema, columnMappings, tuningConfig);
|
||||
}
|
||||
|
||||
@Override
|
||||
@SuppressWarnings("rawtypes")
|
||||
public ExtraInfoHolder makeExtraInfoHolder(final List<SegmentIdWithShardSpec> segmentIdsWithShardSpecs)
|
||||
{
|
||||
return new SegmentGeneratorExtraInfoHolder(segmentIdsWithShardSpecs);
|
||||
}
|
||||
|
||||
private static AppenderatorConfig makeAppenderatorConfig(
|
||||
final MSQTuningConfig tuningConfig,
|
||||
final File persistDirectory,
|
||||
final WorkerMemoryParameters memoryParameters
|
||||
)
|
||||
{
|
||||
return new AppenderatorConfig()
|
||||
{
|
||||
@Override
|
||||
public AppendableIndexSpec getAppendableIndexSpec()
|
||||
{
|
||||
return TuningConfig.DEFAULT_APPENDABLE_INDEX;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxRowsInMemory()
|
||||
{
|
||||
// No need to apportion this amongst memoryParameters.getAppenderatorCount(), because it only exists
|
||||
// to minimize the impact of per-row overheads that are not accounted for by OnheapIncrementalIndex's
|
||||
// maxBytesInMemory handling. For example: overheads related to creating bitmaps during persist.
|
||||
return tuningConfig.getMaxRowsInMemory();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getMaxBytesInMemory()
|
||||
{
|
||||
return memoryParameters.getAppenderatorMaxBytesInMemory();
|
||||
}
|
||||
|
||||
@Override
|
||||
public PartitionsSpec getPartitionsSpec()
|
||||
{
|
||||
// We're not actually doing dynamic partitioning. This segment generator is going to get exactly one segment.
|
||||
return new DynamicPartitionsSpec(Integer.MAX_VALUE, Long.MAX_VALUE);
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexSpec getIndexSpec()
|
||||
{
|
||||
return new IndexSpec();
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexSpec getIndexSpecForIntermediatePersists()
|
||||
{
|
||||
// Disable compression for intermediate persists to reduce direct memory usage.
|
||||
return new IndexSpec(
|
||||
null,
|
||||
CompressionStrategy.UNCOMPRESSED, // Dimensions don't support NONE, so use UNCOMPRESSED
|
||||
CompressionStrategy.NONE, // NONE is more efficient than UNCOMPRESSED
|
||||
CompressionFactory.LongEncodingStrategy.LONGS,
|
||||
null
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isReportParseExceptions()
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxPendingPersists()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isSkipBytesInMemoryOverheadCheck()
|
||||
{
|
||||
return TuningConfig.DEFAULT_SKIP_BYTES_IN_MEMORY_OVERHEAD_CHECK;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Period getIntermediatePersistPeriod()
|
||||
{
|
||||
// Intermediate persist doesn't make much sense for batch jobs.
|
||||
return new Period(Integer.MAX_VALUE);
|
||||
}
|
||||
|
||||
@Override
|
||||
public File getBasePersistDirectory()
|
||||
{
|
||||
return persistDirectory;
|
||||
}
|
||||
|
||||
@Override
|
||||
public AppenderatorConfig withBasePersistDirectory(final File basePersistDirectory)
|
||||
{
|
||||
// Not used.
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory()
|
||||
{
|
||||
// Default SegmentWriteOutMediumFactory.
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxColumnsToMerge()
|
||||
{
|
||||
return memoryParameters.getAppenderatorMaxColumnsToMerge();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@JsonTypeName("segmentGenerator")
|
||||
public static class SegmentGeneratorExtraInfoHolder extends ExtraInfoHolder<List<SegmentIdWithShardSpec>>
|
||||
{
|
||||
@JsonCreator
|
||||
public SegmentGeneratorExtraInfoHolder(@Nullable @JsonProperty(INFO_KEY) final List<SegmentIdWithShardSpec> extra)
|
||||
{
|
||||
super(extra);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,45 @@
|
|||
/*
|
||||
* 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;
|
||||
|
||||
public class TaskReportMSQDestination implements MSQDestination
|
||||
{
|
||||
public static final TaskReportMSQDestination INSTANCE = new TaskReportMSQDestination();
|
||||
static final String TYPE = "taskReport";
|
||||
|
||||
private TaskReportMSQDestination()
|
||||
{
|
||||
// Singleton.
|
||||
}
|
||||
|
||||
@JsonCreator
|
||||
public static TaskReportMSQDestination instance()
|
||||
{
|
||||
return INSTANCE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "TaskReportMSQDestination{}";
|
||||
}
|
||||
}
|
|
@ -0,0 +1,222 @@
|
|||
/*
|
||||
* 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 it.unimi.dsi.fastutil.bytes.ByteArrays;
|
||||
import org.apache.commons.lang.mutable.MutableLong;
|
||||
import org.apache.druid.frame.file.FrameFileHttpResponseHandler;
|
||||
import org.apache.druid.frame.key.ClusterByPartitions;
|
||||
import org.apache.druid.indexing.common.TaskToolbox;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
import org.apache.druid.msq.exec.Worker;
|
||||
import org.apache.druid.msq.kernel.StageId;
|
||||
import org.apache.druid.msq.kernel.WorkOrder;
|
||||
import org.apache.druid.segment.realtime.firehose.ChatHandler;
|
||||
import org.apache.druid.segment.realtime.firehose.ChatHandlers;
|
||||
import org.apache.druid.server.security.Action;
|
||||
import org.apache.druid.utils.CloseableUtils;
|
||||
|
||||
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.QueryParam;
|
||||
import javax.ws.rs.core.Context;
|
||||
import javax.ws.rs.core.MediaType;
|
||||
import javax.ws.rs.core.Response;
|
||||
import javax.ws.rs.core.StreamingOutput;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
|
||||
public class WorkerChatHandler implements ChatHandler
|
||||
{
|
||||
private static final Logger log = new Logger(WorkerChatHandler.class);
|
||||
|
||||
/**
|
||||
* Callers must be able to store an entire chunk in memory. It can't be too large.
|
||||
*/
|
||||
private static final long CHANNEL_DATA_CHUNK_SIZE = 1_000_000;
|
||||
|
||||
private final Worker worker;
|
||||
private final MSQWorkerTask task;
|
||||
private final TaskToolbox toolbox;
|
||||
|
||||
public WorkerChatHandler(TaskToolbox toolbox, Worker worker)
|
||||
{
|
||||
this.worker = worker;
|
||||
this.task = worker.task();
|
||||
this.toolbox = toolbox;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns up to {@link #CHANNEL_DATA_CHUNK_SIZE} bytes of stage output data.
|
||||
*
|
||||
* See {@link org.apache.druid.msq.exec.WorkerClient#fetchChannelData} for the client-side code that calls this API.
|
||||
*/
|
||||
@GET
|
||||
@Path("/channels/{queryId}/{stageNumber}/{partitionNumber}")
|
||||
@Produces(MediaType.APPLICATION_OCTET_STREAM)
|
||||
public Response httpGetChannelData(
|
||||
@PathParam("queryId") final String queryId,
|
||||
@PathParam("stageNumber") final int stageNumber,
|
||||
@PathParam("partitionNumber") final int partitionNumber,
|
||||
@QueryParam("offset") final long offset,
|
||||
@Context final HttpServletRequest req
|
||||
)
|
||||
{
|
||||
ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper());
|
||||
try {
|
||||
final InputStream inputStream = worker.readChannel(queryId, stageNumber, partitionNumber, offset);
|
||||
if (inputStream == null) {
|
||||
return Response.status(Response.Status.NOT_FOUND).build();
|
||||
}
|
||||
|
||||
final Response.ResponseBuilder responseBuilder = Response.ok();
|
||||
|
||||
final byte[] readBuf = new byte[8192];
|
||||
final MutableLong bytesReadTotal = new MutableLong(0L);
|
||||
final int firstRead = inputStream.read(readBuf);
|
||||
|
||||
if (firstRead == -1) {
|
||||
// Empty read means we're at the end of the channel. Set the last fetch header so the client knows this.
|
||||
inputStream.close();
|
||||
return responseBuilder
|
||||
.header(
|
||||
FrameFileHttpResponseHandler.HEADER_LAST_FETCH_NAME,
|
||||
FrameFileHttpResponseHandler.HEADER_LAST_FETCH_VALUE
|
||||
)
|
||||
.entity(ByteArrays.EMPTY_ARRAY)
|
||||
.build();
|
||||
}
|
||||
|
||||
return Response.ok((StreamingOutput) output -> {
|
||||
try {
|
||||
int bytesReadThisCall = firstRead;
|
||||
do {
|
||||
final int bytesToWrite =
|
||||
(int) Math.min(CHANNEL_DATA_CHUNK_SIZE - bytesReadTotal.longValue(), bytesReadThisCall);
|
||||
output.write(readBuf, 0, bytesToWrite);
|
||||
bytesReadTotal.add(bytesReadThisCall);
|
||||
} while (bytesReadTotal.longValue() < CHANNEL_DATA_CHUNK_SIZE
|
||||
&& (bytesReadThisCall = inputStream.read(readBuf)) != -1);
|
||||
}
|
||||
catch (Throwable e) {
|
||||
// Suppress the exception to ensure nothing gets written over the wire once we've sent a 200. The client
|
||||
// will resume from where it left off.
|
||||
log.noStackTrace().warn(
|
||||
e,
|
||||
"Error writing channel for query [%s] stage [%s] partition [%s] offset [%,d] to [%s]",
|
||||
queryId,
|
||||
stageNumber,
|
||||
partitionNumber,
|
||||
offset,
|
||||
req.getRemoteAddr()
|
||||
);
|
||||
}
|
||||
finally {
|
||||
CloseableUtils.closeAll(inputStream, output);
|
||||
}
|
||||
}).build();
|
||||
}
|
||||
catch (IOException e) {
|
||||
return Response.status(Response.Status.INTERNAL_SERVER_ERROR).build();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* See {@link org.apache.druid.msq.exec.WorkerClient#postWorkOrder} for the client-side code that calls this API.
|
||||
*/
|
||||
@POST
|
||||
@Consumes(MediaType.APPLICATION_JSON)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("/workOrder")
|
||||
public Response httpPostWorkOrder(final WorkOrder workOrder, @Context final HttpServletRequest req)
|
||||
{
|
||||
ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper());
|
||||
worker.postWorkOrder(workOrder);
|
||||
return Response.status(Response.Status.ACCEPTED).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* See {@link org.apache.druid.msq.exec.WorkerClient#postResultPartitionBoundaries} for the client-side code that calls this API.
|
||||
*/
|
||||
@POST
|
||||
@Consumes(MediaType.APPLICATION_JSON)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("/resultPartitionBoundaries/{queryId}/{stageNumber}")
|
||||
public Response httpPostResultPartitionBoundaries(
|
||||
final ClusterByPartitions stagePartitionBoundaries,
|
||||
@PathParam("queryId") final String queryId,
|
||||
@PathParam("stageNumber") final int stageNumber,
|
||||
@Context final HttpServletRequest req
|
||||
)
|
||||
{
|
||||
ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper());
|
||||
if (worker.postResultPartitionBoundaries(stagePartitionBoundaries, queryId, stageNumber)) {
|
||||
return Response.status(Response.Status.ACCEPTED).build();
|
||||
} else {
|
||||
return Response.status(Response.Status.BAD_REQUEST).build();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* See {@link org.apache.druid.msq.exec.WorkerClient#postCleanupStage} for the client-side code that calls this API.
|
||||
*/
|
||||
@POST
|
||||
@Path("/cleanupStage/{queryId}/{stageNumber}")
|
||||
public Response httpPostCleanupStage(
|
||||
@PathParam("queryId") final String queryId,
|
||||
@PathParam("stageNumber") final int stageNumber,
|
||||
@Context final HttpServletRequest req
|
||||
)
|
||||
{
|
||||
ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper());
|
||||
worker.postCleanupStage(new StageId(queryId, stageNumber));
|
||||
return Response.status(Response.Status.ACCEPTED).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* See {@link org.apache.druid.msq.exec.WorkerClient#postFinish} for the client-side code that calls this API.
|
||||
*/
|
||||
@POST
|
||||
@Path("/finish")
|
||||
public Response httpPostFinish(@Context final HttpServletRequest req)
|
||||
{
|
||||
ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper());
|
||||
worker.postFinish();
|
||||
return Response.status(Response.Status.ACCEPTED).build();
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* See {@link org.apache.druid.msq.exec.WorkerClient#getCounters} for the client-side code that calls this API.
|
||||
*/
|
||||
@GET
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("/counters")
|
||||
public Response httpGetCounters(@Context final HttpServletRequest req)
|
||||
{
|
||||
ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper());
|
||||
return Response.status(Response.Status.OK).entity(worker.getCounters()).build();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,121 @@
|
|||
/*
|
||||
* 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.error;
|
||||
|
||||
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.StringUtils;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* Base implementation of {@link MSQFault}.
|
||||
*
|
||||
* Implements {@link #equals}, {@link #hashCode()}, and {@link #toString()} using {@link #errorCode} and
|
||||
* {@link #errorMessage}, so faults must either encode all relevant information in the message, or provide
|
||||
* their own implementation of these methods.
|
||||
*/
|
||||
public abstract class BaseMSQFault implements MSQFault
|
||||
{
|
||||
private final String errorCode;
|
||||
|
||||
@Nullable
|
||||
private final String errorMessage;
|
||||
|
||||
BaseMSQFault(final String errorCode, @Nullable final String errorMessage)
|
||||
{
|
||||
this.errorCode = Preconditions.checkNotNull(errorCode, "errorCode");
|
||||
this.errorMessage = errorMessage;
|
||||
}
|
||||
|
||||
BaseMSQFault(
|
||||
final String errorCode,
|
||||
final String errorMessageFormat,
|
||||
final Object errorMessageFirstArg,
|
||||
final Object... errorMessageOtherArgs
|
||||
)
|
||||
{
|
||||
this(errorCode, format(errorMessageFormat, errorMessageFirstArg, errorMessageOtherArgs));
|
||||
}
|
||||
|
||||
BaseMSQFault(final String errorCode)
|
||||
{
|
||||
this(errorCode, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getErrorCode()
|
||||
{
|
||||
return errorCode;
|
||||
}
|
||||
|
||||
@Override
|
||||
@Nullable
|
||||
@JsonProperty
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public String getErrorMessage()
|
||||
{
|
||||
return errorMessage;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
BaseMSQFault that = (BaseMSQFault) o;
|
||||
return Objects.equals(errorCode, that.errorCode) && Objects.equals(errorMessage, that.errorMessage);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(errorCode, errorMessage);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return getCodeWithMessage();
|
||||
}
|
||||
|
||||
private static String format(
|
||||
final String formatString,
|
||||
final Object firstArg,
|
||||
final Object... otherArgs
|
||||
)
|
||||
{
|
||||
final Object[] args = new Object[1 + (otherArgs != null ? otherArgs.length : 0)];
|
||||
|
||||
args[0] = firstArg;
|
||||
|
||||
if (otherArgs != null) {
|
||||
System.arraycopy(otherArgs, 0, args, 1, otherArgs.length);
|
||||
}
|
||||
|
||||
return StringUtils.format(formatString, args);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,72 @@
|
|||
/*
|
||||
* 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.error;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
@JsonTypeName(BroadcastTablesTooLargeFault.CODE)
|
||||
public class BroadcastTablesTooLargeFault extends BaseMSQFault
|
||||
{
|
||||
static final String CODE = "BroadcastTablesTooLarge";
|
||||
|
||||
private final long maxBroadcastTablesSize;
|
||||
|
||||
@JsonCreator
|
||||
public BroadcastTablesTooLargeFault(@JsonProperty("maxBroadcastTablesSize") final long maxBroadcastTablesSize)
|
||||
{
|
||||
super(CODE,
|
||||
"Size of the broadcast tables exceed the memory reserved for them (memory reserved for broadcast tables = %d bytes)",
|
||||
maxBroadcastTablesSize
|
||||
);
|
||||
this.maxBroadcastTablesSize = maxBroadcastTablesSize;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public long getMaxBroadcastTablesSize()
|
||||
{
|
||||
return maxBroadcastTablesSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
if (!super.equals(o)) {
|
||||
return false;
|
||||
}
|
||||
BroadcastTablesTooLargeFault that = (BroadcastTablesTooLargeFault) o;
|
||||
return maxBroadcastTablesSize == that.maxBroadcastTablesSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(super.hashCode(), maxBroadcastTablesSize);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,41 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.msq.indexing.error;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||
|
||||
@JsonTypeName(CanceledFault.CODE)
|
||||
public class CanceledFault extends BaseMSQFault
|
||||
{
|
||||
public static final CanceledFault INSTANCE = new CanceledFault();
|
||||
static final String CODE = "Canceled";
|
||||
|
||||
CanceledFault()
|
||||
{
|
||||
super(CODE, "Query canceled by user or by task shutdown.");
|
||||
}
|
||||
|
||||
@JsonCreator
|
||||
public static CanceledFault instance()
|
||||
{
|
||||
return INSTANCE;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,35 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.msq.indexing.error;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
@JsonTypeName(CannotParseExternalDataFault.CODE)
|
||||
public class CannotParseExternalDataFault extends BaseMSQFault
|
||||
{
|
||||
public static final String CODE = "CannotParseExternalData";
|
||||
|
||||
public CannotParseExternalDataFault(@JsonProperty("errorMessage") String message)
|
||||
{
|
||||
super(CODE, Preconditions.checkNotNull(message, "errorMessage"));
|
||||
}
|
||||
}
|
|
@ -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.error;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
@JsonTypeName(ColumnNameRestrictedFault.CODE)
|
||||
public class ColumnNameRestrictedFault extends BaseMSQFault
|
||||
{
|
||||
static final String CODE = "ColumnNameRestricted";
|
||||
|
||||
private final String columnName;
|
||||
|
||||
@JsonCreator
|
||||
public ColumnNameRestrictedFault(
|
||||
@JsonProperty("columnName") final String columnName
|
||||
)
|
||||
{
|
||||
super(CODE, StringUtils.format(
|
||||
"[%s] column name is reserved for MSQ's internal purpose. Please retry the query after renaming the column.",
|
||||
columnName));
|
||||
this.columnName = Preconditions.checkNotNull(columnName, "columnName");
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getColumnName()
|
||||
{
|
||||
return columnName;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
if (!super.equals(o)) {
|
||||
return false;
|
||||
}
|
||||
ColumnNameRestrictedFault that = (ColumnNameRestrictedFault) o;
|
||||
return Objects.equals(columnName, that.columnName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(super.hashCode(), columnName);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,89 @@
|
|||
/*
|
||||
* 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.error;
|
||||
|
||||
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.common.base.Preconditions;
|
||||
import org.apache.druid.frame.write.UnsupportedColumnTypeException;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Objects;
|
||||
|
||||
@JsonTypeName(ColumnTypeNotSupportedFault.CODE)
|
||||
public class ColumnTypeNotSupportedFault extends BaseMSQFault
|
||||
{
|
||||
static final String CODE = "ColumnTypeNotSupported";
|
||||
|
||||
private final String columnName;
|
||||
|
||||
@Nullable
|
||||
private final ColumnType columnType;
|
||||
|
||||
@JsonCreator
|
||||
public ColumnTypeNotSupportedFault(
|
||||
@JsonProperty("columnName") final String columnName,
|
||||
@JsonProperty("columnType") @Nullable final ColumnType columnType
|
||||
)
|
||||
{
|
||||
super(CODE, UnsupportedColumnTypeException.message(columnName, columnType));
|
||||
this.columnName = Preconditions.checkNotNull(columnName, "columnName");
|
||||
this.columnType = columnType;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getColumnName()
|
||||
{
|
||||
return columnName;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@JsonProperty
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public ColumnType getColumnType()
|
||||
{
|
||||
return columnType;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
if (!super.equals(o)) {
|
||||
return false;
|
||||
}
|
||||
ColumnTypeNotSupportedFault that = (ColumnTypeNotSupportedFault) o;
|
||||
return Objects.equals(columnName, that.columnName) && Objects.equals(columnType, that.columnType);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(super.hashCode(), columnName, columnType);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,80 @@
|
|||
/*
|
||||
* 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.error;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||
import org.apache.druid.msq.guice.MSQDurableStorageModule;
|
||||
import org.apache.druid.msq.util.MultiStageQueryContext;
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
@JsonTypeName(DurableStorageConfigurationFault.CODE)
|
||||
public class DurableStorageConfigurationFault extends BaseMSQFault
|
||||
{
|
||||
static final String CODE = "DurableStorageConfiguration";
|
||||
|
||||
private final String errorMessage;
|
||||
|
||||
@JsonCreator
|
||||
public DurableStorageConfigurationFault(@JsonProperty("message") final String errorMessage)
|
||||
{
|
||||
super(
|
||||
CODE,
|
||||
"Durable storage mode can only be enabled when %s is set to true and "
|
||||
+ "the connector is configured correctly. "
|
||||
+ "Check the documentation on how to enable durable storage mode. "
|
||||
+ "If you want to still query without durable storage mode, set %s to false in the query context. Got error %s",
|
||||
MSQDurableStorageModule.MSQ_INTERMEDIATE_STORAGE_ENABLED,
|
||||
MultiStageQueryContext.CTX_ENABLE_DURABLE_SHUFFLE_STORAGE,
|
||||
errorMessage
|
||||
);
|
||||
this.errorMessage = errorMessage;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getMessage()
|
||||
{
|
||||
return errorMessage;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
if (!super.equals(o)) {
|
||||
return false;
|
||||
}
|
||||
DurableStorageConfigurationFault that = (DurableStorageConfigurationFault) o;
|
||||
return Objects.equals(errorMessage, that.errorMessage);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(super.hashCode(), errorMessage);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,92 @@
|
|||
/*
|
||||
* 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.error;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.msq.counters.CounterNames;
|
||||
import org.apache.druid.msq.counters.CounterSnapshots;
|
||||
import org.apache.druid.msq.counters.CounterSnapshotsTree;
|
||||
import org.apache.druid.msq.counters.WarningCounters;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
|
||||
/**
|
||||
* Keeps a track of the warnings that have been so far and returns if any type has exceeded their designated limit
|
||||
* This class is thread safe
|
||||
*/
|
||||
public class FaultsExceededChecker
|
||||
{
|
||||
final Map<String, Long> maxFaultsAllowedCount;
|
||||
|
||||
public FaultsExceededChecker(final Map<String, Long> maxFaultsAllowedCount)
|
||||
{
|
||||
maxFaultsAllowedCount.forEach(
|
||||
(warning, count) ->
|
||||
Preconditions.checkArgument(
|
||||
count >= 0 || count == -1,
|
||||
StringUtils.format("Invalid limit of %d supplied for warnings of type %s. "
|
||||
+ "Limit can be greater than or equal to -1.", count, warning)
|
||||
)
|
||||
);
|
||||
this.maxFaultsAllowedCount = maxFaultsAllowedCount;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param snapshotsTree WorkerCounters have the count of the warnings generated per worker
|
||||
*
|
||||
* @return An optional which is empty if the faults count in the present in the task counters don't exceed their
|
||||
* prescribed limit, else it contains the errorCode and the maximum allowed faults for that errorCode
|
||||
*/
|
||||
public Optional<Pair<String, Long>> addFaultsAndCheckIfExceeded(CounterSnapshotsTree snapshotsTree)
|
||||
{
|
||||
final Map<Integer, Map<Integer, CounterSnapshots>> snapshotsMap = snapshotsTree.copyMap();
|
||||
|
||||
Map<String, Long> allWarnings = new HashMap<>();
|
||||
for (Map.Entry<Integer, Map<Integer, CounterSnapshots>> stageEntry : snapshotsMap.entrySet()) {
|
||||
for (Map.Entry<Integer, CounterSnapshots> workerEntry : stageEntry.getValue().entrySet()) {
|
||||
final WarningCounters.Snapshot warningsSnapshot =
|
||||
(WarningCounters.Snapshot) workerEntry.getValue().getMap().get(CounterNames.warnings());
|
||||
|
||||
if (warningsSnapshot != null) {
|
||||
for (Map.Entry<String, Long> entry : warningsSnapshot.getWarningCountMap().entrySet()) {
|
||||
allWarnings.compute(
|
||||
entry.getKey(),
|
||||
(ignored, value) -> value == null ? entry.getValue() : value + entry.getValue()
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
for (Map.Entry<String, Long> totalWarningCountEntry : allWarnings.entrySet()) {
|
||||
long limit = maxFaultsAllowedCount.getOrDefault(totalWarningCountEntry.getKey(), -1L);
|
||||
boolean passed = limit == -1 || totalWarningCountEntry.getValue() <= limit;
|
||||
if (!passed) {
|
||||
return Optional.of(Pair.of(totalWarningCountEntry.getKey(), limit));
|
||||
}
|
||||
}
|
||||
return Optional.empty();
|
||||
}
|
||||
|
||||
}
|
|
@ -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.error;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
@JsonTypeName(InsertCannotAllocateSegmentFault.CODE)
|
||||
public class InsertCannotAllocateSegmentFault extends BaseMSQFault
|
||||
{
|
||||
static final String CODE = "InsertCannotAllocateSegment";
|
||||
|
||||
private final String dataSource;
|
||||
private final Interval interval;
|
||||
|
||||
@JsonCreator
|
||||
public InsertCannotAllocateSegmentFault(
|
||||
@JsonProperty("dataSource") final String dataSource,
|
||||
@JsonProperty("interval") final Interval interval
|
||||
)
|
||||
{
|
||||
super(CODE, "Cannot allocate segment for dataSource [%s], interval [%s]", dataSource, interval);
|
||||
this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource");
|
||||
this.interval = Preconditions.checkNotNull(interval, "interval");
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getDataSource()
|
||||
{
|
||||
return dataSource;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Interval getInterval()
|
||||
{
|
||||
return interval;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
if (!super.equals(o)) {
|
||||
return false;
|
||||
}
|
||||
InsertCannotAllocateSegmentFault that = (InsertCannotAllocateSegmentFault) o;
|
||||
return Objects.equals(dataSource, that.dataSource) && Objects.equals(interval, that.interval);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(super.hashCode(), dataSource, interval);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,72 @@
|
|||
/*
|
||||
* 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.error;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
@JsonTypeName(InsertCannotBeEmptyFault.CODE)
|
||||
public class InsertCannotBeEmptyFault extends BaseMSQFault
|
||||
{
|
||||
static final String CODE = "InsertCannotBeEmpty";
|
||||
|
||||
private final String dataSource;
|
||||
|
||||
@JsonCreator
|
||||
public InsertCannotBeEmptyFault(
|
||||
@JsonProperty("dataSource") final String dataSource
|
||||
)
|
||||
{
|
||||
super(CODE, "No rows to insert for dataSource [%s]", dataSource);
|
||||
this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource");
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getDataSource()
|
||||
{
|
||||
return dataSource;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
if (!super.equals(o)) {
|
||||
return false;
|
||||
}
|
||||
InsertCannotBeEmptyFault that = (InsertCannotBeEmptyFault) o;
|
||||
return Objects.equals(dataSource, that.dataSource);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(super.hashCode(), dataSource);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,72 @@
|
|||
/*
|
||||
* 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.error;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
@JsonTypeName(InsertCannotOrderByDescendingFault.CODE)
|
||||
public class InsertCannotOrderByDescendingFault extends BaseMSQFault
|
||||
{
|
||||
static final String CODE = "InsertCannotOrderByDescending";
|
||||
|
||||
private final String columnName;
|
||||
|
||||
@JsonCreator
|
||||
public InsertCannotOrderByDescendingFault(
|
||||
@JsonProperty("columnName") final String columnName
|
||||
)
|
||||
{
|
||||
super(CODE, "Cannot ingest column [%s] in descending order", columnName);
|
||||
this.columnName = Preconditions.checkNotNull(columnName, "columnName");
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getColumnName()
|
||||
{
|
||||
return columnName;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
if (!super.equals(o)) {
|
||||
return false;
|
||||
}
|
||||
InsertCannotOrderByDescendingFault that = (InsertCannotOrderByDescendingFault) o;
|
||||
return Objects.equals(columnName, that.columnName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(super.hashCode(), columnName);
|
||||
}
|
||||
}
|
|
@ -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.indexing.error;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.apache.druid.timeline.SegmentId;
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
public class InsertCannotReplaceExistingSegmentFault extends BaseMSQFault
|
||||
{
|
||||
static final String CODE = "InsertCannotReplaceExistingSegment";
|
||||
|
||||
private final String segmentId;
|
||||
|
||||
public InsertCannotReplaceExistingSegmentFault(@JsonProperty("segmentId") String segmentId)
|
||||
{
|
||||
super(
|
||||
CODE,
|
||||
"Cannot replace existing segment [%s] because it is not within the "
|
||||
+ "bounds specified by replaceExistingTimeChunks",
|
||||
segmentId
|
||||
);
|
||||
this.segmentId = segmentId;
|
||||
}
|
||||
|
||||
public InsertCannotReplaceExistingSegmentFault(final SegmentId segmentId)
|
||||
{
|
||||
this(segmentId.toString());
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getSegmentId()
|
||||
{
|
||||
return segmentId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
if (!super.equals(o)) {
|
||||
return false;
|
||||
}
|
||||
InsertCannotReplaceExistingSegmentFault that = (InsertCannotReplaceExistingSegmentFault) o;
|
||||
return Objects.equals(segmentId, that.segmentId);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(super.hashCode(), segmentId);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,45 @@
|
|||
/*
|
||||
* 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.error;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||
|
||||
@JsonTypeName(InsertLockPreemptedFault.CODE)
|
||||
public class InsertLockPreemptedFault extends BaseMSQFault
|
||||
{
|
||||
public static final InsertLockPreemptedFault INSTANCE = new InsertLockPreemptedFault();
|
||||
static final String CODE = "InsertLockPreempted";
|
||||
|
||||
InsertLockPreemptedFault()
|
||||
{
|
||||
super(
|
||||
CODE,
|
||||
"Insert lock preempted while trying to ingest the data."
|
||||
+ " This can occur if there are higher priority jobs like real-time ingestion running on same time chunks."
|
||||
);
|
||||
}
|
||||
|
||||
@JsonCreator
|
||||
public static InsertLockPreemptedFault instance()
|
||||
{
|
||||
return INSTANCE;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,42 @@
|
|||
/*
|
||||
* 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.error;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||
import org.apache.druid.segment.column.ColumnHolder;
|
||||
|
||||
@JsonTypeName(InsertTimeNullFault.CODE)
|
||||
public class InsertTimeNullFault extends BaseMSQFault
|
||||
{
|
||||
public static final InsertTimeNullFault INSTANCE = new InsertTimeNullFault();
|
||||
static final String CODE = "InsertTimeNull";
|
||||
|
||||
InsertTimeNullFault()
|
||||
{
|
||||
super(CODE, "Null timestamp (%s) encountered during INSERT or REPLACE.", ColumnHolder.TIME_COLUMN_NAME);
|
||||
}
|
||||
|
||||
@JsonCreator
|
||||
public static InsertTimeNullFault instance()
|
||||
{
|
||||
return INSTANCE;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,66 @@
|
|||
/*
|
||||
* 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.error;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
public class InsertTimeOutOfBoundsFault extends BaseMSQFault
|
||||
{
|
||||
static final String CODE = "InsertTimeOutOfBounds";
|
||||
|
||||
private final Interval interval;
|
||||
|
||||
public InsertTimeOutOfBoundsFault(@JsonProperty("interval") Interval interval)
|
||||
{
|
||||
super(CODE, "Query generated time chunk [%s] out of bounds specified by replaceExistingTimeChunks", interval);
|
||||
this.interval = interval;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Interval getInterval()
|
||||
{
|
||||
return interval;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
if (!super.equals(o)) {
|
||||
return false;
|
||||
}
|
||||
InsertTimeOutOfBoundsFault that = (InsertTimeOutOfBoundsFault) o;
|
||||
return Objects.equals(interval, that.interval);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(super.hashCode(), interval);
|
||||
}
|
||||
}
|
|
@ -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.error;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
@JsonTypeName(InvalidNullByteFault.CODE)
|
||||
public class InvalidNullByteFault extends BaseMSQFault
|
||||
{
|
||||
static final String CODE = "InvalidNullByte";
|
||||
|
||||
private final String column;
|
||||
|
||||
@JsonCreator
|
||||
public InvalidNullByteFault(
|
||||
@JsonProperty("column") final String column
|
||||
)
|
||||
{
|
||||
super(CODE, "Invalid null byte in string column [%s]", column);
|
||||
this.column = column;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getColumn()
|
||||
{
|
||||
return column;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
if (!super.equals(o)) {
|
||||
return false;
|
||||
}
|
||||
InvalidNullByteFault that = (InvalidNullByteFault) o;
|
||||
return Objects.equals(column, that.column);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(super.hashCode(), column);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,200 @@
|
|||
/*
|
||||
* 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.error;
|
||||
|
||||
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 com.google.common.base.Throwables;
|
||||
import org.apache.druid.frame.processor.FrameRowTooLargeException;
|
||||
import org.apache.druid.frame.write.UnsupportedColumnTypeException;
|
||||
import org.apache.druid.java.util.common.parsers.ParseException;
|
||||
import org.apache.druid.msq.statistics.TooManyBucketsException;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Objects;
|
||||
|
||||
public class MSQErrorReport
|
||||
{
|
||||
private final String taskId;
|
||||
@Nullable
|
||||
private final String host;
|
||||
@Nullable
|
||||
private final Integer stageNumber;
|
||||
private final MSQFault error;
|
||||
@Nullable
|
||||
private final String exceptionStackTrace;
|
||||
|
||||
@JsonCreator
|
||||
MSQErrorReport(
|
||||
@JsonProperty("taskId") final String taskId,
|
||||
@JsonProperty("host") @Nullable final String host,
|
||||
@JsonProperty("stageNumber") final Integer stageNumber,
|
||||
@JsonProperty("error") final MSQFault fault,
|
||||
@JsonProperty("exceptionStackTrace") @Nullable final String exceptionStackTrace
|
||||
)
|
||||
{
|
||||
this.taskId = Preconditions.checkNotNull(taskId, "taskId");
|
||||
this.host = host;
|
||||
this.stageNumber = stageNumber;
|
||||
this.error = Preconditions.checkNotNull(fault, "error");
|
||||
this.exceptionStackTrace = exceptionStackTrace;
|
||||
}
|
||||
|
||||
public static MSQErrorReport fromFault(
|
||||
final String taskId,
|
||||
@Nullable final String host,
|
||||
@Nullable final Integer stageNumber,
|
||||
final MSQFault fault
|
||||
)
|
||||
{
|
||||
return new MSQErrorReport(taskId, host, stageNumber, fault, null);
|
||||
}
|
||||
|
||||
public static MSQErrorReport fromException(
|
||||
final String taskId,
|
||||
@Nullable final String host,
|
||||
@Nullable final Integer stageNumber,
|
||||
final Throwable e
|
||||
)
|
||||
{
|
||||
return new MSQErrorReport(
|
||||
taskId,
|
||||
host,
|
||||
stageNumber,
|
||||
getFaultFromException(e),
|
||||
Throwables.getStackTraceAsString(e)
|
||||
);
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getTaskId()
|
||||
{
|
||||
return taskId;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@JsonProperty
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public String getHost()
|
||||
{
|
||||
return host;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@JsonProperty
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public Integer getStageNumber()
|
||||
{
|
||||
return stageNumber;
|
||||
}
|
||||
|
||||
@JsonProperty("error")
|
||||
public MSQFault getFault()
|
||||
{
|
||||
return error;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@JsonProperty
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public String getExceptionStackTrace()
|
||||
{
|
||||
return exceptionStackTrace;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
MSQErrorReport that = (MSQErrorReport) o;
|
||||
return Objects.equals(taskId, that.taskId)
|
||||
&& Objects.equals(host, that.host)
|
||||
&& Objects.equals(stageNumber, that.stageNumber)
|
||||
&& Objects.equals(error, that.error)
|
||||
&& Objects.equals(exceptionStackTrace, that.exceptionStackTrace);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(taskId, host, error, exceptionStackTrace);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "MSQErrorReport{" +
|
||||
"taskId='" + taskId + '\'' +
|
||||
", host='" + host + '\'' +
|
||||
", stageNumber=" + stageNumber +
|
||||
", error=" + error +
|
||||
", exceptionStackTrace='" + exceptionStackTrace + '\'' +
|
||||
'}';
|
||||
}
|
||||
|
||||
/**
|
||||
* Magical code that extracts a useful fault from an exception, even if that exception is not necessarily a
|
||||
* {@link MSQException}. This method walks through the causal chain, and also "knows" about various exception
|
||||
* types thrown by other Druid code.
|
||||
*/
|
||||
public static MSQFault getFaultFromException(@Nullable final Throwable e)
|
||||
{
|
||||
// Unwrap exception wrappers to find an underlying fault. The assumption here is that the topmost recognizable
|
||||
// exception should be used to generate the fault code for the entire report.
|
||||
|
||||
Throwable cause = e;
|
||||
|
||||
// This method will grow as we try to add more faults and exceptions
|
||||
// One way of handling this would be to extend the faults to have a method like
|
||||
// public MSQFault fromException(@Nullable Throwable e) which returns the specific fault if it can be reconstructed
|
||||
// from the exception or null. Then instead of having a case per exception, we can have a case per fault, which
|
||||
// should be cool because there is a 1:1 mapping between faults and exceptions (apart from the more geeneric
|
||||
// UnknownFaults and MSQExceptions)
|
||||
while (cause != null) {
|
||||
|
||||
if (cause instanceof MSQException) {
|
||||
return ((MSQException) cause).getFault();
|
||||
} else if (cause instanceof ParseException) {
|
||||
return new CannotParseExternalDataFault(cause.getMessage());
|
||||
} else if (cause instanceof UnsupportedColumnTypeException) {
|
||||
final UnsupportedColumnTypeException unsupportedColumnTypeException = (UnsupportedColumnTypeException) cause;
|
||||
return new ColumnTypeNotSupportedFault(
|
||||
unsupportedColumnTypeException.getColumnName(),
|
||||
unsupportedColumnTypeException.getColumnType()
|
||||
);
|
||||
} else if (cause instanceof TooManyBucketsException) {
|
||||
return new TooManyBucketsFault(((TooManyBucketsException) cause).getMaxBuckets());
|
||||
} else if (cause instanceof FrameRowTooLargeException) {
|
||||
return new RowTooLargeFault(((FrameRowTooLargeException) cause).getMaxFrameSize());
|
||||
} else {
|
||||
cause = cause.getCause();
|
||||
}
|
||||
}
|
||||
|
||||
return UnknownFault.forException(e);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,51 @@
|
|||
/*
|
||||
* 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.error;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
/**
|
||||
* An unchecked exception that holds a {@link MSQFault}.
|
||||
*/
|
||||
public class MSQException extends RuntimeException
|
||||
{
|
||||
private final MSQFault fault;
|
||||
|
||||
public MSQException(
|
||||
@Nullable final Throwable cause,
|
||||
final MSQFault fault
|
||||
)
|
||||
{
|
||||
super(fault.getCodeWithMessage(), cause);
|
||||
this.fault = Preconditions.checkNotNull(fault, "fault");
|
||||
}
|
||||
|
||||
public MSQException(final MSQFault fault)
|
||||
{
|
||||
this(null, fault);
|
||||
}
|
||||
|
||||
public MSQFault getFault()
|
||||
{
|
||||
return fault;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,49 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.msq.indexing.error;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
/**
|
||||
* Error code for multi-stage queries.
|
||||
*
|
||||
* See {@link MSQErrorReport#getFaultFromException} for a mapping of exception type to error code.
|
||||
*/
|
||||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "errorCode")
|
||||
public interface MSQFault
|
||||
{
|
||||
String getErrorCode();
|
||||
|
||||
@Nullable
|
||||
String getErrorMessage();
|
||||
|
||||
default String getCodeWithMessage()
|
||||
{
|
||||
final String message = getErrorMessage();
|
||||
|
||||
if (message != null && !message.isEmpty()) {
|
||||
return getErrorCode() + ": " + message;
|
||||
} else {
|
||||
return getErrorCode();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,96 @@
|
|||
/*
|
||||
* 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.error;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.druid.msq.exec.Limits;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
/**
|
||||
* Limits the number of exceptions that get published to the underlying delegate publisher. This helps
|
||||
* in preventing the spam of exceptions from the worker task to the published source. As such, any implementation
|
||||
* of {@link MSQWarningReportPublisher} that is wrapped in this class cannot be sure that the warning handed off
|
||||
* is trully published
|
||||
*/
|
||||
public class MSQWarningReportLimiterPublisher implements MSQWarningReportPublisher
|
||||
{
|
||||
|
||||
final MSQWarningReportPublisher delegate;
|
||||
final long totalLimit;
|
||||
final Map<String, Long> errorCodeToLimit;
|
||||
final ConcurrentHashMap<String, Long> errorCodeToCurrentCount = new ConcurrentHashMap<>();
|
||||
|
||||
long totalCount = 0L;
|
||||
|
||||
final Object lock = new Object();
|
||||
|
||||
public MSQWarningReportLimiterPublisher(MSQWarningReportPublisher delegate)
|
||||
{
|
||||
this(
|
||||
delegate,
|
||||
Limits.MAX_VERBOSE_WARNINGS,
|
||||
ImmutableMap.of(
|
||||
CannotParseExternalDataFault.CODE, Limits.MAX_VERBOSE_PARSE_EXCEPTIONS
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
public MSQWarningReportLimiterPublisher(
|
||||
MSQWarningReportPublisher delegate,
|
||||
long totalLimit,
|
||||
Map<String, Long> errorCodeToLimit
|
||||
)
|
||||
{
|
||||
this.delegate = delegate;
|
||||
this.errorCodeToLimit = errorCodeToLimit;
|
||||
this.totalLimit = totalLimit;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void publishException(int stageNumber, Throwable e)
|
||||
{
|
||||
String errorCode = MSQErrorReport.getFaultFromException(e).getErrorCode();
|
||||
synchronized (lock) {
|
||||
totalCount = totalCount + 1;
|
||||
errorCodeToCurrentCount.compute(errorCode, (ignored, count) -> count == null ? 1L : count + 1);
|
||||
|
||||
if (totalLimit != -1 && totalCount > totalLimit) {
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
long limitForFault = errorCodeToLimit.getOrDefault(errorCode, -1L);
|
||||
synchronized (lock) {
|
||||
if (limitForFault != -1 && errorCodeToCurrentCount.getOrDefault(errorCode, 0L) > limitForFault) {
|
||||
return;
|
||||
}
|
||||
}
|
||||
delegate.publishException(stageNumber, e);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
delegate.close();
|
||||
}
|
||||
}
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue