You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ratis.apache.org by GitBox <gi...@apache.org> on 2022/11/13 23:34:14 UTC

[GitHub] [ratis] szetszwo opened a new pull request, #782: RATIS-1743. Memory leak in SegmentedRaftLogWorker due to metrics.

szetszwo opened a new pull request, #782:
URL: https://github.com/apache/ratis/pull/782

   See https://issues.apache.org/jira/browse/RATIS-1743


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ratis] adoroszlai commented on pull request #782: RATIS-1743. (superseded)

Posted by GitBox <gi...@apache.org>.
adoroszlai commented on PR #782:
URL: https://github.com/apache/ratis/pull/782#issuecomment-1314816431

   > this approach still cannot fix the failure of TestStorageContainerManagerHA, even with the buffers moved to Queues.
   
   I think that's because we have a similar leak in `LeaderElectionMetrics`.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ratis] szetszwo commented on a diff in pull request #782: RATIS-1743. (superseded)

Posted by GitBox <gi...@apache.org>.
szetszwo commented on code in PR #782:
URL: https://github.com/apache/ratis/pull/782#discussion_r1025720287


##########
ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java:
##########
@@ -132,29 +139,83 @@ synchronized void updateIndex(long i) {
     }
   }
 
+  static class Queues implements ToIntFunction<String> {
+    private final DataBlockingQueue<Task> taskQueue;
+    private final WriteLogTasks writeTasks = new WriteLogTasks();
+
+    Queues(String name, RaftProperties properties) {
+      final SizeInBytes queueByteLimit = RaftServerConfigKeys.Log.queueByteLimit(properties);
+      final int queueElementLimit = RaftServerConfigKeys.Log.queueElementLimit(properties);
+      this.taskQueue = new DataBlockingQueue<>(name, queueByteLimit, queueElementLimit, Task::getSerializedSize);
+    }
+
+    DataBlockingQueue<Task> getTaskQueue() {
+      return taskQueue;
+    }
+
+    WriteLogTasks getWriteTasks() {
+      return writeTasks;
+    }
+
+    @Override
+    public int applyAsInt(String type) {
+      if (type.equals(SegmentedRaftLogMetrics.RAFT_LOG_DATA_QUEUE_SIZE)) {
+        return taskQueue.getNumElements();
+      } else if (type.equals(SegmentedRaftLogMetrics.RAFT_LOG_WORKER_QUEUE_SIZE)) {
+        return writeTasks.getQueueSize();
+      }
+      throw new IllegalArgumentException("Unknown type " + type);
+    }
+
+    @Override
+    public String toString() {
+      return "#tasks=" + taskQueue.getNumElements()
+          + ", #writes=" + writeTasks.getQueueSize();
+    }
+  }
+
+  static class Buffers {
+    private final ByteBuffer writeBuffer;
+    private final Supplier<byte[]> sharedBuffer;

Review Comment:
   @adoroszlai , @SzyWilliam , FYI, just have learned from this article that Java lambda is garbage collected in a different way.  That is the reason that the `Buffers` class here does not work but the `AtomicReference` works.  See https://tanbt.medium.com/java-lambdas-and-the-garbage-collector-part-1-da934e11518a



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ratis] szetszwo commented on pull request #782: RATIS-1743. Memory leak in SegmentedRaftLogWorker due to metrics.

Posted by GitBox <gi...@apache.org>.
szetszwo commented on PR #782:
URL: https://github.com/apache/ratis/pull/782#issuecomment-1314467742

   @adoroszlai , this approach still cannot fix the failure of TestStorageContainerManagerHA, even with the buffers moved to Queues.
   
   If the byte array are created incrementally, TestStorageContainerManagerHA can pass; see #783 .


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ratis] adoroszlai commented on pull request #782: RATIS-1743. (superseded)

Posted by GitBox <gi...@apache.org>.
adoroszlai commented on PR #782:
URL: https://github.com/apache/ratis/pull/782#issuecomment-1315068634

   @szetszwo with the latest commit (74619ec) `TestStorageContainerManagerHA` is passing for me.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ratis] szetszwo commented on pull request #782: RATIS-1743. Memory leak in SegmentedRaftLogWorker due to metrics.

Posted by GitBox <gi...@apache.org>.
szetszwo commented on PR #782:
URL: https://github.com/apache/ratis/pull/782#issuecomment-1316226635

   Tried to fix [LeaderElectionMetrics](https://github.com/apache/ratis/pull/782/commits/e5f08eb1fabf35b3d981620701d0a302e260dedd) but TestStorageContainerManagerHA still fails.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ratis] szetszwo closed pull request #782: RATIS-1743. Memory leak in SegmentedRaftLogWorker due to metrics.

Posted by GitBox <gi...@apache.org>.
szetszwo closed pull request #782: RATIS-1743. Memory leak in SegmentedRaftLogWorker due to metrics.
URL: https://github.com/apache/ratis/pull/782


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ratis] SzyWilliam commented on a diff in pull request #782: RATIS-1743. (superseded)

