szetszwo commented on PR #958:
URL: https://github.com/apache/ratis/pull/958#issuecomment-1789296758

   I wonder if it is more efficient to use `synchronized` to implement 
`ReadIndexQueue` than `ConcurrentSkipListMap`?
   ```java
     static class ReadIndexQueue {
       private final TimeoutExecutor scheduler = TimeoutExecutor.getInstance();
       private final NavigableMap<Long, CompletableFuture<Long>> sorted = new 
TreeMap<>();
       private final TimeDuration readTimeout;
   
       ReadIndexQueue(TimeDuration readTimeout) {
         this.readTimeout = readTimeout;
       }
   
       CompletableFuture<Long> add(long readIndex) {
         final CompletableFuture<Long> returned;
         final boolean create;
         synchronized (this) {
           // The same as computeIfAbsent except that it also tells if a new 
value is created.
           final CompletableFuture<Long> existing = sorted.get(readIndex);
           create = existing == null;
           if (create) {
             returned = new CompletableFuture<>();
             sorted.put(readIndex, returned);
           } else {
             returned = existing;
           }
         }
   
         if (create) {
           scheduler.onTimeout(readTimeout, () -> handleTimeout(readIndex),
               LOG, () -> "Failed to handle read timeout for index " + 
readIndex);
         }
         return returned;
       }
   
       private void handleTimeout(long readIndex) {
         final CompletableFuture<Long> removed;
         synchronized (this) {
           removed = sorted.remove(readIndex);
         }
         if (removed == null) {
           return;
         }
         removed.completeExceptionally(new ReadException("Read timeout " + 
readTimeout + " for index " + readIndex));
       }
   
   
       /** Complete all the entries less than or equal to the given applied 
index. */
       synchronized void complete(Long appliedIndex) {
         final NavigableMap<Long, CompletableFuture<Long>> headMap = 
sorted.headMap(appliedIndex, true);
         headMap.values().forEach(f -> f.complete(appliedIndex));
         headMap.clear();
       }
     }
   ```


-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to