You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2018/12/06 09:28:43 UTC

[GitHub] QiuMM closed pull request #6540: add close method in Cache interface

QiuMM closed pull request #6540: add close method in Cache interface
URL: https://github.com/apache/incubator-druid/pull/6540
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/extensions-contrib/redis-cache/src/main/java/org/apache/druid/client/cache/RedisCache.java b/extensions-contrib/redis-cache/src/main/java/org/apache/druid/client/cache/RedisCache.java
index bf9fb42c400..817a2175f25 100644
--- a/extensions-contrib/redis-cache/src/main/java/org/apache/druid/client/cache/RedisCache.java
+++ b/extensions-contrib/redis-cache/src/main/java/org/apache/druid/client/cache/RedisCache.java
@@ -21,6 +21,7 @@
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
+import org.apache.druid.java.util.common.lifecycle.LifecycleStop;
 import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.java.util.emitter.service.ServiceEmitter;
 import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
@@ -29,6 +30,7 @@
 import redis.clients.jedis.JedisPoolConfig;
 import redis.clients.jedis.exceptions.JedisException;
 
+import java.io.IOException;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -147,6 +149,13 @@ public void close(String namespace)
     // no resources to cleanup
   }
 
+  @Override
+  @LifecycleStop
+  public void close() throws IOException
+  {
+    pool.close();
+  }
+
   @Override
   public CacheStats getStats()
   {
diff --git a/server/src/main/java/org/apache/druid/client/cache/Cache.java b/server/src/main/java/org/apache/druid/client/cache/Cache.java
index 725a880a4da..baeea8f042f 100644
--- a/server/src/main/java/org/apache/druid/client/cache/Cache.java
+++ b/server/src/main/java/org/apache/druid/client/cache/Cache.java
@@ -24,13 +24,14 @@
 import org.apache.druid.java.util.emitter.service.ServiceEmitter;
 
 import javax.annotation.Nullable;
+import java.io.Closeable;
 import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.Map;
 
 /**
  */
-public interface Cache
+public interface Cache extends Closeable
 {
   @Nullable
   byte[] get(NamedKey key);
diff --git a/server/src/main/java/org/apache/druid/client/cache/CaffeineCache.java b/server/src/main/java/org/apache/druid/client/cache/CaffeineCache.java
index ec18cd2d7ab..fa33e083ee8 100644
--- a/server/src/main/java/org/apache/druid/client/cache/CaffeineCache.java
+++ b/server/src/main/java/org/apache/druid/client/cache/CaffeineCache.java
@@ -28,10 +28,12 @@
 import net.jpountz.lz4.LZ4Compressor;
 import net.jpountz.lz4.LZ4Factory;
 import net.jpountz.lz4.LZ4FastDecompressor;
+import org.apache.druid.java.util.common.lifecycle.LifecycleStop;
 import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.java.util.emitter.service.ServiceEmitter;
 import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
 
+import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Map;
 import java.util.OptionalLong;
@@ -114,6 +116,13 @@ public void close(String namespace)
     }
   }
 
+  @Override
+  @LifecycleStop
+  public void close() throws IOException
+  {
+    cache.cleanUp();
+  }
+
   @Override
   public org.apache.druid.client.cache.CacheStats getStats()
   {
diff --git a/server/src/main/java/org/apache/druid/client/cache/HybridCache.java b/server/src/main/java/org/apache/druid/client/cache/HybridCache.java
index d940f716618..98b75f7affd 100644
--- a/server/src/main/java/org/apache/druid/client/cache/HybridCache.java
+++ b/server/src/main/java/org/apache/druid/client/cache/HybridCache.java
@@ -21,10 +21,13 @@
 
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
+import org.apache.druid.java.util.common.lifecycle.LifecycleStop;
 import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.java.util.emitter.service.ServiceEmitter;
 
 import javax.annotation.Nullable;
+import java.io.Closeable;
+import java.io.IOException;
 import java.util.Collections;
 import java.util.Map;
 import java.util.Set;
@@ -127,8 +130,35 @@ public void put(NamedKey key, byte[] value)
   @Override
   public void close(String namespace)
   {
-    level1.close(namespace);
-    level2.close(namespace);
+    Throwable t = null;
+    try {
+      level1.close(namespace);
+    }
+    catch (Throwable t1) {
+      t = t1;
+      throw t1;
+    }
+    finally {
+      if (t != null) {
+        try {
+          level2.close(namespace);
+        }
+        catch (Throwable t2) {
+          t.addSuppressed(t2);
+        }
+      } else {
+        level2.close(namespace);
+      }
+    }
+  }
+
+  @Override
+  @LifecycleStop
+  public void close() throws IOException
+  {
+    try (Closeable closeable1 = level1; Closeable closeable2 = level2) {
+      // Just for closing
+    }
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/druid/client/cache/MapCache.java b/server/src/main/java/org/apache/druid/client/cache/MapCache.java
index e05c35a515c..509dc16fbce 100644
--- a/server/src/main/java/org/apache/druid/client/cache/MapCache.java
+++ b/server/src/main/java/org/apache/druid/client/cache/MapCache.java
@@ -21,8 +21,10 @@
 
 import com.google.common.collect.Maps;
 import com.google.common.primitives.Ints;
+import org.apache.druid.java.util.common.lifecycle.LifecycleStop;
 import org.apache.druid.java.util.emitter.service.ServiceEmitter;
 
+import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -144,6 +146,15 @@ public void close(String namespace)
     }
   }
 
+  @Override
+  @LifecycleStop
+  public void close() throws IOException
+  {
+    baseMap.clear();
+    byteCountingLRUMap.clear();
+    namespaceId.clear();
+  }
+
   private byte[] getNamespaceId(final String identifier)
   {
     synchronized (namespaceId) {
diff --git a/server/src/main/java/org/apache/druid/client/cache/MemcachedCache.java b/server/src/main/java/org/apache/druid/client/cache/MemcachedCache.java
index a3fd26caf07..9950d6a89b8 100644
--- a/server/src/main/java/org/apache/druid/client/cache/MemcachedCache.java
+++ b/server/src/main/java/org/apache/druid/client/cache/MemcachedCache.java
@@ -46,6 +46,7 @@
 import org.apache.druid.collections.ResourceHolder;
 import org.apache.druid.collections.StupidResourceHolder;
 import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.lifecycle.LifecycleStop;
 import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.java.util.emitter.service.ServiceEmitter;
 import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
@@ -596,6 +597,13 @@ public void close(String namespace)
     // no resources to cleanup
   }
 
+  @Override
+  @LifecycleStop
+  public void close() throws IOException
+  {
+    monitor.stop();
+  }
+
   public static final int MAX_PREFIX_LENGTH =
       MemcachedClientIF.MAX_KEY_LENGTH
       - 40 // length of namespace hash
diff --git a/server/src/test/java/org/apache/druid/client/CachingQueryRunnerTest.java b/server/src/test/java/org/apache/druid/client/CachingQueryRunnerTest.java
index b20edd804a0..7ecc7cde3c9 100644
--- a/server/src/test/java/org/apache/druid/client/CachingQueryRunnerTest.java
+++ b/server/src/test/java/org/apache/druid/client/CachingQueryRunnerTest.java
@@ -252,6 +252,11 @@ public void close(String namespace)
       {
       }
 
+      @Override
+      public void close() throws IOException
+      {
+      }
+
       @Override
       public CacheStats getStats()
       {


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org