Skip to content

Commit

Permalink
Add code to wait for segments generated to be loaded on historicals (#…
Browse files Browse the repository at this point in the history
…14322)

Currently, after an MSQ query, the web console is responsible for waiting for the segments to load. It does so by checking if there are any segments loading into the datasource ingested into, which can cause some issues, like in cases where the segments would never be loaded, or would end up waiting for other ingests as well.

This PR shifts this responsibility to the controller, which would have the list of segments created.
  • Loading branch information
adarshsanjeev authored Sep 6, 2023
1 parent 706b57c commit 959148a
Show file tree
Hide file tree
Showing 15 changed files with 1,070 additions and 74 deletions.
24 changes: 23 additions & 1 deletion docs/api-reference/sql-ingestion-api.md
Original file line number Diff line number Diff line change
Expand Up @@ -288,7 +288,19 @@ The response shows an example report for a query.
"startTime": "2022-09-14T22:12:09.266Z",
"durationMs": 28227,
"pendingTasks": 0,
"runningTasks": 2
"runningTasks": 2,
"segmentLoadStatus": {
"state": "SUCCESS",
"dataSource": "kttm_simple",
"startTime": "2022-09-14T23:12:09.266Z",
"duration": 15,
"totalSegments": 1,
"usedSegments": 1,
"precachedSegments": 0,
"onDemandSegments": 0,
"pendingSegments": 0,
"unknownSegments": 0
}
},
"stages": [
{
Expand Down Expand Up @@ -593,6 +605,16 @@ 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.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. |
| `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.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. |
| `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. |
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -63,6 +64,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;
Expand Down Expand Up @@ -292,6 +294,7 @@ public class ControllerImpl implements Controller
private WorkerMemoryParameters workerMemoryParameters;
private boolean isDurableStorageEnabled;
private boolean isFaultToleranceEnabled;
private volatile SegmentLoadWaiter segmentLoadWaiter;

public ControllerImpl(
final MSQControllerTask task,
Expand Down Expand Up @@ -437,6 +440,45 @@ 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.
}
}


try {
releaseTaskLocks();

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 e) {
log.warn(e, "Exception thrown during cleanup. Ignoring it and writing task report.");
}

try {
// Write report even if something went wrong.
final MSQStagesReport stagesReport;
Expand Down Expand Up @@ -488,7 +530,8 @@ public TaskStatus runTask(final Closer closer)
workerWarnings,
queryStartTime,
new Interval(queryStartTime, DateTimes.nowUtc()).toDurationMillis(),
workerTaskLauncher
workerTaskLauncher,
segmentLoadWaiter
),
stagesReport,
countersSnapshot,
Expand All @@ -504,30 +547,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 {
Expand All @@ -536,6 +555,23 @@ public TaskStatus runTask(final Closer closer)
}
}

/**
* Releases the locks obtained by the task.
*/
private void releaseTaskLocks() throws IOException
{
final List<TaskLock> 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()}.
Expand Down Expand Up @@ -875,7 +911,8 @@ public Map<String, TaskReport> liveReports()
workerWarnings,
queryStartTime,
queryStartTime == null ? -1L : new Interval(queryStartTime, DateTimes.nowUtc()).toDurationMillis(),
workerTaskLauncher
workerTaskLauncher,
segmentLoadWaiter
),
makeStageReport(
queryDef,
Expand Down Expand Up @@ -1316,17 +1353,36 @@ private void publishAllSegments(final Set<DataSegment> 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 {
Set<String> versionsToAwait = segmentsWithTombstones.stream().map(DataSegment::getVersion).collect(Collectors.toSet());
segmentLoadWaiter = new SegmentLoadWaiter(
context.injector().getInstance(BrokerClient.class),
context.jsonMapper(),
task.getDataSource(),
versionsToAwait,
segmentsWithTombstones.size(),
true
);
performSegmentPublish(
context.taskActionClient(),
SegmentTransactionalInsertAction.overwriteAction(null, segmentsWithTombstones)
);
}
} else if (!segments.isEmpty()) {
Set<String> versionsToAwait = segments.stream().map(DataSegment::getVersion).collect(Collectors.toSet());
segmentLoadWaiter = new SegmentLoadWaiter(
context.injector().getInstance(BrokerClient.class),
context.jsonMapper(),
task.getDataSource(),
versionsToAwait,
segments.size(),
true
);
// Append mode.
performSegmentPublish(
context.taskActionClient(),
Expand Down Expand Up @@ -2072,7 +2128,8 @@ private static MSQStatusReport makeStatusReport(
final Queue<MSQErrorReport> errorReports,
@Nullable final DateTime queryStartTime,
final long queryDuration,
MSQWorkerTaskLauncher taskLauncher
MSQWorkerTaskLauncher taskLauncher,
final SegmentLoadWaiter segmentLoadWaiter
)
{
int pendingTasks = -1;
Expand All @@ -2083,14 +2140,18 @@ private static MSQStatusReport makeStatusReport(
pendingTasks = workerTaskCount.getPendingWorkerCount();
runningTasks = workerTaskCount.getRunningWorkerCount() + 1; // To account for controller.
}

SegmentLoadWaiter.SegmentLoadWaiterStatus status = segmentLoadWaiter == null ? null : segmentLoadWaiter.status();

return new MSQStatusReport(
taskState,
errorReport,
errorReports,
queryStartTime,
queryDuration,
pendingTasks,
runningTasks
runningTasks,
status
);
}

Expand Down Expand Up @@ -2259,6 +2320,7 @@ private Pair<ControllerQueryKernel, ListenableFuture<?>> run() throws IOExceptio
throwKernelExceptionIfNotUnknown();
}

updateLiveReportMaps();
cleanUpEffectivelyFinishedStages();
return Pair.of(queryKernel, workerTaskLauncherFuture);
}
Expand Down
Loading

0 comments on commit 959148a

Please sign in to comment.