From 85a8a1d805ecc112fa9d430762836b1218e8b4e2 Mon Sep 17 00:00:00 2001 From: Edgar Melendrez Date: Tue, 30 Jul 2024 10:53:59 -0700 Subject: [PATCH 01/12] [Docs]Batch04 - Bitwise numeric functions (#16805) * Batch04 - Bitwise numeric functions * Batch04 - Bitwise numeric functions * minor fixes * rewording bitwise_shift functions * rewording bitwise_shift functions * Update docs/querying/sql-functions.md * applying suggestions --------- Co-authored-by: Benedict Jin --- docs/querying/sql-functions.md | 171 ++++++++++++++++++++++++++++----- 1 file changed, 146 insertions(+), 25 deletions(-) diff --git a/docs/querying/sql-functions.md b/docs/querying/sql-functions.md index 1b2f68c7a89..47a797fde6b 100644 --- a/docs/querying/sql-functions.md +++ b/docs/querying/sql-functions.md @@ -396,67 +396,188 @@ Performs a bitwise XOR operation on all input values. ## BITWISE_AND -`BITWISE_AND(expr1, expr2)` +Returns the bitwise AND between two expressions: `expr1 & expr2`. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `BITWISE_AND(expr1, expr2)` +* **Function type:** Scalar, numeric -Returns the bitwise AND between the two expressions, that is, `expr1 & expr2`. +
Example + +The following example performs the bitwise AND operation `12 & 10`. + +```sql +SELECT BITWISE_AND(12, 10) AS "bitwise_and" +``` +Returns the following: + +| `bitwise_and` | +| -- | +| 8 | +
+ +[Learn more](sql-scalar.md#numeric-functions) ## BITWISE_COMPLEMENT -`BITWISE_COMPLEMENT(expr)` +Returns the bitwise complement (bitwise not) for the expression: `~expr`. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `BITWISE_COMPLEMENT(expr)` +* **Function type:** Scalar, numeric -Returns the bitwise NOT for the expression, that is, `~expr`. +
Example + +The following example performs the bitwise complement operation `~12`. + +```sql +SELECT BITWISE_COMPLEMENT(12) AS "bitwise_complement" +``` +Returns the following: + +| `bitwise_complement` | +| -- | +| -13 | +
+ +[Learn more](sql-scalar.md#numeric-functions) ## BITWISE_CONVERT_DOUBLE_TO_LONG_BITS -`BITWISE_CONVERT_DOUBLE_TO_LONG_BITS(expr)` +Converts the bits of an IEEE 754 floating-point double value to long. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:**`BITWISE_CONVERT_DOUBLE_TO_LONG_BITS(expr)` +* **Function type:** Scalar, numeric + +
Example + +The following example returns the IEEE 754 floating-point double representation of `255` as a long. + +```sql +SELECT BITWISE_CONVERT_DOUBLE_TO_LONG_BITS(255) AS "ieee_754_double_to_long" +``` +Returns the following: + +| `ieee_754_double_to_long` | +| -- | +| `4643176031446892544` | +
+ +[Learn more](sql-scalar.md#numeric-functions) -Converts the bits of an IEEE 754 floating-point double value to a long. ## BITWISE_CONVERT_LONG_BITS_TO_DOUBLE -`BITWISE_CONVERT_LONG_BITS_TO_DOUBLE(expr)` - -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) - Converts a long to the IEEE 754 floating-point double specified by the bits stored in the long. +* **Syntax:**`BITWISE_CONVERT_LONG_BITS_TO_DOUBLE(expr)` +* **Function type:** Scalar, numeric + +
Example + +The following example returns the long representation of `4643176031446892544` as an IEEE 754 floating-point double. + +```sql +SELECT BITWISE_CONVERT_LONG_BITS_TO_DOUBLE(4643176031446892544) AS "long_to_ieee_754_double" +``` +Returns the following: + +| `long_to_ieee_754_double` | +| -- | +| `255` | +
+ +[Learn more](sql-scalar.md#numeric-functions) + ## BITWISE_OR -`BITWISE_OR(expr1, expr2)` +Returns the bitwise OR between the two expressions: `expr1 | expr2`. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `BITWISE_OR(expr1, expr2)` +* **Function type:** Scalar, numeric -Returns the bitwise OR between the two expressions, that is, `expr1 | expr2`. +
Example + +The following example performs the bitwise OR operation `12 | 10`. + +```sql +SELECT BITWISE_OR(12, 10) AS "bitwise_or" +``` +Returns the following: + +| `bitwise_or` | +| -- | +| `14` | +
+ +[Learn more](sql-scalar.md#numeric-functions) ## BITWISE_SHIFT_LEFT -`BITWISE_SHIFT_LEFT(expr1, expr2)` +Returns the bitwise left shift by x positions of an expr: `expr << x`. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `BITWISE_SHIFT_LEFT(expr, x)` +* **Function type:** Scalar, numeric -Returns a bitwise left shift of expr1, that is, `expr1 << expr2`. +
Example + +The following example performs the bitwise SHIFT operation `2 << 3`. + +```sql +SELECT BITWISE_SHIFT_LEFT(2, 3) AS "bitwise_shift_left" +``` +Returns the following: + +| `bitwise_shift_left` | +| -- | +| `16` | +
+ +[Learn more](sql-scalar.md#numeric-functions) ## BITWISE_SHIFT_RIGHT -`BITWISE_SHIFT_RIGHT(expr1, expr2)` +Returns the bitwise right shift by x positions of an expr: `expr >> x`. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `BITWISE_SHIFT_RIGHT(expr, x)` +* **Function type:** Scalar, numeric -Returns a bitwise right shift of expr1, that is, `expr1 >> expr2`. +
Example + +The following example performs the bitwise SHIFT operation `16 >> 3`. + +```sql +SELECT BITWISE_SHIFT_RIGHT(16, 3) AS "bitwise_shift_right" +``` +Returns the following: + +| `bitwise_shift_right` | +| -- | +| `2` | +
+ +[Learn more](sql-scalar.md#numeric-functions) ## BITWISE_XOR -`BITWISE_XOR(expr1, expr2)` +Returns the bitwise exclusive OR between the two expressions: `expr1 ^ expr2`. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `BITWISE_XOR(expr1, expr2)` +* **Function type:** Scalar, numeric -Returns the bitwise exclusive OR between the two expressions, that is, `expr1 ^ expr2`. +
Example + +The following example performs the bitwise XOR operation `12 ^ 10`. + +```sql +SELECT BITWISE_XOR(12, 10) AS "bitwise_xor" +``` +Returns the following: + +| `bitwise_xor` | +| -- | +| `6` | +
+ +[Learn more](sql-scalar.md#numeric-functions) ## BLOOM_FILTER From 3bb6d40285d41d9734cd39bce5c867ea8d978f21 Mon Sep 17 00:00:00 2001 From: Edgar Melendrez Date: Tue, 30 Jul 2024 17:30:01 -0700 Subject: [PATCH 02/12] [docs] batch 5 updating functions (#16812) * batch 5 * Update docs/querying/sql-functions.md * applying suggestions --------- Co-authored-by: Benedict Jin --- docs/querying/sql-functions.md | 128 ++++++++++++++++++++++++++++----- 1 file changed, 110 insertions(+), 18 deletions(-) diff --git a/docs/querying/sql-functions.md b/docs/querying/sql-functions.md index 47a797fde6b..2ec056290b6 100644 --- a/docs/querying/sql-functions.md +++ b/docs/querying/sql-functions.md @@ -831,12 +831,37 @@ Returns the rank for a row within a window without gaps. For example, if two row ## DIV -`DIV(x, y)` - -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) - Returns the result of integer division of `x` by `y`. +* **Syntax:** `DIV(x, y)` +* **Function type:** Scalar, numeric + + + +:::info + + The `DIV` function is not implemented in Druid versions 30.0.0 or earlier. Consider using [`SAFE_DIVIDE`](./sql-functions.md/#safe_divide) instead. + +::: + +[Learn more](sql-scalar.md#numeric-functions) + ## DS_CDF `DS_CDF(expr, splitPoint0, splitPoint1, ...)` @@ -1092,28 +1117,78 @@ Returns a union of HLL sketches. ## HUMAN_READABLE_BINARY_BYTE_FORMAT -`HUMAN_READABLE_BINARY_BYTE_FORMAT(value[, precision])` +Converts an integer byte size into human-readable [IEC](https://en.wikipedia.org/wiki/Binary_prefix) format. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `HUMAN_READABLE_BINARY_BYTE_FORMAT(value[, precision])` +* **Function type:** Scalar, numeric -Converts an integer byte size into human-readable IEC format. +
Example + + The following example converts `1000000` into IEC format. + + ```sql + SELECT HUMAN_READABLE_BINARY_BYTE_FORMAT(1000000, 2) AS "iec_format" + ``` + + Returns the following: + + | `iec_format` | + | -- | + | `976.56 KiB` | + +
+ +[Learn more](sql-scalar.md#numeric-functions) ## HUMAN_READABLE_DECIMAL_BYTE_FORMAT -`HUMAN_READABLE_DECIMAL_BYTE_FORMAT(value[, precision])` +Converts a byte size into human-readable [SI](https://en.wikipedia.org/wiki/Binary_prefix) format. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `HUMAN_READABLE_DECIMAL_BYTE_FORMAT(value[, precision])` +* **Function type:** Scalar, numeric -Converts a byte size into human-readable SI format. +
Example + +The following example converts `1000000` into SI format. + +```sql +SELECT HUMAN_READABLE_DECIMAL_BYTE_FORMAT(1000000, 2) AS "si_format" +``` + +Returns the following: + +|`si_format`| +|--| +|`1.00 MB`| + +
+ +[Learn more](sql-scalar.md#numeric-functions) ## HUMAN_READABLE_DECIMAL_FORMAT -`HUMAN_READABLE_DECIMAL_FORMAT(value[, precision])` - -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) - Converts a byte size into human-readable SI format with single-character units. +* **Syntax:** `HUMAN_READABLE_DECIMAL_FORMAT(value[, precision])` +* **Function type:** Scalar, numeric + +
Example + + The following example converts `1000000` into single character SI format. + +```sql +SELECT HUMAN_READABLE_DECIMAL_FORMAT(1000000, 2) AS "single_character_si_format" +``` + +Returns the following: + +|`single_character_si_format`| +|--| +|`1.00 M`| +
+ +[Learn more](sql-scalar.md#numeric-functions) + ## ICONTAINS_STRING `ICONTAINS_STRING(, str)` @@ -1740,12 +1815,29 @@ Trims characters from the trailing end of an expression. ## SAFE_DIVIDE -`SAFE_DIVIDE(x, y)` - -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) - Returns `x` divided by `y`, guarded on division by 0. +* **Syntax:** `SAFE_DIVIDE(x, y)` +* **Function type:** Scalar, numeric + +
Example + +The following example calculates divisions of integer `78` by integer `10`. + +```sql +SELECT SAFE_DIVIDE(78, 10) AS "safe_division" +``` + +Returns the following: + +|`safe_division`| +|--| +| `7` | + +
+ +[Learn more](sql-scalar.md#numeric-functions) + ## SIN Calculates the trigonometric sine of an angle expressed in radians. From 01f6cfcbf5de7147d2d6e350ea9769fe8d870cdc Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 30 Jul 2024 18:41:24 -0700 Subject: [PATCH 03/12] MSQ worker: Support in-memory shuffles. (#16790) * MSQ worker: Support in-memory shuffles. This patch is a follow-up to #16168, adding worker-side support for in-memory shuffles. Changes include: 1) Worker-side code now respects the same context parameter "maxConcurrentStages" that was added to the controller in #16168. The parameter remains undocumented for now, to give us a chance to more fully develop and test this functionality. 1) WorkerImpl is broken up into WorkerImpl, RunWorkOrder, and RunWorkOrderListener to improve readability. 2) WorkerImpl has a new StageOutputHolder + StageOutputReader concept, which abstract over memory-based or file-based stage results. 3) RunWorkOrder is updated to create in-memory stage output channels when instructed to. 4) ControllerResource is updated to add /doneReadingInput/, so the controller can tell when workers that sort, but do not gather statistics, are done reading their inputs. 5) WorkerMemoryParameters is updated to consider maxConcurrentStages. Additionally, WorkerChatHandler is split into WorkerResource, so as to match ControllerChatHandler and ControllerResource. * Updates for static checks, test coverage. * Fixes. * Remove exception. * Changes from review. * Address static check. * Changes from review. * Improvements to docs and method names. * Update comments, add test. * Additional javadocs. * Fix throws. * Fix worker stopping in tests. * Fix stuck test. --- .../org/apache/druid/msq/exec/Controller.java | 4 +- .../druid/msq/exec/ControllerClient.java | 26 +- .../apache/druid/msq/exec/ControllerImpl.java | 10 +- .../msq/exec/ControllerMemoryParameters.java | 3 +- .../exec/ListeningOutputChannelFactory.java | 74 + .../druid/msq/exec/OutputChannelMode.java | 9 +- .../apache/druid/msq/exec/RunWorkOrder.java | 1051 +++++++ .../druid/msq/exec/RunWorkOrderListener.java | 57 + .../org/apache/druid/msq/exec/Worker.java | 64 +- .../apache/druid/msq/exec/WorkerContext.java | 46 +- .../org/apache/druid/msq/exec/WorkerImpl.java | 2410 ++++++----------- .../msq/exec/WorkerMemoryParameters.java | 82 +- .../msq/exec/WorkerStorageParameters.java | 6 +- .../msq/indexing/IndexerFrameContext.java | 35 +- .../IndexerResourcePermissionMapper.java | 6 + .../msq/indexing/IndexerWorkerContext.java | 110 +- .../druid/msq/indexing/MSQWorkerTask.java | 19 +- .../client/IndexerControllerClient.java | 2 +- .../indexing/client/WorkerChatHandler.java | 313 +-- .../indexing/error/NotEnoughMemoryFault.java | 38 +- .../apache/druid/msq/input/InputSlices.java | 22 +- .../external/ExternalInputSliceReader.java | 13 +- .../apache/druid/msq/kernel/FrameContext.java | 20 +- .../apache/druid/msq/kernel/WorkOrder.java | 2 +- .../msq/kernel/worker/WorkerStageKernel.java | 47 +- .../msq/kernel/worker/WorkerStagePhase.java | 29 +- .../druid/msq/rpc/BaseWorkerClientImpl.java | 4 +- .../druid/msq/rpc/ControllerResource.java | 25 +- .../druid/msq/rpc/MSQResourceUtils.java | 16 + .../msq/rpc/ResourcePermissionMapper.java | 6 +- .../apache/druid/msq/rpc/WorkerResource.java | 391 +++ .../input/MetaInputChannelFactory.java | 115 + .../WorkerOrLocalInputChannelFactory.java | 70 + .../shuffle/output/ByteChunksInputStream.java | 116 + .../output/ChannelStageOutputReader.java | 274 ++ .../shuffle/output/FileStageOutputReader.java | 99 + .../output/FutureReadableFrameChannel.java | 124 + .../shuffle/output/NilStageOutputReader.java | 77 + .../msq/shuffle/output/StageOutputHolder.java | 141 + .../msq/shuffle/output/StageOutputReader.java | 77 + .../msq/exec/MSQDrillWindowQueryTest.java | 11 +- .../apache/druid/msq/exec/WorkerImplTest.java | 54 - .../msq/exec/WorkerMemoryParametersTest.java | 126 +- .../indexing/IndexerWorkerContextTest.java | 9 + .../msq/indexing/WorkerChatHandlerTest.java | 65 +- .../msq/indexing/error/MSQFaultSerdeTest.java | 2 +- .../output/ByteChunksInputStreamTest.java | 149 + .../output/ChannelStageOutputReaderTest.java | 255 ++ .../msq/test/CalciteArraysQueryMSQTest.java | 10 +- .../test/CalciteNestedDataQueryMSQTest.java | 10 +- .../test/CalciteSelectJoinQueryMSQTest.java | 10 +- .../msq/test/CalciteSelectQueryMSQTest.java | 10 +- .../msq/test/CalciteUnionQueryMSQTest.java | 10 +- .../apache/druid/msq/test/MSQTestBase.java | 24 +- .../msq/test/MSQTestControllerClient.java | 4 +- .../msq/test/MSQTestControllerContext.java | 19 +- .../druid/msq/test/MSQTestWorkerClient.java | 12 +- .../druid/msq/test/MSQTestWorkerContext.java | 182 +- .../channel/ReadableFileFrameChannel.java | 8 + .../ReadableFileFrameChannelTest.java | 104 + 60 files changed, 4813 insertions(+), 2294 deletions(-) create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ListeningOutputChannelFactory.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/RunWorkOrder.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/RunWorkOrderListener.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/WorkerResource.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/MetaInputChannelFactory.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/WorkerOrLocalInputChannelFactory.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/ByteChunksInputStream.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/ChannelStageOutputReader.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/FileStageOutputReader.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/FutureReadableFrameChannel.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/NilStageOutputReader.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/StageOutputHolder.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/StageOutputReader.java delete mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerImplTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/output/ByteChunksInputStreamTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/output/ChannelStageOutputReaderTest.java create mode 100644 processing/src/test/java/org/apache/druid/frame/processor/ReadableFileFrameChannelTest.java diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Controller.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Controller.java index f04286dd7c4..d2370b05793 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Controller.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Controller.java @@ -117,9 +117,9 @@ public interface Controller ); /** - * Returns the current list of task ids, ordered by worker number. The Nth task has worker number N. + * Returns the current list of worker IDs, ordered by worker number. The Nth worker has worker number N. */ - List getTaskIds(); + List getWorkerIds(); @Nullable TaskReport.ReportMap liveReports(); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerClient.java index 405ff4fb902..428ce59cd8f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerClient.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerClient.java @@ -23,20 +23,25 @@ import org.apache.druid.msq.counters.CounterSnapshotsTree; import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.apache.druid.msq.kernel.StageDefinition; import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.kernel.WorkOrder; import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation; import javax.annotation.Nullable; +import java.io.Closeable; import java.io.IOException; import java.util.List; /** - * Client for the multi-stage query controller. Used by a Worker task. + * Client for the multi-stage query controller. Used by a {@link Worker}. Each instance is specific to a single query, + * meaning it communicates with a single controller. */ -public interface ControllerClient extends AutoCloseable +public interface ControllerClient extends Closeable { /** - * Client side method to update the controller with partial key statistics information for a particular stage and worker. - * Controller's implementation collates all the information for a stage to fetch key statistics from workers. + * Client side method to update the controller with partial key statistics information for a particular stage + * and worker. The controller collates all the information for a stage to fetch key statistics from workers. + * + * Only used when {@link StageDefinition#mustGatherResultKeyStatistics()}. */ void postPartialKeyStatistics( StageId stageId, @@ -86,11 +91,16 @@ public interface ControllerClient extends AutoCloseable /** * Client side method to inform the controller about the warnings generated by the given worker. */ - void postWorkerWarning( - List MSQErrorReports - ) throws IOException; + void postWorkerWarning(List MSQErrorReports) throws IOException; - List getTaskList() throws IOException; + /** + * Client side method for retrieving the list of worker IDs from the controller. These IDs can be passed to + * {@link WorkerClient} methods to communicate with other workers. Not necessary when the {@link WorkOrder} has + * {@link WorkOrder#getWorkerIds()} set. + * + * @see Controller#getWorkerIds() for the controller side + */ + List getWorkerIds() throws IOException; /** * Close this client. Idempotent. diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index d2d5cc657e6..839839db4e4 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -1171,7 +1171,7 @@ public class ControllerImpl implements Controller } @Override - public List getTaskIds() + public List getWorkerIds() { if (workerManager == null) { return Collections.emptyList(); @@ -1260,7 +1260,7 @@ public class ControllerImpl implements Controller { // Sorted copy of target worker numbers to ensure consistent iteration order. final List workersCopy = Ordering.natural().sortedCopy(workers); - final List workerIds = getTaskIds(); + final List workerIds = getWorkerIds(); final List> workerFutures = new ArrayList<>(workersCopy.size()); try { @@ -1488,7 +1488,7 @@ public class ControllerImpl implements Controller private CounterSnapshotsTree getCountersFromAllTasks() { final CounterSnapshotsTree retVal = new CounterSnapshotsTree(); - final List taskList = getTaskIds(); + final List taskList = getWorkerIds(); final List> futures = new ArrayList<>(); @@ -1508,7 +1508,7 @@ public class ControllerImpl implements Controller private void postFinishToAllTasks() { - final List taskList = getTaskIds(); + final List taskList = getWorkerIds(); final List> futures = new ArrayList<>(); @@ -2963,7 +2963,7 @@ public class ControllerImpl implements Controller } final StageId finalStageId = queryKernel.getStageId(queryDef.getFinalStageDefinition().getStageNumber()); - final List taskIds = getTaskIds(); + final List taskIds = getWorkerIds(); final InputChannelFactory inputChannelFactory; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerMemoryParameters.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerMemoryParameters.java index 8e6fc72b6aa..2ab016e10e4 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerMemoryParameters.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerMemoryParameters.java @@ -91,7 +91,8 @@ public class ControllerMemoryParameters memoryIntrospector.totalMemoryInJvm(), usableMemoryInJvm, numControllersInJvm, - memoryIntrospector.numProcessorsInJvm() + memoryIntrospector.numProcessorsInJvm(), + 0 ) ); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ListeningOutputChannelFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ListeningOutputChannelFactory.java new file mode 100644 index 00000000000..ebaad076387 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ListeningOutputChannelFactory.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.exec; + +import org.apache.druid.frame.processor.OutputChannel; +import org.apache.druid.frame.processor.OutputChannelFactory; +import org.apache.druid.frame.processor.PartitionedOutputChannel; + +import java.io.IOException; + +/** + * Decorator for {@link OutputChannelFactory} that notifies a {@link Listener} whenever a channel is opened. + */ +public class ListeningOutputChannelFactory implements OutputChannelFactory +{ + private final OutputChannelFactory delegate; + private final Listener listener; + + public ListeningOutputChannelFactory(final OutputChannelFactory delegate, final Listener listener) + { + this.delegate = delegate; + this.listener = listener; + } + + @Override + public OutputChannel openChannel(final int partitionNumber) throws IOException + { + return notifyListener(delegate.openChannel(partitionNumber)); + } + + + @Override + public OutputChannel openNilChannel(final int partitionNumber) + { + return notifyListener(delegate.openNilChannel(partitionNumber)); + } + + @Override + public PartitionedOutputChannel openPartitionedChannel( + final String name, + final boolean deleteAfterRead + ) + { + throw new UnsupportedOperationException("Listening to partitioned channels is not supported"); + } + + private OutputChannel notifyListener(OutputChannel channel) + { + listener.channelOpened(channel); + return channel; + } + + public interface Listener + { + void channelOpened(OutputChannel channel); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/OutputChannelMode.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/OutputChannelMode.java index 7e7fc3d3d6f..f42d558a76c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/OutputChannelMode.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/OutputChannelMode.java @@ -32,9 +32,12 @@ import org.apache.druid.msq.kernel.controller.ControllerQueryKernelUtils; public enum OutputChannelMode { /** - * In-memory output channels. Stage shuffle data does not hit disk. This mode requires a consumer stage to run - * at the same time as its corresponding producer stage. See {@link ControllerQueryKernelUtils#computeStageGroups} for the - * logic that determines when we can use in-memory channels. + * In-memory output channels. Stage shuffle data does not hit disk. In-memory channels do not fully buffer stage + * output. They use a blocking queue; see {@link RunWorkOrder#makeStageOutputChannelFactory()}. + * + * Because stage output is not fully buffered, this mode requires a consumer stage to run at the same time as its + * corresponding producer stage. See {@link ControllerQueryKernelUtils#computeStageGroups} for the logic that + * determines when we can use in-memory channels. */ MEMORY("memory"), diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/RunWorkOrder.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/RunWorkOrder.java new file mode 100644 index 00000000000..0173979efee --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/RunWorkOrder.java @@ -0,0 +1,1051 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.exec; + +import com.google.common.base.Function; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +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.MoreExecutors; +import com.google.common.util.concurrent.SettableFuture; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.error.DruidException; +import org.apache.druid.frame.allocation.ArenaMemoryAllocator; +import org.apache.druid.frame.allocation.ArenaMemoryAllocatorFactory; +import org.apache.druid.frame.channel.BlockingQueueFrameChannel; +import org.apache.druid.frame.channel.ByteTracker; +import org.apache.druid.frame.channel.FrameWithPartition; +import org.apache.druid.frame.key.ClusterByPartitions; +import org.apache.druid.frame.processor.BlockingQueueOutputChannelFactory; +import org.apache.druid.frame.processor.Bouncer; +import org.apache.druid.frame.processor.ComposingOutputChannelFactory; +import org.apache.druid.frame.processor.FileOutputChannelFactory; +import org.apache.druid.frame.processor.FrameChannelHashPartitioner; +import org.apache.druid.frame.processor.FrameChannelMixer; +import org.apache.druid.frame.processor.FrameProcessor; +import org.apache.druid.frame.processor.FrameProcessorExecutor; +import org.apache.druid.frame.processor.OutputChannel; +import org.apache.druid.frame.processor.OutputChannelFactory; +import org.apache.druid.frame.processor.OutputChannels; +import org.apache.druid.frame.processor.PartitionedOutputChannel; +import org.apache.druid.frame.processor.SuperSorter; +import org.apache.druid.frame.processor.SuperSorterProgressTracker; +import org.apache.druid.frame.processor.manager.ProcessorManager; +import org.apache.druid.frame.processor.manager.ProcessorManagers; +import org.apache.druid.frame.util.DurableStorageUtils; +import org.apache.druid.frame.write.FrameWriters; +import org.apache.druid.java.util.common.FileUtils; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.UOE; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.msq.counters.CounterNames; +import org.apache.druid.msq.counters.CounterTracker; +import org.apache.druid.msq.indexing.CountingOutputChannelFactory; +import org.apache.druid.msq.indexing.InputChannelFactory; +import org.apache.druid.msq.indexing.InputChannelsImpl; +import org.apache.druid.msq.indexing.processor.KeyStatisticsCollectionProcessor; +import org.apache.druid.msq.input.InputSlice; +import org.apache.druid.msq.input.InputSliceReader; +import org.apache.druid.msq.input.InputSlices; +import org.apache.druid.msq.input.MapInputSliceReader; +import org.apache.druid.msq.input.NilInputSlice; +import org.apache.druid.msq.input.NilInputSliceReader; +import org.apache.druid.msq.input.external.ExternalInputSlice; +import org.apache.druid.msq.input.external.ExternalInputSliceReader; +import org.apache.druid.msq.input.inline.InlineInputSlice; +import org.apache.druid.msq.input.inline.InlineInputSliceReader; +import org.apache.druid.msq.input.lookup.LookupInputSlice; +import org.apache.druid.msq.input.lookup.LookupInputSliceReader; +import org.apache.druid.msq.input.stage.InputChannels; +import org.apache.druid.msq.input.stage.StageInputSlice; +import org.apache.druid.msq.input.stage.StageInputSliceReader; +import org.apache.druid.msq.input.table.SegmentsInputSlice; +import org.apache.druid.msq.input.table.SegmentsInputSliceReader; +import org.apache.druid.msq.kernel.FrameContext; +import org.apache.druid.msq.kernel.FrameProcessorFactory; +import org.apache.druid.msq.kernel.ProcessorsAndChannels; +import org.apache.druid.msq.kernel.ShuffleSpec; +import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.msq.kernel.WorkOrder; +import org.apache.druid.msq.shuffle.output.DurableStorageOutputChannelFactory; +import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; +import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; +import org.apache.druid.utils.CloseableUtils; +import org.checkerframework.checker.nullness.qual.MonotonicNonNull; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; + +/** + * Main worker logic for executing a {@link WorkOrder} in a {@link FrameProcessorExecutor}. + */ +public class RunWorkOrder +{ + private final WorkOrder workOrder; + private final InputChannelFactory inputChannelFactory; + private final CounterTracker counterTracker; + private final FrameProcessorExecutor exec; + private final String cancellationId; + private final int parallelism; + private final WorkerContext workerContext; + private final FrameContext frameContext; + private final RunWorkOrderListener listener; + private final boolean reindex; + private final boolean removeNullBytes; + private final ByteTracker intermediateSuperSorterLocalStorageTracker; + private final AtomicBoolean started = new AtomicBoolean(); + + @MonotonicNonNull + private InputSliceReader inputSliceReader; + @MonotonicNonNull + private OutputChannelFactory workOutputChannelFactory; + @MonotonicNonNull + private OutputChannelFactory shuffleOutputChannelFactory; + @MonotonicNonNull + private ResultAndChannels workResultAndOutputChannels; + @MonotonicNonNull + private SettableFuture stagePartitionBoundariesFuture; + @MonotonicNonNull + private ListenableFuture stageOutputChannelsFuture; + + public RunWorkOrder( + final WorkOrder workOrder, + final InputChannelFactory inputChannelFactory, + final CounterTracker counterTracker, + final FrameProcessorExecutor exec, + final String cancellationId, + final WorkerContext workerContext, + final FrameContext frameContext, + final RunWorkOrderListener listener, + final boolean reindex, + final boolean removeNullBytes + ) + { + this.workOrder = workOrder; + this.inputChannelFactory = inputChannelFactory; + this.counterTracker = counterTracker; + this.exec = exec; + this.cancellationId = cancellationId; + this.parallelism = workerContext.threadCount(); + this.workerContext = workerContext; + this.frameContext = frameContext; + this.listener = listener; + this.reindex = reindex; + this.removeNullBytes = removeNullBytes; + this.intermediateSuperSorterLocalStorageTracker = + new ByteTracker( + frameContext.storageParameters().isIntermediateStorageLimitConfigured() + ? frameContext.storageParameters().getIntermediateSuperSorterStorageMaxLocalBytes() + : Long.MAX_VALUE + ); + } + + /** + * Start execution of the provided {@link WorkOrder} in the provided {@link FrameProcessorExecutor}. + * + * Execution proceeds asynchronously after this method returns. The {@link RunWorkOrderListener} passed to the + * constructor of this instance can be used to track progress. + */ + public void start() throws IOException + { + if (started.getAndSet(true)) { + throw new ISE("Already started"); + } + + final StageDefinition stageDef = workOrder.getStageDefinition(); + + try { + makeInputSliceReader(); + makeWorkOutputChannelFactory(); + makeShuffleOutputChannelFactory(); + makeAndRunWorkProcessors(); + + if (stageDef.doesShuffle()) { + makeAndRunShuffleProcessors(); + } else { + // No shuffling: work output _is_ stage output. Retain read-only versions to reduce memory footprint. + stageOutputChannelsFuture = + Futures.immediateFuture(workResultAndOutputChannels.getOutputChannels().readOnly()); + } + + setUpCompletionCallbacks(); + } + catch (Throwable t) { + // If start() has problems, cancel anything that was already kicked off, and close the FrameContext. + try { + exec.cancel(cancellationId); + } + catch (Throwable t2) { + t.addSuppressed(t2); + } + + CloseableUtils.closeAndSuppressExceptions(frameContext, t::addSuppressed); + throw t; + } + } + + /** + * Settable {@link ClusterByPartitions} future for global sort. Necessary because we don't know ahead of time + * what the boundaries will be. The controller decides based on statistics from all workers. Once the controller + * decides, its decision is written to this future, which allows sorting on workers to proceed. + */ + @Nullable + public SettableFuture getStagePartitionBoundariesFuture() + { + return stagePartitionBoundariesFuture; + } + + private void makeInputSliceReader() + { + if (inputSliceReader != null) { + throw new ISE("inputSliceReader already created"); + } + + final String queryId = workOrder.getQueryDefinition().getQueryId(); + + final InputChannels inputChannels = + new InputChannelsImpl( + workOrder.getQueryDefinition(), + InputSlices.allReadablePartitions(workOrder.getInputs()), + inputChannelFactory, + () -> ArenaMemoryAllocator.createOnHeap(frameContext.memoryParameters().getStandardFrameSize()), + exec, + cancellationId, + removeNullBytes + ); + + inputSliceReader = new MapInputSliceReader( + ImmutableMap., InputSliceReader>builder() + .put(NilInputSlice.class, NilInputSliceReader.INSTANCE) + .put(StageInputSlice.class, new StageInputSliceReader(queryId, inputChannels)) + .put(ExternalInputSlice.class, new ExternalInputSliceReader(frameContext.tempDir("external"))) + .put(InlineInputSlice.class, new InlineInputSliceReader(frameContext.segmentWrangler())) + .put(LookupInputSlice.class, new LookupInputSliceReader(frameContext.segmentWrangler())) + .put(SegmentsInputSlice.class, new SegmentsInputSliceReader(frameContext, reindex)) + .build() + ); + } + + private void makeWorkOutputChannelFactory() + { + if (workOutputChannelFactory != null) { + throw new ISE("processorOutputChannelFactory already created"); + } + + final OutputChannelFactory baseOutputChannelFactory; + + if (workOrder.getStageDefinition().doesShuffle()) { + // Writing to a consumer in the same JVM (which will be set up later on in this method). Use the large frame + // size if we're writing to a SuperSorter, since we'll generate fewer temp files if we use larger frames. + // Otherwise, use the standard frame size. + final int frameSize; + + if (workOrder.getStageDefinition().getShuffleSpec().kind().isSort()) { + frameSize = frameContext.memoryParameters().getLargeFrameSize(); + } else { + frameSize = frameContext.memoryParameters().getStandardFrameSize(); + } + + baseOutputChannelFactory = new BlockingQueueOutputChannelFactory(frameSize); + } else { + // Writing stage output. + baseOutputChannelFactory = makeStageOutputChannelFactory(); + } + + workOutputChannelFactory = new CountingOutputChannelFactory( + baseOutputChannelFactory, + counterTracker.channel(CounterNames.outputChannel()) + ); + } + + private void makeShuffleOutputChannelFactory() + { + shuffleOutputChannelFactory = + new CountingOutputChannelFactory( + makeStageOutputChannelFactory(), + counterTracker.channel(CounterNames.shuffleChannel()) + ); + } + + /** + * Use {@link FrameProcessorFactory#makeProcessors} to create {@link ProcessorsAndChannels}. Executes the + * processors using {@link #exec} and sets the output channels in {@link #workResultAndOutputChannels}. + * + * @param type of {@link StageDefinition#getProcessorFactory()} + * @param return type of {@link FrameProcessor} created by the manager + * @param result type of {@link ProcessorManager#result()} + * @param type of {@link WorkOrder#getExtraInfo()} + */ + private , ProcessorReturnType, ManagerReturnType, ExtraInfoType> void makeAndRunWorkProcessors() + throws IOException + { + if (workResultAndOutputChannels != null) { + throw new ISE("workResultAndOutputChannels already set"); + } + + @SuppressWarnings("unchecked") + final FactoryType processorFactory = (FactoryType) workOrder.getStageDefinition().getProcessorFactory(); + + @SuppressWarnings("unchecked") + final ProcessorsAndChannels processors = + processorFactory.makeProcessors( + workOrder.getStageDefinition(), + workOrder.getWorkerNumber(), + workOrder.getInputs(), + inputSliceReader, + (ExtraInfoType) workOrder.getExtraInfo(), + workOutputChannelFactory, + frameContext, + parallelism, + counterTracker, + listener::onWarning, + removeNullBytes + ); + + final ProcessorManager processorManager = processors.getProcessorManager(); + + final int maxOutstandingProcessors; + + if (processors.getOutputChannels().getAllChannels().isEmpty()) { + // No output channels: run up to "parallelism" processors at once. + maxOutstandingProcessors = Math.max(1, parallelism); + } else { + // If there are output channels, that acts as a ceiling on the number of processors that can run at once. + maxOutstandingProcessors = + Math.max(1, Math.min(parallelism, processors.getOutputChannels().getAllChannels().size())); + } + + final ListenableFuture workResultFuture = exec.runAllFully( + processorManager, + maxOutstandingProcessors, + frameContext.processorBouncer(), + cancellationId + ); + + workResultAndOutputChannels = new ResultAndChannels<>(workResultFuture, processors.getOutputChannels()); + } + + private void makeAndRunShuffleProcessors() + { + if (stageOutputChannelsFuture != null) { + throw new ISE("stageOutputChannelsFuture already set"); + } + + final ShuffleSpec shuffleSpec = workOrder.getStageDefinition().getShuffleSpec(); + + final ShufflePipelineBuilder shufflePipeline = new ShufflePipelineBuilder( + workOrder, + counterTracker, + exec, + cancellationId, + frameContext + ); + + shufflePipeline.initialize(workResultAndOutputChannels); + shufflePipeline.gatherResultKeyStatisticsAndReportDoneReadingInputIfNeeded(); + + switch (shuffleSpec.kind()) { + case MIX: + shufflePipeline.mix(shuffleOutputChannelFactory); + break; + + case HASH: + shufflePipeline.hashPartition(shuffleOutputChannelFactory); + break; + + case HASH_LOCAL_SORT: + final OutputChannelFactory hashOutputChannelFactory; + + if (shuffleSpec.partitionCount() == 1) { + // Single partition; no need to write temporary files. + hashOutputChannelFactory = + new BlockingQueueOutputChannelFactory(frameContext.memoryParameters().getStandardFrameSize()); + } else { + // Multi-partition; write temporary files and then sort each one file-by-file. + hashOutputChannelFactory = + new FileOutputChannelFactory( + frameContext.tempDir("hash-parts"), + frameContext.memoryParameters().getStandardFrameSize(), + null + ); + } + + shufflePipeline.hashPartition(hashOutputChannelFactory); + shufflePipeline.localSort(shuffleOutputChannelFactory); + break; + + case GLOBAL_SORT: + shufflePipeline.globalSort(shuffleOutputChannelFactory, makeGlobalSortPartitionBoundariesFuture()); + break; + + default: + throw new UOE("Cannot handle shuffle kind [%s]", shuffleSpec.kind()); + } + + stageOutputChannelsFuture = shufflePipeline.build(); + } + + private ListenableFuture makeGlobalSortPartitionBoundariesFuture() + { + if (workOrder.getStageDefinition().mustGatherResultKeyStatistics()) { + if (stagePartitionBoundariesFuture != null) { + throw new ISE("Cannot call 'makeGlobalSortPartitionBoundariesFuture' twice"); + } + + return (stagePartitionBoundariesFuture = SettableFuture.create()); + } else { + // Result key stats aren't needed, so the partition boundaries are knowable ahead of time. Compute them now. + final ClusterByPartitions boundaries = + workOrder.getStageDefinition() + .generatePartitionBoundariesForShuffle(null) + .valueOrThrow(); + + return Futures.immediateFuture(boundaries); + } + } + + private void setUpCompletionCallbacks() + { + Futures.addCallback( + Futures.allAsList( + Arrays.asList( + workResultAndOutputChannels.getResultFuture(), + stageOutputChannelsFuture + ) + ), + new FutureCallback>() + { + @Override + public void onSuccess(final List workerResultAndOutputChannelsResolved) + { + final Object resultObject = workerResultAndOutputChannelsResolved.get(0); + final OutputChannels outputChannels = (OutputChannels) workerResultAndOutputChannelsResolved.get(1); + + if (workOrder.getOutputChannelMode() != OutputChannelMode.MEMORY) { + // In non-MEMORY output channel modes, call onOutputChannelAvailable when all work is done. + // (In MEMORY mode, we would have called onOutputChannelAvailable when the channels were created.) + for (final OutputChannel channel : outputChannels.getAllChannels()) { + listener.onOutputChannelAvailable(channel); + } + } + + if (workOrder.getOutputChannelMode().isDurable()) { + // In DURABLE_STORAGE output channel mode, write a success file once all work is done. + writeDurableStorageSuccessFile(); + } + + listener.onSuccess(resultObject); + } + + @Override + public void onFailure(final Throwable t) + { + listener.onFailure(t); + } + }, + Execs.directExecutor() + ); + } + + /** + * Write {@link DurableStorageUtils#SUCCESS_MARKER_FILENAME} for a particular stage, if durable storage is enabled. + */ + private void writeDurableStorageSuccessFile() + { + final DurableStorageOutputChannelFactory durableStorageOutputChannelFactory = + makeDurableStorageOutputChannelFactory( + frameContext.tempDir("durable"), + frameContext.memoryParameters().getStandardFrameSize(), + workOrder.getOutputChannelMode() == OutputChannelMode.DURABLE_STORAGE_QUERY_RESULTS + ); + + try { + durableStorageOutputChannelFactory.createSuccessFile(workerContext.workerId()); + } + catch (IOException e) { + throw new ISE( + e, + "Unable to create success file at location[%s]", + durableStorageOutputChannelFactory.getSuccessFilePath() + ); + } + } + + private OutputChannelFactory makeStageOutputChannelFactory() + { + // Use the standard frame size, since we assume this size when computing how much is needed to merge output + // files from different workers. + final int frameSize = frameContext.memoryParameters().getStandardFrameSize(); + final OutputChannelMode outputChannelMode = workOrder.getOutputChannelMode(); + + switch (outputChannelMode) { + case MEMORY: + // Use ListeningOutputChannelFactory to capture output channels as they are created, rather than when + // work is complete. + return new ListeningOutputChannelFactory( + new BlockingQueueOutputChannelFactory(frameSize), + listener::onOutputChannelAvailable + ); + + case LOCAL_STORAGE: + final File fileChannelDirectory = + frameContext.tempDir(StringUtils.format("output_stage_%06d", workOrder.getStageNumber())); + return new FileOutputChannelFactory(fileChannelDirectory, frameSize, null); + + case DURABLE_STORAGE_INTERMEDIATE: + case DURABLE_STORAGE_QUERY_RESULTS: + return makeDurableStorageOutputChannelFactory( + frameContext.tempDir("durable"), + frameSize, + outputChannelMode == OutputChannelMode.DURABLE_STORAGE_QUERY_RESULTS + ); + + default: + throw DruidException.defensive("No handling for outputChannelMode[%s]", outputChannelMode); + } + } + + private OutputChannelFactory makeSuperSorterIntermediateOutputChannelFactory(final File tmpDir) + { + final int frameSize = frameContext.memoryParameters().getLargeFrameSize(); + final File fileChannelDirectory = + new File(tmpDir, StringUtils.format("intermediate_output_stage_%06d", workOrder.getStageNumber())); + final FileOutputChannelFactory fileOutputChannelFactory = + new FileOutputChannelFactory(fileChannelDirectory, frameSize, intermediateSuperSorterLocalStorageTracker); + + if (workOrder.getOutputChannelMode().isDurable() + && frameContext.storageParameters().isIntermediateStorageLimitConfigured()) { + final boolean isQueryResults = + workOrder.getOutputChannelMode() == OutputChannelMode.DURABLE_STORAGE_QUERY_RESULTS; + return new ComposingOutputChannelFactory( + ImmutableList.of( + fileOutputChannelFactory, + makeDurableStorageOutputChannelFactory(tmpDir, frameSize, isQueryResults) + ), + frameSize + ); + } else { + return fileOutputChannelFactory; + } + } + + private DurableStorageOutputChannelFactory makeDurableStorageOutputChannelFactory( + final File tmpDir, + final int frameSize, + final boolean isQueryResults + ) + { + return DurableStorageOutputChannelFactory.createStandardImplementation( + workOrder.getQueryDefinition().getQueryId(), + workOrder.getWorkerNumber(), + workOrder.getStageNumber(), + workerContext.workerId(), + frameSize, + MSQTasks.makeStorageConnector(workerContext.injector()), + tmpDir, + isQueryResults + ); + } + + /** + * Helper for {@link RunWorkOrder#makeAndRunShuffleProcessors()}. Builds a {@link FrameProcessor} pipeline to + * handle the shuffle. + */ + private class ShufflePipelineBuilder + { + private final WorkOrder workOrder; + private final CounterTracker counterTracker; + private final FrameProcessorExecutor exec; + private final String cancellationId; + private final FrameContext frameContext; + + // Current state of the pipeline. It's a future to allow pipeline construction to be deferred if necessary. + private ListenableFuture> pipelineFuture; + + public ShufflePipelineBuilder( + final WorkOrder workOrder, + final CounterTracker counterTracker, + final FrameProcessorExecutor exec, + final String cancellationId, + final FrameContext frameContext + ) + { + this.workOrder = workOrder; + this.counterTracker = counterTracker; + this.exec = exec; + this.cancellationId = cancellationId; + this.frameContext = frameContext; + } + + /** + * Start the pipeline with the outputs of the main processor. + */ + public void initialize(final ResultAndChannels resultAndChannels) + { + if (pipelineFuture != null) { + throw new ISE("already initialized"); + } + + pipelineFuture = Futures.immediateFuture(resultAndChannels); + } + + /** + * Add {@link FrameChannelMixer}, which mixes all current outputs into a single channel from the provided factory. + */ + public void mix(final OutputChannelFactory outputChannelFactory) + { + // No sorting or statistics gathering, just combining all outputs into one big partition. Use a mixer to get + // everything into one file. Note: even if there is only one output channel, we'll run it through the mixer + // anyway, to ensure the data gets written to a file. (httpGetChannelData requires files.) + + push( + resultAndChannels -> { + final OutputChannel outputChannel = outputChannelFactory.openChannel(0); + + final FrameChannelMixer mixer = + new FrameChannelMixer( + resultAndChannels.getOutputChannels().getAllReadableChannels(), + outputChannel.getWritableChannel() + ); + + return new ResultAndChannels<>( + exec.runFully(mixer, cancellationId), + OutputChannels.wrap(Collections.singletonList(outputChannel.readOnly())) + ); + } + ); + } + + /** + * Add {@link KeyStatisticsCollectionProcessor} if {@link StageDefinition#mustGatherResultKeyStatistics()}. + * + * Calls {@link RunWorkOrderListener#onDoneReadingInput(ClusterByStatisticsSnapshot)} when statistics are gathered. + * If statistics were not needed, calls the listener immediately. + */ + public void gatherResultKeyStatisticsAndReportDoneReadingInputIfNeeded() + { + push( + resultAndChannels -> { + final StageDefinition stageDefinition = workOrder.getStageDefinition(); + final OutputChannels channels = resultAndChannels.getOutputChannels(); + + if (channels.getAllChannels().isEmpty()) { + // No data coming out of this stage. Report empty statistics, if the kernel is expecting statistics. + if (stageDefinition.mustGatherResultKeyStatistics()) { + listener.onDoneReadingInput(ClusterByStatisticsSnapshot.empty()); + } else { + listener.onDoneReadingInput(null); + } + + // Generate one empty channel so the next part of the pipeline has something to do. + final BlockingQueueFrameChannel channel = BlockingQueueFrameChannel.minimal(); + channel.writable().close(); + + final OutputChannel outputChannel = OutputChannel.readOnly( + channel.readable(), + FrameWithPartition.NO_PARTITION + ); + + return new ResultAndChannels<>( + Futures.immediateFuture(null), + OutputChannels.wrap(Collections.singletonList(outputChannel)) + ); + } else if (stageDefinition.mustGatherResultKeyStatistics()) { + return gatherResultKeyStatistics(channels); + } else { + // Report "done reading input" when the input future resolves. + // No need to add any processors to the pipeline. + resultAndChannels.resultFuture.addListener( + () -> listener.onDoneReadingInput(null), + Execs.directExecutor() + ); + return resultAndChannels; + } + } + ); + } + + /** + * Add a {@link SuperSorter} using {@link StageDefinition#getSortKey()} and partition boundaries + * from {@code partitionBoundariesFuture}. + */ + public void globalSort( + final OutputChannelFactory outputChannelFactory, + final ListenableFuture partitionBoundariesFuture + ) + { + pushAsync( + resultAndChannels -> { + final StageDefinition stageDefinition = workOrder.getStageDefinition(); + + final File sorterTmpDir = frameContext.tempDir("super-sort"); + FileUtils.mkdirp(sorterTmpDir); + if (!sorterTmpDir.isDirectory()) { + throw new IOException("Cannot create directory: " + sorterTmpDir); + } + + final WorkerMemoryParameters memoryParameters = frameContext.memoryParameters(); + final SuperSorter sorter = new SuperSorter( + resultAndChannels.getOutputChannels().getAllReadableChannels(), + stageDefinition.getFrameReader(), + stageDefinition.getSortKey(), + partitionBoundariesFuture, + exec, + outputChannelFactory, + makeSuperSorterIntermediateOutputChannelFactory(sorterTmpDir), + memoryParameters.getSuperSorterMaxActiveProcessors(), + memoryParameters.getSuperSorterMaxChannelsPerProcessor(), + -1, + cancellationId, + counterTracker.sortProgress(), + removeNullBytes + ); + + return FutureUtils.transform( + sorter.run(), + sortedChannels -> new ResultAndChannels<>(Futures.immediateFuture(null), sortedChannels) + ); + } + ); + } + + /** + * Add a {@link FrameChannelHashPartitioner} using {@link StageDefinition#getSortKey()}. + */ + public void hashPartition(final OutputChannelFactory outputChannelFactory) + { + pushAsync( + resultAndChannels -> { + final ShuffleSpec shuffleSpec = workOrder.getStageDefinition().getShuffleSpec(); + final int partitions = shuffleSpec.partitionCount(); + + final List outputChannels = new ArrayList<>(); + + for (int i = 0; i < partitions; i++) { + outputChannels.add(outputChannelFactory.openChannel(i)); + } + + final FrameChannelHashPartitioner partitioner = new FrameChannelHashPartitioner( + resultAndChannels.getOutputChannels().getAllReadableChannels(), + outputChannels.stream().map(OutputChannel::getWritableChannel).collect(Collectors.toList()), + workOrder.getStageDefinition().getFrameReader(), + workOrder.getStageDefinition().getClusterBy().getColumns().size(), + FrameWriters.makeRowBasedFrameWriterFactory( + new ArenaMemoryAllocatorFactory(frameContext.memoryParameters().getStandardFrameSize()), + workOrder.getStageDefinition().getSignature(), + workOrder.getStageDefinition().getSortKey(), + removeNullBytes + ) + ); + + final ListenableFuture partitionerFuture = exec.runFully(partitioner, cancellationId); + + final ResultAndChannels retVal = + new ResultAndChannels<>(partitionerFuture, OutputChannels.wrap(outputChannels)); + + if (retVal.getOutputChannels().areReadableChannelsReady()) { + return Futures.immediateFuture(retVal); + } else { + return FutureUtils.transform(partitionerFuture, ignored -> retVal); + } + } + ); + } + + /** + * Add a sequence of {@link SuperSorter}, operating on each current output channel in order, one at a time. + */ + public void localSort(final OutputChannelFactory outputChannelFactory) + { + pushAsync( + resultAndChannels -> { + final StageDefinition stageDefinition = workOrder.getStageDefinition(); + final OutputChannels channels = resultAndChannels.getOutputChannels(); + final List> sortedChannelFutures = new ArrayList<>(); + + ListenableFuture nextFuture = Futures.immediateFuture(null); + + for (final OutputChannel channel : channels.getAllChannels()) { + final File sorterTmpDir = frameContext.tempDir( + StringUtils.format("hash-parts-super-sort-%06d", channel.getPartitionNumber()) + ); + + FileUtils.mkdirp(sorterTmpDir); + + // SuperSorter will try to write to output partition zero; we remap it to the correct partition number. + final OutputChannelFactory partitionOverrideOutputChannelFactory = new OutputChannelFactory() + { + @Override + public OutputChannel openChannel(int expectedZero) throws IOException + { + if (expectedZero != 0) { + throw new ISE("Unexpected part [%s]", expectedZero); + } + + return outputChannelFactory.openChannel(channel.getPartitionNumber()); + } + + @Override + public PartitionedOutputChannel openPartitionedChannel(String name, boolean deleteAfterRead) + { + throw new UnsupportedOperationException(); + } + + @Override + public OutputChannel openNilChannel(int expectedZero) + { + if (expectedZero != 0) { + throw new ISE("Unexpected part [%s]", expectedZero); + } + + return outputChannelFactory.openNilChannel(channel.getPartitionNumber()); + } + }; + + // Chain futures so we only sort one partition at a time. + nextFuture = Futures.transformAsync( + nextFuture, + ignored -> { + final SuperSorter sorter = new SuperSorter( + Collections.singletonList(channel.getReadableChannel()), + stageDefinition.getFrameReader(), + stageDefinition.getSortKey(), + Futures.immediateFuture(ClusterByPartitions.oneUniversalPartition()), + exec, + partitionOverrideOutputChannelFactory, + makeSuperSorterIntermediateOutputChannelFactory(sorterTmpDir), + 1, + 2, + -1, + cancellationId, + + // Tracker is not actually tracked, since it doesn't quite fit into the way we report counters. + // There's a single SuperSorterProgressTrackerCounter per worker, but workers that do local + // sorting have a SuperSorter per partition. + new SuperSorterProgressTracker(), + removeNullBytes + ); + + return FutureUtils.transform(sorter.run(), r -> Iterables.getOnlyElement(r.getAllChannels())); + }, + MoreExecutors.directExecutor() + ); + + sortedChannelFutures.add(nextFuture); + } + + return FutureUtils.transform( + Futures.allAsList(sortedChannelFutures), + sortedChannels -> new ResultAndChannels<>( + Futures.immediateFuture(null), + OutputChannels.wrap(sortedChannels) + ) + ); + } + ); + } + + /** + * Return the (future) output channels for this pipeline. + */ + public ListenableFuture build() + { + if (pipelineFuture == null) { + throw new ISE("Not initialized"); + } + + return Futures.transformAsync( + pipelineFuture, + resultAndChannels -> + Futures.transform( + resultAndChannels.getResultFuture(), + (Function) input -> { + sanityCheckOutputChannels(resultAndChannels.getOutputChannels()); + return resultAndChannels.getOutputChannels(); + }, + Execs.directExecutor() + ), + Execs.directExecutor() + ); + } + + /** + * Adds {@link KeyStatisticsCollectionProcessor}. Called by {@link #gatherResultKeyStatisticsAndReportDoneReadingInputIfNeeded()}. + */ + private ResultAndChannels gatherResultKeyStatistics(final OutputChannels channels) + { + final StageDefinition stageDefinition = workOrder.getStageDefinition(); + final List retVal = new ArrayList<>(); + final List processors = new ArrayList<>(); + + for (final OutputChannel outputChannel : channels.getAllChannels()) { + final BlockingQueueFrameChannel channel = BlockingQueueFrameChannel.minimal(); + retVal.add(OutputChannel.readOnly(channel.readable(), outputChannel.getPartitionNumber())); + + processors.add( + new KeyStatisticsCollectionProcessor( + outputChannel.getReadableChannel(), + channel.writable(), + stageDefinition.getFrameReader(), + stageDefinition.getClusterBy(), + stageDefinition.createResultKeyStatisticsCollector( + frameContext.memoryParameters().getPartitionStatisticsMaxRetainedBytes() + ) + ) + ); + } + + final ListenableFuture clusterByStatisticsCollectorFuture = + exec.runAllFully( + ProcessorManagers.of(processors) + .withAccumulation( + stageDefinition.createResultKeyStatisticsCollector( + frameContext.memoryParameters().getPartitionStatisticsMaxRetainedBytes() + ), + ClusterByStatisticsCollector::addAll + ), + // Run all processors simultaneously. They are lightweight and this keeps things moving. + processors.size(), + Bouncer.unlimited(), + cancellationId + ); + + Futures.addCallback( + clusterByStatisticsCollectorFuture, + new FutureCallback() + { + @Override + public void onSuccess(final ClusterByStatisticsCollector result) + { + listener.onDoneReadingInput(result.snapshot()); + } + + @Override + public void onFailure(Throwable t) + { + listener.onFailure( + new ISE(t, "Failed to gather clusterBy statistics for stage[%s]", stageDefinition.getId()) + ); + } + }, + Execs.directExecutor() + ); + + return new ResultAndChannels<>( + clusterByStatisticsCollectorFuture, + OutputChannels.wrap(retVal) + ); + } + + /** + * Update the {@link #pipelineFuture}. + */ + private void push(final ExceptionalFunction, ResultAndChannels> fn) + { + pushAsync( + channels -> + Futures.immediateFuture(fn.apply(channels)) + ); + } + + /** + * Update the {@link #pipelineFuture} asynchronously. + */ + private void pushAsync(final ExceptionalFunction, ListenableFuture>> fn) + { + if (pipelineFuture == null) { + throw new ISE("Not initialized"); + } + + pipelineFuture = FutureUtils.transform( + Futures.transformAsync( + pipelineFuture, + fn::apply, + Execs.directExecutor() + ), + resultAndChannels -> new ResultAndChannels<>( + resultAndChannels.getResultFuture(), + resultAndChannels.getOutputChannels().readOnly() + ) + ); + } + + /** + * Verifies there is exactly one channel per partition. + */ + private void sanityCheckOutputChannels(final OutputChannels outputChannels) + { + for (int partitionNumber : outputChannels.getPartitionNumbers()) { + final List outputChannelsForPartition = + outputChannels.getChannelsForPartition(partitionNumber); + + Preconditions.checkState(partitionNumber >= 0, "Expected partitionNumber >= 0, but got [%s]", partitionNumber); + Preconditions.checkState( + outputChannelsForPartition.size() == 1, + "Expected one channel for partition [%s], but got [%s]", + partitionNumber, + outputChannelsForPartition.size() + ); + } + } + } + + private static class ResultAndChannels + { + private final ListenableFuture resultFuture; + private final OutputChannels outputChannels; + + public ResultAndChannels( + ListenableFuture resultFuture, + OutputChannels outputChannels + ) + { + this.resultFuture = resultFuture; + this.outputChannels = outputChannels; + } + + public ListenableFuture getResultFuture() + { + return resultFuture; + } + + public OutputChannels getOutputChannels() + { + return outputChannels; + } + } + + private interface ExceptionalFunction + { + R apply(T t) throws Exception; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/RunWorkOrderListener.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/RunWorkOrderListener.java new file mode 100644 index 00000000000..19c3c6570fe --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/RunWorkOrderListener.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.exec; + +import org.apache.druid.frame.processor.OutputChannel; +import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; + +import javax.annotation.Nullable; + +/** + * Listener for various things that may happen during execution of {@link RunWorkOrder#start()}. Listener methods are + * fired in processing threads, so they must be thread-safe, and it is important that they run quickly. + */ +public interface RunWorkOrderListener +{ + /** + * Called when done reading input. If key statistics were gathered, they are provided. + */ + void onDoneReadingInput(@Nullable ClusterByStatisticsSnapshot snapshot); + + /** + * Called when an output channel becomes available for reading by downstream stages. + */ + void onOutputChannelAvailable(OutputChannel outputChannel); + + /** + * Called when the work order has succeeded. + */ + void onSuccess(Object resultObject); + + /** + * Called when a non-fatal exception is encountered. Work continues after this listener fires. + */ + void onWarning(Throwable t); + + /** + * Called when the work order has failed. + */ + void onFailure(Throwable t); +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java index cc5f0fae173..a90068060d8 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java @@ -19,40 +19,44 @@ package org.apache.druid.msq.exec; +import com.google.common.util.concurrent.ListenableFuture; 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 org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; -import javax.annotation.Nullable; -import java.io.IOException; import java.io.InputStream; +/** + * Interface for a multi-stage query (MSQ) worker. Workers are long-lived and are able to run multiple {@link WorkOrder} + * prior to exiting. + * + * @see WorkerImpl the production implementation + */ public interface Worker { /** - * Unique ID for this worker. + * Identifier for this worker. Same as {@link WorkerContext#workerId()}. */ String id(); /** - * The task which this worker runs. + * Runs the worker in the current thread. Surrounding classes provide the execution thread. */ - MSQWorkerTask task(); + void run(); /** - * Runs the worker in the current thread. Surrounding classes provide - * the execution thread. + * Terminate the worker upon a cancellation request. Causes a concurrently-running {@link #run()} method in + * a separate thread to cancel all outstanding work and exit. Does not block. Use {@link #awaitStop()} if you + * would like to wait for {@link #run()} to finish. */ - TaskStatus run() throws Exception; + void stop(); /** - * Terminate the worker upon a cancellation request. + * Wait for {@link #run()} to finish. */ - void stopGracefully(); + void awaitStop(); /** * Report that the controller has failed. The worker must cease work immediately. Cleanup then exit. @@ -63,20 +67,20 @@ public interface Worker // 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 + * Called when the worker receives a new work order. Accepts the work order and schedules it for execution. */ void postWorkOrder(WorkOrder workOrder); /** * Returns the statistics snapshot for the given stageId. This is called from {@link WorkerSketchFetcher} under - * PARALLEL OR AUTO modes. + * {@link ClusterStatisticsMergeMode#PARALLEL} OR {@link ClusterStatisticsMergeMode#AUTO} modes. */ ClusterByStatisticsSnapshot fetchStatisticsSnapshot(StageId stageId); /** * Returns the statistics snapshot for the given stageId which contains only the sketch for the specified timeChunk. - * This is called from {@link WorkerSketchFetcher} under SEQUENTIAL OR AUTO modes. + * This is called from {@link WorkerSketchFetcher} under {@link ClusterStatisticsMergeMode#SEQUENTIAL} or + * {@link ClusterStatisticsMergeMode#AUTO} modes. */ ClusterByStatisticsSnapshot fetchStatisticsSnapshotForTimeChunk(StageId stageId, long timeChunk); @@ -84,26 +88,30 @@ public interface Worker * 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 - ); + boolean postResultPartitionBoundaries(StageId stageId, ClusterByPartitions stagePartitionBoundaries); /** * 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 + * during retry in case of a connection error. * - * Returns a null if the workerOutput for a particular queryId, stageNumber, and partitionNumber is not found. + * The returned future resolves when at least one byte of data is available, or when the channel is finished. + * If the channel is finished, an empty {@link InputStream} is returned. * - * @throws IOException when the worker output is found but there is an error while reading it. + * With {@link OutputChannelMode#MEMORY}, once this method is called with a certain offset, workers are free to + * delete data prior to that offset. (Already-requested offsets will not be re-requested, because + * {@link OutputChannelMode#MEMORY} requires a single reader.) In this mode, if an already-requested offset is + * re-requested for some reason, an error future is returned. + * + * The returned future resolves to null if stage output for a particular queryId, stageNumber, and + * partitionNumber is not found. + * + * Throws an exception when 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; + ListenableFuture readStageOutput(StageId stageId, int partitionNumber, long offset); /** - * Returns the snapshot of the worker counters + * Returns a snapshot of counters. */ CounterSnapshotsTree getCounters(); @@ -115,7 +123,7 @@ public interface Worker void postCleanupStage(StageId stageId); /** - * Called when the work required for the query has been finished + * Called when the worker is no longer needed, and should shut down. */ void postFinish(); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerContext.java index f5e86039c23..666115d774c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerContext.java @@ -21,11 +21,12 @@ 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.StringUtils; import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.msq.indexing.MSQWorkerTask; import org.apache.druid.msq.kernel.FrameContext; +import org.apache.druid.msq.kernel.FrameProcessorFactory; import org.apache.druid.msq.kernel.QueryDefinition; +import org.apache.druid.msq.kernel.WorkOrder; import org.apache.druid.server.DruidNode; import java.io.File; @@ -33,10 +34,21 @@ import java.io.File; /** * Context used by multi-stage query workers. * - * Useful because it allows test fixtures to provide their own implementations. + * Each context is scoped to a {@link Worker} and is shared across all {@link WorkOrder} run by that worker. */ public interface WorkerContext { + /** + * Query ID for this context. + */ + String queryId(); + + /** + * Identifier for this worker that enables the controller, and other workers, to find it. For tasks this is the + * task ID from {@link MSQWorkerTask#getId()}. For persistent servers, this is the server URI. + */ + String workerId(); + ObjectMapper jsonMapper(); // Using an Injector directly because tasks do not have a way to provide their own Guice modules. @@ -49,9 +61,15 @@ public interface WorkerContext void registerWorker(Worker worker, Closer closer); /** - * Creates and fetches the controller client for the provided controller ID. + * Maximum number of {@link WorkOrder} that a {@link Worker} with this context will be asked to execute + * simultaneously. */ - ControllerClient makeControllerClient(String controllerId); + int maxConcurrentStages(); + + /** + * Creates a controller client. + */ + ControllerClient makeControllerClient(); /** * Creates and fetches a {@link WorkerClient}. It is independent of the workerId because the workerId is passed @@ -60,24 +78,24 @@ public interface WorkerContext WorkerClient makeWorkerClient(); /** - * Fetch a directory for temporary outputs + * Directory for temporary outputs. */ File tempDir(); - FrameContext frameContext(QueryDefinition queryDef, int stageNumber); + /** + * Create a context with useful objects required by {@link FrameProcessorFactory#makeProcessors}. + */ + FrameContext frameContext(QueryDefinition queryDef, int stageNumber, OutputChannelMode outputChannelMode); + /** + * Number of available processing threads. + */ int threadCount(); /** - * Fetch node info about self + * Fetch node info about self. */ DruidNode selfNode(); - Bouncer processorBouncer(); DataServerQueryHandlerFactory dataServerQueryHandlerFactory(); - - default File tempDir(int stageNumber, String id) - { - return new File(StringUtils.format("%s/stage_%02d/%s", tempDir(), stageNumber, id)); - } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java index 61939d82373..7d2964eb2f8 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java @@ -19,111 +19,58 @@ package org.apache.druid.msq.exec; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; -import com.google.common.base.Preconditions; import com.google.common.base.Suppliers; +import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; -import com.google.common.util.concurrent.AsyncFunction; -import com.google.common.util.concurrent.FutureCallback; -import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; -import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.SettableFuture; -import it.unimi.dsi.fastutil.bytes.ByteArrays; +import it.unimi.dsi.fastutil.ints.Int2ObjectMap; +import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap; +import it.unimi.dsi.fastutil.ints.IntObjectPair; import org.apache.druid.common.guava.FutureUtils; -import org.apache.druid.frame.allocation.ArenaMemoryAllocator; -import org.apache.druid.frame.allocation.ArenaMemoryAllocatorFactory; -import org.apache.druid.frame.channel.BlockingQueueFrameChannel; -import org.apache.druid.frame.channel.ByteTracker; -import org.apache.druid.frame.channel.FrameWithPartition; -import org.apache.druid.frame.channel.ReadableFileFrameChannel; +import org.apache.druid.error.DruidException; import org.apache.druid.frame.channel.ReadableFrameChannel; -import org.apache.druid.frame.channel.ReadableNilFrameChannel; -import org.apache.druid.frame.file.FrameFile; -import org.apache.druid.frame.file.FrameFileWriter; import org.apache.druid.frame.key.ClusterByPartitions; -import org.apache.druid.frame.processor.BlockingQueueOutputChannelFactory; -import org.apache.druid.frame.processor.Bouncer; -import org.apache.druid.frame.processor.ComposingOutputChannelFactory; -import org.apache.druid.frame.processor.FileOutputChannelFactory; -import org.apache.druid.frame.processor.FrameChannelHashPartitioner; -import org.apache.druid.frame.processor.FrameChannelMixer; -import org.apache.druid.frame.processor.FrameProcessor; import org.apache.druid.frame.processor.FrameProcessorExecutor; import org.apache.druid.frame.processor.OutputChannel; -import org.apache.druid.frame.processor.OutputChannelFactory; -import org.apache.druid.frame.processor.OutputChannels; -import org.apache.druid.frame.processor.PartitionedOutputChannel; -import org.apache.druid.frame.processor.SuperSorter; -import org.apache.druid.frame.processor.SuperSorterProgressTracker; -import org.apache.druid.frame.processor.manager.ProcessorManager; -import org.apache.druid.frame.processor.manager.ProcessorManagers; import org.apache.druid.frame.util.DurableStorageUtils; -import org.apache.druid.frame.write.FrameWriters; -import org.apache.druid.indexer.TaskStatus; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.msq.counters.CounterNames; import org.apache.druid.msq.counters.CounterSnapshotsTree; import org.apache.druid.msq.counters.CounterTracker; -import org.apache.druid.msq.indexing.CountingOutputChannelFactory; import org.apache.druid.msq.indexing.InputChannelFactory; -import org.apache.druid.msq.indexing.InputChannelsImpl; import org.apache.druid.msq.indexing.MSQWorkerTask; import org.apache.druid.msq.indexing.destination.MSQSelectDestination; import org.apache.druid.msq.indexing.error.CanceledFault; import org.apache.druid.msq.indexing.error.CannotParseExternalDataFault; import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.apache.druid.msq.indexing.error.MSQException; -import org.apache.druid.msq.indexing.error.MSQFaultUtils; import org.apache.druid.msq.indexing.error.MSQWarningReportLimiterPublisher; import org.apache.druid.msq.indexing.error.MSQWarningReportPublisher; import org.apache.druid.msq.indexing.error.MSQWarningReportSimplePublisher; import org.apache.druid.msq.indexing.error.MSQWarnings; -import org.apache.druid.msq.indexing.processor.KeyStatisticsCollectionProcessor; -import org.apache.druid.msq.input.InputSlice; -import org.apache.druid.msq.input.InputSliceReader; import org.apache.druid.msq.input.InputSlices; -import org.apache.druid.msq.input.MapInputSliceReader; -import org.apache.druid.msq.input.NilInputSlice; -import org.apache.druid.msq.input.NilInputSliceReader; -import org.apache.druid.msq.input.external.ExternalInputSlice; -import org.apache.druid.msq.input.external.ExternalInputSliceReader; -import org.apache.druid.msq.input.inline.InlineInputSlice; -import org.apache.druid.msq.input.inline.InlineInputSliceReader; -import org.apache.druid.msq.input.lookup.LookupInputSlice; -import org.apache.druid.msq.input.lookup.LookupInputSliceReader; -import org.apache.druid.msq.input.stage.InputChannels; import org.apache.druid.msq.input.stage.ReadablePartition; -import org.apache.druid.msq.input.stage.StageInputSlice; -import org.apache.druid.msq.input.stage.StageInputSliceReader; -import org.apache.druid.msq.input.table.SegmentsInputSlice; -import org.apache.druid.msq.input.table.SegmentsInputSliceReader; import org.apache.druid.msq.kernel.FrameContext; -import org.apache.druid.msq.kernel.FrameProcessorFactory; -import org.apache.druid.msq.kernel.ProcessorsAndChannels; -import org.apache.druid.msq.kernel.ShuffleSpec; import org.apache.druid.msq.kernel.StageDefinition; import org.apache.druid.msq.kernel.StageId; -import org.apache.druid.msq.kernel.StagePartition; import org.apache.druid.msq.kernel.WorkOrder; +import org.apache.druid.msq.kernel.controller.ControllerQueryKernelUtils; import org.apache.druid.msq.kernel.worker.WorkerStageKernel; import org.apache.druid.msq.kernel.worker.WorkerStagePhase; import org.apache.druid.msq.shuffle.input.DurableStorageInputChannelFactory; +import org.apache.druid.msq.shuffle.input.MetaInputChannelFactory; import org.apache.druid.msq.shuffle.input.WorkerInputChannelFactory; -import org.apache.druid.msq.shuffle.output.DurableStorageOutputChannelFactory; -import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; +import org.apache.druid.msq.shuffle.input.WorkerOrLocalInputChannelFactory; +import org.apache.druid.msq.shuffle.output.StageOutputHolder; import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation; import org.apache.druid.msq.util.DecoratedExecutorService; @@ -132,23 +79,14 @@ import org.apache.druid.query.PrioritizedCallable; import org.apache.druid.query.PrioritizedRunnable; import org.apache.druid.query.QueryContext; import org.apache.druid.query.QueryProcessingPool; -import org.apache.druid.rpc.ServiceClosedException; import org.apache.druid.server.DruidNode; import javax.annotation.Nullable; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.File; +import java.io.Closeable; import java.io.IOException; import java.io.InputStream; -import java.io.RandomAccessFile; -import java.nio.channels.Channels; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; import java.util.Comparator; -import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; @@ -156,9 +94,9 @@ import java.util.Set; import java.util.concurrent.BlockingQueue; import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Consumer; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -166,105 +104,93 @@ import java.util.stream.StreamSupport; /** * Interface for a worker of a multi-stage query. + * + * Not scoped to any particular query. There is one of these per {@link MSQWorkerTask}, and one per server for + * long-lived workers. */ public class WorkerImpl implements Worker { private static final Logger log = new Logger(WorkerImpl.class); + /** + * Task object, if this {@link WorkerImpl} was launched from a task. Ideally, this would not be needed, and we + * would be able to get everything we need from {@link WorkerContext}. + */ + @Nullable private final MSQWorkerTask task; private final WorkerContext context; private final DruidNode selfDruidNode; - private final Bouncer processorBouncer; - private final BlockingQueue> kernelManipulationQueue = new LinkedBlockingDeque<>(); - private final ConcurrentHashMap> stageOutputs = new ConcurrentHashMap<>(); - private final ConcurrentHashMap stageCounters = new ConcurrentHashMap<>(); - private final ConcurrentHashMap stageKernelMap = new ConcurrentHashMap<>(); - private final ByteTracker intermediateSuperSorterLocalStorageTracker; - private final boolean durableStageStorageEnabled; - private final WorkerStorageParameters workerStorageParameters; - private final boolean isRemoveNullBytes; + private final BlockingQueue> kernelManipulationQueue = new LinkedBlockingDeque<>(); + private final ConcurrentHashMap> stageOutputs = new ConcurrentHashMap<>(); /** - * Only set for select jobs. + * Pair of {workerNumber, stageId} -> counters. */ - @Nullable - private final MSQSelectDestination selectDestination; + private final ConcurrentHashMap, CounterTracker> stageCounters = new ConcurrentHashMap<>(); /** - * Set once in {@link #runTask} and never reassigned. + * Atomic that is set to true when {@link #run()} starts (or when {@link #stop()} is called before {@link #run()}). + */ + private final AtomicBoolean didRun = new AtomicBoolean(); + + /** + * Future that resolves when {@link #run()} completes. + */ + private final SettableFuture runFuture = SettableFuture.create(); + + /** + * Set once in {@link #run} and never reassigned. This is in a field so {@link #doCancel()} can close it. */ private volatile ControllerClient controllerClient; /** - * Set once in {@link #runTask} and never reassigned. Used by processing threads so we can contact other workers + * Set once in {@link #runInternal} and never reassigned. Used by processing threads so we can contact other workers * during a shuffle. */ private volatile WorkerClient workerClient; /** - * Set to false by {@link #controllerFailed()} as a way of enticing the {@link #runTask} method to exit promptly. + * Set to false by {@link #controllerFailed()} as a way of enticing the {@link #runInternal} method to exit promptly. */ private volatile boolean controllerAlive = true; - public WorkerImpl(MSQWorkerTask task, WorkerContext context) - { - this( - task, - context, - WorkerStorageParameters.createProductionInstance( - context.injector(), - MultiStageQueryContext.isDurableStorageEnabled(QueryContext.of(task.getContext())) - // If Durable Storage is enabled, then super sorter intermediate storage can be enabled. - ) - ); - } - - @VisibleForTesting - public WorkerImpl(MSQWorkerTask task, WorkerContext context, WorkerStorageParameters workerStorageParameters) + public WorkerImpl(@Nullable final MSQWorkerTask task, final WorkerContext context) { this.task = task; this.context = context; this.selfDruidNode = context.selfNode(); - this.processorBouncer = context.processorBouncer(); - QueryContext queryContext = QueryContext.of(task.getContext()); - this.durableStageStorageEnabled = MultiStageQueryContext.isDurableStorageEnabled(queryContext); - this.selectDestination = MultiStageQueryContext.getSelectDestinationOrNull(queryContext); - this.isRemoveNullBytes = MultiStageQueryContext.removeNullBytes(queryContext); - this.workerStorageParameters = workerStorageParameters; - - long maxBytes = workerStorageParameters.isIntermediateStorageLimitConfigured() - ? workerStorageParameters.getIntermediateSuperSorterStorageMaxLocalBytes() - : Long.MAX_VALUE; - this.intermediateSuperSorterLocalStorageTracker = new ByteTracker(maxBytes); } @Override public String id() { - return task.getId(); + return context.workerId(); } @Override - public MSQWorkerTask task() + public void run() { - return task; - } + if (!didRun.compareAndSet(false, true)) { + throw new ISE("already run"); + } - @Override - public TaskStatus run() throws Exception - { try (final Closer closer = Closer.create()) { + final KernelHolders kernelHolders = KernelHolders.create(context, closer); + controllerClient = kernelHolders.getControllerClient(); + + Throwable t = null; Optional maybeErrorReport; try { - maybeErrorReport = runTask(closer); + maybeErrorReport = runInternal(kernelHolders, closer); } catch (Throwable e) { + t = e; maybeErrorReport = Optional.of( MSQErrorReport.fromException( - id(), - MSQTasks.getHostFromSelfNode(selfDruidNode), + context.workerId(), + MSQTasks.getHostFromSelfNode(context.selfNode()), null, e ) @@ -273,78 +199,529 @@ public class WorkerImpl implements Worker if (maybeErrorReport.isPresent()) { final MSQErrorReport errorReport = maybeErrorReport.get(); - final String errorLogMessage = MSQTasks.errorReportToLogMessage(errorReport); - log.warn(errorLogMessage); + final String logMessage = MSQTasks.errorReportToLogMessage(errorReport); + log.warn("%s", logMessage); - closer.register(() -> { - if (controllerAlive && controllerClient != null && selfDruidNode != null) { - controllerClient.postWorkerError(id(), errorReport); - } - }); + if (controllerAlive) { + controllerClient.postWorkerError(context.workerId(), errorReport); + } - return TaskStatus.failure(id(), MSQFaultUtils.generateMessageWithErrorCode(errorReport.getFault())); - } else { - return TaskStatus.success(id()); + if (t != null) { + Throwables.throwIfInstanceOf(t, MSQException.class); + throw new MSQException(t, maybeErrorReport.get().getFault()); + } else { + throw new MSQException(maybeErrorReport.get().getFault()); + } } } + catch (IOException e) { + throw new RuntimeException(e); + } + finally { + runFuture.set(null); + } } /** * Runs worker logic. Returns an empty Optional on success. On failure, returns an error report for errors that * happened in other threads; throws exceptions for errors that happened in the main worker loop. */ - public Optional runTask(final Closer closer) throws Exception + private Optional runInternal(final KernelHolders kernelHolders, final Closer workerCloser) + throws Exception { - this.controllerClient = context.makeControllerClient(task.getControllerTaskId()); - closer.register(controllerClient::close); - closer.register(context.dataServerQueryHandlerFactory()); - context.registerWorker(this, closer); // Uses controllerClient, so must be called after that is initialized - - this.workerClient = new ExceptionWrappingWorkerClient(context.makeWorkerClient()); - closer.register(workerClient::close); - - final KernelHolder kernelHolder = new KernelHolder(); - final String cancellationId = id(); - + context.registerWorker(this, workerCloser); + workerCloser.register(context.dataServerQueryHandlerFactory()); + this.workerClient = workerCloser.register(new ExceptionWrappingWorkerClient(context.makeWorkerClient())); final FrameProcessorExecutor workerExec = new FrameProcessorExecutor(makeProcessingPool()); - // Delete all the stage outputs - closer.register(() -> { - for (final StageId stageId : stageOutputs.keySet()) { - cleanStageOutput(stageId, false); - } - }); + final long maxAllowedParseExceptions; - // Close stage output processors and running futures (if present) - closer.register(() -> { - try { - workerExec.cancel(cancellationId); - } - catch (InterruptedException e) { - // Strange that cancellation would itself be interrupted. Throw an exception, since this is unexpected. - throw new RuntimeException(e); - } - }); + if (task != null) { + maxAllowedParseExceptions = + Long.parseLong(task.getContext() + .getOrDefault(MSQWarnings.CTX_MAX_PARSE_EXCEPTIONS_ALLOWED, Long.MAX_VALUE) + .toString()); + } else { + maxAllowedParseExceptions = 0; + } - long maxAllowedParseExceptions = Long.parseLong(task.getContext().getOrDefault( - MSQWarnings.CTX_MAX_PARSE_EXCEPTIONS_ALLOWED, - Long.MAX_VALUE - ).toString()); - - long maxVerboseParseExceptions; + final long maxVerboseParseExceptions; if (maxAllowedParseExceptions == -1L) { maxVerboseParseExceptions = Limits.MAX_VERBOSE_PARSE_EXCEPTIONS; } else { maxVerboseParseExceptions = Math.min(maxAllowedParseExceptions, Limits.MAX_VERBOSE_PARSE_EXCEPTIONS); } - Set criticalWarningCodes; + final Set criticalWarningCodes; if (maxAllowedParseExceptions == 0) { criticalWarningCodes = ImmutableSet.of(CannotParseExternalDataFault.CODE); } else { criticalWarningCodes = ImmutableSet.of(); } + // Delay removal of kernels so we don't interfere with iteration of kernelHolders.getAllKernelHolders(). + final Set kernelsToRemove = new HashSet<>(); + + while (!kernelHolders.isDone()) { + boolean didSomething = false; + + for (final KernelHolder kernelHolder : kernelHolders.getAllKernelHolders()) { + final WorkerStageKernel kernel = kernelHolder.kernel; + final StageDefinition stageDefinition = kernel.getStageDefinition(); + + // Workers run all work orders they get. There is not (currently) any limit on the number of concurrent work + // orders; we rely on the controller to avoid overloading workers. + if (kernel.getPhase() == WorkerStagePhase.NEW + && kernelHolders.runningKernelCount() < context.maxConcurrentStages()) { + handleNewWorkOrder( + kernelHolder, + controllerClient, + workerExec, + criticalWarningCodes, + maxVerboseParseExceptions + ); + logKernelStatus(kernelHolders.getAllKernels()); + didSomething = true; + } + + if (kernel.getPhase() == WorkerStagePhase.READING_INPUT + && handleReadingInput(kernelHolder, controllerClient)) { + didSomething = true; + logKernelStatus(kernelHolders.getAllKernels()); + } + + if (kernel.getPhase() == WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES + && handleWaitingForResultPartitionBoundaries(kernelHolder)) { + didSomething = true; + logKernelStatus(kernelHolders.getAllKernels()); + } + + if (kernel.getPhase() == WorkerStagePhase.RESULTS_COMPLETE + && handleResultsReady(kernelHolder, controllerClient)) { + didSomething = true; + logKernelStatus(kernelHolders.getAllKernels()); + } + + if (kernel.getPhase() == WorkerStagePhase.FAILED) { + // Return an error report when a work order fails. This is better than throwing an exception, because we can + // include the stage number. + return Optional.of( + MSQErrorReport.fromException( + id(), + MSQTasks.getHostFromSelfNode(selfDruidNode), + stageDefinition.getId().getStageNumber(), + kernel.getException() + ) + ); + } + + if (kernel.getPhase().isTerminal()) { + handleTerminated(kernelHolder); + kernelsToRemove.add(stageDefinition.getId()); + } + } + + for (final StageId stageId : kernelsToRemove) { + kernelHolders.removeKernel(stageId); + } + + kernelsToRemove.clear(); + + if (!didSomething && !kernelHolders.isDone()) { + Consumer nextCommand; + + // Run the next command, waiting for it if necessary. Post counters to the controller every 5 seconds + // while waiting. + do { + postCountersToController(kernelHolders.getControllerClient()); + } while ((nextCommand = kernelManipulationQueue.poll(5, TimeUnit.SECONDS)) == null); + + nextCommand.accept(kernelHolders); + + // Run all pending commands after that one. Helps avoid deep queues. + // After draining the command queue, move on to the next iteration of the worker loop. + while ((nextCommand = kernelManipulationQueue.poll()) != null) { + nextCommand.accept(kernelHolders); + } + + logKernelStatus(kernelHolders.getAllKernels()); + } + } + + // Empty means success. + return Optional.empty(); + } + + /** + * Handle a kernel in state {@link WorkerStagePhase#NEW}. The kernel is transitioned to + * {@link WorkerStagePhase#READING_INPUT} and a {@link RunWorkOrder} instance is created to start executing work. + */ + private void handleNewWorkOrder( + final KernelHolder kernelHolder, + final ControllerClient controllerClient, + final FrameProcessorExecutor workerExec, + final Set criticalWarningCodes, + final long maxVerboseParseExceptions + ) throws IOException + { + final WorkerStageKernel kernel = kernelHolder.kernel; + final WorkOrder workOrder = kernel.getWorkOrder(); + final StageDefinition stageDefinition = workOrder.getStageDefinition(); + final String cancellationId = cancellationIdFor(stageDefinition.getId()); + + log.info( + "Processing work order for stage[%s]%s", + stageDefinition.getId(), + (log.isDebugEnabled() + ? StringUtils.format(", payload[%s]", context.jsonMapper().writeValueAsString(workOrder)) : "") + ); + + final FrameContext frameContext = kernelHolder.processorCloser.register( + context.frameContext( + workOrder.getQueryDefinition(), + stageDefinition.getStageNumber(), + workOrder.getOutputChannelMode() + ) + ); + kernelHolder.processorCloser.register(() -> { + try { + workerExec.cancel(cancellationId); + } + catch (InterruptedException e) { + // Strange that cancellation would itself be interrupted. Log and suppress. + log.warn(e, "Cancellation interrupted for stage[%s]", stageDefinition.getId()); + Thread.currentThread().interrupt(); + } + }); + + // Set up cleanup functions for this work order. + kernelHolder.resultsCloser.register(() -> FileUtils.deleteDirectory(frameContext.tempDir())); + kernelHolder.resultsCloser.register(() -> removeStageOutputChannels(stageDefinition.getId())); + + // Create separate inputChannelFactory per stage, because the list of tasks can grow between stages, and + // so we need to avoid the memoization of controllerClient.getWorkerIds() in baseInputChannelFactory. + final InputChannelFactory inputChannelFactory = + makeBaseInputChannelFactory(workOrder, controllerClient, kernelHolder.processorCloser); + + // Start working on this stage immediately. + kernel.startReading(); + + final QueryContext queryContext = task != null ? QueryContext.of(task.getContext()) : QueryContext.empty(); + final RunWorkOrder runWorkOrder = new RunWorkOrder( + workOrder, + inputChannelFactory, + stageCounters.computeIfAbsent( + IntObjectPair.of(workOrder.getWorkerNumber(), stageDefinition.getId()), + ignored -> new CounterTracker() + ), + workerExec, + cancellationId, + context, + frameContext, + makeRunWorkOrderListener(workOrder, controllerClient, criticalWarningCodes, maxVerboseParseExceptions), + MultiStageQueryContext.isReindex(queryContext), + MultiStageQueryContext.removeNullBytes(queryContext) + ); + + runWorkOrder.start(); + kernelHolder.partitionBoundariesFuture = runWorkOrder.getStagePartitionBoundariesFuture(); + } + + /** + * Handle a kernel in state {@link WorkerStagePhase#READING_INPUT}. + * + * If the worker has finished generating result key statistics, they are posted to the controller and the kernel is + * transitioned to {@link WorkerStagePhase#PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES}. + * + * @return whether kernel state changed + */ + private boolean handleReadingInput( + final KernelHolder kernelHolder, + final ControllerClient controllerClient + ) throws IOException + { + final WorkerStageKernel kernel = kernelHolder.kernel; + if (kernel.hasResultKeyStatisticsSnapshot()) { + if (controllerAlive) { + PartialKeyStatisticsInformation partialKeyStatisticsInformation = + kernel.getResultKeyStatisticsSnapshot() + .partialKeyStatistics(); + + controllerClient.postPartialKeyStatistics( + kernel.getStageDefinition().getId(), + kernel.getWorkOrder().getWorkerNumber(), + partialKeyStatisticsInformation + ); + } + + kernel.startPreshuffleWaitingForResultPartitionBoundaries(); + return true; + } else if (kernel.isDoneReadingInput() + && kernel.getStageDefinition().doesSortDuringShuffle() + && !kernel.getStageDefinition().mustGatherResultKeyStatistics()) { + // Skip postDoneReadingInput when context.maxConcurrentStages() == 1, for backwards compatibility. + // See Javadoc comment on ControllerClient#postDoneReadingInput. + if (controllerAlive && context.maxConcurrentStages() > 1) { + controllerClient.postDoneReadingInput( + kernel.getStageDefinition().getId(), + kernel.getWorkOrder().getWorkerNumber() + ); + } + + kernel.startPreshuffleWritingOutput(); + return true; + } else { + return false; + } + } + + /** + * Handle a kernel in state {@link WorkerStagePhase#PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES}. + * + * If partition boundaries have become available, the {@link KernelHolder#partitionBoundariesFuture} is updated and + * the kernel is transitioned to state {@link WorkerStagePhase#PRESHUFFLE_WRITING_OUTPUT}. + * + * @return whether kernel state changed + */ + private boolean handleWaitingForResultPartitionBoundaries(final KernelHolder kernelHolder) + { + if (kernelHolder.kernel.hasResultPartitionBoundaries()) { + kernelHolder.partitionBoundariesFuture.set(kernelHolder.kernel.getResultPartitionBoundaries()); + kernelHolder.kernel.startPreshuffleWritingOutput(); + return true; + } else { + return false; + } + } + + /** + * Handle a kernel in state {@link WorkerStagePhase#RESULTS_COMPLETE}. If {@link ControllerClient#postResultsComplete} + * has not yet been posted to the controller, it is posted at this time. Otherwise nothing happens. + * + * @return whether kernel state changed + */ + private boolean handleResultsReady(final KernelHolder kernelHolder, final ControllerClient controllerClient) + throws IOException + { + final WorkerStageKernel kernel = kernelHolder.kernel; + final boolean didNotPostYet = + kernel.addPostedResultsComplete(kernel.getStageDefinition().getId(), kernel.getWorkOrder().getWorkerNumber()); + + if (controllerAlive && didNotPostYet) { + controllerClient.postResultsComplete( + kernel.getStageDefinition().getId(), + kernel.getWorkOrder().getWorkerNumber(), + kernel.getResultObject() + ); + } + + return didNotPostYet; + } + + /** + * Handle a kernel in state where {@link WorkerStagePhase#isTerminal()} is true. + */ + private void handleTerminated(final KernelHolder kernelHolder) + { + final WorkerStageKernel kernel = kernelHolder.kernel; + removeStageOutputChannels(kernel.getStageDefinition().getId()); + + if (kernelHolder.kernel.getWorkOrder().getOutputChannelMode().isDurable()) { + removeStageDurableStorageOutput(kernel.getStageDefinition().getId()); + } + } + + @Override + public void stop() + { + // stopGracefully() is called when the containing process is terminated, or when the task is canceled. + log.info("Worker id[%s] canceled.", context.workerId()); + + if (didRun.compareAndSet(false, true)) { + // run() hasn't been called yet. Set runFuture so awaitStop() still works. + runFuture.set(null); + } else { + doCancel(); + } + } + + @Override + public void awaitStop() + { + FutureUtils.getUnchecked(runFuture, false); + } + + @Override + public void controllerFailed() + { + log.info( + "Controller task[%s] for worker[%s] failed. Canceling.", + task != null ? task.getControllerTaskId() : null, + id() + ); + doCancel(); + } + + @Override + public ListenableFuture readStageOutput( + final StageId stageId, + final int partitionNumber, + final long offset + ) + { + return getOrCreateStageOutputHolder(stageId, partitionNumber).readRemotelyFrom(offset); + } + + @Override + public void postWorkOrder(final WorkOrder workOrder) + { + log.info( + "Got work order for stage[%s], workerNumber[%s]", + workOrder.getStageDefinition().getId(), + workOrder.getWorkerNumber() + ); + + if (task != null && task.getWorkerNumber() != workOrder.getWorkerNumber()) { + throw new ISE( + "Worker number mismatch: expected workerNumber[%d], got[%d]", + task.getWorkerNumber(), + workOrder.getWorkerNumber() + ); + } + + final OutputChannelMode outputChannelMode; + + // This stack of conditions can be removed once we can rely on OutputChannelMode always being in the WorkOrder. + // (It will be there for newer controllers; this is a backwards-compatibility thing.) + if (workOrder.hasOutputChannelMode()) { + outputChannelMode = workOrder.getOutputChannelMode(); + } else { + final MSQSelectDestination selectDestination = + task != null + ? MultiStageQueryContext.getSelectDestination(QueryContext.of(task.getContext())) + : MSQSelectDestination.TASKREPORT; + + outputChannelMode = ControllerQueryKernelUtils.getOutputChannelMode( + workOrder.getQueryDefinition(), + workOrder.getStageNumber(), + selectDestination, + task != null && MultiStageQueryContext.isDurableStorageEnabled(QueryContext.of(task.getContext())), + false + ); + } + + final WorkOrder workOrderToUse = workOrder.withOutputChannelMode(outputChannelMode); + kernelManipulationQueue.add( + kernelHolders -> + kernelHolders.addKernel(WorkerStageKernel.create(workOrderToUse)) + ); + } + + @Override + public boolean postResultPartitionBoundaries( + final StageId stageId, + final ClusterByPartitions stagePartitionBoundaries + ) + { + kernelManipulationQueue.add( + kernelHolders -> { + final WorkerStageKernel stageKernel = kernelHolders.getKernelFor(stageId); + + if (stageKernel != null) { + if (!stageKernel.hasResultPartitionBoundaries()) { + stageKernel.setResultPartitionBoundaries(stagePartitionBoundaries); + } else { + // Ignore if partition boundaries are already set. + log.warn("Stage[%s] already has result partition boundaries set. Ignoring new ones.", stageId); + } + } + } + ); + return true; + } + + @Override + public void postCleanupStage(final StageId stageId) + { + log.debug("Received cleanup order for stage[%s].", stageId); + kernelManipulationQueue.add(holder -> { + holder.finishProcessing(stageId); + final WorkerStageKernel kernel = holder.getKernelFor(stageId); + if (kernel != null) { + kernel.setStageFinished(); + } + }); + } + + @Override + public void postFinish() + { + log.debug("Received finish call."); + kernelManipulationQueue.add(KernelHolders::setDone); + } + + @Override + public ClusterByStatisticsSnapshot fetchStatisticsSnapshot(StageId stageId) + { + log.debug("Fetching statistics for stage[%s]", stageId); + final SettableFuture snapshotFuture = SettableFuture.create(); + kernelManipulationQueue.add( + holder -> { + try { + final WorkerStageKernel kernel = holder.getKernelFor(stageId); + if (kernel != null) { + final ClusterByStatisticsSnapshot snapshot = kernel.getResultKeyStatisticsSnapshot(); + if (snapshot == null) { + throw new ISE("Requested statistics snapshot is not generated yet for stage [%s]", stageId); + } + + snapshotFuture.set(snapshot); + } else { + snapshotFuture.setException(new ISE("Stage[%s] has terminated", stageId)); + } + } + catch (Throwable t) { + snapshotFuture.setException(t); + } + } + ); + return FutureUtils.getUnchecked(snapshotFuture, true); + } + + @Override + public ClusterByStatisticsSnapshot fetchStatisticsSnapshotForTimeChunk(StageId stageId, long timeChunk) + { + return fetchStatisticsSnapshot(stageId).getSnapshotForTimeChunk(timeChunk); + } + + @Override + public CounterSnapshotsTree getCounters() + { + final CounterSnapshotsTree retVal = new CounterSnapshotsTree(); + + for (final Map.Entry, CounterTracker> entry : stageCounters.entrySet()) { + retVal.put( + entry.getKey().right().getStageNumber(), + entry.getKey().leftInt(), + entry.getValue().snapshot() + ); + } + + return retVal; + } + + /** + * Create a {@link RunWorkOrderListener} for {@link RunWorkOrder} that hooks back into the {@link KernelHolders} + * in the main loop. + */ + private RunWorkOrderListener makeRunWorkOrderListener( + final WorkOrder workOrder, + final ControllerClient controllerClient, + final Set criticalWarningCodes, + final long maxVerboseParseExceptions + ) + { + final StageId stageId = workOrder.getStageDefinition().getId(); final MSQWarningReportPublisher msqWarningReportPublisher = new MSQWarningReportLimiterPublisher( new MSQWarningReportSimplePublisher( id(), @@ -360,430 +737,136 @@ public class WorkerImpl implements Worker MSQTasks.getHostFromSelfNode(selfDruidNode) ); - closer.register(msqWarningReportPublisher); + return new RunWorkOrderListener() + { + @Override + public void onDoneReadingInput(@Nullable ClusterByStatisticsSnapshot snapshot) + { + kernelManipulationQueue.add( + holder -> { + final WorkerStageKernel kernel = holder.getKernelFor(stageId); + if (kernel != null) { + kernel.setResultKeyStatisticsSnapshot(snapshot); + } + } + ); + } - final Map> partitionBoundariesFutureMap = new HashMap<>(); + @Override + public void onOutputChannelAvailable(OutputChannel channel) + { + ReadableFrameChannel readableChannel = null; - final Map stageFrameContexts = new HashMap<>(); - - while (!kernelHolder.isDone()) { - boolean didSomething = false; - - for (final WorkerStageKernel kernel : kernelHolder.getStageKernelMap().values()) { - final StageDefinition stageDefinition = kernel.getStageDefinition(); - - if (kernel.getPhase() == WorkerStagePhase.NEW) { - - log.info("Processing work order for stage [%d]" + - (log.isDebugEnabled() - ? StringUtils.format( - " with payload [%s]", - context.jsonMapper().writeValueAsString(kernel.getWorkOrder()) - ) : ""), stageDefinition.getId().getStageNumber()); - - // Create separate inputChannelFactory per stage, because the list of tasks can grow between stages, and - // so we need to avoid the memoization in baseInputChannelFactory. - final InputChannelFactory inputChannelFactory = makeBaseInputChannelFactory(closer); - - // Compute memory parameters for all stages, even ones that haven't been assigned yet, so we can fail-fast - // if some won't work. (We expect that all stages will get assigned to the same pool of workers.) - for (final StageDefinition stageDef : kernel.getWorkOrder().getQueryDefinition().getStageDefinitions()) { - stageFrameContexts.computeIfAbsent( - stageDef.getId(), - stageId -> context.frameContext( - kernel.getWorkOrder().getQueryDefinition(), - stageId.getStageNumber() - ) - ); - } - - // Start working on this stage immediately. - kernel.startReading(); - - final RunWorkOrder runWorkOrder = new RunWorkOrder( - kernel, - inputChannelFactory, - stageCounters.computeIfAbsent(stageDefinition.getId(), ignored -> new CounterTracker()), - workerExec, - cancellationId, - context.threadCount(), - stageFrameContexts.get(stageDefinition.getId()), - msqWarningReportPublisher - ); - - runWorkOrder.start(); - - final SettableFuture partitionBoundariesFuture = - runWorkOrder.getStagePartitionBoundariesFuture(); - - if (partitionBoundariesFuture != null) { - if (partitionBoundariesFutureMap.put(stageDefinition.getId(), partitionBoundariesFuture) != null) { - throw new ISE("Work order collision for stage [%s]", stageDefinition.getId()); + try { + readableChannel = channel.getReadableChannel(); + getOrCreateStageOutputHolder(stageId, channel.getPartitionNumber()) + .setChannel(readableChannel); + } + catch (Exception e) { + if (readableChannel != null) { + try { + readableChannel.close(); + } + catch (Throwable e2) { + e.addSuppressed(e2); } } - didSomething = true; - logKernelStatus(kernelHolder.getStageKernelMap().values()); + kernelManipulationQueue.add(holder -> { + throw new RE(e, "Worker completion callback error for stage [%s]", stageId); + }); } + } - if (kernel.getPhase() == WorkerStagePhase.READING_INPUT && kernel.hasResultKeyStatisticsSnapshot()) { - if (controllerAlive) { - PartialKeyStatisticsInformation partialKeyStatisticsInformation = - kernel.getResultKeyStatisticsSnapshot() - .partialKeyStatistics(); + @Override + public void onSuccess(Object resultObject) + { + kernelManipulationQueue.add( + holder -> { + // Call finishProcessing prior to transitioning to RESULTS_COMPLETE, so the FrameContext is closed + // and resources are released. + holder.finishProcessing(stageId); - controllerClient.postPartialKeyStatistics( - stageDefinition.getId(), - kernel.getWorkOrder().getWorkerNumber(), - partialKeyStatisticsInformation - ); - } - kernel.startPreshuffleWaitingForResultPartitionBoundaries(); + final WorkerStageKernel kernel = holder.getKernelFor(stageId); + if (kernel != null) { + kernel.setResultsComplete(resultObject); + } + } + ); + } - didSomething = true; - logKernelStatus(kernelHolder.getStageKernelMap().values()); - } + @Override + public void onWarning(Throwable t) + { + msqWarningReportPublisher.publishException(stageId.getStageNumber(), t); + } - logKernelStatus(kernelHolder.getStageKernelMap().values()); - if (kernel.getPhase() == WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES - && kernel.hasResultPartitionBoundaries()) { - partitionBoundariesFutureMap.get(stageDefinition.getId()).set(kernel.getResultPartitionBoundaries()); - kernel.startPreshuffleWritingOutput(); + @Override + public void onFailure(Throwable t) + { + kernelManipulationQueue.add( + holder -> { + final WorkerStageKernel kernel = holder.getKernelFor(stageId); + if (kernel != null) { + kernel.fail(t); + } + } + ); + } + }; + } - didSomething = true; - logKernelStatus(kernelHolder.getStageKernelMap().values()); - } + private InputChannelFactory makeBaseInputChannelFactory( + final WorkOrder workOrder, + final ControllerClient controllerClient, + final Closer closer + ) + { + return MetaInputChannelFactory.create( + InputSlices.allStageSlices(workOrder.getInputs()), + workOrder.getOutputChannelMode(), + outputChannelMode -> { + switch (outputChannelMode) { + case MEMORY: + case LOCAL_STORAGE: + final Supplier> workerIds; - if (kernel.getPhase() == WorkerStagePhase.RESULTS_READY - && kernel.addPostedResultsComplete(Pair.of( - stageDefinition.getId(), - kernel.getWorkOrder().getWorkerNumber() - ))) { - if (controllerAlive) { - controllerClient.postResultsComplete( - stageDefinition.getId(), - kernel.getWorkOrder().getWorkerNumber(), - kernel.getResultObject() - ); - } - } + if (workOrder.getWorkerIds() != null) { + workerIds = workOrder::getWorkerIds; + } else { + workerIds = Suppliers.memoize( + () -> { + try { + return controllerClient.getWorkerIds(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + ); + } - if (kernel.getPhase() == WorkerStagePhase.FAILED) { - // Better than throwing an exception, because we can include the stage number. - return Optional.of( - MSQErrorReport.fromException( + return new WorkerOrLocalInputChannelFactory( id(), - MSQTasks.getHostFromSelfNode(selfDruidNode), - stageDefinition.getId().getStageNumber(), - kernel.getException() - ) - ); - } - } - - if (!didSomething && !kernelHolder.isDone()) { - Consumer nextCommand; - - do { - postCountersToController(); - } while ((nextCommand = kernelManipulationQueue.poll(5, TimeUnit.SECONDS)) == null); - - nextCommand.accept(kernelHolder); - logKernelStatus(kernelHolder.getStageKernelMap().values()); - } - } - - // Empty means success. - return Optional.empty(); - } - - @Override - public void stopGracefully() - { - // stopGracefully() is called when the containing process is terminated, or when the task is canceled. - log.info("Worker task[%s] canceled.", task.getId()); - doCancel(); - } - - @Override - public void controllerFailed() - { - log.info("Controller task[%s] for worker task[%s] failed. Canceling.", task.getControllerTaskId(), task.getId()); - doCancel(); - } - - @Override - public InputStream readChannel( - final String queryId, - final int stageNumber, - final int partitionNumber, - final long offset - ) throws IOException - { - final StageId stageId = new StageId(queryId, stageNumber); - final StagePartition stagePartition = new StagePartition(stageId, partitionNumber); - final ConcurrentHashMap partitionOutputsForStage = stageOutputs.get(stageId); - - if (partitionOutputsForStage == null) { - return null; - } - final ReadableFrameChannel channel = partitionOutputsForStage.get(partitionNumber); - - if (channel == null) { - return null; - } - - if (channel instanceof ReadableNilFrameChannel) { - // Build an empty frame file. - final ByteArrayOutputStream baos = new ByteArrayOutputStream(); - FrameFileWriter.open(Channels.newChannel(baos), null, ByteTracker.unboundedTracker()).close(); - - final ByteArrayInputStream in = new ByteArrayInputStream(baos.toByteArray()); - - //noinspection ResultOfMethodCallIgnored: OK to ignore since "skip" always works for ByteArrayInputStream. - in.skip(offset); - - return in; - } else if (channel instanceof ReadableFileFrameChannel) { - // Close frameFile once we've returned an input stream: no need to retain a reference to the mmap after that, - // since we aren't using it. - try (final FrameFile frameFile = ((ReadableFileFrameChannel) channel).newFrameFileReference()) { - final RandomAccessFile randomAccessFile = new RandomAccessFile(frameFile.file(), "r"); - - if (offset >= randomAccessFile.length()) { - randomAccessFile.close(); - return new ByteArrayInputStream(ByteArrays.EMPTY_ARRAY); - } else { - randomAccessFile.seek(offset); - return Channels.newInputStream(randomAccessFile.getChannel()); - } - } - } else { - String errorMsg = StringUtils.format( - "Returned server error to client because channel for [%s] is not nil or file-based (class = %s)", - stagePartition, - channel.getClass().getName() - ); - log.error(StringUtils.encodeForFormat(errorMsg)); - - throw new IOException(errorMsg); - } - } - - @Override - public void postWorkOrder(final WorkOrder workOrder) - { - log.info("Got work order for stage [%d]", workOrder.getStageNumber()); - if (task.getWorkerNumber() != workOrder.getWorkerNumber()) { - throw new ISE("Worker number mismatch: expected [%d]", task.getWorkerNumber()); - } - - // Do not add to queue if workerOrder already present. - kernelManipulationQueue.add( - kernelHolder -> - kernelHolder.getStageKernelMap().putIfAbsent( - workOrder.getStageDefinition().getId(), - WorkerStageKernel.create(workOrder) - ) - ); - } - - @Override - public boolean postResultPartitionBoundaries( - final ClusterByPartitions stagePartitionBoundaries, - final String queryId, - final int stageNumber - ) - { - final StageId stageId = new StageId(queryId, stageNumber); - - kernelManipulationQueue.add( - kernelHolder -> { - final WorkerStageKernel stageKernel = kernelHolder.getStageKernelMap().get(stageId); - - if (stageKernel != null) { - if (!stageKernel.hasResultPartitionBoundaries()) { - stageKernel.setResultPartitionBoundaries(stagePartitionBoundaries); - } else { - // Ignore if partition boundaries are already set. - log.warn( - "Stage[%s] already has result partition boundaries set. Ignoring the latest partition boundaries recieved.", - stageId + workerIds, + new WorkerInputChannelFactory(workerClient, workerIds), + this::getOrCreateStageOutputHolder ); - } - } else { - // Ignore the update if we don't have a kernel for this stage. - log.warn("Ignored result partition boundaries call for unknown stage [%s]", stageId); - } - } - ); - return true; - } - @Override - public void postCleanupStage(final StageId stageId) - { - log.info("Cleanup order for stage [%s] received", stageId); - kernelManipulationQueue.add( - holder -> { - cleanStageOutput(stageId, true); - // Mark the stage as FINISHED - WorkerStageKernel stageKernel = holder.getStageKernelMap().get(stageId); - if (stageKernel == null) { - log.warn("Stage id [%s] non existent. Unable to mark the stage kernel for it as FINISHED", stageId); - } else { - stageKernel.setStageFinished(); - } - } - ); - } - - @Override - public void postFinish() - { - log.info("Finish received for task [%s]", task.getId()); - kernelManipulationQueue.add(KernelHolder::setDone); - } - - @Override - public ClusterByStatisticsSnapshot fetchStatisticsSnapshot(StageId stageId) - { - log.info("Fetching statistics for stage [%d]", stageId.getStageNumber()); - if (stageKernelMap.get(stageId) == null) { - throw new ISE("Requested statistics snapshot for non-existent stageId %s.", stageId); - } else if (stageKernelMap.get(stageId).getResultKeyStatisticsSnapshot() == null) { - throw new ISE( - "Requested statistics snapshot is not generated yet for stageId [%s]", - stageId - ); - } else { - return stageKernelMap.get(stageId).getResultKeyStatisticsSnapshot(); - } - } - - @Override - public ClusterByStatisticsSnapshot fetchStatisticsSnapshotForTimeChunk(StageId stageId, long timeChunk) - { - log.debug( - "Fetching statistics for stage [%d] with time chunk [%d] ", - stageId.getStageNumber(), - timeChunk - ); - if (stageKernelMap.get(stageId) == null) { - throw new ISE("Requested statistics snapshot for non-existent stageId [%s].", stageId); - } else if (stageKernelMap.get(stageId).getResultKeyStatisticsSnapshot() == null) { - throw new ISE( - "Requested statistics snapshot is not generated yet for stageId [%s]", - stageId - ); - } else { - return stageKernelMap.get(stageId) - .getResultKeyStatisticsSnapshot() - .getSnapshotForTimeChunk(timeChunk); - } - - } - - - @Override - public CounterSnapshotsTree getCounters() - { - final CounterSnapshotsTree retVal = new CounterSnapshotsTree(); - - for (final Map.Entry entry : stageCounters.entrySet()) { - retVal.put(entry.getKey().getStageNumber(), task().getWorkerNumber(), entry.getValue().snapshot()); - } - - return retVal; - } - - private InputChannelFactory makeBaseInputChannelFactory(final Closer closer) - { - final Supplier> workerTaskList = Suppliers.memoize( - () -> { - try { - return controllerClient.getTaskList(); - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - )::get; - - if (durableStageStorageEnabled) { - return DurableStorageInputChannelFactory.createStandardImplementation( - task.getControllerTaskId(), - MSQTasks.makeStorageConnector(context.injector()), - closer, - false - ); - } else { - return new WorkerOrLocalInputChannelFactory(workerTaskList); - } - } - - private OutputChannelFactory makeStageOutputChannelFactory( - final FrameContext frameContext, - final int stageNumber, - boolean isFinalStage - ) - { - // Use the standard frame size, since we assume this size when computing how much is needed to merge output - // files from different workers. - final int frameSize = frameContext.memoryParameters().getStandardFrameSize(); - - if (durableStageStorageEnabled || (isFinalStage - && MSQSelectDestination.DURABLESTORAGE.equals(selectDestination))) { - return DurableStorageOutputChannelFactory.createStandardImplementation( - task.getControllerTaskId(), - task().getWorkerNumber(), - stageNumber, - task().getId(), - frameSize, - MSQTasks.makeStorageConnector(context.injector()), - context.tempDir(), - (isFinalStage && MSQSelectDestination.DURABLESTORAGE.equals(selectDestination)) - ); - } else { - final File fileChannelDirectory = - new File(context.tempDir(), StringUtils.format("output_stage_%06d", stageNumber)); - - return new FileOutputChannelFactory(fileChannelDirectory, frameSize, null); - } - } - - private OutputChannelFactory makeSuperSorterIntermediateOutputChannelFactory( - final FrameContext frameContext, - final int stageNumber, - final File tmpDir - ) - { - final int frameSize = frameContext.memoryParameters().getLargeFrameSize(); - final File fileChannelDirectory = - new File(tmpDir, StringUtils.format("intermediate_output_stage_%06d", stageNumber)); - final FileOutputChannelFactory fileOutputChannelFactory = - new FileOutputChannelFactory(fileChannelDirectory, frameSize, intermediateSuperSorterLocalStorageTracker); - - if (durableStageStorageEnabled && workerStorageParameters.isIntermediateStorageLimitConfigured()) { - return new ComposingOutputChannelFactory( - ImmutableList.of( - fileOutputChannelFactory, - DurableStorageOutputChannelFactory.createStandardImplementation( + case DURABLE_STORAGE_INTERMEDIATE: + case DURABLE_STORAGE_QUERY_RESULTS: + return DurableStorageInputChannelFactory.createStandardImplementation( task.getControllerTaskId(), - task().getWorkerNumber(), - stageNumber, - task().getId(), - frameSize, MSQTasks.makeStorageConnector(context.injector()), - tmpDir, - false - ) - ), - frameSize - ); - } else { - return fileOutputChannelFactory; - } + closer, + outputChannelMode == OutputChannelMode.DURABLE_STORAGE_QUERY_RESULTS + ); + + default: + throw DruidException.defensive("No handling for output channel mode[%s]", outputChannelMode); + } + } + ); } /** @@ -846,69 +929,75 @@ public class WorkerImpl implements Worker /** * Posts all counters for this worker to the controller. */ - private void postCountersToController() throws IOException + private void postCountersToController(final ControllerClient controllerClient) throws IOException { final CounterSnapshotsTree snapshotsTree = getCounters(); if (controllerAlive && !snapshotsTree.isEmpty()) { - try { - controllerClient.postCounters(id(), snapshotsTree); - } - catch (IOException e) { - if (e.getCause() instanceof ServiceClosedException) { - // Suppress. This can happen if the controller goes away while a postCounters call is in flight. - log.debug(e, "Ignoring failure on postCounters, because controller has gone away."); - } else { - throw e; - } - } + controllerClient.postCounters(id(), snapshotsTree); } } /** - * Cleans up the stage outputs corresponding to the provided stage id. It essentially calls {@code doneReading()} on - * the readable channels corresponding to all the partitions for that stage, and removes it from the {@code stageOutputs} - * map + * Removes and closes all output channels for a stage from {@link #stageOutputs}. */ - private void cleanStageOutput(final StageId stageId, boolean removeDurableStorageFiles) + private void removeStageOutputChannels(final StageId stageId) { // This code is thread-safe because remove() on ConcurrentHashMap will remove and return the removed channel only for // one thread. For the other threads it will return null, therefore we will call doneReading for a channel only once - final ConcurrentHashMap partitionOutputsForStage = stageOutputs.remove(stageId); + final ConcurrentHashMap partitionOutputsForStage = stageOutputs.remove(stageId); // Check for null, this can be the case if this method is called simultaneously from multiple threads. if (partitionOutputsForStage == null) { return; } for (final int partition : partitionOutputsForStage.keySet()) { - final ReadableFrameChannel output = partitionOutputsForStage.remove(partition); - if (output == null) { - continue; - } - output.close(); - } - // One caveat with this approach is that in case of a worker crash, while the MM/Indexer systems will delete their - // temp directories where intermediate results were stored, it won't be the case for the external storage. - // Therefore, the logic for cleaning the stage output in case of a worker/machine crash has to be external. - // We currently take care of this in the controller. - if (durableStageStorageEnabled && removeDurableStorageFiles) { - final String folderName = DurableStorageUtils.getTaskIdOutputsFolderName( - task.getControllerTaskId(), - stageId.getStageNumber(), - task.getWorkerNumber(), - task.getId() - ); - try { - MSQTasks.makeStorageConnector(context.injector()).deleteRecursively(folderName); - } - catch (Exception e) { - // If an error is thrown while cleaning up a file, log it and try to continue with the cleanup - log.warn(e, "Error while cleaning up folder at path " + folderName); + final StageOutputHolder output = partitionOutputsForStage.remove(partition); + if (output != null) { + output.close(); } } } /** - * Called by {@link #stopGracefully()} (task canceled, or containing process shut down) and + * Remove outputs from durable storage for a particular stage. + */ + private void removeStageDurableStorageOutput(final StageId stageId) + { + // One caveat with this approach is that in case of a worker crash, while the MM/Indexer systems will delete their + // temp directories where intermediate results were stored, it won't be the case for the external storage. + // Therefore, the logic for cleaning the stage output in case of a worker/machine crash has to be external. + // We currently take care of this in the controller. + final String folderName = DurableStorageUtils.getTaskIdOutputsFolderName( + task.getControllerTaskId(), + stageId.getStageNumber(), + task.getWorkerNumber(), + context.workerId() + ); + try { + MSQTasks.makeStorageConnector(context.injector()).deleteRecursively(folderName); + } + catch (Exception e) { + // If an error is thrown while cleaning up a file, log it and try to continue with the cleanup + log.warn(e, "Error while cleaning up durable storage path[%s].", folderName); + } + } + + private StageOutputHolder getOrCreateStageOutputHolder(final StageId stageId, final int partitionNumber) + { + return stageOutputs.computeIfAbsent(stageId, ignored1 -> new ConcurrentHashMap<>()) + .computeIfAbsent(partitionNumber, ignored -> new StageOutputHolder()); + } + + /** + * Returns cancellation ID for a particular stage, to be used in {@link FrameProcessorExecutor#cancel(String)}. + */ + private static String cancellationIdFor(final StageId stageId) + { + return stageId.toString(); + } + + /** + * Called by {@link #stop()} (task canceled, or containing process shut down) and * {@link #controllerFailed()}. */ private void doCancel() @@ -935,15 +1024,15 @@ public class WorkerImpl implements Worker /** * Log (at DEBUG level) a string explaining the status of all work assigned to this worker. */ - private static void logKernelStatus(final Collection kernels) + private static void logKernelStatus(final Iterable kernels) { if (log.isDebugEnabled()) { log.debug( "Stages: %s", - kernels.stream() - .sorted(Comparator.comparing(k -> k.getStageDefinition().getStageNumber())) - .map(WorkerImpl::makeKernelStageStatusString) - .collect(Collectors.joining("; ")) + StreamSupport.stream(kernels.spliterator(), false) + .sorted(Comparator.comparing(k -> k.getStageDefinition().getStageNumber())) + .map(WorkerImpl::makeKernelStageStatusString) + .collect(Collectors.joining("; ")) ); } } @@ -978,936 +1067,205 @@ public class WorkerImpl implements Worker } /** - * An {@link InputChannelFactory} that loads data locally when possible, and otherwise connects directly to other - * workers. Used when durable shuffle storage is off. + * Holds {@link WorkerStageKernel} and {@link Closer}, one per {@link WorkOrder}. Also holds {@link ControllerClient}. + * Only manipulated by the main loop. Other threads that need to manipulate kernels must do so through + * {@link #kernelManipulationQueue}. */ - private class WorkerOrLocalInputChannelFactory implements InputChannelFactory + private static class KernelHolders implements Closeable { - private final Supplier> taskList; - private final WorkerInputChannelFactory workerInputChannelFactory; - - public WorkerOrLocalInputChannelFactory(final Supplier> taskList) - { - this.workerInputChannelFactory = new WorkerInputChannelFactory(workerClient, taskList); - this.taskList = taskList; - } - - @Override - public ReadableFrameChannel openChannel(StageId stageId, int workerNumber, int partitionNumber) - { - final String taskId = taskList.get().get(workerNumber); - if (taskId.equals(id())) { - final ConcurrentMap partitionOutputsForStage = stageOutputs.get(stageId); - if (partitionOutputsForStage == null) { - throw new ISE("Unable to find outputs for stage [%s]", stageId); - } - - final ReadableFrameChannel myChannel = partitionOutputsForStage.get(partitionNumber); - - if (myChannel instanceof ReadableFileFrameChannel) { - // Must duplicate the channel to avoid double-closure upon task cleanup. - final FrameFile frameFile = ((ReadableFileFrameChannel) myChannel).newFrameFileReference(); - return new ReadableFileFrameChannel(frameFile); - } else if (myChannel instanceof ReadableNilFrameChannel) { - return myChannel; - } else { - throw new ISE("Output for stage [%s] are stored in an instance of %s which is not " - + "supported", stageId, myChannel.getClass()); - } - } else { - return workerInputChannelFactory.openChannel(stageId, workerNumber, partitionNumber); - } - } - } - - /** - * Main worker logic for executing a {@link WorkOrder}. - */ - private class RunWorkOrder - { - private final WorkerStageKernel kernel; - private final InputChannelFactory inputChannelFactory; - private final CounterTracker counterTracker; - private final FrameProcessorExecutor exec; - private final String cancellationId; - private final int parallelism; - private final FrameContext frameContext; - private final MSQWarningReportPublisher warningPublisher; - - private InputSliceReader inputSliceReader; - private OutputChannelFactory workOutputChannelFactory; - private OutputChannelFactory shuffleOutputChannelFactory; - private ResultAndChannels workResultAndOutputChannels; - private SettableFuture stagePartitionBoundariesFuture; - private ListenableFuture shuffleOutputChannelsFuture; - - public RunWorkOrder( - final WorkerStageKernel kernel, - final InputChannelFactory inputChannelFactory, - final CounterTracker counterTracker, - final FrameProcessorExecutor exec, - final String cancellationId, - final int parallelism, - final FrameContext frameContext, - final MSQWarningReportPublisher warningPublisher - ) - { - this.kernel = kernel; - this.inputChannelFactory = inputChannelFactory; - this.counterTracker = counterTracker; - this.exec = exec; - this.cancellationId = cancellationId; - this.parallelism = parallelism; - this.frameContext = frameContext; - this.warningPublisher = warningPublisher; - } - - private void start() throws IOException - { - final WorkOrder workOrder = kernel.getWorkOrder(); - final StageDefinition stageDef = workOrder.getStageDefinition(); - - final boolean isFinalStage = stageDef.getStageNumber() == workOrder.getQueryDefinition() - .getFinalStageDefinition() - .getStageNumber(); - - makeInputSliceReader(); - makeWorkOutputChannelFactory(isFinalStage); - makeShuffleOutputChannelFactory(isFinalStage); - makeAndRunWorkProcessors(); - - if (stageDef.doesShuffle()) { - makeAndRunShuffleProcessors(); - } else { - // No shuffling: work output _is_ shuffle output. Retain read-only versions to reduce memory footprint. - shuffleOutputChannelsFuture = - Futures.immediateFuture(workResultAndOutputChannels.getOutputChannels().readOnly()); - } - - setUpCompletionCallbacks(isFinalStage); - } + private final WorkerContext workerContext; + private final ControllerClient controllerClient; /** - * Settable {@link ClusterByPartitions} future for global sort. Necessary because we don't know ahead of time - * what the boundaries will be. The controller decides based on statistics from all workers. Once the controller - * decides, its decision is written to this future, which allows sorting on workers to proceed. + * Stage number -> kernel holder. */ - @Nullable - public SettableFuture getStagePartitionBoundariesFuture() - { - return stagePartitionBoundariesFuture; - } + private final Int2ObjectMap holderMap = new Int2ObjectOpenHashMap<>(); - private void makeInputSliceReader() - { - if (inputSliceReader != null) { - throw new ISE("inputSliceReader already created"); - } - - final WorkOrder workOrder = kernel.getWorkOrder(); - final String queryId = workOrder.getQueryDefinition().getQueryId(); - - final InputChannels inputChannels = - new InputChannelsImpl( - workOrder.getQueryDefinition(), - InputSlices.allReadablePartitions(workOrder.getInputs()), - inputChannelFactory, - () -> ArenaMemoryAllocator.createOnHeap(frameContext.memoryParameters().getStandardFrameSize()), - exec, - cancellationId, - MultiStageQueryContext.removeNullBytes(QueryContext.of(task.getContext())) - ); - - inputSliceReader = new MapInputSliceReader( - ImmutableMap., InputSliceReader>builder() - .put(NilInputSlice.class, NilInputSliceReader.INSTANCE) - .put(StageInputSlice.class, new StageInputSliceReader(queryId, inputChannels)) - .put(ExternalInputSlice.class, new ExternalInputSliceReader(frameContext.tempDir())) - .put(InlineInputSlice.class, new InlineInputSliceReader(frameContext.segmentWrangler())) - .put(LookupInputSlice.class, new LookupInputSliceReader(frameContext.segmentWrangler())) - .put( - SegmentsInputSlice.class, - new SegmentsInputSliceReader( - frameContext, - MultiStageQueryContext.isReindex(QueryContext.of(task().getContext())) - ) - ) - .build() - ); - } - - private void makeWorkOutputChannelFactory(boolean isFinalStage) - { - if (workOutputChannelFactory != null) { - throw new ISE("processorOutputChannelFactory already created"); - } - - final OutputChannelFactory baseOutputChannelFactory; - - if (kernel.getStageDefinition().doesShuffle()) { - // Writing to a consumer in the same JVM (which will be set up later on in this method). Use the large frame - // size if we're writing to a SuperSorter, since we'll generate fewer temp files if we use larger frames. - // Otherwise, use the standard frame size. - final int frameSize; - - if (kernel.getStageDefinition().getShuffleSpec().kind().isSort()) { - frameSize = frameContext.memoryParameters().getLargeFrameSize(); - } else { - frameSize = frameContext.memoryParameters().getStandardFrameSize(); - } - - baseOutputChannelFactory = new BlockingQueueOutputChannelFactory(frameSize); - } else { - // Writing stage output. - baseOutputChannelFactory = - makeStageOutputChannelFactory(frameContext, kernel.getStageDefinition().getStageNumber(), isFinalStage); - } - - workOutputChannelFactory = new CountingOutputChannelFactory( - baseOutputChannelFactory, - counterTracker.channel(CounterNames.outputChannel()) - ); - } - - private void makeShuffleOutputChannelFactory(boolean isFinalStage) - { - shuffleOutputChannelFactory = - new CountingOutputChannelFactory( - makeStageOutputChannelFactory(frameContext, kernel.getStageDefinition().getStageNumber(), isFinalStage), - counterTracker.channel(CounterNames.shuffleChannel()) - ); - } - - /** - * Use {@link FrameProcessorFactory#makeProcessors} to create {@link ProcessorsAndChannels}. Executes the - * processors using {@link #exec} and sets the output channels in {@link #workResultAndOutputChannels}. - * - * @param type of {@link StageDefinition#getProcessorFactory()} - * @param return type of {@link FrameProcessor} created by the manager - * @param result type of {@link ProcessorManager#result()} - * @param type of {@link WorkOrder#getExtraInfo()} - */ - private , ProcessorReturnType, ManagerReturnType, ExtraInfoType> void makeAndRunWorkProcessors() - throws IOException - { - if (workResultAndOutputChannels != null) { - throw new ISE("workResultAndOutputChannels already set"); - } - - @SuppressWarnings("unchecked") - final FactoryType processorFactory = (FactoryType) kernel.getStageDefinition().getProcessorFactory(); - - @SuppressWarnings("unchecked") - final ProcessorsAndChannels processors = - processorFactory.makeProcessors( - kernel.getStageDefinition(), - kernel.getWorkOrder().getWorkerNumber(), - kernel.getWorkOrder().getInputs(), - inputSliceReader, - (ExtraInfoType) kernel.getWorkOrder().getExtraInfo(), - workOutputChannelFactory, - frameContext, - parallelism, - counterTracker, - e -> warningPublisher.publishException(kernel.getStageDefinition().getStageNumber(), e), - isRemoveNullBytes - ); - - final ProcessorManager processorManager = processors.getProcessorManager(); - - final int maxOutstandingProcessors; - - if (processors.getOutputChannels().getAllChannels().isEmpty()) { - // No output channels: run up to "parallelism" processors at once. - maxOutstandingProcessors = Math.max(1, parallelism); - } else { - // If there are output channels, that acts as a ceiling on the number of processors that can run at once. - maxOutstandingProcessors = - Math.max(1, Math.min(parallelism, processors.getOutputChannels().getAllChannels().size())); - } - - final ListenableFuture workResultFuture = exec.runAllFully( - processorManager, - maxOutstandingProcessors, - processorBouncer, - cancellationId - ); - - workResultAndOutputChannels = new ResultAndChannels<>(workResultFuture, processors.getOutputChannels()); - } - - private void makeAndRunShuffleProcessors() - { - if (shuffleOutputChannelsFuture != null) { - throw new ISE("shuffleOutputChannelsFuture already set"); - } - - final ShuffleSpec shuffleSpec = kernel.getWorkOrder().getStageDefinition().getShuffleSpec(); - - final ShufflePipelineBuilder shufflePipeline = new ShufflePipelineBuilder( - kernel, - counterTracker, - exec, - cancellationId, - frameContext - ); - - shufflePipeline.initialize(workResultAndOutputChannels); - - switch (shuffleSpec.kind()) { - case MIX: - shufflePipeline.mix(shuffleOutputChannelFactory); - break; - - case HASH: - shufflePipeline.hashPartition(shuffleOutputChannelFactory); - break; - - case HASH_LOCAL_SORT: - final OutputChannelFactory hashOutputChannelFactory; - - if (shuffleSpec.partitionCount() == 1) { - // Single partition; no need to write temporary files. - hashOutputChannelFactory = - new BlockingQueueOutputChannelFactory(frameContext.memoryParameters().getStandardFrameSize()); - } else { - // Multi-partition; write temporary files and then sort each one file-by-file. - hashOutputChannelFactory = - new FileOutputChannelFactory( - context.tempDir(kernel.getStageDefinition().getStageNumber(), "hash-parts"), - frameContext.memoryParameters().getStandardFrameSize(), - null - ); - } - - shufflePipeline.hashPartition(hashOutputChannelFactory); - shufflePipeline.localSort(shuffleOutputChannelFactory); - break; - - case GLOBAL_SORT: - shufflePipeline.gatherResultKeyStatisticsIfNeeded(); - shufflePipeline.globalSort(shuffleOutputChannelFactory, makeGlobalSortPartitionBoundariesFuture()); - break; - - default: - throw new UOE("Cannot handle shuffle kind [%s]", shuffleSpec.kind()); - } - - shuffleOutputChannelsFuture = shufflePipeline.build(); - } - - private ListenableFuture makeGlobalSortPartitionBoundariesFuture() - { - if (kernel.getStageDefinition().mustGatherResultKeyStatistics()) { - if (stagePartitionBoundariesFuture != null) { - throw new ISE("Cannot call 'makeGlobalSortPartitionBoundariesFuture' twice"); - } - - return (stagePartitionBoundariesFuture = SettableFuture.create()); - } else { - return Futures.immediateFuture(kernel.getResultPartitionBoundaries()); - } - } - - private void setUpCompletionCallbacks(boolean isFinalStage) - { - final StageDefinition stageDef = kernel.getStageDefinition(); - - Futures.addCallback( - Futures.allAsList( - Arrays.asList( - workResultAndOutputChannels.getResultFuture(), - shuffleOutputChannelsFuture - ) - ), - new FutureCallback>() - { - @Override - public void onSuccess(final List workerResultAndOutputChannelsResolved) - { - final Object resultObject = workerResultAndOutputChannelsResolved.get(0); - final OutputChannels outputChannels = (OutputChannels) workerResultAndOutputChannelsResolved.get(1); - - for (OutputChannel channel : outputChannels.getAllChannels()) { - try { - stageOutputs.computeIfAbsent(stageDef.getId(), ignored1 -> new ConcurrentHashMap<>()) - .computeIfAbsent(channel.getPartitionNumber(), ignored2 -> channel.getReadableChannel()); - } - catch (Exception e) { - kernelManipulationQueue.add(holder -> { - throw new RE(e, "Worker completion callback error for stage [%s]", stageDef.getId()); - }); - - // Don't make the "setResultsComplete" call below. - return; - } - } - - // Once the outputs channels have been resolved and are ready for reading, write success file, if - // using durable storage. - writeDurableStorageSuccessFileIfNeeded(stageDef.getStageNumber(), isFinalStage); - - kernelManipulationQueue.add(holder -> holder.getStageKernelMap() - .get(stageDef.getId()) - .setResultsComplete(resultObject)); - } - - @Override - public void onFailure(final Throwable t) - { - kernelManipulationQueue.add( - kernelHolder -> - kernelHolder.getStageKernelMap().get(stageDef.getId()).fail(t) - ); - } - }, - MoreExecutors.directExecutor() - ); - } - - /** - * Write {@link DurableStorageUtils#SUCCESS_MARKER_FILENAME} for a particular stage, if durable storage is enabled. - */ - private void writeDurableStorageSuccessFileIfNeeded(final int stageNumber, boolean isFinalStage) - { - final DurableStorageOutputChannelFactory durableStorageOutputChannelFactory; - if (durableStageStorageEnabled || (isFinalStage - && MSQSelectDestination.DURABLESTORAGE.equals(selectDestination))) { - durableStorageOutputChannelFactory = DurableStorageOutputChannelFactory.createStandardImplementation( - task.getControllerTaskId(), - task().getWorkerNumber(), - stageNumber, - task().getId(), - frameContext.memoryParameters().getStandardFrameSize(), - MSQTasks.makeStorageConnector(context.injector()), - context.tempDir(), - (isFinalStage && MSQSelectDestination.DURABLESTORAGE.equals(selectDestination)) - ); - } else { - return; - } - try { - durableStorageOutputChannelFactory.createSuccessFile(task.getId()); - } - catch (IOException e) { - throw new ISE( - e, - "Unable to create the success file [%s] at the location [%s]", - DurableStorageUtils.SUCCESS_MARKER_FILENAME, - durableStorageOutputChannelFactory.getSuccessFilePath() - ); - } - } - } - - /** - * Helper for {@link RunWorkOrder#makeAndRunShuffleProcessors()}. Builds a {@link FrameProcessor} pipeline to - * handle the shuffle. - */ - private class ShufflePipelineBuilder - { - private final WorkerStageKernel kernel; - private final CounterTracker counterTracker; - private final FrameProcessorExecutor exec; - private final String cancellationId; - private final FrameContext frameContext; - - // Current state of the pipeline. It's a future to allow pipeline construction to be deferred if necessary. - private ListenableFuture> pipelineFuture; - - public ShufflePipelineBuilder( - final WorkerStageKernel kernel, - final CounterTracker counterTracker, - final FrameProcessorExecutor exec, - final String cancellationId, - final FrameContext frameContext - ) - { - this.kernel = kernel; - this.counterTracker = counterTracker; - this.exec = exec; - this.cancellationId = cancellationId; - this.frameContext = frameContext; - } - - /** - * Start the pipeline with the outputs of the main processor. - */ - public void initialize(final ResultAndChannels resultAndChannels) - { - if (pipelineFuture != null) { - throw new ISE("already initialized"); - } - - pipelineFuture = Futures.immediateFuture(resultAndChannels); - } - - /** - * Add {@link FrameChannelMixer}, which mixes all current outputs into a single channel from the provided factory. - */ - public void mix(final OutputChannelFactory outputChannelFactory) - { - // No sorting or statistics gathering, just combining all outputs into one big partition. Use a mixer to get - // everything into one file. Note: even if there is only one output channel, we'll run it through the mixer - // anyway, to ensure the data gets written to a file. (httpGetChannelData requires files.) - - push( - resultAndChannels -> { - final OutputChannel outputChannel = outputChannelFactory.openChannel(0); - - final FrameChannelMixer mixer = - new FrameChannelMixer( - resultAndChannels.getOutputChannels().getAllReadableChannels(), - outputChannel.getWritableChannel() - ); - - return new ResultAndChannels<>( - exec.runFully(mixer, cancellationId), - OutputChannels.wrap(Collections.singletonList(outputChannel.readOnly())) - ); - } - ); - } - - /** - * Add {@link KeyStatisticsCollectionProcessor} if {@link StageDefinition#mustGatherResultKeyStatistics()}. - */ - public void gatherResultKeyStatisticsIfNeeded() - { - push( - resultAndChannels -> { - final StageDefinition stageDefinition = kernel.getStageDefinition(); - final OutputChannels channels = resultAndChannels.getOutputChannels(); - - if (channels.getAllChannels().isEmpty()) { - // No data coming out of this processor. Report empty statistics, if the kernel is expecting statistics. - if (stageDefinition.mustGatherResultKeyStatistics()) { - kernelManipulationQueue.add( - holder -> - holder.getStageKernelMap().get(stageDefinition.getId()) - .setResultKeyStatisticsSnapshot(ClusterByStatisticsSnapshot.empty()) - ); - } - - // Generate one empty channel so the SuperSorter has something to do. - final BlockingQueueFrameChannel channel = BlockingQueueFrameChannel.minimal(); - channel.writable().close(); - - final OutputChannel outputChannel = OutputChannel.readOnly( - channel.readable(), - FrameWithPartition.NO_PARTITION - ); - - return new ResultAndChannels<>( - Futures.immediateFuture(null), - OutputChannels.wrap(Collections.singletonList(outputChannel)) - ); - } else if (stageDefinition.mustGatherResultKeyStatistics()) { - return gatherResultKeyStatistics(channels); - } else { - return resultAndChannels; - } - } - ); - } - - /** - * Add a {@link SuperSorter} using {@link StageDefinition#getSortKey()} and partition boundaries - * from {@code partitionBoundariesFuture}. - */ - public void globalSort( - final OutputChannelFactory outputChannelFactory, - final ListenableFuture partitionBoundariesFuture - ) - { - pushAsync( - resultAndChannels -> { - final StageDefinition stageDefinition = kernel.getStageDefinition(); - - final File sorterTmpDir = context.tempDir(stageDefinition.getStageNumber(), "super-sort"); - FileUtils.mkdirp(sorterTmpDir); - if (!sorterTmpDir.isDirectory()) { - throw new IOException("Cannot create directory: " + sorterTmpDir); - } - - final WorkerMemoryParameters memoryParameters = frameContext.memoryParameters(); - final SuperSorter sorter = new SuperSorter( - resultAndChannels.getOutputChannels().getAllReadableChannels(), - stageDefinition.getFrameReader(), - stageDefinition.getSortKey(), - partitionBoundariesFuture, - exec, - outputChannelFactory, - makeSuperSorterIntermediateOutputChannelFactory( - frameContext, - stageDefinition.getStageNumber(), - sorterTmpDir - ), - memoryParameters.getSuperSorterMaxActiveProcessors(), - memoryParameters.getSuperSorterMaxChannelsPerProcessor(), - -1, - cancellationId, - counterTracker.sortProgress(), - isRemoveNullBytes - ); - - return FutureUtils.transform( - sorter.run(), - sortedChannels -> new ResultAndChannels<>(Futures.immediateFuture(null), sortedChannels) - ); - } - ); - } - - /** - * Add a {@link FrameChannelHashPartitioner} using {@link StageDefinition#getSortKey()}. - */ - public void hashPartition(final OutputChannelFactory outputChannelFactory) - { - pushAsync( - resultAndChannels -> { - final ShuffleSpec shuffleSpec = kernel.getStageDefinition().getShuffleSpec(); - final int partitions = shuffleSpec.partitionCount(); - - final List outputChannels = new ArrayList<>(); - - for (int i = 0; i < partitions; i++) { - outputChannels.add(outputChannelFactory.openChannel(i)); - } - - final FrameChannelHashPartitioner partitioner = new FrameChannelHashPartitioner( - resultAndChannels.getOutputChannels().getAllReadableChannels(), - outputChannels.stream().map(OutputChannel::getWritableChannel).collect(Collectors.toList()), - kernel.getStageDefinition().getFrameReader(), - kernel.getStageDefinition().getClusterBy().getColumns().size(), - FrameWriters.makeRowBasedFrameWriterFactory( - new ArenaMemoryAllocatorFactory(frameContext.memoryParameters().getStandardFrameSize()), - kernel.getStageDefinition().getSignature(), - kernel.getStageDefinition().getSortKey(), - isRemoveNullBytes - ) - ); - - final ListenableFuture partitionerFuture = exec.runFully(partitioner, cancellationId); - - final ResultAndChannels retVal = - new ResultAndChannels<>(partitionerFuture, OutputChannels.wrap(outputChannels)); - - if (retVal.getOutputChannels().areReadableChannelsReady()) { - return Futures.immediateFuture(retVal); - } else { - return FutureUtils.transform(partitionerFuture, ignored -> retVal); - } - } - ); - } - - /** - * Add a sequence of {@link SuperSorter}, operating on each current output channel in order, one at a time. - */ - public void localSort(final OutputChannelFactory outputChannelFactory) - { - pushAsync( - resultAndChannels -> { - final StageDefinition stageDefinition = kernel.getStageDefinition(); - final OutputChannels channels = resultAndChannels.getOutputChannels(); - final List> sortedChannelFutures = new ArrayList<>(); - - ListenableFuture nextFuture = Futures.immediateFuture(null); - - for (final OutputChannel channel : channels.getAllChannels()) { - final File sorterTmpDir = context.tempDir( - stageDefinition.getStageNumber(), - StringUtils.format("hash-parts-super-sort-%06d", channel.getPartitionNumber()) - ); - - FileUtils.mkdirp(sorterTmpDir); - - // SuperSorter will try to write to output partition zero; we remap it to the correct partition number. - final OutputChannelFactory partitionOverrideOutputChannelFactory = new OutputChannelFactory() - { - @Override - public OutputChannel openChannel(int expectedZero) throws IOException - { - if (expectedZero != 0) { - throw new ISE("Unexpected part [%s]", expectedZero); - } - - return outputChannelFactory.openChannel(channel.getPartitionNumber()); - } - - @Override - public PartitionedOutputChannel openPartitionedChannel(String name, boolean deleteAfterRead) - { - throw new UnsupportedOperationException(); - } - - @Override - public OutputChannel openNilChannel(int expectedZero) - { - if (expectedZero != 0) { - throw new ISE("Unexpected part [%s]", expectedZero); - } - - return outputChannelFactory.openNilChannel(channel.getPartitionNumber()); - } - }; - - // Chain futures so we only sort one partition at a time. - nextFuture = Futures.transformAsync( - nextFuture, - (AsyncFunction) ignored -> { - final SuperSorter sorter = new SuperSorter( - Collections.singletonList(channel.getReadableChannel()), - stageDefinition.getFrameReader(), - stageDefinition.getSortKey(), - Futures.immediateFuture(ClusterByPartitions.oneUniversalPartition()), - exec, - partitionOverrideOutputChannelFactory, - makeSuperSorterIntermediateOutputChannelFactory( - frameContext, - stageDefinition.getStageNumber(), - sorterTmpDir - ), - 1, - 2, - -1, - cancellationId, - - // Tracker is not actually tracked, since it doesn't quite fit into the way we report counters. - // There's a single SuperSorterProgressTrackerCounter per worker, but workers that do local - // sorting have a SuperSorter per partition. - new SuperSorterProgressTracker(), - isRemoveNullBytes - ); - - return FutureUtils.transform(sorter.run(), r -> Iterables.getOnlyElement(r.getAllChannels())); - }, - MoreExecutors.directExecutor() - ); - - sortedChannelFutures.add(nextFuture); - } - - return FutureUtils.transform( - Futures.allAsList(sortedChannelFutures), - sortedChannels -> new ResultAndChannels<>( - Futures.immediateFuture(null), - OutputChannels.wrap(sortedChannels) - ) - ); - } - ); - } - - /** - * Return the (future) output channels for this pipeline. - */ - public ListenableFuture build() - { - if (pipelineFuture == null) { - throw new ISE("Not initialized"); - } - - return Futures.transformAsync( - pipelineFuture, - (AsyncFunction, OutputChannels>) resultAndChannels -> - Futures.transform( - resultAndChannels.getResultFuture(), - (Function) input -> { - sanityCheckOutputChannels(resultAndChannels.getOutputChannels()); - return resultAndChannels.getOutputChannels(); - }, - MoreExecutors.directExecutor() - ), - MoreExecutors.directExecutor() - ); - } - - /** - * Adds {@link KeyStatisticsCollectionProcessor}. Called by {@link #gatherResultKeyStatisticsIfNeeded()}. - */ - private ResultAndChannels gatherResultKeyStatistics(final OutputChannels channels) - { - final StageDefinition stageDefinition = kernel.getStageDefinition(); - final List retVal = new ArrayList<>(); - final List processors = new ArrayList<>(); - - for (final OutputChannel outputChannel : channels.getAllChannels()) { - final BlockingQueueFrameChannel channel = BlockingQueueFrameChannel.minimal(); - retVal.add(OutputChannel.readOnly(channel.readable(), outputChannel.getPartitionNumber())); - - processors.add( - new KeyStatisticsCollectionProcessor( - outputChannel.getReadableChannel(), - channel.writable(), - stageDefinition.getFrameReader(), - stageDefinition.getClusterBy(), - stageDefinition.createResultKeyStatisticsCollector( - frameContext.memoryParameters().getPartitionStatisticsMaxRetainedBytes() - ) - ) - ); - } - - final ListenableFuture clusterByStatisticsCollectorFuture = - exec.runAllFully( - ProcessorManagers.of(processors) - .withAccumulation( - stageDefinition.createResultKeyStatisticsCollector( - frameContext.memoryParameters().getPartitionStatisticsMaxRetainedBytes() - ), - ClusterByStatisticsCollector::addAll - ), - // Run all processors simultaneously. They are lightweight and this keeps things moving. - processors.size(), - Bouncer.unlimited(), - cancellationId - ); - - Futures.addCallback( - clusterByStatisticsCollectorFuture, - new FutureCallback() - { - @Override - public void onSuccess(final ClusterByStatisticsCollector result) - { - result.logSketches(); - kernelManipulationQueue.add( - holder -> - holder.getStageKernelMap().get(stageDefinition.getId()) - .setResultKeyStatisticsSnapshot(result.snapshot()) - ); - } - - @Override - public void onFailure(Throwable t) - { - kernelManipulationQueue.add( - holder -> { - log.noStackTrace() - .warn(t, "Failed to gather clusterBy statistics for stage [%s]", stageDefinition.getId()); - holder.getStageKernelMap().get(stageDefinition.getId()).fail(t); - } - ); - } - }, - MoreExecutors.directExecutor() - ); - - return new ResultAndChannels<>( - clusterByStatisticsCollectorFuture, - OutputChannels.wrap(retVal) - ); - } - - /** - * Update the {@link #pipelineFuture}. - */ - private void push(final ExceptionalFunction, ResultAndChannels> fn) - { - pushAsync( - channels -> - Futures.immediateFuture(fn.apply(channels)) - ); - } - - /** - * Update the {@link #pipelineFuture} asynchronously. - */ - private void pushAsync(final ExceptionalFunction, ListenableFuture>> fn) - { - if (pipelineFuture == null) { - throw new ISE("Not initialized"); - } - - pipelineFuture = FutureUtils.transform( - Futures.transformAsync( - pipelineFuture, - new AsyncFunction, ResultAndChannels>() - { - @Override - public ListenableFuture> apply(ResultAndChannels t) throws Exception - { - return fn.apply(t); - } - }, - MoreExecutors.directExecutor() - ), - resultAndChannels -> new ResultAndChannels<>( - resultAndChannels.getResultFuture(), - resultAndChannels.getOutputChannels().readOnly() - ) - ); - } - - /** - * Verifies there is exactly one channel per partition. - */ - private void sanityCheckOutputChannels(final OutputChannels outputChannels) - { - for (int partitionNumber : outputChannels.getPartitionNumbers()) { - final List outputChannelsForPartition = - outputChannels.getChannelsForPartition(partitionNumber); - - Preconditions.checkState(partitionNumber >= 0, "Expected partitionNumber >= 0, but got [%s]", partitionNumber); - Preconditions.checkState( - outputChannelsForPartition.size() == 1, - "Expected one channel for partition [%s], but got [%s]", - partitionNumber, - outputChannelsForPartition.size() - ); - } - } - } - - private class KernelHolder - { private boolean done = false; - public Map getStageKernelMap() + private KernelHolders(final WorkerContext workerContext, final ControllerClient controllerClient) { - return stageKernelMap; + this.workerContext = workerContext; + this.controllerClient = controllerClient; } + public static KernelHolders create(final WorkerContext workerContext, final Closer closer) + { + return closer.register(new KernelHolders(workerContext, closer.register(workerContext.makeControllerClient()))); + } + + /** + * Add a {@link WorkerStageKernel} to this holder. Also creates a {@link ControllerClient} for the query ID + * if one does not yet exist. Does nothing if a kernel with the same {@link StageId} is already being tracked. + */ + public void addKernel(final WorkerStageKernel kernel) + { + final StageId stageId = verifyQueryId(kernel.getWorkOrder().getStageDefinition().getId()); + + if (holderMap.putIfAbsent(stageId.getStageNumber(), new KernelHolder(kernel)) != null) { + // Already added. Do nothing. + } + } + + /** + * Called when processing for a stage is complete. Releases processing resources associated with the stage, i.e., + * those that are part of {@link KernelHolder#processorCloser}. + * + * Does not release results-fetching resources, i.e., does not release {@link KernelHolder#resultsCloser}. Those + * resources are released on {@link #removeKernel(StageId)} only. + */ + public void finishProcessing(final StageId stageId) + { + final KernelHolder kernel = holderMap.get(verifyQueryId(stageId).getStageNumber()); + + if (kernel != null) { + try { + kernel.processorCloser.close(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + } + + /** + * Remove the {@link WorkerStageKernel} for a given {@link StageId} from this holder. Closes all the associated + * {@link Closeable}. Removes and closes the {@link ControllerClient} for this query ID, if there are no longer + * any active work orders for that query ID + * + * @throws IllegalStateException if there is no active kernel for this stage + */ + public void removeKernel(final StageId stageId) + { + final KernelHolder removed = holderMap.remove(verifyQueryId(stageId).getStageNumber()); + + if (removed == null) { + throw new ISE("No kernel for stage[%s]", stageId); + } + + try { + removed.processorCloser.close(); + removed.resultsCloser.close(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + /** + * Returns all currently-active kernel holders. + */ + public Iterable getAllKernelHolders() + { + return holderMap.values(); + } + + /** + * Returns all currently-active kernels. + */ + public Iterable getAllKernels() + { + return Iterables.transform(holderMap.values(), holder -> holder.kernel); + } + + /** + * Returns the number of kernels that are in running states, where {@link WorkerStagePhase#isRunning()}. + */ + public int runningKernelCount() + { + int retVal = 0; + for (final KernelHolder holder : holderMap.values()) { + if (holder.kernel.getPhase().isRunning()) { + retVal++; + } + } + + return retVal; + } + + /** + * Return the kernel for a particular {@link StageId}. + * + * @return kernel, or null if there is no active kernel for this stage + */ + @Nullable + public WorkerStageKernel getKernelFor(final StageId stageId) + { + final KernelHolder holder = holderMap.get(verifyQueryId(stageId).getStageNumber()); + if (holder != null) { + return holder.kernel; + } else { + return null; + } + } + + /** + * Retrieves the {@link ControllerClient}, which is shared across all {@link WorkOrder} for this worker. + */ + public ControllerClient getControllerClient() + { + return controllerClient; + } + + /** + * Remove all {@link WorkerStageKernel} and close all {@link ControllerClient}. + */ + @Override + public void close() + { + for (final int stageNumber : ImmutableList.copyOf(holderMap.keySet())) { + final StageId stageId = new StageId(workerContext.queryId(), stageNumber); + + try { + removeKernel(stageId); + } + catch (Exception e) { + log.warn(e, "Failed to remove kernel for stage[%s].", stageId); + } + } + } + + /** + * Check whether {@link #setDone()} has been called. + */ public boolean isDone() { return done; } + /** + * Mark the holder as "done", signaling to the main loop that it should clean up and exit as soon as possible. + */ public void setDone() { this.done = true; } - } - private static class ResultAndChannels - { - private final ListenableFuture resultFuture; - private final OutputChannels outputChannels; - - public ResultAndChannels( - ListenableFuture resultFuture, - OutputChannels outputChannels - ) + private StageId verifyQueryId(final StageId stageId) { - this.resultFuture = resultFuture; - this.outputChannels = outputChannels; - } + if (!stageId.getQueryId().equals(workerContext.queryId())) { + throw new ISE("Unexpected queryId[%s], expected queryId[%s]", stageId.getQueryId(), workerContext.queryId()); + } - public ListenableFuture getResultFuture() - { - return resultFuture; - } - - public OutputChannels getOutputChannels() - { - return outputChannels; + return stageId; } } - private interface ExceptionalFunction + /** + * Holder for a single {@link WorkerStageKernel} and associated items, contained within {@link KernelHolders}. + */ + private static class KernelHolder { - R apply(T t) throws Exception; + private final WorkerStageKernel kernel; + private final Closer processorCloser; + private final Closer resultsCloser; + private SettableFuture partitionBoundariesFuture; + + public KernelHolder(WorkerStageKernel kernel) + { + this.kernel = kernel; + this.processorCloser = Closer.create(); + this.resultsCloser = Closer.create(); + } } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerMemoryParameters.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerMemoryParameters.java index b36b1b4155a..aeaae030e61 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerMemoryParameters.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerMemoryParameters.java @@ -168,29 +168,14 @@ public class WorkerMemoryParameters this.partitionStatisticsMaxRetainedBytes = partitionStatisticsMaxRetainedBytes; } - /** - * Create a production instance for {@link org.apache.druid.msq.indexing.MSQControllerTask}. - */ - public static WorkerMemoryParameters createProductionInstanceForController(final Injector injector) - { - long totalLookupFootprint = computeTotalLookupFootprint(injector); - return createInstance( - Runtime.getRuntime().maxMemory(), - computeNumWorkersInJvm(injector), - computeNumProcessorsInJvm(injector), - 0, - 0, - totalLookupFootprint - ); - } - /** * Create a production instance for {@link org.apache.druid.msq.indexing.MSQWorkerTask}. */ public static WorkerMemoryParameters createProductionInstanceForWorker( final Injector injector, final QueryDefinition queryDef, - final int stageNumber + final int stageNumber, + final int maxConcurrentStages ) { final StageDefinition stageDef = queryDef.getStageDefinition(stageNumber); @@ -212,6 +197,7 @@ public class WorkerMemoryParameters Runtime.getRuntime().maxMemory(), computeNumWorkersInJvm(injector), computeNumProcessorsInJvm(injector), + maxConcurrentStages, numInputWorkers, numHashOutputPartitions, totalLookupFootprint @@ -228,6 +214,7 @@ public class WorkerMemoryParameters * @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 maxConcurrentStages maximum number of concurrent stages per worker. * @param numInputWorkers total number of workers across all input stages. * @param numHashOutputPartitions total number of output partitions, if using hash partitioning; zero if not using * hash partitioning. @@ -237,6 +224,7 @@ public class WorkerMemoryParameters final long maxMemoryInJvm, final int numWorkersInJvm, final int numProcessingThreadsInJvm, + final int maxConcurrentStages, final int numInputWorkers, final int numHashOutputPartitions, final long totalLookupFootprint @@ -257,7 +245,8 @@ public class WorkerMemoryParameters ); final long usableMemoryInJvm = computeUsableMemoryInJvm(maxMemoryInJvm, totalLookupFootprint); final long workerMemory = memoryPerWorker(usableMemoryInJvm, numWorkersInJvm); - final long bundleMemory = memoryPerBundle(usableMemoryInJvm, numWorkersInJvm, numProcessingThreadsInJvm); + final long bundleMemory = + memoryPerBundle(usableMemoryInJvm, numWorkersInJvm, numProcessingThreadsInJvm) / maxConcurrentStages; final long bundleMemoryForInputChannels = memoryNeededForInputChannels(numInputWorkers); final long bundleMemoryForHashPartitioning = memoryNeededForHashPartitioning(numHashOutputPartitions); final long bundleMemoryForProcessing = @@ -268,6 +257,7 @@ public class WorkerMemoryParameters usableMemoryInJvm, numWorkersInJvm, numProcessingThreadsInJvm, + maxConcurrentStages, numHashOutputPartitions ); @@ -281,12 +271,14 @@ public class WorkerMemoryParameters estimateUsableMemory( numWorkersInJvm, numProcessingThreadsInJvm, - PROCESSING_MINIMUM_BYTES + BUFFER_BYTES_FOR_ESTIMATION + bundleMemoryForInputChannels + PROCESSING_MINIMUM_BYTES + BUFFER_BYTES_FOR_ESTIMATION + bundleMemoryForInputChannels, + maxConcurrentStages ), totalLookupFootprint), maxMemoryInJvm, usableMemoryInJvm, numWorkersInJvm, - numProcessingThreadsInJvm + numProcessingThreadsInJvm, + maxConcurrentStages ) ); } @@ -301,14 +293,16 @@ public class WorkerMemoryParameters calculateSuggestedMinMemoryFromUsableMemory( estimateUsableMemory( numWorkersInJvm, - (MIN_SUPER_SORTER_FRAMES + BUFFER_BYTES_FOR_ESTIMATION) * LARGE_FRAME_SIZE + (MIN_SUPER_SORTER_FRAMES + BUFFER_BYTES_FOR_ESTIMATION) * LARGE_FRAME_SIZE, + maxConcurrentStages ), totalLookupFootprint ), maxMemoryInJvm, usableMemoryInJvm, numWorkersInJvm, - numProcessingThreadsInJvm + numProcessingThreadsInJvm, + maxConcurrentStages ) ); } @@ -338,12 +332,14 @@ public class WorkerMemoryParameters estimateUsableMemory( numWorkersInJvm, numProcessingThreadsInJvm, - PROCESSING_MINIMUM_BYTES + BUFFER_BYTES_FOR_ESTIMATION + bundleMemoryForInputChannels + PROCESSING_MINIMUM_BYTES + BUFFER_BYTES_FOR_ESTIMATION + bundleMemoryForInputChannels, + maxConcurrentStages ), totalLookupFootprint), maxMemoryInJvm, usableMemoryInJvm, numWorkersInJvm, - numProcessingThreadsInJvm + numProcessingThreadsInJvm, + maxConcurrentStages ) ); } @@ -352,7 +348,9 @@ public class WorkerMemoryParameters bundleMemoryForProcessing, superSorterMaxActiveProcessors, superSorterMaxChannelsPerProcessor, - Ints.checkedCast(workerMemory) // 100% of worker memory is devoted to partition statistics + + // 100% of worker memory is devoted to partition statistics + Ints.checkedCast(workerMemory / maxConcurrentStages) ); } @@ -459,18 +457,19 @@ public class WorkerMemoryParameters final long usableMemoryInJvm, final int numWorkersInJvm, final int numProcessingThreadsInJvm, + final int maxConcurrentStages, final int numHashOutputPartitions ) { final long bundleMemory = memoryPerBundle(usableMemoryInJvm, numWorkersInJvm, numProcessingThreadsInJvm); - // Compute number of workers that gives us PROCESSING_MINIMUM_BYTES of memory per bundle, while accounting for - // memoryNeededForInputChannels + memoryNeededForHashPartitioning. + // Compute number of workers that gives us PROCESSING_MINIMUM_BYTES of memory per bundle per concurrent stage, while + // accounting for memoryNeededForInputChannels + memoryNeededForHashPartitioning. final int isHashing = numHashOutputPartitions > 0 ? 1 : 0; - return Math.max( - 0, - Ints.checkedCast((bundleMemory - PROCESSING_MINIMUM_BYTES) / ((long) STANDARD_FRAME_SIZE * (1 + isHashing)) - 1) - ); + final long bundleMemoryPerStage = bundleMemory / maxConcurrentStages; + final long maxWorkers = + (bundleMemoryPerStage - PROCESSING_MINIMUM_BYTES) / ((long) STANDARD_FRAME_SIZE * (1 + isHashing)) - 1; + return Math.max(0, Ints.checkedCast(maxWorkers)); } /** @@ -528,7 +527,8 @@ public class WorkerMemoryParameters } /** - * Compute the memory allocated to each processing bundle. Any computation changes done to this method should also be done in its corresponding method {@link WorkerMemoryParameters#estimateUsableMemory(int, int, long)} + * Compute the memory allocated to each processing bundle. Any computation changes done to this method should also be + * done in its corresponding method {@link WorkerMemoryParameters#estimateUsableMemory} */ private static long memoryPerBundle( final long usableMemoryInJvm, @@ -536,6 +536,8 @@ public class WorkerMemoryParameters final int numProcessingThreadsInJvm ) { + // One bundle per worker + one per processor. The worker bundles are used for sorting (SuperSorter) and the + // processing bundles are used for reading input and doing per-partition processing. final int bundleCount = numWorkersInJvm + numProcessingThreadsInJvm; // Need to subtract memoryForWorkers off the top of usableMemoryInJvm, since this is reserved for @@ -553,24 +555,28 @@ public class WorkerMemoryParameters private static long estimateUsableMemory( final int numWorkersInJvm, final int numProcessingThreadsInJvm, - final long estimatedEachBundleMemory + final long estimatedEachBundleMemory, + final int maxConcurrentStages ) { final int bundleCount = numWorkersInJvm + numProcessingThreadsInJvm; - return estimateUsableMemory(numWorkersInJvm, estimatedEachBundleMemory * bundleCount); - + return estimateUsableMemory(numWorkersInJvm, estimatedEachBundleMemory * bundleCount, maxConcurrentStages); } /** * Add overheads to the estimated bundle memoery for all the workers. Checkout {@link WorkerMemoryParameters#memoryPerWorker(long, int)} * for the overhead calculation outside the processing bundles. */ - private static long estimateUsableMemory(final int numWorkersInJvm, final long estimatedTotalBundleMemory) + private static long estimateUsableMemory( + final int numWorkersInJvm, + final long estimatedTotalBundleMemory, + final int maxConcurrentStages + ) { - // Currently, we only add the partition stats overhead since it will be the single largest overhead per worker. final long estimateStatOverHeadPerWorker = PARTITION_STATS_MEMORY_MAX_BYTES; - return estimatedTotalBundleMemory + (estimateStatOverHeadPerWorker * numWorkersInJvm); + final long requiredUsableMemory = estimatedTotalBundleMemory + (estimateStatOverHeadPerWorker * numWorkersInJvm); + return requiredUsableMemory * maxConcurrentStages; } private static long memoryNeededForHashPartitioning(final int numOutputPartitions) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerStorageParameters.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerStorageParameters.java index 59576ec90bf..53e12dd2ab4 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerStorageParameters.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerStorageParameters.java @@ -70,11 +70,13 @@ public class WorkerStorageParameters public static WorkerStorageParameters createProductionInstance( final Injector injector, - final boolean isIntermediateSuperSorterStorageEnabled + final OutputChannelMode outputChannelMode ) { long tmpStorageBytesPerTask = injector.getInstance(TaskConfig.class).getTmpStorageBytesPerTask(); - return createInstance(tmpStorageBytesPerTask, isIntermediateSuperSorterStorageEnabled); + + // If durable storage is enabled, then super sorter intermediate storage should be enabled as well. + return createInstance(tmpStorageBytesPerTask, outputChannelMode.isDurable()); } @VisibleForTesting diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java index e0de5bdc27e..fb6e4a0079f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java @@ -20,9 +20,13 @@ package org.apache.druid.msq.indexing; import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.frame.processor.Bouncer; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.msq.exec.DataServerQueryHandlerFactory; import org.apache.druid.msq.exec.WorkerMemoryParameters; +import org.apache.druid.msq.exec.WorkerStorageParameters; import org.apache.druid.msq.kernel.FrameContext; +import org.apache.druid.msq.kernel.StageId; import org.apache.druid.msq.querykit.DataSegmentProvider; import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.segment.IndexIO; @@ -35,25 +39,31 @@ import java.io.File; public class IndexerFrameContext implements FrameContext { + private final StageId stageId; private final IndexerWorkerContext context; private final IndexIO indexIO; private final DataSegmentProvider dataSegmentProvider; private final WorkerMemoryParameters memoryParameters; + private final WorkerStorageParameters storageParameters; private final DataServerQueryHandlerFactory dataServerQueryHandlerFactory; public IndexerFrameContext( + StageId stageId, IndexerWorkerContext context, IndexIO indexIO, DataSegmentProvider dataSegmentProvider, DataServerQueryHandlerFactory dataServerQueryHandlerFactory, - WorkerMemoryParameters memoryParameters + WorkerMemoryParameters memoryParameters, + WorkerStorageParameters storageParameters ) { + this.stageId = stageId; this.context = context; this.indexIO = indexIO; this.dataSegmentProvider = dataSegmentProvider; - this.dataServerQueryHandlerFactory = dataServerQueryHandlerFactory; this.memoryParameters = memoryParameters; + this.storageParameters = storageParameters; + this.dataServerQueryHandlerFactory = dataServerQueryHandlerFactory; } @Override @@ -90,7 +100,8 @@ public class IndexerFrameContext implements FrameContext @Override public File tempDir() { - return context.tempDir(); + // No need to include query ID; each task handles a single query, so there is no ambiguity. + return new File(context.tempDir(), StringUtils.format("stage_%06d", stageId.getStageNumber())); } @Override @@ -128,4 +139,22 @@ public class IndexerFrameContext implements FrameContext { return memoryParameters; } + + @Override + public Bouncer processorBouncer() + { + return context.injector().getInstance(Bouncer.class); + } + + @Override + public WorkerStorageParameters storageParameters() + { + return storageParameters; + } + + @Override + public void close() + { + // Nothing to close. + } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerResourcePermissionMapper.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerResourcePermissionMapper.java index 30bc75282fa..2dedaf204ec 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerResourcePermissionMapper.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerResourcePermissionMapper.java @@ -52,4 +52,10 @@ public class IndexerResourcePermissionMapper implements ResourcePermissionMapper ) ); } + + @Override + public List getQueryPermissions(String queryId) + { + return getAdminPermissions(); + } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java index 1bd789df769..63358467489 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java @@ -24,9 +24,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.errorprone.annotations.concurrent.GuardedBy; import com.google.inject.Injector; import com.google.inject.Key; -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.guice.annotations.Smile; import org.apache.druid.indexing.common.SegmentCacheManagerFactory; import org.apache.druid.indexing.common.TaskToolbox; @@ -35,16 +33,21 @@ 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.DataServerQueryHandlerFactory; +import org.apache.druid.msq.exec.MemoryIntrospector; +import org.apache.druid.msq.exec.OutputChannelMode; 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.exec.WorkerStorageParameters; import org.apache.druid.msq.indexing.client.IndexerControllerClient; import org.apache.druid.msq.indexing.client.IndexerWorkerClient; import org.apache.druid.msq.indexing.client.WorkerChatHandler; import org.apache.druid.msq.kernel.FrameContext; import org.apache.druid.msq.kernel.QueryDefinition; +import org.apache.druid.msq.util.MultiStageQueryContext; +import org.apache.druid.query.QueryContext; import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.rpc.ServiceClientFactory; import org.apache.druid.rpc.ServiceLocations; @@ -67,37 +70,49 @@ public class IndexerWorkerContext implements WorkerContext private static final long FREQUENCY_CHECK_MILLIS = 1000; private static final long FREQUENCY_CHECK_JITTER = 30; + private final MSQWorkerTask task; private final TaskToolbox toolbox; private final Injector injector; + private final OverlordClient overlordClient; private final IndexIO indexIO; private final TaskDataSegmentProvider dataSegmentProvider; private final DataServerQueryHandlerFactory dataServerQueryHandlerFactory; private final ServiceClientFactory clientFactory; - - @GuardedBy("this") - private OverlordClient overlordClient; + private final MemoryIntrospector memoryIntrospector; + private final int maxConcurrentStages; @GuardedBy("this") private ServiceLocator controllerLocator; public IndexerWorkerContext( + final MSQWorkerTask task, final TaskToolbox toolbox, final Injector injector, + final OverlordClient overlordClient, final IndexIO indexIO, final TaskDataSegmentProvider dataSegmentProvider, - final DataServerQueryHandlerFactory dataServerQueryHandlerFactory, - final ServiceClientFactory clientFactory + final ServiceClientFactory clientFactory, + final MemoryIntrospector memoryIntrospector, + final DataServerQueryHandlerFactory dataServerQueryHandlerFactory ) { + this.task = task; this.toolbox = toolbox; this.injector = injector; + this.overlordClient = overlordClient; this.indexIO = indexIO; this.dataSegmentProvider = dataSegmentProvider; - this.dataServerQueryHandlerFactory = dataServerQueryHandlerFactory; this.clientFactory = clientFactory; + this.memoryIntrospector = memoryIntrospector; + this.dataServerQueryHandlerFactory = dataServerQueryHandlerFactory; + this.maxConcurrentStages = MultiStageQueryContext.getMaxConcurrentStages(QueryContext.of(task.getContext())); } - public static IndexerWorkerContext createProductionInstance(final TaskToolbox toolbox, final Injector injector) + public static IndexerWorkerContext createProductionInstance( + final MSQWorkerTask task, + final TaskToolbox toolbox, + final Injector injector + ) { final IndexIO indexIO = injector.getInstance(IndexIO.class); final SegmentCacheManager segmentCacheManager = @@ -105,28 +120,42 @@ public class IndexerWorkerContext implements WorkerContext .manufacturate(new File(toolbox.getIndexingTmpDir(), "segment-fetch")); final ServiceClientFactory serviceClientFactory = injector.getInstance(Key.get(ServiceClientFactory.class, EscalatedGlobal.class)); + final MemoryIntrospector memoryIntrospector = injector.getInstance(MemoryIntrospector.class); + final OverlordClient overlordClient = + injector.getInstance(OverlordClient.class).withRetryPolicy(StandardRetryPolicy.unlimited()); final ObjectMapper smileMapper = injector.getInstance(Key.get(ObjectMapper.class, Smile.class)); final QueryToolChestWarehouse warehouse = injector.getInstance(QueryToolChestWarehouse.class); return new IndexerWorkerContext( + task, toolbox, injector, + overlordClient, indexIO, - new TaskDataSegmentProvider( - toolbox.getCoordinatorClient(), - segmentCacheManager, - indexIO - ), + new TaskDataSegmentProvider(toolbox.getCoordinatorClient(), segmentCacheManager, indexIO), + serviceClientFactory, + memoryIntrospector, new DataServerQueryHandlerFactory( toolbox.getCoordinatorClient(), serviceClientFactory, smileMapper, warehouse - ), - serviceClientFactory + ) ); } + @Override + public String queryId() + { + return task.getControllerTaskId(); + } + + @Override + public String workerId() + { + return task.getId(); + } + public TaskToolbox toolbox() { return toolbox; @@ -147,7 +176,8 @@ public class IndexerWorkerContext implements WorkerContext @Override public void registerWorker(Worker worker, Closer closer) { - WorkerChatHandler chatHandler = new WorkerChatHandler(toolbox, worker); + final WorkerChatHandler chatHandler = + new WorkerChatHandler(worker, toolbox.getAuthorizerMapper(), task.getDataSource()); toolbox.getChatHandlerProvider().register(worker.id(), chatHandler, false); closer.register(() -> toolbox.getChatHandlerProvider().unregister(worker.id())); closer.register(() -> { @@ -161,7 +191,7 @@ public class IndexerWorkerContext implements WorkerContext // 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()); + final ServiceLocator controllerLocator = makeControllerLocator(task.getControllerTaskId()); periodicControllerCheckerExec.submit(() -> controllerCheckerRunnable(controllerLocator, worker)); } @@ -218,15 +248,21 @@ public class IndexerWorkerContext implements WorkerContext } @Override - public ControllerClient makeControllerClient(String controllerId) + public int maxConcurrentStages() { - final ServiceLocator locator = makeControllerLocator(controllerId); + return maxConcurrentStages; + } + + @Override + public ControllerClient makeControllerClient() + { + final ServiceLocator locator = makeControllerLocator(task.getControllerTaskId()); return new IndexerControllerClient( clientFactory.makeClient( - controllerId, + task.getControllerTaskId(), locator, - new SpecificTaskRetryPolicy(controllerId, StandardRetryPolicy.unlimited()) + new SpecificTaskRetryPolicy(task.getControllerTaskId(), StandardRetryPolicy.unlimited()) ), jsonMapper(), locator @@ -237,37 +273,33 @@ public class IndexerWorkerContext implements WorkerContext public WorkerClient makeWorkerClient() { // Ignore workerId parameter. The workerId is passed into each method of WorkerClient individually. - return new IndexerWorkerClient(clientFactory, makeOverlordClient(), jsonMapper()); + return new IndexerWorkerClient(clientFactory, overlordClient, jsonMapper()); } @Override - public FrameContext frameContext(QueryDefinition queryDef, int stageNumber) + public FrameContext frameContext(QueryDefinition queryDef, int stageNumber, OutputChannelMode outputChannelMode) { return new IndexerFrameContext( + queryDef.getStageDefinition(stageNumber).getId(), this, indexIO, dataSegmentProvider, dataServerQueryHandlerFactory, - WorkerMemoryParameters.createProductionInstanceForWorker(injector, queryDef, stageNumber) + WorkerMemoryParameters.createProductionInstanceForWorker(injector, queryDef, stageNumber, maxConcurrentStages), + WorkerStorageParameters.createProductionInstance(injector, outputChannelMode) ); } @Override public int threadCount() { - return processorBouncer().getMaxCount(); + return memoryIntrospector.numProcessorsInJvm(); } @Override public DruidNode selfNode() { - return injector.getInstance(Key.get(DruidNode.class, Self.class)); - } - - @Override - public Bouncer processorBouncer() - { - return injector.getInstance(Bouncer.class); + return toolbox.getDruidNode(); } @Override @@ -276,21 +308,13 @@ public class IndexerWorkerContext implements WorkerContext return dataServerQueryHandlerFactory; } - 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()); + controllerLocator = new SpecificTaskServiceLocator(controllerId, overlordClient); } return controllerLocator; } + } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTask.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTask.java index b4d18ea390e..31b03d63ba6 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTask.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTask.java @@ -33,10 +33,13 @@ 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.indexing.common.task.Tasks; +import org.apache.druid.java.util.common.logger.Logger; 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 org.apache.druid.msq.indexing.error.MSQException; +import org.apache.druid.msq.indexing.error.MSQFaultUtils; import org.apache.druid.server.security.ResourceAction; import javax.annotation.Nonnull; @@ -48,6 +51,7 @@ import java.util.Set; public class MSQWorkerTask extends AbstractTask { public static final String TYPE = "query_worker"; + private static final Logger log = new Logger(MSQWorkerTask.class); private final String controllerTaskId; private final int workerNumber; @@ -132,18 +136,25 @@ public class MSQWorkerTask extends AbstractTask } @Override - public TaskStatus runTask(final TaskToolbox toolbox) throws Exception + public TaskStatus runTask(final TaskToolbox toolbox) { - final WorkerContext context = IndexerWorkerContext.createProductionInstance(toolbox, injector); + final WorkerContext context = IndexerWorkerContext.createProductionInstance(this, toolbox, injector); worker = new WorkerImpl(this, context); - return worker.run(); + + try { + worker.run(); + return TaskStatus.success(context.workerId()); + } + catch (MSQException e) { + return TaskStatus.failure(context.workerId(), MSQFaultUtils.generateMessageWithErrorCode(e.getFault())); + } } @Override public void stopGracefully(TaskConfig taskConfig) { if (worker != null) { - worker.stopGracefully(); + worker.stop(); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/IndexerControllerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/IndexerControllerClient.java index 81303eb4384..1e31de71a8a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/IndexerControllerClient.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/IndexerControllerClient.java @@ -152,7 +152,7 @@ public class IndexerControllerClient implements ControllerClient } @Override - public List getTaskList() throws IOException + public List getWorkerIds() throws IOException { final BytesFullResponseHolder retVal = doRequest( new RequestBuilder(HttpMethod.GET, "/taskList"), diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/WorkerChatHandler.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/WorkerChatHandler.java index 70d1ab11d38..7c8b86bb9d6 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/WorkerChatHandler.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/WorkerChatHandler.java @@ -19,310 +19,25 @@ package org.apache.druid.msq.indexing.client; -import com.google.common.collect.ImmutableMap; -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.StringUtils; -import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.exec.Worker; -import org.apache.druid.msq.indexing.MSQWorkerTask; -import org.apache.druid.msq.kernel.StageId; -import org.apache.druid.msq.kernel.WorkOrder; -import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; -import org.apache.druid.msq.statistics.serde.ClusterByStatisticsSnapshotSerde; +import org.apache.druid.msq.indexing.IndexerResourcePermissionMapper; +import org.apache.druid.msq.rpc.WorkerResource; import org.apache.druid.segment.realtime.ChatHandler; -import org.apache.druid.segment.realtime.ChatHandlers; -import org.apache.druid.server.security.Action; -import org.apache.druid.utils.CloseableUtils; +import org.apache.druid.segment.realtime.ChatHandlerProvider; +import org.apache.druid.server.security.AuthorizerMapper; -import javax.annotation.Nullable; -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 +/** + * Subclass of {@link WorkerResource} that implements {@link ChatHandler}, suitable for registration + * with a {@link ChatHandlerProvider}. + */ +public class WorkerChatHandler extends WorkerResource 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 + public WorkerChatHandler( + final Worker worker, + final AuthorizerMapper authorizerMapper, + final String dataSource ) { - 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(); - } - } - - @POST - @Path("/keyStatistics/{queryId}/{stageNumber}") - @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_OCTET_STREAM}) - @Consumes(MediaType.APPLICATION_JSON) - public Response httpFetchKeyStatistics( - @PathParam("queryId") final String queryId, - @PathParam("stageNumber") final int stageNumber, - @QueryParam("sketchEncoding") @Nullable final SketchEncoding sketchEncoding, - @Context final HttpServletRequest req - ) - { - ChatHandlers.authorizationCheck(req, Action.READ, task.getDataSource(), toolbox.getAuthorizerMapper()); - ClusterByStatisticsSnapshot clusterByStatisticsSnapshot; - StageId stageId = new StageId(queryId, stageNumber); - try { - clusterByStatisticsSnapshot = worker.fetchStatisticsSnapshot(stageId); - if (SketchEncoding.OCTET_STREAM.equals(sketchEncoding)) { - return Response.status(Response.Status.ACCEPTED) - .type(MediaType.APPLICATION_OCTET_STREAM) - .entity((StreamingOutput) output -> ClusterByStatisticsSnapshotSerde.serialize(output, clusterByStatisticsSnapshot)) - .build(); - } else { - return Response.status(Response.Status.ACCEPTED) - .type(MediaType.APPLICATION_JSON) - .entity(clusterByStatisticsSnapshot) - .build(); - } - } - catch (Exception e) { - String errorMessage = StringUtils.format( - "Invalid request for key statistics for query[%s] and stage[%d]", - queryId, - stageNumber - ); - log.error(e, errorMessage); - return Response.status(Response.Status.BAD_REQUEST) - .entity(ImmutableMap.of("error", errorMessage)) - .build(); - } - } - - @POST - @Path("/keyStatisticsForTimeChunk/{queryId}/{stageNumber}/{timeChunk}") - @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_OCTET_STREAM}) - @Consumes(MediaType.APPLICATION_JSON) - public Response httpFetchKeyStatisticsWithSnapshot( - @PathParam("queryId") final String queryId, - @PathParam("stageNumber") final int stageNumber, - @PathParam("timeChunk") final long timeChunk, - @QueryParam("sketchEncoding") @Nullable final SketchEncoding sketchEncoding, - @Context final HttpServletRequest req - ) - { - ChatHandlers.authorizationCheck(req, Action.READ, task.getDataSource(), toolbox.getAuthorizerMapper()); - ClusterByStatisticsSnapshot snapshotForTimeChunk; - StageId stageId = new StageId(queryId, stageNumber); - try { - snapshotForTimeChunk = worker.fetchStatisticsSnapshotForTimeChunk(stageId, timeChunk); - if (SketchEncoding.OCTET_STREAM.equals(sketchEncoding)) { - return Response.status(Response.Status.ACCEPTED) - .type(MediaType.APPLICATION_OCTET_STREAM) - .entity((StreamingOutput) output -> ClusterByStatisticsSnapshotSerde.serialize(output, snapshotForTimeChunk)) - .build(); - } else { - return Response.status(Response.Status.ACCEPTED) - .type(MediaType.APPLICATION_JSON) - .entity(snapshotForTimeChunk) - .build(); - } - } - catch (Exception e) { - String errorMessage = StringUtils.format( - "Invalid request for key statistics for query[%s], stage[%d] and timeChunk[%d]", - queryId, - stageNumber, - timeChunk - ); - log.error(e, errorMessage); - return Response.status(Response.Status.BAD_REQUEST) - .entity(ImmutableMap.of("error", errorMessage)) - .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(); - } - - /** - * Determines the encoding of key collectors returned by {@link #httpFetchKeyStatistics} and - * {@link #httpFetchKeyStatisticsWithSnapshot}. - */ - public enum SketchEncoding - { - /** - * The key collector is encoded as a byte stream with {@link ClusterByStatisticsSnapshotSerde}. - */ - OCTET_STREAM, - /** - * The key collector is encoded as json - */ - JSON + super(worker, new IndexerResourcePermissionMapper(dataSource), authorizerMapper); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/NotEnoughMemoryFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/NotEnoughMemoryFault.java index 5c80f065eef..6f4b36da1ee 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/NotEnoughMemoryFault.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/NotEnoughMemoryFault.java @@ -20,6 +20,7 @@ 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; @@ -35,6 +36,7 @@ public class NotEnoughMemoryFault extends BaseMSQFault private final long usableMemory; private final int serverWorkers; private final int serverThreads; + private final int maxConcurrentStages; @JsonCreator public NotEnoughMemoryFault( @@ -42,19 +44,23 @@ public class NotEnoughMemoryFault extends BaseMSQFault @JsonProperty("serverMemory") final long serverMemory, @JsonProperty("usableMemory") final long usableMemory, @JsonProperty("serverWorkers") final int serverWorkers, - @JsonProperty("serverThreads") final int serverThreads + @JsonProperty("serverThreads") final int serverThreads, + @JsonProperty("maxConcurrentStages") final int maxConcurrentStages ) { super( CODE, "Not enough memory. Required at least %,d bytes. (total = %,d bytes; usable = %,d bytes; " - + "worker capacity = %,d; processing threads = %,d). Increase JVM memory with the -Xmx option" - + (serverWorkers > 1 ? " or reduce worker capacity on this server" : ""), + + "worker capacity = %,d; processing threads = %,d; concurrent stages = %,d). " + + "Increase JVM memory with the -Xmx option" + + (serverWorkers > 1 ? ", or reduce worker capacity on this server" : "") + + (maxConcurrentStages > 1 ? ", or reduce maxConcurrentStages for this query" : ""), suggestedServerMemory, serverMemory, usableMemory, serverWorkers, - serverThreads + serverThreads, + maxConcurrentStages ); this.suggestedServerMemory = suggestedServerMemory; @@ -62,6 +68,7 @@ public class NotEnoughMemoryFault extends BaseMSQFault this.usableMemory = usableMemory; this.serverWorkers = serverWorkers; this.serverThreads = serverThreads; + this.maxConcurrentStages = maxConcurrentStages; } @JsonProperty @@ -94,6 +101,13 @@ public class NotEnoughMemoryFault extends BaseMSQFault return serverThreads; } + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + public int getMaxConcurrentStages() + { + return maxConcurrentStages; + } + @Override public boolean equals(Object o) { @@ -107,12 +121,12 @@ public class NotEnoughMemoryFault extends BaseMSQFault return false; } NotEnoughMemoryFault that = (NotEnoughMemoryFault) o; - return - suggestedServerMemory == that.suggestedServerMemory - && serverMemory == that.serverMemory - && usableMemory == that.usableMemory - && serverWorkers == that.serverWorkers - && serverThreads == that.serverThreads; + return suggestedServerMemory == that.suggestedServerMemory + && serverMemory == that.serverMemory + && usableMemory == that.usableMemory + && serverWorkers == that.serverWorkers + && serverThreads == that.serverThreads + && maxConcurrentStages == that.maxConcurrentStages; } @Override @@ -124,7 +138,8 @@ public class NotEnoughMemoryFault extends BaseMSQFault serverMemory, usableMemory, serverWorkers, - serverThreads + serverThreads, + maxConcurrentStages ); } @@ -137,6 +152,7 @@ public class NotEnoughMemoryFault extends BaseMSQFault " bytes, usableMemory=" + usableMemory + " bytes, serverWorkers=" + serverWorkers + ", serverThreads=" + serverThreads + + ", maxConcurrentStages=" + maxConcurrentStages + '}'; } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSlices.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSlices.java index 028f1b5bd48..de01235447a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSlices.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSlices.java @@ -41,6 +41,22 @@ public class InputSlices // No instantiation. } + /** + * Returns all {@link StageInputSlice} from the provided list of input slices. Ignores other types of input slices. + */ + public static List allStageSlices(final List slices) + { + final List retVal = new ArrayList<>(); + + for (final InputSlice slice : slices) { + if (slice instanceof StageInputSlice) { + retVal.add((StageInputSlice) slice); + } + } + + return retVal; + } + /** * Combines all {@link StageInputSlice#getPartitions()} from the input slices that are {@link StageInputSlice}. * Ignores other types of input slices. @@ -49,10 +65,8 @@ public class InputSlices { final List partitionsList = new ArrayList<>(); - for (final InputSlice slice : slices) { - if (slice instanceof StageInputSlice) { - partitionsList.add(((StageInputSlice) slice).getPartitions()); - } + for (final StageInputSlice slice : allStageSlices(slices)) { + partitionsList.add(slice.getPartitions()); } return ReadablePartitions.combine(partitionsList); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java index 03aa7cd0fe4..4b68a3bf1b0 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java @@ -31,7 +31,7 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.InlineInputSource; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.msq.counters.ChannelCounters; import org.apache.druid.msq.counters.CounterNames; import org.apache.druid.msq.counters.CounterTracker; @@ -53,6 +53,7 @@ import org.apache.druid.segment.incremental.SimpleRowIngestionMeters; import org.apache.druid.timeline.SegmentId; import java.io.File; +import java.io.IOException; import java.util.Iterator; import java.util.List; import java.util.function.Consumer; @@ -94,7 +95,7 @@ public class ExternalInputSliceReader implements InputSliceReader externalInputSlice.getInputSources(), externalInputSlice.getInputFormat(), externalInputSlice.getSignature(), - temporaryDirectory, + new File(temporaryDirectory, String.valueOf(inputNumber)), counters.channel(CounterNames.inputChannel(inputNumber)).setTotalFiles(slice.fileCount()), counters.warnings(), warningPublisher @@ -128,9 +129,13 @@ public class ExternalInputSliceReader implements InputSliceReader ColumnsFilter.all() ); - if (!temporaryDirectory.exists() && !temporaryDirectory.mkdir()) { - throw new ISE("Cannot create temporary directory at [%s]", temporaryDirectory); + try { + FileUtils.mkdirp(temporaryDirectory); } + catch (IOException e) { + throw new RuntimeException(e); + } + return Iterators.transform( inputSources.iterator(), inputSource -> { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameContext.java index 7db2fa1a9dd..da962a9d393 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameContext.java @@ -20,8 +20,11 @@ package org.apache.druid.msq.kernel; import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.frame.processor.Bouncer; import org.apache.druid.msq.exec.DataServerQueryHandlerFactory; +import org.apache.druid.msq.exec.OutputChannelMode; import org.apache.druid.msq.exec.WorkerMemoryParameters; +import org.apache.druid.msq.exec.WorkerStorageParameters; import org.apache.druid.msq.querykit.DataSegmentProvider; import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.segment.IndexIO; @@ -30,12 +33,16 @@ import org.apache.druid.segment.SegmentWrangler; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.loading.DataSegmentPusher; +import java.io.Closeable; import java.io.File; /** - * Provides services and objects for the functioning of the frame processors + * Provides services and objects for the functioning of the frame processors. Scoped to a specific stage of a + * specific query, i.e., one {@link WorkOrder}. + * + * Generated by {@link org.apache.druid.msq.exec.WorkerContext#frameContext(QueryDefinition, int, OutputChannelMode)}. */ -public interface FrameContext +public interface FrameContext extends Closeable { SegmentWrangler segmentWrangler(); @@ -59,5 +66,14 @@ public interface FrameContext IndexMergerV9 indexMerger(); + Bouncer processorBouncer(); + WorkerMemoryParameters memoryParameters(); + + WorkerStorageParameters storageParameters(); + + default File tempDir(String name) + { + return new File(tempDir(), name); + } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/WorkOrder.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/WorkOrder.java index 201a1783c05..0c857870210 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/WorkOrder.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/WorkOrder.java @@ -109,7 +109,7 @@ public class WorkOrder /** * Worker IDs for this query, if known in advance (at the time the work order is created). May be null, in which - * case workers use {@link ControllerClient#getTaskList()} to find worker IDs. + * case workers use {@link ControllerClient#getWorkerIds()} to find worker IDs. */ @Nullable @JsonProperty("workers") diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/worker/WorkerStageKernel.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/worker/WorkerStageKernel.java index 632b8a8106d..b838092ca71 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/worker/WorkerStageKernel.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/worker/WorkerStageKernel.java @@ -42,6 +42,8 @@ import java.util.Set; * This separation of decision-making from the "real world" allows the decision-making to live in one, * easy-to-follow place. * + * Not thread-safe. + * * @see org.apache.druid.msq.kernel.controller.ControllerQueryKernel state machine on the controller side */ public class WorkerStageKernel @@ -51,9 +53,10 @@ public class WorkerStageKernel private WorkerStagePhase phase = WorkerStagePhase.NEW; - // We read this variable in the main thread and the netty threads @Nullable - private volatile ClusterByStatisticsSnapshot resultKeyStatisticsSnapshot; + private ClusterByStatisticsSnapshot resultKeyStatisticsSnapshot; + + private boolean doneReadingInput; @Nullable private ClusterByPartitions resultPartitionBoundaries; @@ -107,25 +110,25 @@ public class WorkerStageKernel public void startPreshuffleWaitingForResultPartitionBoundaries() { - assertPreshuffleStatisticsNeeded(); + assertPreshuffleStatisticsNeeded(true); transitionTo(WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES); } public void startPreshuffleWritingOutput() { - assertPreshuffleStatisticsNeeded(); transitionTo(WorkerStagePhase.PRESHUFFLE_WRITING_OUTPUT); } - public void setResultKeyStatisticsSnapshot(final ClusterByStatisticsSnapshot resultKeyStatisticsSnapshot) + public void setResultKeyStatisticsSnapshot(@Nullable final ClusterByStatisticsSnapshot resultKeyStatisticsSnapshot) { - assertPreshuffleStatisticsNeeded(); + assertPreshuffleStatisticsNeeded(resultKeyStatisticsSnapshot != null); this.resultKeyStatisticsSnapshot = resultKeyStatisticsSnapshot; + this.doneReadingInput = true; } public void setResultPartitionBoundaries(final ClusterByPartitions resultPartitionBoundaries) { - assertPreshuffleStatisticsNeeded(); + assertPreshuffleStatisticsNeeded(true); this.resultPartitionBoundaries = resultPartitionBoundaries; } @@ -134,6 +137,11 @@ public class WorkerStageKernel return resultKeyStatisticsSnapshot != null; } + public boolean isDoneReadingInput() + { + return doneReadingInput; + } + public boolean hasResultPartitionBoundaries() { return resultPartitionBoundaries != null; @@ -152,10 +160,10 @@ public class WorkerStageKernel @Nullable public Object getResultObject() { - if (phase == WorkerStagePhase.RESULTS_READY || phase == WorkerStagePhase.FINISHED) { + if (phase == WorkerStagePhase.RESULTS_COMPLETE) { return resultObject; } else { - throw new ISE("Results are not ready yet"); + throw new ISE("Results are not ready in phase[%s]", phase); } } @@ -174,7 +182,7 @@ public class WorkerStageKernel throw new NullPointerException("resultObject must not be null"); } - transitionTo(WorkerStagePhase.RESULTS_READY); + transitionTo(WorkerStagePhase.RESULTS_COMPLETE); this.resultObject = resultObject; } @@ -196,16 +204,18 @@ public class WorkerStageKernel } } - public boolean addPostedResultsComplete(Pair stageIdAndWorkerNumber) + public boolean addPostedResultsComplete(StageId stageId, int workerNumber) { - return postedResultsComplete.add(stageIdAndWorkerNumber); + return postedResultsComplete.add(Pair.of(stageId, workerNumber)); } - private void assertPreshuffleStatisticsNeeded() + private void assertPreshuffleStatisticsNeeded(final boolean delivered) { - if (!workOrder.getStageDefinition().mustGatherResultKeyStatistics()) { + if (delivered != workOrder.getStageDefinition().mustGatherResultKeyStatistics()) { throw new ISE( - "Result partitioning is not necessary for stage [%s]", + "Result key statistics %s, but %s, for stage[%s]", + delivered ? "delivered" : "not delivered", + workOrder.getStageDefinition().mustGatherResultKeyStatistics() ? "expected" : "not expected", workOrder.getStageDefinition().getId() ); } @@ -222,7 +232,12 @@ public class WorkerStageKernel ); phase = newPhase; } else { - throw new IAE("Cannot transition from [%s] to [%s]", phase, newPhase); + throw new IAE( + "Cannot transition stage[%s] from[%s] to[%s]", + workOrder.getStageDefinition().getId(), + phase, + newPhase + ); } } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/worker/WorkerStagePhase.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/worker/WorkerStagePhase.java index f54aa52349e..4e59e7d17a8 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/worker/WorkerStagePhase.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/worker/WorkerStagePhase.java @@ -54,11 +54,12 @@ public enum WorkerStagePhase @Override public boolean canTransitionFrom(final WorkerStagePhase priorPhase) { - return priorPhase == PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES; + return priorPhase == PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES /* if globally sorting */ + || priorPhase == READING_INPUT /* if locally sorting */; } }, - RESULTS_READY { + RESULTS_COMPLETE { @Override public boolean canTransitionFrom(final WorkerStagePhase priorPhase) { @@ -70,7 +71,9 @@ public enum WorkerStagePhase @Override public boolean canTransitionFrom(final WorkerStagePhase priorPhase) { - return priorPhase == RESULTS_READY; + // Stages can transition to FINISHED even if they haven't generated all output yet. For example, this is + // possible if the downstream stage is applying a limit. + return priorPhase.compareTo(FINISHED) < 0; } }, @@ -84,4 +87,24 @@ public enum WorkerStagePhase }; public abstract boolean canTransitionFrom(WorkerStagePhase priorPhase); + + /** + * Whether this phase indicates that the stage is no longer running. + */ + public boolean isTerminal() + { + return this == FINISHED || this == FAILED; + } + + /** + * Whether this phase indicates a stage is running and consuming its full complement of resources. + * + * There are still some resources that can be consumed by stages that are not running. For example, in the + * {@link #FINISHED} state, stages can still have data on disk that has not been cleaned-up yet, some pointers + * to that data that still reside in memory, and some counters in memory available for collection by the controller. + */ + public boolean isRunning() + { + return this != NEW && this != RESULTS_COMPLETE && this != FINISHED && this != FAILED; + } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/BaseWorkerClientImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/BaseWorkerClientImpl.java index 8d0fba72a21..fd1a0323d0f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/BaseWorkerClientImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/BaseWorkerClientImpl.java @@ -97,7 +97,7 @@ public abstract class BaseWorkerClientImpl implements WorkerClient "/keyStatistics/%s/%d?sketchEncoding=%s", StringUtils.urlEncode(stageId.getQueryId()), stageId.getStageNumber(), - WorkerChatHandler.SketchEncoding.OCTET_STREAM + WorkerResource.SketchEncoding.OCTET_STREAM ); return getClient(workerId).asyncRequest( @@ -118,7 +118,7 @@ public abstract class BaseWorkerClientImpl implements WorkerClient StringUtils.urlEncode(stageId.getQueryId()), stageId.getStageNumber(), timeChunk, - WorkerChatHandler.SketchEncoding.OCTET_STREAM + WorkerResource.SketchEncoding.OCTET_STREAM ); return getClient(workerId).asyncRequest( diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/ControllerResource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/ControllerResource.java index d3e9eefa86d..cc570ec992a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/ControllerResource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/ControllerResource.java @@ -82,6 +82,27 @@ public class ControllerResource return Response.status(Response.Status.ACCEPTED).build(); } + /** + * Used by subtasks to inform the controller that they are done reading their input, in cases where they would + * not be calling {@link #httpPostPartialKeyStatistics(Object, String, int, int, HttpServletRequest)}. + * + * See {@link ControllerClient#postDoneReadingInput(StageId, int)} for the client-side code that calls this API. + */ + @POST + @Path("/doneReadingInput/{queryId}/{stageNumber}/{workerNumber}") + @Produces(MediaType.APPLICATION_JSON) + @Consumes(MediaType.APPLICATION_JSON) + public Response httpPostDoneReadingInput( + @PathParam("stageNumber") final int stageNumber, + @PathParam("workerNumber") final int workerNumber, + @Context final HttpServletRequest req + ) + { + MSQResourceUtils.authorizeAdminRequest(permissionMapper, authorizerMapper, req); + controller.doneReadingInput(stageNumber, workerNumber); + 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. @@ -166,7 +187,7 @@ public class ControllerResource } /** - * See {@link ControllerClient#getTaskList()} for the client-side code that calls this API. + * See {@link ControllerClient#getWorkerIds} for the client-side code that calls this API. */ @GET @Path("/taskList") @@ -174,7 +195,7 @@ public class ControllerResource public Response httpGetTaskList(@Context final HttpServletRequest req) { MSQResourceUtils.authorizeAdminRequest(permissionMapper, authorizerMapper, req); - return Response.ok(new MSQTaskList(controller.getTaskIds())).build(); + return Response.ok(new MSQTaskList(controller.getWorkerIds())).build(); } /** diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/MSQResourceUtils.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/MSQResourceUtils.java index 30a8179fe0f..8820b4ead5a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/MSQResourceUtils.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/MSQResourceUtils.java @@ -47,4 +47,20 @@ public class MSQResourceUtils throw new ForbiddenException(access.toString()); } } + + public static void authorizeQueryRequest( + final ResourcePermissionMapper permissionMapper, + final AuthorizerMapper authorizerMapper, + final HttpServletRequest request, + final String queryId + ) + { + final List resourceActions = permissionMapper.getQueryPermissions(queryId); + + Access access = AuthorizationUtils.authorizeAllResourceActions(request, resourceActions, authorizerMapper); + + if (!access.isAllowed()) { + throw new ForbiddenException(access.toString()); + } + } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/ResourcePermissionMapper.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/ResourcePermissionMapper.java index 8c79f4fa0e0..0a7fb874f6d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/ResourcePermissionMapper.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/ResourcePermissionMapper.java @@ -23,11 +23,9 @@ import org.apache.druid.server.security.ResourceAction; import java.util.List; -/** - * Provides HTTP resources such as {@link ControllerResource} with information about which permissions are needed - * for requests. - */ public interface ResourcePermissionMapper { List getAdminPermissions(); + + List getQueryPermissions(String queryId); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/WorkerResource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/WorkerResource.java new file mode 100644 index 00000000000..a0bfecff542 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/WorkerResource.java @@ -0,0 +1,391 @@ +/* + * 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.rpc; + +import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; +import com.google.common.collect.ImmutableMap; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.frame.file.FrameFileHttpResponseHandler; +import org.apache.druid.frame.key.ClusterByPartitions; +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.Worker; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.kernel.WorkOrder; +import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; +import org.apache.druid.msq.statistics.serde.ClusterByStatisticsSnapshotSerde; +import org.apache.druid.server.security.AuthorizerMapper; +import org.apache.druid.utils.CloseableUtils; + +import javax.annotation.Nullable; +import javax.servlet.AsyncContext; +import javax.servlet.AsyncEvent; +import javax.servlet.AsyncListener; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +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.InputStream; +import java.io.OutputStream; + +public class WorkerResource +{ + private static final Logger log = new Logger(WorkerResource.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 static final long GET_CHANNEL_DATA_TIMEOUT = 30_000L; + + protected final Worker worker; + protected final ResourcePermissionMapper permissionMapper; + protected final AuthorizerMapper authorizerMapper; + + public WorkerResource( + final Worker worker, + final ResourcePermissionMapper permissionMapper, + final AuthorizerMapper authorizerMapper + ) + { + this.worker = worker; + this.permissionMapper = permissionMapper; + this.authorizerMapper = authorizerMapper; + } + + /** + * 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 + ) + { + MSQResourceUtils.authorizeQueryRequest(permissionMapper, authorizerMapper, req, queryId); + + final ListenableFuture dataFuture = + worker.readStageOutput(new StageId(queryId, stageNumber), partitionNumber, offset); + + final AsyncContext asyncContext = req.startAsync(); + asyncContext.setTimeout(GET_CHANNEL_DATA_TIMEOUT); + asyncContext.addListener( + new AsyncListener() + { + @Override + public void onComplete(AsyncEvent event) + { + } + + @Override + public void onTimeout(AsyncEvent event) + { + HttpServletResponse response = (HttpServletResponse) asyncContext.getResponse(); + response.setStatus(HttpServletResponse.SC_OK); + event.getAsyncContext().complete(); + } + + @Override + public void onError(AsyncEvent event) + { + } + + @Override + public void onStartAsync(AsyncEvent event) + { + } + } + ); + + // Save these items, since "req" becomes inaccessible in future exception handlers. + final String remoteAddr = req.getRemoteAddr(); + final String requestURI = req.getRequestURI(); + + Futures.addCallback( + dataFuture, + new FutureCallback() + { + @Override + public void onSuccess(final InputStream inputStream) + { + HttpServletResponse response = (HttpServletResponse) asyncContext.getResponse(); + + try (final OutputStream outputStream = response.getOutputStream()) { + if (inputStream == null) { + response.setStatus(HttpServletResponse.SC_NOT_FOUND); + } else { + response.setStatus(HttpServletResponse.SC_OK); + response.setContentType(MediaType.APPLICATION_OCTET_STREAM); + + final byte[] readBuf = new byte[8192]; + 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. + response.setHeader( + FrameFileHttpResponseHandler.HEADER_LAST_FETCH_NAME, + FrameFileHttpResponseHandler.HEADER_LAST_FETCH_VALUE + ); + } else { + long bytesReadTotal = 0; + int bytesReadThisCall = firstRead; + do { + final int bytesToWrite = + (int) Math.min(CHANNEL_DATA_CHUNK_SIZE - bytesReadTotal, bytesReadThisCall); + outputStream.write(readBuf, 0, bytesToWrite); + bytesReadTotal += bytesReadThisCall; + } while (bytesReadTotal < CHANNEL_DATA_CHUNK_SIZE + && (bytesReadThisCall = inputStream.read(readBuf)) != -1); + } + } + } + catch (Exception e) { + log.noStackTrace().warn(e, "Could not respond to request from[%s] to[%s]", remoteAddr, requestURI); + } + finally { + CloseableUtils.closeAndSuppressExceptions(inputStream, e -> log.warn("Failed to close output channel")); + asyncContext.complete(); + } + } + + @Override + public void onFailure(Throwable e) + { + if (!dataFuture.isCancelled()) { + try { + HttpServletResponse response = (HttpServletResponse) asyncContext.getResponse(); + response.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR); + asyncContext.complete(); + } + catch (Exception e2) { + e.addSuppressed(e2); + } + + log.noStackTrace().warn(e, "Request failed from[%s] to[%s]", remoteAddr, requestURI); + } + } + }, + Execs.directExecutor() + ); + + return null; + } + + /** + * See {@link org.apache.druid.msq.exec.WorkerClient#postWorkOrder} for the client-side code that calls this API. + */ + @POST + @Consumes({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) + @Path("/workOrder") + public Response httpPostWorkOrder(final WorkOrder workOrder, @Context final HttpServletRequest req) + { + final String queryId = workOrder.getQueryDefinition().getQueryId(); + MSQResourceUtils.authorizeQueryRequest(permissionMapper, authorizerMapper, req, queryId); + 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, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) + @Path("/resultPartitionBoundaries/{queryId}/{stageNumber}") + public Response httpPostResultPartitionBoundaries( + final ClusterByPartitions stagePartitionBoundaries, + @PathParam("queryId") final String queryId, + @PathParam("stageNumber") final int stageNumber, + @Context final HttpServletRequest req + ) + { + MSQResourceUtils.authorizeQueryRequest(permissionMapper, authorizerMapper, req, queryId); + if (worker.postResultPartitionBoundaries(new StageId(queryId, stageNumber), stagePartitionBoundaries)) { + return Response.status(Response.Status.ACCEPTED).build(); + } else { + return Response.status(Response.Status.BAD_REQUEST).build(); + } + } + + @POST + @Path("/keyStatistics/{queryId}/{stageNumber}") + @Consumes({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_OCTET_STREAM}) + public Response httpFetchKeyStatistics( + @PathParam("queryId") final String queryId, + @PathParam("stageNumber") final int stageNumber, + @QueryParam("sketchEncoding") @Nullable final SketchEncoding sketchEncoding, + @Context final HttpServletRequest req + ) + { + MSQResourceUtils.authorizeQueryRequest(permissionMapper, authorizerMapper, req, queryId); + ClusterByStatisticsSnapshot clusterByStatisticsSnapshot; + StageId stageId = new StageId(queryId, stageNumber); + try { + clusterByStatisticsSnapshot = worker.fetchStatisticsSnapshot(stageId); + if (SketchEncoding.OCTET_STREAM.equals(sketchEncoding)) { + return Response.status(Response.Status.ACCEPTED) + .type(MediaType.APPLICATION_OCTET_STREAM) + .entity( + (StreamingOutput) output -> + ClusterByStatisticsSnapshotSerde.serialize(output, clusterByStatisticsSnapshot) + ) + .build(); + } else { + return Response.status(Response.Status.ACCEPTED) + .type(MediaType.APPLICATION_JSON) + .entity(clusterByStatisticsSnapshot) + .build(); + } + } + catch (Exception e) { + String errorMessage = StringUtils.format( + "Invalid request for key statistics for query[%s] and stage[%d]", + queryId, + stageNumber + ); + log.error(e, errorMessage); + return Response.status(Response.Status.BAD_REQUEST) + .entity(ImmutableMap.of("error", errorMessage)) + .build(); + } + } + + @POST + @Path("/keyStatisticsForTimeChunk/{queryId}/{stageNumber}/{timeChunk}") + @Consumes({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_OCTET_STREAM}) + public Response httpFetchKeyStatisticsWithSnapshot( + @PathParam("queryId") final String queryId, + @PathParam("stageNumber") final int stageNumber, + @PathParam("timeChunk") final long timeChunk, + @QueryParam("sketchEncoding") @Nullable final SketchEncoding sketchEncoding, + @Context final HttpServletRequest req + ) + { + MSQResourceUtils.authorizeQueryRequest(permissionMapper, authorizerMapper, req, queryId); + ClusterByStatisticsSnapshot snapshotForTimeChunk; + StageId stageId = new StageId(queryId, stageNumber); + try { + snapshotForTimeChunk = worker.fetchStatisticsSnapshotForTimeChunk(stageId, timeChunk); + if (SketchEncoding.OCTET_STREAM.equals(sketchEncoding)) { + return Response.status(Response.Status.ACCEPTED) + .type(MediaType.APPLICATION_OCTET_STREAM) + .entity( + (StreamingOutput) output -> + ClusterByStatisticsSnapshotSerde.serialize(output, snapshotForTimeChunk) + ) + .build(); + } else { + return Response.status(Response.Status.ACCEPTED) + .type(MediaType.APPLICATION_JSON) + .entity(snapshotForTimeChunk) + .build(); + } + } + catch (Exception e) { + String errorMessage = StringUtils.format( + "Invalid request for key statistics for query[%s], stage[%d] and timeChunk[%d]", + queryId, + stageNumber, + timeChunk + ); + log.error(e, errorMessage); + return Response.status(Response.Status.BAD_REQUEST) + .entity(ImmutableMap.of("error", errorMessage)) + .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 + ) + { + MSQResourceUtils.authorizeQueryRequest(permissionMapper, authorizerMapper, req, queryId); + 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) + { + MSQResourceUtils.authorizeAdminRequest(permissionMapper, authorizerMapper, req); + 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 + "; qs=0.9", SmileMediaTypes.APPLICATION_JACKSON_SMILE + "; qs=0.1"}) + @Path("/counters") + public Response httpGetCounters(@Context final HttpServletRequest req) + { + MSQResourceUtils.authorizeAdminRequest(permissionMapper, authorizerMapper, req); + return Response.status(Response.Status.OK).entity(worker.getCounters()).build(); + } + + /** + * Determines the encoding of key collectors returned by {@link #httpFetchKeyStatistics} and + * {@link #httpFetchKeyStatisticsWithSnapshot}. + */ + public enum SketchEncoding + { + /** + * The key collector is encoded as a byte stream with {@link ClusterByStatisticsSnapshotSerde}. + */ + OCTET_STREAM, + /** + * The key collector is encoded as json + */ + JSON + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/MetaInputChannelFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/MetaInputChannelFactory.java new file mode 100644 index 00000000000..37595050c81 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/MetaInputChannelFactory.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.shuffle.input; + +import it.unimi.dsi.fastutil.ints.Int2ObjectMap; +import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.msq.exec.OutputChannelMode; +import org.apache.druid.msq.indexing.InputChannelFactory; +import org.apache.druid.msq.input.stage.StageInputSlice; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.kernel.WorkOrder; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Function; + +/** + * Meta-factory that wraps {@link #inputChannelFactoryProvider}, and can create various other kinds of factories. + */ +public class MetaInputChannelFactory implements InputChannelFactory +{ + private final Int2ObjectMap stageOutputModeMap; + private final Function inputChannelFactoryProvider; + private final Map inputChannelFactoryMap = new HashMap<>(); + + public MetaInputChannelFactory( + final Int2ObjectMap stageOutputModeMap, + final Function inputChannelFactoryProvider + ) + { + this.stageOutputModeMap = stageOutputModeMap; + this.inputChannelFactoryProvider = inputChannelFactoryProvider; + } + + /** + * Create a meta-factory. + * + * @param slices stage slices from {@link WorkOrder#getInputs()} + * @param defaultOutputChannelMode mode to use when {@link StageInputSlice#getOutputChannelMode()} is null; i.e., + * when running with an older controller + * @param inputChannelFactoryProvider provider of {@link InputChannelFactory} for various {@link OutputChannelMode} + */ + public static MetaInputChannelFactory create( + final List slices, + final OutputChannelMode defaultOutputChannelMode, + final Function inputChannelFactoryProvider + ) + { + final Int2ObjectMap stageOutputModeMap = new Int2ObjectOpenHashMap<>(); + + for (final StageInputSlice slice : slices) { + final OutputChannelMode newMode; + + if (slice.getOutputChannelMode() != null) { + newMode = slice.getOutputChannelMode(); + } else { + newMode = defaultOutputChannelMode; + } + + final OutputChannelMode prevMode = stageOutputModeMap.putIfAbsent( + slice.getStageNumber(), + newMode + ); + + if (prevMode != null && prevMode != newMode) { + throw new ISE( + "Inconsistent output modes for stage[%s], got[%s] and[%s]", + slice.getStageNumber(), + prevMode, + newMode + ); + } + } + + return new MetaInputChannelFactory(stageOutputModeMap, inputChannelFactoryProvider); + } + + @Override + public ReadableFrameChannel openChannel( + final StageId stageId, + final int workerNumber, + final int partitionNumber + ) throws IOException + { + final OutputChannelMode outputChannelMode = stageOutputModeMap.get(stageId.getStageNumber()); + + if (outputChannelMode == null) { + throw new ISE("No output mode for stageNumber[%s]", stageId.getStageNumber()); + } + + return inputChannelFactoryMap.computeIfAbsent(outputChannelMode, inputChannelFactoryProvider) + .openChannel(stageId, workerNumber, partitionNumber); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/WorkerOrLocalInputChannelFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/WorkerOrLocalInputChannelFactory.java new file mode 100644 index 00000000000..08c7176b7c2 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/WorkerOrLocalInputChannelFactory.java @@ -0,0 +1,70 @@ +/* + * 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.shuffle.input; + +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.msq.indexing.InputChannelFactory; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.shuffle.output.StageOutputHolder; + +import java.io.IOException; +import java.util.List; +import java.util.function.Supplier; + +/** + * An {@link InputChannelFactory} that loads data locally when possible, and otherwise connects directly to other + * workers. Used when durable shuffle storage is off. + */ +public class WorkerOrLocalInputChannelFactory implements InputChannelFactory +{ + private final String myId; + private final Supplier> workerIdsSupplier; + private final InputChannelFactory workerInputChannelFactory; + private final StageOutputHolderProvider stageOutputHolderProvider; + + public WorkerOrLocalInputChannelFactory( + final String myId, + final Supplier> workerIdsSupplier, + final InputChannelFactory workerInputChannelFactory, + final StageOutputHolderProvider stageOutputHolderProvider + ) + { + this.myId = myId; + this.workerIdsSupplier = workerIdsSupplier; + this.workerInputChannelFactory = workerInputChannelFactory; + this.stageOutputHolderProvider = stageOutputHolderProvider; + } + + @Override + public ReadableFrameChannel openChannel(StageId stageId, int workerNumber, int partitionNumber) throws IOException + { + final String taskId = workerIdsSupplier.get().get(workerNumber); + if (taskId.equals(myId)) { + return stageOutputHolderProvider.getHolder(stageId, partitionNumber).readLocally(); + } else { + return workerInputChannelFactory.openChannel(stageId, workerNumber, partitionNumber); + } + } + + public interface StageOutputHolderProvider + { + StageOutputHolder getHolder(StageId stageId, int partitionNumber); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/ByteChunksInputStream.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/ByteChunksInputStream.java new file mode 100644 index 00000000000..f623e58f65b --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/ByteChunksInputStream.java @@ -0,0 +1,116 @@ +/* + * 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.shuffle.output; + +import org.apache.druid.error.DruidException; + +import java.io.InputStream; +import java.util.List; + +/** + * Input stream based on a list of byte arrays. + */ +public class ByteChunksInputStream extends InputStream +{ + private final List chunks; + private int chunkNum; + private int positionWithinChunk; + + /** + * Create a new stream wrapping a list of chunks. + * + * @param chunks byte arrays + * @param positionWithinFirstChunk starting position within the first byte array + */ + public ByteChunksInputStream(final List chunks, final int positionWithinFirstChunk) + { + this.chunks = chunks; + this.positionWithinChunk = positionWithinFirstChunk; + this.chunkNum = -1; + advanceChunk(); + } + + @Override + public int read() + { + if (chunkNum >= chunks.size()) { + return -1; + } else { + final byte[] currentChunk = chunks.get(chunkNum); + final byte b = currentChunk[positionWithinChunk++]; + + if (positionWithinChunk == currentChunk.length) { + chunkNum++; + positionWithinChunk = 0; + } + + return b & 0xFF; + } + } + + @Override + public int read(byte[] b) + { + return read(b, 0, b.length); + } + + @Override + public int read(byte[] b, int off, int len) + { + if (len == 0) { + return 0; + } else if (chunkNum >= chunks.size()) { + return -1; + } else { + int r = 0; + + while (r < len && chunkNum < chunks.size()) { + final byte[] currentChunk = chunks.get(chunkNum); + int toReadFromCurrentChunk = Math.min(len - r, currentChunk.length - positionWithinChunk); + System.arraycopy(currentChunk, positionWithinChunk, b, off + r, toReadFromCurrentChunk); + r += toReadFromCurrentChunk; + positionWithinChunk += toReadFromCurrentChunk; + if (positionWithinChunk == currentChunk.length) { + chunkNum++; + positionWithinChunk = 0; + } + } + + return r; + } + } + + @Override + public void close() + { + chunkNum = chunks.size(); + positionWithinChunk = 0; + } + + private void advanceChunk() + { + chunkNum++; + + // Verify nonempty + if (chunkNum < chunks.size() && chunks.get(chunkNum).length == 0) { + throw DruidException.defensive("Empty chunk not allowed"); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/ChannelStageOutputReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/ChannelStageOutputReader.java new file mode 100644 index 00000000000..ec95ca7af6a --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/ChannelStageOutputReader.java @@ -0,0 +1,274 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.shuffle.output; + +import com.google.common.collect.ImmutableList; +import com.google.common.primitives.Ints; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.errorprone.annotations.concurrent.GuardedBy; +import it.unimi.dsi.fastutil.bytes.ByteArrays; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.error.DruidException; +import org.apache.druid.frame.channel.ByteTracker; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.frame.file.FrameFileWriter; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.msq.exec.OutputChannelMode; +import org.apache.druid.msq.kernel.controller.ControllerQueryKernelUtils; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; +import java.util.ArrayDeque; +import java.util.Deque; + +/** + * Reader for the case where stage output is a generic {@link ReadableFrameChannel}. + * + * Because this reader returns an underlying channel directly, it must only be used when it is certain that + * only a single consumer exists, i.e., when using output mode {@link OutputChannelMode#MEMORY}. See + * {@link ControllerQueryKernelUtils#canUseMemoryOutput} for the code that ensures that there is only a single + * consumer in the in-memory case. + */ +public class ChannelStageOutputReader implements StageOutputReader +{ + enum State + { + INIT, + LOCAL, + REMOTE, + CLOSED + } + + private final ReadableFrameChannel channel; + private final FrameFileWriter writer; + + /** + * Pair of chunk size + chunk InputStream. + */ + private final Deque chunks = new ArrayDeque<>(); + + /** + * State of this reader. + */ + @GuardedBy("this") + private State state = State.INIT; + + /** + * Position of {@link #positionWithinFirstChunk} in the first chunk of {@link #chunks}, within the overall stream. + */ + @GuardedBy("this") + private long cursor; + + /** + * Offset of the first chunk in {@link #chunks} which corresponds to {@link #cursor}. + */ + @GuardedBy("this") + private int positionWithinFirstChunk; + + /** + * Whether {@link FrameFileWriter#close()} is called on {@link #writer}. + */ + @GuardedBy("this") + private boolean didCloseWriter; + + public ChannelStageOutputReader(final ReadableFrameChannel channel) + { + this.channel = channel; + this.writer = FrameFileWriter.open(new ChunkAcceptor(), null, ByteTracker.unboundedTracker()); + } + + /** + * Returns an input stream starting at the provided offset. + * + * The returned {@link InputStream} is non-blocking, and is slightly buffered (up to one frame). It does not + * necessarily contain the complete remaining dataset; this means that multiple calls to this method are necessary + * to fetch the complete dataset. + * + * The provided offset must be greater than, or equal to, the offset provided to the prior call. + * + * This class supports either remote or local reads, but not both. Calling both this method and {@link #readLocally()} + * on the same instance of this class is an error. + * + * @param offset offset into the stage output stream + */ + @Override + public synchronized ListenableFuture readRemotelyFrom(final long offset) + { + if (state == State.INIT) { + state = State.REMOTE; + } else if (state == State.LOCAL) { + throw new ISE("Cannot read both remotely and locally"); + } else if (state == State.CLOSED) { + throw new ISE("Closed"); + } + + if (offset < cursor) { + return Futures.immediateFailedFuture( + new ISE("Offset[%,d] no longer available, current cursor is[%,d]", offset, cursor)); + } + + while (chunks.isEmpty() || offset > cursor) { + // Fetch additional chunks if needed. + if (chunks.isEmpty()) { + if (didCloseWriter) { + if (offset == cursor) { + return Futures.immediateFuture(new ByteArrayInputStream(ByteArrays.EMPTY_ARRAY)); + } else { + throw DruidException.defensive( + "Channel finished but cursor[%,d] does not match requested offset[%,d]", + cursor, + offset + ); + } + } else if (channel.isFinished()) { + try { + writer.close(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + + didCloseWriter = true; + continue; + } else if (channel.canRead()) { + try { + writer.writeFrame(channel.read(), FrameFileWriter.NO_PARTITION); + } + catch (Exception e) { + try { + writer.abort(); + } + catch (IOException e2) { + e.addSuppressed(e2); + } + + throw new RuntimeException(e); + } + } else { + return FutureUtils.transformAsync(channel.readabilityFuture(), ignored -> readRemotelyFrom(offset)); + } + } + + // Advance cursor to the provided offset, or the end of the current chunk, whichever is earlier. + final byte[] chunk = chunks.peek(); + final long amountToAdvance = Math.min(offset - cursor, chunk.length - positionWithinFirstChunk); + cursor += amountToAdvance; + positionWithinFirstChunk += Ints.checkedCast(amountToAdvance); + + // Remove first chunk if it is no longer needed. (i.e., if the cursor is at the end of it.) + if (positionWithinFirstChunk == chunk.length) { + chunks.poll(); + positionWithinFirstChunk = 0; + } + } + + if (chunks.isEmpty() || offset != cursor) { + throw DruidException.defensive( + "Expected cursor[%,d] to be caught up to offset[%,d] by this point, and to have nonzero chunks", + cursor, + offset + ); + } + + return Futures.immediateFuture(new ByteChunksInputStream(ImmutableList.copyOf(chunks), positionWithinFirstChunk)); + } + + /** + * Returns the {@link ReadableFrameChannel} that backs this reader. + * + * Callers are responsible for closing the returned channel. Once this method is called, the caller becomes the + * owner of the channel, and this class's {@link #close()} method will no longer close the channel. + * + * Only a single reader is supported. Once this method is called, it cannot be called again. + * + * This class supports either remote or local reads, but not both. Calling both this method and + * {@link #readRemotelyFrom(long)} on the same instance of this class is an error. + */ + @Override + public synchronized ReadableFrameChannel readLocally() + { + if (state == State.INIT) { + state = State.LOCAL; + return channel; + } else if (state == State.REMOTE) { + throw new ISE("Cannot read both remotely and locally"); + } else if (state == State.LOCAL) { + throw new ISE("Cannot read channel multiple times"); + } else { + assert state == State.CLOSED; + throw new ISE("Closed"); + } + } + + /** + * Closes the {@link ReadableFrameChannel} backing this reader, unless {@link #readLocally()} has been called. + * In that case, the caller of {@link #readLocally()} is responsible for closing the channel. + */ + @Override + public synchronized void close() + { + // Call channel.close() unless readLocally() has been called. In that case, we expect the caller to close it. + if (state != State.LOCAL) { + state = State.CLOSED; + channel.close(); + } + } + + /** + * Input stream that can have bytes appended to it, and that can have bytes acknowledged. + */ + private class ChunkAcceptor implements WritableByteChannel + { + private boolean open = true; + + @Override + public int write(final ByteBuffer src) throws IOException + { + if (!open) { + throw new IOException("Closed"); + } + + final int len = src.remaining(); + if (len > 0) { + final byte[] bytes = new byte[len]; + src.get(bytes); + chunks.add(bytes); + } + + return len; + } + + @Override + public boolean isOpen() + { + return open; + } + + @Override + public void close() + { + open = false; + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/FileStageOutputReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/FileStageOutputReader.java new file mode 100644 index 00000000000..29fb7b17ee7 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/FileStageOutputReader.java @@ -0,0 +1,99 @@ +/* + * 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.shuffle.output; + +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import it.unimi.dsi.fastutil.bytes.ByteArrays; +import org.apache.druid.frame.channel.ReadableFileFrameChannel; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.frame.file.FrameFile; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.RandomAccessFile; +import java.nio.channels.Channels; + +/** + * Reader for the case where stage output is stored in a {@link FrameFile} on disk. + */ +public class FileStageOutputReader implements StageOutputReader +{ + private final FrameFile frameFile; + + public FileStageOutputReader(FrameFile frameFile) + { + this.frameFile = frameFile; + } + + /** + * Returns an input stream starting at the provided offset. The file is opened and seeked in-line with this method + * call, so the returned future is always immediately resolved. Callers are responsible for closing the returned + * input stream. + * + * This class supports remote and local reads from the same {@link FrameFile}, which, for example, is useful when + * broadcasting the output of a stage. + * + * @param offset offset into the stage output file + */ + @Override + public ListenableFuture readRemotelyFrom(long offset) + { + try { + final RandomAccessFile randomAccessFile = new RandomAccessFile(frameFile.file(), "r"); + + if (offset >= randomAccessFile.length()) { + randomAccessFile.close(); + return Futures.immediateFuture(new ByteArrayInputStream(ByteArrays.EMPTY_ARRAY)); + } else { + randomAccessFile.seek(offset); + return Futures.immediateFuture(Channels.newInputStream(randomAccessFile.getChannel())); + } + } + catch (Exception e) { + return Futures.immediateFailedFuture(e); + } + } + + /** + * Returns a channel pointing to a fresh {@link FrameFile#newReference()} of the underlying frame file. Callers are + * responsible for closing the returned channel. + * + * This class supports remote and local reads from the same {@link FrameFile}, which, for example, is useful when + * broadcasting the output of a stage. + */ + @Override + public ReadableFrameChannel readLocally() + { + return new ReadableFileFrameChannel(frameFile.newReference()); + } + + /** + * Closes the initial reference to the underlying {@link FrameFile}. Does not close additional references created by + * calls to {@link #readLocally()}; those references are closed when the channel(s) returned by {@link #readLocally()} + * are closed. + */ + @Override + public void close() throws IOException + { + frameFile.close(); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/FutureReadableFrameChannel.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/FutureReadableFrameChannel.java new file mode 100644 index 00000000000..8dcb8786713 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/FutureReadableFrameChannel.java @@ -0,0 +1,124 @@ +/* + * 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.shuffle.output; + +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.java.util.common.logger.Logger; + +import java.util.NoSuchElementException; + +/** + * Channel that wraps a {@link ListenableFuture} of a {@link ReadableFrameChannel}, but acts like a regular (non-future) + * {@link ReadableFrameChannel}. + */ +public class FutureReadableFrameChannel implements ReadableFrameChannel +{ + private static final Logger log = new Logger(FutureReadableFrameChannel.class); + + private final ListenableFuture channelFuture; + private ReadableFrameChannel channel; + + public FutureReadableFrameChannel(final ListenableFuture channelFuture) + { + this.channelFuture = channelFuture; + } + + @Override + public boolean isFinished() + { + if (populateChannel()) { + return channel.isFinished(); + } else { + return false; + } + } + + @Override + public boolean canRead() + { + if (populateChannel()) { + return channel.canRead(); + } else { + return false; + } + } + + @Override + public Frame read() + { + if (populateChannel()) { + return channel.read(); + } else { + throw new NoSuchElementException(); + } + } + + @Override + public ListenableFuture readabilityFuture() + { + if (populateChannel()) { + return channel.readabilityFuture(); + } else { + return FutureUtils.transformAsync(channelFuture, ignored -> readabilityFuture()); + } + } + + @Override + public void close() + { + if (populateChannel()) { + channel.close(); + } else { + channelFuture.cancel(true); + + // In case of a race where channelFuture resolved between populateChannel() and here, the cancel call above would + // have no effect. Guard against this case by checking if the channelFuture has resolved, and if so, close the + // channel here. + try { + final ReadableFrameChannel theChannel = FutureUtils.getUncheckedImmediately(channelFuture); + + try { + theChannel.close(); + } + catch (Throwable t) { + log.noStackTrace().warn(t, "Failed to close channel"); + } + } + catch (Throwable ignored) { + // Suppress. + } + } + } + + private boolean populateChannel() + { + if (channel != null) { + return true; + } else if (channelFuture.isDone()) { + channel = FutureUtils.getUncheckedImmediately(channelFuture); + return true; + } else { + return false; + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/NilStageOutputReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/NilStageOutputReader.java new file mode 100644 index 00000000000..86530dad1d0 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/NilStageOutputReader.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.shuffle.output; + +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.frame.channel.ByteTracker; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.frame.channel.ReadableNilFrameChannel; +import org.apache.druid.frame.file.FrameFileWriter; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.channels.Channels; + +/** + * Reader for the case where stage output is known to be empty. + */ +public class NilStageOutputReader implements StageOutputReader +{ + public static final NilStageOutputReader INSTANCE = new NilStageOutputReader(); + + private static final byte[] EMPTY_FRAME_FILE; + + static { + try { + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + FrameFileWriter.open(Channels.newChannel(baos), null, ByteTracker.unboundedTracker()).close(); + EMPTY_FRAME_FILE = baos.toByteArray(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public ListenableFuture readRemotelyFrom(final long offset) + { + final ByteArrayInputStream in = new ByteArrayInputStream(EMPTY_FRAME_FILE); + + //noinspection ResultOfMethodCallIgnored: OK to ignore since "skip" always works for ByteArrayInputStream. + in.skip(offset); + + return Futures.immediateFuture(in); + } + + @Override + public ReadableFrameChannel readLocally() + { + return ReadableNilFrameChannel.INSTANCE; + } + + @Override + public void close() + { + // Nothing to do. + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/StageOutputHolder.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/StageOutputHolder.java new file mode 100644 index 00000000000..c19519dfb7b --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/StageOutputHolder.java @@ -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. + */ + +package org.apache.druid.msq.shuffle.output; + +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.SettableFuture; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.frame.channel.ReadableFileFrameChannel; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.frame.channel.ReadableNilFrameChannel; +import org.apache.druid.frame.file.FrameFile; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.msq.exec.Worker; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.rpc.WorkerResource; +import org.apache.druid.utils.CloseableUtils; + +import javax.servlet.http.HttpServletRequest; +import java.io.Closeable; +import java.io.InputStream; + +/** + * Container for a {@link StageOutputReader}, which is used to read the output of a stage. + */ +public class StageOutputHolder implements Closeable +{ + private final SettableFuture channelFuture; + private final ListenableFuture readerFuture; + + public StageOutputHolder() + { + this.channelFuture = SettableFuture.create(); + this.readerFuture = FutureUtils.transform(channelFuture, StageOutputHolder::createReader); + } + + /** + * Method for remote reads. + * + * Provides the implementation for {@link Worker#readStageOutput(StageId, int, long)}, which is in turn used by + * {@link WorkerResource#httpGetChannelData(String, int, int, long, HttpServletRequest)}. + * + * @see StageOutputReader#readRemotelyFrom(long) for details on behavior + */ + public ListenableFuture readRemotelyFrom(final long offset) + { + return FutureUtils.transformAsync(readerFuture, reader -> reader.readRemotelyFrom(offset)); + } + + /** + * Method for local reads. + * + * Used instead of {@link #readRemotelyFrom(long)} when a worker is reading a channel from itself, to avoid needless + * HTTP calls to itself. + * + * @see StageOutputReader#readLocally() for details on behavior + */ + public ReadableFrameChannel readLocally() + { + return new FutureReadableFrameChannel(FutureUtils.transform(readerFuture, StageOutputReader::readLocally)); + } + + /** + * Sets the channel that backs {@link #readLocally()} and {@link #readRemotelyFrom(long)}. + */ + public void setChannel(final ReadableFrameChannel channel) + { + if (!channelFuture.set(channel)) { + if (FutureUtils.getUncheckedImmediately(channelFuture) == null) { + throw new ISE("Closed"); + } else { + throw new ISE("Channel already set"); + } + } + } + + @Override + public void close() + { + channelFuture.set(null); + + final StageOutputReader reader; + + try { + reader = FutureUtils.getUnchecked(readerFuture, true); + } + catch (Throwable e) { + // Error creating the reader, nothing to close. Suppress. + return; + } + + if (reader != null) { + CloseableUtils.closeAndWrapExceptions(reader); + } + } + + private static StageOutputReader createReader(final ReadableFrameChannel channel) + { + if (channel == null) { + // Happens if close() was called before the channel resolved. + throw new ISE("Closed"); + } + + if (channel instanceof ReadableNilFrameChannel) { + return NilStageOutputReader.INSTANCE; + } + + if (channel instanceof ReadableFileFrameChannel) { + // Optimized implementation when reading an entire file. + final ReadableFileFrameChannel fileChannel = (ReadableFileFrameChannel) channel; + + if (fileChannel.isEntireFile()) { + final FrameFile frameFile = fileChannel.newFrameFileReference(); + + // Close original channel, so we don't leak a frame file reference. + channel.close(); + + return new FileStageOutputReader(frameFile); + } + } + + // Generic implementation for any other type of channel. + return new ChannelStageOutputReader(channel); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/StageOutputReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/StageOutputReader.java new file mode 100644 index 00000000000..36b993611ca --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/StageOutputReader.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.shuffle.output; + +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.frame.file.FrameFile; +import org.apache.druid.msq.exec.Worker; +import org.apache.druid.msq.kernel.StageId; + +import java.io.Closeable; +import java.io.InputStream; + +/** + * Interface for reading output channels for a particular stage. Each instance of this interface represents a + * stream from a single {@link org.apache.druid.msq.kernel.StagePartition} in {@link FrameFile} format. + * + * @see FileStageOutputReader implementation backed by {@link FrameFile} + * @see ChannelStageOutputReader implementation backed by {@link ReadableFrameChannel} + * @see NilStageOutputReader implementation for an empty channel + */ +public interface StageOutputReader extends Closeable +{ + /** + * Method for remote reads. + * + * This method ultimately backs {@link Worker#readStageOutput(StageId, int, long)}. Refer to that method's + * documentation for details about behavior of the returned future. + * + * Callers are responsible for closing the returned {@link InputStream}. This input stream may encapsulate + * resources that are not closed by this class's {@link #close()} method. + * + * It is implementation-dependent whether calls to this method must have monotonically increasing offsets. + * In particular, {@link ChannelStageOutputReader} requires monotonically increasing offsets, but + * {@link FileStageOutputReader} and {@link NilStageOutputReader} do not. + * + * @param offset offset into the stage output file + * + * @see StageOutputHolder#readRemotelyFrom(long) which uses this method + * @see Worker#readStageOutput(StageId, int, long) for documentation on behavior of the returned future + */ + ListenableFuture readRemotelyFrom(long offset); + + /** + * Method for local reads. + * + * Depending on implementation, this method may or may not be able to be called multiple times, and may or may not + * be able to be mixed with {@link #readRemotelyFrom(long)}. Refer to the specific implementation for more details. + * + * Callers are responsible for closing the returned channel. The returned channel may encapsulate resources that + * are not closed by this class's {@link #close()} method. + * + * It is implementation-dependent whether this method can be called multiple times. In particular, + * {@link ChannelStageOutputReader#readLocally()} can only be called one time, but the implementations in + * {@link FileStageOutputReader} and {@link NilStageOutputReader} can be called multiple times. + * + * @see StageOutputHolder#readLocally() which uses this method + */ + ReadableFrameChannel readLocally(); +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQDrillWindowQueryTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQDrillWindowQueryTest.java index d3a67fdd659..1b2eebe7742 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQDrillWindowQueryTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQDrillWindowQueryTest.java @@ -28,6 +28,7 @@ import org.apache.druid.msq.exec.MSQDrillWindowQueryTest.DrillWindowQueryMSQComp import org.apache.druid.msq.sql.MSQTaskSqlEngine; import org.apache.druid.msq.test.CalciteMSQTestsHelper; import org.apache.druid.msq.test.ExtractResultsFactory; +import org.apache.druid.msq.test.MSQTestBase; import org.apache.druid.msq.test.MSQTestOverlordServiceClient; import org.apache.druid.msq.test.MSQTestTaskActionClient; import org.apache.druid.msq.test.VerifyMSQSupportedNativeQueriesPredicate; @@ -63,15 +64,7 @@ public class MSQDrillWindowQueryTest extends DrillWindowQueryTest Injector injector ) { - final WorkerMemoryParameters workerMemoryParameters = - WorkerMemoryParameters.createInstance( - WorkerMemoryParameters.PROCESSING_MINIMUM_BYTES * 50, - 2, - 10, - 2, - 0, - 0 - ); + final WorkerMemoryParameters workerMemoryParameters = MSQTestBase.makeTestWorkerMemoryParameters(); final MSQTestOverlordServiceClient indexingServiceClient = new MSQTestOverlordServiceClient( queryJsonMapper, injector, diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerImplTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerImplTest.java deleted file mode 100644 index 171f476ebfe..00000000000 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerImplTest.java +++ /dev/null @@ -1,54 +0,0 @@ -/* - * 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.java.util.common.ISE; -import org.apache.druid.msq.indexing.MSQWorkerTask; -import org.apache.druid.msq.kernel.StageId; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.Mock; -import org.mockito.junit.MockitoJUnitRunner; - -import java.util.HashMap; - - -@RunWith(MockitoJUnitRunner.class) -public class WorkerImplTest -{ - @Mock - WorkerContext workerContext; - - @Test - public void testFetchStatsThrows() - { - WorkerImpl worker = new WorkerImpl(new MSQWorkerTask("controller", "ds", 1, new HashMap<>(), 0), workerContext, WorkerStorageParameters.createInstanceForTests(Long.MAX_VALUE)); - Assert.assertThrows(ISE.class, () -> worker.fetchStatisticsSnapshot(new StageId("xx", 1))); - } - - @Test - public void testFetchStatsWithTimeChunkThrows() - { - WorkerImpl worker = new WorkerImpl(new MSQWorkerTask("controller", "ds", 1, new HashMap<>(), 0), workerContext, WorkerStorageParameters.createInstanceForTests(Long.MAX_VALUE)); - Assert.assertThrows(ISE.class, () -> worker.fetchStatisticsSnapshotForTimeChunk(new StageId("xx", 1), 1L)); - } - -} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerMemoryParametersTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerMemoryParametersTest.java index 29614fc0734..1ead2a181fd 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerMemoryParametersTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerMemoryParametersTest.java @@ -32,34 +32,54 @@ public class WorkerMemoryParametersTest @Test public void test_oneWorkerInJvm_alone() { - Assert.assertEquals(params(335_500_000, 1, 41, 75_000_000), create(1_000_000_000, 1, 1, 1, 0, 0)); - Assert.assertEquals(params(223_000_000, 2, 13, 75_000_000), create(1_000_000_000, 1, 2, 1, 0, 0)); - Assert.assertEquals(params(133_000_000, 4, 3, 75_000_000), create(1_000_000_000, 1, 4, 1, 0, 0)); - Assert.assertEquals(params(73_000_000, 3, 2, 75_000_000), create(1_000_000_000, 1, 8, 1, 0, 0)); - Assert.assertEquals(params(49_923_076, 2, 2, 75_000_000), create(1_000_000_000, 1, 12, 1, 0, 0)); + Assert.assertEquals(params(335_500_000, 1, 41, 75_000_000), create(1_000_000_000, 1, 1, 1, 1, 0, 0)); + Assert.assertEquals(params(223_000_000, 2, 13, 75_000_000), create(1_000_000_000, 1, 2, 1, 1, 0, 0)); + Assert.assertEquals(params(133_000_000, 4, 3, 75_000_000), create(1_000_000_000, 1, 4, 1, 1, 0, 0)); + Assert.assertEquals(params(73_000_000, 3, 2, 75_000_000), create(1_000_000_000, 1, 8, 1, 1, 0, 0)); + Assert.assertEquals(params(49_923_076, 2, 2, 75_000_000), create(1_000_000_000, 1, 12, 1, 1, 0, 0)); final MSQException e = Assert.assertThrows( MSQException.class, - () -> create(1_000_000_000, 1, 32, 1, 0, 0) + () -> create(1_000_000_000, 1, 32, 1, 1, 0, 0) ); - Assert.assertEquals(new NotEnoughMemoryFault(1_588_044_000, 1_000_000_000, 750_000_000, 1, 32), e.getFault()); + Assert.assertEquals(new NotEnoughMemoryFault(1_588_044_000, 1_000_000_000, 750_000_000, 1, 32, 1), e.getFault()); - final MSQFault fault = Assert.assertThrows(MSQException.class, () -> create(1_000_000_000, 2, 32, 1, 0, 0)) + final MSQFault fault = Assert.assertThrows(MSQException.class, () -> create(1_000_000_000, 2, 32, 1, 1, 0, 0)) .getFault(); - Assert.assertEquals(new NotEnoughMemoryFault(2024045333, 1_000_000_000, 750_000_000, 2, 32), fault); + Assert.assertEquals(new NotEnoughMemoryFault(2024045333, 1_000_000_000, 750_000_000, 2, 32, 1), fault); + } + @Test + public void test_oneWorkerInJvm_alone_twoConcurrentStages() + { + Assert.assertEquals(params(166_750_000, 1, 20, 37_500_000), create(1_000_000_000, 1, 1, 2, 1, 0, 0)); + Assert.assertEquals(params(110_500_000, 2, 6, 37_500_000), create(1_000_000_000, 1, 2, 2, 1, 0, 0)); + Assert.assertEquals(params(65_500_000, 2, 3, 37_500_000), create(1_000_000_000, 1, 4, 2, 1, 0, 0)); + Assert.assertEquals(params(35_500_000, 1, 3, 37_500_000), create(1_000_000_000, 1, 8, 2, 1, 0, 0)); + + final MSQException e = Assert.assertThrows( + MSQException.class, + () -> create(1_000_000_000, 1, 12, 2, 1, 0, 0) + ); + + Assert.assertEquals(new NotEnoughMemoryFault(1_736_034_666, 1_000_000_000, 750_000_000, 1, 12, 2), e.getFault()); + + final MSQFault fault = Assert.assertThrows(MSQException.class, () -> create(1_000_000_000, 2, 32, 2, 1, 0, 0)) + .getFault(); + + Assert.assertEquals(new NotEnoughMemoryFault(4_048_090_666L, 1_000_000_000, 750_000_000, 2, 32, 2), fault); } @Test public void test_oneWorkerInJvm_twoHundredWorkersInCluster() { - Assert.assertEquals(params(474_000_000, 1, 83, 150_000_000), create(2_000_000_000, 1, 1, 200, 0, 0)); - Assert.assertEquals(params(249_000_000, 2, 27, 150_000_000), create(2_000_000_000, 1, 2, 200, 0, 0)); + Assert.assertEquals(params(474_000_000, 1, 83, 150_000_000), create(2_000_000_000, 1, 1, 1, 200, 0, 0)); + Assert.assertEquals(params(249_000_000, 2, 27, 150_000_000), create(2_000_000_000, 1, 2, 1, 200, 0, 0)); final MSQException e = Assert.assertThrows( MSQException.class, - () -> create(1_000_000_000, 1, 4, 200, 0, 0) + () -> create(1_000_000_000, 1, 4, 1, 200, 0, 0) ); Assert.assertEquals(new TooManyWorkersFault(200, 109), e.getFault()); @@ -68,76 +88,102 @@ public class WorkerMemoryParametersTest @Test public void test_fourWorkersInJvm_twoHundredWorkersInCluster() { - Assert.assertEquals(params(1_014_000_000, 1, 150, 168_750_000), create(9_000_000_000L, 4, 1, 200, 0, 0)); - Assert.assertEquals(params(811_500_000, 2, 62, 168_750_000), create(9_000_000_000L, 4, 2, 200, 0, 0)); - Assert.assertEquals(params(558_375_000, 4, 22, 168_750_000), create(9_000_000_000L, 4, 4, 200, 0, 0)); - Assert.assertEquals(params(305_250_000, 4, 14, 168_750_000), create(9_000_000_000L, 4, 8, 200, 0, 0)); - Assert.assertEquals(params(102_750_000, 4, 8, 168_750_000), create(9_000_000_000L, 4, 16, 200, 0, 0)); + Assert.assertEquals(params(1_014_000_000, 1, 150, 168_750_000), create(9_000_000_000L, 4, 1, 1, 200, 0, 0)); + Assert.assertEquals(params(811_500_000, 2, 62, 168_750_000), create(9_000_000_000L, 4, 2, 1, 200, 0, 0)); + Assert.assertEquals(params(558_375_000, 4, 22, 168_750_000), create(9_000_000_000L, 4, 4, 1, 200, 0, 0)); + Assert.assertEquals(params(305_250_000, 4, 14, 168_750_000), create(9_000_000_000L, 4, 8, 1, 200, 0, 0)); + Assert.assertEquals(params(102_750_000, 4, 8, 168_750_000), create(9_000_000_000L, 4, 16, 1, 200, 0, 0)); final MSQException e = Assert.assertThrows( MSQException.class, - () -> create(8_000_000_000L, 4, 32, 200, 0, 0) + () -> create(8_000_000_000L, 4, 32, 1, 200, 0, 0) ); Assert.assertEquals(new TooManyWorkersFault(200, 124), e.getFault()); // Make sure 124 actually works, and 125 doesn't. (Verify the error message above.) - Assert.assertEquals(params(25_000_000, 4, 3, 150_000_000), create(8_000_000_000L, 4, 32, 124, 0, 0)); + Assert.assertEquals(params(25_000_000, 4, 3, 150_000_000), create(8_000_000_000L, 4, 32, 1, 124, 0, 0)); final MSQException e2 = Assert.assertThrows( MSQException.class, - () -> create(8_000_000_000L, 4, 32, 125, 0, 0) + () -> create(8_000_000_000L, 4, 32, 1, 125, 0, 0) ); Assert.assertEquals(new TooManyWorkersFault(125, 124), e2.getFault()); } @Test - public void test_oneWorkerInJvm_smallWorkerCapacity() + public void test_fourWorkersInJvm_twoHundredWorkersInCluster_twoConcurrentStages() { - // Supersorter max channels per processer are one less than they are usually to account for extra frames that are required while creating composing output channels - Assert.assertEquals(params(41_200_000, 1, 3, 9_600_000), create(128_000_000, 1, 1, 1, 0, 0)); - Assert.assertEquals(params(26_800_000, 1, 1, 9_600_000), create(128_000_000, 1, 2, 1, 0, 0)); + Assert.assertEquals(params(406_500_000, 1, 74, 84_375_000), create(9_000_000_000L, 4, 1, 2, 200, 0, 0)); + Assert.assertEquals(params(305_250_000, 2, 30, 84_375_000), create(9_000_000_000L, 4, 2, 2, 200, 0, 0)); + Assert.assertEquals(params(178_687_500, 4, 10, 84_375_000), create(9_000_000_000L, 4, 4, 2, 200, 0, 0)); + Assert.assertEquals(params(52_125_000, 4, 6, 84_375_000), create(9_000_000_000L, 4, 8, 2, 200, 0, 0)); final MSQException e = Assert.assertThrows( MSQException.class, - () -> create(1_000_000_000, 1, 32, 1, 0, 0) + () -> create(8_000_000_000L, 4, 16, 2, 200, 0, 0) ); - Assert.assertEquals(new NotEnoughMemoryFault(1_588_044_000, 1_000_000_000, 750_000_000, 1, 32), e.getFault()); + + Assert.assertEquals(new TooManyWorkersFault(200, 109), e.getFault()); + + // Make sure 109 actually works, and 110 doesn't. (Verify the error message above.) + Assert.assertEquals(params(25_000_000, 4, 3, 75_000_000), create(8_000_000_000L, 4, 16, 2, 109, 0, 0)); final MSQException e2 = Assert.assertThrows( MSQException.class, - () -> create(128_000_000, 1, 4, 1, 0, 0) + () -> create(8_000_000_000L, 4, 16, 2, 110, 0, 0) ); - Assert.assertEquals(new NotEnoughMemoryFault(580_006_666, 12_8000_000, 96_000_000, 1, 4), e2.getFault()); - final MSQFault fault = Assert.assertThrows(MSQException.class, () -> create(1_000_000_000, 2, 32, 1, 0, 0)) + Assert.assertEquals(new TooManyWorkersFault(110, 109), e2.getFault()); + } + + @Test + public void test_oneWorkerInJvm_smallWorkerCapacity() + { + // Supersorter max channels per processer are one less than they are usually to account for extra frames that are required while creating composing output channels + Assert.assertEquals(params(41_200_000, 1, 3, 9_600_000), create(128_000_000, 1, 1, 1, 1, 0, 0)); + Assert.assertEquals(params(26_800_000, 1, 1, 9_600_000), create(128_000_000, 1, 2, 1, 1, 0, 0)); + + final MSQException e = Assert.assertThrows( + MSQException.class, + () -> create(1_000_000_000, 1, 32, 1, 1, 0, 0) + ); + Assert.assertEquals(new NotEnoughMemoryFault(1_588_044_000, 1_000_000_000, 750_000_000, 1, 32, 1), e.getFault()); + + final MSQException e2 = Assert.assertThrows( + MSQException.class, + () -> create(128_000_000, 1, 4, 1, 1, 0, 0) + ); + Assert.assertEquals(new NotEnoughMemoryFault(580_006_666, 12_8000_000, 96_000_000, 1, 4, 1), e2.getFault()); + + final MSQFault fault = Assert.assertThrows(MSQException.class, () -> create(1_000_000_000, 2, 32, 1, 1, 0, 0)) .getFault(); - Assert.assertEquals(new NotEnoughMemoryFault(2024045333, 1_000_000_000, 750_000_000, 2, 32), fault); + Assert.assertEquals(new NotEnoughMemoryFault(2024045333, 1_000_000_000, 750_000_000, 2, 32, 1), fault); } @Test public void test_fourWorkersInJvm_twoHundredWorkersInCluster_hashPartitions() { - Assert.assertEquals(params(814_000_000, 1, 150, 168_750_000), create(9_000_000_000L, 4, 1, 200, 200, 0)); - Assert.assertEquals(params(611_500_000, 2, 62, 168_750_000), create(9_000_000_000L, 4, 2, 200, 200, 0)); - Assert.assertEquals(params(358_375_000, 4, 22, 168_750_000), create(9_000_000_000L, 4, 4, 200, 200, 0)); - Assert.assertEquals(params(105_250_000, 4, 14, 168_750_000), create(9_000_000_000L, 4, 8, 200, 200, 0)); + Assert.assertEquals(params(814_000_000, 1, 150, 168_750_000), create(9_000_000_000L, 4, 1, 1, 200, 200, 0)); + Assert.assertEquals(params(611_500_000, 2, 62, 168_750_000), create(9_000_000_000L, 4, 2, 1, 200, 200, 0)); + Assert.assertEquals(params(358_375_000, 4, 22, 168_750_000), create(9_000_000_000L, 4, 4, 1, 200, 200, 0)); + Assert.assertEquals(params(105_250_000, 4, 14, 168_750_000), create(9_000_000_000L, 4, 8, 1, 200, 200, 0)); final MSQException e = Assert.assertThrows( MSQException.class, - () -> create(9_000_000_000L, 4, 16, 200, 200, 0) + () -> create(9_000_000_000L, 4, 16, 1, 200, 200, 0) ); Assert.assertEquals(new TooManyWorkersFault(200, 138), e.getFault()); // Make sure 138 actually works, and 139 doesn't. (Verify the error message above.) - Assert.assertEquals(params(26_750_000, 4, 8, 168_750_000), create(9_000_000_000L, 4, 16, 138, 138, 0)); + Assert.assertEquals(params(26_750_000, 4, 8, 168_750_000), create(9_000_000_000L, 4, 16, 1, 138, 138, 0)); final MSQException e2 = Assert.assertThrows( MSQException.class, - () -> create(9_000_000_000L, 4, 16, 139, 139, 0) + () -> create(9_000_000_000L, 4, 16, 1, 139, 139, 0) ); Assert.assertEquals(new TooManyWorkersFault(139, 138), e2.getFault()); @@ -148,10 +194,10 @@ public class WorkerMemoryParametersTest { final MSQException e = Assert.assertThrows( MSQException.class, - () -> WorkerMemoryParameters.createInstance(1, 1, 1, 32, 1, 1) + () -> WorkerMemoryParameters.createInstance(1, 1, 1, 1, 32, 1, 1) ); - Assert.assertEquals(new NotEnoughMemoryFault(554669334, 1, 1, 1, 1), e.getFault()); + Assert.assertEquals(new NotEnoughMemoryFault(554669334, 1, 1, 1, 1, 1), e.getFault()); } @Test @@ -179,6 +225,7 @@ public class WorkerMemoryParametersTest final long maxMemoryInJvm, final int numWorkersInJvm, final int numProcessingThreadsInJvm, + final int maxConcurrentStages, final int numInputWorkers, final int numHashOutputPartitions, final int totalLookUpFootprint @@ -188,6 +235,7 @@ public class WorkerMemoryParametersTest maxMemoryInJvm, numWorkersInJvm, numProcessingThreadsInJvm, + maxConcurrentStages, numInputWorkers, numHashOutputPartitions, totalLookUpFootprint diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/IndexerWorkerContextTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/IndexerWorkerContextTest.java index 583c21d3407..dfb88d17b21 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/IndexerWorkerContextTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/IndexerWorkerContextTest.java @@ -19,6 +19,7 @@ package org.apache.druid.msq.indexing; +import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.Futures; import com.google.inject.Injector; import org.apache.druid.indexing.common.SegmentCacheManagerFactory; @@ -30,6 +31,7 @@ import org.apache.druid.rpc.ServiceLocator; import org.junit.Before; import org.junit.Test; import org.mockito.Mockito; +import org.mockito.quality.Strictness; import java.util.Collections; @@ -44,12 +46,19 @@ public class IndexerWorkerContextTest Mockito.when(injectorMock.getInstance(SegmentCacheManagerFactory.class)) .thenReturn(Mockito.mock(SegmentCacheManagerFactory.class)); + final MSQWorkerTask task = + Mockito.mock(MSQWorkerTask.class, Mockito.withSettings().strictness(Strictness.STRICT_STUBS)); + Mockito.when(task.getContext()).thenReturn(ImmutableMap.of()); + indexerWorkerContext = new IndexerWorkerContext( + task, Mockito.mock(TaskToolbox.class), injectorMock, null, null, null, + null, + null, null ); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/WorkerChatHandlerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/WorkerChatHandlerTest.java index 5d86abd129c..ccf91acb666 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/WorkerChatHandlerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/WorkerChatHandlerTest.java @@ -19,12 +19,8 @@ package org.apache.druid.msq.indexing; -import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.frame.key.ClusterByPartitions; -import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.task.NoopTestTaskReportFileWriter; -import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.ISE; import org.apache.druid.msq.counters.CounterSnapshotsTree; import org.apache.druid.msq.exec.Worker; @@ -32,12 +28,9 @@ import org.apache.druid.msq.indexing.client.WorkerChatHandler; import org.apache.druid.msq.kernel.StageId; import org.apache.druid.msq.kernel.WorkOrder; import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; -import org.apache.druid.segment.IndexIO; -import org.apache.druid.segment.IndexMergerV9; -import org.apache.druid.segment.column.ColumnConfig; -import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthenticationResult; +import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.sql.calcite.util.CalciteTests; import org.junit.After; import org.junit.Assert; @@ -51,15 +44,16 @@ import javax.annotation.Nullable; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.core.Response; import java.io.InputStream; -import java.util.HashMap; public class WorkerChatHandlerTest { private static final StageId TEST_STAGE = new StageId("123", 0); + private static final String DATASOURCE = "foo"; + @Mock private HttpServletRequest req; - private TaskToolbox toolbox; + private AuthorizerMapper authorizerMapper; private AutoCloseable mocks; private final TestWorker worker = new TestWorker(); @@ -67,29 +61,16 @@ public class WorkerChatHandlerTest @Before public void setUp() { - ObjectMapper mapper = new DefaultObjectMapper(); - IndexIO indexIO = new IndexIO(mapper, ColumnConfig.DEFAULT); - IndexMergerV9 indexMerger = new IndexMergerV9( - mapper, - indexIO, - OffHeapMemorySegmentWriteOutMediumFactory.instance() - ); - + authorizerMapper = CalciteTests.TEST_AUTHORIZER_MAPPER; mocks = MockitoAnnotations.openMocks(this); Mockito.when(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) .thenReturn(new AuthenticationResult("druid", "druid", null, null)); - TaskToolbox.Builder builder = new TaskToolbox.Builder(); - toolbox = builder.authorizerMapper(CalciteTests.TEST_AUTHORIZER_MAPPER) - .indexIO(indexIO) - .indexMergerV9(indexMerger) - .taskReportFileWriter(new NoopTestTaskReportFileWriter()) - .build(); } @Test public void testFetchSnapshot() { - WorkerChatHandler chatHandler = new WorkerChatHandler(toolbox, worker); + WorkerChatHandler chatHandler = new WorkerChatHandler(worker, authorizerMapper, DATASOURCE); Assert.assertEquals( ClusterByStatisticsSnapshot.empty(), chatHandler.httpFetchKeyStatistics(TEST_STAGE.getQueryId(), TEST_STAGE.getStageNumber(), null, req) @@ -100,7 +81,7 @@ public class WorkerChatHandlerTest @Test public void testFetchSnapshot404() { - WorkerChatHandler chatHandler = new WorkerChatHandler(toolbox, worker); + WorkerChatHandler chatHandler = new WorkerChatHandler(worker, authorizerMapper, DATASOURCE); Assert.assertEquals( Response.Status.BAD_REQUEST.getStatusCode(), chatHandler.httpFetchKeyStatistics("123", 2, null, req) @@ -111,7 +92,7 @@ public class WorkerChatHandlerTest @Test public void testFetchSnapshotWithTimeChunk() { - WorkerChatHandler chatHandler = new WorkerChatHandler(toolbox, worker); + WorkerChatHandler chatHandler = new WorkerChatHandler(worker, authorizerMapper, DATASOURCE); Assert.assertEquals( ClusterByStatisticsSnapshot.empty(), chatHandler.httpFetchKeyStatisticsWithSnapshot(TEST_STAGE.getQueryId(), TEST_STAGE.getStageNumber(), 1, null, req) @@ -122,7 +103,7 @@ public class WorkerChatHandlerTest @Test public void testFetchSnapshotWithTimeChunk404() { - WorkerChatHandler chatHandler = new WorkerChatHandler(toolbox, worker); + WorkerChatHandler chatHandler = new WorkerChatHandler(worker, authorizerMapper, DATASOURCE); Assert.assertEquals( Response.Status.BAD_REQUEST.getStatusCode(), chatHandler.httpFetchKeyStatisticsWithSnapshot("123", 2, 1, null, req) @@ -133,7 +114,6 @@ public class WorkerChatHandlerTest private static class TestWorker implements Worker { - @Override public String id() { @@ -141,19 +121,13 @@ public class WorkerChatHandlerTest } @Override - public MSQWorkerTask task() + public void run() { - return new MSQWorkerTask("controller", "ds", 1, new HashMap<>(), 0); + } @Override - public TaskStatus run() - { - return null; - } - - @Override - public void stopGracefully() + public void stop() { } @@ -164,6 +138,12 @@ public class WorkerChatHandlerTest } + @Override + public void awaitStop() + { + + } + @Override public void postWorkOrder(WorkOrder workOrder) { @@ -192,9 +172,8 @@ public class WorkerChatHandlerTest @Override public boolean postResultPartitionBoundaries( - ClusterByPartitions stagePartitionBoundaries, - String queryId, - int stageNumber + StageId stageId, + ClusterByPartitions stagePartitionBoundaries ) { return false; @@ -202,7 +181,7 @@ public class WorkerChatHandlerTest @Nullable @Override - public InputStream readChannel(String queryId, int stageNumber, int partitionNumber, long offset) + public ListenableFuture readStageOutput(StageId stageId, int partitionNumber, long offset) { return null; } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java index c33faa40c14..cffc0f78a49 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java @@ -74,7 +74,7 @@ public class MSQFaultSerdeTest )); assertFaultSerde(new InvalidNullByteFault("the source", 1, "the column", "the value", 2)); assertFaultSerde(new InvalidFieldFault("the source", "the column", 1, "the error", "the log msg")); - assertFaultSerde(new NotEnoughMemoryFault(1000, 1000, 900, 1, 2)); + assertFaultSerde(new NotEnoughMemoryFault(1000, 1000, 900, 1, 2, 2)); assertFaultSerde(QueryNotSupportedFault.INSTANCE); assertFaultSerde(new QueryRuntimeFault("new error", "base error")); assertFaultSerde(new QueryRuntimeFault("new error", null)); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/output/ByteChunksInputStreamTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/output/ByteChunksInputStreamTest.java new file mode 100644 index 00000000000..bc349d56c8f --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/output/ByteChunksInputStreamTest.java @@ -0,0 +1,149 @@ +/* + * 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.shuffle.output; + +import com.google.common.collect.ImmutableList; +import org.hamcrest.MatcherAssert; +import org.hamcrest.Matchers; +import org.junit.Assert; +import org.junit.Test; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.List; + +public class ByteChunksInputStreamTest +{ + private final List chunks = ImmutableList.of( + new byte[]{-128, -127, -1, 0, 1, 126, 127}, + new byte[]{0}, + new byte[]{3, 4, 5} + ); + + @Test + public void test_read_fromStart() throws IOException + { + try (final InputStream in = new ByteChunksInputStream(chunks, 0)) { + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + + int c; + while ((c = in.read()) != -1) { + MatcherAssert.assertThat("InputStream#read contract", c, Matchers.greaterThanOrEqualTo(0)); + baos.write(c); + } + + Assert.assertArrayEquals(chunksSubset(0), baos.toByteArray()); + } + } + + @Test + public void test_read_fromSecondByte() throws IOException + { + try (final InputStream in = new ByteChunksInputStream(chunks, 1)) { + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + + int c; + while ((c = in.read()) != -1) { + MatcherAssert.assertThat("InputStream#read contract", c, Matchers.greaterThanOrEqualTo(0)); + baos.write(c); + } + + Assert.assertArrayEquals(chunksSubset(1), baos.toByteArray()); + } + } + + @Test + public void test_read_array1_fromStart() throws IOException + { + try (final InputStream in = new ByteChunksInputStream(chunks, 0)) { + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + final byte[] buf = new byte[2]; + + int r; + while ((r = in.read(buf, 1, 1)) != -1) { + Assert.assertEquals("InputStream#read bytes read", 1, r); + baos.write(buf, 1, 1); + } + + Assert.assertArrayEquals(chunksSubset(0), baos.toByteArray()); + } + } + + @Test + public void test_read_array1_fromSecondByte() throws IOException + { + try (final InputStream in = new ByteChunksInputStream(chunks, 1)) { + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + final byte[] buf = new byte[2]; + + int r; + while ((r = in.read(buf, 1, 1)) != -1) { + Assert.assertEquals("InputStream#read bytes read", 1, r); + baos.write(buf, 1, 1); + } + + Assert.assertArrayEquals(chunksSubset(1), baos.toByteArray()); + } + } + + @Test + public void test_read_array3_fromStart() throws IOException + { + try (final InputStream in = new ByteChunksInputStream(chunks, 0)) { + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + final byte[] buf = new byte[5]; + + int r; + while ((r = in.read(buf, 2, 3)) != -1) { + baos.write(buf, 2, r); + } + + Assert.assertArrayEquals(chunksSubset(0), baos.toByteArray()); + } + } + + @Test + public void test_read_array3_fromSecondByte() throws IOException + { + try (final InputStream in = new ByteChunksInputStream(chunks, 1)) { + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + final byte[] buf = new byte[6]; + + int r; + while ((r = in.read(buf, 2, 3)) != -1) { + baos.write(buf, 2, r); + } + + Assert.assertArrayEquals(chunksSubset(1), baos.toByteArray()); + } + } + + private byte[] chunksSubset(final int positionInFirstChunk) + { + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + + for (int chunk = 0, p = positionInFirstChunk; chunk < chunks.size(); chunk++, p = 0) { + baos.write(chunks.get(chunk), p, chunks.get(chunk).length - p); + } + + return baos.toByteArray(); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/output/ChannelStageOutputReaderTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/output/ChannelStageOutputReaderTest.java new file mode 100644 index 00000000000..927372a3a6a --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/output/ChannelStageOutputReaderTest.java @@ -0,0 +1,255 @@ +/* + * 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.shuffle.output; + +import com.google.common.io.ByteStreams; +import com.google.common.math.IntMath; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.FrameType; +import org.apache.druid.frame.channel.BlockingQueueFrameChannel; +import org.apache.druid.frame.channel.ReadableFileFrameChannel; +import org.apache.druid.frame.file.FrameFile; +import org.apache.druid.frame.read.FrameReader; +import org.apache.druid.frame.testutil.FrameSequenceBuilder; +import org.apache.druid.frame.testutil.FrameTestUtil; +import org.apache.druid.segment.TestIndex; +import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.hamcrest.CoreMatchers; +import org.hamcrest.MatcherAssert; +import org.hamcrest.Matchers; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.internal.matchers.ThrowableCauseMatcher; +import org.junit.internal.matchers.ThrowableMessageMatcher; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.math.RoundingMode; +import java.util.List; + +public class ChannelStageOutputReaderTest extends InitializedNullHandlingTest +{ + private static final int MAX_FRAMES = 10; + private static final int EXPECTED_NUM_ROWS = 1209; + + private final BlockingQueueFrameChannel channel = new BlockingQueueFrameChannel(MAX_FRAMES); + private final ChannelStageOutputReader reader = new ChannelStageOutputReader(channel.readable()); + + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private FrameReader frameReader; + private List frameList; + + @Before + public void setUp() + { + final IncrementalIndex index = TestIndex.getIncrementalTestIndex(); + final IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index); + frameReader = FrameReader.create(adapter.getRowSignature()); + frameList = FrameSequenceBuilder.fromAdapter(adapter) + .frameType(FrameType.ROW_BASED) + .maxRowsPerFrame(IntMath.divide(index.size(), MAX_FRAMES, RoundingMode.CEILING)) + .frames() + .toList(); + } + + @After + public void tearDown() + { + reader.close(); + } + + @Test + public void test_readLocally() throws IOException + { + writeAllFramesToChannel(); + + Assert.assertSame(channel.readable(), reader.readLocally()); + reader.close(); // Won't close the channel, because it's already been returned by readLocally + + final int numRows = FrameTestUtil.readRowsFromFrameChannel(channel.readable(), frameReader).toList().size(); + Assert.assertEquals(EXPECTED_NUM_ROWS, numRows); + } + + @Test + public void test_readLocally_closePriorToRead() throws IOException + { + writeAllFramesToChannel(); + + reader.close(); + + // Can't read the channel after closing the reader + Assert.assertThrows( + IllegalStateException.class, + reader::readLocally + ); + } + + @Test + public void test_readLocally_thenReadRemotely() throws IOException + { + writeAllFramesToChannel(); + + Assert.assertSame(channel.readable(), reader.readLocally()); + + // Can't read remotely after reading locally + Assert.assertThrows( + IllegalStateException.class, + () -> reader.readRemotelyFrom(0) + ); + + // Can still read locally after this error + final int numRows = FrameTestUtil.readRowsFromFrameChannel(channel.readable(), frameReader).toList().size(); + Assert.assertEquals(EXPECTED_NUM_ROWS, numRows); + } + + @Test + public void test_readRemotely_strideBasedOnReturnedChunk() throws IOException + { + // Test that reads entire chunks from readRemotelyFrom. This is a typical usage pattern. + + writeAllFramesToChannel(); + + final File tmpFile = temporaryFolder.newFile(); + + try (final FileOutputStream tmpOut = new FileOutputStream(tmpFile)) { + int numReads = 0; + long offset = 0; + + while (true) { + try (final InputStream in = FutureUtils.getUnchecked(reader.readRemotelyFrom(offset), true)) { + numReads++; + final long bytesWritten = ByteStreams.copy(in, tmpOut); + offset += bytesWritten; + + if (bytesWritten == 0) { + break; + } + } + } + + MatcherAssert.assertThat(numReads, Matchers.greaterThan(1)); + } + + final FrameFile frameFile = FrameFile.open(tmpFile, null); + final int numRows = + FrameTestUtil.readRowsFromFrameChannel(new ReadableFileFrameChannel(frameFile), frameReader).toList().size(); + + Assert.assertEquals(EXPECTED_NUM_ROWS, numRows); + } + + @Test + public void test_readRemotely_strideOneByte() throws IOException + { + // Test that reads one byte at a time from readRemotelyFrom. This helps ensure that there are no edge cases + // in the chunk-reading logic. + + writeAllFramesToChannel(); + + final File tmpFile = temporaryFolder.newFile(); + + try (final FileOutputStream tmpOut = new FileOutputStream(tmpFile)) { + int numReads = 0; + long offset = 0; + + while (true) { + try (final InputStream in = FutureUtils.getUnchecked(reader.readRemotelyFrom(offset), true)) { + numReads++; + final int nextByte = in.read(); + + if (nextByte < 0) { + break; + } + + tmpOut.write(nextByte); + offset++; + } + } + + Assert.assertEquals(numReads, offset + 1); + } + + final FrameFile frameFile = FrameFile.open(tmpFile, null); + final int numRows = + FrameTestUtil.readRowsFromFrameChannel(new ReadableFileFrameChannel(frameFile), frameReader).toList().size(); + + Assert.assertEquals(EXPECTED_NUM_ROWS, numRows); + } + + @Test + public void test_readRemotely_thenLocally() throws IOException + { + writeAllFramesToChannel(); + + // Read remotely + FutureUtils.getUnchecked(reader.readRemotelyFrom(0), true); + + // Then read locally + Assert.assertThrows( + IllegalStateException.class, + reader::readLocally + ); + } + + @Test + public void test_readRemotely_cannotReverse() throws IOException + { + writeAllFramesToChannel(); + + // Read remotely from offset = 1. + final InputStream in = FutureUtils.getUnchecked(reader.readRemotelyFrom(1), true); + final int offset = ByteStreams.toByteArray(in).length; + MatcherAssert.assertThat(offset, Matchers.greaterThan(0)); + + // Then read again from offset = 0; should get an error. + final RuntimeException e = Assert.assertThrows( + RuntimeException.class, + () -> FutureUtils.getUnchecked(reader.readRemotelyFrom(0), true) + ); + + MatcherAssert.assertThat( + e, + ThrowableCauseMatcher.hasCause( + Matchers.allOf( + CoreMatchers.instanceOf(IllegalStateException.class), + ThrowableMessageMatcher.hasMessage(CoreMatchers.startsWith("Offset[0] no longer available")) + ) + ) + ); + } + + private void writeAllFramesToChannel() throws IOException + { + for (Frame frame : frameList) { + channel.writable().write(frame); + } + channel.writable().close(); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteArraysQueryMSQTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteArraysQueryMSQTest.java index b60c6c71d2e..124b4fce258 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteArraysQueryMSQTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteArraysQueryMSQTest.java @@ -64,15 +64,7 @@ public class CalciteArraysQueryMSQTest extends CalciteArraysQueryTest Injector injector ) { - final WorkerMemoryParameters workerMemoryParameters = - WorkerMemoryParameters.createInstance( - WorkerMemoryParameters.PROCESSING_MINIMUM_BYTES * 50, - 2, - 10, - 2, - 0, - 0 - ); + final WorkerMemoryParameters workerMemoryParameters = MSQTestBase.makeTestWorkerMemoryParameters(); final MSQTestOverlordServiceClient indexingServiceClient = new MSQTestOverlordServiceClient( queryJsonMapper, injector, diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteNestedDataQueryMSQTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteNestedDataQueryMSQTest.java index 2d8067e900e..5d4c0994ea0 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteNestedDataQueryMSQTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteNestedDataQueryMSQTest.java @@ -67,15 +67,7 @@ public class CalciteNestedDataQueryMSQTest extends CalciteNestedDataQueryTest Injector injector ) { - final WorkerMemoryParameters workerMemoryParameters = - WorkerMemoryParameters.createInstance( - WorkerMemoryParameters.PROCESSING_MINIMUM_BYTES * 50, - 2, - 10, - 2, - 0, - 0 - ); + final WorkerMemoryParameters workerMemoryParameters = MSQTestBase.makeTestWorkerMemoryParameters(); final MSQTestOverlordServiceClient indexingServiceClient = new MSQTestOverlordServiceClient( queryJsonMapper, injector, diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectJoinQueryMSQTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectJoinQueryMSQTest.java index 317fe30a646..6bbf9c6da5e 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectJoinQueryMSQTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectJoinQueryMSQTest.java @@ -136,15 +136,7 @@ public class CalciteSelectJoinQueryMSQTest Injector injector ) { - final WorkerMemoryParameters workerMemoryParameters = - WorkerMemoryParameters.createInstance( - WorkerMemoryParameters.PROCESSING_MINIMUM_BYTES * 50, - 2, - 10, - 2, - 0, - 0 - ); + final WorkerMemoryParameters workerMemoryParameters = MSQTestBase.makeTestWorkerMemoryParameters(); final MSQTestOverlordServiceClient indexingServiceClient = new MSQTestOverlordServiceClient( queryJsonMapper, injector, diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectQueryMSQTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectQueryMSQTest.java index 3008f9d43b4..2de9229b4ad 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectQueryMSQTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectQueryMSQTest.java @@ -73,15 +73,7 @@ public class CalciteSelectQueryMSQTest extends CalciteQueryTest Injector injector ) { - final WorkerMemoryParameters workerMemoryParameters = - WorkerMemoryParameters.createInstance( - WorkerMemoryParameters.PROCESSING_MINIMUM_BYTES * 50, - 2, - 10, - 2, - 0, - 0 - ); + final WorkerMemoryParameters workerMemoryParameters = MSQTestBase.makeTestWorkerMemoryParameters(); final MSQTestOverlordServiceClient indexingServiceClient = new MSQTestOverlordServiceClient( queryJsonMapper, injector, diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteUnionQueryMSQTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteUnionQueryMSQTest.java index b5d8368b068..e4b678402a8 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteUnionQueryMSQTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteUnionQueryMSQTest.java @@ -79,15 +79,7 @@ public class CalciteUnionQueryMSQTest extends CalciteUnionQueryTest Injector injector ) { - final WorkerMemoryParameters workerMemoryParameters = - WorkerMemoryParameters.createInstance( - WorkerMemoryParameters.PROCESSING_MINIMUM_BYTES * 50, - 2, - 10, - 2, - 0, - 0 - ); + final WorkerMemoryParameters workerMemoryParameters = MSQTestBase.makeTestWorkerMemoryParameters(); final MSQTestOverlordServiceClient indexingServiceClient = new MSQTestOverlordServiceClient( queryJsonMapper, injector, diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index 5f0bd545b7c..2136d96d6d1 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -333,16 +333,7 @@ public class MSQTestBase extends BaseCalciteQueryTest private SegmentCacheManager segmentCacheManager; private TestGroupByBuffers groupByBuffers; - protected final WorkerMemoryParameters workerMemoryParameters = Mockito.spy( - WorkerMemoryParameters.createInstance( - WorkerMemoryParameters.PROCESSING_MINIMUM_BYTES * 50, - 2, - 10, - 2, - 1, - 0 - ) - ); + protected final WorkerMemoryParameters workerMemoryParameters = Mockito.spy(makeTestWorkerMemoryParameters()); protected static class MSQBaseComponentSupplier extends StandardComponentSupplier { @@ -753,6 +744,19 @@ public class MSQTestBase extends BaseCalciteQueryTest return mapper; } + public static WorkerMemoryParameters makeTestWorkerMemoryParameters() + { + return WorkerMemoryParameters.createInstance( + WorkerMemoryParameters.PROCESSING_MINIMUM_BYTES * 50, + 2, + 10, + 1, + 2, + 1, + 0 + ); + } + private String runMultiStageQuery(String query, Map context) { final DirectStatement stmt = sqlStatementFactory.directStatement( diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerClient.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerClient.java index 96e26cba77e..4c7ca61be02 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerClient.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerClient.java @@ -87,9 +87,9 @@ public class MSQTestControllerClient implements ControllerClient } @Override - public List getTaskList() + public List getWorkerIds() { - return controller.getTaskIds(); + return controller.getWorkerIds(); } @Override diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java index 20d31fbd4cf..e6510430203 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java @@ -156,32 +156,33 @@ public class MSQTestControllerContext implements ControllerContext Worker worker = new WorkerImpl( task, new MSQTestWorkerContext( + task.getId(), inMemoryWorkers, controller, mapper, injector, - workerMemoryParameters - ), - workerStorageParameters + workerMemoryParameters, + workerStorageParameters + ) ); inMemoryWorkers.put(task.getId(), worker); statusMap.put(task.getId(), TaskStatus.running(task.getId())); - ListenableFuture future = executor.submit(() -> { + ListenableFuture future = executor.submit(() -> { try { - return worker.run(); + worker.run(); } catch (Exception e) { throw new RuntimeException(e); } }); - Futures.addCallback(future, new FutureCallback() + Futures.addCallback(future, new FutureCallback() { @Override - public void onSuccess(@Nullable TaskStatus result) + public void onSuccess(@Nullable Object result) { - statusMap.put(task.getId(), result); + statusMap.put(task.getId(), TaskStatus.success(task.getId())); } @Override @@ -261,7 +262,7 @@ public class MSQTestControllerContext implements ControllerContext { final Worker worker = inMemoryWorkers.remove(workerId); if (worker != null) { - worker.stopGracefully(); + worker.stop(); } return Futures.immediateFuture(null); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerClient.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerClient.java index 72cb246a43e..65145b5f5c0 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerClient.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerClient.java @@ -80,11 +80,7 @@ public class MSQTestWorkerClient implements WorkerClient ) { try { - inMemoryWorkers.get(workerTaskId).postResultPartitionBoundaries( - partitionBoundaries, - stageId.getQueryId(), - stageId.getStageNumber() - ); + inMemoryWorkers.get(workerTaskId).postResultPartitionBoundaries(stageId, partitionBoundaries); return Futures.immediateFuture(null); } catch (Exception e) { @@ -122,8 +118,7 @@ public class MSQTestWorkerClient implements WorkerClient ) { try (InputStream inputStream = - inMemoryWorkers.get(workerTaskId) - .readChannel(stageId.getQueryId(), stageId.getStageNumber(), partitionNumber, offset)) { + inMemoryWorkers.get(workerTaskId).readStageOutput(stageId, partitionNumber, offset).get()) { byte[] buffer = new byte[8 * 1024]; boolean didRead = false; int bytesRead; @@ -138,12 +133,11 @@ public class MSQTestWorkerClient implements WorkerClient catch (Exception e) { throw new ISE(e, "Error reading frame file channel"); } - } @Override public void close() { - inMemoryWorkers.forEach((k, v) -> v.stopGracefully()); + inMemoryWorkers.forEach((k, v) -> v.stop()); } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java index 14f6f73b24a..082429a9d7b 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java @@ -22,59 +22,69 @@ package org.apache.druid.msq.test; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Injector; import org.apache.druid.frame.processor.Bouncer; -import org.apache.druid.indexer.report.TaskReportFileWriter; -import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.task.NoopTestTaskReportFileWriter; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.msq.exec.Controller; import org.apache.druid.msq.exec.ControllerClient; import org.apache.druid.msq.exec.DataServerQueryHandlerFactory; +import org.apache.druid.msq.exec.OutputChannelMode; 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.indexing.IndexerFrameContext; -import org.apache.druid.msq.indexing.IndexerWorkerContext; +import org.apache.druid.msq.exec.WorkerStorageParameters; import org.apache.druid.msq.kernel.FrameContext; import org.apache.druid.msq.kernel.QueryDefinition; import org.apache.druid.msq.querykit.DataSegmentProvider; +import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9; +import org.apache.druid.segment.SegmentWrangler; import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.incremental.NoopRowIngestionMeters; +import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.loading.DataSegmentPusher; -import org.apache.druid.segment.realtime.NoopChatHandlerProvider; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.server.DruidNode; -import org.apache.druid.server.coordination.DataSegmentAnnouncer; -import org.apache.druid.server.security.AuthTestUtils; import java.io.File; import java.util.Map; public class MSQTestWorkerContext implements WorkerContext { + private final String workerId; private final Controller controller; private final ObjectMapper mapper; private final Injector injector; private final Map inMemoryWorkers; private final File file = FileUtils.createTempDir(); + private final Bouncer bouncer = new Bouncer(1); private final WorkerMemoryParameters workerMemoryParameters; + private final WorkerStorageParameters workerStorageParameters; public MSQTestWorkerContext( + String workerId, Map inMemoryWorkers, Controller controller, ObjectMapper mapper, Injector injector, - WorkerMemoryParameters workerMemoryParameters + WorkerMemoryParameters workerMemoryParameters, + WorkerStorageParameters workerStorageParameters ) { + this.workerId = workerId; this.inMemoryWorkers = inMemoryWorkers; this.controller = controller; this.mapper = mapper; this.injector = injector; this.workerMemoryParameters = workerMemoryParameters; + this.workerStorageParameters = workerStorageParameters; + } + + @Override + public String queryId() + { + return controller.queryId(); } @Override @@ -96,7 +106,13 @@ public class MSQTestWorkerContext implements WorkerContext } @Override - public ControllerClient makeControllerClient(String controllerId) + public String workerId() + { + return workerId; + } + + @Override + public ControllerClient makeControllerClient() { return new MSQTestControllerClient(controller); } @@ -114,42 +130,9 @@ public class MSQTestWorkerContext implements WorkerContext } @Override - public FrameContext frameContext(QueryDefinition queryDef, int stageNumber) + public FrameContext frameContext(QueryDefinition queryDef, int stageNumber, OutputChannelMode outputChannelMode) { - IndexIO indexIO = new IndexIO(mapper, ColumnConfig.DEFAULT); - IndexMergerV9 indexMerger = new IndexMergerV9( - mapper, - indexIO, - OffHeapMemorySegmentWriteOutMediumFactory.instance(), - true - ); - final TaskReportFileWriter reportFileWriter = new NoopTestTaskReportFileWriter(); - - return new IndexerFrameContext( - new IndexerWorkerContext( - new TaskToolbox.Builder() - .segmentPusher(injector.getInstance(DataSegmentPusher.class)) - .segmentAnnouncer(injector.getInstance(DataSegmentAnnouncer.class)) - .jsonMapper(mapper) - .taskWorkDir(tempDir()) - .indexIO(indexIO) - .indexMergerV9(indexMerger) - .taskReportFileWriter(reportFileWriter) - .authorizerMapper(AuthTestUtils.TEST_AUTHORIZER_MAPPER) - .chatHandlerProvider(new NoopChatHandlerProvider()) - .rowIngestionMetersFactory(NoopRowIngestionMeters::new) - .build(), - injector, - indexIO, - null, - null, - null - ), - indexIO, - injector.getInstance(DataSegmentProvider.class), - injector.getInstance(DataServerQueryHandlerFactory.class), - workerMemoryParameters - ); + return new FrameContextImpl(new File(tempDir(), queryDef.getStageDefinition(stageNumber).getId().toString())); } @Override @@ -165,9 +148,9 @@ public class MSQTestWorkerContext implements WorkerContext } @Override - public Bouncer processorBouncer() + public int maxConcurrentStages() { - return injector.getInstance(Bouncer.class); + return 1; } @Override @@ -175,4 +158,109 @@ public class MSQTestWorkerContext implements WorkerContext { return injector.getInstance(DataServerQueryHandlerFactory.class); } + + class FrameContextImpl implements FrameContext + { + private final File tempDir; + + public FrameContextImpl(File tempDir) + { + this.tempDir = tempDir; + } + + @Override + public SegmentWrangler segmentWrangler() + { + return injector.getInstance(SegmentWrangler.class); + } + + @Override + public GroupingEngine groupingEngine() + { + return injector.getInstance(GroupingEngine.class); + } + + @Override + public RowIngestionMeters rowIngestionMeters() + { + return new NoopRowIngestionMeters(); + } + + @Override + public DataSegmentProvider dataSegmentProvider() + { + return injector.getInstance(DataSegmentProvider.class); + } + + @Override + public DataServerQueryHandlerFactory dataServerQueryHandlerFactory() + { + return injector.getInstance(DataServerQueryHandlerFactory.class); + } + + @Override + public File tempDir() + { + return new File(tempDir, "tmp"); + } + + @Override + public ObjectMapper jsonMapper() + { + return mapper; + } + + @Override + public IndexIO indexIO() + { + return new IndexIO(mapper, ColumnConfig.DEFAULT); + } + + @Override + public File persistDir() + { + return new File(tempDir, "persist"); + } + + @Override + public DataSegmentPusher segmentPusher() + { + return injector.getInstance(DataSegmentPusher.class); + } + + @Override + public IndexMergerV9 indexMerger() + { + return new IndexMergerV9( + mapper, + indexIO(), + OffHeapMemorySegmentWriteOutMediumFactory.instance(), + true + ); + } + + @Override + public Bouncer processorBouncer() + { + return bouncer; + } + + @Override + public WorkerMemoryParameters memoryParameters() + { + return workerMemoryParameters; + } + + @Override + public WorkerStorageParameters storageParameters() + { + return workerStorageParameters; + } + + @Override + public void close() + { + + } + } } diff --git a/processing/src/main/java/org/apache/druid/frame/channel/ReadableFileFrameChannel.java b/processing/src/main/java/org/apache/druid/frame/channel/ReadableFileFrameChannel.java index 963a001ad6d..7da6550ccca 100644 --- a/processing/src/main/java/org/apache/druid/frame/channel/ReadableFileFrameChannel.java +++ b/processing/src/main/java/org/apache/druid/frame/channel/ReadableFileFrameChannel.java @@ -104,6 +104,14 @@ public class ReadableFileFrameChannel implements ReadableFrameChannel } } + /** + * Returns whether this channel represents the entire underlying {@link FrameFile}. + */ + public boolean isEntireFile() + { + return currentFrame == 0 && endFrame == frameFile.numFrames(); + } + /** * Returns a new reference to the {@link FrameFile} that this channel is reading from. Callers should close this * reference when done reading. diff --git a/processing/src/test/java/org/apache/druid/frame/processor/ReadableFileFrameChannelTest.java b/processing/src/test/java/org/apache/druid/frame/processor/ReadableFileFrameChannelTest.java new file mode 100644 index 00000000000..9025d182086 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/frame/processor/ReadableFileFrameChannelTest.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.frame.processor; + +import org.apache.druid.frame.FrameType; +import org.apache.druid.frame.channel.ReadableFileFrameChannel; +import org.apache.druid.frame.file.FrameFile; +import org.apache.druid.frame.read.FrameReader; +import org.apache.druid.frame.testutil.FrameSequenceBuilder; +import org.apache.druid.frame.testutil.FrameTestUtil; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.TestIndex; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.IOException; +import java.util.List; + +public class ReadableFileFrameChannelTest extends InitializedNullHandlingTest +{ + private static final int ROWS_PER_FRAME = 20; + + private List> allRows; + private FrameReader frameReader; + private FrameFile frameFile; + + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + + @Before + public void setUp() throws IOException + { + final StorageAdapter adapter = new QueryableIndexStorageAdapter(TestIndex.getNoRollupMMappedTestIndex()); + final File file = FrameTestUtil.writeFrameFile( + FrameSequenceBuilder.fromAdapter(adapter) + .frameType(FrameType.ROW_BASED) + .maxRowsPerFrame(ROWS_PER_FRAME) + .frames(), + temporaryFolder.newFile() + ); + allRows = FrameTestUtil.readRowsFromAdapter(adapter, adapter.getRowSignature(), false).toList(); + frameReader = FrameReader.create(adapter.getRowSignature()); + frameFile = FrameFile.open(file, null, FrameFile.Flag.DELETE_ON_CLOSE); + } + + @After + public void tearDown() throws Exception + { + frameFile.close(); + } + + @Test + public void test_fullFile() + { + final ReadableFileFrameChannel channel = new ReadableFileFrameChannel(frameFile); + Assert.assertTrue(channel.isEntireFile()); + + FrameTestUtil.assertRowsEqual( + Sequences.simple(allRows), + FrameTestUtil.readRowsFromFrameChannel(channel, frameReader) + ); + + Assert.assertFalse(channel.isEntireFile()); + } + + @Test + public void test_partialFile() + { + final ReadableFileFrameChannel channel = new ReadableFileFrameChannel(frameFile, 1, 2); + Assert.assertFalse(channel.isEntireFile()); + + FrameTestUtil.assertRowsEqual( + Sequences.simple(allRows).skip(ROWS_PER_FRAME).limit(ROWS_PER_FRAME), + FrameTestUtil.readRowsFromFrameChannel(channel, frameReader) + ); + + Assert.assertFalse(channel.isEntireFile()); + } +} From bb4d6cc001f0fe1d7972202fcbf01039e83648d9 Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Thu, 1 Aug 2024 10:26:04 +0530 Subject: [PATCH 04/12] Add task report fields in response of SQL statements endpoint (#16808) If the optional query parameter detail is supplied, then the response also includes the following: * A stages object that summarizes information about the different stages being used for query execution, such as stage number, phase, start time, duration, input and output information, processing methods, and partitioning. * A counters object that provides details on the rows, bytes, and files processed at various stages for each worker across different channels, along with sort progress. * A warnings object that provides details about any warnings. --- docs/api-reference/sql-api.md | 432 +++++++++++++++++- .../msq/counters/CounterSnapshotsTree.java | 10 + .../msq/sql/entity/SqlStatementResult.java | 89 ++-- .../sql/resources/SqlStatementResource.java | 40 +- .../msq/util/SqlStatementResourceHelper.java | 39 +- .../sql/entity/SqlStatementResultTest.java | 13 +- .../SqlMSQStatementResourcePostTest.java | 41 +- .../resources/SqlStatementResourceTest.java | 108 ++++- 8 files changed, 682 insertions(+), 90 deletions(-) diff --git a/docs/api-reference/sql-api.md b/docs/api-reference/sql-api.md index bf58d236426..e1fb53bc6eb 100644 --- a/docs/api-reference/sql-api.md +++ b/docs/api-reference/sql-api.md @@ -629,10 +629,21 @@ Retrieves information about the query associated with the given query ID. The re - `sizeInBytes`: the size of the page. - `id`: the page number that you can use to reference a specific page when you get query results. +If the optional query parameter `detail` is supplied, then the response also includes the following: +- A `stages` object that summarizes information about the different stages being used for query execution, such as stage number, phase, start time, duration, input and output information, processing methods, and partitioning. +- A `counters` object that provides details on the rows, bytes, and files processed at various stages for each worker across different channels, along with sort progress. +- A `warnings` object that provides details about any warnings. + #### URL `GET` `/druid/v2/sql/statements/{queryId}` +#### Query parameters +* `detail` (optional) + * Type: Boolean + * Default: false + * Fetch additional details about the query, which includes the information about different stages, counters for each stage, and any warnings. + #### Responses @@ -672,7 +683,7 @@ The following example retrieves the status of a query with specified ID `query-9 ```shell -curl "http://ROUTER_IP:ROUTER_PORT/druid/v2/sql/statements/query-9b93f6f7-ab0e-48f5-986a-3520f84f0804" +curl "http://ROUTER_IP:ROUTER_PORT/druid/v2/sql/statements/query-9b93f6f7-ab0e-48f5-986a-3520f84f0804?detail=true" ``` @@ -680,7 +691,7 @@ curl "http://ROUTER_IP:ROUTER_PORT/druid/v2/sql/statements/query-9b93f6f7-ab0e-4 ```HTTP -GET /druid/v2/sql/statements/query-9b93f6f7-ab0e-48f5-986a-3520f84f0804 HTTP/1.1 +GET /druid/v2/sql/statements/query-9b93f6f7-ab0e-48f5-986a-3520f84f0804?detail=true HTTP/1.1 Host: http://ROUTER_IP:ROUTER_PORT ``` @@ -835,7 +846,422 @@ Host: http://ROUTER_IP:ROUTER_PORT "sizeInBytes": 375 } ] - } + }, + "stages": [ + { + "stageNumber": 0, + "definition": { + "id": "query-9b93f6f7-ab0e-48f5-986a-3520f84f0804_0", + "input": [ + { + "type": "table", + "dataSource": "wikipedia", + "intervals": [ + "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" + ], + "filter": { + "type": "equals", + "column": "user", + "matchValueType": "STRING", + "matchValue": "BlueMoon2662" + }, + "filterFields": [ + "user" + ] + } + ], + "processor": { + "type": "scan", + "query": { + "queryType": "scan", + "dataSource": { + "type": "inputNumber", + "inputNumber": 0 + }, + "intervals": { + "type": "intervals", + "intervals": [ + "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" + ] + }, + "virtualColumns": [ + { + "type": "expression", + "name": "v0", + "expression": "'BlueMoon2662'", + "outputType": "STRING" + } + ], + "resultFormat": "compactedList", + "limit": 1001, + "filter": { + "type": "equals", + "column": "user", + "matchValueType": "STRING", + "matchValue": "BlueMoon2662" + }, + "columns": [ + "__time", + "added", + "channel", + "cityName", + "comment", + "commentLength", + "countryIsoCode", + "countryName", + "deleted", + "delta", + "deltaBucket", + "diffUrl", + "flags", + "isAnonymous", + "isMinor", + "isNew", + "isRobot", + "isUnpatrolled", + "metroCode", + "namespace", + "page", + "regionIsoCode", + "regionName", + "v0" + ], + "context": { + "__resultFormat": "array", + "__user": "allowAll", + "enableWindowing": true, + "executionMode": "async", + "finalize": true, + "maxNumTasks": 2, + "maxParseExceptions": 0, + "queryId": "33b53acb-7533-4880-a81b-51c16c489eab", + "scanSignature": "[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"added\",\"type\":\"LONG\"},{\"name\":\"channel\",\"type\":\"STRING\"},{\"name\":\"cityName\",\"type\":\"STRING\"},{\"name\":\"comment\",\"type\":\"STRING\"},{\"name\":\"commentLength\",\"type\":\"LONG\"},{\"name\":\"countryIsoCode\",\"type\":\"STRING\"},{\"name\":\"countryName\",\"type\":\"STRING\"},{\"name\":\"deleted\",\"type\":\"LONG\"},{\"name\":\"delta\",\"type\":\"LONG\"},{\"name\":\"deltaBucket\",\"type\":\"LONG\"},{\"name\":\"diffUrl\",\"type\":\"STRING\"},{\"name\":\"flags\",\"type\":\"STRING\"},{\"name\":\"isAnonymous\",\"type\":\"STRING\"},{\"name\":\"isMinor\",\"type\":\"STRING\"},{\"name\":\"isNew\",\"type\":\"STRING\"},{\"name\":\"isRobot\",\"type\":\"STRING\"},{\"name\":\"isUnpatrolled\",\"type\":\"STRING\"},{\"name\":\"metroCode\",\"type\":\"STRING\"},{\"name\":\"namespace\",\"type\":\"STRING\"},{\"name\":\"page\",\"type\":\"STRING\"},{\"name\":\"regionIsoCode\",\"type\":\"STRING\"},{\"name\":\"regionName\",\"type\":\"STRING\"},{\"name\":\"v0\",\"type\":\"STRING\"}]", + "sqlOuterLimit": 1001, + "sqlQueryId": "33b53acb-7533-4880-a81b-51c16c489eab", + "sqlStringifyArrays": false + }, + "columnTypes": [ + "LONG", + "LONG", + "STRING", + "STRING", + "STRING", + "LONG", + "STRING", + "STRING", + "LONG", + "LONG", + "LONG", + "STRING", + "STRING", + "STRING", + "STRING", + "STRING", + "STRING", + "STRING", + "STRING", + "STRING", + "STRING", + "STRING", + "STRING", + "STRING" + ], + "granularity": { + "type": "all" + }, + "legacy": false + } + }, + "signature": [ + { + "name": "__boost", + "type": "LONG" + }, + { + "name": "__time", + "type": "LONG" + }, + { + "name": "added", + "type": "LONG" + }, + { + "name": "channel", + "type": "STRING" + }, + { + "name": "cityName", + "type": "STRING" + }, + { + "name": "comment", + "type": "STRING" + }, + { + "name": "commentLength", + "type": "LONG" + }, + { + "name": "countryIsoCode", + "type": "STRING" + }, + { + "name": "countryName", + "type": "STRING" + }, + { + "name": "deleted", + "type": "LONG" + }, + { + "name": "delta", + "type": "LONG" + }, + { + "name": "deltaBucket", + "type": "LONG" + }, + { + "name": "diffUrl", + "type": "STRING" + }, + { + "name": "flags", + "type": "STRING" + }, + { + "name": "isAnonymous", + "type": "STRING" + }, + { + "name": "isMinor", + "type": "STRING" + }, + { + "name": "isNew", + "type": "STRING" + }, + { + "name": "isRobot", + "type": "STRING" + }, + { + "name": "isUnpatrolled", + "type": "STRING" + }, + { + "name": "metroCode", + "type": "STRING" + }, + { + "name": "namespace", + "type": "STRING" + }, + { + "name": "page", + "type": "STRING" + }, + { + "name": "regionIsoCode", + "type": "STRING" + }, + { + "name": "regionName", + "type": "STRING" + }, + { + "name": "v0", + "type": "STRING" + } + ], + "shuffleSpec": { + "type": "mix" + }, + "maxWorkerCount": 1 + }, + "phase": "FINISHED", + "workerCount": 1, + "partitionCount": 1, + "shuffle": "mix", + "output": "localStorage", + "startTime": "2024-07-31T15:20:21.255Z", + "duration": 103 + }, + { + "stageNumber": 1, + "definition": { + "id": "query-9b93f6f7-ab0e-48f5-986a-3520f84f0804_1", + "input": [ + { + "type": "stage", + "stage": 0 + } + ], + "processor": { + "type": "limit", + "limit": 1001 + }, + "signature": [ + { + "name": "__boost", + "type": "LONG" + }, + { + "name": "__time", + "type": "LONG" + }, + { + "name": "added", + "type": "LONG" + }, + { + "name": "channel", + "type": "STRING" + }, + { + "name": "cityName", + "type": "STRING" + }, + { + "name": "comment", + "type": "STRING" + }, + { + "name": "commentLength", + "type": "LONG" + }, + { + "name": "countryIsoCode", + "type": "STRING" + }, + { + "name": "countryName", + "type": "STRING" + }, + { + "name": "deleted", + "type": "LONG" + }, + { + "name": "delta", + "type": "LONG" + }, + { + "name": "deltaBucket", + "type": "LONG" + }, + { + "name": "diffUrl", + "type": "STRING" + }, + { + "name": "flags", + "type": "STRING" + }, + { + "name": "isAnonymous", + "type": "STRING" + }, + { + "name": "isMinor", + "type": "STRING" + }, + { + "name": "isNew", + "type": "STRING" + }, + { + "name": "isRobot", + "type": "STRING" + }, + { + "name": "isUnpatrolled", + "type": "STRING" + }, + { + "name": "metroCode", + "type": "STRING" + }, + { + "name": "namespace", + "type": "STRING" + }, + { + "name": "page", + "type": "STRING" + }, + { + "name": "regionIsoCode", + "type": "STRING" + }, + { + "name": "regionName", + "type": "STRING" + }, + { + "name": "v0", + "type": "STRING" + } + ], + "shuffleSpec": { + "type": "maxCount", + "clusterBy": { + "columns": [ + { + "columnName": "__boost", + "order": "ASCENDING" + } + ] + }, + "partitions": 1 + }, + "maxWorkerCount": 1 + }, + "phase": "FINISHED", + "workerCount": 1, + "partitionCount": 1, + "shuffle": "globalSort", + "output": "localStorage", + "startTime": "2024-07-31T15:20:21.355Z", + "duration": 10, + "sort": true + } + ], + "counters": { + "0": { + "0": { + "input0": { + "type": "channel", + "rows": [ + 24433 + ], + "bytes": [ + 7393933 + ], + "files": [ + 22 + ], + "totalFiles": [ + 22 + ] + } + } + }, + "1": { + "0": { + "sortProgress": { + "type": "sortProgress", + "totalMergingLevels": -1, + "levelToTotalBatches": {}, + "levelToMergedBatches": {}, + "totalMergersForUltimateLevel": -1, + "triviallyComplete": true, + "progressDigest": 1 + } + } + } + }, + "warnings": [] } ``` diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterSnapshotsTree.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterSnapshotsTree.java index 8936e104bd6..dce2fe7ac3a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterSnapshotsTree.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterSnapshotsTree.java @@ -108,4 +108,14 @@ public class CounterSnapshotsTree } } } + + @Override + public String toString() + { + synchronized (snapshotsMap) { + return "CounterSnapshotsTree{" + + "snapshotsMap=" + snapshotsMap + + '}'; + } + } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java index de66550a587..bd33d76adb1 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java @@ -23,12 +23,14 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.error.ErrorResponse; +import org.apache.druid.msq.counters.CounterSnapshotsTree; +import org.apache.druid.msq.indexing.error.MSQErrorReport; +import org.apache.druid.msq.indexing.report.MSQStagesReport; import org.apache.druid.msq.sql.SqlStatementState; import org.joda.time.DateTime; import javax.annotation.Nullable; import java.util.List; -import java.util.Objects; public class SqlStatementResult { @@ -51,6 +53,27 @@ public class SqlStatementResult @Nullable private final ErrorResponse errorResponse; + @Nullable + private final MSQStagesReport stages; + + @Nullable + private final CounterSnapshotsTree counters; + + @Nullable + private final List warnings; + + public SqlStatementResult( + String queryId, + SqlStatementState state, + DateTime createdAt, + List sqlRowSignature, + Long durationMs, + ResultSetInformation resultSetInformation, + ErrorResponse errorResponse + ) + { + this(queryId, state, createdAt, sqlRowSignature, durationMs, resultSetInformation, errorResponse, null, null, null); + } @JsonCreator public SqlStatementResult( @@ -67,8 +90,13 @@ public class SqlStatementResult @Nullable @JsonProperty("result") ResultSetInformation resultSetInformation, @Nullable @JsonProperty("errorDetails") - ErrorResponse errorResponse - + ErrorResponse errorResponse, + @Nullable @JsonProperty("stages") + MSQStagesReport stages, + @Nullable @JsonProperty("counters") + CounterSnapshotsTree counters, + @Nullable @JsonProperty("warnings") + List warnings ) { this.queryId = queryId; @@ -78,6 +106,9 @@ public class SqlStatementResult this.durationMs = durationMs; this.resultSetInformation = resultSetInformation; this.errorResponse = errorResponse; + this.stages = stages; + this.counters = counters; + this.warnings = warnings; } @JsonProperty @@ -130,41 +161,28 @@ public class SqlStatementResult return errorResponse; } - - @Override - public boolean equals(Object o) + @JsonProperty("stages") + @Nullable + @JsonInclude(JsonInclude.Include.NON_NULL) + public MSQStagesReport getStages() { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - SqlStatementResult that = (SqlStatementResult) o; - return Objects.equals(queryId, that.queryId) && state == that.state && Objects.equals( - createdAt, - that.createdAt - ) && Objects.equals(sqlRowSignature, that.sqlRowSignature) && Objects.equals( - durationMs, - that.durationMs - ) && Objects.equals(resultSetInformation, that.resultSetInformation) && Objects.equals( - errorResponse == null ? null : errorResponse.getAsMap(), - that.errorResponse == null ? null : that.errorResponse.getAsMap() - ); + return stages; } - @Override - public int hashCode() + @JsonProperty("counters") + @Nullable + @JsonInclude(JsonInclude.Include.NON_NULL) + public CounterSnapshotsTree getCounters() { - return Objects.hash( - queryId, - state, - createdAt, - sqlRowSignature, - durationMs, - resultSetInformation, - errorResponse == null ? null : errorResponse.getAsMap() - ); + return counters; + } + + @JsonProperty("warnings") + @Nullable + @JsonInclude(JsonInclude.Include.NON_NULL) + public List getWarnings() + { + return warnings; } @Override @@ -180,6 +198,9 @@ public class SqlStatementResult ", errorResponse=" + (errorResponse == null ? "{}" : errorResponse.getAsMap().toString()) + + ", stages=" + stages + + ", counters=" + counters + + ", warnings=" + warnings + '}'; } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java index 4beb2a869ef..1ee5f1030a4 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java @@ -113,6 +113,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Optional; +import java.util.function.Supplier; import java.util.stream.Collectors; @@ -231,7 +232,9 @@ public class SqlStatementResource @Path("/{id}") @Produces(MediaType.APPLICATION_JSON) public Response doGetStatus( - @PathParam("id") final String queryId, @Context final HttpServletRequest req + @PathParam("id") final String queryId, + @QueryParam("detail") boolean detail, + @Context final HttpServletRequest req ) { try { @@ -242,7 +245,8 @@ public class SqlStatementResource queryId, authenticationResult, true, - Action.READ + Action.READ, + detail ); if (sqlStatementResult.isPresent()) { @@ -369,7 +373,8 @@ public class SqlStatementResource queryId, authenticationResult, false, - Action.WRITE + Action.WRITE, + false ); if (sqlStatementResult.isPresent()) { switch (sqlStatementResult.get().getState()) { @@ -479,7 +484,7 @@ public class SqlStatementResource } String taskId = String.valueOf(firstRow[0]); - Optional statementResult = getStatementStatus(taskId, authenticationResult, true, Action.READ); + Optional statementResult = getStatementStatus(taskId, authenticationResult, true, Action.READ, false); if (statementResult.isPresent()) { return Response.status(Response.Status.OK).entity(statementResult.get()).build(); @@ -565,7 +570,8 @@ public class SqlStatementResource String queryId, AuthenticationResult authenticationResult, boolean withResults, - Action forAction + Action forAction, + boolean detail ) throws DruidException { TaskStatusResponse taskResponse = contactOverlord(overlordClient.taskStatus(queryId), queryId); @@ -582,14 +588,29 @@ public class SqlStatementResource MSQControllerTask msqControllerTask = getMSQControllerTaskAndCheckPermission(queryId, authenticationResult, forAction); SqlStatementState sqlStatementState = SqlStatementResourceHelper.getSqlStatementState(statusPlus); + Supplier> msqTaskReportPayloadSupplier = () -> { + try { + return Optional.ofNullable(SqlStatementResourceHelper.getPayload( + contactOverlord(overlordClient.taskReportAsMap(queryId), queryId) + )); + } + catch (DruidException e) { + if (e.getErrorCode().equals("notFound")) { + return Optional.empty(); + } + throw e; + } + }; + if (SqlStatementState.FAILED == sqlStatementState) { return SqlStatementResourceHelper.getExceptionPayload( queryId, taskResponse, statusPlus, sqlStatementState, - contactOverlord(overlordClient.taskReportAsMap(queryId), queryId), - jsonMapper + msqTaskReportPayloadSupplier.get().orElse(null), + jsonMapper, + detail ); } else { Optional> signature = SqlStatementResourceHelper.getSignature(msqControllerTask); @@ -605,7 +626,10 @@ public class SqlStatementResource sqlStatementState, msqControllerTask.getQuerySpec().getDestination() ).orElse(null) : null, - null + null, + detail ? SqlStatementResourceHelper.getQueryStagesReport(msqTaskReportPayloadSupplier.get().orElse(null)) : null, + detail ? SqlStatementResourceHelper.getQueryCounters(msqTaskReportPayloadSupplier.get().orElse(null)) : null, + detail ? SqlStatementResourceHelper.getQueryWarningDetails(msqTaskReportPayloadSupplier.get().orElse(null)) : null )); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SqlStatementResourceHelper.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SqlStatementResourceHelper.java index 4f07dcb2cc0..0820342ba72 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SqlStatementResourceHelper.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SqlStatementResourceHelper.java @@ -250,11 +250,12 @@ public class SqlStatementResourceHelper TaskStatusResponse taskResponse, TaskStatusPlus statusPlus, SqlStatementState sqlStatementState, - TaskReport.ReportMap msqPayload, - ObjectMapper jsonMapper + MSQTaskReportPayload msqTaskReportPayload, + ObjectMapper jsonMapper, + boolean detail ) { - final MSQErrorReport exceptionDetails = getQueryExceptionDetails(getPayload(msqPayload)); + final MSQErrorReport exceptionDetails = getQueryExceptionDetails(msqTaskReportPayload); final MSQFault fault = exceptionDetails == null ? null : exceptionDetails.getFault(); if (exceptionDetails == null || fault == null) { return Optional.of(new SqlStatementResult( @@ -267,7 +268,10 @@ public class SqlStatementResourceHelper DruidException.forPersona(DruidException.Persona.DEVELOPER) .ofCategory(DruidException.Category.UNCATEGORIZED) .build("%s", taskResponse.getStatus().getErrorMsg()) - .toErrorResponse() + .toErrorResponse(), + detail ? getQueryStagesReport(msqTaskReportPayload) : null, + detail ? getQueryCounters(msqTaskReportPayload) : null, + detail ? getQueryWarningDetails(msqTaskReportPayload) : null )); } @@ -293,7 +297,10 @@ public class SqlStatementResourceHelper ex.withContext(exceptionContext); return ex; } - }).toErrorResponse() + }).toErrorResponse(), + detail ? getQueryStagesReport(msqTaskReportPayload) : null, + detail ? getQueryCounters(msqTaskReportPayload) : null, + detail ? getQueryWarningDetails(msqTaskReportPayload) : null )); } @@ -353,7 +360,7 @@ public class SqlStatementResourceHelper } @Nullable - public static MSQStagesReport.Stage getFinalStage(MSQTaskReportPayload msqTaskReportPayload) + public static MSQStagesReport.Stage getFinalStage(@Nullable MSQTaskReportPayload msqTaskReportPayload) { if (msqTaskReportPayload == null || msqTaskReportPayload.getStages().getStages() == null) { return null; @@ -369,11 +376,29 @@ public class SqlStatementResourceHelper } @Nullable - private static MSQErrorReport getQueryExceptionDetails(MSQTaskReportPayload payload) + private static MSQErrorReport getQueryExceptionDetails(@Nullable MSQTaskReportPayload payload) { return payload == null ? null : payload.getStatus().getErrorReport(); } + @Nullable + public static List getQueryWarningDetails(@Nullable MSQTaskReportPayload payload) + { + return payload == null ? null : new ArrayList<>(payload.getStatus().getWarningReports()); + } + + @Nullable + public static MSQStagesReport getQueryStagesReport(@Nullable MSQTaskReportPayload payload) + { + return payload == null ? null : payload.getStages(); + } + + @Nullable + public static CounterSnapshotsTree getQueryCounters(@Nullable MSQTaskReportPayload payload) + { + return payload == null ? null : payload.getCounters(); + } + @Nullable public static MSQTaskReportPayload getPayload(TaskReport.ReportMap reportMap) { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/SqlStatementResultTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/SqlStatementResultTest.java index 03c017b7442..96ef0ac6b1f 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/SqlStatementResultTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/SqlStatementResultTest.java @@ -71,14 +71,6 @@ public class SqlStatementResultTest { Assert.assertEquals(JSON_STRING, MAPPER.writeValueAsString(SQL_STATEMENT_RESULT)); - Assert.assertEquals( - SQL_STATEMENT_RESULT, - MAPPER.readValue(MAPPER.writeValueAsString(SQL_STATEMENT_RESULT), SqlStatementResult.class) - ); - Assert.assertEquals( - SQL_STATEMENT_RESULT.hashCode(), - MAPPER.readValue(MAPPER.writeValueAsString(SQL_STATEMENT_RESULT), SqlStatementResult.class).hashCode() - ); Assert.assertEquals( "SqlStatementResult{" + "queryId='q1'," @@ -87,7 +79,10 @@ public class SqlStatementResultTest + " sqlRowSignature=[ColumnNameAndTypes{colName='_time', sqlTypeName='TIMESTAMP', nativeTypeName='LONG'}, ColumnNameAndTypes{colName='alias', sqlTypeName='VARCHAR', nativeTypeName='STRING'}, ColumnNameAndTypes{colName='market', sqlTypeName='VARCHAR', nativeTypeName='STRING'}]," + " durationInMs=100," + " resultSetInformation=ResultSetInformation{numTotalRows=1, totalSizeInBytes=1, resultFormat=object, records=null, dataSource='ds', pages=[PageInformation{id=0, numRows=null, sizeInBytes=1, worker=null, partition=null}]}," - + " errorResponse={error=druidException, errorCode=QueryNotSupported, persona=USER, category=UNCATEGORIZED, errorMessage=QueryNotSupported, context={}}}", + + " errorResponse={error=druidException, errorCode=QueryNotSupported, persona=USER, category=UNCATEGORIZED, errorMessage=QueryNotSupported, context={}}," + + " stages=null," + + " counters=null," + + " warnings=null}", SQL_STATEMENT_RESULT.toString() ); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlMSQStatementResourcePostTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlMSQStatementResourcePostTest.java index 1b0483d0b5a..cef3e00daa2 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlMSQStatementResourcePostTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlMSQStatementResourcePostTest.java @@ -206,7 +206,7 @@ public class SqlMSQStatementResourcePostTest extends MSQTestBase new ResultSetInformation(0L, 0L, null, "foo1", null, null), null ); - Assert.assertEquals(expected, actual); + assertSqlStatementResult(expected, actual); } @Test @@ -236,7 +236,7 @@ public class SqlMSQStatementResourcePostTest extends MSQTestBase new ResultSetInformation(0L, 0L, null, "foo1", null, null), null ); - Assert.assertEquals(expected, actual); + assertSqlStatementResult(expected, actual); } @Test @@ -282,7 +282,7 @@ public class SqlMSQStatementResourcePostTest extends MSQTestBase } }).toErrorResponse() ); - Assert.assertEquals(expected, actual); + assertSqlStatementResult(expected, actual); } @Test @@ -687,11 +687,11 @@ public class SqlMSQStatementResourcePostTest extends MSQTestBase new ResultSetInformation(NullHandling.sqlCompatible() ? 6L : 5L, 0L, null, "foo1", null, null), null ); - Assert.assertEquals(expected, actual); + assertSqlStatementResult(expected, actual); - Response getResponse = resource.doGetStatus(actual.getQueryId(), SqlStatementResourceTest.makeOkRequest()); + Response getResponse = resource.doGetStatus(actual.getQueryId(), false, SqlStatementResourceTest.makeOkRequest()); Assert.assertEquals(Response.Status.OK.getStatusCode(), getResponse.getStatus()); - Assert.assertEquals(expected, getResponse.getEntity()); + assertSqlStatementResult(expected, (SqlStatementResult) getResponse.getEntity()); Response resultsResponse = resource.doGetResults( actual.getQueryId(), @@ -730,11 +730,11 @@ public class SqlMSQStatementResourcePostTest extends MSQTestBase new ResultSetInformation(NullHandling.sqlCompatible() ? 6L : 5L, 0L, null, "foo1", null, null), null ); - Assert.assertEquals(expected, actual); + assertSqlStatementResult(expected, actual); - Response getResponse = resource.doGetStatus(actual.getQueryId(), SqlStatementResourceTest.makeOkRequest()); + Response getResponse = resource.doGetStatus(actual.getQueryId(), false, SqlStatementResourceTest.makeOkRequest()); Assert.assertEquals(Response.Status.OK.getStatusCode(), getResponse.getStatus()); - Assert.assertEquals(expected, getResponse.getEntity()); + assertSqlStatementResult(expected, (SqlStatementResult) getResponse.getEntity()); Response resultsResponse = resource.doGetResults( actual.getQueryId(), @@ -754,4 +754,27 @@ public class SqlMSQStatementResourcePostTest extends MSQTestBase return context; } + private void assertSqlStatementResult(SqlStatementResult expected, SqlStatementResult actual) + { + Assert.assertEquals(expected.getQueryId(), actual.getQueryId()); + Assert.assertEquals(expected.getCreatedAt(), actual.getCreatedAt()); + Assert.assertEquals(expected.getSqlRowSignature(), actual.getSqlRowSignature()); + Assert.assertEquals(expected.getDurationMs(), actual.getDurationMs()); + Assert.assertEquals(expected.getStages(), actual.getStages()); + Assert.assertEquals(expected.getState(), actual.getState()); + Assert.assertEquals(expected.getWarnings(), actual.getWarnings()); + Assert.assertEquals(expected.getResultSetInformation(), actual.getResultSetInformation()); + + if (actual.getCounters() == null || expected.getCounters() == null) { + Assert.assertEquals(expected.getCounters(), actual.getCounters()); + } else { + Assert.assertEquals(expected.getCounters().toString(), actual.getCounters().toString()); + } + + if (actual.getErrorResponse() == null || expected.getErrorResponse() == null) { + Assert.assertEquals(expected.getErrorResponse(), actual.getErrorResponse()); + } else { + Assert.assertEquals(expected.getErrorResponse().getAsMap(), actual.getErrorResponse().getAsMap()); + } + } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlStatementResourceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlStatementResourceTest.java index a97ee01297f..4ea2993050e 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlStatementResourceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlStatementResourceTest.java @@ -97,6 +97,7 @@ import javax.ws.rs.core.Response; import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.ArrayDeque; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -449,7 +450,11 @@ public class SqlStatementResourceTest extends MSQTestBase ))); - Mockito.when(indexingServiceClient.taskReportAsMap(FINISHED_SELECT_MSQ_QUERY)) + Mockito.when(indexingServiceClient.taskReportAsMap(ArgumentMatchers.eq(FINISHED_SELECT_MSQ_QUERY))) + .thenAnswer(inv -> Futures.immediateFuture(TaskReport.buildTaskReports(selectTaskReport.get()))); + Mockito.when(indexingServiceClient.taskReportAsMap(ArgumentMatchers.eq(ACCEPTED_SELECT_MSQ_QUERY))) + .thenAnswer(inv -> Futures.immediateFuture(TaskReport.buildTaskReports(selectTaskReport.get()))); + Mockito.when(indexingServiceClient.taskReportAsMap(ArgumentMatchers.eq(RUNNING_SELECT_MSQ_QUERY))) .thenAnswer(inv -> Futures.immediateFuture(TaskReport.buildTaskReports(selectTaskReport.get()))); Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(ERRORED_SELECT_MSQ_QUERY))) @@ -584,6 +589,10 @@ public class SqlStatementResourceTest extends MSQTestBase Mockito.when(indexingServiceClient.taskReportAsMap(ArgumentMatchers.eq(FINISHED_INSERT_MSQ_QUERY))) .thenReturn(Futures.immediateFuture(TaskReport.buildTaskReports(MSQ_INSERT_TASK_REPORT))); + Mockito.when(indexingServiceClient.taskReportAsMap(ArgumentMatchers.eq(ACCEPTED_INSERT_MSQ_TASK))) + .thenReturn(Futures.immediateFuture(TaskReport.buildTaskReports(MSQ_INSERT_TASK_REPORT))); + Mockito.when(indexingServiceClient.taskReportAsMap(ArgumentMatchers.eq(RUNNING_INSERT_MSQ_QUERY))) + .thenReturn(Futures.immediateFuture(TaskReport.buildTaskReports(MSQ_INSERT_TASK_REPORT))); Mockito.when(indexingServiceClient.taskPayload(FINISHED_INSERT_MSQ_QUERY)) .thenReturn(Futures.immediateFuture(new TaskPayloadResponse( @@ -690,9 +699,9 @@ public class SqlStatementResourceTest extends MSQTestBase @Test public void testMSQSelectAcceptedQuery() { - Response response = resource.doGetStatus(ACCEPTED_SELECT_MSQ_QUERY, makeOkRequest()); + Response response = resource.doGetStatus(ACCEPTED_SELECT_MSQ_QUERY, false, makeOkRequest()); Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); - Assert.assertEquals( + assertSqlStatementResult( new SqlStatementResult( ACCEPTED_SELECT_MSQ_QUERY, SqlStatementState.ACCEPTED, @@ -702,7 +711,7 @@ public class SqlStatementResourceTest extends MSQTestBase null, null ), - response.getEntity() + (SqlStatementResult) response.getEntity() ); assertExceptionMessage( @@ -724,9 +733,9 @@ public class SqlStatementResourceTest extends MSQTestBase public void testMSQSelectRunningQuery() { - Response response = resource.doGetStatus(RUNNING_SELECT_MSQ_QUERY, makeOkRequest()); + Response response = resource.doGetStatus(RUNNING_SELECT_MSQ_QUERY, false, makeOkRequest()); Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); - Assert.assertEquals( + assertSqlStatementResult( new SqlStatementResult( RUNNING_SELECT_MSQ_QUERY, SqlStatementState.RUNNING, @@ -736,7 +745,7 @@ public class SqlStatementResourceTest extends MSQTestBase null, null ), - response.getEntity() + (SqlStatementResult) response.getEntity() ); assertExceptionMessage( @@ -754,10 +763,40 @@ public class SqlStatementResourceTest extends MSQTestBase ); } + @Test + public void testMSQSelectRunningQueryWithDetail() + { + Response response = resource.doGetStatus(RUNNING_SELECT_MSQ_QUERY, true, makeOkRequest()); + Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); + + SqlStatementResult expectedSqlStatementResult = new SqlStatementResult( + RUNNING_SELECT_MSQ_QUERY, + SqlStatementState.RUNNING, + CREATED_TIME, + COL_NAME_AND_TYPES, + null, + null, + null, + selectTaskReport.get().getPayload().getStages(), + selectTaskReport.get().getPayload().getCounters(), + new ArrayList<>(selectTaskReport.get().getPayload().getStatus().getWarningReports()) + ); + + assertSqlStatementResult( + expectedSqlStatementResult, + (SqlStatementResult) response.getEntity() + ); + + Assert.assertEquals( + Response.Status.ACCEPTED.getStatusCode(), + resource.deleteQuery(RUNNING_SELECT_MSQ_QUERY, makeOkRequest()).getStatus() + ); + } + @Test public void testFinishedSelectMSQQuery() throws Exception { - Response response = resource.doGetStatus(FINISHED_SELECT_MSQ_QUERY, makeOkRequest()); + Response response = resource.doGetStatus(FINISHED_SELECT_MSQ_QUERY, false, makeOkRequest()); Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); Assert.assertEquals(objectMapper.writeValueAsString(new SqlStatementResult( FINISHED_SELECT_MSQ_QUERY, @@ -825,7 +864,7 @@ public class SqlStatementResourceTest extends MSQTestBase public void testFailedMSQQuery() { for (String queryID : ImmutableList.of(ERRORED_SELECT_MSQ_QUERY, ERRORED_INSERT_MSQ_QUERY)) { - assertExceptionMessage(resource.doGetStatus(queryID, makeOkRequest()), FAILURE_MSG, Response.Status.OK); + assertExceptionMessage(resource.doGetStatus(queryID, false, makeOkRequest()), FAILURE_MSG, Response.Status.OK); assertExceptionMessage( resource.doGetResults(queryID, 0L, null, makeOkRequest()), StringUtils.format( @@ -845,9 +884,9 @@ public class SqlStatementResourceTest extends MSQTestBase @Test public void testFinishedInsertMSQQuery() { - Response response = resource.doGetStatus(FINISHED_INSERT_MSQ_QUERY, makeOkRequest()); + Response response = resource.doGetStatus(FINISHED_INSERT_MSQ_QUERY, false, makeOkRequest()); Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); - Assert.assertEquals(new SqlStatementResult( + assertSqlStatementResult(new SqlStatementResult( FINISHED_INSERT_MSQ_QUERY, SqlStatementState.SUCCESS, CREATED_TIME, @@ -855,7 +894,7 @@ public class SqlStatementResourceTest extends MSQTestBase 100L, new ResultSetInformation(null, null, null, "test", null, null), null - ), response.getEntity()); + ), (SqlStatementResult) response.getEntity()); Assert.assertEquals( Response.Status.OK.getStatusCode(), @@ -876,7 +915,7 @@ public class SqlStatementResourceTest extends MSQTestBase public void testNonMSQTasks() { for (String queryID : ImmutableList.of(RUNNING_NON_MSQ_TASK, FAILED_NON_MSQ_TASK, FINISHED_NON_MSQ_TASK)) { - assertNotFound(resource.doGetStatus(queryID, makeOkRequest()), queryID); + assertNotFound(resource.doGetStatus(queryID, false, makeOkRequest()), queryID); assertNotFound(resource.doGetResults(queryID, 0L, null, makeOkRequest()), queryID); assertNotFound(resource.deleteQuery(queryID, makeOkRequest()), queryID); } @@ -885,9 +924,9 @@ public class SqlStatementResourceTest extends MSQTestBase @Test public void testMSQInsertAcceptedQuery() { - Response response = resource.doGetStatus(ACCEPTED_INSERT_MSQ_TASK, makeOkRequest()); + Response response = resource.doGetStatus(ACCEPTED_INSERT_MSQ_TASK, false, makeOkRequest()); Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); - Assert.assertEquals( + assertSqlStatementResult( new SqlStatementResult( ACCEPTED_INSERT_MSQ_TASK, SqlStatementState.ACCEPTED, @@ -897,7 +936,7 @@ public class SqlStatementResourceTest extends MSQTestBase null, null ), - response.getEntity() + (SqlStatementResult) response.getEntity() ); assertExceptionMessage( @@ -918,9 +957,9 @@ public class SqlStatementResourceTest extends MSQTestBase @Test public void testMSQInsertRunningQuery() { - Response response = resource.doGetStatus(RUNNING_INSERT_MSQ_QUERY, makeOkRequest()); + Response response = resource.doGetStatus(RUNNING_INSERT_MSQ_QUERY, false, makeOkRequest()); Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); - Assert.assertEquals( + assertSqlStatementResult( new SqlStatementResult( RUNNING_INSERT_MSQ_QUERY, SqlStatementState.RUNNING, @@ -930,7 +969,7 @@ public class SqlStatementResourceTest extends MSQTestBase null, null ), - response.getEntity() + (SqlStatementResult) response.getEntity() ); assertExceptionMessage( @@ -955,6 +994,7 @@ public class SqlStatementResourceTest extends MSQTestBase Response.Status.OK.getStatusCode(), resource.doGetStatus( RUNNING_SELECT_MSQ_QUERY, + false, makeExpectedReq(makeAuthResultForUser(SUPERUSER)) ).getStatus() ); @@ -984,6 +1024,7 @@ public class SqlStatementResourceTest extends MSQTestBase Response.Status.FORBIDDEN.getStatusCode(), resource.doGetStatus( RUNNING_SELECT_MSQ_QUERY, + false, makeExpectedReq(differentUserAuthResult) ).getStatus() ); @@ -1013,6 +1054,7 @@ public class SqlStatementResourceTest extends MSQTestBase Response.Status.OK.getStatusCode(), resource.doGetStatus( RUNNING_SELECT_MSQ_QUERY, + false, makeExpectedReq(differentUserAuthResult) ).getStatus() ); @@ -1042,6 +1084,7 @@ public class SqlStatementResourceTest extends MSQTestBase Response.Status.FORBIDDEN.getStatusCode(), resource.doGetStatus( RUNNING_SELECT_MSQ_QUERY, + false, makeExpectedReq(differentUserAuthResult) ).getStatus() ); @@ -1071,6 +1114,7 @@ public class SqlStatementResourceTest extends MSQTestBase Response.Status.OK.getStatusCode(), resource.doGetStatus( RUNNING_SELECT_MSQ_QUERY, + false, makeExpectedReq(differentUserAuthResult) ).getStatus() ); @@ -1107,7 +1151,7 @@ public class SqlStatementResourceTest extends MSQTestBase Assert.assertEquals( Response.Status.NOT_FOUND.getStatusCode(), - resource.doGetStatus(taskIdNotFound, makeOkRequest()).getStatus() + resource.doGetStatus(taskIdNotFound, false, makeOkRequest()).getStatus() ); Assert.assertEquals( Response.Status.NOT_FOUND.getStatusCode(), @@ -1124,4 +1168,28 @@ public class SqlStatementResourceTest extends MSQTestBase { Assert.assertEquals(Response.Status.OK.getStatusCode(), resource.isEnabled(makeOkRequest()).getStatus()); } + + private void assertSqlStatementResult(SqlStatementResult expected, SqlStatementResult actual) + { + Assert.assertEquals(expected.getQueryId(), actual.getQueryId()); + Assert.assertEquals(expected.getCreatedAt(), actual.getCreatedAt()); + Assert.assertEquals(expected.getSqlRowSignature(), actual.getSqlRowSignature()); + Assert.assertEquals(expected.getDurationMs(), actual.getDurationMs()); + Assert.assertEquals(expected.getStages(), actual.getStages()); + Assert.assertEquals(expected.getState(), actual.getState()); + Assert.assertEquals(expected.getWarnings(), actual.getWarnings()); + Assert.assertEquals(expected.getResultSetInformation(), actual.getResultSetInformation()); + + if (actual.getCounters() == null || expected.getCounters() == null) { + Assert.assertEquals(expected.getCounters(), actual.getCounters()); + } else { + Assert.assertEquals(expected.getCounters().toString(), actual.getCounters().toString()); + } + + if (actual.getErrorResponse() == null || expected.getErrorResponse() == null) { + Assert.assertEquals(expected.getErrorResponse(), actual.getErrorResponse()); + } else { + Assert.assertEquals(expected.getErrorResponse().getAsMap(), actual.getErrorResponse().getAsMap()); + } + } } From 8c170f7d0e2763cf11282ea17a602397c5efe151 Mon Sep 17 00:00:00 2001 From: Vadim Ogievetsky Date: Thu, 1 Aug 2024 02:30:30 -0700 Subject: [PATCH 05/12] Web console: use stages, counters, and warnings from the new detailed status API (#16809) * stages and counters can be seen on the status reponse * warnings are exposed also * mark as msq when attached * update snapshots * download CSV/TSV null as empty cell --- .../async-query/async-query.mock.ts | 674 +++++++++++++- .../druid-models/async-query/async-query.ts | 5 + .../druid-models/execution/execution.spec.ts | 836 +++++++++++++++++- .../src/druid-models/execution/execution.ts | 6 +- web-console/src/druid-models/stages/stages.ts | 13 +- web-console/src/druid-models/task/task.ts | 4 +- .../workbench-query/workbench-query.ts | 3 +- .../helpers/execution/sql-task-execution.ts | 2 +- web-console/src/utils/download.spec.ts | 5 +- web-console/src/utils/download.ts | 2 + .../recent-query-task-panel.tsx | 2 +- .../views/workbench-view/workbench-view.tsx | 5 +- 12 files changed, 1515 insertions(+), 42 deletions(-) diff --git a/web-console/src/druid-models/async-query/async-query.mock.ts b/web-console/src/druid-models/async-query/async-query.mock.ts index 906f82a753f..c3c0e1d1926 100644 --- a/web-console/src/druid-models/async-query/async-query.mock.ts +++ b/web-console/src/druid-models/async-query/async-query.mock.ts @@ -29,9 +29,9 @@ LIMIT 2 */ export const SUCCESS_ASYNC_STATUS: AsyncStatusResponse = { - queryId: 'query-ad84d20a-c331-4ee9-ac59-83024e369cf1', + queryId: 'query-45f1dafd-8a52-4eb7-9a6c-77840cddd349', state: 'SUCCESS', - createdAt: '2023-07-05T21:33:19.147Z', + createdAt: '2024-07-27T02:39:22.230Z', schema: [ { name: 'channel', @@ -44,10 +44,10 @@ export const SUCCESS_ASYNC_STATUS: AsyncStatusResponse = { nativeType: 'LONG', }, ], - durationMs: 29168, + durationMs: 7183, result: { numTotalRows: 2, - totalSizeInBytes: 116, + totalSizeInBytes: 150, dataSource: '__query_select', sampleRecords: [ ['#en.wikipedia', 6650], @@ -55,12 +55,395 @@ export const SUCCESS_ASYNC_STATUS: AsyncStatusResponse = { ], pages: [ { - numRows: 2, - sizeInBytes: 116, id: 0, + numRows: 2, + sizeInBytes: 150, }, ], }, + stages: [ + { + stageNumber: 0, + definition: { + id: 'query-45f1dafd-8a52-4eb7-9a6c-77840cddd349_0', + input: [ + { + type: 'table', + dataSource: 'wikipedia', + intervals: ['-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z'], + }, + ], + processor: { + type: 'groupByPreShuffle', + query: { + queryType: 'groupBy', + dataSource: { + type: 'inputNumber', + inputNumber: 0, + }, + intervals: { + type: 'intervals', + intervals: ['-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z'], + }, + granularity: { + type: 'all', + }, + dimensions: [ + { + type: 'default', + dimension: 'channel', + outputName: 'd0', + outputType: 'STRING', + }, + ], + aggregations: [ + { + type: 'count', + name: 'a0', + }, + ], + limitSpec: { + type: 'default', + columns: [ + { + dimension: 'a0', + direction: 'descending', + dimensionOrder: { + type: 'numeric', + }, + }, + ], + limit: 2, + }, + context: { + __resultFormat: 'array', + __user: 'allowAll', + executionMode: 'async', + finalize: true, + maxNumTasks: 2, + maxParseExceptions: 0, + queryId: '45f1dafd-8a52-4eb7-9a6c-77840cddd349', + sqlOuterLimit: 1001, + sqlQueryId: '45f1dafd-8a52-4eb7-9a6c-77840cddd349', + sqlStringifyArrays: false, + }, + }, + }, + signature: [ + { + name: 'd0', + type: 'STRING', + }, + { + name: 'a0', + type: 'LONG', + }, + ], + shuffleSpec: { + type: 'maxCount', + clusterBy: { + columns: [ + { + columnName: 'd0', + order: 'ASCENDING', + }, + ], + }, + partitions: 1, + aggregate: true, + }, + maxWorkerCount: 1, + }, + phase: 'FINISHED', + workerCount: 1, + partitionCount: 1, + shuffle: 'globalSort', + output: 'localStorage', + startTime: '2024-07-27T02:39:24.713Z', + duration: 3384, + sort: true, + }, + { + stageNumber: 1, + definition: { + id: 'query-45f1dafd-8a52-4eb7-9a6c-77840cddd349_1', + input: [ + { + type: 'stage', + stage: 0, + }, + ], + processor: { + type: 'groupByPostShuffle', + query: { + queryType: 'groupBy', + dataSource: { + type: 'inputNumber', + inputNumber: 0, + }, + intervals: { + type: 'intervals', + intervals: ['-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z'], + }, + granularity: { + type: 'all', + }, + dimensions: [ + { + type: 'default', + dimension: 'channel', + outputName: 'd0', + outputType: 'STRING', + }, + ], + aggregations: [ + { + type: 'count', + name: 'a0', + }, + ], + limitSpec: { + type: 'default', + columns: [ + { + dimension: 'a0', + direction: 'descending', + dimensionOrder: { + type: 'numeric', + }, + }, + ], + limit: 2, + }, + context: { + __resultFormat: 'array', + __user: 'allowAll', + executionMode: 'async', + finalize: true, + maxNumTasks: 2, + maxParseExceptions: 0, + queryId: '45f1dafd-8a52-4eb7-9a6c-77840cddd349', + sqlOuterLimit: 1001, + sqlQueryId: '45f1dafd-8a52-4eb7-9a6c-77840cddd349', + sqlStringifyArrays: false, + }, + }, + }, + signature: [ + { + name: 'a0', + type: 'LONG', + }, + { + name: '__boost', + type: 'LONG', + }, + { + name: 'd0', + type: 'STRING', + }, + ], + shuffleSpec: { + type: 'maxCount', + clusterBy: { + columns: [ + { + columnName: 'a0', + order: 'DESCENDING', + }, + { + columnName: '__boost', + order: 'ASCENDING', + }, + ], + }, + partitions: 1, + }, + maxWorkerCount: 1, + }, + phase: 'FINISHED', + workerCount: 1, + partitionCount: 1, + shuffle: 'globalSort', + output: 'localStorage', + startTime: '2024-07-27T02:39:28.089Z', + duration: 26, + sort: true, + }, + { + stageNumber: 2, + definition: { + id: 'query-45f1dafd-8a52-4eb7-9a6c-77840cddd349_2', + input: [ + { + type: 'stage', + stage: 1, + }, + ], + processor: { + type: 'limit', + limit: 2, + }, + signature: [ + { + name: 'a0', + type: 'LONG', + }, + { + name: '__boost', + type: 'LONG', + }, + { + name: 'd0', + type: 'STRING', + }, + ], + shuffleSpec: { + type: 'maxCount', + clusterBy: { + columns: [ + { + columnName: 'a0', + order: 'DESCENDING', + }, + { + columnName: '__boost', + order: 'ASCENDING', + }, + ], + }, + partitions: 1, + }, + maxWorkerCount: 1, + }, + phase: 'FINISHED', + workerCount: 1, + partitionCount: 1, + shuffle: 'globalSort', + output: 'localStorage', + startTime: '2024-07-27T02:39:28.112Z', + duration: 12, + sort: true, + }, + ], + counters: { + '0': { + '0': { + input0: { + type: 'channel', + rows: [24433], + bytes: [6525055], + files: [1], + totalFiles: [1], + }, + output: { + type: 'channel', + rows: [51], + bytes: [2335], + frames: [1], + }, + shuffle: { + type: 'channel', + rows: [51], + bytes: [2131], + frames: [1], + }, + sortProgress: { + type: 'sortProgress', + totalMergingLevels: 3, + levelToTotalBatches: { + '0': 1, + '1': 1, + '2': 1, + }, + levelToMergedBatches: { + '0': 1, + '1': 1, + '2': 1, + }, + totalMergersForUltimateLevel: 1, + progressDigest: 1.0, + }, + }, + }, + '1': { + '0': { + input0: { + type: 'channel', + rows: [51], + bytes: [2131], + frames: [1], + }, + output: { + type: 'channel', + rows: [51], + bytes: [2998], + frames: [1], + }, + shuffle: { + type: 'channel', + rows: [51], + bytes: [2794], + frames: [1], + }, + sortProgress: { + type: 'sortProgress', + totalMergingLevels: 3, + levelToTotalBatches: { + '0': 1, + '1': 1, + '2': 1, + }, + levelToMergedBatches: { + '0': 1, + '1': 1, + '2': 1, + }, + totalMergersForUltimateLevel: 1, + progressDigest: 1.0, + }, + }, + }, + '2': { + '0': { + input0: { + type: 'channel', + rows: [51], + bytes: [2794], + frames: [1], + }, + output: { + type: 'channel', + rows: [2], + bytes: [150], + frames: [1], + }, + shuffle: { + type: 'channel', + rows: [2], + bytes: [142], + frames: [1], + }, + sortProgress: { + type: 'sortProgress', + totalMergingLevels: 3, + levelToTotalBatches: { + '0': 1, + '1': 1, + '2': 1, + }, + levelToMergedBatches: { + '0': 1, + '1': 1, + '2': 1, + }, + totalMergersForUltimateLevel: 1, + progressDigest: 1.0, + }, + }, + }, + }, + warnings: [], }; /* @@ -82,18 +465,285 @@ PARTITIONED BY DAY */ export const FAILED_ASYNC_STATUS: AsyncStatusResponse = { - queryId: 'query-36ea273a-bd6d-48de-b890-2d853d879bf8', + queryId: 'query-ea3e36df-ad67-4870-b136-f5616b17d9c4', state: 'FAILED', - createdAt: '2023-07-05T21:40:39.986Z', - durationMs: 11217, + createdAt: '2024-07-26T18:04:59.873Z', + durationMs: 6954, errorDetails: { error: 'druidException', - errorCode: 'UnknownError', + errorCode: 'TooManyWarnings', persona: 'USER', category: 'UNCATEGORIZED', - errorMessage: 'java.io.UncheckedIOException: /', + errorMessage: 'Too many warnings of type CannotParseExternalData generated (max = 2)', context: { - message: 'java.io.UncheckedIOException: /', + maxWarnings: '2', + rootErrorCode: 'CannotParseExternalData', }, }, + stages: [ + { + stageNumber: 0, + definition: { + id: 'query-ea3e36df-ad67-4870-b136-f5616b17d9c4_0', + input: [ + { + type: 'external', + inputSource: { + type: 'http', + uris: ['https://static.imply.io/example-data/kttm-with-issues/kttm-blank-lines.json'], + }, + inputFormat: { + type: 'json', + }, + signature: [ + { + name: 'timestamp', + type: 'STRING', + }, + { + name: 'agent_type', + type: 'STRING', + }, + ], + }, + ], + processor: { + type: 'scan', + query: { + queryType: 'scan', + dataSource: { + type: 'external', + inputSource: { + type: 'http', + uris: [ + 'https://static.imply.io/example-data/kttm-with-issues/kttm-blank-lines.json', + ], + }, + inputFormat: { + type: 'json', + }, + signature: [ + { + name: 'timestamp', + type: 'STRING', + }, + { + name: 'agent_type', + type: 'STRING', + }, + ], + }, + intervals: { + type: 'intervals', + intervals: ['-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z'], + }, + virtualColumns: [ + { + type: 'expression', + name: 'v0', + expression: 'timestamp_parse("timestamp",null,\'UTC\')', + outputType: 'LONG', + }, + ], + resultFormat: 'compactedList', + columns: ['agent_type', 'v0'], + context: { + __resultFormat: 'array', + __timeColumn: 'v0', + __user: 'allowAll', + executionMode: 'async', + finalize: false, + finalizeAggregations: false, + groupByEnableMultiValueUnnesting: false, + maxNumTasks: 2, + maxParseExceptions: 2, + queryId: 'ea3e36df-ad67-4870-b136-f5616b17d9c4', + scanSignature: '[{"name":"agent_type","type":"STRING"},{"name":"v0","type":"LONG"}]', + sqlInsertSegmentGranularity: '"DAY"', + sqlQueryId: 'ea3e36df-ad67-4870-b136-f5616b17d9c4', + sqlReplaceTimeChunks: 'all', + sqlStringifyArrays: false, + waitUntilSegmentsLoad: true, + }, + columnTypes: ['STRING', 'LONG'], + granularity: { + type: 'all', + }, + legacy: false, + }, + }, + signature: [ + { + name: '__bucket', + type: 'LONG', + }, + { + name: '__boost', + type: 'LONG', + }, + { + name: 'agent_type', + type: 'STRING', + }, + { + name: 'v0', + type: 'LONG', + }, + ], + shuffleSpec: { + type: 'targetSize', + clusterBy: { + columns: [ + { + columnName: '__bucket', + order: 'ASCENDING', + }, + { + columnName: '__boost', + order: 'ASCENDING', + }, + ], + bucketByCount: 1, + }, + targetSize: 3000000, + }, + maxWorkerCount: 1, + shuffleCheckHasMultipleValues: true, + }, + phase: 'FAILED', + workerCount: 1, + shuffle: 'globalSort', + output: 'localStorage', + startTime: '2024-07-26T18:05:02.399Z', + duration: 4056, + sort: true, + }, + { + stageNumber: 1, + definition: { + id: 'query-ea3e36df-ad67-4870-b136-f5616b17d9c4_1', + input: [ + { + type: 'stage', + stage: 0, + }, + ], + processor: { + type: 'segmentGenerator', + dataSchema: { + dataSource: 'kttm-blank-lines', + timestampSpec: { + column: '__time', + format: 'millis', + missingValue: null, + }, + dimensionsSpec: { + dimensions: [ + { + type: 'string', + name: 'agent_type', + multiValueHandling: 'SORTED_ARRAY', + createBitmapIndex: true, + }, + ], + dimensionExclusions: ['__time'], + includeAllDimensions: false, + useSchemaDiscovery: false, + }, + metricsSpec: [], + granularitySpec: { + type: 'arbitrary', + queryGranularity: { + type: 'none', + }, + rollup: false, + intervals: ['-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z'], + }, + transformSpec: { + filter: null, + transforms: [], + }, + }, + columnMappings: [ + { + queryColumn: 'v0', + outputColumn: '__time', + }, + { + queryColumn: 'agent_type', + outputColumn: 'agent_type', + }, + ], + tuningConfig: { + maxNumWorkers: 1, + maxRowsInMemory: 100000, + rowsPerSegment: 3000000, + }, + }, + signature: [], + maxWorkerCount: 1, + }, + }, + ], + counters: { + '0': { + '0': { + input0: { + type: 'channel', + rows: [10], + bytes: [7658], + files: [1], + totalFiles: [1], + }, + output: { + type: 'channel', + rows: [10], + bytes: [712], + frames: [1], + }, + sortProgress: { + type: 'sortProgress', + totalMergingLevels: 3, + levelToTotalBatches: { + '0': 1, + '1': 1, + '2': -1, + }, + levelToMergedBatches: {}, + totalMergersForUltimateLevel: -1, + progressDigest: 0.0, + }, + warnings: { + type: 'warnings', + CannotParseExternalData: 3, + }, + }, + }, + }, + warnings: [ + { + taskId: 'query-ea3e36df-ad67-4870-b136-f5616b17d9c4-worker0_0', + host: 'localhost:8101', + stageNumber: 0, + error: { + errorCode: 'CannotParseExternalData', + errorMessage: + 'Unable to parse row [] (Path: https://static.imply.io/example-data/kttm-with-issues/kttm-blank-lines.json, Record: 3, Line: 3)', + }, + exceptionStackTrace: + 'org.apache.druid.java.util.common.parsers.ParseException: Unable to parse row [] (Path: https://static.imply.io/example-data/kttm-with-issues/kttm-blank-lines.json, Record: 3, Line: 3)\n\tat org.apache.druid.data.input.IntermediateRowParsingReader$1.hasNext(IntermediateRowParsingReader.java:80)\n\tat org.apache.druid.java.util.common.parsers.CloseableIterator$1.hasNext(CloseableIterator.java:42)\n\tat org.apache.druid.java.util.common.parsers.CloseableIterator$2.findNextIteratorIfNecessary(CloseableIterator.java:72)\n\tat org.apache.druid.java.util.common.parsers.CloseableIterator$2.hasNext(CloseableIterator.java:93)\n\tat org.apache.druid.java.util.common.parsers.CloseableIterator$1.hasNext(CloseableIterator.java:42)\n\tat org.apache.druid.msq.input.external.ExternalSegment$1$1.hasNext(ExternalSegment.java:94)\n\tat org.apache.druid.java.util.common.guava.BaseSequence$1.next(BaseSequence.java:115)\n\tat org.apache.druid.segment.RowWalker.advance(RowWalker.java:75)\n\tat org.apache.druid.segment.RowBasedCursor.advanceUninterruptibly(RowBasedCursor.java:110)\n\tat org.apache.druid.segment.RowBasedCursor.advance(RowBasedCursor.java:103)\n\tat org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessor.populateFrameWriterAndFlushIfNeeded(ScanQueryFrameProcessor.java:374)\n\tat org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessor.populateFrameWriterAndFlushIfNeededWithExceptionHandling(ScanQueryFrameProcessor.java:334)\n\tat org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessor.runWithSegment(ScanQueryFrameProcessor.java:273)\n\tat org.apache.druid.msq.querykit.BaseLeafFrameProcessor.runIncrementally(BaseLeafFrameProcessor.java:88)\n\tat org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessor.runIncrementally(ScanQueryFrameProcessor.java:157)\n\tat org.apache.druid.frame.processor.FrameProcessors$1FrameProcessorWithBaggage.runIncrementally(FrameProcessors.java:75)\n\tat org.apache.druid.frame.processor.FrameProcessorExecutor$1ExecutorRunnable.runProcessorNow(FrameProcessorExecutor.java:230)\n\tat org.apache.druid.frame.processor.FrameProcessorExecutor$1ExecutorRunnable.run(FrameProcessorExecutor.java:138)\n\tat org.apache.druid.msq.exec.WorkerImpl$1$2.run(WorkerImpl.java:838)\n\tat java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:539)\n\tat java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)\n\tat org.apache.druid.query.PrioritizedListenableFutureTask.run(PrioritizedExecutorService.java:259)\n\tat java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)\n\tat java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)\n\tat java.base/java.lang.Thread.run(Thread.java:840)\nCaused by: com.fasterxml.jackson.databind.exc.MismatchedInputException: No content to map due to end-of-input\n at [Source: (byte[])""; line: 1, column: 0]\n\tat com.fasterxml.jackson.databind.exc.MismatchedInputException.from(MismatchedInputException.java:59)\n\tat com.fasterxml.jackson.databind.ObjectMapper._initForReading(ObjectMapper.java:4688)\n\tat com.fasterxml.jackson.databind.ObjectMapper._readMapAndClose(ObjectMapper.java:4586)\n\tat com.fasterxml.jackson.databind.ObjectMapper.readValue(ObjectMapper.java:3609)\n\tat org.apache.druid.data.input.impl.JsonLineReader.parseInputRows(JsonLineReader.java:75)\n\tat org.apache.druid.data.input.impl.JsonLineReader.parseInputRows(JsonLineReader.java:48)\n\tat org.apache.druid.data.input.IntermediateRowParsingReader$1.hasNext(IntermediateRowParsingReader.java:71)\n\t... 24 more\n', + }, + { + taskId: 'query-ea3e36df-ad67-4870-b136-f5616b17d9c4-worker0_0', + host: 'localhost:8101', + stageNumber: 0, + error: { + errorCode: 'CannotParseExternalData', + errorMessage: + 'Unable to parse row [] (Path: https://static.imply.io/example-data/kttm-with-issues/kttm-blank-lines.json, Record: 6, Line: 7)', + }, + exceptionStackTrace: + 'org.apache.druid.java.util.common.parsers.ParseException: Unable to parse row [] (Path: https://static.imply.io/example-data/kttm-with-issues/kttm-blank-lines.json, Record: 6, Line: 7)\n\tat org.apache.druid.data.input.IntermediateRowParsingReader$1.hasNext(IntermediateRowParsingReader.java:80)\n\tat org.apache.druid.java.util.common.parsers.CloseableIterator$1.hasNext(CloseableIterator.java:42)\n\tat org.apache.druid.java.util.common.parsers.CloseableIterator$2.findNextIteratorIfNecessary(CloseableIterator.java:72)\n\tat org.apache.druid.java.util.common.parsers.CloseableIterator$2.hasNext(CloseableIterator.java:93)\n\tat org.apache.druid.java.util.common.parsers.CloseableIterator$1.hasNext(CloseableIterator.java:42)\n\tat org.apache.druid.msq.input.external.ExternalSegment$1$1.hasNext(ExternalSegment.java:94)\n\tat org.apache.druid.java.util.common.guava.BaseSequence$1.next(BaseSequence.java:115)\n\tat org.apache.druid.segment.RowWalker.advance(RowWalker.java:75)\n\tat org.apache.druid.segment.RowBasedCursor.advanceUninterruptibly(RowBasedCursor.java:110)\n\tat org.apache.druid.segment.RowBasedCursor.advance(RowBasedCursor.java:103)\n\tat org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessor.populateFrameWriterAndFlushIfNeeded(ScanQueryFrameProcessor.java:374)\n\tat org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessor.populateFrameWriterAndFlushIfNeededWithExceptionHandling(ScanQueryFrameProcessor.java:334)\n\tat org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessor.runWithSegment(ScanQueryFrameProcessor.java:273)\n\tat org.apache.druid.msq.querykit.BaseLeafFrameProcessor.runIncrementally(BaseLeafFrameProcessor.java:88)\n\tat org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessor.runIncrementally(ScanQueryFrameProcessor.java:157)\n\tat org.apache.druid.frame.processor.FrameProcessors$1FrameProcessorWithBaggage.runIncrementally(FrameProcessors.java:75)\n\tat org.apache.druid.frame.processor.FrameProcessorExecutor$1ExecutorRunnable.runProcessorNow(FrameProcessorExecutor.java:230)\n\tat org.apache.druid.frame.processor.FrameProcessorExecutor$1ExecutorRunnable.run(FrameProcessorExecutor.java:138)\n\tat org.apache.druid.msq.exec.WorkerImpl$1$2.run(WorkerImpl.java:838)\n\tat java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:539)\n\tat java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)\n\tat org.apache.druid.query.PrioritizedListenableFutureTask.run(PrioritizedExecutorService.java:259)\n\tat java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)\n\tat java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)\n\tat java.base/java.lang.Thread.run(Thread.java:840)\nCaused by: com.fasterxml.jackson.databind.exc.MismatchedInputException: No content to map due to end-of-input\n at [Source: (byte[])""; line: 1, column: 0]\n\tat com.fasterxml.jackson.databind.exc.MismatchedInputException.from(MismatchedInputException.java:59)\n\tat com.fasterxml.jackson.databind.ObjectMapper._initForReading(ObjectMapper.java:4688)\n\tat com.fasterxml.jackson.databind.ObjectMapper._readMapAndClose(ObjectMapper.java:4586)\n\tat com.fasterxml.jackson.databind.ObjectMapper.readValue(ObjectMapper.java:3609)\n\tat org.apache.druid.data.input.impl.JsonLineReader.parseInputRows(JsonLineReader.java:75)\n\tat org.apache.druid.data.input.impl.JsonLineReader.parseInputRows(JsonLineReader.java:48)\n\tat org.apache.druid.data.input.IntermediateRowParsingReader$1.hasNext(IntermediateRowParsingReader.java:71)\n\t... 24 more\n', + }, + ], }; diff --git a/web-console/src/druid-models/async-query/async-query.ts b/web-console/src/druid-models/async-query/async-query.ts index e823ba6a17e..71eec372515 100644 --- a/web-console/src/druid-models/async-query/async-query.ts +++ b/web-console/src/druid-models/async-query/async-query.ts @@ -17,6 +17,8 @@ */ import type { ErrorResponse } from '../../utils'; +import type { Counters, StageDefinition } from '../stages/stages'; +import type { MsqTaskErrorReport } from '../task/task'; export type AsyncState = 'ACCEPTED' | 'RUNNING' | 'SUCCESS' | 'FAILED'; @@ -37,5 +39,8 @@ export interface AsyncStatusResponse { sizeInBytes: number; }[]; }; + stages?: StageDefinition[]; + counters?: Counters; errorDetails?: ErrorResponse; + warnings?: MsqTaskErrorReport[]; } diff --git a/web-console/src/druid-models/execution/execution.spec.ts b/web-console/src/druid-models/execution/execution.spec.ts index 275de4f1757..2938351a22a 100644 --- a/web-console/src/druid-models/execution/execution.spec.ts +++ b/web-console/src/druid-models/execution/execution.spec.ts @@ -622,13 +622,13 @@ describe('Execution', () => { { "id": 0, "numRows": 2, - "sizeInBytes": 116, + "sizeInBytes": 150, }, ], - "duration": 29168, + "duration": 7183, "engine": "sql-msq-task", "error": undefined, - "id": "query-ad84d20a-c331-4ee9-ac59-83024e369cf1", + "id": "query-45f1dafd-8a52-4eb7-9a6c-77840cddd349", "nativeQuery": undefined, "queryContext": undefined, "result": _QueryResult { @@ -663,8 +663,453 @@ describe('Execution', () => { }, "segmentStatus": undefined, "sqlQuery": undefined, - "stages": undefined, - "startTime": 2023-07-05T21:33:19.147Z, + "stages": Stages { + "counters": { + "0": { + "0": { + "input0": { + "bytes": [ + 6525055, + ], + "files": [ + 1, + ], + "rows": [ + 24433, + ], + "totalFiles": [ + 1, + ], + "type": "channel", + }, + "output": { + "bytes": [ + 2335, + ], + "frames": [ + 1, + ], + "rows": [ + 51, + ], + "type": "channel", + }, + "shuffle": { + "bytes": [ + 2131, + ], + "frames": [ + 1, + ], + "rows": [ + 51, + ], + "type": "channel", + }, + "sortProgress": { + "levelToMergedBatches": { + "0": 1, + "1": 1, + "2": 1, + }, + "levelToTotalBatches": { + "0": 1, + "1": 1, + "2": 1, + }, + "progressDigest": 1, + "totalMergersForUltimateLevel": 1, + "totalMergingLevels": 3, + "type": "sortProgress", + }, + }, + }, + "1": { + "0": { + "input0": { + "bytes": [ + 2131, + ], + "frames": [ + 1, + ], + "rows": [ + 51, + ], + "type": "channel", + }, + "output": { + "bytes": [ + 2998, + ], + "frames": [ + 1, + ], + "rows": [ + 51, + ], + "type": "channel", + }, + "shuffle": { + "bytes": [ + 2794, + ], + "frames": [ + 1, + ], + "rows": [ + 51, + ], + "type": "channel", + }, + "sortProgress": { + "levelToMergedBatches": { + "0": 1, + "1": 1, + "2": 1, + }, + "levelToTotalBatches": { + "0": 1, + "1": 1, + "2": 1, + }, + "progressDigest": 1, + "totalMergersForUltimateLevel": 1, + "totalMergingLevels": 3, + "type": "sortProgress", + }, + }, + }, + "2": { + "0": { + "input0": { + "bytes": [ + 2794, + ], + "frames": [ + 1, + ], + "rows": [ + 51, + ], + "type": "channel", + }, + "output": { + "bytes": [ + 150, + ], + "frames": [ + 1, + ], + "rows": [ + 2, + ], + "type": "channel", + }, + "shuffle": { + "bytes": [ + 142, + ], + "frames": [ + 1, + ], + "rows": [ + 2, + ], + "type": "channel", + }, + "sortProgress": { + "levelToMergedBatches": { + "0": 1, + "1": 1, + "2": 1, + }, + "levelToTotalBatches": { + "0": 1, + "1": 1, + "2": 1, + }, + "progressDigest": 1, + "totalMergersForUltimateLevel": 1, + "totalMergingLevels": 3, + "type": "sortProgress", + }, + }, + }, + }, + "stages": [ + { + "definition": { + "id": "query-45f1dafd-8a52-4eb7-9a6c-77840cddd349_0", + "input": [ + { + "dataSource": "wikipedia", + "intervals": [ + "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z", + ], + "type": "table", + }, + ], + "maxWorkerCount": 1, + "processor": { + "query": { + "aggregations": [ + { + "name": "a0", + "type": "count", + }, + ], + "context": { + "__resultFormat": "array", + "__user": "allowAll", + "executionMode": "async", + "finalize": true, + "maxNumTasks": 2, + "maxParseExceptions": 0, + "queryId": "45f1dafd-8a52-4eb7-9a6c-77840cddd349", + "sqlOuterLimit": 1001, + "sqlQueryId": "45f1dafd-8a52-4eb7-9a6c-77840cddd349", + "sqlStringifyArrays": false, + }, + "dataSource": { + "inputNumber": 0, + "type": "inputNumber", + }, + "dimensions": [ + { + "dimension": "channel", + "outputName": "d0", + "outputType": "STRING", + "type": "default", + }, + ], + "granularity": { + "type": "all", + }, + "intervals": { + "intervals": [ + "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z", + ], + "type": "intervals", + }, + "limitSpec": { + "columns": [ + { + "dimension": "a0", + "dimensionOrder": { + "type": "numeric", + }, + "direction": "descending", + }, + ], + "limit": 2, + "type": "default", + }, + "queryType": "groupBy", + }, + "type": "groupByPreShuffle", + }, + "shuffleSpec": { + "aggregate": true, + "clusterBy": { + "columns": [ + { + "columnName": "d0", + "order": "ASCENDING", + }, + ], + }, + "partitions": 1, + "type": "maxCount", + }, + "signature": [ + { + "name": "d0", + "type": "STRING", + }, + { + "name": "a0", + "type": "LONG", + }, + ], + }, + "duration": 3384, + "output": "localStorage", + "partitionCount": 1, + "phase": "FINISHED", + "shuffle": "globalSort", + "sort": true, + "stageNumber": 0, + "startTime": "2024-07-27T02:39:24.713Z", + "workerCount": 1, + }, + { + "definition": { + "id": "query-45f1dafd-8a52-4eb7-9a6c-77840cddd349_1", + "input": [ + { + "stage": 0, + "type": "stage", + }, + ], + "maxWorkerCount": 1, + "processor": { + "query": { + "aggregations": [ + { + "name": "a0", + "type": "count", + }, + ], + "context": { + "__resultFormat": "array", + "__user": "allowAll", + "executionMode": "async", + "finalize": true, + "maxNumTasks": 2, + "maxParseExceptions": 0, + "queryId": "45f1dafd-8a52-4eb7-9a6c-77840cddd349", + "sqlOuterLimit": 1001, + "sqlQueryId": "45f1dafd-8a52-4eb7-9a6c-77840cddd349", + "sqlStringifyArrays": false, + }, + "dataSource": { + "inputNumber": 0, + "type": "inputNumber", + }, + "dimensions": [ + { + "dimension": "channel", + "outputName": "d0", + "outputType": "STRING", + "type": "default", + }, + ], + "granularity": { + "type": "all", + }, + "intervals": { + "intervals": [ + "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z", + ], + "type": "intervals", + }, + "limitSpec": { + "columns": [ + { + "dimension": "a0", + "dimensionOrder": { + "type": "numeric", + }, + "direction": "descending", + }, + ], + "limit": 2, + "type": "default", + }, + "queryType": "groupBy", + }, + "type": "groupByPostShuffle", + }, + "shuffleSpec": { + "clusterBy": { + "columns": [ + { + "columnName": "a0", + "order": "DESCENDING", + }, + { + "columnName": "__boost", + "order": "ASCENDING", + }, + ], + }, + "partitions": 1, + "type": "maxCount", + }, + "signature": [ + { + "name": "a0", + "type": "LONG", + }, + { + "name": "__boost", + "type": "LONG", + }, + { + "name": "d0", + "type": "STRING", + }, + ], + }, + "duration": 26, + "output": "localStorage", + "partitionCount": 1, + "phase": "FINISHED", + "shuffle": "globalSort", + "sort": true, + "stageNumber": 1, + "startTime": "2024-07-27T02:39:28.089Z", + "workerCount": 1, + }, + { + "definition": { + "id": "query-45f1dafd-8a52-4eb7-9a6c-77840cddd349_2", + "input": [ + { + "stage": 1, + "type": "stage", + }, + ], + "maxWorkerCount": 1, + "processor": { + "limit": 2, + "type": "limit", + }, + "shuffleSpec": { + "clusterBy": { + "columns": [ + { + "columnName": "a0", + "order": "DESCENDING", + }, + { + "columnName": "__boost", + "order": "ASCENDING", + }, + ], + }, + "partitions": 1, + "type": "maxCount", + }, + "signature": [ + { + "name": "a0", + "type": "LONG", + }, + { + "name": "__boost", + "type": "LONG", + }, + { + "name": "d0", + "type": "STRING", + }, + ], + }, + "duration": 12, + "output": "localStorage", + "partitionCount": 1, + "phase": "FINISHED", + "shuffle": "globalSort", + "sort": true, + "stageNumber": 2, + "startTime": "2024-07-27T02:39:28.112Z", + "workerCount": 1, + }, + ], + }, + "startTime": 2024-07-27T02:39:22.230Z, "status": "SUCCESS", "usageInfo": undefined, "warnings": undefined, @@ -679,32 +1124,395 @@ describe('Execution', () => { "capacityInfo": undefined, "destination": undefined, "destinationPages": undefined, - "duration": 11217, + "duration": 6954, "engine": "sql-msq-task", "error": { "error": { "category": "UNCATEGORIZED", "context": { - "message": "java.io.UncheckedIOException: /", + "maxWarnings": "2", + "rootErrorCode": "CannotParseExternalData", }, "error": "druidException", - "errorCode": "UnknownError", - "errorMessage": "java.io.UncheckedIOException: /", + "errorCode": "TooManyWarnings", + "errorMessage": "Too many warnings of type CannotParseExternalData generated (max = 2)", "persona": "USER", }, - "taskId": "query-36ea273a-bd6d-48de-b890-2d853d879bf8", + "taskId": "query-ea3e36df-ad67-4870-b136-f5616b17d9c4", }, - "id": "query-36ea273a-bd6d-48de-b890-2d853d879bf8", + "id": "query-ea3e36df-ad67-4870-b136-f5616b17d9c4", "nativeQuery": undefined, "queryContext": undefined, "result": undefined, "segmentStatus": undefined, "sqlQuery": undefined, - "stages": undefined, - "startTime": 2023-07-05T21:40:39.986Z, + "stages": Stages { + "counters": { + "0": { + "0": { + "input0": { + "bytes": [ + 7658, + ], + "files": [ + 1, + ], + "rows": [ + 10, + ], + "totalFiles": [ + 1, + ], + "type": "channel", + }, + "output": { + "bytes": [ + 712, + ], + "frames": [ + 1, + ], + "rows": [ + 10, + ], + "type": "channel", + }, + "sortProgress": { + "levelToMergedBatches": {}, + "levelToTotalBatches": { + "0": 1, + "1": 1, + "2": -1, + }, + "progressDigest": 0, + "totalMergersForUltimateLevel": -1, + "totalMergingLevels": 3, + "type": "sortProgress", + }, + "warnings": { + "CannotParseExternalData": 3, + "type": "warnings", + }, + }, + }, + }, + "stages": [ + { + "definition": { + "id": "query-ea3e36df-ad67-4870-b136-f5616b17d9c4_0", + "input": [ + { + "inputFormat": { + "type": "json", + }, + "inputSource": { + "type": "http", + "uris": [ + "https://static.imply.io/example-data/kttm-with-issues/kttm-blank-lines.json", + ], + }, + "signature": [ + { + "name": "timestamp", + "type": "STRING", + }, + { + "name": "agent_type", + "type": "STRING", + }, + ], + "type": "external", + }, + ], + "maxWorkerCount": 1, + "processor": { + "query": { + "columnTypes": [ + "STRING", + "LONG", + ], + "columns": [ + "agent_type", + "v0", + ], + "context": { + "__resultFormat": "array", + "__timeColumn": "v0", + "__user": "allowAll", + "executionMode": "async", + "finalize": false, + "finalizeAggregations": false, + "groupByEnableMultiValueUnnesting": false, + "maxNumTasks": 2, + "maxParseExceptions": 2, + "queryId": "ea3e36df-ad67-4870-b136-f5616b17d9c4", + "scanSignature": "[{"name":"agent_type","type":"STRING"},{"name":"v0","type":"LONG"}]", + "sqlInsertSegmentGranularity": ""DAY"", + "sqlQueryId": "ea3e36df-ad67-4870-b136-f5616b17d9c4", + "sqlReplaceTimeChunks": "all", + "sqlStringifyArrays": false, + "waitUntilSegmentsLoad": true, + }, + "dataSource": { + "inputFormat": { + "type": "json", + }, + "inputSource": { + "type": "http", + "uris": [ + "https://static.imply.io/example-data/kttm-with-issues/kttm-blank-lines.json", + ], + }, + "signature": [ + { + "name": "timestamp", + "type": "STRING", + }, + { + "name": "agent_type", + "type": "STRING", + }, + ], + "type": "external", + }, + "granularity": { + "type": "all", + }, + "intervals": { + "intervals": [ + "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z", + ], + "type": "intervals", + }, + "legacy": false, + "queryType": "scan", + "resultFormat": "compactedList", + "virtualColumns": [ + { + "expression": "timestamp_parse("timestamp",null,'UTC')", + "name": "v0", + "outputType": "LONG", + "type": "expression", + }, + ], + }, + "type": "scan", + }, + "shuffleCheckHasMultipleValues": true, + "shuffleSpec": { + "clusterBy": { + "bucketByCount": 1, + "columns": [ + { + "columnName": "__bucket", + "order": "ASCENDING", + }, + { + "columnName": "__boost", + "order": "ASCENDING", + }, + ], + }, + "targetSize": 3000000, + "type": "targetSize", + }, + "signature": [ + { + "name": "__bucket", + "type": "LONG", + }, + { + "name": "__boost", + "type": "LONG", + }, + { + "name": "agent_type", + "type": "STRING", + }, + { + "name": "v0", + "type": "LONG", + }, + ], + }, + "duration": 4056, + "output": "localStorage", + "phase": "FAILED", + "shuffle": "globalSort", + "sort": true, + "stageNumber": 0, + "startTime": "2024-07-26T18:05:02.399Z", + "workerCount": 1, + }, + { + "definition": { + "id": "query-ea3e36df-ad67-4870-b136-f5616b17d9c4_1", + "input": [ + { + "stage": 0, + "type": "stage", + }, + ], + "maxWorkerCount": 1, + "processor": { + "columnMappings": [ + { + "outputColumn": "__time", + "queryColumn": "v0", + }, + { + "outputColumn": "agent_type", + "queryColumn": "agent_type", + }, + ], + "dataSchema": { + "dataSource": "kttm-blank-lines", + "dimensionsSpec": { + "dimensionExclusions": [ + "__time", + ], + "dimensions": [ + { + "createBitmapIndex": true, + "multiValueHandling": "SORTED_ARRAY", + "name": "agent_type", + "type": "string", + }, + ], + "includeAllDimensions": false, + "useSchemaDiscovery": false, + }, + "granularitySpec": { + "intervals": [ + "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z", + ], + "queryGranularity": { + "type": "none", + }, + "rollup": false, + "type": "arbitrary", + }, + "metricsSpec": [], + "timestampSpec": { + "column": "__time", + "format": "millis", + "missingValue": null, + }, + "transformSpec": { + "filter": null, + "transforms": [], + }, + }, + "tuningConfig": { + "maxNumWorkers": 1, + "maxRowsInMemory": 100000, + "rowsPerSegment": 3000000, + }, + "type": "segmentGenerator", + }, + "signature": [], + }, + "stageNumber": 1, + }, + ], + }, + "startTime": 2024-07-26T18:04:59.873Z, "status": "FAILED", "usageInfo": undefined, - "warnings": undefined, + "warnings": [ + { + "error": { + "errorCode": "CannotParseExternalData", + "errorMessage": "Unable to parse row [] (Path: https://static.imply.io/example-data/kttm-with-issues/kttm-blank-lines.json, Record: 3, Line: 3)", + }, + "exceptionStackTrace": "org.apache.druid.java.util.common.parsers.ParseException: Unable to parse row [] (Path: https://static.imply.io/example-data/kttm-with-issues/kttm-blank-lines.json, Record: 3, Line: 3) + at org.apache.druid.data.input.IntermediateRowParsingReader$1.hasNext(IntermediateRowParsingReader.java:80) + at org.apache.druid.java.util.common.parsers.CloseableIterator$1.hasNext(CloseableIterator.java:42) + at org.apache.druid.java.util.common.parsers.CloseableIterator$2.findNextIteratorIfNecessary(CloseableIterator.java:72) + at org.apache.druid.java.util.common.parsers.CloseableIterator$2.hasNext(CloseableIterator.java:93) + at org.apache.druid.java.util.common.parsers.CloseableIterator$1.hasNext(CloseableIterator.java:42) + at org.apache.druid.msq.input.external.ExternalSegment$1$1.hasNext(ExternalSegment.java:94) + at org.apache.druid.java.util.common.guava.BaseSequence$1.next(BaseSequence.java:115) + at org.apache.druid.segment.RowWalker.advance(RowWalker.java:75) + at org.apache.druid.segment.RowBasedCursor.advanceUninterruptibly(RowBasedCursor.java:110) + at org.apache.druid.segment.RowBasedCursor.advance(RowBasedCursor.java:103) + at org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessor.populateFrameWriterAndFlushIfNeeded(ScanQueryFrameProcessor.java:374) + at org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessor.populateFrameWriterAndFlushIfNeededWithExceptionHandling(ScanQueryFrameProcessor.java:334) + at org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessor.runWithSegment(ScanQueryFrameProcessor.java:273) + at org.apache.druid.msq.querykit.BaseLeafFrameProcessor.runIncrementally(BaseLeafFrameProcessor.java:88) + at org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessor.runIncrementally(ScanQueryFrameProcessor.java:157) + at org.apache.druid.frame.processor.FrameProcessors$1FrameProcessorWithBaggage.runIncrementally(FrameProcessors.java:75) + at org.apache.druid.frame.processor.FrameProcessorExecutor$1ExecutorRunnable.runProcessorNow(FrameProcessorExecutor.java:230) + at org.apache.druid.frame.processor.FrameProcessorExecutor$1ExecutorRunnable.run(FrameProcessorExecutor.java:138) + at org.apache.druid.msq.exec.WorkerImpl$1$2.run(WorkerImpl.java:838) + at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:539) + at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264) + at org.apache.druid.query.PrioritizedListenableFutureTask.run(PrioritizedExecutorService.java:259) + at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136) + at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635) + at java.base/java.lang.Thread.run(Thread.java:840) + Caused by: com.fasterxml.jackson.databind.exc.MismatchedInputException: No content to map due to end-of-input + at [Source: (byte[])""; line: 1, column: 0] + at com.fasterxml.jackson.databind.exc.MismatchedInputException.from(MismatchedInputException.java:59) + at com.fasterxml.jackson.databind.ObjectMapper._initForReading(ObjectMapper.java:4688) + at com.fasterxml.jackson.databind.ObjectMapper._readMapAndClose(ObjectMapper.java:4586) + at com.fasterxml.jackson.databind.ObjectMapper.readValue(ObjectMapper.java:3609) + at org.apache.druid.data.input.impl.JsonLineReader.parseInputRows(JsonLineReader.java:75) + at org.apache.druid.data.input.impl.JsonLineReader.parseInputRows(JsonLineReader.java:48) + at org.apache.druid.data.input.IntermediateRowParsingReader$1.hasNext(IntermediateRowParsingReader.java:71) + ... 24 more + ", + "host": "localhost:8101", + "stageNumber": 0, + "taskId": "query-ea3e36df-ad67-4870-b136-f5616b17d9c4-worker0_0", + }, + { + "error": { + "errorCode": "CannotParseExternalData", + "errorMessage": "Unable to parse row [] (Path: https://static.imply.io/example-data/kttm-with-issues/kttm-blank-lines.json, Record: 6, Line: 7)", + }, + "exceptionStackTrace": "org.apache.druid.java.util.common.parsers.ParseException: Unable to parse row [] (Path: https://static.imply.io/example-data/kttm-with-issues/kttm-blank-lines.json, Record: 6, Line: 7) + at org.apache.druid.data.input.IntermediateRowParsingReader$1.hasNext(IntermediateRowParsingReader.java:80) + at org.apache.druid.java.util.common.parsers.CloseableIterator$1.hasNext(CloseableIterator.java:42) + at org.apache.druid.java.util.common.parsers.CloseableIterator$2.findNextIteratorIfNecessary(CloseableIterator.java:72) + at org.apache.druid.java.util.common.parsers.CloseableIterator$2.hasNext(CloseableIterator.java:93) + at org.apache.druid.java.util.common.parsers.CloseableIterator$1.hasNext(CloseableIterator.java:42) + at org.apache.druid.msq.input.external.ExternalSegment$1$1.hasNext(ExternalSegment.java:94) + at org.apache.druid.java.util.common.guava.BaseSequence$1.next(BaseSequence.java:115) + at org.apache.druid.segment.RowWalker.advance(RowWalker.java:75) + at org.apache.druid.segment.RowBasedCursor.advanceUninterruptibly(RowBasedCursor.java:110) + at org.apache.druid.segment.RowBasedCursor.advance(RowBasedCursor.java:103) + at org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessor.populateFrameWriterAndFlushIfNeeded(ScanQueryFrameProcessor.java:374) + at org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessor.populateFrameWriterAndFlushIfNeededWithExceptionHandling(ScanQueryFrameProcessor.java:334) + at org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessor.runWithSegment(ScanQueryFrameProcessor.java:273) + at org.apache.druid.msq.querykit.BaseLeafFrameProcessor.runIncrementally(BaseLeafFrameProcessor.java:88) + at org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessor.runIncrementally(ScanQueryFrameProcessor.java:157) + at org.apache.druid.frame.processor.FrameProcessors$1FrameProcessorWithBaggage.runIncrementally(FrameProcessors.java:75) + at org.apache.druid.frame.processor.FrameProcessorExecutor$1ExecutorRunnable.runProcessorNow(FrameProcessorExecutor.java:230) + at org.apache.druid.frame.processor.FrameProcessorExecutor$1ExecutorRunnable.run(FrameProcessorExecutor.java:138) + at org.apache.druid.msq.exec.WorkerImpl$1$2.run(WorkerImpl.java:838) + at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:539) + at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264) + at org.apache.druid.query.PrioritizedListenableFutureTask.run(PrioritizedExecutorService.java:259) + at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136) + at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635) + at java.base/java.lang.Thread.run(Thread.java:840) + Caused by: com.fasterxml.jackson.databind.exc.MismatchedInputException: No content to map due to end-of-input + at [Source: (byte[])""; line: 1, column: 0] + at com.fasterxml.jackson.databind.exc.MismatchedInputException.from(MismatchedInputException.java:59) + at com.fasterxml.jackson.databind.ObjectMapper._initForReading(ObjectMapper.java:4688) + at com.fasterxml.jackson.databind.ObjectMapper._readMapAndClose(ObjectMapper.java:4586) + at com.fasterxml.jackson.databind.ObjectMapper.readValue(ObjectMapper.java:3609) + at org.apache.druid.data.input.impl.JsonLineReader.parseInputRows(JsonLineReader.java:75) + at org.apache.druid.data.input.impl.JsonLineReader.parseInputRows(JsonLineReader.java:48) + at org.apache.druid.data.input.IntermediateRowParsingReader$1.hasNext(IntermediateRowParsingReader.java:71) + ... 24 more + ", + "host": "localhost:8101", + "stageNumber": 0, + "taskId": "query-ea3e36df-ad67-4870-b136-f5616b17d9c4-worker0_0", + }, + ], } `); }); diff --git a/web-console/src/druid-models/execution/execution.ts b/web-console/src/druid-models/execution/execution.ts index dcb5b2d1b93..45bfc517538 100644 --- a/web-console/src/druid-models/execution/execution.ts +++ b/web-console/src/druid-models/execution/execution.ts @@ -194,7 +194,7 @@ export interface ExecutionValue { export class Execution { static USE_TASK_PAYLOAD = true; - static USE_TASK_REPORTS = true; + static USE_TASK_REPORTS = false; static INLINE_DATASOURCE_MARKER = '__query_select'; static getClusterCapacity: (() => Promise) | undefined = @@ -235,7 +235,7 @@ export class Execution { sqlQuery?: string, queryContext?: QueryContext, ): Execution { - const { queryId, schema, result, errorDetails } = asyncSubmitResult; + const { queryId, schema, result, errorDetails, stages, counters, warnings } = asyncSubmitResult; let queryResult: QueryResult | undefined; if (schema && result?.sampleRecords) { @@ -263,6 +263,8 @@ export class Execution { status: Execution.normalizeAsyncState(asyncSubmitResult.state), sqlQuery, queryContext, + stages: Array.isArray(stages) && counters ? new Stages(stages, counters) : undefined, + warnings: Array.isArray(warnings) ? warnings : undefined, error: executionError, destination: typeof result?.dataSource === 'string' diff --git a/web-console/src/druid-models/stages/stages.ts b/web-console/src/druid-models/stages/stages.ts index 04cf22be8f9..0d263a40553 100644 --- a/web-console/src/druid-models/stages/stages.ts +++ b/web-console/src/druid-models/stages/stages.ts @@ -74,6 +74,8 @@ export interface StageDefinition { startTime?: string; duration?: number; sort?: boolean; + shuffle?: string; + output?: string; } export interface ClusterBy { @@ -169,7 +171,7 @@ export type SegmentGenerationProgressFields = | 'rowsPushed'; export interface WarningCounter { - type: 'warning'; + type: 'warnings'; CannotParseExternalData?: number; // More types of warnings might be added later } @@ -192,6 +194,8 @@ function zeroChannelFields(): Record { }; } +export type Counters = Record>; + export class Stages { static readonly QUERY_START_FACTOR = 0.05; static readonly QUERY_END_FACTOR = 0.05; @@ -205,12 +209,9 @@ export class Stages { } public readonly stages: StageDefinition[]; - private readonly counters?: Record>; + private readonly counters?: Counters; - constructor( - stages: StageDefinition[], - counters?: Record>, - ) { + constructor(stages: StageDefinition[], counters?: Counters) { this.stages = stages; this.counters = counters; } diff --git a/web-console/src/druid-models/task/task.ts b/web-console/src/druid-models/task/task.ts index f8e329c3405..1a245508e8d 100644 --- a/web-console/src/druid-models/task/task.ts +++ b/web-console/src/druid-models/task/task.ts @@ -18,7 +18,7 @@ import { C } from '@druid-toolkit/query'; -import type { StageDefinition } from '../stages/stages'; +import type { Counters, StageDefinition } from '../stages/stages'; export type TaskStatus = 'WAITING' | 'PENDING' | 'RUNNING' | 'FAILED' | 'SUCCESS'; export type TaskStatusWithCanceled = TaskStatus | 'CANCELED'; @@ -112,7 +112,7 @@ export interface MsqTaskReportResponse { segmentLoadWaiterStatus?: SegmentLoadWaiterStatus; }; stages: StageDefinition[]; - counters: Record>; + counters: Counters; }; }; error?: any; diff --git a/web-console/src/druid-models/workbench-query/workbench-query.ts b/web-console/src/druid-models/workbench-query/workbench-query.ts index 05818b27606..76bd29a4448 100644 --- a/web-console/src/druid-models/workbench-query/workbench-query.ts +++ b/web-console/src/druid-models/workbench-query/workbench-query.ts @@ -151,11 +151,12 @@ export class WorkbenchQuery { return WorkbenchQuery.enabledQueryEngines; } - static fromEffectiveQueryAndContext(queryString: string, context: QueryContext): WorkbenchQuery { + static fromTaskQueryAndContext(queryString: string, context: QueryContext): WorkbenchQuery { const noSqlOuterLimit = typeof context['sqlOuterLimit'] === 'undefined'; const cleanContext = deleteKeys(context, ['sqlOuterLimit']); let retQuery = WorkbenchQuery.blank() + .changeEngine('sql-msq-task') .changeQueryString(queryString) .changeQueryContext(cleanContext); diff --git a/web-console/src/helpers/execution/sql-task-execution.ts b/web-console/src/helpers/execution/sql-task-execution.ts index ca7c0485bd4..0fa9b090959 100644 --- a/web-console/src/helpers/execution/sql-task-execution.ts +++ b/web-console/src/helpers/execution/sql-task-execution.ts @@ -184,7 +184,7 @@ export async function getTaskExecution( if (!execution) { const statusResp = await Api.instance.get( - `/druid/v2/sql/statements/${encodedId}`, + `/druid/v2/sql/statements/${encodedId}?detail=true`, { cancelToken, }, diff --git a/web-console/src/utils/download.spec.ts b/web-console/src/utils/download.spec.ts index 85d6669acc9..71c051500d0 100644 --- a/web-console/src/utils/download.spec.ts +++ b/web-console/src/utils/download.spec.ts @@ -20,7 +20,10 @@ import { formatForFormat } from './download'; describe('download', () => { it('.formatForFormat', () => { - expect(formatForFormat(null, 'csv')).toEqual('"null"'); + expect(formatForFormat(null, 'csv')).toEqual(''); + expect(formatForFormat(null, 'tsv')).toEqual(''); + expect(formatForFormat('', 'csv')).toEqual('""'); + expect(formatForFormat('null', 'csv')).toEqual('"null"'); expect(formatForFormat('hello\nworld', 'csv')).toEqual('"hello world"'); expect(formatForFormat(123, 'csv')).toEqual('"123"'); expect(formatForFormat(new Date('2021-01-02T03:04:05.678Z'), 'csv')).toEqual( diff --git a/web-console/src/utils/download.ts b/web-console/src/utils/download.ts index aabe2fc1c66..4fb3342e523 100644 --- a/web-console/src/utils/download.ts +++ b/web-console/src/utils/download.ts @@ -43,6 +43,8 @@ export function downloadUrl(url: string, filename: string) { } export function formatForFormat(s: null | string | number | Date, format: 'csv' | 'tsv'): string { + if (s == null) return ''; + // stringify and remove line break const str = stringifyValue(s).replace(/(?:\r\n|\r|\n)/g, ' '); diff --git a/web-console/src/views/workbench-view/recent-query-task-panel/recent-query-task-panel.tsx b/web-console/src/views/workbench-view/recent-query-task-panel/recent-query-task-panel.tsx index 47ba4dd7705..4f254d13407 100644 --- a/web-console/src/views/workbench-view/recent-query-task-panel/recent-query-task-panel.tsx +++ b/web-console/src/views/workbench-view/recent-query-task-panel/recent-query-task-panel.tsx @@ -173,7 +173,7 @@ LIMIT 100`, } onNewTab( - WorkbenchQuery.fromEffectiveQueryAndContext( + WorkbenchQuery.fromTaskQueryAndContext( execution.sqlQuery, execution.queryContext, ).changeLastExecution({ engine: 'sql-msq-task', id: w.taskId }), diff --git a/web-console/src/views/workbench-view/workbench-view.tsx b/web-console/src/views/workbench-view/workbench-view.tsx index a125b8d2987..aaac54e6d4a 100644 --- a/web-console/src/views/workbench-view/workbench-view.tsx +++ b/web-console/src/views/workbench-view/workbench-view.tsx @@ -30,7 +30,8 @@ import type { SqlQuery } from '@druid-toolkit/query'; import { SqlExpression } from '@druid-toolkit/query'; import classNames from 'classnames'; import copy from 'copy-to-clipboard'; -import React, { ComponentProps } from 'react'; +import type { ComponentProps } from 'react'; +import React from 'react'; import { SpecDialog, StringInputDialog } from '../../dialogs'; import type { @@ -464,7 +465,7 @@ export class WorkbenchView extends React.PureComponent Date: Fri, 2 Aug 2024 09:07:15 +0530 Subject: [PATCH 06/12] Fix issues with fetching task reports in SQL statements endpoint for middlemanager (#16832) --- .../apache/druid/msq/sql/resources/SqlStatementResource.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java index 1ee5f1030a4..322727aea92 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java @@ -595,7 +595,7 @@ public class SqlStatementResource )); } catch (DruidException e) { - if (e.getErrorCode().equals("notFound")) { + if (e.getErrorCode().equals("notFound") || e.getMessage().contains("Unable to contact overlord")) { return Optional.empty(); } throw e; From 9b731e8f0a0fb4bc5ef1faa7ad8292cd09159a7b Mon Sep 17 00:00:00 2001 From: zachjsh Date: Fri, 2 Aug 2024 08:48:44 -0400 Subject: [PATCH 07/12] Kinesis Input Format for timestamp, and payload parsing (#16813) * SQL syntax error should target USER persona * * revert change to queryHandler and related tests, based on review comments * * add test * Introduce KinesisRecordEntity to support Kinesis headers in InputFormats * * add kinesisInputFormat and Reader, and tests * * bind KinesisInputFormat class to module * * improve test coverage * * remove references to kafka * * resolve review comments * * remove comment * * fix grammer of comment * * fix comment again * * fix comment again * * more review comments * * add partitionKey * * add check for same timestamp and partitionKey column name * * fix intellij inspection --- .../data/input/kafka/KafkaRecordEntity.java | 2 - .../input/kinesis/KinesisInputFormat.java | 157 +++ .../input/kinesis/KinesisInputReader.java | 256 +++++ .../input/kinesis/KinesisRecordEntity.java | 51 + .../indexing/kinesis/KinesisIndexTask.java | 6 +- .../kinesis/KinesisIndexTaskRunner.java | 10 +- .../kinesis/KinesisIndexingServiceModule.java | 4 +- .../kinesis/KinesisRecordSupplier.java | 28 +- .../kinesis/supervisor/KinesisSupervisor.java | 10 +- .../input/kinesis/KinesisInputFormatTest.java | 940 ++++++++++++++++++ .../kinesis/KinesisIndexTaskTest.java | 94 +- .../kinesis/KinesisRecordSupplierTest.java | 63 +- .../kinesis/KinesisSamplerSpecTest.java | 30 +- .../supervisor/KinesisSupervisorTest.java | 4 +- 14 files changed, 1534 insertions(+), 121 deletions(-) create mode 100644 extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/data/input/kinesis/KinesisInputFormat.java create mode 100644 extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/data/input/kinesis/KinesisInputReader.java create mode 100644 extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/data/input/kinesis/KinesisRecordEntity.java create mode 100644 extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/data/input/kinesis/KinesisInputFormatTest.java diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafka/KafkaRecordEntity.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafka/KafkaRecordEntity.java index 41c2c0a0325..53369cc6ad6 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafka/KafkaRecordEntity.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafka/KafkaRecordEntity.java @@ -33,8 +33,6 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; * key, and timestamp. *

* NOTE: Any records with null values will be skipped, even if they contain non-null keys, or headers - *

- * This functionality is not yet exposed through any built-in InputFormats, but is available for use in extensions. */ public class KafkaRecordEntity extends ByteEntity { diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/data/input/kinesis/KinesisInputFormat.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/data/input/kinesis/KinesisInputFormat.java new file mode 100644 index 00000000000..7d97fffed37 --- /dev/null +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/data/input/kinesis/KinesisInputFormat.java @@ -0,0 +1,157 @@ +/* + * 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.data.input.kinesis; + +import com.amazonaws.services.kinesis.model.Record; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.druid.data.input.InputEntity; +import org.apache.druid.data.input.InputEntityReader; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.impl.JsonInputFormat; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexing.seekablestream.SettableByteEntity; +import org.apache.druid.java.util.common.DateTimes; + +import javax.annotation.Nullable; + +import java.io.File; +import java.util.Objects; + +/** + * Kinesis aware InputFormat. Allows for reading kinesis specific values that are stored in the {@link Record}. At + * this time, this input format only supports reading data from the following record components + *

+ * - {@link Record#data} + * - {@link Record#approximateArrivalTimestamp} + * - {@link Record#partitionKey} + *

+ * This class can be extended easily to read other fields available in the kinesis record. + */ +public class KinesisInputFormat implements InputFormat +{ + private static final String DEFAULT_TIMESTAMP_COLUMN_NAME = "kinesis.timestamp"; + private static final String DEFAULT_PARTITION_KEY_COLUMN_NAME = "kinesis.partitionKey"; + + // Since KinesisInputFormat blends data from record properties, and payload, timestamp spec can be pointing to an + // attribute within one of these 2 sections. To handle scenarios where there is no timestamp value in the payload, we + // induce an artificial timestamp value to avoid unnecessary parser barf out. Users in such situations can use the + // inputFormat's kinesis record timestamp as its primary timestamp. + public static final String DEFAULT_AUTO_TIMESTAMP_STRING = "__kif_auto_timestamp"; + private final TimestampSpec dummyTimestampSpec = new TimestampSpec(DEFAULT_AUTO_TIMESTAMP_STRING, "auto", DateTimes.EPOCH); + + private final InputFormat valueFormat; + private final String timestampColumnName; + private final String partitionKeyColumnName; + + public KinesisInputFormat( + @JsonProperty("valueFormat") InputFormat valueFormat, + @JsonProperty("partitionKeyColumnName") @Nullable String partitionKeyColumnName, + @JsonProperty("timestampColumnName") @Nullable String timestampColumnName + ) + { + this.valueFormat = Preconditions.checkNotNull(valueFormat, "valueFormat must not be null"); + Preconditions.checkState( + !(timestampColumnName != null && timestampColumnName.equals(partitionKeyColumnName)), + "timestampColumnName and partitionKeyColumnName must be different" + ); + this.partitionKeyColumnName = partitionKeyColumnName != null + ? partitionKeyColumnName + : DEFAULT_PARTITION_KEY_COLUMN_NAME; + this.timestampColumnName = timestampColumnName != null ? timestampColumnName : DEFAULT_TIMESTAMP_COLUMN_NAME; + } + + @Override + public boolean isSplittable() + { + return false; + } + + @Override + public InputEntityReader createReader(InputRowSchema inputRowSchema, InputEntity source, File temporaryDirectory) + { + final SettableByteEntity settableByteEntitySource; + if (source instanceof SettableByteEntity) { + settableByteEntitySource = (SettableByteEntity) source; + } else { + settableByteEntitySource = new SettableByteEntity<>(); + settableByteEntitySource.setEntity((KinesisRecordEntity) source); + } + InputRowSchema newInputRowSchema = new InputRowSchema( + dummyTimestampSpec, + inputRowSchema.getDimensionsSpec(), + inputRowSchema.getColumnsFilter(), + inputRowSchema.getMetricNames() + ); + return new KinesisInputReader( + inputRowSchema, + settableByteEntitySource, + JsonInputFormat.withLineSplittable(valueFormat, false).createReader( + newInputRowSchema, + source, + temporaryDirectory + ), + partitionKeyColumnName, + timestampColumnName + ); + } + + @JsonProperty + public InputFormat getValueFormat() + { + return valueFormat; + } + + @Nullable + @JsonProperty + public String getTimestampColumnName() + { + return timestampColumnName; + } + + @Nullable + @JsonProperty + public String getPartitionKeyColumnName() + { + return partitionKeyColumnName; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + KinesisInputFormat that = (KinesisInputFormat) o; + return Objects.equals(valueFormat, that.valueFormat) + && Objects.equals(timestampColumnName, that.timestampColumnName) + && Objects.equals(partitionKeyColumnName, that.partitionKeyColumnName); + } + + @Override + public int hashCode() + { + return Objects.hash(valueFormat, timestampColumnName, partitionKeyColumnName); + } +} diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/data/input/kinesis/KinesisInputReader.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/data/input/kinesis/KinesisInputReader.java new file mode 100644 index 00000000000..d0c30280a2b --- /dev/null +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/data/input/kinesis/KinesisInputReader.java @@ -0,0 +1,256 @@ +/* + * 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.data.input.kinesis; + +import com.google.common.collect.Lists; +import org.apache.druid.data.input.InputEntityReader; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowListPlusRawValues; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.MapBasedInputRow; +import org.apache.druid.data.input.impl.MapInputRowParser; +import org.apache.druid.indexing.seekablestream.SettableByteEntity; +import org.apache.druid.java.util.common.CloseableIterators; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.joda.time.DateTime; + +import java.io.IOException; +import java.util.AbstractMap; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; + +public class KinesisInputReader implements InputEntityReader +{ + + private final InputRowSchema inputRowSchema; + private final SettableByteEntity source; + private final InputEntityReader valueParser; + private final String partitionKeyColumnName; + private final String timestampColumnName; + + public KinesisInputReader( + InputRowSchema inputRowSchema, + SettableByteEntity source, + InputEntityReader valueParser, + String partitionKeyColumnName, + String timestampColumnName + ) + { + this.inputRowSchema = inputRowSchema; + this.source = source; + this.valueParser = valueParser; + this.partitionKeyColumnName = partitionKeyColumnName; + this.timestampColumnName = timestampColumnName; + + } + + @Override + public CloseableIterator read() throws IOException + { + final KinesisRecordEntity record = source.getEntity(); + final Map mergedHeaderMap = extractHeaders(record); + + if (record.getRecord().getData() != null) { + return buildBlendedRows(valueParser, mergedHeaderMap); + } else { + return CloseableIterators.withEmptyBaggage(buildInputRowsForMap(mergedHeaderMap).iterator()); + } + } + + @Override + public CloseableIterator sample() throws IOException + { + final KinesisRecordEntity record = source.getEntity(); + InputRowListPlusRawValues headers = extractHeaderSample(record); + if (record.getRecord().getData() != null) { + return buildBlendedRowsSample(valueParser, headers.getRawValues()); + } else { + final List rows = Collections.singletonList(headers); + return CloseableIterators.withEmptyBaggage(rows.iterator()); + } + } + + private Map extractHeaders(KinesisRecordEntity record) + { + final Map mergedHeaderMap = new HashMap<>(); + mergedHeaderMap.put(timestampColumnName, record.getRecord().getApproximateArrivalTimestamp().getTime()); + mergedHeaderMap.put(partitionKeyColumnName, record.getRecord().getPartitionKey()); + return mergedHeaderMap; + } + + private CloseableIterator buildBlendedRows( + InputEntityReader valueParser, + Map headerKeyList + ) throws IOException + { + return valueParser.read().map( + r -> { + final HashSet newDimensions = new HashSet<>(r.getDimensions()); + final Map event = buildBlendedEventMap(r::getRaw, newDimensions, headerKeyList); + newDimensions.addAll(headerKeyList.keySet()); + // Remove the dummy timestamp added in KinesisInputFormat + newDimensions.remove(KinesisInputFormat.DEFAULT_AUTO_TIMESTAMP_STRING); + + final DateTime timestamp = MapInputRowParser.parseTimestamp(inputRowSchema.getTimestampSpec(), event); + return new MapBasedInputRow( + timestamp, + MapInputRowParser.findDimensions( + inputRowSchema.getTimestampSpec(), + inputRowSchema.getDimensionsSpec(), + newDimensions + ), + event + ); + } + ); + } + + private InputRowListPlusRawValues extractHeaderSample(KinesisRecordEntity record) + { + Map mergedHeaderMap = extractHeaders(record); + return InputRowListPlusRawValues.of(buildInputRowsForMap(mergedHeaderMap), mergedHeaderMap); + } + + private CloseableIterator buildBlendedRowsSample( + InputEntityReader valueParser, + Map headerKeyList + ) throws IOException + { + return valueParser.sample().map( + rowAndValues -> { + if (rowAndValues.getParseException() != null) { + return rowAndValues; + } + List newInputRows = Lists.newArrayListWithCapacity(rowAndValues.getInputRows().size()); + List> newRawRows = Lists.newArrayListWithCapacity(rowAndValues.getRawValues().size()); + + for (Map raw : rowAndValues.getRawValuesList()) { + newRawRows.add(buildBlendedEventMap(raw::get, raw.keySet(), headerKeyList)); + } + for (InputRow r : rowAndValues.getInputRows()) { + if (r != null) { + final HashSet newDimensions = new HashSet<>(r.getDimensions()); + final Map event = buildBlendedEventMap( + r::getRaw, + newDimensions, + headerKeyList + ); + newDimensions.addAll(headerKeyList.keySet()); + // Remove the dummy timestamp added in KinesisInputFormat + newDimensions.remove(KinesisInputFormat.DEFAULT_AUTO_TIMESTAMP_STRING); + newInputRows.add( + new MapBasedInputRow( + inputRowSchema.getTimestampSpec().extractTimestamp(event), + MapInputRowParser.findDimensions( + inputRowSchema.getTimestampSpec(), + inputRowSchema.getDimensionsSpec(), + newDimensions + ), + event + ) + ); + } + } + return InputRowListPlusRawValues.ofList(newRawRows, newInputRows, null); + } + ); + } + + private List buildInputRowsForMap(Map headerKeyList) + { + return Collections.singletonList( + new MapBasedInputRow( + inputRowSchema.getTimestampSpec().extractTimestamp(headerKeyList), + MapInputRowParser.findDimensions( + inputRowSchema.getTimestampSpec(), + inputRowSchema.getDimensionsSpec(), + headerKeyList.keySet() + ), + headerKeyList + ) + ); + } + + private Map buildBlendedEventMap( + Function getRowValue, + Set rowDimensions, + Map fallback + ) + { + final Set keySet = new HashSet<>(fallback.keySet()); + keySet.addAll(rowDimensions); + + return new AbstractMap() + { + @Override + public Object get(Object key) + { + final String skey = (String) key; + final Object val = getRowValue.apply(skey); + if (val == null) { + return fallback.get(skey); + } + return val; + } + + @Override + public Set keySet() + { + return keySet; + } + + @Override + public Set> entrySet() + { + return keySet().stream() + .map( + field -> new Entry() + { + @Override + public String getKey() + { + return field; + } + + @Override + public Object getValue() + { + return get(field); + } + + @Override + public Object setValue(final Object value) + { + throw new UnsupportedOperationException(); + } + } + ) + .collect(Collectors.toCollection(LinkedHashSet::new)); + } + }; + } +} diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/data/input/kinesis/KinesisRecordEntity.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/data/input/kinesis/KinesisRecordEntity.java new file mode 100644 index 00000000000..a490fd8f4c3 --- /dev/null +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/data/input/kinesis/KinesisRecordEntity.java @@ -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.data.input.kinesis; + +import com.amazonaws.services.kinesis.model.Record; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.impl.ByteEntity; +import org.apache.druid.indexing.kinesis.KinesisRecordSupplier; + +/** + * A {@link ByteEntity} generated by {@link KinesisRecordSupplier} and fed to any {@link InputFormat} used by kinesis + * indexing tasks. + *

+ * It can be used as a regular ByteEntity, in which case the kinesis record value is returned, but the {@link #getRecord} + * method also allows Kinesis-aware {@link InputFormat} implementations to read the full kinesis record, including + * timestamp, encrytion key, patition key, and sequence number + *

+ * NOTE: Any records with null values will be returned as records with just only kinesis properties and no data payload + */ +public class KinesisRecordEntity extends ByteEntity +{ + private final Record record; + + public KinesisRecordEntity(Record record) + { + super(record.getData()); + this.record = record; + } + + public Record getRecord() + { + return record; + } +} diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java index fb019f10030..bea69d96c3d 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -27,7 +27,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.inject.name.Named; import org.apache.druid.common.aws.AWSCredentialsConfig; -import org.apache.druid.data.input.impl.ByteEntity; +import org.apache.druid.data.input.kinesis.KinesisRecordEntity; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.task.TaskResource; @@ -46,7 +46,7 @@ import java.util.Collections; import java.util.Map; import java.util.Set; -public class KinesisIndexTask extends SeekableStreamIndexTask +public class KinesisIndexTask extends SeekableStreamIndexTask { private static final String TYPE = "index_kinesis"; @@ -100,7 +100,7 @@ public class KinesisIndexTask extends SeekableStreamIndexTask createTaskRunner() + protected SeekableStreamIndexTaskRunner createTaskRunner() { //noinspection unchecked return new KinesisIndexTaskRunner( diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java index 75f23da0e1f..72e61635912 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java @@ -21,8 +21,8 @@ package org.apache.druid.indexing.kinesis; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.data.input.impl.ByteEntity; import org.apache.druid.data.input.impl.InputRowParser; +import org.apache.druid.data.input.kinesis.KinesisRecordEntity; import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata; @@ -49,7 +49,7 @@ import java.util.Set; import java.util.TreeMap; import java.util.concurrent.ConcurrentMap; -public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner +public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner { private static final EmittingLogger log = new EmittingLogger(KinesisIndexTaskRunner.class); private static final long POLL_TIMEOUT = 100; @@ -81,8 +81,8 @@ public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner> getRecords( - RecordSupplier recordSupplier, TaskToolbox toolbox + protected List> getRecords( + RecordSupplier recordSupplier, TaskToolbox toolbox ) { return recordSupplier.poll(POLL_TIMEOUT); @@ -119,7 +119,7 @@ public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner recordSupplier, + RecordSupplier recordSupplier, Set> assignment ) { diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java index 0cce1a7e698..5ac9022d001 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java @@ -26,6 +26,7 @@ import com.google.common.collect.ImmutableList; import com.google.inject.Binder; import com.google.inject.name.Names; import org.apache.druid.common.aws.AWSCredentialsConfig; +import org.apache.druid.data.input.kinesis.KinesisInputFormat; import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorSpec; import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorTuningConfig; @@ -50,7 +51,8 @@ public class KinesisIndexingServiceModule implements DruidModule new NamedType(KinesisIndexTaskIOConfig.class, SCHEME), new NamedType(KinesisSupervisorTuningConfig.class, SCHEME), new NamedType(KinesisSupervisorSpec.class, SCHEME), - new NamedType(KinesisSamplerSpec.class, SCHEME) + new NamedType(KinesisSamplerSpec.class, SCHEME), + new NamedType(KinesisInputFormat.class, SCHEME) ) ); } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index 36047ce429d..07a0da32a95 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -26,6 +26,7 @@ import com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider; import com.amazonaws.client.builder.AwsClientBuilder; import com.amazonaws.services.kinesis.AmazonKinesis; import com.amazonaws.services.kinesis.AmazonKinesisClientBuilder; +import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord; import com.amazonaws.services.kinesis.model.DescribeStreamRequest; import com.amazonaws.services.kinesis.model.ExpiredIteratorException; import com.amazonaws.services.kinesis.model.GetRecordsRequest; @@ -49,7 +50,7 @@ import com.google.common.collect.Maps; import org.apache.druid.common.aws.AWSClientUtil; import org.apache.druid.common.aws.AWSCredentialsConfig; import org.apache.druid.common.aws.AWSCredentialsUtils; -import org.apache.druid.data.input.impl.ByteEntity; +import org.apache.druid.data.input.kinesis.KinesisRecordEntity; import org.apache.druid.error.DruidException; import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisor; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; @@ -69,7 +70,6 @@ import javax.annotation.Nullable; import java.lang.invoke.MethodHandle; import java.lang.invoke.MethodHandles; import java.lang.reflect.Method; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -94,7 +94,7 @@ import java.util.stream.Collectors; * This class implements a local buffer for storing fetched Kinesis records. Fetching is done * in background threads. */ -public class KinesisRecordSupplier implements RecordSupplier +public class KinesisRecordSupplier implements RecordSupplier { private static final EmittingLogger log = new EmittingLogger(KinesisRecordSupplier.class); private static final long PROVISIONED_THROUGHPUT_EXCEEDED_BACKOFF_MS = 3000; @@ -210,7 +210,7 @@ public class KinesisRecordSupplier implements RecordSupplier currRecord; + OrderedPartitionableRecord currRecord; long recordBufferOfferWaitMillis; try { @@ -248,7 +248,7 @@ public class KinesisRecordSupplier implements RecordSupplier data; + final List data; if (deaggregateHandle == null || getDataHandle == null) { throw new ISE("deaggregateHandle or getDataHandle is null!"); @@ -256,15 +256,15 @@ public class KinesisRecordSupplier implements RecordSupplier(); - final List userRecords = (List) deaggregateHandle.invokeExact( + final List userRecords = (List) deaggregateHandle.invokeExact( Collections.singletonList(kinesisRecord) ); int recordSize = 0; - for (Object userRecord : userRecords) { - ByteEntity byteEntity = new ByteEntity((ByteBuffer) getDataHandle.invoke(userRecord)); - recordSize += byteEntity.getBuffer().array().length; - data.add(byteEntity); + for (UserRecord userRecord : userRecords) { + KinesisRecordEntity kinesisRecordEntity = new KinesisRecordEntity(userRecord); + recordSize += kinesisRecordEntity.getBuffer().array().length; + data.add(kinesisRecordEntity); } @@ -408,7 +408,7 @@ public class KinesisRecordSupplier implements RecordSupplier, PartitionResource> partitionResources = new ConcurrentHashMap<>(); - private MemoryBoundLinkedBlockingQueue> records; + private MemoryBoundLinkedBlockingQueue> records; private final boolean backgroundFetchEnabled; private volatile boolean closed = false; @@ -615,12 +615,12 @@ public class KinesisRecordSupplier implements RecordSupplier> poll(long timeout) + public List> poll(long timeout) { start(); try { - List>> polledRecords = new ArrayList<>(); + List>> polledRecords = new ArrayList<>(); records.drain( polledRecords, @@ -1040,7 +1040,7 @@ public class KinesisRecordSupplier implements RecordSupplier> newQ = + MemoryBoundLinkedBlockingQueue> newQ = new MemoryBoundLinkedBlockingQueue<>(recordBufferSizeBytes); records.stream() diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index a142f414762..2f00c8c16cc 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -25,7 +25,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import org.apache.druid.common.aws.AWSCredentialsConfig; import org.apache.druid.common.utils.IdUtils; -import org.apache.druid.data.input.impl.ByteEntity; +import org.apache.druid.data.input.kinesis.KinesisRecordEntity; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.kinesis.KinesisDataSourceMetadata; @@ -74,7 +74,7 @@ import java.util.stream.Collectors; * tasks to satisfy the desired number of replicas. As tasks complete, new tasks are queued to process the next range of * Kinesis sequences. */ -public class KinesisSupervisor extends SeekableStreamSupervisor +public class KinesisSupervisor extends SeekableStreamSupervisor { private static final EmittingLogger log = new EmittingLogger(KinesisSupervisor.class); @@ -150,7 +150,7 @@ public class KinesisSupervisor extends SeekableStreamSupervisor> createIndexTasks( + protected List> createIndexTasks( int replicas, String baseSequenceName, ObjectMapper sortingMapper, @@ -164,7 +164,7 @@ public class KinesisSupervisor extends SeekableStreamSupervisor context = createBaseTaskContexts(); context.put(CHECKPOINTS_CTX_KEY, checkpoints); - List> taskList = new ArrayList<>(); + List> taskList = new ArrayList<>(); for (int i = 0; i < replicas; i++) { String taskId = IdUtils.getRandomIdWithPrefix(baseSequenceName); taskList.add(new KinesisIndexTask( @@ -183,7 +183,7 @@ public class KinesisSupervisor extends SeekableStreamSupervisor setupRecordSupplier() throws RuntimeException + protected RecordSupplier setupRecordSupplier() throws RuntimeException { KinesisSupervisorIOConfig ioConfig = spec.getIoConfig(); KinesisIndexTaskTuningConfig taskTuningConfig = spec.getTuningConfig(); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/data/input/kinesis/KinesisInputFormatTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/data/input/kinesis/KinesisInputFormatTest.java new file mode 100644 index 00000000000..130f31681de --- /dev/null +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/data/input/kinesis/KinesisInputFormatTest.java @@ -0,0 +1,940 @@ +/* + * 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.data.input.kinesis; + +import com.amazonaws.services.kinesis.model.Record; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.data.input.ColumnsFilter; +import org.apache.druid.data.input.InputEntityReader; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowListPlusRawValues; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.impl.CsvInputFormat; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.JsonInputFormat; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexing.common.TestUtils; +import org.apache.druid.indexing.seekablestream.SettableByteEntity; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.java.util.common.parsers.JSONPathFieldSpec; +import org.apache.druid.java.util.common.parsers.JSONPathFieldType; +import org.apache.druid.java.util.common.parsers.JSONPathSpec; +import org.apache.druid.java.util.common.parsers.ParseException; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collections; +import java.util.Date; +import java.util.List; + +public class KinesisInputFormatTest +{ + static { + NullHandling.initializeForTests(); + } + + + private static final String KINESIS_APPROXIMATE_TIME_DATE = "2024-07-29"; + private static final long KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS = DateTimes.of(KINESIS_APPROXIMATE_TIME_DATE).getMillis(); + private static final String DATA_TIMSTAMP_DATE = "2024-07-30"; + private static final String PARTITION_KEY = "partition_key_1"; + + private static final byte[] SIMPLE_JSON_VALUE_BYTES = StringUtils.toUtf8( + TestUtils.singleQuoteToStandardJson( + "{" + + " 'timestamp': '" + DATA_TIMSTAMP_DATE + "'," + + " 'bar': null," + + " 'foo': 'x'," + + " 'baz': 4," + + " 'o': {'mg': 1}" + + "}" + ) + ); + + private KinesisInputFormat format; + + @Before + public void setUp() + { + format = new KinesisInputFormat( + // Value Format + new JsonInputFormat( + new JSONPathSpec( + true, + ImmutableList.of( + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz", "baz"), + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz2", "baz2"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg", "$.o.mg"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg2", "$.o.mg2"), + new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg", ".o.mg"), + new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg2", ".o.mg2") + ) + ), + null, + null, + false, + false + ), + "kinesis.newts.partitionKey", + "kinesis.newts.timestamp" + ); + } + + @Test + public void testSerde() throws JsonProcessingException + { + final ObjectMapper mapper = new ObjectMapper(); + KinesisInputFormat kif = new KinesisInputFormat( + // Value Format + new JsonInputFormat( + new JSONPathSpec( + true, + ImmutableList.of( + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz", "baz"), + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz2", "baz2"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg", "$.o.mg"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg2", "$.o.mg2"), + new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg", ".o.mg"), + new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg2", ".o.mg2") + ) + ), + null, + null, + false, + false + ), + "kinesis.newts.partitionKey", + "kinesis.newts.timestamp" + ); + Assert.assertEquals(format, kif); + + final byte[] formatBytes = mapper.writeValueAsBytes(format); + final byte[] kifBytes = mapper.writeValueAsBytes(kif); + Assert.assertArrayEquals(formatBytes, kifBytes); + } + + @Test + public void testTimestampFromHeader() throws IOException + { + KinesisRecordEntity inputEntity = makeInputEntity(SIMPLE_JSON_VALUE_BYTES, KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS); + + final InputEntityReader reader = format.createReader( + new InputRowSchema( + new TimestampSpec("kinesis.newts.timestamp", "iso", null), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas( + ImmutableList.of( + "bar", + "foo" + ) + ) + ), + ColumnsFilter.all() + ), + newSettableByteEntity(inputEntity), + null + ); + + final int numExpectedIterations = 1; + try (CloseableIterator iterator = reader.read()) { + int numActualIterations = 0; + while (iterator.hasNext()) { + + final InputRow row = iterator.next(); + // Payload verifications + // this isn't super realistic, since most of these columns are not actually defined in the dimensionSpec + // but test reading them anyway since it isn't technically illegal + + Assert.assertEquals(DateTimes.of(KINESIS_APPROXIMATE_TIME_DATE), row.getTimestamp()); + Assert.assertEquals( + String.valueOf(KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS), + Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp")) + ); + Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey"))); + Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg"))); + Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o")); + + Assert.assertTrue(row.getDimension("root_baz2").isEmpty()); + Assert.assertTrue(row.getDimension("path_omg2").isEmpty()); + Assert.assertTrue(row.getDimension("jq_omg2").isEmpty()); + Assert.assertTrue(row.getDimension("jq_omg2").isEmpty()); + numActualIterations++; + } + + Assert.assertEquals(numExpectedIterations, numActualIterations); + } + } + + @Test + public void testRawSample() throws IOException + { + KinesisRecordEntity inputEntity = makeInputEntity(SIMPLE_JSON_VALUE_BYTES, KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS); + + final InputEntityReader reader = format.createReader( + new InputRowSchema( + new TimestampSpec(KinesisInputFormat.DEFAULT_AUTO_TIMESTAMP_STRING, "auto", DateTimes.EPOCH), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas( + ImmutableList.of( + "bar", + "foo" + ) + ) + ), + ColumnsFilter.all() + ), + newSettableByteEntity(inputEntity), + null + ); + + final int numExpectedIterations = 1; + try (CloseableIterator iterator = reader.sample()) { + int numActualIterations = 0; + while (iterator.hasNext()) { + + final InputRowListPlusRawValues rawValues = iterator.next(); + Assert.assertEquals(1, rawValues.getInputRows().size()); + InputRow row = rawValues.getInputRows().get(0); + // Payload verifications + // this isn't super realistic, since most of these columns are not actually defined in the dimensionSpec + // but test reading them anyway since it isn't technically illegal + + Assert.assertEquals( + String.valueOf(KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS), + Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp")) + ); + Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey"))); + Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg"))); + Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o")); + + Assert.assertTrue(row.getDimension("root_baz2").isEmpty()); + Assert.assertTrue(row.getDimension("path_omg2").isEmpty()); + Assert.assertTrue(row.getDimension("jq_omg2").isEmpty()); + Assert.assertTrue(row.getDimension("jq_omg2").isEmpty()); + + numActualIterations++; + } + + Assert.assertEquals(numExpectedIterations, numActualIterations); + } + } + + @Test + public void testProcessesSampleTimestampFromHeader() throws IOException + { + KinesisRecordEntity inputEntity = makeInputEntity(SIMPLE_JSON_VALUE_BYTES, KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS); + + final InputEntityReader reader = format.createReader( + new InputRowSchema( + new TimestampSpec("kinesis.newts.timestamp", "iso", null), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas( + ImmutableList.of( + "bar", + "foo" + ) + ) + ), + ColumnsFilter.all() + ), + newSettableByteEntity(inputEntity), + null + ); + + final int numExpectedIterations = 1; + try (CloseableIterator iterator = reader.sample()) { + int numActualIterations = 0; + while (iterator.hasNext()) { + + final InputRowListPlusRawValues rawValues = iterator.next(); + Assert.assertEquals(1, rawValues.getInputRows().size()); + InputRow row = rawValues.getInputRows().get(0); + // Payload verifications + // this isn't super realistic, since most of these columns are not actually defined in the dimensionSpec + // but test reading them anyway since it isn't technically illegal + + Assert.assertEquals(DateTimes.of(String.valueOf(KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS)), row.getTimestamp()); + Assert.assertEquals( + String.valueOf(KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS), + Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp")) + ); + Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey"))); + Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg"))); + Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o")); + + Assert.assertTrue(row.getDimension("root_baz2").isEmpty()); + Assert.assertTrue(row.getDimension("path_omg2").isEmpty()); + Assert.assertTrue(row.getDimension("jq_omg2").isEmpty()); + Assert.assertTrue(row.getDimension("jq_omg2").isEmpty()); + + numActualIterations++; + } + + Assert.assertEquals(numExpectedIterations, numActualIterations); + } + } + + @Test + public void testWithMultipleMixedRecordsTimestampFromHeader() throws IOException + { + final byte[][] values = new byte[5][]; + for (int i = 0; i < values.length; i++) { + values[i] = StringUtils.toUtf8( + "{\n" + + " \"timestamp\": \"2024-07-2" + i + "\",\n" + + " \"bar\": null,\n" + + " \"foo\": \"x\",\n" + + " \"baz\": 4,\n" + + " \"index\": " + i + ",\n" + + " \"o\": {\n" + + " \"mg\": 1\n" + + " }\n" + + "}" + ); + } + + SettableByteEntity settableByteEntity = new SettableByteEntity<>(); + + final InputEntityReader reader = format.createReader( + new InputRowSchema( + new TimestampSpec("kinesis.newts.timestamp", "iso", null), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas( + ImmutableList.of( + "bar", + "foo", + "kinesis.newts.timestamp" + ) + ) + ), + ColumnsFilter.all() + ), + settableByteEntity, + null + ); + + for (int i = 0; i < values.length; i++) { + KinesisRecordEntity inputEntity = makeInputEntity(values[i], DateTimes.of("2024-07-1" + i).getMillis()); + settableByteEntity.setEntity(inputEntity); + + final int numExpectedIterations = 1; + try (CloseableIterator iterator = reader.read()) { + int numActualIterations = 0; + while (iterator.hasNext()) { + + final InputRow row = iterator.next(); + + // Payload verification + // this isn't super realistic, since most of these columns are not actually defined in the dimensionSpec + // but test reading them anyway since it isn't technically illegal + Assert.assertEquals(DateTimes.of("2024-07-1" + i), row.getTimestamp()); + Assert.assertEquals( + String.valueOf(DateTimes.of("2024-07-1" + i).getMillis()), + Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp")) + ); + Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey"))); + Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg"))); + Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o")); + Assert.assertEquals(String.valueOf(i), Iterables.getOnlyElement(row.getDimension("index"))); + + Assert.assertTrue(row.getDimension("root_baz2").isEmpty()); + Assert.assertTrue(row.getDimension("path_omg2").isEmpty()); + Assert.assertTrue(row.getDimension("jq_omg2").isEmpty()); + + numActualIterations++; + } + + Assert.assertEquals(numExpectedIterations, numActualIterations); + } + } + } + + @Test + public void testTimestampFromData() throws IOException + { + KinesisRecordEntity inputEntity = makeInputEntity(SIMPLE_JSON_VALUE_BYTES, KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS); + + final InputEntityReader reader = format.createReader( + new InputRowSchema( + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas( + ImmutableList.of( + "bar", + "foo" + ) + ) + ), + ColumnsFilter.all() + ), + newSettableByteEntity(inputEntity), + null + ); + + final int numExpectedIterations = 1; + try (CloseableIterator iterator = reader.read()) { + int numActualIterations = 0; + while (iterator.hasNext()) { + + final InputRow row = iterator.next(); + // Payload verifications + // this isn't super realistic, since most of these columns are not actually defined in the dimensionSpec + // but test reading them anyway since it isn't technically illegal + + Assert.assertEquals(DateTimes.of(DATA_TIMSTAMP_DATE), row.getTimestamp()); + Assert.assertEquals( + String.valueOf(KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS), + Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp")) + ); + Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey"))); + Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg"))); + Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o")); + + Assert.assertTrue(row.getDimension("root_baz2").isEmpty()); + Assert.assertTrue(row.getDimension("path_omg2").isEmpty()); + Assert.assertTrue(row.getDimension("jq_omg2").isEmpty()); + Assert.assertTrue(row.getDimension("jq_omg2").isEmpty()); + numActualIterations++; + } + + Assert.assertEquals(numExpectedIterations, numActualIterations); + } + } + + @Test + public void testWithMultipleMixedRecordsTimestampFromData() throws IOException + { + final byte[][] values = new byte[5][]; + for (int i = 0; i < values.length; i++) { + values[i] = StringUtils.toUtf8( + "{\n" + + " \"timestamp\": \"2024-07-2" + i + "\",\n" + + " \"bar\": null,\n" + + " \"foo\": \"x\",\n" + + " \"baz\": 4,\n" + + " \"index\": " + i + ",\n" + + " \"o\": {\n" + + " \"mg\": 1\n" + + " }\n" + + "}" + ); + } + + SettableByteEntity settableByteEntity = new SettableByteEntity<>(); + + final InputEntityReader reader = format.createReader( + new InputRowSchema( + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas( + ImmutableList.of( + "bar", + "foo", + "kinesis.newts.timestamp" + ) + ) + ), + ColumnsFilter.all() + ), + settableByteEntity, + null + ); + + for (int i = 0; i < values.length; i++) { + KinesisRecordEntity inputEntity = makeInputEntity(values[i], KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS); + settableByteEntity.setEntity(inputEntity); + + final int numExpectedIterations = 1; + try (CloseableIterator iterator = reader.read()) { + int numActualIterations = 0; + while (iterator.hasNext()) { + + final InputRow row = iterator.next(); + + // Payload verification + // this isn't super realistic, since most of these columns are not actually defined in the dimensionSpec + // but test reading them anyway since it isn't technically illegal + Assert.assertEquals(DateTimes.of("2024-07-2" + i), row.getTimestamp()); + Assert.assertEquals( + String.valueOf(DateTimes.of("2024-07-29").getMillis()), + Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp")) + ); + Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey"))); + Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg"))); + Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o")); + Assert.assertEquals(String.valueOf(i), Iterables.getOnlyElement(row.getDimension("index"))); + + Assert.assertTrue(row.getDimension("root_baz2").isEmpty()); + Assert.assertTrue(row.getDimension("path_omg2").isEmpty()); + Assert.assertTrue(row.getDimension("jq_omg2").isEmpty()); + + numActualIterations++; + } + + Assert.assertEquals(numExpectedIterations, numActualIterations); + } + } + } + + @Test + public void testMissingTimestampThrowsException() throws IOException + { + KinesisRecordEntity inputEntity = + makeInputEntity(SIMPLE_JSON_VALUE_BYTES, KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS); + + final InputEntityReader reader = format.createReader( + new InputRowSchema( + new TimestampSpec("time", "iso", null), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas( + ImmutableList.of( + "bar", + "foo", + "kinesis.newts.timestamp" + ) + ) + ), + ColumnsFilter.all() + ), + newSettableByteEntity(inputEntity), + null + ); + + try (CloseableIterator iterator = reader.read()) { + while (iterator.hasNext()) { + Throwable t = Assert.assertThrows(ParseException.class, iterator::next); + Assert.assertTrue( + t.getMessage().startsWith("Timestamp[null] is unparseable! Event: {") + ); + } + } + } + + @Test + public void testWithSchemaDiscoveryKinesisTimestampExcluded() throws IOException + { + KinesisRecordEntity inputEntity = + makeInputEntity(SIMPLE_JSON_VALUE_BYTES, KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS); + + final InputEntityReader reader = format.createReader( + new InputRowSchema( + new TimestampSpec("timestamp", "iso", null), + DimensionsSpec.builder() + .useSchemaDiscovery(true) + .setDimensionExclusions(ImmutableList.of("kinesis.newts.timestamp")) + .build(), + ColumnsFilter.all() + ), + newSettableByteEntity(inputEntity), + null + ); + + final int numExpectedIterations = 1; + try (CloseableIterator iterator = reader.read()) { + int numActualIterations = 0; + while (iterator.hasNext()) { + + final InputRow row = iterator.next(); + List expectedDimensions = Arrays.asList( + "foo", + "root_baz", + "o", + "bar", + "path_omg", + "jq_omg", + "jq_omg2", + "baz", + "root_baz2", + "path_omg2", + "kinesis.newts.partitionKey" + ); + Collections.sort(expectedDimensions); + Collections.sort(row.getDimensions()); + Assert.assertEquals( + expectedDimensions, + row.getDimensions() + ); + + // Payload verifications + Assert.assertEquals(DateTimes.of(DATA_TIMSTAMP_DATE), row.getTimestamp()); + Assert.assertEquals( + String.valueOf(DateTimes.of(KINESIS_APPROXIMATE_TIME_DATE).getMillis()), + Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp")) + ); + Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey"))); + Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg"))); + Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o")); + + Assert.assertTrue(row.getDimension("root_baz2").isEmpty()); + Assert.assertTrue(row.getDimension("path_omg2").isEmpty()); + Assert.assertTrue(row.getDimension("jq_omg2").isEmpty()); + + numActualIterations++; + } + + Assert.assertEquals(numExpectedIterations, numActualIterations); + } + } + + @Test + public void testWithSchemaDiscoveryTimestampFromHeader() throws IOException + { + KinesisRecordEntity inputEntity = + makeInputEntity(SIMPLE_JSON_VALUE_BYTES, KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS); + + final InputEntityReader reader = format.createReader( + new InputRowSchema( + new TimestampSpec("kinesis.newts.timestamp", "iso", null), + DimensionsSpec.builder() + .useSchemaDiscovery(true) + .build(), + ColumnsFilter.all() + ), + newSettableByteEntity(inputEntity), + null + ); + + final int numExpectedIterations = 1; + try (CloseableIterator iterator = reader.read()) { + int numActualIterations = 0; + while (iterator.hasNext()) { + + final InputRow row = iterator.next(); + List expectedDimensions = Arrays.asList( + "foo", + "timestamp", + "root_baz", + "o", + "bar", + "path_omg", + "jq_omg", + "jq_omg2", + "baz", + "root_baz2", + "path_omg2", + "kinesis.newts.partitionKey" + ); + Collections.sort(expectedDimensions); + Collections.sort(row.getDimensions()); + Assert.assertEquals( + expectedDimensions, + row.getDimensions() + ); + + // Payload verifications + Assert.assertEquals(DateTimes.of(KINESIS_APPROXIMATE_TIME_DATE), row.getTimestamp()); + Assert.assertEquals( + String.valueOf(DateTimes.of(KINESIS_APPROXIMATE_TIME_DATE).getMillis()), + Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp")) + ); + Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey"))); + Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg"))); + Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o")); + + Assert.assertTrue(row.getDimension("root_baz2").isEmpty()); + Assert.assertTrue(row.getDimension("path_omg2").isEmpty()); + Assert.assertTrue(row.getDimension("jq_omg2").isEmpty()); + + numActualIterations++; + } + + Assert.assertEquals(numExpectedIterations, numActualIterations); + } + } + + @Test + public void testValueInCsvFormat() throws IOException + { + format = new KinesisInputFormat( + // Value Format + new CsvInputFormat( + Arrays.asList("foo", "bar", "timestamp", "baz"), + null, + false, + false, + 0 + ), + "kinesis.newts.partitionKey", + "kinesis.newts.timestamp" + ); + + KinesisRecordEntity inputEntity = + makeInputEntity(StringUtils.toUtf8("x,,2024-07-30,4"), KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS); + + final InputEntityReader reader = format.createReader( + new InputRowSchema( + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas( + ImmutableList.of( + "bar", + "foo", + "kinesis.newts.timestamp", + "kinesis.newts.partitionKey" + ) + ) + ), + ColumnsFilter.all() + ), + newSettableByteEntity(inputEntity), + null + ); + + final int numExpectedIterations = 1; + try (CloseableIterator iterator = reader.read()) { + int numActualIterations = 0; + while (iterator.hasNext()) { + + final InputRow row = iterator.next(); + Assert.assertEquals( + Arrays.asList( + "bar", + "foo", + "kinesis.newts.timestamp", + "kinesis.newts.partitionKey" + ), + row.getDimensions() + ); + // Payload verifications + // this isn't super realistic, since most of these columns are not actually defined in the dimensionSpec + // but test reading them anyway since it isn't technically illegal + + Assert.assertEquals(DateTimes.of("2024-07-30"), row.getTimestamp()); + Assert.assertEquals( + String.valueOf(DateTimes.of(KINESIS_APPROXIMATE_TIME_DATE).getMillis()), + Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp")) + ); + Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey"))); + Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz"))); + Assert.assertTrue(row.getDimension("bar").isEmpty()); + + numActualIterations++; + } + + Assert.assertEquals(numExpectedIterations, numActualIterations); + } + } + + @Test + public void testWithPartialDeclarationSchemaDiscovery() throws IOException + { + KinesisRecordEntity inputEntity = + makeInputEntity(SIMPLE_JSON_VALUE_BYTES, KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS); + + final InputEntityReader reader = format.createReader( + new InputRowSchema( + new TimestampSpec("timestamp", "iso", null), + DimensionsSpec.builder().setDimensions( + DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar")) + ).useSchemaDiscovery(true).build(), + ColumnsFilter.all() + ), + newSettableByteEntity(inputEntity), + null + ); + + final int numExpectedIterations = 1; + try (CloseableIterator iterator = reader.read()) { + int numActualIterations = 0; + while (iterator.hasNext()) { + + final InputRow row = iterator.next(); + + List expectedDimensions = Arrays.asList( + "bar", + "foo", + "kinesis.newts.timestamp", + "kinesis.newts.partitionKey", + "root_baz", + "o", + "path_omg", + "jq_omg", + "jq_omg2", + "baz", + "root_baz2", + "path_omg2" + ); + Collections.sort(expectedDimensions); + Collections.sort(row.getDimensions()); + Assert.assertEquals( + expectedDimensions, + row.getDimensions() + ); + + // Payload verifications + Assert.assertEquals(DateTimes.of(DATA_TIMSTAMP_DATE), row.getTimestamp()); + Assert.assertEquals( + String.valueOf(DateTimes.of(KINESIS_APPROXIMATE_TIME_DATE).getMillis()), + Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp")) + ); + Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey"))); + Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg"))); + Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o")); + + Assert.assertTrue(row.getDimension("root_baz2").isEmpty()); + Assert.assertTrue(row.getDimension("path_omg2").isEmpty()); + Assert.assertTrue(row.getDimension("jq_omg2").isEmpty()); + + numActualIterations++; + } + + Assert.assertEquals(numExpectedIterations, numActualIterations); + } + } + + @Test + @SuppressWarnings("ResultOfMethodCallIgnored") + public void testValidInputFormatConstruction() + { + InputFormat valueFormat = new JsonInputFormat( + new JSONPathSpec( + true, + ImmutableList.of( + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz", "baz"), + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz2", "baz2"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg", "$.o.mg"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg2", "$.o.mg2"), + new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg", ".o.mg"), + new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg2", ".o.mg2") + ) + ), + null, + null, + false, + false + ); + // null partitionKeyColumnName and null timestampColumnName is valid + new KinesisInputFormat(valueFormat, null, null); + + // non-null partitionKeyColumnName and null timestampColumnName is valid + new KinesisInputFormat(valueFormat, "kinesis.partitionKey", null); + + // null partitionKeyColumnName and non-null timestampColumnName is valid + new KinesisInputFormat(valueFormat, null, "kinesis.timestamp"); + + // non-null partitionKeyColumnName and non-null timestampColumnName is valid + new KinesisInputFormat(valueFormat, "kinesis.partitionKey", "kinesis.timestamp"); + + } + + @Test + @SuppressWarnings("ResultOfMethodCallIgnored") + public void testInvalidInputFormatConstruction() + { + // null value format is invalid + Assert.assertThrows( + "valueFormat must not be null", + NullPointerException.class, + () -> new KinesisInputFormat(null, null, null) + ); + + InputFormat valueFormat = new JsonInputFormat( + new JSONPathSpec( + true, + ImmutableList.of( + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz", "baz"), + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz2", "baz2"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg", "$.o.mg"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg2", "$.o.mg2"), + new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg", ".o.mg"), + new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg2", ".o.mg2") + ) + ), + null, + null, + false, + false + ); + + // partitionKeyColumnName == timestampColumnName is invalid + Assert.assertThrows( + "timestampColumnName and partitionKeyColumnName must be different", + IllegalStateException.class, + () -> new KinesisInputFormat(valueFormat, "kinesis.timestamp", "kinesis.timestamp") + ); + } + + private KinesisRecordEntity makeInputEntity( + byte[] payload, + long kinesisTimestampMillis) + { + return new KinesisRecordEntity( + new Record().withData(ByteBuffer.wrap(payload)) + .withApproximateArrivalTimestamp(new Date(kinesisTimestampMillis)) + .withPartitionKey(PARTITION_KEY) + ); + } + + private SettableByteEntity newSettableByteEntity(KinesisRecordEntity kinesisRecordEntity) + { + SettableByteEntity settableByteEntity = new SettableByteEntity<>(); + settableByteEntity.setEntity(kinesisRecordEntity); + return settableByteEntity; + } +} diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index 527a6738ffe..80bded2031d 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -19,6 +19,7 @@ package org.apache.druid.indexing.kinesis; +import com.amazonaws.services.kinesis.model.Record; import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; @@ -41,6 +42,7 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.FloatDimensionSchema; import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; +import org.apache.druid.data.input.kinesis.KinesisRecordEntity; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.report.IngestionStatsAndErrors; @@ -127,39 +129,39 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase private static final String SHARD_ID0 = "0"; private static final List RECORDS = Arrays.asList( - createRecord("1", "0", jb("2008", "a", "y", "10", "20.0", "1.0")), - createRecord("1", "1", jb("2009", "b", "y", "10", "20.0", "1.0")), - createRecord("1", "2", jb("2010", "c", "y", "10", "20.0", "1.0")), - createRecord("1", "3", jb("2011", "d", "y", "10", "20.0", "1.0")), - createRecord("1", "4", jb("2011", "e", "y", "10", "20.0", "1.0")), - createRecord("1", "5", jb("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")), - createRecord("1", "6", new ByteEntity(StringUtils.toUtf8("unparseable"))), - createRecord("1", "7", new ByteEntity(StringUtils.toUtf8(""))), - createRecord("1", "8", new ByteEntity(StringUtils.toUtf8("{}"))), - createRecord("1", "9", jb("2013", "f", "y", "10", "20.0", "1.0")), - createRecord("1", "10", jb("2049", "f", "y", "notanumber", "20.0", "1.0")), - createRecord("1", "11", jb("2049", "f", "y", "10", "notanumber", "1.0")), - createRecord("1", "12", jb("2049", "f", "y", "10", "20.0", "notanumber")), - createRecord("0", "0", jb("2012", "g", "y", "10", "20.0", "1.0")), - createRecord("0", "1", jb("2011", "h", "y", "10", "20.0", "1.0")) + createRecord("1", "0", kjb("2008", "a", "y", "10", "20.0", "1.0")), + createRecord("1", "1", kjb("2009", "b", "y", "10", "20.0", "1.0")), + createRecord("1", "2", kjb("2010", "c", "y", "10", "20.0", "1.0")), + createRecord("1", "3", kjb("2011", "d", "y", "10", "20.0", "1.0")), + createRecord("1", "4", kjb("2011", "e", "y", "10", "20.0", "1.0")), + createRecord("1", "5", kjb("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")), + createRecord("1", "6", new KinesisRecordEntity(new Record().withData(new ByteEntity(StringUtils.toUtf8("unparseable")).getBuffer()))), + createRecord("1", "7", new KinesisRecordEntity(new Record().withData(new ByteEntity(StringUtils.toUtf8("")).getBuffer()))), + createRecord("1", "8", new KinesisRecordEntity(new Record().withData(new ByteEntity(StringUtils.toUtf8("{}")).getBuffer()))), + createRecord("1", "9", kjb("2013", "f", "y", "10", "20.0", "1.0")), + createRecord("1", "10", kjb("2049", "f", "y", "notanumber", "20.0", "1.0")), + createRecord("1", "11", kjb("2049", "f", "y", "10", "notanumber", "1.0")), + createRecord("1", "12", kjb("2049", "f", "y", "10", "20.0", "notanumber")), + createRecord("0", "0", kjb("2012", "g", "y", "10", "20.0", "1.0")), + createRecord("0", "1", kjb("2011", "h", "y", "10", "20.0", "1.0")) ); private static final List SINGLE_PARTITION_RECORDS = Arrays.asList( - createRecord("1", "0", jb("2008", "a", "y", "10", "20.0", "1.0")), - createRecord("1", "1", jb("2009", "b", "y", "10", "20.0", "1.0")), - createRecord("1", "2", jb("2010", "c", "y", "10", "20.0", "1.0")), - createRecord("1", "3", jb("2011", "d", "y", "10", "20.0", "1.0")), - createRecord("1", "4", jb("2011", "e", "y", "10", "20.0", "1.0")), - createRecord("1", "5", jb("2012", "a", "y", "10", "20.0", "1.0")), - createRecord("1", "6", jb("2013", "b", "y", "10", "20.0", "1.0")), - createRecord("1", "7", jb("2010", "c", "y", "10", "20.0", "1.0")), - createRecord("1", "8", jb("2011", "d", "y", "10", "20.0", "1.0")), - createRecord("1", "9", jb("2011", "e", "y", "10", "20.0", "1.0")), - createRecord("1", "10", jb("2008", "a", "y", "10", "20.0", "1.0")), - createRecord("1", "11", jb("2009", "b", "y", "10", "20.0", "1.0")), - createRecord("1", "12", jb("2010", "c", "y", "10", "20.0", "1.0")), - createRecord("1", "13", jb("2012", "d", "y", "10", "20.0", "1.0")), - createRecord("1", "14", jb("2013", "e", "y", "10", "20.0", "1.0")) + createRecord("1", "0", kjb("2008", "a", "y", "10", "20.0", "1.0")), + createRecord("1", "1", kjb("2009", "b", "y", "10", "20.0", "1.0")), + createRecord("1", "2", kjb("2010", "c", "y", "10", "20.0", "1.0")), + createRecord("1", "3", kjb("2011", "d", "y", "10", "20.0", "1.0")), + createRecord("1", "4", kjb("2011", "e", "y", "10", "20.0", "1.0")), + createRecord("1", "5", kjb("2012", "a", "y", "10", "20.0", "1.0")), + createRecord("1", "6", kjb("2013", "b", "y", "10", "20.0", "1.0")), + createRecord("1", "7", kjb("2010", "c", "y", "10", "20.0", "1.0")), + createRecord("1", "8", kjb("2011", "d", "y", "10", "20.0", "1.0")), + createRecord("1", "9", kjb("2011", "e", "y", "10", "20.0", "1.0")), + createRecord("1", "10", kjb("2008", "a", "y", "10", "20.0", "1.0")), + createRecord("1", "11", kjb("2009", "b", "y", "10", "20.0", "1.0")), + createRecord("1", "12", kjb("2010", "c", "y", "10", "20.0", "1.0")), + createRecord("1", "13", kjb("2012", "d", "y", "10", "20.0", "1.0")), + createRecord("1", "14", kjb("2013", "e", "y", "10", "20.0", "1.0")) ); private static KinesisRecordSupplier recordSupplier; @@ -272,12 +274,12 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase record.getPartitionId(), record.getSequenceNumber(), record.getData().stream() - .map(entity -> new ByteEntity(entity.getBuffer())) + .map(entity -> new KinesisRecordEntity(new Record().withData(entity.getBuffer()))) .collect(Collectors.toList()) ); } - private static List> clone( + private static List> clone( List records, int start, int end @@ -289,14 +291,14 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase /** * Records can only be read once, hence we must use fresh records every time. */ - private static List> clone( + private static List> clone( List records ) { return records.stream().map(KinesisIndexTaskTest::clone).collect(Collectors.toList()); } - private static KinesisRecord createRecord(String partitionId, String sequenceNumber, ByteEntity entity) + private static KinesisRecord createRecord(String partitionId, String sequenceNumber, KinesisRecordEntity entity) { return new KinesisRecord(STREAM, partitionId, sequenceNumber, Collections.singletonList(entity)); } @@ -1697,7 +1699,7 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase maxRowsPerSegment = 2; maxRecordsPerPoll = 1; maxBytesPerPoll = 1_000_000; - List> records = + List> records = clone(SINGLE_PARTITION_RECORDS); recordSupplier.assign(EasyMock.anyObject()); @@ -2148,7 +2150,7 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase recordSupplier.seek(EasyMock.anyObject(), EasyMock.anyString()); EasyMock.expectLastCall().anyTimes(); - List> eosRecord = ImmutableList.of( + List> eosRecord = ImmutableList.of( new OrderedPartitionableRecord<>(STREAM, SHARD_ID1, KinesisSequenceNumber.END_OF_SHARD_MARKER, null) ); @@ -2454,6 +2456,18 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase return task.getRunner().getStatus() == SeekableStreamIndexTaskRunner.Status.READING; } + private static KinesisRecordEntity kjb( + String timestamp, + String dim1, + String dim2, + String dimLong, + String dimFloat, + String met1 + ) + { + return new KinesisRecordEntity(new Record().withData(jb(timestamp, dim1, dim2, dimLong, dimFloat, met1).getBuffer())); + } + @JsonTypeName("index_kinesis") private static class TestableKinesisIndexTask extends KinesisIndexTask { @@ -2497,15 +2511,15 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase /** * Utility class to keep the test code more readable. */ - private static class KinesisRecord extends OrderedPartitionableRecord + private static class KinesisRecord extends OrderedPartitionableRecord { - private final List data; + private final List data; public KinesisRecord( String stream, String partitionId, String sequenceNumber, - List data + List data ) { super(stream, partitionId, sequenceNumber, data); @@ -2514,7 +2528,7 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase @Nonnull @Override - public List getData() + public List getData() { return data; } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java index 5fcf81139eb..7c59ad61ac0 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java @@ -39,6 +39,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import org.apache.druid.data.input.impl.ByteEntity; +import org.apache.druid.data.input.kinesis.KinesisRecordEntity; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.java.util.common.ISE; @@ -99,34 +100,26 @@ public class KinesisRecordSupplierTest extends EasyMockSupport new Record().withData(jb("2012", "g", "y", "10", "20.0", "1.0")).withSequenceNumber("8"), new Record().withData(jb("2011", "h", "y", "10", "20.0", "1.0")).withSequenceNumber("9") ); - private static final List> ALL_RECORDS = ImmutableList.>builder() - .addAll(SHARD0_RECORDS.stream() - .map(x -> new OrderedPartitionableRecord<>( - STREAM, - SHARD_ID0, - x.getSequenceNumber(), - Collections - .singletonList( - new ByteEntity( - x.getData())) - )) - .collect( - Collectors - .toList())) - .addAll(SHARD1_RECORDS.stream() - .map(x -> new OrderedPartitionableRecord<>( - STREAM, - SHARD_ID1, - x.getSequenceNumber(), - Collections - .singletonList( - new ByteEntity( - x.getData())) - )) - .collect( - Collectors - .toList())) - .build(); + private static final List> ALL_RECORDS = ImmutableList.>builder() + .addAll(SHARD0_RECORDS.stream() + .map(x -> new OrderedPartitionableRecord<>( + STREAM, + SHARD_ID0, + x.getSequenceNumber(), + Collections.singletonList(new KinesisRecordEntity(new Record().withData(new ByteEntity(x.getData()).getBuffer()))) + )) + .collect( + Collectors + .toList())) + .addAll(SHARD1_RECORDS.stream() + .map(x -> new OrderedPartitionableRecord<>( + STREAM, + SHARD_ID1, + x.getSequenceNumber(), + Collections.singletonList(new KinesisRecordEntity(new Record().withData(new ByteEntity(x.getData()).getBuffer()))) + )) + .collect(Collectors.toList())) + .build(); private static ByteBuffer jb(String timestamp, String dim1, String dim2, String dimLong, String dimFloat, String met1) @@ -316,7 +309,7 @@ public class KinesisRecordSupplierTest extends EasyMockSupport } // filter out EOS markers - private static List> cleanRecords(List> records) + private static List> cleanRecords(List> records) { return records.stream() .filter(x -> !x.getSequenceNumber() @@ -398,7 +391,7 @@ public class KinesisRecordSupplierTest extends EasyMockSupport Thread.sleep(100); } - List> polledRecords = cleanRecords(recordSupplier.poll( + List> polledRecords = cleanRecords(recordSupplier.poll( POLL_TIMEOUT_MILLIS)); verifyAll(); @@ -457,7 +450,7 @@ public class KinesisRecordSupplierTest extends EasyMockSupport } Assert.assertFalse(recordSupplier.isAnyFetchActive()); - List> polledRecords = cleanRecords(recordSupplier.poll( + List> polledRecords = cleanRecords(recordSupplier.poll( POLL_TIMEOUT_MILLIS)); verifyAll(); @@ -531,7 +524,7 @@ public class KinesisRecordSupplierTest extends EasyMockSupport Thread.sleep(100); } - List> polledRecords = cleanRecords(recordSupplier.poll( + List> polledRecords = cleanRecords(recordSupplier.poll( POLL_TIMEOUT_MILLIS)); verifyAll(); @@ -687,7 +680,7 @@ public class KinesisRecordSupplierTest extends EasyMockSupport Thread.sleep(100); } - OrderedPartitionableRecord firstRecord = recordSupplier.poll(POLL_TIMEOUT_MILLIS).get(0); + OrderedPartitionableRecord firstRecord = recordSupplier.poll(POLL_TIMEOUT_MILLIS).get(0); Assert.assertEquals( ALL_RECORDS.get(7), @@ -705,7 +698,7 @@ public class KinesisRecordSupplierTest extends EasyMockSupport } - OrderedPartitionableRecord record2 = recordSupplier.poll(POLL_TIMEOUT_MILLIS).get(0); + OrderedPartitionableRecord record2 = recordSupplier.poll(POLL_TIMEOUT_MILLIS).get(0); Assert.assertEquals(ALL_RECORDS.get(9), record2); // only one partition in this test. second results come from getRecordsResult0, which has SHARD0_LAG_MILLIS @@ -776,7 +769,7 @@ public class KinesisRecordSupplierTest extends EasyMockSupport Thread.sleep(100); } - List> polledRecords = cleanRecords(recordSupplier.poll( + List> polledRecords = cleanRecords(recordSupplier.poll( POLL_TIMEOUT_MILLIS)); verifyAll(); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java index b0ba730a350..63144c6a935 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java @@ -19,6 +19,7 @@ package org.apache.druid.indexing.kinesis; +import com.amazonaws.services.kinesis.model.Record; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -27,7 +28,6 @@ import org.apache.druid.client.indexing.SamplerResponse; import org.apache.druid.client.indexing.SamplerSpec; import org.apache.druid.common.aws.AWSCredentialsConfig; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.data.input.impl.ByteEntity; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.FloatDimensionSchema; import org.apache.druid.data.input.impl.InputRowParser; @@ -37,6 +37,7 @@ import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.data.input.kinesis.KinesisRecordEntity; import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorIOConfig; import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorSpec; import org.apache.druid.indexing.overlord.sampler.InputSourceSampler; @@ -63,6 +64,7 @@ import org.junit.Assert; import org.junit.Test; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Collections; import java.util.Iterator; @@ -99,7 +101,7 @@ public class KinesisSamplerSpecTest extends EasyMockSupport private final KinesisRecordSupplier recordSupplier = mock(KinesisRecordSupplier.class); - private static List> generateRecords(String stream) + private static List> generateRecords(String stream) { return ImmutableList.of( new OrderedPartitionableRecord<>(stream, "1", "0", jb("2008", "a", "y", "10", "20.0", "1.0")), @@ -115,9 +117,9 @@ public class KinesisSamplerSpecTest extends EasyMockSupport stream, "1", "6", - Collections.singletonList(new ByteEntity(StringUtils.toUtf8("unparseable"))) + Collections.singletonList(new KinesisRecordEntity(new Record().withData(ByteBuffer.wrap(StringUtils.toUtf8("unparseable"))))) ), - new OrderedPartitionableRecord<>(stream, "1", "8", Collections.singletonList(new ByteEntity(StringUtils.toUtf8("{}")))) + new OrderedPartitionableRecord<>(stream, "1", "8", Collections.singletonList(new KinesisRecordEntity(new Record().withData(ByteBuffer.wrap(StringUtils.toUtf8("{}")))))) ); } @@ -428,19 +430,19 @@ public class KinesisSamplerSpecTest extends EasyMockSupport Assert.assertFalse(it.hasNext()); } - private static List jb(String ts, String dim1, String dim2, String dimLong, String dimFloat, String met1) + private static List jb(String ts, String dim1, String dim2, String dimLong, String dimFloat, String met1) { try { - return Collections.singletonList(new ByteEntity(new ObjectMapper().writeValueAsBytes( + return Collections.singletonList(new KinesisRecordEntity(new Record().withData(ByteBuffer.wrap(new ObjectMapper().writeValueAsBytes( ImmutableMap.builder() - .put("timestamp", ts) - .put("dim1", dim1) - .put("dim2", dim2) - .put("dimLong", dimLong) - .put("dimFloat", dimFloat) - .put("met1", met1) - .build() - ))); + .put("timestamp", ts) + .put("dim1", dim1) + .put("dim2", dim2) + .put("dimLong", dimLong) + .put("dimFloat", dimFloat) + .put("met1", met1) + .build() + ))))); } catch (Exception e) { throw new RuntimeException(e); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index 9001f148e99..e6ed27c9cec 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -27,12 +27,12 @@ import com.google.common.collect.ImmutableSet; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.data.input.InputFormat; -import org.apache.druid.data.input.impl.ByteEntity; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.data.input.kinesis.KinesisRecordEntity; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskInfoProvider; @@ -5656,7 +5656,7 @@ public class KinesisSupervisorTest extends EasyMockSupport } @Override - protected RecordSupplier setupRecordSupplier() + protected RecordSupplier setupRecordSupplier() { return supervisorRecordSupplier; } From fe6772a1012d6b3ee8f685899577afbe53b3ca74 Mon Sep 17 00:00:00 2001 From: Abhishek Radhakrishnan Date: Fri, 2 Aug 2024 10:01:55 -0700 Subject: [PATCH 08/12] Rename test builder `MSQTester.setExpectedSegment` (#16837) * Rename setExpectedSegment to setExpectedSegments in MSQTestBase. * Add expected segments for max num segments test cases. --- .../apache/druid/msq/exec/MSQArraysTest.java | 6 +- .../druid/msq/exec/MSQComplexGroupByTest.java | 4 +- .../apache/druid/msq/exec/MSQExportTest.java | 14 ++--- .../apache/druid/msq/exec/MSQInsertTest.java | 59 +++++++++++-------- .../apache/druid/msq/exec/MSQReplaceTest.java | 51 ++++++++-------- .../apache/druid/msq/exec/MSQWindowTest.java | 14 ++--- .../apache/druid/msq/test/MSQTestBase.java | 2 +- 7 files changed, 81 insertions(+), 69 deletions(-) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQArraysTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQArraysTest.java index 3cf4bf12ed1..20a871b953c 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQArraysTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQArraysTest.java @@ -230,7 +230,7 @@ public class MSQArraysTest extends MSQTestBase .setQueryContext(adjustedContext) .setExpectedDataSource("foo") .setExpectedRowSignature(rowSignature) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0))) + .setExpectedSegments(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0))) .setExpectedResultRows( NullHandling.sqlCompatible() ? ImmutableList.of( @@ -278,7 +278,7 @@ public class MSQArraysTest extends MSQTestBase .setQueryContext(adjustedContext) .setExpectedDataSource("foo") .setExpectedRowSignature(rowSignature) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0))) + .setExpectedSegments(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0))) .setExpectedResultRows( ImmutableList.of( new Object[]{0L, null}, @@ -325,7 +325,7 @@ public class MSQArraysTest extends MSQTestBase .setExpectedDataSource("foo1") .setExpectedRowSignature(rowSignature) .setQueryContext(context) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) + .setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) .setExpectedResultRows(expectedRows) .verifyResults(); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQComplexGroupByTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQComplexGroupByTest.java index efddab577e7..a35b0ad0cb2 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQComplexGroupByTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQComplexGroupByTest.java @@ -129,7 +129,7 @@ public class MSQComplexGroupByTest extends MSQTestBase + " GROUP BY 1\n" + " PARTITIONED BY ALL") .setQueryContext(context) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) + .setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) .setExpectedDataSource("foo1") .setExpectedRowSignature(RowSignature.builder() .add("__time", ColumnType.LONG) @@ -257,7 +257,7 @@ public class MSQComplexGroupByTest extends MSQTestBase + " GROUP BY 1\n" + " PARTITIONED BY ALL") .setQueryContext(adjustedContext) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) + .setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) .setExpectedDataSource("foo1") .setExpectedRowSignature(RowSignature.builder() .add("__time", ColumnType.LONG) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQExportTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQExportTest.java index 538cd471420..dde587c0c04 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQExportTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQExportTest.java @@ -57,7 +57,7 @@ public class MSQExportTest extends MSQTestBase .setExpectedDataSource("foo1") .setQueryContext(DEFAULT_MSQ_CONTEXT) .setExpectedRowSignature(rowSignature) - .setExpectedSegment(ImmutableSet.of()) + .setExpectedSegments(ImmutableSet.of()) .setExpectedResultRows(ImmutableList.of()) .verifyResults(); @@ -88,7 +88,7 @@ public class MSQExportTest extends MSQTestBase .setExpectedDataSource("foo1") .setQueryContext(DEFAULT_MSQ_CONTEXT) .setExpectedRowSignature(rowSignature) - .setExpectedSegment(ImmutableSet.of()) + .setExpectedSegments(ImmutableSet.of()) .setExpectedResultRows(ImmutableList.of()) .verifyResults(); @@ -127,7 +127,7 @@ public class MSQExportTest extends MSQTestBase .setExpectedDataSource("foo1") .setQueryContext(queryContext) .setExpectedRowSignature(rowSignature) - .setExpectedSegment(ImmutableSet.of()) + .setExpectedSegments(ImmutableSet.of()) .setExpectedResultRows(ImmutableList.of()) .verifyResults(); @@ -168,7 +168,7 @@ public class MSQExportTest extends MSQTestBase .setExpectedDataSource("foo1") .setQueryContext(DEFAULT_MSQ_CONTEXT) .setExpectedRowSignature(rowSignature) - .setExpectedSegment(ImmutableSet.of()) + .setExpectedSegments(ImmutableSet.of()) .setExpectedResultRows(ImmutableList.of()) .verifyResults(); @@ -219,7 +219,7 @@ public class MSQExportTest extends MSQTestBase .setExpectedDataSource("foo1") .setQueryContext(DEFAULT_MSQ_CONTEXT) .setExpectedRowSignature(rowSignature) - .setExpectedSegment(ImmutableSet.of()) + .setExpectedSegments(ImmutableSet.of()) .setExpectedResultRows(ImmutableList.of()) .verifyResults(); @@ -258,7 +258,7 @@ public class MSQExportTest extends MSQTestBase .setExpectedDataSource("foo1") .setQueryContext(DEFAULT_MSQ_CONTEXT) .setExpectedRowSignature(rowSignature) - .setExpectedSegment(ImmutableSet.of()) + .setExpectedSegments(ImmutableSet.of()) .setExpectedResultRows(ImmutableList.of()) .verifyResults(); @@ -335,7 +335,7 @@ public class MSQExportTest extends MSQTestBase .setExpectedDataSource("foo1") .setQueryContext(queryContext) .setExpectedRowSignature(rowSignature) - .setExpectedSegment(ImmutableSet.of()) + .setExpectedSegments(ImmutableSet.of()) .setExpectedResultRows(ImmutableList.of()) .verifyResults(); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java index 098b143b277..4449090fb69 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java @@ -113,7 +113,7 @@ public class MSQInsertTest extends MSQTestBase .setExpectedDataSource("foo1") .setQueryContext(context) .setExpectedRowSignature(rowSignature) - .setExpectedSegment(expectedFooSegments()) + .setExpectedSegments(expectedFooSegments()) .setExpectedResultRows(expectedRows) .setExpectedMSQSegmentReport( new MSQSegmentReport( @@ -310,7 +310,7 @@ public class MSQInsertTest extends MSQTestBase .setExpectedDataSource("foo1") .setExpectedRowSignature(rowSignature) .setQueryContext(context) - .setExpectedSegment(ImmutableSet.of(SegmentId.of( + .setExpectedSegments(ImmutableSet.of(SegmentId.of( "foo1", Intervals.of("2016-06-27/P1D"), "test", @@ -368,7 +368,7 @@ public class MSQInsertTest extends MSQTestBase .setExpectedDataSource("foo1") .setQueryContext(context) .setExpectedRowSignature(rowSignature) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) + .setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) .setExpectedResultRows(expectedRows) .setExpectedCountersForStageWorkerChannel( CounterSnapshotMatcher @@ -432,7 +432,7 @@ public class MSQInsertTest extends MSQTestBase .setExpectedDataSource("foo1") .setQueryContext(QueryContexts.override(context, ROLLUP_CONTEXT_PARAMS)) .setExpectedRowSignature(rowSignature) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) + .setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) .setExpectedResultRows(expectedRows) .setExpectedRollUp(true) .addExpectedAggregatorFactory(new LongSumAggregatorFactory("cnt", "cnt")) @@ -457,7 +457,7 @@ public class MSQInsertTest extends MSQTestBase .setExpectedDataSource("foo1") .setQueryContext(context) .setExpectedRowSignature(rowSignature) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) + .setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) .setExpectedResultRows(expectedRows) .setExpectedCountersForStageWorkerChannel( CounterSnapshotMatcher @@ -507,7 +507,7 @@ public class MSQInsertTest extends MSQTestBase .setExpectedDataSource("foo1") .setExpectedRowSignature(rowSignature) .setQueryContext(context) - .setExpectedSegment(expectedFooSegments()) + .setExpectedSegments(expectedFooSegments()) .setExpectedResultRows(expectedFooRows()) .verifyResults(); @@ -530,7 +530,7 @@ public class MSQInsertTest extends MSQTestBase .setExpectedDataSource("foo1") .setExpectedRowSignature(rowSignature) .setQueryContext(context) - .setExpectedSegment( + .setExpectedSegments( ImmutableSet.of( SegmentId.of("foo1", Intervals.of("1970-01-01/P1D"), "test", 0) ) @@ -565,7 +565,7 @@ public class MSQInsertTest extends MSQTestBase .setExpectedDataSource("foo1") .setExpectedRowSignature(rowSignature) .setQueryContext(localContext) - .setExpectedSegment( + .setExpectedSegments( ImmutableSet.of( SegmentId.of("foo1", Intervals.of("1970-01-01/P1D"), "test", 0) ) @@ -597,7 +597,7 @@ public class MSQInsertTest extends MSQTestBase .setExpectedDataSource("foo1") .setExpectedRowSignature(rowSignature) .setQueryContext(context) - .setExpectedSegment( + .setExpectedSegments( ImmutableSet.of( SegmentId.of("foo1", Intervals.of("1999-12-31T/P1D"), "test", 0) ) @@ -637,7 +637,7 @@ public class MSQInsertTest extends MSQTestBase .setExpectedDataSource("foo1") .setExpectedRowSignature(rowSignature) .setQueryContext(context) - .setExpectedSegment(expectedFooSegments()) + .setExpectedSegments(expectedFooSegments()) .setExpectedResultRows(expectedRows) .setExpectedCountersForStageWorkerChannel( CounterSnapshotMatcher @@ -686,7 +686,7 @@ public class MSQInsertTest extends MSQTestBase .setExpectedDataSource("foo1") .setExpectedRowSignature(rowSignature) .setQueryContext(context) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) + .setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) .setExpectedResultRows(expectedMultiValueFooRows()) .verifyResults(); } @@ -704,7 +704,7 @@ public class MSQInsertTest extends MSQTestBase .setExpectedDataSource("foo1") .setExpectedRowSignature(rowSignature) .setQueryContext(context) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) + .setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) .setExpectedResultRows(expectedMultiValueFooRows()) .verifyResults(); } @@ -722,7 +722,7 @@ public class MSQInsertTest extends MSQTestBase .setExpectedDataSource("foo1") .setExpectedRowSignature(rowSignature) .setQueryContext(context) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) + .setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) .setExpectedResultRows(expectedMultiValueFooRows()) .verifyResults(); } @@ -740,7 +740,7 @@ public class MSQInsertTest extends MSQTestBase .setExpectedDataSource("foo1") .setExpectedRowSignature(rowSignature) .setQueryContext(context) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) + .setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) .setExpectedResultRows(expectedMultiValueFooRowsGroupBy()) .verifyResults(); } @@ -776,7 +776,7 @@ public class MSQInsertTest extends MSQTestBase .setExpectedDataSource("foo1") .setExpectedRowSignature(rowSignature) .setQueryContext(adjustedContext) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) + .setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) .setExpectedResultRows( NullHandling.replaceWithDefault() ? ImmutableList.of( @@ -812,7 +812,7 @@ public class MSQInsertTest extends MSQTestBase .setExpectedDataSource("foo1") .setExpectedRowSignature(rowSignature) .setQueryContext(adjustedContext) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) + .setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) .setExpectedResultRows( NullHandling.replaceWithDefault() ? ImmutableList.of( @@ -848,7 +848,7 @@ public class MSQInsertTest extends MSQTestBase .setExpectedDataSource("foo1") .setExpectedRowSignature(rowSignature) .setQueryContext(adjustedContext) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) + .setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) .setExpectedResultRows( NullHandling.replaceWithDefault() ? ImmutableList.of( @@ -911,7 +911,7 @@ public class MSQInsertTest extends MSQTestBase .setExpectedRollUp(true) .addExpectedAggregatorFactory(new LongSumAggregatorFactory("cnt", "cnt")) .setExpectedRowSignature(rowSignature) - .setExpectedSegment(expectedFooSegments()) + .setExpectedSegments(expectedFooSegments()) .setExpectedResultRows(expectedRows) .setExpectedCountersForStageWorkerChannel( CounterSnapshotMatcher @@ -969,7 +969,7 @@ public class MSQInsertTest extends MSQTestBase .setExpectedQueryGranularity(Granularities.DAY) .addExpectedAggregatorFactory(new LongSumAggregatorFactory("cnt", "cnt")) .setExpectedRowSignature(rowSignature) - .setExpectedSegment(expectedFooSegments()) + .setExpectedSegments(expectedFooSegments()) .setExpectedResultRows(expectedRows) .setExpectedCountersForStageWorkerChannel( CounterSnapshotMatcher @@ -1043,7 +1043,7 @@ public class MSQInsertTest extends MSQTestBase .setExpectedQueryGranularity(Granularities.DAY) .addExpectedAggregatorFactory(new HyperUniquesAggregatorFactory("cnt", "cnt", false, true)) .setExpectedRowSignature(rowSignature) - .setExpectedSegment(expectedFooSegments()) + .setExpectedSegments(expectedFooSegments()) .setExpectedResultRows(expectedFooRowsWithAggregatedComplexColumn()) .verifyResults(); @@ -1068,7 +1068,7 @@ public class MSQInsertTest extends MSQTestBase .setExpectedRollUp(true) .addExpectedAggregatorFactory(new HyperUniquesAggregatorFactory("cnt", "cnt", false, true)) .setExpectedRowSignature(rowSignature) - .setExpectedSegment(expectedFooSegments()) + .setExpectedSegments(expectedFooSegments()) .setExpectedResultRows(expectedFooRowsWithAggregatedComplexColumn()) .verifyResults(); @@ -1102,7 +1102,7 @@ public class MSQInsertTest extends MSQTestBase .setExpectedDataSource("foo1") .setExpectedRowSignature(rowSignature) .addExpectedAggregatorFactory(new LongSumAggregatorFactory("cnt", "cnt")) - .setExpectedSegment(ImmutableSet.of(SegmentId.of( + .setExpectedSegments(ImmutableSet.of(SegmentId.of( "foo1", Intervals.of("2016-06-27/P1D"), "test", @@ -1171,7 +1171,7 @@ public class MSQInsertTest extends MSQTestBase .setExpectedDataSource("foo1") .setExpectedRowSignature(rowSignature) .addExpectedAggregatorFactory(new LongSumAggregatorFactory("cnt", "cnt")) - .setExpectedSegment(ImmutableSet.of(SegmentId.of( + .setExpectedSegments(ImmutableSet.of(SegmentId.of( "foo1", Intervals.of("2016-06-27/P1D"), "test", @@ -1419,6 +1419,13 @@ public class MSQInsertTest extends MSQTestBase .setQueryContext(context) .setExpectedDataSource("foo") .setExpectedRowSignature(expectedRowSignature) + .setExpectedSegments( + ImmutableSet.of( + SegmentId.of("foo", Intervals.of("2023-01-01/P1D"), "test", 0), + SegmentId.of("foo", Intervals.of("2023-01-01/P1D"), "test", 1), + SegmentId.of("foo", Intervals.of("2023-02-01/P1D"), "test", 0) + ) + ) .setExpectedResultRows( ImmutableList.of( new Object[]{1672531200000L, "day1_1"}, @@ -1491,7 +1498,7 @@ public class MSQInsertTest extends MSQTestBase .setExpectedDataSource("foo1") .setQueryContext(queryContext) .setExpectedRowSignature(rowSignature) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0), SegmentId.of("foo1", Intervals.ETERNITY, "test", 1))) + .setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0), SegmentId.of("foo1", Intervals.ETERNITY, "test", 1))) .setExpectedResultRows(expectedRows) .setExpectedMSQSegmentReport( new MSQSegmentReport( @@ -1538,7 +1545,7 @@ public class MSQInsertTest extends MSQTestBase .setExpectedDataSource("foo1") .setQueryContext(localContext) .setExpectedRowSignature(rowSignature) - .setExpectedSegment(ImmutableSet.of(SegmentId.of( + .setExpectedSegments(ImmutableSet.of(SegmentId.of( "foo1", Intervals.of("2016-06-27/P1D"), "test", @@ -1590,7 +1597,7 @@ public class MSQInsertTest extends MSQTestBase .setExpectedDataSource("foo1") .setQueryContext(localContext) .setExpectedRowSignature(rowSignature) - .setExpectedSegment(ImmutableSet.of(SegmentId.of( + .setExpectedSegments(ImmutableSet.of(SegmentId.of( "foo1", Intervals.of("2016-06-27/P1D"), "test", diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java index 1e8dc474f6e..cbdddf4a46e 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java @@ -142,7 +142,7 @@ public class MSQReplaceTest extends MSQTestBase .setExpectedRowSignature(rowSignature) .setQueryContext(context) .setExpectedDestinationIntervals(Intervals.ONLY_ETERNITY) - .setExpectedSegment( + .setExpectedSegments( ImmutableSet.of( SegmentId.of("foo", Intervals.of("2000-01-01T/P1D"), "test", 0), SegmentId.of("foo", Intervals.of("2000-01-02T/P1D"), "test", 0), @@ -214,7 +214,7 @@ public class MSQReplaceTest extends MSQTestBase "2000-01-02T00:00:00.000Z/2000-01-03T00:00:00.000Z"))) .setExpectedRowSignature(rowSignature) .setQueryContext(context) - .setExpectedSegment(ImmutableSet.of(SegmentId.of( + .setExpectedSegments(ImmutableSet.of(SegmentId.of( "foo", Intervals.of("2000-01-02T/P1D"), "test", @@ -284,7 +284,7 @@ public class MSQReplaceTest extends MSQTestBase "Cannot use RangeShardSpec, RangedShardSpec only supports string CLUSTER BY keys. Using NumberedShardSpec instead." ) ) - .setExpectedSegment(ImmutableSet.of( + .setExpectedSegments(ImmutableSet.of( SegmentId.of( "foo1", Intervals.of("2016-06-27T00:00:00.000Z/2016-06-27T01:00:00.000Z"), @@ -374,7 +374,7 @@ public class MSQReplaceTest extends MSQTestBase .setQueryContext(context) .setExpectedDestinationIntervals(ImmutableList.of(Intervals.of( "2016-06-27T01:00:00.000Z/2016-06-27T02:00:00.000Z"))) - .setExpectedSegment(ImmutableSet.of(SegmentId.of( + .setExpectedSegments(ImmutableSet.of(SegmentId.of( "foo1", Intervals.of("2016-06-27T01:00:00.000Z/2016-06-27T02:00:00.000Z"), "test", @@ -453,7 +453,7 @@ public class MSQReplaceTest extends MSQTestBase .setExpectedRowSignature(rowSignature) .setQueryContext(context) .setExpectedDestinationIntervals(Intervals.ONLY_ETERNITY) - .setExpectedSegment(ImmutableSet.of(SegmentId.of( + .setExpectedSegments(ImmutableSet.of(SegmentId.of( "foo", Intervals.of("2000-01-01T/P1M"), "test", @@ -469,7 +469,7 @@ public class MSQReplaceTest extends MSQTestBase new Object[]{978480000000L, 6.0f} ) ) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0))) + .setExpectedSegments(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0))) .setExpectedCountersForStageWorkerChannel( CounterSnapshotMatcher .with().totalFiles(1), @@ -542,7 +542,7 @@ public class MSQReplaceTest extends MSQTestBase .setExpectedRowSignature(rowSignature) .setQueryContext(context) .setExpectedDestinationIntervals(Intervals.ONLY_ETERNITY) - .setExpectedSegment(ImmutableSet.of(SegmentId.of( + .setExpectedSegments(ImmutableSet.of(SegmentId.of( "foo", Intervals.of("2000-01-01T/P1M"), "test", @@ -558,7 +558,7 @@ public class MSQReplaceTest extends MSQTestBase new Object[]{978480000000L, 6.0f} ) ) - .setExpectedSegment(ImmutableSet.of( + .setExpectedSegments(ImmutableSet.of( SegmentId.of("foo", Intervals.of("2000-01-01T/P1M"), "test", 0), SegmentId.of("foo", Intervals.of("2001-01-01T/P1M"), "test", 0) ) @@ -628,7 +628,7 @@ public class MSQReplaceTest extends MSQTestBase .setExpectedRowSignature(rowSignature) .setQueryContext(context) .setExpectedDestinationIntervals(Collections.singletonList(Intervals.of("2000-01-01T/2000-03-01T"))) - .setExpectedSegment(ImmutableSet.of(SegmentId.of( + .setExpectedSegments(ImmutableSet.of(SegmentId.of( "foo", Intervals.of("2000-01-01T/P1M"), "test", @@ -640,7 +640,7 @@ public class MSQReplaceTest extends MSQTestBase new Object[]{946771200000L, 2.0f} ) ) - .setExpectedSegment(ImmutableSet.of(SegmentId.of( + .setExpectedSegments(ImmutableSet.of(SegmentId.of( "foo", Intervals.of("2000-01-01T/P1M"), "test", @@ -720,7 +720,7 @@ public class MSQReplaceTest extends MSQTestBase .setQueryContext(context) .setExpectedDestinationIntervals(Collections.singletonList(Intervals.of("2000-01-01T/2002-01-01T"))) .setExpectedTombstoneIntervals(ImmutableSet.of(Intervals.of("2001-01-01T/2001-02-01T"))) - .setExpectedSegment(ImmutableSet.of(SegmentId.of( + .setExpectedSegments(ImmutableSet.of(SegmentId.of( "foo", Intervals.of("2000-01-01T/P1M"), "test", @@ -816,6 +816,11 @@ public class MSQReplaceTest extends MSQTestBase .setQueryContext(context) .setExpectedDataSource("foo") .setExpectedRowSignature(expectedRowSignature) + .setExpectedSegments( + ImmutableSet.of( + SegmentId.of("foo", Intervals.ETERNITY, "test", 0) + ) + ) .setExpectedResultRows( ImmutableList.of( new Object[]{1672531200000L, "day1_1"}, @@ -893,7 +898,7 @@ public class MSQReplaceTest extends MSQTestBase .setExpectedRowSignature(rowSignature) .setQueryContext(context) .setExpectedDestinationIntervals(Collections.singletonList(Intervals.of("2000-01-01T/2000-03-01T"))) - .setExpectedSegment(ImmutableSet.of(SegmentId.of( + .setExpectedSegments(ImmutableSet.of(SegmentId.of( "foo", Intervals.of("2000-01-01T/P1M"), "test", @@ -950,7 +955,7 @@ public class MSQReplaceTest extends MSQTestBase .setQueryContext(queryContext) .setExpectedRowSignature(rowSignature) .setExpectedShardSpec(DimensionRangeShardSpec.class) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0), SegmentId.of("foo1", Intervals.ETERNITY, "test", 1))) + .setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0), SegmentId.of("foo1", Intervals.ETERNITY, "test", 1))) .setExpectedResultRows(expectedRows) .setExpectedMSQSegmentReport( new MSQSegmentReport( @@ -1009,7 +1014,7 @@ public class MSQReplaceTest extends MSQTestBase .setQueryContext(context) .setExpectedDestinationIntervals(Collections.singletonList(Intervals.of("2000-01-01T/2002-01-01T"))) .setExpectedTombstoneIntervals(ImmutableSet.of(Intervals.of("2001-01-01T/2001-02-01T"))) - .setExpectedSegment(ImmutableSet.of(SegmentId.of( + .setExpectedSegments(ImmutableSet.of(SegmentId.of( "foo", Intervals.of("2000-01-01T/P1M"), "test", @@ -1071,7 +1076,7 @@ public class MSQReplaceTest extends MSQTestBase Intervals.of("%s/%s", "2000-02-01", Intervals.ETERNITY.getEnd()) ) ) - .setExpectedSegment(ImmutableSet.of(SegmentId.of( + .setExpectedSegments(ImmutableSet.of(SegmentId.of( "foo", Intervals.of("2000-01-01T/P1M"), "test", @@ -1119,7 +1124,7 @@ public class MSQReplaceTest extends MSQTestBase ) .setExpectedRowSignature(rowSignature) .setQueryContext(context) - .setExpectedSegment(expectedFooSegments()) + .setExpectedSegments(expectedFooSegments()) .setExpectedResultRows(expectedFooRows()) .setExpectedLastCompactionState( expectedCompactionState( @@ -1161,7 +1166,7 @@ public class MSQReplaceTest extends MSQTestBase .setExpectedShardSpec(DimensionRangeShardSpec.class) .setExpectedRowSignature(rowSignature) .setQueryContext(context) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) + .setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) .setExpectedResultRows( ImmutableList.of( new Object[]{0L, NullHandling.sqlCompatible() ? "" : null, 1.0f, 1L}, @@ -1206,7 +1211,7 @@ public class MSQReplaceTest extends MSQTestBase .setExpectedRowSignature(rowSignature) .setQueryContext(context) .setExpectedDestinationIntervals(Intervals.ONLY_ETERNITY) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foobar", Intervals.ETERNITY, "test", 0))) + .setExpectedSegments(ImmutableSet.of(SegmentId.of("foobar", Intervals.ETERNITY, "test", 0))) .setExpectedResultRows( ImmutableList.of( new Object[]{946684800000L, 1.0f}, @@ -1248,7 +1253,7 @@ public class MSQReplaceTest extends MSQTestBase .setExpectedRowSignature(rowSignature) .setQueryContext(context) .setExpectedDestinationIntervals(Intervals.ONLY_ETERNITY) - .setExpectedSegment( + .setExpectedSegments( ImmutableSet.of( SegmentId.of( "foobar", @@ -1324,7 +1329,7 @@ public class MSQReplaceTest extends MSQTestBase .setExpectedRowSignature(rowSignature) .setQueryContext(context) .setExpectedDestinationIntervals(Intervals.ONLY_ETERNITY) - .setExpectedSegment(ImmutableSet.of(SegmentId.of( + .setExpectedSegments(ImmutableSet.of(SegmentId.of( "foo", Intervals.of("2000-01-01T/P1M"), "test", @@ -1348,7 +1353,7 @@ public class MSQReplaceTest extends MSQTestBase new Object[]{978480000000L, null} ) ) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0))) + .setExpectedSegments(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0))) .setExpectedCountersForStageWorkerChannel( CounterSnapshotMatcher .with().totalFiles(1), @@ -1399,7 +1404,7 @@ public class MSQReplaceTest extends MSQTestBase .setExpectedRowSignature(rowSignature) .setQueryContext(context) .setExpectedDestinationIntervals(Intervals.ONLY_ETERNITY) - .setExpectedSegment(ImmutableSet.of(SegmentId.of( + .setExpectedSegments(ImmutableSet.of(SegmentId.of( "foo", Intervals.of("2000-01-01T/P1M"), "test", @@ -1421,7 +1426,7 @@ public class MSQReplaceTest extends MSQTestBase new Object[]{978480000000L, 6.0f} ) ) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0))) + .setExpectedSegments(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0))) .setExpectedCountersForStageWorkerChannel( CounterSnapshotMatcher .with().totalFiles(1), diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java index 5cc84ac6ee6..a4ba4113084 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java @@ -1584,7 +1584,7 @@ public class MSQWindowTest extends MSQTestBase new Object[]{978480000000L, 6.0f, 6.0, null} ) ) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) + .setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) .verifyResults(); } @@ -1615,7 +1615,7 @@ public class MSQWindowTest extends MSQTestBase new Object[]{978480000000L, 6.0f, 6.0} ) ) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0))) + .setExpectedSegments(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0))) .verifyResults(); } @@ -1646,7 +1646,7 @@ public class MSQWindowTest extends MSQTestBase new Object[]{978480000000L, 6.0f, 21.0} ) ) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0))) + .setExpectedSegments(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0))) .verifyResults(); } @@ -1677,7 +1677,7 @@ public class MSQWindowTest extends MSQTestBase new Object[]{978480000000L, 6.0f, 21.0} ) ) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0))) + .setExpectedSegments(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0))) .verifyResults(); } @@ -1709,7 +1709,7 @@ public class MSQWindowTest extends MSQTestBase new Object[]{978480000000L, 6.0f, 21.0, 21.0} ) ) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0))) + .setExpectedSegments(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0))) .verifyResults(); } @@ -1741,7 +1741,7 @@ public class MSQWindowTest extends MSQTestBase new Object[]{978480000000L, 6.0f, 6.0, 6.0} ) ) - .setExpectedSegment( + .setExpectedSegments( ImmutableSet.of( SegmentId.of("foo1", Intervals.of("2000-01-01T/P1D"), "test", 0), SegmentId.of("foo1", Intervals.of("2000-01-02T/P1D"), "test", 0), @@ -2045,7 +2045,7 @@ public class MSQWindowTest extends MSQTestBase new Object[]{0L, 129L, "Albuquerque", 140L} ) ) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) + .setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) .verifyResults(); } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index 2136d96d6d1..5b944f9ae50 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -904,7 +904,7 @@ public class MSQTestBase extends BaseCalciteQueryTest return asBuilder(); } - public Builder setExpectedSegment(Set expectedSegments) + public Builder setExpectedSegments(Set expectedSegments) { Preconditions.checkArgument(expectedSegments != null, "Segments cannot be null"); this.expectedSegments = expectedSegments; From 9dc2569f2276ab01b7917ad968250bd5e20ddfb7 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Sat, 3 Aug 2024 00:44:21 -0700 Subject: [PATCH 09/12] Track and emit segment loading rate for HttpLoadQueuePeon on Coordinator (#16691) Design: The loading rate is computed as a moving average of at least the last 10 GiB of successful segment loads. To account for multiple loading threads on a server, we use the concept of a batch to track load times. A batch is a set of segments added by the coordinator to the load queue of a server in one go. Computation: batchDurationMillis = t(load queue becomes empty) - t(first load request in batch is sent to server) batchBytes = total bytes successfully loaded in batch avg loading rate in batch (kbps) = (8 * batchBytes) / batchDurationMillis overall avg loading rate (kbps) = (8 * sumOverWindow(batchBytes)) / sumOverWindow(batchDurationMillis) Changes: - Add `LoadingRateTracker` which computes a moving average load rate based on the last few GBs of successful segment loads. - Emit metric `segment/loading/rateKbps` from the Coordinator. In the future, we may also consider emitting this metric from the historicals themselves. - Add `expectedLoadTimeMillis` to response of API `/druid/coordinator/v1/loadQueue?simple` --- docs/operations/metrics.md | 3 +- .../duty/CollectSegmentAndServerStats.java | 1 + .../loading/CuratorLoadQueuePeon.java | 10 +- .../loading/HttpLoadQueuePeon.java | 59 +++-- .../coordinator/loading/LoadPeonCallback.java | 10 +- .../coordinator/loading/LoadQueuePeon.java | 2 + .../loading/LoadingRateTracker.java | 220 ++++++++++++++++++ .../coordinator/loading/SegmentHolder.java | 29 ++- .../stats/CoordinatorRunStats.java | 4 +- .../server/coordinator/stats/RowKey.java | 7 +- .../druid/server/coordinator/stats/Stats.java | 2 + .../server/http/CoordinatorResource.java | 19 +- .../CollectSegmentAndServerStatsTest.java | 1 + .../loading/HttpLoadQueuePeonTest.java | 118 +++++++--- .../loading/LoadingRateTrackerTest.java | 191 +++++++++++++++ .../loading/TestLoadQueuePeon.java | 6 + .../server/http/CoordinatorResourceTest.java | 26 +++ website/.spelling | 1 + 18 files changed, 626 insertions(+), 83 deletions(-) create mode 100644 server/src/main/java/org/apache/druid/server/coordinator/loading/LoadingRateTracker.java create mode 100644 server/src/test/java/org/apache/druid/server/coordinator/loading/LoadingRateTrackerTest.java diff --git a/docs/operations/metrics.md b/docs/operations/metrics.md index b5be94ba4ba..aa35b0681ec 100644 --- a/docs/operations/metrics.md +++ b/docs/operations/metrics.md @@ -311,7 +311,7 @@ See [Enabling metrics](../configuration/index.md#enabling-metrics) for more deta ## Coordination -These metrics are for the Druid Coordinator and are reset each time the Coordinator runs the coordination logic. +These metrics are emitted by the Druid Coordinator in every run of the corresponding coordinator duty. |Metric|Description|Dimensions|Normal value| |------|-----------|----------|------------| @@ -325,6 +325,7 @@ These metrics are for the Druid Coordinator and are reset each time the Coordina |`segment/dropSkipped/count`|Number of segments that could not be dropped from any server.|`dataSource`, `tier`, `description`|Varies| |`segment/loadQueue/size`|Size in bytes of segments to load.|`server`|Varies| |`segment/loadQueue/count`|Number of segments to load.|`server`|Varies| +|`segment/loading/rateKbps`|Current rate of segment loading on a server in kbps (1000 bits per second). The rate is calculated as a moving average over the last 10 GiB or more of successful segment loads on that server.|`server`|Varies| |`segment/dropQueue/count`|Number of segments to drop.|`server`|Varies| |`segment/loadQueue/assigned`|Number of segments assigned for load or drop to the load queue of a server.|`dataSource`, `server`|Varies| |`segment/loadQueue/success`|Number of segment assignments that completed successfully.|`dataSource`, `server`|Varies| diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CollectSegmentAndServerStats.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CollectSegmentAndServerStats.java index da2f1e1a04a..3056d1c2bd0 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CollectSegmentAndServerStats.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CollectSegmentAndServerStats.java @@ -70,6 +70,7 @@ public class CollectSegmentAndServerStats implements CoordinatorDuty stats.add(Stats.SegmentQueue.BYTES_TO_LOAD, rowKey, queuePeon.getSizeOfSegmentsToLoad()); stats.add(Stats.SegmentQueue.NUM_TO_LOAD, rowKey, queuePeon.getSegmentsToLoad().size()); stats.add(Stats.SegmentQueue.NUM_TO_DROP, rowKey, queuePeon.getSegmentsToDrop().size()); + stats.updateMax(Stats.SegmentQueue.LOAD_RATE_KBPS, rowKey, queuePeon.getLoadRateKbps()); queuePeon.getAndResetStats().forEachStat( (stat, key, statValue) -> diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/CuratorLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/CuratorLoadQueuePeon.java index 52b012a81d4..333d3b0e305 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/CuratorLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/CuratorLoadQueuePeon.java @@ -170,6 +170,12 @@ public class CuratorLoadQueuePeon implements LoadQueuePeon return queuedSize.get(); } + @Override + public long getLoadRateKbps() + { + return 0; + } + @Override public CoordinatorRunStats getAndResetStats() { @@ -179,7 +185,7 @@ public class CuratorLoadQueuePeon implements LoadQueuePeon @Override public void loadSegment(final DataSegment segment, SegmentAction action, @Nullable final LoadPeonCallback callback) { - SegmentHolder segmentHolder = new SegmentHolder(segment, action, callback); + SegmentHolder segmentHolder = new SegmentHolder(segment, action, Duration.ZERO, callback); final SegmentHolder existingHolder = segmentsToLoad.putIfAbsent(segment, segmentHolder); if (existingHolder != null) { existingHolder.addCallback(callback); @@ -193,7 +199,7 @@ public class CuratorLoadQueuePeon implements LoadQueuePeon @Override public void dropSegment(final DataSegment segment, @Nullable final LoadPeonCallback callback) { - SegmentHolder segmentHolder = new SegmentHolder(segment, SegmentAction.DROP, callback); + SegmentHolder segmentHolder = new SegmentHolder(segment, SegmentAction.DROP, Duration.ZERO, callback); final SegmentHolder existingHolder = segmentsToDrop.putIfAbsent(segment, segmentHolder); if (existingHolder != null) { existingHolder.addCallback(callback); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java index cb32f95516b..a72cb22e151 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java @@ -36,6 +36,7 @@ import org.apache.druid.server.coordination.DataSegmentChangeCallback; import org.apache.druid.server.coordination.DataSegmentChangeHandler; import org.apache.druid.server.coordination.DataSegmentChangeRequest; import org.apache.druid.server.coordination.DataSegmentChangeResponse; +import org.apache.druid.server.coordination.SegmentChangeRequestLoad; import org.apache.druid.server.coordination.SegmentChangeStatus; import org.apache.druid.server.coordinator.BytesAccumulatingResponseHandler; import org.apache.druid.server.coordinator.config.HttpLoadQueuePeonConfig; @@ -92,6 +93,7 @@ public class HttpLoadQueuePeon implements LoadQueuePeon private final ConcurrentMap segmentsToLoad = new ConcurrentHashMap<>(); private final ConcurrentMap segmentsToDrop = new ConcurrentHashMap<>(); private final Set segmentsMarkedToDrop = ConcurrentHashMap.newKeySet(); + private final LoadingRateTracker loadingRateTracker = new LoadingRateTracker(); /** * Segments currently in queue ordered by priority and interval. This includes @@ -169,11 +171,10 @@ public class HttpLoadQueuePeon implements LoadQueuePeon synchronized (lock) { final Iterator queuedSegmentIterator = queuedSegments.iterator(); - final long currentTimeMillis = System.currentTimeMillis(); while (newRequests.size() < batchSize && queuedSegmentIterator.hasNext()) { final SegmentHolder holder = queuedSegmentIterator.next(); final DataSegment segment = holder.getSegment(); - if (hasRequestTimedOut(holder, currentTimeMillis)) { + if (holder.hasRequestTimedOut()) { onRequestFailed(holder, "timed out"); queuedSegmentIterator.remove(); if (holder.isLoad()) { @@ -188,9 +189,13 @@ public class HttpLoadQueuePeon implements LoadQueuePeon activeRequestSegments.add(segment); } } + + if (segmentsToLoad.isEmpty()) { + loadingRateTracker.markBatchLoadingFinished(); + } } - if (newRequests.size() == 0) { + if (newRequests.isEmpty()) { log.trace( "[%s]Found no load/drop requests. SegmentsToLoad[%d], SegmentsToDrop[%d], batchSize[%d].", serverId, segmentsToLoad.size(), segmentsToDrop.size(), config.getBatchSize() @@ -201,6 +206,11 @@ public class HttpLoadQueuePeon implements LoadQueuePeon try { log.trace("Sending [%d] load/drop requests to Server[%s].", newRequests.size(), serverId); + final boolean hasLoadRequests = newRequests.stream().anyMatch(r -> r instanceof SegmentChangeRequestLoad); + if (hasLoadRequests && !loadingRateTracker.isLoadingBatch()) { + loadingRateTracker.markBatchLoadingStarted(); + } + BytesAccumulatingResponseHandler responseHandler = new BytesAccumulatingResponseHandler(); ListenableFuture future = httpClient.go( new Request(HttpMethod.POST, changeRequestURL) @@ -234,9 +244,16 @@ public class HttpLoadQueuePeon implements LoadQueuePeon return; } + int numSuccessfulLoads = 0; + long successfulLoadSize = 0; for (DataSegmentChangeResponse e : statuses) { switch (e.getStatus().getState()) { case SUCCESS: + if (e.getRequest() instanceof SegmentChangeRequestLoad) { + ++numSuccessfulLoads; + successfulLoadSize += + ((SegmentChangeRequestLoad) e.getRequest()).getSegment().getSize(); + } case FAILED: handleResponseStatus(e.getRequest(), e.getStatus()); break; @@ -248,6 +265,10 @@ public class HttpLoadQueuePeon implements LoadQueuePeon log.error("Server[%s] returned unknown state in status[%s].", serverId, e.getStatus()); } } + + if (numSuccessfulLoads > 0) { + loadingRateTracker.incrementBytesLoadedInBatch(successfulLoadSize); + } } } catch (Exception ex) { @@ -284,9 +305,7 @@ public class HttpLoadQueuePeon implements LoadQueuePeon log.error( t, "Request[%s] Failed with status[%s]. Reason[%s].", - changeRequestURL, - responseHandler.getStatus(), - responseHandler.getDescription() + changeRequestURL, responseHandler.getStatus(), responseHandler.getDescription() ); } }, @@ -367,7 +386,7 @@ public class HttpLoadQueuePeon implements LoadQueuePeon if (stopped) { return; } - log.info("Stopping load queue peon for server [%s].", serverId); + log.info("Stopping load queue peon for server[%s].", serverId); stopped = true; // Cancel all queued requests @@ -379,6 +398,7 @@ public class HttpLoadQueuePeon implements LoadQueuePeon queuedSegments.clear(); activeRequestSegments.clear(); queuedSize.set(0L); + loadingRateTracker.stop(); stats.get().clear(); } } @@ -387,7 +407,7 @@ public class HttpLoadQueuePeon implements LoadQueuePeon public void loadSegment(DataSegment segment, SegmentAction action, LoadPeonCallback callback) { if (!action.isLoad()) { - log.warn("Invalid load action [%s] for segment [%s] on server [%s].", action, segment.getId(), serverId); + log.warn("Invalid load action[%s] for segment[%s] on server[%s].", action, segment.getId(), serverId); return; } @@ -407,7 +427,7 @@ public class HttpLoadQueuePeon implements LoadQueuePeon if (holder == null) { log.trace("Server[%s] to load segment[%s] queued.", serverId, segment.getId()); queuedSize.addAndGet(segment.getSize()); - holder = new SegmentHolder(segment, action, callback); + holder = new SegmentHolder(segment, action, config.getLoadTimeout(), callback); segmentsToLoad.put(segment, holder); queuedSegments.add(holder); processingExecutor.execute(this::doSegmentManagement); @@ -436,7 +456,7 @@ public class HttpLoadQueuePeon implements LoadQueuePeon if (holder == null) { log.trace("Server[%s] to drop segment[%s] queued.", serverId, segment.getId()); - holder = new SegmentHolder(segment, SegmentAction.DROP, callback); + holder = new SegmentHolder(segment, SegmentAction.DROP, config.getLoadTimeout(), callback); segmentsToDrop.put(segment, holder); queuedSegments.add(holder); processingExecutor.execute(this::doSegmentManagement); @@ -481,6 +501,12 @@ public class HttpLoadQueuePeon implements LoadQueuePeon return queuedSize.get(); } + @Override + public long getLoadRateKbps() + { + return loadingRateTracker.getMovingAverageLoadRateKbps(); + } + @Override public CoordinatorRunStats getAndResetStats() { @@ -505,19 +531,6 @@ public class HttpLoadQueuePeon implements LoadQueuePeon return Collections.unmodifiableSet(segmentsMarkedToDrop); } - /** - * A request is considered to have timed out if the time elapsed since it was - * first sent to the server is greater than the configured load timeout. - * - * @see HttpLoadQueuePeonConfig#getLoadTimeout() - */ - private boolean hasRequestTimedOut(SegmentHolder holder, long currentTimeMillis) - { - return holder.isRequestSentToServer() - && currentTimeMillis - holder.getFirstRequestMillis() - > config.getLoadTimeout().getMillis(); - } - private void onRequestFailed(SegmentHolder holder, String failureCause) { log.error( diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadPeonCallback.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadPeonCallback.java index 2a2163563db..320062514b0 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadPeonCallback.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadPeonCallback.java @@ -20,15 +20,11 @@ package org.apache.druid.server.coordinator.loading; /** + * Callback executed when the load or drop of a segment completes on a server + * either with success or failure. */ +@FunctionalInterface public interface LoadPeonCallback { - /** - * Ideally, this method is called after the load/drop opertion is successfully done, i.e., the historical node - * removes the zookeeper node from loadQueue and announces/unannouces the segment. However, this method will - * also be called in failure scenarios so for implementations of LoadPeonCallback that care about success it - * is important to take extra measures to ensure that whatever side effects they expect to happen upon success - * have happened. Coordinator will have a complete and correct view of the cluster in the next run period. - */ void execute(boolean success); } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadQueuePeon.java index 49e5f9a7c08..8e9989717ab 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadQueuePeon.java @@ -54,6 +54,8 @@ public interface LoadQueuePeon long getSizeOfSegmentsToLoad(); + long getLoadRateKbps(); + CoordinatorRunStats getAndResetStats(); /** diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadingRateTracker.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadingRateTracker.java new file mode 100644 index 00000000000..218a41df514 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadingRateTracker.java @@ -0,0 +1,220 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.coordinator.loading; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.EvictingQueue; +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.Stopwatch; + +import javax.annotation.concurrent.NotThreadSafe; +import java.util.concurrent.atomic.AtomicReference; + +/** + * Tracks the current segment loading rate for a single server. + *

+ * The loading rate is computed as a moving average of the last + * {@link #MOVING_AVERAGE_WINDOW_SIZE} segment batches (or more if any batch was + * smaller than {@link #MIN_ENTRY_SIZE_BYTES}). A batch is defined as a set of + * segments added to the load queue together. Usage: + *

    + *
  • Call {@link #markBatchLoadingStarted()} exactly once to indicate start of + * a batch.
  • + *
  • Call {@link #incrementBytesLoadedInBatch(long)} any number of times to + * increment successful loads done in the batch.
  • + *
  • Call {@link #markBatchLoadingFinished()} exactly once to complete the batch.
  • + *
+ * + *
+ *   batchDurationMillis
+ *   = t(load queue becomes empty) - t(first load request in batch is sent to server)
+ *
+ *   batchBytes = total bytes successfully loaded in batch
+ *
+ *   avg loading rate in batch (kbps) = (8 * batchBytes) / batchDurationMillis
+ *
+ *   overall avg loading rate (kbps)
+ *   = (8 * sumOverWindow(batchBytes)) / sumOverWindow(batchDurationMillis)
+ * 
+ *

+ * This class is currently not required to be thread-safe as the caller + * {@link HttpLoadQueuePeon} itself ensures that the write methods of this class + * are only accessed by one thread at a time. + */ +@NotThreadSafe +public class LoadingRateTracker +{ + public static final int MOVING_AVERAGE_WINDOW_SIZE = 10; + + /** + * Minimum size of a single entry in the moving average window = 1 GiB. + */ + public static final long MIN_ENTRY_SIZE_BYTES = 1 << 30; + + private final EvictingQueue window = EvictingQueue.create(MOVING_AVERAGE_WINDOW_SIZE); + + /** + * Total stats for the whole window. This includes the total from the current + * batch as well. + *

+ * Maintained as an atomic reference to ensure computational correctness in + * {@link #getMovingAverageLoadRateKbps()}. Otherwise, it is possible to have + * a state where bytes have been updated for the entry but not time taken + * (or vice versa). + */ + private final AtomicReference windowTotal = new AtomicReference<>(); + + private Entry currentBatchTotal; + private Entry currentTail; + + private final Stopwatch currentBatchDuration = Stopwatch.createUnstarted(); + + /** + * Marks the start of loading of a batch of segments. This should be called when + * the first request in a batch is sent to the server. + */ + public void markBatchLoadingStarted() + { + if (isLoadingBatch()) { + // Do nothing + return; + } + + currentBatchDuration.restart(); + currentBatchTotal = new Entry(); + + // Add a fresh entry at the tail for this batch + final Entry evictedHead = addNewEntryIfTailIsFull(); + if (evictedHead != null) { + final Entry delta = new Entry(); + delta.bytes -= evictedHead.bytes; + delta.millisElapsed -= evictedHead.millisElapsed; + + windowTotal.updateAndGet(delta::incrementBy); + } + } + + /** + * @return if a batch of segments is currently being loaded. + */ + public boolean isLoadingBatch() + { + return currentBatchDuration.isRunning(); + } + + /** + * Adds the given number of bytes to the total data successfully loaded in the + * current batch. This causes an update of the current load rate. + * + * @throws DruidException if called without making a prior call to + * {@link #markBatchLoadingStarted()}. + */ + public void incrementBytesLoadedInBatch(long loadedBytes) + { + incrementBytesLoadedInBatch(loadedBytes, currentBatchDuration.millisElapsed()); + } + + @VisibleForTesting + void incrementBytesLoadedInBatch(final long bytes, final long batchDurationMillis) + { + if (!isLoadingBatch()) { + throw DruidException.defensive("markBatchLoadingStarted() must be called before tracking load progress."); + } + + final Entry delta = new Entry(); + delta.bytes = bytes; + delta.millisElapsed = batchDurationMillis - currentBatchTotal.millisElapsed; + + currentTail.incrementBy(delta); + currentBatchTotal.incrementBy(delta); + windowTotal.updateAndGet(delta::incrementBy); + } + + /** + * Marks the end of loading of a batch of segments. This method should be called + * when all the requests in the batch have been processed by the server. + */ + public void markBatchLoadingFinished() + { + if (isLoadingBatch()) { + currentBatchDuration.reset(); + currentBatchTotal = null; + } + } + + /** + * Stops this rate tracker and resets its current state. + */ + public void stop() + { + window.clear(); + windowTotal.set(null); + currentTail = null; + currentBatchTotal = null; + currentBatchDuration.reset(); + } + + /** + * Moving average load rate in kbps (1000 bits per second). + */ + public long getMovingAverageLoadRateKbps() + { + final Entry overallTotal = windowTotal.get(); + if (overallTotal == null || overallTotal.millisElapsed <= 0) { + return 0; + } else { + return (8 * overallTotal.bytes) / overallTotal.millisElapsed; + } + } + + /** + * Adds a fresh entry to the queue if the current tail entry is already full. + * + * @return Old head of the queue if it was evicted, null otherwise. + */ + private Entry addNewEntryIfTailIsFull() + { + final Entry oldHead = window.peek(); + + if (currentTail == null || currentTail.bytes >= MIN_ENTRY_SIZE_BYTES) { + currentTail = new Entry(); + window.add(currentTail); + } + + // Compare if the oldHead and the newHead are the same object (not equals) + final Entry newHead = window.peek(); + return newHead == oldHead ? null : oldHead; + } + + private static class Entry + { + long bytes; + long millisElapsed; + + Entry incrementBy(Entry delta) + { + if (delta != null) { + this.bytes += delta.bytes; + this.millisElapsed += delta.millisElapsed; + } + return this; + } + } +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentHolder.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentHolder.java index ce199d65430..9f4a181699d 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentHolder.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentHolder.java @@ -21,18 +21,20 @@ package org.apache.druid.server.coordinator.loading; import com.google.common.collect.ImmutableList; import com.google.common.collect.Ordering; +import org.apache.druid.java.util.common.Stopwatch; import org.apache.druid.server.coordination.DataSegmentChangeRequest; import org.apache.druid.server.coordination.SegmentChangeRequestDrop; import org.apache.druid.server.coordination.SegmentChangeRequestLoad; import org.apache.druid.server.coordinator.DruidCoordinator; +import org.apache.druid.server.coordinator.config.HttpLoadQueuePeonConfig; import org.apache.druid.timeline.DataSegment; +import org.joda.time.Duration; import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Comparator; import java.util.List; import java.util.Objects; -import java.util.concurrent.atomic.AtomicLong; /** * Represents a segment queued for a load or drop operation in a LoadQueuePeon. @@ -57,14 +59,17 @@ public class SegmentHolder implements Comparable private final DataSegmentChangeRequest changeRequest; private final SegmentAction action; + private final Duration requestTimeout; + // Guaranteed to store only non-null elements private final List callbacks = new ArrayList<>(); - private final AtomicLong firstRequestMillis = new AtomicLong(0); + private final Stopwatch sinceRequestSentToServer = Stopwatch.createUnstarted(); private int runsInQueue = 0; public SegmentHolder( DataSegment segment, SegmentAction action, + Duration requestTimeout, @Nullable LoadPeonCallback callback ) { @@ -76,6 +81,7 @@ public class SegmentHolder implements Comparable if (callback != null) { callbacks.add(callback); } + this.requestTimeout = requestTimeout; } public DataSegment getSegment() @@ -124,17 +130,20 @@ public class SegmentHolder implements Comparable public void markRequestSentToServer() { - firstRequestMillis.compareAndSet(0L, System.currentTimeMillis()); + if (!sinceRequestSentToServer.isRunning()) { + sinceRequestSentToServer.start(); + } } - public boolean isRequestSentToServer() + /** + * A request is considered to have timed out if the time elapsed since it was + * first sent to the server is greater than the configured load timeout. + * + * @see HttpLoadQueuePeonConfig#getLoadTimeout() + */ + public boolean hasRequestTimedOut() { - return firstRequestMillis.get() > 0; - } - - public long getFirstRequestMillis() - { - return firstRequestMillis.get(); + return sinceRequestSentToServer.millisElapsed() > requestTimeout.getMillis(); } public int incrementAndGetRunsInQueue() diff --git a/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorRunStats.java b/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorRunStats.java index 5412230c2cc..92b2acc052b 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorRunStats.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorRunStats.java @@ -87,7 +87,7 @@ public class CoordinatorRunStats public long get(CoordinatorStat stat) { - return get(stat, RowKey.EMPTY); + return get(stat, RowKey.empty()); } public long get(CoordinatorStat stat, RowKey rowKey) @@ -196,7 +196,7 @@ public class CoordinatorRunStats public void add(CoordinatorStat stat, long value) { - add(stat, RowKey.EMPTY, value); + add(stat, RowKey.empty(), value); } public void add(CoordinatorStat stat, RowKey rowKey, long value) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/stats/RowKey.java b/server/src/main/java/org/apache/druid/server/coordinator/stats/RowKey.java index b0ee0a2d1f7..874ac79b1d1 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/stats/RowKey.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/stats/RowKey.java @@ -29,7 +29,7 @@ import java.util.Objects; */ public class RowKey { - public static final RowKey EMPTY = new RowKey(Collections.emptyMap()); + private static final RowKey EMPTY = new RowKey(Collections.emptyMap()); private final Map values; private final int hashCode; @@ -52,6 +52,11 @@ public class RowKey return with(dimension, value).build(); } + public static RowKey empty() + { + return EMPTY; + } + public Map getValues() { return values; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java b/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java index e7b901e3ed9..10873d89426 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java @@ -71,6 +71,8 @@ public class Stats = CoordinatorStat.toDebugAndEmit("bytesToLoad", "segment/loadQueue/size"); public static final CoordinatorStat NUM_TO_DROP = CoordinatorStat.toDebugAndEmit("numToDrop", "segment/dropQueue/count"); + public static final CoordinatorStat LOAD_RATE_KBPS + = CoordinatorStat.toDebugAndEmit("loadRateKbps", "segment/loading/rateKbps"); public static final CoordinatorStat ASSIGNED_ACTIONS = CoordinatorStat.toDebugAndEmit("assignedActions", "segment/loadQueue/assigned"); diff --git a/server/src/main/java/org/apache/druid/server/http/CoordinatorResource.java b/server/src/main/java/org/apache/druid/server/http/CoordinatorResource.java index 7a8c3e904dc..bffe2fe99d2 100644 --- a/server/src/main/java/org/apache/druid/server/http/CoordinatorResource.java +++ b/server/src/main/java/org/apache/druid/server/http/CoordinatorResource.java @@ -111,14 +111,23 @@ public class CoordinatorResource return Response.ok( Maps.transformValues( coordinator.getLoadManagementPeons(), - input -> { - long loadSize = input.getSizeOfSegmentsToLoad(); - long dropSize = input.getSegmentsToDrop().stream().mapToLong(DataSegment::getSize).sum(); + peon -> { + long loadSize = peon.getSizeOfSegmentsToLoad(); + long dropSize = peon.getSegmentsToDrop().stream().mapToLong(DataSegment::getSize).sum(); + + // 1 kbps = 1/8 kB/s = 1/8 B/ms + long loadRateKbps = peon.getLoadRateKbps(); + long expectedLoadTimeMillis + = loadRateKbps > 0 && loadSize > 0 + ? (8 * loadSize) / loadRateKbps + : 0; + return new ImmutableMap.Builder<>() - .put("segmentsToLoad", input.getSegmentsToLoad().size()) - .put("segmentsToDrop", input.getSegmentsToDrop().size()) + .put("segmentsToLoad", peon.getSegmentsToLoad().size()) + .put("segmentsToDrop", peon.getSegmentsToDrop().size()) .put("segmentsToLoadSize", loadSize) .put("segmentsToDropSize", dropSize) + .put("expectedLoadTimeMillis", expectedLoadTimeMillis) .build(); } ) diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/CollectSegmentAndServerStatsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/CollectSegmentAndServerStatsTest.java index fcbeeebc726..bb6d0406c69 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/CollectSegmentAndServerStatsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/CollectSegmentAndServerStatsTest.java @@ -62,6 +62,7 @@ public class CollectSegmentAndServerStatsTest CoordinatorRunStats stats = params.getCoordinatorStats(); Assert.assertTrue(stats.hasStat(Stats.SegmentQueue.NUM_TO_LOAD)); Assert.assertTrue(stats.hasStat(Stats.SegmentQueue.NUM_TO_DROP)); + Assert.assertTrue(stats.hasStat(Stats.SegmentQueue.LOAD_RATE_KBPS)); } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java b/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java index 16251130e77..5928f47e0bb 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java @@ -60,9 +60,6 @@ import java.util.Set; import java.util.function.Consumer; import java.util.stream.Collectors; -/** - * - */ public class HttpLoadQueuePeonTest { private static final ObjectMapper MAPPER = TestHelper.makeJsonMapper(); @@ -75,26 +72,22 @@ public class HttpLoadQueuePeonTest private TestHttpClient httpClient; private HttpLoadQueuePeon httpLoadQueuePeon; - private BlockingExecutorService processingExecutor; - private BlockingExecutorService callbackExecutor; - - private final List processedSegments = new ArrayList<>(); @Before public void setUp() { httpClient = new TestHttpClient(); - processingExecutor = new BlockingExecutorService("HttpLoadQueuePeonTest-%s"); - callbackExecutor = new BlockingExecutorService("HttpLoadQueuePeonTest-cb"); - processedSegments.clear(); - httpLoadQueuePeon = new HttpLoadQueuePeon( "http://dummy:4000", MAPPER, httpClient, new HttpLoadQueuePeonConfig(null, null, 10), - new WrappingScheduledExecutorService("HttpLoadQueuePeonTest-%s", processingExecutor, true), - callbackExecutor + new WrappingScheduledExecutorService( + "HttpLoadQueuePeonTest-%s", + httpClient.processingExecutor, + true + ), + httpClient.callbackExecutor ); httpLoadQueuePeon.start(); } @@ -117,13 +110,12 @@ public class HttpLoadQueuePeonTest httpLoadQueuePeon .loadSegment(segments.get(3), SegmentAction.MOVE_TO, markSegmentProcessed(segments.get(3))); - // Send requests to server - processingExecutor.finishAllPendingTasks(); + httpClient.sendRequestToServerAndHandleResponse(); Assert.assertEquals(segments, httpClient.segmentsSentToServer); // Verify that all callbacks are executed - callbackExecutor.finishAllPendingTasks(); - Assert.assertEquals(segments, processedSegments); + httpClient.executeCallbacks(); + Assert.assertEquals(segments, httpClient.processedSegments); } @Test @@ -170,8 +162,7 @@ public class HttpLoadQueuePeonTest Collections.shuffle(actions); actions.forEach(QueueAction::invoke); - // Send one batch of requests to the server - processingExecutor.finishAllPendingTasks(); + httpClient.sendRequestToServerAndHandleResponse(); // Verify that all segments are sent to the server in the expected order Assert.assertEquals(segmentsDay1, httpClient.segmentsSentToServer); @@ -194,7 +185,7 @@ public class HttpLoadQueuePeonTest Collections.shuffle(segmentsDay2); // Assign segments to the actions in their order of priority - // Priority order: action (drop, priorityLoad, etc), then interval (new then old) + // Order: action (drop, priorityLoad, etc.), then interval (new then old) List actions = Arrays.asList( QueueAction.of(segmentsDay2.get(0), s -> httpLoadQueuePeon.dropSegment(s, null)), QueueAction.of(segmentsDay1.get(0), s -> httpLoadQueuePeon.dropSegment(s, null)), @@ -212,8 +203,7 @@ public class HttpLoadQueuePeonTest Collections.shuffle(actions); actions.forEach(QueueAction::invoke); - // Send one batch of requests to the server - processingExecutor.finishNextPendingTask(); + httpClient.sendRequestToServerAndHandleResponse(); // Verify that all segments are sent to the server in the expected order Assert.assertEquals(expectedSegmentOrder, httpClient.segmentsSentToServer); @@ -230,7 +220,7 @@ public class HttpLoadQueuePeonTest Assert.assertTrue(cancelled); Assert.assertEquals(0, httpLoadQueuePeon.getSegmentsToLoad().size()); - Assert.assertTrue(processedSegments.isEmpty()); + Assert.assertTrue(httpClient.processedSegments.isEmpty()); } @Test @@ -244,7 +234,7 @@ public class HttpLoadQueuePeonTest Assert.assertTrue(cancelled); Assert.assertTrue(httpLoadQueuePeon.getSegmentsToDrop().isEmpty()); - Assert.assertTrue(processedSegments.isEmpty()); + Assert.assertTrue(httpClient.processedSegments.isEmpty()); } @Test @@ -254,8 +244,7 @@ public class HttpLoadQueuePeonTest httpLoadQueuePeon.loadSegment(segment, SegmentAction.REPLICATE, markSegmentProcessed(segment)); Assert.assertTrue(httpLoadQueuePeon.getSegmentsToLoad().contains(segment)); - // Send the request to the server - processingExecutor.finishNextPendingTask(); + httpClient.sendRequestToServer(); Assert.assertTrue(httpClient.segmentsSentToServer.contains(segment)); // Segment is still in queue but operation cannot be cancelled @@ -263,8 +252,7 @@ public class HttpLoadQueuePeonTest boolean cancelled = httpLoadQueuePeon.cancelOperation(segment); Assert.assertFalse(cancelled); - // Handle response from server - processingExecutor.finishNextPendingTask(); + httpClient.handleResponseFromServer(); // Segment has been removed from queue Assert.assertTrue(httpLoadQueuePeon.getSegmentsToLoad().isEmpty()); @@ -272,8 +260,8 @@ public class HttpLoadQueuePeonTest Assert.assertFalse(cancelled); // Execute callbacks and verify segment is fully processed - callbackExecutor.finishAllPendingTasks(); - Assert.assertTrue(processedSegments.contains(segment)); + httpClient.executeCallbacks(); + Assert.assertTrue(httpClient.processedSegments.contains(segment)); } @Test @@ -287,14 +275,59 @@ public class HttpLoadQueuePeonTest Assert.assertFalse(httpLoadQueuePeon.cancelOperation(segment)); } + @Test + public void testLoadRateIsZeroWhenNoLoadHasFinishedYet() + { + httpLoadQueuePeon.loadSegment(segments.get(0), SegmentAction.LOAD, null); + httpClient.sendRequestToServer(); + Assert.assertEquals(1, httpLoadQueuePeon.getSegmentsToLoad().size()); + Assert.assertEquals(0, httpLoadQueuePeon.getLoadRateKbps()); + } + + @Test + public void testLoadRateIsUnchangedByDrops() throws InterruptedException + { + // Drop a segment after a small delay + final long millisTakenToDropSegment = 10; + httpLoadQueuePeon.dropSegment(segments.get(0), null); + httpClient.sendRequestToServer(); + Thread.sleep(millisTakenToDropSegment); + httpClient.handleResponseFromServer(); + + // Verify that load rate is still zero + Assert.assertEquals(0, httpLoadQueuePeon.getLoadRateKbps()); + } + + @Test + public void testLoadRateIsChangedWhenLoadSucceeds() throws InterruptedException + { + // Load a segment after a small delay + final long millisTakenToLoadSegment = 10; + httpLoadQueuePeon.loadSegment(segments.get(0), SegmentAction.LOAD, null); + httpClient.sendRequestToServer(); + Thread.sleep(millisTakenToLoadSegment); + httpClient.handleResponseFromServer(); + + // Verify that load rate has been updated + long expectedRateKbps = (8 * segments.get(0).getSize()) / millisTakenToLoadSegment; + long observedRateKbps = httpLoadQueuePeon.getLoadRateKbps(); + Assert.assertTrue( + observedRateKbps > expectedRateKbps / 2 + && observedRateKbps <= expectedRateKbps + ); + } + private LoadPeonCallback markSegmentProcessed(DataSegment segment) { - return success -> processedSegments.add(segment); + return success -> httpClient.processedSegments.add(segment); } private static class TestHttpClient implements HttpClient, DataSegmentChangeHandler { - private final List segmentsSentToServer = new ArrayList<>(); + final BlockingExecutorService processingExecutor = new BlockingExecutorService("HttpLoadQueuePeonTest-%s"); + final BlockingExecutorService callbackExecutor = new BlockingExecutorService("HttpLoadQueuePeonTest-cb"); + final List processedSegments = new ArrayList<>(); + final List segmentsSentToServer = new ArrayList<>(); @Override public ListenableFuture go( @@ -353,6 +386,27 @@ public class HttpLoadQueuePeonTest { segmentsSentToServer.add(segment); } + + void sendRequestToServerAndHandleResponse() + { + sendRequestToServer(); + handleResponseFromServer(); + } + + void sendRequestToServer() + { + processingExecutor.finishNextPendingTask(); + } + + void handleResponseFromServer() + { + processingExecutor.finishAllPendingTasks(); + } + + void executeCallbacks() + { + callbackExecutor.finishAllPendingTasks(); + } } /** diff --git a/server/src/test/java/org/apache/druid/server/coordinator/loading/LoadingRateTrackerTest.java b/server/src/test/java/org/apache/druid/server/coordinator/loading/LoadingRateTrackerTest.java new file mode 100644 index 00000000000..fafa1f3e0e3 --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/coordinator/loading/LoadingRateTrackerTest.java @@ -0,0 +1,191 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.coordinator.loading; + +import org.apache.druid.error.DruidException; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.Random; + +public class LoadingRateTrackerTest +{ + private LoadingRateTracker tracker; + + @Before + public void setup() + { + tracker = new LoadingRateTracker(); + } + + @Test + public void testUpdateThrowsExceptionIfBatchNotStarted() + { + DruidException e = Assert.assertThrows( + DruidException.class, + () -> tracker.incrementBytesLoadedInBatch(1000, 10) + ); + Assert.assertEquals( + "markBatchLoadingStarted() must be called before tracking load progress.", + e.getMessage() + ); + } + + @Test + public void testRateIsZeroWhenEmpty() + { + Assert.assertEquals(0, tracker.getMovingAverageLoadRateKbps()); + } + + @Test + public void testRateIsZeroAfterStop() + { + tracker.markBatchLoadingStarted(); + tracker.incrementBytesLoadedInBatch(1000, 10); + Assert.assertEquals(8 * 1000 / 10, tracker.getMovingAverageLoadRateKbps()); + + tracker.stop(); + Assert.assertEquals(0, tracker.getMovingAverageLoadRateKbps()); + } + + @Test + public void testRateAfter2UpdatesInBatch() + { + tracker.markBatchLoadingStarted(); + tracker.incrementBytesLoadedInBatch(1000, 10); + Assert.assertEquals(8 * 1000 / 10, tracker.getMovingAverageLoadRateKbps()); + + tracker.incrementBytesLoadedInBatch(1000, 15); + Assert.assertEquals(8 * 2000 / 15, tracker.getMovingAverageLoadRateKbps()); + } + + @Test + public void testRateAfter2Batches() + { + tracker.markBatchLoadingStarted(); + tracker.incrementBytesLoadedInBatch(1000, 10); + Assert.assertEquals(8 * 1000 / 10, tracker.getMovingAverageLoadRateKbps()); + tracker.markBatchLoadingFinished(); + + tracker.markBatchLoadingStarted(); + tracker.incrementBytesLoadedInBatch(1000, 5); + Assert.assertEquals(8 * 2000 / 15, tracker.getMovingAverageLoadRateKbps()); + tracker.markBatchLoadingFinished(); + } + + @Test + public void test100UpdatesInABatch() + { + final Random random = new Random(1001); + + tracker.markBatchLoadingStarted(); + + long totalUpdateBytes = 0; + long monoticBatchDuration = 0; + for (int i = 0; i < 100; ++i) { + long updateBytes = 1 + random.nextInt(1000); + monoticBatchDuration = 1 + random.nextInt(10); + + tracker.incrementBytesLoadedInBatch(updateBytes, monoticBatchDuration); + + totalUpdateBytes += updateBytes; + Assert.assertEquals(8 * totalUpdateBytes / monoticBatchDuration, tracker.getMovingAverageLoadRateKbps()); + } + + tracker.markBatchLoadingFinished(); + Assert.assertEquals(8 * totalUpdateBytes / monoticBatchDuration, tracker.getMovingAverageLoadRateKbps()); + } + + @Test + public void testRateIsMovingAverage() + { + final Random random = new Random(1001); + final int windowSize = LoadingRateTracker.MOVING_AVERAGE_WINDOW_SIZE; + final long minEntrySizeBytes = LoadingRateTracker.MIN_ENTRY_SIZE_BYTES; + + // Add batch updates to fill up the window size + long[] updateBytes = new long[windowSize]; + long[] updateMillis = new long[windowSize]; + + long totalBytes = 0; + long totalMillis = 0; + for (int i = 0; i < windowSize; ++i) { + updateBytes[i] = minEntrySizeBytes + random.nextInt((int) minEntrySizeBytes); + updateMillis[i] = 1 + random.nextInt(1000); + + totalBytes += updateBytes[i]; + totalMillis += updateMillis[i]; + + tracker.markBatchLoadingStarted(); + tracker.incrementBytesLoadedInBatch(updateBytes[i], updateMillis[i]); + Assert.assertEquals( + 8 * totalBytes / totalMillis, + tracker.getMovingAverageLoadRateKbps() + ); + tracker.markBatchLoadingFinished(); + } + + // Add another batch update + long latestUpdateBytes = 1; + long latestUpdateMillis = 1 + random.nextInt(1000); + tracker.markBatchLoadingStarted(); + tracker.incrementBytesLoadedInBatch(latestUpdateBytes, latestUpdateMillis); + tracker.markBatchLoadingFinished(); + + // Verify that the average window has moved + totalBytes = totalBytes - updateBytes[0] + latestUpdateBytes; + totalMillis = totalMillis - updateMillis[0] + latestUpdateMillis; + Assert.assertEquals( + 8 * totalBytes / totalMillis, + tracker.getMovingAverageLoadRateKbps() + ); + } + + @Test + public void testWindowMovesOnlyAfterMinSizeUpdates() + { + final Random random = new Random(1001); + + long totalBytes = 0; + long totalMillis = 0; + + final int windowSize = LoadingRateTracker.MOVING_AVERAGE_WINDOW_SIZE; + final long minEntrySizeBytes = LoadingRateTracker.MIN_ENTRY_SIZE_BYTES; + + for (int i = 0; i < windowSize * 10; ++i) { + long updateBytes = 1 + random.nextInt((int) minEntrySizeBytes / 100); + long updateMillis = 1 + random.nextInt(1000); + + totalBytes += updateBytes; + totalMillis += updateMillis; + + tracker.markBatchLoadingStarted(); + tracker.incrementBytesLoadedInBatch(updateBytes, updateMillis); + tracker.markBatchLoadingFinished(); + + // Verify that the average window doesn't move + Assert.assertEquals( + 8 * totalBytes / totalMillis, + tracker.getMovingAverageLoadRateKbps() + ); + } + } +} diff --git a/server/src/test/java/org/apache/druid/server/coordinator/loading/TestLoadQueuePeon.java b/server/src/test/java/org/apache/druid/server/coordinator/loading/TestLoadQueuePeon.java index c496c37e3ec..71ebc6fd8cf 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/loading/TestLoadQueuePeon.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/loading/TestLoadQueuePeon.java @@ -56,6 +56,12 @@ public class TestLoadQueuePeon implements LoadQueuePeon return 0; } + @Override + public long getLoadRateKbps() + { + return 0; + } + @Override public CoordinatorRunStats getAndResetStats() { diff --git a/server/src/test/java/org/apache/druid/server/http/CoordinatorResourceTest.java b/server/src/test/java/org/apache/druid/server/http/CoordinatorResourceTest.java index c25417a8d13..9d2b81f6aa1 100644 --- a/server/src/test/java/org/apache/druid/server/http/CoordinatorResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/CoordinatorResourceTest.java @@ -21,6 +21,7 @@ package org.apache.druid.server.http; import com.google.common.collect.ImmutableMap; import org.apache.druid.server.coordinator.DruidCoordinator; +import org.apache.druid.server.coordinator.loading.TestLoadQueuePeon; import org.easymock.EasyMock; import org.junit.After; import org.junit.Assert; @@ -73,4 +74,29 @@ public class CoordinatorResourceTest Assert.assertEquals(ImmutableMap.of("leader", false), response2.getEntity()); Assert.assertEquals(404, response2.getStatus()); } + + @Test + public void testGetLoadStatusSimple() + { + EasyMock.expect(mock.getLoadManagementPeons()) + .andReturn(ImmutableMap.of("hist1", new TestLoadQueuePeon())) + .once(); + EasyMock.replay(mock); + + final Response response = new CoordinatorResource(mock).getLoadQueue("true", null); + Assert.assertEquals( + ImmutableMap.of( + "hist1", + ImmutableMap.of( + "segmentsToDrop", 0, + "segmentsToLoad", 0, + "segmentsToLoadSize", 0L, + "segmentsToDropSize", 0L, + "expectedLoadTimeMillis", 0L + ) + ), + response.getEntity() + ); + Assert.assertEquals(200, response.getStatus()); + } } diff --git a/website/.spelling b/website/.spelling index a80900b3edb..1ac39471571 100644 --- a/website/.spelling +++ b/website/.spelling @@ -382,6 +382,7 @@ json_query json_query_array json_value karlkfi +kbps kerberos keystore keytool From 31b43753fbf15caa395ad6f3b4c52e41332f5a82 Mon Sep 17 00:00:00 2001 From: Abhishek Radhakrishnan Date: Sat, 3 Aug 2024 10:23:44 -0700 Subject: [PATCH 10/12] Add `druid.indexing.formats.stringMultiValueHandlingMode` system config (#16822) This patch introduces an optional cluster configuration, druid.indexing.formats.stringMultiValueHandlingMode, allowing operators to override the default mode SORTED_SET for string dimensions. The possible values for the config are SORTED_SET, SORTED_ARRAY, or ARRAY (SORTED_SET is the default). Case insensitive values are allowed. While this cluster property allows users to manage the multi-value handling mode for string dimension types, it's recommended to migrate to using real array types instead of MVDs. This fixes a long-standing issue where compaction will honor the configured cluster wide property instead of rewriting it as the default SORTED_ARRAY always, even if the data was originally ingested with ARRAY or SORTED_SET. --- .../GroupByDeserializationBenchmark.java | 6 +-- .../frame/FrameChannelMergerBenchmark.java | 4 +- .../benchmark/query/SqlGroupByBenchmark.java | 6 +-- .../druid/msq/exec/MSQComplexGroupByTest.java | 6 +-- .../apache/druid/msq/test/MSQTestBase.java | 8 +-- .../data/input/impl/DimensionSchema.java | 4 +- ...ataModule.java => BuiltInTypesModule.java} | 50 +++++++++++++---- .../segment/DefaultColumnFormatConfig.java | 47 ++++++++++++++-- .../druid/segment/StringDimensionIndexer.java | 2 +- .../nested/NestedDataComplexTypeSerde.java | 4 +- .../druid/frame/write/FrameWriterTest.java | 4 +- ...eTest.java => BuiltInTypesModuleTest.java} | 54 +++++++++++++++++-- .../apache/druid/math/expr/FunctionTest.java | 4 +- .../druid/query/NestedDataTestUtils.java | 4 +- .../groupby/NestedDataGroupByQueryTest.java | 6 +-- .../groupby/NestedGroupByArrayQueryTest.java | 6 +-- ...lumnGroupByColumnSelectorStrategyTest.java | 4 +- .../query/scan/NestedDataScanQueryTest.java | 6 +-- .../NestedDataTimeseriesQueryTest.java | 4 +- .../query/topn/NestedDataTopNQueryTest.java | 6 +-- .../segment/AutoTypeColumnIndexerTest.java | 4 +- .../DefaultColumnFormatsConfigTest.java | 7 ++- .../NestedDataColumnIndexerV4Test.java | 4 +- .../segment/NestedDataColumnSchemaTest.java | 4 +- .../segment/data/IncrementalIndexTest.java | 4 +- .../ArrayContainsElementFilterTests.java | 4 +- .../druid/segment/filter/BaseFilterTest.java | 4 +- .../segment/filter/EqualityFilterTests.java | 4 +- .../segment/generator/SegmentGenerator.java | 4 +- .../IncrementalIndexIngestionTest.java | 4 +- .../IncrementalIndexMultiValueSpecTest.java | 4 +- .../IncrementalIndexStorageAdapterTest.java | 4 +- .../nested/NestedDataColumnSupplierTest.java | 4 +- .../NestedDataColumnSupplierV4Test.java | 4 +- .../NestedFieldColumnSelectorsTest.java | 6 +-- .../ScalarDoubleColumnSupplierTest.java | 4 +- .../nested/ScalarLongColumnSupplierTest.java | 4 +- .../ScalarStringColumnSupplierTest.java | 4 +- .../nested/VariantColumnSupplierTest.java | 4 +- .../initialization/CoreInjectorBuilder.java | 4 +- .../org/apache/druid/cli/DumpSegmentTest.java | 16 +++--- .../sql/calcite/CalciteArraysQueryTest.java | 4 +- .../calcite/CalciteNestedDataQueryTest.java | 6 +-- .../sql/calcite/util/SqlTestFramework.java | 3 +- 44 files changed, 232 insertions(+), 117 deletions(-) rename processing/src/main/java/org/apache/druid/guice/{NestedDataModule.java => BuiltInTypesModule.java} (68%) rename processing/src/test/java/org/apache/druid/guice/{NestedDataModuleTest.java => BuiltInTypesModuleTest.java} (65%) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByDeserializationBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByDeserializationBenchmark.java index c3ebbe5122f..ee1ec34dc3b 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByDeserializationBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByDeserializationBenchmark.java @@ -23,7 +23,7 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.jackson.AggregatorsModule; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Pair; @@ -68,7 +68,7 @@ public class GroupByDeserializationBenchmark static { NullHandling.initializeForTests(); - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); AggregatorsModule.registerComplexMetricsAndSerde(); } @@ -93,7 +93,7 @@ public class GroupByDeserializationBenchmark public void setup() throws JsonProcessingException { final ObjectMapper undecoratedMapper = TestHelper.makeJsonMapper(); - undecoratedMapper.registerModules(NestedDataModule.getJacksonModulesList()); + undecoratedMapper.registerModules(BuiltInTypesModule.getJacksonModulesList()); undecoratedMapper.registerModule(new AggregatorsModule()); final Pair sqlQueryAndResultRow = sqlQueryAndResultRow( numDimensions, diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/frame/FrameChannelMergerBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/frame/FrameChannelMergerBenchmark.java index a864bcd3419..a8b90e56eea 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/frame/FrameChannelMergerBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/frame/FrameChannelMergerBenchmark.java @@ -37,7 +37,7 @@ import org.apache.druid.frame.processor.FrameProcessorExecutor; import org.apache.druid.frame.read.FrameReader; import org.apache.druid.frame.testutil.FrameSequenceBuilder; import org.apache.druid.frame.write.FrameWriters; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.NonnullPair; @@ -85,7 +85,7 @@ public class FrameChannelMergerBenchmark { static { NullHandling.initializeForTests(); - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); } private static final String KEY = "key"; diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlGroupByBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlGroupByBenchmark.java index 80b6647a0ee..bf521b83163 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlGroupByBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlGroupByBenchmark.java @@ -24,7 +24,7 @@ import com.google.common.collect.ImmutableSet; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; @@ -89,7 +89,7 @@ public class SqlGroupByBenchmark static { NullHandling.initializeForTests(); ExpressionProcessing.initializeForTests(); - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); } private static final Logger log = new Logger(SqlGroupByBenchmark.class); @@ -331,7 +331,7 @@ public class SqlGroupByBenchmark // Hacky and pollutes global namespace, but it is fine since benchmarks are run in isolation. Wasn't able // to work up a cleaner way of doing it by modifying the injector. - CalciteTests.getJsonMapper().registerModules(NestedDataModule.getJacksonModulesList()); + CalciteTests.getJsonMapper().registerModules(BuiltInTypesModule.getJacksonModulesList()); final DruidSchemaCatalog rootSchema = CalciteTests.createMockRootSchema(conglomerate, walker, plannerConfig, AuthTestUtils.TEST_AUTHORIZER_MAPPER); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQComplexGroupByTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQComplexGroupByTest.java index a35b0ad0cb2..da4f975b24a 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQComplexGroupByTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQComplexGroupByTest.java @@ -25,7 +25,7 @@ import com.google.common.collect.ImmutableSet; import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.data.input.impl.systemfield.SystemFields; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.msq.indexing.MSQSpec; import org.apache.druid.msq.indexing.MSQTuningConfig; @@ -64,7 +64,7 @@ import java.util.Map; public class MSQComplexGroupByTest extends MSQTestBase { static { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); } private String dataFileNameJsonString; @@ -109,7 +109,7 @@ public class MSQComplexGroupByTest extends MSQTestBase dataFileSignature ); - objectMapper.registerModules(NestedDataModule.getJacksonModulesList()); + objectMapper.registerModules(BuiltInTypesModule.getJacksonModulesList()); } @MethodSource("data") diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index 5b944f9ae50..080ba843204 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -48,6 +48,7 @@ import org.apache.druid.discovery.NodeRole; import org.apache.druid.frame.channel.FrameChannelSequence; import org.apache.druid.frame.processor.Bouncer; import org.apache.druid.frame.testutil.FrameTestUtil; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.guice.DruidInjectorBuilder; import org.apache.druid.guice.DruidSecondaryModule; import org.apache.druid.guice.ExpressionModule; @@ -55,7 +56,6 @@ import org.apache.druid.guice.GuiceInjectors; import org.apache.druid.guice.IndexingServiceTuningConfigModule; import org.apache.druid.guice.JoinableFactoryModule; import org.apache.druid.guice.JsonConfigProvider; -import org.apache.druid.guice.NestedDataModule; import org.apache.druid.guice.SegmentWranglerModule; import org.apache.druid.guice.StartupInjectorBuilder; import org.apache.druid.guice.annotations.EscalatedGlobal; @@ -357,8 +357,8 @@ public class MSQTestBase extends BaseCalciteQueryTest { // We want this module to bring InputSourceModule along for the ride. binder.install(new InputSourceModule()); - binder.install(new NestedDataModule()); - NestedDataModule.registerHandlersAndSerde(); + binder.install(new BuiltInTypesModule()); + BuiltInTypesModule.registerHandlersAndSerde(); SqlBindings.addOperatorConversion(binder, ExternalOperatorConversion.class); SqlBindings.addOperatorConversion(binder, HttpOperatorConversion.class); SqlBindings.addOperatorConversion(binder, InlineOperatorConversion.class); @@ -521,7 +521,7 @@ public class MSQTestBase extends BaseCalciteQueryTest objectMapper = setupObjectMapper(injector); objectMapper.registerModules(new StorageConnectorModule().getJacksonModules()); objectMapper.registerModules(sqlModule.getJacksonModules()); - objectMapper.registerModules(NestedDataModule.getJacksonModulesList()); + objectMapper.registerModules(BuiltInTypesModule.getJacksonModulesList()); doReturn(mock(Request.class)).when(brokerClient).makeRequest(any(), anyString()); diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/DimensionSchema.java b/processing/src/main/java/org/apache/druid/data/input/impl/DimensionSchema.java index a65ddb7b9ae..b5933e722d8 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/DimensionSchema.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/DimensionSchema.java @@ -26,6 +26,7 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonValue; import com.google.common.base.Strings; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.guice.annotations.PublicApi; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.emitter.EmittingLogger; @@ -110,10 +111,9 @@ public abstract class DimensionSchema return name == null ? ofDefault() : valueOf(StringUtils.toUpperCase(name)); } - // this can be system configuration public static MultiValueHandling ofDefault() { - return SORTED_ARRAY; + return BuiltInTypesModule.getStringMultiValueHandlingMode(); } } diff --git a/processing/src/main/java/org/apache/druid/guice/NestedDataModule.java b/processing/src/main/java/org/apache/druid/guice/BuiltInTypesModule.java similarity index 68% rename from processing/src/main/java/org/apache/druid/guice/NestedDataModule.java rename to processing/src/main/java/org/apache/druid/guice/BuiltInTypesModule.java index daaf4ff2f65..dc9701e7d8b 100644 --- a/processing/src/main/java/org/apache/druid/guice/NestedDataModule.java +++ b/processing/src/main/java/org/apache/druid/guice/BuiltInTypesModule.java @@ -25,6 +25,7 @@ import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.common.annotations.VisibleForTesting; import com.google.inject.Binder; import com.google.inject.Provides; +import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.initialization.DruidModule; import org.apache.druid.segment.DefaultColumnFormatConfig; import org.apache.druid.segment.DimensionHandler; @@ -38,11 +39,19 @@ import org.apache.druid.segment.nested.StructuredDataJsonSerializer; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.virtual.NestedFieldVirtualColumn; +import javax.annotation.Nullable; import java.util.Collections; import java.util.List; -public class NestedDataModule implements DruidModule +public class BuiltInTypesModule implements DruidModule { + /** + * Initialized with a default value so tests can just get it via {@link #getStringMultiValueHandlingMode} without any + * explicit initialization. In production, this default may be overridden if a value is configured via + * {@link #initDimensionHandlerAndMvHandlingMode(DefaultColumnFormatConfig)}. + */ + private static DimensionSchema.MultiValueHandling STRING_MV_MODE = DimensionSchema.MultiValueHandling.SORTED_ARRAY; + @Override public List getJacksonModules() { @@ -53,14 +62,15 @@ public class NestedDataModule implements DruidModule public void configure(Binder binder) { registerSerde(); - // binding our side effect class to the lifecycle causes registerHandler to be called on service start, allowing - // use of the config to get the system default format version - LifecycleModule.register(binder, SideEffectHandlerRegisterer.class); + // binding our side effect classes to the lifecycle causes the initDimensionHandlerAndMvHandlingMode to be + // called on service start, allowing use of the config to get the system default format version and string multi + // value handling mode. + LifecycleModule.register(binder, SideEffectRegisterer.class); } @Provides @LazySingleton - public SideEffectHandlerRegisterer registerHandler(DefaultColumnFormatConfig formatsConfig) + public SideEffectRegisterer initDimensionHandlerAndMvHandlingMode(DefaultColumnFormatConfig formatsConfig) { if (formatsConfig.getNestedColumnFormatVersion() != null && formatsConfig.getNestedColumnFormatVersion() == 4) { DimensionHandlerUtils.registerDimensionHandlerProvider( @@ -73,7 +83,25 @@ public class NestedDataModule implements DruidModule new NestedCommonFormatHandlerProvider() ); } - return new SideEffectHandlerRegisterer(); + + setStringMultiValueHandlingModeIfConfigured(formatsConfig.getStringMultiValueHandlingMode()); + return new SideEffectRegisterer(); + } + + private static void setStringMultiValueHandlingModeIfConfigured(@Nullable String stringMultiValueHandlingMode) + { + if (stringMultiValueHandlingMode != null) { + STRING_MV_MODE = DimensionSchema.MultiValueHandling.fromString(stringMultiValueHandlingMode); + } + } + + /** + * @return the configured string multi value handling mode from the system config if set; otherwise, returns + * the default. + */ + public static DimensionSchema.MultiValueHandling getStringMultiValueHandlingMode() + { + return STRING_MV_MODE; } public static List getJacksonModulesList() @@ -126,13 +154,15 @@ public class NestedDataModule implements DruidModule return new NestedDataColumnHandlerV4(dimensionName); } } + /** - * this is used as a vehicle to register the correct version of the system default nested column handler by side - * effect with the help of binding to {@link org.apache.druid.java.util.common.lifecycle.Lifecycle} so that - * {@link #registerHandler(DefaultColumnFormatConfig)} can be called with the injected + * this is used as a vehicle to register the correct version of the system default nested column handler and multi + * value handling mode by side effect with the help of binding to + * {@link org.apache.druid.java.util.common.lifecycle.Lifecycle} so that + * {@link #initDimensionHandlerAndMvHandlingMode(DefaultColumnFormatConfig)} can be called with the injected * {@link DefaultColumnFormatConfig}. */ - public static class SideEffectHandlerRegisterer + public static class SideEffectRegisterer { // nothing to see here } diff --git a/processing/src/main/java/org/apache/druid/segment/DefaultColumnFormatConfig.java b/processing/src/main/java/org/apache/druid/segment/DefaultColumnFormatConfig.java index 3f41fb88c03..4df37f85b35 100644 --- a/processing/src/main/java/org/apache/druid/segment/DefaultColumnFormatConfig.java +++ b/processing/src/main/java/org/apache/druid/segment/DefaultColumnFormatConfig.java @@ -21,9 +21,11 @@ package org.apache.druid.segment; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.error.DruidException; import javax.annotation.Nullable; +import java.util.Arrays; import java.util.Objects; public class DefaultColumnFormatConfig @@ -39,16 +41,44 @@ public class DefaultColumnFormatConfig } } + private static void validateMultiValueHandlingMode(@Nullable String stringMultiValueHandlingMode) + { + if (stringMultiValueHandlingMode != null) { + try { + DimensionSchema.MultiValueHandling.fromString(stringMultiValueHandlingMode); + } + catch (IllegalArgumentException e) { + throw DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + "Invalid value[%s] specified for 'druid.indexing.formats.stringMultiValueHandlingMode'." + + " Supported values are [%s].", + stringMultiValueHandlingMode, + Arrays.toString(DimensionSchema.MultiValueHandling.values()) + ); + } + } + } + + @Nullable @JsonProperty("nestedColumnFormatVersion") private final Integer nestedColumnFormatVersion; + @Nullable + @JsonProperty("stringMultiValueHandlingMode") + private final String stringMultiValueHandlingMode; + @JsonCreator public DefaultColumnFormatConfig( - @JsonProperty("nestedColumnFormatVersion") @Nullable Integer nestedColumnFormatVersion + @JsonProperty("nestedColumnFormatVersion") @Nullable Integer nestedColumnFormatVersion, + @JsonProperty("stringMultiValueHandlingMode") @Nullable String stringMultiValueHandlingMode ) { + validateNestedFormatVersion(nestedColumnFormatVersion); + validateMultiValueHandlingMode(stringMultiValueHandlingMode); + this.nestedColumnFormatVersion = nestedColumnFormatVersion; - validateNestedFormatVersion(this.nestedColumnFormatVersion); + this.stringMultiValueHandlingMode = stringMultiValueHandlingMode; } @Nullable @@ -58,6 +88,13 @@ public class DefaultColumnFormatConfig return nestedColumnFormatVersion; } + @Nullable + @JsonProperty("stringMultiValueHandlingMode") + public String getStringMultiValueHandlingMode() + { + return stringMultiValueHandlingMode; + } + @Override public boolean equals(Object o) { @@ -68,13 +105,14 @@ public class DefaultColumnFormatConfig return false; } DefaultColumnFormatConfig that = (DefaultColumnFormatConfig) o; - return Objects.equals(nestedColumnFormatVersion, that.nestedColumnFormatVersion); + return Objects.equals(nestedColumnFormatVersion, that.nestedColumnFormatVersion) + && Objects.equals(stringMultiValueHandlingMode, that.stringMultiValueHandlingMode); } @Override public int hashCode() { - return Objects.hash(nestedColumnFormatVersion); + return Objects.hash(nestedColumnFormatVersion, stringMultiValueHandlingMode); } @Override @@ -82,6 +120,7 @@ public class DefaultColumnFormatConfig { return "DefaultColumnFormatConfig{" + "nestedColumnFormatVersion=" + nestedColumnFormatVersion + + ", stringMultiValueHandlingMode=" + stringMultiValueHandlingMode + '}'; } } diff --git a/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java b/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java index 923b48ca478..46d45024437 100644 --- a/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java @@ -66,7 +66,7 @@ public class StringDimensionIndexer extends DictionaryEncodedColumnIndexer gadget.getInstance(BuiltInTypesModule.SideEffectRegisterer.class) + ); + Assert.assertTrue(exception.getMessage().contains( + "Invalid value[boo] specified for 'druid.indexing.formats.stringMultiValueHandlingMode'." + + " Supported values are [[SORTED_ARRAY, SORTED_SET, ARRAY]]." + )); } private Injector makeInjector(Properties props) @@ -104,7 +148,7 @@ public class NestedDataModuleTest binder -> { JsonConfigProvider.bind(binder, "druid.indexing.formats", DefaultColumnFormatConfig.class); }, - new NestedDataModule() + new BuiltInTypesModule() ) ); diff --git a/processing/src/test/java/org/apache/druid/math/expr/FunctionTest.java b/processing/src/test/java/org/apache/druid/math/expr/FunctionTest.java index 56b3583031f..521c00f2f17 100644 --- a/processing/src/test/java/org/apache/druid/math/expr/FunctionTest.java +++ b/processing/src/test/java/org/apache/druid/math/expr/FunctionTest.java @@ -24,7 +24,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import org.apache.druid.common.config.NullHandling; import org.apache.druid.error.DruidException; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; @@ -66,7 +66,7 @@ public class FunctionTest extends InitializedNullHandlingTest TypeStrategiesTest.NULLABLE_TEST_PAIR_TYPE.getComplexTypeName(), new TypeStrategiesTest.NullableLongPairTypeStrategy() ); - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); } @Before diff --git a/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java b/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java index 1ff564106c5..23215b95d1e 100644 --- a/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java +++ b/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java @@ -31,7 +31,7 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; @@ -166,7 +166,7 @@ public class NestedDataTestUtils static { JSON_MAPPER = TestHelper.makeJsonMapper(); - JSON_MAPPER.registerModules(NestedDataModule.getJacksonModulesList()); + JSON_MAPPER.registerModules(BuiltInTypesModule.getJacksonModulesList()); } public static List createSimpleSegmentsTsv( diff --git a/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java b/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java index 77f86bdc4c5..6c0ff3b4951 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java @@ -22,7 +22,7 @@ package org.apache.druid.query.groupby; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.io.Closer; @@ -82,10 +82,10 @@ public class NestedDataGroupByQueryTest extends InitializedNullHandlingTest String vectorize ) { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); this.vectorize = QueryContexts.Vectorize.fromString(vectorize); this.helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper( - NestedDataModule.getJacksonModulesList(), + BuiltInTypesModule.getJacksonModulesList(), config, tempFolder ); diff --git a/processing/src/test/java/org/apache/druid/query/groupby/NestedGroupByArrayQueryTest.java b/processing/src/test/java/org/apache/druid/query/groupby/NestedGroupByArrayQueryTest.java index d305cb95444..f0e581291d3 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/NestedGroupByArrayQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/NestedGroupByArrayQueryTest.java @@ -22,7 +22,7 @@ package org.apache.druid.query.groupby; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.io.Closer; @@ -74,10 +74,10 @@ public class NestedGroupByArrayQueryTest String vectorize ) { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); this.vectorize = QueryContexts.Vectorize.fromString(vectorize); this.helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper( - NestedDataModule.getJacksonModulesList(), + BuiltInTypesModule.getJacksonModulesList(), config, tempFolder ); diff --git a/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/column/NestedColumnGroupByColumnSelectorStrategyTest.java b/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/column/NestedColumnGroupByColumnSelectorStrategyTest.java index a35432cc80e..bdce3d693e8 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/column/NestedColumnGroupByColumnSelectorStrategyTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/column/NestedColumnGroupByColumnSelectorStrategyTest.java @@ -21,7 +21,7 @@ package org.apache.druid.query.groupby.epinephelinae.column; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.query.IterableRowsCursorHelper; import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.groupby.epinephelinae.GroupByColumnSelectorStrategyFactory; @@ -44,7 +44,7 @@ import java.util.List; public class NestedColumnGroupByColumnSelectorStrategyTest extends InitializedNullHandlingTest { static { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); } private static final GroupByColumnSelectorStrategyFactory STRATEGY_FACTORY = new GroupByColumnSelectorStrategyFactory(); diff --git a/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java b/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java index 64c68c943f5..80fa1fa0832 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java @@ -24,7 +24,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; 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; @@ -78,8 +78,8 @@ public class NestedDataScanQueryTest extends InitializedNullHandlingTest public NestedDataScanQueryTest() { - NestedDataModule.registerHandlersAndSerde(); - List mods = NestedDataModule.getJacksonModulesList(); + BuiltInTypesModule.registerHandlersAndSerde(); + List mods = BuiltInTypesModule.getJacksonModulesList(); this.helper = AggregationTestHelper.createScanQueryAggregationTestHelper(mods, tempFolder); this.closer = Closer.create(); } diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java index 815a6ed9951..f555d81cefe 100644 --- a/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java @@ -22,7 +22,7 @@ package org.apache.druid.query.timeseries; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.io.Closer; @@ -98,7 +98,7 @@ public class NestedDataTimeseriesQueryTest extends InitializedNullHandlingTest ) { this.helper = AggregationTestHelper.createTimeseriesQueryAggregationTestHelper( - NestedDataModule.getJacksonModulesList(), + BuiltInTypesModule.getJacksonModulesList(), tempFolder ); this.segmentsGenerator = segmentsGenerator; diff --git a/processing/src/test/java/org/apache/druid/query/topn/NestedDataTopNQueryTest.java b/processing/src/test/java/org/apache/druid/query/topn/NestedDataTopNQueryTest.java index 392df70e1d7..cb3b5fbfc9a 100644 --- a/processing/src/test/java/org/apache/druid/query/topn/NestedDataTopNQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/topn/NestedDataTopNQueryTest.java @@ -21,7 +21,7 @@ package org.apache.druid.query.topn; import com.google.common.collect.ImmutableList; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; @@ -72,9 +72,9 @@ public class NestedDataTopNQueryTest extends InitializedNullHandlingTest BiFunction> segmentGenerator ) { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); this.helper = AggregationTestHelper.createTopNQueryAggregationTestHelper( - NestedDataModule.getJacksonModulesList(), + BuiltInTypesModule.getJacksonModulesList(), tempFolder ); this.segmentsGenerator = segmentGenerator; diff --git a/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java b/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java index e397267907e..73a2b6db50a 100644 --- a/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java +++ b/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java @@ -25,7 +25,7 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; 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; @@ -62,7 +62,7 @@ public class AutoTypeColumnIndexerTest extends InitializedNullHandlingTest @BeforeClass public static void setup() { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); } @Test diff --git a/processing/src/test/java/org/apache/druid/segment/DefaultColumnFormatsConfigTest.java b/processing/src/test/java/org/apache/druid/segment/DefaultColumnFormatsConfigTest.java index f259fc1dc23..c4de7e41f2b 100644 --- a/processing/src/test/java/org/apache/druid/segment/DefaultColumnFormatsConfigTest.java +++ b/processing/src/test/java/org/apache/druid/segment/DefaultColumnFormatsConfigTest.java @@ -22,6 +22,7 @@ package org.apache.druid.segment; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.jackson.DefaultObjectMapper; import org.junit.Assert; import org.junit.Test; @@ -33,21 +34,23 @@ public class DefaultColumnFormatsConfigTest @Test public void testDefaultsSerde() throws JsonProcessingException { - DefaultColumnFormatConfig defaultColumnFormatConfig = new DefaultColumnFormatConfig(null); + DefaultColumnFormatConfig defaultColumnFormatConfig = new DefaultColumnFormatConfig(null, null); String there = MAPPER.writeValueAsString(defaultColumnFormatConfig); DefaultColumnFormatConfig andBack = MAPPER.readValue(there, DefaultColumnFormatConfig.class); Assert.assertEquals(defaultColumnFormatConfig, andBack); Assert.assertNull(andBack.getNestedColumnFormatVersion()); + Assert.assertNull(andBack.getStringMultiValueHandlingMode()); } @Test public void testDefaultsSerdeOverride() throws JsonProcessingException { - DefaultColumnFormatConfig defaultColumnFormatConfig = new DefaultColumnFormatConfig(4); + DefaultColumnFormatConfig defaultColumnFormatConfig = new DefaultColumnFormatConfig(4, "ARRAY"); String there = MAPPER.writeValueAsString(defaultColumnFormatConfig); DefaultColumnFormatConfig andBack = MAPPER.readValue(there, DefaultColumnFormatConfig.class); Assert.assertEquals(defaultColumnFormatConfig, andBack); Assert.assertEquals(4, (int) andBack.getNestedColumnFormatVersion()); + Assert.assertEquals(DimensionSchema.MultiValueHandling.ARRAY.toString(), andBack.getStringMultiValueHandlingMode()); } @Test diff --git a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java index e670ae10003..359d2de58cb 100644 --- a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java +++ b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java @@ -25,7 +25,7 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; 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; @@ -61,7 +61,7 @@ public class NestedDataColumnIndexerV4Test extends InitializedNullHandlingTest @BeforeClass public static void setup() { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); } @Test diff --git a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnSchemaTest.java b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnSchemaTest.java index 8bfcfdefa4f..847f8a9d196 100644 --- a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnSchemaTest.java +++ b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnSchemaTest.java @@ -29,8 +29,8 @@ import org.junit.Test; public class NestedDataColumnSchemaTest { - private static final DefaultColumnFormatConfig DEFAULT_CONFIG = new DefaultColumnFormatConfig(null); - private static final DefaultColumnFormatConfig DEFAULT_CONFIG_V4 = new DefaultColumnFormatConfig(4); + private static final DefaultColumnFormatConfig DEFAULT_CONFIG = new DefaultColumnFormatConfig(null, null); + private static final DefaultColumnFormatConfig DEFAULT_CONFIG_V4 = new DefaultColumnFormatConfig(4, null); private static final ObjectMapper MAPPER; private static final ObjectMapper MAPPER_V4; diff --git a/processing/src/test/java/org/apache/druid/segment/data/IncrementalIndexTest.java b/processing/src/test/java/org/apache/druid/segment/data/IncrementalIndexTest.java index 4a104510f8f..3143d2f7be7 100644 --- a/processing/src/test/java/org/apache/druid/segment/data/IncrementalIndexTest.java +++ b/processing/src/test/java/org/apache/druid/segment/data/IncrementalIndexTest.java @@ -33,7 +33,7 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.data.input.Row; import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; @@ -104,7 +104,7 @@ public class IncrementalIndexTest extends InitializedNullHandlingTest public IncrementalIndexTest(String indexType, String mode, boolean isPreserveExistingMetrics) throws JsonProcessingException { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); this.isPreserveExistingMetrics = isPreserveExistingMetrics; indexCreator = closer.closeLater(new IncrementalIndexCreator(indexType, (builder, args) -> builder .setSimpleTestingIndexSchema("rollup".equals(mode), isPreserveExistingMetrics, (AggregatorFactory[]) args[0]) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/ArrayContainsElementFilterTests.java b/processing/src/test/java/org/apache/druid/segment/filter/ArrayContainsElementFilterTests.java index 2418941bf73..f45885790dc 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/ArrayContainsElementFilterTests.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/ArrayContainsElementFilterTests.java @@ -27,7 +27,7 @@ import com.google.common.collect.ImmutableMap; import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.common.config.NullHandling; import org.apache.druid.error.DruidException; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Pair; import org.apache.druid.query.filter.ArrayContainsElementFilter; @@ -1071,7 +1071,7 @@ public class ArrayContainsElementFilterTests Assert.assertFalse(Arrays.equals(f1.getCacheKey(), f2.getCacheKey())); Assert.assertArrayEquals(f1.getCacheKey(), f3.getCacheKey()); - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); f1 = new ArrayContainsElementFilter( "x", ColumnType.NESTED_DATA, diff --git a/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java index d9eb693ab76..179e4e96d10 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java @@ -41,7 +41,7 @@ import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.frame.FrameType; import org.apache.druid.frame.segment.FrameSegment; import org.apache.druid.frame.segment.FrameStorageAdapter; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; @@ -445,7 +445,7 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest @Before public void setUp() throws Exception { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); String className = getClass().getName(); Map> adaptersForClass = adapterCache.get().get(className); if (adaptersForClass == null) { diff --git a/processing/src/test/java/org/apache/druid/segment/filter/EqualityFilterTests.java b/processing/src/test/java/org/apache/druid/segment/filter/EqualityFilterTests.java index fd87969a042..97be448e61a 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/EqualityFilterTests.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/EqualityFilterTests.java @@ -30,7 +30,7 @@ import com.google.common.collect.TreeRangeSet; import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.common.config.NullHandling; import org.apache.druid.error.DruidException; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Pair; import org.apache.druid.math.expr.ExprEval; @@ -1760,7 +1760,7 @@ public class EqualityFilterTests Assert.assertFalse(Arrays.equals(f1.getCacheKey(), f2.getCacheKey())); Assert.assertArrayEquals(f1.getCacheKey(), f3.getCacheKey()); - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); f1 = new EqualityFilter("x", ColumnType.NESTED_DATA, ImmutableMap.of("x", ImmutableList.of(1, 2, 3)), null); f1_2 = new EqualityFilter("x", ColumnType.NESTED_DATA, ImmutableMap.of("x", ImmutableList.of(1, 2, 3)), null); f2 = new EqualityFilter("x", ColumnType.NESTED_DATA, ImmutableMap.of("x", ImmutableList.of(1, 2, 3, 4)), null); diff --git a/processing/src/test/java/org/apache/druid/segment/generator/SegmentGenerator.java b/processing/src/test/java/org/apache/druid/segment/generator/SegmentGenerator.java index 3c7ff2389c6..5afb9fa0412 100644 --- a/processing/src/test/java/org/apache/druid/segment/generator/SegmentGenerator.java +++ b/processing/src/test/java/org/apache/druid/segment/generator/SegmentGenerator.java @@ -26,7 +26,7 @@ import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.MapInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; @@ -140,7 +140,7 @@ public class SegmentGenerator implements Closeable { // In case we need to generate hyperUniques or json ComplexMetrics.registerSerde(HyperUniquesSerde.TYPE_NAME, new HyperUniquesSerde()); - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); final String dataHash = Hashing.sha256() .newHasher() diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexIngestionTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexIngestionTest.java index 4ef0a0c69d9..77e0470c548 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexIngestionTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexIngestionTest.java @@ -22,7 +22,7 @@ package org.apache.druid.segment.incremental; import com.fasterxml.jackson.core.JsonProcessingException; import com.google.common.collect.ImmutableMap; import org.apache.druid.data.input.MapBasedInputRow; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.aggregation.Aggregator; import org.apache.druid.query.aggregation.LongMaxAggregator; @@ -50,7 +50,7 @@ public class IncrementalIndexIngestionTest extends InitializedNullHandlingTest public IncrementalIndexIngestionTest(String indexType) throws JsonProcessingException { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); indexCreator = closer.closeLater(new IncrementalIndexCreator(indexType, (builder, args) -> builder .setIndexSchema((IncrementalIndexSchema) args[0]) .setMaxRowCount(MAX_ROWS) diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexMultiValueSpecTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexMultiValueSpecTest.java index b980c0251b6..80c8207ed60 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexMultiValueSpecTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexMultiValueSpecTest.java @@ -27,7 +27,7 @@ import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.CloserRule; @@ -56,7 +56,7 @@ public class IncrementalIndexMultiValueSpecTest extends InitializedNullHandlingT public IncrementalIndexMultiValueSpecTest(String indexType) throws JsonProcessingException { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); indexCreator = closer.closeLater(new IncrementalIndexCreator(indexType, (builder, args) -> builder .setIndexSchema((IncrementalIndexSchema) args[0]) .setMaxRowCount(10_000) diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java index 0d0f979f715..09f78e066fb 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java @@ -27,7 +27,7 @@ import org.apache.druid.collections.CloseableStupidPool; import org.apache.druid.collections.ResourceHolder; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.MapBasedInputRow; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; 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; @@ -97,7 +97,7 @@ public class IncrementalIndexStorageAdapterTest extends InitializedNullHandlingT public IncrementalIndexStorageAdapterTest(String indexType) throws JsonProcessingException { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); indexCreator = closer.closeLater(new IncrementalIndexCreator(indexType, (builder, args) -> builder .setSimpleTestingIndexSchema(new CountAggregatorFactory("cnt")) .setMaxRowCount(1_000) diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java index cd85768e5ac..1ebd1731c33 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java @@ -29,7 +29,7 @@ import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.collections.bitmap.ImmutableBitmap; import org.apache.druid.collections.bitmap.WrappedRoaringBitmap; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; 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.io.smoosh.FileSmoosher; @@ -149,7 +149,7 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest @BeforeClass public static void staticSetup() { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); } @Before diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4Test.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4Test.java index bcc40b333c4..c7b1facec56 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4Test.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4Test.java @@ -28,7 +28,7 @@ import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.collections.bitmap.RoaringBitmapFactory; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; 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.io.smoosh.FileSmoosher; @@ -141,7 +141,7 @@ public class NestedDataColumnSupplierV4Test extends InitializedNullHandlingTest @BeforeClass public static void staticSetup() { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); } @Before diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnSelectorsTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnSelectorsTest.java index 86455f12c6e..e46b7893a0d 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnSelectorsTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnSelectorsTest.java @@ -22,7 +22,7 @@ package org.apache.druid.segment.nested; import com.fasterxml.jackson.databind.Module; import com.google.common.collect.ImmutableList; import org.apache.druid.error.DruidException; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; 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; @@ -81,8 +81,8 @@ public class NestedFieldColumnSelectorsTest extends InitializedNullHandlingTest public NestedFieldColumnSelectorsTest() { - NestedDataModule.registerHandlersAndSerde(); - List mods = NestedDataModule.getJacksonModulesList(); + BuiltInTypesModule.registerHandlersAndSerde(); + List mods = BuiltInTypesModule.getJacksonModulesList(); this.helper = AggregationTestHelper.createScanQueryAggregationTestHelper( mods, tempFolder diff --git a/processing/src/test/java/org/apache/druid/segment/nested/ScalarDoubleColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/ScalarDoubleColumnSupplierTest.java index 5d76e18f94c..3f237b9396f 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/ScalarDoubleColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/ScalarDoubleColumnSupplierTest.java @@ -26,7 +26,7 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; 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.io.smoosh.FileSmoosher; @@ -105,7 +105,7 @@ public class ScalarDoubleColumnSupplierTest extends InitializedNullHandlingTest @BeforeClass public static void staticSetup() { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); } @Before diff --git a/processing/src/test/java/org/apache/druid/segment/nested/ScalarLongColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/ScalarLongColumnSupplierTest.java index 4661d72cdbb..5fd8ddd299c 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/ScalarLongColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/ScalarLongColumnSupplierTest.java @@ -26,7 +26,7 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; 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.io.smoosh.FileSmoosher; @@ -105,7 +105,7 @@ public class ScalarLongColumnSupplierTest extends InitializedNullHandlingTest @BeforeClass public static void staticSetup() { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); } @Before diff --git a/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java index 5f0c1d020c8..ee7ef05149e 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java @@ -26,7 +26,7 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; 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.io.smoosh.FileSmoosher; @@ -105,7 +105,7 @@ public class ScalarStringColumnSupplierTest extends InitializedNullHandlingTest @BeforeClass public static void staticSetup() { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); } @Before diff --git a/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java index 6ea4509bef7..4a9423b87bc 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java @@ -24,7 +24,7 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; 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.io.smoosh.FileSmoosher; @@ -167,7 +167,7 @@ public class VariantColumnSupplierTest extends InitializedNullHandlingTest @BeforeClass public static void staticSetup() { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); } @Parameterized.Parameters(name = "data = {0}") diff --git a/server/src/main/java/org/apache/druid/initialization/CoreInjectorBuilder.java b/server/src/main/java/org/apache/druid/initialization/CoreInjectorBuilder.java index e07ac5ed115..902d2855a0a 100644 --- a/server/src/main/java/org/apache/druid/initialization/CoreInjectorBuilder.java +++ b/server/src/main/java/org/apache/druid/initialization/CoreInjectorBuilder.java @@ -24,6 +24,7 @@ import org.apache.druid.curator.CuratorModule; import org.apache.druid.curator.discovery.DiscoveryModule; import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.AnnouncerModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.guice.CoordinatorDiscoveryModule; import org.apache.druid.guice.DruidInjectorBuilder; import org.apache.druid.guice.DruidSecondaryModule; @@ -34,7 +35,6 @@ import org.apache.druid.guice.JavaScriptModule; import org.apache.druid.guice.LifecycleModule; import org.apache.druid.guice.LocalDataStorageDruidModule; import org.apache.druid.guice.MetadataConfigModule; -import org.apache.druid.guice.NestedDataModule; import org.apache.druid.guice.ServerModule; import org.apache.druid.guice.ServerViewModule; import org.apache.druid.guice.StartupLoggingModule; @@ -113,7 +113,7 @@ public class CoreInjectorBuilder extends DruidInjectorBuilder new StorageNodeModule(), new JettyServerModule(), new ExpressionModule(), - new NestedDataModule(), + new BuiltInTypesModule(), new DiscoveryModule(), new ServerViewModule(), new MetadataConfigModule(), diff --git a/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java b/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java index 9369717bf1f..75c8bb6b143 100644 --- a/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java +++ b/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java @@ -31,7 +31,7 @@ import org.apache.druid.collections.bitmap.RoaringBitmapFactory; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.guice.StartupInjectorBuilder; import org.apache.druid.guice.annotations.Json; import org.apache.druid.initialization.ServerInjectorBuilder; @@ -84,7 +84,7 @@ public class DumpSegmentTest extends InitializedNullHandlingTest public DumpSegmentTest() { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); this.closer = Closer.create(); } @@ -156,15 +156,15 @@ public class DumpSegmentTest extends InitializedNullHandlingTest { Injector injector = Mockito.mock(Injector.class); ObjectMapper mapper = TestHelper.makeJsonMapper(); - mapper.registerModules(NestedDataModule.getJacksonModulesList()); + mapper.registerModules(BuiltInTypesModule.getJacksonModulesList()); mapper.setInjectableValues( new InjectableValues.Std() .addValue(ExprMacroTable.class.getName(), TestExprMacroTable.INSTANCE) .addValue(ObjectMapper.class.getName(), mapper) - .addValue(DefaultColumnFormatConfig.class, new DefaultColumnFormatConfig(null)) + .addValue(DefaultColumnFormatConfig.class, new DefaultColumnFormatConfig(null, null)) ); Mockito.when(injector.getInstance(Key.get(ObjectMapper.class, Json.class))).thenReturn(mapper); - Mockito.when(injector.getInstance(DefaultColumnFormatConfig.class)).thenReturn(new DefaultColumnFormatConfig(null)); + Mockito.when(injector.getInstance(DefaultColumnFormatConfig.class)).thenReturn(new DefaultColumnFormatConfig(null, null)); List segments = createSegments(tempFolder, closer); QueryableIndex queryableIndex = segments.get(0).asQueryableIndex(); @@ -196,15 +196,15 @@ public class DumpSegmentTest extends InitializedNullHandlingTest { Injector injector = Mockito.mock(Injector.class); ObjectMapper mapper = TestHelper.makeJsonMapper(); - mapper.registerModules(NestedDataModule.getJacksonModulesList()); + mapper.registerModules(BuiltInTypesModule.getJacksonModulesList()); mapper.setInjectableValues( new InjectableValues.Std() .addValue(ExprMacroTable.class.getName(), TestExprMacroTable.INSTANCE) .addValue(ObjectMapper.class.getName(), mapper) - .addValue(DefaultColumnFormatConfig.class, new DefaultColumnFormatConfig(null)) + .addValue(DefaultColumnFormatConfig.class, new DefaultColumnFormatConfig(null, null)) ); Mockito.when(injector.getInstance(Key.get(ObjectMapper.class, Json.class))).thenReturn(mapper); - Mockito.when(injector.getInstance(DefaultColumnFormatConfig.class)).thenReturn(new DefaultColumnFormatConfig(null)); + Mockito.when(injector.getInstance(DefaultColumnFormatConfig.class)).thenReturn(new DefaultColumnFormatConfig(null, null)); List segments = createSegments(tempFolder, closer); QueryableIndex queryableIndex = segments.get(0).asQueryableIndex(); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java index 2fe1919f6a1..2e89c2eeb04 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java @@ -24,8 +24,8 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import org.apache.calcite.avatica.SqlType; import org.apache.druid.common.config.NullHandling; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.guice.DruidInjectorBuilder; -import org.apache.druid.guice.NestedDataModule; import org.apache.druid.java.util.common.HumanReadableBytes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; @@ -130,7 +130,7 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest public void configureGuice(DruidInjectorBuilder builder) { super.configureGuice(builder); - builder.addModule(new NestedDataModule()); + builder.addModule(new BuiltInTypesModule()); } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java index 9e6e5da2bb0..5efc59420ad 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java @@ -34,8 +34,8 @@ import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.error.DruidException; import org.apache.druid.error.DruidExceptionMatcher; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.guice.DruidInjectorBuilder; -import org.apache.druid.guice.NestedDataModule; import org.apache.druid.java.util.common.HumanReadableBytes; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.math.expr.ExprMacroTable; @@ -196,7 +196,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest public void configureGuice(DruidInjectorBuilder builder) { super.configureGuice(builder); - builder.addModule(new NestedDataModule()); + builder.addModule(new BuiltInTypesModule()); } @SuppressWarnings("resource") @@ -207,7 +207,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest final Injector injector ) { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); final QueryableIndex index = IndexBuilder.create() .tmpDir(tempDirProducer.newTempFolder()) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java index d195e0cefb9..5d710600a11 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java @@ -548,7 +548,7 @@ public class SqlTestFramework { binder.bind(DruidOperatorTable.class).in(LazySingleton.class); binder.bind(DataSegment.PruneSpecsHolder.class).toInstance(DataSegment.PruneSpecsHolder.DEFAULT); - binder.bind(DefaultColumnFormatConfig.class).toInstance(new DefaultColumnFormatConfig(null)); + binder.bind(DefaultColumnFormatConfig.class).toInstance(new DefaultColumnFormatConfig(null, null)); } @Provides @@ -620,7 +620,6 @@ public class SqlTestFramework .addModule(new SqlAggregationModule()) .addModule(new ExpressionModule()) .addModule(new TestSetupModule(builder)); - builder.componentSupplier.configureGuice(injectorBuilder); ServiceInjectorBuilder serviceInjector = new ServiceInjectorBuilder(injectorBuilder); From c7eacd079e74cd7c96caaed9d083d8c42554f873 Mon Sep 17 00:00:00 2001 From: Sree Charan Manamala Date: Mon, 5 Aug 2024 11:27:51 +0530 Subject: [PATCH 11/12] fallback SQL IN filter to expression filter when VirtualColumnRegistry is null (#16836) --- .../ScalarInArrayOperatorConversion.java | 3 ++ .../sql/calcite/CalciteSubqueryTest.java | 35 +++++++++++++++++++ 2 files changed, 38 insertions(+) diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ScalarInArrayOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ScalarInArrayOperatorConversion.java index 8a18ce73df2..33e49697f52 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ScalarInArrayOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ScalarInArrayOperatorConversion.java @@ -94,6 +94,9 @@ public class ScalarInArrayOperatorConversion extends DirectOperatorConversion } else if (scalarExpression.isSimpleExtraction() && plannerContext.isUseLegacyInFilter()) { scalarColumn = scalarExpression.getSimpleExtraction().getColumn(); scalarExtractionFn = scalarExpression.getSimpleExtraction().getExtractionFn(); + } else if (virtualColumnRegistry == null) { + // virtual column registry unavailable, fallback to expression filter + return null; } else { scalarColumn = virtualColumnRegistry.getOrCreateVirtualColumnForExpression( scalarExpression, diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java index c1236162fa0..1452358950a 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java @@ -58,8 +58,12 @@ import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.ExtractionDimensionSpec; import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.extraction.SubstringDimExtractionFn; +import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.filter.InDimFilter; +import org.apache.druid.query.filter.TypedInFilter; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; +import org.apache.druid.query.groupby.orderby.NoopLimitSpec; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.scan.ScanQuery; @@ -92,6 +96,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -1601,6 +1606,36 @@ public class CalciteSubqueryTest extends BaseCalciteQueryTest ); } + @MethodSource("constructorFeeder") + @ParameterizedTest(name = "{0}") + public void testScalarInArrayToUseHavingFilter(String testName, Map queryContext) + { + DimFilter filter = NullHandling.replaceWithDefault() + ? new InDimFilter("v0", new HashSet<>(Arrays.asList("1", "17"))) + : new TypedInFilter("v0", ColumnType.LONG, null, ImmutableList.of(1, 17), null); + testQuery( + "select countryName from " + + "(select countryName, length(countryName) as cname from wikipedia group by countryName) " + + "where SCALAR_IN_ARRAY(cname, ARRAY[17, 1])", + queryContext, + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(new TableDataSource(CalciteTests.WIKIPEDIA)) + .setInterval(querySegmentSpec(Intervals.ETERNITY)) + .setVirtualColumns(expressionVirtualColumn("v0", "strlen(\"countryName\")", ColumnType.LONG)) + .setDimFilter(filter) + .setGranularity(Granularities.ALL) + .setDimensions(new DefaultDimensionSpec("countryName", "d0", ColumnType.STRING)) + .setLimitSpec(NoopLimitSpec.instance()) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{"Republic of Korea"} + ) + ); + } + public static class SubqueryComponentSupplier extends SqlTestFramework.StandardComponentSupplier { From bb23ace518bc2192ed8b6590e93bac5a99d6d6b2 Mon Sep 17 00:00:00 2001 From: Zoltan Haindrich Date: Mon, 5 Aug 2024 08:59:48 +0000 Subject: [PATCH 12/12] builtintypes instead nesteddata --- .../quidem/ExposedAsBrokerQueryComponentSupplierWrapper.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/quidem-ut/src/main/java/org/apache/druid/quidem/ExposedAsBrokerQueryComponentSupplierWrapper.java b/quidem-ut/src/main/java/org/apache/druid/quidem/ExposedAsBrokerQueryComponentSupplierWrapper.java index d688144b050..f853dd317d1 100644 --- a/quidem-ut/src/main/java/org/apache/druid/quidem/ExposedAsBrokerQueryComponentSupplierWrapper.java +++ b/quidem-ut/src/main/java/org/apache/druid/quidem/ExposedAsBrokerQueryComponentSupplierWrapper.java @@ -43,6 +43,7 @@ import org.apache.druid.discovery.DruidNodeDiscoveryProvider; import org.apache.druid.guice.AnnouncerModule; import org.apache.druid.guice.BrokerProcessingModule; import org.apache.druid.guice.BrokerServiceModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.guice.CoordinatorDiscoveryModule; import org.apache.druid.guice.DruidInjectorBuilder; import org.apache.druid.guice.ExpressionModule; @@ -56,7 +57,6 @@ import org.apache.druid.guice.LazySingleton; import org.apache.druid.guice.LifecycleModule; import org.apache.druid.guice.LocalDataStorageDruidModule; import org.apache.druid.guice.MetadataConfigModule; -import org.apache.druid.guice.NestedDataModule; import org.apache.druid.guice.QueryRunnerFactoryModule; import org.apache.druid.guice.SegmentWranglerModule; import org.apache.druid.guice.ServerModule; @@ -273,6 +273,7 @@ public class ExposedAsBrokerQueryComponentSupplierWrapper implements QueryCompon */ private void installForServerModules(CoreInjectorBuilder builder) { + builder.add( new Log4jShutterDownerModule(), new LifecycleModule(), @@ -291,7 +292,7 @@ public class ExposedAsBrokerQueryComponentSupplierWrapper implements QueryCompon new StorageNodeModule(), new JettyServerModule(), new ExpressionModule(), - new NestedDataModule(), + new BuiltInTypesModule(), new DiscoveryModule(), new ServerViewModule(), new MetadataConfigModule(),