Posted by GitBox <gi...@apache.org>.
SzyWilliam commented on code in PR #782:
URL: https://github.com/apache/ratis/pull/782#discussion_r1025879696


##########
ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java:
##########
@@ -132,29 +139,83 @@ synchronized void updateIndex(long i) {
     }
   }
 
+  static class Queues implements ToIntFunction<String> {
+    private final DataBlockingQueue<Task> taskQueue;
+    private final WriteLogTasks writeTasks = new WriteLogTasks();
+
+    Queues(String name, RaftProperties properties) {
+      final SizeInBytes queueByteLimit = RaftServerConfigKeys.Log.queueByteLimit(properties);
+      final int queueElementLimit = RaftServerConfigKeys.Log.queueElementLimit(properties);
+      this.taskQueue = new DataBlockingQueue<>(name, queueByteLimit, queueElementLimit, Task::getSerializedSize);
+    }
+
+    DataBlockingQueue<Task> getTaskQueue() {
+      return taskQueue;
+    }
+
+    WriteLogTasks getWriteTasks() {
+      return writeTasks;
+    }
+
+    @Override
+    public int applyAsInt(String type) {
+      if (type.equals(SegmentedRaftLogMetrics.RAFT_LOG_DATA_QUEUE_SIZE)) {
+        return taskQueue.getNumElements();
+      } else if (type.equals(SegmentedRaftLogMetrics.RAFT_LOG_WORKER_QUEUE_SIZE)) {
+        return writeTasks.getQueueSize();
+      }
+      throw new IllegalArgumentException("Unknown type " + type);
+    }
+
+    @Override
+    public String toString() {
+      return "#tasks=" + taskQueue.getNumElements()
+          + ", #writes=" + writeTasks.getQueueSize();
+    }
+  }
+
+  static class Buffers {
+    private final ByteBuffer writeBuffer;
+    private final Supplier<byte[]> sharedBuffer;

Review Comment:
   @szetszwo Thanks for sharing! Learned a lot from this article.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ratis] szetszwo commented on a diff in pull request #782: RATIS-1743. Memory leak in SegmentedRaftLogWorker due to metrics.

Posted by GitBox <gi...@apache.org>.
szetszwo commented on code in PR #782:
URL: https://github.com/apache/ratis/pull/782#discussion_r1021846956


##########
ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java:
##########
@@ -208,8 +243,7 @@ synchronized void updateIndex(long i) {
     this.workerThread = new Thread(this::run, name);
 
     // Server Id can be null in unit tests
-    metricRegistry.addDataQueueSizeGauge(queue);
-    metricRegistry.addLogWorkerQueueSizeGauge(writeTasks.q);
+    metricRegistry.addQueueSizeGauges(queues);
     metricRegistry.addFlushBatchSizeGauge(() -> (Gauge<Integer>) () -> flushBatchSize);

Review Comment:
   @adoroszlai ,  good catch.  Just have changed it to use AtomicInteger.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ratis] adoroszlai commented on a diff in pull request #782: RATIS-1743. Memory leak in SegmentedRaftLogWorker due to metrics.

Posted by GitBox <gi...@apache.org>.
adoroszlai commented on code in PR #782:
URL: https://github.com/apache/ratis/pull/782#discussion_r1021480921


##########
ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java:
##########
@@ -208,8 +243,7 @@ synchronized void updateIndex(long i) {
     this.workerThread = new Thread(this::run, name);
 
     // Server Id can be null in unit tests
-    metricRegistry.addDataQueueSizeGauge(queue);
-    metricRegistry.addLogWorkerQueueSizeGauge(writeTasks.q);
+    metricRegistry.addQueueSizeGauges(queues);
     metricRegistry.addFlushBatchSizeGauge(() -> (Gauge<Integer>) () -> flushBatchSize);

Review Comment:
   The `syncBatchSize` gauge this creates still holds `SegmentedRaftLogWorker` after `close()`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ratis] adoroszlai commented on a diff in pull request #782: RATIS-1743. Memory leak in SegmentedRaftLogWorker due to metrics.

Posted by GitBox <gi...@apache.org>.
adoroszlai commented on code in PR #782:
URL: https://github.com/apache/ratis/pull/782#discussion_r1021941234


