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

ctubbsii pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/master by this push:
     new 9f256c5  Fix #723 Replace finalize method uses with Cleaner (#1329)
9f256c5 is described below

commit 9f256c531b42e9f890ce86e21f3b0e1a7e0acc78
Author: Christopher Tubbs <ctubb...@apache.org>
AuthorDate: Tue Aug 27 18:28:28 2019 -0400

    Fix #723 Replace finalize method uses with Cleaner (#1329)
    
    Replaces deprecated and ill-advised uses of `Object.finalize()` with
    Cleaner.
    
    * Create a CleanerUtil to contain cleanable actions in a separate class
      to avoid them blocking garbage collection of the objects to be cleaned
    * Consolidate similar cleanable actions into a single implementation
    
    Also cleaned up some of the NativeMap code, especially:
    
    * Replaced repeated checks for deleted NM with a private method
    * Make NMSKVI conform to SKVI spec by throwing NoSuchElementException
    instead of IllegalStateException
    * Add message parameters to some Exceptions which didn't have a message
    * Remove redundant UnsupportedOperationException on remove() (default
    method from interface is sufficient)
---
 .../core/client/MultiTableBatchWriter.java         |   3 +-
 .../core/clientImpl/MultiTableBatchWriterImpl.java |  44 +++----
 .../core/clientImpl/TabletServerBatchDeleter.java  |   2 +-
 .../core/clientImpl/TabletServerBatchReader.java   |  59 ++++-----
 .../core/clientImpl/TabletServerBatchWriter.java   |   3 +-
 .../core/singletons/SingletonReservation.java      |  44 ++++---
 .../accumulo/core/util/SimpleThreadPool.java       |  21 ++++
 .../accumulo/core/util/cleaner/CleanerUtil.java    | 129 +++++++++++++++++++
 .../server/constraints/MetadataConstraints.java    |  16 +--
 .../server/master/state/MetaDataTableScanner.java  |  72 +++++------
 .../org/apache/accumulo/tserver/NativeMap.java     | 137 +++++++++------------
 .../accumulo/tserver/NativeMapCleanerUtil.java     |  54 ++++++++
 .../apache/accumulo/test/BatchWriterIterator.java  |  66 +++++-----
 .../test/functional/MasterAssignmentIT.java        |   3 +-
 .../accumulo/test/master/SuspendedTabletsIT.java   |   3 +-
 .../accumulo/test/performance/NullTserver.java     |   3 +-
 16 files changed, 404 insertions(+), 255 deletions(-)

diff --git 
a/core/src/main/java/org/apache/accumulo/core/client/MultiTableBatchWriter.java 
b/core/src/main/java/org/apache/accumulo/core/client/MultiTableBatchWriter.java
index 8ea675d..0d9983e 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/client/MultiTableBatchWriter.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/client/MultiTableBatchWriter.java
@@ -21,7 +21,7 @@ package org.apache.accumulo.core.client;
  * each table, each has its own memory and network resources. Using this class 
these resources may
  * be shared among multiple tables.
  */
