Skip to content
Merged
Changes from 1 commit
Commits
Show all changes
37 commits
Select commit Hold shift + click to select a range
e8b0a23
Add published segment cache in broker
Jan 22, 2019
6ec4911
Change the DataSegment interner so it's not based on DataSEgment's eq…
Jan 27, 2019
8b6d453
Merge branch 'master' of github.com:druid-io/druid into broker-segmen…
Jan 27, 2019
281600d
Use separate interner for realtime and historical segments
Jan 28, 2019
cdc751e
Merge branch 'master' of github.com:druid-io/druid into broker-segmen…
Jan 28, 2019
8fed80a
Remove trueEquals as it's not used anymore, change log message
Jan 28, 2019
0d0f89f
PR comments
Jan 29, 2019
8fc84b3
PR comments
Jan 29, 2019
adf133f
Merge branch 'master' of github.com:druid-io/druid into broker-segmen…
Jan 29, 2019
d46edc5
Fix tests
Jan 29, 2019
92601d0
Merge branch 'master' of github.com:druid-io/druid into broker-segmen…
Jan 29, 2019
a993482
PR comments
Jan 30, 2019
a4cbcfc
Few more modification to
Jan 30, 2019
e376df3
Merge branch 'master' of github.com:druid-io/druid into broker-segmen…
Jan 30, 2019
0fbb48c
minor changes
Jan 30, 2019
8df2d96
PR comments
Jan 31, 2019
d5b7d79
PR comments
Jan 31, 2019
2bfa396
Merge branch 'master' of github.com:druid-io/druid into broker-segmen…
Jan 31, 2019
632d741
Make the segment cache in broker off by default
Feb 1, 2019
bcc6513
Merge branch 'master' of github.com:druid-io/druid into broker-segmen…
Feb 1, 2019
c41f085
Add doc for new planner config
Feb 1, 2019
07a80b1
Update documentation
Feb 1, 2019
d83eb33
Merge branch 'master' of github.com:druid-io/druid into broker-segmen…
Feb 1, 2019
a440c0c
PR comments
Feb 1, 2019
b385345
Merge branch 'master' of github.com:druid-io/druid into broker-segmen…
Feb 1, 2019
981b080
some more changes
Feb 1, 2019
3a94cae
PR comments
Feb 1, 2019
0032a31
Merge branch 'master' of github.com:druid-io/druid into broker-segmen…
Feb 1, 2019
ad28458
fix test
Feb 1, 2019
33751a4
Merge branch 'master' of github.com:druid-io/druid into broker-segmen…
Feb 1, 2019
2ddb7a1
remove unintentional change, whether to synchronize on lifecycleLock …
Feb 1, 2019
1b37493
minor changes
Feb 1, 2019
60dbf41
some changes to initialization
Feb 2, 2019
8860053
use pollPeriodInMS
Feb 2, 2019
ca21779
Add boolean cachePopulated to check if first poll succeeds
Feb 2, 2019
3a70f39
Remove poll from start()
Feb 2, 2019
e2a9af7
take the log message out of condition in stop()
Feb 2, 2019
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -106,10 +106,15 @@ public void start()
}
try {
if (isCacheEnabled) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks like this class violates the Single Responsibility Principle, see #7392.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

hmm I need to read up more on Single Responsibility Principle, another way to look at it, is the responsibility of this class is to get metadata segments, the public method is getPublishedSegments, then it kind of does one thing i.e. get metadata segments.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Probably SRP is a wrong link in this case, cohesion may be a more correct reference. Anyway, it seems to me that isCacheEnabled replaces a perfectly healthy polymorphism "CachingMetadataSegmentView"/"DirectMetadataSegmentView" here. Somebody may want to add "CachingWithDirectMetadataStoreAccess" (vs. using Coordinator as a proxy), etc.

scheduledExec.schedule(new PollTask(), 0, TimeUnit.MILLISECONDS);
lifecycleLock.started();
log.info("MetadataSegmentView Started.");
try {
poll();
}
finally {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please add a catch block to emit exceptions.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

added catch block

scheduledExec.schedule(new PollTask(), 0, TimeUnit.MILLISECONDS);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please set the proper wait time instead of starting poll() immediately.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yeah, changed to use pollPeriodinMS

}
}
lifecycleLock.started();
log.info("MetadataSegmentView Started.");
}
finally {
lifecycleLock.exitStart();
Expand Down Expand Up @@ -161,6 +166,10 @@ private void poll()
public Iterator<DataSegment> getPublishedSegments()
{
if (isCacheEnabled) {
Preconditions.checkState(
lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS),
Copy link
Contributor

@jon-wei jon-wei Feb 2, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggest changing this precondition check message to "lifecycle start not completed", since the lifecycle starting doesn't necessarily indicate that the cache is ready (the initial poll() might have failed).

We could adjust this class so that the publishedSegments map is initially null, and initialize it upon the first successful getMetadataSegments() call in poll(). Then, in this method, add a check for "map != null" with the "hold on, still syncing published segments" message if the check fails.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

made changes, so the message correctly implies that published segments are still syncing.

"hold on, still syncing published segments"
);
return publishedSegments.keySet().iterator();
} else {
return getMetadataSegments(
Expand All @@ -173,7 +182,7 @@ public Iterator<DataSegment> getPublishedSegments()
}

// Note that coordinator must be up to get segments
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

See #7391

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

will resolve with #7391 , thanks.

private static JsonParserIterator<DataSegment> getMetadataSegments(
private JsonParserIterator<DataSegment> getMetadataSegments(
DruidLeaderClient coordinatorClient,
ObjectMapper jsonMapper,
BytesAccumulatingResponseHandler responseHandler,
Expand Down Expand Up @@ -226,20 +235,23 @@ private class PollTask implements Runnable
@Override
public void run()
{
long delayMS = pollPeriodinMS;
try {
final long pollStartTime = System.nanoTime();
poll();
final long pollEndTime = System.nanoTime();
final long pollTimeNS = pollEndTime - pollStartTime;
final long pollTimeMS = TimeUnit.NANOSECONDS.toMillis(pollTimeNS);
final long delayMS = Math.max(pollPeriodinMS - pollTimeMS, 0);
if (!Thread.currentThread().isInterrupted()) {
scheduledExec.schedule(new PollTask(), delayMS, TimeUnit.MILLISECONDS);
}
delayMS = Math.max(pollPeriodinMS - pollTimeMS, 0);
}
catch (Exception e) {
log.makeAlert(e, "Problem polling Coordinator.").emit();
}
finally {
if (!Thread.currentThread().isInterrupted()) {
scheduledExec.schedule(new PollTask(), delayMS, TimeUnit.MILLISECONDS);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why doesn't this class use ScheduledExecutorService.sheduleAtFixedRate() or sheduleWithFixedDelay()?

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

i changed to use schedule because if poll is executed with fixed delay, there might be a possibility of two or more polls to be executed at the same time, if there's huge GC or getMetadataSegments takes longer than poll period.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Both scheduleAtFixedRate and sheduleWithFixedDelay explicitly prohibit concurrent execution in their Javadocs. Also, this is not imaginably possible, because your sheduledExec has only a single thread.

}
}
}
}

Expand Down