-
Notifications
You must be signed in to change notification settings - Fork 3.8k
Inject new task count calculation during the rollover #18860
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: master
Are you sure you want to change the base?
Changes from all commits
e5a0512
08eddc3
d6894c7
2d8ba11
7a853ab
1109f5b
004b10d
6ce6562
7788e38
5936fd0
640287c
2722e0d
2830ea1
fd654e2
24fdd70
833cb88
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change | ||||
|---|---|---|---|---|---|---|
|
|
@@ -63,6 +63,7 @@ | |||||
| import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; | ||||||
| import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager; | ||||||
| import org.apache.druid.indexing.overlord.supervisor.autoscaler.LagStats; | ||||||
| import org.apache.druid.indexing.overlord.supervisor.autoscaler.SupervisorTaskAutoScaler; | ||||||
| import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata; | ||||||
| import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers; | ||||||
| import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; | ||||||
|
|
@@ -885,7 +886,7 @@ public String getType() | |||||
|
|
||||||
| /** | ||||||
| * Tag for identifying this supervisor in thread-names, listeners, etc. tag = (type + supervisorId). | ||||||
| */ | ||||||
| */ | ||||||
| private final String supervisorTag; | ||||||
| private final TaskInfoProvider taskInfoProvider; | ||||||
| private final RowIngestionMetersFactory rowIngestionMetersFactory; | ||||||
|
|
@@ -926,6 +927,12 @@ public String getType() | |||||
| private volatile boolean lifecycleStarted = false; | ||||||
| private final ServiceEmitter emitter; | ||||||
|
|
||||||
| /** | ||||||
| * Reference to the autoscaler, used for rollover-based scale-down decisions. | ||||||
| * Wired by {@link SupervisorManager} after supervisor creation. | ||||||
| */ | ||||||
| private volatile SupervisorTaskAutoScaler taskAutoScaler; | ||||||
|
|
||||||
| // snapshots latest sequences from the stream to be verified in the next run cycle of inactive stream check | ||||||
| private final Map<PartitionIdType, SequenceOffsetType> previousSequencesFromStream = new HashMap<>(); | ||||||
| private long lastActiveTimeMillis; | ||||||
|
|
@@ -1306,7 +1313,7 @@ public void tryInit() | |||||
| if (log.isDebugEnabled()) { | ||||||
| log.debug( | ||||||
| "Handled notice[%s] from notices queue in [%d] ms, " | ||||||
| + "current notices queue size [%d] for supervisor[%s] for datasource[%s].", | ||||||
| + "current notices queue size [%d] for supervisor[%s] for datasource[%s].", | ||||||
| noticeType, noticeHandleTime.millisElapsed(), getNoticesQueueSize(), supervisorId, dataSource | ||||||
| ); | ||||||
| } | ||||||
|
|
@@ -1677,6 +1684,13 @@ private List<ParseExceptionReport> getCurrentParseErrors() | |||||
| return limitedParseErrors; | ||||||
| } | ||||||
|
|
||||||
| @Override | ||||||
| public SupervisorTaskAutoScaler createAutoscaler() | ||||||
| { | ||||||
| this.taskAutoScaler = spec.createAutoscaler(this); | ||||||
| return this.taskAutoScaler; | ||||||
| } | ||||||
|
|
||||||
| @VisibleForTesting | ||||||
| public TaskGroup addTaskGroupToActivelyReadingTaskGroup( | ||||||
| int taskGroupId, | ||||||
|
|
@@ -3338,6 +3352,7 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException | |||||
|
|
||||||
| final AtomicInteger numStoppedTasks = new AtomicInteger(); | ||||||
| // Sort task groups by start time to prioritize early termination of earlier groups, then iterate for processing | ||||||
| // Sort task groups by start time to prioritize early termination of earlier groups, then iterate for processing | ||||||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. duplicate comment. |
||||||
| activelyReadingTaskGroups.entrySet().stream().sorted( | ||||||
| Comparator.comparingLong( | ||||||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Please revert all the formatting changes in this file.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Done. |
||||||
| taskGroupEntry -> computeEarliestTaskStartTime(taskGroupEntry.getValue()).getMillis() | ||||||
|
|
@@ -3376,7 +3391,6 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException | |||||
| } | ||||||
| } | ||||||
| }); | ||||||
|
|
||||||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Please retain the newline for clean separation of code. |
||||||
| List<Either<Throwable, Map<PartitionIdType, SequenceOffsetType>>> results = coalesceAndAwait(futures); | ||||||
| for (int j = 0; j < results.size(); j++) { | ||||||
| Integer groupId = futureGroupIds.get(j); | ||||||
|
|
@@ -3428,6 +3442,17 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException | |||||
| // remove this task group from the list of current task groups now that it has been handled | ||||||
| activelyReadingTaskGroups.remove(groupId); | ||||||
| } | ||||||
|
|
||||||
| if (taskAutoScaler != null && activelyReadingTaskGroups.isEmpty()) { | ||||||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Please move this entire new logic into a new private method and add a short javadoc to it. |
||||||
| int rolloverTaskCount = taskAutoScaler.computeTaskCountForRollover(); | ||||||
| if (rolloverTaskCount > 0 && rolloverTaskCount < ioConfig.getTaskCount()) { | ||||||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Should we also allow scale up on task rollover? |
||||||
| log.info("Cost-based autoscaler recommends scaling down to [%d] tasks during rollover", rolloverTaskCount); | ||||||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Suggested change
|
||||||
| changeTaskCountInIOConfig(rolloverTaskCount); | ||||||
| // Here force reset the supervisor state to be re-calculated on the next iteration of runInternal() call. | ||||||
| // This seems the best way to inject task amount recalculation during the rollover. | ||||||
| clearAllocationInfo(); | ||||||
| } | ||||||
| } | ||||||
| } | ||||||
|
|
||||||
| private DateTime computeEarliestTaskStartTime(TaskGroup group) | ||||||
|
|
@@ -4330,6 +4355,15 @@ public SeekableStreamSupervisorIOConfig getIoConfig() | |||||
| return ioConfig; | ||||||
| } | ||||||
|
|
||||||
| /** | ||||||
| * Sets the autoscaler reference for rollover-based scale-down decisions. | ||||||
| * Called by {@link SupervisorManager} after supervisor creation. | ||||||
| */ | ||||||
| public void setTaskAutoScaler(@Nullable SupervisorTaskAutoScaler taskAutoScaler) | ||||||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is this still needed? |
||||||
| { | ||||||
| this.taskAutoScaler = taskAutoScaler; | ||||||
| } | ||||||
|
|
||||||
| @Override | ||||||
| public void checkpoint(int taskGroupId, DataSourceMetadata checkpointMetadata) | ||||||
| { | ||||||
|
|
@@ -4688,9 +4722,9 @@ protected void emitLag() | |||||
|
|
||||||
| // Try emitting lag even with stale metrics provided that none of the partitions has negative lag | ||||||
| final long staleMillis = sequenceLastUpdated == null | ||||||
| ? 0 | ||||||
| : DateTimes.nowUtc().getMillis() | ||||||
| - (tuningConfig.getOffsetFetchPeriod().getMillis() + sequenceLastUpdated.getMillis()); | ||||||
| ? 0 | ||||||
| : DateTimes.nowUtc().getMillis() | ||||||
| - (tuningConfig.getOffsetFetchPeriod().getMillis() + sequenceLastUpdated.getMillis()); | ||||||
| if (staleMillis > 0 && partitionLags.values().stream().anyMatch(x -> x < 0)) { | ||||||
| // Log at most once every twenty supervisor runs to reduce noise in the logs | ||||||
| if ((staleMillis / getIoConfig().getPeriod().getMillis()) % 20 == 0) { | ||||||
|
|
||||||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for adding this verification.
Can we update the existing scale down test to verify that scaling was actually skipped in that case?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Here it is actually should not be skipped :)