You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ratis.apache.org by lj...@apache.org on 2019/02/25 19:33:26 UTC

[incubator-ratis] branch master updated: RATIS-489. Server RetryCache should not use size-based eviction. Contributed by Tsz Wo Nicholas Sze.

This is an automated email from the ASF dual-hosted git repository.

ljain pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-ratis.git


The following commit(s) were added to refs/heads/master by this push:
     new a0f98c1  RATIS-489. Server RetryCache should not use size-based eviction. Contributed by Tsz Wo Nicholas Sze.
a0f98c1 is described below

commit a0f98c1eb8d1f755433a669c0ddbff0d9601b97c
Author: Lokesh Jain <lj...@apache.org>
AuthorDate: Tue Feb 26 01:03:02 2019 +0530

    RATIS-489. Server RetryCache should not use size-based eviction. Contributed by Tsz Wo Nicholas Sze.
---
 .../java/org/apache/ratis/server/RaftServerConfigKeys.java   | 12 ------------
 .../java/org/apache/ratis/server/impl/RaftServerImpl.java    |  3 +--
 .../main/java/org/apache/ratis/server/impl/RetryCache.java   | 12 ++++--------
 .../org/apache/ratis/server/impl/RetryCacheTestUtil.java     |  4 ++--
 4 files changed, 7 insertions(+), 24 deletions(-)

diff --git a/ratis-server/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java b/ratis-server/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java
index ab8e50f..9cbdf2f 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java
@@ -17,7 +17,6 @@
  */
 package org.apache.ratis.server;
 
-import org.apache.ratis.conf.ConfUtils;
 import org.apache.ratis.conf.RaftProperties;
 import org.apache.ratis.util.SizeInBytes;
 import org.apache.ratis.util.TimeDuration;
@@ -357,17 +356,6 @@ public interface RaftServerConfigKeys {
   interface RetryCache {
     String PREFIX = RaftServerConfigKeys.PREFIX + ".retrycache";
 
-    String CAPACITY_KEY = PREFIX + ".capacity";
-    int CAPACITY_DEFAULT = 4096;
-    static int capacity(RaftProperties properties) {
-      return ConfUtils.getInt(properties::getInt, CAPACITY_KEY, CAPACITY_DEFAULT, getDefaultLog(),
-          ConfUtils.requireMin(0));
-    }
-
-    static void setCapacity(RaftProperties properties, int capacity) {
-      setInt(properties::setInt, CAPACITY_KEY, capacity);
-    }
-
     String EXPIRY_TIME_KEY = PREFIX + ".expirytime";
     TimeDuration EXPIRY_TIME_DEFAULT = TimeDuration.valueOf(60, TimeUnit.SECONDS);
     static TimeDuration expiryTime(RaftProperties properties) {
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
index 2cd8d40..494a596 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
@@ -107,9 +107,8 @@ public class RaftServerImpl implements RaftServerProtocol, RaftServerAsynchronou
   }
 
   private RetryCache initRetryCache(RaftProperties prop) {
-    final int capacity = RaftServerConfigKeys.RetryCache.capacity(prop);
     final TimeDuration expireTime = RaftServerConfigKeys.RetryCache.expiryTime(prop);
-    return new RetryCache(capacity, expireTime);
+    return new RetryCache(expireTime);
   }
 
   LogAppender newLogAppender(
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RetryCache.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RetryCache.java
index 887f39c..4ac898b 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RetryCache.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RetryCache.java
@@ -20,7 +20,6 @@ package org.apache.ratis.server.impl;
 import java.io.Closeable;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.ratis.protocol.ClientId;
 import org.apache.ratis.protocol.RaftClientReply;
@@ -35,7 +34,6 @@ import org.slf4j.LoggerFactory;
 
 public class RetryCache implements Closeable {
   static final Logger LOG = LoggerFactory.getLogger(RetryCache.class);
-  private static final int MIN_CAPACITY = 128;
 
   static class CacheKey {
     private final ClientId clientId;
@@ -153,14 +151,12 @@ public class RetryCache implements Closeable {
   private final Cache<CacheKey, CacheEntry> cache;
 
   /**
-   * @param capacity the capacity of the cache
    * @param expirationTime time for an entry to expire in milliseconds
    */
-  RetryCache(int capacity, TimeDuration expirationTime) {
-    capacity = Math.max(capacity, MIN_CAPACITY);
-    cache = CacheBuilder.newBuilder().maximumSize(capacity)
-        .expireAfterWrite(expirationTime.toLong(TimeUnit.MILLISECONDS),
-            TimeUnit.MILLISECONDS).build();
+  RetryCache(TimeDuration expirationTime) {
+    cache = CacheBuilder.newBuilder()
+        .expireAfterWrite(expirationTime.getDuration(), expirationTime.getUnit())
+        .build();
   }
 
   CacheEntry getOrCreateEntry(ClientId clientId, long callId) {
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/RetryCacheTestUtil.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/RetryCacheTestUtil.java
index 0212eab..a8a2465 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/impl/RetryCacheTestUtil.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/RetryCacheTestUtil.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -27,7 +27,7 @@ import java.util.concurrent.TimeUnit;
 
 public class RetryCacheTestUtil {
   public static RetryCache createRetryCache(){
-    return new RetryCache(5000, TimeDuration.valueOf(60, TimeUnit.SECONDS));
+    return new RetryCache(TimeDuration.valueOf(60, TimeUnit.SECONDS));
   }
 
   public static void createEntry(RetryCache cache, LogEntryProto logEntry){