ratis icon indicating copy to clipboard operation
ratis copied to clipboard

Testing zero-copy bugs fixes (not for merging)

Open szetszwo opened this issue 1 year ago • 2 comments

This fix will be split into multiple JIRAs: RATIS-2164, RATIS-2151, RATIS-2173

The following are the bugs found so far:

  1. LeakDetector: asserted allLeaks is non-empty but printed "allLeaks.size = 0"
    • Another bug: Tracks are added to the set before calling retain. Without calling retain at all, it is not a leak.
  2. SimpleTracing and AdvancedTracing: the methods should be synchronized.
    • Minor presentation problem: AdvancedTracing should have a single track list instead of retainsTraces and releaseTraces.
  3. GrpcClientProtocolService.UnorderedRequestStreamObserver.processClientRequest(..) should use try-finally.
  4. GrpcLogAppender.appendLog(..) calls release() incorrectly for exception.
  5. LogAppenderDefault.sendAppendEntriesWithRetries(..) calls release() incorrectly for exception.
  6. LogSegment cache can release an entry multiple times.
  7. LogSegment.loadCache(..) should call retain() for cache hit.
  8. SegmentedRaftLog.retainLog(..): between getting the entry and calling retain(), 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 #1153
  9. SegmentedRaftLog.retainEntryWithData(..) should release for exception.
  10. Test bug: the log entries stored in SimpleStateMachine4Testing can be released.
  11. LogSegment: New entries can be added after EntryCache is closed.
  12. MemoryRaftLog has similar problems as in SegmentedRaftLog.
  13. SegmentedRaftLogWorker should clean up unfinished tasks in the queue after stopped running.

szetszwo avatar Sep 28 '24 16:09 szetszwo

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.

szetszwo avatar Oct 07 '24 11:10 szetszwo

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:

  1. The current JIRA RATIS-2164 for fixing LeakDetector. (I will leave this PR as-is and submit another PR.)
  2. RATIS-2151 for fixing gRPC.
  3. RATIS-2159 for fixing other non-gRPC cases.

I will see if (2) and (3) needed to be further split.

BTW, we should move LeakDetector enabling from MiniRaftClusterWithGrpc to MiniRaftCluster. It will be able to detect more failures.

szetszwo avatar Oct 07 '24 19:10 szetszwo