##########
ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java:
##########
@@ -208,8 +243,7 @@ synchronized void updateIndex(long i) {
     this.workerThread = new Thread(this::run, name);
 
     // Server Id can be null in unit tests
-    metricRegistry.addDataQueueSizeGauge(queue);
-    metricRegistry.addLogWorkerQueueSizeGauge(writeTasks.q);
+    metricRegistry.addQueueSizeGauges(queues);
     metricRegistry.addFlushBatchSizeGauge(() -> (Gauge<Integer>) () -> flushBatchSize);

Review Comment:
   The following POC resolves this specific leak (please feel free to adapt):
   
   ```diff
   diff --git ratis-server/src/main/java/org/apache/ratis/server/metrics/SegmentedRaftLogMetrics.java ratis-server/src/main/java/org/apache/ratis/server/metrics/SegmentedRaftLogMetrics.java
   index 50d582ac..d213563b 100644
   --- ratis-server/src/main/java/org/apache/ratis/server/metrics/SegmentedRaftLogMetrics.java
   +++ ratis-server/src/main/java/org/apache/ratis/server/metrics/SegmentedRaftLogMetrics.java
   @@ -24,6 +24,7 @@ import com.codahale.metrics.Timer;
    import org.apache.ratis.protocol.RaftGroupMemberId;
    import org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogCache;
    
   +import java.util.concurrent.atomic.AtomicInteger;
    import java.util.concurrent.atomic.AtomicReference;
    import java.util.function.ToIntFunction;
    
   @@ -113,8 +114,10 @@ public class SegmentedRaftLogMetrics extends RaftLogMetricsBase {
        });
      }
    
   -  public void addFlushBatchSizeGauge(MetricRegistry.MetricSupplier<Gauge> supplier) {
   -    registry.gauge(RAFT_LOG_SYNC_BATCH_SIZE, supplier);
   +  public AtomicInteger addFlushBatchSizeGauge() {
   +    MutableIntegerGaugeSupplier migs = new MutableIntegerGaugeSupplier();
   +    registry.gauge(RAFT_LOG_SYNC_BATCH_SIZE, migs);
   +    return migs.asAtomicInteger();
      }
    
      private Timer getTimer(String timerName) {
   @@ -181,4 +184,23 @@ public class SegmentedRaftLogMetrics extends RaftLogMetricsBase {
      public void onStateMachineDataReadTimeout() {
        registry.counter(RAFT_LOG_STATEMACHINE_DATA_READ_TIMEOUT_COUNT).inc();
      }
   +
   +  public static class MutableIntegerGaugeSupplier implements MetricRegistry.MetricSupplier<Gauge>, Gauge<Integer> {
   +    private final AtomicInteger value = new AtomicInteger();
   +
   +    public AtomicInteger asAtomicInteger() {
   +      return value;
   +    }
   +
   +    @Override
   +    public Gauge<Integer> newMetric() {
   +      return this;
   +    }
   +
   +    @Override
   +    public Integer getValue() {
   +      return value.get();
   +    }
   +  }
   +
    }
   diff --git ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java
   index a5b12eb9..fa4516e1 100644
   --- ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java
   +++ ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java
   @@ -211,7 +211,7 @@ class SegmentedRaftLogWorker {
      private final long segmentMaxSize;
      private final long preallocatedSize;
      private final RaftServer.Division server;
   -  private final AtomicInteger flushBatchSize = new AtomicInteger();
   +  private final AtomicInteger flushBatchSize;
    
      private final boolean asyncFlush;
      private final boolean unsafeFlush;
   @@ -243,7 +243,7 @@ class SegmentedRaftLogWorker {
    
        // Server Id can be null in unit tests
        metricRegistry.addQueueSizeGauges(queues);
   -    metricRegistry.addFlushBatchSizeGauge(() -> flushBatchSize::get);
   +    flushBatchSize = metricRegistry.addFlushBatchSizeGauge();
        this.logFlushTimer = metricRegistry.getFlushTimer();
        this.raftLogSyncTimer = metricRegistry.getRaftLogSyncTimer();
        this.raftLogQueueingTimer = metricRegistry.getRaftLogQueueTimer(
   ```
   
   With this, we still have a leak due to `lastLeaderElapsedTime` gauge in `LeaderElectionMetrics` (at least).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ratis] adoroszlai commented on a diff in pull request #782: RATIS-1743. Memory leak in SegmentedRaftLogWorker due to metrics.

Posted by GitBox <gi...@apache.org>.
adoroszlai commented on code in PR #782:
URL: https://github.com/apache/ratis/pull/782#discussion_r1021909823


##########
ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java:
##########
@@ -208,8 +243,7 @@ synchronized void updateIndex(long i) {
     this.workerThread = new Thread(this::run, name);
 
     // Server Id can be null in unit tests
-    metricRegistry.addDataQueueSizeGauge(queue);
-    metricRegistry.addLogWorkerQueueSizeGauge(writeTasks.q);
+    metricRegistry.addQueueSizeGauges(queues);
     metricRegistry.addFlushBatchSizeGauge(() -> (Gauge<Integer>) () -> flushBatchSize);

Review Comment:
   Unfortunately the lambda still has a reference to the enclosing `SegmentedRaftLogWorker` instance.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ratis] szetszwo commented on pull request #782: RATIS-1743. (superseded)

