You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by gi...@apache.org on 2019/04/17 15:05:03 UTC

[incubator-druid] branch master updated: Java 9 compatible cleaner operations (#7487)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 4322ce3  Java 9 compatible cleaner operations (#7487)
4322ce3 is described below

commit 4322ce33032434f55e8b104cd0e3f617a2a991b6
Author: Xavier Léauté <xv...@users.noreply.github.com>
AuthorDate: Wed Apr 17 08:04:52 2019 -0700

    Java 9 compatible cleaner operations (#7487)
    
    Java 9 removed support for sun.misc.Cleaner in favor of
    java.lang.ref.Cleaner. This change adds a thin abstraction to switch
    between Cleaner implementations based on JDK version at runtime
---
 .../ReferenceCountingResourceHolder.java           |   6 +-
 .../org/apache/druid/collections/StupidPool.java   |  27 ++--
 .../apache/druid/java/util/common/Cleaners.java    | 160 +++++++++++++++++++++
 .../lookup/namespace/cache/CacheScheduler.java     |  20 +--
 .../OffHeapNamespaceExtractionCacheManager.java    |  26 ++--
 .../druid/client/cache/MemcacheClientPool.java     |   8 +-
 6 files changed, 204 insertions(+), 43 deletions(-)

diff --git a/core/src/main/java/org/apache/druid/collections/ReferenceCountingResourceHolder.java b/core/src/main/java/org/apache/druid/collections/ReferenceCountingResourceHolder.java
index c721b85..715921b 100644
--- a/core/src/main/java/org/apache/druid/collections/ReferenceCountingResourceHolder.java
+++ b/core/src/main/java/org/apache/druid/collections/ReferenceCountingResourceHolder.java
@@ -19,9 +19,9 @@
 
 package org.apache.druid.collections;
 
+import org.apache.druid.java.util.common.Cleaners;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.logger.Logger;
-import sun.misc.Cleaner;
 
 import java.io.Closeable;
 import java.io.IOException;
@@ -49,13 +49,13 @@ public class ReferenceCountingResourceHolder<T> implements ResourceHolder<T>
    * to be used directly.
    */
   @SuppressWarnings("unused")
-  private final Cleaner cleaner;
+  private final Object cleanable;
 
   public ReferenceCountingResourceHolder(final T object, final Closeable closer)
   {
     this.object = object;
     this.closer = closer;
-    this.cleaner = Cleaner.create(this, new CloserRunnable(object, closer, refCount));
+    this.cleanable = Cleaners.register(this, new CloserRunnable(object, closer, refCount));
   }
 
   public static <T extends Closeable> ReferenceCountingResourceHolder<T> fromCloseable(final T object)
diff --git a/core/src/main/java/org/apache/druid/collections/StupidPool.java b/core/src/main/java/org/apache/druid/collections/StupidPool.java
index 76ba790..a69a8fd 100644
--- a/core/src/main/java/org/apache/druid/collections/StupidPool.java
+++ b/core/src/main/java/org/apache/druid/collections/StupidPool.java
@@ -22,9 +22,9 @@ package org.apache.druid.collections;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Supplier;
+import org.apache.druid.java.util.common.Cleaners;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.logger.Logger;
-import sun.misc.Cleaner;
 
 import java.lang.ref.WeakReference;
 import java.util.Queue;
@@ -118,7 +118,7 @@ public class StupidPool<T> implements NonBlockingPool<T>
     ObjectLeakNotifier notifier = new ObjectLeakNotifier(this);
     // Using objectId as referent for Cleaner, because if the object itself (e. g. ByteBuffer) is leaked after taken
     // from the pool, and the ResourceHolder is not closed, Cleaner won't notify about the leak.
-    return new ObjectResourceHolder(object, objectId, Cleaner.create(objectId, notifier), notifier);
+    return new ObjectResourceHolder(object, objectId, Cleaners.register(objectId, notifier), notifier);
   }
 
   @VisibleForTesting
@@ -133,7 +133,7 @@ public class StupidPool<T> implements NonBlockingPool<T>
     return leakedObjectsCounter.get();
   }
 