-public interface MultiTableBatchWriter {
+public interface MultiTableBatchWriter extends AutoCloseable {
 
   /**
    * Returns a BatchWriter for a particular table.
@@ -54,6 +54,7 @@ public interface MultiTableBatchWriter {
    *           when queued mutations are unable to be inserted
    *
    */
+  @Override
   void close() throws MutationsRejectedException;
 
   /**
diff --git 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/MultiTableBatchWriterImpl.java
 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/MultiTableBatchWriterImpl.java
index 0f5b973..068b41e 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/MultiTableBatchWriterImpl.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/MultiTableBatchWriterImpl.java
@@ -18,6 +18,7 @@ package org.apache.accumulo.core.clientImpl;
 
 import static com.google.common.base.Preconditions.checkArgument;
 
+import java.lang.ref.Cleaner.Cleanable;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 
@@ -29,6 +30,7 @@ import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.TableOfflineException;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.util.cleaner.CleanerUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -37,13 +39,12 @@ import 
com.google.common.util.concurrent.UncheckedExecutionException;
 public class MultiTableBatchWriterImpl implements MultiTableBatchWriter {
 
   private static final Logger log = 
LoggerFactory.getLogger(MultiTableBatchWriterImpl.class);
-  private AtomicBoolean closed;
 
   private class TableBatchWriter implements BatchWriter {
 
-    private TableId tableId;
+    private final TableId tableId;
 
-    TableBatchWriter(TableId tableId) {
+    private TableBatchWriter(TableId tableId) {
       this.tableId = tableId;
     }
 
@@ -69,47 +70,34 @@ public class MultiTableBatchWriterImpl implements 
MultiTableBatchWriter {
       throw new UnsupportedOperationException(
           "Must flush all tables, can not flush an individual table");
     }
-
   }
 
-  private TabletServerBatchWriter bw;
-  private ConcurrentHashMap<TableId,BatchWriter> tableWriters;
+  private final ConcurrentHashMap<TableId,BatchWriter> tableWriters = new 
ConcurrentHashMap<>();
+  private final AtomicBoolean closed = new AtomicBoolean(false);
   private final ClientContext context;
+  private final TabletServerBatchWriter bw;
+  private final Cleanable cleanable;
 
-  public MultiTableBatchWriterImpl(ClientContext context, BatchWriterConfig 
config) {
+  MultiTableBatchWriterImpl(ClientContext context, BatchWriterConfig config) {
     checkArgument(context != null, "context is null");
     checkArgument(config != null, "config is null");
     this.context = context;
     this.bw = new TabletServerBatchWriter(context, config);
-    tableWriters = new ConcurrentHashMap<>();
-    this.closed = new AtomicBoolean(false);
+    this.cleanable = CleanerUtil.unclosed(this, MultiTableBatchWriter.class, 
closed, log, bw);
   }
 
   @Override
   public boolean isClosed() {
-    return this.closed.get();
+    return closed.get();
   }
 
   @Override
   public void close() throws MutationsRejectedException {
-    this.closed.set(true);
-    bw.close();
-  }
-
-  // WARNING: do not rely upon finalize to close this class. Finalize is not 
guaranteed to be
-  // called.
-  @Override
-  protected void finalize() {
-    if (!closed.get()) {
-      log.warn("{} not shutdown; did you forget to call close()?",
-          MultiTableBatchWriterImpl.class.getSimpleName());
-      try {
-        close();
-      } catch (MutationsRejectedException mre) {
-        log.error(MultiTableBatchWriterImpl.class.getSimpleName() + " internal 
error.", mre);
-        throw new RuntimeException(
-            "Exception when closing " + 
MultiTableBatchWriterImpl.class.getSimpleName(), mre);
-      }
+    if (closed.compareAndSet(false, true)) {
+      // deregister cleanable, but it won't run because it checks
+      // the value of closed first, which is now true
+      cleanable.clean();
+      bw.close();
     }
   }
 
diff --git 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchDeleter.java
 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchDeleter.java
index ce8092e..9d5555f 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchDeleter.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchDeleter.java
@@ -40,7 +40,7 @@ public class TabletServerBatchDeleter extends 
TabletServerBatchReader implements
 
   public TabletServerBatchDeleter(ClientContext context, TableId tableId,
       Authorizations authorizations, int numQueryThreads, BatchWriterConfig 
bwConfig) {
-    super(context, tableId, authorizations, numQueryThreads);
+    super(context, BatchDeleter.class, tableId, authorizations, 
numQueryThreads);
     this.context = context;
     this.tableId = tableId;
     this.bwConfig = bwConfig;
diff --git 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReader.java
 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReader.java
index 0a6777e..3f56668 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReader.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReader.java
@@ -18,11 +18,13 @@ package org.apache.accumulo.core.clientImpl;
 
 import static com.google.common.base.Preconditions.checkArgument;
 
+import java.lang.ref.Cleaner.Cleanable;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.Map.Entry;
-import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.accumulo.core.client.BatchScanner;
 import org.apache.accumulo.core.data.Key;
@@ -31,31 +33,31 @@ import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.SimpleThreadPool;
+import org.apache.accumulo.core.util.cleaner.CleanerUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 public class TabletServerBatchReader extends ScannerOptions implements 
BatchScanner {
   private static final Logger log = 
LoggerFactory.getLogger(TabletServerBatchReader.class);
+  private static final AtomicInteger nextBatchReaderInstance = new 
AtomicInteger(1);
 
-  private TableId tableId;
-  private int numThreads;
-  private ExecutorService queryThreadPool;
-
+  private final int batchReaderInstance = 
nextBatchReaderInstance.getAndIncrement();
+  private final TableId tableId;
+  private final int numThreads;
+  private final SimpleThreadPool queryThreadPool;
   private final ClientContext context;
-  private ArrayList<Range> ranges;
-
-  private Authorizations authorizations = Authorizations.EMPTY;
-  private Throwable ex = null;
+  private final Authorizations authorizations;
+  private final AtomicBoolean closed = new AtomicBoolean(false);
+  private final Cleanable cleanable;
 
-  private static int nextBatchReaderInstance = 1;
+  private ArrayList<Range> ranges = null;
 
-  private static synchronized int getNextBatchReaderInstance() {
-    return nextBatchReaderInstance++;
+  public TabletServerBatchReader(ClientContext context, TableId tableId,
+      Authorizations authorizations, int numQueryThreads) {
+    this(context, BatchScanner.class, tableId, authorizations, 
numQueryThreads);
   }
 
-  private final int batchReaderInstance = getNextBatchReaderInstance();
-
-  public TabletServerBatchReader(ClientContext context, TableId tableId,
+  protected TabletServerBatchReader(ClientContext context, Class<?> 
scopeClass, TableId tableId,
       Authorizations authorizations, int numQueryThreads) {
     checkArgument(context != null, "context is null");
     checkArgument(tableId != null, "tableId is null");
@@ -67,14 +69,17 @@ public class TabletServerBatchReader extends ScannerOptions 
implements BatchScan
 
     queryThreadPool =
         new SimpleThreadPool(numQueryThreads, "batch scanner " + 
batchReaderInstance + "-");
-
-    ranges = null;
-    ex = new Throwable();
+    cleanable = CleanerUtil.unclosed(this, scopeClass, closed, log, 
queryThreadPool.asCloseable());
   }
 
   @Override
   public void close() {
-    queryThreadPool.shutdownNow();
+    if (closed.compareAndSet(false, true)) {
+      // deregister cleanable, but it won't run because it checks
+      // the value of closed first, which is now true
+      cleanable.clean();
+      queryThreadPool.shutdownNow();
+    }
   }
 
   @Override
@@ -82,29 +87,17 @@ public class TabletServerBatchReader extends ScannerOptions 
implements BatchScan
     return authorizations;
   }
 
-  // WARNING: do not rely upon finalize to close this class. Finalize is not 
guaranteed to be
-  // called.
-  @Override
-  protected void finalize() {
-    if (!queryThreadPool.isShutdown()) {
-      log.warn(TabletServerBatchReader.class.getSimpleName()
-          + " not shutdown; did you forget to call close()?", ex);
-      close();
-    }
-  }
-
   @Override
   public void setRanges(Collection<Range> ranges) {
     if (ranges == null || ranges.size() == 0) {
       throw new IllegalArgumentException("ranges must be non null and contain 
at least 1 range");
     }
 
-    if (queryThreadPool.isShutdown()) {
+    if (closed.get()) {
       throw new IllegalStateException("batch reader closed");
     }
 
     this.ranges = new ArrayList<>(ranges);
-
   }
 
   @Override
@@ -113,7 +106,7 @@ public class TabletServerBatchReader extends ScannerOptions 
implements BatchScan
       throw new IllegalStateException("ranges not set");
     }
 
-    if (queryThreadPool.isShutdown()) {
+    if (closed.get()) {
       throw new IllegalStateException("batch reader closed");
     }
 
diff --git 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchWriter.java
 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchWriter.java
index 8b766c6..97fe2a4 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchWriter.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchWriter.java
@@ -100,7 +100,7 @@ import com.google.common.base.Joiner;
  *   + when a mutation enters the system memory is incremented
  *   + when a mutation successfully leaves the system memory is decremented
  */
-public class TabletServerBatchWriter {
+public class TabletServerBatchWriter implements AutoCloseable {
 
   private static final Logger log = 
LoggerFactory.getLogger(TabletServerBatchWriter.class);
 
@@ -324,6 +324,7 @@ public class TabletServerBatchWriter {
     }
   }
 
+  @Override
   public synchronized void close() throws MutationsRejectedException {
 
     if (closed)
diff --git 
a/core/src/main/java/org/apache/accumulo/core/singletons/SingletonReservation.java
 
b/core/src/main/java/org/apache/accumulo/core/singletons/SingletonReservation.java
index 67242af..cb542cb 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/singletons/SingletonReservation.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/singletons/SingletonReservation.java
@@ -17,6 +17,11 @@
 
 package org.apache.accumulo.core.singletons;
 
+import java.lang.ref.Cleaner.Cleanable;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.util.cleaner.CleanerUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -25,41 +30,34 @@ import org.slf4j.LoggerFactory;
  */
 public class SingletonReservation implements AutoCloseable {
 
-  // volatile so finalize does not need to synchronize to reliably read
-  protected volatile boolean closed = false;
-
-  private static Logger log = 
LoggerFactory.getLogger(SingletonReservation.class);
+  private static final Logger log = 
LoggerFactory.getLogger(SingletonReservation.class);
 
-  private final Exception e;
+  // AtomicBoolean so cleaner doesn't need to synchronize to reliably read
+  private final AtomicBoolean closed = new AtomicBoolean(false);
+  private final Cleanable cleanable;
 
   public SingletonReservation() {
-    e = new Exception();
+    cleanable = CleanerUtil.unclosed(this, AccumuloClient.class, closed, log, 
null);
   }
 
   @Override
-  public synchronized void close() {
-    if (closed) {
-      return;
-    }
-    closed = true;
-    SingletonManager.releaseRerservation();
-  }
-
-  @Override
-  protected void finalize() throws Throwable {
-    try {
-      if (!closed) {
-        log.warn("An Accumulo Client was garbage collected without being 
closed.", e);
-      }
-    } finally {
-      super.finalize();
+  public void close() {
+    if (closed.compareAndSet(false, true)) {
+      // deregister cleanable, but it won't run because it checks
+      // the value of closed first, which is now true
+      cleanable.clean();
+      SingletonManager.releaseRerservation();
     }
   }
 
   private static class NoopSingletonReservation extends SingletonReservation {
     NoopSingletonReservation() {
-      closed = true;
+      super();
+      super.closed.set(true);
+      // deregister the cleaner
+      super.cleanable.clean();
     }
+
   }
 
   private static final SingletonReservation NOOP = new 
NoopSingletonReservation();
diff --git 
a/core/src/main/java/org/apache/accumulo/core/util/SimpleThreadPool.java 
b/core/src/main/java/org/apache/accumulo/core/util/SimpleThreadPool.java
index 91fedf0..39f8f94 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/SimpleThreadPool.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/SimpleThreadPool.java
@@ -37,4 +37,25 @@ public class SimpleThreadPool extends ThreadPoolExecutor {
     allowCoreThreadTimeOut(true);
   }
 
+  /**
+   * Wrap this with a trivial object whose {@link AutoCloseable#close()} 
method calls
+   * {@link #shutdownNow()}.
+   */
+  public CloseableSimpleThreadPool asCloseable() {
+    return new CloseableSimpleThreadPool(this);
+  }
+
+  public static class CloseableSimpleThreadPool implements AutoCloseable {
+    private final SimpleThreadPool stp;
+
+    public CloseableSimpleThreadPool(SimpleThreadPool simpleThreadPool) {
+      this.stp = simpleThreadPool;
+    }
+
+    @Override
+    public void close() {
+      stp.shutdownNow();
+    }
+  }
+
 }
diff --git 
a/core/src/main/java/org/apache/accumulo/core/util/cleaner/CleanerUtil.java 
b/core/src/main/java/org/apache/accumulo/core/util/cleaner/CleanerUtil.java
new file mode 100644
index 0000000..db1602e
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/util/cleaner/CleanerUtil.java
@@ -0,0 +1,129 @@
+/*
+ * 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.accumulo.core.util.cleaner;
+
+import static java.util.Objects.requireNonNull;
+
+import java.lang.ref.Cleaner;
+import java.lang.ref.Cleaner.Cleanable;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.slf4j.Logger;
+
+/**
+ * This class collects all the cleaner actions executed in various parts of 
the code.
+ *
+ * <p>
+ * These actions replace the use of finalizers, which are deprecated in Java 9 
and later, and should
+ * be avoided. These actions are triggered by their respective objects when 
those objects become
+ * phantom reachable.
+ *
+ * <p>
+ * In the "unclosed*" methods below, the object should have been closed 
(implements AutoCloseable).
+ * We could possibly consolidate these into a single method which only warns, 
and doesn't try to
+ * clean up. We could also delete them entirely, since it is the caller's 
responsibility to close
+ * AutoCloseable resources, not the object's own responsibility to detect that 
it wasn't closed.
+ */
+public class CleanerUtil {
+
+  public static final Cleaner CLEANER = Cleaner.create();
+
+  /**
+   * Register an action to warn about caller failing to close an {@link 
AutoCloseable} object.
+   *
+   * <p>
+   * This task will register a generic action to:
+   * <ol>
+   * <li>check that the monitored object wasn't closed,
+   * <li>log a warning that the monitored object was not closed,
+   * <li>attempt to close a resource within the object, and
+   * <li>log an error if the resource cannot be closed for any reason
+   * </ol>
+   *
+   * @param obj
+   *          the object to monitor for becoming phantom-reachable without 
having been closed
+   * @param objClass
+   *          the class whose simple name will be used in the log message for 
<code>o</code>
+   *          (usually an interface name, rather than the actual impl name of 
the object)
+   * @param closed
+   *          a flag to check whether <code>o</code> has already been closed
+   * @param log
+   *          the logger to use when emitting error/warn messages
+   * @param closeable
+   *          the resource within <code>o</code> to close when <code>o</code> 
is cleaned; must not
+   *          contain a reference to the <code>monitoredObject</code> or it 
won't become
+   *          phantom-reachable and will never be cleaned
+   * @return the registered {@link Cleanable} from {@link 
Cleaner#register(Object, Runnable)}
+   */
+  public static Cleanable unclosed(AutoCloseable obj, Class<?> objClass, 
AtomicBoolean closed,
+      Logger log, AutoCloseable closeable) {
+    String className = requireNonNull(objClass).getSimpleName();
+    requireNonNull(closed);
+    requireNonNull(log);
+    String closeableClassName = closeable == null ? null : 
closeable.getClass().getSimpleName();
+
+    // capture the stack trace during setup for logging later, so user can 
find unclosed object
+    var stackTrace = new Exception();
+
+    // register the action to run when obj becomes phantom-reachable or clean 
is explicitly called
+    return CLEANER.register(obj, () -> {
+      if (closed.get()) {
+        // already closed; nothing to do
+        return;
+      }
+      log.warn("{} found unreferenced without calling close()", className, 
stackTrace);
+      if (closeable != null) {
+        try {
+          closeable.close();
+        } catch (Exception e1) {
+          log.error("{} internal error; exception closing {}", objClass, 
closeableClassName, e1);
+        }
+      }
+    });
+  }
+
+  // this done for the BatchWriterIterator test code; I don't trust that 
pattern, but
+  // registering a cleaner is something any user is probably going to have to 
do to clean up
+  // resources used in an iterator, until iterators properly implement their 
own close()
+  public static Cleanable batchWriterAndClientCloser(Object o, Logger log, 
BatchWriter bw,
+      AccumuloClient client) {
+    requireNonNull(log);
+    requireNonNull(bw);
+    requireNonNull(client);
+    return CLEANER.register(o, () -> {
+      try {
+        bw.close();
+      } catch (MutationsRejectedException e) {
+        log.error("Failed to close BatchWriter; some mutations may not be 
applied", e);
+      } finally {
+        client.close();
+      }
+    });
+  }
+
+  // this is dubious; MetadataConstraints should probably use the ZooCache 
provided by context
+  // can be done in a follow-on action; for now, this merely replaces the 
previous finalizer
+  public static Cleanable zooCacheClearer(Object o, ZooCache zc) {
+    requireNonNull(zc);
+    return CLEANER.register(o, zc::clear);
+  }
+
+}
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/constraints/MetadataConstraints.java
 
b/server/base/src/main/java/org/apache/accumulo/server/constraints/MetadataConstraints.java
index 0b59364..783b515 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/constraints/MetadataConstraints.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/constraints/MetadataConstraints.java
@@ -41,6 +41,7 @@ import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.Da
 import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
 import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ScanFileColumnFamily;
 import org.apache.accumulo.core.util.ColumnFQ;
+import org.apache.accumulo.core.util.cleaner.CleanerUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
 import org.apache.accumulo.fate.zookeeper.ZooLock;
 import org.apache.accumulo.fate.zookeeper.ZooUtil;
@@ -53,14 +54,13 @@ import org.slf4j.LoggerFactory;
 
 public class MetadataConstraints implements Constraint {
 
-  private ZooCache zooCache = null;
-  private String zooRoot = null;
-
   private static final Logger log = 
LoggerFactory.getLogger(MetadataConstraints.class);
 
-  private static boolean[] validTableNameChars = new boolean[256];
+  private ZooCache zooCache = null;
+  private String zooRoot = null;
 
-  {
+  private static final boolean[] validTableNameChars = new boolean[256];
+  static {
     for (int i = 0; i < 256; i++) {
       validTableNameChars[i] =
           ((i >= 'a' && i <= 'z') || (i >= '0' && i <= '9')) || i == '!' || i 
== '+';
@@ -265,6 +265,7 @@ public class MetadataConstraints implements Constraint {
             .equals(TabletsSection.ServerColumnFamily.LOCK_COLUMN)) {
           if (zooCache == null) {
             zooCache = new ZooCache(context.getZooReaderWriter(), null);
+            CleanerUtil.zooCacheClearer(this, zooCache);
           }
 
           if (zooRoot == null) {
@@ -327,9 +328,4 @@ public class MetadataConstraints implements Constraint {
     return null;
   }
 
-  @Override
-  protected void finalize() {
-    if (zooCache != null)
-      zooCache.clear();
-  }
 }
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java
 
b/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java
index f718790..f1c4bdb 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java
@@ -17,6 +17,7 @@
 package org.apache.accumulo.server.master.state;
 
 import java.io.IOException;
+import java.lang.ref.Cleaner.Cleanable;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -24,22 +25,25 @@ import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map.Entry;
+import java.util.NoSuchElementException;
 import java.util.SortedMap;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.accumulo.core.client.BatchScanner;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.ScannerBase;
+import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.iterators.user.WholeRowIterator;
-import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ChoppedColumnFamily;
 import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.cleaner.CleanerUtil;
 import 
org.apache.accumulo.server.master.state.TabletLocationState.BadLocationStateException;
 import org.apache.hadoop.io.Text;
 import org.slf4j.Logger;
@@ -48,28 +52,23 @@ import org.slf4j.LoggerFactory;
 public class MetaDataTableScanner implements 
ClosableIterator<TabletLocationState> {
   private static final Logger log = 
LoggerFactory.getLogger(MetaDataTableScanner.class);
 
-  BatchScanner mdScanner = null;
-  Iterator<Entry<Key,Value>> iter = null;
-
-  public MetaDataTableScanner(ClientContext context, Range range, CurrentState 
state) {
-    this(context, range, state, MetadataTable.NAME);
-  }
+  private final Cleanable cleanable;
+  private final BatchScanner mdScanner;
+  private final Iterator<Entry<Key,Value>> iter;
+  private final AtomicBoolean closed = new AtomicBoolean(false);
 
   MetaDataTableScanner(ClientContext context, Range range, CurrentState state, 
String tableName) {
     // scan over metadata table, looking for tablets in the wrong state based 
on the live servers
     // and online tables
     try {
       mdScanner = context.createBatchScanner(tableName, Authorizations.EMPTY, 
8);
-      configureScanner(mdScanner, state);
-      mdScanner.setRanges(Collections.singletonList(range));
-      iter = mdScanner.iterator();
-    } catch (Exception ex) {
-      if (mdScanner != null)
-        mdScanner.close();
-      iter = null;
-      mdScanner = null;
-      throw new RuntimeException(ex);
+    } catch (TableNotFoundException e) {
+      throw new IllegalStateException("Metadata table " + tableName + " should 
exist", e);
     }
+    cleanable = CleanerUtil.unclosed(this, MetaDataTableScanner.class, closed, 
log, mdScanner);
+    configureScanner(mdScanner, state);
+    mdScanner.setRanges(Collections.singletonList(range));
+    iter = mdScanner.iterator();
   }
 
   public static void configureScanner(ScannerBase scanner, CurrentState state) 
{
@@ -95,31 +94,25 @@ public class MetaDataTableScanner implements 
ClosableIterator<TabletLocationStat
     scanner.addScanIterator(tabletChange);
   }
 
-  public MetaDataTableScanner(ClientContext context, Range range) {
-    this(context, range, MetadataTable.NAME);
-  }
-
   public MetaDataTableScanner(ClientContext context, Range range, String 
tableName) {
     this(context, range, null, tableName);
   }
 
   @Override
   public void close() {
-    if (iter != null) {
+    if (closed.compareAndSet(false, true)) {
+      // deregister cleanable, but it won't run because it checks
+      // the value of closed first, which is now true
+      cleanable.clean();
       mdScanner.close();
-      iter = null;
     }
   }
 
   @Override
-  protected void finalize() {
-    close();
-  }
-
-  @Override
   public boolean hasNext() {
-    if (iter == null)
+    if (closed.get()) {
       return false;
+    }
     boolean result = iter.hasNext();
     if (!result) {
       close();
@@ -129,7 +122,15 @@ public class MetaDataTableScanner implements 
ClosableIterator<TabletLocationStat
 
   @Override
   public TabletLocationState next() {
-    return fetch();
+    if (closed.get()) {
+      throw new NoSuchElementException(this.getClass().getSimpleName() + " is 
closed");
+    }
+    try {
+      Entry<Key,Value> e = iter.next();
+      return createTabletLocationState(e.getKey(), e.getValue());
+    } catch (IOException | BadLocationStateException ex) {
+      throw new RuntimeException(ex);
+    }
   }
 
   public static TabletLocationState createTabletLocationState(Key k, Value v)
@@ -187,17 +188,4 @@ public class MetaDataTableScanner implements 
ClosableIterator<TabletLocationStat
     return new TabletLocationState(extent, future, current, last, suspend, 
walogs, chopped);
   }
 
-  private TabletLocationState fetch() {
-    try {
-      Entry<Key,Value> e = iter.next();
-      return createTabletLocationState(e.getKey(), e.getValue());
-    } catch (IOException | BadLocationStateException ex) {
-      throw new RuntimeException(ex);
-    }
-  }
-
-  @Override
-  public void remove() {
-    throw new RuntimeException("Unimplemented");
-  }
 }
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/NativeMap.java 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/NativeMap.java
index 45bbee6..1c8f2b4 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/NativeMap.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/NativeMap.java
@@ -17,6 +17,7 @@
 package org.apache.accumulo.tserver;
 
 import java.io.File;
+import java.lang.ref.Cleaner.Cleanable;
 import java.util.AbstractMap.SimpleImmutableEntry;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -28,6 +29,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.NoSuchElementException;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
@@ -179,7 +181,7 @@ public class NativeMap implements 
Iterable<Map.Entry<Key,Value>> {
     return false;
   }
 
-  private long nmPointer;
+  private final AtomicLong nmPtr = new AtomicLong(0);
 
   private final ReadWriteLock rwLock;
   private final Lock rlock;
@@ -248,6 +250,12 @@ public class NativeMap implements 
Iterable<Map.Entry<Key,Value>> {
     }
   }
 
+  // package private visibility for NativeMapCleanerUtil use,
+  // without affecting ABI of existing native interface
+  static void _deleteNativeMap(long nmPtr) {
+    deleteNativeMap(nmPtr);
+  }
+
   private static native long createNMI(long nmp, int[] fieldLens);
 
   private static native long createNMI(long nmp, byte[] row, byte[] cf, byte[] 
cq, byte[] cv,
@@ -262,6 +270,12 @@ public class NativeMap implements 
Iterable<Map.Entry<Key,Value>> {
 
   private static native void deleteNMI(long nmiPointer);
 
+  // package private visibility for NativeMapCleanerUtil use,
+  // without affecting ABI of existing native interface
+  static void _deleteNMI(long nmiPointer) {
+    deleteNMI(nmiPointer);
+  }
+
   private class ConcurrentIterator implements Iterator<Map.Entry<Key,Value>> {
 
     // in order to get good performance when there are multiple threads 
reading, need to read a lot
@@ -363,11 +377,6 @@ public class NativeMap implements 
Iterable<Map.Entry<Key,Value>> {
       return ret;
     }
 
-    @Override
-    public void remove() {
-      throw new UnsupportedOperationException();
-    }
-
     public void delete() {
       source.delete();
     }
@@ -385,40 +394,42 @@ public class NativeMap implements 
Iterable<Map.Entry<Key,Value>> {
      *
      */
 
-    private long nmiPointer;
+    private final AtomicLong nmiPtr = new AtomicLong(0);
     private boolean hasNext;
     private int expectedModCount;
     private int[] fieldsLens = new int[7];
     private byte[] lastRow;
+    private final Cleanable cleanableNMI;
 
     // it is assumed the read lock is held when this method is called
     NMIterator(Key key) {
 
-      if (nmPointer == 0) {
-        throw new IllegalStateException();
-      }
+      final long nmPointer = nmPtr.get();
+      checkDeletedNM(nmPointer);
 
       expectedModCount = modCount;
 
-      nmiPointer = createNMI(nmPointer, key.getRowData().toArray(),
+      final long nmiPointer = createNMI(nmPointer, key.getRowData().toArray(),
           key.getColumnFamilyData().toArray(), 
key.getColumnQualifierData().toArray(),
           key.getColumnVisibilityData().toArray(), key.getTimestamp(), 
key.isDeleted(), fieldsLens);
 
       hasNext = nmiPointer != 0;
+
+      nmiPtr.set(nmiPointer);
+      cleanableNMI = NativeMapCleanerUtil.deleteNMIterator(this, nmiPtr);
     }
 
     // delete is synchronized on a per iterator basis want to ensure only one
     // thread deletes an iterator w/o acquiring the global write lock...
     // there is no contention among concurrent readers for deleting their 
iterators
     public synchronized void delete() {
-      if (nmiPointer == 0) {
-        return;
+      final long nmiPointer = nmiPtr.getAndSet(0);
+      if (nmiPointer != 0) {
+        // deregister cleanable, but it won't run because it checks
+        // the value of nmiPtr first, which is now 0
+        cleanableNMI.clean();
+        deleteNMI(nmiPointer);
       }
-
-      // log.debug("Deleting native map iterator pointer");
-
-      deleteNMI(nmiPointer);
-      nmiPointer = 0;
     }
 
     @Override
@@ -429,10 +440,7 @@ public class NativeMap implements 
Iterable<Map.Entry<Key,Value>> {
     // it is assumed the read lock is held when this method is called
     // this method only needs to be called once per read lock acquisition
     private void doNextPreCheck() {
-      if (nmPointer == 0) {
-        throw new IllegalStateException();
-      }
-
+      checkDeletedNM(nmPtr.get());
       if (modCount != expectedModCount) {
         throw new ConcurrentModificationException();
       }
@@ -448,6 +456,7 @@ public class NativeMap implements 
Iterable<Map.Entry<Key,Value>> {
         throw new NoSuchElementException();
       }
 
+      final long nmiPointer = nmiPtr.get();
       if (nmiPointer == 0) {
         throw new IllegalStateException("Native Map Iterator Deleted");
       }
@@ -475,41 +484,28 @@ public class NativeMap implements 
Iterable<Map.Entry<Key,Value>> {
       return new SimpleImmutableEntry<>(k, v);
     }
 
-    @Override
-    public void remove() {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    protected void finalize() throws Throwable {
-      super.finalize();
-      if (nmiPointer != 0) {
-        // log.debug("Deleting native map iterator pointer in finalize");
-        deleteNMI(nmiPointer);
-      }
-    }
-
   }
 
+  private final Cleanable cleanableNM;
+
   public NativeMap() {
-    nmPointer = createNativeMap();
+    final long nmPointer = createNativeMap();
+    nmPtr.set(nmPointer);
+    cleanableNM = NativeMapCleanerUtil.deleteNM(this, log, nmPtr);
     rwLock = new ReentrantReadWriteLock();
     rlock = rwLock.readLock();
     wlock = rwLock.writeLock();
     log.debug(String.format("Allocated native map 0x%016x", nmPointer));
   }
 
-  @Override
-  protected void finalize() throws Throwable {
-    super.finalize();
-    if (nmPointer != 0) {
-      log.warn(String.format("Deallocating native map 0x%016x in finalize", 
nmPointer));
-      deleteNativeMap(nmPointer);
+  private static void checkDeletedNM(final long nmPointer) {
+    if (nmPointer == 0) {
+      throw new IllegalStateException("Native Map Deleted");
     }
   }
 
-  private int _mutate(Mutation mutation, int mutationCount) {
-
+  // assumes wlock
+  private int _mutate(final long nmPointer, Mutation mutation, int 
mutationCount) {
     List<ColumnUpdate> updates = mutation.getUpdates();
     if (updates.size() == 1) {
       ColumnUpdate update = updates.get(0);
@@ -534,16 +530,15 @@ public class NativeMap implements 
Iterable<Map.Entry<Key,Value>> {
 
       wlock.lock();
       try {
-        if (nmPointer == 0) {
-          throw new IllegalStateException("Native Map Deleted");
-        }
+        final long nmPointer = nmPtr.get();
+        checkDeletedNM(nmPointer);
 
         modCount++;
 
         int count = 0;
         while (iter.hasNext() && count < 10) {
           Mutation mutation = iter.next();
-          mutationCount = _mutate(mutation, mutationCount);
+          mutationCount = _mutate(nmPointer, mutation, mutationCount);
           count += mutation.size();
         }
       } finally {
@@ -556,9 +551,8 @@ public class NativeMap implements 
Iterable<Map.Entry<Key,Value>> {
   public void put(Key key, Value value) {
     wlock.lock();
     try {
-      if (nmPointer == 0) {
-        throw new IllegalStateException("Native Map Deleted");
-      }
+      final long nmPointer = nmPtr.get();
+      checkDeletedNM(nmPointer);
 
       modCount++;
 
@@ -593,10 +587,8 @@ public class NativeMap implements 
Iterable<Map.Entry<Key,Value>> {
   public int size() {
     rlock.lock();
     try {
-      if (nmPointer == 0) {
-        throw new IllegalStateException("Native Map Deleted");
-      }
-
+      final long nmPointer = nmPtr.get();
+      checkDeletedNM(nmPointer);
       return sizeNM(nmPointer);
     } finally {
       rlock.unlock();
@@ -606,10 +598,8 @@ public class NativeMap implements 
Iterable<Map.Entry<Key,Value>> {
   public long getMemoryUsed() {
     rlock.lock();
     try {
-      if (nmPointer == 0) {
-        throw new IllegalStateException("Native Map Deleted");
-      }
-
+      final long nmPointer = nmPtr.get();
+      checkDeletedNM(nmPointer);
       return memoryUsedNM(nmPointer);
     } finally {
       rlock.unlock();
@@ -620,10 +610,8 @@ public class NativeMap implements 
Iterable<Map.Entry<Key,Value>> {
   public Iterator<Map.Entry<Key,Value>> iterator() {
     rlock.lock();
     try {
-      if (nmPointer == 0) {
-        throw new IllegalStateException("Native Map Deleted");
-      }
-
+      final long nmPointer = nmPtr.get();
+      checkDeletedNM(nmPointer);
       return new ConcurrentIterator();
     } finally {
       rlock.unlock();
@@ -633,11 +621,8 @@ public class NativeMap implements 
Iterable<Map.Entry<Key,Value>> {
   public Iterator<Map.Entry<Key,Value>> iterator(Key startKey) {
     rlock.lock();
     try {
-
-      if (nmPointer == 0) {
-        throw new IllegalStateException("Native Map Deleted");
-      }
-
+      final long nmPointer = nmPtr.get();
+      checkDeletedNM(nmPointer);
       return new ConcurrentIterator(startKey);
     } finally {
       rlock.unlock();
@@ -647,13 +632,13 @@ public class NativeMap implements 
Iterable<Map.Entry<Key,Value>> {
   public void delete() {
     wlock.lock();
     try {
-      if (nmPointer == 0) {
-        throw new IllegalStateException("Native Map Deleted");
-      }
-
+      final long nmPointer = nmPtr.getAndSet(0);
+      checkDeletedNM(nmPointer);
+      // deregister cleanable, but it won't run because it checks
+      // the value of nmPtr first, which is now 0
+      cleanableNM.clean();
       log.debug(String.format("Deallocating native map 0x%016x", nmPointer));
       deleteNativeMap(nmPointer);
-      nmPointer = 0;
     } finally {
       wlock.unlock();
     }
@@ -704,7 +689,7 @@ public class NativeMap implements 
Iterable<Map.Entry<Key,Value>> {
     public void next() {
 
       if (entry == null)
-        throw new IllegalStateException();
+        throw new NoSuchElementException();
 
       // checking the interrupt flag for every call to next had bad a bad 
performance impact
       // so check it every 100th time
@@ -753,7 +738,7 @@ public class NativeMap implements 
Iterable<Map.Entry<Key,Value>> {
     @Override
     public void init(SortedKeyValueIterator<Key,Value> source, 
Map<String,String> options,
         IteratorEnvironment env) {
-      throw new UnsupportedOperationException();
+      throw new UnsupportedOperationException("init");
     }
 
     @Override
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/NativeMapCleanerUtil.java
 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/NativeMapCleanerUtil.java
new file mode 100644
index 0000000..4cd5d92
--- /dev/null
+++ 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/NativeMapCleanerUtil.java
@@ -0,0 +1,54 @@
+/*
+ * 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.accumulo.tserver;
+
+import static java.util.Objects.requireNonNull;
+
+import java.lang.ref.Cleaner.Cleanable;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.accumulo.core.util.cleaner.CleanerUtil;
+import org.slf4j.Logger;
+
+/**
+ * A cleaner utility for NativeMap code, in the same spirit as {@link 
CleanerUtil}.
+ */
+public class NativeMapCleanerUtil {
+
+  public static Cleanable deleteNM(Object obj, Logger log, AtomicLong nmPtr) {
+    requireNonNull(nmPtr);
+    requireNonNull(log);
+    return CleanerUtil.CLEANER.register(obj, () -> {
+      long nmPointer = nmPtr.get();
+      if (nmPointer != 0) {
+        log.warn(String.format("Deallocating native map 0x%016x in finalize", 
nmPointer));
+        NativeMap._deleteNativeMap(nmPointer);
+      }
+    });
+  }
+
+  public static Cleanable deleteNMIterator(Object obj, AtomicLong nmiPtr) {
+    requireNonNull(nmiPtr);
+    return CleanerUtil.CLEANER.register(obj, () -> {
+      long nmiPointer = nmiPtr.get();
+      if (nmiPointer != 0) {
+        NativeMap._deleteNMI(nmiPointer);
+      }
+    });
+  }
+
+}
diff --git 
a/test/src/main/java/org/apache/accumulo/test/BatchWriterIterator.java 
b/test/src/main/java/org/apache/accumulo/test/BatchWriterIterator.java
index 4bf65e8..53916d5 100644
--- a/test/src/main/java/org/apache/accumulo/test/BatchWriterIterator.java
+++ b/test/src/main/java/org/apache/accumulo/test/BatchWriterIterator.java
@@ -29,7 +29,6 @@ import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.clientImpl.ClientInfo;
@@ -42,6 +41,7 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.IteratorEnvironment;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.iterators.WrappingIterator;
+import org.apache.accumulo.core.util.cleaner.CleanerUtil;
 import org.apache.accumulo.test.util.SerializationUtil;
 import org.apache.hadoop.io.Text;
 import org.slf4j.Logger;
@@ -61,6 +61,26 @@ import org.slf4j.LoggerFactory;
 public class BatchWriterIterator extends WrappingIterator {
   private static final Logger log = 
LoggerFactory.getLogger(BatchWriterIterator.class);
 
+  private static final String OPT_sleepAfterFirstWrite = 
"sleepAfterFirstWrite";
+  private static final String OPT_numEntriesToWritePerEntry = 
"numEntriesToWritePerEntry";
+  private static final String OPT_batchWriterTimeout = "batchWriterTimeout";
+  private static final String OPT_batchWriterMaxMemory = 
"batchWriterMaxMemory";
+  private static final String OPT_clearCacheAfterFirstWrite = 
"clearCacheAfterFirstWrite";
+  private static final String OPT_splitAfterFirstWrite = 
"splitAfterFirstWrite";
+
+  private static final String ZOOKEEPERHOST = "zookeeperHost";
+  private static final String INSTANCENAME = "instanceName";
+  private static final String TABLENAME = "tableName";
+  private static final String USERNAME = "username";
+  private static final String ZOOKEEPERTIMEOUT = "zookeeperTimeout";
+  // base64 encoding of token
+  private static final String AUTHENTICATION_TOKEN = "authenticationToken";
+  // class of token
+  private static final String AUTHENTICATION_TOKEN_CLASS = 
"authenticationTokenClass";
+  private static final String SUCCESS_STRING = "success";
+
+  public static final Value SUCCESS_VALUE = new 
Value(SUCCESS_STRING.getBytes());
+
   private Map<String,String> originalOptions; // remembered for deepCopy
 
   private int sleepAfterFirstWrite = 0;
@@ -69,34 +89,17 @@ public class BatchWriterIterator extends WrappingIterator {
   private long batchWriterMaxMemory = 0;
   private boolean clearCacheAfterFirstWrite = false;
   private boolean splitAfterFirstWrite = false;
-
-  public static final String OPT_sleepAfterFirstWrite = "sleepAfterFirstWrite",
-      OPT_numEntriesToWritePerEntry = "numEntriesToWritePerEntry",
-      OPT_batchWriterTimeout = "batchWriterTimeout",
-      OPT_batchWriterMaxMemory = "batchWriterMaxMemory",
-      OPT_clearCacheAfterFirstWrite = "clearCacheAfterFirstWrite",
-      OPT_splitAfterFirstWrite = "splitAfterFirstWrite";
-
   private String instanceName;
   private String tableName;
   private String zookeeperHost;
   private int zookeeperTimeout = -1;
   private String username;
   private AuthenticationToken auth = null;
-
-  public static final String ZOOKEEPERHOST = "zookeeperHost", INSTANCENAME = 
"instanceName",
-      TABLENAME = "tableName", USERNAME = "username", ZOOKEEPERTIMEOUT = 
"zookeeperTimeout",
-      AUTHENTICATION_TOKEN = "authenticationToken", // base64 encoding of token
-      AUTHENTICATION_TOKEN_CLASS = "authenticationTokenClass"; // class of 
token
-
   private BatchWriter batchWriter;
   private boolean firstWrite = true;
   private Value topValue = null;
   private AccumuloClient accumuloClient;
 
-  public static final String SUCCESS_STRING = "success";
-  public static final Value SUCCESS_VALUE = new 
Value(SUCCESS_STRING.getBytes());
-
   public static IteratorSetting iteratorSetting(int priority, int 
sleepAfterFirstWrite,
       long batchWriterTimeout, long batchWriterMaxMemory, int 
numEntriesToWrite, String tableName,
       AccumuloClient accumuloClient, AuthenticationToken token, boolean 
clearCacheAfterFirstWrite,
@@ -167,13 +170,8 @@ public class BatchWriterIterator extends WrappingIterator {
   }
 
   private void initBatchWriter() {
-    try {
-      accumuloClient = Accumulo.newClient().to(instanceName, 
zookeeperHost).as(username, auth)
-          .zkTimeout(zookeeperTimeout).build();
-    } catch (Exception e) {
-      log.error("failed to connect to Accumulo instance " + instanceName, e);
-      throw new RuntimeException(e);
-    }
+    accumuloClient = Accumulo.newClient().to(instanceName, 
zookeeperHost).as(username, auth)
+        .zkTimeout(zookeeperTimeout).build();
 
     BatchWriterConfig bwc = new BatchWriterConfig();
     bwc.setMaxMemory(batchWriterMaxMemory);
@@ -183,8 +181,14 @@ public class BatchWriterIterator extends WrappingIterator {
       batchWriter = accumuloClient.createBatchWriter(tableName, bwc);
     } catch (TableNotFoundException e) {
       log.error(tableName + " does not exist in instance " + instanceName, e);
+      accumuloClient.close();
       throw new RuntimeException(e);
+    } catch (RuntimeException e) {
+      accumuloClient.close();
+      throw e;
     }
+    // this is dubious, but necessary since iterators aren't closeable
+    CleanerUtil.batchWriterAndClientCloser(this, log, batchWriter, 
accumuloClient);
   }
 
   /**
@@ -229,18 +233,6 @@ public class BatchWriterIterator extends WrappingIterator {
   }
 
   @Override
-  protected void finalize() throws Throwable {
-    super.finalize();
-    try {
-      batchWriter.close();
-    } catch (MutationsRejectedException e) {
-      log.error("Failed to close BatchWriter; some mutations may not be 
applied", e);
-    } finally {
-      accumuloClient.close();
-    }
-  }
-
-  @Override
   public void next() throws IOException {
     super.next();
     if (hasTop())
diff --git 
a/test/src/main/java/org/apache/accumulo/test/functional/MasterAssignmentIT.java
 
b/test/src/main/java/org/apache/accumulo/test/functional/MasterAssignmentIT.java
index cd1af04..2414677 100644
--- 
a/test/src/main/java/org/apache/accumulo/test/functional/MasterAssignmentIT.java
+++ 
b/test/src/main/java/org/apache/accumulo/test/functional/MasterAssignmentIT.java
@@ -27,6 +27,7 @@ import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 import org.apache.accumulo.fate.util.UtilWaitThread;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
@@ -88,7 +89,7 @@ public class MasterAssignmentIT extends 
AccumuloClusterHarness {
 
   private TabletLocationState getTabletLocationState(AccumuloClient c, String 
tableId) {
     try (MetaDataTableScanner s = new MetaDataTableScanner((ClientContext) c,
-        new Range(TabletsSection.getRow(TableId.of(tableId), null)))) {
+        new Range(TabletsSection.getRow(TableId.of(tableId), null)), 
MetadataTable.NAME)) {
       return s.next();
     }
   }
diff --git 
a/test/src/main/java/org/apache/accumulo/test/master/SuspendedTabletsIT.java 
b/test/src/main/java/org/apache/accumulo/test/master/SuspendedTabletsIT.java
index 9b5ee89..b6d887f 100644
--- a/test/src/main/java/org/apache/accumulo/test/master/SuspendedTabletsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/master/SuspendedTabletsIT.java
@@ -49,6 +49,7 @@ import org.apache.accumulo.core.conf.ClientProperty;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
@@ -304,7 +305,7 @@ public class SuspendedTabletsIT extends ConfigurableMacBase 
{
     private void scan(ClientContext ctx, String tableName) {
       Map<String,String> idMap = ctx.tableOperations().tableIdMap();
       String tableId = Objects.requireNonNull(idMap.get(tableName));
-      try (MetaDataTableScanner scanner = new MetaDataTableScanner(ctx, new 
Range())) {
+      try (var scanner = new MetaDataTableScanner(ctx, new Range(), 
MetadataTable.NAME)) {
         while (scanner.hasNext()) {
           TabletLocationState tls = scanner.next();
 
diff --git 
a/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java 
b/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java
index 13d3b28..6a94f96 100644
--- a/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java
+++ b/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java
@@ -52,6 +52,7 @@ import org.apache.accumulo.core.dataImpl.thrift.TSummaries;
 import org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest;
 import org.apache.accumulo.core.dataImpl.thrift.UpdateErrors;
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
 import org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction;
 import org.apache.accumulo.core.tabletserver.thrift.ActiveScan;
@@ -309,7 +310,7 @@ public class NullTserver {
     // read the locations for the table
     Range tableRange = new KeyExtent(tableId, null, null).toMetadataRange();
     List<Assignment> assignments = new ArrayList<>();
-    try (MetaDataTableScanner s = new MetaDataTableScanner(context, 
tableRange)) {
+    try (var s = new MetaDataTableScanner(context, tableRange, 
MetadataTable.NAME)) {
       long randomSessionID = opts.port;
       TServerInstance instance = new TServerInstance(addr, randomSessionID);
 

Reply via email to