Posted by GitBox <gi...@apache.org>.
szetszwo commented on PR #782:
URL: https://github.com/apache/ratis/pull/782#issuecomment-1315700875

   > ...`TestStorageContainerManagerHA` is passing for me.
   
   @adoroszlai , You are right.  Let's merge this then.  I somehow thought that this was not able to fix the problem.  Thanks a lot!
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ratis] szetszwo commented on a diff in pull request #782: RATIS-1743. Memory leak in SegmentedRaftLogWorker due to metrics.

Posted by GitBox <gi...@apache.org>.
szetszwo commented on code in PR #782:
URL: https://github.com/apache/ratis/pull/782#discussion_r1021957351


##########
ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java:
##########
@@ -208,8 +243,7 @@ synchronized void updateIndex(long i) {
     this.workerThread = new Thread(this::run, name);
 
     // Server Id can be null in unit tests
-    metricRegistry.addDataQueueSizeGauge(queue);
-    metricRegistry.addLogWorkerQueueSizeGauge(writeTasks.q);
+    metricRegistry.addQueueSizeGauges(queues);
     metricRegistry.addFlushBatchSizeGauge(() -> (Gauge<Integer>) () -> flushBatchSize);

Review Comment:
   @adoroszlai , thanks for catching it.  I mean to pass the AtomicInteger and use lambda in SegmentedRaftLogMetrics.  Then, it won't have a reference to SegmentedRaftLogWorker.  Let me try it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ratis] adoroszlai commented on a diff in pull request #782: RATIS-1743. Memory leak in SegmentedRaftLogWorker due to metrics.

Posted by GitBox <gi...@apache.org>.
adoroszlai commented on code in PR #782:
URL: https://github.com/apache/ratis/pull/782#discussion_r1021480921


##########
ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java:
##########
@@ -208,8 +243,7 @@ synchronized void updateIndex(long i) {
     this.workerThread = new Thread(this::run, name);
 
     // Server Id can be null in unit tests
-    metricRegistry.addDataQueueSizeGauge(queue);
-    metricRegistry.addLogWorkerQueueSizeGauge(writeTasks.q);
+    metricRegistry.addQueueSizeGauges(queues);
     metricRegistry.addFlushBatchSizeGauge(() -> (Gauge<Integer>) () -> flushBatchSize);

Review Comment:
   The `syncBatchSize` gauge (created in `addFlushBatchSizeGauge`) still holds `SegmentedRaftLogWorker` after `close()`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ratis] szetszwo commented on a diff in pull request #782: RATIS-1743. Memory leak in SegmentedRaftLogWorker due to metrics.

Posted by GitBox <gi...@apache.org>.
szetszwo commented on code in PR #782:
URL: https://github.com/apache/ratis/pull/782#discussion_r1023449543


##########
ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java:
##########
@@ -132,29 +139,78 @@ synchronized void updateIndex(long i) {
     }
   }
 
+  static class Queues implements ToIntFunction<String> {
+    private final DataBlockingQueue<Task> taskQueue;
+    private final WriteLogTasks writeTasks = new WriteLogTasks();
+
+    private final ByteBuffer writeBuffer;
+    private final Supplier<byte[]> sharedBuffer;
+
+    Queues(String name, RaftProperties properties) {
+      final SizeInBytes queueByteLimit = RaftServerConfigKeys.Log.queueByteLimit(properties);
+      final int queueElementLimit = RaftServerConfigKeys.Log.queueElementLimit(properties);
+      this.taskQueue = new DataBlockingQueue<>(name, queueByteLimit, queueElementLimit, Task::getSerializedSize);
+
+      final int bufferSize = RaftServerConfigKeys.Log.writeBufferSize(properties).getSizeInt();
+      this.writeBuffer = ByteBuffer.allocateDirect(bufferSize);
+      final int logEntryLimit = RaftServerConfigKeys.Log.Appender.bufferByteLimit(properties).getSizeInt();
+      // 4 bytes (serialized size) + logEntryLimit + 4 bytes (checksum)
+      this.sharedBuffer = () -> new byte[logEntryLimit + 8];

Review Comment:
   @adoroszlai , The buffer is not shared so that TestStorageContainerManagerHA can pass.  If we use `MemoizedSupplier` as below, it will fail.
   ```java
         this.sharedBuffer = MemoizedSupplier.valueOf(() -> new byte[logEntryLimit + 8]);
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ratis] szetszwo commented on pull request #782: RATIS-1743. Memory leak in SegmentedRaftLogWorker due to metrics.

Posted by GitBox <gi...@apache.org>.
szetszwo commented on PR #782:
URL: https://github.com/apache/ratis/pull/782#issuecomment-1316311298

   This was superseded by #784.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org