This is an automated email from the ASF dual-hosted git repository. mmiller pushed a commit to branch main in repository https://gitbox.apache.org/repos/asf/accumulo.git
The following commit(s) were added to refs/heads/main by this push: new bc921ba Make CleanUp shutdownNow close Connector (#2332) bc921ba is described below commit bc921baf1f96adf1ccb5aec7a8f8a5b6ae66b3bd Author: Mike Miller <mmil...@apache.org> AuthorDate: Thu Oct 28 11:28:34 2021 -0400 Make CleanUp shutdownNow close Connector (#2332) * Recent changes in #2303 made static ThriftTransportPool a part of ClientContext. In order to allow legacy Connector clean up the threads as it did previously, make the shutdownNow method take a connector and call close on it. Made null an option in case the Connector is unavailable to pass as a parameter. * Fixes failure in CleanUpIT. Partial fix of #2327 --- core/src/main/java/org/apache/accumulo/core/util/CleanUp.java | 11 ++++++++++- .../java/org/apache/accumulo/test/functional/CleanUpIT.java | 2 +- 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/util/CleanUp.java b/core/src/main/java/org/apache/accumulo/core/util/CleanUp.java index af83fa9..0b31365 100644 --- a/core/src/main/java/org/apache/accumulo/core/util/CleanUp.java +++ b/core/src/main/java/org/apache/accumulo/core/util/CleanUp.java @@ -21,6 +21,8 @@ package org.apache.accumulo.core.util; import java.util.Set; import org.apache.accumulo.core.client.AccumuloClient; +import org.apache.accumulo.core.client.Connector; +import org.apache.accumulo.core.clientImpl.ConnectorImpl; import org.apache.accumulo.core.singletons.SingletonManager; import org.apache.accumulo.core.singletons.SingletonManager.Mode; import org.slf4j.Logger; @@ -47,10 +49,17 @@ public class CleanUp { /** * kills all threads created by internal Accumulo singleton resources. After this method is * called, no Connector will work in the current classloader. + * + * @param conn + * If available, Connector object to close resources on. Will accept null otherwise. */ - public static void shutdownNow() { + public static void shutdownNow(Connector conn) { SingletonManager.setMode(Mode.CLIENT); waitForZooKeeperClientThreads(); + if (conn != null) { + ConnectorImpl connImpl = (ConnectorImpl) conn; + connImpl.getAccumuloClient().close(); + } } /** diff --git a/test/src/main/java/org/apache/accumulo/test/functional/CleanUpIT.java b/test/src/main/java/org/apache/accumulo/test/functional/CleanUpIT.java index 34309cd..4e3c153 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/CleanUpIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/CleanUpIT.java @@ -110,7 +110,7 @@ public class CleanUpIT extends SharedMiniClusterBase { fail("Not seeing expected threads. Saw " + threadCount); } - org.apache.accumulo.core.util.CleanUp.shutdownNow(); + org.apache.accumulo.core.util.CleanUp.shutdownNow(conn); Mutation m2 = new Mutation("r2"); m2.put("cf1", "cq1", 1, "6");