-
Notifications
You must be signed in to change notification settings - Fork 3.7k
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
Cache segment metadata on the Overlord to speed up segment allocation and other task actions #17653
base: master
Are you sure you want to change the base?
Conversation
server/src/main/java/org/apache/druid/metadata/segment/cache/SqlSegmentsMetadataCache.java
Fixed
Show fixed
Hide fixed
server/src/main/java/org/apache/druid/metadata/segment/cache/SegmentsMetadataCache.java
Fixed
Show fixed
Hide fixed
server/src/main/java/org/apache/druid/metadata/segment/cache/DatasourceSegmentCache.java
Fixed
Show fixed
Hide fixed
@kfaraz thank you for the changes. Could you please call out the dependency on #17545 in the description? |
Thanks for the suggestion, @AmatyaAvadhanula . I have added an |
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.
I had a couple of questions about certain choices, which I have left as comments.
I was also hoping to understand the pros and cons of having pending segments and segments being accessed by the same ReadWriteLock in the cache.
Otherwise, the segment allocation changes look good to me.
Will try to wrap up the review of the caching mechanism soon.
|
||
HeapMemoryDatasourceSegmentCache(String dataSource) | ||
{ | ||
super(true); |
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.
Could you please add more details explaining the usage of a fair lock?
I recall a discussion where we wanted to change the lock in VersionedIntervalTimeline to false as well.
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.
Sure, will add that in the javadoc.
Edit: I haven't really given much thought to whether we should go with fair or not for the cache. Reading through the javadoc of ReentrantReadWriteLock, fair had seemed an appropriate choice. But I will take another look and document whatever we decide to do.
/** | ||
* Not being used right now. Could allow lookup of visible segments for a given interval. | ||
*/ | ||
private final SegmentTimeline usedSegmentTimeline = SegmentTimeline.forSegments(Set.of()); |
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.
Adding a timeline with potentially expensive operations because of its own lock seems risky and wasteful given that we are not using it.
Is there a reason we aren't using a TreeMap: Interval -> (DataSegment / SegmentId) and use it instead?
I believe the perf impact would be significant when there are several intervals and segments.
We are creating a Timeline after fetching segments anyway.
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.
Yeah, the timeline is not really being used in the code right now. I will just get rid of it for the time being.
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.
Is there a reason we aren't using a TreeMap: Interval -> (DataSegment / SegmentId) and use it instead?
I believe the perf impact would be significant when there are several intervals and segments.
Hmm, let me evaluate this once. I had decided against it originally since most segment searches are for overlapping intervals rather than exact matches. But I guess we can have some additional logic to prune out intervals which are disjoint, thus benefiting perf as you point out.
Other searches are by segment ID, so keeping an id -> segment
map helped.
Eventually, we would most likely end up keeping a timeline itself.
That timeline could also be used to replace the timeline maintained in SqlSegmentsMetadataManager
used by CompactionScheduler
(and also the coordinator).
There are certain transactions that modify both pending segments and regular segments. |
@kfaraz this is listed under "pending items". What race conditions are you aware of? |
Thanks for calling this out, @gianm .
I will take another pass through the code, just to ensure that I haven't missed anything, adding comments where necessary. |
@AmatyaAvadhanula , thanks for the suggestions! I have removed the timeline and added an interval map instead. |
Thanks for the notes! |
segmentAllocationQueue.becomeLeader(); | ||
taskMaster.becomeHalfLeader(taskRunner, taskQueue); | ||
} | ||
|
||
@Override | ||
public void stop() | ||
{ | ||
segmentMetadataCache.stopBeingLeader(); |
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.
Generally, the order of items in stop()
should be the reverse of the order in start()
, in case there are dependencies.
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.
Fixed.
@@ -228,6 +230,10 @@ public void configure(Binder binder) | |||
JsonConfigProvider.bind(binder, "druid.indexer.task.default", DefaultTaskConfig.class); | |||
binder.bind(RetryPolicyFactory.class).in(LazySingleton.class); | |||
|
|||
binder.bind(SegmentMetadataCache.class) |
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.
Is this meant to override the binding to NoopSegmentMetadataCache
in SQLMetadataStorageDruidModule
? I thought multiple bindings typically weren't allowed.
I wonder why we need the binding outside of CliOverlord at all- why do other server types need to create one?
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.
Ideally, the other server types shouldn't need the binding at all.
But we have this dep graph:
CoreInjectorBuilder
-> DerbyMetadataStorageModule
-> SQLMetadataStorageModule
-> SqlSegmentMetadataTransactionFactory
(required for IndexerSQLMetadataStorageCoordinator
) -> SegmentMetadataCache
.
CoreInjectorBuilder
should not even load DerbyMetadataStorageModule
, it should be loaded only in CliOverlord
and CliCoordinator
. But I decided to make this change in a separate PR so that I could test it out properly, just to be on the safe side.
Please let me know if you think I should include it in this PR itself.
{ | ||
return pollDuration; | ||
this.pollDuration = Configs.valueOrDefault(pollDuration, Period.minutes(1)); | ||
this.useCache = Configs.valueOrDefault(useCache, true); |
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.
If this is what controls whether the new caching feature is enabled, then want this to be false
by default for now. We could change the default to true
when it's more proven out.
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.
Yes, I had enabled this temporarily to have all UTs and ITs work with the cache enabled.
I will disable it now as all tests seem to work as expected.
There are already some tests which run in both modes: cache enabled and disabled.
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.
Fixed, default is now false (disabled).
int maxPartitionNum = -1; | ||
for (String id : unusedSegmentIds) { | ||
final SegmentId segmentId = SegmentId.tryParse(datasource, id); | ||
if (segmentId == null) { |
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.
Warn (or perhaps even throw?) if the segment ID is unparseable, since in that case, the method may not be returning the correct answer.
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.
Fixed, throws exception now.
for (List<String> partition : partitionedSegmentIds) { | ||
fetchedSegments.addAll(retrieveSegmentBatchById(datasource, partition, false)); | ||
fetchedSegments.add(retrieveSegmentBatchById(datasource, partition, false)); |
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.
I wonder what this code will do exactly. There will be multiple CloseableIterator
from retrieveSegmentBatchById
existing at once. What effect does that have?
Does the metadata query get made lazily when the iterator first has hasNext()
called? If so then it would lead to the metadata queries being issued sequentially, which seems fine. But, if the query is issued as part of iterator creation, this would lead to quite a lot of simultaneously open queries, which might cause problems with the metadata store.
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 the suggestion! Added a CloseableIterator which keeps only one result set open at a time.
while (currentCacheState == CacheState.LEADER_FIRST_SYNC_PENDING | ||
|| currentCacheState == CacheState.LEADER_FIRST_SYNC_STARTED) { | ||
try { | ||
cacheStateLock.wait(5 * 60_000); |
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.
use a static constant please. Btw, if the intent here is to have a specific timeout on waiting for sync, it should be checked again after wait
returns (and then continue to wait
if the timeout hasn't been reached yet). It is possible for wait
to return early in case of spurious wakeup.
To ensure the thread wakes up timely, all of the cache state transitions should include a notifyAll
.
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.
No, the intent is just to avoid waiting forever.
The spurious wakeup is handled by verifyCacheIsReady
(renamed to verifyCacheIsUsableAndAwaitSync()
) itself.
cacheStateLock.wait(5 * 60_000); | ||
} | ||
catch (Exception e) { | ||
log.noStackTrace().error(e, "Error while waiting for cache to be ready"); |
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.
I think the only error wait
will throw during normal operation is InterruptedException
. Consider special-casing that, and logging at a lower level. Other exception types can continue to be logged at error level and re-thrown.
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.
added catch for InterruptedException
.
} | ||
catch (Throwable t) { | ||
log.error(t, "Error occurred while polling metadata store"); | ||
log.makeAlert(t, "Error occurred while polling metadata store"); |
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.
Missing call to emit()
after makeAlert
. Also, no reason to call both log.error
and log.makeAlert
. The alert is logged when it is emitted.
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 catching this!
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.
fixed.
throw DruidException.defensive("Cache has not been started yet"); | ||
} | ||
|
||
currentCacheState = CacheState.LEADER_FIRST_SYNC_PENDING; |
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.
Interrupt and re-start the current sync? That could help the leader gain leadership faster.
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.
Updated.
* <li>Emit metrics</li> | ||
* </ul> | ||
*/ | ||
private void syncWithMetadataStore() |
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.
Have you been able to benchmark this method with a cluster with lots of segments (e.g. millions)? It will need to complete before allocation can work post-leadership-election, so I'm wondering how much time that will take.
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.
Not yet. I am about to start testing on some clusters with a large number of segments. I will share the results here once the testing is done.
@gianm , thanks for the review! |
Description
The Overlord performs several metadata operations on the tables
druid_segments
anddruid_pendingSegments
,such as segment commit, allocation, upgrade and mark used / unused.
Segment allocation, in particular, involves several reads/writes to the metadata store and can often become a bottleneck,
causing ingestion to slow down. This effect is particularly pronounced when streaming ingestion is enabled for
multiple datasources or if there is a lot of late arriving data.
This patch adds an in-memory segment cache to the Overlord to speed up all segment metadata operations.
Assumptions
Design
Summary
druid.manager.segments.useCache
to enable cacheSegment metadata transaction with cache enabled
Lifecycle of cache
druid.manager.segments.useCache=true
on the Overlordstart()
cache is called putting it in STANDBY mode.druid.manager.segments.pollDuration
to do the following:stop()
cache is called stopping the pollContents of cache
The cache maintains the following fields for every datasource.
Map<String, DataSegmentPlus> idToUsedSegment
Set<String> unusedSegmentIds
Map<Interval, Map<String, Integer>> intervalVersionToHighestUnusedPartitionNumber
Map<Interval, Map<String, PendingSegmentRecord>> intervalToPendingSegments
Code changes
SegmentsMetadataManagerConfig.useCache
DatasourceSegmentMetadataReader
DatasourceSegmentMetadataWriter
HeapMemorySegmentMetadataCache
DatasourceSegmentCache
SegmentMetadataTransaction
SqlSegmentMetadataTransaction
CachedSegmentMetadataTransaction
SqlSegmentMetadataTransactionFactory
IndexerSQLMetadataStorageCoordinator
SqlSegmentsMetadataQuery
SqlSegmentMetadataTransaction
Testing
IndexerSQLMetadataStorageCoordinatorTest
SegmentAllocateActionTest
SegmentAllocationQueueTest
DatasourceSegmentCacheTest
Pending items
Release note
Add Overlord runtime property
druid.manager.segments.useCache
(default valuefalse
).Set this to
true
to turn on segment metadata caching on the Overlord. This allows segment metadata operationssuch as reads and segment allocation to be sped up significantly.
Upgrade notes
The flag
druid.manager.segments.useCache
to enable the segment cache should be turned on only whenDruid has been upgraded to a version containing both this patch #17653 and #17545 .
When Druid is being downgraded to an older, the feature flag must first be turned off.
This PR has: