-
Notifications
You must be signed in to change notification settings - Fork 455
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
Make garbage collecting wal and recovery files faster #5399
base: 2.1
Are you sure you want to change the base?
Conversation
Uses the same property and technique used for deleting RFiles to delete wal and recovery files. Closes apache#5397
Kicked off IT build |
Full IT build successful |
server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
Show resolved
Hide resolved
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.
Changes look good to me.
The only suggestion I have is using a separate GC_WAL_DELETE_THREADS
property.
In the current code, user table activity will actively delay WAL cleanup processing as the wals are processed at the end of a GC run cycle before flushing the metadata and root tables.
If the goal is to speed up GC of the wals and recovery files, then I don't see a reason why the WAL logs have to be handled at the end of the GC run cycle vs just always be running as a separate task thread.
GC_DELETE_THREADS
is most likely going to be set higher than needed for the WALs since it's based on file activity in the system while the upper limit of WALs should be based on the amount of tservers * tserver.wal.max.referenced
and tserver churn (dead/recovered/etc).
If we move to having the WALs GC'd in a separate thread or just pulled into a different GC process, then using a separate property would make it easier to avoid exceeding the max amount of available threads on the server.
} catch (InterruptedException e1) { | ||
log.error("{}", e1.getMessage(), e1); | ||
} | ||
return counter.get(); |
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.
The code was incrementing this deleted counter, should it still do that?
return counter.get(); | |
status.currentLog.deleted += counter.get(); | |
return counter.get(); |
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.
Yep, I forgot to increment the counter. I created it though, so I was half way there.
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.
On second look, counter is being passed to removeFile, and incremented there.
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 counter is being incrementing removeFile, but the old code used to increment status.currentLog.deleted
which may be used for display on the monitor and logging. Seems like that is no longer incremented.
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 in 7c54495
fs.deleteRecursively(path); | ||
} | ||
counter.incrementAndGet(); | ||
} catch (FileNotFoundException ex) { |
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 was wondering if we need to handle any other exception types here as an uncaught exception can bubble up and kill the thead pool. It's pretty common to catch all exceptions as a catch all.
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.
The previous code just bubbled the RuntimeExceptions up the call stack. I could call submit
instead, test the futures, and return any errors up the stack.
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 you used futures that might be better, you could wait for them to finish instead of relying on the thread pool to shutdown for completion. Either way could work, I just figured I'd mention it as I'm not too familiar with this code but I noticed we were only catching those specific exceptions so was curious if a random runtime exception would cause issues with the new thread pool.
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 to use futures in 7c54495
Added in 7c54495 |
} | ||
|
||
while (!futures.isEmpty()) { |
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.
The get() method in a future waits until its done or failed, so should be able to loop through all futures once calling get.
futures.forEach((path,future)->{
try {
future.get();
} catch (InterruptedException | ExecutionException e) {
throw new RuntimeException("Uncaught exception deleting recovery log file" + path,
e);
}
});
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 was thinking the faster I process the ones that are complete and remove from the map, the faster I'm giving back memory to the VM. I probably need to put a wait at the bottom of the loop though. If the first one the iterator returns is the last one submitted, then I would be waiting until they are all done.
} | ||
} | ||
} | ||
deleteThreadPool.shutdown(); |
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 an exception is throw the pool would not be shut down.
} | ||
deleteThreadPool.shutdown(); | ||
try { | ||
while (!deleteThreadPool.awaitTermination(1000, TimeUnit.MILLISECONDS)) { // empty |
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 remove this code now that futures are being waited on.
Uses the same property and technique used for deleting RFiles to delete wal and recovery files.
Closes #5397