jihoonson commented on a change in pull request #10213:
URL: https://github.com/apache/druid/pull/10213#discussion_r589116331
##########
File path:
server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java
##########
@@ -361,21 +386,33 @@ private void entryRemoved(SegmentHolder segmentHolder,
String path)
);
}
- private void failAssign(SegmentHolder segmentHolder)
+ private void failAssign(SegmentHolder segmentHolder, boolean handleTimeout)
{
- failAssign(segmentHolder, null);
+ failAssign(segmentHolder, handleTimeout, null);
}
- private void failAssign(SegmentHolder segmentHolder, Exception e)
+ private void failAssign(SegmentHolder segmentHolder, boolean handleTimeout,
Exception e)
{
if (e != null) {
log.error(e, "Server[%s], throwable caught when submitting [%s].",
basePath, segmentHolder);
Review comment:
I'm not sure why we don't emit exceptions currently (using
`EmittingLogger.makeAlert()`), but should we? At least for the segment loading
timeout error, it would be nice to emit those errors so that cluster operators
can notice there is something going wrong with segment loading.
##########
File path:
server/src/main/java/org/apache/druid/server/coordinator/SegmentReplicantLookup.java
##########
@@ -59,7 +59,10 @@ public static SegmentReplicantLookup make(DruidCluster
cluster)
if (numReplicants == null) {
numReplicants = 0;
}
- loadingSegments.put(segment.getId(), server.getTier(), numReplicants
+ 1);
+ // Timed out segments need to be replicated in another server for
faster availability
+ if (!serverHolder.getPeon().getTimedOutSegments().contains(segment))
{
+ loadingSegments.put(segment.getId(), server.getTier(),
numReplicants + 1);
Review comment:
As @himanshug pointed out in
https://github.com/apache/druid/issues/10193#issuecomment-790941778, there
could be two types of slow segment loading.
- There are a few historicals being slow in segment loading in the cluster.
This can be caused by unbalanced load queues or some intermittent failures.
- Historicals are OK, but ingestion might outpace the ability to load
segments.
This particular change in `SegmentReplicantLookup` could help in the former
case, but make things worse in the latter case. In an extreme case, all
historicals could have the same set of timed-out segments in their load queue.
This might be still OK though, because, if that's the case, Druid cannot get
out of that state by itself anyway. The system administrator should add more
historicals or use more threads for parallel segment loading. However, we
should provide relevant data so that system administrators can tell what's
happening. I left another comment about emitting exceptions to provide such
data.
##########
File path:
server/src/main/java/org/apache/druid/server/coordinator/SegmentReplicantLookup.java
##########
@@ -59,7 +59,10 @@ public static SegmentReplicantLookup make(DruidCluster
cluster)
if (numReplicants == null) {
numReplicants = 0;
}
- loadingSegments.put(segment.getId(), server.getTier(), numReplicants
+ 1);
+ // Timed out segments need to be replicated in another server for
faster availability
+ if (!serverHolder.getPeon().getTimedOutSegments().contains(segment))
{
+ loadingSegments.put(segment.getId(), server.getTier(),
numReplicants + 1);
Review comment:
`loadingSegments` is not just a set of segments loading anymore. Please
add some javadoc in `SegmentReplicantLookup` about this.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]