-
Notifications
You must be signed in to change notification settings - Fork 415
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
Testing zero-copy bugs fixes (not for merging) #1156
base: master
Are you sure you want to change the base?
Conversation
@@ -55,13 +59,56 @@ | |||
*/ | |||
public class LeakDetector { | |||
private static final Logger LOG = LoggerFactory.getLogger(LeakDetector.class); | |||
|
|||
private static class LeakTrackerSet { | |||
private final Set<LeakTracker> set = Collections.newSetFromMap(new ConcurrentHashMap<>()); |
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.
All methods are synchronized, do we still need to use ConcurrentHashMap?
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.
You are right! We don't need ConcurrentHashMap
anymore.
@@ -46,6 +47,8 @@ public class DataBlockingQueue<E> extends DataQueue<E> { | |||
private final Condition notFull = lock.newCondition(); | |||
private final Condition notEmpty = lock.newCondition(); | |||
|
|||
private boolean closed = 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.
Why not use atomic?
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.
There is already a lock
.
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 see. :)
@@ -46,6 +47,8 @@ public class DataBlockingQueue<E> extends DataQueue<E> { | |||
private final Condition notFull = lock.newCondition(); | |||
private final Condition notEmpty = lock.newCondition(); | |||
|
|||
private boolean closed = 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 see. :)
Finally, it is able to pass all the tests (with a few retries). Note that there are probably some other zero copy bugs. Will fix them separately. |
This can pass all the tests (with a few retries). Since this change is quite big (56kB) and non-trivial, I will split this to a few JIRAs:
I will see if (2) and (3) needed to be further split. BTW, we should move |
This fix will be split into multiple JIRAs: RATIS-2164, RATIS-2151, RATIS-2173
The following are the bugs found so far:
LeakDetector
: assertedallLeaks
is non-empty but printed "allLeaks.size = 0"retain
. Without callingretain
at all, it is not a leak.SimpleTracing
andAdvancedTracing
: the methods should be synchronized.AdvancedTracing
should have a single track list instead ofretainsTraces
andreleaseTraces
.GrpcClientProtocolService.UnorderedRequestStreamObserver.processClientRequest(..)
should use try-finally.GrpcLogAppender.appendLog(..)
callsrelease()
incorrectly for exception.LogAppenderDefault.sendAppendEntriesWithRetries(..)
callsrelease()
incorrectly for exception.LogSegment
cache can release an entry multiple times.LogSegment.loadCache(..)
should callretain()
for cache hit.SegmentedRaftLog.retainLog(..)
: between getting the entry and callingretain()
, the entry can be released. The "fail to retain" exception, if there is any, can be ignored since It is the same as a cache miss. See RATIS-2159. TestRaftWithSimulatedRpc could "fail to retain". #1153SegmentedRaftLog.retainEntryWithData(..)
should release for exception.SimpleStateMachine4Testing
can be released.LogSegment
: New entries can be added after EntryCache is closed.MemoryRaftLog
has similar problems as inSegmentedRaftLog
.SegmentedRaftLogWorker
should clean up unfinished tasks in the queue after stopped running.