-  private void tryReturnToPool(T object, ObjectId objectId, Cleaner cleaner, ObjectLeakNotifier notifier)
+  private void tryReturnToPool(T object, ObjectId objectId, Cleaners.Cleanable cleanable, ObjectLeakNotifier notifier)
   {
     long currentPoolSize;
     do {
@@ -142,7 +142,8 @@ public class StupidPool<T> implements NonBlockingPool<T>
         notifier.disable();
         // Effectively does nothing, because notifier is disabled above. The purpose of this call is to deregister the
         // cleaner from the internal global linked list of all cleaners in the JVM, and let it be reclaimed itself.
-        cleaner.clean();
+        cleanable.clean();
+
         // Important to use the objectId after notifier.disable() (in the logging statement below), otherwise VM may
         // already decide that the objectId is unreachable and run Cleaner before notifier.disable(), that would be
         // reported as a false-positive "leak". Ideally reachabilityFence(objectId) should be inserted here.
@@ -150,8 +151,8 @@ public class StupidPool<T> implements NonBlockingPool<T>
         return;
       }
     } while (!poolSize.compareAndSet(currentPoolSize, currentPoolSize + 1));
-    if (!objects.offer(new ObjectResourceHolder(object, objectId, cleaner, notifier))) {
-      impossibleOffsetFailed(object, objectId, cleaner, notifier);
+    if (!objects.offer(new ObjectResourceHolder(object, objectId, cleanable, notifier))) {
+      impossibleOffsetFailed(object, objectId, cleanable, notifier);
     }
   }
 
