Skip to content
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

RATIS-1893. In SegmentedRaftLogCache, start a daemon thread to checkAndEvictCache. #924

Merged
merged 4 commits into from
Sep 24, 2023

Conversation

szetszwo
Copy link
Contributor

Copy link
Contributor

@OneSizeFitsQuorum OneSizeFitsQuorum left a comment

Choose a reason for hiding this comment

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

Hi, @szetszwo
I salute you for acting so quickly!

I'm wondering if an asynchronous thread would be a more elegant design. It looks like it would require a big change in cache concurrency control.

If my subsequent review holds true, perhaps we could contemplate managing it within the primary appendLog flow but trigger it less frequently, maybe when transitioning the raftlog. This could potentially reduce implementation complexity.

What's your opinion?

@@ -505,6 +507,7 @@ public CompletableFuture<Long> onSnapshotInstalled(long lastSnapshotIndex) {
public void close() throws IOException {
try(AutoCloseableLock writeLock = writeLock()) {
super.close();
cacheEviction.close();
cache.close();
Copy link
Contributor

@OneSizeFitsQuorum OneSizeFitsQuorum Sep 20, 2023

Choose a reason for hiding this comment

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

It looks like the cache is now being called by multiple threads concurrently. Should we add more concurrency control logic inside the cache?Otherwise, concurrent traversal and modification of arrays such as closedSegments could fail

Copy link
Contributor Author

Choose a reason for hiding this comment

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

The cache is thread safe and it is already called by multiple threads; see RATIS-430.

Copy link
Contributor

Choose a reason for hiding this comment

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

got it

}

try {
runMethod.run();
Copy link
Contributor

Choose a reason for hiding this comment

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

It looks like the thread will never sleep after the first signal, even if there is no write, it will always be busy executing the run function. I'm not sure if I've got it wrong

Copy link
Contributor

@ivandika3 ivandika3 Sep 20, 2023

Choose a reason for hiding this comment

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

If I understand correctly, once the runMethod (e.g. checkAndEvictCache) finishes, it will loop back to awaitForSignal.await which will block until signal() is sent.

Also, I think that this ensures checkAndEvictCache run sequentially. If cacheEviction.signal() is sent multiple times for each append entry, checkAndEvictCache only runs it if runMethod finished running. So this might avoid the call for checkAndEvictCache for every append entry.

Please correct me if I'm wrong.

Copy link
Contributor

Choose a reason for hiding this comment

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

Sorry, it's my fault! I was wondering why getAndSet was used here, now I understand, very clever design!

Copy link
Contributor

Choose a reason for hiding this comment

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

So this might avoid the call for checkAndEvictCache for every append entry.

Yes, it definitely reduces the number of calls ~ so that leaves the concurrency control part above to see how we can do it ~

@szetszwo
Copy link
Contributor Author

I'm wondering if an asynchronous thread would be a more elegant design. It looks like it would require a big change in cache concurrency control.

The daemon in AwaitToRun is an asynchronous thread.

If my subsequent review holds true, perhaps we could contemplate managing it within the primary appendLog flow but trigger it less frequently, maybe when transitioning the raftlog. This could potentially reduce implementation complexity.

What's your opinion?

In appendEntryImpl, how about we evict cache only after rollOpenSegment?

Copy link
Contributor

@OneSizeFitsQuorum OneSizeFitsQuorum left a comment

Choose a reason for hiding this comment

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

Awesome work!
With asynchronous threading we were able to avoid the effect of checkAndEvictCache on write latency, and by checking only when Raft logs are switched, we were able to further reduce the frequency of checks, thus reducing CPU resource consumption!

@@ -382,26 +385,29 @@ protected CompletableFuture<Long> appendEntryImpl(LogEntryProto entry) {
final Timekeeper.Context appendEntryTimerContext = getRaftLogMetrics().startAppendEntryTimer();
validateLogEntry(entry);
final LogSegment currentOpenSegment = cache.getOpenSegment();
boolean rollOpenSegment = false;
Copy link
Contributor

Choose a reason for hiding this comment

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

do we need to check here when addOpenSegment?

Copy link
Contributor

Choose a reason for hiding this comment

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

Looking at the contents of shouldEvict, it seems that a check is not necessary because nothing will change.

    final CacheInfo closedSegmentsCacheInfo = closedSegments.getCacheInfo();
    if (closedSegmentsCacheInfo.getCount() > maxCachedSegments) {
      return true;
    }

    final long size = closedSegmentsCacheInfo.getSize()
        + Optional.ofNullable(openSegment).map(LogSegment::getTotalCacheSize).orElse(0L);
    return size > maxSegmentCacheSize;
  }

Copy link
Contributor Author

Choose a reason for hiding this comment

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

For rollOpenSegment, we will close the open segement so it increases the count. For addOpenSegment, nothing will change as you mentioned.

Copy link
Member

@SzyWilliam SzyWilliam left a comment

Choose a reason for hiding this comment

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

+1 the changes look good. Thanks @szetszwo for supporting! I'll include the patch to test the latency improvements later.
Just a small question, shall we also evict the cache here?

@szetszwo
Copy link
Contributor Author

@SzyWilliam , thanks a lot for reviewing this!

..., shall we also evict the cache here?

Good catch! We should signal cacheEviction there. Just have updated the change. Please take a look.

Copy link
Member

@SzyWilliam SzyWilliam left a comment

Choose a reason for hiding this comment

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

+1 LGTM. Thanks @szetszwo again for the patch!
Also thanks @OneSizeFitsQuorum and @ivandika3 for the reviews!

@SzyWilliam SzyWilliam merged commit 0e136f3 into apache:master Sep 24, 2023
10 checks passed
symious pushed a commit to symious/ratis that referenced this pull request Mar 21, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants