From 2539f8643f5a3c1a66652cba84ceed84416d5b40 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Sat, 20 May 2023 15:59:12 +0530 Subject: [PATCH 01/19] Add code to wait for segments generated to be loaded on historicals --- docs/multi-stage-query/api.md | 17 +- .../apache/druid/msq/exec/ControllerImpl.java | 75 ++++--- .../druid/msq/exec/SegmentLoadAwaiter.java | 185 ++++++++++++++++++ .../msq/indexing/report/MSQStatusReport.java | 16 +- .../indexing/report/MSQTaskReportTest.java | 29 ++- .../msq/test/MSQTestControllerContext.java | 1 + 6 files changed, 290 insertions(+), 33 deletions(-) create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadAwaiter.java diff --git a/docs/multi-stage-query/api.md b/docs/multi-stage-query/api.md index 19e1e11c4cc0..4a92ff5064e5 100644 --- a/docs/multi-stage-query/api.md +++ b/docs/multi-stage-query/api.md @@ -255,7 +255,15 @@ The response shows an example report for a query. "startTime": "2022-09-14T22:12:09.266Z", "durationMs": 28227, "pendingTasks": 0, - "runningTasks": 2 + "runningTasks": 2, + "segmentLoadWaiterStatus": { + "state": "SUCCESS", + "dataSource": "kttm_simple", + "startTime": "2022-09-14T23:12:09.266Z", + "duration": 15, + "totalSegments": 1, + "segmentsLeft": 0 + } }, "stages": [ { @@ -583,6 +591,13 @@ The following table describes the response fields when you retrieve a report for | `multiStageQuery.payload.stages[].definition.processor` | An object defining the processor logic. | | `multiStageQuery.payload.stages[].definition.signature` | The output signature of the stage. | +| `multiStageQuery.payload.status.segmentLoadAwaiterStatus` | Segment loading awaiter container. Not present till the segments have been published | +| `multiStageQuery.payload.status.segmentLoadAwaiterStatus.state` | RUNNING, SUCCESS, FAILED or TIMED_OUT | +| `multiStageQuery.payload.status.segmentLoadAwaiterStatus.startTime` | Start time of the awaiter | +| `multiStageQuery.payload.status.segmentLoadAwaiterStatus.duration` | The number of milliseconds that the awaiter has been running | +| `multiStageQuery.payload.status.segmentLoadAwaiterStatus.totalSegments` | The total number of segments generated by the job. This includes tombstone segments (if any). | +| `multiStageQuery.payload.status.segmentLoadAwaiterStatus.segmentsLeft` | The total number of segments remaining to be loaded. | + ## Cancel a query task #### Request 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 b39249d8111f..00657a80fc33 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 @@ -291,6 +291,7 @@ public class ControllerImpl implements Controller private WorkerMemoryParameters workerMemoryParameters; private boolean isDurableStorageEnabled; private boolean isFaultToleranceEnabled; + private volatile SegmentLoadAwaiter segmentLoadAwaiter; public ControllerImpl( final MSQControllerTask task, @@ -428,6 +429,37 @@ public TaskStatus runTask(final Closer closer) } } + if (queryKernel != null && queryKernel.isSuccess()) { + // If successful, encourage the tasks to exit successfully. + postFinishToAllTasks(); + workerTaskLauncher.stop(false); + } else { + // If not successful, cancel running tasks. + if (workerTaskLauncher != null) { + workerTaskLauncher.stop(true); + } + } + + // Wait for worker tasks to exit. Ignore their return status. At this point, we've done everything we need to do, + // so we don't care about the task exit status. + if (workerTaskRunnerFuture != null) { + try { + workerTaskRunnerFuture.get(); + } + catch (Exception ignored) { + // Suppress. + } + } + + cleanUpDurableStorageIfNeeded(); + + if (queryKernel != null && queryKernel.isSuccess()) { + if (segmentLoadAwaiter != null) { + // If successful and there are segments a + segmentLoadAwaiter.awaitSegmentLoad(); + } + } + try { // Write report even if something went wrong. final MSQStagesReport stagesReport; @@ -478,7 +510,8 @@ public TaskStatus runTask(final Closer closer) workerWarnings, queryStartTime, new Interval(queryStartTime, DateTimes.nowUtc()).toDurationMillis(), - workerTaskLauncher + workerTaskLauncher, + segmentLoadAwaiter ), stagesReport, countersSnapshot, @@ -494,30 +527,6 @@ public TaskStatus runTask(final Closer closer) log.warn(e, "Error encountered while writing task report. Skipping."); } - if (queryKernel != null && queryKernel.isSuccess()) { - // If successful, encourage the tasks to exit successfully. - postFinishToAllTasks(); - workerTaskLauncher.stop(false); - } else { - // If not successful, cancel running tasks. - if (workerTaskLauncher != null) { - workerTaskLauncher.stop(true); - } - } - - // Wait for worker tasks to exit. Ignore their return status. At this point, we've done everything we need to do, - // so we don't care about the task exit status. - if (workerTaskRunnerFuture != null) { - try { - workerTaskRunnerFuture.get(); - } - catch (Exception ignored) { - // Suppress. - } - } - - cleanUpDurableStorageIfNeeded(); - if (taskStateForReport == TaskState.SUCCESS) { return TaskStatus.success(id()); } else { @@ -845,7 +854,8 @@ public Map liveReports() workerWarnings, queryStartTime, queryStartTime == null ? -1L : new Interval(queryStartTime, DateTimes.nowUtc()).toDurationMillis(), - workerTaskLauncher + workerTaskLauncher, + segmentLoadAwaiter ), makeStageReport( queryDef, @@ -1271,17 +1281,20 @@ private void publishAllSegments(final Set segments) throws IOExcept if (segmentsWithTombstones.isEmpty()) { // Nothing to publish, only drop. We already validated that the intervalsToDrop do not have any // partially-overlapping segments, so it's safe to drop them as intervals instead of as specific segments. + // This should not need a segment load wait as segments are marked as unused immediately. for (final Interval interval : intervalsToDrop) { context.taskActionClient() .submit(new MarkSegmentsAsUnusedAction(task.getDataSource(), interval)); } } else { + segmentLoadAwaiter = new SegmentLoadAwaiter(context.coordinatorClient(), task.getDataSource(), segmentsWithTombstones); performSegmentPublish( context.taskActionClient(), SegmentTransactionalInsertAction.overwriteAction(null, null, segmentsWithTombstones) ); } } else if (!segments.isEmpty()) { + segmentLoadAwaiter = new SegmentLoadAwaiter(context.coordinatorClient(), task.getDataSource(), segments); // Append mode. performSegmentPublish( context.taskActionClient(), @@ -2035,7 +2048,8 @@ private static MSQStatusReport makeStatusReport( final Queue errorReports, @Nullable final DateTime queryStartTime, final long queryDuration, - MSQWorkerTaskLauncher taskLauncher + MSQWorkerTaskLauncher taskLauncher, + final SegmentLoadAwaiter segmentLoadWaiter ) { int pendingTasks = -1; @@ -2046,6 +2060,9 @@ private static MSQStatusReport makeStatusReport( pendingTasks = workerTaskCount.getPendingWorkerCount(); runningTasks = workerTaskCount.getRunningWorkerCount() + 1; // To account for controller. } + + SegmentLoadAwaiter.Status status = segmentLoadWaiter == null ? null : segmentLoadWaiter.status(); + return new MSQStatusReport( taskState, errorReport, @@ -2053,7 +2070,8 @@ private static MSQStatusReport makeStatusReport( queryStartTime, queryDuration, pendingTasks, - runningTasks + runningTasks, + status ); } @@ -2222,6 +2240,7 @@ private Pair> run() throws IOExceptio throwKernelExceptionIfNotUnknown(); } + updateLiveReportMaps(); cleanUpEffectivelyFinishedStages(); return Pair.of(queryKernel, workerTaskLauncherFuture); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadAwaiter.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadAwaiter.java new file mode 100644 index 000000000000..e5c9a681f52c --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadAwaiter.java @@ -0,0 +1,185 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.exec; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.client.coordinator.CoordinatorClient; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.timeline.DataSegment; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +import java.util.Iterator; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +/** + * Class that periodically checks with the coordinator if all the segments generated are loaded and + * blocks till it is complete. This will account and not wait for segments that would never be loaded due to + * load rules. Only used if the query generates new segments or tombstones. + *
+ * If an exception is thrown during operation, this will simply log the exception and exit without failing the task, + * since the segments have already been published successfully, and should be loaded eventually. + *
+ * If the segments are not loaded by {@link #DEFAULT_WAIT_TIMEOUT}, logs a warning and exits. + */ +public class SegmentLoadAwaiter +{ + private static final Logger log = new Logger(SegmentLoadAwaiter.class); + private static final long DEFAULT_WAIT_TIMEOUT = TimeUnit.MINUTES.toMillis(10); + private static final long SLEEP_DURATION = TimeUnit.SECONDS.toMillis(15); + + private final CoordinatorClient coordinatorClient; + private final String dataSource; + private final Set segmentsToAwait; + private volatile Status status; + + public SegmentLoadAwaiter(CoordinatorClient coordinatorClient, String dataSource, Set dataSegments) + { + this.coordinatorClient = coordinatorClient; + this.dataSource = dataSource; + this.segmentsToAwait = dataSegments.stream() + .map(DataSegment::toDescriptor) + .collect(Collectors.toSet()); + } + + /** + * Uses coordinator client to check if all segments in {@link #segmentsToAwait} have finished handing off. Hand off + * is when a segment has finished loading, and immedietly returns true if it would not be loaded according to the + * load rules. + *
+ * If an exception is thrown during operation, this will log the exception and return without failing the task, + * since the segments have already been published successfully, and should be loaded eventually. + *
+ * Only expected to be called from the main controller thread. + */ + public void awaitSegmentLoad() + { + int totalSegments = segmentsToAwait.size(); + DateTime startTime = DateTime.now(); + + try { + while (!segmentsToAwait.isEmpty()) { + + long runningMillis = new Interval(startTime, DateTime.now()).toDurationMillis(); + if (runningMillis > DEFAULT_WAIT_TIMEOUT) { + log.warn("Timeout %s exceeded while waiting for segments to load. Exiting.", DEFAULT_WAIT_TIMEOUT); + status = new Status(State.TIMED_OUT, startTime, runningMillis, totalSegments, segmentsToAwait.size()); + return; + } + + Iterator iterator = segmentsToAwait.iterator(); + while (iterator.hasNext()) { + SegmentDescriptor descriptor = iterator.next(); + Boolean handOffComplete = coordinatorClient.isHandOffComplete(dataSource, descriptor); + if (Boolean.TRUE.equals(handOffComplete)) { + iterator.remove(); + } + } + + if (segmentsToAwait.isEmpty()) { + break; + } else { + // Update the status + runningMillis = new Interval(startTime, DateTime.now()).toDurationMillis(); + status = new Status(State.RUNNING, startTime, runningMillis, totalSegments, segmentsToAwait.size()); + + // Sleep for a while before retrying + Thread.sleep(SLEEP_DURATION); + } + } + } catch (Exception e) { + log.warn(e, "Exception occurred while waiting for segments to load. Exiting."); + long runningMillis = new Interval(startTime, DateTime.now()).toDurationMillis(); + status = new Status(State.FAILED, startTime, runningMillis, totalSegments, segmentsToAwait.size()); + } + + long runningMillis = new Interval(startTime, DateTime.now()).toDurationMillis(); + status = new Status(State.SUCCESS, startTime, runningMillis, totalSegments, segmentsToAwait.size()); + } + + public Status status() { + return status; + } + + public static class Status { + private final State state; + private final DateTime startTime; + private final long duration; + private final int totalSegments; + private final int segmentsLeft; + + @JsonCreator + public Status( + @JsonProperty("state") State state, + @JsonProperty("startTime") DateTime startTime, + @JsonProperty("duration") long duration, + @JsonProperty("totalSegments") int totalSegments, + @JsonProperty("segmentsLeft") int segmentsLeft + ) + { + this.state = state; + this.startTime = startTime; + this.duration = duration; + this.totalSegments = totalSegments; + this.segmentsLeft = segmentsLeft; + } + + @JsonProperty + public State getState() + { + return state; + } + + @JsonProperty + public DateTime getStartTime() + { + return startTime; + } + + @JsonProperty + public long getDuration() + { + return duration; + } + + @JsonProperty + public long getTotalSegments() + { + return totalSegments; + } + + @JsonProperty + public int getSegmentsLeft() + { + return segmentsLeft; + } + } + + public enum State { + RUNNING, + SUCCESS, + FAILED, + TIMED_OUT + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQStatusReport.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQStatusReport.java index 3791bc82e16a..6c10035cd85b 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQStatusReport.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQStatusReport.java @@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import org.apache.druid.indexer.TaskState; +import org.apache.druid.msq.exec.SegmentLoadAwaiter; import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.joda.time.DateTime; @@ -50,6 +51,9 @@ public class MSQStatusReport private final int runningTasks; + @Nullable + private final SegmentLoadAwaiter.Status segmentLoadAwaiterStatus; + @JsonCreator public MSQStatusReport( @JsonProperty("status") TaskState status, @@ -58,7 +62,8 @@ public MSQStatusReport( @JsonProperty("startTime") @Nullable DateTime startTime, @JsonProperty("durationMs") long durationMs, @JsonProperty("pendingTasks") int pendingTasks, - @JsonProperty("runningTasks") int runningTasks + @JsonProperty("runningTasks") int runningTasks, + @JsonProperty("segmentLoadAwaiterStatus") @Nullable SegmentLoadAwaiter.Status segmentLoadAwaiterStatus ) { this.status = Preconditions.checkNotNull(status, "status"); @@ -68,6 +73,7 @@ public MSQStatusReport( this.durationMs = durationMs; this.pendingTasks = pendingTasks; this.runningTasks = runningTasks; + this.segmentLoadAwaiterStatus = segmentLoadAwaiterStatus; } @JsonProperty @@ -117,6 +123,14 @@ public long getDurationMs() return durationMs; } + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public SegmentLoadAwaiter.Status getSegmentLoadAwaiterStatus() + { + return segmentLoadAwaiterStatus; + } + @Override public boolean equals(Object o) { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java index aeca792024a9..cf95e2a23371 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java @@ -34,6 +34,7 @@ import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.msq.counters.CounterSnapshotsTree; +import org.apache.druid.msq.exec.SegmentLoadAwaiter; import org.apache.druid.msq.guice.MSQIndexingModule; import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.apache.druid.msq.indexing.error.TooManyColumnsFault; @@ -44,6 +45,7 @@ import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; +import org.joda.time.DateTime; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -90,10 +92,17 @@ public void testSerdeResultsReport() throws Exception new Object[]{"bar"} ); + SegmentLoadAwaiter.Status status = new SegmentLoadAwaiter.Status( + SegmentLoadAwaiter.State.RUNNING, DateTime.now(), + 200L, + 50, + 10 + ); + final MSQTaskReport report = new MSQTaskReport( TASK_ID, new MSQTaskReportPayload( - new MSQStatusReport(TaskState.SUCCESS, null, new ArrayDeque<>(), null, 0, 1, 2), + new MSQStatusReport(TaskState.SUCCESS, null, new ArrayDeque<>(), null, 0, 1, 2, status), MSQStagesReport.create( QUERY_DEFINITION, ImmutableMap.of(), @@ -142,11 +151,18 @@ public void testSerdeResultsReport() throws Exception @Test public void testSerdeErrorReport() throws Exception { + SegmentLoadAwaiter.Status status = new SegmentLoadAwaiter.Status( + SegmentLoadAwaiter.State.FAILED, DateTime.now(), + 200L, + 50, + 10 + ); + final MSQErrorReport errorReport = MSQErrorReport.fromFault(TASK_ID, HOST, 0, new TooManyColumnsFault(10, 5)); final MSQTaskReport report = new MSQTaskReport( TASK_ID, new MSQTaskReportPayload( - new MSQStatusReport(TaskState.FAILED, errorReport, new ArrayDeque<>(), null, 0, 1, 2), + new MSQStatusReport(TaskState.FAILED, errorReport, new ArrayDeque<>(), null, 0, 1, 2, status), MSQStagesReport.create( QUERY_DEFINITION, ImmutableMap.of(), @@ -179,10 +195,17 @@ public void testSerdeErrorReport() throws Exception @Test public void testWriteTaskReport() throws Exception { + SegmentLoadAwaiter.Status status = new SegmentLoadAwaiter.Status( + SegmentLoadAwaiter.State.SUCCESS, DateTime.now(), + 200L, + 50, + 0 + ); + final MSQTaskReport report = new MSQTaskReport( TASK_ID, new MSQTaskReportPayload( - new MSQStatusReport(TaskState.SUCCESS, null, new ArrayDeque<>(), null, 0, 1, 2), + new MSQStatusReport(TaskState.SUCCESS, null, new ArrayDeque<>(), null, 0, 1, 2, status), MSQStagesReport.create( QUERY_DEFINITION, ImmutableMap.of(), diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java index 58ee01f00872..0465453e15ac 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java @@ -110,6 +110,7 @@ public MSQTestControllerContext( .collect(Collectors.toList()) ) ); + Mockito.when(coordinatorClient.isHandOffComplete(ArgumentMatchers.anyString(), ArgumentMatchers.any())).thenReturn(Boolean.TRUE); this.workerMemoryParameters = workerMemoryParameters; } From 92b432d5e5a3fbf09bedf2ccedf7731fa7f5e593 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Tue, 13 Jun 2023 11:02:26 +0530 Subject: [PATCH 02/19] Add broker client and use it to query broker for segment load status --- .../apache/druid/msq/exec/ControllerImpl.java | 10 +- .../druid/msq/exec/SegmentLoadAwaiter.java | 215 +++++++++++++----- .../msq/indexing/report/MSQStatusReport.java | 6 +- .../indexing/report/MSQTaskReportTest.java | 17 +- .../apache/druid/discovery/BrokerClient.java | 140 ++++++++++++ .../apache/druid/discovery/ClientUtils.java | 58 +++++ .../druid/discovery/DruidLeaderClient.java | 36 +-- 7 files changed, 383 insertions(+), 99 deletions(-) create mode 100644 server/src/main/java/org/apache/druid/discovery/BrokerClient.java create mode 100644 server/src/main/java/org/apache/druid/discovery/ClientUtils.java 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 00657a80fc33..6a58b41f20b3 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 @@ -455,7 +455,7 @@ public TaskStatus runTask(final Closer closer) if (queryKernel != null && queryKernel.isSuccess()) { if (segmentLoadAwaiter != null) { - // If successful and there are segments a + // If successful and there are segments created, segmentLoadAwaiter should wait for them to become available. segmentLoadAwaiter.awaitSegmentLoad(); } } @@ -1287,14 +1287,16 @@ private void publishAllSegments(final Set segments) throws IOExcept .submit(new MarkSegmentsAsUnusedAction(task.getDataSource(), interval)); } } else { - segmentLoadAwaiter = new SegmentLoadAwaiter(context.coordinatorClient(), task.getDataSource(), segmentsWithTombstones); + Set versionsToAwait = segmentsWithTombstones.stream().map(DataSegment::getVersion).collect(Collectors.toSet()); + segmentLoadAwaiter = new SegmentLoadAwaiter(context, task.getDataSource(), versionsToAwait, segmentsWithTombstones.size()); performSegmentPublish( context.taskActionClient(), SegmentTransactionalInsertAction.overwriteAction(null, null, segmentsWithTombstones) ); } } else if (!segments.isEmpty()) { - segmentLoadAwaiter = new SegmentLoadAwaiter(context.coordinatorClient(), task.getDataSource(), segments); + Set versionsToAwait = segments.stream().map(DataSegment::getVersion).collect(Collectors.toSet()); + segmentLoadAwaiter = new SegmentLoadAwaiter(context, task.getDataSource(), versionsToAwait, segments.size()); // Append mode. performSegmentPublish( context.taskActionClient(), @@ -2061,7 +2063,7 @@ private static MSQStatusReport makeStatusReport( runningTasks = workerTaskCount.getRunningWorkerCount() + 1; // To account for controller. } - SegmentLoadAwaiter.Status status = segmentLoadWaiter == null ? null : segmentLoadWaiter.status(); + SegmentLoadAwaiter.SegmentLoadAwaiterStatus status = segmentLoadWaiter == null ? null : segmentLoadWaiter.status(); return new MSQStatusReport( taskState, diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadAwaiter.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadAwaiter.java index e5c9a681f52c..19d61d9b65df 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadAwaiter.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadAwaiter.java @@ -21,52 +21,70 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.client.coordinator.CoordinatorClient; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.discovery.BrokerClient; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.RE; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.query.SegmentDescriptor; -import org.apache.druid.timeline.DataSegment; +import org.apache.druid.java.util.http.client.Request; +import org.apache.druid.java.util.http.client.response.StringFullResponseHolder; +import org.apache.druid.sql.http.ResultFormat; +import org.apache.druid.sql.http.SqlQuery; +import org.jboss.netty.handler.codec.http.HttpMethod; import org.joda.time.DateTime; import org.joda.time.Interval; +import javax.servlet.http.HttpServletResponse; +import java.util.HashMap; import java.util.Iterator; +import java.util.Map; import java.util.Set; import java.util.concurrent.TimeUnit; -import java.util.stream.Collectors; /** - * Class that periodically checks with the coordinator if all the segments generated are loaded and - * blocks till it is complete. This will account and not wait for segments that would never be loaded due to - * load rules. Only used if the query generates new segments or tombstones. + * Class that periodically checks with the broker if all the segments generated are loaded by querying the sys table + * and blocks till it is complete. This will account for and not wait for segments that would never be loaded due to + * load rules. Should only be called if the query generates new segments or tombstones. *
* If an exception is thrown during operation, this will simply log the exception and exit without failing the task, * since the segments have already been published successfully, and should be loaded eventually. *
- * If the segments are not loaded by {@link #DEFAULT_WAIT_TIMEOUT}, logs a warning and exits. + * If the segments are not loaded within {@link #TIMEOUT_DURATION_MILLIS} milliseconds, this logs a warning and exits + * for the same reason. */ public class SegmentLoadAwaiter { private static final Logger log = new Logger(SegmentLoadAwaiter.class); - private static final long DEFAULT_WAIT_TIMEOUT = TimeUnit.MINUTES.toMillis(10); - private static final long SLEEP_DURATION = TimeUnit.SECONDS.toMillis(15); + private static final long INITIAL_SLEEP_DURATION_MILLIS = TimeUnit.SECONDS.toMillis(5); + private static final long SLEEP_DURATION_MILLIS = TimeUnit.SECONDS.toMillis(5); + private static final long TIMEOUT_DURATION_MILLIS = TimeUnit.MINUTES.toMillis(10); + private static final String LOAD_QUERY = "SELECT COUNT(*) AS totalSegments, " + + "COUNT(*) FILTER (WHERE is_available = 0 AND is_published = 1) AS loadingSegments " + + "FROM sys.segments " + + "WHERE datasource = '%s' AND is_overshadowed = 0 AND version = '%s'"; - private final CoordinatorClient coordinatorClient; - private final String dataSource; - private final Set segmentsToAwait; - private volatile Status status; + private final BrokerClient brokerClient; + private final ObjectMapper objectMapper; + // Map of version vs latest load status. + private final Map versionToLoadStatusMap; + private final String datasource; + private final Set versionsToAwait; + private volatile SegmentLoadAwaiterStatus status; - public SegmentLoadAwaiter(CoordinatorClient coordinatorClient, String dataSource, Set dataSegments) + public SegmentLoadAwaiter(ControllerContext context, String datasource, Set versionsToAwait, int initialSegmentCount) { - this.coordinatorClient = coordinatorClient; - this.dataSource = dataSource; - this.segmentsToAwait = dataSegments.stream() - .map(DataSegment::toDescriptor) - .collect(Collectors.toSet()); + this.brokerClient = context.injector().getInstance(BrokerClient.class); + this.objectMapper = context.jsonMapper(); + this.datasource = datasource; + this.versionsToAwait = versionsToAwait; + this.versionToLoadStatusMap = new HashMap<>(); + this.status = new SegmentLoadAwaiterStatus(State.INIT, null, 0, initialSegmentCount, initialSegmentCount); } /** - * Uses coordinator client to check if all segments in {@link #segmentsToAwait} have finished handing off. Hand off - * is when a segment has finished loading, and immedietly returns true if it would not be loaded according to the - * load rules. + * Uses broker client to check if all segments created by the ingestion have been loaded and updates the {@link #status)} + * periodically. *
* If an exception is thrown during operation, this will log the exception and return without failing the task, * since the segments have already been published successfully, and should be loaded eventually. @@ -75,54 +93,110 @@ public SegmentLoadAwaiter(CoordinatorClient coordinatorClient, String dataSource */ public void awaitSegmentLoad() { - int totalSegments = segmentsToAwait.size(); - DateTime startTime = DateTime.now(); + DateTime startTime = DateTimes.nowUtc(); try { - while (!segmentsToAwait.isEmpty()) { + // Sleep for a short duration to allow the segments that were just created to reflect in broker queries. + // This avoids a race condition where the broker returns nothing as it is not aware of the new segments yet, + // as this cannot be differentiated from the case where the segments have been dropped by load rules already. + Thread.sleep(INITIAL_SLEEP_DURATION_MILLIS); - long runningMillis = new Interval(startTime, DateTime.now()).toDurationMillis(); - if (runningMillis > DEFAULT_WAIT_TIMEOUT) { - log.warn("Timeout %s exceeded while waiting for segments to load. Exiting.", DEFAULT_WAIT_TIMEOUT); - status = new Status(State.TIMED_OUT, startTime, runningMillis, totalSegments, segmentsToAwait.size()); + while (!versionsToAwait.isEmpty()) { + // Check the timeout and exit if exceeded. + long runningMillis = new Interval(startTime, DateTimes.nowUtc()).toDurationMillis(); + if (runningMillis > TIMEOUT_DURATION_MILLIS) { + log.warn("Runtime [%s] exceeded timeout [%s] while waiting for segments to load. Exiting.", runningMillis, TIMEOUT_DURATION_MILLIS); + updateStatus(State.TIMED_OUT, startTime); return; } - Iterator iterator = segmentsToAwait.iterator(); + Iterator iterator = versionsToAwait.iterator(); + + // Query the broker for all pending versions while (iterator.hasNext()) { - SegmentDescriptor descriptor = iterator.next(); - Boolean handOffComplete = coordinatorClient.isHandOffComplete(dataSource, descriptor); - if (Boolean.TRUE.equals(handOffComplete)) { + String version = iterator.next(); + + // Fetch the load status for this version from the broker + VersionLoadStatus loadStatus = fetchLoadStatusForVersion(version); + versionToLoadStatusMap.put(version, loadStatus); + + // If loading is done for this stage, remove it from future loops. + if (loadStatus.isLoadingComplete()) { iterator.remove(); } } - if (segmentsToAwait.isEmpty()) { - break; - } else { - // Update the status - runningMillis = new Interval(startTime, DateTime.now()).toDurationMillis(); - status = new Status(State.RUNNING, startTime, runningMillis, totalSegments, segmentsToAwait.size()); + if (!versionsToAwait.isEmpty()) { + // Update the status. + updateStatus(State.RUNNING, startTime); - // Sleep for a while before retrying - Thread.sleep(SLEEP_DURATION); + // Sleep for a while before retrying. + Thread.sleep(SLEEP_DURATION_MILLIS); } } - } catch (Exception e) { + } + catch (Exception e) { log.warn(e, "Exception occurred while waiting for segments to load. Exiting."); - long runningMillis = new Interval(startTime, DateTime.now()).toDurationMillis(); - status = new Status(State.FAILED, startTime, runningMillis, totalSegments, segmentsToAwait.size()); + + // Update the status and return. + updateStatus(State.FAILED, startTime); + return; + } + // Update the status. + updateStatus(State.SUCCESS, startTime); + } + + /** + * Updates the {@link #status} with the latest details based on {@link #versionToLoadStatusMap} + */ + private void updateStatus(State state, DateTime startTime) + { + int totalSegmentCount = 0, pendingSegmentCount = 0; + for (Map.Entry entry : versionToLoadStatusMap.entrySet()) { + totalSegmentCount += entry.getValue().getTotalSegments(); + pendingSegmentCount += entry.getValue().getLoadingSegments(); } - long runningMillis = new Interval(startTime, DateTime.now()).toDurationMillis(); - status = new Status(State.SUCCESS, startTime, runningMillis, totalSegments, segmentsToAwait.size()); + long runningMillis = new Interval(startTime, DateTimes.nowUtc()).toDurationMillis(); + status = new SegmentLoadAwaiterStatus(state, startTime, runningMillis, totalSegmentCount, pendingSegmentCount); } - public Status status() { + /** + * Uses {@link #brokerClient} to fetch latest load status for a given version. Converts the response into a + * {@link VersionLoadStatus} and returns it. + */ + private VersionLoadStatus fetchLoadStatusForVersion(String version) throws Exception + { + Request request = brokerClient.makeRequest(HttpMethod.POST, "/druid/v2/sql/"); + SqlQuery sqlQuery = new SqlQuery(StringUtils.format(LOAD_QUERY, datasource, version), + ResultFormat.OBJECTLINES, + false, false, false, null, null); + request.setContent("application/json", objectMapper.writeValueAsBytes(sqlQuery)); + + StringFullResponseHolder responseHolder = brokerClient.sendQuery(request); + if (responseHolder.getStatus().getCode() != HttpServletResponse.SC_OK) { + throw new RE("HTTP request to[%s] failed", request.getUrl()); + } + + String response = responseHolder.getContent(); + 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); + } else { + return objectMapper.readValue(response, VersionLoadStatus.class); + } + } + + /** + * Returns the current status of the load. + */ + public SegmentLoadAwaiterStatus status() + { return status; } - public static class Status { + public static class SegmentLoadAwaiterStatus + { private final State state; private final DateTime startTime; private final long duration; @@ -130,8 +204,8 @@ public static class Status { private final int segmentsLeft; @JsonCreator - public Status( - @JsonProperty("state") State state, + public SegmentLoadAwaiterStatus( + @JsonProperty("state") SegmentLoadAwaiter.State state, @JsonProperty("startTime") DateTime startTime, @JsonProperty("duration") long duration, @JsonProperty("totalSegments") int totalSegments, @@ -146,7 +220,7 @@ public Status( } @JsonProperty - public State getState() + public SegmentLoadAwaiter.State getState() { return state; } @@ -176,10 +250,45 @@ public int getSegmentsLeft() } } - public enum State { + public enum State + { + INIT, RUNNING, SUCCESS, FAILED, TIMED_OUT } + + public static class VersionLoadStatus + { + private final int totalSegments; + private final int loadingSegments; + + @JsonCreator + public VersionLoadStatus( + @JsonProperty("totalSegments") int total_segments, + @JsonProperty("loadingSegments") int loading_segments + ) + { + this.totalSegments = total_segments; + this.loadingSegments = loading_segments; + } + + @JsonProperty("totalSegments") + public int getTotalSegments() + { + return totalSegments; + } + + @JsonProperty("loadingSegments") + public int getLoadingSegments() + { + return loadingSegments; + } + + public boolean isLoadingComplete() + { + return loadingSegments == 0; + } + } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQStatusReport.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQStatusReport.java index 6c10035cd85b..47beed500c35 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQStatusReport.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQStatusReport.java @@ -52,7 +52,7 @@ public class MSQStatusReport private final int runningTasks; @Nullable - private final SegmentLoadAwaiter.Status segmentLoadAwaiterStatus; + private final SegmentLoadAwaiter.SegmentLoadAwaiterStatus segmentLoadAwaiterStatus; @JsonCreator public MSQStatusReport( @@ -63,7 +63,7 @@ public MSQStatusReport( @JsonProperty("durationMs") long durationMs, @JsonProperty("pendingTasks") int pendingTasks, @JsonProperty("runningTasks") int runningTasks, - @JsonProperty("segmentLoadAwaiterStatus") @Nullable SegmentLoadAwaiter.Status segmentLoadAwaiterStatus + @JsonProperty("segmentLoadAwaiterStatus") @Nullable SegmentLoadAwaiter.SegmentLoadAwaiterStatus segmentLoadAwaiterStatus ) { this.status = Preconditions.checkNotNull(status, "status"); @@ -126,7 +126,7 @@ public long getDurationMs() @Nullable @JsonProperty @JsonInclude(JsonInclude.Include.NON_NULL) - public SegmentLoadAwaiter.Status getSegmentLoadAwaiterStatus() + public SegmentLoadAwaiter.SegmentLoadAwaiterStatus getSegmentLoadAwaiterStatus() { return segmentLoadAwaiterStatus; } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java index cf95e2a23371..eb1bf67f498a 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java @@ -30,6 +30,7 @@ import org.apache.druid.indexer.TaskState; import org.apache.druid.indexing.common.SingleFileTaskReportFileWriter; import org.apache.druid.indexing.common.TaskReport; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.common.guava.Yielders; @@ -45,7 +46,6 @@ import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.joda.time.DateTime; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -92,8 +92,9 @@ public void testSerdeResultsReport() throws Exception new Object[]{"bar"} ); - SegmentLoadAwaiter.Status status = new SegmentLoadAwaiter.Status( - SegmentLoadAwaiter.State.RUNNING, DateTime.now(), + SegmentLoadAwaiter.SegmentLoadAwaiterStatus status = new SegmentLoadAwaiter.SegmentLoadAwaiterStatus( + SegmentLoadAwaiter.State.RUNNING, + DateTimes.nowUtc(), 200L, 50, 10 @@ -151,8 +152,9 @@ public void testSerdeResultsReport() throws Exception @Test public void testSerdeErrorReport() throws Exception { - SegmentLoadAwaiter.Status status = new SegmentLoadAwaiter.Status( - SegmentLoadAwaiter.State.FAILED, DateTime.now(), + SegmentLoadAwaiter.SegmentLoadAwaiterStatus status = new SegmentLoadAwaiter.SegmentLoadAwaiterStatus( + SegmentLoadAwaiter.State.FAILED, + DateTimes.nowUtc(), 200L, 50, 10 @@ -195,8 +197,9 @@ public void testSerdeErrorReport() throws Exception @Test public void testWriteTaskReport() throws Exception { - SegmentLoadAwaiter.Status status = new SegmentLoadAwaiter.Status( - SegmentLoadAwaiter.State.SUCCESS, DateTime.now(), + SegmentLoadAwaiter.SegmentLoadAwaiterStatus status = new SegmentLoadAwaiter.SegmentLoadAwaiterStatus( + SegmentLoadAwaiter.State.SUCCESS, + DateTimes.nowUtc(), 200L, 50, 0 diff --git a/server/src/main/java/org/apache/druid/discovery/BrokerClient.java b/server/src/main/java/org/apache/druid/discovery/BrokerClient.java new file mode 100644 index 000000000000..88d91671f464 --- /dev/null +++ b/server/src/main/java/org/apache/druid/discovery/BrokerClient.java @@ -0,0 +1,140 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.discovery; + +import com.google.common.base.Throwables; +import com.google.inject.Inject; +import org.apache.druid.guice.annotations.EscalatedGlobal; +import org.apache.druid.java.util.common.IOE; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.RE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.java.util.http.client.Request; +import org.apache.druid.java.util.http.client.response.StringFullResponseHandler; +import org.apache.druid.java.util.http.client.response.StringFullResponseHolder; +import org.jboss.netty.channel.ChannelException; +import org.jboss.netty.handler.codec.http.HttpMethod; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; + +import java.io.IOException; +import java.net.MalformedURLException; +import java.net.URL; +import java.nio.charset.StandardCharsets; +import java.util.concurrent.ExecutionException; + +/** + * This class facilitates interaction with Broker. + */ +public class BrokerClient +{ + private final Logger log = new Logger(BrokerClient.class); + private static final int MAX_RETRIES = 5; + + private final HttpClient httpClient; + private final DruidNodeDiscovery druidNodeDiscovery; + + @Inject + public BrokerClient( + @EscalatedGlobal HttpClient httpClient, + DruidNodeDiscoveryProvider druidNodeDiscoveryProvider + ) + { + this.httpClient = httpClient; + this.druidNodeDiscovery = druidNodeDiscoveryProvider.getForNodeRole(NodeRole.BROKER); + } + + /** + * Creates and returns a {@link Request} after choosing a broker. + */ + public Request makeRequest(HttpMethod httpMethod, String urlPath) throws IOException + { + String host = ClientUtils.pickOneHost(druidNodeDiscovery); + + if (host == null) { + throw new IOE("No known server"); + } + return new Request(httpMethod, new URL(StringUtils.format("%s%s", host, urlPath))); + } + + public StringFullResponseHolder sendQuery(Request request) throws Exception + { + StringFullResponseHandler responseHandler = new StringFullResponseHandler(StandardCharsets.UTF_8); + + for (int counter = 0; counter < MAX_RETRIES; counter++) { + + final StringFullResponseHolder fullResponseHolder; + + try { + try { + fullResponseHolder = httpClient.go(request, responseHandler).get(); + } + catch (ExecutionException e) { + // Unwrap IOExceptions and ChannelExceptions, re-throw others + Throwables.propagateIfInstanceOf(e.getCause(), IOException.class); + Throwables.propagateIfInstanceOf(e.getCause(), ChannelException.class); + throw new RE(e, "HTTP request to[%s] failed", request.getUrl()); + } + } + catch (IOException | ChannelException ex) { + // can happen if the node is stopped. + log.warn(ex, "Request[%s] failed.", request.getUrl()); + request = getNewRequestUrl(request); + continue; + } + HttpResponseStatus responseStatus = fullResponseHolder.getResponse().getStatus(); + if (HttpResponseStatus.SERVICE_UNAVAILABLE.equals(responseStatus) + || HttpResponseStatus.GATEWAY_TIMEOUT.equals(responseStatus)) { + log.warn( + "Request[%s] received a %s response. Attempt %s/%s", + request.getUrl(), + responseStatus, + counter + 1, + MAX_RETRIES + ); + request = getNewRequestUrl(request); + } else { + return fullResponseHolder; + } + } + + throw new IOE("Retries exhausted, couldn't fulfill request to [%s].", request.getUrl()); + } + + private Request getNewRequestUrl(Request oldRequest) + { + try { + return ClientUtils.withUrl( + oldRequest, + new URL(StringUtils.format("%s%s", ClientUtils.pickOneHost(druidNodeDiscovery), oldRequest.getUrl().getPath())) + ); + } + catch (MalformedURLException e) { + // Not an IOException; this is our own fault. + throw new ISE( + e, + "failed to build url with path[%] and query string [%s].", + oldRequest.getUrl().getPath(), + oldRequest.getUrl().getQuery() + ); + } + } +} diff --git a/server/src/main/java/org/apache/druid/discovery/ClientUtils.java b/server/src/main/java/org/apache/druid/discovery/ClientUtils.java new file mode 100644 index 000000000000..cadf5bafca60 --- /dev/null +++ b/server/src/main/java/org/apache/druid/discovery/ClientUtils.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.discovery; + +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.http.client.Request; + +import javax.annotation.Nullable; +import java.net.URL; +import java.util.Iterator; + +/** + * Utils class for shared client methods + */ +public class ClientUtils +{ + @Nullable + public static String pickOneHost(DruidNodeDiscovery druidNodeDiscovery) + { + Iterator iter = druidNodeDiscovery.getAllNodes().iterator(); + if (iter.hasNext()) { + DiscoveryDruidNode node = iter.next(); + return StringUtils.format( + "%s://%s", + node.getDruidNode().getServiceScheme(), + node.getDruidNode().getHostAndPortToUse() + ); + } + return null; + } + + public static Request withUrl(Request old, URL url) + { + Request req = new Request(old.getMethod(), url); + req.addHeaderValues(old.getHeaders()); + if (old.hasContent()) { + req.setContent(old.getContent()); + } + return req; + } +} diff --git a/server/src/main/java/org/apache/druid/discovery/DruidLeaderClient.java b/server/src/main/java/org/apache/druid/discovery/DruidLeaderClient.java index 3bc29da40f46..d9a37ff5533a 100644 --- a/server/src/main/java/org/apache/druid/discovery/DruidLeaderClient.java +++ b/server/src/main/java/org/apache/druid/discovery/DruidLeaderClient.java @@ -39,12 +39,10 @@ import org.jboss.netty.handler.codec.http.HttpMethod; import org.jboss.netty.handler.codec.http.HttpResponseStatus; -import javax.annotation.Nullable; import java.io.IOException; import java.net.MalformedURLException; import java.net.URL; import java.nio.charset.StandardCharsets; -import java.util.Iterator; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; @@ -202,7 +200,7 @@ public > H go( redirectUrl.getPort() )); - request = withUrl(request, redirectUrl); + request = ClientUtils.withUrl(request, redirectUrl); } else if (HttpResponseStatus.SERVICE_UNAVAILABLE.equals(responseStatus) || HttpResponseStatus.GATEWAY_TIMEOUT.equals(responseStatus)) { log.warn( @@ -260,7 +258,7 @@ private String getCurrentKnownLeader(final boolean cached) throws IOException { final String leader = currentKnownLeader.accumulateAndGet( null, - (current, given) -> current == null || !cached ? pickOneHost() : current + (current, given) -> current == null || !cached ? ClientUtils.pickOneHost(druidNodeDiscovery) : current ); if (leader == null) { @@ -270,43 +268,17 @@ private String getCurrentKnownLeader(final boolean cached) throws IOException } } - @Nullable - private String pickOneHost() - { - Iterator iter = druidNodeDiscovery.getAllNodes().iterator(); - if (iter.hasNext()) { - DiscoveryDruidNode node = iter.next(); - return StringUtils.format( - "%s://%s", - node.getDruidNode().getServiceScheme(), - node.getDruidNode().getHostAndPortToUse() - ); - } - - return null; - } - - private Request withUrl(Request old, URL url) - { - Request req = new Request(old.getMethod(), url); - req.addHeaderValues(old.getHeaders()); - if (old.hasContent()) { - req.setContent(old.getContent()); - } - return req; - } - private Request getNewRequestUrlInvalidatingCache(Request oldRequest) throws IOException { try { Request newRequest; if (oldRequest.getUrl().getQuery() == null) { - newRequest = withUrl( + newRequest = ClientUtils.withUrl( oldRequest, new URL(StringUtils.format("%s%s", getCurrentKnownLeader(false), oldRequest.getUrl().getPath())) ); } else { - newRequest = withUrl( + newRequest = ClientUtils.withUrl( oldRequest, new URL(StringUtils.format( "%s%s?%s", From 68210ba11ada87618f29fa836ffe15b81a5f4ba0 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Wed, 14 Jun 2023 10:15:48 +0530 Subject: [PATCH 03/19] Temp --- .../apache/druid/msq/exec/ControllerImpl.java | 18 ++-- ...oadAwaiter.java => SegmentLoadWaiter.java} | 56 +++++++---- .../msq/indexing/report/MSQStatusReport.java | 8 +- .../druid/msq/exec/SegmentLoadWaiterTest.java | 95 +++++++++++++++++++ .../indexing/report/MSQTaskReportTest.java | 14 +-- .../apache/druid/msq/test/MSQTestBase.java | 17 +++- .../apache/druid/discovery/BrokerClient.java | 8 +- 7 files changed, 172 insertions(+), 44 deletions(-) rename extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/{SegmentLoadAwaiter.java => SegmentLoadWaiter.java} (85%) create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/SegmentLoadWaiterTest.java 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 6a58b41f20b3..d0890d5fdad1 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 @@ -291,7 +291,7 @@ public class ControllerImpl implements Controller private WorkerMemoryParameters workerMemoryParameters; private boolean isDurableStorageEnabled; private boolean isFaultToleranceEnabled; - private volatile SegmentLoadAwaiter segmentLoadAwaiter; + private volatile SegmentLoadWaiter segmentLoadWaiter; public ControllerImpl( final MSQControllerTask task, @@ -454,9 +454,9 @@ public TaskStatus runTask(final Closer closer) cleanUpDurableStorageIfNeeded(); if (queryKernel != null && queryKernel.isSuccess()) { - if (segmentLoadAwaiter != null) { + if (segmentLoadWaiter != null) { // If successful and there are segments created, segmentLoadAwaiter should wait for them to become available. - segmentLoadAwaiter.awaitSegmentLoad(); + segmentLoadWaiter.awaitSegmentLoad(); } } @@ -511,7 +511,7 @@ public TaskStatus runTask(final Closer closer) queryStartTime, new Interval(queryStartTime, DateTimes.nowUtc()).toDurationMillis(), workerTaskLauncher, - segmentLoadAwaiter + segmentLoadWaiter ), stagesReport, countersSnapshot, @@ -855,7 +855,7 @@ public Map liveReports() queryStartTime, queryStartTime == null ? -1L : new Interval(queryStartTime, DateTimes.nowUtc()).toDurationMillis(), workerTaskLauncher, - segmentLoadAwaiter + segmentLoadWaiter ), makeStageReport( queryDef, @@ -1288,7 +1288,7 @@ private void publishAllSegments(final Set segments) throws IOExcept } } else { Set versionsToAwait = segmentsWithTombstones.stream().map(DataSegment::getVersion).collect(Collectors.toSet()); - segmentLoadAwaiter = new SegmentLoadAwaiter(context, task.getDataSource(), versionsToAwait, segmentsWithTombstones.size()); + segmentLoadWaiter = new SegmentLoadWaiter(context, task.getDataSource(), versionsToAwait, segmentsWithTombstones.size()); performSegmentPublish( context.taskActionClient(), SegmentTransactionalInsertAction.overwriteAction(null, null, segmentsWithTombstones) @@ -1296,7 +1296,7 @@ private void publishAllSegments(final Set segments) throws IOExcept } } else if (!segments.isEmpty()) { Set versionsToAwait = segments.stream().map(DataSegment::getVersion).collect(Collectors.toSet()); - segmentLoadAwaiter = new SegmentLoadAwaiter(context, task.getDataSource(), versionsToAwait, segments.size()); + segmentLoadWaiter = new SegmentLoadWaiter(context, task.getDataSource(), versionsToAwait, segments.size()); // Append mode. performSegmentPublish( context.taskActionClient(), @@ -2051,7 +2051,7 @@ private static MSQStatusReport makeStatusReport( @Nullable final DateTime queryStartTime, final long queryDuration, MSQWorkerTaskLauncher taskLauncher, - final SegmentLoadAwaiter segmentLoadWaiter + final SegmentLoadWaiter segmentLoadWaiter ) { int pendingTasks = -1; @@ -2063,7 +2063,7 @@ private static MSQStatusReport makeStatusReport( runningTasks = workerTaskCount.getRunningWorkerCount() + 1; // To account for controller. } - SegmentLoadAwaiter.SegmentLoadAwaiterStatus status = segmentLoadWaiter == null ? null : segmentLoadWaiter.status(); + SegmentLoadWaiter.SegmentLoadAwaiterStatus status = segmentLoadWaiter == null ? null : segmentLoadWaiter.status(); return new MSQStatusReport( taskState, diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadAwaiter.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java similarity index 85% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadAwaiter.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java index 19d61d9b65df..ac6067e0274e 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadAwaiter.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java @@ -20,26 +20,26 @@ package org.apache.druid.msq.exec; import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; import org.apache.druid.discovery.BrokerClient; import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.http.client.Request; -import org.apache.druid.java.util.http.client.response.StringFullResponseHolder; import org.apache.druid.sql.http.ResultFormat; import org.apache.druid.sql.http.SqlQuery; import org.jboss.netty.handler.codec.http.HttpMethod; import org.joda.time.DateTime; import org.joda.time.Interval; -import javax.servlet.http.HttpServletResponse; import java.util.HashMap; import java.util.Iterator; import java.util.Map; import java.util.Set; +import java.util.TreeSet; import java.util.concurrent.TimeUnit; /** @@ -53,9 +53,9 @@ * If the segments are not loaded within {@link #TIMEOUT_DURATION_MILLIS} milliseconds, this logs a warning and exits * for the same reason. */ -public class SegmentLoadAwaiter +public class SegmentLoadWaiter { - private static final Logger log = new Logger(SegmentLoadAwaiter.class); + private static final Logger log = new Logger(SegmentLoadWaiter.class); private static final long INITIAL_SLEEP_DURATION_MILLIS = TimeUnit.SECONDS.toMillis(5); private static final long SLEEP_DURATION_MILLIS = TimeUnit.SECONDS.toMillis(5); private static final long TIMEOUT_DURATION_MILLIS = TimeUnit.MINUTES.toMillis(10); @@ -70,18 +70,31 @@ public class SegmentLoadAwaiter private final Map versionToLoadStatusMap; private final String datasource; private final Set versionsToAwait; + private final boolean doWait; private volatile SegmentLoadAwaiterStatus status; - public SegmentLoadAwaiter(ControllerContext context, String datasource, Set versionsToAwait, int initialSegmentCount) + public SegmentLoadWaiter(ControllerContext context, String datasource, Set versionsToAwait, int initialSegmentCount) { this.brokerClient = context.injector().getInstance(BrokerClient.class); this.objectMapper = context.jsonMapper(); this.datasource = datasource; - this.versionsToAwait = versionsToAwait; + this.versionsToAwait = new TreeSet<>(versionsToAwait); this.versionToLoadStatusMap = new HashMap<>(); this.status = new SegmentLoadAwaiterStatus(State.INIT, null, 0, initialSegmentCount, initialSegmentCount); + this.doWait = true; } + @VisibleForTesting + SegmentLoadWaiter(BrokerClient brokerClient, ObjectMapper objectMapper, String datasource, Set versionsToAwait, int initialSegmentCount, boolean doWait) + { + this.brokerClient = brokerClient; + this.objectMapper = objectMapper; + this.datasource = datasource; + this.versionsToAwait = new TreeSet<>(versionsToAwait); + this.versionToLoadStatusMap = new HashMap<>(); + this.status = new SegmentLoadAwaiterStatus(State.INIT, null, 0, initialSegmentCount, initialSegmentCount); + this.doWait = doWait; + } /** * Uses broker client to check if all segments created by the ingestion have been loaded and updates the {@link #status)} * periodically. @@ -99,7 +112,7 @@ public void awaitSegmentLoad() // Sleep for a short duration to allow the segments that were just created to reflect in broker queries. // This avoids a race condition where the broker returns nothing as it is not aware of the new segments yet, // as this cannot be differentiated from the case where the segments have been dropped by load rules already. - Thread.sleep(INITIAL_SLEEP_DURATION_MILLIS); + waitIfNeeded(INITIAL_SLEEP_DURATION_MILLIS); while (!versionsToAwait.isEmpty()) { // Check the timeout and exit if exceeded. @@ -131,7 +144,7 @@ public void awaitSegmentLoad() updateStatus(State.RUNNING, startTime); // Sleep for a while before retrying. - Thread.sleep(SLEEP_DURATION_MILLIS); + waitIfNeeded(SLEEP_DURATION_MILLIS); } } } @@ -146,6 +159,12 @@ public void awaitSegmentLoad() updateStatus(State.SUCCESS, startTime); } + private void waitIfNeeded(long waitTimeMillis) throws Exception { + if (doWait) { + Thread.sleep(waitTimeMillis); + } + } + /** * Updates the {@link #status} with the latest details based on {@link #versionToLoadStatusMap} */ @@ -173,12 +192,8 @@ private VersionLoadStatus fetchLoadStatusForVersion(String version) throws Excep false, false, false, null, null); request.setContent("application/json", objectMapper.writeValueAsBytes(sqlQuery)); - StringFullResponseHolder responseHolder = brokerClient.sendQuery(request); - if (responseHolder.getStatus().getCode() != HttpServletResponse.SC_OK) { - throw new RE("HTTP request to[%s] failed", request.getUrl()); - } + String response = brokerClient.sendQuery(request); - String response = responseHolder.getContent(); 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); @@ -205,7 +220,7 @@ public static class SegmentLoadAwaiterStatus @JsonCreator public SegmentLoadAwaiterStatus( - @JsonProperty("state") SegmentLoadAwaiter.State state, + @JsonProperty("state") SegmentLoadWaiter.State state, @JsonProperty("startTime") DateTime startTime, @JsonProperty("duration") long duration, @JsonProperty("totalSegments") int totalSegments, @@ -220,7 +235,7 @@ public SegmentLoadAwaiterStatus( } @JsonProperty - public SegmentLoadAwaiter.State getState() + public SegmentLoadWaiter.State getState() { return state; } @@ -266,12 +281,12 @@ public static class VersionLoadStatus @JsonCreator public VersionLoadStatus( - @JsonProperty("totalSegments") int total_segments, - @JsonProperty("loadingSegments") int loading_segments + @JsonProperty("totalSegments") int totalSegments, + @JsonProperty("loadingSegments") int loadingSegments ) { - this.totalSegments = total_segments; - this.loadingSegments = loading_segments; + this.totalSegments = totalSegments; + this.loadingSegments = loadingSegments; } @JsonProperty("totalSegments") @@ -286,6 +301,7 @@ public int getLoadingSegments() return loadingSegments; } + @JsonIgnore public boolean isLoadingComplete() { return loadingSegments == 0; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQStatusReport.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQStatusReport.java index 47beed500c35..fc799f76c282 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQStatusReport.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQStatusReport.java @@ -24,7 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import org.apache.druid.indexer.TaskState; -import org.apache.druid.msq.exec.SegmentLoadAwaiter; +import org.apache.druid.msq.exec.SegmentLoadWaiter; import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.joda.time.DateTime; @@ -52,7 +52,7 @@ public class MSQStatusReport private final int runningTasks; @Nullable - private final SegmentLoadAwaiter.SegmentLoadAwaiterStatus segmentLoadAwaiterStatus; + private final SegmentLoadWaiter.SegmentLoadAwaiterStatus segmentLoadAwaiterStatus; @JsonCreator public MSQStatusReport( @@ -63,7 +63,7 @@ public MSQStatusReport( @JsonProperty("durationMs") long durationMs, @JsonProperty("pendingTasks") int pendingTasks, @JsonProperty("runningTasks") int runningTasks, - @JsonProperty("segmentLoadAwaiterStatus") @Nullable SegmentLoadAwaiter.SegmentLoadAwaiterStatus segmentLoadAwaiterStatus + @JsonProperty("segmentLoadAwaiterStatus") @Nullable SegmentLoadWaiter.SegmentLoadAwaiterStatus segmentLoadAwaiterStatus ) { this.status = Preconditions.checkNotNull(status, "status"); @@ -126,7 +126,7 @@ public long getDurationMs() @Nullable @JsonProperty @JsonInclude(JsonInclude.Include.NON_NULL) - public SegmentLoadAwaiter.SegmentLoadAwaiterStatus getSegmentLoadAwaiterStatus() + public SegmentLoadWaiter.SegmentLoadAwaiterStatus getSegmentLoadAwaiterStatus() { return segmentLoadAwaiterStatus; } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/SegmentLoadWaiterTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/SegmentLoadWaiterTest.java new file mode 100644 index 000000000000..95a7faae5bff --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/SegmentLoadWaiterTest.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.exec; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.discovery.BrokerClient; +import org.apache.druid.java.util.http.client.Request; +import org.junit.Test; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +public class SegmentLoadWaiterTest +{ + private static final String TEST_DATASOURCE = "testDatasource"; + + private SegmentLoadWaiter segmentLoadWaiter; + + private BrokerClient brokerClient; + + /** + * Single version created, loaded after 3 attempts + */ + @Test + public void testSingleVersionWaitsForLoadCorrectly() throws Exception + { + brokerClient = mock(BrokerClient.class); + + doReturn(mock(Request.class)).when(brokerClient).makeRequest(any(), anyString()); + doAnswer(new Answer() + { + int timesInvoked = 0; + @Override + public Object answer(InvocationOnMock invocation) throws Throwable + { + timesInvoked += 1; + SegmentLoadWaiter.VersionLoadStatus loadStatus = new SegmentLoadWaiter.VersionLoadStatus(5, 5 - timesInvoked); + return new ObjectMapper().writeValueAsString(loadStatus); + } + }).when(brokerClient).sendQuery(any()); + segmentLoadWaiter = new SegmentLoadWaiter(brokerClient, new ObjectMapper(), TEST_DATASOURCE, ImmutableSet.of("version1"), 5, false); + segmentLoadWaiter.awaitSegmentLoad(); + + verify(brokerClient, times(5)).sendQuery(any()); + } + + @Test + public void testMultipleVersionWaitsForLoadCorrectly() throws Exception + { + brokerClient = mock(BrokerClient.class); + + doReturn(mock(Request.class)).when(brokerClient).makeRequest(any(), anyString()); + doAnswer(new Answer() + { + int timesInvoked = 0; + @Override + public Object answer(InvocationOnMock invocation) throws Throwable + { + timesInvoked += 1; + SegmentLoadWaiter.VersionLoadStatus loadStatus = new SegmentLoadWaiter.VersionLoadStatus(5, 5 - timesInvoked); + return new ObjectMapper().writeValueAsString(loadStatus); + } + }).when(brokerClient).sendQuery(any()); + segmentLoadWaiter = new SegmentLoadWaiter(brokerClient, new ObjectMapper(), TEST_DATASOURCE, ImmutableSet.of("version1"), 5, false); + segmentLoadWaiter.awaitSegmentLoad(); + + verify(brokerClient, times(5)).sendQuery(any()); + } + +} \ No newline at end of file diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java index eb1bf67f498a..874107504f73 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java @@ -35,7 +35,7 @@ import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.msq.counters.CounterSnapshotsTree; -import org.apache.druid.msq.exec.SegmentLoadAwaiter; +import org.apache.druid.msq.exec.SegmentLoadWaiter; import org.apache.druid.msq.guice.MSQIndexingModule; import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.apache.druid.msq.indexing.error.TooManyColumnsFault; @@ -92,8 +92,8 @@ public void testSerdeResultsReport() throws Exception new Object[]{"bar"} ); - SegmentLoadAwaiter.SegmentLoadAwaiterStatus status = new SegmentLoadAwaiter.SegmentLoadAwaiterStatus( - SegmentLoadAwaiter.State.RUNNING, + SegmentLoadWaiter.SegmentLoadAwaiterStatus status = new SegmentLoadWaiter.SegmentLoadAwaiterStatus( + SegmentLoadWaiter.State.RUNNING, DateTimes.nowUtc(), 200L, 50, @@ -152,8 +152,8 @@ public void testSerdeResultsReport() throws Exception @Test public void testSerdeErrorReport() throws Exception { - SegmentLoadAwaiter.SegmentLoadAwaiterStatus status = new SegmentLoadAwaiter.SegmentLoadAwaiterStatus( - SegmentLoadAwaiter.State.FAILED, + SegmentLoadWaiter.SegmentLoadAwaiterStatus status = new SegmentLoadWaiter.SegmentLoadAwaiterStatus( + SegmentLoadWaiter.State.FAILED, DateTimes.nowUtc(), 200L, 50, @@ -197,8 +197,8 @@ public void testSerdeErrorReport() throws Exception @Test public void testWriteTaskReport() throws Exception { - SegmentLoadAwaiter.SegmentLoadAwaiterStatus status = new SegmentLoadAwaiter.SegmentLoadAwaiterStatus( - SegmentLoadAwaiter.State.SUCCESS, + SegmentLoadWaiter.SegmentLoadAwaiterStatus status = new SegmentLoadWaiter.SegmentLoadAwaiterStatus( + SegmentLoadWaiter.State.SUCCESS, DateTimes.nowUtc(), 200L, 50, diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index e44d1974b8a3..f1597cba3f2d 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -42,6 +42,7 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; +import org.apache.druid.discovery.BrokerClient; import org.apache.druid.discovery.NodeRole; import org.apache.druid.frame.testutil.FrameTestUtil; import org.apache.druid.guice.DruidInjectorBuilder; @@ -71,6 +72,7 @@ import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.http.client.Request; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.metadata.input.InputSourceModule; import org.apache.druid.msq.counters.CounterNames; @@ -79,6 +81,7 @@ import org.apache.druid.msq.counters.QueryCounterSnapshot; import org.apache.druid.msq.exec.ClusterStatisticsMergeMode; import org.apache.druid.msq.exec.Controller; +import org.apache.druid.msq.exec.SegmentLoadWaiter; import org.apache.druid.msq.exec.WorkerMemoryParameters; import org.apache.druid.msq.guice.MSQDurableStorageModule; import org.apache.druid.msq.guice.MSQExternalDataSourceModule; @@ -200,6 +203,10 @@ import static org.apache.druid.sql.calcite.util.CalciteTests.DATASOURCE2; import static org.apache.druid.sql.calcite.util.TestDataBuilder.ROWS1; import static org.apache.druid.sql.calcite.util.TestDataBuilder.ROWS2; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; /** * Base test runner for running MSQ unit tests. It sets up multi stage query execution environment @@ -341,7 +348,7 @@ public void tearDown2() // which depends on the object mapper that the injector will provide, once it // is built, but has not yet been build while we build the SQL engine. @Before - public void setUp2() + public void setUp2() throws Exception { groupByBuffers = TestGroupByBuffers.createDefault(); @@ -369,6 +376,7 @@ public void setUp2() segmentManager = new MSQTestSegmentManager(segmentCacheManager, indexIO); + BrokerClient brokerClient = mock(BrokerClient.class); List modules = ImmutableList.of( binder -> { DruidProcessingConfig druidProcessingConfig = new DruidProcessingConfig() @@ -455,7 +463,8 @@ public String getFormatString() ), new MSQExternalDataSourceModule(), new LookylooModule(), - new SegmentWranglerModule() + new SegmentWranglerModule(), + binder -> binder.bind(BrokerClient.class).toInstance(brokerClient) ); // adding node role injection to the modules, since CliPeon would also do that through run method Injector injector = new CoreInjectorBuilder(new StartupInjectorBuilder().build(), ImmutableSet.of(NodeRole.PEON)) @@ -465,6 +474,10 @@ public String getFormatString() objectMapper = setupObjectMapper(injector); objectMapper.registerModules(sqlModule.getJacksonModules()); + doReturn(mock(Request.class)).when(brokerClient).makeRequest(any(), anyString()); + SegmentLoadWaiter.VersionLoadStatus loadStatus = new SegmentLoadWaiter.VersionLoadStatus(1, 0); + doReturn(new String(objectMapper.writeValueAsBytes(loadStatus))).when(brokerClient).sendQuery(any()); + testTaskActionClient = Mockito.spy(new MSQTestTaskActionClient(objectMapper)); indexingServiceClient = new MSQTestOverlordServiceClient( objectMapper, diff --git a/server/src/main/java/org/apache/druid/discovery/BrokerClient.java b/server/src/main/java/org/apache/druid/discovery/BrokerClient.java index 88d91671f464..c60bede1b67e 100644 --- a/server/src/main/java/org/apache/druid/discovery/BrokerClient.java +++ b/server/src/main/java/org/apache/druid/discovery/BrokerClient.java @@ -35,6 +35,7 @@ import org.jboss.netty.handler.codec.http.HttpMethod; import org.jboss.netty.handler.codec.http.HttpResponseStatus; +import javax.servlet.http.HttpServletResponse; import java.io.IOException; import java.net.MalformedURLException; import java.net.URL; @@ -75,7 +76,7 @@ public Request makeRequest(HttpMethod httpMethod, String urlPath) throws IOExcep return new Request(httpMethod, new URL(StringUtils.format("%s%s", host, urlPath))); } - public StringFullResponseHolder sendQuery(Request request) throws Exception + public String sendQuery(Request request) throws Exception { StringFullResponseHandler responseHandler = new StringFullResponseHandler(StandardCharsets.UTF_8); @@ -111,8 +112,11 @@ public StringFullResponseHolder sendQuery(Request request) throws Exception MAX_RETRIES ); request = getNewRequestUrl(request); + } else if (responseStatus.getCode() != HttpServletResponse.SC_OK) { + log.warn("Request[%s] failed with error", request.getUrl(), responseStatus.getCode()); + continue; } else { - return fullResponseHolder; + return fullResponseHolder.getContent(); } } From 52aef1d05f03e4459e56e0783d00de9883e939fb Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Wed, 5 Jul 2023 17:03:23 +0530 Subject: [PATCH 04/19] Update query with replication factor --- docs/api-reference/sql-ingestion-api.md | 13 +++++----- .../apache/druid/msq/exec/ControllerImpl.java | 6 ++--- .../druid/msq/exec/SegmentLoadWaiter.java | 26 +++++++++---------- .../msq/indexing/MSQWorkerTaskLauncher.java | 10 ++++--- .../msq/indexing/report/MSQStatusReport.java | 10 +++---- .../druid/msq/exec/SegmentLoadWaiterTest.java | 4 +-- .../indexing/report/MSQTaskReportTest.java | 6 ++--- .../msq/sql/SqlStatementResourceTest.java | 6 +++-- 8 files changed, 43 insertions(+), 38 deletions(-) diff --git a/docs/api-reference/sql-ingestion-api.md b/docs/api-reference/sql-ingestion-api.md index c02dec15a6b5..c187b1e8a8e5 100644 --- a/docs/api-reference/sql-ingestion-api.md +++ b/docs/api-reference/sql-ingestion-api.md @@ -570,6 +570,12 @@ The following table describes the response fields when you retrieve a report for | `multiStageQuery.payload.status.durationMs` | Milliseconds elapsed after the query has started running. -1 denotes that the query hasn't started running yet. | | `multiStageQuery.payload.status.pendingTasks` | Number of tasks that are not fully started. -1 denotes that the number is currently unknown. | | `multiStageQuery.payload.status.runningTasks` | Number of currently running tasks. Should be at least 1 since the controller is included. | +| `multiStageQuery.payload.status.segmentLoadWaiterStatus` | Segment loading waiter container. Not present till the segments have been published | +| `multiStageQuery.payload.status.segmentLoadWaiterStatus.state` | RUNNING, SUCCESS, FAILED or TIMED_OUT | +| `multiStageQuery.payload.status.segmentLoadWaiterStatus.startTime` | Start time of the awaiter | +| `multiStageQuery.payload.status.segmentLoadWaiterStatus.duration` | The number of milliseconds that the awaiter has been running | +| `multiStageQuery.payload.status.segmentLoadWaiterStatus.totalSegments` | The total number of segments generated by the job. This includes tombstone segments (if any). | +| `multiStageQuery.payload.status.segmentLoadWaiterStatus.segmentsLeft` | The total number of segments remaining to be loaded. | | `multiStageQuery.payload.status.errorReport` | Error object. Only present if there was an error. | | `multiStageQuery.payload.status.errorReport.taskId` | The task that reported the error, if known. May be a controller task or a worker task. | | `multiStageQuery.payload.status.errorReport.host` | The hostname and port of the task that reported the error, if known. | @@ -593,13 +599,6 @@ The following table describes the response fields when you retrieve a report for | `multiStageQuery.payload.stages[].definition.processor` | An object defining the processor logic. | | `multiStageQuery.payload.stages[].definition.signature` | The output signature of the stage. | -| `multiStageQuery.payload.status.segmentLoadAwaiterStatus` | Segment loading awaiter container. Not present till the segments have been published | -| `multiStageQuery.payload.status.segmentLoadAwaiterStatus.state` | RUNNING, SUCCESS, FAILED or TIMED_OUT | -| `multiStageQuery.payload.status.segmentLoadAwaiterStatus.startTime` | Start time of the awaiter | -| `multiStageQuery.payload.status.segmentLoadAwaiterStatus.duration` | The number of milliseconds that the awaiter has been running | -| `multiStageQuery.payload.status.segmentLoadAwaiterStatus.totalSegments` | The total number of segments generated by the job. This includes tombstone segments (if any). | -| `multiStageQuery.payload.status.segmentLoadAwaiterStatus.segmentsLeft` | The total number of segments remaining to be loaded. | - ## Cancel a query task #### Request 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 0d0a4dfe1654..994999caf0a3 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 @@ -458,8 +458,8 @@ public TaskStatus runTask(final Closer closer) if (queryKernel != null && queryKernel.isSuccess()) { if (segmentLoadWaiter != null) { - // If successful and there are segments created, segmentLoadAwaiter should wait for them to become available. - segmentLoadWaiter.awaitSegmentLoad(); + // If successful and there are segments created, segmentLoadWaiter should wait for them to become available. + segmentLoadWaiter.waitForSegmentsToLoad(); } } @@ -2088,7 +2088,7 @@ private static MSQStatusReport makeStatusReport( runningTasks = workerTaskCount.getRunningWorkerCount() + 1; // To account for controller. } - SegmentLoadWaiter.SegmentLoadAwaiterStatus status = segmentLoadWaiter == null ? null : segmentLoadWaiter.status(); + SegmentLoadWaiter.SegmentLoadWaiterStatus status = segmentLoadWaiter == null ? null : segmentLoadWaiter.status(); return new MSQStatusReport( taskState, diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java index ac6067e0274e..c51fb2c70312 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java @@ -59,9 +59,9 @@ public class SegmentLoadWaiter private static final long INITIAL_SLEEP_DURATION_MILLIS = TimeUnit.SECONDS.toMillis(5); private static final long SLEEP_DURATION_MILLIS = TimeUnit.SECONDS.toMillis(5); private static final long TIMEOUT_DURATION_MILLIS = TimeUnit.MINUTES.toMillis(10); - private static final String LOAD_QUERY = "SELECT COUNT(*) AS totalSegments, " - + "COUNT(*) FILTER (WHERE is_available = 0 AND is_published = 1) AS loadingSegments " - + "FROM sys.segments " + private static final String LOAD_QUERY = "SELECT COUNT(*) AS totalSegments,\n" + + "COUNT(*) FILTER (WHERE is_available = 0 AND is_published = 1 AND replication_factor != 0) AS loadingSegments\n" + + "FROM sys.segments\n" + "WHERE datasource = '%s' AND is_overshadowed = 0 AND version = '%s'"; private final BrokerClient brokerClient; @@ -71,7 +71,7 @@ public class SegmentLoadWaiter private final String datasource; private final Set versionsToAwait; private final boolean doWait; - private volatile SegmentLoadAwaiterStatus status; + private volatile SegmentLoadWaiterStatus status; public SegmentLoadWaiter(ControllerContext context, String datasource, Set versionsToAwait, int initialSegmentCount) { @@ -80,7 +80,7 @@ public SegmentLoadWaiter(ControllerContext context, String datasource, Set(versionsToAwait); this.versionToLoadStatusMap = new HashMap<>(); - this.status = new SegmentLoadAwaiterStatus(State.INIT, null, 0, initialSegmentCount, initialSegmentCount); + this.status = new SegmentLoadWaiterStatus(State.INIT, null, 0, initialSegmentCount, initialSegmentCount); this.doWait = true; } @@ -92,7 +92,7 @@ public SegmentLoadWaiter(ControllerContext context, String datasource, Set(versionsToAwait); this.versionToLoadStatusMap = new HashMap<>(); - this.status = new SegmentLoadAwaiterStatus(State.INIT, null, 0, initialSegmentCount, initialSegmentCount); + this.status = new SegmentLoadWaiterStatus(State.INIT, null, 0, initialSegmentCount, initialSegmentCount); this.doWait = doWait; } /** @@ -104,14 +104,14 @@ public SegmentLoadWaiter(ControllerContext context, String datasource, Set * Only expected to be called from the main controller thread. */ - public void awaitSegmentLoad() + public void waitForSegmentsToLoad() { DateTime startTime = DateTimes.nowUtc(); try { // Sleep for a short duration to allow the segments that were just created to reflect in broker queries. - // This avoids a race condition where the broker returns nothing as it is not aware of the new segments yet, - // as this cannot be differentiated from the case where the segments have been dropped by load rules already. + // This avoids a race condition where the broker returns an empty response as it is not yet aware of the new segments + // yet. This cannot be differentiated from the case where the segments have been dropped due to load rules already. waitIfNeeded(INITIAL_SLEEP_DURATION_MILLIS); while (!versionsToAwait.isEmpty()) { @@ -177,7 +177,7 @@ private void updateStatus(State state, DateTime startTime) } long runningMillis = new Interval(startTime, DateTimes.nowUtc()).toDurationMillis(); - status = new SegmentLoadAwaiterStatus(state, startTime, runningMillis, totalSegmentCount, pendingSegmentCount); + status = new SegmentLoadWaiterStatus(state, startTime, runningMillis, totalSegmentCount, pendingSegmentCount); } /** @@ -205,12 +205,12 @@ private VersionLoadStatus fetchLoadStatusForVersion(String version) throws Excep /** * Returns the current status of the load. */ - public SegmentLoadAwaiterStatus status() + public SegmentLoadWaiterStatus status() { return status; } - public static class SegmentLoadAwaiterStatus + public static class SegmentLoadWaiterStatus { private final State state; private final DateTime startTime; @@ -219,7 +219,7 @@ public static class SegmentLoadAwaiterStatus private final int segmentsLeft; @JsonCreator - public SegmentLoadAwaiterStatus( + public SegmentLoadWaiterStatus( @JsonProperty("state") SegmentLoadWaiter.State state, @JsonProperty("startTime") DateTime startTime, @JsonProperty("duration") long duration, diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java index 44ad42251a5e..cddbfead87f9 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java @@ -465,9 +465,13 @@ private void runNewTasks() public WorkerCount getWorkerTaskCount() { synchronized (taskIds) { - int runningTasks = fullyStartedTasks.size(); - int pendingTasks = desiredTaskCount - runningTasks; - return new WorkerCount(runningTasks, pendingTasks); + if (stopFuture.isDone()) { + return new WorkerCount(0, 0); + } else { + int runningTasks = fullyStartedTasks.size(); + int pendingTasks = desiredTaskCount - runningTasks; + return new WorkerCount(runningTasks, pendingTasks); + } } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQStatusReport.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQStatusReport.java index fc799f76c282..ea721d84f474 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQStatusReport.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQStatusReport.java @@ -52,7 +52,7 @@ public class MSQStatusReport private final int runningTasks; @Nullable - private final SegmentLoadWaiter.SegmentLoadAwaiterStatus segmentLoadAwaiterStatus; + private final SegmentLoadWaiter.SegmentLoadWaiterStatus segmentLoadWaiterStatus; @JsonCreator public MSQStatusReport( @@ -63,7 +63,7 @@ public MSQStatusReport( @JsonProperty("durationMs") long durationMs, @JsonProperty("pendingTasks") int pendingTasks, @JsonProperty("runningTasks") int runningTasks, - @JsonProperty("segmentLoadAwaiterStatus") @Nullable SegmentLoadWaiter.SegmentLoadAwaiterStatus segmentLoadAwaiterStatus + @JsonProperty("segmentLoadWaiterStatus") @Nullable SegmentLoadWaiter.SegmentLoadWaiterStatus segmentLoadWaiterStatus ) { this.status = Preconditions.checkNotNull(status, "status"); @@ -73,7 +73,7 @@ public MSQStatusReport( this.durationMs = durationMs; this.pendingTasks = pendingTasks; this.runningTasks = runningTasks; - this.segmentLoadAwaiterStatus = segmentLoadAwaiterStatus; + this.segmentLoadWaiterStatus = segmentLoadWaiterStatus; } @JsonProperty @@ -126,9 +126,9 @@ public long getDurationMs() @Nullable @JsonProperty @JsonInclude(JsonInclude.Include.NON_NULL) - public SegmentLoadWaiter.SegmentLoadAwaiterStatus getSegmentLoadAwaiterStatus() + public SegmentLoadWaiter.SegmentLoadWaiterStatus getSegmentLoadWaiterStatus() { - return segmentLoadAwaiterStatus; + return segmentLoadWaiterStatus; } @Override diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/SegmentLoadWaiterTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/SegmentLoadWaiterTest.java index 95a7faae5bff..bbc93a964024 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/SegmentLoadWaiterTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/SegmentLoadWaiterTest.java @@ -64,7 +64,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable } }).when(brokerClient).sendQuery(any()); segmentLoadWaiter = new SegmentLoadWaiter(brokerClient, new ObjectMapper(), TEST_DATASOURCE, ImmutableSet.of("version1"), 5, false); - segmentLoadWaiter.awaitSegmentLoad(); + segmentLoadWaiter.waitForSegmentsToLoad(); verify(brokerClient, times(5)).sendQuery(any()); } @@ -87,7 +87,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable } }).when(brokerClient).sendQuery(any()); segmentLoadWaiter = new SegmentLoadWaiter(brokerClient, new ObjectMapper(), TEST_DATASOURCE, ImmutableSet.of("version1"), 5, false); - segmentLoadWaiter.awaitSegmentLoad(); + segmentLoadWaiter.waitForSegmentsToLoad(); verify(brokerClient, times(5)).sendQuery(any()); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java index e8db7e6abb15..63cd350ef949 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java @@ -92,7 +92,7 @@ public void testSerdeResultsReport() throws Exception new Object[]{"bar"} ); - SegmentLoadWaiter.SegmentLoadAwaiterStatus status = new SegmentLoadWaiter.SegmentLoadAwaiterStatus( + SegmentLoadWaiter.SegmentLoadWaiterStatus status = new SegmentLoadWaiter.SegmentLoadWaiterStatus( SegmentLoadWaiter.State.RUNNING, DateTimes.nowUtc(), 200L, @@ -152,7 +152,7 @@ public void testSerdeResultsReport() throws Exception @Test public void testSerdeErrorReport() throws Exception { - SegmentLoadWaiter.SegmentLoadAwaiterStatus status = new SegmentLoadWaiter.SegmentLoadAwaiterStatus( + SegmentLoadWaiter.SegmentLoadWaiterStatus status = new SegmentLoadWaiter.SegmentLoadWaiterStatus( SegmentLoadWaiter.State.FAILED, DateTimes.nowUtc(), 200L, @@ -197,7 +197,7 @@ public void testSerdeErrorReport() throws Exception @Test public void testWriteTaskReport() throws Exception { - SegmentLoadWaiter.SegmentLoadAwaiterStatus status = new SegmentLoadWaiter.SegmentLoadAwaiterStatus( + SegmentLoadWaiter.SegmentLoadWaiterStatus status = new SegmentLoadWaiter.SegmentLoadWaiterStatus( SegmentLoadWaiter.State.SUCCESS, DateTimes.nowUtc(), 200L, diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java index 6a19f3f792ef..84a7919242ab 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java @@ -233,7 +233,8 @@ public class SqlStatementResourceTest extends MSQTestBase null, 0, 1, - 2 + 2, + null ), MSQStagesReport.create( MSQTaskReportTest.QUERY_DEFINITION, @@ -281,7 +282,8 @@ public class SqlStatementResourceTest extends MSQTestBase null, 0, 1, - 2 + 2, + null ), MSQStagesReport.create( MSQTaskReportTest.QUERY_DEFINITION, From 529a577d0d808d380e872c93a36b228fddad654c Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Thu, 27 Jul 2023 14:16:42 +0530 Subject: [PATCH 05/19] Cleanup code --- docs/api-reference/sql-ingestion-api.md | 2 +- .../apache/druid/msq/exec/SegmentLoadWaiter.java | 14 ++++++++++---- .../druid/msq/exec/SegmentLoadWaiterTest.java | 2 +- .../msq/indexing/report/MSQTaskReportTest.java | 2 +- .../org/apache/druid/msq/test/MSQTestBase.java | 2 +- 5 files changed, 14 insertions(+), 8 deletions(-) diff --git a/docs/api-reference/sql-ingestion-api.md b/docs/api-reference/sql-ingestion-api.md index c187b1e8a8e5..6ec9819454a9 100644 --- a/docs/api-reference/sql-ingestion-api.md +++ b/docs/api-reference/sql-ingestion-api.md @@ -571,7 +571,7 @@ The following table describes the response fields when you retrieve a report for | `multiStageQuery.payload.status.pendingTasks` | Number of tasks that are not fully started. -1 denotes that the number is currently unknown. | | `multiStageQuery.payload.status.runningTasks` | Number of currently running tasks. Should be at least 1 since the controller is included. | | `multiStageQuery.payload.status.segmentLoadWaiterStatus` | Segment loading waiter container. Not present till the segments have been published | -| `multiStageQuery.payload.status.segmentLoadWaiterStatus.state` | RUNNING, SUCCESS, FAILED or TIMED_OUT | +| `multiStageQuery.payload.status.segmentLoadWaiterStatus.state` | WAITING, SUCCESS, FAILED or TIMED_OUT | | `multiStageQuery.payload.status.segmentLoadWaiterStatus.startTime` | Start time of the awaiter | | `multiStageQuery.payload.status.segmentLoadWaiterStatus.duration` | The number of milliseconds that the awaiter has been running | | `multiStageQuery.payload.status.segmentLoadWaiterStatus.totalSegments` | The total number of segments generated by the job. This includes tombstone segments (if any). | diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java index c51fb2c70312..62e7d22fd2a4 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; @@ -35,6 +36,7 @@ import org.joda.time.DateTime; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.util.HashMap; import java.util.Iterator; import java.util.Map; @@ -95,6 +97,7 @@ public SegmentLoadWaiter(ControllerContext context, String datasource, Set Date: Mon, 7 Aug 2023 10:08:24 +0530 Subject: [PATCH 06/19] Code cleanup --- docs/api-reference/sql-ingestion-api.md | 2 +- .../druid/msq/indexing/MSQWorkerTaskLauncher.java | 15 ++++++++++++++- .../druid/msq/test/MSQTestControllerContext.java | 1 - .../org/apache/druid/discovery/BrokerClient.java | 8 ++++---- 4 files changed, 19 insertions(+), 7 deletions(-) diff --git a/docs/api-reference/sql-ingestion-api.md b/docs/api-reference/sql-ingestion-api.md index 6ec9819454a9..7ab0bf0236c8 100644 --- a/docs/api-reference/sql-ingestion-api.md +++ b/docs/api-reference/sql-ingestion-api.md @@ -571,7 +571,7 @@ The following table describes the response fields when you retrieve a report for | `multiStageQuery.payload.status.pendingTasks` | Number of tasks that are not fully started. -1 denotes that the number is currently unknown. | | `multiStageQuery.payload.status.runningTasks` | Number of currently running tasks. Should be at least 1 since the controller is included. | | `multiStageQuery.payload.status.segmentLoadWaiterStatus` | Segment loading waiter container. Not present till the segments have been published | -| `multiStageQuery.payload.status.segmentLoadWaiterStatus.state` | WAITING, SUCCESS, FAILED or TIMED_OUT | +| `multiStageQuery.payload.status.segmentLoadWaiterStatus.state` | INIT, WAITING, SUCCESS, FAILED or TIMED_OUT | | `multiStageQuery.payload.status.segmentLoadWaiterStatus.startTime` | Start time of the awaiter | | `multiStageQuery.payload.status.segmentLoadWaiterStatus.duration` | The number of milliseconds that the awaiter has been running | | `multiStageQuery.payload.status.segmentLoadWaiterStatus.totalSegments` | The total number of segments generated by the job. This includes tombstone segments (if any). | diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java index bb25982c2fb6..5e4c951ad1bc 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java @@ -182,7 +182,20 @@ public ListenableFuture start() */ public void stop(final boolean interrupt) { - if (state.compareAndSet(State.STARTED, State.STOPPED)) { + if (state.compareAndSet(State.NEW, State.STOPPED)) { + state.set(State.STOPPED); + if (interrupt) { + cancelTasksOnStop.set(true); + } + + synchronized (taskIds) { + // Wake up sleeping mainLoop. + taskIds.notifyAll(); + } + exec.shutdown(); + stopFuture.set(null); + } + else if (state.compareAndSet(State.STARTED, State.STOPPED)) { if (interrupt) { cancelTasksOnStop.set(true); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java index 719c7767459a..7c37723ecc80 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java @@ -112,7 +112,6 @@ public MSQTestControllerContext( .collect(Collectors.toList()) ) ); - Mockito.when(coordinatorClient.isHandOffComplete(ArgumentMatchers.anyString(), ArgumentMatchers.any())).thenReturn(Boolean.TRUE); this.workerMemoryParameters = workerMemoryParameters; } diff --git a/server/src/main/java/org/apache/druid/discovery/BrokerClient.java b/server/src/main/java/org/apache/druid/discovery/BrokerClient.java index c60bede1b67e..36b36c3e152e 100644 --- a/server/src/main/java/org/apache/druid/discovery/BrokerClient.java +++ b/server/src/main/java/org/apache/druid/discovery/BrokerClient.java @@ -92,12 +92,12 @@ public String sendQuery(Request request) throws Exception // Unwrap IOExceptions and ChannelExceptions, re-throw others Throwables.propagateIfInstanceOf(e.getCause(), IOException.class); Throwables.propagateIfInstanceOf(e.getCause(), ChannelException.class); - throw new RE(e, "HTTP request to[%s] failed", request.getUrl()); + throw new RE(e, "HTTP request to [%s] failed", request.getUrl()); } } catch (IOException | ChannelException ex) { // can happen if the node is stopped. - log.warn(ex, "Request[%s] failed.", request.getUrl()); + log.warn(ex, "Request [%s] failed.", request.getUrl()); request = getNewRequestUrl(request); continue; } @@ -105,7 +105,7 @@ public String sendQuery(Request request) throws Exception if (HttpResponseStatus.SERVICE_UNAVAILABLE.equals(responseStatus) || HttpResponseStatus.GATEWAY_TIMEOUT.equals(responseStatus)) { log.warn( - "Request[%s] received a %s response. Attempt %s/%s", + "Request [%s] received a [%s] response. Attempt [%s]/[%s]", request.getUrl(), responseStatus, counter + 1, @@ -113,7 +113,7 @@ public String sendQuery(Request request) throws Exception ); request = getNewRequestUrl(request); } else if (responseStatus.getCode() != HttpServletResponse.SC_OK) { - log.warn("Request[%s] failed with error", request.getUrl(), responseStatus.getCode()); + log.warn("Request [%s] failed with error code [%s]", request.getUrl(), responseStatus.getCode()); continue; } else { return fullResponseHolder.getContent(); From d86ad22e0fc0b6ee9b645a60e1ac0e5d6fbda127 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Wed, 9 Aug 2023 11:23:41 +0530 Subject: [PATCH 07/19] Improve coverage --- .../druid/msq/exec/SegmentLoadWaiter.java | 3 +- .../apache/druid/discovery/BrokerClient.java | 2 +- .../druid/discovery/BrokerClientTest.java | 102 ++++++++++++++++++ .../discovery/DruidLeaderClientTest.java | 2 +- 4 files changed, 106 insertions(+), 3 deletions(-) create mode 100644 server/src/test/java/org/apache/druid/discovery/BrokerClientTest.java diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java index 62e7d22fd2a4..d4b3abc4a41a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java @@ -37,6 +37,7 @@ import org.joda.time.Interval; import javax.annotation.Nullable; +import javax.ws.rs.core.MediaType; import java.util.HashMap; import java.util.Iterator; import java.util.Map; @@ -194,7 +195,7 @@ private VersionLoadStatus fetchLoadStatusForVersion(String version) throws Excep SqlQuery sqlQuery = new SqlQuery(StringUtils.format(LOAD_QUERY, datasource, version), ResultFormat.OBJECTLINES, false, false, false, null, null); - request.setContent("application/json", objectMapper.writeValueAsBytes(sqlQuery)); + request.setContent(MediaType.APPLICATION_JSON, objectMapper.writeValueAsBytes(sqlQuery)); String response = brokerClient.sendQuery(request); diff --git a/server/src/main/java/org/apache/druid/discovery/BrokerClient.java b/server/src/main/java/org/apache/druid/discovery/BrokerClient.java index 36b36c3e152e..b25944549123 100644 --- a/server/src/main/java/org/apache/druid/discovery/BrokerClient.java +++ b/server/src/main/java/org/apache/druid/discovery/BrokerClient.java @@ -101,6 +101,7 @@ public String sendQuery(Request request) throws Exception request = getNewRequestUrl(request); continue; } + HttpResponseStatus responseStatus = fullResponseHolder.getResponse().getStatus(); if (HttpResponseStatus.SERVICE_UNAVAILABLE.equals(responseStatus) || HttpResponseStatus.GATEWAY_TIMEOUT.equals(responseStatus)) { @@ -114,7 +115,6 @@ public String sendQuery(Request request) throws Exception request = getNewRequestUrl(request); } else if (responseStatus.getCode() != HttpServletResponse.SC_OK) { log.warn("Request [%s] failed with error code [%s]", request.getUrl(), responseStatus.getCode()); - continue; } else { return fullResponseHolder.getContent(); } diff --git a/server/src/test/java/org/apache/druid/discovery/BrokerClientTest.java b/server/src/test/java/org/apache/druid/discovery/BrokerClientTest.java new file mode 100644 index 000000000000..a85f5227ae01 --- /dev/null +++ b/server/src/test/java/org/apache/druid/discovery/BrokerClientTest.java @@ -0,0 +1,102 @@ +/* + * 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.discovery; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Module; +import com.google.inject.name.Names; +import org.apache.druid.guice.GuiceInjectors; +import org.apache.druid.guice.Jerseys; +import org.apache.druid.guice.JsonConfigProvider; +import org.apache.druid.guice.LazySingleton; +import org.apache.druid.guice.LifecycleModule; +import org.apache.druid.guice.annotations.Self; +import org.apache.druid.initialization.Initialization; +import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.java.util.http.client.Request; +import org.apache.druid.server.DruidNode; +import org.apache.druid.server.initialization.BaseJettyTest; +import org.apache.druid.server.initialization.jetty.JettyServerInitializer; +import org.easymock.EasyMock; +import org.eclipse.jetty.server.Server; +import org.jboss.netty.handler.codec.http.HttpMethod; +import org.junit.Assert; +import org.junit.Test; + +import java.nio.charset.StandardCharsets; + +public class BrokerClientTest extends BaseJettyTest +{ + + private DiscoveryDruidNode discoveryDruidNode; + private HttpClient httpClient; + + @Override + protected Injector setupInjector() + { + final DruidNode node = new DruidNode("test", "localhost", false, null, null, true, false); + discoveryDruidNode = new DiscoveryDruidNode(node, NodeRole.PEON, ImmutableMap.of()); + + Injector injector = Initialization.makeInjectorWithModules( + GuiceInjectors.makeStartupInjector(), ImmutableList.of( + binder -> { + JsonConfigProvider.bindInstance( + binder, + Key.get(DruidNode.class, Self.class), + node + ); + binder.bind(Integer.class).annotatedWith(Names.named("port")).toInstance(node.getPlaintextPort()); + binder.bind(JettyServerInitializer.class).to(DruidLeaderClientTest.TestJettyServerInitializer.class).in( + LazySingleton.class); + Jerseys.addResource(binder, DruidLeaderClientTest.SimpleResource.class); + LifecycleModule.register(binder, Server.class); + } + ) + ); + httpClient = injector.getInstance(BaseJettyTest.ClientHolder.class).getClient(); + return injector; + } + + @Test + public void testSimple() throws Exception + { + DruidNodeDiscovery druidNodeDiscovery = EasyMock.createMock(DruidNodeDiscovery.class); + EasyMock.expect(druidNodeDiscovery.getAllNodes()).andReturn( + ImmutableList.of(discoveryDruidNode) + ); + + DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class); + EasyMock.expect(druidNodeDiscoveryProvider.getForNodeRole(NodeRole.BROKER)).andReturn(druidNodeDiscovery); + + EasyMock.replay(druidNodeDiscovery, druidNodeDiscoveryProvider); + + BrokerClient brokerClient = new BrokerClient( + httpClient, + druidNodeDiscoveryProvider + ); + + Request request = brokerClient.makeRequest(HttpMethod.POST, "/simple/direct"); + request.setContent("hello".getBytes(StandardCharsets.UTF_8)); + Assert.assertEquals("hello", brokerClient.sendQuery(request)); + } +} diff --git a/server/src/test/java/org/apache/druid/discovery/DruidLeaderClientTest.java b/server/src/test/java/org/apache/druid/discovery/DruidLeaderClientTest.java index 6b8f32ba1906..3bda8dd5168d 100644 --- a/server/src/test/java/org/apache/druid/discovery/DruidLeaderClientTest.java +++ b/server/src/test/java/org/apache/druid/discovery/DruidLeaderClientTest.java @@ -278,7 +278,7 @@ public void testFindCurrentLeader() Assert.assertEquals("http://localhost:1234/", druidLeaderClient.findCurrentLeader()); } - private static class TestJettyServerInitializer implements JettyServerInitializer + static class TestJettyServerInitializer implements JettyServerInitializer { @Override public void initialize(Server server, Injector injector) From 47da704adaf93094dd93a49f768e7187b4f102a7 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Wed, 9 Aug 2023 17:51:01 +0530 Subject: [PATCH 08/19] Resolve build failures --- docs/api-reference/sql-ingestion-api.md | 8 ++++---- .../druid/msq/indexing/MSQWorkerTaskLauncher.java | 3 +-- .../org/apache/druid/discovery/BrokerClient.java | 3 +-- .../apache/druid/discovery/BrokerClientTest.java | 13 ++++--------- website/.spelling | 1 + 5 files changed, 11 insertions(+), 17 deletions(-) diff --git a/docs/api-reference/sql-ingestion-api.md b/docs/api-reference/sql-ingestion-api.md index 7ab0bf0236c8..2a33ed7fb9ff 100644 --- a/docs/api-reference/sql-ingestion-api.md +++ b/docs/api-reference/sql-ingestion-api.md @@ -570,10 +570,10 @@ The following table describes the response fields when you retrieve a report for | `multiStageQuery.payload.status.durationMs` | Milliseconds elapsed after the query has started running. -1 denotes that the query hasn't started running yet. | | `multiStageQuery.payload.status.pendingTasks` | Number of tasks that are not fully started. -1 denotes that the number is currently unknown. | | `multiStageQuery.payload.status.runningTasks` | Number of currently running tasks. Should be at least 1 since the controller is included. | -| `multiStageQuery.payload.status.segmentLoadWaiterStatus` | Segment loading waiter container. Not present till the segments have been published | -| `multiStageQuery.payload.status.segmentLoadWaiterStatus.state` | INIT, WAITING, SUCCESS, FAILED or TIMED_OUT | -| `multiStageQuery.payload.status.segmentLoadWaiterStatus.startTime` | Start time of the awaiter | -| `multiStageQuery.payload.status.segmentLoadWaiterStatus.duration` | The number of milliseconds that the awaiter has been running | +| `multiStageQuery.payload.status.segmentLoadWaiterStatus` | Segment loading waiter container. Only present after the segments have been published. | +| `multiStageQuery.payload.status.segmentLoadWaiterStatus.state` | Either INIT, WAITING, SUCCESS, FAILED or TIMED_OUT. | +| `multiStageQuery.payload.status.segmentLoadWaiterStatus.startTime` | Time since which the controller has been waiting for the segments to finish loading. | +| `multiStageQuery.payload.status.segmentLoadWaiterStatus.duration` | The duration in milliseconds that the controller has been waiting for the segments to load. | | `multiStageQuery.payload.status.segmentLoadWaiterStatus.totalSegments` | The total number of segments generated by the job. This includes tombstone segments (if any). | | `multiStageQuery.payload.status.segmentLoadWaiterStatus.segmentsLeft` | The total number of segments remaining to be loaded. | | `multiStageQuery.payload.status.errorReport` | Error object. Only present if there was an error. | diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java index 5e4c951ad1bc..dcc81d868644 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java @@ -194,8 +194,7 @@ public void stop(final boolean interrupt) } exec.shutdown(); stopFuture.set(null); - } - else if (state.compareAndSet(State.STARTED, State.STOPPED)) { + } else if (state.compareAndSet(State.STARTED, State.STOPPED)) { if (interrupt) { cancelTasksOnStop.set(true); } diff --git a/server/src/main/java/org/apache/druid/discovery/BrokerClient.java b/server/src/main/java/org/apache/druid/discovery/BrokerClient.java index b25944549123..6636a5cfc498 100644 --- a/server/src/main/java/org/apache/druid/discovery/BrokerClient.java +++ b/server/src/main/java/org/apache/druid/discovery/BrokerClient.java @@ -71,7 +71,7 @@ public Request makeRequest(HttpMethod httpMethod, String urlPath) throws IOExcep String host = ClientUtils.pickOneHost(druidNodeDiscovery); if (host == null) { - throw new IOE("No known server"); + throw new IOE("No known server."); } return new Request(httpMethod, new URL(StringUtils.format("%s%s", host, urlPath))); } @@ -81,7 +81,6 @@ public String sendQuery(Request request) throws Exception StringFullResponseHandler responseHandler = new StringFullResponseHandler(StandardCharsets.UTF_8); for (int counter = 0; counter < MAX_RETRIES; counter++) { - final StringFullResponseHolder fullResponseHolder; try { diff --git a/server/src/test/java/org/apache/druid/discovery/BrokerClientTest.java b/server/src/test/java/org/apache/druid/discovery/BrokerClientTest.java index a85f5227ae01..f52110ab9822 100644 --- a/server/src/test/java/org/apache/druid/discovery/BrokerClientTest.java +++ b/server/src/test/java/org/apache/druid/discovery/BrokerClientTest.java @@ -23,7 +23,6 @@ import com.google.common.collect.ImmutableMap; import com.google.inject.Injector; import com.google.inject.Key; -import com.google.inject.Module; import com.google.inject.name.Names; import org.apache.druid.guice.GuiceInjectors; import org.apache.druid.guice.Jerseys; @@ -47,7 +46,6 @@ public class BrokerClientTest extends BaseJettyTest { - private DiscoveryDruidNode discoveryDruidNode; private HttpClient httpClient; @@ -55,10 +53,10 @@ public class BrokerClientTest extends BaseJettyTest protected Injector setupInjector() { final DruidNode node = new DruidNode("test", "localhost", false, null, null, true, false); - discoveryDruidNode = new DiscoveryDruidNode(node, NodeRole.PEON, ImmutableMap.of()); + discoveryDruidNode = new DiscoveryDruidNode(node, NodeRole.BROKER, ImmutableMap.of()); Injector injector = Initialization.makeInjectorWithModules( - GuiceInjectors.makeStartupInjector(), ImmutableList.of( + GuiceInjectors.makeStartupInjector(), ImmutableList.of( binder -> { JsonConfigProvider.bindInstance( binder, @@ -66,8 +64,7 @@ protected Injector setupInjector() node ); binder.bind(Integer.class).annotatedWith(Names.named("port")).toInstance(node.getPlaintextPort()); - binder.bind(JettyServerInitializer.class).to(DruidLeaderClientTest.TestJettyServerInitializer.class).in( - LazySingleton.class); + binder.bind(JettyServerInitializer.class).to(DruidLeaderClientTest.TestJettyServerInitializer.class).in(LazySingleton.class); Jerseys.addResource(binder, DruidLeaderClientTest.SimpleResource.class); LifecycleModule.register(binder, Server.class); } @@ -81,9 +78,7 @@ protected Injector setupInjector() public void testSimple() throws Exception { DruidNodeDiscovery druidNodeDiscovery = EasyMock.createMock(DruidNodeDiscovery.class); - EasyMock.expect(druidNodeDiscovery.getAllNodes()).andReturn( - ImmutableList.of(discoveryDruidNode) - ); + EasyMock.expect(druidNodeDiscovery.getAllNodes()).andReturn(ImmutableList.of(discoveryDruidNode)); DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class); EasyMock.expect(druidNodeDiscoveryProvider.getForNodeRole(NodeRole.BROKER)).andReturn(druidNodeDiscovery); diff --git a/website/.spelling b/website/.spelling index cc7a09882d92..ace30aaf69d4 100644 --- a/website/.spelling +++ b/website/.spelling @@ -744,6 +744,7 @@ TooManyWorkers NotEnoughMemory WorkerFailed WorkerRpcFailed +TIMED_OUT # MSQ context parameters maxNumTasks taskAssignment From e2b1787c941149c6ac3dd9340ed46e03d192fcfc Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Fri, 11 Aug 2023 10:55:04 +0530 Subject: [PATCH 09/19] Address review comments --- .../apache/druid/msq/exec/ControllerImpl.java | 19 +++++++++++++++++-- .../druid/msq/exec/SegmentLoadWaiter.java | 15 +-------------- .../apache/druid/discovery/BrokerClient.java | 19 ++++++++++--------- .../apache/druid/discovery/ClientUtils.java | 2 +- 4 files changed, 29 insertions(+), 26 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 7e4557b8393b..bfaa2372c437 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 @@ -44,6 +44,7 @@ import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.discovery.BrokerClient; import org.apache.druid.frame.allocation.ArenaMemoryAllocator; import org.apache.druid.frame.channel.FrameChannelSequence; import org.apache.druid.frame.key.ClusterBy; @@ -1316,7 +1317,14 @@ private void publishAllSegments(final Set segments) throws IOExcept } } else { Set versionsToAwait = segmentsWithTombstones.stream().map(DataSegment::getVersion).collect(Collectors.toSet()); - segmentLoadWaiter = new SegmentLoadWaiter(context, task.getDataSource(), versionsToAwait, segmentsWithTombstones.size()); + segmentLoadWaiter = new SegmentLoadWaiter( + context.injector().getInstance(BrokerClient.class), + context.jsonMapper(), + task.getDataSource(), + versionsToAwait, + segmentsWithTombstones.size(), + true + ); performSegmentPublish( context.taskActionClient(), SegmentTransactionalInsertAction.overwriteAction(null, null, segmentsWithTombstones) @@ -1324,7 +1332,14 @@ private void publishAllSegments(final Set segments) throws IOExcept } } else if (!segments.isEmpty()) { Set versionsToAwait = segments.stream().map(DataSegment::getVersion).collect(Collectors.toSet()); - segmentLoadWaiter = new SegmentLoadWaiter(context, task.getDataSource(), versionsToAwait, segments.size()); + segmentLoadWaiter = new SegmentLoadWaiter( + context.injector().getInstance(BrokerClient.class), + context.jsonMapper(), + task.getDataSource(), + versionsToAwait, + segmentsWithTombstones.size(), + true + ); // Append mode. performSegmentPublish( context.taskActionClient(), diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java index d4b3abc4a41a..036f1da19149 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java @@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.annotations.VisibleForTesting; import org.apache.druid.discovery.BrokerClient; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.StringUtils; @@ -76,19 +75,7 @@ public class SegmentLoadWaiter private final boolean doWait; private volatile SegmentLoadWaiterStatus status; - public SegmentLoadWaiter(ControllerContext context, String datasource, Set versionsToAwait, int initialSegmentCount) - { - this.brokerClient = context.injector().getInstance(BrokerClient.class); - this.objectMapper = context.jsonMapper(); - this.datasource = datasource; - this.versionsToAwait = new TreeSet<>(versionsToAwait); - this.versionToLoadStatusMap = new HashMap<>(); - this.status = new SegmentLoadWaiterStatus(State.INIT, null, 0, initialSegmentCount, initialSegmentCount); - this.doWait = true; - } - - @VisibleForTesting - SegmentLoadWaiter(BrokerClient brokerClient, ObjectMapper objectMapper, String datasource, Set versionsToAwait, int initialSegmentCount, boolean doWait) + public SegmentLoadWaiter(BrokerClient brokerClient, ObjectMapper objectMapper, String datasource, Set versionsToAwait, int initialSegmentCount, boolean doWait) { this.brokerClient = brokerClient; this.objectMapper = objectMapper; diff --git a/server/src/main/java/org/apache/druid/discovery/BrokerClient.java b/server/src/main/java/org/apache/druid/discovery/BrokerClient.java index 6636a5cfc498..a4384259898c 100644 --- a/server/src/main/java/org/apache/druid/discovery/BrokerClient.java +++ b/server/src/main/java/org/apache/druid/discovery/BrokerClient.java @@ -21,9 +21,9 @@ import com.google.common.base.Throwables; import com.google.inject.Inject; +import org.apache.druid.error.DruidException; import org.apache.druid.guice.annotations.EscalatedGlobal; import org.apache.druid.java.util.common.IOE; -import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; @@ -50,16 +50,16 @@ public class BrokerClient private final Logger log = new Logger(BrokerClient.class); private static final int MAX_RETRIES = 5; - private final HttpClient httpClient; + private final HttpClient brokerHttpClient; private final DruidNodeDiscovery druidNodeDiscovery; @Inject public BrokerClient( - @EscalatedGlobal HttpClient httpClient, + @EscalatedGlobal HttpClient brokerHttpClient, DruidNodeDiscoveryProvider druidNodeDiscoveryProvider ) { - this.httpClient = httpClient; + this.brokerHttpClient = brokerHttpClient; this.druidNodeDiscovery = druidNodeDiscoveryProvider.getForNodeRole(NodeRole.BROKER); } @@ -71,7 +71,9 @@ public Request makeRequest(HttpMethod httpMethod, String urlPath) throws IOExcep String host = ClientUtils.pickOneHost(druidNodeDiscovery); if (host == null) { - throw new IOE("No known server."); + throw DruidException.forPersona(DruidException.Persona.ADMIN) + .ofCategory(DruidException.Category.NOT_FOUND) + .build("A leader node could not be found for [%s] service. Check the logs to validate that service is healthy.", NodeRole.BROKER); } return new Request(httpMethod, new URL(StringUtils.format("%s%s", host, urlPath))); } @@ -85,7 +87,7 @@ public String sendQuery(Request request) throws Exception try { try { - fullResponseHolder = httpClient.go(request, responseHandler).get(); + fullResponseHolder = brokerHttpClient.go(request, responseHandler).get(); } catch (ExecutionException e) { // Unwrap IOExceptions and ChannelExceptions, re-throw others @@ -132,9 +134,8 @@ private Request getNewRequestUrl(Request oldRequest) } catch (MalformedURLException e) { // Not an IOException; this is our own fault. - throw new ISE( - e, - "failed to build url with path[%] and query string [%s].", + throw DruidException.defensive( + "Failed to build url with path[%] and query string [%s].", oldRequest.getUrl().getPath(), oldRequest.getUrl().getQuery() ); diff --git a/server/src/main/java/org/apache/druid/discovery/ClientUtils.java b/server/src/main/java/org/apache/druid/discovery/ClientUtils.java index cadf5bafca60..1a6fd1974368 100644 --- a/server/src/main/java/org/apache/druid/discovery/ClientUtils.java +++ b/server/src/main/java/org/apache/druid/discovery/ClientUtils.java @@ -51,7 +51,7 @@ public static Request withUrl(Request old, URL url) Request req = new Request(old.getMethod(), url); req.addHeaderValues(old.getHeaders()); if (old.hasContent()) { - req.setContent(old.getContent()); + req.setContent(old.getContent().copy()); } return req; } From cc5456b216cc699dd2f958510a3a62a91422ebb0 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Tue, 22 Aug 2023 11:46:02 +0530 Subject: [PATCH 10/19] Address review comments --- docs/api-reference/sql-ingestion-api.md | 21 ++-- .../apache/druid/msq/exec/ControllerImpl.java | 2 +- .../druid/msq/exec/SegmentLoadWaiter.java | 114 +++++++++++++----- .../druid/msq/exec/SegmentLoadWaiterTest.java | 12 +- .../indexing/report/MSQTaskReportTest.java | 13 +- .../apache/druid/msq/test/MSQTestBase.java | 2 +- .../apache/druid/discovery/ClientUtils.java | 8 +- 7 files changed, 124 insertions(+), 48 deletions(-) diff --git a/docs/api-reference/sql-ingestion-api.md b/docs/api-reference/sql-ingestion-api.md index eea63e11de9c..c41cfad1c332 100644 --- a/docs/api-reference/sql-ingestion-api.md +++ b/docs/api-reference/sql-ingestion-api.md @@ -289,13 +289,16 @@ The response shows an example report for a query. "durationMs": 28227, "pendingTasks": 0, "runningTasks": 2, - "segmentLoadWaiterStatus": { + "segmentLoadStatus": { "state": "SUCCESS", "dataSource": "kttm_simple", "startTime": "2022-09-14T23:12:09.266Z", "duration": 15, "totalSegments": 1, - "segmentsLeft": 0 + "usedSegments": 1, + "coldSegments": 0, + "pendingSegments": 0, + "unknownSegments": 0 } }, "stages": [ @@ -601,12 +604,14 @@ The following table describes the response fields when you retrieve a report for | `multiStageQuery.payload.status.durationMs` | Milliseconds elapsed after the query has started running. -1 denotes that the query hasn't started running yet. | | `multiStageQuery.payload.status.pendingTasks` | Number of tasks that are not fully started. -1 denotes that the number is currently unknown. | | `multiStageQuery.payload.status.runningTasks` | Number of currently running tasks. Should be at least 1 since the controller is included. | -| `multiStageQuery.payload.status.segmentLoadWaiterStatus` | Segment loading waiter container. Only present after the segments have been published. | -| `multiStageQuery.payload.status.segmentLoadWaiterStatus.state` | Either INIT, WAITING, SUCCESS, FAILED or TIMED_OUT. | -| `multiStageQuery.payload.status.segmentLoadWaiterStatus.startTime` | Time since which the controller has been waiting for the segments to finish loading. | -| `multiStageQuery.payload.status.segmentLoadWaiterStatus.duration` | The duration in milliseconds that the controller has been waiting for the segments to load. | -| `multiStageQuery.payload.status.segmentLoadWaiterStatus.totalSegments` | The total number of segments generated by the job. This includes tombstone segments (if any). | -| `multiStageQuery.payload.status.segmentLoadWaiterStatus.segmentsLeft` | The total number of segments remaining to be loaded. | +| `multiStageQuery.payload.status.segmentLoadStatus` | Segment loading container. Only present after the segments have been published. | +| `multiStageQuery.payload.status.segmentLoadStatus.state` | Either INIT, WAITING, SUCCESS, FAILED or TIMED_OUT. | +| `multiStageQuery.payload.status.segmentLoadStatus.duration` | The duration in milliseconds that the controller has been waiting for the segments to load. | +| `multiStageQuery.payload.status.segmentLoadStatus.totalSegments` | The total number of segments generated by the job. This includes tombstone segments (if any). | +| `multiStageQuery.payload.status.segmentLoadStatus.usedSegments` | The number of segments which are marked as used based on the load rules. Unused segments can be cleaned up at any time. | +| `multiStageQuery.payload.status.segmentLoadStatus.coldSegments` | The number of segments which are marked as cold based on the load rules. These segments will not be loaded on any historical. | +| `multiStageQuery.payload.status.segmentLoadStatus.pendingSegments` | The number of segments remaining to be loaded. | +| `multiStageQuery.payload.status.segmentLoadStatus.unknownSegments` | The number of segments whose status is unknown. | | `multiStageQuery.payload.status.errorReport` | Error object. Only present if there was an error. | | `multiStageQuery.payload.status.errorReport.taskId` | The task that reported the error, if known. May be a controller task or a worker task. | | `multiStageQuery.payload.status.errorReport.host` | The hostname and port of the task that reported the error, if known. | 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 bfb24e0562bb..f21bbf44a276 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 @@ -1337,7 +1337,7 @@ private void publishAllSegments(final Set segments) throws IOExcept context.jsonMapper(), task.getDataSource(), versionsToAwait, - segmentsWithTombstones.size(), + segments.size(), true ); // Append mode. diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java index 036f1da19149..b238c475b95f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java @@ -61,8 +61,10 @@ public class SegmentLoadWaiter private static final long INITIAL_SLEEP_DURATION_MILLIS = TimeUnit.SECONDS.toMillis(5); private static final long SLEEP_DURATION_MILLIS = TimeUnit.SECONDS.toMillis(5); private static final long TIMEOUT_DURATION_MILLIS = TimeUnit.MINUTES.toMillis(10); - private static final String LOAD_QUERY = "SELECT COUNT(*) AS totalSegments,\n" - + "COUNT(*) FILTER (WHERE is_available = 0 AND is_published = 1 AND replication_factor != 0) AS loadingSegments\n" + private static final String LOAD_QUERY = "SELECT COUNT(*) AS usedSegments,\n" + + "COUNT(*) FILTER (WHERE is_published = 1 AND replication_factor = 0) AS coldSegments,\n" + + "COUNT(*) FILTER (WHERE is_available = 0 AND is_published = 1 AND replication_factor != 0) AS pendingSegments,\n" + + "COUNT(*) FILTER (WHERE replication_factor = -1) AS unknownSegments\n" + "FROM sys.segments\n" + "WHERE datasource = '%s' AND is_overshadowed = 0 AND version = '%s'"; @@ -72,17 +74,26 @@ public class SegmentLoadWaiter private final Map versionToLoadStatusMap; private final String datasource; private final Set versionsToAwait; + private final int totalSegmentsGenerated; private final boolean doWait; private volatile SegmentLoadWaiterStatus status; - public SegmentLoadWaiter(BrokerClient brokerClient, ObjectMapper objectMapper, String datasource, Set versionsToAwait, int initialSegmentCount, boolean doWait) + public SegmentLoadWaiter( + BrokerClient brokerClient, + ObjectMapper objectMapper, + String datasource, + Set versionsToAwait, + int totalSegmentsGenerated, + boolean doWait + ) { this.brokerClient = brokerClient; this.objectMapper = objectMapper; this.datasource = datasource; this.versionsToAwait = new TreeSet<>(versionsToAwait); this.versionToLoadStatusMap = new HashMap<>(); - this.status = new SegmentLoadWaiterStatus(State.INIT, null, 0, initialSegmentCount, initialSegmentCount); + this.totalSegmentsGenerated = totalSegmentsGenerated; + this.status = new SegmentLoadWaiterStatus(State.INIT, null, 0, totalSegmentsGenerated, 0, 0, 0, totalSegmentsGenerated); this.doWait = doWait; } @@ -162,14 +173,16 @@ private void waitIfNeeded(long waitTimeMillis) throws Exception */ private void updateStatus(State state, DateTime startTime) { - int totalSegmentCount = 0, pendingSegmentCount = 0; + int pendingSegmentCount = 0, usedSegmentsCount = 0, coldSegmentCount = 0, unknownSegmentCount = 0; for (Map.Entry entry : versionToLoadStatusMap.entrySet()) { - totalSegmentCount += entry.getValue().getTotalSegments(); - pendingSegmentCount += entry.getValue().getLoadingSegments(); + usedSegmentsCount += entry.getValue().getUsedSegments(); + coldSegmentCount += entry.getValue().getColdSegments(); + unknownSegmentCount += entry.getValue().getUnknownSegments(); + pendingSegmentCount += entry.getValue().getPendingSegments(); } long runningMillis = new Interval(startTime, DateTimes.nowUtc()).toDurationMillis(); - status = new SegmentLoadWaiterStatus(state, startTime, runningMillis, totalSegmentCount, pendingSegmentCount); + status = new SegmentLoadWaiterStatus(state, startTime, runningMillis, totalSegmentsGenerated, usedSegmentsCount, coldSegmentCount, pendingSegmentCount, unknownSegmentCount); } /** @@ -188,7 +201,7 @@ private VersionLoadStatus fetchLoadStatusForVersion(String version) throws Excep 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); + return new VersionLoadStatus(0, 0, 0, 0); } else { return objectMapper.readValue(response, VersionLoadStatus.class); } @@ -208,7 +221,10 @@ public static class SegmentLoadWaiterStatus private final DateTime startTime; private final long duration; private final int totalSegments; - private final int segmentsLeft; + private final int usedSegments; + private final int coldSegments; + private final int pendingSegments; + private final int unknownSegments; @JsonCreator public SegmentLoadWaiterStatus( @@ -216,14 +232,20 @@ public SegmentLoadWaiterStatus( @JsonProperty("startTime") @Nullable DateTime startTime, @JsonProperty("duration") long duration, @JsonProperty("totalSegments") int totalSegments, - @JsonProperty("segmentsLeft") int segmentsLeft + @JsonProperty("usedSegments") int usedSegments, + @JsonProperty("coldSegments") int coldSegments, + @JsonProperty("pendingSegments") int pendingSegments, + @JsonProperty("unknownSegments") int unknownSegments ) { this.state = state; this.startTime = startTime; this.duration = duration; this.totalSegments = totalSegments; - this.segmentsLeft = segmentsLeft; + this.usedSegments = usedSegments; + this.coldSegments = coldSegments; + this.pendingSegments = pendingSegments; + this.unknownSegments = unknownSegments; } @JsonProperty @@ -253,9 +275,27 @@ public long getTotalSegments() } @JsonProperty - public int getSegmentsLeft() + public int getUsedSegments() { - return segmentsLeft; + return usedSegments; + } + + @JsonProperty + public int getColdSegments() + { + return coldSegments; + } + + @JsonProperty + public int getPendingSegments() + { + return pendingSegments; + } + + @JsonProperty + public int getUnknownSegments() + { + return unknownSegments; } } @@ -270,35 +310,53 @@ public enum State public static class VersionLoadStatus { - private final int totalSegments; - private final int loadingSegments; + private final int usedSegments; + private final int coldSegments; + private final int pendingSegments; + private final int unknownSegments; @JsonCreator public VersionLoadStatus( - @JsonProperty("totalSegments") int totalSegments, - @JsonProperty("loadingSegments") int loadingSegments + @JsonProperty("usedSegments") int usedSegments, + @JsonProperty("coldSegments") int coldSegments, + @JsonProperty("pendingSegments") int pendingSegments, + @JsonProperty("unknownSegments") int unknownSegments ) { - this.totalSegments = totalSegments; - this.loadingSegments = loadingSegments; + this.usedSegments = usedSegments; + this.coldSegments = coldSegments; + this.pendingSegments = pendingSegments; + this.unknownSegments = unknownSegments; } - @JsonProperty("totalSegments") - public int getTotalSegments() + @JsonProperty + public int getUsedSegments() { - return totalSegments; + return usedSegments; } - @JsonProperty("loadingSegments") - public int getLoadingSegments() + @JsonProperty + public int getColdSegments() { - return loadingSegments; + return coldSegments; + } + + @JsonProperty + public int getPendingSegments() + { + return pendingSegments; + } + + @JsonProperty + public int getUnknownSegments() + { + return unknownSegments; } @JsonIgnore public boolean isLoadingComplete() { - return loadingSegments == 0; - } + return pendingSegments == 0; + } // TODO: change? } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/SegmentLoadWaiterTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/SegmentLoadWaiterTest.java index 35fdba6dab93..8f6875a472e4 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/SegmentLoadWaiterTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/SegmentLoadWaiterTest.java @@ -52,14 +52,14 @@ public void testSingleVersionWaitsForLoadCorrectly() throws Exception brokerClient = mock(BrokerClient.class); doReturn(mock(Request.class)).when(brokerClient).makeRequest(any(), anyString()); - doAnswer(new Answer() + doAnswer(new Answer() { int timesInvoked = 0; @Override - public Object answer(InvocationOnMock invocation) throws Throwable + public String answer(InvocationOnMock invocation) throws Throwable { timesInvoked += 1; - SegmentLoadWaiter.VersionLoadStatus loadStatus = new SegmentLoadWaiter.VersionLoadStatus(5, 5 - timesInvoked); + SegmentLoadWaiter.VersionLoadStatus loadStatus = new SegmentLoadWaiter.VersionLoadStatus(5, 0,5 - timesInvoked, 0); return new ObjectMapper().writeValueAsString(loadStatus); } }).when(brokerClient).sendQuery(any()); @@ -75,14 +75,14 @@ public void testMultipleVersionWaitsForLoadCorrectly() throws Exception brokerClient = mock(BrokerClient.class); doReturn(mock(Request.class)).when(brokerClient).makeRequest(any(), anyString()); - doAnswer(new Answer() + doAnswer(new Answer() { int timesInvoked = 0; @Override - public Object answer(InvocationOnMock invocation) throws Throwable + public String answer(InvocationOnMock invocation) throws Throwable { timesInvoked += 1; - SegmentLoadWaiter.VersionLoadStatus loadStatus = new SegmentLoadWaiter.VersionLoadStatus(5, 5 - timesInvoked); + SegmentLoadWaiter.VersionLoadStatus loadStatus = new SegmentLoadWaiter.VersionLoadStatus(5, 0,5 - timesInvoked, 0); return new ObjectMapper().writeValueAsString(loadStatus); } }).when(brokerClient).sendQuery(any()); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java index 64951127e784..50ff023f47e4 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java @@ -96,8 +96,11 @@ public void testSerdeResultsReport() throws Exception SegmentLoadWaiter.State.WAITING, DateTimes.nowUtc(), 200L, + 100, + 80, 50, - 10 + 10, + 0 ); final MSQTaskReport report = new MSQTaskReport( @@ -156,8 +159,11 @@ public void testSerdeErrorReport() throws Exception SegmentLoadWaiter.State.FAILED, DateTimes.nowUtc(), 200L, + 100, + 80, 50, - 10 + 10, + 0 ); final MSQErrorReport errorReport = MSQErrorReport.fromFault(TASK_ID, HOST, 0, new TooManyColumnsFault(10, 5)); @@ -201,7 +207,10 @@ public void testWriteTaskReport() throws Exception SegmentLoadWaiter.State.SUCCESS, DateTimes.nowUtc(), 200L, + 100, + 80, 50, + 10, 0 ); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index fcacbdc49bc1..fe4b199c4d27 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -486,7 +486,7 @@ public String getFormatString() objectMapper.registerModules(sqlModule.getJacksonModules()); doReturn(mock(Request.class)).when(brokerClient).makeRequest(any(), anyString()); - SegmentLoadWaiter.VersionLoadStatus loadStatus = new SegmentLoadWaiter.VersionLoadStatus(1, 0); + SegmentLoadWaiter.VersionLoadStatus loadStatus = new SegmentLoadWaiter.VersionLoadStatus(1, 0, 0, 0); // TODO? doReturn(StringUtils.fromUtf8(objectMapper.writeValueAsBytes(loadStatus))).when(brokerClient).sendQuery(any()); testTaskActionClient = Mockito.spy(new MSQTestTaskActionClient(objectMapper)); diff --git a/server/src/main/java/org/apache/druid/discovery/ClientUtils.java b/server/src/main/java/org/apache/druid/discovery/ClientUtils.java index 1a6fd1974368..4115bc1a68f0 100644 --- a/server/src/main/java/org/apache/druid/discovery/ClientUtils.java +++ b/server/src/main/java/org/apache/druid/discovery/ClientUtils.java @@ -19,12 +19,15 @@ package org.apache.druid.discovery; +import com.google.common.collect.Lists; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.http.client.Request; import javax.annotation.Nullable; import java.net.URL; import java.util.Iterator; +import java.util.List; +import java.util.Random; /** * Utils class for shared client methods @@ -35,8 +38,9 @@ public class ClientUtils public static String pickOneHost(DruidNodeDiscovery druidNodeDiscovery) { Iterator iter = druidNodeDiscovery.getAllNodes().iterator(); - if (iter.hasNext()) { - DiscoveryDruidNode node = iter.next(); + List discoveryDruidNodeList = Lists.newArrayList(iter); + if (!discoveryDruidNodeList.isEmpty()) { + DiscoveryDruidNode node = discoveryDruidNodeList.get(new Random().nextInt(discoveryDruidNodeList.size())); return StringUtils.format( "%s://%s", node.getDruidNode().getServiceScheme(), From 2d9db10f1ff789b1c46a38f8bad5c0bf6bb327d6 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Tue, 22 Aug 2023 15:42:48 +0530 Subject: [PATCH 11/19] Address review comments --- docs/api-reference/sql-ingestion-api.md | 2 + .../apache/druid/msq/exec/ControllerImpl.java | 22 +++++-- .../druid/msq/exec/SegmentLoadWaiter.java | 60 ++++++++++++++----- .../druid/msq/exec/SegmentLoadWaiterTest.java | 4 +- .../indexing/report/MSQTaskReportTest.java | 3 + .../apache/druid/msq/test/MSQTestBase.java | 3 - 6 files changed, 69 insertions(+), 25 deletions(-) diff --git a/docs/api-reference/sql-ingestion-api.md b/docs/api-reference/sql-ingestion-api.md index c41cfad1c332..9caf7cf51ee3 100644 --- a/docs/api-reference/sql-ingestion-api.md +++ b/docs/api-reference/sql-ingestion-api.md @@ -296,6 +296,7 @@ The response shows an example report for a query. "duration": 15, "totalSegments": 1, "usedSegments": 1, + "hotSegments": 0, "coldSegments": 0, "pendingSegments": 0, "unknownSegments": 0 @@ -609,6 +610,7 @@ The following table describes the response fields when you retrieve a report for | `multiStageQuery.payload.status.segmentLoadStatus.duration` | The duration in milliseconds that the controller has been waiting for the segments to load. | | `multiStageQuery.payload.status.segmentLoadStatus.totalSegments` | The total number of segments generated by the job. This includes tombstone segments (if any). | | `multiStageQuery.payload.status.segmentLoadStatus.usedSegments` | The number of segments which are marked as used based on the load rules. Unused segments can be cleaned up at any time. | +| `multiStageQuery.payload.status.segmentLoadStatus.hotSegments` | The number of segments which are marked as hot based on the load rules. These segments will be loaded and served by a historical. | | `multiStageQuery.payload.status.segmentLoadStatus.coldSegments` | The number of segments which are marked as cold based on the load rules. These segments will not be loaded on any historical. | | `multiStageQuery.payload.status.segmentLoadStatus.pendingSegments` | The number of segments remaining to be loaded. | | `multiStageQuery.payload.status.segmentLoadStatus.unknownSegments` | The number of segments whose status is unknown. | 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 f21bbf44a276..3e708616e7c0 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 @@ -63,6 +63,7 @@ import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.TaskReport; import org.apache.druid.indexing.common.actions.LockListAction; +import org.apache.druid.indexing.common.actions.LockReleaseAction; import org.apache.druid.indexing.common.actions.MarkSegmentsAsUnusedAction; import org.apache.druid.indexing.common.actions.SegmentAllocateAction; import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; @@ -453,13 +454,24 @@ public TaskStatus runTask(final Closer closer) } } - cleanUpDurableStorageIfNeeded(); - if (queryKernel != null && queryKernel.isSuccess()) { - if (segmentLoadWaiter != null) { - // If successful and there are segments created, segmentLoadWaiter should wait for them to become available. - segmentLoadWaiter.waitForSegmentsToLoad(); + try { + final List locks = context.taskActionClient().submit(new LockListAction()); + for (final TaskLock lock : locks) { + context.taskActionClient().submit(new LockReleaseAction(lock.getInterval())); } + + cleanUpDurableStorageIfNeeded(); + + if (queryKernel != null && queryKernel.isSuccess()) { + if (segmentLoadWaiter != null) { + // If successful and there are segments created, segmentLoadWaiter should wait for them to become available. + segmentLoadWaiter.waitForSegmentsToLoad(); + } + } + } + catch (Exception ignored) { + // Ignore and just write the task report. } try { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java index b238c475b95f..73184df42a28 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java @@ -43,6 +43,7 @@ import java.util.Set; import java.util.TreeSet; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; /** * Class that periodically checks with the broker if all the segments generated are loaded by querying the sys table @@ -58,10 +59,10 @@ public class SegmentLoadWaiter { private static final Logger log = new Logger(SegmentLoadWaiter.class); - private static final long INITIAL_SLEEP_DURATION_MILLIS = TimeUnit.SECONDS.toMillis(5); private static final long SLEEP_DURATION_MILLIS = TimeUnit.SECONDS.toMillis(5); private static final long TIMEOUT_DURATION_MILLIS = TimeUnit.MINUTES.toMillis(10); private static final String LOAD_QUERY = "SELECT COUNT(*) AS usedSegments,\n" + + "COUNT(*) FILTER (WHERE is_published = 1 AND replication_factor > 0) AS hotSegments,\n" + "COUNT(*) FILTER (WHERE is_published = 1 AND replication_factor = 0) AS coldSegments,\n" + "COUNT(*) FILTER (WHERE is_available = 0 AND is_published = 1 AND replication_factor != 0) AS pendingSegments,\n" + "COUNT(*) FILTER (WHERE replication_factor = -1) AS unknownSegments\n" @@ -76,7 +77,7 @@ public class SegmentLoadWaiter private final Set versionsToAwait; private final int totalSegmentsGenerated; private final boolean doWait; - private volatile SegmentLoadWaiterStatus status; + private final AtomicReference status; public SegmentLoadWaiter( BrokerClient brokerClient, @@ -93,7 +94,7 @@ public SegmentLoadWaiter( this.versionsToAwait = new TreeSet<>(versionsToAwait); this.versionToLoadStatusMap = new HashMap<>(); this.totalSegmentsGenerated = totalSegmentsGenerated; - this.status = new SegmentLoadWaiterStatus(State.INIT, null, 0, totalSegmentsGenerated, 0, 0, 0, totalSegmentsGenerated); + this.status = new AtomicReference<>(new SegmentLoadWaiterStatus(State.INIT, null, 0, totalSegmentsGenerated, 0, 0, 0, 0, totalSegmentsGenerated)); this.doWait = doWait; } @@ -109,13 +110,9 @@ public SegmentLoadWaiter( public void waitForSegmentsToLoad() { DateTime startTime = DateTimes.nowUtc(); + boolean hasAnySegmentBeenLoaded = false; try { - // Sleep for a short duration to allow the segments that were just created to reflect in broker queries. - // This avoids a race condition where the broker returns an empty response as it is not yet aware of the new segments - // yet. This cannot be differentiated from the case where the segments have been dropped due to load rules already. - waitIfNeeded(INITIAL_SLEEP_DURATION_MILLIS); - while (!versionsToAwait.isEmpty()) { // Check the timeout and exit if exceeded. long runningMillis = new Interval(startTime, DateTimes.nowUtc()).toDurationMillis(); @@ -135,8 +132,10 @@ public void waitForSegmentsToLoad() VersionLoadStatus loadStatus = fetchLoadStatusForVersion(version); versionToLoadStatusMap.put(version, loadStatus); + hasAnySegmentBeenLoaded = hasAnySegmentBeenLoaded || loadStatus.getUsedSegments() > 0; + // If loading is done for this stage, remove it from future loops. - if (loadStatus.isLoadingComplete()) { + if (hasAnySegmentBeenLoaded && loadStatus.isLoadingComplete()) { iterator.remove(); } } @@ -173,16 +172,29 @@ private void waitIfNeeded(long waitTimeMillis) throws Exception */ private void updateStatus(State state, DateTime startTime) { - int pendingSegmentCount = 0, usedSegmentsCount = 0, coldSegmentCount = 0, unknownSegmentCount = 0; + int pendingSegmentCount = 0, usedSegmentsCount = 0, hotSegmentCount = 0, coldSegmentCount = 0, unknownSegmentCount = 0; for (Map.Entry entry : versionToLoadStatusMap.entrySet()) { usedSegmentsCount += entry.getValue().getUsedSegments(); + hotSegmentCount += entry.getValue().getHotSegments(); coldSegmentCount += entry.getValue().getColdSegments(); unknownSegmentCount += entry.getValue().getUnknownSegments(); pendingSegmentCount += entry.getValue().getPendingSegments(); } long runningMillis = new Interval(startTime, DateTimes.nowUtc()).toDurationMillis(); - status = new SegmentLoadWaiterStatus(state, startTime, runningMillis, totalSegmentsGenerated, usedSegmentsCount, coldSegmentCount, pendingSegmentCount, unknownSegmentCount); + status.set( + new SegmentLoadWaiterStatus( + state, + startTime, + runningMillis, + totalSegmentsGenerated, + usedSegmentsCount, + hotSegmentCount, + coldSegmentCount, + pendingSegmentCount, + unknownSegmentCount + ) + ); } /** @@ -201,7 +213,7 @@ private VersionLoadStatus fetchLoadStatusForVersion(String version) throws Excep 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); + return new VersionLoadStatus(0, 0, 0, 0, 0); } else { return objectMapper.readValue(response, VersionLoadStatus.class); } @@ -212,7 +224,7 @@ private VersionLoadStatus fetchLoadStatusForVersion(String version) throws Excep */ public SegmentLoadWaiterStatus status() { - return status; + return status.get(); } public static class SegmentLoadWaiterStatus @@ -222,6 +234,7 @@ public static class SegmentLoadWaiterStatus private final long duration; private final int totalSegments; private final int usedSegments; + private final int hotSegments; private final int coldSegments; private final int pendingSegments; private final int unknownSegments; @@ -233,6 +246,7 @@ public SegmentLoadWaiterStatus( @JsonProperty("duration") long duration, @JsonProperty("totalSegments") int totalSegments, @JsonProperty("usedSegments") int usedSegments, + @JsonProperty("hotSegments") int hotSegments, @JsonProperty("coldSegments") int coldSegments, @JsonProperty("pendingSegments") int pendingSegments, @JsonProperty("unknownSegments") int unknownSegments @@ -243,6 +257,7 @@ public SegmentLoadWaiterStatus( this.duration = duration; this.totalSegments = totalSegments; this.usedSegments = usedSegments; + this.hotSegments = hotSegments; this.coldSegments = coldSegments; this.pendingSegments = pendingSegments; this.unknownSegments = unknownSegments; @@ -280,6 +295,12 @@ public int getUsedSegments() return usedSegments; } + @JsonProperty + public int getHotSegments() + { + return hotSegments; + } + @JsonProperty public int getColdSegments() { @@ -311,6 +332,7 @@ public enum State public static class VersionLoadStatus { private final int usedSegments; + private final int hotSegments; private final int coldSegments; private final int pendingSegments; private final int unknownSegments; @@ -318,12 +340,14 @@ public static class VersionLoadStatus @JsonCreator public VersionLoadStatus( @JsonProperty("usedSegments") int usedSegments, + @JsonProperty("hotSegments") int hotSegments, @JsonProperty("coldSegments") int coldSegments, @JsonProperty("pendingSegments") int pendingSegments, @JsonProperty("unknownSegments") int unknownSegments ) { this.usedSegments = usedSegments; + this.hotSegments = hotSegments; this.coldSegments = coldSegments; this.pendingSegments = pendingSegments; this.unknownSegments = unknownSegments; @@ -335,6 +359,12 @@ public int getUsedSegments() return usedSegments; } + @JsonProperty + public int getHotSegments() + { + return hotSegments; + } + @JsonProperty public int getColdSegments() { @@ -356,7 +386,7 @@ public int getUnknownSegments() @JsonIgnore public boolean isLoadingComplete() { - return pendingSegments == 0; - } // TODO: change? + return pendingSegments == 0 && (usedSegments == hotSegments + coldSegments); + } } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/SegmentLoadWaiterTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/SegmentLoadWaiterTest.java index 8f6875a472e4..e14fa5faec26 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/SegmentLoadWaiterTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/SegmentLoadWaiterTest.java @@ -59,7 +59,7 @@ public void testSingleVersionWaitsForLoadCorrectly() throws Exception public String answer(InvocationOnMock invocation) throws Throwable { timesInvoked += 1; - SegmentLoadWaiter.VersionLoadStatus loadStatus = new SegmentLoadWaiter.VersionLoadStatus(5, 0,5 - timesInvoked, 0); + SegmentLoadWaiter.VersionLoadStatus loadStatus = new SegmentLoadWaiter.VersionLoadStatus(5, timesInvoked, 0, 5 - timesInvoked, 0); return new ObjectMapper().writeValueAsString(loadStatus); } }).when(brokerClient).sendQuery(any()); @@ -82,7 +82,7 @@ public void testMultipleVersionWaitsForLoadCorrectly() throws Exception public String answer(InvocationOnMock invocation) throws Throwable { timesInvoked += 1; - SegmentLoadWaiter.VersionLoadStatus loadStatus = new SegmentLoadWaiter.VersionLoadStatus(5, 0,5 - timesInvoked, 0); + SegmentLoadWaiter.VersionLoadStatus loadStatus = new SegmentLoadWaiter.VersionLoadStatus(5, timesInvoked, 0, 5 - timesInvoked, 0); return new ObjectMapper().writeValueAsString(loadStatus); } }).when(brokerClient).sendQuery(any()); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java index 50ff023f47e4..ef50008d48e2 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java @@ -98,6 +98,7 @@ public void testSerdeResultsReport() throws Exception 200L, 100, 80, + 30, 50, 10, 0 @@ -161,6 +162,7 @@ public void testSerdeErrorReport() throws Exception 200L, 100, 80, + 30, 50, 10, 0 @@ -209,6 +211,7 @@ public void testWriteTaskReport() throws Exception 200L, 100, 80, + 30, 50, 10, 0 diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index fe4b199c4d27..9acae11ab8c3 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -83,7 +83,6 @@ import org.apache.druid.msq.counters.QueryCounterSnapshot; import org.apache.druid.msq.exec.ClusterStatisticsMergeMode; import org.apache.druid.msq.exec.Controller; -import org.apache.druid.msq.exec.SegmentLoadWaiter; import org.apache.druid.msq.exec.WorkerMemoryParameters; import org.apache.druid.msq.guice.MSQDurableStorageModule; import org.apache.druid.msq.guice.MSQExternalDataSourceModule; @@ -486,8 +485,6 @@ public String getFormatString() objectMapper.registerModules(sqlModule.getJacksonModules()); doReturn(mock(Request.class)).when(brokerClient).makeRequest(any(), anyString()); - SegmentLoadWaiter.VersionLoadStatus loadStatus = new SegmentLoadWaiter.VersionLoadStatus(1, 0, 0, 0); // TODO? - doReturn(StringUtils.fromUtf8(objectMapper.writeValueAsBytes(loadStatus))).when(brokerClient).sendQuery(any()); testTaskActionClient = Mockito.spy(new MSQTestTaskActionClient(objectMapper)); indexingServiceClient = new MSQTestOverlordServiceClient( From 2401c0c393a67b3723032bcc10cbf62ff76d8378 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Tue, 22 Aug 2023 16:50:17 +0530 Subject: [PATCH 12/19] Address review comments --- .../apache/druid/msq/exec/ControllerImpl.java | 4 +- .../apache/druid/discovery/BrokerClient.java | 69 +++++++------------ .../apache/druid/discovery/ClientUtils.java | 4 +- .../druid/discovery/BrokerClientTest.java | 2 +- 4 files changed, 28 insertions(+), 51 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 3e708616e7c0..e9af520b2da5 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 @@ -470,8 +470,8 @@ public TaskStatus runTask(final Closer closer) } } } - catch (Exception ignored) { - // Ignore and just write the task report. + catch (Exception e) { + log.warn(e, "Exception thrown during cleanup. Ignoring it and writing task report."); } try { diff --git a/server/src/main/java/org/apache/druid/discovery/BrokerClient.java b/server/src/main/java/org/apache/druid/discovery/BrokerClient.java index a4384259898c..a83ca98103ce 100644 --- a/server/src/main/java/org/apache/druid/discovery/BrokerClient.java +++ b/server/src/main/java/org/apache/druid/discovery/BrokerClient.java @@ -19,14 +19,12 @@ package org.apache.druid.discovery; -import com.google.common.base.Throwables; import com.google.inject.Inject; import org.apache.druid.error.DruidException; import org.apache.druid.guice.annotations.EscalatedGlobal; import org.apache.druid.java.util.common.IOE; -import org.apache.druid.java.util.common.RE; +import org.apache.druid.java.util.common.RetryUtils; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.http.client.HttpClient; import org.apache.druid.java.util.http.client.Request; import org.apache.druid.java.util.http.client.response.StringFullResponseHandler; @@ -47,7 +45,6 @@ */ public class BrokerClient { - private final Logger log = new Logger(BrokerClient.class); private static final int MAX_RETRIES = 5; private final HttpClient brokerHttpClient; @@ -78,50 +75,30 @@ public Request makeRequest(HttpMethod httpMethod, String urlPath) throws IOExcep return new Request(httpMethod, new URL(StringUtils.format("%s%s", host, urlPath))); } - public String sendQuery(Request request) throws Exception + public String sendQuery(final Request request) throws Exception { - StringFullResponseHandler responseHandler = new StringFullResponseHandler(StandardCharsets.UTF_8); + return RetryUtils.retry( + () -> { + Request newRequestUrl = getNewRequestUrl(request); + final StringFullResponseHolder fullResponseHolder = brokerHttpClient.go(newRequestUrl, new StringFullResponseHandler(StandardCharsets.UTF_8)).get(); - for (int counter = 0; counter < MAX_RETRIES; counter++) { - final StringFullResponseHolder fullResponseHolder; - - try { - try { - fullResponseHolder = brokerHttpClient.go(request, responseHandler).get(); - } - catch (ExecutionException e) { - // Unwrap IOExceptions and ChannelExceptions, re-throw others - Throwables.propagateIfInstanceOf(e.getCause(), IOException.class); - Throwables.propagateIfInstanceOf(e.getCause(), ChannelException.class); - throw new RE(e, "HTTP request to [%s] failed", request.getUrl()); - } - } - catch (IOException | ChannelException ex) { - // can happen if the node is stopped. - log.warn(ex, "Request [%s] failed.", request.getUrl()); - request = getNewRequestUrl(request); - continue; - } - - HttpResponseStatus responseStatus = fullResponseHolder.getResponse().getStatus(); - if (HttpResponseStatus.SERVICE_UNAVAILABLE.equals(responseStatus) - || HttpResponseStatus.GATEWAY_TIMEOUT.equals(responseStatus)) { - log.warn( - "Request [%s] received a [%s] response. Attempt [%s]/[%s]", - request.getUrl(), - responseStatus, - counter + 1, - MAX_RETRIES - ); - request = getNewRequestUrl(request); - } else if (responseStatus.getCode() != HttpServletResponse.SC_OK) { - log.warn("Request [%s] failed with error code [%s]", request.getUrl(), responseStatus.getCode()); - } else { - return fullResponseHolder.getContent(); - } - } - - throw new IOE("Retries exhausted, couldn't fulfill request to [%s].", request.getUrl()); + HttpResponseStatus responseStatus = fullResponseHolder.getResponse().getStatus(); + if (HttpResponseStatus.SERVICE_UNAVAILABLE.equals(responseStatus) + || HttpResponseStatus.GATEWAY_TIMEOUT.equals(responseStatus)) { + throw new IOE(StringUtils.format("Request to broker failed due to failed response status: [%s]", responseStatus)); + } else if (responseStatus.getCode() != HttpServletResponse.SC_OK) { + throw new IOE(StringUtils.format("Request to broker failed due to failed response code: [%s]", responseStatus.getCode())); + } + return fullResponseHolder.getContent(); + }, + (throwable) -> { + if (throwable instanceof ExecutionException) { + return throwable.getCause() instanceof IOException || throwable.getCause() instanceof ChannelException; + } + return throwable instanceof IOE; + }, + MAX_RETRIES + ); } private Request getNewRequestUrl(Request oldRequest) diff --git a/server/src/main/java/org/apache/druid/discovery/ClientUtils.java b/server/src/main/java/org/apache/druid/discovery/ClientUtils.java index 4115bc1a68f0..b9c53343c0d7 100644 --- a/server/src/main/java/org/apache/druid/discovery/ClientUtils.java +++ b/server/src/main/java/org/apache/druid/discovery/ClientUtils.java @@ -27,7 +27,7 @@ import java.net.URL; import java.util.Iterator; import java.util.List; -import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; /** * Utils class for shared client methods @@ -40,7 +40,7 @@ public static String pickOneHost(DruidNodeDiscovery druidNodeDiscovery) Iterator iter = druidNodeDiscovery.getAllNodes().iterator(); List discoveryDruidNodeList = Lists.newArrayList(iter); if (!discoveryDruidNodeList.isEmpty()) { - DiscoveryDruidNode node = discoveryDruidNodeList.get(new Random().nextInt(discoveryDruidNodeList.size())); + DiscoveryDruidNode node = discoveryDruidNodeList.get(ThreadLocalRandom.current().nextInt(discoveryDruidNodeList.size())); return StringUtils.format( "%s://%s", node.getDruidNode().getServiceScheme(), diff --git a/server/src/test/java/org/apache/druid/discovery/BrokerClientTest.java b/server/src/test/java/org/apache/druid/discovery/BrokerClientTest.java index f52110ab9822..4d0e675d520d 100644 --- a/server/src/test/java/org/apache/druid/discovery/BrokerClientTest.java +++ b/server/src/test/java/org/apache/druid/discovery/BrokerClientTest.java @@ -78,7 +78,7 @@ protected Injector setupInjector() public void testSimple() throws Exception { DruidNodeDiscovery druidNodeDiscovery = EasyMock.createMock(DruidNodeDiscovery.class); - EasyMock.expect(druidNodeDiscovery.getAllNodes()).andReturn(ImmutableList.of(discoveryDruidNode)); + EasyMock.expect(druidNodeDiscovery.getAllNodes()).andReturn(ImmutableList.of(discoveryDruidNode)).anyTimes(); DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class); EasyMock.expect(druidNodeDiscoveryProvider.getForNodeRole(NodeRole.BROKER)).andReturn(druidNodeDiscovery); From 23c2f89a6718c5ea3ed89ef718f18873bd86451d Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Tue, 22 Aug 2023 17:09:30 +0530 Subject: [PATCH 13/19] Update names --- docs/api-reference/sql-ingestion-api.md | 8 +-- .../druid/msq/exec/SegmentLoadWaiter.java | 56 +++++++++---------- 2 files changed, 32 insertions(+), 32 deletions(-) diff --git a/docs/api-reference/sql-ingestion-api.md b/docs/api-reference/sql-ingestion-api.md index 9caf7cf51ee3..d2def6e64d05 100644 --- a/docs/api-reference/sql-ingestion-api.md +++ b/docs/api-reference/sql-ingestion-api.md @@ -296,8 +296,8 @@ The response shows an example report for a query. "duration": 15, "totalSegments": 1, "usedSegments": 1, - "hotSegments": 0, - "coldSegments": 0, + "precachedSegments": 0, + "asyncOnlySegments": 0, "pendingSegments": 0, "unknownSegments": 0 } @@ -610,8 +610,8 @@ The following table describes the response fields when you retrieve a report for | `multiStageQuery.payload.status.segmentLoadStatus.duration` | The duration in milliseconds that the controller has been waiting for the segments to load. | | `multiStageQuery.payload.status.segmentLoadStatus.totalSegments` | The total number of segments generated by the job. This includes tombstone segments (if any). | | `multiStageQuery.payload.status.segmentLoadStatus.usedSegments` | The number of segments which are marked as used based on the load rules. Unused segments can be cleaned up at any time. | -| `multiStageQuery.payload.status.segmentLoadStatus.hotSegments` | The number of segments which are marked as hot based on the load rules. These segments will be loaded and served by a historical. | -| `multiStageQuery.payload.status.segmentLoadStatus.coldSegments` | The number of segments which are marked as cold based on the load rules. These segments will not be loaded on any historical. | +| `multiStageQuery.payload.status.segmentLoadStatus.precachedSegments` | The number of segments which are marked as precached and served by historicals, as per the load rules. | +| `multiStageQuery.payload.status.segmentLoadStatus.asyncOnlySegments` | The number of segments which are not loaded on any historical, as per the load rules. | | `multiStageQuery.payload.status.segmentLoadStatus.pendingSegments` | The number of segments remaining to be loaded. | | `multiStageQuery.payload.status.segmentLoadStatus.unknownSegments` | The number of segments whose status is unknown. | | `multiStageQuery.payload.status.errorReport` | Error object. Only present if there was an error. | diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java index 73184df42a28..a670dc2f4bfb 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java @@ -62,8 +62,8 @@ public class SegmentLoadWaiter private static final long SLEEP_DURATION_MILLIS = TimeUnit.SECONDS.toMillis(5); private static final long TIMEOUT_DURATION_MILLIS = TimeUnit.MINUTES.toMillis(10); private static final String LOAD_QUERY = "SELECT COUNT(*) AS usedSegments,\n" - + "COUNT(*) FILTER (WHERE is_published = 1 AND replication_factor > 0) AS hotSegments,\n" - + "COUNT(*) FILTER (WHERE is_published = 1 AND replication_factor = 0) AS coldSegments,\n" + + "COUNT(*) FILTER (WHERE is_published = 1 AND replication_factor > 0) AS precachedSegments,\n" + + "COUNT(*) FILTER (WHERE is_published = 1 AND replication_factor = 0) AS asyncOnlySegments,\n" + "COUNT(*) FILTER (WHERE is_available = 0 AND is_published = 1 AND replication_factor != 0) AS pendingSegments,\n" + "COUNT(*) FILTER (WHERE replication_factor = -1) AS unknownSegments\n" + "FROM sys.segments\n" @@ -172,11 +172,11 @@ private void waitIfNeeded(long waitTimeMillis) throws Exception */ private void updateStatus(State state, DateTime startTime) { - int pendingSegmentCount = 0, usedSegmentsCount = 0, hotSegmentCount = 0, coldSegmentCount = 0, unknownSegmentCount = 0; + int pendingSegmentCount = 0, usedSegmentsCount = 0, precachedSegmentCount = 0, asyncOnlySegmentCount = 0, unknownSegmentCount = 0; for (Map.Entry entry : versionToLoadStatusMap.entrySet()) { usedSegmentsCount += entry.getValue().getUsedSegments(); - hotSegmentCount += entry.getValue().getHotSegments(); - coldSegmentCount += entry.getValue().getColdSegments(); + precachedSegmentCount += entry.getValue().getPrecachedSegments(); + asyncOnlySegmentCount += entry.getValue().getAsyncOnlySegments(); unknownSegmentCount += entry.getValue().getUnknownSegments(); pendingSegmentCount += entry.getValue().getPendingSegments(); } @@ -189,8 +189,8 @@ private void updateStatus(State state, DateTime startTime) runningMillis, totalSegmentsGenerated, usedSegmentsCount, - hotSegmentCount, - coldSegmentCount, + precachedSegmentCount, + asyncOnlySegmentCount, pendingSegmentCount, unknownSegmentCount ) @@ -234,8 +234,8 @@ public static class SegmentLoadWaiterStatus private final long duration; private final int totalSegments; private final int usedSegments; - private final int hotSegments; - private final int coldSegments; + private final int precachedSegments; + private final int asyncOnlySegments; private final int pendingSegments; private final int unknownSegments; @@ -246,8 +246,8 @@ public SegmentLoadWaiterStatus( @JsonProperty("duration") long duration, @JsonProperty("totalSegments") int totalSegments, @JsonProperty("usedSegments") int usedSegments, - @JsonProperty("hotSegments") int hotSegments, - @JsonProperty("coldSegments") int coldSegments, + @JsonProperty("precachedSegments") int precachedSegments, + @JsonProperty("asyncOnlySegments") int asyncOnlySegments, @JsonProperty("pendingSegments") int pendingSegments, @JsonProperty("unknownSegments") int unknownSegments ) @@ -257,8 +257,8 @@ public SegmentLoadWaiterStatus( this.duration = duration; this.totalSegments = totalSegments; this.usedSegments = usedSegments; - this.hotSegments = hotSegments; - this.coldSegments = coldSegments; + this.precachedSegments = precachedSegments; + this.asyncOnlySegments = asyncOnlySegments; this.pendingSegments = pendingSegments; this.unknownSegments = unknownSegments; } @@ -296,15 +296,15 @@ public int getUsedSegments() } @JsonProperty - public int getHotSegments() + public int getPrecachedSegments() { - return hotSegments; + return precachedSegments; } @JsonProperty - public int getColdSegments() + public int getAsyncOnlySegments() { - return coldSegments; + return asyncOnlySegments; } @JsonProperty @@ -332,23 +332,23 @@ public enum State public static class VersionLoadStatus { private final int usedSegments; - private final int hotSegments; - private final int coldSegments; + private final int precachedSegments; + private final int asyncOnlySegments; private final int pendingSegments; private final int unknownSegments; @JsonCreator public VersionLoadStatus( @JsonProperty("usedSegments") int usedSegments, - @JsonProperty("hotSegments") int hotSegments, - @JsonProperty("coldSegments") int coldSegments, + @JsonProperty("precachedSegments") int precachedSegments, + @JsonProperty("asyncOnlySegments") int asyncOnlySegments, @JsonProperty("pendingSegments") int pendingSegments, @JsonProperty("unknownSegments") int unknownSegments ) { this.usedSegments = usedSegments; - this.hotSegments = hotSegments; - this.coldSegments = coldSegments; + this.precachedSegments = precachedSegments; + this.asyncOnlySegments = asyncOnlySegments; this.pendingSegments = pendingSegments; this.unknownSegments = unknownSegments; } @@ -360,15 +360,15 @@ public int getUsedSegments() } @JsonProperty - public int getHotSegments() + public int getPrecachedSegments() { - return hotSegments; + return precachedSegments; } @JsonProperty - public int getColdSegments() + public int getAsyncOnlySegments() { - return coldSegments; + return asyncOnlySegments; } @JsonProperty @@ -386,7 +386,7 @@ public int getUnknownSegments() @JsonIgnore public boolean isLoadingComplete() { - return pendingSegments == 0 && (usedSegments == hotSegments + coldSegments); + return pendingSegments == 0 && (usedSegments == precachedSegments + asyncOnlySegments); } } } From c3234745a14faab28d5ad424d69440f8c8cba818 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Tue, 22 Aug 2023 17:30:12 +0530 Subject: [PATCH 14/19] Update names --- docs/api-reference/sql-ingestion-api.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/api-reference/sql-ingestion-api.md b/docs/api-reference/sql-ingestion-api.md index d2def6e64d05..6ee5fb0d6f0b 100644 --- a/docs/api-reference/sql-ingestion-api.md +++ b/docs/api-reference/sql-ingestion-api.md @@ -607,6 +607,7 @@ The following table describes the response fields when you retrieve a report for | `multiStageQuery.payload.status.runningTasks` | Number of currently running tasks. Should be at least 1 since the controller is included. | | `multiStageQuery.payload.status.segmentLoadStatus` | Segment loading container. Only present after the segments have been published. | | `multiStageQuery.payload.status.segmentLoadStatus.state` | Either INIT, WAITING, SUCCESS, FAILED or TIMED_OUT. | +| `multiStageQuery.payload.status.segmentLoadStatus.startTime` | Time since which the controller has been waiting for the segments to finish loading. | | `multiStageQuery.payload.status.segmentLoadStatus.duration` | The duration in milliseconds that the controller has been waiting for the segments to load. | | `multiStageQuery.payload.status.segmentLoadStatus.totalSegments` | The total number of segments generated by the job. This includes tombstone segments (if any). | | `multiStageQuery.payload.status.segmentLoadStatus.usedSegments` | The number of segments which are marked as used based on the load rules. Unused segments can be cleaned up at any time. | From 6b990730dd973ee211e754f1bbe2161be9797e0a Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Tue, 22 Aug 2023 22:21:05 +0530 Subject: [PATCH 15/19] Increase coverage --- .../druid/discovery/BrokerClientTest.java | 59 ++++++++++++++++++- 1 file changed, 58 insertions(+), 1 deletion(-) diff --git a/server/src/test/java/org/apache/druid/discovery/BrokerClientTest.java b/server/src/test/java/org/apache/druid/discovery/BrokerClientTest.java index 4d0e675d520d..333882a43051 100644 --- a/server/src/test/java/org/apache/druid/discovery/BrokerClientTest.java +++ b/server/src/test/java/org/apache/druid/discovery/BrokerClientTest.java @@ -42,6 +42,11 @@ import org.junit.Assert; import org.junit.Test; +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; import java.nio.charset.StandardCharsets; public class BrokerClientTest extends BaseJettyTest @@ -65,7 +70,7 @@ protected Injector setupInjector() ); binder.bind(Integer.class).annotatedWith(Names.named("port")).toInstance(node.getPlaintextPort()); binder.bind(JettyServerInitializer.class).to(DruidLeaderClientTest.TestJettyServerInitializer.class).in(LazySingleton.class); - Jerseys.addResource(binder, DruidLeaderClientTest.SimpleResource.class); + Jerseys.addResource(binder, SimpleResource.class); LifecycleModule.register(binder, Server.class); } ) @@ -94,4 +99,56 @@ public void testSimple() throws Exception request.setContent("hello".getBytes(StandardCharsets.UTF_8)); Assert.assertEquals("hello", brokerClient.sendQuery(request)); } + + @Test + public void testError() throws Exception + { + DruidNodeDiscovery druidNodeDiscovery = EasyMock.createMock(DruidNodeDiscovery.class); + EasyMock.expect(druidNodeDiscovery.getAllNodes()).andReturn(ImmutableList.of(discoveryDruidNode)).anyTimes(); + + DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class); + EasyMock.expect(druidNodeDiscoveryProvider.getForNodeRole(NodeRole.BROKER)).andReturn(druidNodeDiscovery); + + EasyMock.replay(druidNodeDiscovery, druidNodeDiscoveryProvider); + + BrokerClient brokerClient = new BrokerClient( + httpClient, + druidNodeDiscoveryProvider + ); + + Request request = brokerClient.makeRequest(HttpMethod.POST, "/simple/flakey"); + request.setContent("hello".getBytes(StandardCharsets.UTF_8)); + Assert.assertEquals("hello", brokerClient.sendQuery(request)); + } + + @Path("/simple") + public static class SimpleResource + { + private static int attempt = 0; + + @POST + @Path("/direct") + @Produces(MediaType.APPLICATION_JSON) + public Response direct(String input) + { + if ("hello".equals(input)) { + return Response.ok("hello").build(); + } else { + return Response.serverError().build(); + } + } + + @POST + @Path("/flakey") + @Produces(MediaType.APPLICATION_JSON) + public Response redirecting() + { + if (attempt > 2) { + return Response.ok("hello").build(); + } else { + attempt += 1; + return Response.status(504).build(); + } + } + } } From 7675c8cfb2384d1c78b78c14c6f3f82c010ee20b Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Tue, 22 Aug 2023 22:59:34 +0530 Subject: [PATCH 16/19] Fix spelling --- website/.spelling | 1 + 1 file changed, 1 insertion(+) diff --git a/website/.spelling b/website/.spelling index c802a4647322..f23e87486213 100644 --- a/website/.spelling +++ b/website/.spelling @@ -440,6 +440,7 @@ preemptible prefetch prefetched prefetching +precached prepend prepended prepending From f709e71f0b2e0741c2066fb0130cc72753d5911d Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Tue, 5 Sep 2023 10:08:51 +0530 Subject: [PATCH 17/19] Add java docs --- docs/api-reference/sql-ingestion-api.md | 4 +- .../apache/druid/msq/exec/ControllerImpl.java | 22 +++++-- .../druid/msq/exec/SegmentLoadWaiter.java | 60 ++++++++++++++----- 3 files changed, 65 insertions(+), 21 deletions(-) diff --git a/docs/api-reference/sql-ingestion-api.md b/docs/api-reference/sql-ingestion-api.md index 6ee5fb0d6f0b..3daadfa5085f 100644 --- a/docs/api-reference/sql-ingestion-api.md +++ b/docs/api-reference/sql-ingestion-api.md @@ -297,7 +297,7 @@ The response shows an example report for a query. "totalSegments": 1, "usedSegments": 1, "precachedSegments": 0, - "asyncOnlySegments": 0, + "onDemandSegments": 0, "pendingSegments": 0, "unknownSegments": 0 } @@ -612,7 +612,7 @@ The following table describes the response fields when you retrieve a report for | `multiStageQuery.payload.status.segmentLoadStatus.totalSegments` | The total number of segments generated by the job. This includes tombstone segments (if any). | | `multiStageQuery.payload.status.segmentLoadStatus.usedSegments` | The number of segments which are marked as used based on the load rules. Unused segments can be cleaned up at any time. | | `multiStageQuery.payload.status.segmentLoadStatus.precachedSegments` | The number of segments which are marked as precached and served by historicals, as per the load rules. | -| `multiStageQuery.payload.status.segmentLoadStatus.asyncOnlySegments` | The number of segments which are not loaded on any historical, as per the load rules. | +| `multiStageQuery.payload.status.segmentLoadStatus.onDemandSegments` | The number of segments which are not loaded on any historical, as per the load rules. | | `multiStageQuery.payload.status.segmentLoadStatus.pendingSegments` | The number of segments remaining to be loaded. | | `multiStageQuery.payload.status.segmentLoadStatus.unknownSegments` | The number of segments whose status is unknown. | | `multiStageQuery.payload.status.errorReport` | Error object. Only present if there was an error. | 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 e9af520b2da5..feef36474a71 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 @@ -456,10 +456,7 @@ public TaskStatus runTask(final Closer closer) try { - final List locks = context.taskActionClient().submit(new LockListAction()); - for (final TaskLock lock : locks) { - context.taskActionClient().submit(new LockReleaseAction(lock.getInterval())); - } + releaseTaskLocks(); cleanUpDurableStorageIfNeeded(); @@ -550,6 +547,23 @@ public TaskStatus runTask(final Closer closer) } } + /** + * Releases the locks obtained by the task. + */ + private void releaseTaskLocks() throws IOException + { + final List locks; + try { + locks = context.taskActionClient().submit(new LockListAction()); + for (final TaskLock lock : locks) { + context.taskActionClient().submit(new LockReleaseAction(lock.getInterval())); + } + } + catch (IOException e) { + throw new IOException("Failed to release locks", e); + } + } + /** * Adds some logic to {@link #kernelManipulationQueue}, where it will, in due time, be executed by the main * controller loop in {@link RunQueryUntilDone#run()}. diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java index a670dc2f4bfb..d572f0991886 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java @@ -61,9 +61,22 @@ public class SegmentLoadWaiter private static final Logger log = new Logger(SegmentLoadWaiter.class); private static final long SLEEP_DURATION_MILLIS = TimeUnit.SECONDS.toMillis(5); private static final long TIMEOUT_DURATION_MILLIS = TimeUnit.MINUTES.toMillis(10); + + /** + * The query sent to the broker. This query uses replication_factor to determine how many copies of a segment has to be + * loaded as per the load rules. + * - If a segment is not used, the broker will not have any information about it, hence, a COUNT(*) should return this count. + * - If replication_factor is more than 0, the segment will be loaded on historicals and needs to be waited for. + * - If replication_factor is 0, that means that the segment will never be loaded on a historical and does not need to + * be waited for. + * - If replication_factor is -1, the replication factor is not known currently and will become known after a load rule + * evaluation. + *
+ * See https://github.com/apache/druid/pull/14403 for more details about replication_factor + */ private static final String LOAD_QUERY = "SELECT COUNT(*) AS usedSegments,\n" + "COUNT(*) FILTER (WHERE is_published = 1 AND replication_factor > 0) AS precachedSegments,\n" - + "COUNT(*) FILTER (WHERE is_published = 1 AND replication_factor = 0) AS asyncOnlySegments,\n" + + "COUNT(*) FILTER (WHERE is_published = 1 AND replication_factor = 0) AS onDemandSegments,\n" + "COUNT(*) FILTER (WHERE is_available = 0 AND is_published = 1 AND replication_factor != 0) AS pendingSegments,\n" + "COUNT(*) FILTER (WHERE replication_factor = -1) AS unknownSegments\n" + "FROM sys.segments\n" @@ -172,11 +185,11 @@ private void waitIfNeeded(long waitTimeMillis) throws Exception */ private void updateStatus(State state, DateTime startTime) { - int pendingSegmentCount = 0, usedSegmentsCount = 0, precachedSegmentCount = 0, asyncOnlySegmentCount = 0, unknownSegmentCount = 0; + int pendingSegmentCount = 0, usedSegmentsCount = 0, precachedSegmentCount = 0, onDemandSegmentCount = 0, unknownSegmentCount = 0; for (Map.Entry entry : versionToLoadStatusMap.entrySet()) { usedSegmentsCount += entry.getValue().getUsedSegments(); precachedSegmentCount += entry.getValue().getPrecachedSegments(); - asyncOnlySegmentCount += entry.getValue().getAsyncOnlySegments(); + onDemandSegmentCount += entry.getValue().getOnDemandSegments(); unknownSegmentCount += entry.getValue().getUnknownSegments(); pendingSegmentCount += entry.getValue().getPendingSegments(); } @@ -190,7 +203,7 @@ private void updateStatus(State state, DateTime startTime) totalSegmentsGenerated, usedSegmentsCount, precachedSegmentCount, - asyncOnlySegmentCount, + onDemandSegmentCount, pendingSegmentCount, unknownSegmentCount ) @@ -235,7 +248,7 @@ public static class SegmentLoadWaiterStatus private final int totalSegments; private final int usedSegments; private final int precachedSegments; - private final int asyncOnlySegments; + private final int onDemandSegments; private final int pendingSegments; private final int unknownSegments; @@ -247,7 +260,7 @@ public SegmentLoadWaiterStatus( @JsonProperty("totalSegments") int totalSegments, @JsonProperty("usedSegments") int usedSegments, @JsonProperty("precachedSegments") int precachedSegments, - @JsonProperty("asyncOnlySegments") int asyncOnlySegments, + @JsonProperty("onDemandSegments") int onDemandSegments, @JsonProperty("pendingSegments") int pendingSegments, @JsonProperty("unknownSegments") int unknownSegments ) @@ -258,7 +271,7 @@ public SegmentLoadWaiterStatus( this.totalSegments = totalSegments; this.usedSegments = usedSegments; this.precachedSegments = precachedSegments; - this.asyncOnlySegments = asyncOnlySegments; + this.onDemandSegments = onDemandSegments; this.pendingSegments = pendingSegments; this.unknownSegments = unknownSegments; } @@ -302,9 +315,9 @@ public int getPrecachedSegments() } @JsonProperty - public int getAsyncOnlySegments() + public int getOnDemandSegments() { - return asyncOnlySegments; + return onDemandSegments; } @JsonProperty @@ -322,10 +335,27 @@ public int getUnknownSegments() public enum State { + /** + * Initial state after being initialised with the segment versions and before #waitForSegmentsToLoad has been called. + */ INIT, + /** + * All segments that need to be loaded have not yet been loaded. The load status is perodically being queried from + * the broker. + */ WAITING, + /** + * All segments which need to be loaded have been loaded, and the SegmentLoadWaiter exited successfully. + */ SUCCESS, + /** + * An exception occurred while checking load status. The SegmentLoadWaiter exited without failing the task. + */ FAILED, + /** + * The time spent waiting for segments to load exceeded org.apache.druid.msq.exec.SegmentLoadWaiter#TIMEOUT_DURATION_MILLIS. + * The SegmentLoadWaiter exited without failing the task. + */ TIMED_OUT } @@ -333,7 +363,7 @@ public static class VersionLoadStatus { private final int usedSegments; private final int precachedSegments; - private final int asyncOnlySegments; + private final int onDemandSegments; private final int pendingSegments; private final int unknownSegments; @@ -341,14 +371,14 @@ public static class VersionLoadStatus public VersionLoadStatus( @JsonProperty("usedSegments") int usedSegments, @JsonProperty("precachedSegments") int precachedSegments, - @JsonProperty("asyncOnlySegments") int asyncOnlySegments, + @JsonProperty("onDemandSegments") int onDemandSegments, @JsonProperty("pendingSegments") int pendingSegments, @JsonProperty("unknownSegments") int unknownSegments ) { this.usedSegments = usedSegments; this.precachedSegments = precachedSegments; - this.asyncOnlySegments = asyncOnlySegments; + this.onDemandSegments = onDemandSegments; this.pendingSegments = pendingSegments; this.unknownSegments = unknownSegments; } @@ -366,9 +396,9 @@ public int getPrecachedSegments() } @JsonProperty - public int getAsyncOnlySegments() + public int getOnDemandSegments() { - return asyncOnlySegments; + return onDemandSegments; } @JsonProperty @@ -386,7 +416,7 @@ public int getUnknownSegments() @JsonIgnore public boolean isLoadingComplete() { - return pendingSegments == 0 && (usedSegments == precachedSegments + asyncOnlySegments); + return pendingSegments == 0 && (usedSegments == precachedSegments + onDemandSegments); } } } From 41aa0a5defdfd0fafcbbe0fd0c244acd9bff9ca8 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Tue, 5 Sep 2023 11:03:12 +0530 Subject: [PATCH 18/19] Update error message --- .../java/org/apache/druid/discovery/BrokerClient.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/apache/druid/discovery/BrokerClient.java b/server/src/main/java/org/apache/druid/discovery/BrokerClient.java index a83ca98103ce..bc97c2490ef4 100644 --- a/server/src/main/java/org/apache/druid/discovery/BrokerClient.java +++ b/server/src/main/java/org/apache/druid/discovery/BrokerClient.java @@ -85,9 +85,13 @@ public String sendQuery(final Request request) throws Exception HttpResponseStatus responseStatus = fullResponseHolder.getResponse().getStatus(); if (HttpResponseStatus.SERVICE_UNAVAILABLE.equals(responseStatus) || HttpResponseStatus.GATEWAY_TIMEOUT.equals(responseStatus)) { - throw new IOE(StringUtils.format("Request to broker failed due to failed response status: [%s]", responseStatus)); + throw DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.RUNTIME_FAILURE) + .build("Request to broker failed due to failed response status: [%s]", responseStatus); } else if (responseStatus.getCode() != HttpServletResponse.SC_OK) { - throw new IOE(StringUtils.format("Request to broker failed due to failed response code: [%s]", responseStatus.getCode())); + throw DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.RUNTIME_FAILURE) + .build("Request to broker failed due to failed response code: [%s]", responseStatus.getCode()); } return fullResponseHolder.getContent(); }, From 7ca94651a117308af42fcd926d6d70b733bc7ddf Mon Sep 17 00:00:00 2001 From: Karan Kumar Date: Wed, 6 Sep 2023 10:35:27 +0530 Subject: [PATCH 19/19] Update extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java --- .../main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java index d572f0991886..3a54c41e4104 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadWaiter.java @@ -65,7 +65,7 @@ public class SegmentLoadWaiter /** * The query sent to the broker. This query uses replication_factor to determine how many copies of a segment has to be * loaded as per the load rules. - * - If a segment is not used, the broker will not have any information about it, hence, a COUNT(*) should return this count. + * - If a segment is not used, the broker will not have any information about it, hence, a COUNT(*) should return the used count only. * - If replication_factor is more than 0, the segment will be loaded on historicals and needs to be waited for. * - If replication_factor is 0, that means that the segment will never be loaded on a historical and does not need to * be waited for.