@@ -159,13 +160,13 @@ public class StupidPool<T> implements NonBlockingPool<T>
    * This should be impossible, because {@link ConcurrentLinkedQueue#offer(Object)} event don't have `return false;` in
    * it's body in OpenJDK 8.
    */
-  private void impossibleOffsetFailed(T object, ObjectId objectId, Cleaner cleaner, ObjectLeakNotifier notifier)
+  private void impossibleOffsetFailed(T object, ObjectId objectId, Cleaners.Cleanable cleanable, ObjectLeakNotifier notifier)
   {
     poolSize.decrementAndGet();
     notifier.disable();
     // Effectively does nothing, because notifier is disabled above. The purpose of this call is to deregister the
     // cleaner from the internal global linked list of all cleaners in the JVM, and let it be reclaimed itself.
-    cleaner.clean();
+    cleanable.clean();
     log.error(
         new ISE("Queue offer failed"),
         "Could not offer object [%s] back into the queue, objectId [%s]",
@@ -178,19 +179,19 @@ public class StupidPool<T> implements NonBlockingPool<T>
   {
     private final AtomicReference<T> objectRef;
     private ObjectId objectId;
-    private Cleaner cleaner;
+    private Cleaners.Cleanable cleanable;
     private ObjectLeakNotifier notifier;
 
     ObjectResourceHolder(
         final T object,
         final ObjectId objectId,
-        final Cleaner cleaner,
+        final Cleaners.Cleanable cleanable,
         final ObjectLeakNotifier notifier
     )
     {
       this.objectRef = new AtomicReference<>(object);
       this.objectId = objectId;
-      this.cleaner = cleaner;
+      this.cleanable = cleanable;
       this.notifier = notifier;
     }
 
@@ -213,7 +214,7 @@ public class StupidPool<T> implements NonBlockingPool<T>
       final T object = objectRef.get();
       if (object != null && objectRef.compareAndSet(object, null)) {
         try {
-          tryReturnToPool(object, objectId, cleaner, notifier);
+          tryReturnToPool(object, objectId, cleanable, notifier);
         }
         finally {
           // Need to null reference to objectId because if ObjectResourceHolder is closed, but leaked, this reference
@@ -221,7 +222,7 @@ public class StupidPool<T> implements NonBlockingPool<T>
           // again.
           objectId = null;
           // Nulling cleaner and notifier is not strictly needed, but harmless for sure.
-          cleaner = null;
+          cleanable = null;
           notifier = null;
         }
       }
diff --git a/core/src/main/java/org/apache/druid/java/util/common/Cleaners.java b/core/src/main/java/org/apache/druid/java/util/common/Cleaners.java
new file mode 100644
index 0000000..2de6a3e
--- /dev/null
+++ b/core/src/main/java/org/apache/druid/java/util/common/Cleaners.java
@@ -0,0 +1,160 @@
+/*
+ * 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
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.java.util.common;
+
+import org.apache.druid.utils.JvmUtils;
+
+import java.lang.invoke.MethodHandle;
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.MethodType;
+
+public class Cleaners
+{
+  public interface Cleanable
+  {
+    void clean();
+  }
+
+  public interface Cleaner
+  {
+    Cleanable register(Object object, Runnable runnable);
+  }
+
+  private static final Cleaner CLEANER;
+  private static final RuntimeException CLEANER_NOT_SUPPORTED_EXCEPTION;
+
+  static {
+    Cleaners.Cleaner cleaner = null;
+    RuntimeException exception = null;
+    try {
+      cleaner = takeMeToTheCleaners();
+    }
+    catch (RuntimeException e) {
+      exception = e;
+    }
+    if (cleaner != null) {
+      CLEANER = cleaner;
+      CLEANER_NOT_SUPPORTED_EXCEPTION = null;
+    } else {
+      CLEANER = null;
+      CLEANER_NOT_SUPPORTED_EXCEPTION = exception;
+    }
+  }
+
+  private static Cleaner takeMeToTheCleaners()
+  {
+    final MethodHandles.Lookup lookup = MethodHandles.lookup();
+    try {
+      if (JvmUtils.isIsJava9Compatible()) {
+        return lookupCleanerJava9(lookup);
+      } else {
+        return lookupCleanerJava8(lookup);
+      }
+    }
+    catch (ReflectiveOperationException | RuntimeException e) {
+      throw new UnsupportedOperationException("Cleaning is not support on this platform, because internal " +
+                                              "Java APIs are not compatible with this Druid version", e);
+    }
+  }
+
+  private static Cleaner lookupCleanerJava9(MethodHandles.Lookup lookup) throws ReflectiveOperationException
+  {
+    Class<?> cleaner = Class.forName("java.lang.ref.Cleaner");
+    Class<?> cleanable = Class.forName("java.lang.ref.Cleaner$Cleanable");
+
+    MethodHandle create = lookup.findStatic(cleaner, "create", MethodType.methodType(cleaner));
+
+    Object theCleaner;
+    try {
+      theCleaner = create.invoke();
+    }
+    catch (Throwable t) {
+      throw new RuntimeException("Unable to create cleaner", t);
+    }
+
+    MethodHandle register = lookup.findVirtual(
+        cleaner,
+        "register",
+        MethodType.methodType(cleanable, Object.class, Runnable.class)
+    ).bindTo(theCleaner);
+
+    MethodHandle clean = lookup.findVirtual(cleanable, "clean", MethodType.methodType(void.class));
+
+    return new CleanerImpl(register, clean);
+  }
+
+  private static Cleaner lookupCleanerJava8(MethodHandles.Lookup lookup) throws ReflectiveOperationException
+  {
+    Class<?> cleaner = Class.forName("sun.misc.Cleaner");
+    MethodHandle register = lookup.findStatic(
+        cleaner,
+        "create",
+        MethodType.methodType(cleaner, Object.class, Runnable.class)
+    );
+
+    MethodHandle clean = lookup.findVirtual(cleaner, "clean", MethodType.methodType(void.class));
+    return new CleanerImpl(register, clean);
+  }
+
+  public static Cleanable register(Object object, Runnable runnable)
+  {
+    if (CLEANER == null) {
+      throw new UnsupportedOperationException(CLEANER_NOT_SUPPORTED_EXCEPTION);
+    }
+
+    return CLEANER.register(object, runnable);
+  }
+
+  private static class CleanerImpl implements Cleaner
+  {
+    private final MethodHandle register;
+    private final MethodHandle clean;
+
+    private CleanerImpl(MethodHandle register, MethodHandle clean)
+    {
+      this.register = register;
+      this.clean = clean;
+    }
+
+    @Override
+    public Cleanable register(Object object, Runnable runnable)
+    {
+      try {
+        Object cleanable = (Object) register.invoke(object, runnable);
+        return createCleanable(clean, cleanable);
+      }
+      catch (Throwable t) {
+        throw new RuntimeException("Unable to register cleaning action", t);
+      }
+    }
+
+    private static Cleanable createCleanable(MethodHandle clean, Object cleanable)
+    {
+      return () -> {
+        try {
+          clean.invoke(cleanable);
+        }
+        catch (Throwable t) {
+          throw new RuntimeException("Unable to run cleaning action", t);
+        }
+      };
+    }
+  }
+}
diff --git a/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/server/lookup/namespace/cache/CacheScheduler.java b/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/server/lookup/namespace/cache/CacheScheduler.java
index f8a965d..fb2e229 100644
--- a/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/server/lookup/namespace/cache/CacheScheduler.java
+++ b/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/server/lookup/namespace/cache/CacheScheduler.java
@@ -24,6 +24,7 @@ import com.google.common.base.Throwables;
 import com.google.inject.Inject;
 import org.apache.druid.concurrent.ConcurrentAwaitableCounter;
 import org.apache.druid.guice.LazySingleton;
+import org.apache.druid.java.util.common.Cleaners;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.common.logger.Logger;
@@ -31,7 +32,6 @@ import org.apache.druid.java.util.emitter.service.ServiceEmitter;
 import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
 import org.apache.druid.query.lookup.namespace.CacheGenerator;
 import org.apache.druid.query.lookup.namespace.ExtractionNamespace;
-import sun.misc.Cleaner;
 
 import javax.annotation.Nullable;
 import java.util.IdentityHashMap;
@@ -138,8 +138,8 @@ public final class CacheScheduler
   /**
    * This class effectively contains the whole state and most of the logic of {@link Entry}, need to be a separate class
    * because the Entry must not be referenced from the runnable executed in {@link #cacheManager}'s ExecutorService,
-   * that would be a leak preventing the Entry to be collected by GC, and therefore {@link #entryCleaner} to be run by
-   * the JVM. Also, {@link #entryCleaner} must not reference the Entry through it's Runnable hunk.
+   * that would be a leak preventing the Entry to be collected by GC, and therefore {@link #entryCleanable} to be run by
+   * the JVM. Also, {@link #entryCleanable} must not reference the Entry through it's Runnable hunk.
    */
   public class EntryImpl<T extends ExtractionNamespace> implements AutoCloseable
   {
@@ -147,7 +147,7 @@ public final class CacheScheduler
     private final String asString;
     private final AtomicReference<CacheState> cacheStateHolder = new AtomicReference<CacheState>(NoCache.CACHE_NOT_INITIALIZED);
     private final Future<?> updaterFuture;
-    private final Cleaner entryCleaner;
+    private final Cleaners.Cleanable entryCleanable;
     private final CacheGenerator<T> cacheGenerator;
     private final ConcurrentAwaitableCounter updateCounter = new ConcurrentAwaitableCounter();
     private final CountDownLatch startLatch = new CountDownLatch(1);
@@ -158,7 +158,7 @@ public final class CacheScheduler
         this.namespace = namespace;
         this.asString = StringUtils.format("namespace [%s] : %s", namespace, super.toString());
         this.updaterFuture = schedule(namespace);
-        this.entryCleaner = createCleaner(entry);
+        this.entryCleanable = createCleaner(entry);
         this.cacheGenerator = cacheGenerator;
         activeEntries.incrementAndGet();
       }
@@ -167,9 +167,9 @@ public final class CacheScheduler
       }
     }
 
-    private Cleaner createCleaner(Entry<T> entry)
+    private Cleaners.Cleanable createCleaner(Entry<T> entry)
     {
-      return Cleaner.create(entry, new Runnable()
+      return Cleaners.register(entry, new Runnable()
       {
         @Override
         public void run()
@@ -290,10 +290,10 @@ public final class CacheScheduler
       if (!doClose(true)) {
         log.error("Cache for %s has already been closed", this);
       }
-      // This Cleaner.clean() call effectively just removes the Cleaner from the internal linked list of all cleaners.
+      // This clean() call effectively just removes the object from the internal linked list of all cleanables.
       // It will delegate to closeFromCleaner() which will be a no-op because cacheStateHolder is already set to
       // ENTRY_CLOSED.
-      entryCleaner.clean();
+      entryCleanable.clean();
     }
 
     private void closeFromCleaner()
@@ -311,7 +311,7 @@ public final class CacheScheduler
           t.addSuppressed(e);
         }
         Throwables.propagateIfInstanceOf(t, Error.class);
-        // Must not throw exceptions in the cleaner thread, run by the JVM.
+        // Must not throw exceptions in the cleaner thread, possibly run by the JVM.
       }
     }
 
diff --git a/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManager.java b/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManager.java
index 8a4615d..7f10615 100644
--- a/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManager.java
+++ b/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManager.java
@@ -21,6 +21,7 @@ package org.apache.druid.server.lookup.namespace.cache;
 
 import com.google.common.base.Throwables;
 import com.google.inject.Inject;
+import org.apache.druid.java.util.common.Cleaners;
 import org.apache.druid.java.util.common.lifecycle.Lifecycle;
 import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.java.util.emitter.service.ServiceEmitter;
@@ -29,7 +30,6 @@ import org.apache.druid.server.lookup.namespace.NamespaceExtractionConfig;
 import org.mapdb.DB;
 import org.mapdb.DBMaker;
 import org.mapdb.HTreeMap;
-import sun.misc.Cleaner;
 
 import java.io.File;
 import java.io.IOException;
@@ -54,9 +54,9 @@ public class OffHeapNamespaceExtractionCacheManager extends NamespaceExtractionC
      *
      * <p>In case of actual race, we don't wait in those methods until the other one, which manages to switch this flag
      * first, completes. This could result into the situation that neither one completes, if the JVM is shutting down
-     * and the thread from which {@link Cleaner#clean()} (delegating to {@link #run()}) is called started the disposal
+     * and the thread from which {@link Cleaners.Cleanable#clean()} (delegating to {@link #run()}) is called started the disposal
      * operation, then more deterministic shutdown hook / lifecycle.stop(), which may call {@link #disposeManually()}
-     * completed early, and then the whole process shuts down before {@link Cleaner#clean()} completes, because shutdown
+     * completed early, and then the whole process shuts down before {@link Cleaners.Cleanable#clean()} completes, because shutdown
      * is not blocked by it. However this should be harmless because anyway we remove the whole MapDB's file in
      * lifecycle.stop() (see {@link OffHeapNamespaceExtractionCacheManager#OffHeapNamespaceExtractionCacheManager}).
      * However if we persist off-heap DB between JVM runs, this decision should be revised.
@@ -69,7 +69,7 @@ public class OffHeapNamespaceExtractionCacheManager extends NamespaceExtractionC
     }
 
     /**
-     * To be called by the JVM via {@link Cleaner#clean()}. The only difference from {@link #disposeManually()} is
+     * To be called by the JVM via {@link Cleaners.Cleanable#clean()}. The only difference from {@link #disposeManually()} is
      * exception treatment.
      */
     @Override
@@ -89,7 +89,7 @@ public class OffHeapNamespaceExtractionCacheManager extends NamespaceExtractionC
             t.addSuppressed(e);
           }
           Throwables.propagateIfInstanceOf(t, Error.class);
-          // Must not throw exceptions in the cleaner thread, run by the JVM.
+          // Must not throw exceptions in the cleaner thread, possibly run in the JVM.
         }
       }
     }
@@ -118,12 +118,12 @@ public class OffHeapNamespaceExtractionCacheManager extends NamespaceExtractionC
   private static class MapDbCacheDisposerAndCleaner
   {
     final MapDbCacheDisposer cacheDisposer;
-    final Cleaner cleaner;
+    final Cleaners.Cleanable cleanable;
 
-    private MapDbCacheDisposerAndCleaner(MapDbCacheDisposer cacheDisposer, Cleaner cleaner)
+    private MapDbCacheDisposerAndCleaner(MapDbCacheDisposer cacheDisposer, Cleaners.Cleanable cleanable)
     {
       this.cacheDisposer = cacheDisposer;
-      this.cleaner = cleaner;
+      this.cleanable = cleanable;
     }
   }
 
@@ -196,8 +196,8 @@ public class OffHeapNamespaceExtractionCacheManager extends NamespaceExtractionC
       mapDbKey = Long.toString(mapDbKeyCounter.getAndIncrement());
       try {
         HTreeMap<String, String> hTreeMap = mmapDB.createHashMap(mapDbKey).make();
-        // Access MapDB's HTreeMap and create a cleaner via proxy, because there is no 100% confidence that there are
-        // no memory leaks in MapDB and in OffHeapCacheManager. Otherwise JVM will never be able to clean the cleaner
+        // Access MapDB's HTreeMap and create a cleanable via proxy, because there is no 100% confidence that there are
+        // no memory leaks in MapDB and in OffHeapCacheManager. Otherwise JVM will never be able to clean the cleanable
         // and dispose leaked cache.
         cache = new CacheProxy(hTreeMap);
         cacheCount.incrementAndGet();
@@ -211,10 +211,10 @@ public class OffHeapNamespaceExtractionCacheManager extends NamespaceExtractionC
     // Cleaner is "the second level of defence". Normally all users of createCache() must call disposeCache() with
     // the returned CacheHandler instance manually. But if they don't do this for whatever reason, JVM will cleanup
     // the cache itself.
-    Cleaner cleaner = Cleaner.create(cache, cacheDisposer);
+    Cleaners.Cleanable cleanable = Cleaners.register(cache, cacheDisposer);
     MapDbCacheDisposerAndCleaner disposerAndCleaner = new MapDbCacheDisposerAndCleaner(
         cacheDisposer,
-        cleaner
+        cleanable
     );
     return new CacheHandler(this, cache, disposerAndCleaner);
   }
@@ -226,7 +226,7 @@ public class OffHeapNamespaceExtractionCacheManager extends NamespaceExtractionC
     disposerAndCleaner.cacheDisposer.disposeManually();
     // This clean() call effectively just removes the Cleaner from the internal linked list of all cleaners.
     // The thunk.run() will be a no-op because cacheDisposer.disposed is already set to true.
-    disposerAndCleaner.cleaner.clean();
+    disposerAndCleaner.cleanable.clean();
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/druid/client/cache/MemcacheClientPool.java b/server/src/main/java/org/apache/druid/client/cache/MemcacheClientPool.java
index fad6578..60b5f22 100644
--- a/server/src/main/java/org/apache/druid/client/cache/MemcacheClientPool.java
+++ b/server/src/main/java/org/apache/druid/client/cache/MemcacheClientPool.java
@@ -24,8 +24,8 @@ import com.google.common.base.Preconditions;
 import com.google.common.base.Supplier;
 import net.spy.memcached.MemcachedClientIF;
 import org.apache.druid.collections.ResourceHolder;
+import org.apache.druid.java.util.common.Cleaners;
 import org.apache.druid.java.util.common.logger.Logger;
-import sun.misc.Cleaner;
 
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
@@ -93,16 +93,16 @@ final class MemcacheClientPool implements Supplier<ResourceHolder<MemcachedClien
     private final AtomicInteger count = new AtomicInteger(0);
     private final MemcachedClientIF clientIF;
     /**
-     * The point of Cleaner is to be referenced. Action is performed when it becomes unreachable, so it doesn't need
+     * The point of cleanable is to be referenced. Action is performed when it becomes unreachable, so it doesn't need
      * to be used directly.
      */
     @SuppressWarnings("unused")
-    private final Cleaner cleaner;
+    private final Cleaners.Cleanable cleanable;
 
     private CountingHolder(final MemcachedClientIF clientIF)
     {
       this.clientIF = clientIF;
-      cleaner = Cleaner.create(this, new ClientLeakNotifier(count, clientIF));
+      cleanable = Cleaners.register(this, new ClientLeakNotifier(count, clientIF));
     }
   }
 


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