-
Notifications
You must be signed in to change notification settings - Fork 3.8k
Add metric and simulation test for turbo loading mode #17830
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
Changes from all commits
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 |
|---|---|---|
|
|
@@ -20,6 +20,7 @@ | |
| package org.apache.druid.server.coordination; | ||
|
|
||
| import com.google.common.annotations.VisibleForTesting; | ||
| import com.google.common.base.Throwables; | ||
| import com.google.common.cache.Cache; | ||
| import com.google.common.cache.CacheBuilder; | ||
| import com.google.common.collect.ImmutableList; | ||
|
|
@@ -57,7 +58,7 @@ | |
| * Responsible for loading and dropping of segments by a process that can serve segments. | ||
| */ | ||
| @ManageLifecycle | ||
| public class SegmentLoadDropHandler implements DataSegmentChangeHandler | ||
| public class SegmentLoadDropHandler | ||
| { | ||
| private static final EmittingLogger log = new EmittingLogger(SegmentLoadDropHandler.class); | ||
|
|
||
|
|
@@ -125,6 +126,7 @@ public SegmentLoadDropHandler( | |
| this.normalLoadExec = normalLoadExec; | ||
| this.turboLoadExec = turboLoadExec; | ||
|
|
||
| // Allow core threads to time out to save resources when not in turbo mode | ||
| this.turboLoadExec.allowCoreThreadTimeOut(true); | ||
|
|
||
| this.segmentsToDelete = new ConcurrentSkipListSet<>(); | ||
|
|
@@ -141,12 +143,15 @@ public Map<String, SegmentRowCountDistribution> getRowCountDistributionPerDataso | |
| return segmentManager.getRowCountDistribution(); | ||
| } | ||
|
|
||
| @Override | ||
| public void addSegment(DataSegment segment, @Nullable DataSegmentChangeCallback callback) | ||
| public void addSegment( | ||
|
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. We could have an internal overloaded method instead if we wanted to keep the interface
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. Replied to the other comment. The contract of |
||
| DataSegment segment, | ||
| @Nullable DataSegmentChangeCallback callback, | ||
| SegmentLoadingMode loadingMode | ||
| ) | ||
| { | ||
| SegmentChangeStatus result = null; | ||
| try { | ||
| log.info("Loading segment[%s]", segment.getId()); | ||
| log.info("Loading segment[%s] in mode[%s]", segment.getId(), loadingMode); | ||
| /* | ||
| The lock below is used to prevent a race condition when the scheduled runnable in removeSegment() starts, | ||
| and if (segmentsToDelete.remove(segment)) returns true, in which case historical will start deleting segment | ||
|
|
@@ -179,13 +184,14 @@ each time when addSegment() is called, it has to wait for the lock in order to m | |
| throw new SegmentLoadingException(e, "Failed to announce segment[%s]", segment.getId()); | ||
| } | ||
|
|
||
| result = SegmentChangeStatus.SUCCESS; | ||
| result = SegmentChangeStatus.success(loadingMode); | ||
| } | ||
| catch (Throwable e) { | ||
| log.makeAlert(e, "Failed to load segment") | ||
| .addData("segment", segment) | ||
| .emit(); | ||
| result = SegmentChangeStatus.failed(e.toString()); | ||
| Throwable rootCause = Throwables.getRootCause(e); | ||
| result = SegmentChangeStatus.failed(rootCause.toString(), loadingMode); | ||
| } | ||
| finally { | ||
| updateRequestStatus(new SegmentChangeRequestLoad(segment), result); | ||
|
|
@@ -195,7 +201,6 @@ each time when addSegment() is called, it has to wait for the lock in order to m | |
| } | ||
| } | ||
|
|
||
| @Override | ||
| public void removeSegment(DataSegment segment, @Nullable DataSegmentChangeCallback callback) | ||
| { | ||
| removeSegment(segment, callback, true); | ||
|
|
@@ -242,7 +247,7 @@ void removeSegment( | |
| runnable.run(); | ||
| } | ||
|
|
||
| result = SegmentChangeStatus.SUCCESS; | ||
| result = SegmentChangeStatus.success(); | ||
| } | ||
| catch (Exception e) { | ||
| log.makeAlert(e, "Failed to remove segment") | ||
|
|
@@ -322,11 +327,13 @@ public void addSegment( | |
| @Nullable DataSegmentChangeCallback callback | ||
| ) | ||
| { | ||
| requestStatuses.put(changeRequest, new AtomicReference<>(SegmentChangeStatus.PENDING)); | ||
| getExecutorService(segmentLoadingMode).submit( | ||
| final SegmentChangeStatus pendingStatus = SegmentChangeStatus.pending(segmentLoadingMode); | ||
| requestStatuses.put(changeRequest, new AtomicReference<>(pendingStatus)); | ||
| getLoadingExecutor(segmentLoadingMode).submit( | ||
| () -> SegmentLoadDropHandler.this.addSegment( | ||
| ((SegmentChangeRequestLoad) changeRequest).getSegment(), | ||
| () -> resolveWaitingFutures() | ||
| () -> resolveWaitingFutures(), | ||
| segmentLoadingMode | ||
| ) | ||
| ); | ||
| } | ||
|
|
@@ -337,7 +344,7 @@ public void removeSegment( | |
| @Nullable DataSegmentChangeCallback callback | ||
| ) | ||
| { | ||
| requestStatuses.put(changeRequest, new AtomicReference<>(SegmentChangeStatus.PENDING)); | ||
| requestStatuses.put(changeRequest, new AtomicReference<>(SegmentChangeStatus.pending())); | ||
| SegmentLoadDropHandler.this.removeSegment( | ||
| ((SegmentChangeRequestDrop) changeRequest).getSegment(), | ||
| () -> resolveWaitingFutures(), | ||
|
|
@@ -428,7 +435,7 @@ public boolean cancel(boolean interruptIfRunning) | |
| } | ||
| } | ||
|
|
||
| private ExecutorService getExecutorService(SegmentLoadingMode loadingMode) | ||
| private ExecutorService getLoadingExecutor(SegmentLoadingMode loadingMode) | ||
| { | ||
| return loadingMode == SegmentLoadingMode.TURBO ? turboLoadExec : normalLoadExec; | ||
| } | ||
|
|
||
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.
What's the reason to remove the interface?
I think this class is one of the main implementations of it, so if required, isn't it possible to modify the interface itself instead?
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.
It wouldn't make sense to modify the interface since a
DataSegmentChangeRequestitself is not aware of the loading mode.Having
SegmentLoadDropHandlerimplement the handler interface doesn't seem to serve any real purpose, since we are already creating a separate handler insideprocessRequestmethod. That handler then calls the outsideaddorremovemethods. So I decided to just get rid of that interface.For now, this seemed the cleanest approach. We can add it back in the future, if needed.
Please let me know what you think.