From e910f0b3fad582c2bc85b0cd5b9fd8d2c303dd2a Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Thu, 5 Oct 2023 13:09:05 +0530 Subject: [PATCH] Address review comments --- docs/multi-stage-query/reference.md | 2 +- .../apache/druid/msq/exec/ControllerImpl.java | 2 ++ .../msq/exec/SegmentLoadStatusFetcher.java | 19 +++++++++++++------ .../msq/util/MultiStageQueryContext.java | 2 +- 4 files changed, 17 insertions(+), 8 deletions(-) diff --git a/docs/multi-stage-query/reference.md b/docs/multi-stage-query/reference.md index f0ec20b46ddb..010bbff2a270 100644 --- a/docs/multi-stage-query/reference.md +++ b/docs/multi-stage-query/reference.md @@ -246,7 +246,7 @@ The following table lists the context parameters for the MSQ task engine: | `durableShuffleStorage` | SELECT, INSERT, REPLACE

Whether to use durable storage for shuffle mesh. To use this feature, configure the durable storage at the server level using `druid.msq.intermediate.storage.enable=true`). If these properties are not configured, any query with the context variable `durableShuffleStorage=true` fails with a configuration error.

| `false` | | `faultTolerance` | SELECT, INSERT, REPLACE

Whether to turn on fault tolerance mode or not. Failed workers are retried based on [Limits](#limits). Cannot be used when `durableShuffleStorage` is explicitly set to false. | `false` | | `selectDestination` | SELECT

Controls where the final result of the select query is written.
Use `taskReport`(the default) to write select results to the task report. This is not scalable since task reports size explodes for large results
Use `durableStorage` to write results to durable storage location. For large results sets, its recommended to use `durableStorage` . To configure durable storage see [`this`](#durable-storage) section. | `taskReport` | -| `segmentLoadWait` | INSERT, REPLACE

Whether the controller should wait for segments to be loaded before exiting. If this is true, the controller queries the broker and waits till the segments created (if any) have been loaded by the load rules. The controller also provides this information in the live reports and task reports. If this is false, the controller exits immediately after finishing the query. | `false` | +| `waitTillSegmentsLoad` | INSERT, REPLACE

If set, the ingest query waits for the generated segment to be loaded before exiting, else the ingest query exits without waiting. The task and live reports contain the information about the status of loading segments if this flag is set. This will ensure that any future queries made after the ingestion exits will include results from the ingestion. The drawback is that the controller task will stall till the segments are loaded. | `false` | ## Joins 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 bbd97c0afd44..ddb85821c79a 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 @@ -473,6 +473,8 @@ public TaskStatus runTask(final Closer closer) if (shouldWaitForSegmentLoad && segmentLoadWaiter != null) { // If successful, there are segments created and segment load is enabled, segmentLoadWaiter should wait // for them to become available. + log.info("Controller will now wait for segments to be loaded. The query has already finished executing," + + " and results will be included once the segments are loaded, even if this query is cancelled now."); segmentLoadWaiter.waitForSegmentsToLoad(); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadStatusFetcher.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadStatusFetcher.java index d4268fe48d1c..17f46bad23a2 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadStatusFetcher.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadStatusFetcher.java @@ -145,6 +145,7 @@ public void waitForSegmentsToLoad() final AtomicReference hasAnySegmentBeenLoaded = new AtomicReference<>(false); try { FutureUtils.getUnchecked(executorService.submit(() -> { + long lastLogMillis = -TimeUnit.MINUTES.toMillis(1); try { while (!(hasAnySegmentBeenLoaded.get() && versionLoadStatusReference.get().isLoadingComplete())) { // Check the timeout and exit if exceeded. @@ -159,11 +160,14 @@ public void waitForSegmentsToLoad() return; } - log.debug( - "Fetching segment load status for datasource[%s] from broker for segment versions[%s]", - datasource, - versionsInClauseString - ); + if (runningMillis - lastLogMillis >= TimeUnit.MINUTES.toMillis(1)) { + lastLogMillis = runningMillis; + log.info( + "Fetching segment load status for datasource[%s] from broker for segment versions[%s]", + datasource, + versionsInClauseString + ); + } // Fetch the load status from the broker VersionLoadStatus loadStatus = fetchLoadStatusFromBroker(); @@ -240,7 +244,10 @@ private VersionLoadStatus fetchLoadStatusFromBroker() throws Exception request.setContent(MediaType.APPLICATION_JSON, objectMapper.writeValueAsBytes(sqlQuery)); String response = brokerClient.sendQuery(request); - if (response.trim().isEmpty()) { + if (response == null) { + // Unable to query broker + return new VersionLoadStatus(0, 0, 0, 0, totalSegmentsGenerated); + } else if (response.trim().isEmpty()) { // If no segments are returned for a version, all segments have been dropped by a drop rule. return new VersionLoadStatus(0, 0, 0, 0, 0); } else { 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 081926ebddde..98dcd471d0fe 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 @@ -97,7 +97,7 @@ public class MultiStageQueryContext public static final String CTX_FAULT_TOLERANCE = "faultTolerance"; public static final boolean DEFAULT_FAULT_TOLERANCE = false; - public static final String CTX_SEGMENT_LOAD_WAIT = "segmentLoadWait"; + public static final String CTX_SEGMENT_LOAD_WAIT = "waitTillSegmentsLoad"; public static final boolean DEFAULT_SEGMENT_LOAD_WAIT = false; public static final String CTX_MAX_INPUT_BYTES_PER_WORKER = "maxInputBytesPerWorker";