From 0c6681517ccb4dc2fb33e3a5fe649cf9515e0fd6 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Mon, 21 Nov 2022 20:39:07 +0530 Subject: [PATCH 01/34] Update versions for 25.0 release --- distribution/docker/docker-compose.yml | 2 +- web-console/src/links.ts | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/distribution/docker/docker-compose.yml b/distribution/docker/docker-compose.yml index 58b7a47a4b4a..9faa35389bf2 100644 --- a/distribution/docker/docker-compose.yml +++ b/distribution/docker/docker-compose.yml @@ -49,7 +49,7 @@ services: - ZOO_MY_ID=1 coordinator: - image: apache/druid:0.24.0 + image: apache/druid:25.0.0 container_name: coordinator volumes: - druid_shared:/opt/shared diff --git a/web-console/src/links.ts b/web-console/src/links.ts index d9bd341d4f67..5c391d838e44 100644 --- a/web-console/src/links.ts +++ b/web-console/src/links.ts @@ -19,7 +19,7 @@ import hasOwnProp from 'has-own-prop'; // This is set to the latest available version and should be updated to the next version before release -const DRUID_DOCS_VERSION = 'latest'; +const DRUID_DOCS_VERSION = '25.0.0'; function fillVersion(str: string): string { return str.replace(/\{\{VERSION}}/g, DRUID_DOCS_VERSION); From 35580fe74dbea81eb54ae75aec3c40edc5885568 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 22 Nov 2022 14:37:29 +0530 Subject: [PATCH 02/34] Fix web-console snapshots (#13408) * Fix web-console snapshots * Revert changes to package and package-lock.json --- .../header-bar/__snapshots__/header-bar.spec.tsx.snap | 2 +- .../__snapshots__/retention-dialog.spec.tsx.snap | 2 +- .../__snapshots__/execution-error-pane.spec.tsx.snap | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/web-console/src/components/header-bar/__snapshots__/header-bar.spec.tsx.snap b/web-console/src/components/header-bar/__snapshots__/header-bar.spec.tsx.snap index 92ea6c77b87c..d549afd9ae12 100644 --- a/web-console/src/components/header-bar/__snapshots__/header-bar.spec.tsx.snap +++ b/web-console/src/components/header-bar/__snapshots__/header-bar.spec.tsx.snap @@ -325,7 +325,7 @@ exports[`HeaderBar matches snapshot 1`] = ` diff --git a/web-console/src/views/workbench-view/execution-error-pane/__snapshots__/execution-error-pane.spec.tsx.snap b/web-console/src/views/workbench-view/execution-error-pane/__snapshots__/execution-error-pane.spec.tsx.snap index de73ea9dc9a7..24573eea258b 100644 --- a/web-console/src/views/workbench-view/execution-error-pane/__snapshots__/execution-error-pane.spec.tsx.snap +++ b/web-console/src/views/workbench-view/execution-error-pane/__snapshots__/execution-error-pane.spec.tsx.snap @@ -9,7 +9,7 @@ exports[`ExecutionErrorPane matches snapshot 1`] = ` className="error-message-text" > TooManyWarnings From f4731433582410de4c960cbaf8133c4667e575d5 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Wed, 23 Nov 2022 18:11:29 +0530 Subject: [PATCH 03/34] Add sequential sketch merging to MSQ (#13205) (#13414) * Add sketch fetching framework * Refactor code to support sequential merge * Update worker sketch fetcher * Refactor sketch fetcher * Refactor sketch fetcher * Add context parameter and threshold to trigger sequential merge * Fix test * Add integration test for non sequential merge * Address review comments * Address review comments * Address review comments * Resolve maxRetainedBytes * Add new classes * Renamed key statistics information class * Rename fetchStatisticsSnapshotForTimeChunk function * Address review comments * Address review comments * Update documentation and add comments * Resolve build issues * Resolve build issues * Change worker APIs to async * Address review comments * Resolve build issues * Add null time check * Update integration tests * Address review comments * Add log messages and comments * Resolve build issues * Add unit tests * Add unit tests * Fix timing issue in tests --- docs/multi-stage-query/reference.md | 22 ++ .../msq/exec/ClusterStatisticsMergeMode.java | 44 +++ .../org/apache/druid/msq/exec/Controller.java | 8 +- .../druid/msq/exec/ControllerClient.java | 10 +- .../apache/druid/msq/exec/ControllerImpl.java | 63 +++- .../exec/ExceptionWrappingWorkerClient.java | 18 + .../org/apache/druid/msq/exec/Worker.java | 13 + .../apache/druid/msq/exec/WorkerClient.java | 22 ++ .../org/apache/druid/msq/exec/WorkerImpl.java | 26 +- .../druid/msq/exec/WorkerSketchFetcher.java | 340 ++++++++++++++++++ .../msq/indexing/ControllerChatHandler.java | 16 +- .../msq/indexing/IndexerControllerClient.java | 10 +- .../msq/indexing/IndexerWorkerClient.java | 43 +++ .../druid/msq/indexing/WorkerChatHandler.java | 40 +++ .../controller/ControllerQueryKernel.java | 46 ++- .../controller/ControllerStagePhase.java | 13 +- .../controller/ControllerStageTracker.java | 127 ++++--- .../ClusterByStatisticsCollectorImpl.java | 20 +- .../ClusterByStatisticsSnapshot.java | 31 +- .../CompleteKeyStatisticsInformation.java | 82 +++++ .../DelegateOrMinKeyCollectorSnapshot.java | 3 + .../msq/statistics/DistinctKeySnapshot.java | 3 + .../msq/statistics/KeyCollectorSnapshot.java | 9 + .../PartialKeyStatisticsInformation.java | 67 ++++ .../QuantilesSketchKeyCollectorSnapshot.java | 3 + .../msq/util/MultiStageQueryContext.java | 15 + .../exec/WorkerSketchFetcherAutoModeTest.java | 139 +++++++ .../msq/exec/WorkerSketchFetcherTest.java | 295 +++++++++++++++ .../BaseControllerQueryKernelTest.java | 27 +- .../ControllerQueryKernelTests.java | 55 ++- ...tialKeyStatisticsInformationSerdeTest.java | 62 ++++ .../msq/test/MSQTestControllerClient.java | 10 +- .../druid/msq/test/MSQTestWorkerClient.java | 24 ++ integration-tests-ex/README.md | 2 +- integration-tests-ex/cases/pom.xml | 5 + .../msq/ITKeyStatisticsSketchMergeMode.java | 206 +++++++++++ .../apache/druid/frame/key/RowKeyReader.java | 22 ++ .../druid/frame/key/RowKeyReaderTest.java | 38 ++ website/.spelling | 3 + 39 files changed, 1850 insertions(+), 132 deletions(-) create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ClusterStatisticsMergeMode.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/CompleteKeyStatisticsInformation.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/PartialKeyStatisticsInformation.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherAutoModeTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/PartialKeyStatisticsInformationSerdeTest.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITKeyStatisticsSketchMergeMode.java diff --git a/docs/multi-stage-query/reference.md b/docs/multi-stage-query/reference.md index 3550566874b8..a4bcbfc27b1f 100644 --- a/docs/multi-stage-query/reference.md +++ b/docs/multi-stage-query/reference.md @@ -203,6 +203,28 @@ The following table lists the context parameters for the MSQ task engine: | `maxParseExceptions`| SELECT, INSERT, REPLACE

Maximum number of parse exceptions that are ignored while executing the query before it stops with `TooManyWarningsFault`. To ignore all the parse exceptions, set the value to -1.| 0 | | `rowsPerSegment` | INSERT or REPLACE

The number of rows per segment to target. The actual number of rows per segment may be somewhat higher or lower than this number. In most cases, use the default. For general information about sizing rows per segment, see [Segment Size Optimization](../operations/segment-optimization.md). | 3,000,000 | | `indexSpec` | INSERT or REPLACE

An [`indexSpec`](../ingestion/ingestion-spec.md#indexspec) to use when generating segments. May be a JSON string or object. See [Front coding](../ingestion/ingestion-spec.md#front-coding) for details on configuring an `indexSpec` with front coding. | See [`indexSpec`](../ingestion/ingestion-spec.md#indexspec). | +| `clusterStatisticsMergeMode` | Whether to use parallel or sequential mode for merging of the worker sketches. Can be `PARALLEL`, `SEQUENTIAL` or `AUTO`. See [Sketch Merging Mode](#sketch-merging-mode) for more information. | `AUTO` | + +## Sketch Merging Mode +This section details the advantages and performance of various Cluster By Statistics Merge Modes. + +If a query requires key statistics to generate partition boundaries, key statistics are gathered by the workers while +reading rows from the datasource. These statistics must be transferred to the controller to be merged together. +`clusterStatisticsMergeMode` configures the way in which this happens. + +`PARALLEL` mode fetches the key statistics for all time chunks from all workers together and the controller then downsamples +the sketch if it does not fit in memory. This is faster than `SEQUENTIAL` mode as there is less over head in fetching sketches +for all time chunks together. This is good for small sketches which won't be downsampled even if merged together or if +accuracy in segment sizing for the ingestion is not very important. + +`SEQUENTIAL` mode fetches the sketches in ascending order of time and generates the partition boundaries for one time +chunk at a time. This gives more working memory to the controller for merging sketches, which results in less +downsampling and thus, more accuracy. There is, however, a time overhead on fetching sketches in sequential order. This is +good for cases where accuracy is important. + +`AUTO` mode tries to find the best approach based on number of workers and size of input rows. If there are more +than 100 workers or if the combined sketch size among all workers is more than 1GB, `SEQUENTIAL` is chosen, otherwise, +`PARALLEL` is chosen. ## Durable Storage This section enumerates the advantages and performance implications of enabling durable storage while executing MSQ tasks. diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ClusterStatisticsMergeMode.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ClusterStatisticsMergeMode.java new file mode 100644 index 000000000000..92ed82ff5e11 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ClusterStatisticsMergeMode.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.exec; + +/** + * Mode which dictates how {@link WorkerSketchFetcher} gets sketches for the partition boundaries from workers. + */ +public enum ClusterStatisticsMergeMode +{ + /** + * Fetches sketch in sequential order based on time. Slower due to overhead, but more accurate. + */ + SEQUENTIAL, + + /** + * Fetch all sketches from the worker at once. Faster to generate partitions, but less accurate. + */ + PARALLEL, + + /** + * Tries to decide between sequential and parallel modes based on the number of workers and size of the input + * + * If there are more than 100 workers or if the combined sketch size among all workers is more than + * 1,000,000,000 bytes, SEQUENTIAL mode is chosen, otherwise, PARALLEL mode is chosen. + */ + AUTO +} 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 07730de45e7a..fe010b219e49 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 @@ -27,7 +27,7 @@ import org.apache.druid.msq.counters.CounterSnapshotsTree; import org.apache.druid.msq.indexing.MSQControllerTask; import org.apache.druid.msq.indexing.error.MSQErrorReport; -import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; +import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation; import javax.annotation.Nullable; import java.util.List; @@ -81,9 +81,11 @@ public String getId() // Worker-to-controller messages /** - * Provide a {@link ClusterByStatisticsSnapshot} for shuffling stages. + * Accepts a {@link PartialKeyStatisticsInformation} and updates the controller key statistics information. If all key + * statistics have been gathered, enqueues the task with the {@link WorkerSketchFetcher} to generate partiton boundaries. + * This is intended to be called by the {@link org.apache.druid.msq.indexing.ControllerChatHandler}. */ - void updateStatus(int stageNumber, int workerNumber, Object keyStatisticsObject); + void updatePartialKeyStatisticsInformation(int stageNumber, int workerNumber, Object partialKeyStatisticsInformationObject); /** * System error reported by a subtask. Note that the errors are organized by 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 faf1c3ff5e98..9d197b76dd17 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 @@ -22,7 +22,7 @@ import org.apache.druid.msq.counters.CounterSnapshotsTree; import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.apache.druid.msq.kernel.StageId; -import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; +import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation; import javax.annotation.Nullable; import java.io.IOException; @@ -34,13 +34,13 @@ public interface ControllerClient extends AutoCloseable { /** - * Client side method to update the controller with key statistics for a particular stage and worker. - * Controller's implementation collates all the key statistics for a stage to generate the partition boundaries. + * 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. */ - void postKeyStatistics( + void postPartialKeyStatistics( StageId stageId, int workerNumber, - ClusterByStatisticsSnapshot keyStatistics + PartialKeyStatisticsInformation partialKeyStatisticsInformation ) throws IOException; /** 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 a4812292fb15..cafc0f389258 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 @@ -64,6 +64,7 @@ import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Either; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; @@ -107,6 +108,7 @@ import org.apache.druid.msq.indexing.error.MSQWarningReportLimiterPublisher; import org.apache.druid.msq.indexing.error.MSQWarnings; import org.apache.druid.msq.indexing.error.QueryNotSupportedFault; +import org.apache.druid.msq.indexing.error.TooManyPartitionsFault; import org.apache.druid.msq.indexing.error.TooManyWarningsFault; import org.apache.druid.msq.indexing.error.UnknownFault; import org.apache.druid.msq.indexing.report.MSQResultsReport; @@ -149,7 +151,8 @@ import org.apache.druid.msq.shuffle.DurableStorageInputChannelFactory; import org.apache.druid.msq.shuffle.DurableStorageUtils; import org.apache.druid.msq.shuffle.WorkerInputChannelFactory; -import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; +import org.apache.druid.msq.statistics.CompleteKeyStatisticsInformation; +import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation; import org.apache.druid.msq.util.DimensionSchemaUtils; import org.apache.druid.msq.util.IntervalUtils; import org.apache.druid.msq.util.MSQFutureUtils; @@ -201,6 +204,7 @@ import java.util.Set; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ThreadLocalRandom; @@ -259,6 +263,7 @@ public class ControllerImpl implements Controller // For live reports. Written by the main controller thread, read by HTTP threads. private final ConcurrentHashMap stagePartitionCountsForLiveReports = new ConcurrentHashMap<>(); + private WorkerSketchFetcher workerSketchFetcher; // Time at which the query started. // For live reports. Written by the main controller thread, read by HTTP threads. private volatile DateTime queryStartTime = null; @@ -519,6 +524,15 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer) context.registerController(this, closer); this.netClient = new ExceptionWrappingWorkerClient(context.taskClientFor(this)); + ClusterStatisticsMergeMode clusterStatisticsMergeMode = + MultiStageQueryContext.getClusterStatisticsMergeMode(task.getQuerySpec().getQuery().context()); + + log.debug("Query [%s] cluster statistics merge mode is set to %s.", id(), clusterStatisticsMergeMode); + + int statisticsMaxRetainedBytes = WorkerMemoryParameters.createProductionInstanceForController(context.injector()) + .getPartitionStatisticsMaxRetainedBytes(); + this.workerSketchFetcher = new WorkerSketchFetcher(netClient, clusterStatisticsMergeMode, statisticsMaxRetainedBytes); + closer.register(netClient::close); final boolean isDurableStorageEnabled = @@ -565,10 +579,12 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer) } /** - * Provide a {@link ClusterByStatisticsSnapshot} for shuffling stages. + * Accepts a {@link PartialKeyStatisticsInformation} and updates the controller key statistics information. If all key + * statistics information has been gathered, enqueues the task with the {@link WorkerSketchFetcher} to generate + * partiton boundaries. This is intended to be called by the {@link org.apache.druid.msq.indexing.ControllerChatHandler}. */ @Override - public void updateStatus(int stageNumber, int workerNumber, Object keyStatisticsObject) + public void updatePartialKeyStatisticsInformation(int stageNumber, int workerNumber, Object partialKeyStatisticsInformationObject) { addToKernelManipulationQueue( queryKernel -> { @@ -582,9 +598,9 @@ public void updateStatus(int stageNumber, int workerNumber, Object keyStatistics stageDef.getShuffleSpec().get().doesAggregateByClusterKey() ); - final ClusterByStatisticsSnapshot keyStatistics; + final PartialKeyStatisticsInformation partialKeyStatisticsInformation; try { - keyStatistics = mapper.convertValue(keyStatisticsObject, ClusterByStatisticsSnapshot.class); + partialKeyStatisticsInformation = mapper.convertValue(partialKeyStatisticsInformationObject, PartialKeyStatisticsInformation.class); } catch (IllegalArgumentException e) { throw new IAE( @@ -595,7 +611,36 @@ public void updateStatus(int stageNumber, int workerNumber, Object keyStatistics ); } - queryKernel.addResultKeyStatisticsForStageAndWorker(stageId, workerNumber, keyStatistics); + queryKernel.addPartialKeyStatisticsForStageAndWorker(stageId, workerNumber, partialKeyStatisticsInformation); + + if (queryKernel.getStagePhase(stageId).equals(ControllerStagePhase.MERGING_STATISTICS)) { + List workerTaskIds = workerTaskLauncher.getTaskList(); + CompleteKeyStatisticsInformation completeKeyStatisticsInformation = + queryKernel.getCompleteKeyStatisticsInformation(stageId); + + // Queue the sketch fetching task into the worker sketch fetcher. + CompletableFuture> clusterByPartitionsCompletableFuture = + workerSketchFetcher.submitFetcherTask( + completeKeyStatisticsInformation, + workerTaskIds, + stageDef + ); + + // Add the listener to handle completion. + clusterByPartitionsCompletableFuture.whenComplete((clusterByPartitionsEither, throwable) -> { + addToKernelManipulationQueue(holder -> { + if (throwable != null) { + holder.failStageForReason(stageId, UnknownFault.forException(throwable)); + } else if (clusterByPartitionsEither.isError()) { + holder.failStageForReason(stageId, new TooManyPartitionsFault(stageDef.getMaxPartitionCount())); + } else { + log.debug("Query [%s] Partition boundaries generated for stage %s", id(), stageId); + holder.setClusterByPartitionBoundaries(stageId, clusterByPartitionsEither.valueOrThrow()); + } + holder.transitionStageKernel(stageId, queryKernel.getStagePhase(stageId)); + }); + }); + } } ); } @@ -1959,11 +2004,7 @@ public RunQueryUntilDone( this.queryDef = queryDef; this.inputSpecSlicerFactory = inputSpecSlicerFactory; this.closer = closer; - this.queryKernel = new ControllerQueryKernel( - queryDef, - WorkerMemoryParameters.createProductionInstanceForController(context.injector()) - .getPartitionStatisticsMaxRetainedBytes() - ); + this.queryKernel = new ControllerQueryKernel(queryDef); } /** diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExceptionWrappingWorkerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExceptionWrappingWorkerClient.java index 1c3cc39987c7..3d78b7c9ced4 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExceptionWrappingWorkerClient.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExceptionWrappingWorkerClient.java @@ -31,6 +31,7 @@ import org.apache.druid.msq.indexing.error.WorkerRpcFailedFault; 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; @@ -55,6 +56,23 @@ public ListenableFuture postWorkOrder(String workerTaskId, WorkOrder workO return wrap(workerTaskId, client, c -> c.postWorkOrder(workerTaskId, workOrder)); } + @Override + public ListenableFuture fetchClusterByStatisticsSnapshot(String workerTaskId, String queryId, int stageNumber) + { + return client.fetchClusterByStatisticsSnapshot(workerTaskId, queryId, stageNumber); + } + + @Override + public ListenableFuture fetchClusterByStatisticsSnapshotForTimeChunk( + String workerTaskId, + String queryId, + int stageNumber, + long timeChunk + ) + { + return client.fetchClusterByStatisticsSnapshotForTimeChunk(workerTaskId, queryId, stageNumber, timeChunk); + } + @Override public ListenableFuture postResultPartitionBoundaries( final String workerTaskId, 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 f069c91e145a..cc5f0fae1732 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 @@ -25,6 +25,7 @@ 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; @@ -67,6 +68,18 @@ public interface Worker */ void postWorkOrder(WorkOrder workOrder); + /** + * Returns the statistics snapshot for the given stageId. This is called from {@link WorkerSketchFetcher} under + * PARALLEL OR 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. + */ + ClusterByStatisticsSnapshot fetchStatisticsSnapshotForTimeChunk(StageId stageId, long timeChunk); + /** * Called when the worker chat handler recieves the result partition boundaries for a particular stageNumber * and queryId diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java index 8f4c7bac239a..5c02a79f89a3 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java @@ -25,6 +25,7 @@ import org.apache.druid.msq.counters.CounterSnapshotsTree; import org.apache.druid.msq.kernel.StageId; import org.apache.druid.msq.kernel.WorkOrder; +import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; import java.io.IOException; @@ -38,6 +39,27 @@ public interface WorkerClient extends AutoCloseable */ ListenableFuture postWorkOrder(String workerTaskId, WorkOrder workOrder); + /** + * Fetches the {@link ClusterByStatisticsSnapshot} from a worker. This is intended to be used by the + * {@link WorkerSketchFetcher} under PARALLEL or AUTO modes. + */ + ListenableFuture fetchClusterByStatisticsSnapshot( + String workerTaskId, + String queryId, + int stageNumber + ); + + /** + * Fetches a {@link ClusterByStatisticsSnapshot} which contains only the sketch of the specified timeChunk. + * This is intended to be used by the {@link WorkerSketchFetcher} under SEQUENTIAL or AUTO modes. + */ + ListenableFuture fetchClusterByStatisticsSnapshotForTimeChunk( + String workerTaskId, + String queryId, + int stageNumber, + long timeChunk + ); + /** * Worker's client method to inform it of the partition boundaries for the given stage. This is usually invoked by the * controller after collating the result statistics from all the workers processing the query 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 5b68041d0e97..49d6f9080d7c 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 @@ -106,6 +106,7 @@ import org.apache.druid.msq.shuffle.WorkerInputChannelFactory; import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; +import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation; import org.apache.druid.msq.util.DecoratedExecutorService; import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.PrioritizedCallable; @@ -159,6 +160,7 @@ public class WorkerImpl implements Worker 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 boolean durableStageStorageEnabled; /** @@ -365,10 +367,14 @@ public Optional runTask(final Closer closer) throws Exception if (kernel.getPhase() == WorkerStagePhase.READING_INPUT && kernel.hasResultKeyStatisticsSnapshot()) { if (controllerAlive) { - controllerClient.postKeyStatistics( + PartialKeyStatisticsInformation partialKeyStatisticsInformation = + kernel.getResultKeyStatisticsSnapshot() + .partialKeyStatistics(); + + controllerClient.postPartialKeyStatistics( stageDefinition.getId(), kernel.getWorkOrder().getWorkerNumber(), - kernel.getResultKeyStatisticsSnapshot() + partialKeyStatisticsInformation ); } kernel.startPreshuffleWaitingForResultPartitionBoundaries(); @@ -562,6 +568,19 @@ public void postFinish() kernelManipulationQueue.add(KernelHolder::setDone); } + @Override + public ClusterByStatisticsSnapshot fetchStatisticsSnapshot(StageId stageId) + { + return stageKernelMap.get(stageId).getResultKeyStatisticsSnapshot(); + } + + @Override + public ClusterByStatisticsSnapshot fetchStatisticsSnapshotForTimeChunk(StageId stageId, long timeChunk) + { + ClusterByStatisticsSnapshot snapshot = stageKernelMap.get(stageId).getResultKeyStatisticsSnapshot(); + return snapshot.getSnapshotForTimeChunk(timeChunk); + } + @Override public CounterSnapshotsTree getCounters() { @@ -1273,9 +1292,8 @@ public ReadableFrameChannel openChannel(StageId stageId, int workerNumber, int p } } - private static class KernelHolder + private class KernelHolder { - private final Map stageKernelMap = new HashMap<>(); private boolean done = false; public Map getStageKernelMap() diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java new file mode 100644 index 000000000000..3482b50daaff --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java @@ -0,0 +1,340 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.exec; + +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.frame.key.ClusterByPartition; +import org.apache.druid.frame.key.ClusterByPartitions; +import org.apache.druid.java.util.common.Either; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; +import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; +import org.apache.druid.msq.statistics.CompleteKeyStatisticsInformation; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.stream.IntStream; + +/** + * Queues up fetching sketches from workers and progressively generates partitions boundaries. + */ +public class WorkerSketchFetcher +{ + private static final Logger log = new Logger(WorkerSketchFetcher.class); + private static final int DEFAULT_THREAD_COUNT = 4; + // If the combined size of worker sketches is more than this threshold, SEQUENTIAL merging mode is used. + static final long BYTES_THRESHOLD = 1_000_000_000L; + // If there are more workers than this threshold, SEQUENTIAL merging mode is used. + static final long WORKER_THRESHOLD = 100; + + private final ClusterStatisticsMergeMode clusterStatisticsMergeMode; + private final int statisticsMaxRetainedBytes; + private final WorkerClient workerClient; + private final ExecutorService executorService; + + public WorkerSketchFetcher(WorkerClient workerClient, ClusterStatisticsMergeMode clusterStatisticsMergeMode, int statisticsMaxRetainedBytes) + { + this.workerClient = workerClient; + this.clusterStatisticsMergeMode = clusterStatisticsMergeMode; + this.executorService = Executors.newFixedThreadPool(DEFAULT_THREAD_COUNT); + this.statisticsMaxRetainedBytes = statisticsMaxRetainedBytes; + } + + /** + * Submits a request to fetch and generate partitions for the given worker statistics and returns a future for it. It + * decides based on the statistics if it should fetch sketches one by one or together. + */ + public CompletableFuture> submitFetcherTask( + CompleteKeyStatisticsInformation completeKeyStatisticsInformation, + List workerTaskIds, + StageDefinition stageDefinition + ) + { + ClusterBy clusterBy = stageDefinition.getClusterBy(); + + switch (clusterStatisticsMergeMode) { + case SEQUENTIAL: + return sequentialTimeChunkMerging(completeKeyStatisticsInformation, stageDefinition, workerTaskIds); + case PARALLEL: + return inMemoryFullSketchMerging(stageDefinition, workerTaskIds); + case AUTO: + if (clusterBy.getBucketByCount() == 0) { + log.debug("Query [%s] AUTO mode: chose PARALLEL mode to merge key statistics", stageDefinition.getId().getQueryId()); + // If there is no time clustering, there is no scope for sequential merge + return inMemoryFullSketchMerging(stageDefinition, workerTaskIds); + } else if (stageDefinition.getMaxWorkerCount() > WORKER_THRESHOLD || completeKeyStatisticsInformation.getBytesRetained() > BYTES_THRESHOLD) { + log.debug("Query [%s] AUTO mode: chose SEQUENTIAL mode to merge key statistics", stageDefinition.getId().getQueryId()); + return sequentialTimeChunkMerging(completeKeyStatisticsInformation, stageDefinition, workerTaskIds); + } + log.debug("Query [%s] AUTO mode: chose PARALLEL mode to merge key statistics", stageDefinition.getId().getQueryId()); + return inMemoryFullSketchMerging(stageDefinition, workerTaskIds); + default: + throw new IllegalStateException("No fetching strategy found for mode: " + clusterStatisticsMergeMode); + } + } + + /** + * Fetches the full {@link ClusterByStatisticsCollector} from all workers and generates partition boundaries from them. + * This is faster than fetching them timechunk by timechunk but the collector will be downsampled till it can fit + * on the controller, resulting in less accurate partition boundries. + */ + CompletableFuture> inMemoryFullSketchMerging( + StageDefinition stageDefinition, + List workerTaskIds + ) + { + CompletableFuture> partitionFuture = new CompletableFuture<>(); + + // Create a new key statistics collector to merge worker sketches into + final ClusterByStatisticsCollector mergedStatisticsCollector = + stageDefinition.createResultKeyStatisticsCollector(statisticsMaxRetainedBytes); + final int workerCount = workerTaskIds.size(); + // Guarded by synchronized mergedStatisticsCollector + final Set finishedWorkers = new HashSet<>(); + + // Submit a task for each worker to fetch statistics + IntStream.range(0, workerCount).forEach(workerNo -> { + executorService.submit(() -> { + ListenableFuture snapshotFuture = + workerClient.fetchClusterByStatisticsSnapshot( + workerTaskIds.get(workerNo), + stageDefinition.getId().getQueryId(), + stageDefinition.getStageNumber() + ); + partitionFuture.whenComplete((result, exception) -> { + if (exception != null || (result != null && result.isError())) { + snapshotFuture.cancel(true); + } + }); + + try { + ClusterByStatisticsSnapshot clusterByStatisticsSnapshot = snapshotFuture.get(); + if (clusterByStatisticsSnapshot == null) { + throw new ISE("Worker %s returned null sketch, this should never happen", workerNo); + } + synchronized (mergedStatisticsCollector) { + mergedStatisticsCollector.addAll(clusterByStatisticsSnapshot); + finishedWorkers.add(workerNo); + + if (finishedWorkers.size() == workerCount) { + log.debug("Query [%s] Received all statistics, generating partitions", stageDefinition.getId().getQueryId()); + partitionFuture.complete(stageDefinition.generatePartitionsForShuffle(mergedStatisticsCollector)); + } + } + } + catch (Exception e) { + synchronized (mergedStatisticsCollector) { + partitionFuture.completeExceptionally(e); + mergedStatisticsCollector.clear(); + } + } + }); + }); + return partitionFuture; + } + + /** + * Fetches cluster statistics from all workers and generates partition boundaries from them one time chunk at a time. + * This takes longer due to the overhead of fetching sketches, however, this prevents any loss in accuracy from + * downsampling on the controller. + */ + CompletableFuture> sequentialTimeChunkMerging( + CompleteKeyStatisticsInformation completeKeyStatisticsInformation, + StageDefinition stageDefinition, + List workerTaskIds + ) + { + SequentialFetchStage sequentialFetchStage = new SequentialFetchStage( + stageDefinition, + workerTaskIds, + completeKeyStatisticsInformation.getTimeSegmentVsWorkerMap().entrySet().iterator() + ); + sequentialFetchStage.submitFetchingTasksForNextTimeChunk(); + return sequentialFetchStage.getPartitionFuture(); + } + + private class SequentialFetchStage + { + private final StageDefinition stageDefinition; + private final List workerTaskIds; + private final Iterator>> timeSegmentVsWorkerIdIterator; + private final CompletableFuture> partitionFuture; + // Final sorted list of partition boundaries. This is appended to after statistics for each time chunk are gathered. + private final List finalPartitionBoundries; + + public SequentialFetchStage( + StageDefinition stageDefinition, + List workerTaskIds, + Iterator>> timeSegmentVsWorkerIdIterator + ) + { + this.finalPartitionBoundries = new ArrayList<>(); + this.stageDefinition = stageDefinition; + this.workerTaskIds = workerTaskIds; + this.timeSegmentVsWorkerIdIterator = timeSegmentVsWorkerIdIterator; + this.partitionFuture = new CompletableFuture<>(); + } + + /** + * Submits the tasks to fetch key statistics for the time chunk pointed to by {@link #timeSegmentVsWorkerIdIterator}. + * Once the statistics have been gathered from all workers which have them, generates partitions and adds it to + * {@link #finalPartitionBoundries}, stiching the partitions between time chunks using + * {@link #abutAndAppendPartitionBoundries(List, List)} to make them continuous. + * + * The time chunks returned by {@link #timeSegmentVsWorkerIdIterator} should be in ascending order for the partitions + * to be generated correctly. + * + * If {@link #timeSegmentVsWorkerIdIterator} doesn't have any more values, assumes that partition boundaries have + * been successfully generated and completes {@link #partitionFuture} with the result. + * + * Completes the future with an error as soon as the number of partitions exceed max partition count for the stage + * definition. + */ + public void submitFetchingTasksForNextTimeChunk() + { + if (!timeSegmentVsWorkerIdIterator.hasNext()) { + partitionFuture.complete(Either.value(new ClusterByPartitions(finalPartitionBoundries))); + } else { + Map.Entry> entry = timeSegmentVsWorkerIdIterator.next(); + // Time chunk for which partition boundries are going to be generated for + Long timeChunk = entry.getKey(); + Set workerIdsWithTimeChunk = entry.getValue(); + // Create a new key statistics collector to merge worker sketches into + ClusterByStatisticsCollector mergedStatisticsCollector = + stageDefinition.createResultKeyStatisticsCollector(statisticsMaxRetainedBytes); + // Guarded by synchronized mergedStatisticsCollector + Set finishedWorkers = new HashSet<>(); + + log.debug("Query [%s]. Submitting request for statistics for time chunk %s to %s workers", + stageDefinition.getId().getQueryId(), + timeChunk, + workerIdsWithTimeChunk.size()); + + // Submits a task for every worker which has a certain time chunk + for (int workerNo : workerIdsWithTimeChunk) { + executorService.submit(() -> { + ListenableFuture snapshotFuture = + workerClient.fetchClusterByStatisticsSnapshotForTimeChunk( + workerTaskIds.get(workerNo), + stageDefinition.getId().getQueryId(), + stageDefinition.getStageNumber(), + timeChunk + ); + partitionFuture.whenComplete((result, exception) -> { + if (exception != null || (result != null && result.isError())) { + snapshotFuture.cancel(true); + } + }); + + try { + ClusterByStatisticsSnapshot snapshotForTimeChunk = snapshotFuture.get(); + if (snapshotForTimeChunk == null) { + throw new ISE("Worker %s returned null sketch for %s, this should never happen", workerNo, timeChunk); + } + synchronized (mergedStatisticsCollector) { + mergedStatisticsCollector.addAll(snapshotForTimeChunk); + finishedWorkers.add(workerNo); + + if (finishedWorkers.size() == workerIdsWithTimeChunk.size()) { + Either longClusterByPartitionsEither = + stageDefinition.generatePartitionsForShuffle(mergedStatisticsCollector); + + log.debug("Query [%s]. Received all statistics for time chunk %s, generating partitions", + stageDefinition.getId().getQueryId(), + timeChunk); + + long totalPartitionCount = finalPartitionBoundries.size() + getPartitionCountFromEither(longClusterByPartitionsEither); + if (totalPartitionCount > stageDefinition.getMaxPartitionCount()) { + // Fail fast if more partitions than the maximum have been reached. + partitionFuture.complete(Either.error(totalPartitionCount)); + mergedStatisticsCollector.clear(); + } else { + List timeSketchPartitions = longClusterByPartitionsEither.valueOrThrow().ranges(); + abutAndAppendPartitionBoundries(finalPartitionBoundries, timeSketchPartitions); + log.debug("Query [%s]. Finished generating partitions for time chunk %s, total count so far %s", + stageDefinition.getId().getQueryId(), + timeChunk, + finalPartitionBoundries.size()); + submitFetchingTasksForNextTimeChunk(); + } + } + } + } + catch (Exception e) { + synchronized (mergedStatisticsCollector) { + partitionFuture.completeExceptionally(e); + mergedStatisticsCollector.clear(); + } + } + }); + } + } + } + + /** + * Takes a list of sorted {@link ClusterByPartitions} {@param timeSketchPartitions} and adds it to a sorted list + * {@param finalPartitionBoundries}. If {@param finalPartitionBoundries} is not empty, the end time of the last + * partition of {@param finalPartitionBoundries} is changed to abut with the starting time of the first partition + * of {@param timeSketchPartitions}. + * + * This is used to make the partitions generated continuous. + */ + private void abutAndAppendPartitionBoundries( + List finalPartitionBoundries, + List timeSketchPartitions + ) + { + if (!finalPartitionBoundries.isEmpty()) { + // Stitch up the end time of the last partition with the start time of the first partition. + ClusterByPartition clusterByPartition = finalPartitionBoundries.remove(finalPartitionBoundries.size() - 1); + finalPartitionBoundries.add(new ClusterByPartition(clusterByPartition.getStart(), timeSketchPartitions.get(0).getStart())); + } + finalPartitionBoundries.addAll(timeSketchPartitions); + } + + public CompletableFuture> getPartitionFuture() + { + return partitionFuture; + } + } + + /** + * Gets the partition size from an {@link Either}. If it is an error, the long denotes the number of partitions + * (in the case of creating too many partitions), otherwise checks the size of the list. + */ + private static long getPartitionCountFromEither(Either either) + { + if (either.isError()) { + return either.error(); + } else { + return either.valueOrThrow().size(); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ControllerChatHandler.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ControllerChatHandler.java index 1857d83708c7..8fa04ce6d90b 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ControllerChatHandler.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ControllerChatHandler.java @@ -26,7 +26,8 @@ import org.apache.druid.msq.exec.Controller; import org.apache.druid.msq.exec.ControllerClient; import org.apache.druid.msq.indexing.error.MSQErrorReport; -import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation; import org.apache.druid.segment.realtime.firehose.ChatHandler; import org.apache.druid.segment.realtime.firehose.ChatHandlers; import org.apache.druid.server.security.Action; @@ -58,16 +59,17 @@ public ControllerChatHandler(TaskToolbox toolbox, Controller controller) } /** - * Used by subtasks to post {@link ClusterByStatisticsSnapshot} for shuffling stages. + * Used by subtasks to post {@link PartialKeyStatisticsInformation} for shuffling stages. * - * See {@link ControllerClient#postKeyStatistics} for the client-side code that calls this API. + * See {@link ControllerClient#postPartialKeyStatistics(StageId, int, PartialKeyStatisticsInformation)} + * for the client-side code that calls this API. */ @POST - @Path("/keyStatistics/{queryId}/{stageNumber}/{workerNumber}") + @Path("/partialKeyStatisticsInformation/{queryId}/{stageNumber}/{workerNumber}") @Produces(MediaType.APPLICATION_JSON) @Consumes(MediaType.APPLICATION_JSON) - public Response httpPostKeyStatistics( - final Object keyStatisticsObject, + public Response httpPostPartialKeyStatistics( + final Object partialKeyStatisticsObject, @PathParam("queryId") final String queryId, @PathParam("stageNumber") final int stageNumber, @PathParam("workerNumber") final int workerNumber, @@ -75,7 +77,7 @@ public Response httpPostKeyStatistics( ) { ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper()); - controller.updateStatus(stageNumber, workerNumber, keyStatisticsObject); + controller.updatePartialKeyStatisticsInformation(stageNumber, workerNumber, partialKeyStatisticsObject); return Response.status(Response.Status.ACCEPTED).build(); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerClient.java index 3a6c9e78797b..1ff5952a8958 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerClient.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerClient.java @@ -29,7 +29,7 @@ import org.apache.druid.msq.exec.ControllerClient; import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.apache.druid.msq.kernel.StageId; -import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; +import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation; import org.apache.druid.rpc.IgnoreHttpResponseHandler; import org.apache.druid.rpc.RequestBuilder; import org.apache.druid.rpc.ServiceClient; @@ -59,14 +59,14 @@ public IndexerControllerClient( } @Override - public void postKeyStatistics( + public void postPartialKeyStatistics( StageId stageId, int workerNumber, - ClusterByStatisticsSnapshot keyStatistics + PartialKeyStatisticsInformation partialKeyStatisticsInformation ) throws IOException { final String path = StringUtils.format( - "/keyStatistics/%s/%s/%d", + "/partialKeyStatisticsInformation/%s/%d/%d", StringUtils.urlEncode(stageId.getQueryId()), stageId.getStageNumber(), workerNumber @@ -74,7 +74,7 @@ public void postKeyStatistics( doRequest( new RequestBuilder(HttpMethod.POST, path) - .jsonContent(jsonMapper, keyStatistics), + .jsonContent(jsonMapper, partialKeyStatisticsInformation), IgnoreHttpResponseHandler.INSTANCE ); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerClient.java index 430c264291f8..980c7f97bee5 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerClient.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerClient.java @@ -41,6 +41,7 @@ import org.apache.druid.msq.exec.WorkerClient; import org.apache.druid.msq.kernel.StageId; import org.apache.druid.msq.kernel.WorkOrder; +import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; import org.apache.druid.rpc.IgnoreHttpResponseHandler; import org.apache.druid.rpc.RequestBuilder; import org.apache.druid.rpc.ServiceClient; @@ -103,6 +104,48 @@ public ListenableFuture postWorkOrder(String workerTaskId, WorkOrder workO ); } + @Override + public ListenableFuture fetchClusterByStatisticsSnapshot( + String workerTaskId, + String queryId, + int stageNumber + ) + { + String path = StringUtils.format("/keyStatistics/%s/%d", + StringUtils.urlEncode(queryId), + stageNumber); + + return FutureUtils.transform( + getClient(workerTaskId).asyncRequest( + new RequestBuilder(HttpMethod.POST, path), + new BytesFullResponseHandler() + ), + holder -> deserialize(holder, new TypeReference() {}) + ); + } + + @Override + public ListenableFuture fetchClusterByStatisticsSnapshotForTimeChunk( + String workerTaskId, + String queryId, + int stageNumber, + long timeChunk + ) + { + String path = StringUtils.format("/keyStatisticsForTimeChunk/%s/%d/%d", + StringUtils.urlEncode(queryId), + stageNumber, + timeChunk); + + return FutureUtils.transform( + getClient(workerTaskId).asyncRequest( + new RequestBuilder(HttpMethod.POST, path), + new BytesFullResponseHandler() + ), + holder -> deserialize(holder, new TypeReference() {}) + ); + } + @Override public ListenableFuture postResultPartitionBoundaries( String workerTaskId, diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/WorkerChatHandler.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/WorkerChatHandler.java index 06fd8f36caf7..dd6ea7cb7124 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/WorkerChatHandler.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/WorkerChatHandler.java @@ -28,6 +28,7 @@ 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.segment.realtime.firehose.ChatHandler; import org.apache.druid.segment.realtime.firehose.ChatHandlers; import org.apache.druid.server.security.Action; @@ -179,6 +180,45 @@ public Response httpPostResultPartitionBoundaries( } } + @POST + @Path("/keyStatistics/{queryId}/{stageNumber}") + @Produces(MediaType.APPLICATION_JSON) + @Consumes(MediaType.APPLICATION_JSON) + public Response httpFetchKeyStatistics( + @PathParam("queryId") final String queryId, + @PathParam("stageNumber") final int stageNumber, + @Context final HttpServletRequest req + ) + { + ChatHandlers.authorizationCheck(req, Action.READ, task.getDataSource(), toolbox.getAuthorizerMapper()); + ClusterByStatisticsSnapshot clusterByStatisticsSnapshot; + StageId stageId = new StageId(queryId, stageNumber); + clusterByStatisticsSnapshot = worker.fetchStatisticsSnapshot(stageId); + return Response.status(Response.Status.ACCEPTED) + .entity(clusterByStatisticsSnapshot) + .build(); + } + + @POST + @Path("/keyStatisticsForTimeChunk/{queryId}/{stageNumber}/{timeChunk}") + @Produces(MediaType.APPLICATION_JSON) + @Consumes(MediaType.APPLICATION_JSON) + public Response httpSketch( + @PathParam("queryId") final String queryId, + @PathParam("stageNumber") final int stageNumber, + @PathParam("timeChunk") final long timeChunk, + @Context final HttpServletRequest req + ) + { + ChatHandlers.authorizationCheck(req, Action.READ, task.getDataSource(), toolbox.getAuthorizerMapper()); + ClusterByStatisticsSnapshot snapshotForTimeChunk; + StageId stageId = new StageId(queryId, stageNumber); + snapshotForTimeChunk = worker.fetchStatisticsSnapshotForTimeChunk(stageId, timeChunk); + return Response.status(Response.Status.ACCEPTED) + .entity(snapshotForTimeChunk) + .build(); + } + /** * See {@link org.apache.druid.msq.exec.WorkerClient#postCleanupStage} for the client-side code that calls this API. */ diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java index 9de5c692c990..d2e9f198f04b 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java @@ -41,7 +41,8 @@ import org.apache.druid.msq.kernel.StageId; import org.apache.druid.msq.kernel.WorkOrder; import org.apache.druid.msq.kernel.WorkerAssignmentStrategy; -import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; +import org.apache.druid.msq.statistics.CompleteKeyStatisticsInformation; +import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation; import javax.annotation.Nullable; import java.util.HashMap; @@ -65,7 +66,6 @@ public class ControllerQueryKernel { private final QueryDefinition queryDef; - private final int partitionStatisticsMaxRetainedBytes; /** * Stage ID -> tracker for that stage. An extension of the state of this kernel. @@ -107,10 +107,9 @@ public class ControllerQueryKernel */ private final Set effectivelyFinishedStages = new HashSet<>(); - public ControllerQueryKernel(final QueryDefinition queryDef, final int partitionStatisticsMaxRetainedBytes) + public ControllerQueryKernel(final QueryDefinition queryDef) { this.queryDef = queryDef; - this.partitionStatisticsMaxRetainedBytes = partitionStatisticsMaxRetainedBytes; this.inflowMap = ImmutableMap.copyOf(computeStageInflowMap(queryDef)); this.outflowMap = ImmutableMap.copyOf(computeStageOutflowMap(queryDef)); @@ -266,8 +265,7 @@ private void createNewKernels( stageDef, stageWorkerCountMap, slicer, - assignmentStrategy, - partitionStatisticsMaxRetainedBytes + assignmentStrategy ); stageTracker.put(nextStage, stageKernel); } @@ -334,6 +332,22 @@ public ClusterByPartitions getResultPartitionBoundariesForStage(final StageId st return getStageKernelOrThrow(stageId).getResultPartitionBoundaries(); } + /** + * Delegates call to {@link ControllerStageTracker#getCompleteKeyStatisticsInformation()} + */ + public CompleteKeyStatisticsInformation getCompleteKeyStatisticsInformation(final StageId stageId) + { + return getStageKernelOrThrow(stageId).getCompleteKeyStatisticsInformation(); + } + + /** + * Delegates call to {@link ControllerStageTracker#setClusterByPartitionBoundaries(ClusterByPartitions)} ()} + */ + public void setClusterByPartitionBoundaries(final StageId stageId, ClusterByPartitions clusterByPartitions) + { + getStageKernelOrThrow(stageId).setClusterByPartitionBoundaries(clusterByPartitions); + } + /** * Delegates call to {@link ControllerStageTracker#collectorEncounteredAnyMultiValueField()} */ @@ -390,22 +404,24 @@ public WorkerInputs getWorkerInputsForStage(final StageId stageId) } /** - * Delegates call to {@link ControllerStageTracker#addResultKeyStatisticsForWorker(int, ClusterByStatisticsSnapshot)}. + * Delegates call to {@link ControllerStageTracker#addPartialKeyStatisticsForWorker(int, PartialKeyStatisticsInformation)}. * If calling this causes transition for the stage kernel, then this gets registered in this query kernel */ - public void addResultKeyStatisticsForStageAndWorker( + public void addPartialKeyStatisticsForStageAndWorker( final StageId stageId, final int workerNumber, - final ClusterByStatisticsSnapshot snapshot + final PartialKeyStatisticsInformation partialKeyStatisticsInformation ) { - ControllerStagePhase newPhase = getStageKernelOrThrow(stageId).addResultKeyStatisticsForWorker( + ControllerStageTracker stageKernel = getStageKernelOrThrow(stageId); + ControllerStagePhase newPhase = stageKernel.addPartialKeyStatisticsForWorker( workerNumber, - snapshot + partialKeyStatisticsInformation ); - // If the phase is POST_READING or FAILED, that implies the kernel has transitioned. We need to account for that + // If the kernel phase has transitioned, we need to account for that. switch (newPhase) { + case MERGING_STATISTICS: case POST_READING: case FAILED: transitionStageKernel(stageId, newPhase); @@ -436,6 +452,12 @@ public MSQFault getFailureReasonForStage(final StageId stageId) return getStageKernelOrThrow(stageId).getFailureReason(); } + public void failStageForReason(final StageId stageId, MSQFault fault) + { + getStageKernelOrThrow(stageId).failForReason(fault); + transitionStageKernel(stageId, ControllerStagePhase.FAILED); + } + /** * Delegates call to {@link ControllerStageTracker#fail()} and registers this transition to FAILED in this query kernel */ diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStagePhase.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStagePhase.java index 6686292c1d85..896b345b4d22 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStagePhase.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStagePhase.java @@ -48,6 +48,17 @@ public boolean canTransitionFrom(final ControllerStagePhase priorPhase) } }, + // Waiting to fetch key statistics in the background from the workers and incrementally generate partitions. + // This phase is only transitioned to once all partialKeyInformation are recieved from workers. + // Transitioning to this phase should also enqueue the task to fetch key statistics to WorkerSketchFetcher. + MERGING_STATISTICS { + @Override + public boolean canTransitionFrom(final ControllerStagePhase priorPhase) + { + return priorPhase == READING_INPUT; + } + }, + // Post the inputs have been read and mapped to frames, in the `POST_READING` stage, we pre-shuffle and determing the partition boundaries. // This step for a stage spits out the statistics of the data as a whole (and not just the individual records). This // phase is not required in non-pre shuffle contexts. @@ -55,7 +66,7 @@ public boolean canTransitionFrom(final ControllerStagePhase priorPhase) @Override public boolean canTransitionFrom(final ControllerStagePhase priorPhase) { - return priorPhase == READING_INPUT; + return priorPhase == MERGING_STATISTICS; } }, diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java index 3ad01a513c80..bf53239335c5 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerStageTracker.java @@ -28,6 +28,7 @@ import org.apache.druid.java.util.common.Either; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.msq.indexing.error.InsertTimeNullFault; import org.apache.druid.msq.indexing.error.MSQFault; import org.apache.druid.msq.indexing.error.TooManyPartitionsFault; import org.apache.druid.msq.indexing.error.UnknownFault; @@ -38,11 +39,12 @@ import org.apache.druid.msq.input.stage.StageInputSlice; import org.apache.druid.msq.kernel.StageDefinition; import org.apache.druid.msq.kernel.WorkerAssignmentStrategy; -import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; -import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; +import org.apache.druid.msq.statistics.CompleteKeyStatisticsInformation; +import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation; import javax.annotation.Nullable; import java.util.List; +import java.util.TreeMap; /** * Controller-side state machine for each stage. Used by {@link ControllerQueryKernel} to form the overall state @@ -57,13 +59,13 @@ class ControllerStageTracker private final int workerCount; private final WorkerInputs workerInputs; - private final IntSet workersWithResultKeyStatistics = new IntAVLTreeSet(); + private final IntSet workersWithReportedKeyStatistics = new IntAVLTreeSet(); private final IntSet workersWithResultsComplete = new IntAVLTreeSet(); private ControllerStagePhase phase = ControllerStagePhase.NEW; @Nullable - private final ClusterByStatisticsCollector resultKeyStatisticsCollector; + public final CompleteKeyStatisticsInformation completeKeyStatisticsInformation; // Result partitions and where they can be read from. @Nullable @@ -81,8 +83,7 @@ class ControllerStageTracker private ControllerStageTracker( final StageDefinition stageDef, - final WorkerInputs workerInputs, - final int partitionStatisticsMaxRetainedBytes + final WorkerInputs workerInputs ) { this.stageDef = stageDef; @@ -90,11 +91,11 @@ private ControllerStageTracker( this.workerInputs = workerInputs; if (stageDef.mustGatherResultKeyStatistics()) { - this.resultKeyStatisticsCollector = - stageDef.createResultKeyStatisticsCollector(partitionStatisticsMaxRetainedBytes); + this.completeKeyStatisticsInformation = + new CompleteKeyStatisticsInformation(new TreeMap<>(), false, 0); } else { - this.resultKeyStatisticsCollector = null; - generateResultPartitionsAndBoundaries(); + this.completeKeyStatisticsInformation = null; + generateResultPartitionsAndBoundariesWithoutKeyStatistics(); } } @@ -107,12 +108,11 @@ static ControllerStageTracker create( final StageDefinition stageDef, final Int2IntMap stageWorkerCountMap, final InputSpecSlicer slicer, - final WorkerAssignmentStrategy assignmentStrategy, - final int partitionStatisticsMaxRetainedBytes + final WorkerAssignmentStrategy assignmentStrategy ) { final WorkerInputs workerInputs = WorkerInputs.create(stageDef, stageWorkerCountMap, slicer, assignmentStrategy); - return new ControllerStageTracker(stageDef, workerInputs, partitionStatisticsMaxRetainedBytes); + return new ControllerStageTracker(stageDef, workerInputs); } /** @@ -175,18 +175,12 @@ ClusterByPartitions getResultPartitionBoundaries() */ boolean collectorEncounteredAnyMultiValueField() { - if (resultKeyStatisticsCollector == null) { + if (completeKeyStatisticsInformation == null) { throw new ISE("Stage does not gather result key statistics"); - } else if (resultPartitions == null) { + } else if (workersWithReportedKeyStatistics.size() != workerCount) { throw new ISE("Result key statistics are not ready"); } else { - for (int i = 0; i < resultKeyStatisticsCollector.getClusterBy().getColumns().size(); i++) { - if (resultKeyStatisticsCollector.hasMultipleValues(i)) { - return true; - } - } - - return false; + return completeKeyStatisticsInformation.hasMultipleValues(); } } @@ -219,10 +213,6 @@ void start() */ void finish() { - if (resultKeyStatisticsCollector != null) { - resultKeyStatisticsCollector.clear(); - } - transitionTo(ControllerStagePhase.FINISHED); } @@ -234,23 +224,31 @@ WorkerInputs getWorkerInputs() return workerInputs; } + /** + * Returns the merged key statistics. + */ + @Nullable + public CompleteKeyStatisticsInformation getCompleteKeyStatisticsInformation() + { + return completeKeyStatisticsInformation; + } + /** * Adds result key statistics for a particular worker number. If statistics have already been added for this worker, * then this call ignores the new ones and does nothing. * * @param workerNumber the worker - * @param snapshot worker statistics + * @param partialKeyStatisticsInformation partial key statistics */ - ControllerStagePhase addResultKeyStatisticsForWorker( + ControllerStagePhase addPartialKeyStatisticsForWorker( final int workerNumber, - final ClusterByStatisticsSnapshot snapshot + final PartialKeyStatisticsInformation partialKeyStatisticsInformation ) { if (phase != ControllerStagePhase.READING_INPUT) { throw new ISE("Cannot add result key statistics from stage [%s]", phase); } - - if (resultKeyStatisticsCollector == null) { + if (!stageDef.mustGatherResultKeyStatistics() || !stageDef.doesShuffle() || completeKeyStatisticsInformation == null) { throw new ISE("Stage does not gather result key statistics"); } @@ -259,16 +257,21 @@ ControllerStagePhase addResultKeyStatisticsForWorker( } try { - if (workersWithResultKeyStatistics.add(workerNumber)) { - resultKeyStatisticsCollector.addAll(snapshot); + if (workersWithReportedKeyStatistics.add(workerNumber)) { - if (workersWithResultKeyStatistics.size() == workerCount) { - generateResultPartitionsAndBoundaries(); + if (partialKeyStatisticsInformation.getTimeSegments().contains(null)) { + // Time should not contain null value + failForReason(InsertTimeNullFault.instance()); + return getPhase(); + } + + completeKeyStatisticsInformation.mergePartialInformation(workerNumber, partialKeyStatisticsInformation); + + if (workersWithReportedKeyStatistics.size() == workerCount) { + // All workers have sent the partial key statistics information. + // Transition to MERGING_STATISTICS state to queue fetch clustering statistics from workers. + transitionTo(ControllerStagePhase.MERGING_STATISTICS); - // Phase can become FAILED after generateResultPartitionsAndBoundaries, if there were too many partitions. - if (phase != ControllerStagePhase.FAILED) { - transitionTo(ControllerStagePhase.POST_READING); - } } } } @@ -280,6 +283,33 @@ ControllerStagePhase addResultKeyStatisticsForWorker( return getPhase(); } + /** + * Sets the {@link #resultPartitions} and {@link #resultPartitionBoundaries} and transitions the phase to POST_READING. + */ + void setClusterByPartitionBoundaries(ClusterByPartitions clusterByPartitions) + { + if (resultPartitions != null) { + throw new ISE("Result partitions have already been generated"); + } + + if (!stageDef.mustGatherResultKeyStatistics()) { + throw new ISE("Result partitions does not require key statistics, should not have set partition boundries here"); + } + + if (!ControllerStagePhase.MERGING_STATISTICS.equals(getPhase())) { + throw new ISE("Cannot set partition boundires from key statistics from stage [%s]", getPhase()); + } + + this.resultPartitionBoundaries = clusterByPartitions; + this.resultPartitions = ReadablePartitions.striped( + stageDef.getStageNumber(), + workerCount, + clusterByPartitions.size() + ); + + transitionTo(ControllerStagePhase.POST_READING); + } + /** * Accepts and sets the results that each worker produces for this particular stage * @@ -339,12 +369,11 @@ void fail() } /** - * Sets {@link #resultPartitions} (always) and {@link #resultPartitionBoundaries}. + * Sets {@link #resultPartitions} (always) and {@link #resultPartitionBoundaries} without using key statistics. * - * If {@link StageDefinition#mustGatherResultKeyStatistics()} is true, this method cannot be called until after - * statistics have been provided to {@link #addResultKeyStatisticsForWorker} for all workers. + * If {@link StageDefinition#mustGatherResultKeyStatistics()} is true, this method should not be called. */ - private void generateResultPartitionsAndBoundaries() + private void generateResultPartitionsAndBoundariesWithoutKeyStatistics() { if (resultPartitions != null) { throw new ISE("Result partitions have already been generated"); @@ -353,12 +382,12 @@ private void generateResultPartitionsAndBoundaries() final int stageNumber = stageDef.getStageNumber(); if (stageDef.doesShuffle()) { - if (stageDef.mustGatherResultKeyStatistics() && workersWithResultKeyStatistics.size() != workerCount) { - throw new ISE("Cannot generate result partitions without all worker statistics"); + if (stageDef.mustGatherResultKeyStatistics()) { + throw new ISE("Cannot generate result partitions without key statistics"); } final Either maybeResultPartitionBoundaries = - stageDef.generatePartitionsForShuffle(resultKeyStatisticsCollector); + stageDef.generatePartitionsForShuffle(null); if (maybeResultPartitionBoundaries.isError()) { failForReason(new TooManyPartitionsFault(stageDef.getMaxPartitionCount())); @@ -397,15 +426,11 @@ private void generateResultPartitionsAndBoundaries() * * @param fault reason why this stage has failed */ - private void failForReason(final MSQFault fault) + void failForReason(final MSQFault fault) { transitionTo(ControllerStagePhase.FAILED); this.failureReason = fault; - - if (resultKeyStatisticsCollector != null) { - resultKeyStatisticsCollector.clear(); - } } void transitionTo(final ControllerStagePhase newPhase) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollectorImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollectorImpl.java index 13ab5dc01cee..bd010c555f17 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollectorImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollectorImpl.java @@ -35,6 +35,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.SortedMap; @@ -56,7 +57,7 @@ public class ClusterByStatisticsCollectorImpl implements ClusterByStatisticsColl private final boolean[] hasMultipleValues; - private final int maxRetainedBytes; + private final long maxRetainedBytes; private final int maxBuckets; private long totalRetainedBytes; @@ -64,7 +65,7 @@ private ClusterByStatisticsCollectorImpl( final ClusterBy clusterBy, final RowKeyReader keyReader, final KeyCollectorFactory keyCollectorFactory, - final int maxRetainedBytes, + final long maxRetainedBytes, final int maxBuckets, final boolean checkHasMultipleValues ) @@ -86,7 +87,7 @@ private ClusterByStatisticsCollectorImpl( public static ClusterByStatisticsCollector create( final ClusterBy clusterBy, final RowSignature signature, - final int maxRetainedBytes, + final long maxRetainedBytes, final int maxBuckets, final boolean aggregate, final boolean checkHasMultipleValues @@ -167,7 +168,7 @@ public ClusterByStatisticsCollector addAll(final ClusterByStatisticsCollector ot public ClusterByStatisticsCollector addAll(final ClusterByStatisticsSnapshot snapshot) { // Add all key collectors from the other collector. - for (ClusterByStatisticsSnapshot.Bucket otherBucket : snapshot.getBuckets()) { + for (ClusterByStatisticsSnapshot.Bucket otherBucket : snapshot.getBuckets().values()) { //noinspection rawtypes, unchecked final KeyCollector otherKeyCollector = ((KeyCollectorFactory) keyCollectorFactory).fromSnapshot(otherBucket.getKeyCollectorSnapshot()); @@ -315,13 +316,20 @@ public ClusterByStatisticsSnapshot snapshot() { assertRetainedByteCountsAreTrackedCorrectly(); - final List bucketSnapshots = new ArrayList<>(); + final Map bucketSnapshots = new HashMap<>(); + final RowKeyReader trimmedRowReader = keyReader.trimmedKeyReader(clusterBy.getBucketByCount()); for (final Map.Entry bucketEntry : buckets.entrySet()) { //noinspection rawtypes, unchecked final KeyCollectorSnapshot keyCollectorSnapshot = ((KeyCollectorFactory) keyCollectorFactory).toSnapshot(bucketEntry.getValue().keyCollector); - bucketSnapshots.add(new ClusterByStatisticsSnapshot.Bucket(bucketEntry.getKey(), keyCollectorSnapshot)); + Long bucketKey = Long.MIN_VALUE; + + // If there is a clustering on time, read the first field from each bucket and add it to the snapshots. + if (clusterBy.getBucketByCount() == 1) { + bucketKey = (Long) trimmedRowReader.read(bucketEntry.getKey(), 0); + } + bucketSnapshots.put(bucketKey, new ClusterByStatisticsSnapshot.Bucket(bucketEntry.getKey(), keyCollectorSnapshot, totalRetainedBytes)); } final IntSet hasMultipleValuesSet; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsSnapshot.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsSnapshot.java index 6b16890254a0..e54253ad2186 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsSnapshot.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsSnapshot.java @@ -23,22 +23,23 @@ import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; import org.apache.druid.frame.key.RowKey; import javax.annotation.Nullable; import java.util.Collections; -import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Set; public class ClusterByStatisticsSnapshot { - private final List buckets; + private final Map buckets; private final Set hasMultipleValues; @JsonCreator ClusterByStatisticsSnapshot( - @JsonProperty("buckets") final List buckets, + @JsonProperty("buckets") final Map buckets, @JsonProperty("hasMultipleValues") @Nullable final Set hasMultipleValues ) { @@ -48,15 +49,21 @@ public class ClusterByStatisticsSnapshot public static ClusterByStatisticsSnapshot empty() { - return new ClusterByStatisticsSnapshot(Collections.emptyList(), null); + return new ClusterByStatisticsSnapshot(Collections.emptyMap(), null); } @JsonProperty("buckets") - List getBuckets() + Map getBuckets() { return buckets; } + public ClusterByStatisticsSnapshot getSnapshotForTimeChunk(long timeChunk) + { + Bucket bucket = buckets.get(timeChunk); + return new ClusterByStatisticsSnapshot(ImmutableMap.of(timeChunk, bucket), null); + } + @JsonProperty("hasMultipleValues") @JsonInclude(JsonInclude.Include.NON_EMPTY) Set getHasMultipleValues() @@ -64,6 +71,15 @@ Set getHasMultipleValues() return hasMultipleValues; } + public PartialKeyStatisticsInformation partialKeyStatistics() + { + double bytesRetained = 0; + for (ClusterByStatisticsSnapshot.Bucket bucket : buckets.values()) { + bytesRetained += bucket.bytesRetained; + } + return new PartialKeyStatisticsInformation(buckets.keySet(), !getHasMultipleValues().isEmpty(), bytesRetained); + } + @Override public boolean equals(Object o) { @@ -86,16 +102,19 @@ public int hashCode() static class Bucket { private final RowKey bucketKey; + private final double bytesRetained; private final KeyCollectorSnapshot keyCollectorSnapshot; @JsonCreator Bucket( @JsonProperty("bucketKey") RowKey bucketKey, - @JsonProperty("data") KeyCollectorSnapshot keyCollectorSnapshot + @JsonProperty("data") KeyCollectorSnapshot keyCollectorSnapshot, + @JsonProperty("bytesRetained") double bytesRetained ) { this.bucketKey = Preconditions.checkNotNull(bucketKey, "bucketKey"); this.keyCollectorSnapshot = Preconditions.checkNotNull(keyCollectorSnapshot, "data"); + this.bytesRetained = bytesRetained; } @JsonProperty diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/CompleteKeyStatisticsInformation.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/CompleteKeyStatisticsInformation.java new file mode 100644 index 000000000000..8a915fc8cd8e --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/CompleteKeyStatisticsInformation.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.statistics; + +import com.google.common.collect.ImmutableSortedMap; + +import java.util.HashSet; +import java.util.Set; +import java.util.SortedMap; + +/** + * Class maintained by the controller to merge {@link PartialKeyStatisticsInformation} sent by the worker. + */ +public class CompleteKeyStatisticsInformation +{ + private final SortedMap> timeSegmentVsWorkerMap; + + private boolean multipleValues; + + private double bytesRetained; + + public CompleteKeyStatisticsInformation( + final SortedMap> timeChunks, + boolean multipleValues, + double bytesRetained + ) + { + this.timeSegmentVsWorkerMap = timeChunks; + this.multipleValues = multipleValues; + this.bytesRetained = bytesRetained; + } + + /** + * Merges the {@link PartialKeyStatisticsInformation} into the complete key statistics information object. + * {@link #timeSegmentVsWorkerMap} is updated in sorted order with the timechunks from + * {@param partialKeyStatisticsInformation}, {@link #multipleValues} is set to true if + * {@param partialKeyStatisticsInformation} contains multipleValues and the bytes retained by the partial sketch + * is added to {@link #bytesRetained}. + */ + public void mergePartialInformation(int workerNumber, PartialKeyStatisticsInformation partialKeyStatisticsInformation) + { + for (Long timeSegment : partialKeyStatisticsInformation.getTimeSegments()) { + this.timeSegmentVsWorkerMap + .computeIfAbsent(timeSegment, key -> new HashSet<>()) + .add(workerNumber); + } + this.multipleValues = this.multipleValues || partialKeyStatisticsInformation.hasMultipleValues(); + this.bytesRetained += bytesRetained; + } + + public SortedMap> getTimeSegmentVsWorkerMap() + { + return ImmutableSortedMap.copyOfSorted(timeSegmentVsWorkerMap); + } + + public boolean hasMultipleValues() + { + return multipleValues; + } + + public double getBytesRetained() + { + return bytesRetained; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DelegateOrMinKeyCollectorSnapshot.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DelegateOrMinKeyCollectorSnapshot.java index f0191da126b2..5e8da8df0c22 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DelegateOrMinKeyCollectorSnapshot.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DelegateOrMinKeyCollectorSnapshot.java @@ -22,16 +22,19 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; import org.apache.druid.frame.key.RowKey; import org.apache.druid.java.util.common.ISE; import javax.annotation.Nullable; import java.util.Objects; +@JsonTypeName(DelegateOrMinKeyCollectorSnapshot.TYPE) public class DelegateOrMinKeyCollectorSnapshot implements KeyCollectorSnapshot { static final String FIELD_SNAPSHOT = "snapshot"; static final String FIELD_MIN_KEY = "minKey"; + static final String TYPE = "delegate"; private final T snapshot; private final RowKey minKey; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DistinctKeySnapshot.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DistinctKeySnapshot.java index 44b95d3874f6..47d46f72cabb 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DistinctKeySnapshot.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DistinctKeySnapshot.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Preconditions; import org.apache.druid.collections.SerializablePair; import org.apache.druid.frame.key.RowKey; @@ -31,8 +32,10 @@ import java.util.Map; import java.util.Objects; +@JsonTypeName(DistinctKeySnapshot.TYPE) public class DistinctKeySnapshot implements KeyCollectorSnapshot { + static final String TYPE = "distinct"; private final List> keys; private final int spaceReductionFactor; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/KeyCollectorSnapshot.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/KeyCollectorSnapshot.java index ad359cb1e60e..d1e717c37ec9 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/KeyCollectorSnapshot.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/KeyCollectorSnapshot.java @@ -19,9 +19,18 @@ package org.apache.druid.msq.statistics; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; + /** * Marker interface for deserialization. */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "collectorType") +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = DelegateOrMinKeyCollectorSnapshot.TYPE, value = DelegateOrMinKeyCollectorSnapshot.class), + @JsonSubTypes.Type(name = QuantilesSketchKeyCollectorSnapshot.TYPE, value = QuantilesSketchKeyCollectorSnapshot.class), + @JsonSubTypes.Type(name = DistinctKeySnapshot.TYPE, value = DistinctKeySnapshot.class), +}) public interface KeyCollectorSnapshot { } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/PartialKeyStatisticsInformation.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/PartialKeyStatisticsInformation.java new file mode 100644 index 000000000000..535af8dafb0a --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/PartialKeyStatisticsInformation.java @@ -0,0 +1,67 @@ +/* + * 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.statistics; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Set; + +/** + * Class sent by worker to controller after reading input to generate partition boundries. + */ +public class PartialKeyStatisticsInformation +{ + private final Set timeSegments; + + private final boolean multipleValues; + + private final double bytesRetained; + + @JsonCreator + public PartialKeyStatisticsInformation( + @JsonProperty("timeSegments") Set timeSegments, + @JsonProperty("multipleValues") boolean hasMultipleValues, + @JsonProperty("bytesRetained") double bytesRetained + ) + { + this.timeSegments = timeSegments; + this.multipleValues = hasMultipleValues; + this.bytesRetained = bytesRetained; + } + + @JsonProperty("timeSegments") + public Set getTimeSegments() + { + return timeSegments; + } + + @JsonProperty("multipleValues") + public boolean hasMultipleValues() + { + return multipleValues; + } + + @JsonProperty("bytesRetained") + public double getBytesRetained() + { + return bytesRetained; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollectorSnapshot.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollectorSnapshot.java index 1b555ac3f944..5784afd49a0f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollectorSnapshot.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollectorSnapshot.java @@ -21,11 +21,14 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; import java.util.Objects; +@JsonTypeName(QuantilesSketchKeyCollectorSnapshot.TYPE) public class QuantilesSketchKeyCollectorSnapshot implements KeyCollectorSnapshot { + static final String TYPE = "quantile"; private final String encodedSketch; private final double averageKeyLength; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java index c7c61fd30c76..7c589f2326f1 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java @@ -25,6 +25,7 @@ import com.google.common.annotations.VisibleForTesting; import com.opencsv.RFC4180Parser; import com.opencsv.RFC4180ParserBuilder; +import org.apache.druid.msq.exec.ClusterStatisticsMergeMode; import org.apache.druid.msq.kernel.WorkerAssignmentStrategy; import org.apache.druid.msq.sql.MSQMode; import org.apache.druid.query.QueryContext; @@ -58,6 +59,8 @@ public class MultiStageQueryContext private static final boolean DEFAULT_FINALIZE_AGGREGATIONS = true; public static final String CTX_ENABLE_DURABLE_SHUFFLE_STORAGE = "durableShuffleStorage"; + public static final String CTX_CLUSTER_STATISTICS_MERGE_MODE = "clusterStatisticsMergeMode"; + public static final String DEFAULT_CLUSTER_STATISTICS_MERGE_MODE = ClusterStatisticsMergeMode.AUTO.toString(); private static final boolean DEFAULT_ENABLE_DURABLE_SHUFFLE_STORAGE = false; public static final String CTX_DESTINATION = "destination"; @@ -93,6 +96,18 @@ public static boolean isDurableStorageEnabled(final QueryContext queryContext) ); } + public static ClusterStatisticsMergeMode getClusterStatisticsMergeMode(QueryContext queryContext) + { + return ClusterStatisticsMergeMode.valueOf( + String.valueOf( + queryContext.getString( + CTX_CLUSTER_STATISTICS_MERGE_MODE, + DEFAULT_CLUSTER_STATISTICS_MERGE_MODE + ) + ) + ); + } + public static boolean isFinalizeAggregations(final QueryContext queryContext) { return queryContext.getBoolean( diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherAutoModeTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherAutoModeTest.java new file mode 100644 index 000000000000..42f6f0437f59 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherAutoModeTest.java @@ -0,0 +1,139 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.exec; + +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.statistics.CompleteKeyStatisticsInformation; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import java.util.Collections; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +public class WorkerSketchFetcherAutoModeTest +{ + @Mock + private CompleteKeyStatisticsInformation completeKeyStatisticsInformation; + @Mock + private StageDefinition stageDefinition; + @Mock + private ClusterBy clusterBy; + private AutoCloseable mocks; + private WorkerSketchFetcher target; + + @Before + public void setUp() + { + mocks = MockitoAnnotations.openMocks(this); + + target = spy(new WorkerSketchFetcher(mock(WorkerClient.class), ClusterStatisticsMergeMode.AUTO, 300_000_000)); + // Don't actually try to fetch sketches + doReturn(null).when(target).inMemoryFullSketchMerging(any(), any()); + doReturn(null).when(target).sequentialTimeChunkMerging(any(), any(), any()); + + doReturn(StageId.fromString("1_1")).when(stageDefinition).getId(); + doReturn(clusterBy).when(stageDefinition).getClusterBy(); + } + + @After + public void tearDown() throws Exception + { + mocks.close(); + } + + @Test + public void test_submitFetcherTask_belowThresholds_ShouldBeParallel() + { + // Bytes below threshold + doReturn(10.0).when(completeKeyStatisticsInformation).getBytesRetained(); + + // Cluster by bucket count not 0 + doReturn(1).when(clusterBy).getBucketByCount(); + + // Worker count below threshold + doReturn(1).when(stageDefinition).getMaxWorkerCount(); + + target.submitFetcherTask(completeKeyStatisticsInformation, Collections.emptyList(), stageDefinition); + verify(target, times(1)).inMemoryFullSketchMerging(any(), any()); + verify(target, times(0)).sequentialTimeChunkMerging(any(), any(), any()); + } + + @Test + public void test_submitFetcherTask_workerCountAboveThreshold_shouldBeSequential() + { + // Bytes below threshold + doReturn(10.0).when(completeKeyStatisticsInformation).getBytesRetained(); + + // Cluster by bucket count not 0 + doReturn(1).when(clusterBy).getBucketByCount(); + + // Worker count below threshold + doReturn((int) WorkerSketchFetcher.WORKER_THRESHOLD + 1).when(stageDefinition).getMaxWorkerCount(); + + target.submitFetcherTask(completeKeyStatisticsInformation, Collections.emptyList(), stageDefinition); + verify(target, times(0)).inMemoryFullSketchMerging(any(), any()); + verify(target, times(1)).sequentialTimeChunkMerging(any(), any(), any()); + } + + @Test + public void test_submitFetcherTask_noClusterByColumns_shouldBeParallel() + { + // Bytes above threshold + doReturn(WorkerSketchFetcher.BYTES_THRESHOLD + 10.0).when(completeKeyStatisticsInformation).getBytesRetained(); + + // Cluster by bucket count 0 + doReturn(ClusterBy.none()).when(stageDefinition).getClusterBy(); + + // Worker count above threshold + doReturn((int) WorkerSketchFetcher.WORKER_THRESHOLD + 1).when(stageDefinition).getMaxWorkerCount(); + + target.submitFetcherTask(completeKeyStatisticsInformation, Collections.emptyList(), stageDefinition); + verify(target, times(1)).inMemoryFullSketchMerging(any(), any()); + verify(target, times(0)).sequentialTimeChunkMerging(any(), any(), any()); + } + + @Test + public void test_submitFetcherTask_bytesRetainedAboveThreshold_shouldBeSequential() + { + // Bytes above threshold + doReturn(WorkerSketchFetcher.BYTES_THRESHOLD + 10.0).when(completeKeyStatisticsInformation).getBytesRetained(); + + // Cluster by bucket count not 0 + doReturn(1).when(clusterBy).getBucketByCount(); + + // Worker count below threshold + doReturn(1).when(stageDefinition).getMaxWorkerCount(); + + target.submitFetcherTask(completeKeyStatisticsInformation, Collections.emptyList(), stageDefinition); + verify(target, times(0)).inMemoryFullSketchMerging(any(), any()); + verify(target, times(1)).sequentialTimeChunkMerging(any(), any(), any()); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherTest.java new file mode 100644 index 000000000000..54c9a792e558 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherTest.java @@ -0,0 +1,295 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.exec; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.ImmutableSortedMap; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.frame.key.ClusterByPartition; +import org.apache.druid.frame.key.ClusterByPartitions; +import org.apache.druid.frame.key.RowKey; +import org.apache.druid.java.util.common.Either; +import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; +import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; +import org.apache.druid.msq.statistics.CompleteKeyStatisticsInformation; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import java.util.List; +import java.util.Queue; +import java.util.Set; +import java.util.SortedMap; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.ExecutionException; + +import static org.easymock.EasyMock.mock; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +public class WorkerSketchFetcherTest +{ + @Mock + private CompleteKeyStatisticsInformation completeKeyStatisticsInformation; + @Mock + private StageDefinition stageDefinition; + @Mock + private ClusterBy clusterBy; + @Mock + private ClusterByStatisticsCollector mergedClusterByStatisticsCollector1; + @Mock + private ClusterByStatisticsCollector mergedClusterByStatisticsCollector2; + @Mock + private WorkerClient workerClient; + private ClusterByPartitions expectedPartitions1; + private ClusterByPartitions expectedPartitions2; + private AutoCloseable mocks; + private WorkerSketchFetcher target; + + @Before + public void setUp() + { + mocks = MockitoAnnotations.openMocks(this); + doReturn(StageId.fromString("1_1")).when(stageDefinition).getId(); + doReturn(clusterBy).when(stageDefinition).getClusterBy(); + doReturn(25_000).when(stageDefinition).getMaxPartitionCount(); + + expectedPartitions1 = new ClusterByPartitions(ImmutableList.of(new ClusterByPartition(mock(RowKey.class), mock(RowKey.class)))); + expectedPartitions2 = new ClusterByPartitions(ImmutableList.of(new ClusterByPartition(mock(RowKey.class), mock(RowKey.class)))); + + doReturn(Either.value(expectedPartitions1)).when(stageDefinition).generatePartitionsForShuffle(eq(mergedClusterByStatisticsCollector1)); + doReturn(Either.value(expectedPartitions2)).when(stageDefinition).generatePartitionsForShuffle(eq(mergedClusterByStatisticsCollector2)); + + doReturn( + mergedClusterByStatisticsCollector1, + mergedClusterByStatisticsCollector2 + ).when(stageDefinition).createResultKeyStatisticsCollector(anyInt()); + } + + @After + public void tearDown() throws Exception + { + mocks.close(); + } + + @Test + public void test_submitFetcherTask_parallelFetch_workerThrowsException_shouldCancelOtherTasks() throws Exception + { + // Store futures in a queue + final Queue> futureQueue = new ConcurrentLinkedQueue<>(); + final List workerIds = ImmutableList.of("0", "1", "2", "3"); + final CountDownLatch latch = new CountDownLatch(workerIds.size()); + + target = spy(new WorkerSketchFetcher(workerClient, ClusterStatisticsMergeMode.PARALLEL, 300_000_000)); + + // When fetching snapshots, return a mock and add future to queue + doAnswer(invocation -> { + ListenableFuture snapshotListenableFuture = + spy(Futures.immediateFuture(mock(ClusterByStatisticsSnapshot.class))); + futureQueue.add(snapshotListenableFuture); + latch.countDown(); + latch.await(); + return snapshotListenableFuture; + }).when(workerClient).fetchClusterByStatisticsSnapshot(any(), any(), anyInt()); + + // Cause a worker to fail instead of returning the result + doAnswer(invocation -> { + latch.countDown(); + latch.await(); + return Futures.immediateFailedFuture(new InterruptedException("interrupted")); + }).when(workerClient).fetchClusterByStatisticsSnapshot(eq("2"), any(), anyInt()); + + CompletableFuture> eitherCompletableFuture = target.submitFetcherTask( + completeKeyStatisticsInformation, + workerIds, + stageDefinition + ); + + // Assert that the final result is failed and all other task futures are also cancelled. + Assert.assertThrows(CompletionException.class, eitherCompletableFuture::join); + Thread.sleep(1000); + + Assert.assertTrue(eitherCompletableFuture.isCompletedExceptionally()); + // Verify that the statistics collector was cleared due to the error. + verify(mergedClusterByStatisticsCollector1, times(1)).clear(); + // Verify that other task futures were requested to be cancelled. + Assert.assertFalse(futureQueue.isEmpty()); + for (ListenableFuture snapshotFuture : futureQueue) { + verify(snapshotFuture, times(1)).cancel(eq(true)); + } + } + + @Test + public void test_submitFetcherTask_parallelFetch_mergePerformedCorrectly() + throws ExecutionException, InterruptedException + { + // Store snapshots in a queue + final Queue snapshotQueue = new ConcurrentLinkedQueue<>(); + final List workerIds = ImmutableList.of("0", "1", "2", "3", "4"); + final CountDownLatch latch = new CountDownLatch(workerIds.size()); + + target = spy(new WorkerSketchFetcher(workerClient, ClusterStatisticsMergeMode.PARALLEL, 300_000_000)); + + // When fetching snapshots, return a mock and add it to queue + doAnswer(invocation -> { + ClusterByStatisticsSnapshot snapshot = mock(ClusterByStatisticsSnapshot.class); + snapshotQueue.add(snapshot); + latch.countDown(); + return Futures.immediateFuture(snapshot); + }).when(workerClient).fetchClusterByStatisticsSnapshot(any(), any(), anyInt()); + + CompletableFuture> eitherCompletableFuture = target.submitFetcherTask( + completeKeyStatisticsInformation, + workerIds, + stageDefinition + ); + + // Assert that the final result is complete and all other sketches returned have been merged. + eitherCompletableFuture.join(); + Thread.sleep(1000); + Assert.assertTrue(eitherCompletableFuture.isDone() && !eitherCompletableFuture.isCompletedExceptionally()); + Assert.assertFalse(snapshotQueue.isEmpty()); + // Verify that all statistics were added to controller. + for (ClusterByStatisticsSnapshot snapshot : snapshotQueue) { + verify(mergedClusterByStatisticsCollector1, times(1)).addAll(eq(snapshot)); + } + // Check that the partitions returned by the merged collector is returned by the final future. + Assert.assertEquals(expectedPartitions1, eitherCompletableFuture.get().valueOrThrow()); + } + + @Test + public void test_submitFetcherTask_sequentialFetch_workerThrowsException_shouldCancelOtherTasks() throws Exception + { + // Store futures in a queue + final Queue> futureQueue = new ConcurrentLinkedQueue<>(); + + SortedMap> timeSegmentVsWorkerMap = ImmutableSortedMap.of(1L, ImmutableSet.of(0, 1, 2), 2L, ImmutableSet.of(0, 1, 4)); + doReturn(timeSegmentVsWorkerMap).when(completeKeyStatisticsInformation).getTimeSegmentVsWorkerMap(); + + final CyclicBarrier barrier = new CyclicBarrier(3); + target = spy(new WorkerSketchFetcher(workerClient, ClusterStatisticsMergeMode.SEQUENTIAL, 300_000_000)); + + // When fetching snapshots, return a mock and add future to queue + doAnswer(invocation -> { + ListenableFuture snapshotListenableFuture = + spy(Futures.immediateFuture(mock(ClusterByStatisticsSnapshot.class))); + futureQueue.add(snapshotListenableFuture); + barrier.await(); + return snapshotListenableFuture; + }).when(workerClient).fetchClusterByStatisticsSnapshotForTimeChunk(anyString(), anyString(), anyInt(), anyLong()); + + // Cause a worker in the second time chunk to fail instead of returning the result + doAnswer(invocation -> { + barrier.await(); + return Futures.immediateFailedFuture(new InterruptedException("interrupted")); + }).when(workerClient).fetchClusterByStatisticsSnapshotForTimeChunk(eq("4"), any(), anyInt(), eq(2L)); + + CompletableFuture> eitherCompletableFuture = target.submitFetcherTask( + completeKeyStatisticsInformation, + ImmutableList.of("0", "1", "2", "3", "4"), + stageDefinition + ); + + // Assert that the final result is failed and all other task futures are also cancelled. + Assert.assertThrows(CompletionException.class, eitherCompletableFuture::join); + Thread.sleep(1000); + + Assert.assertTrue(eitherCompletableFuture.isCompletedExceptionally()); + // Verify that the correct statistics collector was cleared due to the error. + verify(mergedClusterByStatisticsCollector1, times(0)).clear(); + verify(mergedClusterByStatisticsCollector2, times(1)).clear(); + // Verify that other task futures were requested to be cancelled. + Assert.assertFalse(futureQueue.isEmpty()); + for (ListenableFuture snapshotFuture : futureQueue) { + verify(snapshotFuture, times(1)).cancel(eq(true)); + } + } + + @Test + public void test_submitFetcherTask_sequentialFetch_mergePerformedCorrectly() + throws ExecutionException, InterruptedException + { + // Store snapshots in a queue + final Queue snapshotQueue = new ConcurrentLinkedQueue<>(); + + SortedMap> timeSegmentVsWorkerMap = ImmutableSortedMap.of(1L, ImmutableSet.of(0, 1, 2), 2L, ImmutableSet.of(0, 1, 4)); + doReturn(timeSegmentVsWorkerMap).when(completeKeyStatisticsInformation).getTimeSegmentVsWorkerMap(); + + final CyclicBarrier barrier = new CyclicBarrier(3); + target = spy(new WorkerSketchFetcher(workerClient, ClusterStatisticsMergeMode.SEQUENTIAL, 300_000_000)); + + // When fetching snapshots, return a mock and add it to queue + doAnswer(invocation -> { + ClusterByStatisticsSnapshot snapshot = mock(ClusterByStatisticsSnapshot.class); + snapshotQueue.add(snapshot); + barrier.await(); + return Futures.immediateFuture(snapshot); + }).when(workerClient).fetchClusterByStatisticsSnapshotForTimeChunk(any(), any(), anyInt(), anyLong()); + + CompletableFuture> eitherCompletableFuture = target.submitFetcherTask( + completeKeyStatisticsInformation, + ImmutableList.of("0", "1", "2", "3", "4"), + stageDefinition + ); + + // Assert that the final result is complete and all other sketches returned have been merged. + eitherCompletableFuture.join(); + Thread.sleep(1000); + + Assert.assertTrue(eitherCompletableFuture.isDone() && !eitherCompletableFuture.isCompletedExceptionally()); + Assert.assertFalse(snapshotQueue.isEmpty()); + // Verify that all statistics were added to controller. + snapshotQueue.stream().limit(3).forEach(snapshot -> { + verify(mergedClusterByStatisticsCollector1, times(1)).addAll(eq(snapshot)); + }); + snapshotQueue.stream().skip(3).limit(3).forEach(snapshot -> { + verify(mergedClusterByStatisticsCollector2, times(1)).addAll(eq(snapshot)); + }); + ClusterByPartitions expectedResult = + new ClusterByPartitions( + ImmutableList.of( + new ClusterByPartition(expectedPartitions1.get(0).getStart(), expectedPartitions2.get(0).getStart()), + new ClusterByPartition(expectedPartitions2.get(0).getStart(), expectedPartitions2.get(0).getEnd()) + ) + ); + // Check that the partitions returned by the merged collector is returned by the final future. + Assert.assertEquals(expectedResult, eitherCompletableFuture.get().valueOrThrow()); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/BaseControllerQueryKernelTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/BaseControllerQueryKernelTest.java index c31d8c69fa99..f6b0ea9f08bb 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/BaseControllerQueryKernelTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/BaseControllerQueryKernelTest.java @@ -21,6 +21,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; +import org.apache.druid.frame.key.ClusterByPartitions; import org.apache.druid.frame.key.KeyTestUtils; import org.apache.druid.frame.key.RowKey; import org.apache.druid.java.util.common.IAE; @@ -31,6 +32,7 @@ import org.apache.druid.msq.input.stage.StageInputSpec; import org.apache.druid.msq.input.stage.StageInputSpecSlicer; import org.apache.druid.msq.kernel.QueryDefinition; +import org.apache.druid.msq.kernel.StageDefinition; import org.apache.druid.msq.kernel.StageId; import org.apache.druid.msq.kernel.WorkerAssignmentStrategy; import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; @@ -80,7 +82,7 @@ private ControllerQueryKernelTester(int numWorkers) public ControllerQueryKernelTester queryDefinition(QueryDefinition queryDefinition) { this.queryDefinition = Preconditions.checkNotNull(queryDefinition); - this.controllerQueryKernel = new ControllerQueryKernel(queryDefinition, 10_000_000); + this.controllerQueryKernel = new ControllerQueryKernel(queryDefinition); return this; } @@ -121,10 +123,10 @@ public ControllerQueryKernelTester setupStage( if (queryDefinition.getStageDefinition(stageNumber).mustGatherResultKeyStatistics()) { for (int i = 0; i < numWorkers; ++i) { - controllerQueryKernel.addResultKeyStatisticsForStageAndWorker( + controllerQueryKernel.addPartialKeyStatisticsForStageAndWorker( new StageId(queryDefinition.getQueryId(), stageNumber), i, - ClusterByStatisticsSnapshot.empty() + ClusterByStatisticsSnapshot.empty().partialKeyStatistics() ); } } else { @@ -238,7 +240,7 @@ public void finishStage(int stageNumber, boolean strict) controllerQueryKernel.finishStage(new StageId(queryDefinition.getQueryId(), stageNumber), strict); } - public void addResultKeyStatisticsForStageAndWorker(int stageNumber, int workerNumber) + public ClusterByStatisticsCollector addResultKeyStatisticsForStageAndWorker(int stageNumber, int workerNumber) { Preconditions.checkArgument(initialized); @@ -254,11 +256,12 @@ public void addResultKeyStatisticsForStageAndWorker(int stageNumber, int workerN keyStatsCollector.add(key, 1); } - controllerQueryKernel.addResultKeyStatisticsForStageAndWorker( + controllerQueryKernel.addPartialKeyStatisticsForStageAndWorker( new StageId(queryDefinition.getQueryId(), stageNumber), workerNumber, - keyStatsCollector.snapshot() + keyStatsCollector.snapshot().partialKeyStatistics() ); + return keyStatsCollector; } public void setResultsCompleteForStageAndWorker(int stageNumber, int workerNumber) @@ -271,6 +274,18 @@ public void setResultsCompleteForStageAndWorker(int stageNumber, int workerNumbe ); } + public void setPartitionBoundaries(int stageNumber, ClusterByStatisticsCollector clusterByStatisticsCollector) + { + Preconditions.checkArgument(initialized); + StageId stageId = new StageId(queryDefinition.getQueryId(), stageNumber); + StageDefinition stageDefinition = controllerQueryKernel.getStageDefinition(stageId); + ClusterByPartitions clusterByPartitions = + stageDefinition + .generatePartitionsForShuffle(clusterByStatisticsCollector) + .valueOrThrow(); + controllerQueryKernel.setClusterByPartitionBoundaries(stageId, clusterByPartitions); + } + public void failStage(int stageNumber) { Preconditions.checkArgument(initialized); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernelTests.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernelTests.java index 3094bf344d2f..f2b6139e9985 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernelTests.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernelTests.java @@ -20,6 +20,7 @@ package org.apache.druid.msq.kernel.controller; import com.google.common.collect.ImmutableSet; +import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; import org.junit.Assert; import org.junit.Test; @@ -146,8 +147,13 @@ public void testCompleteDAGExecutionForMultipleWorkers() Assert.assertEquals(ImmutableSet.of(0), newStageNumbers); Assert.assertEquals(ImmutableSet.of(), effectivelyFinishedStageNumbers); controllerQueryKernelTester.startStage(0); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 0); - controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + ClusterByStatisticsCollector clusterByStatisticsCollector = + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker( + 0, + 0 + ); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.setPartitionBoundaries(0, clusterByStatisticsCollector); controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.RESULTS_READY); @@ -156,9 +162,20 @@ public void testCompleteDAGExecutionForMultipleWorkers() Assert.assertEquals(ImmutableSet.of(1), newStageNumbers); Assert.assertEquals(ImmutableSet.of(), effectivelyFinishedStageNumbers); controllerQueryKernelTester.startStage(1); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(1, 0); + clusterByStatisticsCollector = + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker( + 1, + 0 + ); controllerQueryKernelTester.assertStagePhase(1, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(1, 1); + clusterByStatisticsCollector.addAll( + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker( + 1, + 1 + ) + ); + controllerQueryKernelTester.assertStagePhase(1, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.setPartitionBoundaries(1, clusterByStatisticsCollector); controllerQueryKernelTester.assertStagePhase(1, ControllerStagePhase.POST_READING); controllerQueryKernelTester.setResultsCompleteForStageAndWorker(1, 0); controllerQueryKernelTester.assertStagePhase(1, ControllerStagePhase.POST_READING); @@ -182,9 +199,19 @@ public void testCompleteDAGExecutionForMultipleWorkers() Assert.assertEquals(ImmutableSet.of(1), effectivelyFinishedStageNumbers); controllerQueryKernelTester.startStage(3); controllerQueryKernelTester.assertStagePhase(3, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(3, 0); + ClusterByStatisticsCollector clusterByStatisticsCollector3 = + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker( + 3, + 0 + ); controllerQueryKernelTester.assertStagePhase(3, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(3, 1); + ClusterByStatisticsCollector clusterByStatisticsCollector4 = + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker( + 3, + 1 + ); + controllerQueryKernelTester.assertStagePhase(3, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.setPartitionBoundaries(3, clusterByStatisticsCollector3.addAll(clusterByStatisticsCollector4)); controllerQueryKernelTester.assertStagePhase(3, ControllerStagePhase.POST_READING); controllerQueryKernelTester.setResultsCompleteForStageAndWorker(3, 0); controllerQueryKernelTester.assertStagePhase(3, ControllerStagePhase.POST_READING); @@ -217,11 +244,21 @@ public void testTransitionsInShufflingStagesAndMultipleWorkers() controllerQueryKernelTester.createAndGetNewStageNumbers(); controllerQueryKernelTester.startStage(0); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 0); + ClusterByStatisticsCollector clusterByStatisticsCollector = + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker( + 0, + 0 + ); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.READING_INPUT); - controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker(0, 1); - controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); + clusterByStatisticsCollector.addAll( + controllerQueryKernelTester.addResultKeyStatisticsForStageAndWorker( + 0, + 1 + ) + ); + controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.MERGING_STATISTICS); + controllerQueryKernelTester.setPartitionBoundaries(0, clusterByStatisticsCollector); controllerQueryKernelTester.setResultsCompleteForStageAndWorker(0, 0); controllerQueryKernelTester.assertStagePhase(0, ControllerStagePhase.POST_READING); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/PartialKeyStatisticsInformationSerdeTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/PartialKeyStatisticsInformationSerdeTest.java new file mode 100644 index 000000000000..c23106c0299d --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/PartialKeyStatisticsInformationSerdeTest.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.statistics; + +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.msq.guice.MSQIndexingModule; +import org.apache.druid.segment.TestHelper; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class PartialKeyStatisticsInformationSerdeTest +{ + private ObjectMapper objectMapper; + + @Before + public void setUp() + { + objectMapper = TestHelper.makeJsonMapper(); + objectMapper.registerModules(new MSQIndexingModule().getJacksonModules()); + objectMapper.enable(JsonParser.Feature.STRICT_DUPLICATE_DETECTION); + } + + @Test + public void testSerde() throws JsonProcessingException + { + PartialKeyStatisticsInformation partialInformation = new PartialKeyStatisticsInformation( + ImmutableSet.of(2L, 3L), + false, + 0.0 + ); + + final String json = objectMapper.writeValueAsString(partialInformation); + final PartialKeyStatisticsInformation deserializedKeyStatistics = objectMapper.readValue( + json, + PartialKeyStatisticsInformation.class + ); + Assert.assertEquals(json, partialInformation.getTimeSegments(), deserializedKeyStatistics.getTimeSegments()); + Assert.assertEquals(json, partialInformation.hasMultipleValues(), deserializedKeyStatistics.hasMultipleValues()); + Assert.assertEquals(json, partialInformation.getBytesRetained(), deserializedKeyStatistics.getBytesRetained(), 0); + } +} 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 5b088b71d5bb..06b0bae29690 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 @@ -25,7 +25,7 @@ import org.apache.druid.msq.exec.ControllerClient; import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.apache.druid.msq.kernel.StageId; -import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; +import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation; import javax.annotation.Nullable; import java.util.List; @@ -40,17 +40,17 @@ public MSQTestControllerClient(Controller controller) } @Override - public void postKeyStatistics( + public void postPartialKeyStatistics( StageId stageId, int workerNumber, - ClusterByStatisticsSnapshot keyStatistics + PartialKeyStatisticsInformation partialKeyStatisticsInformation ) { try { - controller.updateStatus(stageId.getStageNumber(), workerNumber, keyStatistics); + controller.updatePartialKeyStatisticsInformation(stageId.getStageNumber(), workerNumber, partialKeyStatisticsInformation); } catch (Exception e) { - throw new ISE(e, "unable to post key statistics"); + throw new ISE(e, "unable to post partial key statistics"); } } 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 094c1bd16cf7..ae892c34500a 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 @@ -29,6 +29,7 @@ import org.apache.druid.msq.exec.WorkerClient; import org.apache.druid.msq.kernel.StageId; import org.apache.druid.msq.kernel.WorkOrder; +import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; import java.io.InputStream; import java.util.Arrays; @@ -50,6 +51,29 @@ public ListenableFuture postWorkOrder(String workerTaskId, WorkOrder workO return Futures.immediateFuture(null); } + @Override + public ListenableFuture fetchClusterByStatisticsSnapshot( + String workerTaskId, + String queryId, + int stageNumber + ) + { + StageId stageId = new StageId(queryId, stageNumber); + return Futures.immediateFuture(inMemoryWorkers.get(workerTaskId).fetchStatisticsSnapshot(stageId)); + } + + @Override + public ListenableFuture fetchClusterByStatisticsSnapshotForTimeChunk( + String workerTaskId, + String queryId, + int stageNumber, + long timeChunk + ) + { + StageId stageId = new StageId(queryId, stageNumber); + return Futures.immediateFuture(inMemoryWorkers.get(workerTaskId).fetchStatisticsSnapshotForTimeChunk(stageId, timeChunk)); + } + @Override public ListenableFuture postResultPartitionBoundaries( String workerTaskId, diff --git a/integration-tests-ex/README.md b/integration-tests-ex/README.md index 9c29ec101fe9..3a17c6e7c700 100644 --- a/integration-tests-ex/README.md +++ b/integration-tests-ex/README.md @@ -71,7 +71,7 @@ Start the cluster: ```bash cd $DRUID_DEV/integration-tests-ex/cases -./cluster.sh up +./cluster.sh up ``` Where `` is one of the test categories. Then launch the diff --git a/integration-tests-ex/cases/pom.xml b/integration-tests-ex/cases/pom.xml index cf781f6f8885..92a632d1f14c 100644 --- a/integration-tests-ex/cases/pom.xml +++ b/integration-tests-ex/cases/pom.xml @@ -178,6 +178,11 @@ + + org.apache.druid + druid-sql + 25.0.0-SNAPSHOT + org.apache.druid.extensions druid-multi-stage-query diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITKeyStatisticsSketchMergeMode.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITKeyStatisticsSketchMergeMode.java new file mode 100644 index 000000000000..c0f22f2c762d --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITKeyStatisticsSketchMergeMode.java @@ -0,0 +1,206 @@ +/* + * 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.testsEx.msq; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import com.google.inject.Inject; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.msq.exec.ClusterStatisticsMergeMode; +import org.apache.druid.msq.sql.SqlTaskStatus; +import org.apache.druid.msq.util.MultiStageQueryContext; +import org.apache.druid.sql.http.SqlQuery; +import org.apache.druid.testing.IntegrationTestingConfig; +import org.apache.druid.testing.clients.CoordinatorResourceTestClient; +import org.apache.druid.testing.clients.SqlResourceTestClient; +import org.apache.druid.testing.utils.DataLoaderHelper; +import org.apache.druid.testing.utils.MsqTestQueryHelper; +import org.apache.druid.testsEx.categories.MultiStageQuery; +import org.apache.druid.testsEx.config.DruidTestRunner; +import org.junit.Assert; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +@RunWith(DruidTestRunner.class) +@Category(MultiStageQuery.class) +public class ITKeyStatisticsSketchMergeMode +{ + @Inject + private MsqTestQueryHelper msqHelper; + + @Inject + private SqlResourceTestClient msqClient; + + @Inject + private IntegrationTestingConfig config; + + @Inject + private ObjectMapper jsonMapper; + + @Inject + private DataLoaderHelper dataLoaderHelper; + + @Inject + private CoordinatorResourceTestClient coordinatorClient; + + private static final String QUERY_FILE = "/multi-stage-query/wikipedia_msq_select_query1.json"; + + + @Test + public void testMsqIngestionParallelMerging() throws Exception + { + String datasource = "dst"; + + // Clear up the datasource from the previous runs + coordinatorClient.unloadSegmentsForDataSource(datasource); + + String queryLocal = + StringUtils.format( + "INSERT INTO %s\n" + + "SELECT\n" + + " TIME_PARSE(\"timestamp\") AS __time,\n" + + " isRobot,\n" + + " diffUrl,\n" + + " added,\n" + + " countryIsoCode,\n" + + " regionName,\n" + + " channel,\n" + + " flags,\n" + + " delta,\n" + + " isUnpatrolled,\n" + + " isNew,\n" + + " deltaBucket,\n" + + " isMinor,\n" + + " isAnonymous,\n" + + " deleted,\n" + + " cityName,\n" + + " metroCode,\n" + + " namespace,\n" + + " comment,\n" + + " page,\n" + + " commentLength,\n" + + " countryName,\n" + + " user,\n" + + " regionIsoCode\n" + + "FROM TABLE(\n" + + " EXTERN(\n" + + " '{\"type\":\"local\",\"files\":[\"/resources/data/batch_index/json/wikipedia_index_data1.json\"]}',\n" + + " '{\"type\":\"json\"}',\n" + + " '[{\"type\":\"string\",\"name\":\"timestamp\"},{\"type\":\"string\",\"name\":\"isRobot\"},{\"type\":\"string\",\"name\":\"diffUrl\"},{\"type\":\"long\",\"name\":\"added\"},{\"type\":\"string\",\"name\":\"countryIsoCode\"},{\"type\":\"string\",\"name\":\"regionName\"},{\"type\":\"string\",\"name\":\"channel\"},{\"type\":\"string\",\"name\":\"flags\"},{\"type\":\"long\",\"name\":\"delta\"},{\"type\":\"string\",\"name\":\"isUnpatrolled\"},{\"type\":\"string\",\"name\":\"isNew\"},{\"type\":\"double\",\"name\":\"deltaBucket\"},{\"type\":\"string\",\"name\":\"isMinor\"},{\"type\":\"string\",\"name\":\"isAnonymous\"},{\"type\":\"long\",\"name\":\"deleted\"},{\"type\":\"string\",\"name\":\"cityName\"},{\"type\":\"long\",\"name\":\"metroCode\"},{\"type\":\"string\",\"name\":\"namespace\"},{\"type\":\"string\",\"name\":\"comment\"},{\"type\":\"string\",\"name\":\"page\"},{\"type\":\"long\",\"name\":\"commentLength\"},{\"type\":\"string\",\"name\":\"countryName\"},{\"type\":\"string\",\"name\":\"user\"},{\"type\":\"string\",\"name\":\"regionIsoCode\"}]'\n" + + " )\n" + + ")\n" + + "PARTITIONED BY DAY\n" + + "CLUSTERED BY \"__time\"", + datasource + ); + + ImmutableMap context = ImmutableMap.of( + MultiStageQueryContext.CTX_CLUSTER_STATISTICS_MERGE_MODE, + ClusterStatisticsMergeMode.PARALLEL + ); + + // Submit the task and wait for the datasource to get loaded + SqlQuery sqlQuery = new SqlQuery(queryLocal, null, false, false, false, context, null); + SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTask(sqlQuery); + + if (sqlTaskStatus.getState().isFailure()) { + Assert.fail(StringUtils.format( + "Unable to start the task successfully.\nPossible exception: %s", + sqlTaskStatus.getError() + )); + } + + msqHelper.pollTaskIdForCompletion(sqlTaskStatus.getTaskId()); + dataLoaderHelper.waitUntilDatasourceIsReady(datasource); + + msqHelper.testQueriesFromFile(QUERY_FILE, datasource); + } + + @Test + public void testMsqIngestionSequentialMerging() throws Exception + { + String datasource = "dst"; + + // Clear up the datasource from the previous runs + coordinatorClient.unloadSegmentsForDataSource(datasource); + + String queryLocal = + StringUtils.format( + "INSERT INTO %s\n" + + "SELECT\n" + + " TIME_PARSE(\"timestamp\") AS __time,\n" + + " isRobot,\n" + + " diffUrl,\n" + + " added,\n" + + " countryIsoCode,\n" + + " regionName,\n" + + " channel,\n" + + " flags,\n" + + " delta,\n" + + " isUnpatrolled,\n" + + " isNew,\n" + + " deltaBucket,\n" + + " isMinor,\n" + + " isAnonymous,\n" + + " deleted,\n" + + " cityName,\n" + + " metroCode,\n" + + " namespace,\n" + + " comment,\n" + + " page,\n" + + " commentLength,\n" + + " countryName,\n" + + " user,\n" + + " regionIsoCode\n" + + "FROM TABLE(\n" + + " EXTERN(\n" + + " '{\"type\":\"local\",\"files\":[\"/resources/data/batch_index/json/wikipedia_index_data1.json\"]}',\n" + + " '{\"type\":\"json\"}',\n" + + " '[{\"type\":\"string\",\"name\":\"timestamp\"},{\"type\":\"string\",\"name\":\"isRobot\"},{\"type\":\"string\",\"name\":\"diffUrl\"},{\"type\":\"long\",\"name\":\"added\"},{\"type\":\"string\",\"name\":\"countryIsoCode\"},{\"type\":\"string\",\"name\":\"regionName\"},{\"type\":\"string\",\"name\":\"channel\"},{\"type\":\"string\",\"name\":\"flags\"},{\"type\":\"long\",\"name\":\"delta\"},{\"type\":\"string\",\"name\":\"isUnpatrolled\"},{\"type\":\"string\",\"name\":\"isNew\"},{\"type\":\"double\",\"name\":\"deltaBucket\"},{\"type\":\"string\",\"name\":\"isMinor\"},{\"type\":\"string\",\"name\":\"isAnonymous\"},{\"type\":\"long\",\"name\":\"deleted\"},{\"type\":\"string\",\"name\":\"cityName\"},{\"type\":\"long\",\"name\":\"metroCode\"},{\"type\":\"string\",\"name\":\"namespace\"},{\"type\":\"string\",\"name\":\"comment\"},{\"type\":\"string\",\"name\":\"page\"},{\"type\":\"long\",\"name\":\"commentLength\"},{\"type\":\"string\",\"name\":\"countryName\"},{\"type\":\"string\",\"name\":\"user\"},{\"type\":\"string\",\"name\":\"regionIsoCode\"}]'\n" + + " )\n" + + ")\n" + + "PARTITIONED BY DAY\n" + + "CLUSTERED BY \"__time\"", + datasource + ); + + ImmutableMap context = ImmutableMap.of( + MultiStageQueryContext.CTX_CLUSTER_STATISTICS_MERGE_MODE, + ClusterStatisticsMergeMode.SEQUENTIAL + ); + + // Submit the task and wait for the datasource to get loaded + SqlQuery sqlQuery = new SqlQuery(queryLocal, null, false, false, false, context, null); + SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTask(sqlQuery); + + if (sqlTaskStatus.getState().isFailure()) { + Assert.fail(StringUtils.format( + "Unable to start the task successfully.\nPossible exception: %s", + sqlTaskStatus.getError() + )); + } + + msqHelper.pollTaskIdForCompletion(sqlTaskStatus.getTaskId()); + dataLoaderHelper.waitUntilDatasourceIsReady(datasource); + + msqHelper.testQueriesFromFile(QUERY_FILE, datasource); + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/key/RowKeyReader.java b/processing/src/main/java/org/apache/druid/frame/key/RowKeyReader.java index d3ca6d035c49..3102b103c9c7 100644 --- a/processing/src/main/java/org/apache/druid/frame/key/RowKeyReader.java +++ b/processing/src/main/java/org/apache/druid/frame/key/RowKeyReader.java @@ -130,6 +130,28 @@ public boolean hasMultipleValues(final RowKey key, final int fieldNumber) } } + /** + * Trims the key reader to a particular fieldCount. Used to read keys trimmed by {@link #trim(RowKey, int)}. + */ + public RowKeyReader trimmedKeyReader(int trimmedFieldCount) + { + final RowSignature.Builder newSignature = RowSignature.builder(); + + if (trimmedFieldCount > signature.size()) { + throw new IAE("Cannot trim to [%,d] fields, only have [%,d] fields", trimmedFieldCount, signature); + } + + for (int i = 0; i < trimmedFieldCount; i++) { + final String columnName = signature.getColumnName(i); + final ColumnType columnType = + Preconditions.checkNotNull(signature.getColumnType(i).orElse(null), "Type for column [%s]", columnName); + + newSignature.add(columnName, columnType); + } + + return RowKeyReader.create(newSignature.build()); + } + /** * Trim a key to a particular fieldCount. The returned key may be a copy, but is not guaranteed to be. */ diff --git a/processing/src/test/java/org/apache/druid/frame/key/RowKeyReaderTest.java b/processing/src/test/java/org/apache/druid/frame/key/RowKeyReaderTest.java index 7a9131729539..48d940384270 100644 --- a/processing/src/test/java/org/apache/druid/frame/key/RowKeyReaderTest.java +++ b/processing/src/test/java/org/apache/druid/frame/key/RowKeyReaderTest.java @@ -30,6 +30,7 @@ import org.junit.internal.matchers.ThrowableMessageMatcher; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.stream.IntStream; @@ -144,4 +145,41 @@ public void test_trim_beyondFullLength() MatcherAssert.assertThat(e, ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString("Cannot trim"))); } + + @Test + public void test_trimmedKeyReader_zero() + { + RowKey trimmedKey = keyReader.trim(key, 0); + RowKeyReader trimmedKeyReader = keyReader.trimmedKeyReader(0); + + Assert.assertEquals( + Collections.emptyList(), + trimmedKeyReader.read(trimmedKey) + ); + } + + @Test + public void test_trimmedKeyReader_one() + { + RowKey trimmedKey = keyReader.trim(key, 1); + RowKeyReader trimmedKeyReader = keyReader.trimmedKeyReader(1); + + Assert.assertEquals( + objects.subList(0, 1), + trimmedKeyReader.read(trimmedKey) + ); + } + + @Test + public void test_trimmedKeyReader_oneLessThanFullLength() + { + final int numFields = signature.size() - 1; + RowKey trimmedKey = keyReader.trim(key, numFields); + RowKeyReader trimmedKeyReader = keyReader.trimmedKeyReader(numFields); + + Assert.assertEquals( + objects.subList(0, numFields), + trimmedKeyReader.read(trimmedKey) + ); + } } diff --git a/website/.spelling b/website/.spelling index c27c4d8ccc80..c2eb66104ee8 100644 --- a/website/.spelling +++ b/website/.spelling @@ -68,6 +68,9 @@ Double.NEGATIVE_INFINITY Double.NEGATIVE_INFINITY. Double.POSITIVE_INFINITY Double.POSITIVE_INFINITY. +downsampled +downsamples +downsampling Dropwizard dropwizard DruidInputSource From 1f107cbfd3a4e071c3ac4a87ae4b0fa52d372b0c Mon Sep 17 00:00:00 2001 From: Jill Osborne Date: Thu, 24 Nov 2022 02:23:57 +0000 Subject: [PATCH 04/34] Backport firehose doc changes (#13419) * Backport firehose PR 12981 * Update migrate-from-firehose-ingestion.md --- .../migrate-from-firehose-ingestion.md | 209 ++++++++++++++++++ docs/ingestion/native-batch-firehose.md | 11 +- website/sidebars.json | 2 + 3 files changed, 216 insertions(+), 6 deletions(-) create mode 100644 docs/ingestion/migrate-from-firehose-ingestion.md diff --git a/docs/ingestion/migrate-from-firehose-ingestion.md b/docs/ingestion/migrate-from-firehose-ingestion.md new file mode 100644 index 000000000000..c5d3f6946f21 --- /dev/null +++ b/docs/ingestion/migrate-from-firehose-ingestion.md @@ -0,0 +1,209 @@ +--- +id: migrate-from-firehose +title: "Migrate from firehose to input source ingestion" +sidebar_label: "Migrate from firehose" +--- + + + +Apache deprecated support for Druid firehoses in version 0.17. Support for firehose ingestion will be removed in version 26.0. + +If you're using a firehose for batch ingestion, we strongly recommend that you follow the instructions on this page to transition to using native batch ingestion input sources as soon as possible. + +Firehose ingestion doesn't work with newer Druid versions, so you must be using an ingestion spec with a defined input source before you upgrade. + +## Migrate from firehose ingestion to an input source + +To migrate from firehose ingestion, you can use the Druid console to update your ingestion spec, or you can update it manually. + +### Use the Druid console + +To update your ingestion spec using the Druid console, open the console and copy your spec into the **Edit spec** stage of the data loader. + +Druid converts the spec into one with a defined input source. For example, it converts the [example firehose ingestion spec](#example-firehose-ingestion-spec) below into the [example ingestion spec after migration](#example-ingestion-spec-after-migration). + +If you're unable to use the console or you have problems with the console method, the alternative is to update your ingestion spec manually. + +### Update your ingestion spec manually + +To update your ingestion spec manually, copy your existing spec into a new file. Refer to [Native batch ingestion with firehose (Deprecated)](./native-batch-firehose.md) for a description of firehose properties. + +Edit the new file as follows: + +1. In the `ioConfig` component, replace the `firehose` definition with an `inputSource` definition for your chosen input source. See [Native batch input sources](./native-batch-input-source.md) for details. +2. Move the `timeStampSpec` definition from `parser.parseSpec` to the `dataSchema` component. +3. Move the `dimensionsSpec` definition from `parser.parseSpec` to the `dataSchema` component. +4. Move the `format` definition from `parser.parseSpec` to an `inputFormat` definition in `ioConfig`. +5. Delete the `parser` definition. +6. Save the file. +
You can check the format of your new ingestion file against the [migrated example](#example-ingestion-spec-after-migration) below. +7. Test the new ingestion spec with a temporary data source. +8. Once you've successfully ingested sample data with the new spec, stop firehose ingestion and switch to the new spec. + +When the transition is complete, you can upgrade Druid to the latest version. See the [Druid release notes](https://druid.apache.org/downloads.html) for upgrade instructions. + +### Example firehose ingestion spec + +An example firehose ingestion spec is as follows: + +```json +{ + "type" : "index", + "spec" : { + "dataSchema" : { + "dataSource" : "wikipedia", + "metricsSpec" : [ + { + "type" : "count", + "name" : "count" + }, + { + "type" : "doubleSum", + "name" : "added", + "fieldName" : "added" + }, + { + "type" : "doubleSum", + "name" : "deleted", + "fieldName" : "deleted" + }, + { + "type" : "doubleSum", + "name" : "delta", + "fieldName" : "delta" + } + ], + "granularitySpec" : { + "type" : "uniform", + "segmentGranularity" : "DAY", + "queryGranularity" : "NONE", + "intervals" : [ "2013-08-31/2013-09-01" ] + }, + "parser": { + "type": "string", + "parseSpec": { + "format": "json", + "timestampSpec" : { + "column" : "timestamp", + "format" : "auto" + }, + "dimensionsSpec" : { + "dimensions": ["country", "page","language","user","unpatrolled","newPage","robot","anonymous","namespace","continent","region","city"], + "dimensionExclusions" : [] + } + } + } + }, + "ioConfig" : { + "type" : "index", + "firehose" : { + "type" : "local", + "baseDir" : "examples/indexing/", + "filter" : "wikipedia_data.json" + } + }, + "tuningConfig" : { + "type" : "index", + "partitionsSpec": { + "type": "single_dim", + "partitionDimension": "country", + "targetRowsPerSegment": 5000000 + } + } + } +} +``` + +### Example ingestion spec after migration + +The following example illustrates the result of migrating the [example firehose ingestion spec](#example-firehose-ingestion-spec) to a spec with an input source: + +```json +{ + "type" : "index", + "spec" : { + "dataSchema" : { + "dataSource" : "wikipedia", + "timestampSpec" : { + "column" : "timestamp", + "format" : "auto" + }, + "dimensionsSpec" : { + "dimensions": ["country", "page","language","user","unpatrolled","newPage","robot","anonymous","namespace","continent","region","city"], + "dimensionExclusions" : [] + }, + "metricsSpec" : [ + { + "type" : "count", + "name" : "count" + }, + { + "type" : "doubleSum", + "name" : "added", + "fieldName" : "added" + }, + { + "type" : "doubleSum", + "name" : "deleted", + "fieldName" : "deleted" + }, + { + "type" : "doubleSum", + "name" : "delta", + "fieldName" : "delta" + } + ], + "granularitySpec" : { + "type" : "uniform", + "segmentGranularity" : "DAY", + "queryGranularity" : "NONE", + "intervals" : [ "2013-08-31/2013-09-01" ] + } + }, + "ioConfig" : { + "type" : "index", + "inputSource" : { + "type" : "local", + "baseDir" : "examples/indexing/", + "filter" : "wikipedia_data.json" + }, + "inputFormat": { + "type": "json" + } + }, + "tuningConfig" : { + "type" : "index", + "partitionsSpec": { + "type": "single_dim", + "partitionDimension": "country", + "targetRowsPerSegment": 5000000 + } + } + } +} +``` + +## Learn more + +For more information, see the following pages: + +- [Ingestion](./index.md): Overview of the Druid ingestion process. +- [Native batch ingestion](./native-batch.md): Description of the supported native batch indexing tasks. +- [Ingestion spec reference](./ingestion-spec.md): Description of the components and properties in the ingestion spec. diff --git a/docs/ingestion/native-batch-firehose.md b/docs/ingestion/native-batch-firehose.md index 4e2cad97fc1a..ca848e725c87 100644 --- a/docs/ingestion/native-batch-firehose.md +++ b/docs/ingestion/native-batch-firehose.md @@ -1,6 +1,6 @@ --- id: native-batch-firehose -title: "Native batch ingestion with firehose" +title: "Native batch ingestion with firehose (Deprecated)" sidebar_label: "Firehose (deprecated)" --- @@ -23,14 +23,13 @@ sidebar_label: "Firehose (deprecated)" ~ under the License. --> - -Firehoses are deprecated in 0.17.0. It's highly recommended to use the [Native batch ingestion input sources](./native-batch-input-source.md) instead. +> Firehose ingestion is deprecated. See [Migrate from firehose to input source ingestion](./migrate-from-firehose-ingestion.md) for instructions on migrating from firehose ingestion to using native batch ingestion input sources. There are several firehoses readily available in Druid, some are meant for examples, others can be used directly in a production environment. ## StaticS3Firehose -> You need to include the [`druid-s3-extensions`](../development/extensions-core/s3.md) as an extension to use the StaticS3Firehose. +You need to include the [`druid-s3-extensions`](../development/extensions-core/s3.md) as an extension to use the StaticS3Firehose. This firehose ingests events from a predefined list of S3 objects. This firehose is _splittable_ and can be used by the [Parallel task](./native-batch.md). @@ -62,7 +61,7 @@ Note that prefetching or caching isn't that useful in the Parallel task. ## StaticGoogleBlobStoreFirehose -> You need to include the [`druid-google-extensions`](../development/extensions-core/google.md) as an extension to use the StaticGoogleBlobStoreFirehose. +You need to include the [`druid-google-extensions`](../development/extensions-core/google.md) as an extension to use the StaticGoogleBlobStoreFirehose. This firehose ingests events, similar to the StaticS3Firehose, but from an Google Cloud Store. @@ -112,7 +111,7 @@ Google Blobs: ## HDFSFirehose -> You need to include the [`druid-hdfs-storage`](../development/extensions-core/hdfs.md) as an extension to use the HDFSFirehose. +You need to include the [`druid-hdfs-storage`](../development/extensions-core/hdfs.md) as an extension to use the HDFSFirehose. This firehose ingests events from a predefined list of files from the HDFS storage. This firehose is _splittable_ and can be used by the [Parallel task](./native-batch.md). diff --git a/website/sidebars.json b/website/sidebars.json index a398f9fd3642..1ebc2140279f 100644 --- a/website/sidebars.json +++ b/website/sidebars.json @@ -57,6 +57,8 @@ "ids": [ "ingestion/native-batch", "ingestion/native-batch-input-sources", + "ingestion/migrate-from-firehose", + "ingestion/native-batch-firehose", "ingestion/hadoop" ] }, From cca9118f55e87cc217a597db2ba0738c315b00fc Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 24 Nov 2022 18:01:11 +0530 Subject: [PATCH 05/34] Port CVE suppressions from 24.0.1 (#13415) (#13430) * Suppress jackson-databind CVE-2022-42003 and CVE-2022-42004 (cherry picked from commit 1f4d892c9a2dbc3ce6df1481fd4c6d242ba0ea8d) * Suppress CVEs (cherry picked from commit ed55baa8fa7d7f914a0addabb072d9ed47e1cd9f) * Suppress vulnerabilities from druid-website package (cherry picked from commit c0fb364f8049d53cd704e414e2ffeab6c49b012e) * Add more suppressions for website package (cherry picked from commit 9bba569ebd52c5480bf4219c420ed78eb053701f) Co-authored-by: Rohan Garg <7731512+rohangarg@users.noreply.github.com> --- owasp-dependency-check-suppressions.xml | 100 +++++++++++++++++++++++- 1 file changed, 97 insertions(+), 3 deletions(-) diff --git a/owasp-dependency-check-suppressions.xml b/owasp-dependency-check-suppressions.xml index 6ffb3b9f2e1f..a09ed507cc83 100644 --- a/owasp-dependency-check-suppressions.xml +++ b/owasp-dependency-check-suppressions.xml @@ -88,6 +88,17 @@ ^pkg:maven/net\.minidev/accessors\-smart@.*$ CVE-2021-27568 + + + + ^pkg:maven/com\.fasterxml\.jackson\.core/jackson\-databind@.*$ + CVE-2022-42003 + CVE-2022-42004 + @@ -209,6 +220,15 @@ CVE-2018-1320 CVE-2019-0205 + + + + ^pkg:maven/org\.codehaus\.jettison/jettison@1.*$ + CVE-2022-40149 + CVE-2022-40150 + CVE-2019-12399 CVE-2018-17196 + + + ^pkg:maven/org\.apache\.kafka/kafka\-clients@.*$ + CVE-2022-34917 + + + ^pkg:maven/org\.ini4j/ini4j@.*$ + CVE-2022-41404 - CVE-2022-36364 + CVE-2022-39135 + + + + ^pkg:maven/org\.apache\.calcite/calcite\-core@.*$ + CVE-2020-13955 - CVE-2022-31197 - 1084597 + + + ^pkg:npm/d3\-color@.*$ + 1084597 + + + + ^pkg:maven/com\.google\.protobuf/protobuf\-java@.*$ + CVE-2022-3171 + + + + ^pkg:maven/com\.google\.protobuf/protobuf\-java\-util@.*$ + CVE-2022-3171 + + + + ^pkg:npm/ansi\-regex@.*$ + 1084697 + CVE-2021-3807 + + + + ^pkg:npm/glob\-parent@.*$ + 1081884 + CVE-2020-28469 + + + + ^pkg:npm/minimatch@.*$ + 1084765 + + + + ^pkg:npm/y18n@.*$ + 1070209 + CVE-2020-7774 + From 1046c75a5b3bdfb3e18e6a643806834de9559139 Mon Sep 17 00:00:00 2001 From: Tejaswini Bandlamudi <96047043+tejaswini-imply@users.noreply.github.com> Date: Sat, 26 Nov 2022 07:43:23 +0530 Subject: [PATCH 06/34] Fixes reindexing bug with filter on long column (#13386) (#13438) * fixes BlockLayoutColumnarLongs close method to nullify internal buffer. * fixes other BlockLayoutColumnar supplier close methods to nullify internal buffers. * fix spotbugs (cherry picked from commit b091b32f21f22cdb8daa48de06d18116d76f6780) --- .../input/DruidSegmentInputFormatTest.java | 3 +- .../input/DruidSegmentReaderTest.java | 355 +++++++++++------- .../BlockLayoutColumnarDoublesSupplier.java | 6 + .../BlockLayoutColumnarFloatsSupplier.java | 6 + .../BlockLayoutColumnarLongsSupplier.java | 8 + 5 files changed, 245 insertions(+), 133 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidSegmentInputFormatTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidSegmentInputFormatTest.java index 910371a56699..e4faea1c069c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidSegmentInputFormatTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidSegmentInputFormatTest.java @@ -19,6 +19,7 @@ package org.apache.druid.indexing.input; +import com.google.common.collect.ImmutableList; import org.apache.druid.data.input.ColumnsFilter; import org.apache.druid.data.input.InputEntityReader; import org.apache.druid.data.input.InputRowSchema; @@ -50,7 +51,7 @@ public void testDruidSegmentInputEntityReader() DruidSegmentInputFormat format = new DruidSegmentInputFormat(null, null); InputEntityReader reader = format.createReader( INPUT_ROW_SCHEMA, - DruidSegmentReaderTest.makeInputEntity(Intervals.of("2000/P1D"), null), + DruidSegmentReaderTest.makeInputEntity(Intervals.of("2000/P1D"), null, ImmutableList.of("s", "d"), ImmutableList.of("cnt", "met_s")), null ); Assert.assertTrue(reader instanceof DruidSegmentReader); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidSegmentReaderTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidSegmentReaderTest.java index 1638e79e9a70..ff4b50cee481 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidSegmentReaderTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidSegmentReaderTest.java @@ -31,6 +31,7 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.DoubleDimensionSchema; import org.apache.druid.data.input.impl.FileEntity; +import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.hll.HyperLogLogCollector; @@ -41,8 +42,11 @@ import org.apache.druid.java.util.common.guava.BaseSequence.IteratorMaker; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; +import org.apache.druid.query.filter.NotDimFilter; +import org.apache.druid.query.filter.OrDimFilter; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.IndexIO; @@ -80,67 +84,44 @@ public class DruidSegmentReaderTest extends NullHandlingTest private File segmentDirectory; private final IndexIO indexIO = TestHelper.getTestIndexIO(); + private DimensionsSpec dimensionsSpec; + private List metrics; + private List rows; @Before public void setUp() throws IOException { // Write a segment with two rows in it, with columns: s (string), d (double), cnt (long), met_s (complex). - final IncrementalIndex incrementalIndex = - IndexBuilder.create() - .schema( - new IncrementalIndexSchema.Builder() - .withDimensionsSpec( - new DimensionsSpec( - ImmutableList.of( - StringDimensionSchema.create("s"), - new DoubleDimensionSchema("d") - ) - ) - ) - .withMetrics( - new CountAggregatorFactory("cnt"), - new HyperUniquesAggregatorFactory("met_s", "s") - ) - .withRollup(false) - .build() - ) - .rows( - ImmutableList.of( - new MapBasedInputRow( - DateTimes.of("2000"), - ImmutableList.of("s", "d"), - ImmutableMap.builder() - .put("s", "foo") - .put("d", 1.23) - .build() - ), - new MapBasedInputRow( - DateTimes.of("2000T01"), - ImmutableList.of("s", "d"), - ImmutableMap.builder() - .put("s", "bar") - .put("d", 4.56) - .build() - ) - ) - ) - .buildIncrementalIndex(); - - segmentDirectory = temporaryFolder.newFolder(); + dimensionsSpec = new DimensionsSpec( + ImmutableList.of( + StringDimensionSchema.create("strCol"), + new DoubleDimensionSchema("dblCol") + ) + ); + metrics = ImmutableList.of( + new CountAggregatorFactory("cnt"), + new HyperUniquesAggregatorFactory("met_s", "strCol") + ); + rows = ImmutableList.of( + new MapBasedInputRow( + DateTimes.of("2000"), + ImmutableList.of("strCol", "dblCol"), + ImmutableMap.builder() + .put("strCol", "foo") + .put("dblCol", 1.23) + .build() + ), + new MapBasedInputRow( + DateTimes.of("2000T01"), + ImmutableList.of("strCol", "dblCol"), + ImmutableMap.builder() + .put("strCol", "bar") + .put("dblCol", 4.56) + .build() + ) + ); - try { - TestHelper.getTestIndexMergerV9( - OnHeapMemorySegmentWriteOutMediumFactory.instance() - ).persist( - incrementalIndex, - segmentDirectory, - new IndexSpec(), - null - ); - } - finally { - incrementalIndex.close(); - } + createTestSetup(); } @Test @@ -152,8 +133,8 @@ public void testReader() throws IOException new TimestampSpec("__time", "millis", DateTimes.of("1971")), new DimensionsSpec( ImmutableList.of( - StringDimensionSchema.create("s"), - new DoubleDimensionSchema("d") + StringDimensionSchema.create("strCol"), + new DoubleDimensionSchema("dblCol") ) ), ColumnsFilter.all(), @@ -165,22 +146,22 @@ public void testReader() throws IOException ImmutableList.of( new MapBasedInputRow( DateTimes.of("2000"), - ImmutableList.of("s", "d"), + ImmutableList.of("strCol", "dblCol"), ImmutableMap.builder() .put("__time", DateTimes.of("2000T").getMillis()) - .put("s", "foo") - .put("d", 1.23d) + .put("strCol", "foo") + .put("dblCol", 1.23d) .put("cnt", 1L) .put("met_s", makeHLLC("foo")) .build() ), new MapBasedInputRow( DateTimes.of("2000T01"), - ImmutableList.of("s", "d"), + ImmutableList.of("strCol", "dblCol"), ImmutableMap.builder() .put("__time", DateTimes.of("2000T01").getMillis()) - .put("s", "bar") - .put("d", 4.56d) + .put("strCol", "bar") + .put("dblCol", 4.56d) .put("cnt", 1L) .put("met_s", makeHLLC("bar")) .build() @@ -190,6 +171,74 @@ public void testReader() throws IOException ); } + @Test + public void testReaderWhenFilteringOnLongColumn() throws IOException + { + dimensionsSpec = new DimensionsSpec( + ImmutableList.of( + new LongDimensionSchema("longCol"), + StringDimensionSchema.create("a"), + StringDimensionSchema.create("b") + ) + ); + metrics = ImmutableList.of(); + + List columnNames = ImmutableList.of("longCol", "a", "b"); + rows = ImmutableList.of( + new MapBasedInputRow( + DateTimes.utc(1667115726217L), + columnNames, + ImmutableMap.builder() + .put("__time", 1667115726217L) + .put("longCol", 0L) + .put("a", "foo1") + .put("b", "bar1") + .build() + ), + new MapBasedInputRow( + DateTimes.utc(1667115726224L), + columnNames, + ImmutableMap.builder() + .put("__time", 1667115726224L) + .put("longCol", 0L) + .put("a", "foo2") + .put("b", "bar2") + .build() + ), + new MapBasedInputRow( + DateTimes.utc(1667115726128L), + columnNames, + ImmutableMap.builder() + .put("__time", 1667115726128L) + .put("longCol", 5L) + .put("a", "foo3") + .put("b", "bar3") + .build() + ) + ); + + createTestSetup(); + + final DruidSegmentReader reader = new DruidSegmentReader( + makeInputEntityWithParams(Intervals.of("2022-10-30/2022-10-31"), columnNames, null), + indexIO, + new TimestampSpec("__time", "iso", null), + dimensionsSpec, + ColumnsFilter.all(), + new OrDimFilter( + new SelectorDimFilter("longCol", "5", null), + new NotDimFilter(new SelectorDimFilter("a", "foo1", null)), + new NotDimFilter(new SelectorDimFilter("b", "bar1", null)) + ), + temporaryFolder.newFolder() + ); + + List expectedRows = new ArrayList<>(); + expectedRows.add(rows.get(2)); + expectedRows.add(rows.get(1)); + Assert.assertEquals(expectedRows, readRows(reader)); + } + @Test public void testDruidTombstoneSegmentReader() throws IOException { @@ -235,8 +284,8 @@ public void testReaderAutoTimestampFormat() throws IOException new TimestampSpec("__time", "auto", DateTimes.of("1971")), new DimensionsSpec( ImmutableList.of( - StringDimensionSchema.create("s"), - new DoubleDimensionSchema("d") + StringDimensionSchema.create("strCol"), + new DoubleDimensionSchema("dblCol") ) ), ColumnsFilter.all(), @@ -248,22 +297,22 @@ public void testReaderAutoTimestampFormat() throws IOException ImmutableList.of( new MapBasedInputRow( DateTimes.of("2000"), - ImmutableList.of("s", "d"), + ImmutableList.of("strCol", "dblCol"), ImmutableMap.builder() .put("__time", DateTimes.of("2000T").getMillis()) - .put("s", "foo") - .put("d", 1.23d) + .put("strCol", "foo") + .put("dblCol", 1.23d) .put("cnt", 1L) .put("met_s", makeHLLC("foo")) .build() ), new MapBasedInputRow( DateTimes.of("2000T01"), - ImmutableList.of("s", "d"), + ImmutableList.of("strCol", "dblCol"), ImmutableMap.builder() .put("__time", DateTimes.of("2000T01").getMillis()) - .put("s", "bar") - .put("d", 4.56d) + .put("strCol", "bar") + .put("dblCol", 4.56d) .put("cnt", 1L) .put("met_s", makeHLLC("bar")) .build() @@ -280,7 +329,7 @@ public void testReaderWithDimensionExclusions() throws IOException makeInputEntity(Intervals.of("2000/P1D")), indexIO, new TimestampSpec("__time", "millis", DateTimes.of("1971")), - DimensionsSpec.builder().setDimensionExclusions(ImmutableList.of("__time", "s", "cnt", "met_s")).build(), + DimensionsSpec.builder().setDimensionExclusions(ImmutableList.of("__time", "strCol", "cnt", "met_s")).build(), ColumnsFilter.all(), null, temporaryFolder.newFolder() @@ -290,22 +339,22 @@ public void testReaderWithDimensionExclusions() throws IOException ImmutableList.of( new MapBasedInputRow( DateTimes.of("2000"), - ImmutableList.of("d"), + ImmutableList.of("dblCol"), ImmutableMap.builder() .put("__time", DateTimes.of("2000T").getMillis()) - .put("s", "foo") - .put("d", 1.23d) + .put("strCol", "foo") + .put("dblCol", 1.23d) .put("cnt", 1L) .put("met_s", makeHLLC("foo")) .build() ), new MapBasedInputRow( DateTimes.of("2000T01"), - ImmutableList.of("d"), + ImmutableList.of("dblCol"), ImmutableMap.builder() .put("__time", DateTimes.of("2000T01").getMillis()) - .put("s", "bar") - .put("d", 4.56d) + .put("strCol", "bar") + .put("dblCol", 4.56d) .put("cnt", 1L) .put("met_s", makeHLLC("bar")) .build() @@ -324,11 +373,11 @@ public void testReaderWithInclusiveColumnsFilter() throws IOException new TimestampSpec("__time", "millis", DateTimes.of("1971")), new DimensionsSpec( ImmutableList.of( - StringDimensionSchema.create("s"), - new DoubleDimensionSchema("d") + StringDimensionSchema.create("strCol"), + new DoubleDimensionSchema("dblCol") ) ), - ColumnsFilter.inclusionBased(ImmutableSet.of("__time", "s", "d")), + ColumnsFilter.inclusionBased(ImmutableSet.of("__time", "strCol", "dblCol")), null, temporaryFolder.newFolder() ); @@ -337,20 +386,20 @@ public void testReaderWithInclusiveColumnsFilter() throws IOException ImmutableList.of( new MapBasedInputRow( DateTimes.of("2000"), - ImmutableList.of("s", "d"), + ImmutableList.of("strCol", "dblCol"), ImmutableMap.builder() .put("__time", DateTimes.of("2000T").getMillis()) - .put("s", "foo") - .put("d", 1.23d) + .put("strCol", "foo") + .put("dblCol", 1.23d) .build() ), new MapBasedInputRow( DateTimes.of("2000T01"), - ImmutableList.of("s", "d"), + ImmutableList.of("strCol", "dblCol"), ImmutableMap.builder() .put("__time", DateTimes.of("2000T01").getMillis()) - .put("s", "bar") - .put("d", 4.56d) + .put("strCol", "bar") + .put("dblCol", 4.56d) .build() ) ), @@ -367,11 +416,11 @@ public void testReaderWithInclusiveColumnsFilterNoTimestamp() throws IOException new TimestampSpec("__time", "millis", DateTimes.of("1971")), new DimensionsSpec( ImmutableList.of( - StringDimensionSchema.create("s"), - new DoubleDimensionSchema("d") + StringDimensionSchema.create("strCol"), + new DoubleDimensionSchema("dblCol") ) ), - ColumnsFilter.inclusionBased(ImmutableSet.of("s", "d")), + ColumnsFilter.inclusionBased(ImmutableSet.of("strCol", "dblCol")), null, temporaryFolder.newFolder() ); @@ -380,18 +429,18 @@ public void testReaderWithInclusiveColumnsFilterNoTimestamp() throws IOException ImmutableList.of( new MapBasedInputRow( DateTimes.of("1971"), - ImmutableList.of("s", "d"), + ImmutableList.of("strCol", "dblCol"), ImmutableMap.builder() - .put("s", "foo") - .put("d", 1.23d) + .put("strCol", "foo") + .put("dblCol", 1.23d) .build() ), new MapBasedInputRow( DateTimes.of("1971"), - ImmutableList.of("s", "d"), + ImmutableList.of("strCol", "dblCol"), ImmutableMap.builder() - .put("s", "bar") - .put("d", 4.56d) + .put("strCol", "bar") + .put("dblCol", 4.56d) .build() ) ), @@ -408,12 +457,12 @@ public void testReaderWithFilter() throws IOException new TimestampSpec("__time", "millis", DateTimes.of("1971")), new DimensionsSpec( ImmutableList.of( - StringDimensionSchema.create("s"), - new DoubleDimensionSchema("d") + StringDimensionSchema.create("strCol"), + new DoubleDimensionSchema("dblCol") ) ), ColumnsFilter.all(), - new SelectorDimFilter("d", "1.23", null), + new SelectorDimFilter("dblCol", "1.23", null), temporaryFolder.newFolder() ); @@ -421,11 +470,11 @@ public void testReaderWithFilter() throws IOException ImmutableList.of( new MapBasedInputRow( DateTimes.of("2000"), - ImmutableList.of("s", "d"), + ImmutableList.of("strCol", "dblCol"), ImmutableMap.builder() .put("__time", DateTimes.of("2000T").getMillis()) - .put("s", "foo") - .put("d", 1.23d) + .put("strCol", "foo") + .put("dblCol", 1.23d) .put("cnt", 1L) .put("met_s", makeHLLC("foo")) .build() @@ -441,11 +490,11 @@ public void testReaderTimestampFromDouble() throws IOException final DruidSegmentReader reader = new DruidSegmentReader( makeInputEntity(Intervals.of("2000/P1D")), indexIO, - new TimestampSpec("d", "posix", null), + new TimestampSpec("dblCol", "posix", null), new DimensionsSpec( ImmutableList.of( - StringDimensionSchema.create("s"), - new DoubleDimensionSchema("d") + StringDimensionSchema.create("strCol"), + new DoubleDimensionSchema("dblCol") ) ), ColumnsFilter.all(), @@ -457,22 +506,22 @@ public void testReaderTimestampFromDouble() throws IOException ImmutableList.of( new MapBasedInputRow( DateTimes.of("1970-01-01T00:00:01.000Z"), - ImmutableList.of("s", "d"), + ImmutableList.of("strCol", "dblCol"), ImmutableMap.builder() .put("__time", DateTimes.of("2000T").getMillis()) - .put("s", "foo") - .put("d", 1.23d) + .put("strCol", "foo") + .put("dblCol", 1.23d) .put("cnt", 1L) .put("met_s", makeHLLC("foo")) .build() ), new MapBasedInputRow( DateTimes.of("1970-01-01T00:00:04.000Z"), - ImmutableList.of("s", "d"), + ImmutableList.of("strCol", "dblCol"), ImmutableMap.builder() .put("__time", DateTimes.of("2000T01").getMillis()) - .put("s", "bar") - .put("d", 4.56d) + .put("strCol", "bar") + .put("dblCol", 4.56d) .put("cnt", 1L) .put("met_s", makeHLLC("bar")) .build() @@ -491,8 +540,8 @@ public void testReaderTimestampAsPosixIncorrectly() throws IOException new TimestampSpec("__time", "posix", null), new DimensionsSpec( ImmutableList.of( - StringDimensionSchema.create("s"), - new DoubleDimensionSchema("d") + StringDimensionSchema.create("strCol"), + new DoubleDimensionSchema("dblCol") ) ), ColumnsFilter.all(), @@ -504,22 +553,22 @@ public void testReaderTimestampAsPosixIncorrectly() throws IOException ImmutableList.of( new MapBasedInputRow( DateTimes.of("31969-04-01T00:00:00.000Z"), - ImmutableList.of("s", "d"), + ImmutableList.of("strCol", "dblCol"), ImmutableMap.builder() .put("__time", DateTimes.of("2000T").getMillis()) - .put("s", "foo") - .put("d", 1.23d) + .put("strCol", "foo") + .put("dblCol", 1.23d) .put("cnt", 1L) .put("met_s", makeHLLC("foo")) .build() ), new MapBasedInputRow( DateTimes.of("31969-05-12T16:00:00.000Z"), - ImmutableList.of("s", "d"), + ImmutableList.of("strCol", "dblCol"), ImmutableMap.builder() .put("__time", DateTimes.of("2000T01").getMillis()) - .put("s", "bar") - .put("d", 4.56d) + .put("strCol", "bar") + .put("dblCol", 4.56d) .put("cnt", 1L) .put("met_s", makeHLLC("bar")) .build() @@ -538,8 +587,8 @@ public void testReaderTimestampSpecDefault() throws IOException new TimestampSpec(null, null, DateTimes.of("1971")), new DimensionsSpec( ImmutableList.of( - StringDimensionSchema.create("s"), - new DoubleDimensionSchema("d") + StringDimensionSchema.create("strCol"), + new DoubleDimensionSchema("dblCol") ) ), ColumnsFilter.all(), @@ -551,22 +600,22 @@ public void testReaderTimestampSpecDefault() throws IOException ImmutableList.of( new MapBasedInputRow( DateTimes.of("1971"), - ImmutableList.of("s", "d"), + ImmutableList.of("strCol", "dblCol"), ImmutableMap.builder() .put("__time", DateTimes.of("2000T").getMillis()) - .put("s", "foo") - .put("d", 1.23d) + .put("strCol", "foo") + .put("dblCol", 1.23d) .put("cnt", 1L) .put("met_s", makeHLLC("foo")) .build() ), new MapBasedInputRow( DateTimes.of("1971"), - ImmutableList.of("s", "d"), + ImmutableList.of("strCol", "dblCol"), ImmutableMap.builder() .put("__time", DateTimes.of("2000T01").getMillis()) - .put("s", "bar") - .put("d", 4.56d) + .put("strCol", "bar") + .put("dblCol", 4.56d) .put("cnt", 1L) .put("met_s", makeHLLC("bar")) .build() @@ -623,10 +672,20 @@ public void close() private DruidSegmentInputEntity makeInputEntity(final Interval interval) { - return makeInputEntity(interval, segmentDirectory); + return makeInputEntity(interval, segmentDirectory, ImmutableList.of("strCol", "dblCol"), ImmutableList.of("cnt", "met_s")); + } + + private DruidSegmentInputEntity makeInputEntityWithParams(final Interval interval, final List dimensions, final List metrics) + { + return makeInputEntity(interval, segmentDirectory, dimensions, metrics); } - public static DruidSegmentInputEntity makeInputEntity(final Interval interval, final File segmentDirectory) + public static DruidSegmentInputEntity makeInputEntity( + final Interval interval, + final File segmentDirectory, + final List dimensions, + final List metrics + ) { return new DruidSegmentInputEntity( new SegmentCacheManager() @@ -669,9 +728,9 @@ public void loadSegmentIntoPageCache(DataSegment segment, ExecutorService exec) }, DataSegment.builder() .dataSource("ds") - .dimensions(ImmutableList.of("s", "d")) - .metrics(ImmutableList.of("cnt", "met_s")) - .interval(Intervals.of("2000/P1D")) + .dimensions(dimensions) + .metrics(metrics) + .interval(interval) .version("1") .size(0) .build(), @@ -765,4 +824,36 @@ private static HyperLogLogCollector makeHLLC(final String... values) } return collector; } + + private void createTestSetup() throws IOException + { + final IncrementalIndex incrementalIndex = + IndexBuilder.create() + .schema( + new IncrementalIndexSchema.Builder() + .withDimensionsSpec(dimensionsSpec) + .withMetrics(metrics.toArray(new AggregatorFactory[0])) + .withRollup(false) + .build() + ) + .rows(rows) + .buildIncrementalIndex(); + + segmentDirectory = temporaryFolder.newFolder(); + + try { + TestHelper.getTestIndexMergerV9( + OnHeapMemorySegmentWriteOutMediumFactory.instance() + ).persist( + incrementalIndex, + segmentDirectory, + new IndexSpec(), + null + ); + } + finally { + incrementalIndex.close(); + } + } + } diff --git a/processing/src/main/java/org/apache/druid/segment/data/BlockLayoutColumnarDoublesSupplier.java b/processing/src/main/java/org/apache/druid/segment/data/BlockLayoutColumnarDoublesSupplier.java index 28b3c5d2b6c3..98a7ab51f987 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/BlockLayoutColumnarDoublesSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/data/BlockLayoutColumnarDoublesSupplier.java @@ -22,6 +22,7 @@ import com.google.common.base.Supplier; import org.apache.druid.collections.ResourceHolder; +import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.nio.DoubleBuffer; @@ -82,10 +83,12 @@ private class BlockLayoutColumnarDoubles implements ColumnarDoubles final Indexed> singleThreadedDoubleBuffers = baseDoubleBuffers.singleThreaded(); int currBufferNum = -1; + @Nullable ResourceHolder holder; /** * doubleBuffer's position must be 0 */ + @Nullable DoubleBuffer doubleBuffer; @Override @@ -180,7 +183,10 @@ protected void loadBuffer(int bufferNum) public void close() { if (holder != null) { + currBufferNum = -1; holder.close(); + holder = null; + doubleBuffer = null; } } diff --git a/processing/src/main/java/org/apache/druid/segment/data/BlockLayoutColumnarFloatsSupplier.java b/processing/src/main/java/org/apache/druid/segment/data/BlockLayoutColumnarFloatsSupplier.java index c11ba18ec3df..26d7c798c79e 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/BlockLayoutColumnarFloatsSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/data/BlockLayoutColumnarFloatsSupplier.java @@ -22,6 +22,7 @@ import com.google.common.base.Supplier; import org.apache.druid.collections.ResourceHolder; +import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.nio.FloatBuffer; @@ -82,10 +83,12 @@ private class BlockLayoutColumnarFloats implements ColumnarFloats final Indexed> singleThreadedFloatBuffers = baseFloatBuffers.singleThreaded(); int currBufferNum = -1; + @Nullable ResourceHolder holder; /** * floatBuffer's position must be 0 */ + @Nullable FloatBuffer floatBuffer; @Override @@ -180,7 +183,10 @@ protected void loadBuffer(int bufferNum) public void close() { if (holder != null) { + currBufferNum = -1; holder.close(); + holder = null; + floatBuffer = null; } } diff --git a/processing/src/main/java/org/apache/druid/segment/data/BlockLayoutColumnarLongsSupplier.java b/processing/src/main/java/org/apache/druid/segment/data/BlockLayoutColumnarLongsSupplier.java index 05473320553f..29a0748bcee5 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/BlockLayoutColumnarLongsSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/data/BlockLayoutColumnarLongsSupplier.java @@ -22,6 +22,7 @@ import com.google.common.base.Supplier; import org.apache.druid.collections.ResourceHolder; +import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.nio.LongBuffer; @@ -123,11 +124,14 @@ private class BlockLayoutColumnarLongs implements ColumnarLongs final Indexed> singleThreadedLongBuffers = baseLongBuffers.singleThreaded(); int currBufferNum = -1; + @Nullable ResourceHolder holder; + @Nullable ByteBuffer buffer; /** * longBuffer's position must be 0 */ + @Nullable LongBuffer longBuffer; @Override @@ -204,7 +208,11 @@ protected void loadBuffer(int bufferNum) public void close() { if (holder != null) { + currBufferNum = -1; holder.close(); + holder = null; + buffer = null; + longBuffer = null; } } From 7916770886388d29a06b2d6596f6270f50242c71 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Sat, 26 Nov 2022 04:11:26 -0800 Subject: [PATCH 07/34] Add mechanism for 'safe' memory reads for complex types (#13361) (#13421) * we can read where we want to we can leave your bounds behind 'cause if the memory is not there we really don't care and we'll crash this process of mine --- .../hll/HllSketchMergeComplexMetricSerde.java | 15 +- .../hll/HllSketchObjectStrategy.java | 10 + .../KllDoublesSketchComplexMetricSerde.java | 2 +- .../kll/KllDoublesSketchObjectStrategy.java | 13 + .../kll/KllDoublesSketchOperations.java | 21 + .../KllFloatsSketchComplexMetricSerde.java | 2 +- .../kll/KllFloatsSketchObjectStrategy.java | 13 + .../kll/KllFloatsSketchOperations.java | 21 + .../DoublesSketchComplexMetricSerde.java | 2 +- .../DoublesSketchObjectStrategy.java | 13 + .../quantiles/DoublesSketchOperations.java | 20 + .../theta/SketchConstantPostAggregator.java | 2 +- .../datasketches/theta/SketchHolder.java | 22 + .../theta/SketchHolderObjectStrategy.java | 14 + .../theta/SketchMergeComplexMetricSerde.java | 2 +- ...fDoublesSketchMergeComplexMetricSerde.java | 2 +- .../ArrayOfDoublesSketchObjectStrategy.java | 13 +- .../tuple/ArrayOfDoublesSketchOperations.java | 24 +- .../hll/HllSketchObjectStrategyTest.java | 77 +++ ...llDoublesSketchComplexMetricSerdeTest.java | 44 ++ .../kll/KllDoublesSketchOperationsTest.java | 51 ++ ...KllFloatsSketchComplexMetricSerdeTest.java | 44 ++ .../kll/KllFloatsSketchOperationsTest.java | 51 ++ .../DoublesSketchComplexMetricSerdeTest.java | 43 ++ .../DoublesSketchOperationsTest.java | 50 ++ .../theta/SketchHolderObjectStrategyTest.java | 79 +++ .../datasketches/theta/SketchHolderTest.java | 52 ++ ...rrayOfDoublesSketchObjectStrategyTest.java | 70 +++ .../ArrayOfDoublesSketchOperationsTest.java | 55 ++ .../ObjectStrategyComplexTypeStrategy.java | 2 +- .../druid/segment/data/ObjectStrategy.java | 27 + .../druid/segment/data/SafeWritableBase.java | 450 ++++++++++++++++ .../segment/data/SafeWritableBuffer.java | 501 ++++++++++++++++++ .../segment/data/SafeWritableMemory.java | 417 +++++++++++++++ .../segment/data/SafeWritableBufferTest.java | 224 ++++++++ .../segment/data/SafeWritableMemoryTest.java | 359 +++++++++++++ 36 files changed, 2796 insertions(+), 11 deletions(-) create mode 100644 extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchObjectStrategyTest.java create mode 100644 extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchOperationsTest.java create mode 100644 extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchOperationsTest.java create mode 100644 extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchOperationsTest.java create mode 100644 extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchHolderObjectStrategyTest.java create mode 100644 extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchHolderTest.java create mode 100644 extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchObjectStrategyTest.java create mode 100644 extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchOperationsTest.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/SafeWritableBase.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/SafeWritableBuffer.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/SafeWritableMemory.java create mode 100644 processing/src/test/java/org/apache/druid/segment/data/SafeWritableBufferTest.java create mode 100644 processing/src/test/java/org/apache/druid/segment/data/SafeWritableMemoryTest.java diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeComplexMetricSerde.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeComplexMetricSerde.java index c8ac48ab186e..1063bbdfec1a 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeComplexMetricSerde.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeComplexMetricSerde.java @@ -28,6 +28,7 @@ import org.apache.druid.segment.column.ColumnBuilder; import org.apache.druid.segment.data.GenericIndexed; import org.apache.druid.segment.data.ObjectStrategy; +import org.apache.druid.segment.data.SafeWritableMemory; import org.apache.druid.segment.serde.ComplexColumnPartSupplier; import org.apache.druid.segment.serde.ComplexMetricExtractor; import org.apache.druid.segment.serde.ComplexMetricSerde; @@ -70,7 +71,7 @@ public HllSketch extractValue(final InputRow inputRow, final String metricName) if (object == null) { return null; } - return deserializeSketch(object); + return deserializeSketchSafe(object); } }; } @@ -98,6 +99,18 @@ static HllSketch deserializeSketch(final Object object) throw new IAE("Object is not of a type that can be deserialized to an HllSketch:" + object.getClass().getName()); } + static HllSketch deserializeSketchSafe(final Object object) + { + if (object instanceof String) { + return HllSketch.wrap(SafeWritableMemory.wrap(StringUtils.decodeBase64(((String) object).getBytes(StandardCharsets.UTF_8)))); + } else if (object instanceof byte[]) { + return HllSketch.wrap(SafeWritableMemory.wrap((byte[]) object)); + } else if (object instanceof HllSketch) { + return (HllSketch) object; + } + throw new IAE("Object is not of a type that can be deserialized to an HllSketch:" + object.getClass().getName()); + } + // support large columns @Override public GenericColumnSerializer getSerializer(final SegmentWriteOutMedium segmentWriteOutMedium, final String column) diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchObjectStrategy.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchObjectStrategy.java index 34145863fdf8..65257b22b796 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchObjectStrategy.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchObjectStrategy.java @@ -22,7 +22,9 @@ import org.apache.datasketches.hll.HllSketch; import org.apache.datasketches.memory.Memory; import org.apache.druid.segment.data.ObjectStrategy; +import org.apache.druid.segment.data.SafeWritableMemory; +import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.nio.ByteOrder; @@ -55,4 +57,12 @@ public byte[] toBytes(final HllSketch sketch) return sketch.toCompactByteArray(); } + @Nullable + @Override + public HllSketch fromByteBufferSafe(ByteBuffer buffer, int numBytes) + { + return HllSketch.wrap( + SafeWritableMemory.wrap(buffer, ByteOrder.LITTLE_ENDIAN).region(buffer.position(), numBytes) + ); + } } diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchComplexMetricSerde.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchComplexMetricSerde.java index 4c18a9785607..e5249853ac34 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchComplexMetricSerde.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchComplexMetricSerde.java @@ -91,7 +91,7 @@ public Object extractValue(final InputRow inputRow, final String metricName) if (object == null || object instanceof KllDoublesSketch || object instanceof Memory) { return object; } - return KllDoublesSketchOperations.deserialize(object); + return KllDoublesSketchOperations.deserializeSafe(object); } }; } diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchObjectStrategy.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchObjectStrategy.java index 97e670a625a5..17cb94e2fcf8 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchObjectStrategy.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchObjectStrategy.java @@ -23,7 +23,9 @@ import org.apache.datasketches.kll.KllDoublesSketch; import org.apache.datasketches.memory.Memory; import org.apache.druid.segment.data.ObjectStrategy; +import org.apache.druid.segment.data.SafeWritableMemory; +import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.nio.ByteOrder; @@ -60,4 +62,15 @@ public byte[] toBytes(final KllDoublesSketch sketch) return sketch.toByteArray(); } + @Nullable + @Override + public KllDoublesSketch fromByteBufferSafe(ByteBuffer buffer, int numBytes) + { + if (numBytes == 0) { + return KllDoublesSketchOperations.EMPTY_SKETCH; + } + return KllDoublesSketch.wrap( + SafeWritableMemory.wrap(buffer, ByteOrder.LITTLE_ENDIAN).region(buffer.position(), numBytes) + ); + } } diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchOperations.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchOperations.java index 57cb51747101..6da454d7f868 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchOperations.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchOperations.java @@ -23,6 +23,7 @@ import org.apache.datasketches.memory.Memory; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.segment.data.SafeWritableMemory; import java.nio.charset.StandardCharsets; @@ -46,6 +47,16 @@ public static KllDoublesSketch deserialize(final Object serializedSketch) ); } + public static KllDoublesSketch deserializeSafe(final Object serializedSketch) + { + if (serializedSketch instanceof String) { + return deserializeFromBase64EncodedStringSafe((String) serializedSketch); + } else if (serializedSketch instanceof byte[]) { + return deserializeFromByteArraySafe((byte[]) serializedSketch); + } + return deserialize(serializedSketch); + } + public static KllDoublesSketch deserializeFromBase64EncodedString(final String str) { return deserializeFromByteArray(StringUtils.decodeBase64(str.getBytes(StandardCharsets.UTF_8))); @@ -56,4 +67,14 @@ public static KllDoublesSketch deserializeFromByteArray(final byte[] data) return KllDoublesSketch.wrap(Memory.wrap(data)); } + public static KllDoublesSketch deserializeFromBase64EncodedStringSafe(final String str) + { + return deserializeFromByteArraySafe(StringUtils.decodeBase64(str.getBytes(StandardCharsets.UTF_8))); + } + + public static KllDoublesSketch deserializeFromByteArraySafe(final byte[] data) + { + return KllDoublesSketch.wrap(SafeWritableMemory.wrap(data)); + } + } diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchComplexMetricSerde.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchComplexMetricSerde.java index 4a71befe0c66..175b307ec33e 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchComplexMetricSerde.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchComplexMetricSerde.java @@ -91,7 +91,7 @@ public Object extractValue(final InputRow inputRow, final String metricName) if (object == null || object instanceof KllFloatsSketch || object instanceof Memory) { return object; } - return KllFloatsSketchOperations.deserialize(object); + return KllFloatsSketchOperations.deserializeSafe(object); } }; } diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchObjectStrategy.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchObjectStrategy.java index ff177a2f54f1..93ff0a7dba52 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchObjectStrategy.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchObjectStrategy.java @@ -23,7 +23,9 @@ import org.apache.datasketches.kll.KllFloatsSketch; import org.apache.datasketches.memory.Memory; import org.apache.druid.segment.data.ObjectStrategy; +import org.apache.druid.segment.data.SafeWritableMemory; +import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.nio.ByteOrder; @@ -60,4 +62,15 @@ public byte[] toBytes(final KllFloatsSketch sketch) return sketch.toByteArray(); } + @Nullable + @Override + public KllFloatsSketch fromByteBufferSafe(ByteBuffer buffer, int numBytes) + { + if (numBytes == 0) { + return KllFloatsSketchOperations.EMPTY_SKETCH; + } + return KllFloatsSketch.wrap( + SafeWritableMemory.wrap(buffer, ByteOrder.LITTLE_ENDIAN).region(buffer.position(), numBytes) + ); + } } diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchOperations.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchOperations.java index e32b67b2549a..02fb615da4dc 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchOperations.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchOperations.java @@ -23,6 +23,7 @@ import org.apache.datasketches.memory.Memory; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.segment.data.SafeWritableMemory; import java.nio.charset.StandardCharsets; @@ -46,6 +47,16 @@ public static KllFloatsSketch deserialize(final Object serializedSketch) ); } + public static KllFloatsSketch deserializeSafe(final Object serializedSketch) + { + if (serializedSketch instanceof String) { + return deserializeFromBase64EncodedStringSafe((String) serializedSketch); + } else if (serializedSketch instanceof byte[]) { + return deserializeFromByteArraySafe((byte[]) serializedSketch); + } + return deserialize(serializedSketch); + } + public static KllFloatsSketch deserializeFromBase64EncodedString(final String str) { return deserializeFromByteArray(StringUtils.decodeBase64(str.getBytes(StandardCharsets.UTF_8))); @@ -56,4 +67,14 @@ public static KllFloatsSketch deserializeFromByteArray(final byte[] data) return KllFloatsSketch.wrap(Memory.wrap(data)); } + public static KllFloatsSketch deserializeFromBase64EncodedStringSafe(final String str) + { + return deserializeFromByteArraySafe(StringUtils.decodeBase64(str.getBytes(StandardCharsets.UTF_8))); + } + + public static KllFloatsSketch deserializeFromByteArraySafe(final byte[] data) + { + return KllFloatsSketch.wrap(SafeWritableMemory.wrap(data)); + } + } diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchComplexMetricSerde.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchComplexMetricSerde.java index d97b5f8c6dd3..3614f214c708 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchComplexMetricSerde.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchComplexMetricSerde.java @@ -92,7 +92,7 @@ public Object extractValue(final InputRow inputRow, final String metricName) if (object == null || object instanceof DoublesSketch || object instanceof Memory) { return object; } - return DoublesSketchOperations.deserialize(object); + return DoublesSketchOperations.deserializeSafe(object); } }; } diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchObjectStrategy.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchObjectStrategy.java index 826de9378f8c..569b60bf03f7 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchObjectStrategy.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchObjectStrategy.java @@ -23,7 +23,9 @@ import org.apache.datasketches.memory.Memory; import org.apache.datasketches.quantiles.DoublesSketch; import org.apache.druid.segment.data.ObjectStrategy; +import org.apache.druid.segment.data.SafeWritableMemory; +import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.nio.ByteOrder; @@ -60,4 +62,15 @@ public byte[] toBytes(final DoublesSketch sketch) return sketch.toByteArray(true); } + @Nullable + @Override + public DoublesSketch fromByteBufferSafe(ByteBuffer buffer, int numBytes) + { + if (numBytes == 0) { + return DoublesSketchOperations.EMPTY_SKETCH; + } + return DoublesSketch.wrap( + SafeWritableMemory.wrap(buffer, ByteOrder.LITTLE_ENDIAN).region(buffer.position(), numBytes) + ); + } } diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchOperations.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchOperations.java index e30fb9bdae83..a2ca197c11ac 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchOperations.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchOperations.java @@ -23,6 +23,7 @@ import org.apache.datasketches.quantiles.DoublesSketch; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.segment.data.SafeWritableMemory; import java.nio.charset.StandardCharsets; @@ -46,6 +47,16 @@ public static DoublesSketch deserialize(final Object serializedSketch) ); } + public static DoublesSketch deserializeSafe(final Object serializedSketch) + { + if (serializedSketch instanceof String) { + return deserializeFromBase64EncodedStringSafe((String) serializedSketch); + } else if (serializedSketch instanceof byte[]) { + return deserializeFromByteArraySafe((byte[]) serializedSketch); + } + return deserialize(serializedSketch); + } + public static DoublesSketch deserializeFromBase64EncodedString(final String str) { return deserializeFromByteArray(StringUtils.decodeBase64(str.getBytes(StandardCharsets.UTF_8))); @@ -56,4 +67,13 @@ public static DoublesSketch deserializeFromByteArray(final byte[] data) return DoublesSketch.wrap(Memory.wrap(data)); } + public static DoublesSketch deserializeFromBase64EncodedStringSafe(final String str) + { + return deserializeFromByteArraySafe(StringUtils.decodeBase64(str.getBytes(StandardCharsets.UTF_8))); + } + + public static DoublesSketch deserializeFromByteArraySafe(final byte[] data) + { + return DoublesSketch.wrap(SafeWritableMemory.wrap(data)); + } } diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchConstantPostAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchConstantPostAggregator.java index b3541bd50649..64c182a0d62a 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchConstantPostAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchConstantPostAggregator.java @@ -51,7 +51,7 @@ public SketchConstantPostAggregator(@JsonProperty("name") String name, @JsonProp Preconditions.checkArgument(value != null && !value.isEmpty(), "Constant value cannot be null or empty, expecting base64 encoded sketch string"); this.value = value; - this.sketchValue = SketchHolder.deserialize(value); + this.sketchValue = SketchHolder.deserializeSafe(value); } @Override diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchHolder.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchHolder.java index 59ca453bb25a..838b4ae91f0f 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchHolder.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchHolder.java @@ -34,6 +34,7 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.segment.data.SafeWritableMemory; import javax.annotation.Nullable; @@ -224,6 +225,17 @@ public static SketchHolder deserialize(Object serializedSketch) ); } + public static SketchHolder deserializeSafe(Object serializedSketch) + { + if (serializedSketch instanceof String) { + return SketchHolder.of(deserializeFromBase64EncodedStringSafe((String) serializedSketch)); + } else if (serializedSketch instanceof byte[]) { + return SketchHolder.of(deserializeFromByteArraySafe((byte[]) serializedSketch)); + } + + return deserialize(serializedSketch); + } + private static Sketch deserializeFromBase64EncodedString(String str) { return deserializeFromByteArray(StringUtils.decodeBase64(StringUtils.toUtf8(str))); @@ -234,6 +246,16 @@ private static Sketch deserializeFromByteArray(byte[] data) return deserializeFromMemory(Memory.wrap(data)); } + private static Sketch deserializeFromBase64EncodedStringSafe(String str) + { + return deserializeFromByteArraySafe(StringUtils.decodeBase64(StringUtils.toUtf8(str))); + } + + private static Sketch deserializeFromByteArraySafe(byte[] data) + { + return deserializeFromMemory(SafeWritableMemory.wrap(data)); + } + private static Sketch deserializeFromMemory(Memory mem) { if (Sketch.getSerializationVersion(mem) < 3) { diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchHolderObjectStrategy.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchHolderObjectStrategy.java index e98bc3d95a04..96fafe826211 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchHolderObjectStrategy.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchHolderObjectStrategy.java @@ -23,6 +23,7 @@ import org.apache.datasketches.memory.Memory; import org.apache.datasketches.theta.Sketch; import org.apache.druid.segment.data.ObjectStrategy; +import org.apache.druid.segment.data.SafeWritableMemory; import javax.annotation.Nullable; import java.nio.ByteBuffer; @@ -66,4 +67,17 @@ public byte[] toBytes(@Nullable SketchHolder obj) return ByteArrays.EMPTY_ARRAY; } } + + @Nullable + @Override + public SketchHolder fromByteBufferSafe(ByteBuffer buffer, int numBytes) + { + if (numBytes == 0) { + return SketchHolder.EMPTY; + } + + return SketchHolder.of( + SafeWritableMemory.wrap(buffer, ByteOrder.LITTLE_ENDIAN).region(buffer.position(), numBytes) + ); + } } diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchMergeComplexMetricSerde.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchMergeComplexMetricSerde.java index a824312c0ef7..4f3ecfae2915 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchMergeComplexMetricSerde.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchMergeComplexMetricSerde.java @@ -59,7 +59,7 @@ public Class extractedClass() public SketchHolder extractValue(InputRow inputRow, String metricName) { final Object object = inputRow.getRaw(metricName); - return object == null ? null : SketchHolder.deserialize(object); + return object == null ? null : SketchHolder.deserializeSafe(object); } }; } diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchMergeComplexMetricSerde.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchMergeComplexMetricSerde.java index 19c8da292b4e..028bcdc35491 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchMergeComplexMetricSerde.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchMergeComplexMetricSerde.java @@ -60,7 +60,7 @@ public Object extractValue(final InputRow inputRow, final String metricName) if (object == null || object instanceof ArrayOfDoublesSketch) { return object; } - return ArrayOfDoublesSketchOperations.deserialize(object); + return ArrayOfDoublesSketchOperations.deserializeSafe(object); } }; } diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchObjectStrategy.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchObjectStrategy.java index 1ae950e068f8..f893c83b5707 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchObjectStrategy.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchObjectStrategy.java @@ -23,6 +23,7 @@ import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketch; import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketches; import org.apache.druid.segment.data.ObjectStrategy; +import org.apache.druid.segment.data.SafeWritableMemory; import javax.annotation.Nullable; import java.nio.ByteBuffer; @@ -48,7 +49,9 @@ public Class getClazz() @Override public ArrayOfDoublesSketch fromByteBuffer(final ByteBuffer buffer, final int numBytes) { - return ArrayOfDoublesSketches.wrapSketch(Memory.wrap(buffer, ByteOrder.LITTLE_ENDIAN).region(buffer.position(), numBytes)); + return ArrayOfDoublesSketches.wrapSketch( + Memory.wrap(buffer, ByteOrder.LITTLE_ENDIAN).region(buffer.position(), numBytes) + ); } @Override @@ -61,4 +64,12 @@ public byte[] toBytes(@Nullable final ArrayOfDoublesSketch sketch) return sketch.toByteArray(); } + @Nullable + @Override + public ArrayOfDoublesSketch fromByteBufferSafe(ByteBuffer buffer, int numBytes) + { + return ArrayOfDoublesSketches.wrapSketch( + SafeWritableMemory.wrap(buffer, ByteOrder.LITTLE_ENDIAN).region(buffer.position(), numBytes) + ); + } } diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchOperations.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchOperations.java index b1658a995714..2768858ffe32 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchOperations.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchOperations.java @@ -30,6 +30,7 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.segment.data.SafeWritableMemory; import java.nio.charset.StandardCharsets; @@ -115,6 +116,17 @@ public static ArrayOfDoublesSketch deserialize(final Object serializedSketch) throw new ISE("Object is not of a type that can deserialize to sketch: %s", serializedSketch.getClass()); } + public static ArrayOfDoublesSketch deserializeSafe(final Object serializedSketch) + { + if (serializedSketch instanceof String) { + return deserializeFromBase64EncodedStringSafe((String) serializedSketch); + } else if (serializedSketch instanceof byte[]) { + return deserializeFromByteArraySafe((byte[]) serializedSketch); + } + + return deserialize(serializedSketch); + } + public static ArrayOfDoublesSketch deserializeFromBase64EncodedString(final String str) { return deserializeFromByteArray(StringUtils.decodeBase64(str.getBytes(StandardCharsets.UTF_8))); @@ -122,8 +134,16 @@ public static ArrayOfDoublesSketch deserializeFromBase64EncodedString(final Stri public static ArrayOfDoublesSketch deserializeFromByteArray(final byte[] data) { - final Memory mem = Memory.wrap(data); - return ArrayOfDoublesSketches.wrapSketch(mem); + return ArrayOfDoublesSketches.wrapSketch(Memory.wrap(data)); + } + + public static ArrayOfDoublesSketch deserializeFromBase64EncodedStringSafe(final String str) + { + return deserializeFromByteArraySafe(StringUtils.decodeBase64(str.getBytes(StandardCharsets.UTF_8))); } + public static ArrayOfDoublesSketch deserializeFromByteArraySafe(final byte[] data) + { + return ArrayOfDoublesSketches.wrapSketch(SafeWritableMemory.wrap(data)); + } } diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchObjectStrategyTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchObjectStrategyTest.java new file mode 100644 index 000000000000..ff1eb947408e --- /dev/null +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchObjectStrategyTest.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.query.aggregation.datasketches.hll; + +import org.apache.datasketches.SketchesArgumentException; +import org.apache.datasketches.hll.HllSketch; +import org.apache.druid.java.util.common.StringUtils; +import org.junit.Assert; +import org.junit.Test; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +public class HllSketchObjectStrategyTest +{ + @Test + public void testSafeRead() + { + HllSketch sketch = new HllSketch(); + sketch.update(new int[]{1, 2, 3}); + + final byte[] bytes = sketch.toCompactByteArray(); + + ByteBuffer buf = ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN); + HllSketchObjectStrategy objectStrategy = new HllSketchObjectStrategy(); + + // valid sketch should not explode when copied, which reads the memory + objectStrategy.fromByteBufferSafe(buf, bytes.length).copy(); + + // corrupted sketch should fail with a regular java buffer exception + for (int subset = 3; subset < bytes.length - 1; subset++) { + final byte[] garbage2 = new byte[subset]; + for (int i = 0; i < garbage2.length; i++) { + garbage2[i] = buf.get(i); + } + + final ByteBuffer buf2 = ByteBuffer.wrap(garbage2).order(ByteOrder.LITTLE_ENDIAN); + Assert.assertThrows( + IndexOutOfBoundsException.class, + () -> objectStrategy.fromByteBufferSafe(buf2, garbage2.length).copy() + ); + } + + // non sketch that is too short to contain header should fail with regular java buffer exception + final byte[] garbage = new byte[]{0x01, 0x02}; + final ByteBuffer buf3 = ByteBuffer.wrap(garbage).order(ByteOrder.LITTLE_ENDIAN); + Assert.assertThrows( + IndexOutOfBoundsException.class, + () -> objectStrategy.fromByteBufferSafe(buf3, garbage.length).copy() + ); + + // non sketch that is long enough to check (this one doesn't actually need 'safe' read) + final byte[] garbageLonger = StringUtils.toUtf8("notasketch"); + final ByteBuffer buf4 = ByteBuffer.wrap(garbageLonger).order(ByteOrder.LITTLE_ENDIAN); + Assert.assertThrows( + SketchesArgumentException.class, + () -> objectStrategy.fromByteBufferSafe(buf4, garbageLonger.length).copy() + ); + } +} diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchComplexMetricSerdeTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchComplexMetricSerdeTest.java index 3628c5e6212f..0ae46bef496b 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchComplexMetricSerdeTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchComplexMetricSerdeTest.java @@ -23,10 +23,14 @@ import com.google.common.collect.ImmutableMap; import org.apache.datasketches.kll.KllDoublesSketch; import org.apache.druid.data.input.MapBasedInputRow; +import org.apache.druid.segment.data.ObjectStrategy; import org.apache.druid.segment.serde.ComplexMetricExtractor; import org.junit.Assert; import org.junit.Test; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + public class KllDoublesSketchComplexMetricSerdeTest { @Test @@ -92,4 +96,44 @@ public void testExtractorOnLeadingDecimalNumber() Assert.assertEquals(1, sketch.getNumRetained()); Assert.assertEquals(0.1d, sketch.getMaxValue(), 0.01d); } + + @Test + public void testSafeRead() + { + final KllDoublesSketchComplexMetricSerde serde = new KllDoublesSketchComplexMetricSerde(); + final ObjectStrategy objectStrategy = serde.getObjectStrategy(); + + KllDoublesSketch sketch = KllDoublesSketch.newHeapInstance(); + sketch.update(1.1); + sketch.update(1.2); + final byte[] bytes = sketch.toByteArray(); + + ByteBuffer buf = ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN); + + // valid sketch should not explode when converted to byte array, which reads the memory + objectStrategy.fromByteBufferSafe(buf, bytes.length).toByteArray(); + + // corrupted sketch should fail with a regular java buffer exception, not all subsets actually fail with the same + // index out of bounds exceptions, but at least this many do + for (int subset = 3; subset < 24; subset++) { + final byte[] garbage2 = new byte[subset]; + for (int i = 0; i < garbage2.length; i++) { + garbage2[i] = buf.get(i); + } + + final ByteBuffer buf2 = ByteBuffer.wrap(garbage2).order(ByteOrder.LITTLE_ENDIAN); + Assert.assertThrows( + IndexOutOfBoundsException.class, + () -> objectStrategy.fromByteBufferSafe(buf2, garbage2.length).toByteArray() + ); + } + + // non sketch that is too short to contain header should fail with regular java buffer exception + final byte[] garbage = new byte[]{0x01, 0x02}; + final ByteBuffer buf3 = ByteBuffer.wrap(garbage).order(ByteOrder.LITTLE_ENDIAN); + Assert.assertThrows( + IndexOutOfBoundsException.class, + () -> objectStrategy.fromByteBufferSafe(buf3, garbage.length).toByteArray() + ); + } } diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchOperationsTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchOperationsTest.java new file mode 100644 index 000000000000..d2b0e3839848 --- /dev/null +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchOperationsTest.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.query.aggregation.datasketches.kll; + +import org.apache.datasketches.kll.KllDoublesSketch; +import org.apache.druid.java.util.common.StringUtils; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; + +public class KllDoublesSketchOperationsTest +{ + @Test + public void testDeserializeSafe() + { + KllDoublesSketch sketch = KllDoublesSketch.newHeapInstance(); + sketch.update(1.1); + sketch.update(1.2); + final byte[] bytes = sketch.toByteArray(); + final String base64 = StringUtils.encodeBase64String(bytes); + + Assert.assertArrayEquals(bytes, KllDoublesSketchOperations.deserializeSafe(sketch).toByteArray()); + Assert.assertArrayEquals(bytes, KllDoublesSketchOperations.deserializeSafe(bytes).toByteArray()); + Assert.assertArrayEquals(bytes, KllDoublesSketchOperations.deserializeSafe(base64).toByteArray()); + + final byte[] trunacted = Arrays.copyOfRange(bytes, 0, 20); + Assert.assertThrows(IndexOutOfBoundsException.class, () -> KllDoublesSketchOperations.deserializeSafe(trunacted)); + Assert.assertThrows( + IndexOutOfBoundsException.class, + () -> KllDoublesSketchOperations.deserializeSafe(StringUtils.encodeBase64String(trunacted)) + ); + } +} diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchComplexMetricSerdeTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchComplexMetricSerdeTest.java index 5ff441df1c14..c6b8c310221c 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchComplexMetricSerdeTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchComplexMetricSerdeTest.java @@ -23,10 +23,14 @@ import com.google.common.collect.ImmutableMap; import org.apache.datasketches.kll.KllFloatsSketch; import org.apache.druid.data.input.MapBasedInputRow; +import org.apache.druid.segment.data.ObjectStrategy; import org.apache.druid.segment.serde.ComplexMetricExtractor; import org.junit.Assert; import org.junit.Test; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + public class KllFloatsSketchComplexMetricSerdeTest { @Test @@ -92,4 +96,44 @@ public void testExtractorOnLeadingDecimalNumber() Assert.assertEquals(1, sketch.getNumRetained()); Assert.assertEquals(0.1d, sketch.getMaxValue(), 0.01d); } + + @Test + public void testSafeRead() + { + final KllFloatsSketchComplexMetricSerde serde = new KllFloatsSketchComplexMetricSerde(); + final ObjectStrategy objectStrategy = serde.getObjectStrategy(); + + KllFloatsSketch sketch = KllFloatsSketch.newHeapInstance(); + sketch.update(1.1f); + sketch.update(1.2f); + final byte[] bytes = sketch.toByteArray(); + + ByteBuffer buf = ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN); + + // valid sketch should not explode when converted to byte array, which reads the memory + objectStrategy.fromByteBufferSafe(buf, bytes.length).toByteArray(); + + // corrupted sketch should fail with a regular java buffer exception, not all subsets actually fail with the same + // index out of bounds exceptions, but at least this many do + for (int subset = 3; subset < 24; subset++) { + final byte[] garbage2 = new byte[subset]; + for (int i = 0; i < garbage2.length; i++) { + garbage2[i] = buf.get(i); + } + + final ByteBuffer buf2 = ByteBuffer.wrap(garbage2).order(ByteOrder.LITTLE_ENDIAN); + Assert.assertThrows( + IndexOutOfBoundsException.class, + () -> objectStrategy.fromByteBufferSafe(buf2, garbage2.length).toByteArray() + ); + } + + // non sketch that is too short to contain header should fail with regular java buffer exception + final byte[] garbage = new byte[]{0x01, 0x02}; + final ByteBuffer buf3 = ByteBuffer.wrap(garbage).order(ByteOrder.LITTLE_ENDIAN); + Assert.assertThrows( + IndexOutOfBoundsException.class, + () -> objectStrategy.fromByteBufferSafe(buf3, garbage.length).toByteArray() + ); + } } diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchOperationsTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchOperationsTest.java new file mode 100644 index 000000000000..613b38c6601a --- /dev/null +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchOperationsTest.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.query.aggregation.datasketches.kll; + +import org.apache.datasketches.kll.KllFloatsSketch; +import org.apache.druid.java.util.common.StringUtils; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; + +public class KllFloatsSketchOperationsTest +{ + @Test + public void testDeserializeSafe() + { + KllFloatsSketch sketch = KllFloatsSketch.newHeapInstance(); + sketch.update(1.1f); + sketch.update(1.2f); + final byte[] bytes = sketch.toByteArray(); + final String base64 = StringUtils.encodeBase64String(bytes); + + Assert.assertArrayEquals(bytes, KllFloatsSketchOperations.deserializeSafe(sketch).toByteArray()); + Assert.assertArrayEquals(bytes, KllFloatsSketchOperations.deserializeSafe(bytes).toByteArray()); + Assert.assertArrayEquals(bytes, KllFloatsSketchOperations.deserializeSafe(base64).toByteArray()); + + final byte[] trunacted = Arrays.copyOfRange(bytes, 0, 20); + Assert.assertThrows(IndexOutOfBoundsException.class, () -> KllFloatsSketchOperations.deserializeSafe(trunacted)); + Assert.assertThrows( + IndexOutOfBoundsException.class, + () -> KllFloatsSketchOperations.deserializeSafe(StringUtils.encodeBase64String(trunacted)) + ); + } +} diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchComplexMetricSerdeTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchComplexMetricSerdeTest.java index e198c7704257..7dc82baee925 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchComplexMetricSerdeTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchComplexMetricSerdeTest.java @@ -22,11 +22,16 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.datasketches.quantiles.DoublesSketch; +import org.apache.datasketches.quantiles.DoublesUnion; import org.apache.druid.data.input.MapBasedInputRow; +import org.apache.druid.segment.data.ObjectStrategy; import org.apache.druid.segment.serde.ComplexMetricExtractor; import org.junit.Assert; import org.junit.Test; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + public class DoublesSketchComplexMetricSerdeTest { @Test @@ -92,4 +97,42 @@ public void testExtractorOnLeadingDecimalNumber() Assert.assertEquals(1, sketch.getRetainedItems()); Assert.assertEquals(0.1d, sketch.getMaxValue(), 0.01d); } + + @Test + public void testSafeRead() + { + final DoublesSketchComplexMetricSerde serde = new DoublesSketchComplexMetricSerde(); + DoublesUnion union = DoublesUnion.builder().setMaxK(1024).build(); + union.update(1.1); + final byte[] bytes = union.toByteArray(); + + ByteBuffer buf = ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN); + ObjectStrategy objectStrategy = serde.getObjectStrategy(); + + // valid sketch should not explode when copied, which reads the memory + objectStrategy.fromByteBufferSafe(buf, bytes.length).toByteArray(true); + + // corrupted sketch should fail with a regular java buffer exception + for (int subset = 3; subset < 15; subset++) { + final byte[] garbage2 = new byte[subset]; + for (int i = 0; i < garbage2.length; i++) { + garbage2[i] = buf.get(i); + } + + final ByteBuffer buf2 = ByteBuffer.wrap(garbage2).order(ByteOrder.LITTLE_ENDIAN); + Assert.assertThrows( + "i " + subset, + IndexOutOfBoundsException.class, + () -> objectStrategy.fromByteBufferSafe(buf2, garbage2.length).toByteArray(true) + ); + } + + // non sketch that is too short to contain header should fail with regular java buffer exception + final byte[] garbage = new byte[]{0x01, 0x02}; + final ByteBuffer buf3 = ByteBuffer.wrap(garbage).order(ByteOrder.LITTLE_ENDIAN); + Assert.assertThrows( + IndexOutOfBoundsException.class, + () -> objectStrategy.fromByteBufferSafe(buf3, garbage.length).toByteArray(true) + ); + } } diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchOperationsTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchOperationsTest.java new file mode 100644 index 000000000000..38e5d39a91b8 --- /dev/null +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchOperationsTest.java @@ -0,0 +1,50 @@ +/* + * 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.query.aggregation.datasketches.quantiles; + +import org.apache.datasketches.quantiles.DoublesUnion; +import org.apache.druid.java.util.common.StringUtils; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; + +public class DoublesSketchOperationsTest +{ + @Test + public void testDeserializeSafe() + { + DoublesUnion union = DoublesUnion.builder().setMaxK(1024).build(); + union.update(1.1); + final byte[] bytes = union.getResult().toByteArray(); + final String base64 = StringUtils.encodeBase64String(bytes); + + Assert.assertArrayEquals(bytes, DoublesSketchOperations.deserializeSafe(union.getResult()).toByteArray()); + Assert.assertArrayEquals(bytes, DoublesSketchOperations.deserializeSafe(bytes).toByteArray()); + Assert.assertArrayEquals(bytes, DoublesSketchOperations.deserializeSafe(base64).toByteArray()); + + final byte[] trunacted = Arrays.copyOfRange(bytes, 0, 4); + Assert.assertThrows(IndexOutOfBoundsException.class, () -> DoublesSketchOperations.deserializeSafe(trunacted)); + Assert.assertThrows( + IndexOutOfBoundsException.class, + () -> DoublesSketchOperations.deserializeSafe(StringUtils.encodeBase64(trunacted)) + ); + } +} diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchHolderObjectStrategyTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchHolderObjectStrategyTest.java new file mode 100644 index 000000000000..5619facd5f62 --- /dev/null +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchHolderObjectStrategyTest.java @@ -0,0 +1,79 @@ +/* + * 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.query.aggregation.datasketches.theta; + +import org.apache.datasketches.Family; +import org.apache.datasketches.SketchesArgumentException; +import org.apache.datasketches.theta.SetOperation; +import org.apache.datasketches.theta.Union; +import org.apache.druid.java.util.common.StringUtils; +import org.junit.Assert; +import org.junit.Test; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +public class SketchHolderObjectStrategyTest +{ + @Test + public void testSafeRead() + { + SketchHolderObjectStrategy objectStrategy = new SketchHolderObjectStrategy(); + Union union = (Union) SetOperation.builder().setNominalEntries(1024).build(Family.UNION); + union.update(1234L); + + final byte[] bytes = union.getResult().toByteArray(); + + ByteBuffer buf = ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN); + + // valid sketch should not explode when copied, which reads the memory + objectStrategy.fromByteBufferSafe(buf, bytes.length).getSketch().compact().getCompactBytes(); + + // corrupted sketch should fail with a regular java buffer exception + for (int subset = 3; subset < bytes.length - 1; subset++) { + final byte[] garbage2 = new byte[subset]; + for (int i = 0; i < garbage2.length; i++) { + garbage2[i] = buf.get(i); + } + + final ByteBuffer buf2 = ByteBuffer.wrap(garbage2).order(ByteOrder.LITTLE_ENDIAN); + Assert.assertThrows( + IndexOutOfBoundsException.class, + () -> objectStrategy.fromByteBufferSafe(buf2, garbage2.length).getSketch().compact().getCompactBytes() + ); + } + + // non sketch that is too short to contain header should fail with regular java buffer exception + final byte[] garbage = new byte[]{0x01, 0x02}; + final ByteBuffer buf3 = ByteBuffer.wrap(garbage).order(ByteOrder.LITTLE_ENDIAN); + Assert.assertThrows( + IndexOutOfBoundsException.class, + () -> objectStrategy.fromByteBufferSafe(buf3, garbage.length).getSketch().compact().getCompactBytes() + ); + + // non sketch that is long enough to check (this one doesn't actually need 'safe' read) + final byte[] garbageLonger = StringUtils.toUtf8("notasketch"); + final ByteBuffer buf4 = ByteBuffer.wrap(garbageLonger).order(ByteOrder.LITTLE_ENDIAN); + Assert.assertThrows( + SketchesArgumentException.class, + () -> objectStrategy.fromByteBufferSafe(buf4, garbageLonger.length).getSketch().compact().getCompactBytes() + ); + } +} diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchHolderTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchHolderTest.java new file mode 100644 index 000000000000..ef68fdeb8c58 --- /dev/null +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchHolderTest.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.query.aggregation.datasketches.theta; + +import org.apache.datasketches.Family; +import org.apache.datasketches.theta.SetOperation; +import org.apache.datasketches.theta.Union; +import org.apache.druid.java.util.common.StringUtils; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; + +public class SketchHolderTest +{ + @Test + public void testDeserializeSafe() + { + Union union = (Union) SetOperation.builder().setNominalEntries(1024).build(Family.UNION); + union.update(1234L); + final byte[] bytes = union.getResult().toByteArray(); + final String base64 = StringUtils.encodeBase64String(bytes); + + Assert.assertArrayEquals(bytes, SketchHolder.deserializeSafe(union.getResult()).getSketch().toByteArray()); + Assert.assertArrayEquals(bytes, SketchHolder.deserializeSafe(bytes).getSketch().toByteArray()); + Assert.assertArrayEquals(bytes, SketchHolder.deserializeSafe(base64).getSketch().toByteArray()); + + final byte[] trunacted = Arrays.copyOfRange(bytes, 0, 10); + Assert.assertThrows(IndexOutOfBoundsException.class, () -> SketchHolder.deserializeSafe(trunacted)); + Assert.assertThrows( + IndexOutOfBoundsException.class, + () -> SketchHolder.deserializeSafe(StringUtils.encodeBase64String(trunacted)) + ); + } +} diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchObjectStrategyTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchObjectStrategyTest.java new file mode 100644 index 000000000000..ee59ddf57645 --- /dev/null +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchObjectStrategyTest.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.query.aggregation.datasketches.tuple; + +import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesUpdatableSketch; +import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesUpdatableSketchBuilder; +import org.junit.Assert; +import org.junit.Test; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +public class ArrayOfDoublesSketchObjectStrategyTest +{ + @Test + public void testSafeRead() + { + ArrayOfDoublesSketchObjectStrategy objectStrategy = new ArrayOfDoublesSketchObjectStrategy(); + ArrayOfDoublesUpdatableSketch sketch = new ArrayOfDoublesUpdatableSketchBuilder().setNominalEntries(1024) + .setNumberOfValues(4) + .build(); + sketch.update(1L, new double[]{1.0, 2.0, 3.0, 4.0}); + + final byte[] bytes = sketch.compact().toByteArray(); + + ByteBuffer buf = ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN); + + // valid sketch should not explode when copied, which reads the memory + objectStrategy.fromByteBufferSafe(buf, bytes.length).compact().toByteArray(); + + // corrupted sketch should fail with a regular java buffer exception + for (int subset = 3; subset < bytes.length - 1; subset++) { + final byte[] garbage2 = new byte[subset]; + for (int i = 0; i < garbage2.length; i++) { + garbage2[i] = buf.get(i); + } + + final ByteBuffer buf2 = ByteBuffer.wrap(garbage2).order(ByteOrder.LITTLE_ENDIAN); + Assert.assertThrows( + IndexOutOfBoundsException.class, + () -> objectStrategy.fromByteBufferSafe(buf2, garbage2.length).compact().toByteArray() + ); + } + + // non sketch that is too short to contain header should fail with regular java buffer exception + final byte[] garbage = new byte[]{0x01, 0x02}; + final ByteBuffer buf3 = ByteBuffer.wrap(garbage).order(ByteOrder.LITTLE_ENDIAN); + Assert.assertThrows( + IndexOutOfBoundsException.class, + () -> objectStrategy.fromByteBufferSafe(buf3, garbage.length).compact().toByteArray() + ); + } +} diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchOperationsTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchOperationsTest.java new file mode 100644 index 000000000000..415f3acab970 --- /dev/null +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchOperationsTest.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.query.aggregation.datasketches.tuple; + +import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesUpdatableSketch; +import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesUpdatableSketchBuilder; +import org.apache.druid.java.util.common.StringUtils; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; + +public class ArrayOfDoublesSketchOperationsTest +{ + @Test + public void testDeserializeSafe() + { + ArrayOfDoublesSketchObjectStrategy objectStrategy = new ArrayOfDoublesSketchObjectStrategy(); + ArrayOfDoublesUpdatableSketch sketch = new ArrayOfDoublesUpdatableSketchBuilder().setNominalEntries(1024) + .setNumberOfValues(4) + .build(); + sketch.update(1L, new double[]{1.0, 2.0, 3.0, 4.0}); + + final byte[] bytes = sketch.toByteArray(); + final String base64 = StringUtils.encodeBase64String(bytes); + + Assert.assertArrayEquals(bytes, ArrayOfDoublesSketchOperations.deserializeSafe(sketch).toByteArray()); + Assert.assertArrayEquals(bytes, ArrayOfDoublesSketchOperations.deserializeSafe(bytes).toByteArray()); + Assert.assertArrayEquals(bytes, ArrayOfDoublesSketchOperations.deserializeSafe(base64).toByteArray()); + + final byte[] trunacted = Arrays.copyOfRange(bytes, 0, 10); + Assert.assertThrows(IndexOutOfBoundsException.class, () -> ArrayOfDoublesSketchOperations.deserializeSafe(trunacted)); + Assert.assertThrows( + IndexOutOfBoundsException.class, + () -> ArrayOfDoublesSketchOperations.deserializeSafe(StringUtils.encodeBase64String(trunacted)) + ); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/column/ObjectStrategyComplexTypeStrategy.java b/processing/src/main/java/org/apache/druid/segment/column/ObjectStrategyComplexTypeStrategy.java index 351f2665d05e..d05ba208585c 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/ObjectStrategyComplexTypeStrategy.java +++ b/processing/src/main/java/org/apache/druid/segment/column/ObjectStrategyComplexTypeStrategy.java @@ -90,6 +90,6 @@ public int compare(T o1, T o2) @Override public T fromBytes(byte[] value) { - return objectStrategy.fromByteBuffer(ByteBuffer.wrap(value), value.length); + return objectStrategy.fromByteBufferSafe(ByteBuffer.wrap(value), value.length); } } diff --git a/processing/src/main/java/org/apache/druid/segment/data/ObjectStrategy.java b/processing/src/main/java/org/apache/druid/segment/data/ObjectStrategy.java index 8a53fc57a7d7..eba97d04bbbe 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/ObjectStrategy.java +++ b/processing/src/main/java/org/apache/druid/segment/data/ObjectStrategy.java @@ -79,4 +79,31 @@ default void writeTo(T val, WriteOutBytes out) throws IOException out.write(bytes); } } + + /** + * Convert values from their underlying byte representation, when the underlying bytes might be corrupted or + * maliciously constructed + * + * Implementations of this method absolutely must never perform any sun.misc.Unsafe based memory read or write + * operations from instructions contained in the data read from this buffer without first validating the data. If the + * data cannot be validated, all read and write operations from instructions in this data must be done directly with + * the {@link ByteBuffer} methods, or using {@link SafeWritableMemory} if + * {@link org.apache.datasketches.memory.Memory} is employed to materialize the value. + * + * Implementations of this method may change the given buffer's mark, or limit, and position. + * + * Implementations of this method may not store the given buffer in a field of the "deserialized" object, + * need to use {@link ByteBuffer#slice()}, {@link ByteBuffer#asReadOnlyBuffer()} or {@link ByteBuffer#duplicate()} in + * this case. + * + * + * @param buffer buffer to read value from + * @param numBytes number of bytes used to store the value, starting at buffer.position() + * @return an object created from the given byte buffer representation + */ + @Nullable + default T fromByteBufferSafe(ByteBuffer buffer, int numBytes) + { + return fromByteBuffer(buffer, numBytes); + } } diff --git a/processing/src/main/java/org/apache/druid/segment/data/SafeWritableBase.java b/processing/src/main/java/org/apache/druid/segment/data/SafeWritableBase.java new file mode 100644 index 000000000000..df2fc14d0535 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/SafeWritableBase.java @@ -0,0 +1,450 @@ +/* + * 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.segment.data; + +import com.google.common.base.Preconditions; +import com.google.common.primitives.Ints; +import org.apache.datasketches.memory.BaseState; +import org.apache.datasketches.memory.MemoryRequestServer; +import org.apache.datasketches.memory.WritableMemory; +import org.apache.datasketches.memory.internal.BaseStateImpl; +import org.apache.datasketches.memory.internal.UnsafeUtil; +import org.apache.datasketches.memory.internal.XxHash64; +import org.apache.druid.java.util.common.StringUtils; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +/** + * Base class for making a regular {@link ByteBuffer} look like a {@link org.apache.datasketches.memory.Memory} or + * {@link org.apache.datasketches.memory.Buffer}. All methods delegate directly to the {@link ByteBuffer} rather + * than using 'unsafe' reads. + * + * @see SafeWritableMemory + * @see SafeWritableBuffer + */ + +@SuppressWarnings("unused") +public abstract class SafeWritableBase implements BaseState +{ + static final MemoryRequestServer SAFE_HEAP_REQUEST_SERVER = new HeapByteBufferMemoryRequestServer(); + + final ByteBuffer buffer; + + public SafeWritableBase(ByteBuffer buffer) + { + this.buffer = buffer; + } + + public MemoryRequestServer getMemoryRequestServer() + { + return SAFE_HEAP_REQUEST_SERVER; + } + + public boolean getBoolean(long offsetBytes) + { + return getByte(Ints.checkedCast(offsetBytes)) != 0; + } + + public byte getByte(long offsetBytes) + { + return buffer.get(Ints.checkedCast(offsetBytes)); + } + + public char getChar(long offsetBytes) + { + return buffer.getChar(Ints.checkedCast(offsetBytes)); + } + + public double getDouble(long offsetBytes) + { + return buffer.getDouble(Ints.checkedCast(offsetBytes)); + } + + public float getFloat(long offsetBytes) + { + return buffer.getFloat(Ints.checkedCast(offsetBytes)); + } + + public int getInt(long offsetBytes) + { + return buffer.getInt(Ints.checkedCast(offsetBytes)); + } + + public long getLong(long offsetBytes) + { + return buffer.getLong(Ints.checkedCast(offsetBytes)); + } + + public short getShort(long offsetBytes) + { + return buffer.getShort(Ints.checkedCast(offsetBytes)); + } + + public void putBoolean(long offsetBytes, boolean value) + { + buffer.put(Ints.checkedCast(offsetBytes), (byte) (value ? 1 : 0)); + } + + public void putByte(long offsetBytes, byte value) + { + buffer.put(Ints.checkedCast(offsetBytes), value); + } + + public void putChar(long offsetBytes, char value) + { + buffer.putChar(Ints.checkedCast(offsetBytes), value); + } + + public void putDouble(long offsetBytes, double value) + { + buffer.putDouble(Ints.checkedCast(offsetBytes), value); + } + + public void putFloat(long offsetBytes, float value) + { + buffer.putFloat(Ints.checkedCast(offsetBytes), value); + } + + public void putInt(long offsetBytes, int value) + { + buffer.putInt(Ints.checkedCast(offsetBytes), value); + } + + public void putLong(long offsetBytes, long value) + { + buffer.putLong(Ints.checkedCast(offsetBytes), value); + } + + public void putShort(long offsetBytes, short value) + { + buffer.putShort(Ints.checkedCast(offsetBytes), value); + } + + @Override + public ByteOrder getTypeByteOrder() + { + return buffer.order(); + } + + @Override + public boolean isByteOrderCompatible(ByteOrder byteOrder) + { + return buffer.order().equals(byteOrder); + } + + @Override + public ByteBuffer getByteBuffer() + { + return buffer; + } + + @Override + public long getCapacity() + { + return buffer.capacity(); + } + + @Override + public long getCumulativeOffset() + { + return 0; + } + + @Override + public long getCumulativeOffset(long offsetBytes) + { + return offsetBytes; + } + + @Override + public long getRegionOffset() + { + return 0; + } + + @Override + public long getRegionOffset(long offsetBytes) + { + return offsetBytes; + } + + @Override + public boolean hasArray() + { + return false; + } + + @Override + public long xxHash64(long offsetBytes, long lengthBytes, long seed) + { + return hash(buffer, offsetBytes, lengthBytes, seed); + } + + @Override + public long xxHash64(long in, long seed) + { + return XxHash64.hash(in, seed); + } + + @Override + public boolean hasByteBuffer() + { + return true; + } + + @Override + public boolean isDirect() + { + return false; + } + + @Override + public boolean isReadOnly() + { + return false; + } + + @Override + public boolean isSameResource(Object that) + { + return this.equals(that); + } + + @Override + public boolean isValid() + { + return true; + } + + @Override + public void checkValidAndBounds(long offsetBytes, long lengthBytes) + { + Preconditions.checkArgument( + Ints.checkedCast(offsetBytes) < buffer.limit(), + "start offset %s is greater than buffer limit %s", + offsetBytes, + buffer.limit() + ); + Preconditions.checkArgument( + Ints.checkedCast(offsetBytes + lengthBytes) < buffer.limit(), + "end offset %s is greater than buffer limit %s", + offsetBytes + lengthBytes, + buffer.limit() + ); + } + + /** + * Adapted from {@link BaseStateImpl#toHexString(String, long, int)} + */ + @Override + public String toHexString(String header, long offsetBytes, int lengthBytes) + { + final String klass = this.getClass().getSimpleName(); + final String s1 = StringUtils.format("(..., %d, %d)", offsetBytes, lengthBytes); + final long hcode = hashCode() & 0XFFFFFFFFL; + final String call = ".toHexString" + s1 + ", hashCode: " + hcode; + String sb = "### " + klass + " SUMMARY ###" + UnsafeUtil.LS + + "Header Comment : " + header + UnsafeUtil.LS + + "Call Parameters : " + call; + return toHex(this, sb, offsetBytes, lengthBytes); + } + + /** + * Adapted from {@link BaseStateImpl#toHex(BaseStateImpl, String, long, int)} + */ + static String toHex( + final SafeWritableBase state, + final String preamble, + final long offsetBytes, + final int lengthBytes + ) + { + final String lineSeparator = UnsafeUtil.LS; + final long capacity = state.getCapacity(); + UnsafeUtil.checkBounds(offsetBytes, lengthBytes, capacity); + final StringBuilder sb = new StringBuilder(); + final String uObjStr; + final long uObjHeader; + uObjStr = "null"; + uObjHeader = 0; + final ByteBuffer bb = state.getByteBuffer(); + final String bbStr = bb == null ? "null" + : bb.getClass().getSimpleName() + ", " + (bb.hashCode() & 0XFFFFFFFFL); + final MemoryRequestServer memReqSvr = state.getMemoryRequestServer(); + final String memReqStr = memReqSvr != null + ? memReqSvr.getClass().getSimpleName() + ", " + (memReqSvr.hashCode() & 0XFFFFFFFFL) + : "null"; + final long cumBaseOffset = state.getCumulativeOffset(); + sb.append(preamble).append(lineSeparator); + sb.append("UnsafeObj, hashCode : ").append(uObjStr).append(lineSeparator); + sb.append("UnsafeObjHeader : ").append(uObjHeader).append(lineSeparator); + sb.append("ByteBuf, hashCode : ").append(bbStr).append(lineSeparator); + sb.append("RegionOffset : ").append(state.getRegionOffset()).append(lineSeparator); + sb.append("Capacity : ").append(capacity).append(lineSeparator); + sb.append("CumBaseOffset : ").append(cumBaseOffset).append(lineSeparator); + sb.append("MemReq, hashCode : ").append(memReqStr).append(lineSeparator); + sb.append("Valid : ").append(state.isValid()).append(lineSeparator); + sb.append("Read Only : ").append(state.isReadOnly()).append(lineSeparator); + sb.append("Type Byte Order : ").append(state.getTypeByteOrder()).append(lineSeparator); + sb.append("Native Byte Order : ").append(ByteOrder.nativeOrder()).append(lineSeparator); + sb.append("JDK Runtime Version : ").append(UnsafeUtil.JDK).append(lineSeparator); + //Data detail + sb.append("Data, littleEndian : 0 1 2 3 4 5 6 7"); + + for (long i = 0; i < lengthBytes; i++) { + final int b = state.getByte(cumBaseOffset + offsetBytes + i) & 0XFF; + if (i % 8 == 0) { //row header + sb.append(StringUtils.format("%n%20s: ", offsetBytes + i)); + } + sb.append(StringUtils.format("%02x ", b)); + } + sb.append(lineSeparator); + + return sb.toString(); + } + + // copied from datasketches-memory XxHash64.java + private static final long P1 = -7046029288634856825L; + private static final long P2 = -4417276706812531889L; + private static final long P3 = 1609587929392839161L; + private static final long P4 = -8796714831421723037L; + private static final long P5 = 2870177450012600261L; + + /** + * Adapted from {@link XxHash64#hash(Object, long, long, long)} to work with {@link ByteBuffer} + */ + static long hash(ByteBuffer memory, long cumOffsetBytes, final long lengthBytes, final long seed) + { + long hash; + long remaining = lengthBytes; + int offset = Ints.checkedCast(cumOffsetBytes); + + if (remaining >= 32) { + long v1 = seed + P1 + P2; + long v2 = seed + P2; + long v3 = seed; + long v4 = seed - P1; + + do { + v1 += memory.getLong(offset) * P2; + v1 = Long.rotateLeft(v1, 31); + v1 *= P1; + + v2 += memory.getLong(offset + 8) * P2; + v2 = Long.rotateLeft(v2, 31); + v2 *= P1; + + v3 += memory.getLong(offset + 16) * P2; + v3 = Long.rotateLeft(v3, 31); + v3 *= P1; + + v4 += memory.getLong(offset + 24) * P2; + v4 = Long.rotateLeft(v4, 31); + v4 *= P1; + + offset += 32; + remaining -= 32; + } while (remaining >= 32); + + hash = Long.rotateLeft(v1, 1) + + Long.rotateLeft(v2, 7) + + Long.rotateLeft(v3, 12) + + Long.rotateLeft(v4, 18); + + v1 *= P2; + v1 = Long.rotateLeft(v1, 31); + v1 *= P1; + hash ^= v1; + hash = (hash * P1) + P4; + + v2 *= P2; + v2 = Long.rotateLeft(v2, 31); + v2 *= P1; + hash ^= v2; + hash = (hash * P1) + P4; + + v3 *= P2; + v3 = Long.rotateLeft(v3, 31); + v3 *= P1; + hash ^= v3; + hash = (hash * P1) + P4; + + v4 *= P2; + v4 = Long.rotateLeft(v4, 31); + v4 *= P1; + hash ^= v4; + hash = (hash * P1) + P4; + } else { //end remaining >= 32 + hash = seed + P5; + } + + hash += lengthBytes; + + while (remaining >= 8) { + long k1 = memory.getLong(offset); + k1 *= P2; + k1 = Long.rotateLeft(k1, 31); + k1 *= P1; + hash ^= k1; + hash = (Long.rotateLeft(hash, 27) * P1) + P4; + offset += 8; + remaining -= 8; + } + + if (remaining >= 4) { //treat as unsigned ints + hash ^= (memory.getInt(offset) & 0XFFFF_FFFFL) * P1; + hash = (Long.rotateLeft(hash, 23) * P2) + P3; + offset += 4; + remaining -= 4; + } + + while (remaining != 0) { //treat as unsigned bytes + hash ^= (memory.get(offset) & 0XFFL) * P5; + hash = Long.rotateLeft(hash, 11) * P1; + --remaining; + ++offset; + } + + hash ^= hash >>> 33; + hash *= P2; + hash ^= hash >>> 29; + hash *= P3; + hash ^= hash >>> 32; + return hash; + } + + private static class HeapByteBufferMemoryRequestServer implements MemoryRequestServer + { + @Override + public WritableMemory request(WritableMemory currentWritableMemory, long capacityBytes) + { + ByteBuffer newBuffer = ByteBuffer.allocate(Ints.checkedCast(capacityBytes)); + newBuffer.order(currentWritableMemory.getTypeByteOrder()); + return new SafeWritableMemory(newBuffer); + } + + @Override + public void requestClose(WritableMemory memToClose, WritableMemory newMemory) + { + // do nothing + } + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/SafeWritableBuffer.java b/processing/src/main/java/org/apache/druid/segment/data/SafeWritableBuffer.java new file mode 100644 index 000000000000..3da7e70b457c --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/SafeWritableBuffer.java @@ -0,0 +1,501 @@ +/* + * 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.segment.data; + +import com.google.common.primitives.Ints; +import org.apache.datasketches.memory.BaseBuffer; +import org.apache.datasketches.memory.Buffer; +import org.apache.datasketches.memory.Memory; +import org.apache.datasketches.memory.WritableBuffer; +import org.apache.datasketches.memory.WritableMemory; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +/** + * Safety first! Don't trust something whose contents you locations to read and write stuff to, but need a + * {@link Buffer} or {@link WritableBuffer}? use this! + *

+ * Delegates everything to an underlying {@link ByteBuffer} so all read and write operations will have bounds checks + * built in rather than using 'unsafe'. + */ +public class SafeWritableBuffer extends SafeWritableBase implements WritableBuffer +{ + private int start; + private int end; + + public SafeWritableBuffer(ByteBuffer buffer) + { + super(buffer); + this.start = 0; + this.buffer.position(0); + this.end = buffer.capacity(); + } + + @Override + public WritableBuffer writableDuplicate() + { + return writableDuplicate(buffer.order()); + } + + @Override + public WritableBuffer writableDuplicate(ByteOrder byteOrder) + { + ByteBuffer dupe = buffer.duplicate(); + dupe.order(byteOrder); + WritableBuffer duplicate = new SafeWritableBuffer(dupe); + duplicate.setStartPositionEnd(start, buffer.position(), end); + return duplicate; + } + + @Override + public WritableBuffer writableRegion() + { + ByteBuffer dupe = buffer.duplicate().order(buffer.order()); + dupe.position(start); + dupe.limit(end); + ByteBuffer remaining = buffer.slice(); + remaining.order(dupe.order()); + return new SafeWritableBuffer(remaining); + } + + @Override + public WritableBuffer writableRegion(long offsetBytes, long capacityBytes, ByteOrder byteOrder) + { + ByteBuffer dupe = buffer.duplicate(); + dupe.position(Ints.checkedCast(offsetBytes)); + dupe.limit(dupe.position() + Ints.checkedCast(capacityBytes)); + return new SafeWritableBuffer(dupe.slice().order(byteOrder)); + } + + @Override + public WritableMemory asWritableMemory(ByteOrder byteOrder) + { + ByteBuffer dupe = buffer.duplicate(); + dupe.order(byteOrder); + return new SafeWritableMemory(dupe); + } + + @Override + public void putBoolean(boolean value) + { + buffer.put((byte) (value ? 1 : 0)); + } + + @Override + public void putBooleanArray(boolean[] srcArray, int srcOffsetBooleans, int lengthBooleans) + { + for (int i = 0; i < lengthBooleans; i++) { + putBoolean(srcArray[srcOffsetBooleans + i]); + } + } + + @Override + public void putByte(byte value) + { + buffer.put(value); + } + + @Override + public void putByteArray(byte[] srcArray, int srcOffsetBytes, int lengthBytes) + { + buffer.put(srcArray, srcOffsetBytes, lengthBytes); + } + + @Override + public void putChar(char value) + { + buffer.putChar(value); + } + + @Override + public void putCharArray(char[] srcArray, int srcOffsetChars, int lengthChars) + { + for (int i = 0; i < lengthChars; i++) { + buffer.putChar(srcArray[srcOffsetChars + i]); + } + } + + @Override + public void putDouble(double value) + { + buffer.putDouble(value); + } + + @Override + public void putDoubleArray(double[] srcArray, int srcOffsetDoubles, int lengthDoubles) + { + for (int i = 0; i < lengthDoubles; i++) { + buffer.putDouble(srcArray[srcOffsetDoubles + i]); + } + } + + @Override + public void putFloat(float value) + { + buffer.putFloat(value); + } + + @Override + public void putFloatArray(float[] srcArray, int srcOffsetFloats, int lengthFloats) + { + for (int i = 0; i < lengthFloats; i++) { + buffer.putFloat(srcArray[srcOffsetFloats + i]); + } + } + + @Override + public void putInt(int value) + { + buffer.putInt(value); + } + + @Override + public void putIntArray(int[] srcArray, int srcOffsetInts, int lengthInts) + { + for (int i = 0; i < lengthInts; i++) { + buffer.putInt(srcArray[srcOffsetInts + i]); + } + } + + @Override + public void putLong(long value) + { + buffer.putLong(value); + } + + @Override + public void putLongArray(long[] srcArray, int srcOffsetLongs, int lengthLongs) + { + for (int i = 0; i < lengthLongs; i++) { + buffer.putLong(srcArray[srcOffsetLongs + i]); + } + } + + @Override + public void putShort(short value) + { + buffer.putShort(value); + } + + @Override + public void putShortArray(short[] srcArray, int srcOffsetShorts, int lengthShorts) + { + for (int i = 0; i < lengthShorts; i++) { + buffer.putShort(srcArray[srcOffsetShorts + i]); + } + } + + @Override + public Object getArray() + { + return null; + } + + @Override + public void clear() + { + fill((byte) 0); + } + + @Override + public void fill(byte value) + { + while (buffer.hasRemaining() && buffer.position() < end) { + buffer.put(value); + } + } + + @Override + public Buffer duplicate() + { + return writableDuplicate(); + } + + @Override + public Buffer duplicate(ByteOrder byteOrder) + { + return writableDuplicate(byteOrder); + } + + @Override + public Buffer region() + { + return writableRegion(); + } + + @Override + public Buffer region(long offsetBytes, long capacityBytes, ByteOrder byteOrder) + { + return writableRegion(offsetBytes, capacityBytes, byteOrder); + } + + @Override + public Memory asMemory(ByteOrder byteOrder) + { + return asWritableMemory(byteOrder); + } + + @Override + public boolean getBoolean() + { + return buffer.get() == 0 ? false : true; + } + + @Override + public void getBooleanArray(boolean[] dstArray, int dstOffsetBooleans, int lengthBooleans) + { + for (int i = 0; i < lengthBooleans; i++) { + dstArray[dstOffsetBooleans + i] = getBoolean(); + } + } + + @Override + public byte getByte() + { + return buffer.get(); + } + + @Override + public void getByteArray(byte[] dstArray, int dstOffsetBytes, int lengthBytes) + { + for (int i = 0; i < lengthBytes; i++) { + dstArray[dstOffsetBytes + i] = buffer.get(); + } + } + + @Override + public char getChar() + { + return buffer.getChar(); + } + + @Override + public void getCharArray(char[] dstArray, int dstOffsetChars, int lengthChars) + { + for (int i = 0; i < lengthChars; i++) { + dstArray[dstOffsetChars + i] = buffer.getChar(); + } + } + + @Override + public double getDouble() + { + return buffer.getDouble(); + } + + @Override + public void getDoubleArray(double[] dstArray, int dstOffsetDoubles, int lengthDoubles) + { + for (int i = 0; i < lengthDoubles; i++) { + dstArray[dstOffsetDoubles + i] = buffer.getDouble(); + } + } + + @Override + public float getFloat() + { + return buffer.getFloat(); + } + + @Override + public void getFloatArray(float[] dstArray, int dstOffsetFloats, int lengthFloats) + { + for (int i = 0; i < lengthFloats; i++) { + dstArray[dstOffsetFloats + i] = buffer.getFloat(); + } + } + + @Override + public int getInt() + { + return buffer.getInt(); + } + + @Override + public void getIntArray(int[] dstArray, int dstOffsetInts, int lengthInts) + { + for (int i = 0; i < lengthInts; i++) { + dstArray[dstOffsetInts + i] = buffer.getInt(); + } + } + + @Override + public long getLong() + { + return buffer.getLong(); + } + + @Override + public void getLongArray(long[] dstArray, int dstOffsetLongs, int lengthLongs) + { + for (int i = 0; i < lengthLongs; i++) { + dstArray[dstOffsetLongs + i] = buffer.getLong(); + } + } + + @Override + public short getShort() + { + return buffer.getShort(); + } + + @Override + public void getShortArray(short[] dstArray, int dstOffsetShorts, int lengthShorts) + { + for (int i = 0; i < lengthShorts; i++) { + dstArray[dstOffsetShorts + i] = buffer.getShort(); + } + } + + @Override + public int compareTo( + long thisOffsetBytes, + long thisLengthBytes, + Buffer that, + long thatOffsetBytes, + long thatLengthBytes + ) + { + final int thisLength = Ints.checkedCast(thisLengthBytes); + final int thatLength = Ints.checkedCast(thatLengthBytes); + + final int commonLength = Math.min(thisLength, thatLength); + + for (int i = 0; i < commonLength; i++) { + final int cmp = Byte.compare(getByte(thisOffsetBytes + i), that.getByte(thatOffsetBytes + i)); + if (cmp != 0) { + return cmp; + } + } + + return Integer.compare(thisLength, thatLength); + } + + @Override + public BaseBuffer incrementPosition(long increment) + { + buffer.position(buffer.position() + Ints.checkedCast(increment)); + return this; + } + + @Override + public BaseBuffer incrementAndCheckPosition(long increment) + { + checkInvariants(start, buffer.position() + increment, end, buffer.capacity()); + return incrementPosition(increment); + } + + @Override + public long getEnd() + { + return end; + } + + @Override + public long getPosition() + { + return buffer.position(); + } + + @Override + public long getStart() + { + return start; + } + + @Override + public long getRemaining() + { + return buffer.remaining(); + } + + @Override + public boolean hasRemaining() + { + return buffer.hasRemaining(); + } + + @Override + public BaseBuffer resetPosition() + { + buffer.position(start); + return this; + } + + @Override + public BaseBuffer setPosition(long position) + { + buffer.position(Ints.checkedCast(position)); + return this; + } + + @Override + public BaseBuffer setAndCheckPosition(long position) + { + checkInvariants(start, position, end, buffer.capacity()); + return setPosition(position); + } + + @Override + public BaseBuffer setStartPositionEnd(long start, long position, long end) + { + this.start = Ints.checkedCast(start); + this.end = Ints.checkedCast(end); + buffer.position(Ints.checkedCast(position)); + buffer.limit(this.end); + return this; + } + + @Override + public BaseBuffer setAndCheckStartPositionEnd(long start, long position, long end) + { + checkInvariants(start, position, end, buffer.capacity()); + return setStartPositionEnd(start, position, end); + } + + @Override + public boolean equalTo(long thisOffsetBytes, Object that, long thatOffsetBytes, long lengthBytes) + { + if (!(that instanceof SafeWritableBuffer)) { + return false; + } + return compareTo(thisOffsetBytes, lengthBytes, (SafeWritableBuffer) that, thatOffsetBytes, lengthBytes) == 0; + } + + /** + * Adapted from {@link org.apache.datasketches.memory.internal.BaseBufferImpl#checkInvariants(long, long, long, long)} + */ + static void checkInvariants(final long start, final long pos, final long end, final long cap) + { + if ((start | pos | end | cap | (pos - start) | (end - pos) | (cap - end)) < 0L) { + throw new IllegalArgumentException( + "Violation of Invariants: " + + "start: " + start + + " <= pos: " + pos + + " <= end: " + end + + " <= cap: " + cap + + "; (pos - start): " + (pos - start) + + ", (end - pos): " + (end - pos) + + ", (cap - end): " + (cap - end) + ); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/SafeWritableMemory.java b/processing/src/main/java/org/apache/druid/segment/data/SafeWritableMemory.java new file mode 100644 index 000000000000..9006ac5cec9e --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/SafeWritableMemory.java @@ -0,0 +1,417 @@ +/* + * 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.segment.data; + +import com.google.common.primitives.Ints; +import org.apache.datasketches.memory.Buffer; +import org.apache.datasketches.memory.Memory; +import org.apache.datasketches.memory.Utf8CodingException; +import org.apache.datasketches.memory.WritableBuffer; +import org.apache.datasketches.memory.WritableMemory; +import org.apache.druid.java.util.common.StringUtils; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.channels.WritableByteChannel; + +/** + * Safety first! Don't trust something whose contents you locations to read and write stuff to, but need a + * {@link Memory} or {@link WritableMemory}? use this! + *

+ * Delegates everything to an underlying {@link ByteBuffer} so all read and write operations will have bounds checks + * built in rather than using 'unsafe'. + */ +public class SafeWritableMemory extends SafeWritableBase implements WritableMemory +{ + public static SafeWritableMemory wrap(byte[] bytes) + { + return wrap(ByteBuffer.wrap(bytes).order(ByteOrder.nativeOrder()), 0, bytes.length); + } + + public static SafeWritableMemory wrap(ByteBuffer buffer) + { + return wrap(buffer.duplicate().order(buffer.order()), 0, buffer.capacity()); + } + + public static SafeWritableMemory wrap(ByteBuffer buffer, ByteOrder byteOrder) + { + return wrap(buffer.duplicate().order(byteOrder), 0, buffer.capacity()); + } + + public static SafeWritableMemory wrap(ByteBuffer buffer, int offset, int size) + { + final ByteBuffer dupe = buffer.duplicate().order(buffer.order()); + dupe.position(offset); + dupe.limit(offset + size); + return new SafeWritableMemory(dupe.slice().order(buffer.order())); + } + + public SafeWritableMemory(ByteBuffer buffer) + { + super(buffer); + } + + @Override + public Memory region(long offsetBytes, long capacityBytes, ByteOrder byteOrder) + { + return writableRegion(offsetBytes, capacityBytes, byteOrder); + } + + @Override + public Buffer asBuffer(ByteOrder byteOrder) + { + return asWritableBuffer(byteOrder); + } + + @Override + public void getBooleanArray(long offsetBytes, boolean[] dstArray, int dstOffsetBooleans, int lengthBooleans) + { + final int offset = Ints.checkedCast(offsetBytes); + for (int j = 0; j < lengthBooleans; j++) { + dstArray[dstOffsetBooleans + j] = buffer.get(offset + j) != 0; + } + } + + @Override + public void getByteArray(long offsetBytes, byte[] dstArray, int dstOffsetBytes, int lengthBytes) + { + final int offset = Ints.checkedCast(offsetBytes); + for (int j = 0; j < lengthBytes; j++) { + dstArray[dstOffsetBytes + j] = buffer.get(offset + j); + } + } + + @Override + public void getCharArray(long offsetBytes, char[] dstArray, int dstOffsetChars, int lengthChars) + { + final int offset = Ints.checkedCast(offsetBytes); + for (int j = 0; j < lengthChars; j++) { + dstArray[dstOffsetChars + j] = buffer.getChar(offset + (j * Character.BYTES)); + } + } + + @Override + public int getCharsFromUtf8(long offsetBytes, int utf8LengthBytes, Appendable dst) + throws IOException, Utf8CodingException + { + ByteBuffer dupe = buffer.asReadOnlyBuffer().order(buffer.order()); + dupe.position(Ints.checkedCast(offsetBytes)); + String s = StringUtils.fromUtf8(dupe, utf8LengthBytes); + dst.append(s); + return s.length(); + } + + @Override + public int getCharsFromUtf8(long offsetBytes, int utf8LengthBytes, StringBuilder dst) throws Utf8CodingException + { + ByteBuffer dupe = buffer.asReadOnlyBuffer().order(buffer.order()); + dupe.position(Ints.checkedCast(offsetBytes)); + String s = StringUtils.fromUtf8(dupe, utf8LengthBytes); + dst.append(s); + return s.length(); + } + + @Override + public void getDoubleArray(long offsetBytes, double[] dstArray, int dstOffsetDoubles, int lengthDoubles) + { + final int offset = Ints.checkedCast(offsetBytes); + for (int j = 0; j < lengthDoubles; j++) { + dstArray[dstOffsetDoubles + j] = buffer.getDouble(offset + (j * Double.BYTES)); + } + } + + @Override + public void getFloatArray(long offsetBytes, float[] dstArray, int dstOffsetFloats, int lengthFloats) + { + final int offset = Ints.checkedCast(offsetBytes); + for (int j = 0; j < lengthFloats; j++) { + dstArray[dstOffsetFloats + j] = buffer.getFloat(offset + (j * Float.BYTES)); + } + } + + @Override + public void getIntArray(long offsetBytes, int[] dstArray, int dstOffsetInts, int lengthInts) + { + final int offset = Ints.checkedCast(offsetBytes); + for (int j = 0; j < lengthInts; j++) { + dstArray[dstOffsetInts + j] = buffer.getInt(offset + (j * Integer.BYTES)); + } + } + + @Override + public void getLongArray(long offsetBytes, long[] dstArray, int dstOffsetLongs, int lengthLongs) + { + final int offset = Ints.checkedCast(offsetBytes); + for (int j = 0; j < lengthLongs; j++) { + dstArray[dstOffsetLongs + j] = buffer.getLong(offset + (j * Long.BYTES)); + } + } + + @Override + public void getShortArray(long offsetBytes, short[] dstArray, int dstOffsetShorts, int lengthShorts) + { + final int offset = Ints.checkedCast(offsetBytes); + for (int j = 0; j < lengthShorts; j++) { + dstArray[dstOffsetShorts + j] = buffer.getShort(offset + (j * Short.BYTES)); + } + } + + @Override + public int compareTo( + long thisOffsetBytes, + long thisLengthBytes, + Memory that, + long thatOffsetBytes, + long thatLengthBytes + ) + { + final int thisLength = Ints.checkedCast(thisLengthBytes); + final int thatLength = Ints.checkedCast(thatLengthBytes); + + final int commonLength = Math.min(thisLength, thatLength); + + for (int i = 0; i < commonLength; i++) { + final int cmp = Byte.compare(getByte(thisOffsetBytes + i), that.getByte(thatOffsetBytes + i)); + if (cmp != 0) { + return cmp; + } + } + + return Integer.compare(thisLength, thatLength); + } + + @Override + public void copyTo(long srcOffsetBytes, WritableMemory destination, long dstOffsetBytes, long lengthBytes) + { + int offset = Ints.checkedCast(srcOffsetBytes); + for (int i = 0; i < lengthBytes; i++) { + destination.putByte(dstOffsetBytes + i, buffer.get(offset + i)); + } + } + + @Override + public void writeTo(long offsetBytes, long lengthBytes, WritableByteChannel out) throws IOException + { + ByteBuffer dupe = buffer.duplicate(); + dupe.position(Ints.checkedCast(offsetBytes)); + dupe.limit(dupe.position() + Ints.checkedCast(lengthBytes)); + ByteBuffer view = dupe.slice(); + view.order(buffer.order()); + out.write(view); + } + + @Override + public boolean equalTo(long thisOffsetBytes, Object that, long thatOffsetBytes, long lengthBytes) + { + if (!(that instanceof SafeWritableMemory)) { + return false; + } + return compareTo(thisOffsetBytes, lengthBytes, (SafeWritableMemory) that, thatOffsetBytes, lengthBytes) == 0; + } + + + @Override + public WritableMemory writableRegion(long offsetBytes, long capacityBytes, ByteOrder byteOrder) + { + final ByteBuffer dupe = buffer.duplicate().order(buffer.order()); + final int sizeBytes = Ints.checkedCast(capacityBytes); + dupe.position(Ints.checkedCast(offsetBytes)); + dupe.limit(dupe.position() + sizeBytes); + final ByteBuffer view = dupe.slice(); + view.order(byteOrder); + return new SafeWritableMemory(view); + } + + @Override + public WritableBuffer asWritableBuffer(ByteOrder byteOrder) + { + return new SafeWritableBuffer(buffer.duplicate().order(byteOrder)); + } + + @Override + public void putBooleanArray(long offsetBytes, boolean[] srcArray, int srcOffsetBooleans, int lengthBooleans) + { + final int offset = Ints.checkedCast(offsetBytes); + for (int i = 0; i < lengthBooleans; i++) { + buffer.put(offset + i, (byte) (srcArray[i + srcOffsetBooleans] ? 1 : 0)); + } + } + + @Override + public void putByteArray(long offsetBytes, byte[] srcArray, int srcOffsetBytes, int lengthBytes) + { + final int offset = Ints.checkedCast(offsetBytes); + for (int i = 0; i < lengthBytes; i++) { + buffer.put(offset + i, srcArray[srcOffsetBytes + i]); + } + } + + @Override + public void putCharArray(long offsetBytes, char[] srcArray, int srcOffsetChars, int lengthChars) + { + final int offset = Ints.checkedCast(offsetBytes); + for (int i = 0; i < lengthChars; i++) { + buffer.putChar(offset + (i * Character.BYTES), srcArray[srcOffsetChars + i]); + } + } + + @Override + public long putCharsToUtf8(long offsetBytes, CharSequence src) + { + final byte[] bytes = StringUtils.toUtf8(src.toString()); + putByteArray(offsetBytes, bytes, 0, bytes.length); + return bytes.length; + } + + @Override + public void putDoubleArray(long offsetBytes, double[] srcArray, int srcOffsetDoubles, int lengthDoubles) + { + final int offset = Ints.checkedCast(offsetBytes); + for (int i = 0; i < lengthDoubles; i++) { + buffer.putDouble(offset + (i * Double.BYTES), srcArray[srcOffsetDoubles + i]); + } + } + + @Override + public void putFloatArray(long offsetBytes, float[] srcArray, int srcOffsetFloats, int lengthFloats) + { + final int offset = Ints.checkedCast(offsetBytes); + for (int i = 0; i < lengthFloats; i++) { + buffer.putFloat(offset + (i * Float.BYTES), srcArray[srcOffsetFloats + i]); + } + } + + @Override + public void putIntArray(long offsetBytes, int[] srcArray, int srcOffsetInts, int lengthInts) + { + final int offset = Ints.checkedCast(offsetBytes); + for (int i = 0; i < lengthInts; i++) { + buffer.putInt(offset + (i * Integer.BYTES), srcArray[srcOffsetInts + i]); + } + } + + @Override + public void putLongArray(long offsetBytes, long[] srcArray, int srcOffsetLongs, int lengthLongs) + { + final int offset = Ints.checkedCast(offsetBytes); + for (int i = 0; i < lengthLongs; i++) { + buffer.putLong(offset + (i * Long.BYTES), srcArray[srcOffsetLongs + i]); + } + } + + @Override + public void putShortArray(long offsetBytes, short[] srcArray, int srcOffsetShorts, int lengthShorts) + { + final int offset = Ints.checkedCast(offsetBytes); + for (int i = 0; i < lengthShorts; i++) { + buffer.putShort(offset + (i * Short.BYTES), srcArray[srcOffsetShorts + i]); + } + } + + @Override + public long getAndAddLong(long offsetBytes, long delta) + { + final int offset = Ints.checkedCast(offsetBytes); + final long currentValue; + synchronized (buffer) { + currentValue = buffer.getLong(offset); + buffer.putLong(offset, currentValue + delta); + } + return currentValue; + } + + @Override + public boolean compareAndSwapLong(long offsetBytes, long expect, long update) + { + final int offset = Ints.checkedCast(offsetBytes); + synchronized (buffer) { + final long actual = buffer.getLong(offset); + if (expect == actual) { + buffer.putLong(offset, update); + return true; + } + } + return false; + } + + @Override + public long getAndSetLong(long offsetBytes, long newValue) + { + int offset = Ints.checkedCast(offsetBytes); + synchronized (buffer) { + long l = buffer.getLong(offset); + buffer.putLong(offset, newValue); + return l; + } + } + + @Override + public Object getArray() + { + return null; + } + + @Override + public void clear() + { + fill((byte) 0); + } + + @Override + public void clear(long offsetBytes, long lengthBytes) + { + fill(offsetBytes, lengthBytes, (byte) 0); + } + + @Override + public void clearBits(long offsetBytes, byte bitMask) + { + final int offset = Ints.checkedCast(offsetBytes); + int value = buffer.get(offset) & 0XFF; + value &= ~bitMask; + buffer.put(offset, (byte) value); + } + + @Override + public void fill(byte value) + { + for (int i = 0; i < buffer.capacity(); i++) { + buffer.put(i, value); + } + } + + @Override + public void fill(long offsetBytes, long lengthBytes, byte value) + { + int offset = Ints.checkedCast(offsetBytes); + int length = Ints.checkedCast(lengthBytes); + for (int i = 0; i < length; i++) { + buffer.put(offset + i, value); + } + } + + @Override + public void setBits(long offsetBytes, byte bitMask) + { + final int offset = Ints.checkedCast(offsetBytes); + buffer.put(offset, (byte) (buffer.get(offset) | bitMask)); + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/data/SafeWritableBufferTest.java b/processing/src/test/java/org/apache/druid/segment/data/SafeWritableBufferTest.java new file mode 100644 index 000000000000..f432b7c167ca --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/data/SafeWritableBufferTest.java @@ -0,0 +1,224 @@ +/* + * 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.segment.data; + +import org.apache.datasketches.memory.Buffer; +import org.apache.datasketches.memory.WritableBuffer; +import org.junit.Assert; +import org.junit.Test; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +public class SafeWritableBufferTest +{ + private static final int CAPACITY = 1024; + + @Test + public void testPutAndGet() + { + WritableBuffer b1 = getBuffer(); + Assert.assertEquals(0, b1.getPosition()); + b1.putByte((byte) 0x01); + Assert.assertEquals(1, b1.getPosition()); + b1.putBoolean(true); + Assert.assertEquals(2, b1.getPosition()); + b1.putBoolean(false); + Assert.assertEquals(3, b1.getPosition()); + b1.putChar('c'); + Assert.assertEquals(5, b1.getPosition()); + b1.putDouble(1.1); + Assert.assertEquals(13, b1.getPosition()); + b1.putFloat(1.1f); + Assert.assertEquals(17, b1.getPosition()); + b1.putInt(100); + Assert.assertEquals(21, b1.getPosition()); + b1.putLong(1000L); + Assert.assertEquals(29, b1.getPosition()); + b1.putShort((short) 15); + Assert.assertEquals(31, b1.getPosition()); + b1.resetPosition(); + + Assert.assertEquals(0x01, b1.getByte()); + Assert.assertTrue(b1.getBoolean()); + Assert.assertFalse(b1.getBoolean()); + Assert.assertEquals('c', b1.getChar()); + Assert.assertEquals(1.1, b1.getDouble(), 0.0); + Assert.assertEquals(1.1f, b1.getFloat(), 0.0); + Assert.assertEquals(100, b1.getInt()); + Assert.assertEquals(1000L, b1.getLong()); + Assert.assertEquals(15, b1.getShort()); + } + + @Test + public void testPutAndGetArrays() + { + WritableBuffer buffer = getBuffer(); + final byte[] b1 = new byte[]{0x01, 0x02, 0x08, 0x08}; + final byte[] b2 = new byte[b1.length]; + + final boolean[] bool1 = new boolean[]{true, false, false, true}; + final boolean[] bool2 = new boolean[bool1.length]; + + final char[] chars1 = new char[]{'a', 'b', 'c', 'd'}; + final char[] chars2 = new char[chars1.length]; + + final double[] double1 = new double[]{1.1, -2.2, 3.3, 4.4}; + final double[] double2 = new double[double1.length]; + + final float[] float1 = new float[]{1.1f, 2.2f, -3.3f, 4.4f}; + final float[] float2 = new float[float1.length]; + + final int[] ints1 = new int[]{1, 2, -3, 4}; + final int[] ints2 = new int[ints1.length]; + + final long[] longs1 = new long[]{1L, -2L, 3L, -14L}; + final long[] longs2 = new long[ints1.length]; + + final short[] shorts1 = new short[]{1, -2, 3, -14}; + final short[] shorts2 = new short[ints1.length]; + + buffer.putByteArray(b1, 0, 2); + buffer.putByteArray(b1, 2, b1.length - 2); + buffer.putBooleanArray(bool1, 0, bool1.length); + buffer.putCharArray(chars1, 0, chars1.length); + buffer.putDoubleArray(double1, 0, double1.length); + buffer.putFloatArray(float1, 0, float1.length); + buffer.putIntArray(ints1, 0, ints1.length); + buffer.putLongArray(longs1, 0, longs1.length); + buffer.putShortArray(shorts1, 0, shorts1.length); + long pos = buffer.getPosition(); + buffer.resetPosition(); + buffer.getByteArray(b2, 0, b1.length); + buffer.getBooleanArray(bool2, 0, bool1.length); + buffer.getCharArray(chars2, 0, chars1.length); + buffer.getDoubleArray(double2, 0, double1.length); + buffer.getFloatArray(float2, 0, float1.length); + buffer.getIntArray(ints2, 0, ints1.length); + buffer.getLongArray(longs2, 0, longs1.length); + buffer.getShortArray(shorts2, 0, shorts1.length); + + Assert.assertArrayEquals(b1, b2); + Assert.assertArrayEquals(bool1, bool2); + Assert.assertArrayEquals(chars1, chars2); + for (int i = 0; i < double1.length; i++) { + Assert.assertEquals(double1[i], double2[i], 0.0); + } + for (int i = 0; i < float1.length; i++) { + Assert.assertEquals(float1[i], float2[i], 0.0); + } + Assert.assertArrayEquals(ints1, ints2); + Assert.assertArrayEquals(longs1, longs2); + Assert.assertArrayEquals(shorts1, shorts2); + + Assert.assertEquals(pos, buffer.getPosition()); + } + + @Test + public void testStartEndRegionAndDuplicate() + { + WritableBuffer buffer = getBuffer(); + Assert.assertEquals(0, buffer.getPosition()); + Assert.assertEquals(0, buffer.getStart()); + Assert.assertEquals(CAPACITY, buffer.getEnd()); + Assert.assertEquals(CAPACITY, buffer.getRemaining()); + Assert.assertEquals(CAPACITY, buffer.getCapacity()); + Assert.assertTrue(buffer.hasRemaining()); + buffer.fill((byte) 0x07); + buffer.setAndCheckStartPositionEnd(10L, 15L, 100L); + Assert.assertEquals(15L, buffer.getPosition()); + Assert.assertEquals(10L, buffer.getStart()); + Assert.assertEquals(100L, buffer.getEnd()); + Assert.assertEquals(85L, buffer.getRemaining()); + Assert.assertEquals(CAPACITY, buffer.getCapacity()); + buffer.fill((byte) 0x70); + buffer.resetPosition(); + Assert.assertEquals(10L, buffer.getPosition()); + for (int i = 0; i < 90; i++) { + if (i < 5) { + Assert.assertEquals(0x07, buffer.getByte()); + } else { + Assert.assertEquals(0x70, buffer.getByte()); + } + } + buffer.setAndCheckPosition(50); + + Buffer duplicate = buffer.duplicate(); + Assert.assertEquals(buffer.getStart(), duplicate.getStart()); + Assert.assertEquals(buffer.getPosition(), duplicate.getPosition()); + Assert.assertEquals(buffer.getEnd(), duplicate.getEnd()); + Assert.assertEquals(buffer.getRemaining(), duplicate.getRemaining()); + Assert.assertEquals(buffer.getCapacity(), duplicate.getCapacity()); + + duplicate.resetPosition(); + for (int i = 0; i < 90; i++) { + if (i < 5) { + Assert.assertEquals(0x07, duplicate.getByte()); + } else { + Assert.assertEquals(0x70, duplicate.getByte()); + } + } + + Buffer region = buffer.region(5L, 105L, buffer.getTypeByteOrder()); + Assert.assertEquals(0, region.getStart()); + Assert.assertEquals(0, region.getPosition()); + Assert.assertEquals(105L, region.getEnd()); + Assert.assertEquals(105L, region.getRemaining()); + Assert.assertEquals(105L, region.getCapacity()); + + for (int i = 0; i < 105; i++) { + if (i < 10) { + Assert.assertEquals(0x07, region.getByte()); + } else if (i < 95) { + Assert.assertEquals(0x70, region.getByte()); + } else { + Assert.assertEquals(0x07, region.getByte()); + } + } + } + + @Test + public void testFill() + { + WritableBuffer buffer = getBuffer(); + WritableBuffer anotherBuffer = getBuffer(); + + buffer.fill((byte) 0x0F); + anotherBuffer.fill((byte) 0x0F); + Assert.assertTrue(buffer.equalTo(0L, anotherBuffer, 0L, CAPACITY)); + + anotherBuffer.setPosition(100); + anotherBuffer.clear(); + Assert.assertFalse(buffer.equalTo(0L, anotherBuffer, 0L, CAPACITY)); + Assert.assertTrue(buffer.equalTo(0L, anotherBuffer, 0L, 100L)); + } + + private WritableBuffer getBuffer() + { + return getBuffer(CAPACITY); + } + + private WritableBuffer getBuffer(int capacity) + { + final ByteBuffer aBuffer = ByteBuffer.allocate(capacity).order(ByteOrder.LITTLE_ENDIAN); + SafeWritableBuffer memory = new SafeWritableBuffer(aBuffer); + return memory; + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/data/SafeWritableMemoryTest.java b/processing/src/test/java/org/apache/druid/segment/data/SafeWritableMemoryTest.java new file mode 100644 index 000000000000..786443f43ed6 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/data/SafeWritableMemoryTest.java @@ -0,0 +1,359 @@ +/* + * 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.segment.data; + +import org.apache.datasketches.memory.Memory; +import org.apache.datasketches.memory.WritableMemory; +import org.apache.datasketches.memory.internal.UnsafeUtil; +import org.junit.Assert; +import org.junit.Test; + +import java.io.CharArrayWriter; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +public class SafeWritableMemoryTest +{ + private static final int CAPACITY = 1024; + + @Test + public void testPutAndGet() + { + final WritableMemory memory = getMemory(); + memory.putByte(3L, (byte) 0x01); + Assert.assertEquals(memory.getByte(3L), 0x01); + + memory.putBoolean(1L, true); + Assert.assertTrue(memory.getBoolean(1L)); + memory.putBoolean(1L, false); + Assert.assertFalse(memory.getBoolean(1L)); + + memory.putChar(10L, 'c'); + Assert.assertEquals('c', memory.getChar(10L)); + + memory.putDouble(14L, 3.3); + Assert.assertEquals(3.3, memory.getDouble(14L), 0.0); + + memory.putFloat(27L, 3.3f); + Assert.assertEquals(3.3f, memory.getFloat(27L), 0.0); + + memory.putInt(11L, 1234); + Assert.assertEquals(1234, memory.getInt(11L)); + + memory.putLong(500L, 500L); + Assert.assertEquals(500L, memory.getLong(500L)); + + memory.putShort(11L, (short) 15); + Assert.assertEquals(15, memory.getShort(11L)); + + long l = memory.getAndSetLong(900L, 10L); + Assert.assertEquals(0L, l); + l = memory.getAndSetLong(900L, 100L); + Assert.assertEquals(10L, l); + l = memory.getAndAddLong(900L, 10L); + Assert.assertEquals(100L, l); + Assert.assertEquals(110L, memory.getLong(900L)); + Assert.assertTrue(memory.compareAndSwapLong(900L, 110L, 120L)); + Assert.assertFalse(memory.compareAndSwapLong(900L, 110L, 120L)); + Assert.assertEquals(120L, memory.getLong(900L)); + } + + @Test + public void testPutAndGetArrays() + { + final WritableMemory memory = getMemory(); + final byte[] b1 = new byte[]{0x01, 0x02, 0x08, 0x08}; + final byte[] b2 = new byte[b1.length]; + memory.putByteArray(12L, b1, 0, 3); + memory.putByteArray(15L, b1, 3, 1); + memory.getByteArray(12L, b2, 0, 3); + memory.getByteArray(15L, b2, 3, 1); + Assert.assertArrayEquals(b1, b2); + + final boolean[] bool1 = new boolean[]{true, false, false, true}; + final boolean[] bool2 = new boolean[bool1.length]; + memory.putBooleanArray(100L, bool1, 0, 2); + memory.putBooleanArray(102L, bool1, 2, 2); + memory.getBooleanArray(100L, bool2, 0, 2); + memory.getBooleanArray(102L, bool2, 2, 2); + Assert.assertArrayEquals(bool1, bool2); + + final char[] chars1 = new char[]{'a', 'b', 'c', 'd'}; + final char[] chars2 = new char[chars1.length]; + memory.putCharArray(10L, chars1, 0, 4); + memory.getCharArray(10L, chars2, 0, chars1.length); + Assert.assertArrayEquals(chars1, chars2); + + final double[] double1 = new double[]{1.1, -2.2, 3.3, 4.4}; + final double[] double2 = new double[double1.length]; + memory.putDoubleArray(100L, double1, 0, 1); + memory.putDoubleArray(100L + Double.BYTES, double1, 1, 3); + memory.getDoubleArray(100L, double2, 0, 2); + memory.getDoubleArray(100L + (2 * Double.BYTES), double2, 2, 2); + for (int i = 0; i < double1.length; i++) { + Assert.assertEquals(double1[i], double2[i], 0.0); + } + + final float[] float1 = new float[]{1.1f, 2.2f, -3.3f, 4.4f}; + final float[] float2 = new float[float1.length]; + memory.putFloatArray(100L, float1, 0, 1); + memory.putFloatArray(100L + Float.BYTES, float1, 1, 3); + memory.getFloatArray(100L, float2, 0, 2); + memory.getFloatArray(100L + (2 * Float.BYTES), float2, 2, 2); + for (int i = 0; i < float1.length; i++) { + Assert.assertEquals(float1[i], float2[i], 0.0); + } + + final int[] ints1 = new int[]{1, 2, -3, 4}; + final int[] ints2 = new int[ints1.length]; + memory.putIntArray(100L, ints1, 0, 1); + memory.putIntArray(100L + Integer.BYTES, ints1, 1, 3); + memory.getIntArray(100L, ints2, 0, 2); + memory.getIntArray(100L + (2 * Integer.BYTES), ints2, 2, 2); + Assert.assertArrayEquals(ints1, ints2); + + final long[] longs1 = new long[]{1L, -2L, 3L, -14L}; + final long[] longs2 = new long[ints1.length]; + memory.putLongArray(100L, longs1, 0, 1); + memory.putLongArray(100L + Long.BYTES, longs1, 1, 3); + memory.getLongArray(100L, longs2, 0, 2); + memory.getLongArray(100L + (2 * Long.BYTES), longs2, 2, 2); + Assert.assertArrayEquals(longs1, longs2); + + final short[] shorts1 = new short[]{1, -2, 3, -14}; + final short[] shorts2 = new short[ints1.length]; + memory.putShortArray(100L, shorts1, 0, 1); + memory.putShortArray(100L + Short.BYTES, shorts1, 1, 3); + memory.getShortArray(100L, shorts2, 0, 2); + memory.getShortArray(100L + (2 * Short.BYTES), shorts2, 2, 2); + Assert.assertArrayEquals(shorts1, shorts2); + } + + @Test + public void testFill() + { + final byte theByte = 0x01; + final byte anotherByte = 0x02; + final WritableMemory memory = getMemory(); + final int halfWay = (int) (memory.getCapacity() / 2); + + memory.fill(theByte); + for (int i = 0; i < memory.getCapacity(); i++) { + Assert.assertEquals(theByte, memory.getByte(i)); + } + + memory.fill(halfWay, memory.getCapacity() - halfWay, anotherByte); + for (int i = 0; i < memory.getCapacity(); i++) { + if (i < halfWay) { + Assert.assertEquals(theByte, memory.getByte(i)); + } else { + Assert.assertEquals(anotherByte, memory.getByte(i)); + } + } + + memory.clear(halfWay, memory.getCapacity() - halfWay); + for (int i = 0; i < memory.getCapacity(); i++) { + if (i < halfWay) { + Assert.assertEquals(theByte, memory.getByte(i)); + } else { + Assert.assertEquals(0, memory.getByte(i)); + } + } + + memory.setBits(halfWay - 1, anotherByte); + Assert.assertEquals(0x03, memory.getByte(halfWay - 1)); + memory.clearBits(halfWay - 1, theByte); + Assert.assertEquals(anotherByte, memory.getByte(halfWay - 1)); + + memory.clear(); + for (int i = 0; i < memory.getCapacity(); i++) { + Assert.assertEquals(0, memory.getByte(i)); + } + } + + @Test + public void testStringStuff() throws IOException + { + WritableMemory memory = getMemory(); + String s1 = "hello "; + memory.putCharsToUtf8(10L, s1); + + StringBuilder builder = new StringBuilder(); + memory.getCharsFromUtf8(10L, s1.length(), builder); + Assert.assertEquals(s1, builder.toString()); + + CharArrayWriter someAppendable = new CharArrayWriter(); + memory.getCharsFromUtf8(10L, s1.length(), someAppendable); + Assert.assertEquals(s1, someAppendable.toString()); + } + + @Test + public void testRegion() + { + WritableMemory memory = getMemory(); + Assert.assertEquals(CAPACITY, memory.getCapacity()); + Assert.assertEquals(0, memory.getCumulativeOffset()); + Assert.assertEquals(10L, memory.getCumulativeOffset(10L)); + Assert.assertThrows( + IllegalArgumentException.class, + () -> memory.checkValidAndBounds(CAPACITY - 10, 11L) + ); + + final byte[] someBytes = new byte[]{0x01, 0x02, 0x03, 0x04}; + memory.putByteArray(10L, someBytes, 0, someBytes.length); + + Memory region = memory.region(10L, someBytes.length); + Assert.assertEquals(someBytes.length, region.getCapacity()); + Assert.assertEquals(0, region.getCumulativeOffset()); + Assert.assertEquals(2L, region.getCumulativeOffset(2L)); + Assert.assertThrows( + IllegalArgumentException.class, + () -> region.checkValidAndBounds(2L, 4L) + ); + + final byte[] andBack = new byte[someBytes.length]; + region.getByteArray(0L, andBack, 0, someBytes.length); + Assert.assertArrayEquals(someBytes, andBack); + + Memory differentOrderRegion = memory.region(10L, someBytes.length, ByteOrder.BIG_ENDIAN); + // different order + Assert.assertFalse(region.isByteOrderCompatible(differentOrderRegion.getTypeByteOrder())); + // contents are equal tho + Assert.assertTrue(region.equalTo(0L, differentOrderRegion, 0L, someBytes.length)); + } + + @Test + public void testCompareAndEquals() + { + WritableMemory memory = getMemory(); + final byte[] someBytes = new byte[]{0x01, 0x02, 0x03, 0x04}; + final byte[] shorterSameBytes = new byte[]{0x01, 0x02, 0x03}; + final byte[] differentBytes = new byte[]{0x02, 0x02, 0x03, 0x04}; + memory.putByteArray(10L, someBytes, 0, someBytes.length); + memory.putByteArray(400L, someBytes, 0, someBytes.length); + memory.putByteArray(200L, shorterSameBytes, 0, shorterSameBytes.length); + memory.putByteArray(500L, differentBytes, 0, differentBytes.length); + + Assert.assertEquals(0, memory.compareTo(10L, someBytes.length, memory, 400L, someBytes.length)); + Assert.assertEquals(4, memory.compareTo(10L, someBytes.length, memory, 200L, someBytes.length)); + Assert.assertEquals(-1, memory.compareTo(10L, someBytes.length, memory, 500L, differentBytes.length)); + + WritableMemory memory2 = getMemory(); + memory2.putByteArray(0L, someBytes, 0, someBytes.length); + + Assert.assertEquals(0, memory.compareTo(10L, someBytes.length, memory2, 0L, someBytes.length)); + + Assert.assertTrue(memory.equalTo(10L, memory2, 0L, someBytes.length)); + + WritableMemory memory3 = getMemory(); + memory2.copyTo(0L, memory3, 0L, CAPACITY); + Assert.assertTrue(memory2.equalTo(0L, memory3, 0L, CAPACITY)); + } + + @Test + public void testHash() + { + WritableMemory memory = getMemory(); + final long[] someLongs = new long[]{1L, 10L, 100L, 1000L, 10000L}; + final int[] someInts = new int[]{1, 2, 3}; + final byte[] someBytes = new byte[]{0x01, 0x02, 0x03}; + final int longsLength = Long.BYTES * someLongs.length; + final int someIntsLength = Integer.BYTES * someInts.length; + final int totalLength = longsLength + someIntsLength + someBytes.length; + memory.putLongArray(2L, someLongs, 0, someLongs.length); + memory.putIntArray(2L + longsLength, someInts, 0, someInts.length); + memory.putByteArray(2L + longsLength + someIntsLength, someBytes, 0, someBytes.length); + Memory memory2 = Memory.wrap(memory.getByteBuffer(), ByteOrder.LITTLE_ENDIAN); + Assert.assertEquals( + memory2.xxHash64(2L, totalLength, 0), + memory.xxHash64(2L, totalLength, 0) + ); + + Assert.assertEquals( + memory2.xxHash64(2L, 0), + memory.xxHash64(2L, 0) + ); + } + + @Test + public void testToHexString() + { + + final byte[] bytes = new byte[]{0x00, 0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07}; + final WritableMemory memory = getMemory(bytes.length); + memory.putByteArray(0L, bytes, 0, bytes.length); + final long hcode = memory.hashCode() & 0XFFFFFFFFL; + final long bufferhcode = memory.getByteBuffer().hashCode() & 0XFFFFFFFFL; + final long reqhcode = memory.getMemoryRequestServer().hashCode() & 0XFFFFFFFFL; + Assert.assertEquals( + "### SafeWritableMemory SUMMARY ###\n" + + "Header Comment : test memory dump\n" + + "Call Parameters : .toHexString(..., 0, 8), hashCode: " + hcode + "\n" + + "UnsafeObj, hashCode : null\n" + + "UnsafeObjHeader : 0\n" + + "ByteBuf, hashCode : HeapByteBuffer, " + bufferhcode + "\n" + + "RegionOffset : 0\n" + + "Capacity : 8\n" + + "CumBaseOffset : 0\n" + + "MemReq, hashCode : HeapByteBufferMemoryRequestServer, " + reqhcode + "\n" + + "Valid : true\n" + + "Read Only : false\n" + + "Type Byte Order : LITTLE_ENDIAN\n" + + "Native Byte Order : LITTLE_ENDIAN\n" + + "JDK Runtime Version : " + UnsafeUtil.JDK + "\n" + + "Data, littleEndian : 0 1 2 3 4 5 6 7\n" + + " 0: 00 01 02 03 04 05 06 07 \n", + memory.toHexString("test memory dump", 0, bytes.length) + ); + } + + @Test + public void testMisc() + { + WritableMemory memory = getMemory(10); + WritableMemory memory2 = memory.getMemoryRequestServer().request(memory, 20); + Assert.assertEquals(20, memory2.getCapacity()); + + Assert.assertFalse(memory2.hasArray()); + + Assert.assertFalse(memory2.isReadOnly()); + Assert.assertFalse(memory2.isDirect()); + Assert.assertTrue(memory2.isValid()); + Assert.assertTrue(memory2.hasByteBuffer()); + + Assert.assertFalse(memory2.isSameResource(memory)); + Assert.assertTrue(memory2.isSameResource(memory2)); + + // does nothing + memory.getMemoryRequestServer().requestClose(memory, memory2); + } + + private WritableMemory getMemory() + { + return getMemory(CAPACITY); + } + + private WritableMemory getMemory(int capacity) + { + final ByteBuffer aBuffer = ByteBuffer.allocate(capacity).order(ByteOrder.LITTLE_ENDIAN); + return SafeWritableMemory.wrap(aBuffer); + } +} From d9a79f01ec69c3c09e13a22a52d70c6b7506834b Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Sat, 26 Nov 2022 04:12:22 -0800 Subject: [PATCH 08/34] fix off by one error in nested column range index (#13405) (#13422) --- ...NestedFieldLiteralColumnIndexSupplier.java | 5 +- ...edFieldLiteralColumnIndexSupplierTest.java | 118 ++++++++++++++---- 2 files changed, 94 insertions(+), 29 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnIndexSupplier.java index 373dcc822f2e..36c39f251b79 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnIndexSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnIndexSupplier.java @@ -228,10 +228,10 @@ private IntIntPair getLocalRangeFromDictionary( // valid global index in local dictionary, start here localStartIndex = localFound; } - // global end index is exclusive already, so we don't adjust local end index even for missing values + int localEndFound = localDictionary.indexOf(globalEndIndex); if (localEndFound < 0) { - localEndIndex = -localEndFound; + localEndIndex = -(localEndFound + 1); } else { localEndIndex = localEndFound; } @@ -239,6 +239,7 @@ private IntIntPair getLocalRangeFromDictionary( localStartIndex = Math.min(localStartIndex, localDictionary.size()); localEndIndex = Math.max(localStartIndex, Math.min(localDictionary.size(), localEndIndex)); + return new IntIntImmutablePair(localStartIndex, localEndIndex); } diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnIndexSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnIndexSupplierTest.java index 15b40b729f07..25fec68bb82b 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnIndexSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnIndexSupplierTest.java @@ -89,6 +89,9 @@ public void setup() throws IOException stringWriter.write("fo"); stringWriter.write("foo"); stringWriter.write("fooo"); + stringWriter.write("g"); + stringWriter.write("gg"); + stringWriter.write("ggg"); stringWriter.write("z"); writeToBuffer(stringBuffer, stringWriter); @@ -197,13 +200,44 @@ public void testSingleTypeStringColumnRangeIndex() throws IOException Assert.assertNotNull(rangeIndex); // 10 rows + // global: [null, a, b, fo, foo, fooo, g, gg, ggg, z] // local: [b, foo, fooo, z] // column: [foo, b, fooo, b, z, fooo, z, b, b, foo] - BitmapColumnIndex forRange = rangeIndex.forRange("f", true, "g", true); + BitmapColumnIndex forRange = rangeIndex.forRange(null, false, "a", false); Assert.assertNotNull(forRange); - Assert.assertEquals(0.4, forRange.estimateSelectivity(10), 0.0); + Assert.assertEquals(0.0, forRange.estimateSelectivity(10), 0.0); ImmutableBitmap bitmap = forRange.computeBitmapResult(bitmapResultFactory); + checkBitmap(bitmap); + + forRange = rangeIndex.forRange(null, false, "b", false); + Assert.assertNotNull(forRange); + Assert.assertEquals(0.4, forRange.estimateSelectivity(10), 0.0); + bitmap = forRange.computeBitmapResult(bitmapResultFactory); + checkBitmap(bitmap, 1, 3, 7, 8); + + forRange = rangeIndex.forRange("b", false, "fon", false); + Assert.assertNotNull(forRange); + Assert.assertEquals(0.4, forRange.estimateSelectivity(10), 0.0); + bitmap = forRange.computeBitmapResult(bitmapResultFactory); + checkBitmap(bitmap, 1, 3, 7, 8); + + forRange = rangeIndex.forRange("bb", false, "fon", false); + Assert.assertNotNull(forRange); + Assert.assertEquals(0.0, forRange.estimateSelectivity(10), 0.0); + bitmap = forRange.computeBitmapResult(bitmapResultFactory); + checkBitmap(bitmap); + + forRange = rangeIndex.forRange("b", true, "foo", false); + Assert.assertNotNull(forRange); + Assert.assertEquals(0.2, forRange.estimateSelectivity(10), 0.0); + bitmap = forRange.computeBitmapResult(bitmapResultFactory); + checkBitmap(bitmap, 0, 9); + + forRange = rangeIndex.forRange("f", true, "g", true); + Assert.assertNotNull(forRange); + Assert.assertEquals(0.4, forRange.estimateSelectivity(10), 0.0); + bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap, 0, 2, 5, 9); forRange = rangeIndex.forRange(null, false, "g", true); @@ -257,6 +291,36 @@ public void testSingleTypeStringColumnRangeIndex() throws IOException Assert.assertEquals(1.0, forRange.estimateSelectivity(10), 0.0); bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9); + + forRange = rangeIndex.forRange(null, true, "foa", false); + Assert.assertEquals(0.4, forRange.estimateSelectivity(10), 0.0); + bitmap = forRange.computeBitmapResult(bitmapResultFactory); + checkBitmap(bitmap, 1, 3, 7, 8); + + forRange = rangeIndex.forRange(null, true, "foooa", false); + Assert.assertEquals(0.8, forRange.estimateSelectivity(10), 0.0); + bitmap = forRange.computeBitmapResult(bitmapResultFactory); + checkBitmap(bitmap, 0, 1, 2, 3, 5, 7, 8, 9); + + forRange = rangeIndex.forRange("foooa", true, "ggg", false); + Assert.assertEquals(0.0, forRange.estimateSelectivity(10), 0.0); + bitmap = forRange.computeBitmapResult(bitmapResultFactory); + checkBitmap(bitmap); + + forRange = rangeIndex.forRange("g", true, "gg", false); + Assert.assertEquals(0.0, forRange.estimateSelectivity(10), 0.0); + bitmap = forRange.computeBitmapResult(bitmapResultFactory); + checkBitmap(bitmap); + + forRange = rangeIndex.forRange("z", true, "zz", false); + Assert.assertEquals(0.0, forRange.estimateSelectivity(10), 0.0); + bitmap = forRange.computeBitmapResult(bitmapResultFactory); + checkBitmap(bitmap); + + forRange = rangeIndex.forRange("z", false, "zz", false); + Assert.assertEquals(0.2, forRange.estimateSelectivity(10), 0.0); + bitmap = forRange.computeBitmapResult(bitmapResultFactory); + checkBitmap(bitmap, 4, 6); } @Test @@ -1123,7 +1187,7 @@ private NestedFieldLiteralColumnIndexSupplier makeSingleTypeStringSupplier() // 10 rows // globals: [ - // [null, a, b, fo, foo, fooo, z], + // [null, a, b, fo, foo, fooo, g, gg, ggg, z], // [1, 2, 3, 5, 100, 300, 9000], // [1.0, 1.1, 1.2, 2.0, 2.5, 3.3, 6.6, 9.9] // ] @@ -1143,7 +1207,7 @@ private NestedFieldLiteralColumnIndexSupplier makeSingleTypeStringSupplier() bitmapWriter.write(fillBitmap(2, 5)); // z - localDictionaryWriter.write(6); + localDictionaryWriter.write(9); bitmapWriter.write(fillBitmap(4, 6)); writeToBuffer(localDictionaryBuffer, localDictionaryWriter); @@ -1193,7 +1257,7 @@ private NestedFieldLiteralColumnIndexSupplier makeSingleTypeStringWithNullsSu bitmapWriter.open(); // 10 rows // globals: [ - // [null, a, b, fo, foo, fooo, z], + // [null, a, b, fo, foo, fooo, g, gg, ggg, z], // [1, 2, 3, 5, 100, 300, 9000], // [1.0, 1.1, 1.2, 2.0, 2.5, 3.3, 6.6, 9.9] // ] @@ -1217,7 +1281,7 @@ private NestedFieldLiteralColumnIndexSupplier makeSingleTypeStringWithNullsSu bitmapWriter.write(fillBitmap(2, 5)); // z - localDictionaryWriter.write(6); + localDictionaryWriter.write(9); bitmapWriter.write(fillBitmap(4, 6)); writeToBuffer(localDictionaryBuffer, localDictionaryWriter); @@ -1276,19 +1340,19 @@ private NestedFieldLiteralColumnIndexSupplier makeSingleTypeLongSupplier() th // column: [100, 1, 300, 1, 3, 3, 100, 300, 300, 1] // 1 - localDictionaryWriter.write(7); + localDictionaryWriter.write(10); bitmapWriter.write(fillBitmap(1, 3, 9)); // 3 - localDictionaryWriter.write(9); + localDictionaryWriter.write(12); bitmapWriter.write(fillBitmap(4, 5)); // 100 - localDictionaryWriter.write(11); + localDictionaryWriter.write(14); bitmapWriter.write(fillBitmap(0, 6)); // 300 - localDictionaryWriter.write(12); + localDictionaryWriter.write(15); bitmapWriter.write(fillBitmap(2, 7, 8)); writeToBuffer(localDictionaryBuffer, localDictionaryWriter); @@ -1351,19 +1415,19 @@ private NestedFieldLiteralColumnIndexSupplier makeSingleTypeLongSupplierWithN bitmapWriter.write(fillBitmap(2, 5, 8)); // 1 - localDictionaryWriter.write(7); + localDictionaryWriter.write(10); bitmapWriter.write(fillBitmap(1, 3, 9)); // 3 - localDictionaryWriter.write(9); + localDictionaryWriter.write(12); bitmapWriter.write(fillBitmap(4)); // 100 - localDictionaryWriter.write(11); + localDictionaryWriter.write(14); bitmapWriter.write(fillBitmap(0, 6)); // 300 - localDictionaryWriter.write(12); + localDictionaryWriter.write(15); bitmapWriter.write(fillBitmap(7)); writeToBuffer(localDictionaryBuffer, localDictionaryWriter); @@ -1422,19 +1486,19 @@ private NestedFieldLiteralColumnIndexSupplier makeSingleTypeDoubleSupplier() // column: [1.1, 1.1, 1.2, 3.3, 1.2, 6.6, 3.3, 1.2, 1.1, 3.3] // 1.1 - localDictionaryWriter.write(15); + localDictionaryWriter.write(18); bitmapWriter.write(fillBitmap(0, 1, 8)); // 1.2 - localDictionaryWriter.write(16); + localDictionaryWriter.write(19); bitmapWriter.write(fillBitmap(2, 4, 7)); // 3.3 - localDictionaryWriter.write(19); + localDictionaryWriter.write(22); bitmapWriter.write(fillBitmap(3, 6, 9)); // 6.6 - localDictionaryWriter.write(20); + localDictionaryWriter.write(23); bitmapWriter.write(fillBitmap(5)); writeToBuffer(localDictionaryBuffer, localDictionaryWriter); @@ -1497,19 +1561,19 @@ private NestedFieldLiteralColumnIndexSupplier makeSingleTypeDoubleSupplierWit bitmapWriter.write(fillBitmap(1, 3, 6)); // 1.1 - localDictionaryWriter.write(15); + localDictionaryWriter.write(18); bitmapWriter.write(fillBitmap(0, 8)); // 1.2 - localDictionaryWriter.write(16); + localDictionaryWriter.write(19); bitmapWriter.write(fillBitmap(2, 4, 7)); // 3.3 - localDictionaryWriter.write(19); + localDictionaryWriter.write(22); bitmapWriter.write(fillBitmap(9)); // 6.6 - localDictionaryWriter.write(20); + localDictionaryWriter.write(23); bitmapWriter.write(fillBitmap(5)); writeToBuffer(localDictionaryBuffer, localDictionaryWriter); @@ -1576,23 +1640,23 @@ private NestedFieldLiteralColumnIndexSupplier makeVariantSupplierWithNull() t bitmapWriter.write(fillBitmap(1, 9)); // z - localDictionaryWriter.write(6); + localDictionaryWriter.write(9); bitmapWriter.write(fillBitmap(6)); // 1 - localDictionaryWriter.write(7); + localDictionaryWriter.write(10); bitmapWriter.write(fillBitmap(0, 5)); // 300 - localDictionaryWriter.write(12); + localDictionaryWriter.write(15); bitmapWriter.write(fillBitmap(4)); // 1.1 - localDictionaryWriter.write(15); + localDictionaryWriter.write(18); bitmapWriter.write(fillBitmap(8)); // 9.9 - localDictionaryWriter.write(21); + localDictionaryWriter.write(24); bitmapWriter.write(fillBitmap(3)); writeToBuffer(localDictionaryBuffer, localDictionaryWriter); From 8e7a32ae9c1de1499937bff86d5b83de61a58799 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 29 Nov 2022 07:56:07 +0530 Subject: [PATCH 09/34] Add MetricsVerifier to simplify verification of metric values in tests (#13442) (#13444) --- .../java/util/metrics/MetricsVerifier.java | 98 +++++++++++++++++++ .../java/util/metrics/StubServiceEmitter.java | 38 +++++-- .../query/groupby/GroupByQueryRunnerTest.java | 3 +- .../druid/query/scan/ScanQueryRunnerTest.java | 3 +- .../timeseries/TimeseriesQueryRunnerTest.java | 6 +- .../simulate/BalancingStrategiesTest.java | 12 +-- .../simulate/CoordinatorSimulation.java | 7 +- .../CoordinatorSimulationBaseTest.java | 88 ++++------------- .../CoordinatorSimulationBuilder.java | 6 +- .../simulate/SegmentLoadingTest.java | 4 +- .../metrics/TaskCountStatsMonitorTest.java | 15 +-- .../TaskSlotCountStatsMonitorTest.java | 15 +-- .../WorkerTaskCountStatsMonitorTest.java | 59 ++++++----- .../druid/sql/http/SqlResourceTest.java | 12 +-- 14 files changed, 205 insertions(+), 161 deletions(-) create mode 100644 core/src/test/java/org/apache/druid/java/util/metrics/MetricsVerifier.java diff --git a/core/src/test/java/org/apache/druid/java/util/metrics/MetricsVerifier.java b/core/src/test/java/org/apache/druid/java/util/metrics/MetricsVerifier.java new file mode 100644 index 000000000000..c0f2eaeaf15f --- /dev/null +++ b/core/src/test/java/org/apache/druid/java/util/metrics/MetricsVerifier.java @@ -0,0 +1,98 @@ +/* + * 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.java.util.metrics; + +import org.junit.Assert; + +import java.util.List; +import java.util.Map; + +/** + * Test utility to extract and verify metric values. + */ +public interface MetricsVerifier +{ + /** + * Verifies that no event has been emitted for the given metric. + */ + default void verifyNotEmitted(String metricName) + { + verifyEmitted(metricName, 0); + } + + /** + * Verifies that the metric was emitted the expected number of times. + */ + default void verifyEmitted(String metricName, int times) + { + verifyEmitted(metricName, null, times); + } + + /** + * Verifies that the metric was emitted for the given dimension filters the + * expected number of times. + */ + default void verifyEmitted(String metricName, Map dimensionFilters, int times) + { + Assert.assertEquals( + "Metric was emitted unexpected number of times.", + times, + getMetricValues(metricName, dimensionFilters).size() + ); + } + + /** + * Verifies the value of the specified metric emitted in the previous run. + */ + default void verifyValue(String metricName, Number expectedValue) + { + verifyValue(metricName, null, expectedValue); + } + + /** + * Verifies the value of the event corresponding to the specified metric and + * dimensionFilters emitted in the previous run. + */ + default void verifyValue(String metricName, Map dimensionFilters, Number expectedValue) + { + Assert.assertEquals(expectedValue, getValue(metricName, dimensionFilters)); + } + + /** + * Gets the value of the event corresponding to the specified metric and + * dimensionFilters. + */ + default Number getValue(String metricName, Map dimensionFilters) + { + List values = getMetricValues(metricName, dimensionFilters); + Assert.assertEquals( + "Metric must have been emitted exactly once for the given dimensions.", + 1, + values.size() + ); + return values.get(0); + } + + /** + * Gets the metric values for the specified dimension filters. + */ + List getMetricValues(String metricName, Map dimensionFilters); + +} diff --git a/core/src/test/java/org/apache/druid/java/util/metrics/StubServiceEmitter.java b/core/src/test/java/org/apache/druid/java/util/metrics/StubServiceEmitter.java index 653dc8a08aae..395245815792 100644 --- a/core/src/test/java/org/apache/druid/java/util/metrics/StubServiceEmitter.java +++ b/core/src/test/java/org/apache/druid/java/util/metrics/StubServiceEmitter.java @@ -24,12 +24,15 @@ import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; -public class StubServiceEmitter extends ServiceEmitter +public class StubServiceEmitter extends ServiceEmitter implements MetricsVerifier { private final List events = new ArrayList<>(); - private final List metricEvents = new ArrayList<>(); + private final Map> metricEvents = new HashMap<>(); public StubServiceEmitter(String service, String host) { @@ -40,7 +43,9 @@ public StubServiceEmitter(String service, String host) public void emit(Event event) { if (event instanceof ServiceMetricEvent) { - metricEvents.add((ServiceMetricEvent) event); + ServiceMetricEvent metricEvent = (ServiceMetricEvent) event; + metricEvents.computeIfAbsent(metricEvent.getMetric(), name -> new ArrayList<>()) + .add(metricEvent); } events.add(event); } @@ -53,12 +58,29 @@ public List getEvents() return events; } - /** - * Gets all the metric events emitted since the previous {@link #flush()}. - */ - public List getMetricEvents() + @Override + public List getMetricValues( + String metricName, + Map dimensionFilters + ) { - return metricEvents; + final List values = new ArrayList<>(); + final List events = + metricEvents.getOrDefault(metricName, Collections.emptyList()); + final Map filters = + dimensionFilters == null ? Collections.emptyMap() : dimensionFilters; + for (ServiceMetricEvent event : events) { + final Map userDims = event.getUserDims(); + boolean match = filters.keySet().stream() + .map(d -> filters.get(d).equals(userDims.get(d))) + .reduce((a, b) -> a && b) + .orElse(true); + if (match) { + values.add(event.getValue()); + } + } + + return values; } @Override diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java index f9060db4c94c..47be50997f23 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java @@ -757,8 +757,7 @@ public void testGroupBy() query, serviceEmitter ); - Assert.assertEquals(1, serviceEmitter.getEvents().size()); - Assert.assertEquals(vectorize, serviceEmitter.getEvents().get(0).toMap().getOrDefault("vectorized", null)); + serviceEmitter.verifyEmitted("query/wait/time", ImmutableMap.of("vectorized", vectorize), 1); TestHelper.assertExpectedObjects(expectedResults, results, "groupBy"); } diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java index 31a7a454f374..4840a1ccfd18 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java @@ -230,8 +230,7 @@ public void testFullOnSelect() 0, 3 ); - Assert.assertEquals(1, stubServiceEmitter.getEvents().size()); - Assert.assertEquals(false, stubServiceEmitter.getEvents().get(0).toMap().getOrDefault("vectorized", null)); + stubServiceEmitter.verifyEmitted("query/wait/time", ImmutableMap.of("vectorized", false), 1); verify(expectedResults, populateNullColumnAtLastForQueryableIndexCase(results, "null_column")); } diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java index 74155ce51c54..d9d73e67effb 100644 --- a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java @@ -317,11 +317,7 @@ public void testFullOnTimeseries() ++count; } - Assert.assertEquals(1, stubServiceEmitter.getEvents().size()); - Assert.assertEquals( - vectorize, - stubServiceEmitter.getEvents().get(0).toMap().getOrDefault("vectorized", null) - ); + stubServiceEmitter.verifyEmitted("query/wait/time", ImmutableMap.of("vectorized", vectorize), 1); Assert.assertEquals(lastResult.toString(), expectedLast, lastResult.getTimestamp()); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/BalancingStrategiesTest.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/BalancingStrategiesTest.java index fd1c6c4d0f57..f135280825e9 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/BalancingStrategiesTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/BalancingStrategiesTest.java @@ -76,8 +76,8 @@ public void testFreshClusterGetsBalanced() runCoordinatorCycle(); loadQueuedSegments(); verifyValue(Metric.ASSIGNED_COUNT, 1000L); - verifyNoEvent(Metric.MOVED_COUNT); - verifyNoEvent(Metric.UNMOVED_COUNT); + verifyNotEmitted(Metric.MOVED_COUNT); + verifyNotEmitted(Metric.UNMOVED_COUNT); for (DruidServer historical : historicals) { Assert.assertEquals(200, historical.getTotalSegments()); @@ -113,8 +113,8 @@ public void testClusterGetsBalancedWhenServerIsAdded() runCoordinatorCycle(); loadQueuedSegments(); verifyValue(Metric.ASSIGNED_COUNT, 1000L); - verifyNoEvent(Metric.MOVED_COUNT); - verifyNoEvent(Metric.UNMOVED_COUNT); + verifyNotEmitted(Metric.MOVED_COUNT); + verifyNotEmitted(Metric.UNMOVED_COUNT); // Verify that each server is equally loaded for (DruidServer historical : historicals) { @@ -161,8 +161,8 @@ public void testClusterGetsBalancedWhenServerIsRemoved() runCoordinatorCycle(); loadQueuedSegments(); verifyValue(Metric.ASSIGNED_COUNT, 1000L); - verifyNoEvent(Metric.MOVED_COUNT); - verifyNoEvent(Metric.UNMOVED_COUNT); + verifyNotEmitted(Metric.MOVED_COUNT); + verifyNotEmitted(Metric.UNMOVED_COUNT); // Verify that each server is equally loaded for (DruidServer historical : historicals) { diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulation.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulation.java index f79a9bf48041..c0565c19ba9d 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulation.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulation.java @@ -20,7 +20,7 @@ package org.apache.druid.server.coordinator.simulate; import org.apache.druid.client.DruidServer; -import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; +import org.apache.druid.java.util.metrics.MetricsVerifier; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; import org.apache.druid.timeline.DataSegment; @@ -81,9 +81,10 @@ interface CoordinatorState DruidServer getInventoryView(String serverName); /** - * Returns the metric events emitted in the previous coordinator run. + * Returns a MetricsVerifier which can be used to extract and verify the + * metric values emitted in the previous coordinator run. */ - List getMetricEvents(); + MetricsVerifier getMetricsVerifier(); /** * Gets the load percentage of the specified datasource as seen by the coordinator. diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBaseTest.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBaseTest.java index d4e8b7760de0..84c6886d9d40 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBaseTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBaseTest.java @@ -21,7 +21,7 @@ import org.apache.druid.client.DruidServer; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; +import org.apache.druid.java.util.metrics.MetricsVerifier; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; import org.apache.druid.server.coordinator.CreateDataSegments; @@ -32,8 +32,6 @@ import org.junit.Assert; import org.junit.Before; -import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -51,13 +49,15 @@ * leading to flakiness in the tests. The simulation sets this field to true by * default. */ -public abstract class CoordinatorSimulationBaseTest - implements CoordinatorSimulation.CoordinatorState, CoordinatorSimulation.ClusterState +public abstract class CoordinatorSimulationBaseTest implements + CoordinatorSimulation.CoordinatorState, + CoordinatorSimulation.ClusterState, + MetricsVerifier { static final double DOUBLE_DELTA = 10e-9; private CoordinatorSimulation sim; - private final Map> latestMetricEvents = new HashMap<>(); + private MetricsVerifier metricsVerifier; @Before public abstract void setUp(); @@ -78,25 +78,19 @@ void startSimulation(CoordinatorSimulation simulation) { this.sim = simulation; simulation.start(); + this.metricsVerifier = this.sim.coordinator().getMetricsVerifier(); } @Override public void runCoordinatorCycle() { - latestMetricEvents.clear(); sim.coordinator().runCoordinatorCycle(); - - // Extract the metric values of this run - for (ServiceMetricEvent event : sim.coordinator().getMetricEvents()) { - latestMetricEvents.computeIfAbsent(event.getMetric(), m -> new ArrayList<>()) - .add(event); - } } @Override - public List getMetricEvents() + public MetricsVerifier getMetricsVerifier() { - return sim.coordinator().getMetricEvents(); + return null; } @Override @@ -153,61 +147,13 @@ void verifyDatasourceIsFullyLoaded(String datasource) Assert.assertEquals(100.0, getLoadPercentage(datasource), DOUBLE_DELTA); } - void verifyNoEvent(String metricName) - { - Assert.assertTrue(getMetricValues(metricName, null).isEmpty()); - } - - /** - * Verifies the value of the specified metric emitted in the previous run. - */ - void verifyValue(String metricName, Number expectedValue) - { - verifyValue(metricName, null, expectedValue); - } - - /** - * Verifies the value of the event corresponding to the specified metric and - * dimensionFilters emitted in the previous run. - */ - void verifyValue(String metricName, Map dimensionFilters, Number expectedValue) - { - Assert.assertEquals(expectedValue, getValue(metricName, dimensionFilters)); - } - - /** - * Gets the value of the event corresponding to the specified metric and - * dimensionFilters emitted in the previous run. - */ - Number getValue(String metricName, Map dimensionFilters) - { - List values = getMetricValues(metricName, dimensionFilters); - Assert.assertEquals( - "Metric must have been emitted exactly once for the given dimensions.", - 1, - values.size() - ); - return values.get(0); - } - - private List getMetricValues(String metricName, Map dimensionFilters) + @Override + public List getMetricValues( + String metricName, + Map dimensionFilters + ) { - final List values = new ArrayList<>(); - final List events = latestMetricEvents.getOrDefault(metricName, Collections.emptyList()); - final Map filters = dimensionFilters == null - ? Collections.emptyMap() : dimensionFilters; - for (ServiceMetricEvent event : events) { - final Map userDims = event.getUserDims(); - boolean match = filters.keySet().stream() - .map(d -> filters.get(d).equals(userDims.get(d))) - .reduce((a, b) -> a && b) - .orElse(true); - if (match) { - values.add(event.getValue()); - } - } - - return values; + return metricsVerifier.getMetricValues(metricName, dimensionFilters); } // Utility methods @@ -237,13 +183,13 @@ static CoordinatorDynamicConfig createDynamicConfig( /** * Creates a map containing dimension key-values to filter out metric events. */ - static Map filter(String... dimensionValues) + static Map filter(String... dimensionValues) { if (dimensionValues.length < 2 || dimensionValues.length % 2 == 1) { throw new IllegalArgumentException("Dimension key-values must be specified in pairs."); } - final Map filters = new HashMap<>(); + final Map filters = new HashMap<>(); for (int i = 0; i < dimensionValues.length; ) { filters.put(dimensionValues[i], dimensionValues[i + 1]); i += 2; diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java index 01308d82e73f..7066183c3ac6 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java @@ -32,8 +32,8 @@ import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; import org.apache.druid.java.util.common.lifecycle.Lifecycle; import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.java.util.metrics.MetricsVerifier; import org.apache.druid.java.util.metrics.StubServiceEmitter; import org.apache.druid.server.coordinator.BalancerStrategyFactory; import org.apache.druid.server.coordinator.CachingCostBalancerStrategyConfig; @@ -392,9 +392,9 @@ public double getLoadPercentage(String datasource) } @Override - public List getMetricEvents() + public MetricsVerifier getMetricsVerifier() { - return new ArrayList<>(env.serviceEmitter.getMetricEvents()); + return env.serviceEmitter; } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/SegmentLoadingTest.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/SegmentLoadingTest.java index 1edeab8a370f..e04ebe532df5 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/SegmentLoadingTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/SegmentLoadingTest.java @@ -148,7 +148,7 @@ public void testDropHappensAfterTargetReplicationOnEveryTier() startSimulation(sim); runCoordinatorCycle(); - verifyNoEvent(Metric.DROPPED_COUNT); + verifyNotEmitted(Metric.DROPPED_COUNT); int totalAssignedInRun1 = getValue(Metric.ASSIGNED_COUNT, filter(DruidMetrics.TIER, Tier.T2)).intValue() + getValue(Metric.ASSIGNED_COUNT, filter(DruidMetrics.TIER, Tier.T3)).intValue(); @@ -158,7 +158,7 @@ public void testDropHappensAfterTargetReplicationOnEveryTier() runCoordinatorCycle(); loadQueuedSegments(); - verifyNoEvent(Metric.DROPPED_COUNT); + verifyNotEmitted(Metric.DROPPED_COUNT); int totalLoadedAfterRun2 = historicalT21.getTotalSegments() + historicalT22.getTotalSegments() + historicalT31.getTotalSegments() + historicalT32.getTotalSegments(); diff --git a/server/src/test/java/org/apache/druid/server/metrics/TaskCountStatsMonitorTest.java b/server/src/test/java/org/apache/druid/server/metrics/TaskCountStatsMonitorTest.java index 24d89ab957a2..c6a47d56c427 100644 --- a/server/src/test/java/org/apache/druid/server/metrics/TaskCountStatsMonitorTest.java +++ b/server/src/test/java/org/apache/druid/server/metrics/TaskCountStatsMonitorTest.java @@ -75,15 +75,10 @@ public void testMonitor() final StubServiceEmitter emitter = new StubServiceEmitter("service", "host"); monitor.doMonitor(emitter); Assert.assertEquals(5, emitter.getEvents().size()); - Assert.assertEquals("task/success/count", emitter.getEvents().get(0).toMap().get("metric")); - Assert.assertEquals(1L, emitter.getEvents().get(0).toMap().get("value")); - Assert.assertEquals("task/failed/count", emitter.getEvents().get(1).toMap().get("metric")); - Assert.assertEquals(1L, emitter.getEvents().get(1).toMap().get("value")); - Assert.assertEquals("task/running/count", emitter.getEvents().get(2).toMap().get("metric")); - Assert.assertEquals(1L, emitter.getEvents().get(2).toMap().get("value")); - Assert.assertEquals("task/pending/count", emitter.getEvents().get(3).toMap().get("metric")); - Assert.assertEquals(1L, emitter.getEvents().get(3).toMap().get("value")); - Assert.assertEquals("task/waiting/count", emitter.getEvents().get(4).toMap().get("metric")); - Assert.assertEquals(1L, emitter.getEvents().get(4).toMap().get("value")); + emitter.verifyValue("task/success/count", 1L); + emitter.verifyValue("task/failed/count", 1L); + emitter.verifyValue("task/running/count", 1L); + emitter.verifyValue("task/pending/count", 1L); + emitter.verifyValue("task/waiting/count", 1L); } } diff --git a/server/src/test/java/org/apache/druid/server/metrics/TaskSlotCountStatsMonitorTest.java b/server/src/test/java/org/apache/druid/server/metrics/TaskSlotCountStatsMonitorTest.java index be33a4c98b68..0fed1c9b6bc1 100644 --- a/server/src/test/java/org/apache/druid/server/metrics/TaskSlotCountStatsMonitorTest.java +++ b/server/src/test/java/org/apache/druid/server/metrics/TaskSlotCountStatsMonitorTest.java @@ -75,15 +75,10 @@ public void testMonitor() final StubServiceEmitter emitter = new StubServiceEmitter("service", "host"); monitor.doMonitor(emitter); Assert.assertEquals(5, emitter.getEvents().size()); - Assert.assertEquals("taskSlot/total/count", emitter.getEvents().get(0).toMap().get("metric")); - Assert.assertEquals(1L, emitter.getEvents().get(0).toMap().get("value")); - Assert.assertEquals("taskSlot/idle/count", emitter.getEvents().get(1).toMap().get("metric")); - Assert.assertEquals(1L, emitter.getEvents().get(1).toMap().get("value")); - Assert.assertEquals("taskSlot/used/count", emitter.getEvents().get(2).toMap().get("metric")); - Assert.assertEquals(1L, emitter.getEvents().get(2).toMap().get("value")); - Assert.assertEquals("taskSlot/lazy/count", emitter.getEvents().get(3).toMap().get("metric")); - Assert.assertEquals(1L, emitter.getEvents().get(3).toMap().get("value")); - Assert.assertEquals("taskSlot/blacklisted/count", emitter.getEvents().get(4).toMap().get("metric")); - Assert.assertEquals(1L, emitter.getEvents().get(4).toMap().get("value")); + emitter.verifyValue("taskSlot/total/count", 1L); + emitter.verifyValue("taskSlot/idle/count", 1L); + emitter.verifyValue("taskSlot/used/count", 1L); + emitter.verifyValue("taskSlot/lazy/count", 1L); + emitter.verifyValue("taskSlot/blacklisted/count", 1L); } } diff --git a/server/src/test/java/org/apache/druid/server/metrics/WorkerTaskCountStatsMonitorTest.java b/server/src/test/java/org/apache/druid/server/metrics/WorkerTaskCountStatsMonitorTest.java index 01a97b7a9a03..fadb2f988269 100644 --- a/server/src/test/java/org/apache/druid/server/metrics/WorkerTaskCountStatsMonitorTest.java +++ b/server/src/test/java/org/apache/druid/server/metrics/WorkerTaskCountStatsMonitorTest.java @@ -20,10 +20,10 @@ package org.apache.druid.server.metrics; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.inject.Guice; import com.google.inject.Injector; -import com.google.inject.Module; import org.apache.druid.discovery.NodeRole; import org.apache.druid.java.util.metrics.StubServiceEmitter; import org.junit.Assert; @@ -143,24 +143,18 @@ public String getWorkerVersion() injectorForMiddleManager = Guice.createInjector( ImmutableList.of( - (Module) binder -> { - binder.bind(WorkerTaskCountStatsProvider.class).toInstance(statsProvider); - } + binder -> binder.bind(WorkerTaskCountStatsProvider.class).toInstance(statsProvider) ) ); injectorForMiddleManagerNullStats = Guice.createInjector( ImmutableList.of( - (Module) binder -> { - binder.bind(WorkerTaskCountStatsProvider.class).toInstance(nullStatsProvider); - } + binder -> binder.bind(WorkerTaskCountStatsProvider.class).toInstance(nullStatsProvider) ) ); injectorForPeon = Guice.createInjector( - ImmutableList.of( - (Module) binder -> {} - ) + ImmutableList.of(binder -> {}) ); } @@ -172,26 +166,31 @@ public void testMonitor() final StubServiceEmitter emitter = new StubServiceEmitter("service", "host"); monitor.doMonitor(emitter); Assert.assertEquals(5, emitter.getEvents().size()); - Assert.assertEquals("worker/task/failed/count", emitter.getEvents().get(0).toMap().get("metric")); - Assert.assertEquals("workerCategory", emitter.getEvents().get(0).toMap().get("category")); - Assert.assertEquals("workerVersion", emitter.getEvents().get(0).toMap().get("workerVersion")); - Assert.assertEquals(4L, emitter.getEvents().get(0).toMap().get("value")); - Assert.assertEquals("worker/task/success/count", emitter.getEvents().get(1).toMap().get("metric")); - Assert.assertEquals("workerCategory", emitter.getEvents().get(1).toMap().get("category")); - Assert.assertEquals("workerVersion", emitter.getEvents().get(1).toMap().get("workerVersion")); - Assert.assertEquals(2L, emitter.getEvents().get(1).toMap().get("value")); - Assert.assertEquals("worker/taskSlot/idle/count", emitter.getEvents().get(2).toMap().get("metric")); - Assert.assertEquals("workerCategory", emitter.getEvents().get(2).toMap().get("category")); - Assert.assertEquals("workerVersion", emitter.getEvents().get(2).toMap().get("workerVersion")); - Assert.assertEquals(3L, emitter.getEvents().get(2).toMap().get("value")); - Assert.assertEquals("worker/taskSlot/total/count", emitter.getEvents().get(3).toMap().get("metric")); - Assert.assertEquals("workerCategory", emitter.getEvents().get(3).toMap().get("category")); - Assert.assertEquals("workerVersion", emitter.getEvents().get(3).toMap().get("workerVersion")); - Assert.assertEquals(5L, emitter.getEvents().get(3).toMap().get("value")); - Assert.assertEquals("worker/taskSlot/used/count", emitter.getEvents().get(4).toMap().get("metric")); - Assert.assertEquals("workerCategory", emitter.getEvents().get(4).toMap().get("category")); - Assert.assertEquals("workerVersion", emitter.getEvents().get(4).toMap().get("workerVersion")); - Assert.assertEquals(1L, emitter.getEvents().get(4).toMap().get("value")); + emitter.verifyValue( + "worker/task/failed/count", + ImmutableMap.of("category", "workerCategory", "workerVersion", "workerVersion"), + 4L + ); + emitter.verifyValue( + "worker/task/success/count", + ImmutableMap.of("category", "workerCategory", "workerVersion", "workerVersion"), + 2L + ); + emitter.verifyValue( + "worker/taskSlot/idle/count", + ImmutableMap.of("category", "workerCategory", "workerVersion", "workerVersion"), + 3L + ); + emitter.verifyValue( + "worker/taskSlot/total/count", + ImmutableMap.of("category", "workerCategory", "workerVersion", "workerVersion"), + 5L + ); + emitter.verifyValue( + "worker/taskSlot/used/count", + ImmutableMap.of("category", "workerCategory", "workerVersion", "workerVersion"), + 1L + ); } @Test diff --git a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java index d3287530c55b..42fa66cbaa71 100644 --- a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java +++ b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java @@ -425,15 +425,9 @@ public void testSqlLifecycleMetrics() throws Exception ); checkSqlRequestLog(true); Assert.assertTrue(lifecycleManager.getAll("id").isEmpty()); - Set metricNames = ImmutableSet.of("sqlQuery/time", "sqlQuery/bytes", "sqlQuery/planningTimeMs"); - Assert.assertEquals(3, stubServiceEmitter.getEvents().size()); - for (String metricName : metricNames) { - Assert.assertTrue( - stubServiceEmitter.getEvents() - .stream() - .anyMatch(event -> event.toMap().containsValue(metricName)) - ); - } + stubServiceEmitter.verifyEmitted("sqlQuery/time", 1); + stubServiceEmitter.verifyValue("sqlQuery/bytes", 27L); + stubServiceEmitter.verifyEmitted("sqlQuery/planningTimeMs", 1); } From 38e5fad98989f31a55201c160ab9e456beda8886 Mon Sep 17 00:00:00 2001 From: Jill Osborne Date: Tue, 29 Nov 2022 02:29:42 +0000 Subject: [PATCH 10/34] Experimental features backport (#13443) (#13348) * Update and document experimental features (cherry picked from commit ccbf3abee9a341d8f5c80f37e7506088f10252bf) * Updated (cherry picked from commit d7b8fae15fc389ddbac1a57121ff965adb63f2af) * Update experimental-features.md * Updated after review (cherry picked from commit 975ae240ca2c1e008cd0ef7ad968beb6edd3413c) * Updated (cherry picked from commit eb8268ec44490134c16656245b8f73a8fcb550f6) * Update materialized-view.md (cherry picked from commit 53c3bde505abc861d22896b46204b65ec1e7851f) * Update experimental-features.md (cherry picked from commit 77148f7e97f5e1bef6a3474ee698ce37e6213307) --- docs/configuration/index.md | 4 +- docs/development/experimental-features.md | 68 +++++++++++++++++++ .../extensions-core/druid-lookups.md | 3 - .../kafka-supervisor-reference.md | 2 - .../extensions-core/kinesis-ingestion.md | 2 - docs/querying/lookups.md | 4 +- 6 files changed, 72 insertions(+), 11 deletions(-) create mode 100644 docs/development/experimental-features.md diff --git a/docs/configuration/index.md b/docs/configuration/index.md index 6fb3201d98f4..8ac9382b7e21 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -1375,7 +1375,7 @@ For GCE's properties, please refer to the [gce-extensions](../development/extens This section contains the configuration options for the processes that reside on Data servers (MiddleManagers/Peons and Historicals) in the suggested [three-server configuration](../design/processes.md#server-types). -Configuration options for the experimental [Indexer process](../design/indexer.md) are also provided here. +Configuration options for the [Indexer process](../design/indexer.md) are also provided here. ### MiddleManager and Peons @@ -2188,7 +2188,7 @@ Supported query contexts: |Key|Description|Default| |---|-----------|-------| |`druid.expressions.useStrictBooleans`|Controls the behavior of Druid boolean operators and functions, if set to `true` all boolean values will be either a `1` or `0`. See [expression documentation](../misc/math-expr.md#logical-operator-modes)|false| -|`druid.expressions.allowNestedArrays`|If enabled, Druid array expressions can create nested arrays. This is experimental and should be used with caution.|false| +|`druid.expressions.allowNestedArrays`|If enabled, Druid array expressions can create nested arrays.|false| ### Router #### Router Process Configs diff --git a/docs/development/experimental-features.md b/docs/development/experimental-features.md new file mode 100644 index 000000000000..a68fc47411af --- /dev/null +++ b/docs/development/experimental-features.md @@ -0,0 +1,68 @@ +--- +id: experimental-features +title: "Experimental features" +--- + + + +The following features are marked [experimental](./experimental.md) in the Druid docs. + +This document includes each page that mentions an experimental feature. To graduate a feature, remove all mentions of its experimental status on all relevant pages. + +Note that this document does not track the status of contrib extensions, some of which are experimental. + +## SQL-based ingestion + +- [SQL-based ingestion](../multi-stage-query/index.md) +- [SQL-based ingestion concepts](../multi-stage-query/concepts.md) +- [SQL-based ingestion and multi-stage query task API](../multi-stage-query/api.md) + +## Nested columns + +- [Nested columns](../querying/nested-columns.md) + +## Indexer process + +- [Indexer process](../design/indexer.md) +- [Processes and servers](../design/processes.md#indexer-process-optional) + +## Kubernetes + +- [Kubernetes](../development/extensions-core/kubernetes.md) + +## Segment locking + +- [Configuration reference](../configuration/index.md#overlord-operations) +- [Task reference](../ingestion/tasks.md#locking) +- [Design](../design/architecture.md#availability-and-consistency) + +## Moments sketch + +- [Aggregations](../querying/aggregations.md#moments-sketch-experimental) + +## Front coding + +- [Ingestion spec reference](../ingestion/ingestion-spec.md#front-coding) + +## Other configuration properties + +- [Configuration reference](../configuration/index.md) + - `CLOSED_SEGMENTS_SINKS` mode + - Expression processing configuration `druid.expressions.allowNestedArrays` diff --git a/docs/development/extensions-core/druid-lookups.md b/docs/development/extensions-core/druid-lookups.md index b44f9620bd0a..5b19508c2375 100644 --- a/docs/development/extensions-core/druid-lookups.md +++ b/docs/development/extensions-core/druid-lookups.md @@ -22,9 +22,6 @@ title: "Cached Lookup Module" ~ under the License. --> - -> Please note that this is an experimental module and the development/testing still at early stage. Feel free to try it and give us your feedback. - ## Description This Apache Druid module provides a per-lookup caching mechanism for JDBC data sources. The main goal of this cache is to speed up the access to a high latency lookup sources and to provide a caching isolation for every lookup source. diff --git a/docs/development/extensions-core/kafka-supervisor-reference.md b/docs/development/extensions-core/kafka-supervisor-reference.md index 210207302f2d..9dc80d0c6585 100644 --- a/docs/development/extensions-core/kafka-supervisor-reference.md +++ b/docs/development/extensions-core/kafka-supervisor-reference.md @@ -56,8 +56,6 @@ This topic contains configuration reference information for the Apache Kafka sup ## Task Autoscaler Properties -> Note that Task AutoScaler is currently designated as experimental. - | Property | Description | Required | | ------------- | ------------- | ------------- | | `enableTaskAutoScaler` | Enable or disable autoscaling. `false` or blank disables the `autoScaler` even when `autoScalerConfig` is not null| no (default == false) | diff --git a/docs/development/extensions-core/kinesis-ingestion.md b/docs/development/extensions-core/kinesis-ingestion.md index 84036665ec9e..31682b54a291 100644 --- a/docs/development/extensions-core/kinesis-ingestion.md +++ b/docs/development/extensions-core/kinesis-ingestion.md @@ -149,8 +149,6 @@ Where the file `supervisor-spec.json` contains a Kinesis supervisor spec: #### Task Autoscaler Properties -> Note that Task AutoScaler is currently designated as experimental. - | Property | Description | Required | | ------------- | ------------- | ------------- | | `enableTaskAutoScaler` | Enable or disable the auto scaler. When false or absent, Druid disables the `autoScaler` even when `autoScalerConfig` is not null.| no (default == false) | diff --git a/docs/querying/lookups.md b/docs/querying/lookups.md index 57b406afb199..860a3ed2e277 100644 --- a/docs/querying/lookups.md +++ b/docs/querying/lookups.md @@ -115,8 +115,8 @@ will not detect this automatically. Dynamic Configuration --------------------- -> Dynamic lookup configuration is an [experimental](../development/experimental.md) feature. Static -> configuration is no longer supported. +> Static configuration is no longer supported. + The following documents the behavior of the cluster-wide config which is accessible through the Coordinator. The configuration is propagated through the concept of "tier" of servers. A "tier" is defined as a group of services which should receive a set of lookups. From 41257019b599ead100034729126d76e003326778 Mon Sep 17 00:00:00 2001 From: Jill Osborne Date: Tue, 29 Nov 2022 21:58:18 +0000 Subject: [PATCH 11/34] Update nested columns docs (#13424) * Update nested columns docs * Update nested-columns.md --- docs/ingestion/data-formats.md | 4 ++-- docs/ingestion/schema-design.md | 9 ++++----- docs/querying/nested-columns.md | 20 ++++++++++---------- 3 files changed, 16 insertions(+), 17 deletions(-) diff --git a/docs/ingestion/data-formats.md b/docs/ingestion/data-formats.md index eb08df0cf7a8..557060a5e663 100644 --- a/docs/ingestion/data-formats.md +++ b/docs/ingestion/data-formats.md @@ -606,9 +606,9 @@ For example: ### FlattenSpec -The `flattenSpec` object bridges the gap between potentially nested input data, such as Avro or ORC, and Druid's flat data model. It is an object within the `inputFormat` object. +You can use the `flattenSpec` object to flatten nested data, as an alternative to the Druid [nested columns](../querying/nested-columns.md) feature, and for nested input formats unsupported by the feature. It is an object within the `inputFormat` object. -> If you have nested JSON data, you can ingest and store JSON in an Apache Druid column as a `COMPLEX` data type. See [Nested columns](../querying/nested-columns.md) for more information. +See [Nested columns](../querying/nested-columns.md) for information on ingesting and storing nested data in an Apache Druid column as a `COMPLEX` data type. Configure your `flattenSpec` as follows: diff --git a/docs/ingestion/schema-design.md b/docs/ingestion/schema-design.md index 10e6ea82cd2b..f006e792bc44 100644 --- a/docs/ingestion/schema-design.md +++ b/docs/ingestion/schema-design.md @@ -116,14 +116,13 @@ naturally emitted. It is also useful if you want to combine timeseries and non-t Similar to log aggregation systems, Druid offers inverted indexes for fast searching and filtering. Druid's search capabilities are generally less developed than these systems, and its analytical capabilities are generally more developed. The main data modeling differences between Druid and these systems are that when ingesting data into Druid, -you must be more explicit. Druid columns have types specific upfront and Druid does not, at this time, natively support -nested data. +you must be more explicit. Druid columns have types specific upfront. Tips for modeling log data in Druid: * If you don't know ahead of time what columns you'll want to ingest, use an empty dimensions list to trigger [automatic detection of dimension columns](#schema-less-dimensions). -* If you have nested data, flatten it using a [`flattenSpec`](./ingestion-spec.md#flattenspec). +* If you have nested data, you can ingest it using the [nested columns](../querying/nested-columns.md) feature or flatten it using a [`flattenSpec`](./ingestion-spec.md#flattenspec). * Consider enabling [rollup](./rollup.md) if you have mainly analytical use cases for your log data. This will mean you lose the ability to retrieve individual events from Druid, but you potentially gain substantial compression and query performance boosts. @@ -198,9 +197,9 @@ like `MILLIS_TO_TIMESTAMP`, `TIME_FLOOR`, and others. If you're using native Dru ### Nested dimensions -You can ingest and store nested JSON in a Druid column as a `COMPLEX` data type. See [Nested columns](../querying/nested-columns.md) for more information. +You can ingest and store nested data in a Druid column as a `COMPLEX` data type. See [Nested columns](../querying/nested-columns.md) for more information. -If you want to ingest nested data in a format other than JSON—for example Avro, ORC, and Parquet—you must use the `flattenSpec` object to flatten it. For example, if you have data of the following form: +If you want to ingest nested data in a format unsupported by the nested columns feature, you must use the `flattenSpec` object to flatten it. For example, if you have data of the following form: ```json { "foo": { "bar": 3 } } diff --git a/docs/querying/nested-columns.md b/docs/querying/nested-columns.md index e8dc628c8fc5..77af91ddff89 100644 --- a/docs/querying/nested-columns.md +++ b/docs/querying/nested-columns.md @@ -23,17 +23,17 @@ sidebar_label: Nested columns ~ under the License. --> -> Nested columns is an experimental feature available starting in Apache Druid 24.0. Like most experimental features, functionality documented on this page is subject to change in future releases. However, the COMPLEX column type includes versioning to provide backward compatible support in future releases. We strongly encourage you to experiment with nested columns in your development environment to evaluate that they meet your use case. If so, you can use them in production scenarios. Review the release notes and this page to stay up to date with changes. - Apache Druid supports directly storing nested data structures in `COMPLEX` columns. `COMPLEX` columns store a copy of the structured data in JSON format and specialized internal columns and indexes for nested literal values—STRING, LONG, and DOUBLE types. An optimized [virtual column](./virtual-columns.md#nested-field-virtual-column) allows Druid to read and filter these values at speeds consistent with standard Druid LONG, DOUBLE, and STRING columns. Druid [SQL JSON functions](./sql-json-functions.md) allow you to extract, transform, and create `COMPLEX` values in SQL queries, using the specialized virtual columns where appropriate. You can use the [JSON nested columns functions](../misc/math-expr.md#json-functions) in [native queries](./querying.md) using [expression virtual columns](./virtual-columns.md#expression-virtual-column), and in native ingestion with a [`transformSpec`](../ingestion/ingestion-spec.md#transformspec). You can use the JSON functions in INSERT and REPLACE statements in SQL-based ingestion, or in a `transformSpec` in native ingestion as an alternative to using a [`flattenSpec`](../ingestion/data-formats.md#flattenspec) object to "flatten" nested data for ingestion. +Druid supports directly ingesting nested data with the following formats: JSON, Parquet, Avro, ORC. + ## Example nested data -The examples in this topic use the data in [`nested_example_data.json`](https://static.imply.io/data/nested_example_data.json). The file contains a simple facsimile of an order tracking and shipping table. +The examples in this topic use the JSON data in [`nested_example_data.json`](https://static.imply.io/data/nested_example_data.json). The file contains a simple facsimile of an order tracking and shipping table. When pretty-printed, a sample row in `nested_example_data` looks like this: @@ -63,7 +63,7 @@ When pretty-printed, a sample row in `nested_example_data` looks like this: ## Native batch ingestion -For native batch ingestion, you can use the [JSON nested columns functions](./sql-json-functions.md) to extract nested data as an alternative to using the [`flattenSpec`](../ingestion/data-formats.md#flattenspec) input format. +For native batch ingestion, you can use the [SQL JSON functions](./sql-json-functions.md) to extract nested data as an alternative to using the [`flattenSpec`](../ingestion/data-formats.md#flattenspec) input format. To configure a dimension as a nested data type, specify the `json` type for the dimension in the `dimensions` list in the `dimensionsSpec` property of your ingestion spec. @@ -124,7 +124,7 @@ For example, the following ingestion spec instructs Druid to ingest `shipTo` and ### Transform data during batch ingestion -You can use the [JSON nested columns functions](./sql-json-functions.md) to transform JSON data and reference the transformed data in your ingestion spec. +You can use the [SQL JSON functions](./sql-json-functions.md) to transform nested data and reference the transformed data in your ingestion spec. To do this, define the output name and expression in the `transforms` list in the `transformSpec` object of your ingestion spec. @@ -192,7 +192,7 @@ For example, the following ingestion spec extracts `firstName`, `lastName` and ` ## SQL-based ingestion -To ingest nested data using multi-stage query architecture, specify `COMPLEX` as the value for `type` when you define the row signature—`shipTo` and `details` in the following example ingestion spec: +To ingest nested data using SQL-based ingestion, specify `COMPLEX` as the value for `type` when you define the row signature—`shipTo` and `details` in the following example ingestion spec: ![SQL-based ingestion](../assets/nested-msq-ingestion.png) @@ -297,7 +297,7 @@ The [Kafka tutorial](../tutorials/tutorial-kafka.md) guides you through the step ### Transform data during SQL-based ingestion -You can use the [JSON nested columns functions](./sql-json-functions.md) to transform JSON data in your ingestion query. +You can use the [SQL JSON functions](./sql-json-functions.md) to transform nested data in your ingestion query. For example, the following ingestion query is the SQL-based version of the [previous batch example](#transform-data-during-batch-ingestion)—it extracts `firstName`, `lastName`, and `address` from `shipTo` and creates a composite JSON object containing `product`, `details`, and `department`. @@ -326,7 +326,7 @@ PARTITIONED BY ALL ## Ingest a JSON string as COMPLEX -If your source data uses a string representation of your JSON column, you can still ingest the data as `COMPLEX` as follows: +If your source data contains serialized JSON strings, you can ingest the data as `COMPLEX` as follows: - During native batch ingestion, call the `parse_json` function in a `transform` object in the `transformSpec`. - During SQL-based ingestion, use the PARSE_JSON keyword within your SELECT statement to transform the string values to JSON. - If you are concerned that your data may not contain valid JSON, you can use `try_parse_json` for native batch or `TRY_PARSE_JSON` for SQL-based ingestion. For cases where the column does not contain valid JSON, Druid inserts a null value. @@ -563,7 +563,7 @@ In addition to `JSON_VALUE`, Druid offers a number of operators that focus on tr - `PARSE_JSON` - `TO_JSON_STRING` -These functions are primarily intended for use with the multi-stage query architecture to transform data during insert operations, but they also work in traditional Druid SQL queries. Because most of these functions output JSON objects, they have the same limitations when used in traditional Druid queries as interacting with the JSON objects directly. +These functions are primarily intended for use with SQL-based ingestion to transform data during insert operations, but they also work in traditional Druid SQL queries. Because most of these functions output JSON objects, they have the same limitations when used in traditional Druid queries as interacting with the JSON objects directly. #### Example query: Return results in a JSON object @@ -663,7 +663,7 @@ Before you start using the nested columns feature, consider the following known - Directly using `COMPLEX` columns and expressions is not well integrated into the Druid query engine. It can result in errors or undefined behavior when grouping and filtering, and when you use `COMPLEX` objects as inputs to aggregators. As a workaround, consider using `TO_JSON_STRING` to coerce the values to strings before you perform these operations. - Directly using array-typed outputs from `JSON_KEYS` and `JSON_PATHS` is moderately supported by the Druid query engine. You can group on these outputs, and there are a number of array expressions that can operate on these values, such as `ARRAY_CONCAT_AGG`. However, some operations are not well defined for use outside array-specific functions, such as filtering using `=` or `IS NULL`. - Input validation for JSON SQL operators is currently incomplete, which sometimes results in undefined behavior or unhelpful error messages. -- Ingesting JSON columns with a very complex nested structure is potentially an expensive operation and may require you to tune ingestion tasks and/or cluster parameters to account for increased memory usage or overall task run time. When you tune your ingestion configuration, treat each nested literal field inside a JSON object as a flattened top-level Druid column. +- Ingesting data with a very complex nested structure is potentially an expensive operation and may require you to tune ingestion tasks and/or cluster parameters to account for increased memory usage or overall task run time. When you tune your ingestion configuration, treat each nested literal field inside an object as a flattened top-level Druid column. ## Further reading From 753d7702bc2cf9949587bd5d99fba95102da0963 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 29 Nov 2022 14:43:00 -0800 Subject: [PATCH 12/34] fix issues with nested data conversion (#13407) (#13448) --- .../util/common/parsers/ObjectFlatteners.java | 13 +++++- .../common/parsers/ObjectFlattenersTest.java | 14 ++++++- .../orc-extensions/example/test_simple.orc | Bin 0 -> 506 bytes .../druid/data/input/orc/OrcReaderTest.java | 39 ++++++++++++++++++ 4 files changed, 63 insertions(+), 3 deletions(-) create mode 100644 extensions-core/orc-extensions/example/test_simple.orc diff --git a/core/src/main/java/org/apache/druid/java/util/common/parsers/ObjectFlatteners.java b/core/src/main/java/org/apache/druid/java/util/common/parsers/ObjectFlatteners.java index 4e955a4d50b8..98f087333a55 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/parsers/ObjectFlatteners.java +++ b/core/src/main/java/org/apache/druid/java/util/common/parsers/ObjectFlatteners.java @@ -29,6 +29,7 @@ import java.util.AbstractMap; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.LinkedHashMap; import java.util.LinkedHashSet; @@ -250,15 +251,23 @@ default Function makeJsonTreeExtractor(List nodes) */ default Map toMap(T obj) { - return (Map) toPlainJavaType(obj); + final Object mapOrNull = toPlainJavaType(obj); + if (mapOrNull == null) { + return Collections.emptyMap(); + } + return (Map) mapOrNull; } /** * Recursively traverse "json" object using a {@link JsonProvider}, converting to Java {@link Map} and {@link List}, * potentially transforming via {@link #finalizeConversionForMap} as we go */ + @Nullable default Object toPlainJavaType(Object o) { + if (o == null) { + return null; + } final JsonProvider jsonProvider = getJsonProvider(); if (jsonProvider.isMap(o)) { Map actualMap = new HashMap<>(); @@ -287,7 +296,7 @@ default Object toPlainJavaType(Object o) return finalizeConversionForMap(actualList); } // unknown, just pass it through - return o; + return finalizeConversionForMap(o); } /** diff --git a/core/src/test/java/org/apache/druid/java/util/common/parsers/ObjectFlattenersTest.java b/core/src/test/java/org/apache/druid/java/util/common/parsers/ObjectFlattenersTest.java index 2b610690db0a..e0b0fbcc510c 100644 --- a/core/src/test/java/org/apache/druid/java/util/common/parsers/ObjectFlattenersTest.java +++ b/core/src/test/java/org/apache/druid/java/util/common/parsers/ObjectFlattenersTest.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import org.junit.Assert; import org.junit.Test; @@ -32,12 +33,14 @@ public class ObjectFlattenersTest { private static final String SOME_JSON = "{\"foo\": null, \"bar\": 1}"; + + private static final ObjectFlatteners.FlattenerMaker FLATTENER_MAKER = new JSONFlattenerMaker(true); private static final ObjectFlattener FLATTENER = ObjectFlatteners.create( new JSONPathSpec( true, ImmutableList.of(new JSONPathFieldSpec(JSONPathFieldType.PATH, "extract", "$.bar")) ), - new JSONFlattenerMaker(true) + FLATTENER_MAKER ); private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); @@ -62,4 +65,13 @@ public void testToMap() throws JsonProcessingException Assert.assertNull(flat.get("foo")); Assert.assertEquals(1, flat.get("bar")); } + + @Test + public void testToMapNull() throws JsonProcessingException + { + JsonNode node = OBJECT_MAPPER.readTree("null"); + Map flat = FLATTENER.toMap(node); + Assert.assertNull(FLATTENER_MAKER.toPlainJavaType(node)); + Assert.assertEquals(ImmutableMap.of(), flat); + } } diff --git a/extensions-core/orc-extensions/example/test_simple.orc b/extensions-core/orc-extensions/example/test_simple.orc new file mode 100644 index 0000000000000000000000000000000000000000..31c5391fb687961c1837637c7954841c96235b02 GIT binary patch literal 506 zcmeYda+YOa;Nsz8VE_Ul77nHWh5!bJ$D5u%N@-wJROVAtmDNjBW>!{?Wp-BJGBnu0 zq^#JoKrklYka`0ngBnn&(9=f=4NQuDd~UjFdTz%U+!)M&qCro<1_<)GsoLpHdo)SO zS881S%MxY5S4l!>dv>mXClL9Pw|wPN^O9;eT>H}$Xd zi^#fO;Bj1y+1htb|D`3DoqC;iYHvS(WOup#gsJSOtiITM=}=w^&P({Qq@wOO;_;Hi#gPtSJm8z&B&@VI_Jpu>rWXUzc*4#f-> z@hKk|Up;)1a3$$kQVLJP7NLg;A&(v>Y3gYji7-tTl40!O6aHf(F*$zS`-7&-bG b1sEk78d#K=714cMB#Ff;oHIZFTlJD96o literal 0 HcmV?d00001 diff --git a/extensions-core/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcReaderTest.java b/extensions-core/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcReaderTest.java index d0f012245b03..03083f61ebc8 100644 --- a/extensions-core/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcReaderTest.java +++ b/extensions-core/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcReaderTest.java @@ -29,6 +29,7 @@ import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.FileEntity; +import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.parsers.CloseableIterator; @@ -569,6 +570,44 @@ public void testNestedArray() throws IOException } } + @Test + public void testSimpleNullValues() throws IOException + { + final InputFormat inputFormat = new OrcInputFormat( + new JSONPathSpec( + true, + ImmutableList.of() + ), + null, + new Configuration() + ); + final InputEntityReader reader = createReader( + new TimestampSpec("timestamp", "auto", null), + new DimensionsSpec( + ImmutableList.of( + new StringDimensionSchema("c1"), + new StringDimensionSchema("c2") + ) + ), + inputFormat, + "example/test_simple.orc" + ); + try (CloseableIterator iterator = reader.read()) { + Assert.assertTrue(iterator.hasNext()); + InputRow row = iterator.next(); + + Assert.assertEquals(DateTimes.of("2022-01-01T00:00:00.000Z"), row.getTimestamp()); + Assert.assertEquals("true", Iterables.getOnlyElement(row.getDimension("c1"))); + Assert.assertEquals("str1", Iterables.getOnlyElement(row.getDimension("c2"))); + + row = iterator.next(); + Assert.assertEquals(DateTimes.of("2022-01-02T00:00:00.000Z"), row.getTimestamp()); + Assert.assertEquals(ImmutableList.of(), row.getDimension("c1")); + Assert.assertEquals(ImmutableList.of(), row.getDimension("c2")); + Assert.assertFalse(iterator.hasNext()); + } + } + private InputEntityReader createReader( TimestampSpec timestampSpec, DimensionsSpec dimensionsSpec, From 23500a4c28767c0a74b2557f8966e13720b7511b Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 29 Nov 2022 19:33:39 -0800 Subject: [PATCH 13/34] fix KafkaInputFormat with nested columns by delegating to underlying inputRow map instead of eagerly copying (#13406) (#13447) --- .../input/kafkainput/KafkaInputReader.java | 177 ++++++++++++------ .../kafkainput/KafkaInputFormatTest.java | 7 +- 2 files changed, 124 insertions(+), 60 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaInputReader.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaInputReader.java index 458955e58070..56e94215940b 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaInputReader.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaInputReader.java @@ -30,24 +30,24 @@ import org.apache.druid.indexing.seekablestream.SettableByteEntity; import org.apache.druid.java.util.common.CloseableIterators; import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.java.util.common.parsers.ParseException; import javax.annotation.Nullable; - 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 KafkaInputReader implements InputEntityReader { - private static final Logger log = new Logger(KafkaInputReader.class); - private final InputRowSchema inputRowSchema; private final SettableByteEntity source; private final Function headerParserSupplier; @@ -85,7 +85,60 @@ public KafkaInputReader( this.timestampColumnName = timestampColumnName; } - private List getFinalDimensionList(HashSet newDimensions) + @Override + public CloseableIterator read() throws IOException + { + final KafkaRecordEntity record = source.getEntity(); + final Map mergedHeaderMap = new HashMap<>(); + if (headerParserSupplier != null) { + KafkaHeaderReader headerParser = headerParserSupplier.apply(record); + List> headerList = headerParser.read(); + for (Pair ele : headerList) { + mergedHeaderMap.put(ele.lhs, ele.rhs); + } + } + + // Add kafka record timestamp to the mergelist, we will skip record timestamp if the same key exists already in + // the header list + mergedHeaderMap.putIfAbsent(timestampColumnName, record.getRecord().timestamp()); + + InputEntityReader keyParser = (keyParserSupplier == null) ? null : keyParserSupplier.apply(record); + if (keyParser != null) { + try (CloseableIterator keyIterator = keyParser.read()) { + // Key currently only takes the first row and ignores the rest. + if (keyIterator.hasNext()) { + // Return type for the key parser should be of type MapBasedInputRow + // Parsers returning other types are not compatible currently. + MapBasedInputRow keyRow = (MapBasedInputRow) keyIterator.next(); + // Add the key to the mergeList only if the key string is not already present + mergedHeaderMap.putIfAbsent( + keyColumnName, + keyRow.getEvent().entrySet().stream().findFirst().get().getValue() + ); + } + } + catch (ClassCastException e) { + throw new IOException( + "Unsupported keyFormat. KafkaInputformat only supports input format that return MapBasedInputRow rows" + ); + } + } + + // Ignore tombstone records that have null values. + if (record.getRecord().value() != null) { + return buildBlendedRows(valueParser, mergedHeaderMap); + } else { + return buildRowsWithoutValuePayload(mergedHeaderMap); + } + } + + @Override + public CloseableIterator sample() throws IOException + { + return read().map(row -> InputRowListPlusRawValues.of(row, ((MapBasedInputRow) row).getEvent())); + } + + private List getFinalDimensionList(Set newDimensions) { final List schemaDimensions = inputRowSchema.getDimensionsSpec().getDimensionNames(); if (!schemaDimensions.isEmpty()) { @@ -97,11 +150,14 @@ private List getFinalDimensionList(HashSet newDimensions) } } - private CloseableIterator buildBlendedRows(InputEntityReader valueParser, Map headerKeyList) throws IOException + private CloseableIterator buildBlendedRows( + InputEntityReader valueParser, + Map headerKeyList + ) throws IOException { return valueParser.read().map( r -> { - MapBasedInputRow valueRow; + final MapBasedInputRow valueRow; try { // Return type for the value parser should be of type MapBasedInputRow // Parsers returning other types are not compatible currently. @@ -113,14 +169,9 @@ private CloseableIterator buildBlendedRows(InputEntityReader valuePars "Unsupported input format in valueFormat. KafkaInputFormat only supports input format that return MapBasedInputRow rows" ); } - Map event = new HashMap<>(headerKeyList); - /* Currently we prefer payload attributes if there is a collision in names. - We can change this beahvior in later changes with a config knob. This default - behavior lets easy porting of existing inputFormats to the new one without any changes. - */ - event.putAll(valueRow.getEvent()); - - HashSet newDimensions = new HashSet(valueRow.getDimensions()); + + final Map event = buildBlendedEventMap(valueRow.getEvent(), headerKeyList); + final HashSet newDimensions = new HashSet<>(valueRow.getDimensions()); newDimensions.addAll(headerKeyList.keySet()); // Remove the dummy timestamp added in KafkaInputFormat newDimensions.remove(KafkaInputFormat.DEFAULT_AUTO_TIMESTAMP_STRING); @@ -136,60 +187,70 @@ private CloseableIterator buildBlendedRows(InputEntityReader valuePars private CloseableIterator buildRowsWithoutValuePayload(Map headerKeyList) { - HashSet newDimensions = new HashSet(headerKeyList.keySet()); - InputRow row = new MapBasedInputRow( + final InputRow row = new MapBasedInputRow( inputRowSchema.getTimestampSpec().extractTimestamp(headerKeyList), - getFinalDimensionList(newDimensions), + getFinalDimensionList(headerKeyList.keySet()), headerKeyList ); - List rows = Collections.singletonList(row); + final List rows = Collections.singletonList(row); return CloseableIterators.withEmptyBaggage(rows.iterator()); } - @Override - public CloseableIterator read() throws IOException + /** + * Builds a map that blends two {@link Map}, presenting the combined keyset of both maps, and preferring to read + * from the first map and falling back to the second map if the value is not present. + * + * This strategy is used rather than just copying the values of the keyset into a new map so that any 'flattening' + * machinery (such as {@link Map} created by {@link org.apache.druid.java.util.common.parsers.ObjectFlatteners}) is + * still in place to be lazily evaluated instead of eagerly copying. + */ + private static Map buildBlendedEventMap(Map map, Map fallback) { - KafkaRecordEntity record = source.getEntity(); - Map mergeMap = new HashMap<>(); - if (headerParserSupplier != null) { - KafkaHeaderReader headerParser = headerParserSupplier.apply(record); - List> headerList = headerParser.read(); - for (Pair ele : headerList) { - mergeMap.put(ele.lhs, ele.rhs); - } - } - - // Add kafka record timestamp to the mergelist, we will skip record timestamp if the same key exists already in the header list - mergeMap.putIfAbsent(timestampColumnName, record.getRecord().timestamp()); + final Set keySet = new HashSet<>(fallback.keySet()); + keySet.addAll(map.keySet()); - InputEntityReader keyParser = (keyParserSupplier == null) ? null : keyParserSupplier.apply(record); - if (keyParser != null) { - try (CloseableIterator keyIterator = keyParser.read()) { - // Key currently only takes the first row and ignores the rest. - if (keyIterator.hasNext()) { - // Return type for the key parser should be of type MapBasedInputRow - // Parsers returning other types are not compatible currently. - MapBasedInputRow keyRow = (MapBasedInputRow) keyIterator.next(); - // Add the key to the mergeList only if the key string is not already present - mergeMap.putIfAbsent(keyColumnName, keyRow.getEvent().entrySet().stream().findFirst().get().getValue()); - } + return new AbstractMap() + { + @Override + public Object get(Object key) + { + return map.getOrDefault((String) key, fallback.get(key)); } - catch (ClassCastException e) { - throw new IOException("Unsupported input format in keyFormat. KafkaInputformat only supports input format that return MapBasedInputRow rows"); + + @Override + public Set keySet() + { + return keySet; } - } - // Ignore tombstone records that have null values. - if (record.getRecord().value() != null) { - return buildBlendedRows(valueParser, mergeMap); - } else { - return buildRowsWithoutValuePayload(mergeMap); - } - } + @Override + public Set> entrySet() + { + return keySet().stream() + .map( + field -> new Entry() + { + @Override + public String getKey() + { + return field; + } - @Override - public CloseableIterator sample() throws IOException - { - return read().map(row -> InputRowListPlusRawValues.of(row, ((MapBasedInputRow) row).getEvent())); + @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/kafka-indexing-service/src/test/java/org/apache/druid/data/input/kafkainput/KafkaInputFormatTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/data/input/kafkainput/KafkaInputFormatTest.java index fe0b89e996f8..a45730005a9c 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/data/input/kafkainput/KafkaInputFormatTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/data/input/kafkainput/KafkaInputFormatTest.java @@ -22,12 +22,12 @@ 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.data.input.ColumnsFilter; import org.apache.druid.data.input.InputEntityReader; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowSchema; -import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.TimestampSpec; @@ -205,6 +205,7 @@ public void testWithHeaderKeyAndValue() throws IOException 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")); // Header verification Assert.assertEquals("application/json", Iterables.getOnlyElement(row.getDimension("kafka.newheader.encoding"))); @@ -342,7 +343,6 @@ public byte[] value() while (iterator.hasNext()) { final InputRow row = iterator.next(); - final MapBasedInputRow mrow = (MapBasedInputRow) row; // Payload verifications Assert.assertEquals(DateTimes.of("2021-06-24"), row.getTimestamp()); Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo"))); @@ -350,6 +350,7 @@ public byte[] value() 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")); // Header verification Assert.assertEquals("application/json", Iterables.getOnlyElement(row.getDimension("kafka.newheader.encoding"))); @@ -444,6 +445,7 @@ public void testWithOutKeyAndHeaderSpecs() throws IOException 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")); numActualIterations++; } @@ -521,6 +523,7 @@ public void testWithMultipleMixedRecords() throws IOException 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"))); From 13b9dfde7e52536b1fe1bc76458d9668f80e650e Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Wed, 30 Nov 2022 14:34:33 +0530 Subject: [PATCH 14/34] ServiceClient: More robust redirect handling. (#13413) (#13445) Detects self-redirects, redirect loops, long redirect chains, and redirects to unknown servers. Treat all of these cases as an unavailable service, retrying if the retry policy allows it. Previously, some of these cases would lead to a prompt, unretryable error. This caused clients contacting an Overlord during a leader change to fail with error messages like: org.apache.druid.rpc.RpcException: Service [overlord] redirected too many times Additionally, a slight refactor of callbacks in ServiceClientImpl improves readability of the flow through onSuccess. Co-authored-by: Gian Merlino --- .../org/apache/druid/rpc/ServiceClient.java | 6 +- .../apache/druid/rpc/ServiceClientImpl.java | 237 ++++++++++++------ .../rpc/ServiceNotAvailableException.java | 9 +- .../druid/rpc/ServiceClientImplTest.java | 102 +++++++- 4 files changed, 272 insertions(+), 82 deletions(-) diff --git a/server/src/main/java/org/apache/druid/rpc/ServiceClient.java b/server/src/main/java/org/apache/druid/rpc/ServiceClient.java index cb33f713d988..1c416f88a1dc 100644 --- a/server/src/main/java/org/apache/druid/rpc/ServiceClient.java +++ b/server/src/main/java/org/apache/druid/rpc/ServiceClient.java @@ -49,8 +49,10 @@ public interface ServiceClient * encountered error. * * Redirects from 3xx responses are followed up to a chain length of {@link #MAX_REDIRECTS} and do not consume - * attempts. Redirects are validated against the targets returned by {@link ServiceLocator}: the client will not - * follow a redirect to a target that does not appear in the returned {@link ServiceLocations}. + * attempts. Redirects are validated against the targets returned by {@link ServiceLocator}: the client will only + * follow redirects to targets that appear in {@link ServiceLocations}. If the client encounters a redirect to an + * unknown target, or if a redirect loop or self-redirect is detected, it is treated as an unavailable service and + * an attempt is consumed. * * If the service is unavailable at the time an attempt is made, the client will automatically retry based on * {@link ServiceRetryPolicy#retryNotAvailable()}. If true, an attempt is consumed and the client will try to locate diff --git a/server/src/main/java/org/apache/druid/rpc/ServiceClientImpl.java b/server/src/main/java/org/apache/druid/rpc/ServiceClientImpl.java index 1445b943e96e..eca2cfdc5a2e 100644 --- a/server/src/main/java/org/apache/druid/rpc/ServiceClientImpl.java +++ b/server/src/main/java/org/apache/druid/rpc/ServiceClientImpl.java @@ -21,6 +21,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; @@ -88,7 +89,7 @@ public ListenableFuture asyncRequest( ) { final SettableFuture retVal = SettableFuture.create(); - tryRequest(requestBuilder, handler, retVal, 0, 0); + tryRequest(requestBuilder, handler, retVal, 0, ImmutableSet.of()); return retVal; } @@ -98,21 +99,35 @@ public ServiceClientImpl withRetryPolicy(ServiceRetryPolicy newRetryPolicy) return new ServiceClientImpl(serviceName, httpClient, serviceLocator, newRetryPolicy, connectExec); } + /** + * Internal helper used by {@link #asyncRequest(RequestBuilder, HttpResponseHandler)}. + * + * Handles retries by calling itself back in {@link #connectExec} with an incremented {@code attemptNumber}. + * + * @param requestBuilder request builder from call to {@link #asyncRequest} + * @param handler handler from call to {@link #asyncRequest} + * @param retVal return future generated by {@link #asyncRequest} + * @param attemptNumber attempt number; starts at 0 and is incremented on each retry + * @param redirectLocations redirect locations observed from the server on this attempt; used for detecting redirect + * loops and for limiting redirect chain length to {@link #MAX_REDIRECTS}. Cleared when + * a new attempt is issued. + */ private void tryRequest( final RequestBuilder requestBuilder, final HttpResponseHandler handler, final SettableFuture retVal, final long attemptNumber, - final int redirectCount + final ImmutableSet redirectLocations ) { whenServiceReady( - serviceLocation -> { + serviceLocations -> { if (retVal.isCancelled()) { // Return early if the caller canceled the return future. return; } + final ServiceLocation serviceLocation = pick(serviceLocations); final long nextAttemptNumber = attemptNumber + 1; if (serviceLocation == null) { @@ -128,7 +143,7 @@ private void tryRequest( ); connectExec.schedule( - () -> tryRequest(requestBuilder, handler, retVal, attemptNumber + 1, redirectCount), + () -> tryRequest(requestBuilder, handler, retVal, nextAttemptNumber, ImmutableSet.of()), backoffMs, TimeUnit.MILLISECONDS ); @@ -172,71 +187,15 @@ public void onSuccess(@Nullable final Either 1) { - // There were retries. Log at INFO level to provide the user some closure. - log.info( - "Service [%s] request [%s %s] completed.", - serviceName, - request.getMethod(), - request.getUrl() - ); - } else { - // No retries. Log at debug level to avoid cluttering the logs. - log.debug( - "Service [%s] request [%s %s] completed.", - serviceName, - request.getMethod(), - request.getUrl() - ); - } - - // Will not throw, because we checked result.isValue() earlier. - retVal.set(result.valueOrThrow()); + handleResultValue(result.valueOrThrow()); } else { final StringFullResponseHolder errorHolder = result != null ? result.error() : null; if (errorHolder != null && isRedirect(errorHolder.getResponse().getStatus())) { - // Redirect. Update preferredLocationNoPath if appropriate, then reissue. - final String newUri = result.error().getResponse().headers().get("Location"); - - if (redirectCount >= MAX_REDIRECTS) { - retVal.setException(new RpcException( - "Service [%s] redirected too many times [%d] to invalid url %s", - serviceName, - redirectCount, - newUri - )); - } else { - // Update preferredLocationNoPath if we got a redirect. - final ServiceLocation redirectLocationNoPath = serviceLocationNoPathFromUri(newUri); - - if (redirectLocationNoPath != null) { - preferredLocationNoPath.set(redirectLocationNoPath); - connectExec.submit( - () -> tryRequest(requestBuilder, handler, retVal, attemptNumber, redirectCount + 1) - ); - } else { - retVal.setException( - new RpcException( - "Service [%s] redirected [%d] times to invalid URL [%s]", - serviceName, - redirectCount, - newUri - ) - ); - } - } + handleRedirect(errorHolder); } else if (shouldTry(nextAttemptNumber) && (errorHolder == null || retryPolicy.retryHttpResponse(errorHolder.getResponse()))) { - // Retryable server response (or null errorHolder, which means null result, which can happen - // if the HttpClient encounters an exception in the midst of response processing). - final long backoffMs = computeBackoffMs(retryPolicy, attemptNumber); - log.noStackTrace().info(buildErrorMessage(request, errorHolder, backoffMs, nextAttemptNumber)); - connectExec.schedule( - () -> tryRequest(requestBuilder, handler, retVal, attemptNumber + 1, redirectCount), - backoffMs, - TimeUnit.MILLISECONDS - ); + handleRetryableErrorResponse(errorHolder); } else if (errorHolder != null) { // Nonretryable server response. retVal.setException(new HttpResponseException(errorHolder)); @@ -264,7 +223,7 @@ public void onFailure(final Throwable t) log.noStackTrace().info(t, buildErrorMessage(request, null, backoffMs, nextAttemptNumber)); connectExec.schedule( - () -> tryRequest(requestBuilder, handler, retVal, attemptNumber + 1, redirectCount), + () -> tryRequest(requestBuilder, handler, retVal, nextAttemptNumber, ImmutableSet.of()), backoffMs, TimeUnit.MILLISECONDS ); @@ -277,6 +236,135 @@ public void onFailure(final Throwable t) retVal.setException(new RpcException(t, "Service [%s] handler exited unexpectedly", serviceName)); } } + + /** + * Handles HTTP 2xx responses from the server. + */ + private void handleResultValue(final FinalType value) + { + if (nextAttemptNumber > 1) { + // There were retries. Log at INFO level to provide the user some closure. + log.info( + "Service [%s] request [%s %s] completed.", + serviceName, + request.getMethod(), + request.getUrl() + ); + } else { + // No retries. Log at debug level to avoid cluttering the logs. + log.debug( + "Service [%s] request [%s %s] completed.", + serviceName, + request.getMethod(), + request.getUrl() + ); + } + + // Will not throw, because we checked result.isValue() earlier. + retVal.set(value); + } + + /** + * Handles retryable HTTP error responses from the server. + */ + private void handleRetryableErrorResponse(final StringFullResponseHolder errorHolder) + { + // Retryable server response (or null errorHolder, which means null result, which can happen + // if the HttpClient encounters an exception in the midst of response processing). + final long backoffMs = computeBackoffMs(retryPolicy, attemptNumber); + log.info(buildErrorMessage(request, errorHolder, backoffMs, nextAttemptNumber)); + connectExec.schedule( + () -> tryRequest(requestBuilder, handler, retVal, nextAttemptNumber, ImmutableSet.of()), + backoffMs, + TimeUnit.MILLISECONDS + ); + } + + /** + * Handles HTTP redirect responses from the server. + */ + private void handleRedirect(final StringFullResponseHolder errorHolder) + { + // Redirect. Update preferredLocationNoPath if appropriate, then reissue. + final String newUri = errorHolder.getResponse().headers().get("Location"); + final ServiceLocation redirectLocationNoPath = serviceLocationNoPathFromUri(newUri); + + if (redirectLocationNoPath == null) { + // Redirect to invalid URL. Something is wrong with the server: fail immediately + // without retries. + retVal.setException( + new RpcException( + "Service [%s] redirected to invalid URL [%s]", + serviceName, + newUri + ) + ); + } else if (serviceLocations.getLocations() + .stream() + .anyMatch(loc -> serviceLocationNoPath(loc) + .equals(redirectLocationNoPath))) { + // Valid redirect, to a server that is one of the known locations. + final boolean isRedirectLoop = redirectLocations.contains(newUri); + final boolean isRedirectChainTooLong = redirectLocations.size() >= MAX_REDIRECTS; + + if (isRedirectLoop || isRedirectChainTooLong) { + // Treat redirect loops, or too-long redirect chains, as unavailable services. + if (retryPolicy.retryNotAvailable() && shouldTry(nextAttemptNumber)) { + final long backoffMs = computeBackoffMs(retryPolicy, attemptNumber); + + log.info( + "Service [%s] issued too many redirects on attempt #%d; retrying in %,d ms.", + serviceName, + nextAttemptNumber, + backoffMs + ); + + connectExec.schedule( + () -> tryRequest(requestBuilder, handler, retVal, nextAttemptNumber, ImmutableSet.of()), + backoffMs, + TimeUnit.MILLISECONDS + ); + } else { + retVal.setException(new ServiceNotAvailableException(serviceName, "issued too many redirects")); + } + } else { + // Valid redirect. Follow it without incrementing the attempt number. + preferredLocationNoPath.set(redirectLocationNoPath); + final ImmutableSet newRedirectLocations = + ImmutableSet.builder().addAll(redirectLocations).add(newUri).build(); + connectExec.submit( + () -> tryRequest(requestBuilder, handler, retVal, attemptNumber, newRedirectLocations) + ); + } + } else { + // Redirect to a server that is not one of the known locations. Treat service as unavailable. + if (retryPolicy.retryNotAvailable() && shouldTry(nextAttemptNumber)) { + final long backoffMs = computeBackoffMs(retryPolicy, attemptNumber); + + log.info( + "Service [%s] issued redirect to unknown URL [%s] on attempt #%d; retrying in %,d ms.", + serviceName, + newUri, + nextAttemptNumber, + backoffMs + ); + + connectExec.schedule( + () -> tryRequest(requestBuilder, handler, retVal, nextAttemptNumber, ImmutableSet.of()), + backoffMs, + TimeUnit.MILLISECONDS + ); + } else { + retVal.setException( + new ServiceNotAvailableException( + serviceName, + "issued redirect to unknown URL [%s]", + newUri + ) + ); + } + } + } }, connectExec ); @@ -285,7 +373,7 @@ public void onFailure(final Throwable t) ); } - private void whenServiceReady(final Consumer callback, final SettableFuture retVal) + private void whenServiceReady(final Consumer callback, final SettableFuture retVal) { Futures.addCallback( serviceLocator.locate(), @@ -300,8 +388,7 @@ public void onSuccess(final ServiceLocations locations) } try { - final ServiceLocation location = pick(locations); - callback.accept(location); + callback.accept(locations); } catch (Throwable t) { // It's a bug if this happens. The purpose of this line is to help us debug what went wrong. @@ -328,10 +415,7 @@ private ServiceLocation pick(final ServiceLocations locations) if (preferred != null) { // Preferred location is set. Use it if it's one of known locations. for (final ServiceLocation location : locations.getLocations()) { - final ServiceLocation locationNoPath = - new ServiceLocation(location.getHost(), location.getPlaintextPort(), location.getTlsPort(), ""); - - if (locationNoPath.equals(preferred)) { + if (serviceLocationNoPath(location).equals(preferred)) { return location; } } @@ -392,6 +476,9 @@ static long computeBackoffMs(final ServiceRetryPolicy retryPolicy, final long at ); } + /** + * Returns a {@link ServiceLocation} without a path component, based on a URI. + */ @Nullable @VisibleForTesting static ServiceLocation serviceLocationNoPathFromUri(@Nullable final String uriString) @@ -423,6 +510,14 @@ static ServiceLocation serviceLocationNoPathFromUri(@Nullable final String uriSt } } + /** + * Returns a {@link ServiceLocation} without its path. + */ + static ServiceLocation serviceLocationNoPath(final ServiceLocation location) + { + return new ServiceLocation(location.getHost(), location.getPlaintextPort(), location.getTlsPort(), ""); + } + @VisibleForTesting static boolean isRedirect(final HttpResponseStatus responseStatus) { diff --git a/server/src/main/java/org/apache/druid/rpc/ServiceNotAvailableException.java b/server/src/main/java/org/apache/druid/rpc/ServiceNotAvailableException.java index 290fa2cf7ab7..843ac2dfa54f 100644 --- a/server/src/main/java/org/apache/druid/rpc/ServiceNotAvailableException.java +++ b/server/src/main/java/org/apache/druid/rpc/ServiceNotAvailableException.java @@ -19,13 +19,20 @@ package org.apache.druid.rpc; +import org.apache.druid.java.util.common.StringUtils; + /** * Returned by {@link ServiceClient#asyncRequest} when a request has failed because the service is not available. */ public class ServiceNotAvailableException extends RpcException { + public ServiceNotAvailableException(final String serviceName, final String reason, final Object... reasonArgs) + { + super("Service [%s] %s", serviceName, StringUtils.format(reason, reasonArgs)); + } + public ServiceNotAvailableException(final String serviceName) { - super("Service [%s] is not available", serviceName); + this(serviceName, "is not available"); } } diff --git a/server/src/test/java/org/apache/druid/rpc/ServiceClientImplTest.java b/server/src/test/java/org/apache/druid/rpc/ServiceClientImplTest.java index 48f922e6ab6b..dc8bba87d139 100644 --- a/server/src/test/java/org/apache/druid/rpc/ServiceClientImplTest.java +++ b/server/src/test/java/org/apache/druid/rpc/ServiceClientImplTest.java @@ -67,6 +67,9 @@ public class ServiceClientImplTest private static final String SERVICE_NAME = "test-service"; private static final ServiceLocation SERVER1 = new ServiceLocation("example.com", -1, 8888, "/q"); private static final ServiceLocation SERVER2 = new ServiceLocation("example.com", -1, 9999, "/q"); + private static final ServiceLocation SERVER3 = new ServiceLocation("example.com", -1, 1111, "/q"); + private static final ServiceLocation SERVER4 = new ServiceLocation("example.com", -1, 2222, "/q"); + private static final ServiceLocation SERVER5 = new ServiceLocation("example.com", -1, 3333, "/q"); private ScheduledExecutorService exec; @@ -270,7 +273,63 @@ public void test_request_followRedirect() throws Exception } @Test - public void test_request_tooManyRedirects() + public void test_request_tooLongRedirectChain() + { + final RequestBuilder requestBuilder = new RequestBuilder(HttpMethod.GET, "/foo"); + + // Redirect chain longer than max length. + stubLocatorCall(locations(SERVER1, SERVER2, SERVER3, SERVER4, SERVER5)); + expectHttpCall(requestBuilder, SERVER1) + .thenReturn(redirectResponse(requestBuilder.build(SERVER2).getUrl().toString())); + expectHttpCall(requestBuilder, SERVER2) + .thenReturn(redirectResponse(requestBuilder.build(SERVER3).getUrl().toString())); + expectHttpCall(requestBuilder, SERVER3) + .thenReturn(redirectResponse(requestBuilder.build(SERVER4).getUrl().toString())); + expectHttpCall(requestBuilder, SERVER4) + .thenReturn(redirectResponse(requestBuilder.build(SERVER5).getUrl().toString())); + + serviceClient = makeServiceClient(StandardRetryPolicy.noRetries()); + + final ExecutionException e = Assert.assertThrows( + ExecutionException.class, + () -> doRequest(serviceClient, requestBuilder) + ); + + MatcherAssert.assertThat(e.getCause(), CoreMatchers.instanceOf(ServiceNotAvailableException.class)); + MatcherAssert.assertThat( + e.getCause(), + ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString("issued too many redirects")) + ); + } + + @Test + public void test_request_tooLongRedirectChainRetry() throws Exception + { + final RequestBuilder requestBuilder = new RequestBuilder(HttpMethod.GET, "/foo"); + final ImmutableMap expectedResponseObject = ImmutableMap.of("foo", "bar"); + + // Redirect chain longer than max length. Can be followed across retries. + stubLocatorCall(locations(SERVER1, SERVER2, SERVER3, SERVER4, SERVER5)); + expectHttpCall(requestBuilder, SERVER1) + .thenReturn(redirectResponse(requestBuilder.build(SERVER2).getUrl().toString())); + expectHttpCall(requestBuilder, SERVER2) + .thenReturn(redirectResponse(requestBuilder.build(SERVER3).getUrl().toString())); + expectHttpCall(requestBuilder, SERVER3) + .thenReturn(redirectResponse(requestBuilder.build(SERVER4).getUrl().toString())); + expectHttpCall(requestBuilder, SERVER4) + .thenReturn(redirectResponse(requestBuilder.build(SERVER5).getUrl().toString())); + expectHttpCall(requestBuilder, SERVER5) + .thenReturn(valueResponse(expectedResponseObject)); + + serviceClient = makeServiceClient(StandardRetryPolicy.builder().maxAttempts(2).build()); + + final Map response = doRequest(serviceClient, requestBuilder); + + Assert.assertEquals(expectedResponseObject, response); + } + + @Test + public void test_request_selfRedirectLoop() { final RequestBuilder requestBuilder = new RequestBuilder(HttpMethod.GET, "/foo"); @@ -279,17 +338,43 @@ public void test_request_tooManyRedirects() expectHttpCall(requestBuilder, SERVER1) .thenReturn(redirectResponse(requestBuilder.build(SERVER1).getUrl().toString())); - serviceClient = makeServiceClient(StandardRetryPolicy.unlimited()); + serviceClient = makeServiceClient(StandardRetryPolicy.builder().maxAttempts(10).build()); final ExecutionException e = Assert.assertThrows( ExecutionException.class, () -> doRequest(serviceClient, requestBuilder) ); - MatcherAssert.assertThat(e.getCause(), CoreMatchers.instanceOf(RpcException.class)); + MatcherAssert.assertThat(e.getCause(), CoreMatchers.instanceOf(ServiceNotAvailableException.class)); MatcherAssert.assertThat( e.getCause(), - ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString("redirected too many times")) + ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString("issued too many redirects")) + ); + } + + @Test + public void test_request_twoServerRedirectLoop() + { + final RequestBuilder requestBuilder = new RequestBuilder(HttpMethod.GET, "/foo"); + + // Endless redirects between the same two servers. + stubLocatorCall(locations(SERVER1, SERVER2)); + expectHttpCall(requestBuilder, SERVER1) + .thenReturn(redirectResponse(requestBuilder.build(SERVER2).getUrl().toString())); + expectHttpCall(requestBuilder, SERVER2) + .thenReturn(redirectResponse(requestBuilder.build(SERVER1).getUrl().toString())); + + serviceClient = makeServiceClient(StandardRetryPolicy.builder().maxAttempts(10).build()); + + final ExecutionException e = Assert.assertThrows( + ExecutionException.class, + () -> doRequest(serviceClient, requestBuilder) + ); + + MatcherAssert.assertThat(e.getCause(), CoreMatchers.instanceOf(ServiceNotAvailableException.class)); + MatcherAssert.assertThat( + e.getCause(), + ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString("issued too many redirects")) ); } @@ -314,7 +399,7 @@ public void test_request_redirectInvalid() MatcherAssert.assertThat( e.getCause(), ThrowableMessageMatcher.hasMessage( - CoreMatchers.containsString("redirected [0] times to invalid URL [invalid-url]")) + CoreMatchers.containsString("redirected to invalid URL [invalid-url]")) ); } @@ -338,7 +423,7 @@ public void test_request_redirectNil() MatcherAssert.assertThat(e.getCause(), CoreMatchers.instanceOf(RpcException.class)); MatcherAssert.assertThat( e.getCause(), - ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString("redirected [0] times to invalid URL [null]")) + ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString("redirected to invalid URL [null]")) ); } @@ -359,10 +444,11 @@ public void test_request_dontFollowRedirectToUnknownServer() () -> doRequest(serviceClient, requestBuilder) ); - MatcherAssert.assertThat(e.getCause(), CoreMatchers.instanceOf(RpcException.class)); + MatcherAssert.assertThat(e.getCause(), CoreMatchers.instanceOf(ServiceNotAvailableException.class)); MatcherAssert.assertThat( e.getCause(), - ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString("redirected too many times")) + ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( + "issued redirect to unknown URL [https://example.com:9999/q/foo]")) ); } From 8bf4b68b1a2f84cb6737f6ee50a7fab99c0673aa Mon Sep 17 00:00:00 2001 From: Laksh Singla Date: Wed, 30 Nov 2022 15:38:35 +0530 Subject: [PATCH 15/34] Convert errors based on implicit type conversion in multi value arrays to parse exception in MSQ (#13366) (#13454) * initial commit * fix test * push the json changes * reduce the area of the try..catch * Trigger Build * review --- .../apache/druid/msq/exec/MSQSelectTest.java | 58 +++++++++++++++++++ .../unparseable-mv-string-array.json | 3 + .../frame/write/RowBasedFrameWriter.java | 10 +++- .../write/columnar/ColumnarFrameWriter.java | 12 +++- 4 files changed, 78 insertions(+), 5 deletions(-) create mode 100644 extensions-core/multi-stage-query/src/test/resources/unparseable-mv-string-array.json diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java index c83481757d8f..26d566b31e8c 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java @@ -31,6 +31,7 @@ import org.apache.druid.msq.indexing.ColumnMappings; import org.apache.druid.msq.indexing.MSQSpec; import org.apache.druid.msq.indexing.MSQTuningConfig; +import org.apache.druid.msq.indexing.error.CannotParseExternalDataFault; import org.apache.druid.msq.shuffle.DurableStorageUtils; import org.apache.druid.msq.test.MSQTestBase; import org.apache.druid.query.InlineDataSource; @@ -1253,6 +1254,63 @@ public void testGroupByOnFooWithDurableStoragePathAssertions() throws IOExceptio .write(ArgumentMatchers.endsWith("__success")); } + @Test + public void testMultiValueStringWithIncorrectType() throws IOException + { + final File toRead = getResourceAsTemporaryFile("/unparseable-mv-string-array.json"); + final String toReadAsJson = queryFramework().queryJsonMapper().writeValueAsString(toRead.getAbsolutePath()); + + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("language", ColumnType.STRING_ARRAY) + .build(); + + final GroupByQuery expectedQuery = + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions(dimensions(new DefaultDimensionSpec("__time", "d0", ColumnType.LONG))) + .build(); + + + testSelectQuery() + .setSql("WITH\n" + + "kttm_data AS (\n" + + "SELECT * FROM TABLE(\n" + + " EXTERN(\n" + + " '{ \"files\": [" + toReadAsJson + "],\"type\":\"local\"}',\n" + + " '{\"type\":\"json\"}',\n" + + " '[{\"name\":\"timestamp\",\"type\":\"string\"},{\"name\":\"agent_category\",\"type\":\"string\"},{\"name\":\"agent_type\",\"type\":\"string\"},{\"name\":\"browser\",\"type\":\"string\"},{\"name\":\"browser_version\",\"type\":\"string\"},{\"name\":\"city\",\"type\":\"string\"},{\"name\":\"continent\",\"type\":\"string\"},{\"name\":\"country\",\"type\":\"string\"},{\"name\":\"version\",\"type\":\"string\"},{\"name\":\"event_type\",\"type\":\"string\"},{\"name\":\"event_subtype\",\"type\":\"string\"},{\"name\":\"loaded_image\",\"type\":\"string\"},{\"name\":\"adblock_list\",\"type\":\"string\"},{\"name\":\"forwarded_for\",\"type\":\"string\"},{\"name\":\"language\",\"type\":\"string\"},{\"name\":\"number\",\"type\":\"long\"},{\"name\":\"os\",\"type\":\"string\"},{\"name\":\"path\",\"type\":\"string\"},{\"name\":\"platform\",\"type\":\"string\"},{\"name\":\"referrer\",\"type\":\"string\"},{\"name\":\"referrer_host\",\"type\":\"string\"},{\"name\":\"region\",\"type\":\"string\"},{\"name\":\"remote_address\",\"type\":\"string\"},{\"name\":\"screen\",\"type\":\"string\"},{\"name\":\"session\",\"type\":\"string\"},{\"name\":\"session_length\",\"type\":\"long\"},{\"name\":\"timezone\",\"type\":\"string\"},{\"name\":\"timezone_offset\",\"type\":\"long\"},{\"name\":\"window\",\"type\":\"string\"}]'\n" + + " )\n" + + "))\n" + + "\n" + + "SELECT\n" + + " FLOOR(TIME_PARSE(\"timestamp\") TO MINUTE) AS __time,\n" + + " MV_TO_ARRAY(\"language\") AS \"language\"\n" + + "FROM kttm_data") + .setExpectedRowSignature(rowSignature) + .setExpectedResultRows(ImmutableList.of( + new Object[]{1566691200000L, ImmutableList.of("en")}, + new Object[]{1566691200000L, ImmutableList.of("en", "es", "es-419", "es-MX")}, + new Object[]{1566691200000L, ImmutableList.of("en", "es", "es-419", "es-US")} + )) + .setExpectedMSQSpec( + MSQSpec + .builder() + .query(expectedQuery) + .columnMappings(new ColumnMappings( + ImmutableList.of( + new ColumnMapping("d0", "__time"), + new ColumnMapping("a0", "cnt") + ) + )) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .build()) + .setExpectedMSQFault(new CannotParseExternalDataFault( + "Unable to add the row to the frame. Type conversion might be required.")) + .verifyResults(); + } @Nonnull private List expectedMultiValueFooRowsGroup() diff --git a/extensions-core/multi-stage-query/src/test/resources/unparseable-mv-string-array.json b/extensions-core/multi-stage-query/src/test/resources/unparseable-mv-string-array.json new file mode 100644 index 000000000000..57b9a7709b09 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/resources/unparseable-mv-string-array.json @@ -0,0 +1,3 @@ +{"timestamp":"2019-08-25T00:00:00.031Z","agent_category":"Personal computer","agent_type":"Browser","browser":"Chrome","browser_version":"76.0.3809.100","city":"Rosario","continent":"South America","country":"Argentina","version":"1.9.6","event_type":"PercentClear","event_subtype":"55","loaded_image":"http://www.koalastothemax.com/img/koalas2.jpg","adblock_list":"NoAdblock","forwarded_for":"181.13.41.82","language":[{},{}],"number":"16","os":"Windows 7","path":"http://www.koalastothemax.com/","platform":"Windows","referrer":"Direct","referrer_host":"Direct","region":"Santa Fe","remote_address":"172.31.57.89","screen":"1680x1050","session":"S56194838","session_length":76261,"timezone":"N/A","timezone_offset":"180","window":"1680x939"} +{"timestamp":"2019-08-25T00:00:00.059Z","agent_category":"Smartphone","agent_type":"Mobile Browser","browser":"Chrome Mobile","browser_version":"50.0.2661.89","city":"Nuevo Casas Grandes","continent":"North America","country":"Mexico","version":"1.9.6","event_type":"PercentClear","event_subtype":"85","loaded_image":"https://koalastothemax.com/img/koalas1.jpg","adblock_list":"NoAdblock","forwarded_for":"177.242.100.0","language":["en","es","es-419","es-MX"],"number":"24","os":"Android","path":"https://koalastothemax.com/","platform":"Android","referrer":"https://www.google.com/","referrer_host":"www.google.com","region":"Chihuahua","remote_address":"172.31.11.5","screen":"320x570","session":"S46093731","session_length":252689,"timezone":"CDT","timezone_offset":"300","window":"540x743"} +{"timestamp":"2019-08-25T00:00:00.178Z","agent_category":"Personal computer","agent_type":"Browser","browser":"Chrome","browser_version":"76.0.3809.100","city":"Luis Guillon","continent":"South America","country":"Argentina","version":"1.9.6","event_type":"PercentClear","event_subtype":"90","loaded_image":"http://www.koalastothemax.com/img/koalas.jpg","adblock_list":"NoAdblock","forwarded_for":"181.46.136.44","language":["en","es","es-419","es-US"],"number":"24","os":"Windows 7","path":"http://www.koalastothemax.com/","platform":"Windows","referrer":"Direct","referrer_host":"Direct","region":"Buenos Aires","remote_address":"172.31.11.5","screen":"1366x768","session":"S13352079","session_length":1753602,"timezone":"N/A","timezone_offset":"180","window":"1366x652"} diff --git a/processing/src/main/java/org/apache/druid/frame/write/RowBasedFrameWriter.java b/processing/src/main/java/org/apache/druid/frame/write/RowBasedFrameWriter.java index 2bf4b343080c..2a8af4b9e8f8 100644 --- a/processing/src/main/java/org/apache/druid/frame/write/RowBasedFrameWriter.java +++ b/processing/src/main/java/org/apache/druid/frame/write/RowBasedFrameWriter.java @@ -32,6 +32,7 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.utils.CloseableUtils; @@ -118,8 +119,13 @@ public boolean addSelection() return false; } - if (!writeData()) { - return false; + try { + if (!writeData()) { + return false; + } + } + catch (Exception e) { + throw new ParseException("", e, "Unable to add the row to the frame. Type conversion might be required."); } final MemoryRange rowOffsetCursor = rowOffsetMemory.cursor(); diff --git a/processing/src/main/java/org/apache/druid/frame/write/columnar/ColumnarFrameWriter.java b/processing/src/main/java/org/apache/druid/frame/write/columnar/ColumnarFrameWriter.java index 76f5a17d27af..eff5213ae100 100644 --- a/processing/src/main/java/org/apache/druid/frame/write/columnar/ColumnarFrameWriter.java +++ b/processing/src/main/java/org/apache/druid/frame/write/columnar/ColumnarFrameWriter.java @@ -30,6 +30,7 @@ import org.apache.druid.frame.write.FrameWriter; import org.apache.druid.frame.write.FrameWriterUtils; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.segment.column.RowSignature; import javax.annotation.Nullable; @@ -74,11 +75,16 @@ public boolean addSelection() } int i = 0; - for (; i < columnWriters.size(); i++) { - if (!columnWriters.get(i).addSelection()) { - break; + try { + for (; i < columnWriters.size(); i++) { + if (!columnWriters.get(i).addSelection()) { + break; + } } } + catch (Exception e) { + throw new ParseException("", e, "Unable to add the row to the frame. Type conversion might be required."); + } if (i < columnWriters.size()) { // Add failed, clean up. From ff3c83fde0feaf27750b7ef2bb74547aafb7bc3d Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Thu, 1 Dec 2022 09:53:06 +0530 Subject: [PATCH 16/34] Fix an issue with WorkerSketchFetcher not terminating on shutdown (#13459) (#13464) * Fix an issue with WorkerSketchFetcher not terminating on shutdown * Change threadpool name --- .../org/apache/druid/msq/exec/ControllerImpl.java | 5 +++-- .../apache/druid/msq/exec/WorkerSketchFetcher.java | 12 +++++++++--- 2 files changed, 12 insertions(+), 5 deletions(-) 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 cafc0f389258..318c33a759c7 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 @@ -524,6 +524,8 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer) context.registerController(this, closer); this.netClient = new ExceptionWrappingWorkerClient(context.taskClientFor(this)); + closer.register(netClient::close); + ClusterStatisticsMergeMode clusterStatisticsMergeMode = MultiStageQueryContext.getClusterStatisticsMergeMode(task.getQuerySpec().getQuery().context()); @@ -532,8 +534,7 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer) int statisticsMaxRetainedBytes = WorkerMemoryParameters.createProductionInstanceForController(context.injector()) .getPartitionStatisticsMaxRetainedBytes(); this.workerSketchFetcher = new WorkerSketchFetcher(netClient, clusterStatisticsMergeMode, statisticsMaxRetainedBytes); - - closer.register(netClient::close); + closer.register(workerSketchFetcher::close); final boolean isDurableStorageEnabled = MultiStageQueryContext.isDurableStorageEnabled(task.getQuerySpec().getQuery().context()); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java index 3482b50daaff..c4118a9d38e0 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java @@ -25,6 +25,7 @@ import org.apache.druid.frame.key.ClusterByPartitions; import org.apache.druid.java.util.common.Either; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.kernel.StageDefinition; import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; @@ -39,13 +40,12 @@ import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; import java.util.stream.IntStream; /** * Queues up fetching sketches from workers and progressively generates partitions boundaries. */ -public class WorkerSketchFetcher +public class WorkerSketchFetcher implements AutoCloseable { private static final Logger log = new Logger(WorkerSketchFetcher.class); private static final int DEFAULT_THREAD_COUNT = 4; @@ -63,7 +63,7 @@ public WorkerSketchFetcher(WorkerClient workerClient, ClusterStatisticsMergeMode { this.workerClient = workerClient; this.clusterStatisticsMergeMode = clusterStatisticsMergeMode; - this.executorService = Executors.newFixedThreadPool(DEFAULT_THREAD_COUNT); + this.executorService = Execs.multiThreaded(DEFAULT_THREAD_COUNT, "SketchFetcherThreadPool-%d"); this.statisticsMaxRetainedBytes = statisticsMaxRetainedBytes; } @@ -337,4 +337,10 @@ private static long getPartitionCountFromEither(Either Date: Wed, 30 Nov 2022 23:05:04 -0800 Subject: [PATCH 17/34] Web console: backport fixes to 25.0 (#13449) * add ability to make inputFormat part of the example datasets (#13402) * Web console: Index spec dialog (#13425) * add index spec dialog * add sanpshot * Web console: be more robust to aux queries failing and improve kill tasks (#13431) * be more robust to aux queries failing * feedback fixes * remove empty block * fix spelling * remove killAllDataSources from the console * don't render duration if aggregated (#13455) --- .../src/components/auto-form/auto-form.scss | 12 +- .../src/components/auto-form/auto-form.tsx | 55 ++- .../form-group-with-info.scss | 14 + .../table-clickable-cell.scss | 4 + .../table-clickable-cell.tsx | 11 +- .../warning-checklist/warning-checklist.tsx | 18 +- .../async-action-dialog.tsx | 2 +- .../index-spec-dialog.spec.tsx.snap | 317 ++++++++++++++ .../index-spec-dialog/index-spec-dialog.scss | 36 ++ .../index-spec-dialog.spec.tsx | 44 ++ .../index-spec-dialog/index-spec-dialog.tsx | 88 ++++ web-console/src/dialogs/index.ts | 1 + .../kill-datasource-dialog.tsx | 110 +++++ .../compaction-status.spec.ts | 41 +- .../compaction-status/compaction-status.ts | 22 +- .../coordinator-dynamic-config.tsx | 13 +- .../druid-models/index-spec/index-spec.tsx | 158 +++++++ web-console/src/druid-models/index.ts | 1 + .../ingestion-spec/ingestion-spec.tsx | 120 ++---- .../workbench-query/workbench-query-part.ts | 4 +- .../workbench-query/workbench-query.spec.ts | 2 +- .../src/helpers/spec-conversion.spec.ts | 6 + web-console/src/helpers/spec-conversion.ts | 5 + .../datasources-view/datasources-view.tsx | 390 ++++++++++-------- .../views/ingestion-view/ingestion-view.tsx | 3 +- .../src/views/services-view/services-view.tsx | 215 ++++++---- .../input-source-step/example-inputs.ts | 67 ++- .../input-source-step/input-source-step.tsx | 36 +- .../workbench-view/run-panel/run-panel.tsx | 130 +++--- 29 files changed, 1408 insertions(+), 517 deletions(-) create mode 100644 web-console/src/dialogs/index-spec-dialog/__snapshots__/index-spec-dialog.spec.tsx.snap create mode 100644 web-console/src/dialogs/index-spec-dialog/index-spec-dialog.scss create mode 100644 web-console/src/dialogs/index-spec-dialog/index-spec-dialog.spec.tsx create mode 100644 web-console/src/dialogs/index-spec-dialog/index-spec-dialog.tsx create mode 100644 web-console/src/dialogs/kill-datasource-dialog/kill-datasource-dialog.tsx create mode 100644 web-console/src/druid-models/index-spec/index-spec.tsx diff --git a/web-console/src/components/auto-form/auto-form.scss b/web-console/src/components/auto-form/auto-form.scss index 5523f0f8173f..c303abc294f5 100644 --- a/web-console/src/components/auto-form/auto-form.scss +++ b/web-console/src/components/auto-form/auto-form.scss @@ -16,16 +16,8 @@ * limitations under the License. */ -@import '../../variables'; - .auto-form { - // Popover in info label - label.#{$bp-ns}-label { - position: relative; - - .#{$bp-ns}-text-muted { - position: absolute; - right: 0; - } + .custom-input input { + cursor: pointer; } } diff --git a/web-console/src/components/auto-form/auto-form.tsx b/web-console/src/components/auto-form/auto-form.tsx index 1e56ef2b725d..146de61b6272 100644 --- a/web-console/src/components/auto-form/auto-form.tsx +++ b/web-console/src/components/auto-form/auto-form.tsx @@ -16,7 +16,14 @@ * limitations under the License. */ -import { Button, ButtonGroup, FormGroup, Intent, NumericInput } from '@blueprintjs/core'; +import { + Button, + ButtonGroup, + FormGroup, + InputGroup, + Intent, + NumericInput, +} from '@blueprintjs/core'; import { IconNames } from '@blueprintjs/icons'; import React from 'react'; @@ -46,7 +53,8 @@ export interface Field { | 'boolean' | 'string-array' | 'json' - | 'interval'; + | 'interval' + | 'custom'; defaultValue?: any; emptyValue?: any; suggestions?: Functor; @@ -64,6 +72,13 @@ export interface Field { valueAdjustment?: (value: any) => any; adjustment?: (model: Partial) => Partial; issueWithValue?: (value: any) => string | undefined; + + customSummary?: (v: any) => string; + customDialog?: (o: { + value: any; + onValueChange: (v: any) => void; + onClose: () => void; + }) => JSX.Element; } interface ComputedFieldValues { @@ -84,6 +99,7 @@ export interface AutoFormProps { export interface AutoFormState { showMore: boolean; + customDialog?: JSX.Element; } export class AutoForm> extends React.PureComponent< @@ -395,6 +411,36 @@ export class AutoForm> extends React.PureComponent ); } + private renderCustomInput(field: Field): JSX.Element { + const { model } = this.props; + const { required, defaultValue, modelValue } = AutoForm.computeFieldValues(model, field); + const effectiveValue = modelValue || defaultValue; + + const onEdit = () => { + this.setState({ + customDialog: field.customDialog?.({ + value: effectiveValue, + onValueChange: v => this.fieldChange(field, v), + onClose: () => { + this.setState({ customDialog: undefined }); + }, + }), + }); + }; + + return ( + } + onClick={onEdit} + /> + ); + } + renderFieldInput(field: Field) { switch (field.type) { case 'number': @@ -413,6 +459,8 @@ export class AutoForm> extends React.PureComponent return this.renderJsonInput(field); case 'interval': return this.renderIntervalInput(field); + case 'custom': + return this.renderCustomInput(field); default: throw new Error(`unknown field type '${field.type}'`); } @@ -464,7 +512,7 @@ export class AutoForm> extends React.PureComponent render(): JSX.Element { const { fields, model, showCustom } = this.props; - const { showMore } = this.state; + const { showMore, customDialog } = this.state; let shouldShowMore = false; const shownFields = fields.filter(field => { @@ -489,6 +537,7 @@ export class AutoForm> extends React.PureComponent {model && shownFields.map(this.renderField)} {model && showCustom && showCustom(model) && this.renderCustom()} {shouldShowMore && this.renderMoreOrLess()} + {customDialog} ); } diff --git a/web-console/src/components/form-group-with-info/form-group-with-info.scss b/web-console/src/components/form-group-with-info/form-group-with-info.scss index c9587cb088fa..a64c6d29274c 100644 --- a/web-console/src/components/form-group-with-info/form-group-with-info.scss +++ b/web-console/src/components/form-group-with-info/form-group-with-info.scss @@ -19,6 +19,20 @@ @import '../../variables'; .form-group-with-info { + label.#{$bp-ns}-label { + position: relative; + + .#{$bp-ns}-text-muted { + position: absolute; + right: 0; + + // This is only needed because BP4 alerts are too agro in setting CSS on icons + .#{$bp-ns}-icon { + margin-right: 0; + } + } + } + .#{$bp-ns}-text-muted .#{$bp-ns}-popover2-target { margin-top: 0; } diff --git a/web-console/src/components/table-clickable-cell/table-clickable-cell.scss b/web-console/src/components/table-clickable-cell/table-clickable-cell.scss index d6f6f8b2d7f7..5c5991df54e2 100644 --- a/web-console/src/components/table-clickable-cell/table-clickable-cell.scss +++ b/web-console/src/components/table-clickable-cell/table-clickable-cell.scss @@ -24,6 +24,10 @@ overflow: hidden; text-overflow: ellipsis; + &.disabled { + cursor: not-allowed; + } + .hover-icon { position: absolute; top: $table-cell-v-padding; diff --git a/web-console/src/components/table-clickable-cell/table-clickable-cell.tsx b/web-console/src/components/table-clickable-cell/table-clickable-cell.tsx index cc8cfd71e56d..7e4c66fdd5e5 100644 --- a/web-console/src/components/table-clickable-cell/table-clickable-cell.tsx +++ b/web-console/src/components/table-clickable-cell/table-clickable-cell.tsx @@ -27,18 +27,23 @@ export interface TableClickableCellProps { onClick: MouseEventHandler; hoverIcon?: IconName; title?: string; + disabled?: boolean; children?: ReactNode; } export const TableClickableCell = React.memo(function TableClickableCell( props: TableClickableCellProps, ) { - const { className, onClick, hoverIcon, title, children } = props; + const { className, onClick, hoverIcon, title, disabled, children } = props; return ( -

+
{children} - {hoverIcon && } + {hoverIcon && !disabled && }
); }); diff --git a/web-console/src/components/warning-checklist/warning-checklist.tsx b/web-console/src/components/warning-checklist/warning-checklist.tsx index 449ad970045d..5c74cbdb08a0 100644 --- a/web-console/src/components/warning-checklist/warning-checklist.tsx +++ b/web-console/src/components/warning-checklist/warning-checklist.tsx @@ -17,29 +17,31 @@ */ import { Switch } from '@blueprintjs/core'; -import React, { useState } from 'react'; +import React, { ReactNode, useState } from 'react'; export interface WarningChecklistProps { - checks: string[]; - onChange: (allChecked: boolean) => void; + checks: ReactNode[]; + onChange(allChecked: boolean): void; } export const WarningChecklist = React.memo(function WarningChecklist(props: WarningChecklistProps) { const { checks, onChange } = props; - const [checked, setChecked] = useState>({}); + const [checked, setChecked] = useState>({}); - function doCheck(check: string) { + function doCheck(checkIndex: number) { const newChecked = { ...checked }; - newChecked[check] = !newChecked[check]; + newChecked[checkIndex] = !newChecked[checkIndex]; setChecked(newChecked); - onChange(checks.every(check => newChecked[check])); + onChange(checks.every((_, i) => newChecked[i])); } return (
{checks.map((check, i) => ( - doCheck(check)} /> + doCheck(i)}> + {check} + ))}
); diff --git a/web-console/src/dialogs/async-action-dialog/async-action-dialog.tsx b/web-console/src/dialogs/async-action-dialog/async-action-dialog.tsx index f892936bab77..0d8cf385a5b6 100644 --- a/web-console/src/dialogs/async-action-dialog/async-action-dialog.tsx +++ b/web-console/src/dialogs/async-action-dialog/async-action-dialog.tsx @@ -47,7 +47,7 @@ export interface AsyncActionDialogProps { intent?: Intent; successText: string; failText: string; - warningChecks?: string[]; + warningChecks?: ReactNode[]; children?: ReactNode; } diff --git a/web-console/src/dialogs/index-spec-dialog/__snapshots__/index-spec-dialog.spec.tsx.snap b/web-console/src/dialogs/index-spec-dialog/__snapshots__/index-spec-dialog.spec.tsx.snap new file mode 100644 index 000000000000..57d989621b70 --- /dev/null +++ b/web-console/src/dialogs/index-spec-dialog/__snapshots__/index-spec-dialog.spec.tsx.snap @@ -0,0 +1,317 @@ +// Jest Snapshot v1, https://goo.gl/fbAQLP + +exports[`IndexSpecDialog matches snapshot with indexSpec 1`] = ` + + +
+ + Encoding format for STRING value dictionaries used by STRING and COMPLEX<json> columns. + , + "label": "String dictionary encoding", + "name": "stringDictionaryEncoding.type", + "suggestions": Array [ + "utf8", + "frontCoded", + ], + "type": "string", + }, + Object { + "defaultValue": 4, + "defined": [Function], + "info": + The number of values to place in a bucket to perform delta encoding. Must be a power of 2, maximum is 128. + , + "label": "String dictionary encoding bucket size", + "max": 128, + "min": 1, + "name": "stringDictionaryEncoding.bucketSize", + "type": "number", + }, + Object { + "defaultValue": "roaring", + "info": + Compression format for bitmap indexes. + , + "label": "Bitmap type", + "name": "bitmap.type", + "suggestions": Array [ + "roaring", + "concise", + ], + "type": "string", + }, + Object { + "defaultValue": true, + "defined": [Function], + "info": + Controls whether or not run-length encoding will be used when it is determined to be more space-efficient. + , + "label": "Bitmap compress run on serialization", + "name": "bitmap.compressRunOnSerialization", + "type": "boolean", + }, + Object { + "defaultValue": "lz4", + "info": + Compression format for dimension columns. + , + "name": "dimensionCompression", + "suggestions": Array [ + "lz4", + "lzf", + "zstd", + "uncompressed", + ], + "type": "string", + }, + Object { + "defaultValue": "longs", + "info": + Encoding format for long-typed columns. Applies regardless of whether they are dimensions or metrics. + + auto + + encodes the values using offset or lookup table depending on column cardinality, and store them with variable size. + + longs + + stores the value as-is with 8 bytes each. + , + "name": "longEncoding", + "suggestions": Array [ + "longs", + "auto", + ], + "type": "string", + }, + Object { + "defaultValue": "lz4", + "info": + Compression format for primitive type metric columns. + , + "name": "metricCompression", + "suggestions": Array [ + "lz4", + "lzf", + "zstd", + "uncompressed", + ], + "type": "string", + }, + Object { + "defaultValue": "lz4", + "info": + Compression format to use for nested column raw data. + , + "label": "JSON compression", + "name": "jsonCompression", + "suggestions": Array [ + "lz4", + "lzf", + "zstd", + "uncompressed", + ], + "type": "string", + }, + ] + } + model={ + Object { + "dimensionCompression": "lzf", + } + } + onChange={[Function]} + /> +
+
+
+ + +
+
+
+`; + +exports[`IndexSpecDialog matches snapshot without compactionConfig 1`] = ` + + +
+ + Encoding format for STRING value dictionaries used by STRING and COMPLEX<json> columns. + , + "label": "String dictionary encoding", + "name": "stringDictionaryEncoding.type", + "suggestions": Array [ + "utf8", + "frontCoded", + ], + "type": "string", + }, + Object { + "defaultValue": 4, + "defined": [Function], + "info": + The number of values to place in a bucket to perform delta encoding. Must be a power of 2, maximum is 128. + , + "label": "String dictionary encoding bucket size", + "max": 128, + "min": 1, + "name": "stringDictionaryEncoding.bucketSize", + "type": "number", + }, + Object { + "defaultValue": "roaring", + "info": + Compression format for bitmap indexes. + , + "label": "Bitmap type", + "name": "bitmap.type", + "suggestions": Array [ + "roaring", + "concise", + ], + "type": "string", + }, + Object { + "defaultValue": true, + "defined": [Function], + "info": + Controls whether or not run-length encoding will be used when it is determined to be more space-efficient. + , + "label": "Bitmap compress run on serialization", + "name": "bitmap.compressRunOnSerialization", + "type": "boolean", + }, + Object { + "defaultValue": "lz4", + "info": + Compression format for dimension columns. + , + "name": "dimensionCompression", + "suggestions": Array [ + "lz4", + "lzf", + "zstd", + "uncompressed", + ], + "type": "string", + }, + Object { + "defaultValue": "longs", + "info": + Encoding format for long-typed columns. Applies regardless of whether they are dimensions or metrics. + + auto + + encodes the values using offset or lookup table depending on column cardinality, and store them with variable size. + + longs + + stores the value as-is with 8 bytes each. + , + "name": "longEncoding", + "suggestions": Array [ + "longs", + "auto", + ], + "type": "string", + }, + Object { + "defaultValue": "lz4", + "info": + Compression format for primitive type metric columns. + , + "name": "metricCompression", + "suggestions": Array [ + "lz4", + "lzf", + "zstd", + "uncompressed", + ], + "type": "string", + }, + Object { + "defaultValue": "lz4", + "info": + Compression format to use for nested column raw data. + , + "label": "JSON compression", + "name": "jsonCompression", + "suggestions": Array [ + "lz4", + "lzf", + "zstd", + "uncompressed", + ], + "type": "string", + }, + ] + } + model={Object {}} + onChange={[Function]} + /> +
+
+
+ + +
+
+
+`; diff --git a/web-console/src/dialogs/index-spec-dialog/index-spec-dialog.scss b/web-console/src/dialogs/index-spec-dialog/index-spec-dialog.scss new file mode 100644 index 000000000000..e7cc53ee47dd --- /dev/null +++ b/web-console/src/dialogs/index-spec-dialog/index-spec-dialog.scss @@ -0,0 +1,36 @@ +/* + * 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. + */ + +@import '../../variables'; + +.index-spec-dialog { + &.#{$bp-ns}-dialog { + height: 70vh; + } + + .form-json-selector { + margin: 15px; + } + + .content { + margin: 0 15px 10px 0; + padding: 0 5px 0 15px; + flex: 1; + overflow: auto; + } +} diff --git a/web-console/src/dialogs/index-spec-dialog/index-spec-dialog.spec.tsx b/web-console/src/dialogs/index-spec-dialog/index-spec-dialog.spec.tsx new file mode 100644 index 000000000000..68f7f56b885b --- /dev/null +++ b/web-console/src/dialogs/index-spec-dialog/index-spec-dialog.spec.tsx @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import { shallow } from 'enzyme'; +import React from 'react'; + +import { IndexSpecDialog } from './index-spec-dialog'; + +describe('IndexSpecDialog', () => { + it('matches snapshot without compactionConfig', () => { + const compactionDialog = shallow( + {}} onSave={() => {}} indexSpec={undefined} />, + ); + expect(compactionDialog).toMatchSnapshot(); + }); + + it('matches snapshot with indexSpec', () => { + const compactionDialog = shallow( + {}} + onSave={() => {}} + indexSpec={{ + dimensionCompression: 'lzf', + }} + />, + ); + expect(compactionDialog).toMatchSnapshot(); + }); +}); diff --git a/web-console/src/dialogs/index-spec-dialog/index-spec-dialog.tsx b/web-console/src/dialogs/index-spec-dialog/index-spec-dialog.tsx new file mode 100644 index 000000000000..4c870df45af2 --- /dev/null +++ b/web-console/src/dialogs/index-spec-dialog/index-spec-dialog.tsx @@ -0,0 +1,88 @@ +/* + * 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. + */ + +import { Button, Classes, Dialog, Intent } from '@blueprintjs/core'; +import React, { useState } from 'react'; + +import { AutoForm, FormJsonSelector, FormJsonTabs, JsonInput } from '../../components'; +import { INDEX_SPEC_FIELDS, IndexSpec } from '../../druid-models'; + +import './index-spec-dialog.scss'; + +export interface IndexSpecDialogProps { + title?: string; + onClose: () => void; + onSave: (indexSpec: IndexSpec) => void; + indexSpec: IndexSpec | undefined; +} + +export const IndexSpecDialog = React.memo(function IndexSpecDialog(props: IndexSpecDialogProps) { + const { title, indexSpec, onSave, onClose } = props; + + const [currentTab, setCurrentTab] = useState('form'); + const [currentIndexSpec, setCurrentIndexSpec] = useState(indexSpec || {}); + const [jsonError, setJsonError] = useState(); + + const issueWithCurrentIndexSpec = AutoForm.issueWithModel(currentIndexSpec, INDEX_SPEC_FIELDS); + + return ( + + +
+ {currentTab === 'form' ? ( + setCurrentIndexSpec(m)} + /> + ) : ( + { + setCurrentIndexSpec(v); + setJsonError(undefined); + }} + onError={setJsonError} + issueWithValue={value => AutoForm.issueWithModel(value, INDEX_SPEC_FIELDS)} + height="100%" + /> + )} +
+
+
+
+
+
+ ); +}); diff --git a/web-console/src/dialogs/index.ts b/web-console/src/dialogs/index.ts index 9509442c8bd2..588257c84e74 100644 --- a/web-console/src/dialogs/index.ts +++ b/web-console/src/dialogs/index.ts @@ -24,6 +24,7 @@ export * from './diff-dialog/diff-dialog'; export * from './doctor-dialog/doctor-dialog'; export * from './edit-context-dialog/edit-context-dialog'; export * from './history-dialog/history-dialog'; +export * from './kill-datasource-dialog/kill-datasource-dialog'; export * from './lookup-edit-dialog/lookup-edit-dialog'; export * from './numeric-input-dialog/numeric-input-dialog'; export * from './overlord-dynamic-config-dialog/overlord-dynamic-config-dialog'; diff --git a/web-console/src/dialogs/kill-datasource-dialog/kill-datasource-dialog.tsx b/web-console/src/dialogs/kill-datasource-dialog/kill-datasource-dialog.tsx new file mode 100644 index 000000000000..3eb7e9fdf243 --- /dev/null +++ b/web-console/src/dialogs/kill-datasource-dialog/kill-datasource-dialog.tsx @@ -0,0 +1,110 @@ +/* + * 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. + */ + +import { Code, Intent } from '@blueprintjs/core'; +import React, { useState } from 'react'; + +import { FormGroupWithInfo, PopoverText } from '../../components'; +import { SuggestibleInput } from '../../components/suggestible-input/suggestible-input'; +import { Api } from '../../singletons'; +import { uniq } from '../../utils'; +import { AsyncActionDialog } from '../async-action-dialog/async-action-dialog'; + +function getSuggestions(): string[] { + // Default to a data 24h ago so as not to cause a conflict between streaming ingestion and kill tasks + const end = new Date(Date.now() - 24 * 60 * 60 * 1000).toISOString(); + const startOfDay = end.slice(0, 10); + const startOfMonth = end.slice(0, 7) + '-01'; + const startOfYear = end.slice(0, 4) + '-01-01'; + + return uniq([ + `1000-01-01/${startOfDay}`, + `1000-01-01/${startOfMonth}`, + `1000-01-01/${startOfYear}`, + '1000-01-01/3000-01-01', + ]); +} + +export interface KillDatasourceDialogProps { + datasource: string; + onClose(): void; + onSuccess(): void; +} + +export const KillDatasourceDialog = function KillDatasourceDialog( + props: KillDatasourceDialogProps, +) { + const { datasource, onClose, onSuccess } = props; + const suggestions = getSuggestions(); + const [interval, setInterval] = useState(suggestions[0]); + + return ( + { + const resp = await Api.instance.delete( + `/druid/coordinator/v1/datasources/${Api.encodePath( + datasource, + )}?kill=true&interval=${Api.encodePath(interval)}`, + {}, + ); + return resp.data; + }} + confirmButtonText="Permanently delete unused segments" + successText="Kill task was issued. Unused segments in datasource will be deleted" + failText="Failed submit kill task" + intent={Intent.DANGER} + onClose={onClose} + onSuccess={onSuccess} + warningChecks={[ + <> + I understand that this operation will delete all metadata about the unused segments of{' '} + {datasource} and removes them from deep storage. + , + 'I understand that this operation cannot be undone.', + ]} + > +

+ Are you sure you want to permanently delete unused segments in {datasource}? +

+

This action is not reversible and the data deleted will be lost.

+ +

+ The range of time over which to delete unused segments specified in ISO8601 interval + format. +

+

+ If you have streaming ingestion running make sure that your interval range doe not + overlap with intervals where streaming data is being added - otherwise the kill task + will not start. +

+ + } + > + setInterval(s || '')} + suggestions={suggestions} + /> +
+
+ ); +}; diff --git a/web-console/src/druid-models/compaction-status/compaction-status.spec.ts b/web-console/src/druid-models/compaction-status/compaction-status.spec.ts index 8ed0c5141366..9d1254090bf5 100644 --- a/web-console/src/druid-models/compaction-status/compaction-status.spec.ts +++ b/web-console/src/druid-models/compaction-status/compaction-status.spec.ts @@ -18,11 +18,7 @@ import { CompactionConfig } from '../compaction-config/compaction-config'; -import { - CompactionStatus, - formatCompactionConfigAndStatus, - zeroCompactionStatus, -} from './compaction-status'; +import { CompactionStatus, formatCompactionInfo, zeroCompactionStatus } from './compaction-status'; describe('compaction status', () => { const BASIC_CONFIG: CompactionConfig = {}; @@ -61,27 +57,30 @@ describe('compaction status', () => { }); it('formatCompactionConfigAndStatus', () => { - expect(formatCompactionConfigAndStatus(undefined, undefined)).toEqual('Not enabled'); + expect(formatCompactionInfo({})).toEqual('Not enabled'); - expect(formatCompactionConfigAndStatus(BASIC_CONFIG, undefined)).toEqual('Awaiting first run'); + expect(formatCompactionInfo({ config: BASIC_CONFIG })).toEqual('Awaiting first run'); - expect(formatCompactionConfigAndStatus(undefined, ZERO_STATUS)).toEqual('Not enabled'); + expect(formatCompactionInfo({ status: ZERO_STATUS })).toEqual('Not enabled'); - expect(formatCompactionConfigAndStatus(BASIC_CONFIG, ZERO_STATUS)).toEqual('Running'); + expect(formatCompactionInfo({ config: BASIC_CONFIG, status: ZERO_STATUS })).toEqual('Running'); expect( - formatCompactionConfigAndStatus(BASIC_CONFIG, { - dataSource: 'tbl', - scheduleStatus: 'RUNNING', - bytesAwaitingCompaction: 0, - bytesCompacted: 100, - bytesSkipped: 0, - segmentCountAwaitingCompaction: 0, - segmentCountCompacted: 10, - segmentCountSkipped: 0, - intervalCountAwaitingCompaction: 0, - intervalCountCompacted: 10, - intervalCountSkipped: 0, + formatCompactionInfo({ + config: BASIC_CONFIG, + status: { + dataSource: 'tbl', + scheduleStatus: 'RUNNING', + bytesAwaitingCompaction: 0, + bytesCompacted: 100, + bytesSkipped: 0, + segmentCountAwaitingCompaction: 0, + segmentCountCompacted: 10, + segmentCountSkipped: 0, + intervalCountAwaitingCompaction: 0, + intervalCountCompacted: 10, + intervalCountSkipped: 0, + }, }), ).toEqual('Fully compacted'); }); diff --git a/web-console/src/druid-models/compaction-status/compaction-status.ts b/web-console/src/druid-models/compaction-status/compaction-status.ts index 2982d9b69e17..d17f2c44fda4 100644 --- a/web-console/src/druid-models/compaction-status/compaction-status.ts +++ b/web-console/src/druid-models/compaction-status/compaction-status.ts @@ -50,19 +50,19 @@ export function zeroCompactionStatus(compactionStatus: CompactionStatus): boolea ); } -export function formatCompactionConfigAndStatus( - compactionConfig: CompactionConfig | undefined, - compactionStatus: CompactionStatus | undefined, -) { - if (compactionConfig) { - if (compactionStatus) { - if ( - compactionStatus.bytesAwaitingCompaction === 0 && - !zeroCompactionStatus(compactionStatus) - ) { +export interface CompactionInfo { + config?: CompactionConfig; + status?: CompactionStatus; +} + +export function formatCompactionInfo(compaction: CompactionInfo) { + const { config, status } = compaction; + if (config) { + if (status) { + if (status.bytesAwaitingCompaction === 0 && !zeroCompactionStatus(status)) { return 'Fully compacted'; } else { - return capitalizeFirst(compactionStatus.scheduleStatus); + return capitalizeFirst(status.scheduleStatus); } } else { return 'Awaiting first run'; diff --git a/web-console/src/druid-models/coordinator-dynamic-config/coordinator-dynamic-config.tsx b/web-console/src/druid-models/coordinator-dynamic-config/coordinator-dynamic-config.tsx index eeb25db09c4b..ca957309ff83 100644 --- a/web-console/src/druid-models/coordinator-dynamic-config/coordinator-dynamic-config.tsx +++ b/web-console/src/druid-models/coordinator-dynamic-config/coordinator-dynamic-config.tsx @@ -69,20 +69,9 @@ export const COORDINATOR_DYNAMIC_CONFIG_FIELDS: Field[ ), }, - { - name: 'killAllDataSources', - type: 'boolean', - defaultValue: false, - info: ( - <> - Send kill tasks for ALL dataSources if property druid.coordinator.kill.on is - true. If this is set to true then killDataSourceWhitelist must not be specified - or be empty list. - - ), - }, { name: 'killDataSourceWhitelist', + label: 'Kill datasource whitelist', type: 'string-array', emptyValue: [], info: ( diff --git a/web-console/src/druid-models/index-spec/index-spec.tsx b/web-console/src/druid-models/index-spec/index-spec.tsx new file mode 100644 index 000000000000..1a4246299661 --- /dev/null +++ b/web-console/src/druid-models/index-spec/index-spec.tsx @@ -0,0 +1,158 @@ +/* + * 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. + */ + +import { Code } from '@blueprintjs/core'; +import React from 'react'; + +import { Field } from '../../components'; +import { deepGet } from '../../utils'; + +export interface IndexSpec { + bitmap?: Bitmap; + dimensionCompression?: string; + stringDictionaryEncoding?: { type: 'utf8' | 'frontCoded'; bucketSize: number }; + metricCompression?: string; + longEncoding?: string; + jsonCompression?: string; +} + +export interface Bitmap { + type: string; + compressRunOnSerialization?: boolean; +} + +export function summarizeIndexSpec(indexSpec: IndexSpec | undefined): string { + if (!indexSpec) return ''; + + const { stringDictionaryEncoding, bitmap, longEncoding } = indexSpec; + + const ret: string[] = []; + if (stringDictionaryEncoding) { + switch (stringDictionaryEncoding.type) { + case 'frontCoded': + ret.push(`frontCoded(${stringDictionaryEncoding.bucketSize || 4})`); + break; + + default: + ret.push(stringDictionaryEncoding.type); + break; + } + } + + if (bitmap) { + ret.push(bitmap.type); + } + + if (longEncoding) { + ret.push(longEncoding); + } + + return ret.join('; '); +} + +export const INDEX_SPEC_FIELDS: Field[] = [ + { + name: 'stringDictionaryEncoding.type', + label: 'String dictionary encoding', + type: 'string', + defaultValue: 'utf8', + suggestions: ['utf8', 'frontCoded'], + info: ( + <> + Encoding format for STRING value dictionaries used by STRING and COMPLEX<json> + columns. + + ), + }, + { + name: 'stringDictionaryEncoding.bucketSize', + label: 'String dictionary encoding bucket size', + type: 'number', + defaultValue: 4, + min: 1, + max: 128, + defined: spec => deepGet(spec, 'stringDictionaryEncoding.type') === 'frontCoded', + info: ( + <> + The number of values to place in a bucket to perform delta encoding. Must be a power of 2, + maximum is 128. + + ), + }, + + { + name: 'bitmap.type', + label: 'Bitmap type', + type: 'string', + defaultValue: 'roaring', + suggestions: ['roaring', 'concise'], + info: <>Compression format for bitmap indexes., + }, + { + name: 'bitmap.compressRunOnSerialization', + label: 'Bitmap compress run on serialization', + type: 'boolean', + defaultValue: true, + defined: spec => (deepGet(spec, 'bitmap.type') || 'roaring') === 'roaring', + info: ( + <> + Controls whether or not run-length encoding will be used when it is determined to be more + space-efficient. + + ), + }, + + { + name: 'dimensionCompression', + type: 'string', + defaultValue: 'lz4', + suggestions: ['lz4', 'lzf', 'zstd', 'uncompressed'], + info: <>Compression format for dimension columns., + }, + + { + name: 'longEncoding', + type: 'string', + defaultValue: 'longs', + suggestions: ['longs', 'auto'], + info: ( + <> + Encoding format for long-typed columns. Applies regardless of whether they are dimensions or + metrics. auto encodes the values using offset or lookup table depending on + column cardinality, and store them with variable size. longs stores the value + as-is with 8 bytes each. + + ), + }, + { + name: 'metricCompression', + type: 'string', + defaultValue: 'lz4', + suggestions: ['lz4', 'lzf', 'zstd', 'uncompressed'], + info: <>Compression format for primitive type metric columns., + }, + + { + name: 'jsonCompression', + label: 'JSON compression', + type: 'string', + defaultValue: 'lz4', + suggestions: ['lz4', 'lzf', 'zstd', 'uncompressed'], + info: <>Compression format to use for nested column raw data. , + }, +]; diff --git a/web-console/src/druid-models/index.ts b/web-console/src/druid-models/index.ts index 359ba70440b8..0b4ad6b65f70 100644 --- a/web-console/src/druid-models/index.ts +++ b/web-console/src/druid-models/index.ts @@ -25,6 +25,7 @@ export * from './execution/execution'; export * from './external-config/external-config'; export * from './filter/filter'; export * from './flatten-spec/flatten-spec'; +export * from './index-spec/index-spec'; export * from './ingest-query-pattern/ingest-query-pattern'; export * from './ingestion-spec/ingestion-spec'; export * from './input-format/input-format'; diff --git a/web-console/src/druid-models/ingestion-spec/ingestion-spec.tsx b/web-console/src/druid-models/ingestion-spec/ingestion-spec.tsx index b68052abd471..f144f4c2ad29 100644 --- a/web-console/src/druid-models/ingestion-spec/ingestion-spec.tsx +++ b/web-console/src/druid-models/ingestion-spec/ingestion-spec.tsx @@ -21,6 +21,7 @@ import { range } from 'd3-array'; import React from 'react'; import { AutoForm, ExternalLink, Field } from '../../components'; +import { IndexSpecDialog } from '../../dialogs/index-spec-dialog/index-spec-dialog'; import { getLink } from '../../links'; import { allowKeys, @@ -44,6 +45,7 @@ import { getDimensionSpecs, getDimensionSpecType, } from '../dimension-spec/dimension-spec'; +import { IndexSpec, summarizeIndexSpec } from '../index-spec/index-spec'; import { InputFormat, issueWithInputFormat } from '../input-format/input-format'; import { FILTER_SUGGESTIONS, @@ -1379,6 +1381,7 @@ export interface TuningConfig { partitionsSpec?: PartitionsSpec; maxPendingPersists?: number; indexSpec?: IndexSpec; + indexSpecForIntermediatePersists?: IndexSpec; forceExtendableShardSpecs?: boolean; forceGuaranteedRollup?: boolean; reportParseExceptions?: boolean; @@ -1869,103 +1872,38 @@ const TUNING_FORM_FIELDS: Field[] = [ }, { - name: 'spec.tuningConfig.indexSpec.bitmap.type', - label: 'Index bitmap type', - type: 'string', - defaultValue: 'roaring', - suggestions: ['concise', 'roaring'], + name: 'spec.tuningConfig.indexSpec', + type: 'custom', hideInMore: true, - info: <>Compression format for bitmap indexes., - }, - { - name: 'spec.tuningConfig.indexSpec.bitmap.compressRunOnSerialization', - type: 'boolean', - defaultValue: true, - defined: spec => deepGet(spec, 'spec.tuningConfig.indexSpec.bitmap.type') === 'roaring', - info: ( - <> - Controls whether or not run-length encoding will be used when it is determined to be more - space-efficient. - + info: <>Defines segment storage format options to use at indexing time., + placeholder: 'Default index spec', + customSummary: summarizeIndexSpec, + customDialog: ({ value, onValueChange, onClose }) => ( + ), }, - - { - name: 'spec.tuningConfig.indexSpec.dimensionCompression', - label: 'Index dimension compression', - type: 'string', - defaultValue: 'lz4', - suggestions: ['lz4', 'lzf', 'zstd', 'uncompressed'], - hideInMore: true, - info: <>Compression format for dimension columns., - }, - { - name: 'spec.tuningConfig.indexSpec.stringDictionaryEncoding.type', - label: 'Index string dictionary encoding', - type: 'string', - defaultValue: 'utf8', - suggestions: ['utf8', 'frontCoded'], + name: 'spec.tuningConfig.indexSpecForIntermediatePersists', + type: 'custom', hideInMore: true, info: ( <> - Encoding format for STRING value dictionaries used by STRING and COMPLEX<json> - columns. + Defines segment storage format options to use at indexing time for intermediate persisted + temporary segments. ), - }, - { - name: 'spec.tuningConfig.indexSpec.stringDictionaryEncoding.bucketSize', - label: 'Index string dictionary encoding bucket size', - type: 'number', - defaultValue: 4, - min: 1, - max: 128, - defined: spec => - deepGet(spec, 'spec.tuningConfig.indexSpec.stringDictionaryEncoding.type') === 'frontCoded', - hideInMore: true, - info: ( - <> - The number of values to place in a bucket to perform delta encoding. Must be a power of 2, - maximum is 128. - + placeholder: 'Default index spec', + customSummary: summarizeIndexSpec, + customDialog: ({ value, onValueChange, onClose }) => ( + ), }, - { - name: 'spec.tuningConfig.indexSpec.metricCompression', - label: 'Index metric compression', - type: 'string', - defaultValue: 'lz4', - suggestions: ['lz4', 'lzf', 'zstd', 'uncompressed'], - hideInMore: true, - info: <>Compression format for primitive type metric columns., - }, - { - name: 'spec.tuningConfig.indexSpec.longEncoding', - label: 'Index long encoding', - type: 'string', - defaultValue: 'longs', - suggestions: ['longs', 'auto'], - hideInMore: true, - info: ( - <> - Encoding format for long-typed columns. Applies regardless of whether they are dimensions or - metrics. auto encodes the values using offset or lookup table depending on - column cardinality, and store them with variable size. longs stores the value - as-is with 8 bytes each. - - ), - }, - { - name: 'spec.tuningConfig.indexSpec.jsonCompression', - label: 'Index JSON compression', - type: 'string', - defaultValue: 'lz4', - suggestions: ['lz4', 'lzf', 'zstd', 'uncompressed'], - hideInMore: true, - info: <>Compression format to use for nested column raw data. , - }, { name: 'spec.tuningConfig.splitHintSpec.maxSplitSize', type: 'number', @@ -2172,18 +2110,6 @@ export function getTuningFormFields() { return TUNING_FORM_FIELDS; } -export interface IndexSpec { - bitmap?: Bitmap; - dimensionCompression?: string; - metricCompression?: string; - longEncoding?: string; -} - -export interface Bitmap { - type: string; - compressRunOnSerialization?: boolean; -} - // -------------- export function updateIngestionType( diff --git a/web-console/src/druid-models/workbench-query/workbench-query-part.ts b/web-console/src/druid-models/workbench-query/workbench-query-part.ts index 604cfb013213..5e4afb453f09 100644 --- a/web-console/src/druid-models/workbench-query/workbench-query-part.ts +++ b/web-console/src/druid-models/workbench-query/workbench-query-part.ts @@ -62,10 +62,10 @@ export class WorkbenchQueryPart { static getIngestDatasourceFromQueryFragment(queryFragment: string): string | undefined { // Assuming the queryFragment is no parsable find the prefix that look like: // REPLACEINTOSELECT - const matchInsertReplaceIndex = queryFragment.match(/(?:INSERT|REPLACE)\s+INTO/)?.index; + const matchInsertReplaceIndex = queryFragment.match(/(?:INSERT|REPLACE)\s+INTO/i)?.index; if (typeof matchInsertReplaceIndex !== 'number') return; - const matchEnd = queryFragment.match(/\b(?:SELECT|WITH)\b|$/); + const matchEnd = queryFragment.match(/\b(?:SELECT|WITH)\b|$/i); const fragmentQuery = SqlQuery.maybeParse( queryFragment.substring(matchInsertReplaceIndex, matchEnd?.index) + ' SELECT * FROM t', ); diff --git a/web-console/src/druid-models/workbench-query/workbench-query.spec.ts b/web-console/src/druid-models/workbench-query/workbench-query.spec.ts index 5d7e2615098c..9af0fb240700 100644 --- a/web-console/src/druid-models/workbench-query/workbench-query.spec.ts +++ b/web-console/src/druid-models/workbench-query/workbench-query.spec.ts @@ -465,7 +465,7 @@ describe('WorkbenchQuery', () => { it('works with INSERT (unparsable)', () => { const sql = sane` -- Some comment - INSERT INTO trips2 + INSERT into trips2 SELECT TIME_PARSE(pickup_datetime) AS __time, * diff --git a/web-console/src/helpers/spec-conversion.spec.ts b/web-console/src/helpers/spec-conversion.spec.ts index 0239da185415..2f6aa59f51e2 100644 --- a/web-console/src/helpers/spec-conversion.spec.ts +++ b/web-console/src/helpers/spec-conversion.spec.ts @@ -106,6 +106,9 @@ describe('spec conversion', () => { partitionDimension: 'isRobot', targetRowsPerSegment: 150000, }, + indexSpec: { + dimensionCompression: 'lzf', + }, forceGuaranteedRollup: true, maxNumConcurrentSubTasks: 4, maxParseExceptions: 3, @@ -159,6 +162,9 @@ describe('spec conversion', () => { maxParseExceptions: 3, finalizeAggregations: false, maxNumTasks: 5, + indexSpec: { + dimensionCompression: 'lzf', + }, }); }); diff --git a/web-console/src/helpers/spec-conversion.ts b/web-console/src/helpers/spec-conversion.ts index d35433917f59..9b76e787dd61 100644 --- a/web-console/src/helpers/spec-conversion.ts +++ b/web-console/src/helpers/spec-conversion.ts @@ -70,6 +70,11 @@ export function convertSpecToSql(spec: any): QueryWithContext { groupByEnableMultiValueUnnesting: false, }; + const indexSpec = deepGet(spec, 'spec.tuningConfig.indexSpec'); + if (indexSpec) { + context.indexSpec = indexSpec; + } + const lines: string[] = []; const rollup = deepGet(spec, 'spec.dataSchema.granularitySpec.rollup') ?? true; diff --git a/web-console/src/views/datasources-view/datasources-view.tsx b/web-console/src/views/datasources-view/datasources-view.tsx index 7f384fc8bc2a..c38a42b63938 100644 --- a/web-console/src/views/datasources-view/datasources-view.tsx +++ b/web-console/src/views/datasources-view/datasources-view.tsx @@ -36,12 +36,18 @@ import { TableColumnSelector, ViewControlBar, } from '../../components'; -import { AsyncActionDialog, CompactionDialog, RetentionDialog } from '../../dialogs'; +import { + AsyncActionDialog, + CompactionDialog, + KillDatasourceDialog, + RetentionDialog, +} from '../../dialogs'; import { DatasourceTableActionDialog } from '../../dialogs/datasource-table-action-dialog/datasource-table-action-dialog'; import { CompactionConfig, + CompactionInfo, CompactionStatus, - formatCompactionConfigAndStatus, + formatCompactionInfo, QueryWithContext, zeroCompactionStatus, } from '../../druid-models'; @@ -208,9 +214,8 @@ function segmentGranularityCountsToRank(row: DatasourceQueryResultRow): number { } interface Datasource extends DatasourceQueryResultRow { - readonly rules: Rule[]; - readonly compactionConfig?: CompactionConfig; - readonly compactionStatus?: CompactionStatus; + readonly rules?: Rule[]; + readonly compaction?: CompactionInfo; readonly unused?: boolean; } @@ -220,7 +225,7 @@ function makeUnusedDatasource(datasource: string): Datasource { interface DatasourcesAndDefaultRules { readonly datasources: Datasource[]; - readonly defaultRules: Rule[]; + readonly defaultRules?: Rule[]; } interface RetentionDialogOpenOn { @@ -433,43 +438,85 @@ ORDER BY 1`; let unused: string[] = []; if (showUnused) { - const unusedResp = await Api.instance.get( - '/druid/coordinator/v1/metadata/datasources?includeUnused', + try { + unused = ( + await Api.instance.get( + '/druid/coordinator/v1/metadata/datasources?includeUnused', + ) + ).data.filter(d => !seen[d]); + } catch { + AppToaster.show({ + icon: IconNames.ERROR, + intent: Intent.DANGER, + message: 'Could not get the list of unused datasources', + }); + } + } + + let rules: Record = {}; + try { + rules = (await Api.instance.get>('/druid/coordinator/v1/rules')) + .data; + } catch { + AppToaster.show({ + icon: IconNames.ERROR, + intent: Intent.DANGER, + message: 'Could not get load rules', + }); + } + + let compactionConfigs: Record | undefined; + try { + const compactionConfigsResp = await Api.instance.get<{ + compactionConfigs: CompactionConfig[]; + }>('/druid/coordinator/v1/config/compaction'); + compactionConfigs = lookupBy( + compactionConfigsResp.data.compactionConfigs || [], + c => c.dataSource, ); - unused = unusedResp.data.filter(d => !seen[d]); + } catch { + AppToaster.show({ + icon: IconNames.ERROR, + intent: Intent.DANGER, + message: 'Could not get compaction configs', + }); } - const rulesResp = await Api.instance.get>( - '/druid/coordinator/v1/rules', - ); - const rules = rulesResp.data; - - const compactionConfigsResp = await Api.instance.get<{ - compactionConfigs: CompactionConfig[]; - }>('/druid/coordinator/v1/config/compaction'); - const compactionConfigs = lookupBy( - compactionConfigsResp.data.compactionConfigs || [], - c => c.dataSource, - ); - - const compactionStatusesResp = await Api.instance.get<{ latestStatus: CompactionStatus[] }>( - '/druid/coordinator/v1/compaction/status', - ); - const compactionStatuses = lookupBy( - compactionStatusesResp.data.latestStatus || [], - c => c.dataSource, - ); + let compactionStatuses: Record | undefined; + if (compactionConfigs) { + // Don't bother getting the statuses if we can not even get the configs + try { + const compactionStatusesResp = await Api.instance.get<{ + latestStatus: CompactionStatus[]; + }>('/druid/coordinator/v1/compaction/status'); + compactionStatuses = lookupBy( + compactionStatusesResp.data.latestStatus || [], + c => c.dataSource, + ); + } catch { + AppToaster.show({ + icon: IconNames.ERROR, + intent: Intent.DANGER, + message: 'Could not get compaction statuses', + }); + } + } return { datasources: datasources.concat(unused.map(makeUnusedDatasource)).map(ds => { return { ...ds, - rules: rules[ds.datasource] || [], - compactionConfig: compactionConfigs[ds.datasource], - compactionStatus: compactionStatuses[ds.datasource], + rules: rules[ds.datasource], + compaction: + compactionConfigs && compactionStatuses + ? { + config: compactionConfigs[ds.datasource], + status: compactionStatuses[ds.datasource], + } + : undefined, }; }), - defaultRules: rules[DEFAULT_RULES_KEY] || [], + defaultRules: rules[DEFAULT_RULES_KEY], }; }, onStateChange: datasourcesAndDefaultRulesState => { @@ -633,36 +680,15 @@ ORDER BY 1`; if (!killDatasource) return; return ( - { - const resp = await Api.instance.delete( - `/druid/coordinator/v1/datasources/${Api.encodePath( - killDatasource, - )}?kill=true&interval=1000/3000`, - {}, - ); - return resp.data; - }} - confirmButtonText="Permanently delete unused segments" - successText="Kill task was issued. Unused segments in datasource will be deleted" - failText="Failed submit kill task" - intent={Intent.DANGER} + { this.setState({ killDatasource: undefined }); }} onSuccess={() => { this.fetchDatasourceData(); }} - warningChecks={[ - `I understand that this operation will delete all metadata about the unused segments of ${killDatasource} and removes them from deep storage.`, - 'I understand that this operation cannot be undone.', - ]} - > -

- {`Are you sure you want to permanently delete unused segments in '${killDatasource}'?`} -

-

This action is not reversible and the data deleted will be lost.

-
+ /> ); } @@ -756,20 +782,20 @@ ORDER BY 1`; this.setState({ retentionDialogOpenOn: undefined }); setTimeout(() => { this.setState(state => { - const datasourcesAndDefaultRules = state.datasourcesAndDefaultRulesState.data; - if (!datasourcesAndDefaultRules) return {}; + const defaultRules = state.datasourcesAndDefaultRulesState.data?.defaultRules; + if (!defaultRules) return {}; return { retentionDialogOpenOn: { datasource: '_default', - rules: datasourcesAndDefaultRules.defaultRules, + rules: defaultRules, }, }; }); }, 50); }; - private readonly saveCompaction = async (compactionConfig: any) => { + private readonly saveCompaction = async (compactionConfig: CompactionConfig) => { if (!compactionConfig) return; try { await Api.instance.post(`/druid/coordinator/v1/config/compaction`, compactionConfig); @@ -819,8 +845,8 @@ ORDER BY 1`; getDatasourceActions( datasource: string, unused: boolean | undefined, - rules: Rule[], - compactionConfig: CompactionConfig | undefined, + rules: Rule[] | undefined, + compactionInfo: CompactionInfo | undefined, ): BasicAction[] { const { goToQuery, goToTask, capabilities } = this.props; @@ -863,82 +889,83 @@ ORDER BY 1`; }, ]; } else { - return goToActions.concat([ - { - icon: IconNames.AUTOMATIC_UPDATES, - title: 'Edit retention rules', - onAction: () => { - this.setState({ - retentionDialogOpenOn: { - datasource, - rules, - }, - }); + return goToActions.concat( + compact([ + { + icon: IconNames.AUTOMATIC_UPDATES, + title: 'Edit retention rules', + onAction: () => { + this.setState({ + retentionDialogOpenOn: { + datasource, + rules: rules || [], + }, + }); + }, }, - }, - { - icon: IconNames.REFRESH, - title: 'Mark as used all segments (will lead to reapplying retention rules)', - onAction: () => - this.setState({ - datasourceToMarkAllNonOvershadowedSegmentsAsUsedIn: datasource, - }), - }, - { - icon: IconNames.COMPRESSED, - title: 'Edit compaction configuration', - onAction: () => { - this.setState({ - compactionDialogOpenOn: { - datasource, - compactionConfig, - }, - }); + { + icon: IconNames.REFRESH, + title: 'Mark as used all segments (will lead to reapplying retention rules)', + onAction: () => + this.setState({ + datasourceToMarkAllNonOvershadowedSegmentsAsUsedIn: datasource, + }), }, - }, - { - icon: IconNames.EXPORT, - title: 'Mark as used segments by interval', - - onAction: () => - this.setState({ - datasourceToMarkSegmentsByIntervalIn: datasource, - useUnuseAction: 'use', - }), - }, - { - icon: IconNames.IMPORT, - title: 'Mark as unused segments by interval', - - onAction: () => - this.setState({ - datasourceToMarkSegmentsByIntervalIn: datasource, - useUnuseAction: 'unuse', - }), - }, - { - icon: IconNames.IMPORT, - title: 'Mark as unused all segments', - intent: Intent.DANGER, - onAction: () => this.setState({ datasourceToMarkAsUnusedAllSegmentsIn: datasource }), - }, - { - icon: IconNames.TRASH, - title: 'Delete unused segments (issue kill task)', - intent: Intent.DANGER, - onAction: () => this.setState({ killDatasource: datasource }), - }, - ]); + compactionInfo + ? { + icon: IconNames.COMPRESSED, + title: 'Edit compaction configuration', + onAction: () => { + this.setState({ + compactionDialogOpenOn: { + datasource, + compactionConfig: compactionInfo.config, + }, + }); + }, + } + : undefined, + { + icon: IconNames.EXPORT, + title: 'Mark as used segments by interval', + + onAction: () => + this.setState({ + datasourceToMarkSegmentsByIntervalIn: datasource, + useUnuseAction: 'use', + }), + }, + { + icon: IconNames.IMPORT, + title: 'Mark as unused segments by interval', + + onAction: () => + this.setState({ + datasourceToMarkSegmentsByIntervalIn: datasource, + useUnuseAction: 'unuse', + }), + }, + { + icon: IconNames.IMPORT, + title: 'Mark as unused all segments', + intent: Intent.DANGER, + onAction: () => this.setState({ datasourceToMarkAsUnusedAllSegmentsIn: datasource }), + }, + { + icon: IconNames.TRASH, + title: 'Delete unused segments (issue kill task)', + intent: Intent.DANGER, + onAction: () => this.setState({ killDatasource: datasource }), + }, + ]), + ); } } private renderRetentionDialog(): JSX.Element | undefined { const { retentionDialogOpenOn, tiersState, datasourcesAndDefaultRulesState } = this.state; - const { defaultRules } = datasourcesAndDefaultRulesState.data || { - datasources: [], - defaultRules: [], - }; - if (!retentionDialogOpenOn) return; + const defaultRules = datasourcesAndDefaultRulesState.data?.defaultRules; + if (!retentionDialogOpenOn || !defaultRules) return; return ( !d.unused); @@ -1009,8 +1034,8 @@ ORDER BY 1`; const replicatedSizeValues = datasources.map(d => formatReplicatedSize(d.replicated_size)); const leftToBeCompactedValues = datasources.map(d => - d.compactionStatus - ? formatLeftToBeCompacted(d.compactionStatus.bytesAwaitingCompaction) + d.compaction?.status + ? formatLeftToBeCompacted(d.compaction?.status.bytesAwaitingCompaction) : '-', ); @@ -1297,24 +1322,26 @@ ORDER BY 1`; Header: 'Compaction', show: capabilities.hasCoordinatorAccess() && visibleColumns.shown('Compaction'), id: 'compactionStatus', - accessor: row => Boolean(row.compactionStatus), + accessor: row => Boolean(row.compaction?.status), filterable: false, width: 150, Cell: ({ original }) => { - const { datasource, compactionConfig, compactionStatus } = original as Datasource; + const { datasource, compaction } = original as Datasource; return ( + disabled={!compaction} + onClick={() => { + if (!compaction) return; this.setState({ compactionDialogOpenOn: { datasource, - compactionConfig, + compactionConfig: compaction.config, }, - }) - } + }); + }} hoverIcon={IconNames.EDIT} > - {formatCompactionConfigAndStatus(compactionConfig, compactionStatus)} + {compaction ? formatCompactionInfo(compaction) : 'Could not get compaction info'} ); }, @@ -1324,17 +1351,22 @@ ORDER BY 1`; show: capabilities.hasCoordinatorAccess() && visibleColumns.shown('% Compacted'), id: 'percentCompacted', width: 200, - accessor: ({ compactionStatus }) => - compactionStatus && compactionStatus.bytesCompacted - ? compactionStatus.bytesCompacted / - (compactionStatus.bytesAwaitingCompaction + compactionStatus.bytesCompacted) - : 0, + accessor: ({ compaction }) => { + const status = compaction?.status; + return status?.bytesCompacted + ? status.bytesCompacted / (status.bytesAwaitingCompaction + status.bytesCompacted) + : 0; + }, filterable: false, className: 'padded', Cell: ({ original }) => { - const { compactionStatus } = original as Datasource; + const { compaction } = original as Datasource; + if (!compaction) { + return 'Could not get compaction info'; + } - if (!compactionStatus || zeroCompactionStatus(compactionStatus)) { + const { status } = compaction; + if (!status || zeroCompactionStatus(status)) { return ( <>  {' '} @@ -1348,20 +1380,14 @@ ORDER BY 1`; <> {' '}  {' '} {' '} @@ -1369,8 +1395,8 @@ ORDER BY 1`; - (compactionStatus && compactionStatus.bytesAwaitingCompaction) || 0, + accessor: ({ compaction }) => { + const status = compaction?.status; + return status?.bytesAwaitingCompaction || 0; + }, filterable: false, className: 'padded', Cell: ({ original }) => { - const { compactionStatus } = original as Datasource; + const { compaction } = original as Datasource; + if (!compaction) { + return 'Could not get compaction info'; + } - if (!compactionStatus) { + const { status } = compaction; + if (!status) { return ; } return ( ); @@ -1408,26 +1440,30 @@ ORDER BY 1`; Header: 'Retention', show: capabilities.hasCoordinatorAccess() && visibleColumns.shown('Retention'), id: 'retention', - accessor: row => row.rules.length, + accessor: row => row.rules?.length || 0, filterable: false, width: 200, Cell: ({ original }) => { const { datasource, rules } = original as Datasource; return ( + disabled={!defaultRules} + onClick={() => { + if (!defaultRules) return; this.setState({ retentionDialogOpenOn: { datasource, - rules, + rules: rules || [], }, - }) - } + }); + }} hoverIcon={IconNames.EDIT} > - {rules.length + {rules?.length ? DatasourcesView.formatRules(rules) - : `Cluster default: ${DatasourcesView.formatRules(defaultRules)}`} + : defaultRules + ? `Cluster default: ${DatasourcesView.formatRules(defaultRules)}` + : 'Could not get default rules'} ); }, @@ -1440,12 +1476,12 @@ ORDER BY 1`; width: ACTION_COLUMN_WIDTH, filterable: false, Cell: ({ value: datasource, original }) => { - const { unused, rules, compactionConfig } = original as Datasource; + const { unused, rules, compaction } = original as Datasource; const datasourceActions = this.getDatasourceActions( datasource, unused, rules, - compactionConfig, + compaction, ); return ( 0) { return formatDuration(value); } diff --git a/web-console/src/views/services-view/services-view.tsx b/web-console/src/views/services-view/services-view.tsx index aa2a934533e7..9dff93c8ab7d 100644 --- a/web-console/src/views/services-view/services-view.tsx +++ b/web-console/src/views/services-view/services-view.tsx @@ -36,11 +36,12 @@ import { import { AsyncActionDialog } from '../../dialogs'; import { QueryWithContext } from '../../druid-models'; import { STANDARD_TABLE_PAGE_SIZE, STANDARD_TABLE_PAGE_SIZE_OPTIONS } from '../../react-table'; -import { Api } from '../../singletons'; +import { Api, AppToaster } from '../../singletons'; import { Capabilities, CapabilitiesMode, deepGet, + filterMap, formatBytes, formatBytesCompact, hasPopoverOpen, @@ -117,7 +118,7 @@ export interface ServicesViewState { visibleColumns: LocalStorageBackedVisibility; } -interface ServiceQueryResultRow { +interface ServiceResultRow { readonly service: string; readonly service_type: string; readonly tier: string; @@ -127,16 +128,18 @@ interface ServiceQueryResultRow { readonly max_size: NumberLike; readonly plaintext_port: number; readonly tls_port: number; + loadQueueInfo?: LoadQueueInfo; + workerInfo?: WorkerInfo; } -interface LoadQueueStatus { +interface LoadQueueInfo { readonly segmentsToDrop: NumberLike; readonly segmentsToDropSize: NumberLike; readonly segmentsToLoad: NumberLike; readonly segmentsToLoadSize: NumberLike; } -interface MiddleManagerQueryResultRow { +interface WorkerInfo { readonly availabilityGroups: string[]; readonly blacklistedUntil: string | null; readonly currCapacityUsed: NumberLike; @@ -153,11 +156,6 @@ interface MiddleManagerQueryResultRow { }; } -interface ServiceResultRow - extends ServiceQueryResultRow, - Partial, - Partial {} - export class ServicesView extends React.PureComponent { private readonly serviceQueryManager: QueryManager; @@ -198,7 +196,7 @@ ORDER BY ) DESC, "service" DESC`; - static async getServices(): Promise { + static async getServices(): Promise { const allServiceResp = await Api.instance.get('/druid/coordinator/v1/servers?simple'); const allServices = allServiceResp.data; return allServices.map((s: any) => { @@ -228,7 +226,7 @@ ORDER BY this.serviceQueryManager = new QueryManager({ processQuery: async capabilities => { - let services: ServiceQueryResultRow[]; + let services: ServiceResultRow[]; if (capabilities.hasSql()) { services = await queryDruidSql({ query: ServicesView.SERVICE_SQL }); } else if (capabilities.hasCoordinatorAccess()) { @@ -238,50 +236,49 @@ ORDER BY } if (capabilities.hasCoordinatorAccess()) { - const loadQueueResponse = await Api.instance.get( - '/druid/coordinator/v1/loadqueue?simple', - ); - const loadQueues: Record = loadQueueResponse.data; - services = services.map(s => { - const loadQueueInfo = loadQueues[s.service]; - if (loadQueueInfo) { - s = { ...s, ...loadQueueInfo }; - } - return s; - }); + try { + const loadQueueInfos = ( + await Api.instance.get>( + '/druid/coordinator/v1/loadqueue?simple', + ) + ).data; + services.forEach(s => { + s.loadQueueInfo = loadQueueInfos[s.service]; + }); + } catch { + AppToaster.show({ + icon: IconNames.ERROR, + intent: Intent.DANGER, + message: 'There was an error getting the load queue info', + }); + } } if (capabilities.hasOverlordAccess()) { - let middleManagers: MiddleManagerQueryResultRow[]; try { - const middleManagerResponse = await Api.instance.get('/druid/indexer/v1/workers'); - middleManagers = middleManagerResponse.data; + const workerInfos = (await Api.instance.get('/druid/indexer/v1/workers')) + .data; + + const workerInfoLookup: Record = lookupBy( + workerInfos, + m => m.worker?.host, + ); + + services.forEach(s => { + s.workerInfo = workerInfoLookup[s.service]; + }); } catch (e) { + // Swallow this error because it simply a reflection of a local task runner. if ( - e.response && - typeof e.response.data === 'object' && - e.response.data.error === 'Task Runner does not support worker listing' + deepGet(e, 'response.data.error') !== 'Task Runner does not support worker listing' ) { - // Swallow this error because it simply a reflection of a local task runner. - middleManagers = []; - } else { - // Otherwise re-throw. - throw e; + AppToaster.show({ + icon: IconNames.ERROR, + intent: Intent.DANGER, + message: 'There was an error getting the worker info', + }); } } - - const middleManagersLookup: Record = lookupBy( - middleManagers, - m => m.worker.host, - ); - - services = services.map(s => { - const middleManagerInfo = middleManagersLookup[s.service]; - if (middleManagerInfo) { - s = { ...s, ...middleManagerInfo }; - } - return s; - }); } return services; @@ -372,7 +369,8 @@ ORDER BY id: 'tier', width: 180, accessor: row => { - return row.tier ? row.tier : row.worker ? row.worker.category : null; + if (row.tier) return row.tier; + return deepGet(row, 'workerInfo.worker.category'); }, Cell: this.renderFilterableCell('tier'), }, @@ -451,9 +449,11 @@ ORDER BY className: 'padded', accessor: row => { if (oneOf(row.service_type, 'middle_manager', 'indexer')) { - return row.worker - ? (Number(row.currCapacityUsed) || 0) / Number(row.worker.capacity) - : null; + const { workerInfo } = row; + if (!workerInfo) return 0; + return ( + (Number(workerInfo.currCapacityUsed) || 0) / Number(workerInfo.worker?.capacity) + ); } else { return row.max_size ? Number(row.curr_size) / Number(row.max_size) : null; } @@ -469,15 +469,21 @@ ORDER BY case 'indexer': case 'middle_manager': { - const originalMiddleManagers: ServiceResultRow[] = row.subRows.map( - r => r._original, + const workerInfos: WorkerInfo[] = filterMap( + row.subRows, + r => r._original.workerInfo, ); + + if (!workerInfos.length) { + return 'Could not get worker infos'; + } + const totalCurrCapacityUsed = sum( - originalMiddleManagers, - s => Number(s.currCapacityUsed) || 0, + workerInfos, + w => Number(w.currCapacityUsed) || 0, ); const totalWorkerCapacity = sum( - originalMiddleManagers, + workerInfos, s => deepGet(s, 'worker.capacity') || 0, ); return `${totalCurrCapacityUsed} / ${totalWorkerCapacity} (total slots)`; @@ -496,8 +502,12 @@ ORDER BY case 'indexer': case 'middle_manager': { - const currCapacityUsed = deepGet(row, 'original.currCapacityUsed') || 0; - const capacity = deepGet(row, 'original.worker.capacity'); + if (!deepGet(row, 'original.workerInfo')) { + return 'Could not get capacity info'; + } + const currCapacityUsed = + deepGet(row, 'original.workerInfo.currCapacityUsed') || 0; + const capacity = deepGet(row, 'original.workerInfo.worker.capacity'); if (typeof capacity === 'number') { return `Slots used: ${currCapacityUsed} of ${capacity}`; } else { @@ -518,30 +528,58 @@ ORDER BY filterable: false, className: 'padded', accessor: row => { - if (oneOf(row.service_type, 'middle_manager', 'indexer')) { - if (deepGet(row, 'worker.version') === '') return 'Disabled'; + switch (row.service_type) { + case 'middle_manager': + case 'indexer': { + if (deepGet(row, 'worker.version') === '') return 'Disabled'; + const { workerInfo } = row; + if (!workerInfo) { + return 'Could not get detail info'; + } - const details: string[] = []; - if (row.lastCompletedTaskTime) { - details.push(`Last completed task: ${row.lastCompletedTaskTime}`); + const details: string[] = []; + if (workerInfo.lastCompletedTaskTime) { + details.push(`Last completed task: ${workerInfo.lastCompletedTaskTime}`); + } + if (workerInfo.blacklistedUntil) { + details.push(`Blacklisted until: ${workerInfo.blacklistedUntil}`); + } + return details.join(' '); } - if (row.blacklistedUntil) { - details.push(`Blacklisted until: ${row.blacklistedUntil}`); + + case 'coordinator': + case 'overlord': + return row.is_leader === 1 ? 'Leader' : ''; + + case 'historical': { + const { loadQueueInfo } = row; + if (!loadQueueInfo) return 0; + return ( + (Number(loadQueueInfo.segmentsToLoad) || 0) + + (Number(loadQueueInfo.segmentsToDrop) || 0) + ); } - return details.join(' '); - } else if (oneOf(row.service_type, 'coordinator', 'overlord')) { - return row.is_leader === 1 ? 'Leader' : ''; - } else { - return (Number(row.segmentsToLoad) || 0) + (Number(row.segmentsToDrop) || 0); + + default: + return 0; } }, Cell: row => { if (row.aggregated) return ''; const { service_type } = row.original; switch (service_type) { + case 'middle_manager': + case 'indexer': + case 'coordinator': + case 'overlord': + return row.value; + case 'historical': { + const { loadQueueInfo } = row.original; + if (!loadQueueInfo) return 'Could not get load queue info'; + const { segmentsToLoad, segmentsToLoadSize, segmentsToDrop, segmentsToDropSize } = - row.original; + loadQueueInfo; return formatQueues( segmentsToLoad, segmentsToLoadSize, @@ -550,23 +588,31 @@ ORDER BY ); } - case 'indexer': - case 'middle_manager': - case 'coordinator': - case 'overlord': - return row.value; - default: return ''; } }, Aggregated: row => { if (row.row._pivotVal !== 'historical') return ''; - const originals: ServiceResultRow[] = row.subRows.map(r => r._original); - const segmentsToLoad = sum(originals, s => Number(s.segmentsToLoad) || 0); - const segmentsToLoadSize = sum(originals, s => Number(s.segmentsToLoadSize) || 0); - const segmentsToDrop = sum(originals, s => Number(s.segmentsToDrop) || 0); - const segmentsToDropSize = sum(originals, s => Number(s.segmentsToDropSize) || 0); + const loadQueueInfos: LoadQueueInfo[] = filterMap( + row.subRows, + r => r._original.loadQueueInfo, + ); + + if (!loadQueueInfos.length) { + return 'Could not get load queue infos'; + } + + const segmentsToLoad = sum(loadQueueInfos, s => Number(s.segmentsToLoad) || 0); + const segmentsToLoadSize = sum( + loadQueueInfos, + s => Number(s.segmentsToLoadSize) || 0, + ); + const segmentsToDrop = sum(loadQueueInfos, s => Number(s.segmentsToDrop) || 0); + const segmentsToDropSize = sum( + loadQueueInfos, + s => Number(s.segmentsToDropSize) || 0, + ); return formatQueues( segmentsToLoad, segmentsToLoadSize, @@ -580,13 +626,14 @@ ORDER BY show: capabilities.hasOverlordAccess() && visibleColumns.shown(ACTION_COLUMN_LABEL), id: ACTION_COLUMN_ID, width: ACTION_COLUMN_WIDTH, - accessor: row => row.worker, + accessor: row => row.workerInfo, filterable: false, Cell: ({ value, aggregated }) => { if (aggregated) return ''; if (!value) return null; - const disabled = value.version === ''; - const workerActions = this.getWorkerActions(value.host, disabled); + const { worker } = value; + const disabled = worker.version === ''; + const workerActions = this.getWorkerActions(worker.host, disabled); return ; }, Aggregated: () => '', diff --git a/web-console/src/views/workbench-view/input-source-step/example-inputs.ts b/web-console/src/views/workbench-view/input-source-step/example-inputs.ts index e58dfacca35e..a74f1754b179 100644 --- a/web-console/src/views/workbench-view/input-source-step/example-inputs.ts +++ b/web-console/src/views/workbench-view/input-source-step/example-inputs.ts @@ -16,15 +16,74 @@ * limitations under the License. */ -import { InputSource } from '../../../druid-models'; +import { InputFormat, InputSource } from '../../../druid-models'; -export interface ExampleInputSource { +export interface ExampleInput { name: string; description: string; inputSource: InputSource; + inputFormat?: InputFormat; } -export const EXAMPLE_INPUT_SOURCES: ExampleInputSource[] = [ +const TRIPS_INPUT_FORMAT: InputFormat = { + type: 'csv', + findColumnsFromHeader: false, + columns: [ + 'trip_id', + 'vendor_id', + 'pickup_datetime', + 'dropoff_datetime', + 'store_and_fwd_flag', + 'rate_code_id', + 'pickup_longitude', + 'pickup_latitude', + 'dropoff_longitude', + 'dropoff_latitude', + 'passenger_count', + 'trip_distance', + 'fare_amount', + 'extra', + 'mta_tax', + 'tip_amount', + 'tolls_amount', + 'ehail_fee', + 'improvement_surcharge', + 'total_amount', + 'payment_type', + 'trip_type', + 'pickup', + 'dropoff', + 'cab_type', + 'precipitation', + 'snow_depth', + 'snowfall', + 'max_temperature', + 'min_temperature', + 'average_wind_speed', + 'pickup_nyct2010_gid', + 'pickup_ctlabel', + 'pickup_borocode', + 'pickup_boroname', + 'pickup_ct2010', + 'pickup_boroct2010', + 'pickup_cdeligibil', + 'pickup_ntacode', + 'pickup_ntaname', + 'pickup_puma', + 'dropoff_nyct2010_gid', + 'dropoff_ctlabel', + 'dropoff_borocode', + 'dropoff_boroname', + 'dropoff_ct2010', + 'dropoff_boroct2010', + 'dropoff_cdeligibil', + 'dropoff_ntacode', + 'dropoff_ntaname', + 'dropoff_puma', + ], +}; + +export const EXAMPLE_INPUTS: ExampleInput[] = [ { name: 'Wikipedia', description: 'One day of wikipedia edits (JSON)', @@ -62,6 +121,7 @@ export const EXAMPLE_INPUT_SOURCES: ExampleInputSource[] = [ 'https://static.imply.io/example-data/trips/trips_xac.csv.gz', ], }, + inputFormat: TRIPS_INPUT_FORMAT, }, { name: 'NYC Taxi cabs (all files)', @@ -145,6 +205,7 @@ export const EXAMPLE_INPUT_SOURCES: ExampleInputSource[] = [ 'https://static.imply.io/example-data/trips/trips_xcv.csv.gz', ], }, + inputFormat: TRIPS_INPUT_FORMAT, }, { name: 'FlightCarrierOnTime (1 month)', diff --git a/web-console/src/views/workbench-view/input-source-step/input-source-step.tsx b/web-console/src/views/workbench-view/input-source-step/input-source-step.tsx index 211271c62c3d..f144e8f975d5 100644 --- a/web-console/src/views/workbench-view/input-source-step/input-source-step.tsx +++ b/web-console/src/views/workbench-view/input-source-step/input-source-step.tsx @@ -55,7 +55,7 @@ import { UrlBaser } from '../../../singletons'; import { filterMap, IntermediateQueryState } from '../../../utils'; import { postToSampler, SampleSpec } from '../../../utils/sampler'; -import { EXAMPLE_INPUT_SOURCES } from './example-inputs'; +import { EXAMPLE_INPUTS } from './example-inputs'; import { InputSourceInfo } from './input-source-info'; import './input-source-step.scss'; @@ -81,16 +81,15 @@ export const InputSourceStep = React.memo(function InputSourceStep(props: InputS const [inputSource, setInputSource] = useState | string | undefined>( initInputSource, ); - const exampleInputSource = EXAMPLE_INPUT_SOURCES.find( - ({ name }) => name === inputSource, - )?.inputSource; + const exampleInput = EXAMPLE_INPUTS.find(({ name }) => name === inputSource); const [guessedInputFormatState, connectQueryManager] = useQueryManager< - InputSource, + { inputSource: InputSource; suggestedInputFormat?: InputFormat }, InputFormat, Execution >({ - processQuery: async (inputSource: InputSource, cancelToken) => { + processQuery: async ({ inputSource, suggestedInputFormat }, cancelToken) => { + let guessedInputFormat: InputFormat | undefined; if (mode === 'sampler') { const sampleSpec: SampleSpec = { type: 'index_parallel', @@ -127,7 +126,7 @@ export const InputSourceStep = React.memo(function InputSourceStep(props: InputS ); if (!sampleLines.length) throw new Error('No data returned from sampler'); - return guessInputFormat(sampleLines); + guessedInputFormat = guessInputFormat(sampleLines); } else { const tableExpression = externalConfigToTableExpression({ inputSource, @@ -151,8 +150,14 @@ export const InputSourceStep = React.memo(function InputSourceStep(props: InputS ); if (result instanceof IntermediateQueryState) return result; - return resultToInputFormat(result); + guessedInputFormat = resultToInputFormat(result); } + + if (suggestedInputFormat?.type === guessedInputFormat.type) { + return suggestedInputFormat; + } + + return guessedInputFormat; }, backgroundStatusCheck: async (execution, query, cancelToken) => { const result = await executionBackgroundResultStatusCheck(execution, query, cancelToken); @@ -164,7 +169,7 @@ export const InputSourceStep = React.memo(function InputSourceStep(props: InputS useEffect(() => { const guessedInputFormat = guessedInputFormatState.data; if (!guessedInputFormat) return; - onSet(exampleInputSource || (inputSource as any), guessedInputFormat); + onSet(exampleInput?.inputSource || (inputSource as any), guessedInputFormat); // eslint-disable-next-line react-hooks/exhaustive-deps }, [guessedInputFormatState]); @@ -217,7 +222,7 @@ export const InputSourceStep = React.memo(function InputSourceStep(props: InputS selectedValue={inputSource} onChange={e => setInputSource(e.currentTarget.value)} > - {EXAMPLE_INPUT_SOURCES.map((e, i) => ( + {EXAMPLE_INPUTS.map((e, i) => ( { - if (!exampleInputSource) return; - connectQueryManager.runQuery(exampleInputSource); + if (!exampleInput) return; + connectQueryManager.runQuery({ + inputSource: exampleInput.inputSource, + suggestedInputFormat: exampleInput.inputFormat, + }); }} /> ) : inputSource ? ( @@ -324,7 +332,7 @@ export const InputSourceStep = React.memo(function InputSourceStep(props: InputS } onClick={() => { if (!AutoForm.isValidModel(inputSource, INPUT_SOURCE_FIELDS)) return; - connectQueryManager.runQuery(inputSource); + connectQueryManager.runQuery({ inputSource }); }} /> ) : undefined} diff --git a/web-console/src/views/workbench-view/run-panel/run-panel.tsx b/web-console/src/views/workbench-view/run-panel/run-panel.tsx index 572120c9e730..7299760b4621 100644 --- a/web-console/src/views/workbench-view/run-panel/run-panel.tsx +++ b/web-console/src/views/workbench-view/run-panel/run-panel.tsx @@ -33,6 +33,7 @@ import React, { useCallback, useMemo, useState } from 'react'; import { MenuCheckbox, MenuTristate } from '../../../components'; import { EditContextDialog, StringInputDialog } from '../../../dialogs'; +import { IndexSpecDialog } from '../../../dialogs/index-spec-dialog/index-spec-dialog'; import { changeDurableShuffleStorage, changeFinalizeAggregations, @@ -51,9 +52,12 @@ import { getUseApproximateCountDistinct, getUseApproximateTopN, getUseCache, + IndexSpec, + QueryContext, + summarizeIndexSpec, WorkbenchQuery, } from '../../../druid-models'; -import { pluralIfNeeded, tickIcon } from '../../../utils'; +import { deepGet, pluralIfNeeded, tickIcon } from '../../../utils'; import { MaxTasksButton } from '../max-tasks-button/max-tasks-button'; import './run-panel.scss'; @@ -94,6 +98,7 @@ export const RunPanel = React.memo(function RunPanel(props: RunPanelProps) { const { query, onQueryChange, onRun, moreMenu, loading, small, queryEngines } = props; const [editContextDialogOpen, setEditContextDialogOpen] = useState(false); const [customTimezoneDialogOpen, setCustomTimezoneDialogOpen] = useState(false); + const [indexSpecDialogSpec, setIndexSpecDialogSpec] = useState(); const emptyQuery = query.isEmptyQuery(); const ingestMode = query.isIngestQuery(); @@ -104,6 +109,7 @@ export const RunPanel = React.memo(function RunPanel(props: RunPanelProps) { const finalizeAggregations = getFinalizeAggregations(queryContext); const groupByEnableMultiValueUnnesting = getGroupByEnableMultiValueUnnesting(queryContext); const durableShuffleStorage = getDurableShuffleStorage(queryContext); + const indexSpec: IndexSpec | undefined = deepGet(queryContext, 'indexSpec'); const useApproximateCountDistinct = getUseApproximateCountDistinct(queryContext); const useApproximateTopN = getUseApproximateTopN(queryContext); const useCache = getUseCache(queryContext); @@ -157,6 +163,10 @@ export const RunPanel = React.memo(function RunPanel(props: RunPanelProps) { ); } + function changeQueryContext(queryContext: QueryContext) { + onQueryChange(query.changeQueryContext(queryContext)); + } + const availableEngines = ([undefined] as (DruidEngine | undefined)[]).concat(queryEngines); function offsetOptions(): JSX.Element[] { @@ -170,9 +180,7 @@ export const RunPanel = React.memo(function RunPanel(props: RunPanelProps) { icon={tickIcon(offset === timezone)} text={offset} shouldDismissPopover={false} - onClick={() => { - onQueryChange(query.changeQueryContext(changeTimezone(queryContext, offset))); - }} + onClick={() => changeQueryContext(changeTimezone(queryContext, offset))} />, ); } @@ -233,11 +241,7 @@ export const RunPanel = React.memo(function RunPanel(props: RunPanelProps) { icon={tickIcon(!timezone)} text="Default" shouldDismissPopover={false} - onClick={() => { - onQueryChange( - query.changeQueryContext(changeTimezone(queryContext, undefined)), - ); - }} + onClick={() => changeQueryContext(changeTimezone(queryContext, undefined))} /> {NAMED_TIMEZONES.map(namedTimezone => ( @@ -246,11 +250,9 @@ export const RunPanel = React.memo(function RunPanel(props: RunPanelProps) { icon={tickIcon(namedTimezone === timezone)} text={namedTimezone} shouldDismissPopover={false} - onClick={() => { - onQueryChange( - query.changeQueryContext(changeTimezone(queryContext, namedTimezone)), - ); - }} + onClick={() => + changeQueryContext(changeTimezone(queryContext, namedTimezone)) + } /> ))} @@ -276,11 +278,9 @@ export const RunPanel = React.memo(function RunPanel(props: RunPanelProps) { key={String(v)} icon={tickIcon(v === maxParseExceptions)} text={v === -1 ? '∞ (-1)' : String(v)} - onClick={() => { - onQueryChange( - query.changeQueryContext(changeMaxParseExceptions(queryContext, v)), - ); - }} + onClick={() => + changeQueryContext(changeMaxParseExceptions(queryContext, v)) + } shouldDismissPopover={false} /> ))} @@ -290,35 +290,36 @@ export const RunPanel = React.memo(function RunPanel(props: RunPanelProps) { text="Finalize aggregations" value={finalizeAggregations} undefinedEffectiveValue={!ingestMode} - onValueChange={v => { - onQueryChange( - query.changeQueryContext(changeFinalizeAggregations(queryContext, v)), - ); - }} + onValueChange={v => + changeQueryContext(changeFinalizeAggregations(queryContext, v)) + } /> { - onQueryChange( - query.changeQueryContext( - changeGroupByEnableMultiValueUnnesting(queryContext, v), - ), - ); + onValueChange={v => + changeQueryContext(changeGroupByEnableMultiValueUnnesting(queryContext, v)) + } + /> + { + setIndexSpecDialogSpec(indexSpec || {}); }} /> { - onQueryChange( - query.changeQueryContext( - changeDurableShuffleStorage(queryContext, !durableShuffleStorage), - ), - ); - }} + onChange={() => + changeQueryContext( + changeDurableShuffleStorage(queryContext, !durableShuffleStorage), + ) + } /> ) : ( @@ -326,22 +327,16 @@ export const RunPanel = React.memo(function RunPanel(props: RunPanelProps) { { - onQueryChange( - query.changeQueryContext(changeUseCache(queryContext, !useCache)), - ); - }} + onChange={() => changeQueryContext(changeUseCache(queryContext, !useCache))} /> { - onQueryChange( - query.changeQueryContext( - changeUseApproximateTopN(queryContext, !useApproximateTopN), - ), - ); - }} + onChange={() => + changeQueryContext( + changeUseApproximateTopN(queryContext, !useApproximateTopN), + ) + } /> )} @@ -349,16 +344,14 @@ export const RunPanel = React.memo(function RunPanel(props: RunPanelProps) { { - onQueryChange( - query.changeQueryContext( - changeUseApproximateCountDistinct( - queryContext, - !useApproximateCountDistinct, - ), + onChange={() => + changeQueryContext( + changeUseApproximateCountDistinct( + queryContext, + !useApproximateCountDistinct, ), - ); - }} + ) + } /> )} {effectiveEngine === 'sql-msq-task' && ( - - onQueryChange(query.changeQueryContext(queryContext)) - } - /> + )} )} @@ -399,10 +387,7 @@ export const RunPanel = React.memo(function RunPanel(props: RunPanelProps) { {editContextDialogOpen && ( { - if (!onQueryChange) return; - onQueryChange(query.changeQueryContext(newContext)); - }} + onQueryContextChange={changeQueryContext} onClose={() => { setEditContextDialogOpen(false); }} @@ -413,10 +398,17 @@ export const RunPanel = React.memo(function RunPanel(props: RunPanelProps) { title="Custom timezone" placeholder="Etc/UTC" maxLength={50} - onSubmit={tz => onQueryChange(query.changeQueryContext(changeTimezone(queryContext, tz)))} + onSubmit={tz => changeQueryContext(changeTimezone(queryContext, tz))} onClose={() => setCustomTimezoneDialogOpen(false)} /> )} + {indexSpecDialogSpec && ( + setIndexSpecDialogSpec(undefined)} + onSave={indexSpec => changeQueryContext({ ...queryContext, indexSpec })} + indexSpec={indexSpecDialogSpec} + /> + )}
); }); From 054e4e96bf7da9a7ecd83639ef46ccf42157c1bb Mon Sep 17 00:00:00 2001 From: Jill Osborne Date: Thu, 1 Dec 2022 10:04:33 +0000 Subject: [PATCH 18/34] Update experimental features doc (#13462) (cherry picked from commit 994d7c2d2c22009cccda30d70fa9345fb22e1175) --- docs/development/experimental-features.md | 10 +--------- 1 file changed, 1 insertion(+), 9 deletions(-) diff --git a/docs/development/experimental-features.md b/docs/development/experimental-features.md index a68fc47411af..30d8c2f77c2b 100644 --- a/docs/development/experimental-features.md +++ b/docs/development/experimental-features.md @@ -26,7 +26,7 @@ The following features are marked [experimental](./experimental.md) in the Druid This document includes each page that mentions an experimental feature. To graduate a feature, remove all mentions of its experimental status on all relevant pages. -Note that this document does not track the status of contrib extensions, some of which are experimental. +Note that this document does not track the status of contrib extensions, all of which are considered experimental. ## SQL-based ingestion @@ -34,10 +34,6 @@ Note that this document does not track the status of contrib extensions, some of - [SQL-based ingestion concepts](../multi-stage-query/concepts.md) - [SQL-based ingestion and multi-stage query task API](../multi-stage-query/api.md) -## Nested columns - -- [Nested columns](../querying/nested-columns.md) - ## Indexer process - [Indexer process](../design/indexer.md) @@ -53,10 +49,6 @@ Note that this document does not track the status of contrib extensions, some of - [Task reference](../ingestion/tasks.md#locking) - [Design](../design/architecture.md#availability-and-consistency) -## Moments sketch - -- [Aggregations](../querying/aggregations.md#moments-sketch-experimental) - ## Front coding - [Ingestion spec reference](../ingestion/ingestion-spec.md#front-coding) From baf6ca427b0556049510a4a3a9d2836ddbbd4936 Mon Sep 17 00:00:00 2001 From: Jill Osborne Date: Fri, 2 Dec 2022 00:35:27 +0000 Subject: [PATCH 19/34] LDAP docs backport (#13453) * Update LDAP configuration docs (cherry picked from commit e74bd89bab690dacf538c49cd8093e0e5075b3e5) * Updated after review (cherry picked from commit 882e0b2ccc451b40dc56b59c0fa1ddcbc92cc558) * Update auth-ldap.md Updated. (cherry picked from commit d4f0797cee70c1bbf9a7bdfd903cb6d9266abdb0) * Update auth-ldap.md (cherry picked from commit fbec7b29278630fe16ccb1105fce7ab3142b9d3c) * Updated spelling file (cherry picked from commit ef5316b2640875f461fcbe8de3ff219a57d3b8eb) * Update docs/operations/auth-ldap.md Co-authored-by: Charles Smith (cherry picked from commit 1a9b42af992fe6797e84573c65a8d3ade3155c82) * Update docs/operations/auth-ldap.md Co-authored-by: Charles Smith (cherry picked from commit 1018d9a0f4d23d58acc7d4dcd4ba20d522a21791) * Update docs/operations/auth-ldap.md Co-authored-by: Charles Smith (cherry picked from commit dd81b3f7dea7aaa27f78394e21d61710354e53d9) * Update auth-ldap.md (cherry picked from commit f0655cfb81c27999980563685d08e3ba5a599bb1) --- .../extensions-core/druid-basic-security.md | 420 ++++++++++++++---- docs/operations/auth-ldap.md | 295 ++++++++---- docs/operations/security-overview.md | 75 +--- website/.spelling | 3 + 4 files changed, 542 insertions(+), 251 deletions(-) diff --git a/docs/development/extensions-core/druid-basic-security.md b/docs/development/extensions-core/druid-basic-security.md index b6698b4f153d..4e042b4405f3 100644 --- a/docs/development/extensions-core/druid-basic-security.md +++ b/docs/development/extensions-core/druid-basic-security.md @@ -53,12 +53,29 @@ To set the value for the configuration properties, add them to the common runtim ### General properties -|Property|Description|Default|required| -|--------|-----------|-------|--------| -|`druid.auth.basic.common.pollingPeriod`|Defines in milliseconds how often processes should poll the Coordinator for the current Druid metadata store authenticator/authorizer state.|60000|No| -|`druid.auth.basic.common.maxRandomDelay`|Defines in milliseconds the amount of random delay to add to the pollingPeriod, to spread polling requests across time.|6000|No| -|`druid.auth.basic.common.maxSyncRetries`|Determines how many times a service will retry if the authentication/authorization Druid metadata store state sync with the Coordinator fails.|10|No| -|`druid.auth.basic.common.cacheDirectory`|If defined, snapshots of the basic Authenticator and Authorizer Druid metadata store caches will be stored on disk in this directory. If this property is defined, when a service is starting, it will attempt to initialize its caches from these on-disk snapshots, if the service is unable to initialize its state by communicating with the Coordinator.|null|No| +**`druid.auth.basic.common.pollingPeriod`** + +Defines in milliseconds how often processes should poll the Coordinator for the current Druid metadata store authenticator/authorizer state.
+         **Required**: No
+         **Default**: 60000 + +**`druid.auth.basic.common.maxRandomDelay`** + +Defines in milliseconds the amount of random delay to add to the pollingPeriod, to spread polling requests across time.
+         **Required**: No
+         **Default**: 6000 + +**`druid.auth.basic.common.maxSyncRetries`** + +Determines how many times a service will retry if the authentication/authorization Druid metadata store state sync with the Coordinator fails.
+         **Required**: No
+         **Default**: 10 + +**`druid.auth.basic.common.cacheDirectory`** + +If defined, snapshots of the basic Authenticator and Authorizer Druid metadata store caches will be stored on disk in this directory. If this property is defined, when a service is starting, it will attempt to initialize its caches from these on-disk snapshots, if the service is unable to initialize its state by communicating with the Coordinator.
+         **Required**: No
+         **Default**: null ### Authenticator @@ -96,16 +113,55 @@ The remaining examples of authenticator configuration use either `MyBasicMetadat #### Properties for Druid metadata store user authentication -|Property|Description|Default|required| -|--------|-----------|-------|--------| -|`druid.auth.authenticator.MyBasicMetadataAuthenticator.initialAdminPassword`|Initial [Password Provider](../../operations/password-provider.md) for the automatically created default admin user. If no password is specified, the default admin user will not be created. If the default admin user already exists, setting this property will not affect its password.|null|No| -|`druid.auth.authenticator.MyBasicMetadataAuthenticator.initialInternalClientPassword`|Initial [Password Provider](../../operations/password-provider.md) for the default internal system user, used for internal process communication. If no password is specified, the default internal system user will not be created. If the default internal system user already exists, setting this property will not affect its password.|null|No| -|`druid.auth.authenticator.MyBasicMetadataAuthenticator.enableCacheNotifications`|If true, the Coordinator will notify Druid processes whenever a configuration change to this Authenticator occurs, allowing them to immediately update their state without waiting for polling.|true|No| -|`druid.auth.authenticator.MyBasicMetadataAuthenticator.cacheNotificationTimeout`|The timeout in milliseconds for the cache notifications.|5000|No| -|`druid.auth.authenticator.MyBasicMetadataAuthenticator.credentialIterations`|Number of iterations to use for password hashing. See [Credential iterations and API performance](#credential-iterations-and-api-performance)|10000|No| -|`druid.auth.authenticator.MyBasicMetadataAuthenticator.credentialsValidator.type`|The type of credentials store (metadata) to validate requests credentials.|metadata|No| -|`druid.auth.authenticator.MyBasicMetadataAuthenticator.skipOnFailure`|If true and the request credential doesn't exists or isn't fully configured in the credentials store, the request will proceed to next Authenticator in the chain.|false|No| -|`druid.auth.authenticator.MyBasicMetadataAuthenticator.authorizerName`|Authorizer that requests should be directed to|N/A|Yes| + +**`druid.auth.authenticator.MyBasicMetadataAuthenticator.initialAdminPassword`** + +Initial [Password Provider](../../operations/password-provider.md) for the automatically created default admin user. If no password is specified, the default admin user will not be created. If the default admin user already exists, setting this property will not affect its password.
+         **Required**: No
+         **Default**: null + +**`druid.auth.authenticator.MyBasicMetadataAuthenticator.initialInternalClientPassword`** + +Initial [Password Provider](../../operations/password-provider.md) for the default internal system user, used for internal process communication. If no password is specified, the default internal system user will not be created. If the default internal system user already exists, setting this property will not affect its password.
+         **Required**: No
+         **Default**: null + +**`druid.auth.authenticator.MyBasicMetadataAuthenticator.enableCacheNotifications`** + +If true, the Coordinator will notify Druid processes whenever a configuration change to this Authenticator occurs, allowing them to immediately update their state without waiting for polling.
+         **Required**: No
+         **Default**: True + +**`druid.auth.authenticator.MyBasicMetadataAuthenticator.cacheNotificationTimeout`** + +The timeout in milliseconds for the cache notifications.
+         **Required**: No
+         **Default**: 5000 + +**`druid.auth.authenticator.MyBasicMetadataAuthenticator.credentialIterations`** + +Number of iterations to use for password hashing. See [Credential iterations and API performance](#credential-iterations-and-api-performance)
+         **Required**: No
+         **Default**: 10000 + +**`druid.auth.authenticator.MyBasicMetadataAuthenticator.credentialsValidator.type`** + +The type of credentials store (metadata) to validate requests credentials.
+         **Required**: No
+         **Default**: metadata + +**`druid.auth.authenticator.MyBasicMetadataAuthenticator.skipOnFailure`** + +If true and the request credential doesn't exists or isn't fully configured in the credentials store, the request will proceed to next Authenticator in the chain.
+         **Required**: No
+         **Default**: false + +**`druid.auth.authenticator.MyBasicMetadataAuthenticator.authorizerName`** + +Authorizer that requests should be directed to.
+         **Required**: Yes
+         **Default**: N/A + ##### Credential iterations and API performance @@ -121,25 +177,107 @@ If Druid uses the default credentials validator (i.e., `credentialsValidator.typ #### Properties for LDAP user authentication -|Property|Description|Default|required| -|--------|-----------|-------|--------| -|`druid.auth.authenticator.MyBasicLDAPAuthenticator.initialAdminPassword`|Initial [Password Provider](../../operations/password-provider.md) for the automatically created default admin user. If no password is specified, the default admin user will not be created. If the default admin user already exists, setting this property will not affect its password.|null|No| -|`druid.auth.authenticator.MyBasicLDAPAuthenticator.initialInternalClientPassword`|Initial [Password Provider](../../operations/password-provider.md) for the default internal system user, used for internal process communication. If no password is specified, the default internal system user will not be created. If the default internal system user already exists, setting this property will not affect its password.|null|No| -|`druid.auth.authenticator.MyBasicLDAPAuthenticator.enableCacheNotifications`|If true, the Coordinator will notify Druid processes whenever a configuration change to this Authenticator occurs, allowing them to immediately update their state without waiting for polling.|true|No| -|`druid.auth.authenticator.MyBasicLDAPAuthenticator.cacheNotificationTimeout`|The timeout in milliseconds for the cache notifications.|5000|No| -|`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialIterations`|Number of iterations to use for password hashing.|10000|No| -|`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.type`|The type of credentials store (ldap) to validate requests credentials.|metadata|No| -|`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.url`|URL of the LDAP server.|null|Yes| -|`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.bindUser`|LDAP bind user username.|null|Yes| -|`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.bindPassword`|[Password Provider](../../operations/password-provider.md) LDAP bind user password.|null|Yes| -|`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.baseDn`|The point from where the LDAP server will search for users.|null|Yes| -|`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.userSearch`|The filter/expression to use for the search. For example, (&(sAMAccountName=%s)(objectClass=user))|null|Yes| -|`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.userAttribute`|The attribute id identifying the attribute that will be returned as part of the search. For example, sAMAccountName. |null|Yes| -|`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.credentialVerifyDuration`|The duration in seconds for how long valid credentials are verifiable within the cache when not requested.|600|No| -|`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.credentialMaxDuration`|The max duration in seconds for valid credentials that can reside in cache regardless of how often they are requested.|3600|No| -|`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.credentialCacheSize`|The valid credentials cache size. The cache uses a LRU policy.|100|No| -|`druid.auth.authenticator.MyBasicLDAPAuthenticator.skipOnFailure`|If true and the request credential doesn't exists or isn't fully configured in the credentials store, the request will proceed to next Authenticator in the chain.|false|No| -|`druid.auth.authenticator.MyBasicLDAPAuthenticator.authorizerName`|Authorizer that requests should be directed to.|N/A|Yes| +**`druid.auth.authenticator.MyBasicLDAPAuthenticator.initialAdminPassword`** + +Initial [Password Provider](../../operations/password-provider.md) for the automatically created default admin user. If no password is specified, the default admin user will not be created. If the default admin user already exists, setting this property will not affect its password.
+         **Required**: No
+         **Default**: null + +**`druid.auth.authenticator.MyBasicLDAPAuthenticator.initialInternalClientPassword`** + +Initial [Password Provider](../../operations/password-provider.md) for the default internal system user, used for internal process communication. If no password is specified, the default internal system user will not be created. If the default internal system user already exists, setting this property will not affect its password.
+         **Required**: No
+         **Default**: null + +**`druid.auth.authenticator.MyBasicLDAPAuthenticator.enableCacheNotifications`** + +If true, the Coordinator will notify Druid processes whenever a configuration change to this Authenticator occurs, allowing them to immediately update their state without waiting for polling.
+         **Required**: No
+         **Default**: true + +**`druid.auth.authenticator.MyBasicLDAPAuthenticator.cacheNotificationTimeout`** + +The timeout in milliseconds for the cache notifications.
+         **Required**: No
+         **Default**: 5000 + +**`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialIterations`** + +Number of iterations to use for password hashing.
+         **Required**: No
+         **Default**: 10000 + +**`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.type`** + +The type of credentials store (ldap) to validate requests credentials.
+         **Required**: No
+         **Default**: metadata + +**`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.url`** + +URL of the LDAP server.
+         **Required**: Yes
+         **Default**: null + +**`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.bindUser`** + +LDAP bind user username.
+         **Required**: Yes
+         **Default**: null + +**`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.bindPassword`** + +[Password Provider](../../operations/password-provider.md) LDAP bind user password.
+         **Required**: Yes
+         **Default**: null + +**`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.baseDn`** + +The point from where the LDAP server will search for users.
+         **Required**: Yes
+         **Default**: null + +**`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.userSearch`** + +The filter/expression to use for the search. For example, (&(sAMAccountName=%s)(objectClass=user))
+         **Required**: Yes
+         **Default**: null + +**`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.userAttribute`** + +The attribute id identifying the attribute that will be returned as part of the search. For example, sAMAccountName.
+         **Required**: Yes
+         **Default**: null + +**`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.credentialVerifyDuration`** + +The duration in seconds for how long valid credentials are verifiable within the cache when not requested.
+         **Required**: No
+         **Default**: 600 + +**`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.credentialMaxDuration`** + +The max duration in seconds for valid credentials that can reside in cache regardless of how often they are requested.
+         **Required**: No
+         **Default**: 3600 + +**`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.credentialCacheSize`** + +The valid credentials cache size. The cache uses a LRU policy.
+         **Required**: No
+         **Default**: 100 + +**`druid.auth.authenticator.MyBasicLDAPAuthenticator.skipOnFailure`** + +If true and the request credential doesn't exists or isn't fully configured in the credentials store, the request will proceed to next Authenticator in the chain.
+         **Required**: No
+         **Default**: false + +**`druid.auth.authenticator.MyBasicLDAPAuthenticator.authorizerName`** + +Authorizer that requests should be directed to.
+         **Required**: Yes
+         **Default**: N/A ### Escalator @@ -155,11 +293,24 @@ druid.escalator.authorizerName=MyBasicMetadataAuthorizer ``` #### Properties -|Property|Description|Default|required| -|--------|-----------|-------|--------| -|`druid.escalator.internalClientUsername`|The escalator will use this username for requests made as the internal system user.|n/a|Yes| -|`druid.escalator.internalClientPassword`|The escalator will use this [Password Provider](../../operations/password-provider.md) for requests made as the internal system user.|n/a|Yes| -|`druid.escalator.authorizerName`|Authorizer that requests should be directed to.|n/a|Yes| + +**`druid.escalator.internalClientUsername`** + +The escalator will use this username for requests made as the internal system user.
+         **Required**: Yes
+         **Default**: N/A + +**`druid.escalator.internalClientPassword`** + +The escalator will use this [Password Provider](../../operations/password-provider.md) for requests made as the internal system user.
+         **Required**: Yes
+         **Default**: N/A + +**`druid.escalator.authorizerName`** + +Authorizer that requests should be directed to.
+         **Required**: Yes
+         **Default**: N/A ### Authorizer @@ -182,24 +333,131 @@ druid.auth.authorizer.MyBasicMetadataAuthorizer.type=basic The examples in the rest of this article use `MyBasicMetadataAuthorizer` or `MyBasicLDAPAuthorizer` as the authorizer name. #### Properties for Druid metadata store user authorization -|Property|Description|Default|required| -|--------|-----------|-------|--------| -|`druid.auth.authorizer.MyBasicMetadataAuthorizer.enableCacheNotifications`|If true, the Coordinator will notify Druid processes whenever a configuration change to this Authorizer occurs, allowing them to immediately update their state without waiting for polling.|true|No| -|`druid.auth.authorizer.MyBasicMetadataAuthorizer.cacheNotificationTimeout`|The timeout in milliseconds for the cache notifications.|5000|No| -|`druid.auth.authorizer.MyBasicMetadataAuthorizer.initialAdminUser`|The initial admin user with role defined in initialAdminRole property if specified, otherwise the default admin role will be assigned.|admin|No| -|`druid.auth.authorizer.MyBasicMetadataAuthorizer.initialAdminRole`|The initial admin role to create if it doesn't already exists.|admin|No| -|`druid.auth.authorizer.MyBasicMetadataAuthorizer.roleProvider.type`|The type of role provider to authorize requests credentials.|metadata|No + +**`druid.auth.authorizer.MyBasicMetadataAuthorizer.enableCacheNotifications`** + +If true, the Coordinator will notify Druid processes whenever a configuration change to this Authorizer occurs, allowing them to immediately update their state without waiting for polling.
+         **Required**: No
+         **Default**: true + +**`druid.auth.authorizer.MyBasicMetadataAuthorizer.cacheNotificationTimeout`** + +The timeout in milliseconds for the cache notifications.
+         **Required**: No
+         **Default**: 5000 + +**`druid.auth.authorizer.MyBasicMetadataAuthorizer.initialAdminUser`** + +The initial admin user with role defined in initialAdminRole property if specified, otherwise the default admin role will be assigned.
+         **Required**: No
+         **Default**: admin + +**`druid.auth.authorizer.MyBasicMetadataAuthorizer.initialAdminRole`** + +The initial admin role to create if it doesn't already exists.
+         **Required**: No
+         **Default**: admin + +**`druid.auth.authorizer.MyBasicMetadataAuthorizer.roleProvider.type`** + +The type of role provider to authorize requests credentials.
+         **Required**: No
+         **Default**: metadata #### Properties for LDAP user authorization -|Property|Description|Default|required| -|--------|-----------|-------|--------| -|`druid.auth.authorizer.MyBasicLDAPAuthorizer.enableCacheNotifications`|If true, the Coordinator will notify Druid processes whenever a configuration change to this Authorizer occurs, allowing them to immediately update their state without waiting for polling.|true|No| -|`druid.auth.authorizer.MyBasicLDAPAuthorizer.cacheNotificationTimeout`|The timeout in milliseconds for the cache notifications.|5000|No| -|`druid.auth.authorizer.MyBasicLDAPAuthorizer.initialAdminUser`|The initial admin user with role defined in initialAdminRole property if specified, otherwise the default admin role will be assigned.|admin|No| -|`druid.auth.authorizer.MyBasicLDAPAuthorizer.initialAdminRole`|The initial admin role to create if it doesn't already exists.|admin|No| -|`druid.auth.authorizer.MyBasicLDAPAuthorizer.initialAdminGroupMapping`|The initial admin group mapping with role defined in initialAdminRole property if specified, otherwise the default admin role will be assigned. The name of this initial admin group mapping will be set to adminGroupMapping|null|No| -|`druid.auth.authorizer.MyBasicLDAPAuthorizer.roleProvider.type`|The type of role provider (ldap) to authorize requests credentials.|metadata|No -|`druid.auth.authorizer.MyBasicLDAPAuthorizer.roleProvider.groupFilters`|Array of LDAP group filters used to filter out the allowed set of groups returned from LDAP search. Filters can be begin with *, or end with ,* to provide configurational flexibility to limit or filter allowed set of groups available to LDAP Authorizer.|null|No| + +**`druid.auth.authorizer.MyBasicLDAPAuthorizer.enableCacheNotifications`** + +If true, the Coordinator will notify Druid processes whenever a configuration change to this Authorizer occurs, allowing them to immediately update their state without waiting for polling.
+         **Required**: No
+         **Default**: true + +**`druid.auth.authorizer.MyBasicLDAPAuthorizer.cacheNotificationTimeout`** + +The timeout in milliseconds for the cache notifications.
+         **Required**: No
+         **Default**: 5000 + +**`druid.auth.authorizer.MyBasicLDAPAuthorizer.initialAdminUser`** + +The initial admin user with role defined in initialAdminRole property if specified, otherwise the default admin role will be assigned.
+         **Required**: No
+         **Default**: admin + +**`druid.auth.authorizer.MyBasicLDAPAuthorizer.initialAdminRole`** + +The initial admin role to create if it doesn't already exists.
+         **Required**: No
+         **Default**: admin + +**`druid.auth.authorizer.MyBasicLDAPAuthorizer.initialAdminGroupMapping`** + +The initial admin group mapping with role defined in initialAdminRole property if specified, otherwise the default admin role will be assigned. The name of this initial admin group mapping will be set to adminGroupMapping
+         **Required**: No
+         **Default**: null + +**`druid.auth.authorizer.MyBasicLDAPAuthorizer.roleProvider.type`** + +The type of role provider (ldap) to authorize requests credentials.
+         **Required**: No
+         **Default**: metadata + +**`druid.auth.authorizer.MyBasicLDAPAuthorizer.roleProvider.groupFilters`** + +Array of LDAP group filters used to filter out the allowed set of groups returned from LDAP search. Filters can be begin with *, or end with ,* to provide configurational flexibility to limit or filter allowed set of groups available to LDAP Authorizer.
+         **Required**: No
+         **Default**: null + +#### Properties for LDAPS + +Use the following properties to configure Druid authentication with LDAP over TLS (LDAPS). See [Configure LDAP authentication](../../operations/auth-ldap.md) for more information. + +**`druid.auth.basic.ssl.protocol`** + +SSL protocol to use. The TLS version is 1.2.
+         **Required**: Yes
+         **Default**: tls + +**`druid.auth.basic.ssl.trustStorePath`** + +Path to the trust store file.
+         **Required**: Yes
+         **Default**: N/A + +**`druid.auth.basic.ssl.trustStorePassword`** + +Password to access the trust store file.
+         **Required**: Yes
+         **Default**: N/A + +**`druid.auth.basic.ssl.trustStoreType`** + +Format of the trust store file. For Java the format is jks.
+         **Required**: No
+         **Default**: jks + +**`druid.auth.basic.ssl.trustStoreAlgorithm`** + +Algorithm used by the trust manager to validate certificate chains.
+         **Required**: No
+         **Default**: N/A + +**`druid.auth.basic.ssl.trustStorePassword`** + +Password details that enable access to the truststore.
+         **Required**: No
+         **Default**: N/A + +Example LDAPS configuration: + +```json +druid.auth.basic.ssl.protocol=tls +druid.auth.basic.ssl.trustStorePath=/usr/local/druid-path/certs/truststore.jks +druid.auth.basic.ssl.trustStorePassword=xxxxx +druid.auth.basic.ssl.trustStoreType=jks +druid.auth.basic.ssl.trustStoreAlgorithm=PKIX +``` +You can configure `druid.auth.basic.ssl.trustStorePassword` to be a plain text password or you can set the password as an environment variable. See [Password providers](../../operations/password-provider.md) for more information. ## Usage @@ -213,19 +471,19 @@ Root path: `/druid-ext/basic-security/authentication` Each API endpoint includes {authenticatorName}, specifying which Authenticator instance is being configured. ##### User/Credential Management -`GET(/druid-ext/basic-security/authentication/db/{authenticatorName}/users)` +`GET(/druid-ext/basic-security/authentication/db/{authenticatorName}/users)`
Return a list of all user names. -`GET(/druid-ext/basic-security/authentication/db/{authenticatorName}/users/{userName})` +`GET(/druid-ext/basic-security/authentication/db/{authenticatorName}/users/{userName})`
Return the name and credentials information of the user with name {userName} -`POST(/druid-ext/basic-security/authentication/db/{authenticatorName}/users/{userName})` +`POST(/druid-ext/basic-security/authentication/db/{authenticatorName}/users/{userName})`
Create a new user with name {userName} -`DELETE(/druid-ext/basic-security/authentication/db/{authenticatorName}/users/{userName})` +`DELETE(/druid-ext/basic-security/authentication/db/{authenticatorName}/users/{userName})`
Delete the user with name {userName} -`POST(/druid-ext/basic-security/authentication/db/{authenticatorName}/users/{userName}/credentials)` +`POST(/druid-ext/basic-security/authentication/db/{authenticatorName}/users/{userName}/credentials)`
Assign a password used for HTTP basic authentication for {userName} Content: JSON password request object @@ -238,20 +496,20 @@ Example request body: ``` ##### Cache Load Status -`GET(/druid-ext/basic-security/authentication/loadStatus)` +`GET(/druid-ext/basic-security/authentication/loadStatus)`
Return the current load status of the local caches of the authentication Druid metadata store. #### Authorization API -Root path: `/druid-ext/basic-security/authorization` +Root path: `/druid-ext/basic-security/authorization`
Each API endpoint includes {authorizerName}, specifying which Authorizer instance is being configured. ##### User Creation/Deletion -`GET(/druid-ext/basic-security/authorization/db/{authorizerName}/users)` +`GET(/druid-ext/basic-security/authorization/db/{authorizerName}/users)`
Return a list of all user names. -`GET(/druid-ext/basic-security/authorization/db/{authorizerName}/users/{userName})` +`GET(/druid-ext/basic-security/authorization/db/{authorizerName}/users/{userName})`
Return the name and role information of the user with name {userName} Example output: @@ -338,20 +596,20 @@ The `resourceNamePattern` is a compiled version of the resource name regex. It i } ``` -`POST(/druid-ext/basic-security/authorization/db/{authorizerName}/users/{userName})` +`POST(/druid-ext/basic-security/authorization/db/{authorizerName}/users/{userName})`
Create a new user with name {userName} -`DELETE(/druid-ext/basic-security/authorization/db/{authorizerName}/users/{userName})` +`DELETE(/druid-ext/basic-security/authorization/db/{authorizerName}/users/{userName})`
Delete the user with name {userName} ##### Group mapping Creation/Deletion -`GET(/druid-ext/basic-security/authorization/db/{authorizerName}/groupMappings)` +`GET(/druid-ext/basic-security/authorization/db/{authorizerName}/groupMappings)`
Return a list of all group mappings. -`GET(/druid-ext/basic-security/authorization/db/{authorizerName}/groupMappings/{groupMappingName})` +`GET(/druid-ext/basic-security/authorization/db/{authorizerName}/groupMappings/{groupMappingName})`
Return the group mapping and role information of the group mapping with name {groupMappingName} -`POST(/druid-ext/basic-security/authorization/db/{authorizerName}/groupMappings/{groupMappingName})` +`POST(/druid-ext/basic-security/authorization/db/{authorizerName}/groupMappings/{groupMappingName})`
Create a new group mapping with name {groupMappingName} Content: JSON group mapping object Example request body: @@ -366,14 +624,14 @@ Example request body: } ``` -`DELETE(/druid-ext/basic-security/authorization/db/{authorizerName}/groupMappings/{groupMappingName})` +`DELETE(/druid-ext/basic-security/authorization/db/{authorizerName}/groupMappings/{groupMappingName})`
Delete the group mapping with name {groupMappingName} #### Role Creation/Deletion -`GET(/druid-ext/basic-security/authorization/db/{authorizerName}/roles)` +`GET(/druid-ext/basic-security/authorization/db/{authorizerName}/roles)`
Return a list of all role names. -`GET(/druid-ext/basic-security/authorization/db/{authorizerName}/roles/{roleName})` +`GET(/druid-ext/basic-security/authorization/db/{authorizerName}/roles/{roleName})`
Return name and permissions for the role named {roleName}. Example output: @@ -427,30 +685,30 @@ Example output: ``` -`POST(/druid-ext/basic-security/authorization/db/{authorizerName}/roles/{roleName})` +`POST(/druid-ext/basic-security/authorization/db/{authorizerName}/roles/{roleName})`
Create a new role with name {roleName}. Content: username string -`DELETE(/druid-ext/basic-security/authorization/db/{authorizerName}/roles/{roleName})` +`DELETE(/druid-ext/basic-security/authorization/db/{authorizerName}/roles/{roleName})`
Delete the role with name {roleName}. #### Role Assignment -`POST(/druid-ext/basic-security/authorization/db/{authorizerName}/users/{userName}/roles/{roleName})` +`POST(/druid-ext/basic-security/authorization/db/{authorizerName}/users/{userName}/roles/{roleName})`
Assign role {roleName} to user {userName}. -`DELETE(/druid-ext/basic-security/authorization/db/{authorizerName}/users/{userName}/roles/{roleName})` +`DELETE(/druid-ext/basic-security/authorization/db/{authorizerName}/users/{userName}/roles/{roleName})`
Unassign role {roleName} from user {userName} -`POST(/druid-ext/basic-security/authorization/db/{authorizerName}/groupMappings/{groupMappingName}/roles/{roleName})` +`POST(/druid-ext/basic-security/authorization/db/{authorizerName}/groupMappings/{groupMappingName}/roles/{roleName})`
Assign role {roleName} to group mapping {groupMappingName}. -`DELETE(/druid-ext/basic-security/authorization/db/{authorizerName}/groupMappings/{groupMappingName}/roles/{roleName})` +`DELETE(/druid-ext/basic-security/authorization/db/{authorizerName}/groupMappings/{groupMappingName}/roles/{roleName})`
Unassign role {roleName} from group mapping {groupMappingName} #### Permissions -`POST(/druid-ext/basic-security/authorization/db/{authorizerName}/roles/{roleName}/permissions)` +`POST(/druid-ext/basic-security/authorization/db/{authorizerName}/roles/{roleName}/permissions)`
Set the permissions of {roleName}. This replaces the previous set of permissions on the role. Content: List of JSON Resource-Action objects, e.g.: @@ -479,5 +737,5 @@ The "name" field for resources in the permission definitions are regexes used to Please see [Defining permissions](../../operations/security-user-auth.md#defining-permissions) for more details. ##### Cache Load Status -`GET(/druid-ext/basic-security/authorization/loadStatus)` +`GET(/druid-ext/basic-security/authorization/loadStatus)`
Return the current load status of the local caches of the authorization Druid metadata store. diff --git a/docs/operations/auth-ldap.md b/docs/operations/auth-ldap.md index b9603a0c38fd..5a2d8b7e292f 100644 --- a/docs/operations/auth-ldap.md +++ b/docs/operations/auth-ldap.md @@ -1,6 +1,7 @@ --- id: auth-ldap -title: "LDAP auth" +title: "Configure LDAP authentication" +sidebar_label: "LDAP auth" --- +You can use [Lightweight Directory Access Protocol (LDAP)](https://en.wikipedia.org/wiki/Lightweight_Directory_Access_Protocol) to secure access to Apache Druid. This topic describes how to set up Druid authentication and authorization with LDAP and LDAP over TLS (LDAPS). The examples on this page show the configuration for an Active Directory LDAP system. -This page describes how to set up Druid user authentication and authorization through LDAP. The first step is to enable LDAP authentication and authorization for Druid. You then map an LDAP group to roles and assign permissions to roles. +The first step is to enable LDAP authentication and authorization for Druid. You then map an LDAP group to Druid roles and assign permissions to those roles. After you've completed this configuration you can optionally choose to enable LDAPS to make LDAP traffic confidential and secure. -## Enable LDAP in Druid +## Prerequisites -Before starting, verify that the active directory is reachable from the Druid Master servers. Command line tools such as `ldapsearch` and `ldapwhoami`, which are included with OpenLDAP, are useful for this testing.  +Before you start to configure LDAP for Druid, test your LDAP connection and perform a sample search. -### Check the connection +### Check your LDAP connection -First test that the basic connection and user credential works. For example, given a user `uuser1@example.com`, try: +Test your LDAP connection to verify it works with user credentials. Later in the process you [configure Druid for LDAP authentication](#configure-druid-for-ldap-authentication) with this user as the `bindUser`. + +The following example command tests the connection for the user `myuser@example.com`. Insert your LDAP server IP address. Modify the port number of your LDAP instance if it listens on a port other than `389`. ```bash -ldapwhoami -vv -H ldap://:389  -D"uuser1@example.com" -W +ldapwhoami -vv -H ldap://ip_address:389 -D "myuser@example.com" -W ``` -Enter the password associated with the user when prompted and verify that the command succeeded. If it didn't, try the following troubleshooting steps: +Enter the password for the user when prompted and verify that the command succeeded. If it failed, check the following: + +- Make sure you're using the correct port for your LDAP instance. +- Check if a network firewall is preventing connections to the LDAP port. +- Review your LDAP implementation details to see whether you need to specifically allow LDAP clients at the LDAP server. If so, add the Druid Coordinator server to the allow list. -* Verify that you've used the correct port for your LDAP instance. By default, the LDAP port is 389, but double-check with your LDAP admin if unable to connect. -* Check whether a network firewall is not preventing connections to the LDAP port. -* Check whether LDAP clients need to be specifically whitelisted at the LDAP server to be able to reach it. If so, add the Druid Coordinator server to the AD whitelist. +### Test your LDAP search +Once your LDAP connection is working, search for a user. For example, the following command searches for the user `myuser` in an Active Directory system. The `sAMAccountName` attribute is specific to Active Directory and contains the authenticated user identity: -### Check the search criteria +```bash +ldapsearch -x -W -H ldap://ip_address:389 -D "cn=admin,dc=example,dc=com" -b "dc=example,dc=com" "(sAMAccountName=myuser)" + +``` -After verifying basic connectivity, check your search criteria. For example, the command for searching for user `uuser1@example.com ` is as follows: +The `memberOf` attribute in the results shows the groups the user belongs to. For example, the following response shows that the user is a member of the `mygroup` group: ```bash -ldapsearch -x -W -H ldap://  -D"uuser1@example.com" -b "dc=example,dc=com" "(sAMAccountName=uuser1)" +memberOf: cn=mygroup,ou=groups,dc=example,dc=com ``` -Note the `memberOf` attribute in the results; it shows the groups that the user belongs to. You will use this value to map the LDAP group to the Druid roles later. This attribute may be implemented differently on different types of LDAP servers. For instance, some LDAP servers may support recursive groupings, and some may not. Some LDAP server implementations may not have any object classes that contain this attribute altogether. If your LDAP server does not use the `memberOf` attribute, then Druid will not be able to determine a user's group membership using LDAP. The sAMAccountName attribute used in this example contains the authenticated user identity. This is an attribute of an object class specific to Microsoft Active Directory. The object classes and attribute used in your LDAP server may be different. +You use this information to map the LDAP group to Druid roles in a later step. + +> Druid uses the `memberOf` attribute to determine a group's membership using LDAP. If your LDAP server implementation doesn't include this attribute, you must complete some additional steps when you [map LDAP groups to Druid roles](#map-ldap-groups-to-druid-roles). + +## Configure Druid for LDAP authentication + +To configure Druid to use LDAP authentication, follow these steps. See [Configuration reference](../configuration/index.md) for the location of the configuration files. + +1. Create a user in your LDAP system that you'll use both for internal communication with Druid and as the LDAP initial admin user. See [Security overview](./security-overview.md) for more information. +In the example below, the LDAP user is `internal@example.com`. -## Configure Druid user authentication with LDAP/Active Directory  +2. Enable the `druid-basic-security` extension in the `common.runtime.properties` file. -1. Enable the `druid-basic-security` extension in the `common.runtime.properties` file. See [Security Overview](security-overview.md) for details. -2. As a best practice, create a user in LDAP to be used for internal communication with Druid. -3. In `common.runtime.properties`, update LDAP-related properties, as shown in the following listing:  - ``` - druid.auth.authenticatorChain=["ldap"] - druid.auth.authenticator.ldap.type=basic - druid.auth.authenticator.ldap.enableCacheNotifications=true - druid.auth.authenticator.ldap.credentialsValidator.type=ldap - druid.auth.authenticator.ldap.credentialsValidator.url=ldap://: - druid.auth.authenticator.ldap.credentialsValidator.bindUser= - druid.auth.authenticator.ldap.credentialsValidator.bindPassword= - druid.auth.authenticator.ldap.credentialsValidator.baseDn= - druid.auth.authenticator.ldap.credentialsValidator.userSearch= - druid.auth.authenticator.ldap.credentialsValidator.userAttribute=sAMAccountName - druid.auth.authenticator.ldap.authorizerName=ldapauth - druid.escalator.type=basic - druid.escalator.internalClientUsername= - druid.escalator.internalClientPassword=Welcome123 - druid.escalator.authorizerName=ldapauth - druid.auth.authorizers=["ldapauth"] - druid.auth.authorizer.ldapauth.type=basic - druid.auth.authorizer.ldapauth.initialAdminUser=AD user who acts as the initial admin user, e.g.: internal@example.com> - druid.auth.authorizer.ldapauth.initialAdminRole=admin - druid.auth.authorizer.ldapauth.roleProvider.type=ldap +3. In the `common.runtime.properties` file, add the following lines for LDAP properties and substitute the values for your own. See [Druid basic security](../development/extensions-core/druid-basic-security.md#properties-for-ldap-user-authentication) for details about these properties. + + ``` + druid.auth.authenticatorChain=["ldap"] + druid.auth.authenticator.ldap.type=basic + druid.auth.authenticator.ldap.enableCacheNotifications=true + druid.auth.authenticator.ldap.credentialsValidator.type=ldap + druid.auth.authenticator.ldap.credentialsValidator.url=ldap://ip_address:port + druid.auth.authenticator.ldap.credentialsValidator.bindUser=administrator@example.com + druid.auth.authenticator.ldap.credentialsValidator.bindPassword=adminpassword + druid.auth.authenticator.ldap.credentialsValidator.baseDn=dc=example,dc=com + druid.auth.authenticator.ldap.credentialsValidator.userSearch=(&(sAMAccountName=%s)(objectClass=user)) + druid.auth.authenticator.ldap.credentialsValidator.userAttribute=sAMAccountName + druid.auth.authenticator.ldap.authorizerName=ldapauth + druid.escalator.type=basic + druid.escalator.internalClientUsername=internal@example.com + druid.escalator.internalClientPassword=internaluserpassword + druid.escalator.authorizerName=ldapauth + druid.auth.authorizers=["ldapauth"] + druid.auth.authorizer.ldapauth.type=basic + druid.auth.authorizer.ldapauth.initialAdminUser=internal@example.com + druid.auth.authorizer.ldapauth.initialAdminRole=admin + druid.auth.authorizer.ldapauth.roleProvider.type=ldap ``` + Note the following: - Notice that the LDAP user created in the previous step, `internal@example.com`, serves as the internal client user and the initial admin user. + - `bindUser`: A user for connecting to LDAP. This should be the same user you used to [test your LDAP search](#test-your-ldap-search). + - `userSearch`: Your LDAP search syntax. + - `userAttribute`: The user search attribute. + - `internal@example.com` is the LDAP user you created in step 1. In the example it serves as both the internal client user and the initial admin user. -## Use LDAP groups to assign roles + > In the above example, the [Druid escalator](../development/extensions-core/druid-basic-security.md#escalator) and LDAP initial admin user are set to the same user - `internal@example.com`. If the escalator is set to a different user, you must follow steps 4 and 5 to create the group mapping and allocate initial roles before the rest of the cluster can function. -You can map LDAP groups to a role in Druid. Members in the group get access to the permissions of the corresponding role.  +4. Save your group mapping to a JSON file. An example file `groupmap.json` looks like this: + + ``` + { + "name": "mygroupmap", + "groupPattern": "CN=mygroup,CN=Users,DC=example,DC=com", + "roles": [ + "readRole" + ] + } + ``` + In the example, the LDAP group `mygroup` maps to Druid role `readRole` and the name of the mapping is `mygroupmap`. +5. Use the Druid API to create the group mapping and allocate initial roles according to your JSON file. The following example uses curl to create the mapping defined in `groupmap.json` for the LDAP group `mygroup`: + + ``` + curl -i -v -H "Content-Type: application/json" -u internal -X POST -d @groupmap.json http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/groupMappings/mygroupmap + ``` +6. Check that the group mapping was created successfully. The following example request lists all group mappings: -### Step 1: Create a role + ``` + curl -i -v -H "Content-Type: application/json" -u internal -X GET http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/groupMappings + ``` -First create the role in Druid using the Druid REST API. +## Map LDAP groups to Druid roles -Creating a role involves submitting a POST request to the Coordinator process.  +Once you've completed the initial setup and mapping, you can map more LDAP groups to Druid roles. Members of an LDAP group get access to the permissions of the corresponding Druid role. -The following REST APIs to create the role to read access for datasource, config, state. +### Create a Druid role -> As mentioned, the REST API calls need to address the Coordinator node. The examples used below use localhost as the Coordinator host and 8081 as the port. Adjust these settings according to your deployment. +To create a Druid role, you can submit a POST request to the Coordinator process using the Druid REST API or you can use the Druid console. -Call the following API to create role `readRole` .  +The examples below use `localhost` as the Coordinator host and `8081` as the port. Amend these properties according to the details of your deployment. + +Example request to create a role named `readRole`: ``` -curl -i -v  -H "Content-Type: application/json" -u internal -X POST http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/roles/readRole +curl -i -v -H "Content-Type: application/json" -u internal -X POST http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/roles/readRole ``` -Check that the role has been created successfully by entering the following: +Check that Druid created the role successfully. The following example request lists all roles: ``` -curl -i -v  -H "Content-Type: application/json" -u internal -X GET  http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/roles +curl -i -v -H "Content-Type: application/json" -u internal -X GET http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/roles ``` +### Add permissions to the Druid role -### Step 2: Add permissions to a role  - -You can now add one or more permission to the role. The following example adds read-only access to a `wikipedia` data source.   +Once you have a Druid role you can add permissions to it. The following example adds read-only access to a `wikipedia` data source. Given the following JSON in a file named `perm.json`: ``` -[{ "resource": { "name": "wikipedia", "type": "DATASOURCE" }, "action": "READ" } -,{ "resource": { "name": ".*", "type": "STATE" }, "action": "READ" }, -{ "resource": {"name": ".*", "type": "CONFIG"}, "action": "READ"}] -``` - -The following command associates the permissions in the JSON file with the role - -``` -curl -i -v  -H "Content-Type: application/json" -u internal -X POST -d@perm.json  http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/roles/readRole/permissions +[ + { "resource": { "name": "wikipedia", "type": "DATASOURCE" }, "action": "READ" }, + { "resource": { "name": ".*", "type": "STATE" }, "action": "READ" }, + { "resource": {"name": ".*", "type": "CONFIG"}, "action": "READ"} +] ``` -Note that the STATE and CONFIG permissions in `perm.json` are needed to see the data source in the web console. If only querying permissions are needed, the READ action is sufficient: +The following request associates the permissions in the JSON file with the `readRole` role: ``` -[{ "resource": { "name": "wikipedia", "type": "DATASOURCE" }, "action": "READ" }] +curl -i -v -H "Content-Type: application/json" -u internal -X POST -d@perm.json http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/roles/readRole/permissions ``` -You can also provide the name in the form of regular expression. For example, to give access to all data sources starting with `wiki`, specify the name as  `{ "name": "wiki.*", .....`. +Druid users need the `STATE` and `CONFIG` permissions to view the data source in the Druid console. If you only want to assign querying permissions you can apply just the `READ` permission with the first line in the `perm.json` file. +You can also provide the data source name in the form of a regular expression. For example, to give access to all data sources starting with `wiki`, you would specify the data source name as `{ "name": "wiki.*" }` . -### Step 3: Create group Mapping  +### Create the group mapping -The following shows an example of a group to role mapping. It assumes that a group named `group1` exists in the directory. Also assuming the following role mapping in a file named `groupmap.json`: +You can now map an LDAP group to the Druid role. The following example request creates a mapping with name `mygroupmap`. It assumes that a group named `mygroup` exists in the directory. ``` { -    "name": "group1map", -    "groupPattern": "CN=group1,CN=Users,DC=example,DC=com", -    "roles": [ -        "readRole" -    ] + "name": "mygroupmap", + "groupPattern": "CN=mygroup,CN=Users,DC=example,DC=com", + "roles": [ + "readRole" + ] } ``` -You can configure the mapping as follows: +The following example request configures the mapping—the role mapping is in the file `groupmap.json`. See [Configure Druid for LDAP authentication](#configure-druid-for-ldap-authentication) for the contents of an example file. ``` -curl -i -v  -H "Content-Type: application/json" -u internal -X POST -d @groupmap.json http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/groupMappings/group1map +curl -i -v -H "Content-Type: application/json" -u internal -X POST -d @groupmap.json http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/groupMappings/mygroupmap ``` -To check whether the group mapping was created successfully, run the following command: +To check whether the group mapping was created successfully, the following request lists all group mappings: ``` -curl -i -v  -H "Content-Type: application/json" -u internal -X GET http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/groupMappings +curl -i -v -H "Content-Type: application/json" -u internal -X GET http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/groupMappings ``` -To check the details of a specific group mapping, use the following: +The following example request returns the details of the `mygroupmap` group: ``` -curl -i -v  -H "Content-Type: application/json" -u internal -X GET http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/groupMappings/group1map +curl -i -v -H "Content-Type: application/json" -u internal -X GET http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/groupMappings/mygroupmap ``` -To add additional roles to the group mapping, use the following API: +The following example request adds the role `queryRole` to the `mygroupmap` mapping: ``` -curl -i -v  -H "Content-Type: application/json" -u internal -X POST http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/groupMappings/group1/roles/  +curl -i -v -H "Content-Type: application/json" -u internal -X POST http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/groupMappings/mygroup/roles/queryrole ``` -In the next two steps you will be creating a user, and assigning previously created roles to it. These steps are only needed in the following cases: - - - Your LDAP server does not support the `memberOf` attribute, or - - You want to configure a user with additional roles that are not mapped to the group(s) that the user is a member of - - If this is not the case for your scenario, you can skip these steps. +### Add an LDAP user to Druid and assign a role -### Step 4. Create a user +You only need to complete this step if: +- Your LDAP user doesn't belong to any of your LDAP groups, or +- You want to configure a user with additional Druid roles that are not mapped to the LDAP groups that the user belongs to. -Once LDAP is enabled, only user passwords are verified with LDAP. You add the LDAP user to Druid as follows:  +Example request to add the LDAP user `myuser` to Druid: ``` -curl -i -v  -H "Content-Type: application/json" -u internal -X POST http://localhost:8081/druid-ext/basic-security/authentication/db/ldap/users/  +curl -i -v -H "Content-Type: application/json" -u internal -X POST http://localhost:8081/druid-ext/basic-security/authentication/db/ldap/users/myuser ``` -### Step 5. Assign the role to the user - -The following command shows how to assign a role to a user: +Example request to assign the `myuser` user to the `queryRole` role: ``` -curl -i -v  -H "Content-Type: application/json" -u internal -X POST http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/users//roles/  +curl -i -v -H "Content-Type: application/json" -u internal -X POST http://localhost:8081/druid-ext/basic-security/authentication/db/ldap/users/myuser/roles/queryRole ``` -For more information about security and the basic security extension, see [Security Overview](security-overview.md). +## Enable LDAP over TLS (LDAPS) + +Once you've configured LDAP authentication in Druid, you can optionally make LDAP traffic confidential and secure by using Transport Layer Security (TLS)—previously Secure Socket Layer(SSL)—technology. + +Configuring LDAPS establishes trust between Druid and the LDAP server. + +## Prerequisites + +Before you start to set up LDAPS in Druid, you must [configure Druid for LDAP authentication](#configure-druid-for-ldap-authentication). You also need: + +- A certificate issued by a public certificate authority (CA) or a self-signed certificate by an internal CA. +- The root certificate for the CA that signed the certificate for the LDAP server. If you're using a common public CA, the certificate may already be in the Java truststore. Otherwise you need to import the certificate for the CA. + +## Configure Druid for LDAPS + +Complete the following steps to set up LDAPS for Druid. See [Configuration reference](../configuration/index.md) for the location of the configuration files. + +1. Import the CA certificate for your LDAP server or a self-signed certificate into the truststore location saved as `druid.client.https.trustStorePath` in your `common.runtime.properties` file. + + ``` + keytool -import -trustcacerts -keystore path/to/cacerts -storepass truststorepassword -alias aliasName -file path/to/certificate.cer + ``` + + Replace `path/to/cacerts` with the path to your truststore, `truststorepassword` with your truststore password, `aliasName` with an alias name for the keystore, and `path/to/certificate.cer` with the location and name of your certificate. For example: + + ``` + keytool -import -trustcacerts -keystore /Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home/jre/lib/security/cacerts -storepass mypassword -alias myAlias -file /etc/ssl/certs/my-certificate.cer + ``` + +2. If the root certificate for the CA isn't already in the Java truststore, import it: + + ``` + keytool -importcert -keystore path/to/cacerts -storepass truststorepassword -alias aliasName -file path/to/certificate.cer + ``` + + Replace `path/to/cacerts` with the path to your truststore, `truststorepassword` with your truststore password, `aliasName` with an alias name for the keystore, and `path/to/certificate.cer` with the location and name of your certificate. For example: + + ``` + keytool -importcert -keystore /Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home/jre/lib/security/cacerts -storepass mypassword -alias myAlias -file /etc/ssl/certs/my-certificate.cer + ``` + +3. In your `common.runtime.properties` file, add the following lines to the LDAP configuration section, substituting your own truststore path and password: + + ``` + druid.auth.basic.ssl.trustStorePath=/Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home/jre/lib/security/cacerts + druid.auth.basic.ssl.protocol=TLS + druid.auth.basic.ssl.trustStorePassword=xxxxxx + ``` + + See [Druid basic security](../development/extensions-core/druid-basic-security.md#properties-for-ldaps) for details about these properties. + +4. You can optionally configure additional LDAPS properties in the `common.runtime.properties` file. See [Druid basic security](../development/extensions-core/druid-basic-security.md#properties-for-ldaps) for more information. + +5. Restart Druid. + + +## Troubleshooting tips + +The following are some ideas to help you troubleshoot issues with LDAP and LDAPS. + +### Check the coordinator logs + +If your LDAP connection isn't working, check the coordinator logs. See [Logging](../configuration/logging.md) for details. + +### Check the Druid escalator configuration + +If the coordinator is working but the rest of the cluster isn't, check the escalator configuration. See the [Configuration reference](../configuration/index.md) for details. You can also check other service logs to see why the services are unable to fetch authorization details from the coordinator. + +### Check your LDAP server response time + +If a user can log in to the Druid console but the landing page shows a 401 error, check your LDAP server response time. In a large organization with a high number of LDAP users, LDAP may be slow to respond, and this can result in a connection timeout. diff --git a/docs/operations/security-overview.md b/docs/operations/security-overview.md index 028cff04f55a..3fa80e24adc2 100644 --- a/docs/operations/security-overview.md +++ b/docs/operations/security-overview.md @@ -250,80 +250,7 @@ The following steps walk through a sample setup procedure: ## Configuring an LDAP authenticator -As an alternative to using the basic metadata authenticator, you can use LDAP to authenticate users. The following steps provide an overview of the setup procedure. For more information on these settings, see [Properties for LDAP user authentication](../development/extensions-core/druid-basic-security.md#properties-for-ldap-user-authentication). - -1. In `common.runtime.properties`, add LDAP to the authenticator chain in the order in which you want requests to be evaluated. For example: - ``` - # Druid basic security - druid.auth.authenticatorChain=["ldap", "MyBasicMetadataAuthenticator"] - ``` - -2. Configure LDAP settings in `common.runtime.properties` as appropriate for your LDAP scheme and system. For example: - ``` - druid.auth.authenticator.ldap.type=basic - druid.auth.authenticator.ldap.enableCacheNotifications=true - druid.auth.authenticator.ldap.credentialsValidator.type=ldap - druid.auth.authenticator.ldap.credentialsValidator.url=ldap://ad_host:389 - druid.auth.authenticator.ldap.credentialsValidator.bindUser=ad_admin_user - druid.auth.authenticator.ldap.credentialsValidator.bindPassword=ad_admin_password - druid.auth.authenticator.ldap.credentialsValidator.baseDn=dc=example,dc=com - druid.auth.authenticator.ldap.credentialsValidator.userSearch=(&(sAMAccountName=%s)(objectClass=user)) - druid.auth.authenticator.ldap.credentialsValidator.userAttribute=sAMAccountName - druid.auth.authenticator.ldap.authorizerName=ldapauth - druid.escalator.type=basic - druid.escalator.internalClientUsername=ad_interal_user - druid.escalator.internalClientPassword=Welcome123 - druid.escalator.authorizerName=ldapauth - druid.auth.authorizers=["ldapauth"] - druid.auth.authorizer.ldapauth.type=basic - druid.auth.authorizer.ldapauth.initialAdminUser= - druid.auth.authorizer.ldapauth.initialAdminRole=admin - druid.auth.authorizer.ldapauth.roleProvider.type=ldap - ``` - -3. Use the Druid API to create the group mapping and allocate initial roles. For example, using curl and given a group named `group1` in the directory, run: - ``` - curl -i -v -H "Content-Type: application/json" -u internal -X POST -d @groupmap.json http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/groupMappings/group1map - ``` - The `groupmap.json` file contents would be something like: - ``` - { - "name": "group1map", - "groupPattern": "CN=group1,CN=Users,DC=example,DC=com", - "roles": [ - "readRole" - ] - } - ``` -4. Check if the group mapping is created successfully by executing the following API. This lists all group mappings. - ``` - curl -i -v -H "Content-Type: application/json" -u internal -X GET http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/groupMappings - ``` - - Alternatively, to check the details of a specific group mapping, use the following API: - ``` - curl -i -v -H "Content-Type: application/json" -u internal -X GET http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/groupMappings/group1map - ``` - -5. To add additional roles to the group mapping, use the following API: - ``` - curl -i -v -H "Content-Type: application/json" -u internal -X POST http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/groupMappings/group1/roles/ - ``` - -6. Add the LDAP user to Druid. To add a user, use the following authentication API: - ``` - curl -i -v -H "Content-Type: application/json" -u internal -X POST http://localhost:8081/druid-ext/basic-security/authentication/db/ldap/users/ - ``` - -7. Use the following command to assign the role to a user: - ``` - curl -i -v -H "Content-Type: application/json" -u internal -X POST http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/users//roles/ - ``` - - - -Congratulations, you have configured permissions for user-assigned roles in Druid! - +As an alternative to using the basic metadata authenticator, you can use LDAP to authenticate users. See [Configure LDAP authentication](./auth-ldap.md) for information on configuring Druid for LDAP and LDAPS. ## Druid security trust model Within Druid's trust model there users can have different authorization levels: diff --git a/website/.spelling b/website/.spelling index c2eb66104ee8..d02f4f846799 100644 --- a/website/.spelling +++ b/website/.spelling @@ -129,6 +129,7 @@ JDK JDK7 JDK8 JKS +jks JMX JRE JS @@ -147,6 +148,7 @@ Kerberos KeyStores Kinesis Kubernetes +LDAPS LRU LZ4 LZO @@ -203,6 +205,7 @@ SYSTEM_TABLE TCP TGT TLS +tls TopN TopNs UI From c04ecde4e85de5279704706f79ca38454070c766 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Mon, 5 Dec 2022 19:34:15 +0530 Subject: [PATCH 20/34] Add SegmentAllocationQueue to batch SegmentAllocateActions (#13369) (#13493) In a cluster with a large number of streaming tasks (~1000), SegmentAllocateActions on the overlord can often take very long intervals of time to finish thus causing spikes in the `task/action/run/time`. This may result in lag building up while a task waits for a segment to get allocated. The root causes are: - large number of metadata calls made to the segments and pending segments tables - `giant` lock held in `TaskLockbox.tryLock()` to acquire task locks and allocate segments Since the contention typically arises when several tasks of the same datasource try to allocate segments for the same interval/granularity, the allocation run times can be improved by batching the requests together. Changes - Add flags - `druid.indexer.tasklock.batchSegmentAllocation` (default `false`) - `druid.indexer.tasklock.batchAllocationMaxWaitTime` (in millis) (default `1000`) - Add methods `canPerformAsync` and `performAsync` to `TaskAction` - Submit each allocate action to a `SegmentAllocationQueue`, and add to correct batch - Process batch after `batchAllocationMaxWaitTime` - Acquire `giant` lock just once per batch in `TaskLockbox` - Reduce metadata calls by batching statements together and updating query filters - Except for batching, retain the whole behaviour (order of steps, retries, etc.) - Respond to leadership changes and fail items in queue when not leader - Emit batch and request level metrics --- .../druid/java/util/common/Intervals.java | 29 + .../druid/java/util/common/IntervalsTest.java | 81 ++ .../druid/indexing/common/TaskToolbox.java | 10 +- .../common/actions/LocalTaskActionClient.java | 20 +- .../common/actions/SegmentAllocateAction.java | 18 + .../actions/SegmentAllocateRequest.java | 77 ++ .../common/actions/SegmentAllocateResult.java | 49 ++ .../actions/SegmentAllocationQueue.java | 716 ++++++++++++++++++ .../indexing/common/actions/TaskAction.java | 12 + .../common/actions/TaskActionToolbox.java | 32 + .../druid/indexing/overlord/TaskLockbox.java | 324 +++++++- .../druid/indexing/overlord/TaskMaster.java | 20 +- .../overlord/config/TaskLockConfig.java | 16 + .../actions/SegmentAllocateActionBuilder.java | 138 ++++ .../actions/SegmentAllocateActionTest.java | 102 ++- .../actions/SegmentAllocationQueueTest.java | 349 +++++++++ .../common/actions/TaskActionTestKit.java | 28 +- .../indexing/overlord/http/OverlordTest.java | 4 +- ...TestIndexerMetadataStorageCoordinator.java | 14 + .../IndexerMetadataStorageCoordinator.java | 22 + .../overlord/SegmentCreateRequest.java | 78 ++ .../IndexerSQLMetadataStorageCoordinator.java | 594 +++++++++++++-- .../overlord/SegmentCreateRequestTest.java | 46 ++ ...exerSQLMetadataStorageCoordinatorTest.java | 269 +++---- 24 files changed, 2752 insertions(+), 296 deletions(-) create mode 100644 core/src/test/java/org/apache/druid/java/util/common/IntervalsTest.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateRequest.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateResult.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionBuilder.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueueTest.java create mode 100644 server/src/main/java/org/apache/druid/indexing/overlord/SegmentCreateRequest.java create mode 100644 server/src/test/java/org/apache/druid/indexing/overlord/SegmentCreateRequestTest.java diff --git a/core/src/main/java/org/apache/druid/java/util/common/Intervals.java b/core/src/main/java/org/apache/druid/java/util/common/Intervals.java index b7a1f37cf1c3..96f858fd4be2 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/Intervals.java +++ b/core/src/main/java/org/apache/druid/java/util/common/Intervals.java @@ -20,10 +20,13 @@ package org.apache.druid.java.util.common; import com.google.common.collect.ImmutableList; +import org.apache.druid.java.util.common.guava.Comparators; import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.chrono.ISOChronology; +import javax.annotation.Nullable; + public final class Intervals { public static final Interval ETERNITY = utc(JodaUtils.MIN_INSTANT, JodaUtils.MAX_INSTANT); @@ -68,6 +71,32 @@ public static boolean isEternity(final Interval interval) return ETERNITY.equals(interval); } + /** + * Finds an interval from the given set of sortedIntervals which overlaps with + * the searchInterval. If multiple candidate intervals overlap with the + * searchInterval, the "smallest" interval based on the + * {@link Comparators#intervalsByStartThenEnd()} is returned. + * + * @param searchInterval Interval which should overlap with the result + * @param sortedIntervals Candidate overlapping intervals, sorted in ascending + * order, using {@link Comparators#intervalsByStartThenEnd()}. + * @return The first overlapping interval, if one exists, otherwise null. + */ + @Nullable + public static Interval findOverlappingInterval(Interval searchInterval, Interval[] sortedIntervals) + { + for (Interval interval : sortedIntervals) { + if (interval.overlaps(searchInterval)) { + return interval; + } else if (interval.getStart().isAfter(searchInterval.getEnd())) { + // Intervals after this cannot have an overlap + return null; + } + } + + return null; + } + private Intervals() { } diff --git a/core/src/test/java/org/apache/druid/java/util/common/IntervalsTest.java b/core/src/test/java/org/apache/druid/java/util/common/IntervalsTest.java new file mode 100644 index 000000000000..59eac8d5a991 --- /dev/null +++ b/core/src/test/java/org/apache/druid/java/util/common/IntervalsTest.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.java.util.common; + +import org.apache.druid.java.util.common.guava.Comparators; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; + +public class IntervalsTest +{ + + @Test + public void testFindOverlappingInterval() + { + final Interval[] sortedIntervals = new Interval[]{ + Intervals.of("2019/2020"), + Intervals.of("2021/2022"), + Intervals.of("2021-04-01/2021-05-01"), + Intervals.of("2022/2023") + }; + Arrays.sort(sortedIntervals, Comparators.intervalsByStartThenEnd()); + + // Search interval outside the bounds of the sorted intervals + Assert.assertNull( + Intervals.findOverlappingInterval(Intervals.of("2018/2019"), sortedIntervals) + ); + Assert.assertNull( + Intervals.findOverlappingInterval(Intervals.of("2023/2024"), sortedIntervals) + ); + + // Search interval within bounds, overlap exists + // Fully overlapping interval + Assert.assertEquals( + Intervals.of("2021/2022"), + Intervals.findOverlappingInterval(Intervals.of("2021/2022"), sortedIntervals) + ); + + // Partially overlapping interval + Assert.assertEquals( + Intervals.of("2022/2023"), + Intervals.findOverlappingInterval(Intervals.of("2022-01-01/2022-01-02"), sortedIntervals) + ); + + Assert.assertEquals( + Intervals.of("2021/2022"), + Intervals.findOverlappingInterval(Intervals.of("2021-06-01/2021-07-01"), sortedIntervals) + ); + + // Overlap with multiple intervals, "smallest" one is returned + Assert.assertEquals( + Intervals.of("2021/2022"), + Intervals.findOverlappingInterval(Intervals.of("2021-03-01/2021-04-01"), sortedIntervals) + ); + + // Search interval within bounds, no overlap + Assert.assertNull( + Intervals.findOverlappingInterval(Intervals.of("2020-01-02/2020-03-03"), sortedIntervals) + ); + } + +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java index 6b2fad9610b3..897db9750354 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java @@ -20,7 +20,6 @@ package org.apache.druid.indexing.common; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Multimap; @@ -329,14 +328,7 @@ public void publishSegments(Iterable segments) throws IOException // Request segment pushes for each set final Multimap segmentMultimap = Multimaps.index( segments, - new Function() - { - @Override - public Interval apply(DataSegment segment) - { - return segment.getInterval(); - } - } + DataSegment::getInterval ); for (final Collection segmentCollection : segmentMultimap.asMap().values()) { getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.copyOf(segmentCollection))); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LocalTaskActionClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LocalTaskActionClient.java index 27e0bcbaa29d..a07977c6a564 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LocalTaskActionClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LocalTaskActionClient.java @@ -30,6 +30,7 @@ import javax.annotation.Nullable; import java.util.Map; +import java.util.concurrent.TimeUnit; public class LocalTaskActionClient implements TaskActionClient { @@ -76,11 +77,28 @@ public RetType submit(TaskAction taskAction) } final long performStartTime = System.currentTimeMillis(); - final RetType result = taskAction.perform(task, toolbox); + final RetType result = performAction(taskAction); emitTimerMetric("task/action/run/time", taskAction, System.currentTimeMillis() - performStartTime); return result; } + private R performAction(TaskAction taskAction) + { + try { + final R result; + if (taskAction.canPerformAsync(task, toolbox)) { + result = taskAction.performAsync(task, toolbox).get(5, TimeUnit.MINUTES); + } else { + result = taskAction.perform(task, toolbox); + } + + return result; + } + catch (Throwable t) { + throw new RuntimeException(t); + } + } + private void emitTimerMetric(final String metric, final TaskAction action, final long time) { final ServiceMetricEvent.Builder metricBuilder = ServiceMetricEvent.builder(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java index f61dad98168d..f0fae4a8617d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java @@ -47,6 +47,7 @@ import java.util.HashSet; import java.util.List; import java.util.Set; +import java.util.concurrent.Future; import java.util.concurrent.ThreadLocalRandom; import java.util.stream.Collectors; @@ -180,6 +181,23 @@ public TypeReference getReturnTypeReference() }; } + @Override + public boolean canPerformAsync(Task task, TaskActionToolbox toolbox) + { + return toolbox.canBatchSegmentAllocation(); + } + + @Override + public Future performAsync(Task task, TaskActionToolbox toolbox) + { + if (!toolbox.canBatchSegmentAllocation()) { + throw new ISE("Batched segment allocation is disabled"); + } + return toolbox.getSegmentAllocationQueue().add( + new SegmentAllocateRequest(task, this, MAX_ATTEMPTS) + ); + } + @Override public SegmentIdWithShardSpec perform( final Task task, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateRequest.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateRequest.java new file mode 100644 index 000000000000..adac7523f442 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateRequest.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.indexing.common.actions; + +import org.apache.druid.indexing.common.task.Task; +import org.joda.time.Interval; +import org.joda.time.chrono.ISOChronology; + +/** + * Request received by the overlord for segment allocation. + */ +public class SegmentAllocateRequest +{ + private final Task task; + private final SegmentAllocateAction action; + private final int maxAttempts; + private final Interval rowInterval; + + private int attempts; + + public SegmentAllocateRequest(Task task, SegmentAllocateAction action, int maxAttempts) + { + this.task = task; + this.action = action; + this.maxAttempts = maxAttempts; + this.rowInterval = action.getQueryGranularity() + .bucket(action.getTimestamp()) + .withChronology(ISOChronology.getInstanceUTC()); + } + + public Task getTask() + { + return task; + } + + public SegmentAllocateAction getAction() + { + return action; + } + + public void incrementAttempts() + { + ++attempts; + } + + public boolean canRetry() + { + return attempts < maxAttempts; + } + + public int getAttempts() + { + return attempts; + } + + public Interval getRowInterval() + { + return rowInterval; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateResult.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateResult.java new file mode 100644 index 000000000000..995bfb9d31d0 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateResult.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.actions; + +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; + +public class SegmentAllocateResult +{ + private final SegmentIdWithShardSpec segmentId; + private final String errorMessage; + + public SegmentAllocateResult(SegmentIdWithShardSpec segmentId, String errorMessage) + { + this.segmentId = segmentId; + this.errorMessage = errorMessage; + } + + public SegmentIdWithShardSpec getSegmentId() + { + return segmentId; + } + + public String getErrorMessage() + { + return errorMessage; + } + + public boolean isSuccess() + { + return segmentId != null; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java new file mode 100644 index 000000000000..9ed53d99faf5 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java @@ -0,0 +1,716 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.actions; + +import com.google.inject.Inject; +import org.apache.druid.guice.ManageLifecycle; +import org.apache.druid.indexing.common.LockGranularity; +import org.apache.druid.indexing.common.task.IndexTaskUtils; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import org.apache.druid.indexing.overlord.Segments; +import org.apache.druid.indexing.overlord.TaskLockbox; +import org.apache.druid.indexing.overlord.config.TaskLockConfig; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.java.util.common.guava.Comparators; +import org.apache.druid.java.util.common.lifecycle.LifecycleStart; +import org.apache.druid.java.util.common.lifecycle.LifecycleStop; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; +import org.apache.druid.query.DruidMetrics; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.timeline.DataSegment; +import org.joda.time.Interval; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.TreeSet; +import java.util.concurrent.BlockingDeque; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Future; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; + +/** + * Queue for {@link SegmentAllocateRequest}s. + */ +@ManageLifecycle +public class SegmentAllocationQueue +{ + private static final Logger log = new Logger(SegmentAllocationQueue.class); + + private static final int MAX_QUEUE_SIZE = 2000; + + private final long maxWaitTimeMillis; + + private final TaskLockbox taskLockbox; + private final ScheduledExecutorService executor; + private final IndexerMetadataStorageCoordinator metadataStorage; + private final AtomicBoolean isLeader = new AtomicBoolean(false); + private final ServiceEmitter emitter; + + private final ConcurrentHashMap keyToBatch = new ConcurrentHashMap<>(); + private final BlockingDeque processingQueue = new LinkedBlockingDeque<>(MAX_QUEUE_SIZE); + + @Inject + public SegmentAllocationQueue( + TaskLockbox taskLockbox, + TaskLockConfig taskLockConfig, + IndexerMetadataStorageCoordinator metadataStorage, + ServiceEmitter emitter, + ScheduledExecutorFactory executorFactory + ) + { + this.emitter = emitter; + this.taskLockbox = taskLockbox; + this.metadataStorage = metadataStorage; + this.maxWaitTimeMillis = taskLockConfig.getBatchAllocationMaxWaitTime(); + + this.executor = taskLockConfig.isBatchSegmentAllocation() + ? executorFactory.create(1, "SegmentAllocQueue-%s") : null; + } + + @LifecycleStart + public void start() + { + if (isEnabled()) { + log.info("Initializing segment allocation queue."); + scheduleQueuePoll(maxWaitTimeMillis); + } + } + + @LifecycleStop + public void stop() + { + if (isEnabled()) { + log.info("Tearing down segment allocation queue."); + executor.shutdownNow(); + } + } + + public void becomeLeader() + { + if (!isLeader.compareAndSet(false, true)) { + log.info("Already the leader. Queue processing has started."); + } else if (isEnabled()) { + log.info("Elected leader. Starting queue processing."); + } else { + log.info( + "Elected leader but batched segment allocation is disabled. " + + "Segment allocation queue will not be used." + ); + } + } + + public void stopBeingLeader() + { + if (!isLeader.compareAndSet(true, false)) { + log.info("Already surrendered leadership. Queue processing is stopped."); + } else if (isEnabled()) { + log.info("Not leader anymore. Stopping queue processing."); + } else { + log.info("Not leader anymore. Segment allocation queue is already disabled."); + } + } + + public boolean isEnabled() + { + return executor != null && !executor.isShutdown(); + } + + private void scheduleQueuePoll(long delay) + { + executor.schedule(this::processBatchesDue, delay, TimeUnit.MILLISECONDS); + } + + /** + * Gets the number of batches currently in the queue. + */ + public int size() + { + return processingQueue.size(); + } + + /** + * Queues a SegmentAllocateRequest. The returned future may complete successfully + * with a non-null value or with a non-null value. + */ + public Future add(SegmentAllocateRequest request) + { + if (!isLeader.get()) { + throw new ISE("Cannot allocate segment if not leader."); + } else if (!isEnabled()) { + throw new ISE("Batched segment allocation is disabled."); + } + + final AllocateRequestKey requestKey = new AllocateRequestKey(request, maxWaitTimeMillis); + final AtomicReference> futureReference = new AtomicReference<>(); + + // Possible race condition: + // t1 -> new batch is added to queue or batch already exists in queue + // t2 -> executor pops batch, processes all requests in it + // t1 -> new request is added to dangling batch and is never picked up + // Solution: For existing batch, call keyToBatch.remove() on the key to + // wait on keyToBatch.compute() to finish before proceeding with processBatch(). + // For new batch, keyToBatch.remove() would not wait as key is not in map yet + // but a new batch is unlikely to be due immediately, so it won't get popped right away. + keyToBatch.compute(requestKey, (key, existingBatch) -> { + if (existingBatch == null) { + AllocateRequestBatch newBatch = new AllocateRequestBatch(key); + futureReference.set(newBatch.add(request)); + return addBatchToQueue(newBatch) ? newBatch : null; + } else { + futureReference.set(existingBatch.add(request)); + return existingBatch; + } + }); + + return futureReference.get(); + } + + /** + * Tries to add the given batch to the processing queue. Fails all the pending + * requests in the batch if we are not leader or if the queue is full. + */ + private boolean addBatchToQueue(AllocateRequestBatch batch) + { + batch.key.resetQueueTime(); + if (!isLeader.get()) { + batch.failPendingRequests("Cannot allocate segment if not leader"); + return false; + } else if (processingQueue.offer(batch.key)) { + log.debug("Added a new batch [%s] to queue.", batch.key); + return true; + } else { + batch.failPendingRequests( + "Segment allocation queue is full. Check the metric `task/action/batch/runTime` " + + "to determine if metadata operations are slow." + ); + return false; + } + } + + /** + * Tries to add the given batch to the processing queue. If a batch already + * exists for this key, transfers all the requests from this batch to the + * existing one. + */ + private void requeueBatch(AllocateRequestBatch batch) + { + log.info("Requeueing [%d] failed requests in batch [%s].", batch.size(), batch.key); + keyToBatch.compute(batch.key, (key, existingBatch) -> { + if (existingBatch == null) { + return addBatchToQueue(batch) ? batch : null; + } else { + // Merge requests from this batch to existing one + existingBatch.transferRequestsFrom(batch); + return existingBatch; + } + }); + } + + private void processBatchesDue() + { + clearQueueIfNotLeader(); + + // Process all batches which are due + log.debug("Processing batches which are due. Queue size [%d].", processingQueue.size()); + int numProcessedBatches = 0; + + AllocateRequestKey nextKey = processingQueue.peekFirst(); + while (nextKey != null && nextKey.isDue()) { + processingQueue.pollFirst(); + AllocateRequestBatch nextBatch = keyToBatch.remove(nextKey); + + boolean processed; + try { + processed = processBatch(nextBatch); + } + catch (Throwable t) { + nextBatch.failPendingRequests(t); + processed = true; + log.error(t, "Error while processing batch [%s]", nextKey); + } + + // Requeue if not fully processed yet + if (processed) { + ++numProcessedBatches; + } else { + requeueBatch(nextBatch); + } + + nextKey = processingQueue.peek(); + } + + // Schedule the next round of processing + final long nextScheduleDelay; + if (processingQueue.isEmpty()) { + nextScheduleDelay = maxWaitTimeMillis; + } else { + nextKey = processingQueue.peek(); + long timeElapsed = System.currentTimeMillis() - nextKey.getQueueTime(); + nextScheduleDelay = Math.max(0, maxWaitTimeMillis - timeElapsed); + } + scheduleQueuePoll(nextScheduleDelay); + log.info("Processed [%d] batches, next execution in [%d ms]", numProcessedBatches, nextScheduleDelay); + } + + /** + * Removes items from the queue as long as we are not leader. + */ + private void clearQueueIfNotLeader() + { + int failedBatches = 0; + AllocateRequestKey nextKey = processingQueue.peekFirst(); + while (nextKey != null && !isLeader.get()) { + processingQueue.pollFirst(); + AllocateRequestBatch nextBatch = keyToBatch.remove(nextKey); + nextBatch.failPendingRequests("Cannot allocate segment if not leader"); + ++failedBatches; + + nextKey = processingQueue.peekFirst(); + } + if (failedBatches > 0) { + log.info("Not leader. Failed [%d] batches, remaining in queue [%d].", failedBatches, processingQueue.size()); + } + } + + /** + * Processes the given batch. Returns true if the batch was completely processed + * and should not be requeued. + */ + private boolean processBatch(AllocateRequestBatch requestBatch) + { + final AllocateRequestKey requestKey = requestBatch.key; + if (requestBatch.isEmpty()) { + return true; + } else if (!isLeader.get()) { + requestBatch.failPendingRequests("Cannot allocate segment if not leader"); + return true; + } + + log.debug( + "Processing [%d] requests for batch [%s], queue time [%s].", + requestBatch.size(), + requestKey, + requestKey.getQueueTime() + ); + + final long startTimeMillis = System.currentTimeMillis(); + final int batchSize = requestBatch.size(); + emitBatchMetric("task/action/batch/size", batchSize, requestKey); + emitBatchMetric("task/action/batch/queueTime", (startTimeMillis - requestKey.getQueueTime()), requestKey); + + final Set usedSegments = retrieveUsedSegments(requestKey); + final int successCount = allocateSegmentsForBatch(requestBatch, usedSegments); + + emitBatchMetric("task/action/batch/attempts", 1L, requestKey); + emitBatchMetric("task/action/batch/runTime", (System.currentTimeMillis() - startTimeMillis), requestKey); + log.info("Successfully processed [%d / %d] requests in batch [%s].", successCount, batchSize, requestKey); + + if (requestBatch.isEmpty()) { + log.debug("All requests in batch [%s] have been processed.", requestKey); + return true; + } + + // Requeue the batch only if used segments have changed + log.debug("There are [%d] failed requests in batch [%s].", requestBatch.size(), requestKey); + final Set updatedUsedSegments = retrieveUsedSegments(requestKey); + + if (updatedUsedSegments.equals(usedSegments)) { + requestBatch.failPendingRequests("Allocation failed probably due to conflicting segments."); + return true; + } else { + log.debug("Used segments have changed. Requeuing failed requests."); + return false; + } + } + + private Set retrieveUsedSegments(AllocateRequestKey key) + { + return new HashSet<>( + metadataStorage.retrieveUsedSegmentsForInterval( + key.dataSource, + key.preferredAllocationInterval, + Segments.ONLY_VISIBLE + ) + ); + } + + private int allocateSegmentsForBatch(AllocateRequestBatch requestBatch, Set usedSegments) + { + int successCount = 0; + + // Find requests whose row interval overlaps with an existing used segment + final Set allRequests = requestBatch.getRequests(); + final Set requestsWithNoOverlappingSegment = new HashSet<>(); + + if (usedSegments.isEmpty()) { + requestsWithNoOverlappingSegment.addAll(allRequests); + } else { + final Interval[] sortedUsedSegmentIntervals = getSortedIntervals(usedSegments); + final Map> overlapIntervalToRequests = new HashMap<>(); + + for (SegmentAllocateRequest request : allRequests) { + // If there is an overlapping used segment, the interval of the used segment + // is the only candidate for allocation for this request + final Interval overlappingInterval = Intervals.findOverlappingInterval( + request.getRowInterval(), + sortedUsedSegmentIntervals + ); + + if (overlappingInterval == null) { + requestsWithNoOverlappingSegment.add(request); + } else if (overlappingInterval.contains(request.getRowInterval())) { + // Found an enclosing interval, use this for allocation + overlapIntervalToRequests.computeIfAbsent(overlappingInterval, i -> new ArrayList<>()) + .add(request); + } else { + // There is no valid allocation interval for this request due to a + // partially overlapping used segment. Need not do anything right now. + // The request will be retried upon requeueing the batch. + } + } + + // Try to allocate segments for the identified used segment intervals. + // Do not retry the failed requests with other intervals unless the batch is requeued. + for (Map.Entry> entry : overlapIntervalToRequests.entrySet()) { + successCount += allocateSegmentsForInterval( + entry.getKey(), + entry.getValue(), + requestBatch + ); + } + } + + // For requests that do not overlap with a used segment, first try to allocate + // using the preferred granularity, then smaller granularities + final Set pendingRequests = new HashSet<>(requestsWithNoOverlappingSegment); + for (Granularity granularity : + Granularity.granularitiesFinerThan(requestBatch.key.preferredSegmentGranularity)) { + Map> requestsByInterval = + getRequestsByInterval(pendingRequests, granularity); + + for (Map.Entry> entry : requestsByInterval.entrySet()) { + successCount += allocateSegmentsForInterval( + entry.getKey(), + entry.getValue(), + requestBatch + ); + pendingRequests.retainAll(requestBatch.getRequests()); + } + } + + return successCount; + } + + private Interval[] getSortedIntervals(Set usedSegments) + { + TreeSet sortedSet = new TreeSet<>(Comparators.intervalsByStartThenEnd()); + usedSegments.forEach(segment -> sortedSet.add(segment.getInterval())); + return sortedSet.toArray(new Interval[0]); + } + + /** + * Tries to allocate segments for the given requests over the specified interval. + * Returns the number of requests for which segments were successfully allocated. + */ + private int allocateSegmentsForInterval( + Interval tryInterval, + List requests, + AllocateRequestBatch requestBatch + ) + { + if (requests.isEmpty()) { + return 0; + } + + final AllocateRequestKey requestKey = requestBatch.key; + log.debug( + "Trying allocation for [%d] requests, interval [%s] in batch [%s]", + requests.size(), + tryInterval, + requestKey + ); + + final List results = taskLockbox.allocateSegments( + requests, + requestKey.dataSource, + tryInterval, + requestKey.skipSegmentLineageCheck, + requestKey.lockGranularity + ); + + int successfulRequests = 0; + for (int i = 0; i < requests.size(); ++i) { + SegmentAllocateRequest request = requests.get(i); + SegmentAllocateResult result = results.get(i); + if (result.isSuccess()) { + ++successfulRequests; + } + + requestBatch.handleResult(result, request); + } + + return successfulRequests; + } + + private Map> getRequestsByInterval( + Set requests, + Granularity tryGranularity + ) + { + final Map> tryIntervalToRequests = new HashMap<>(); + for (SegmentAllocateRequest request : requests) { + Interval tryInterval = tryGranularity.bucket(request.getAction().getTimestamp()); + if (tryInterval.contains(request.getRowInterval())) { + tryIntervalToRequests.computeIfAbsent(tryInterval, i -> new ArrayList<>()).add(request); + } + } + return tryIntervalToRequests; + } + + private void emitTaskMetric(String metric, long value, SegmentAllocateRequest request) + { + final ServiceMetricEvent.Builder metricBuilder = ServiceMetricEvent.builder(); + IndexTaskUtils.setTaskDimensions(metricBuilder, request.getTask()); + metricBuilder.setDimension("taskActionType", SegmentAllocateAction.TYPE); + emitter.emit(metricBuilder.build(metric, value)); + } + + private void emitBatchMetric(String metric, long value, AllocateRequestKey key) + { + final ServiceMetricEvent.Builder metricBuilder = ServiceMetricEvent.builder(); + metricBuilder.setDimension("taskActionType", SegmentAllocateAction.TYPE); + metricBuilder.setDimension(DruidMetrics.DATASOURCE, key.dataSource); + metricBuilder.setDimension(DruidMetrics.INTERVAL, key.preferredAllocationInterval.toString()); + emitter.emit(metricBuilder.build(metric, value)); + } + + /** + * A batch of segment allocation requests. + */ + private class AllocateRequestBatch + { + private final AllocateRequestKey key; + + /** + * Map from allocate requests (represents a single SegmentAllocateAction) + * to the future of allocated segment id. + *

+ * This must be accessed through methods synchronized on this batch. + * It is to avoid races between a new request being added just when the batch + * is being processed. + */ + private final Map> + requestToFuture = new HashMap<>(); + + AllocateRequestBatch(AllocateRequestKey key) + { + this.key = key; + } + + synchronized Future add(SegmentAllocateRequest request) + { + log.debug("Adding request to batch [%s]: %s", key, request.getAction()); + return requestToFuture.computeIfAbsent(request, req -> new CompletableFuture<>()); + } + + synchronized void transferRequestsFrom(AllocateRequestBatch batch) + { + requestToFuture.putAll(batch.requestToFuture); + batch.requestToFuture.clear(); + } + + synchronized Set getRequests() + { + return new HashSet<>(requestToFuture.keySet()); + } + + synchronized void failPendingRequests(String reason) + { + failPendingRequests(new ISE(reason)); + } + + synchronized void failPendingRequests(Throwable cause) + { + if (!requestToFuture.isEmpty()) { + log.warn("Failing [%d] requests in batch due to [%s]. Batch key: %s", size(), cause.getMessage(), key); + requestToFuture.values().forEach(future -> future.completeExceptionally(cause)); + requestToFuture.keySet().forEach( + request -> emitTaskMetric("task/action/failed/count", 1L, request) + ); + requestToFuture.clear(); + } + } + + synchronized void handleResult(SegmentAllocateResult result, SegmentAllocateRequest request) + { + request.incrementAttempts(); + + if (result.isSuccess()) { + emitTaskMetric("task/action/success/count", 1L, request); + requestToFuture.remove(request).complete(result.getSegmentId()); + } else if (request.canRetry()) { + log.info( + "Allocation failed in attempt [%d] due to error [%s]. Can still retry. Action: %s", + request.getAttempts(), + result.getErrorMessage(), + request.getAction() + ); + } else { + emitTaskMetric("task/action/failed/count", 1L, request); + log.error( + "Failing allocate action after [%d] attempts. Latest error [%s]. Action: %s", + request.getAttempts(), + result.getErrorMessage(), + request.getAction() + ); + requestToFuture.remove(request).completeExceptionally(new ISE(result.getErrorMessage())); + } + } + + synchronized boolean isEmpty() + { + return requestToFuture.isEmpty(); + } + + synchronized int size() + { + return requestToFuture.size(); + } + } + + /** + * Key to identify a batch of allocation requests. + */ + private static class AllocateRequestKey + { + private long queueTimeMillis; + private final long maxWaitTimeMillis; + + private final String dataSource; + private final String groupId; + private final Interval preferredAllocationInterval; + private final Granularity preferredSegmentGranularity; + + private final boolean skipSegmentLineageCheck; + private final LockGranularity lockGranularity; + + private final boolean useNonRootGenPartitionSpace; + + private final int hash; + + /** + * Creates a new key for the given request. The batch for a unique key will + * always contain a single request. + */ + AllocateRequestKey(SegmentAllocateRequest request, long maxWaitTimeMillis) + { + final SegmentAllocateAction action = request.getAction(); + final Task task = request.getTask(); + + this.dataSource = action.getDataSource(); + this.groupId = task.getGroupId(); + this.skipSegmentLineageCheck = action.isSkipSegmentLineageCheck(); + this.lockGranularity = action.getLockGranularity(); + this.useNonRootGenPartitionSpace = action.getPartialShardSpec() + .useNonRootGenerationPartitionSpace(); + this.preferredSegmentGranularity = action.getPreferredSegmentGranularity(); + this.preferredAllocationInterval = action.getPreferredSegmentGranularity() + .bucket(action.getTimestamp()); + + this.hash = Objects.hash( + skipSegmentLineageCheck, + useNonRootGenPartitionSpace, + dataSource, + groupId, + preferredAllocationInterval, + lockGranularity + ); + + this.maxWaitTimeMillis = maxWaitTimeMillis; + } + + void resetQueueTime() + { + queueTimeMillis = System.currentTimeMillis(); + } + + long getQueueTime() + { + return queueTimeMillis; + } + + boolean isDue() + { + return System.currentTimeMillis() - queueTimeMillis >= maxWaitTimeMillis; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + AllocateRequestKey that = (AllocateRequestKey) o; + return skipSegmentLineageCheck == that.skipSegmentLineageCheck + && useNonRootGenPartitionSpace == that.useNonRootGenPartitionSpace + && dataSource.equals(that.dataSource) + && groupId.equals(that.groupId) + && preferredAllocationInterval.equals(that.preferredAllocationInterval) + && lockGranularity == that.lockGranularity; + } + + @Override + public int hashCode() + { + return hash; + } + + @Override + public String toString() + { + return "{" + + "ds='" + dataSource + '\'' + + ", gr='" + groupId + '\'' + + ", lock=" + lockGranularity + + ", invl=" + preferredAllocationInterval + + ", slc=" + skipSegmentLineageCheck + + '}'; + } + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java index 559039d96ef6..18e373727790 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java @@ -24,6 +24,8 @@ import com.fasterxml.jackson.core.type.TypeReference; import org.apache.druid.indexing.common.task.Task; +import java.util.concurrent.Future; + @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = TaskAction.TYPE_FIELD) @JsonSubTypes(value = { @JsonSubTypes.Type(name = "lockAcquire", value = TimeChunkLockAcquireAction.class), @@ -58,6 +60,16 @@ public interface TaskAction boolean isAudited(); + default boolean canPerformAsync(Task task, TaskActionToolbox toolbox) + { + return false; + } + + default Future performAsync(Task task, TaskActionToolbox toolbox) + { + throw new UnsupportedOperationException(); + } + @Override String toString(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskActionToolbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskActionToolbox.java index 134a9bf6c72e..7d001ecbcb9a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskActionToolbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskActionToolbox.java @@ -35,6 +35,7 @@ public class TaskActionToolbox { private final TaskLockbox taskLockbox; private final TaskStorage taskStorage; + private final SegmentAllocationQueue segmentAllocationQueue; private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; private final ServiceEmitter emitter; private final SupervisorManager supervisorManager; @@ -46,6 +47,7 @@ public TaskActionToolbox( TaskLockbox taskLockbox, TaskStorage taskStorage, IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, + SegmentAllocationQueue segmentAllocationQueue, ServiceEmitter emitter, SupervisorManager supervisorManager, @Json ObjectMapper jsonMapper @@ -57,6 +59,27 @@ public TaskActionToolbox( this.emitter = emitter; this.supervisorManager = supervisorManager; this.jsonMapper = jsonMapper; + this.segmentAllocationQueue = segmentAllocationQueue; + } + + public TaskActionToolbox( + TaskLockbox taskLockbox, + TaskStorage taskStorage, + IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, + ServiceEmitter emitter, + SupervisorManager supervisorManager, + @Json ObjectMapper jsonMapper + ) + { + this( + taskLockbox, + taskStorage, + indexerMetadataStorageCoordinator, + null, + emitter, + supervisorManager, + jsonMapper + ); } public TaskLockbox getTaskLockbox() @@ -103,4 +126,13 @@ public Optional getTaskRunner() return Optional.absent(); } + public SegmentAllocationQueue getSegmentAllocationQueue() + { + return segmentAllocationQueue; + } + + public boolean canBatchSegmentAllocation() + { + return segmentAllocationQueue != null && segmentAllocationQueue.isEnabled(); + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java index 24eebec94471..ca4d4526557b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java @@ -20,7 +20,6 @@ package org.apache.druid.indexing.overlord; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; import com.google.common.base.Objects; import com.google.common.base.Preconditions; import com.google.common.collect.ComparisonChain; @@ -34,9 +33,13 @@ import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.TimeChunkLock; +import org.apache.druid.indexing.common.actions.SegmentAllocateAction; +import org.apache.druid.indexing.common.actions.SegmentAllocateRequest; +import org.apache.druid.indexing.common.actions.SegmentAllocateResult; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; @@ -446,6 +449,140 @@ public LockResult tryLock(final Task task, final LockRequest request) } } + /** + * Attempts to allocate segments for the given requests. Each request contains + * a {@link Task} and a {@link SegmentAllocateAction}. This method tries to + * acquire the task locks on the required intervals/segments and then performs + * a batch allocation of segments. It is possible that some requests succeed + * successfully and others failed. In that case, only the failed ones should be + * retried. + * + * @param requests List of allocation requests + * @param dataSource Datasource for which segment is to be allocated. + * @param interval Interval for which segment is to be allocated. + * @param skipSegmentLineageCheck Whether lineage check is to be skipped + * (this is true for streaming ingestion) + * @param lockGranularity Granularity of task lock + * @return List of allocation results in the same order as the requests. + */ + public List allocateSegments( + List requests, + String dataSource, + Interval interval, + boolean skipSegmentLineageCheck, + LockGranularity lockGranularity + ) + { + log.info("Allocating [%d] segments for datasource [%s], interval [%s]", requests.size(), dataSource, interval); + final boolean isTimeChunkLock = lockGranularity == LockGranularity.TIME_CHUNK; + + final AllocationHolderList holderList = new AllocationHolderList(requests, interval); + holderList.getPending().forEach(this::verifyTaskIsActive); + + giant.lock(); + try { + if (isTimeChunkLock) { + // For time-chunk locking, segment must be allocated only after acquiring the lock + holderList.getPending().forEach(holder -> acquireTaskLock(holder, true)); + allocateSegmentIds(dataSource, interval, skipSegmentLineageCheck, holderList.getPending()); + } else { + allocateSegmentIds(dataSource, interval, skipSegmentLineageCheck, holderList.getPending()); + holderList.getPending().forEach(holder -> acquireTaskLock(holder, false)); + } + + holderList.getPending().forEach(holder -> addTaskAndPersistLocks(holder, isTimeChunkLock)); + } + finally { + giant.unlock(); + } + + return holderList.getResults(); + } + + /** + * Marks the segment allocation as failed if the underlying task is not active. + */ + private void verifyTaskIsActive(SegmentAllocationHolder holder) + { + final String taskId = holder.task.getId(); + if (!activeTasks.contains(taskId)) { + holder.markFailed("Unable to grant lock to inactive Task [%s]", taskId); + } + } + + /** + * Creates a task lock request and creates or finds the lock for that request. + * Marks the segment allocation as failed if the lock could not be acquired or + * was revoked. + */ + private void acquireTaskLock(SegmentAllocationHolder holder, boolean isTimeChunkLock) + { + final LockRequest lockRequest; + if (isTimeChunkLock) { + lockRequest = new TimeChunkLockRequest(holder.lockRequest); + } else { + lockRequest = new SpecificSegmentLockRequest(holder.lockRequest, holder.allocatedSegment); + } + + // Create or find the task lock for the created lock request + final TaskLockPosse posseToUse = createOrFindLockPosse(lockRequest); + final TaskLock acquiredLock = posseToUse == null ? null : posseToUse.getTaskLock(); + if (posseToUse == null) { + holder.markFailed("Could not find or create lock posse."); + } else if (acquiredLock.isRevoked()) { + holder.markFailed("Lock was revoked."); + } else { + holder.setAcquiredLock(posseToUse, lockRequest.getInterval()); + } + } + + /** + * Adds the task to the found lock posse if not already added and updates + * in the metadata store. Marks the segment allocation as failed if the update + * did not succeed. + */ + private void addTaskAndPersistLocks(SegmentAllocationHolder holder, boolean isTimeChunkLock) + { + final Task task = holder.task; + final TaskLock acquiredLock = holder.acquiredLock; + + if (holder.taskLockPosse.addTask(task)) { + log.info("Added task [%s] to TaskLock [%s]", task.getId(), acquiredLock); + + // This can also be batched later + boolean success = updateLockInStorage(task, acquiredLock); + if (success) { + holder.markSucceeded(); + } else { + final Integer partitionId = isTimeChunkLock + ? null : ((SegmentLock) acquiredLock).getPartitionId(); + unlock(task, holder.lockRequestInterval, partitionId); + holder.markFailed("Could not update task lock in metadata store."); + } + } else { + log.info("Task [%s] already present in TaskLock [%s]", task.getId(), acquiredLock.getGroupId()); + holder.markSucceeded(); + } + } + + private boolean updateLockInStorage(Task task, TaskLock taskLock) + { + try { + taskStorage.addLock(task.getId(), taskLock); + return true; + } + catch (Exception e) { + log.makeAlert("Failed to persist lock in storage") + .addData("task", task.getId()) + .addData("dataSource", taskLock.getDataSource()) + .addData("interval", taskLock.getInterval()) + .addData("version", taskLock.getVersion()) + .emit(); + + return false; + } + } + private TaskLockPosse createOrFindLockPosse(LockRequest request) { Preconditions.checkState(!(request instanceof LockRequestForNewSegment), "Can't handle LockRequestForNewSegment"); @@ -541,7 +678,6 @@ && areAllEqualOrHigherPriorityLocksSharedOrRevoked(conflictPosses, request.getPr * monotonicity and that callers specifying {@code preferredVersion} are doing the right thing. * * @param request request to lock - * * @return a new {@link TaskLockPosse} */ private TaskLockPosse createNewTaskLockPosse(LockRequest request) @@ -550,7 +686,10 @@ private TaskLockPosse createNewTaskLockPosse(LockRequest request) try { final TaskLockPosse posseToUse = new TaskLockPosse(request.toLock()); running.computeIfAbsent(request.getDataSource(), k -> new TreeMap<>()) - .computeIfAbsent(request.getInterval().getStart(), k -> new TreeMap<>(Comparators.intervalsByStartThenEnd())) + .computeIfAbsent( + request.getInterval().getStart(), + k -> new TreeMap<>(Comparators.intervalsByStartThenEnd()) + ) .computeIfAbsent(request.getInterval(), k -> new ArrayList<>()) .add(posseToUse); @@ -561,6 +700,45 @@ private TaskLockPosse createNewTaskLockPosse(LockRequest request) } } + /** + * Makes a call to the {@link #metadataStorageCoordinator} to allocate segments + * for the given requests. Updates the holder with the allocated segment if + * the allocation succeeds, otherwise marks it as failed. + */ + private void allocateSegmentIds( + String dataSource, + Interval interval, + boolean skipSegmentLineageCheck, + Collection holders + ) + { + if (holders.isEmpty()) { + return; + } + + final List createRequests = + holders.stream() + .map(SegmentAllocationHolder::getSegmentRequest) + .collect(Collectors.toList()); + + Map allocatedSegments = + metadataStorageCoordinator.allocatePendingSegments( + dataSource, + interval, + skipSegmentLineageCheck, + createRequests + ); + + for (SegmentAllocationHolder holder : holders) { + SegmentIdWithShardSpec segmentId = allocatedSegments.get(holder.getSegmentRequest()); + if (segmentId == null) { + holder.markFailed("Storage coordinator could not allocate segment."); + } else { + holder.setAllocatedSegment(segmentId); + } + } + } + private SegmentIdWithShardSpec allocateSegmentId(LockRequestForNewSegment request, String version) { return metadataStorageCoordinator.allocatePendingSegment( @@ -577,7 +755,7 @@ private SegmentIdWithShardSpec allocateSegmentId(LockRequestForNewSegment reques /** * Perform the given action with a guarantee that the locks of the task are not revoked in the middle of action. This * method first checks that all locks for the given task and intervals are valid and perform the right action. - * + *

* The given action should be finished as soon as possible because all other methods in this class are blocked until * this method is finished. * @@ -611,7 +789,7 @@ private boolean isTaskLocksValid(Task task, List intervals) .allMatch(interval -> { final List lockPosses = getOnlyTaskLockPosseContainingInterval(task, interval); return lockPosses.stream().map(TaskLockPosse::getTaskLock).noneMatch( - lock -> lock.isRevoked() + TaskLock::isRevoked ); }); } @@ -664,7 +842,9 @@ protected void revokeLock(String taskId, TaskLock lock) final TaskLock revokedLock = lock.revokedCopy(); taskStorage.replaceLock(taskId, lock, revokedLock); - final List possesHolder = running.get(task.getDataSource()).get(lock.getInterval().getStart()).get(lock.getInterval()); + final List possesHolder = running.get(task.getDataSource()) + .get(lock.getInterval().getStart()) + .get(lock.getInterval()); final TaskLockPosse foundPosse = possesHolder.stream() .filter(posse -> posse.getTaskLock().equals(lock)) .findFirst() @@ -692,16 +872,7 @@ public List findLocksForTask(final Task task) giant.lock(); try { - return Lists.transform( - findLockPossesForTask(task), new Function() - { - @Override - public TaskLock apply(TaskLockPosse taskLockPosse) - { - return taskLockPosse.getTaskLock(); - } - } - ); + return Lists.transform(findLockPossesForTask(task), TaskLockPosse::getTaskLock); } finally { giant.unlock(); @@ -778,7 +949,7 @@ public void unlock(final Task task, final Interval interval) * Release lock held for a task on a particular interval. Does nothing if the task does not currently * hold the mentioned lock. * - * @param task task to unlock + * @param task task to unlock * @param interval interval to unlock */ public void unlock(final Task task, final Interval interval, @Nullable Integer partitionId) @@ -1233,7 +1404,7 @@ public boolean equals(Object o) TaskLockPosse that = (TaskLockPosse) o; return java.util.Objects.equals(taskLock, that.taskLock) && - java.util.Objects.equals(taskIds, that.taskIds); + java.util.Objects.equals(taskIds, that.taskIds); } @Override @@ -1251,4 +1422,121 @@ public String toString() .toString(); } } + + /** + * Maintains a list of pending allocation holders. + */ + private static class AllocationHolderList + { + final List all = new ArrayList<>(); + final Set pending = new HashSet<>(); + final Set recentlyCompleted = new HashSet<>(); + + AllocationHolderList(List requests, Interval interval) + { + for (SegmentAllocateRequest request : requests) { + SegmentAllocationHolder holder = new SegmentAllocationHolder(request, interval, this); + all.add(holder); + pending.add(holder); + } + } + + void markCompleted(SegmentAllocationHolder holder) + { + recentlyCompleted.add(holder); + } + + Set getPending() + { + pending.removeAll(recentlyCompleted); + recentlyCompleted.clear(); + return pending; + } + + + List getResults() + { + return all.stream().map(holder -> holder.result).collect(Collectors.toList()); + } + } + + /** + * Contains the task, request, lock and final result for a segment allocation. + */ + private static class SegmentAllocationHolder + { + final AllocationHolderList list; + + final Task task; + final Interval allocateInterval; + final SegmentAllocateAction action; + final LockRequestForNewSegment lockRequest; + SegmentCreateRequest segmentRequest; + + TaskLock acquiredLock; + TaskLockPosse taskLockPosse; + Interval lockRequestInterval; + SegmentIdWithShardSpec allocatedSegment; + SegmentAllocateResult result; + + SegmentAllocationHolder(SegmentAllocateRequest request, Interval allocateInterval, AllocationHolderList list) + { + this.list = list; + this.allocateInterval = allocateInterval; + this.task = request.getTask(); + this.action = request.getAction(); + + this.lockRequest = new LockRequestForNewSegment( + action.getLockGranularity(), + action.getTaskLockType(), + task.getGroupId(), + action.getDataSource(), + allocateInterval, + action.getPartialShardSpec(), + task.getPriority(), + action.getSequenceName(), + action.getPreviousSegmentId(), + action.isSkipSegmentLineageCheck() + ); + } + + SegmentCreateRequest getSegmentRequest() + { + // Initialize the first time this is requested + if (segmentRequest == null) { + segmentRequest = new SegmentCreateRequest( + action.getSequenceName(), + action.getPreviousSegmentId(), + acquiredLock == null ? lockRequest.getVersion() : acquiredLock.getVersion(), + action.getPartialShardSpec() + ); + } + + return segmentRequest; + } + + void markFailed(String msgFormat, Object... args) + { + list.markCompleted(this); + result = new SegmentAllocateResult(null, StringUtils.format(msgFormat, args)); + } + + void markSucceeded() + { + list.markCompleted(this); + result = new SegmentAllocateResult(allocatedSegment, null); + } + + void setAllocatedSegment(SegmentIdWithShardSpec segmentId) + { + this.allocatedSegment = segmentId; + } + + void setAcquiredLock(TaskLockPosse lockPosse, Interval lockRequestInterval) + { + this.taskLockPosse = lockPosse; + this.acquiredLock = lockPosse == null ? null : lockPosse.getTaskLock(); + this.lockRequestInterval = lockRequestInterval; + } + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskMaster.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskMaster.java index 7b9101cf1f22..28c623fdadb4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskMaster.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskMaster.java @@ -26,6 +26,7 @@ import org.apache.druid.discovery.DruidLeaderSelector; import org.apache.druid.discovery.DruidLeaderSelector.Listener; import org.apache.druid.guice.annotations.Self; +import org.apache.druid.indexing.common.actions.SegmentAllocationQueue; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; import org.apache.druid.indexing.common.task.Task; @@ -91,7 +92,8 @@ public TaskMaster( final ServiceEmitter emitter, final SupervisorManager supervisorManager, final OverlordHelperManager overlordHelperManager, - @IndexingService final DruidLeaderSelector overlordLeaderSelector + @IndexingService final DruidLeaderSelector overlordLeaderSelector, + final SegmentAllocationQueue segmentAllocationQueue ) { this.supervisorManager = supervisorManager; @@ -136,6 +138,22 @@ public void becomeLeader() leaderLifecycle.addManagedInstance(taskQueue); leaderLifecycle.addManagedInstance(supervisorManager); leaderLifecycle.addManagedInstance(overlordHelperManager); + leaderLifecycle.addHandler( + new Lifecycle.Handler() + { + @Override + public void start() + { + segmentAllocationQueue.becomeLeader(); + } + + @Override + public void stop() + { + segmentAllocationQueue.stopBeingLeader(); + } + } + ); leaderLifecycle.addHandler( new Lifecycle.Handler() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/TaskLockConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/TaskLockConfig.java index 96f14759f78a..acbc318baabc 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/TaskLockConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/TaskLockConfig.java @@ -30,8 +30,24 @@ public class TaskLockConfig @JsonProperty private boolean forceTimeChunkLock = true; + @JsonProperty + private boolean batchSegmentAllocation = false; + + @JsonProperty + private long batchAllocationMaxWaitTime = 500L; + public boolean isForceTimeChunkLock() { return forceTimeChunkLock; } + + public boolean isBatchSegmentAllocation() + { + return batchSegmentAllocation; + } + + public long getBatchAllocationMaxWaitTime() + { + return batchAllocationMaxWaitTime; + } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionBuilder.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionBuilder.java new file mode 100644 index 000000000000..2e9cd70bbdbc --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionBuilder.java @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.actions; + +import org.apache.druid.indexing.common.LockGranularity; +import org.apache.druid.indexing.common.TaskLockType; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.timeline.partition.PartialShardSpec; +import org.joda.time.DateTime; + +public class SegmentAllocateActionBuilder +{ + private String dataSource; + private DateTime timestamp; + private Granularity queryGranularity; + private Granularity preferredSegmentGranularity; + private String sequenceName; + private String previousSegmentId; + private boolean skipSegmentLineageCheck; + private PartialShardSpec partialShardSpec; + private LockGranularity lockGranularity; + private TaskLockType taskLockType; + private Task task; + + public SegmentAllocateActionBuilder forDatasource(String dataSource) + { + this.dataSource = dataSource; + return this; + } + + public SegmentAllocateActionBuilder forTimestamp(DateTime timestamp) + { + this.timestamp = timestamp; + return this; + } + + public SegmentAllocateActionBuilder forTimestamp(String instant) + { + this.timestamp = DateTimes.of(instant); + return this; + } + + public SegmentAllocateActionBuilder withQueryGranularity(Granularity queryGranularity) + { + this.queryGranularity = queryGranularity; + return this; + } + + public SegmentAllocateActionBuilder withSegmentGranularity(Granularity segmentGranularity) + { + this.preferredSegmentGranularity = segmentGranularity; + return this; + } + + public SegmentAllocateActionBuilder withSequenceName(String sequenceName) + { + this.sequenceName = sequenceName; + return this; + } + + public SegmentAllocateActionBuilder withPreviousSegmentId(String previousSegmentId) + { + this.previousSegmentId = previousSegmentId; + return this; + } + + public SegmentAllocateActionBuilder withSkipLineageCheck(boolean skipLineageCheck) + { + this.skipSegmentLineageCheck = skipLineageCheck; + return this; + } + + public SegmentAllocateActionBuilder withPartialShardSpec(PartialShardSpec partialShardSpec) + { + this.partialShardSpec = partialShardSpec; + return this; + } + + public SegmentAllocateActionBuilder withLockGranularity(LockGranularity lockGranularity) + { + this.lockGranularity = lockGranularity; + return this; + } + + public SegmentAllocateActionBuilder withTaskLockType(TaskLockType taskLockType) + { + this.taskLockType = taskLockType; + return this; + } + + public SegmentAllocateActionBuilder forTask(Task task) + { + this.dataSource = task.getDataSource(); + this.sequenceName = task.getId(); + this.task = task; + return this; + } + + public SegmentAllocateRequest build() + { + return new SegmentAllocateRequest(task, buildAction(), 1); + } + + public SegmentAllocateAction buildAction() + { + return new SegmentAllocateAction( + dataSource, + timestamp, + queryGranularity, + preferredSegmentGranularity, + sequenceName, + previousSegmentId, + skipSegmentLineageCheck, + partialShardSpec, + lockGranularity, + taskLockType + ); + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java index a7e85a027e61..c8861a92cd3f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java @@ -20,7 +20,6 @@ package org.apache.druid.indexing.common.actions; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Predicate; import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; @@ -51,6 +50,7 @@ import org.easymock.EasyMock; import org.joda.time.DateTime; import org.joda.time.Period; +import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -64,6 +64,8 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @RunWith(Parameterized.class) @@ -79,20 +81,26 @@ public class SegmentAllocateActionTest private static final DateTime PARTY_TIME = DateTimes.of("1999"); private static final DateTime THE_DISTANT_FUTURE = DateTimes.of("3000"); + private final boolean useBatch; private final LockGranularity lockGranularity; - @Parameterized.Parameters(name = "{0}") + private SegmentAllocationQueue allocationQueue; + + @Parameterized.Parameters(name = "granularity = {0}, useBatch = {1}") public static Iterable constructorFeeder() { return ImmutableList.of( - new Object[]{LockGranularity.SEGMENT}, - new Object[]{LockGranularity.TIME_CHUNK} + new Object[]{LockGranularity.SEGMENT, true}, + new Object[]{LockGranularity.SEGMENT, false}, + new Object[]{LockGranularity.TIME_CHUNK, true}, + new Object[]{LockGranularity.TIME_CHUNK, false} ); } - public SegmentAllocateActionTest(LockGranularity lockGranularity) + public SegmentAllocateActionTest(LockGranularity lockGranularity, boolean useBatch) { this.lockGranularity = lockGranularity; + this.useBatch = useBatch; } @Before @@ -101,6 +109,19 @@ public void setUp() ServiceEmitter emitter = EasyMock.createMock(ServiceEmitter.class); EmittingLogger.registerEmitter(emitter); EasyMock.replay(emitter); + allocationQueue = taskActionTestKit.getTaskActionToolbox().getSegmentAllocationQueue(); + if (allocationQueue != null) { + allocationQueue.start(); + allocationQueue.becomeLeader(); + } + } + + @After + public void tearDown() + { + if (allocationQueue != null) { + allocationQueue.stop(); + } } @Test @@ -288,29 +309,11 @@ public void testResumeSequence() if (lockGranularity == LockGranularity.TIME_CHUNK) { final TaskLock partyLock = Iterables.getOnlyElement( FluentIterable.from(taskActionTestKit.getTaskLockbox().findLocksForTask(task)) - .filter( - new Predicate() - { - @Override - public boolean apply(TaskLock input) - { - return input.getInterval().contains(PARTY_TIME); - } - } - ) + .filter(input -> input.getInterval().contains(PARTY_TIME)) ); final TaskLock futureLock = Iterables.getOnlyElement( FluentIterable.from(taskActionTestKit.getTaskLockbox().findLocksForTask(task)) - .filter( - new Predicate() - { - @Override - public boolean apply(TaskLock input) - { - return input.getInterval().contains(THE_DISTANT_FUTURE); - } - } - ) + .filter(input -> input.getInterval().contains(THE_DISTANT_FUTURE)) ); assertSameIdentifier( @@ -446,29 +449,11 @@ public void testMultipleSequences() if (lockGranularity == LockGranularity.TIME_CHUNK) { final TaskLock partyLock = Iterables.getOnlyElement( FluentIterable.from(taskActionTestKit.getTaskLockbox().findLocksForTask(task)) - .filter( - new Predicate() - { - @Override - public boolean apply(TaskLock input) - { - return input.getInterval().contains(PARTY_TIME); - } - } - ) + .filter(input -> input.getInterval().contains(PARTY_TIME)) ); final TaskLock futureLock = Iterables.getOnlyElement( FluentIterable.from(taskActionTestKit.getTaskLockbox().findLocksForTask(task)) - .filter( - new Predicate() - { - @Override - public boolean apply(TaskLock input) - { - return input.getInterval().contains(THE_DISTANT_FUTURE); - } - } - ) + .filter(input -> input.getInterval().contains(THE_DISTANT_FUTURE)) ); assertSameIdentifier( @@ -990,21 +975,26 @@ private SegmentIdWithShardSpec allocate( lockGranularity, null ); - return action.perform(task, taskActionTestKit.getTaskActionToolbox()); + + try { + if (useBatch) { + return action.performAsync(task, taskActionTestKit.getTaskActionToolbox()) + .get(5, TimeUnit.SECONDS); + } else { + return action.perform(task, taskActionTestKit.getTaskActionToolbox()); + } + } + catch (ExecutionException e) { + return null; + } + catch (Exception e) { + throw new RuntimeException(e); + } } private void assertSameIdentifier(final SegmentIdWithShardSpec expected, final SegmentIdWithShardSpec actual) { Assert.assertEquals(expected, actual); - Assert.assertEquals(expected.getShardSpec().getPartitionNum(), actual.getShardSpec().getPartitionNum()); - Assert.assertEquals(expected.getShardSpec().getClass(), actual.getShardSpec().getClass()); - - if (expected.getShardSpec().getClass() == NumberedShardSpec.class - && actual.getShardSpec().getClass() == NumberedShardSpec.class) { - Assert.assertEquals(expected.getShardSpec().getNumCorePartitions(), actual.getShardSpec().getNumCorePartitions()); - } else if (expected.getShardSpec().getClass() == LinearShardSpec.class - && actual.getShardSpec().getClass() == LinearShardSpec.class) { - // do nothing - } + Assert.assertEquals(expected.getShardSpec(), actual.getShardSpec()); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueueTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueueTest.java new file mode 100644 index 000000000000..536e9ffac24c --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueueTest.java @@ -0,0 +1,349 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.actions; + +import org.apache.druid.indexing.common.LockGranularity; +import org.apache.druid.indexing.common.TaskLockType; +import org.apache.druid.indexing.common.task.NoopTask; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.config.TaskLockConfig; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.metrics.StubServiceEmitter; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.server.coordinator.simulate.BlockingExecutorService; +import org.apache.druid.server.coordinator.simulate.WrappingScheduledExecutorService; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +public class SegmentAllocationQueueTest +{ + @Rule + public TaskActionTestKit taskActionTestKit = new TaskActionTestKit(); + + private static final String DS_WIKI = "wiki"; + private static final String DS_KOALA = "koala"; + + private SegmentAllocationQueue allocationQueue; + + private StubServiceEmitter emitter; + private BlockingExecutorService executor; + + @Before + public void setUp() + { + executor = new BlockingExecutorService("alloc-test-exec"); + emitter = new StubServiceEmitter("overlord", "alloc-test"); + + final TaskLockConfig lockConfig = new TaskLockConfig() + { + @Override + public boolean isBatchSegmentAllocation() + { + return true; + } + + @Override + public long getBatchAllocationMaxWaitTime() + { + return 0; + } + }; + + allocationQueue = new SegmentAllocationQueue( + taskActionTestKit.getTaskLockbox(), + lockConfig, + taskActionTestKit.getMetadataStorageCoordinator(), + emitter, + (corePoolSize, nameFormat) + -> new WrappingScheduledExecutorService(nameFormat, executor, false) + ); + allocationQueue.start(); + allocationQueue.becomeLeader(); + } + + @After + public void tearDown() + { + if (allocationQueue != null) { + allocationQueue.stop(); + } + if (executor != null) { + executor.shutdownNow(); + } + emitter.flush(); + } + + @Test + public void testBatchWithMultipleTimestamps() + { + verifyAllocationWithBatching( + allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + .forTimestamp("2022-01-01T01:00:00") + .withSegmentGranularity(Granularities.DAY) + .withQueryGranularity(Granularities.SECOND) + .withLockGranularity(LockGranularity.TIME_CHUNK) + .withSequenceName("seq_1") + .build(), + allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + .forTimestamp("2022-01-01T02:00:00") + .withSegmentGranularity(Granularities.DAY) + .withQueryGranularity(Granularities.SECOND) + .withLockGranularity(LockGranularity.TIME_CHUNK) + .withSequenceName("seq_2") + .build(), + true + ); + } + + @Test + public void testBatchWithExclusiveLocks() + { + verifyAllocationWithBatching( + allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + .withTaskLockType(TaskLockType.EXCLUSIVE).build(), + allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + .withTaskLockType(TaskLockType.EXCLUSIVE).build(), + true + ); + } + + @Test + public void testBatchWithSharedLocks() + { + verifyAllocationWithBatching( + allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + .withTaskLockType(TaskLockType.SHARED).build(), + allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + .withTaskLockType(TaskLockType.SHARED).build(), + true + ); + } + + @Test + public void testBatchWithMultipleQueryGranularities() + { + verifyAllocationWithBatching( + allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + .withQueryGranularity(Granularities.SECOND).build(), + allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + .withQueryGranularity(Granularities.MINUTE).build(), + true + ); + } + + @Test + public void testMultipleDatasourcesCannotBatch() + { + verifyAllocationWithBatching( + allocateRequest().forTask(createTask(DS_WIKI, "group_1")).build(), + allocateRequest().forTask(createTask(DS_KOALA, "group_1")).build(), + false + ); + } + + @Test + public void testMultipleGroupIdsCannotBatch() + { + verifyAllocationWithBatching( + allocateRequest().forTask(createTask(DS_WIKI, "group_2")).build(), + allocateRequest().forTask(createTask(DS_WIKI, "group_3")).build(), + false + ); + } + + @Test + public void testMultipleLockGranularitiesCannotBatch() + { + verifyAllocationWithBatching( + allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + .withLockGranularity(LockGranularity.TIME_CHUNK).build(), + allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + .withLockGranularity(LockGranularity.SEGMENT).build(), + false + ); + } + + @Test + public void testMultipleAllocateIntervalsCannotBatch() + { + verifyAllocationWithBatching( + allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + .forTimestamp("2022-01-01") + .withSegmentGranularity(Granularities.DAY).build(), + allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + .forTimestamp("2022-01-02") + .withSegmentGranularity(Granularities.DAY).build(), + false + ); + } + + @Test + public void testConflictingPendingSegment() + { + SegmentAllocateRequest hourSegmentRequest = + allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + .withSegmentGranularity(Granularities.HOUR) + .build(); + Future hourSegmentFuture = allocationQueue.add(hourSegmentRequest); + + SegmentAllocateRequest halfHourSegmentRequest = + allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + .withSegmentGranularity(Granularities.THIRTY_MINUTE) + .build(); + Future halfHourSegmentFuture = allocationQueue.add(halfHourSegmentRequest); + + executor.finishNextPendingTask(); + + Assert.assertNotNull(getSegmentId(hourSegmentFuture)); + Throwable t = Assert.assertThrows(ISE.class, () -> getSegmentId(halfHourSegmentFuture)); + Assert.assertEquals("Storage coordinator could not allocate segment.", t.getMessage()); + } + + @Test + public void testFullAllocationQueue() + { + for (int i = 0; i < 2000; ++i) { + SegmentAllocateRequest request = + allocateRequest().forTask(createTask(DS_WIKI, "group_" + i)).build(); + allocationQueue.add(request); + } + + SegmentAllocateRequest request = + allocateRequest().forTask(createTask(DS_WIKI, "next_group")).build(); + Future future = allocationQueue.add(request); + + // Verify that the future is already complete and segment allocation has failed + Throwable t = Assert.assertThrows(ISE.class, () -> getSegmentId(future)); + Assert.assertEquals( + "Segment allocation queue is full. Check the metric `task/action/batch/runTime` " + + "to determine if metadata operations are slow.", + t.getMessage() + ); + } + + @Test + public void testMultipleRequestsForSameSegment() + { + final List> segmentFutures = new ArrayList<>(); + for (int i = 0; i < 10; ++i) { + SegmentAllocateRequest request = + allocateRequest().forTask(createTask(DS_WIKI, "group_" + i)) + .withSequenceName("sequence_1") + .withPreviousSegmentId("segment_1") + .build(); + segmentFutures.add(allocationQueue.add(request)); + } + + executor.finishNextPendingTask(); + + SegmentIdWithShardSpec segmentId1 = getSegmentId(segmentFutures.get(0)); + + for (Future future : segmentFutures) { + Assert.assertEquals(getSegmentId(future), segmentId1); + } + } + + @Test + public void testMaxWaitTime() + { + // Verify that the batch is due yet + } + + @Test + public void testRequestsFailOnLeaderChange() + { + final List> segmentFutures = new ArrayList<>(); + for (int i = 0; i < 10; ++i) { + SegmentAllocateRequest request = + allocateRequest().forTask(createTask(DS_WIKI, "group_" + i)).build(); + segmentFutures.add(allocationQueue.add(request)); + } + + allocationQueue.stopBeingLeader(); + executor.finishNextPendingTask(); + + for (Future future : segmentFutures) { + Throwable t = Assert.assertThrows(ISE.class, () -> getSegmentId(future)); + Assert.assertEquals("Cannot allocate segment if not leader", t.getMessage()); + } + } + + private void verifyAllocationWithBatching( + SegmentAllocateRequest a, + SegmentAllocateRequest b, + boolean canBatch + ) + { + Assert.assertEquals(0, allocationQueue.size()); + final Future futureA = allocationQueue.add(a); + final Future futureB = allocationQueue.add(b); + + final int expectedCount = canBatch ? 1 : 2; + Assert.assertEquals(expectedCount, allocationQueue.size()); + + executor.finishNextPendingTask(); + emitter.verifyEmitted("task/action/batch/size", expectedCount); + + Assert.assertNotNull(getSegmentId(futureA)); + Assert.assertNotNull(getSegmentId(futureB)); + } + + private SegmentIdWithShardSpec getSegmentId(Future future) + { + try { + return future.get(5, TimeUnit.SECONDS); + } + catch (ExecutionException e) { + throw new ISE(e.getCause().getMessage()); + } + catch (InterruptedException | TimeoutException e) { + throw new RuntimeException(e); + } + } + + private SegmentAllocateActionBuilder allocateRequest() + { + return new SegmentAllocateActionBuilder() + .forDatasource(DS_WIKI) + .forTimestamp("2022-01-01") + .withLockGranularity(LockGranularity.TIME_CHUNK) + .withTaskLockType(TaskLockType.SHARED) + .withQueryGranularity(Granularities.SECOND) + .withSegmentGranularity(Granularities.HOUR); + } + + private Task createTask(String datasource, String groupId) + { + Task task = new NoopTask(null, groupId, datasource, 0, 0, null, null, null); + taskActionTestKit.getTaskLockbox().add(task); + return task; + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java index 2d6b22732a69..189d3fe8779c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java @@ -27,7 +27,10 @@ import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.TaskLockbox; import org.apache.druid.indexing.overlord.TaskStorage; +import org.apache.druid.indexing.overlord.config.TaskLockConfig; import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; +import org.apache.druid.java.util.common.concurrent.ScheduledExecutors; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; import org.apache.druid.metadata.MetadataStorageConnectorConfig; import org.apache.druid.metadata.MetadataStorageTablesConfig; @@ -99,11 +102,34 @@ public int getSqlMetadataMaxRetry() Suppliers.ofInstance(metadataStorageTablesConfig), testDerbyConnector ); + final ServiceEmitter noopEmitter = new NoopServiceEmitter(); + final TaskLockConfig taskLockConfig = new TaskLockConfig() + { + @Override + public boolean isBatchSegmentAllocation() + { + return true; + } + + @Override + public long getBatchAllocationMaxWaitTime() + { + return 10L; + } + }; + taskActionToolbox = new TaskActionToolbox( taskLockbox, taskStorage, metadataStorageCoordinator, - new NoopServiceEmitter(), + new SegmentAllocationQueue( + taskLockbox, + taskLockConfig, + metadataStorageCoordinator, + noopEmitter, + ScheduledExecutors::fixed + ), + noopEmitter, EasyMock.createMock(SupervisorManager.class), objectMapper ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java index 6f9a01c25b0b..3ada645ff88a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java @@ -40,6 +40,7 @@ import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TimeChunkLock; +import org.apache.druid.indexing.common.actions.SegmentAllocationQueue; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; import org.apache.druid.indexing.common.config.TaskStorageConfig; import org.apache.druid.indexing.common.task.NoopTask; @@ -226,7 +227,8 @@ public MockTaskRunner get() serviceEmitter, supervisorManager, EasyMock.createNiceMock(OverlordHelperManager.class), - new TestDruidLeaderSelector() + new TestDruidLeaderSelector(), + EasyMock.createNiceMock(SegmentAllocationQueue.class) ); EmittingLogger.registerEmitter(serviceEmitter); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java index 2dcff7694678..d64bd1d22263 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java @@ -25,6 +25,7 @@ import com.google.common.collect.Sets; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import org.apache.druid.indexing.overlord.SegmentCreateRequest; import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.jackson.DefaultObjectMapper; @@ -36,8 +37,10 @@ import javax.annotation.Nullable; import java.util.ArrayList; +import java.util.Collections; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; public class TestIndexerMetadataStorageCoordinator implements IndexerMetadataStorageCoordinator @@ -124,6 +127,17 @@ public Set announceHistoricalSegments(Set segments) return ImmutableSet.copyOf(added); } + @Override + public Map allocatePendingSegments( + String dataSource, + Interval interval, + boolean skipSegmentLineageCheck, + List requests + ) + { + return Collections.emptyMap(); + } + @Override public SegmentPublishResult announceHistoricalSegments( Set segments, diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java index acb617f78a4c..b3c70f0cdbe9 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java @@ -31,6 +31,7 @@ import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Set; /** @@ -154,6 +155,27 @@ Collection retrieveUsedSegmentsForIntervals( */ Set announceHistoricalSegments(Set segments) throws IOException; + /** + * Allocates pending segments for the given requests in the pending segments table. + * The segment id allocated for a request will not be given out again unless a + * request is made with the same {@link SegmentCreateRequest}. + * + * @param dataSource dataSource for which to allocate a segment + * @param interval interval for which to allocate a segment + * @param skipSegmentLineageCheck if true, perform lineage validation using previousSegmentId for this sequence. + * Should be set to false if replica tasks would index events in same order + * @param requests Requests for which to allocate segments. All + * the requests must share the same partition space. + * @return Map from request to allocated segment id. The map does not contain + * entries for failed requests. + */ + Map allocatePendingSegments( + String dataSource, + Interval interval, + boolean skipSegmentLineageCheck, + List requests + ); + /** * Allocate a new pending segment in the pending segments table. This segment identifier will never be given out * again, unless another call is made with the same dataSource, sequenceName, and previousSegmentId. diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/SegmentCreateRequest.java b/server/src/main/java/org/apache/druid/indexing/overlord/SegmentCreateRequest.java new file mode 100644 index 000000000000..b43e46d8e7a5 --- /dev/null +++ b/server/src/main/java/org/apache/druid/indexing/overlord/SegmentCreateRequest.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.overlord; + +import org.apache.druid.timeline.partition.PartialShardSpec; + +/** + * Contains information used by {@link IndexerMetadataStorageCoordinator} for + * creating a new segment. + *

+ * The {@code sequenceName} and {@code previousSegmentId} fields are meant to + * make it easy for two independent ingestion tasks to produce the same series + * of segments. + */ +public class SegmentCreateRequest +{ + // DO NOT IMPLEMENT equals or hashCode for this class as each request must be + // treated as unique even if it is for the same parameters + + private final String version; + private final String sequenceName; + private final String previousSegmentId; + private final PartialShardSpec partialShardSpec; + + public SegmentCreateRequest( + String sequenceName, + String previousSegmentId, + String version, + PartialShardSpec partialShardSpec + ) + { + this.sequenceName = sequenceName; + this.previousSegmentId = previousSegmentId == null ? "" : previousSegmentId; + this.version = version; + this.partialShardSpec = partialShardSpec; + } + + public String getSequenceName() + { + return sequenceName; + } + + /** + * Non-null previous segment id. This can be used for persisting to the + * pending segments table in the metadata store. + */ + public String getPreviousSegmentId() + { + return previousSegmentId; + } + + public String getVersion() + { + return version; + } + + public PartialShardSpec getPartialShardSpec() + { + return partialShardSpec; + } +} diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index cc42d77e1b70..bb7759a8b5d6 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -28,12 +28,14 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; +import com.google.common.hash.Hasher; import com.google.common.hash.Hashing; import com.google.common.io.BaseEncoding; import com.google.inject.Inject; import org.apache.commons.lang.StringEscapeUtils; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import org.apache.druid.indexing.overlord.SegmentCreateRequest; import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.DateTimes; @@ -75,10 +77,13 @@ import javax.validation.constraints.NotNull; import java.io.IOException; import java.sql.ResultSet; +import java.sql.SQLException; 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; @@ -221,6 +226,10 @@ public int markSegmentsAsUnusedWithinInterval(String dataSource, Interval interv return numSegmentsMarkedUnused; } + /** + * Fetches all the pending segments, whose interval overlaps with the given + * search interval from the metadata store. + */ private Set getPendingSegmentsForIntervalWithHandle( final Handle handle, final String dataSource, @@ -481,6 +490,23 @@ public int getSqlMetadataMaxRetry() return SQLMetadataConnector.DEFAULT_MAX_TRIES; } + @Override + public Map allocatePendingSegments( + String dataSource, + Interval allocateInterval, + boolean skipSegmentLineageCheck, + List requests + ) + { + Preconditions.checkNotNull(dataSource, "dataSource"); + Preconditions.checkNotNull(allocateInterval, "interval"); + + final Interval interval = allocateInterval.withChronology(ISOChronology.getInstanceUTC()); + return connector.retryWithHandle( + handle -> allocatePendingSegments(handle, dataSource, interval, skipSegmentLineageCheck, requests) + ); + } + @Override public SegmentIdWithShardSpec allocatePendingSegment( final String dataSource, @@ -599,6 +625,81 @@ private SegmentIdWithShardSpec allocatePendingSegmentWithSegmentLineageCheck( return newIdentifier; } + private Map allocatePendingSegments( + final Handle handle, + final String dataSource, + final Interval interval, + final boolean skipSegmentLineageCheck, + final List requests + ) throws IOException + { + final Map existingSegmentIds; + if (skipSegmentLineageCheck) { + existingSegmentIds = getExistingSegmentIdsSkipLineageCheck(handle, dataSource, interval, requests); + } else { + existingSegmentIds = getExistingSegmentIdsWithLineageCheck(handle, dataSource, interval, requests); + } + + // For every request see if a segment id already exists + final Map allocatedSegmentIds = new HashMap<>(); + final List requestsForNewSegments = new ArrayList<>(); + for (SegmentCreateRequest request : requests) { + CheckExistingSegmentIdResult existingSegmentId = existingSegmentIds.get(request); + if (existingSegmentId == null || !existingSegmentId.found) { + requestsForNewSegments.add(request); + } else if (existingSegmentId.segmentIdentifier != null) { + log.info("Found valid existing segment [%s] for request.", existingSegmentId.segmentIdentifier); + allocatedSegmentIds.put(request, existingSegmentId.segmentIdentifier); + } else { + log.info("Found clashing existing segment [%s] for request.", existingSegmentId); + } + } + + // For each of the remaining requests, create a new segment + final Map createdSegments = + createNewSegments(handle, dataSource, interval, skipSegmentLineageCheck, requestsForNewSegments); + + // SELECT -> INSERT can fail due to races; callers must be prepared to retry. + // Avoiding ON DUPLICATE KEY since it's not portable. + // Avoiding try/catch since it may cause inadvertent transaction-splitting. + + // UNIQUE key for the row, ensuring we don't have more than one segment per sequence per interval. + // Using a single column instead of (sequence_name, sequence_prev_id) as some MySQL storage engines + // have difficulty with large unique keys (see https://github.com/apache/druid/issues/2319) + insertPendingSegmentsIntoMetastore( + handle, + createdSegments, + dataSource, + interval, + skipSegmentLineageCheck + ); + + allocatedSegmentIds.putAll(createdSegments); + return allocatedSegmentIds; + } + + @SuppressWarnings("UnstableApiUsage") + private String getSequenceNameAndPrevIdSha( + SegmentCreateRequest request, + Interval interval, + boolean skipSegmentLineageCheck + ) + { + final Hasher hasher = Hashing.sha1().newHasher() + .putBytes(StringUtils.toUtf8(request.getSequenceName())) + .putByte((byte) 0xff); + if (skipSegmentLineageCheck) { + hasher + .putLong(interval.getStartMillis()) + .putLong(interval.getEndMillis()); + } else { + hasher + .putBytes(StringUtils.toUtf8(request.getPreviousSegmentId())); + } + + return BaseEncoding.base16().encode(hasher.hash().asBytes()); + } + @Nullable private SegmentIdWithShardSpec allocatePendingSegment( final Handle handle, @@ -631,7 +732,6 @@ private SegmentIdWithShardSpec allocatePendingSegment( ); if (result.found) { - // The found existing segment identifier can be null if its interval doesn't match with the given interval return result.segmentIdentifier; } @@ -672,6 +772,95 @@ private SegmentIdWithShardSpec allocatePendingSegment( return newIdentifier; } + /** + * Returns a map from sequenceName to segment id. + */ + private Map getExistingSegmentIdsSkipLineageCheck( + Handle handle, + String dataSource, + Interval interval, + List requests + ) throws IOException + { + final Query> query = handle + .createQuery( + StringUtils.format( + "SELECT sequence_name, payload " + + "FROM %s WHERE " + + "dataSource = :dataSource AND " + + "start = :start AND " + + "%2$send%2$s = :end", + dbTables.getPendingSegmentsTable(), + connector.getQuoteString() + ) + ) + .bind("dataSource", dataSource) + .bind("start", interval.getStart().toString()) + .bind("end", interval.getEnd().toString()); + + final ResultIterator dbSegments = query + .map((index, r, ctx) -> PendingSegmentsRecord.fromResultSet(r)) + .iterator(); + + // Map from sequenceName to segment id + final Map sequenceToSegmentId = new HashMap<>(); + while (dbSegments.hasNext()) { + final PendingSegmentsRecord record = dbSegments.next(); + final SegmentIdWithShardSpec segmentId = + jsonMapper.readValue(record.getPayload(), SegmentIdWithShardSpec.class); + sequenceToSegmentId.put(record.getSequenceName(), segmentId); + } + + final Map requestToResult = new HashMap<>(); + for (SegmentCreateRequest request : requests) { + SegmentIdWithShardSpec segmentId = sequenceToSegmentId.get(request.getSequenceName()); + requestToResult.put(request, new CheckExistingSegmentIdResult(segmentId != null, segmentId)); + } + + return requestToResult; + } + + /** + * Returns a map from sequenceName to segment id. + */ + private Map getExistingSegmentIdsWithLineageCheck( + Handle handle, + String dataSource, + Interval interval, + List requests + ) throws IOException + { + // This cannot be batched because there doesn't seem to be a clean option: + // 1. WHERE must have sequence_name and sequence_prev_id but not start or end. + // (sequence columns are used to find the matching segment whereas start and + // end are used to determine if the found segment is valid or not) + // 2. IN filters on sequence_name and sequence_prev_id might perform worse than individual SELECTs? + // 3. IN filter on sequence_name alone might be a feasible option worth evaluating + final String sql = StringUtils.format( + "SELECT payload FROM %s WHERE " + + "dataSource = :dataSource AND " + + "sequence_name = :sequence_name AND " + + "sequence_prev_id = :sequence_prev_id", + dbTables.getPendingSegmentsTable() + ); + + final Map requestToResult = new HashMap<>(); + for (SegmentCreateRequest request : requests) { + CheckExistingSegmentIdResult result = checkAndGetExistingSegmentId( + handle.createQuery(sql) + .bind("dataSource", dataSource) + .bind("sequence_name", request.getSequenceName()) + .bind("sequence_prev_id", request.getPreviousSegmentId()), + interval, + request.getSequenceName(), + request.getPreviousSegmentId() + ); + requestToResult.put(request, result); + } + + return requestToResult; + } + private CheckExistingSegmentIdResult checkAndGetExistingSegmentId( final Query> query, final Interval interval, @@ -686,50 +875,36 @@ private CheckExistingSegmentIdResult checkAndGetExistingSegmentId( } final List existingBytes = boundQuery.map(ByteArrayMapper.FIRST).list(); - if (!existingBytes.isEmpty()) { + if (existingBytes.isEmpty()) { + return new CheckExistingSegmentIdResult(false, null); + } else { final SegmentIdWithShardSpec existingIdentifier = jsonMapper.readValue( Iterables.getOnlyElement(existingBytes), SegmentIdWithShardSpec.class ); - if (existingIdentifier.getInterval().getStartMillis() == interval.getStartMillis() - && existingIdentifier.getInterval().getEndMillis() == interval.getEndMillis()) { - if (previousSegmentId == null) { - log.info("Found existing pending segment [%s] for sequence[%s] in DB", existingIdentifier, sequenceName); - } else { - log.info( - "Found existing pending segment [%s] for sequence[%s] (previous = [%s]) in DB", - existingIdentifier, - sequenceName, - previousSegmentId - ); - } + if (existingIdentifier.getInterval().isEqual(interval)) { + log.info( + "Found existing pending segment [%s] for sequence[%s] (previous = [%s]) in DB", + existingIdentifier, + sequenceName, + previousSegmentId + ); return new CheckExistingSegmentIdResult(true, existingIdentifier); } else { - if (previousSegmentId == null) { - log.warn( - "Cannot use existing pending segment [%s] for sequence[%s] in DB, " - + "does not match requested interval[%s]", - existingIdentifier, - sequenceName, - interval - ); - } else { - log.warn( - "Cannot use existing pending segment [%s] for sequence[%s] (previous = [%s]) in DB, " - + "does not match requested interval[%s]", - existingIdentifier, - sequenceName, - previousSegmentId, - interval - ); - } + log.warn( + "Cannot use existing pending segment [%s] for sequence[%s] (previous = [%s]) in DB, " + + "does not match requested interval[%s]", + existingIdentifier, + sequenceName, + previousSegmentId, + interval + ); return new CheckExistingSegmentIdResult(true, null); } } - return new CheckExistingSegmentIdResult(false, null); } private static class CheckExistingSegmentIdResult @@ -745,6 +920,48 @@ private static class CheckExistingSegmentIdResult } } + private void insertPendingSegmentsIntoMetastore( + Handle handle, + Map createdSegments, + String dataSource, + Interval interval, + boolean skipSegmentLineageCheck + ) throws JsonProcessingException + { + final PreparedBatch insertBatch = handle.prepareBatch( + StringUtils.format( + "INSERT INTO %1$s (id, dataSource, created_date, start, %2$send%2$s, sequence_name, sequence_prev_id, " + + "sequence_name_prev_id_sha1, payload) " + + "VALUES (:id, :dataSource, :created_date, :start, :end, :sequence_name, :sequence_prev_id, " + + ":sequence_name_prev_id_sha1, :payload)", + dbTables.getPendingSegmentsTable(), + connector.getQuoteString() + )); + + // Deduplicate the segment ids by inverting the map + Map segmentIdToRequest = new HashMap<>(); + createdSegments.forEach((request, segmentId) -> segmentIdToRequest.put(segmentId, request)); + + for (Map.Entry entry : segmentIdToRequest.entrySet()) { + final SegmentCreateRequest request = entry.getValue(); + final SegmentIdWithShardSpec segmentId = entry.getKey(); + insertBatch.add() + .bind("id", segmentId.toString()) + .bind("dataSource", dataSource) + .bind("created_date", DateTimes.nowUtc().toString()) + .bind("start", interval.getStart().toString()) + .bind("end", interval.getEnd().toString()) + .bind("sequence_name", request.getSequenceName()) + .bind("sequence_prev_id", request.getPreviousSegmentId()) + .bind( + "sequence_name_prev_id_sha1", + getSequenceNameAndPrevIdSha(request, interval, skipSegmentLineageCheck) + ) + .bind("payload", jsonMapper.writeValueAsBytes(segmentId)); + } + insertBatch.execute(); + } + private void insertPendingSegmentIntoMetastore( Handle handle, SegmentIdWithShardSpec newIdentifier, @@ -777,6 +994,204 @@ private void insertPendingSegmentIntoMetastore( .execute(); } + private Map createNewSegments( + Handle handle, + String dataSource, + Interval interval, + boolean skipSegmentLineageCheck, + List requests + ) throws IOException + { + if (requests.isEmpty()) { + return Collections.emptyMap(); + } + + // Get the time chunk and associated data segments for the given interval, if any + final List> existingChunks = + getTimelineForIntervalsWithHandle(handle, dataSource, Collections.singletonList(interval)) + .lookup(interval); + + if (existingChunks.size() > 1) { + // Not possible to expand more than one chunk with a single segment. + log.warn( + "Cannot allocate new segments for dataSource[%s], interval[%s]: already have [%,d] chunks.", + dataSource, + interval, + existingChunks.size() + ); + return Collections.emptyMap(); + } + + // Shard spec of any of the requests (as they are all compatible) can be used to + // identify existing shard specs that share partition space with the requested ones. + final PartialShardSpec partialShardSpec = requests.get(0).getPartialShardSpec(); + + // max partitionId of published data segments which share the same partition space. + SegmentIdWithShardSpec committedMaxId = null; + + @Nullable + final String versionOfExistingChunk; + if (existingChunks.isEmpty()) { + versionOfExistingChunk = null; + } else { + TimelineObjectHolder existingHolder = Iterables.getOnlyElement(existingChunks); + versionOfExistingChunk = existingHolder.getVersion(); + + // Don't use the stream API for performance. + for (DataSegment segment : FluentIterable + .from(existingHolder.getObject()) + .transform(PartitionChunk::getObject) + // Here we check only the segments of the shardSpec which shares the same partition space with the given + // partialShardSpec. Note that OverwriteShardSpec doesn't share the partition space with others. + // See PartitionIds. + .filter(segment -> segment.getShardSpec().sharePartitionSpace(partialShardSpec))) { + if (committedMaxId == null + || committedMaxId.getShardSpec().getPartitionNum() < segment.getShardSpec().getPartitionNum()) { + committedMaxId = SegmentIdWithShardSpec.fromDataSegment(segment); + } + } + } + + + // Fetch the pending segments for this interval to determine max partitionId + // across all shard specs (published + pending). + // A pending segment having a higher partitionId must also be considered + // to avoid clashes when inserting the pending segment created here. + final Set pendingSegments = + getPendingSegmentsForIntervalWithHandle(handle, dataSource, interval); + + final Map createdSegments = new HashMap<>(); + final Map sequenceHashToSegment = new HashMap<>(); + + for (SegmentCreateRequest request : requests) { + // Check if the required segment has already been created in this batch + final String sequenceHash = getSequenceNameAndPrevIdSha(request, interval, skipSegmentLineageCheck); + + final SegmentIdWithShardSpec createdSegment; + if (sequenceHashToSegment.containsKey(sequenceHash)) { + createdSegment = sequenceHashToSegment.get(sequenceHash); + } else { + createdSegment = createNewSegment( + request, + dataSource, + interval, + versionOfExistingChunk, + committedMaxId, + pendingSegments + ); + + // Add to pendingSegments to consider for partitionId + if (createdSegment != null) { + pendingSegments.add(createdSegment); + sequenceHashToSegment.put(sequenceHash, createdSegment); + log.info("Created new segment [%s]", createdSegment); + } + } + + if (createdSegment != null) { + createdSegments.put(request, createdSegment); + } + } + + log.info("Created [%d] new segments for [%d] allocate requests.", sequenceHashToSegment.size(), requests.size()); + return createdSegments; + } + + private SegmentIdWithShardSpec createNewSegment( + SegmentCreateRequest request, + String dataSource, + Interval interval, + String versionOfExistingChunk, + SegmentIdWithShardSpec committedMaxId, + Set pendingSegments + ) + { + final PartialShardSpec partialShardSpec = request.getPartialShardSpec(); + final String existingVersion = request.getVersion(); + + // Include the committedMaxId while computing the overallMaxId + if (committedMaxId != null) { + pendingSegments.add(committedMaxId); + } + + // If there is an existing chunk, find the max id with the same version as the existing chunk. + // There may still be a pending segment with a higher version (but no corresponding used segments) + // which may generate a clash with an existing segment once the new id is generated + final SegmentIdWithShardSpec overallMaxId = + pendingSegments.stream() + .filter(id -> id.getShardSpec().sharePartitionSpace(partialShardSpec)) + .filter(id -> versionOfExistingChunk == null + || id.getVersion().equals(versionOfExistingChunk)) + .max(Comparator.comparing(SegmentIdWithShardSpec::getVersion) + .thenComparing(id -> id.getShardSpec().getPartitionNum())) + .orElse(null); + + // Determine the version of the new segment + final String newSegmentVersion; + if (versionOfExistingChunk != null) { + newSegmentVersion = versionOfExistingChunk; + } else if (overallMaxId != null) { + newSegmentVersion = overallMaxId.getVersion(); + } else { + // this is the first segment for this interval + newSegmentVersion = null; + } + + if (overallMaxId == null) { + // When appending segments, null overallMaxId means that we are allocating the very initial + // segment for this time chunk. + // This code is executed when the Overlord coordinates segment allocation, which is either you append segments + // or you use segment lock. Since the core partitions set is not determined for appended segments, we set + // it 0. When you use segment lock, the core partitions set doesn't work with it. We simply set it 0 so that the + // OvershadowableManager handles the atomic segment update. + final int newPartitionId = partialShardSpec.useNonRootGenerationPartitionSpace() + ? PartitionIds.NON_ROOT_GEN_START_PARTITION_ID + : PartitionIds.ROOT_GEN_START_PARTITION_ID; + + String version = newSegmentVersion == null ? existingVersion : newSegmentVersion; + return new SegmentIdWithShardSpec( + dataSource, + interval, + version, + partialShardSpec.complete(jsonMapper, newPartitionId, 0) + ); + } else if (!overallMaxId.getInterval().equals(interval) + || overallMaxId.getVersion().compareTo(existingVersion) > 0) { + log.warn( + "Cannot allocate new segment for dataSource[%s], interval[%s], existingVersion[%s]: conflicting segment[%s].", + dataSource, + interval, + existingVersion, + overallMaxId + ); + return null; + } else if (committedMaxId != null + && committedMaxId.getShardSpec().getNumCorePartitions() + == SingleDimensionShardSpec.UNKNOWN_NUM_CORE_PARTITIONS) { + log.warn( + "Cannot allocate new segment because of unknown core partition size of segment[%s], shardSpec[%s]", + committedMaxId, + committedMaxId.getShardSpec() + ); + return null; + } else { + // The number of core partitions must always be chosen from the set of used segments in the SegmentTimeline. + // When the core partitions have been dropped, using pending segments may lead to an incorrect state + // where the chunk is believed to have core partitions and queries results are incorrect. + + return new SegmentIdWithShardSpec( + dataSource, + interval, + Preconditions.checkNotNull(newSegmentVersion, "newSegmentVersion"), + partialShardSpec.complete( + jsonMapper, + overallMaxId.getShardSpec().getPartitionNum() + 1, + committedMaxId == null ? 0 : committedMaxId.getShardSpec().getNumCorePartitions() + ) + ); + } + } + /** * This function creates a new segment for the given datasource/interval/etc. A critical * aspect of the creation is to make sure that the new version & new partition number will make @@ -818,15 +1233,18 @@ private SegmentIdWithShardSpec createNewSegment( return null; } else { - // max partitionId of the committed shardSpecs which share the same partition space. + // max partitionId of published data segments which share the same partition space. SegmentIdWithShardSpec committedMaxId = null; - // max partitionId of the all shardSpecs including the pending ones which share the same partition space. - SegmentIdWithShardSpec overallMaxId; - if (!existingChunks.isEmpty()) { + @Nullable + final String versionOfExistingChunk; + if (existingChunks.isEmpty()) { + versionOfExistingChunk = null; + } else { TimelineObjectHolder existingHolder = Iterables.getOnlyElement(existingChunks); + versionOfExistingChunk = existingHolder.getVersion(); - //noinspection ConstantConditions + // Don't use the stream API for performance. for (DataSegment segment : FluentIterable .from(existingHolder.getObject()) .transform(PartitionChunk::getObject) @@ -834,8 +1252,6 @@ private SegmentIdWithShardSpec createNewSegment( // partialShardSpec. Note that OverwriteShardSpec doesn't share the partition space with others. // See PartitionIds. .filter(segment -> segment.getShardSpec().sharePartitionSpace(partialShardSpec))) { - // Don't use the stream API for performance. - // Note that this will compute the max id of existing, visible, data segments in the time chunk: if (committedMaxId == null || committedMaxId.getShardSpec().getPartitionNum() < segment.getShardSpec().getPartitionNum()) { committedMaxId = SegmentIdWithShardSpec.fromDataSegment(segment); @@ -843,63 +1259,41 @@ private SegmentIdWithShardSpec createNewSegment( } } - // Get the version of the existing chunk, we might need it in some of the cases below - // to compute the new identifier's version - @Nullable - final String versionOfExistingChunk; - if (!existingChunks.isEmpty()) { - // remember only one chunk possible for given interval so get the first & only one - versionOfExistingChunk = existingChunks.get(0).getVersion(); - } else { - versionOfExistingChunk = null; - } - - // next, we need to enrich the overallMaxId computed with committed segments with the information of the pending segments - // it is possible that a pending segment has a higher id in which case we need that, it will work, - // and it will avoid clashes when inserting the new pending segment later in the caller of this method + // Fetch the pending segments for this interval to determine max partitionId + // across all shard specs (published + pending). + // A pending segment having a higher partitionId must also be considered + // to avoid clashes when inserting the pending segment created here. final Set pendings = getPendingSegmentsForIntervalWithHandle( handle, dataSource, interval ); - // Make sure we add the committed max id we obtained from the segments table: if (committedMaxId != null) { pendings.add(committedMaxId); } - // Now compute the overallMaxId with all the information: pendings + segments: - // The versionOfExistingChunks filter is ensure that we pick the max id with the version of the existing chunk - // in the case that there may be a pending segment with a higher version but no corresponding used segments + + // If there is an existing chunk, find the max id with the same version as the existing chunk. + // There may still be a pending segment with a higher version (but no corresponding used segments) // which may generate a clash with an existing segment once the new id is generated + final SegmentIdWithShardSpec overallMaxId; overallMaxId = pendings.stream() .filter(id -> id.getShardSpec().sharePartitionSpace(partialShardSpec)) - .filter(id -> versionOfExistingChunk == null || id.getVersion() - .equals(versionOfExistingChunk)) - .max((id1, id2) -> { - final int versionCompare = id1.getVersion().compareTo(id2.getVersion()); - if (versionCompare != 0) { - return versionCompare; - } else { - return Integer.compare( - id1.getShardSpec().getPartitionNum(), - id2.getShardSpec().getPartitionNum() - ); - } - }) + .filter(id -> versionOfExistingChunk == null + || id.getVersion().equals(versionOfExistingChunk)) + .max(Comparator.comparing(SegmentIdWithShardSpec::getVersion) + .thenComparing(id -> id.getShardSpec().getPartitionNum())) .orElse(null); - // The following code attempts to compute the new version, if this - // new version is not null at the end of next block then it will be - // used as the new version in the case for initial or appended segment + + // Determine the version of the new segment final String newSegmentVersion; if (versionOfExistingChunk != null) { - // segment version overrides, so pick that now that we know it exists newSegmentVersion = versionOfExistingChunk; - } else if (!pendings.isEmpty() && overallMaxId != null) { - // there is no visible segments in the time chunk, so pick the max id of pendings, as computed above + } else if (overallMaxId != null) { newSegmentVersion = overallMaxId.getVersion(); } else { - // no segments, no pendings, so this must be the very first segment created for this interval + // this is the first segment for this interval newSegmentVersion = null; } @@ -940,7 +1334,7 @@ private SegmentIdWithShardSpec createNewSegment( ); return null; } else { - // The number of core partitions must always be chosen from the set of used segments in the VersionedIntervalTimeline. + // The number of core partitions must always be chosen from the set of used segments in the SegmentTimeline. // When the core partitions have been dropped, using pending segments may lead to an incorrect state // where the chunk is believed to have core partitions and queries results are incorrect. @@ -1472,4 +1866,46 @@ public int removeDataSourceMetadataOlderThan(long timestamp, @NotNull Set + *

  • sequence_name
  • + *
  • payload
  • + * + */ + static PendingSegmentsRecord fromResultSet(ResultSet resultSet) + { + try { + return new PendingSegmentsRecord( + resultSet.getString(1), + resultSet.getBytes(2) + ); + } + catch (SQLException e) { + throw new RuntimeException(e); + } + } + + PendingSegmentsRecord(String sequenceName, byte[] payload) + { + this.payload = payload; + this.sequenceName = sequenceName; + } + + public byte[] getPayload() + { + return payload; + } + + public String getSequenceName() + { + return sequenceName; + } + } + } diff --git a/server/src/test/java/org/apache/druid/indexing/overlord/SegmentCreateRequestTest.java b/server/src/test/java/org/apache/druid/indexing/overlord/SegmentCreateRequestTest.java new file mode 100644 index 000000000000..33641a8417da --- /dev/null +++ b/server/src/test/java/org/apache/druid/indexing/overlord/SegmentCreateRequestTest.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.overlord; + +import org.apache.druid.timeline.partition.NumberedPartialShardSpec; +import org.apache.druid.timeline.partition.PartialShardSpec; +import org.junit.Assert; +import org.junit.Test; + +public class SegmentCreateRequestTest +{ + + @Test + public void testNullPreviousSegmentId() + { + final PartialShardSpec partialShardSpec = NumberedPartialShardSpec.instance(); + SegmentCreateRequest request = new SegmentCreateRequest( + "sequence", + null, + "version", + partialShardSpec + ); + Assert.assertEquals("sequence", request.getSequenceName()); + Assert.assertEquals("", request.getPreviousSegmentId()); + Assert.assertEquals("version", request.getVersion()); + Assert.assertEquals(partialShardSpec, request.getPartialShardSpec()); + } + +} diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index d7000f8f4d20..f7af9611e51a 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -27,6 +27,7 @@ import org.apache.druid.data.input.StringTuple; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.ObjectMetadata; +import org.apache.druid.indexing.overlord.SegmentCreateRequest; import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.DateTimes; @@ -59,7 +60,6 @@ import org.junit.rules.ExpectedException; import org.skife.jdbi.v2.Handle; import org.skife.jdbi.v2.PreparedBatch; -import org.skife.jdbi.v2.tweak.HandleCallback; import org.skife.jdbi.v2.util.StringMapper; import java.io.IOException; @@ -371,40 +371,12 @@ private void markAllSegmentsUnused(Set segments) Assert.assertEquals( 1, (int) derbyConnector.getDBI().withHandle( - new HandleCallback() - { - @Override - public Integer withHandle(Handle handle) - { - String request = StringUtils.format( - "UPDATE %s SET used = false WHERE id = :id", - derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable() - ); - return handle.createStatement(request).bind("id", segment.getId().toString()).execute(); - } - } - ) - ); - } - } - - private void markAllSegmentsUsed(Set segments) - { - for (final DataSegment segment : segments) { - Assert.assertEquals( - 1, - (int) derbyConnector.getDBI().withHandle( - new HandleCallback() - { - @Override - public Integer withHandle(Handle handle) - { - String request = StringUtils.format( - "UPDATE %s SET used = true WHERE id = :id", - derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable() - ); - return handle.createStatement(request).bind("id", segment.getId().toString()).execute(); - } + handle -> { + String request = StringUtils.format( + "UPDATE %s SET used = false WHERE id = :id", + derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable() + ); + return handle.createStatement(request).bind("id", segment.getId().toString()).execute(); } ) ); @@ -415,32 +387,19 @@ private List retrievePendingSegmentIds() { final String table = derbyConnectorRule.metadataTablesConfigSupplier().get().getPendingSegmentsTable(); return derbyConnector.retryWithHandle( - new HandleCallback>() - { - @Override - public List withHandle(Handle handle) - { - return handle.createQuery("SELECT id FROM " + table + " ORDER BY id") - .map(StringMapper.FIRST) - .list(); - } - } + handle -> handle.createQuery("SELECT id FROM " + table + " ORDER BY id") + .map(StringMapper.FIRST) + .list() ); } + private List retrieveUsedSegmentIds() { final String table = derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(); return derbyConnector.retryWithHandle( - new HandleCallback>() - { - @Override - public List withHandle(Handle handle) - { - return handle.createQuery("SELECT id FROM " + table + " WHERE used = true ORDER BY id") - .map(StringMapper.FIRST) - .list(); - } - } + handle -> handle.createQuery("SELECT id FROM " + table + " WHERE used = true ORDER BY id") + .map(StringMapper.FIRST) + .list() ); } @@ -448,16 +407,9 @@ private List retrieveUnusedSegmentIds() { final String table = derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(); return derbyConnector.retryWithHandle( - new HandleCallback>() - { - @Override - public List withHandle(Handle handle) - { - return handle.createQuery("SELECT id FROM " + table + " WHERE used = false ORDER BY id") - .map(StringMapper.FIRST) - .list(); - } - } + handle -> handle.createQuery("SELECT id FROM " + table + " WHERE used = false ORDER BY id") + .map(StringMapper.FIRST) + .list() ); } @@ -466,39 +418,34 @@ private Boolean insertUsedSegments(Set dataSegments) { final String table = derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(); return derbyConnector.retryWithHandle( - new HandleCallback() - { - @Override - public Boolean withHandle(Handle handle) throws Exception - { - PreparedBatch preparedBatch = handle.prepareBatch( - StringUtils.format( - "INSERT INTO %1$s (id, dataSource, created_date, start, %2$send%2$s, partitioned, version, used, payload) " - + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)", - table, - derbyConnector.getQuoteString() - ) - ); - for (DataSegment segment : dataSegments) { - preparedBatch.add() - .bind("id", segment.getId().toString()) - .bind("dataSource", segment.getDataSource()) - .bind("created_date", DateTimes.nowUtc().toString()) - .bind("start", segment.getInterval().getStart().toString()) - .bind("end", segment.getInterval().getEnd().toString()) - .bind("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? false : true) - .bind("version", segment.getVersion()) - .bind("used", true) - .bind("payload", mapper.writeValueAsBytes(segment)); - } - - final int[] affectedRows = preparedBatch.execute(); - final boolean succeeded = Arrays.stream(affectedRows).allMatch(eachAffectedRows -> eachAffectedRows == 1); - if (!succeeded) { - throw new ISE("Failed to publish segments to DB"); - } - return true; + handle -> { + PreparedBatch preparedBatch = handle.prepareBatch( + StringUtils.format( + "INSERT INTO %1$s (id, dataSource, created_date, start, %2$send%2$s, partitioned, version, used, payload) " + + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)", + table, + derbyConnector.getQuoteString() + ) + ); + for (DataSegment segment : dataSegments) { + preparedBatch.add() + .bind("id", segment.getId().toString()) + .bind("dataSource", segment.getDataSource()) + .bind("created_date", DateTimes.nowUtc().toString()) + .bind("start", segment.getInterval().getStart().toString()) + .bind("end", segment.getInterval().getEnd().toString()) + .bind("partitioned", !(segment.getShardSpec() instanceof NoneShardSpec)) + .bind("version", segment.getVersion()) + .bind("used", true) + .bind("payload", mapper.writeValueAsBytes(segment)); } + + final int[] affectedRows = preparedBatch.execute(); + final boolean succeeded = Arrays.stream(affectedRows).allMatch(eachAffectedRows -> eachAffectedRows == 1); + if (!succeeded) { + throw new ISE("Failed to publish segments to DB"); + } + return true; } ); } @@ -561,12 +508,12 @@ public void testAnnounceHistoricalSegments() throws IOException ); } - List segmentIds = segments.stream().map(segment -> segment.getId().toString()).collect(Collectors.toList()); - segmentIds.sort(Comparator.naturalOrder()); - Assert.assertEquals( - segmentIds, - retrieveUsedSegmentIds() - ); + List segmentIds = segments.stream() + .map(segment -> segment.getId().toString()) + .sorted(Comparator.naturalOrder()) + .collect(Collectors.toList()); + + Assert.assertEquals(segmentIds, retrieveUsedSegmentIds()); // Should not update dataSource metadata. Assert.assertEquals(0, metadataUpdateCounter.get()); @@ -823,10 +770,7 @@ public void testTransactionalAnnounceFailSegmentDropFailWithRetry() throws IOExc retrieveUsedSegmentIds() ); - DataSegment nonExistingSegment = defaultSegment4; - - Set dropSegments = ImmutableSet.of(existingSegment1, nonExistingSegment); - + Set dropSegments = ImmutableSet.of(existingSegment1, defaultSegment4); final SegmentPublishResult result1 = coordinator.announceHistoricalSegments( SEGMENTS, dropSegments, @@ -1576,17 +1520,19 @@ public void testAllocatePendingSegment() } /** - * This test simulates an issue detected on the field consisting of the following sequence of events: - * - A kafka stream segment was created on a given interval - * - Later, after the above was published, another segment on same interval was created by the stream - * - Later, after the above was published, another segment on same interval was created by the stream - * - Later a compaction was issued for the three segments above - * - Later, after the above was published, another segment on same interval was created by the stream - * - Later, the compacted segment got dropped due to a drop rule + * This test verifies the behaviour in the following sequence of events: + * - create segment1 for an interval and publish + * - create segment2 for same interval and publish + * - create segment3 for same interval and publish + * - compact all segments above and publish new segments + * - create segment4 for the same interval + * - drop the compacted segment + * - create segment5 for the same interval + * - verify that the id for segment5 is correct * - Later, after the above was dropped, another segment on same interval was created by the stream but this - * time there was an integrity violation in the pending segments table because the - * {@link IndexerSQLMetadataStorageCoordinator#createNewSegment(Handle, String, Interval, PartialShardSpec, String)} - * method returned an segment id that already existed in the pending segments table + * time there was an integrity violation in the pending segments table because the + * {@link IndexerSQLMetadataStorageCoordinator#createNewSegment(Handle, String, Interval, PartialShardSpec, String)} + * method returned an segment id that already existed in the pending segments table */ @Test public void testAllocatePendingSegmentAfterDroppingExistingSegment() @@ -1690,13 +1636,13 @@ public void testAllocatePendingSegmentAfterDroppingExistingSegment() /** * Slightly different that the above test but that involves reverted compaction - 1) used segments of version = A, id = 0, 1, 2 - 2) overwrote segments of version = B, id = 0 <= compaction - 3) marked segments unused for version = A, id = 0, 1, 2 <= overshadowing - 4) pending segment of version = B, id = 1 <= appending new data, aborted - 5) reverted compaction, mark segments used for version = A, id = 0, 1, 2, and mark compacted segments unused - 6) used segments of version = A, id = 0, 1, 2 - 7) pending segment of version = B, id = 1 + * 1) used segments of version = A, id = 0, 1, 2 + * 2) overwrote segments of version = B, id = 0 <= compaction + * 3) marked segments unused for version = A, id = 0, 1, 2 <= overshadowing + * 4) pending segment of version = B, id = 1 <= appending new data, aborted + * 5) reverted compaction, mark segments used for version = A, id = 0, 1, 2, and mark compacted segments unused + * 6) used segments of version = A, id = 0, 1, 2 + * 7) pending segment of version = B, id = 1 */ @Test public void testAnotherAllocatePendingSegmentAfterRevertingCompaction() @@ -1842,13 +1788,13 @@ public void testAnotherAllocatePendingSegmentAfterRevertingCompaction() // used segment: version = A, id = 0,1,2 // unused segment: version = B, id = 0 List pendings = retrievePendingSegmentIds(); - Assert.assertTrue(pendings.size() == 4); + Assert.assertEquals(4, pendings.size()); List used = retrieveUsedSegmentIds(); - Assert.assertTrue(used.size() == 3); + Assert.assertEquals(3, used.size()); List unused = retrieveUnusedSegmentIds(); - Assert.assertTrue(unused.size() == 1); + Assert.assertEquals(1, unused.size()); // Simulate one more append load final SegmentIdWithShardSpec identifier4 = coordinator.allocatePendingSegment( @@ -1886,7 +1832,71 @@ public void testAnotherAllocatePendingSegmentAfterRevertingCompaction() Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_A_3", ids.get(3)); } - + + @Test + public void testAllocatePendingSegments() + { + final PartialShardSpec partialShardSpec = NumberedPartialShardSpec.instance(); + final String dataSource = "ds"; + final Interval interval = Intervals.of("2017-01-01/2017-02-01"); + final String sequenceName = "seq"; + + final SegmentCreateRequest request = new SegmentCreateRequest(sequenceName, null, "v1", partialShardSpec); + final SegmentIdWithShardSpec segmentId0 = coordinator.allocatePendingSegments( + dataSource, + interval, + false, + Collections.singletonList(request) + ).get(request); + + Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_v1", segmentId0.toString()); + + final SegmentCreateRequest request1 = + new SegmentCreateRequest(sequenceName, segmentId0.toString(), segmentId0.getVersion(), partialShardSpec); + final SegmentIdWithShardSpec segmentId1 = coordinator.allocatePendingSegments( + dataSource, + interval, + false, + Collections.singletonList(request1) + ).get(request1); + + Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_v1_1", segmentId1.toString()); + + final SegmentCreateRequest request2 = + new SegmentCreateRequest(sequenceName, segmentId1.toString(), segmentId1.getVersion(), partialShardSpec); + final SegmentIdWithShardSpec segmentId2 = coordinator.allocatePendingSegments( + dataSource, + interval, + false, + Collections.singletonList(request2) + ).get(request2); + + Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_v1_2", segmentId2.toString()); + + final SegmentCreateRequest request3 = + new SegmentCreateRequest(sequenceName, segmentId1.toString(), segmentId1.getVersion(), partialShardSpec); + final SegmentIdWithShardSpec segmentId3 = coordinator.allocatePendingSegments( + dataSource, + interval, + false, + Collections.singletonList(request3) + ).get(request3); + + Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_v1_2", segmentId3.toString()); + Assert.assertEquals(segmentId2, segmentId3); + + final SegmentCreateRequest request4 = + new SegmentCreateRequest("seq1", null, "v1", partialShardSpec); + final SegmentIdWithShardSpec segmentId4 = coordinator.allocatePendingSegments( + dataSource, + interval, + false, + Collections.singletonList(request4) + ).get(request4); + + Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_v1_3", segmentId4.toString()); + } + @Test public void testNoPendingSegmentsAndOneUsedSegment() { @@ -1923,9 +1933,8 @@ public void testNoPendingSegmentsAndOneUsedSegment() true ); Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_A_1", identifier.toString()); - - } + } @Test From 7d106e436011f376bd706871a5029122a8612679 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 6 Dec 2022 13:45:17 +0530 Subject: [PATCH 21/34] Docs: Update docs for coordinator dynamic config (#13494) (#13495) * Update docs for useBatchedSegmentSampler * Update docs for round robin assigment --- docs/configuration/index.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/configuration/index.md b/docs/configuration/index.md index 8ac9382b7e21..9d4d240004d0 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -939,7 +939,7 @@ Issuing a GET request at the same URL will return the spec that is currently in |`mergeBytesLimit`|The maximum total uncompressed size in bytes of segments to merge.|524288000L| |`mergeSegmentsLimit`|The maximum number of segments that can be in a single [append task](../ingestion/tasks.md).|100| |`maxSegmentsToMove`|The maximum number of segments that can be moved at any given time.|5| -|`useBatchedSegmentSampler`|Boolean flag for whether or not we should use the Reservoir Sampling with a reservoir of size k instead of fixed size 1 to pick segments to move. This option can be enabled to speed up segment balancing process, especially if there are huge number of segments in the cluster or if there are too many segments to move.|false| +|`useBatchedSegmentSampler`|Deprecated. Boolean flag for whether or not we should use the Reservoir Sampling with a reservoir of size k instead of fixed size 1 to pick segments to move. This option can be enabled to speed up the sampling of segments to be balanced, especially if there is a large number of segments in the cluster or if there are too many segments to move.|true| |`percentOfSegmentsToConsiderPerMove`|Deprecated. This will eventually be phased out by the batched segment sampler. You can enable the batched segment sampler now by setting the dynamic Coordinator config, `useBatchedSegmentSampler`, to `true`. Note that if you choose to enable the batched segment sampler, `percentOfSegmentsToConsiderPerMove` will no longer have any effect on balancing. If `useBatchedSegmentSampler == false`, this config defines the percentage of the total number of segments in the cluster that are considered every time a segment needs to be selected for a move. Druid orders servers by available capacity ascending (the least available capacity first) and then iterates over the servers. For each server, Druid iterates over the segments on the server, considering them for moving. The default config of 100% means that every segment on every server is a candidate to be moved. This should make sense for most small to medium-sized clusters. However, an admin may find it preferable to drop this value lower if they don't think that it is worthwhile to consider every single segment in the cluster each time it is looking for a segment to move.|100| |`replicantLifetime`|The maximum number of Coordinator runs for a segment to be replicated before we start alerting.|15| |`replicationThrottleLimit`|The maximum number of segments that can be replicated at one time.|10| @@ -948,6 +948,7 @@ Issuing a GET request at the same URL will return the spec that is currently in |`killDataSourceWhitelist`|List of specific data sources for which kill tasks are sent if property `druid.coordinator.kill.on` is true. This can be a list of comma-separated data source names or a JSON array.|none| |`killPendingSegmentsSkipList`|List of data sources for which pendingSegments are _NOT_ cleaned up if property `druid.coordinator.kill.pendingSegments.on` is true. This can be a list of comma-separated data sources or a JSON array.|none| |`maxSegmentsInNodeLoadingQueue`|The maximum number of segments that could be queued for loading to any given server. This parameter could be used to speed up segments loading process, especially if there are "slow" nodes in the cluster (with low loading speed) or if too much segments scheduled to be replicated to some particular node (faster loading could be preferred to better segments distribution). Desired value depends on segments loading speed, acceptable replication time and number of nodes. Value 1000 could be a start point for a rather big cluster. Default value is 100. |100| +|`useRoundRobinSegmentAssignment`|Boolean flag for whether segments should be assigned to historicals in a round robin fashion. When disabled, segment assignment is done using the chosen balancer strategy. When enabled, this can speed up segment assignments leaving balancing to move the segments to their optimal locations (based on the balancer strategy) lazily. |false| |`decommissioningNodes`| List of historical servers to 'decommission'. Coordinator will not assign new segments to 'decommissioning' servers, and segments will be moved away from them to be placed on non-decommissioning servers at the maximum rate specified by `decommissioningMaxPercentOfMaxSegmentsToMove`.|none| |`decommissioningMaxPercentOfMaxSegmentsToMove`| Upper limit of segments the Coordinator can move from decommissioning servers to active non-decommissioning servers during a single run. This value is relative to the total maximum number of segments that can be moved at any given time based upon the value of `maxSegmentsToMove`.

    If `decommissioningMaxPercentOfMaxSegmentsToMove` is 0, the Coordinator does not move segments to decommissioning servers, effectively putting them in a type of "maintenance" mode. In this case, decommissioning servers do not participate in balancing or assignment by load rules. The Coordinator still considers segments on decommissioning servers as candidates to replicate on active servers.

    Decommissioning can stall if there are no available active servers to move the segments to. You can use the maximum percent of decommissioning segment movements to prioritize balancing or to decrease commissioning time to prevent active servers from being overloaded. The value must be between 0 and 100.|70| |`pauseCoordination`| Boolean flag for whether or not the coordinator should execute its various duties of coordinating the cluster. Setting this to true essentially pauses all coordination work while allowing the API to remain up. Duties that are paused include all classes that implement the `CoordinatorDuty` Interface. Such duties include: Segment balancing, Segment compaction, Emission of metrics controlled by the dynamic coordinator config `emitBalancingStats`, Submitting kill tasks for unused segments (if enabled), Logging of used segments in the cluster, Marking of newly unused or overshadowed segments, Matching and execution of load/drop rules for used segments, Unloading segments that are no longer marked as used from Historical servers. An example of when an admin may want to pause coordination would be if they are doing deep storage maintenance on HDFS Name Nodes with downtime and don't want the coordinator to be directing Historical Nodes to hit the Name Node with API requests until maintenance is done and the deep store is declared healthy for use again. |false| From 789922ac1b6ea7c46297fcd65d5214a724385862 Mon Sep 17 00:00:00 2001 From: Jill Osborne Date: Wed, 7 Dec 2022 09:38:05 +0000 Subject: [PATCH 22/34] Update to native ingestion doc - backport (#13483) * Update to native ingestion doc (cherry picked from commit aba83f289dd4dfbf5414ef808c8e35440835b957) * Update native-batch.md * Update native-batch.md --- docs/ingestion/native-batch.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/ingestion/native-batch.md b/docs/ingestion/native-batch.md index 7106a9a000a3..e8d83f34046a 100644 --- a/docs/ingestion/native-batch.md +++ b/docs/ingestion/native-batch.md @@ -679,8 +679,7 @@ Returns the task attempt history of the worker task spec of the given id, or HTT While ingesting data using the parallel task indexing, Druid creates segments from the input data and pushes them. For segment pushing, the parallel task index supports the following segment pushing modes based upon your type of [rollup](./rollup.md): -- Bulk pushing mode: Used for perfect rollup. Druid pushes every segment at the very end of the index task. Until then, Druid stores created segments in memory and local storage of the service running the index task. This mode can cause problems if you have limited storage capacity, and is not recommended to use in production. -To enable bulk pushing mode, set `forceGuaranteedRollup` in your TuningConfig. You cannot use bulk pushing with `appendToExisting` in your IOConfig. +- Bulk pushing mode: Used for perfect rollup. Druid pushes every segment at the very end of the index task. Until then, Druid stores created segments in memory and local storage of the service running the index task. To enable bulk pushing mode, set `forceGuaranteedRollup` to `true` in your tuning config. You cannot use bulk pushing with `appendToExisting` in your IOConfig. - Incremental pushing mode: Used for best-effort rollup. Druid pushes segments are incrementally during the course of the indexing task. The index task collects data and stores created segments in the memory and disks of the services running the task until the total number of collected rows exceeds `maxTotalRows`. At that point the index task immediately pushes all segments created up until that moment, cleans up pushed segments, and continues to ingest the remaining data. ## Capacity planning From 888311cf933f63a119aa32d6df312cf99b316a72 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 8 Dec 2022 09:53:03 +0530 Subject: [PATCH 23/34] Use version 25.0.0 in docker-compose.yml --- distribution/docker/docker-compose.yml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/distribution/docker/docker-compose.yml b/distribution/docker/docker-compose.yml index 9faa35389bf2..1d9ec5d10d55 100644 --- a/distribution/docker/docker-compose.yml +++ b/distribution/docker/docker-compose.yml @@ -65,7 +65,7 @@ services: - environment broker: - image: apache/druid:0.24.0 + image: apache/druid:25.0.0 container_name: broker volumes: - broker_var:/opt/druid/var @@ -81,7 +81,7 @@ services: - environment historical: - image: apache/druid:0.24.0 + image: apache/druid:25.0.0 container_name: historical volumes: - druid_shared:/opt/shared @@ -98,7 +98,7 @@ services: - environment middlemanager: - image: apache/druid:0.24.0 + image: apache/druid:25.0.0 container_name: middlemanager volumes: - druid_shared:/opt/shared @@ -116,7 +116,7 @@ services: - environment router: - image: apache/druid:0.24.0 + image: apache/druid:25.0.0 container_name: router volumes: - router_var:/opt/druid/var From 63780ed08f0442824fb8200516ce1151e07e5aee Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 7 Dec 2022 20:26:26 -0800 Subject: [PATCH 24/34] fix issue with jetty graceful shutdown of data servers when druid.serverview.type=http (#13499) (#13515) * fix issue with http server inventory view blocking data node http server shutdown with long polling * adjust * fix test inspections --- .../apache/druid/guice/AnnouncerModule.java | 2 +- .../BatchDataSegmentAnnouncer.java | 8 ++++ .../coordination/ChangeRequestHistory.java | 19 +++++++++ .../ChangeRequestHistoryTest.java | 41 +++++++++++++++++++ 4 files changed, 69 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java b/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java index 97a468542237..6b0c96641a65 100644 --- a/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java +++ b/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java @@ -54,7 +54,7 @@ public void configure(Binder binder) JsonConfigProvider.bind(binder, "druid.announcer", BatchDataSegmentAnnouncerConfig.class); JsonConfigProvider.bind(binder, "druid.announcer", DataSegmentAnnouncerProvider.class); binder.bind(DataSegmentAnnouncer.class).toProvider(DataSegmentAnnouncerProvider.class); - binder.bind(BatchDataSegmentAnnouncer.class).in(LazySingleton.class); + binder.bind(BatchDataSegmentAnnouncer.class).in(ManageLifecycleAnnouncements.class); if (isZkEnabled) { binder.bind(DataSegmentServerAnnouncer.class).to(CuratorDataSegmentServerAnnouncer.class).in(LazySingleton.class); diff --git a/server/src/main/java/org/apache/druid/server/coordination/BatchDataSegmentAnnouncer.java b/server/src/main/java/org/apache/druid/server/coordination/BatchDataSegmentAnnouncer.java index aa03ec6f4594..3d766be09832 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/BatchDataSegmentAnnouncer.java +++ b/server/src/main/java/org/apache/druid/server/coordination/BatchDataSegmentAnnouncer.java @@ -36,6 +36,7 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.lifecycle.LifecycleStop; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.server.initialization.BatchDataSegmentAnnouncerConfig; import org.apache.druid.server.initialization.ZkPathsConfig; @@ -129,6 +130,13 @@ public BatchDataSegmentAnnouncer( this(server, config, zkPaths, () -> announcer, jsonMapper, ZkEnablementConfig.ENABLED); } + @LifecycleStop + public void stop() + { + changes.stop(); + } + + @Override public void announceSegment(DataSegment segment) throws IOException { diff --git a/server/src/main/java/org/apache/druid/server/coordination/ChangeRequestHistory.java b/server/src/main/java/org/apache/druid/server/coordination/ChangeRequestHistory.java index 65c8dbd62473..a0c45b4a3352 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/ChangeRequestHistory.java +++ b/server/src/main/java/org/apache/druid/server/coordination/ChangeRequestHistory.java @@ -32,6 +32,7 @@ import java.util.ArrayList; import java.util.LinkedHashMap; +import java.util.LinkedHashSet; import java.util.List; import java.util.Map; import java.util.concurrent.ExecutorService; @@ -43,6 +44,7 @@ * * Clients call {@link #getRequestsSince} to get updates since given counter. */ + public class ChangeRequestHistory { private static int MAX_SIZE = 1000; @@ -74,11 +76,24 @@ public ChangeRequestHistory(int maxSize) this.singleThreadedExecutor = Execs.singleThreaded("SegmentChangeRequestHistory"); } + public void stop() + { + singleThreadedExecutor.shutdownNow(); + final LinkedHashSet> futures = new LinkedHashSet<>(waitingFutures.keySet()); + waitingFutures.clear(); + for (CustomSettableFuture theFuture : futures) { + theFuture.setException(new IllegalStateException("Server is shutting down.")); + } + } + /** * Add batch of segment changes update. */ public synchronized void addChangeRequests(List requests) { + if (singleThreadedExecutor.isShutdown()) { + return; + } for (T request : requests) { changes.add(new Holder<>(request, getLastCounter().inc())); } @@ -108,6 +123,10 @@ public synchronized void addChangeRequest(T request) public synchronized ListenableFuture> getRequestsSince(final Counter counter) { final CustomSettableFuture future = new CustomSettableFuture<>(waitingFutures); + if (singleThreadedExecutor.isShutdown()) { + future.setException(new IllegalStateException("Server is shutting down.")); + return future; + } if (counter.counter < 0) { future.setException(new IAE("counter[%s] must be >= 0", counter)); diff --git a/server/src/test/java/org/apache/druid/server/coordination/ChangeRequestHistoryTest.java b/server/src/test/java/org/apache/druid/server/coordination/ChangeRequestHistoryTest.java index 82a67b038ed6..ecba53a8a031 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/ChangeRequestHistoryTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/ChangeRequestHistoryTest.java @@ -25,6 +25,7 @@ import org.junit.Assert; import org.junit.Test; +import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -171,4 +172,44 @@ public void testNonImmediateFuture() throws Exception Assert.assertEquals(1, snapshot.getCounter().getCounter()); Assert.assertEquals(1, snapshot.getRequests().size()); } + + @Test + public void testStop() + { + final ChangeRequestHistory history = new ChangeRequestHistory(); + + ListenableFuture> future = history.getRequestsSince( + ChangeRequestHistory.Counter.ZERO + ); + Assert.assertEquals(1, history.waitingFutures.size()); + + final AtomicBoolean callbackExcecuted = new AtomicBoolean(false); + Futures.addCallback( + future, + new FutureCallback>() + { + @Override + public void onSuccess(ChangeRequestsSnapshot result) + { + callbackExcecuted.set(true); + } + + @Override + public void onFailure(Throwable t) + { + callbackExcecuted.set(true); + } + } + ); + + history.stop(); + // any new change requests should be ignored, there should be no waiting futures, and open futures should be resolved + history.addChangeRequest(new SegmentChangeRequestNoop()); + Assert.assertEquals(0, history.waitingFutures.size()); + Assert.assertTrue(callbackExcecuted.get()); + Assert.assertTrue(future.isDone()); + + Throwable thrown = Assert.assertThrows(ExecutionException.class, future::get); + Assert.assertEquals("java.lang.IllegalStateException: Server is shutting down.", thrown.getMessage()); + } } From 7cb08ee2dd4eed227045bbf7ff00dba054cf72e4 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 8 Dec 2022 10:15:44 +0530 Subject: [PATCH 25/34] Limit max batch size for segment allocation, add docs (#13503) (#13517) Changes: - Limit max batch size in `SegmentAllocationQueue` to 500 - Rename `batchAllocationMaxWaitTime` to `batchAllocationWaitTime` since the actual wait time may exceed this configured value. - Replace usage of `SegmentInsertAction` in `TaskToolbox` with `SegmentTransactionalInsertAction` --- docs/configuration/index.md | 2 + docs/ingestion/tasks.md | 20 +++++++++ docs/operations/metrics.md | 10 ++++- .../druid/indexing/common/TaskToolbox.java | 8 +++- .../actions/SegmentAllocationQueue.java | 44 +++++++++++++++---- .../overlord/config/TaskLockConfig.java | 6 +-- .../actions/SegmentAllocationQueueTest.java | 19 +++++++- .../common/actions/TaskActionTestKit.java | 2 +- 8 files changed, 94 insertions(+), 17 deletions(-) diff --git a/docs/configuration/index.md b/docs/configuration/index.md index 9d4d240004d0..5629be9f4ed0 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -1103,6 +1103,8 @@ These Overlord static configurations can be defined in the `overlord/runtime.pro |`druid.indexer.storage.type`|Choices are "local" or "metadata". Indicates whether incoming tasks should be stored locally (in heap) or in metadata storage. "local" is mainly for internal testing while "metadata" is recommended in production because storing incoming tasks in metadata storage allows for tasks to be resumed if the Overlord should fail.|local| |`druid.indexer.storage.recentlyFinishedThreshold`|Duration of time to store task results. Default is 24 hours. If you have hundreds of tasks running in a day, consider increasing this threshold.|PT24H| |`druid.indexer.tasklock.forceTimeChunkLock`|_**Setting this to false is still experimental**_
    If set, all tasks are enforced to use time chunk lock. If not set, each task automatically chooses a lock type to use. This configuration can be overwritten by setting `forceTimeChunkLock` in the [task context](../ingestion/tasks.md#context). See [Task Locking & Priority](../ingestion/tasks.md#context) for more details about locking in tasks.|true| +|`druid.indexer.tasklock.batchSegmentAllocation`| If set to true, Druid performs segment allocate actions in batches to improve throughput and reduce the average `task/action/run/time`. See [batching `segmentAllocate` actions](../ingestion/tasks.md#batching-segmentallocate-actions) for details.|false| +|`druid.indexer.tasklock.batchAllocationWaitTime`|Number of milliseconds after Druid adds the first segment allocate action to a batch, until it executes the batch. Allows the batch to add more requests and improve the average segment allocation run time. This configuration takes effect only if `batchSegmentAllocation` is enabled.|500| |`druid.indexer.task.default.context`|Default task context that is applied to all tasks submitted to the Overlord. Any default in this config does not override neither the context values the user provides nor `druid.indexer.tasklock.forceTimeChunkLock`.|empty context| |`druid.indexer.queue.maxSize`|Maximum number of active tasks at one time.|Integer.MAX_VALUE| |`druid.indexer.queue.startDelay`|Sleep this long before starting Overlord queue management. This can be useful to give a cluster time to re-orient itself after e.g. a widespread network issue.|PT1M| diff --git a/docs/ingestion/tasks.md b/docs/ingestion/tasks.md index c8a2e915d472..5afbadb3d43a 100644 --- a/docs/ingestion/tasks.md +++ b/docs/ingestion/tasks.md @@ -343,6 +343,26 @@ You can override the task priority by setting your priority in the task context "priority" : 100 } ``` + + +## Task actions + +Task actions are overlord actions performed by tasks during their lifecycle. Some typical task actions are: +- `lockAcquire`: acquires a time-chunk lock on an interval for the task +- `lockRelease`: releases a lock acquired by the task on an interval +- `segmentTransactionalInsert`: publishes new segments created by a task and optionally overwrites and/or drops existing segments in a single transaction +- `segmentAllocate`: allocates pending segments to a task to write rows + +### Batching `segmentAllocate` actions + +In a cluster with several concurrent tasks, `segmentAllocate` actions on the overlord can take a long time to finish, causing spikes in the `task/action/run/time`. This can result in ingestion lag building up while a task waits for a segment to be allocated. +The root cause of such spikes is likely to be one or more of the following: +- several concurrent tasks trying to allocate segments for the same datasource and interval +- large number of metadata calls made to the segments and pending segments tables +- concurrency limitations while acquiring a task lock required for allocating a segment + +Since the contention typically arises from tasks allocating segments for the same datasource and interval, you can improve the run times by batching the actions together. +To enable batched segment allocation on the overlord, set `druid.indexer.tasklock.batchSegmentAllocation` to `true`. See [overlord configuration](../configuration/index.md#overlord-operations) for more details. diff --git a/docs/operations/metrics.md b/docs/operations/metrics.md index 8822f3fea73d..4e3c961dcb8d 100644 --- a/docs/operations/metrics.md +++ b/docs/operations/metrics.md @@ -230,8 +230,14 @@ Note: If the JVM does not support CPU time measurement for the current thread, ` |------|-----------|------------------------------------------------------------|------------| |`task/run/time`|Milliseconds taken to run a task.| `dataSource`, `taskId`, `taskType`, `taskStatus`|Varies| |`task/pending/time`|Milliseconds taken for a task to wait for running.| `dataSource`, `taskId`, `taskType`|Varies| -|`task/action/log/time`|Milliseconds taken to log a task action to the audit log.| `dataSource`, `taskId`, `taskType`|< 1000 (subsecond)| -|`task/action/run/time`|Milliseconds taken to execute a task action.| `dataSource`, `taskId`, `taskType`|Varies from subsecond to a few seconds, based on action type.| +|`task/action/log/time`|Milliseconds taken to log a task action to the audit log.| `dataSource`, `taskId`, `taskType`, `taskActionType`|< 1000 (subsecond)| +|`task/action/run/time`|Milliseconds taken to execute a task action.| `dataSource`, `taskId`, `taskType`, `taskActionType`|Varies from subsecond to a few seconds, based on action type.| +|`task/action/success/count`|Number of task actions that were executed successfully during the emission period. Currently only being emitted for [batched `segmentAllocate` actions](../ingestion/tasks.md#batching-segmentallocate-actions).| `dataSource`, `taskId`, `taskType`, `taskActionType`|Varies| +|`task/action/failed/count`|Number of task actions that failed during the emission period. Currently only being emitted for [batched `segmentAllocate` actions](../ingestion/tasks.md#batching-segmentallocate-actions).| `dataSource`, `taskId`, `taskType`, `taskActionType`|Varies| +|`task/action/batch/queueTime`|Milliseconds spent by a batch of task actions in queue. Currently only being emitted for [batched `segmentAllocate` actions](../ingestion/tasks.md#batching-segmentallocate-actions).| `dataSource`, `taskActionType`, `interval`|Varies based on the `batchAllocationWaitTime` and number of batches in queue.| +|`task/action/batch/runTime`|Milliseconds taken to execute a batch of task actions. Currently only being emitted for [batched `segmentAllocate` actions](../ingestion/tasks.md#batching-segmentallocate-actions).| `dataSource`, `taskActionType`, `interval`|Varies from subsecond to a few seconds, based on action type and batch size.| +|`task/action/batch/size`|Number of task actions in a batch that was executed during the emission period. Currently only being emitted for [batched `segmentAllocate` actions](../ingestion/tasks.md#batching-segmentallocate-actions).| `dataSource`, `taskActionType`, `interval`|Varies based on number of concurrent task actions.| +|`task/action/batch/attempts`|Number of execution attempts for a single batch of task actions. Currently only being emitted for [batched `segmentAllocate` actions](../ingestion/tasks.md#batching-segmentallocate-actions).| `dataSource`, `taskActionType`, `interval`|1 if there are no failures or retries.| |`segment/added/bytes`|Size in bytes of new segments created.| `dataSource`, `taskId`, `taskType`, `interval`|Varies| |`segment/moved/bytes`|Size in bytes of segments moved/archived via the Move Task.| `dataSource`, `taskId`, `taskType`, `interval`|Varies| |`segment/nuked/bytes`|Size in bytes of segments deleted via the Kill Task.| `dataSource`, `taskId`, `taskType`, `interval`|Varies| diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java index 897db9750354..03a4849e602e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java @@ -32,7 +32,7 @@ import org.apache.druid.discovery.DataNodeService; import org.apache.druid.discovery.DruidNodeAnnouncer; import org.apache.druid.discovery.LookupNodeService; -import org.apache.druid.indexing.common.actions.SegmentInsertAction; +import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTaskClientProvider; @@ -331,7 +331,11 @@ public void publishSegments(Iterable segments) throws IOException DataSegment::getInterval ); for (final Collection segmentCollection : segmentMultimap.asMap().values()) { - getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.copyOf(segmentCollection))); + getTaskActionClient().submit( + SegmentTransactionalInsertAction.appendAction( + ImmutableSet.copyOf(segmentCollection), null, null + ) + ); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java index 9ed53d99faf5..d60149d6245c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java @@ -70,6 +70,7 @@ public class SegmentAllocationQueue private static final Logger log = new Logger(SegmentAllocationQueue.class); private static final int MAX_QUEUE_SIZE = 2000; + private static final int MAX_BATCH_SIZE = 500; private final long maxWaitTimeMillis; @@ -94,7 +95,7 @@ public SegmentAllocationQueue( this.emitter = emitter; this.taskLockbox = taskLockbox; this.metadataStorage = metadataStorage; - this.maxWaitTimeMillis = taskLockConfig.getBatchAllocationMaxWaitTime(); + this.maxWaitTimeMillis = taskLockConfig.getBatchAllocationWaitTime(); this.executor = taskLockConfig.isBatchSegmentAllocation() ? executorFactory.create(1, "SegmentAllocQueue-%s") : null; @@ -173,7 +174,7 @@ public Future add(SegmentAllocateRequest request) throw new ISE("Batched segment allocation is disabled."); } - final AllocateRequestKey requestKey = new AllocateRequestKey(request, maxWaitTimeMillis); + final AllocateRequestKey requestKey = getKeyForAvailableBatch(request); final AtomicReference> futureReference = new AtomicReference<>(); // Possible race condition: @@ -198,6 +199,24 @@ public Future add(SegmentAllocateRequest request) return futureReference.get(); } + /** + * Returns the key for a batch that is not added to the queue yet and/or has + * available space. Throws an exception if the queue is already full and no + * batch has available capacity. + */ + private AllocateRequestKey getKeyForAvailableBatch(SegmentAllocateRequest request) + { + for (int batchIncrementalId = 0; batchIncrementalId < MAX_QUEUE_SIZE; ++batchIncrementalId) { + AllocateRequestKey nextKey = new AllocateRequestKey(request, maxWaitTimeMillis, batchIncrementalId); + AllocateRequestBatch nextBatch = keyToBatch.get(nextKey); + if (nextBatch == null || nextBatch.size() < MAX_BATCH_SIZE) { + return nextKey; + } + } + + throw new ISE("Allocation queue is at capacity, all batches are full."); + } + /** * Tries to add the given batch to the processing queue. Fails all the pending * requests in the batch if we are not leader or if the queue is full. @@ -616,6 +635,11 @@ synchronized int size() */ private static class AllocateRequestKey { + /** + * ID to distinguish between two batches for the same datasource, groupId, etc. + */ + private final int batchIncrementalId; + private long queueTimeMillis; private final long maxWaitTimeMillis; @@ -635,11 +659,12 @@ private static class AllocateRequestKey * Creates a new key for the given request. The batch for a unique key will * always contain a single request. */ - AllocateRequestKey(SegmentAllocateRequest request, long maxWaitTimeMillis) + AllocateRequestKey(SegmentAllocateRequest request, long maxWaitTimeMillis, int batchIncrementalId) { final SegmentAllocateAction action = request.getAction(); final Task task = request.getTask(); + this.batchIncrementalId = batchIncrementalId; this.dataSource = action.getDataSource(); this.groupId = task.getGroupId(); this.skipSegmentLineageCheck = action.isSkipSegmentLineageCheck(); @@ -651,10 +676,11 @@ private static class AllocateRequestKey .bucket(action.getTimestamp()); this.hash = Objects.hash( - skipSegmentLineageCheck, - useNonRootGenPartitionSpace, dataSource, groupId, + batchIncrementalId, + skipSegmentLineageCheck, + useNonRootGenPartitionSpace, preferredAllocationInterval, lockGranularity ); @@ -687,10 +713,11 @@ public boolean equals(Object o) return false; } AllocateRequestKey that = (AllocateRequestKey) o; - return skipSegmentLineageCheck == that.skipSegmentLineageCheck - && useNonRootGenPartitionSpace == that.useNonRootGenPartitionSpace - && dataSource.equals(that.dataSource) + return dataSource.equals(that.dataSource) && groupId.equals(that.groupId) + && batchIncrementalId == that.batchIncrementalId + && skipSegmentLineageCheck == that.skipSegmentLineageCheck + && useNonRootGenPartitionSpace == that.useNonRootGenPartitionSpace && preferredAllocationInterval.equals(that.preferredAllocationInterval) && lockGranularity == that.lockGranularity; } @@ -707,6 +734,7 @@ public String toString() return "{" + "ds='" + dataSource + '\'' + ", gr='" + groupId + '\'' + + ", incId=" + batchIncrementalId + ", lock=" + lockGranularity + ", invl=" + preferredAllocationInterval + ", slc=" + skipSegmentLineageCheck + diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/TaskLockConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/TaskLockConfig.java index acbc318baabc..c860e4e1d156 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/TaskLockConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/TaskLockConfig.java @@ -34,7 +34,7 @@ public class TaskLockConfig private boolean batchSegmentAllocation = false; @JsonProperty - private long batchAllocationMaxWaitTime = 500L; + private long batchAllocationWaitTime = 500L; public boolean isForceTimeChunkLock() { @@ -46,8 +46,8 @@ public boolean isBatchSegmentAllocation() return batchSegmentAllocation; } - public long getBatchAllocationMaxWaitTime() + public long getBatchAllocationWaitTime() { - return batchAllocationMaxWaitTime; + return batchAllocationWaitTime; } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueueTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueueTest.java index 536e9ffac24c..974b3096f92d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueueTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueueTest.java @@ -71,7 +71,7 @@ public boolean isBatchSegmentAllocation() } @Override - public long getBatchAllocationMaxWaitTime() + public long getBatchAllocationWaitTime() { return 0; } @@ -249,6 +249,23 @@ public void testFullAllocationQueue() ); } + @Test + public void testMaxBatchSize() + { + for (int i = 0; i < 500; ++i) { + SegmentAllocateRequest request = + allocateRequest().forTask(createTask(DS_WIKI, "group_1")).build(); + allocationQueue.add(request); + } + + // Verify that next request is added to a new batch + Assert.assertEquals(1, allocationQueue.size()); + SegmentAllocateRequest request = + allocateRequest().forTask(createTask(DS_WIKI, "group_1")).build(); + allocationQueue.add(request); + Assert.assertEquals(2, allocationQueue.size()); + } + @Test public void testMultipleRequestsForSameSegment() { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java index 189d3fe8779c..eebf78a7ddcb 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java @@ -112,7 +112,7 @@ public boolean isBatchSegmentAllocation() } @Override - public long getBatchAllocationMaxWaitTime() + public long getBatchAllocationWaitTime() { return 10L; } From 2effa544597db8fb33a525e9be31de184b33b3d7 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Thu, 8 Dec 2022 12:44:14 +0530 Subject: [PATCH 26/34] Remove stray reference to fix OOM while merging sketches (#13475) (#13529) * Remove stray reference to fix OOM while merging sketches * Update future to add result from executor service * Update tests and address review comments * Address review comments * Moved mock * Close threadpool on teardown * Remove worker task cancel --- .../druid/msq/exec/WorkerSketchFetcher.java | 36 ++++--- .../msq/exec/WorkerSketchFetcherTest.java | 99 +------------------ 2 files changed, 19 insertions(+), 116 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java index c4118a9d38e0..dc6f21990587 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java @@ -59,7 +59,11 @@ public class WorkerSketchFetcher implements AutoCloseable private final WorkerClient workerClient; private final ExecutorService executorService; - public WorkerSketchFetcher(WorkerClient workerClient, ClusterStatisticsMergeMode clusterStatisticsMergeMode, int statisticsMaxRetainedBytes) + public WorkerSketchFetcher( + WorkerClient workerClient, + ClusterStatisticsMergeMode clusterStatisticsMergeMode, + int statisticsMaxRetainedBytes + ) { this.workerClient = workerClient; this.clusterStatisticsMergeMode = clusterStatisticsMergeMode; @@ -86,14 +90,14 @@ public CompletableFuture> submitFetcherTask( return inMemoryFullSketchMerging(stageDefinition, workerTaskIds); case AUTO: if (clusterBy.getBucketByCount() == 0) { - log.debug("Query [%s] AUTO mode: chose PARALLEL mode to merge key statistics", stageDefinition.getId().getQueryId()); + log.info("Query [%s] AUTO mode: chose PARALLEL mode to merge key statistics", stageDefinition.getId().getQueryId()); // If there is no time clustering, there is no scope for sequential merge return inMemoryFullSketchMerging(stageDefinition, workerTaskIds); } else if (stageDefinition.getMaxWorkerCount() > WORKER_THRESHOLD || completeKeyStatisticsInformation.getBytesRetained() > BYTES_THRESHOLD) { - log.debug("Query [%s] AUTO mode: chose SEQUENTIAL mode to merge key statistics", stageDefinition.getId().getQueryId()); + log.info("Query [%s] AUTO mode: chose SEQUENTIAL mode to merge key statistics", stageDefinition.getId().getQueryId()); return sequentialTimeChunkMerging(completeKeyStatisticsInformation, stageDefinition, workerTaskIds); } - log.debug("Query [%s] AUTO mode: chose PARALLEL mode to merge key statistics", stageDefinition.getId().getQueryId()); + log.info("Query [%s] AUTO mode: chose PARALLEL mode to merge key statistics", stageDefinition.getId().getQueryId()); return inMemoryFullSketchMerging(stageDefinition, workerTaskIds); default: throw new IllegalStateException("No fetching strategy found for mode: " + clusterStatisticsMergeMode); @@ -128,12 +132,6 @@ CompletableFuture> inMemoryFullSketchMerging( stageDefinition.getId().getQueryId(), stageDefinition.getStageNumber() ); - partitionFuture.whenComplete((result, exception) -> { - if (exception != null || (result != null && result.isError())) { - snapshotFuture.cancel(true); - } - }); - try { ClusterByStatisticsSnapshot clusterByStatisticsSnapshot = snapshotFuture.get(); if (clusterByStatisticsSnapshot == null) { @@ -151,12 +149,15 @@ CompletableFuture> inMemoryFullSketchMerging( } catch (Exception e) { synchronized (mergedStatisticsCollector) { - partitionFuture.completeExceptionally(e); - mergedStatisticsCollector.clear(); + if (!partitionFuture.isDone()) { + partitionFuture.completeExceptionally(e); + mergedStatisticsCollector.clear(); + } } } }); }); + return partitionFuture; } @@ -247,11 +248,6 @@ public void submitFetchingTasksForNextTimeChunk() stageDefinition.getStageNumber(), timeChunk ); - partitionFuture.whenComplete((result, exception) -> { - if (exception != null || (result != null && result.isError())) { - snapshotFuture.cancel(true); - } - }); try { ClusterByStatisticsSnapshot snapshotForTimeChunk = snapshotFuture.get(); @@ -289,8 +285,10 @@ public void submitFetchingTasksForNextTimeChunk() } catch (Exception e) { synchronized (mergedStatisticsCollector) { - partitionFuture.completeExceptionally(e); - mergedStatisticsCollector.clear(); + if (!partitionFuture.isDone()) { + partitionFuture.completeExceptionally(e); + mergedStatisticsCollector.clear(); + } } } }); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherTest.java index 54c9a792e558..83fb73043bd9 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerSketchFetcherTest.java @@ -23,7 +23,6 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.ImmutableSortedMap; import com.google.common.util.concurrent.Futures; -import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.frame.key.ClusterBy; import org.apache.druid.frame.key.ClusterByPartition; import org.apache.druid.frame.key.ClusterByPartitions; @@ -46,7 +45,6 @@ import java.util.Set; import java.util.SortedMap; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CompletionException; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.CountDownLatch; import java.util.concurrent.CyclicBarrier; @@ -56,7 +54,6 @@ import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyLong; -import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; @@ -107,52 +104,8 @@ public void setUp() public void tearDown() throws Exception { mocks.close(); - } - - @Test - public void test_submitFetcherTask_parallelFetch_workerThrowsException_shouldCancelOtherTasks() throws Exception - { - // Store futures in a queue - final Queue> futureQueue = new ConcurrentLinkedQueue<>(); - final List workerIds = ImmutableList.of("0", "1", "2", "3"); - final CountDownLatch latch = new CountDownLatch(workerIds.size()); - - target = spy(new WorkerSketchFetcher(workerClient, ClusterStatisticsMergeMode.PARALLEL, 300_000_000)); - - // When fetching snapshots, return a mock and add future to queue - doAnswer(invocation -> { - ListenableFuture snapshotListenableFuture = - spy(Futures.immediateFuture(mock(ClusterByStatisticsSnapshot.class))); - futureQueue.add(snapshotListenableFuture); - latch.countDown(); - latch.await(); - return snapshotListenableFuture; - }).when(workerClient).fetchClusterByStatisticsSnapshot(any(), any(), anyInt()); - - // Cause a worker to fail instead of returning the result - doAnswer(invocation -> { - latch.countDown(); - latch.await(); - return Futures.immediateFailedFuture(new InterruptedException("interrupted")); - }).when(workerClient).fetchClusterByStatisticsSnapshot(eq("2"), any(), anyInt()); - - CompletableFuture> eitherCompletableFuture = target.submitFetcherTask( - completeKeyStatisticsInformation, - workerIds, - stageDefinition - ); - - // Assert that the final result is failed and all other task futures are also cancelled. - Assert.assertThrows(CompletionException.class, eitherCompletableFuture::join); - Thread.sleep(1000); - - Assert.assertTrue(eitherCompletableFuture.isCompletedExceptionally()); - // Verify that the statistics collector was cleared due to the error. - verify(mergedClusterByStatisticsCollector1, times(1)).clear(); - // Verify that other task futures were requested to be cancelled. - Assert.assertFalse(futureQueue.isEmpty()); - for (ListenableFuture snapshotFuture : futureQueue) { - verify(snapshotFuture, times(1)).cancel(eq(true)); + if (target != null) { + target.close(); } } @@ -194,54 +147,6 @@ public void test_submitFetcherTask_parallelFetch_mergePerformedCorrectly() Assert.assertEquals(expectedPartitions1, eitherCompletableFuture.get().valueOrThrow()); } - @Test - public void test_submitFetcherTask_sequentialFetch_workerThrowsException_shouldCancelOtherTasks() throws Exception - { - // Store futures in a queue - final Queue> futureQueue = new ConcurrentLinkedQueue<>(); - - SortedMap> timeSegmentVsWorkerMap = ImmutableSortedMap.of(1L, ImmutableSet.of(0, 1, 2), 2L, ImmutableSet.of(0, 1, 4)); - doReturn(timeSegmentVsWorkerMap).when(completeKeyStatisticsInformation).getTimeSegmentVsWorkerMap(); - - final CyclicBarrier barrier = new CyclicBarrier(3); - target = spy(new WorkerSketchFetcher(workerClient, ClusterStatisticsMergeMode.SEQUENTIAL, 300_000_000)); - - // When fetching snapshots, return a mock and add future to queue - doAnswer(invocation -> { - ListenableFuture snapshotListenableFuture = - spy(Futures.immediateFuture(mock(ClusterByStatisticsSnapshot.class))); - futureQueue.add(snapshotListenableFuture); - barrier.await(); - return snapshotListenableFuture; - }).when(workerClient).fetchClusterByStatisticsSnapshotForTimeChunk(anyString(), anyString(), anyInt(), anyLong()); - - // Cause a worker in the second time chunk to fail instead of returning the result - doAnswer(invocation -> { - barrier.await(); - return Futures.immediateFailedFuture(new InterruptedException("interrupted")); - }).when(workerClient).fetchClusterByStatisticsSnapshotForTimeChunk(eq("4"), any(), anyInt(), eq(2L)); - - CompletableFuture> eitherCompletableFuture = target.submitFetcherTask( - completeKeyStatisticsInformation, - ImmutableList.of("0", "1", "2", "3", "4"), - stageDefinition - ); - - // Assert that the final result is failed and all other task futures are also cancelled. - Assert.assertThrows(CompletionException.class, eitherCompletableFuture::join); - Thread.sleep(1000); - - Assert.assertTrue(eitherCompletableFuture.isCompletedExceptionally()); - // Verify that the correct statistics collector was cleared due to the error. - verify(mergedClusterByStatisticsCollector1, times(0)).clear(); - verify(mergedClusterByStatisticsCollector2, times(1)).clear(); - // Verify that other task futures were requested to be cancelled. - Assert.assertFalse(futureQueue.isEmpty()); - for (ListenableFuture snapshotFuture : futureQueue) { - verify(snapshotFuture, times(1)).cancel(eq(true)); - } - } - @Test public void test_submitFetcherTask_sequentialFetch_mergePerformedCorrectly() throws ExecutionException, InterruptedException From 977792d03d452742efa39fedabaeb05924a691a2 Mon Sep 17 00:00:00 2001 From: Karan Kumar Date: Sat, 10 Dec 2022 08:30:47 +0530 Subject: [PATCH 27/34] MSQ: Only look at sqlInsertSegmentGranularity on the outer query. (#13537) (#13542) The planner sets sqlInsertSegmentGranularity in its context when using PARTITIONED BY, which sets it on every native query in the stack (as all native queries for a SQL query typically have the same context). QueryKit would interpret that as a request to configure bucketing for all native queries. This isn't useful, as bucketing is only used for the penultimate stage in INSERT / REPLACE. So, this patch modifies QueryKit to only look at sqlInsertSegmentGranularity on the outermost query. As an additional change, this patch switches the static ObjectMapper to use the processwide ObjectMapper for deserializing Granularities. Saves an ObjectMapper instance, and ensures that if there are any special serdes registered for Granularity, we'll pick them up. (cherry picked from commit 55814888f54ebd6a909c60cc0638bf4ea306349f) Co-authored-by: Gian Merlino --- .../apache/druid/msq/exec/ControllerImpl.java | 2 +- .../druid/msq/querykit/DataSourcePlan.java | 16 +++++++++++++++- .../apache/druid/msq/querykit/QueryKitUtils.java | 15 ++++++++------- .../GroupByPostShuffleFrameProcessor.java | 14 ++++++++++---- .../GroupByPostShuffleFrameProcessorFactory.java | 3 ++- .../msq/querykit/groupby/GroupByQueryKit.java | 11 ++++++++++- .../querykit/scan/ScanQueryFrameProcessor.java | 7 +++++-- .../scan/ScanQueryFrameProcessorFactory.java | 3 ++- .../druid/msq/querykit/scan/ScanQueryKit.java | 3 ++- .../apache/druid/msq/sql/MSQTaskSqlEngine.java | 1 + .../scan/ScanQueryFrameProcessorTest.java | 4 +++- 11 files changed, 59 insertions(+), 20 deletions(-) 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 318c33a759c7..528baa4c27d1 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 @@ -989,7 +989,7 @@ private QueryKit makeQueryControllerToolKit() final Map, QueryKit> kitMap = ImmutableMap., QueryKit>builder() .put(ScanQuery.class, new ScanQueryKit(context.jsonMapper())) - .put(GroupByQuery.class, new GroupByQueryKit()) + .put(GroupByQuery.class, new GroupByQueryKit(context.jsonMapper())) .build(); return new MultiQueryKit(kitMap); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSourcePlan.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSourcePlan.java index a5c61c5bd75e..30544cf31bff 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSourcePlan.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSourcePlan.java @@ -51,6 +51,7 @@ import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.sql.calcite.external.ExternalDataSource; +import org.apache.druid.sql.calcite.parser.DruidSqlInsert; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -67,6 +68,16 @@ */ public class DataSourcePlan { + /** + * A map with {@link DruidSqlInsert#SQL_INSERT_SEGMENT_GRANULARITY} set to null, so we can clear it from the context + * of subqueries. + */ + private static final Map CONTEXT_MAP_NO_SEGMENT_GRANULARITY = new HashMap<>(); + + static { + CONTEXT_MAP_NO_SEGMENT_GRANULARITY.put(DruidSqlInsert.SQL_INSERT_SEGMENT_GRANULARITY, null); + } + private final DataSource newDataSource; private final List inputSpecs; private final IntSet broadcastInputs; @@ -247,7 +258,10 @@ private static DataSourcePlan forQuery( { final QueryDefinition subQueryDef = queryKit.makeQueryDefinition( queryId, - dataSource.getQuery(), + + // Subqueries ignore SQL_INSERT_SEGMENT_GRANULARITY, even if set in the context. It's only used for the + // outermost query, and setting it for the subquery makes us erroneously add bucketing where it doesn't belong. + dataSource.getQuery().withOverriddenContext(CONTEXT_MAP_NO_SEGMENT_GRANULARITY), queryKit, ShuffleSpecFactories.subQueryWithMaxWorkerCount(maxWorkerCount), maxWorkerCount, diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/QueryKitUtils.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/QueryKitUtils.java index fcd723291650..1f863a8c735e 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/QueryKitUtils.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/QueryKitUtils.java @@ -24,7 +24,6 @@ import org.apache.calcite.sql.dialect.CalciteSqlDialect; import org.apache.druid.frame.key.ClusterBy; import org.apache.druid.frame.key.SortColumn; -import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; @@ -76,15 +75,16 @@ public class QueryKitUtils */ public static final String CTX_TIME_COLUMN_NAME = "__timeColumn"; - private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper(); - - public static Granularity getSegmentGranularityFromContext(@Nullable final Map context) + public static Granularity getSegmentGranularityFromContext( + final ObjectMapper objectMapper, + @Nullable final Map context + ) { final Object o = context == null ? null : context.get(DruidSqlInsert.SQL_INSERT_SEGMENT_GRANULARITY); if (o instanceof String) { try { - return OBJECT_MAPPER.readValue((String) o, Granularity.class); + return objectMapper.readValue((String) o, Granularity.class); } catch (JsonProcessingException e) { throw new ISE("Invalid segment granularity [%s]", o); @@ -188,9 +188,10 @@ public static RowSignature sortableSignature( * @throws IllegalArgumentException if the provided granularity is not supported */ @Nullable - public static VirtualColumn makeSegmentGranularityVirtualColumn(final Query query) + public static VirtualColumn makeSegmentGranularityVirtualColumn(final ObjectMapper jsonMapper, final Query query) { - final Granularity segmentGranularity = QueryKitUtils.getSegmentGranularityFromContext(query.getContext()); + final Granularity segmentGranularity = + QueryKitUtils.getSegmentGranularityFromContext(jsonMapper, query.getContext()); final String timeColumnName = query.context().getString(QueryKitUtils.CTX_TIME_COLUMN_NAME); if (timeColumnName == null || Granularities.ALL.equals(segmentGranularity)) { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPostShuffleFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPostShuffleFrameProcessor.java index a44c14e3cdc2..207fe53de03a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPostShuffleFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPostShuffleFrameProcessor.java @@ -19,6 +19,7 @@ package org.apache.druid.msq.querykit.groupby; +import com.fasterxml.jackson.databind.ObjectMapper; import it.unimi.dsi.fastutil.ints.IntSet; import org.apache.druid.frame.Frame; import org.apache.druid.frame.FrameType; @@ -92,7 +93,8 @@ public GroupByPostShuffleFrameProcessor( final FrameReader frameReader, final RowSignature resultSignature, final ClusterBy clusterBy, - final MemoryAllocator allocator + final MemoryAllocator allocator, + final ObjectMapper jsonMapper ) { this.query = query; @@ -107,7 +109,7 @@ public GroupByPostShuffleFrameProcessor( this.finalizeFn = makeFinalizeFn(query); this.havingSpec = cloneHavingSpec(query); this.columnSelectorFactoryForFrameWriter = - makeVirtualColumnsForFrameWriter(query).wrap( + makeVirtualColumnsForFrameWriter(jsonMapper, query).wrap( RowBasedGrouperHelper.createResultRowBasedColumnSelectorFactory( query, () -> outputRow, @@ -311,9 +313,13 @@ private static HavingSpec cloneHavingSpec(final GroupByQuery query) * Create virtual columns containing "bonus" fields that should be attached to the {@link FrameWriter} for * this processor. Kept in sync with the signature generated by {@link GroupByQueryKit}. */ - private static VirtualColumns makeVirtualColumnsForFrameWriter(final GroupByQuery query) + private static VirtualColumns makeVirtualColumnsForFrameWriter( + final ObjectMapper jsonMapper, + final GroupByQuery query + ) { - final VirtualColumn segmentGranularityVirtualColumn = QueryKitUtils.makeSegmentGranularityVirtualColumn(query); + final VirtualColumn segmentGranularityVirtualColumn = + QueryKitUtils.makeSegmentGranularityVirtualColumn(jsonMapper, query); if (segmentGranularityVirtualColumn == null) { return VirtualColumns.EMPTY; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPostShuffleFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPostShuffleFrameProcessorFactory.java index 5987eb02fae0..ffb8bacf5e62 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPostShuffleFrameProcessorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPostShuffleFrameProcessorFactory.java @@ -118,7 +118,8 @@ public ProcessorsAndChannels, Long> makeProcessors( readableInput.getChannelFrameReader(), stageDefinition.getSignature(), stageDefinition.getClusterBy(), - outputChannel.getFrameMemoryAllocator() + outputChannel.getFrameMemoryAllocator(), + frameContext.jsonMapper() ); } ); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByQueryKit.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByQueryKit.java index 411fe118a29e..402d2dfa3d8c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByQueryKit.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByQueryKit.java @@ -19,6 +19,7 @@ package org.apache.druid.msq.querykit.groupby; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import org.apache.druid.frame.key.ClusterBy; import org.apache.druid.frame.key.SortColumn; @@ -56,6 +57,13 @@ public class GroupByQueryKit implements QueryKit { + private final ObjectMapper jsonMapper; + + public GroupByQueryKit(ObjectMapper jsonMapper) + { + this.jsonMapper = jsonMapper; + } + @Override public QueryDefinition makeQueryDefinition( final String queryId, @@ -85,7 +93,8 @@ public QueryDefinition makeQueryDefinition( final GroupByQuery queryToRun = (GroupByQuery) originalQuery.withDataSource(dataSourcePlan.getNewDataSource()); final int firstStageNumber = Math.max(minStageNumber, queryDefBuilder.getNextStageNumber()); - final Granularity segmentGranularity = QueryKitUtils.getSegmentGranularityFromContext(queryToRun.getContext()); + final Granularity segmentGranularity = + QueryKitUtils.getSegmentGranularityFromContext(jsonMapper, queryToRun.getContext()); final RowSignature intermediateSignature = computeIntermediateSignature(queryToRun); final ClusterBy resultClusterBy = QueryKitUtils.clusterByWithSegmentGranularity(computeClusterByForResults(queryToRun), segmentGranularity); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java index 307d274c73bf..0482e2715dc7 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java @@ -19,6 +19,7 @@ package org.apache.druid.msq.querykit.scan; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; import it.unimi.dsi.fastutil.ints.Int2ObjectMap; @@ -92,7 +93,8 @@ public ScanQueryFrameProcessor( final ResourceHolder outputChannel, final ResourceHolder frameWriterFactoryHolder, @Nullable final AtomicLong runningCountForLimit, - final long memoryReservedForBroadcastJoin + final long memoryReservedForBroadcastJoin, + final ObjectMapper jsonMapper ) { super( @@ -111,7 +113,8 @@ public ScanQueryFrameProcessor( final List frameWriterVirtualColumns = new ArrayList<>(); frameWriterVirtualColumns.add(partitionBoostVirtualColumn); - final VirtualColumn segmentGranularityVirtualColumn = QueryKitUtils.makeSegmentGranularityVirtualColumn(query); + final VirtualColumn segmentGranularityVirtualColumn = + QueryKitUtils.makeSegmentGranularityVirtualColumn(jsonMapper, query); if (segmentGranularityVirtualColumn != null) { frameWriterVirtualColumns.add(segmentGranularityVirtualColumn); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessorFactory.java index 2a948fd45625..bda53af69641 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessorFactory.java @@ -100,7 +100,8 @@ protected FrameProcessor makeProcessor( allocatorHolder )), runningCountForLimit, - frameContext.memoryParameters().getBroadcastJoinMemory() + frameContext.memoryParameters().getBroadcastJoinMemory(), + frameContext.jsonMapper() ); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryKit.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryKit.java index 5bfb70b52c91..9e44f152eb2d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryKit.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryKit.java @@ -116,7 +116,8 @@ public QueryDefinition makeQueryDefinition( signatureToUse = scanSignature; } else { final RowSignature.Builder signatureBuilder = RowSignature.builder().addAll(scanSignature); - final Granularity segmentGranularity = QueryKitUtils.getSegmentGranularityFromContext(queryToRun.getContext()); + final Granularity segmentGranularity = + QueryKitUtils.getSegmentGranularityFromContext(jsonMapper, queryToRun.getContext()); final List clusterByColumns = new ArrayList<>(); // Add regular orderBys. diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java index a91844114dda..2ec08e03783d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java @@ -207,6 +207,7 @@ private static void validateInsert( try { segmentGranularity = QueryKitUtils.getSegmentGranularityFromContext( + plannerContext.getJsonMapper(), plannerContext.queryContextMap() ); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessorTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessorTest.java index 2ea2958c7368..d93e8df42df9 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessorTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessorTest.java @@ -37,6 +37,7 @@ import org.apache.druid.frame.write.FrameWriter; import org.apache.druid.frame.write.FrameWriterFactory; import org.apache.druid.frame.write.FrameWriters; +import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.concurrent.Execs; @@ -152,7 +153,8 @@ public void close() }, new LazyResourceHolder<>(() -> Pair.of(frameWriterFactory, () -> {})), null, - 0L + 0L, + new DefaultObjectMapper() ); ListenableFuture retVal = exec.runFully(processor, null); From 348c9f64f9ca12803b8dcefe596d1f9e58db41ca Mon Sep 17 00:00:00 2001 From: Vadim Ogievetsky Date: Fri, 9 Dec 2022 19:03:55 -0800 Subject: [PATCH 28/34] [Backport] backport ux bug fixes to 25 (#13533) * Web console: add arrayOfDoublesSketch and other small fixes (#13486) * add padding and keywords * add arrayOfDoubles * Update docs/development/extensions-core/datasketches-tuple.md Co-authored-by: Charles Smith * Update docs/development/extensions-core/datasketches-tuple.md Co-authored-by: Charles Smith * Update docs/development/extensions-core/datasketches-tuple.md Co-authored-by: Charles Smith * Update docs/development/extensions-core/datasketches-tuple.md Co-authored-by: Charles Smith * Update docs/development/extensions-core/datasketches-tuple.md Co-authored-by: Charles Smith * partiton int * fix docs Co-authored-by: Charles Smith * Web console: improve compaction status display (#13523) * improve compaction status display * even more accurate * fix snapshot * MSQ: Improve TooManyBuckets error message, improve error docs. (#13525) 1) Edited the TooManyBuckets error message to mention PARTITIONED BY instead of segmentGranularity. 2) Added error-code-specific anchors in the docs. 3) Add information to various error codes in the docs about common causes and solutions. * update error anchors (#13527) * update snapshot Co-authored-by: Charles Smith Co-authored-by: Gian Merlino --- .../extensions-core/datasketches-tuple.md | 43 +++++- docs/multi-stage-query/concepts.md | 3 +- docs/multi-stage-query/known-issues.md | 8 +- docs/multi-stage-query/reference.md | 70 ++++----- .../indexing/error/TooManyBucketsFault.java | 2 +- web-console/lib/keywords.js | 3 + web-console/script/create-sql-docs.js | 4 +- .../src/bootstrap/react-table-defaults.tsx | 4 +- .../segment-timeline/segment-timeline.tsx | 2 +- .../compaction-dialog/compaction-dialog.scss | 5 + .../compaction-dialog/compaction-dialog.tsx | 27 +++- .../compaction-config/compaction-config.tsx | 29 +++- .../compaction-status.spec.ts | 146 +++++++++++++----- .../compaction-status/compaction-status.ts | 23 ++- .../ingest-query-pattern.ts | 3 +- .../druid-models/metric-spec/metric-spec.tsx | 43 ++++++ .../workbench-query/workbench-query.ts | 8 +- .../helpers/execution/sql-task-execution.ts | 9 +- web-console/src/links.ts | 2 +- .../src/react-table/react-table-extra.scss | 4 + .../datasources-view.spec.tsx.snap | 2 +- .../datasources-view/datasources-view.tsx | 2 +- .../sql-data-loader-view.tsx | 2 + .../connect-external-data-dialog.tsx | 18 ++- .../execution-error-pane.spec.tsx.snap | 2 +- .../execution-error-pane.tsx | 4 +- .../input-source-step/example-inputs.ts | 3 + .../input-source-step/input-source-step.tsx | 12 +- .../views/workbench-view/workbench-view.tsx | 9 +- 29 files changed, 372 insertions(+), 120 deletions(-) diff --git a/docs/development/extensions-core/datasketches-tuple.md b/docs/development/extensions-core/datasketches-tuple.md index fc4f74d5c81d..c9a05b5ab197 100644 --- a/docs/development/extensions-core/datasketches-tuple.md +++ b/docs/development/extensions-core/datasketches-tuple.md @@ -39,19 +39,52 @@ druid.extensions.loadList=["druid-datasketches"] "name" : , "fieldName" : , "nominalEntries": , - "numberOfValues" : , - "metricColumns" : + "metricColumns" : , + "numberOfValues" : } ``` |property|description|required?| |--------|-----------|---------| |type|This String should always be "arrayOfDoublesSketch"|yes| -|name|A String for the output (result) name of the calculation.|yes| +|name|String representing the output column to store sketch values.|yes| |fieldName|A String for the name of the input field.|yes| |nominalEntries|Parameter that determines the accuracy and size of the sketch. Higher k means higher accuracy but more space to store sketches. Must be a power of 2. See the [Theta sketch accuracy](https://datasketches.apache.org/docs/Theta/ThetaErrorTable) for details. |no, defaults to 16384| -|numberOfValues|Number of values associated with each distinct key. |no, defaults to 1| -|metricColumns|If building sketches from raw data, an array of names of the input columns containing numeric values to be associated with each distinct key.|no, defaults to empty array| +|metricColumns|When building sketches from raw data, an array input column that contain numeric values to associate with each distinct key. If not provided, assumes `fieldName` is an `arrayOfDoublesSketch`|no, if not provided `fieldName` is assumed to be an arrayOfDoublesSketch| +|numberOfValues|Number of values associated with each distinct key. |no, defaults to the length of `metricColumns` if provided and 1 otherwise| + +You can use the `arrayOfDoublesSketch` aggregator to: + +- Build a sketch from raw data. In this case, set `metricColumns` to an array. +- Build a sketch from an existing `ArrayOfDoubles` sketch . In this case, leave `metricColumns` unset and set the `fieldName` to an `ArrayOfDoubles` sketch with `numberOfValues` doubles. At ingestion time, you must base64 encode `ArrayOfDoubles` sketches at ingestion time. + +#### Example on top of raw data + +Compute a theta of unique users. For each user store the `added` and `deleted` scores. The new sketch column will be called `users_theta`. + +```json +{ + "type": "arrayOfDoublesSketch", + "name": "users_theta", + "fieldName": "user", + "nominalEntries": 16384, + "metricColumns": ["added", "deleted"], +} +``` + +#### Example ingesting a precomputed sketch column + +Ingest a sketch column called `user_sketches` that has a base64 encoded value of two doubles in its array and store it in a column called `users_theta`. + +```json +{ + "type": "arrayOfDoublesSketch", + "name": "users_theta", + "fieldName": "user_sketches", + "nominalEntries": 16384, + "numberOfValues": 2, +} +``` ### Post Aggregators diff --git a/docs/multi-stage-query/concepts.md b/docs/multi-stage-query/concepts.md index 44e5ea43d427..da0e774152d6 100644 --- a/docs/multi-stage-query/concepts.md +++ b/docs/multi-stage-query/concepts.md @@ -233,7 +233,8 @@ happens: The [`maxNumTasks`](./reference.md#context-parameters) query parameter determines the maximum number of tasks your query will use, including the one `query_controller` task. Generally, queries perform better with more workers. The lowest possible value of `maxNumTasks` is two (one worker and one controller). Do not set this higher than the number of -free slots available in your cluster; doing so will result in a [TaskStartTimeout](reference.md#error-codes) error. +free slots available in your cluster; doing so will result in a [TaskStartTimeout](reference.md#error_TaskStartTimeout) +error. When [reading external data](#extern), EXTERN can read multiple files in parallel across different worker tasks. However, EXTERN does not split individual files across multiple worker tasks. If you have a diff --git a/docs/multi-stage-query/known-issues.md b/docs/multi-stage-query/known-issues.md index c76ab57aa7ac..648d3c297b47 100644 --- a/docs/multi-stage-query/known-issues.md +++ b/docs/multi-stage-query/known-issues.md @@ -33,16 +33,18 @@ sidebar_label: Known issues - Worker task stage outputs are stored in the working directory given by `druid.indexer.task.baseDir`. Stages that generate a large amount of output data may exhaust all available disk space. In this case, the query fails with -an [UnknownError](./reference.md#error-codes) with a message including "No space left on device". +an [UnknownError](./reference.md#error_UnknownError) with a message including "No space left on device". ## SELECT - SELECT from a Druid datasource does not include unpublished real-time data. - GROUPING SETS and UNION ALL are not implemented. Queries using these features return a - [QueryNotSupported](reference.md#error-codes) error. + [QueryNotSupported](reference.md#error_QueryNotSupported) error. -- For some COUNT DISTINCT queries, you'll encounter a [QueryNotSupported](reference.md#error-codes) error that includes `Must not have 'subtotalsSpec'` as one of its causes. This is caused by the planner attempting to use GROUPING SETs, which are not implemented. +- For some COUNT DISTINCT queries, you'll encounter a [QueryNotSupported](reference.md#error_QueryNotSupported) error + that includes `Must not have 'subtotalsSpec'` as one of its causes. This is caused by the planner attempting to use + GROUPING SETs, which are not implemented. - The numeric varieties of the EARLIEST and LATEST aggregators do not work properly. Attempting to use the numeric varieties of these aggregators lead to an error like diff --git a/docs/multi-stage-query/reference.md b/docs/multi-stage-query/reference.md index a4bcbfc27b1f..ae9bc106ca8c 100644 --- a/docs/multi-stage-query/reference.md +++ b/docs/multi-stage-query/reference.md @@ -249,14 +249,14 @@ The following table lists query limits: | Limit | Value | Error if exceeded | |---|---|---| -| Size of an individual row written to a frame. Row size when written to a frame may differ from the original row size. | 1 MB | `RowTooLarge` | -| Number of segment-granular time chunks encountered during ingestion. | 5,000 | `TooManyBuckets` | -| Number of input files/segments per worker. | 10,000 | `TooManyInputFiles` | -| Number of output partitions for any one stage. Number of segments generated during ingestion. |25,000 | `TooManyPartitions` | -| Number of output columns for any one stage. | 2,000 | `TooManyColumns` | -| Number of cluster by columns that can appear in a stage | 1,500 | `TooManyClusteredByColumns` | -| Number of workers for any one stage. | Hard limit is 1,000. Memory-dependent soft limit may be lower. | `TooManyWorkers` | -| Maximum memory occupied by broadcasted tables. | 30% of each [processor memory bundle](concepts.md#memory-usage). | `BroadcastTablesTooLarge` | +| Size of an individual row written to a frame. Row size when written to a frame may differ from the original row size. | 1 MB | [`RowTooLarge`](#error_RowTooLarge) | +| Number of segment-granular time chunks encountered during ingestion. | 5,000 | [`TooManyBuckets`](#error_TooManyBuckets) | +| Number of input files/segments per worker. | 10,000 | [`TooManyInputFiles`](#error_TooManyInputFiles) | +| Number of output partitions for any one stage. Number of segments generated during ingestion. |25,000 | [`TooManyPartitions`](#error_TooManyPartitions) | +| Number of output columns for any one stage. | 2,000 | [`TooManyColumns`](#error_TooManyColumns) | +| Number of cluster by columns that can appear in a stage | 1,500 | [`TooManyClusteredByColumns`](#error_TooManyClusteredByColumns) | +| Number of workers for any one stage. | Hard limit is 1,000. Memory-dependent soft limit may be lower. | [`TooManyWorkers`](#error_TooManyWorkers) | +| Maximum memory occupied by broadcasted tables. | 30% of each [processor memory bundle](concepts.md#memory-usage). | [`BroadcastTablesTooLarge`](#error_BroadcastTablesTooLarge) | @@ -266,30 +266,30 @@ The following table describes error codes you may encounter in the `multiStageQu | Code | Meaning | Additional fields | |---|---|---| -| `BroadcastTablesTooLarge` | The size of the broadcast tables used in the right hand side of the join exceeded the memory reserved for them in a worker task.

    Try increasing the peon memory or reducing the size of the broadcast tables. | `maxBroadcastTablesSize`: Memory reserved for the broadcast tables, measured in bytes. | -| `Canceled` | The query was canceled. Common reasons for cancellation:

    • User-initiated shutdown of the controller task via the `/druid/indexer/v1/task/{taskId}/shutdown` API.
    • Restart or failure of the server process that was running the controller task.
    | | -| `CannotParseExternalData` | A worker task could not parse data from an external datasource. | `errorMessage`: More details on why parsing failed. | -| `ColumnNameRestricted` | The query uses a restricted column name. | `columnName`: The restricted column name. | -| `ColumnTypeNotSupported` | The column type is not supported. This can be because:

    • Support for writing or reading from a particular column type is not supported.
    • The query attempted to use a column type that is not supported by the frame format. This occurs with ARRAY types, which are not yet implemented for frames.
    | `columnName`: The column name with an unsupported type.

    `columnType`: The unknown column type. | -| `InsertCannotAllocateSegment` | The controller task could not allocate a new segment ID due to conflict with existing segments or pending segments. Common reasons for such conflicts:

    • Attempting to mix different granularities in the same intervals of the same datasource.
    • Prior ingestions that used non-extendable shard specs.
    | `dataSource`

    `interval`: The interval for the attempted new segment allocation. | -| `InsertCannotBeEmpty` | An INSERT or REPLACE query did not generate any output rows in a situation where output rows are required for success. This can happen for INSERT or REPLACE queries with `PARTITIONED BY` set to something other than `ALL` or `ALL TIME`. | `dataSource` | -| `InsertCannotOrderByDescending` | An INSERT query contained a `CLUSTERED BY` expression in descending order. Druid's segment generation code only supports ascending order. | `columnName` | -| `InsertCannotReplaceExistingSegment` | A REPLACE query cannot proceed because an existing segment partially overlaps those bounds, and the portion within the bounds is not fully overshadowed by query results.

    There are two ways to address this without modifying your query:
    • Shrink the OVERLAP filter to match the query results.
    • Expand the OVERLAP filter to fully contain the existing segment.
    | `segmentId`: The existing segment
    -| `InsertLockPreempted` | An INSERT or REPLACE query was canceled by a higher-priority ingestion job, such as a real-time ingestion task. | | -| `InsertTimeNull` | An INSERT or REPLACE query encountered a null timestamp in the `__time` field.

    This can happen due to using an expression like `TIME_PARSE(timestamp) AS __time` with a timestamp that cannot be parsed. (TIME_PARSE returns null when it cannot parse a timestamp.) In this case, try parsing your timestamps using a different function or pattern.

    If your timestamps may genuinely be null, consider using COALESCE to provide a default value. One option is CURRENT_TIMESTAMP, which represents the start time of the job. | -| `InsertTimeOutOfBounds` | A REPLACE query generated a timestamp outside the bounds of the TIMESTAMP parameter for your OVERWRITE WHERE clause.

    To avoid this error, verify that the you specified is valid. | `interval`: time chunk interval corresponding to the out-of-bounds timestamp | -| `InvalidNullByte` | A string column included a null byte. Null bytes in strings are not permitted. | `column`: The column that included the null byte | -| `QueryNotSupported` | QueryKit could not translate the provided native query to a multi-stage query.

    This can happen if the query uses features that aren't supported, like GROUPING SETS. | | -| `RowTooLarge` | The query tried to process a row that was too large to write to a single frame. See the [Limits](#limits) table for the specific limit on frame size. Note that the effective maximum row size is smaller than the maximum frame size due to alignment considerations during frame writing. | `maxFrameSize`: The limit on the frame size. | -| `TaskStartTimeout` | Unable to launch all the worker tasks in time.

    There might be insufficient available slots to start all the worker tasks simultaneously.

    Try splitting up the query into smaller chunks with lesser `maxNumTasks` number. Another option is to increase capacity. | `numTasks`: The number of tasks attempted to launch. | -| `TooManyBuckets` | Exceeded the number of partition buckets for a stage. Partition buckets are only used for `segmentGranularity` during INSERT queries. The most common reason for this error is that your `segmentGranularity` is too narrow relative to the data. See the [Limits](#limits) table for the specific limit. | `maxBuckets`: The limit on buckets. | -| `TooManyInputFiles` | Exceeded the number of input files/segments per worker. See the [Limits](#limits) table for the specific limit. | `numInputFiles`: The total number of input files/segments for the stage.

    `maxInputFiles`: The maximum number of input files/segments per worker per stage.

    `minNumWorker`: The minimum number of workers required for a successful run. | -| `TooManyPartitions` | Exceeded the number of partitions for a stage. The most common reason for this is that the final stage of an INSERT or REPLACE query generated too many segments. See the [Limits](#limits) table for the specific limit. | `maxPartitions`: The limit on partitions which was exceeded | -| `TooManyClusteredByColumns` | Exceeded the number of cluster by columns for a stage. See the [Limits](#limits) table for the specific limit. | `numColumns`: The number of columns requested.

    `maxColumns`: The limit on columns which was exceeded.`stage`: The stage number exceeding the limit

    | -| `TooManyColumns` | Exceeded the number of columns for a stage. See the [Limits](#limits) table for the specific limit. | `numColumns`: The number of columns requested.

    `maxColumns`: The limit on columns which was exceeded. | -| `TooManyWarnings` | Exceeded the allowed number of warnings of a particular type. | `rootErrorCode`: The error code corresponding to the exception that exceeded the required limit.

    `maxWarnings`: Maximum number of warnings that are allowed for the corresponding `rootErrorCode`. | -| `TooManyWorkers` | Exceeded the supported number of workers running simultaneously. See the [Limits](#limits) table for the specific limit. | `workers`: The number of simultaneously running workers that exceeded a hard or soft limit. This may be larger than the number of workers in any one stage if multiple stages are running simultaneously.

    `maxWorkers`: The hard or soft limit on workers that was exceeded. | -| `NotEnoughMemory` | Insufficient memory to launch a stage. | `serverMemory`: The amount of memory available to a single process.

    `serverWorkers`: The number of workers running in a single process.

    `serverThreads`: The number of threads in a single process. | -| `WorkerFailed` | A worker task failed unexpectedly. | `errorMsg`

    `workerTaskId`: The ID of the worker task. | -| `WorkerRpcFailed` | A remote procedure call to a worker task failed and could not recover. | `workerTaskId`: the id of the worker task | -| `UnknownError` | All other errors. | `message` | +| `BroadcastTablesTooLarge` | The size of the broadcast tables used in the right hand side of the join exceeded the memory reserved for them in a worker task.

    Try increasing the peon memory or reducing the size of the broadcast tables. | `maxBroadcastTablesSize`: Memory reserved for the broadcast tables, measured in bytes. | +| `Canceled` | The query was canceled. Common reasons for cancellation:

    • User-initiated shutdown of the controller task via the `/druid/indexer/v1/task/{taskId}/shutdown` API.
    • Restart or failure of the server process that was running the controller task.
    | | +| `CannotParseExternalData` | A worker task could not parse data from an external datasource. | `errorMessage`: More details on why parsing failed. | +| `ColumnNameRestricted` | The query uses a restricted column name. | `columnName`: The restricted column name. | +| `ColumnTypeNotSupported` | The column type is not supported. This can be because:

    • Support for writing or reading from a particular column type is not supported.
    • The query attempted to use a column type that is not supported by the frame format. This occurs with ARRAY types, which are not yet implemented for frames.
    | `columnName`: The column name with an unsupported type.

    `columnType`: The unknown column type. | +| `InsertCannotAllocateSegment` | The controller task could not allocate a new segment ID due to conflict with existing segments or pending segments. Common reasons for such conflicts:

    • Attempting to mix different granularities in the same intervals of the same datasource.
    • Prior ingestions that used non-extendable shard specs.
    | `dataSource`

    `interval`: The interval for the attempted new segment allocation. | +| `InsertCannotBeEmpty` | An INSERT or REPLACE query did not generate any output rows in a situation where output rows are required for success. This can happen for INSERT or REPLACE queries with `PARTITIONED BY` set to something other than `ALL` or `ALL TIME`. | `dataSource` | +| `InsertCannotOrderByDescending` | An INSERT query contained a `CLUSTERED BY` expression in descending order. Druid's segment generation code only supports ascending order. | `columnName` | +| `InsertCannotReplaceExistingSegment` | A REPLACE query cannot proceed because an existing segment partially overlaps those bounds, and the portion within the bounds is not fully overshadowed by query results.

    There are two ways to address this without modifying your query:
    • Shrink the OVERLAP filter to match the query results.
    • Expand the OVERLAP filter to fully contain the existing segment.
    | `segmentId`: The existing segment
    +| `InsertLockPreempted` | An INSERT or REPLACE query was canceled by a higher-priority ingestion job, such as a real-time ingestion task. | | +| `InsertTimeNull` | An INSERT or REPLACE query encountered a null timestamp in the `__time` field.

    This can happen due to using an expression like `TIME_PARSE(timestamp) AS __time` with a timestamp that cannot be parsed. (TIME_PARSE returns null when it cannot parse a timestamp.) In this case, try parsing your timestamps using a different function or pattern.

    If your timestamps may genuinely be null, consider using COALESCE to provide a default value. One option is CURRENT_TIMESTAMP, which represents the start time of the job. | +| `InsertTimeOutOfBounds` | A REPLACE query generated a timestamp outside the bounds of the TIMESTAMP parameter for your OVERWRITE WHERE clause.

    To avoid this error, verify that the you specified is valid. | `interval`: time chunk interval corresponding to the out-of-bounds timestamp | +| `InvalidNullByte` | A string column included a null byte. Null bytes in strings are not permitted. | `column`: The column that included the null byte | +| `QueryNotSupported` | QueryKit could not translate the provided native query to a multi-stage query.

    This can happen if the query uses features that aren't supported, like GROUPING SETS. | | +| `RowTooLarge` | The query tried to process a row that was too large to write to a single frame. See the [Limits](#limits) table for specific limits on frame size. Note that the effective maximum row size is smaller than the maximum frame size due to alignment considerations during frame writing. | `maxFrameSize`: The limit on the frame size. | +| `TaskStartTimeout` | Unable to launch all the worker tasks in time.

    There might be insufficient available slots to start all the worker tasks simultaneously.

    Try splitting up the query into smaller chunks with lesser `maxNumTasks` number. Another option is to increase capacity. | `numTasks`: The number of tasks attempted to launch. | +| `TooManyBuckets` | Exceeded the maximum number of partition buckets for a stage (5,000 partition buckets).
    < br />Partition buckets are created for each [`PARTITIONED BY`](#partitioned-by) time chunk for INSERT and REPLACE queries. The most common reason for this error is that your `PARTITIONED BY` is too narrow relative to your data. | `maxBuckets`: The limit on partition buckets. | +| `TooManyInputFiles` | Exceeded the maximum number of input files or segments per worker (10,000 files or segments).

    If you encounter this limit, consider adding more workers, or breaking up your query into smaller queries that process fewer files or segments per query. | `numInputFiles`: The total number of input files/segments for the stage.

    `maxInputFiles`: The maximum number of input files/segments per worker per stage.

    `minNumWorker`: The minimum number of workers required for a successful run. | +| `TooManyPartitions` | Exceeded the maximum number of partitions for a stage (25,000 partitions).

    This can occur with INSERT or REPLACE statements that generate large numbers of segments, since each segment is associated with a partition. If you encounter this limit, consider breaking up your INSERT or REPLACE statement into smaller statements that process less data per statement. | `maxPartitions`: The limit on partitions which was exceeded | +| `TooManyClusteredByColumns` | Exceeded the maximum number of clustering columns for a stage (1,500 columns).

    This can occur with `CLUSTERED BY`, `ORDER BY`, or `GROUP BY` with a large number of columns. | `numColumns`: The number of columns requested.

    `maxColumns`: The limit on columns which was exceeded.`stage`: The stage number exceeding the limit

    | +| `TooManyColumns` | Exceeded the maximum number of columns for a stage (2,000 columns). | `numColumns`: The number of columns requested.

    `maxColumns`: The limit on columns which was exceeded. | +| `TooManyWarnings` | Exceeded the maximum allowed number of warnings of a particular type. | `rootErrorCode`: The error code corresponding to the exception that exceeded the required limit.

    `maxWarnings`: Maximum number of warnings that are allowed for the corresponding `rootErrorCode`. | +| `TooManyWorkers` | Exceeded the maximum number of simultaneously-running workers. See the [Limits](#limits) table for more details. | `workers`: The number of simultaneously running workers that exceeded a hard or soft limit. This may be larger than the number of workers in any one stage if multiple stages are running simultaneously.

    `maxWorkers`: The hard or soft limit on workers that was exceeded. If this is lower than the hard limit (1,000 workers), then you can increase the limit by adding more memory to each task. | +| `NotEnoughMemory` | Insufficient memory to launch a stage. | `serverMemory`: The amount of memory available to a single process.

    `serverWorkers`: The number of workers running in a single process.

    `serverThreads`: The number of threads in a single process. | +| `WorkerFailed` | A worker task failed unexpectedly. | `errorMsg`

    `workerTaskId`: The ID of the worker task. | +| `WorkerRpcFailed` | A remote procedure call to a worker task failed and could not recover. | `workerTaskId`: the id of the worker task | +| `UnknownError` | All other errors. | `message` | diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyBucketsFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyBucketsFault.java index 8af20d091910..fdad421e6490 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyBucketsFault.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyBucketsFault.java @@ -41,7 +41,7 @@ public TooManyBucketsFault(@JsonProperty("maxBuckets") final int maxBuckets) super( CODE, "Too many partition buckets (max = %,d); try breaking your query up into smaller queries or " - + "using a wider segmentGranularity", + + "using a wider PARTITIONED BY", maxBuckets ); this.maxBuckets = maxBuckets; diff --git a/web-console/lib/keywords.js b/web-console/lib/keywords.js index e34b2daf45be..bc81153dd77e 100644 --- a/web-console/lib/keywords.js +++ b/web-console/lib/keywords.js @@ -61,6 +61,9 @@ exports.SQL_KEYWORDS = [ 'REPLACE INTO', 'OVERWRITE', 'RETURNING', + 'OVER', + 'PARTITION BY', + 'WINDOW', ]; exports.SQL_EXPRESSION_PARTS = [ diff --git a/web-console/script/create-sql-docs.js b/web-console/script/create-sql-docs.js index 6af65006f8ef..13ed438915ba 100755 --- a/web-console/script/create-sql-docs.js +++ b/web-console/script/create-sql-docs.js @@ -52,9 +52,7 @@ function convertMarkdownToHtml(markdown) { // Concert to markdown markdown = snarkdown(markdown); - return markdown - .replace(/
    /g, '

    ') // Double up the
    s - .replace(/]*>(.*?)<\/a>/g, '$1'); // Remove links + return markdown.replace(/]*>(.*?)<\/a>/g, '$1'); // Remove links } const readDoc = async () => { diff --git a/web-console/src/bootstrap/react-table-defaults.tsx b/web-console/src/bootstrap/react-table-defaults.tsx index 4c31928064cd..139a13bcd5a6 100644 --- a/web-console/src/bootstrap/react-table-defaults.tsx +++ b/web-console/src/bootstrap/react-table-defaults.tsx @@ -53,12 +53,12 @@ export function bootstrapReactTable() { .map((row: any) => row[column.id]); const previewCount = countBy(previewValues); return ( - +
    {Object.keys(previewCount) .sort() .map(v => `${v} (${previewCount[v]})`) .join(', ')} - +
    ); }, defaultPageSize: 20, diff --git a/web-console/src/components/segment-timeline/segment-timeline.tsx b/web-console/src/components/segment-timeline/segment-timeline.tsx index c138e82dff25..f8cef06189b6 100644 --- a/web-console/src/components/segment-timeline/segment-timeline.tsx +++ b/web-console/src/components/segment-timeline/segment-timeline.tsx @@ -278,7 +278,7 @@ ORDER BY "start" DESC`; intervals = await queryDruidSql({ query: SegmentTimeline.getSqlQuery(startDate, endDate), }); - datasources = uniq(intervals.map(r => r.datasource)); + datasources = uniq(intervals.map(r => r.datasource).sort()); } else if (capabilities.hasCoordinatorAccess()) { const startIso = startDate.toISOString(); diff --git a/web-console/src/dialogs/compaction-dialog/compaction-dialog.scss b/web-console/src/dialogs/compaction-dialog/compaction-dialog.scss index e3ca37b14ea0..499df985c9e1 100644 --- a/web-console/src/dialogs/compaction-dialog/compaction-dialog.scss +++ b/web-console/src/dialogs/compaction-dialog/compaction-dialog.scss @@ -23,6 +23,11 @@ height: 80vh; } + .legacy-callout { + width: auto; + margin: 10px 15px 0; + } + .form-json-selector { margin: 15px; } diff --git a/web-console/src/dialogs/compaction-dialog/compaction-dialog.tsx b/web-console/src/dialogs/compaction-dialog/compaction-dialog.tsx index d63501b1b0b3..3b5456e7d049 100644 --- a/web-console/src/dialogs/compaction-dialog/compaction-dialog.tsx +++ b/web-console/src/dialogs/compaction-dialog/compaction-dialog.tsx @@ -16,11 +16,16 @@ * limitations under the License. */ -import { Button, Classes, Dialog, Intent } from '@blueprintjs/core'; +import { Button, Callout, Classes, Code, Dialog, Intent } from '@blueprintjs/core'; import React, { useState } from 'react'; import { AutoForm, FormJsonSelector, FormJsonTabs, JsonInput } from '../../components'; -import { COMPACTION_CONFIG_FIELDS, CompactionConfig } from '../../druid-models'; +import { + COMPACTION_CONFIG_FIELDS, + CompactionConfig, + compactionConfigHasLegacyInputSegmentSizeBytesSet, +} from '../../druid-models'; +import { deepDelete, formatBytesCompact } from '../../utils'; import './compaction-dialog.scss'; @@ -55,13 +60,29 @@ export const CompactionDialog = React.memo(function CompactionDialog(props: Comp canOutsideClickClose={false} title={`Compaction config: ${datasource}`} > + {compactionConfigHasLegacyInputSegmentSizeBytesSet(currentConfig) && ( + +

    + You current config sets the legacy inputSegmentSizeBytes to{' '} + {formatBytesCompact(currentConfig.inputSegmentSizeBytes!)} it is + recommended to unset this property. +

    +

    +