You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ma...@apache.org on 2020/07/09 21:01:44 UTC

[lucene-solr] 02/23: #1 A few additions to address TestCloudConsistency fail with a few related cleanups and a couple other test fail fixes.

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

markrmiller pushed a commit to branch reference_impl
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git

commit cd2ded5e8be8b305427932f7b810e0b2d453c65c
Author: markrmiller@gmail.com <ma...@gmail.com>
AuthorDate: Tue Jun 9 11:59:45 2020 -0500

    #1 A few additions to address TestCloudConsistency fail with a few related cleanups and a couple other test fail fixes.
---
 .../solr/cloud/ShardLeaderElectionContext.java     | 14 +-----
 .../java/org/apache/solr/cloud/ZkController.java   | 56 ++++++++--------------
 .../java/org/apache/solr/cloud/ZkShardTerms.java   |  5 ++
 .../solr/cloud/api/collections/AliasCmd.java       |  4 +-
 .../apache/solr/cloud/overseer/SliceMutator.java   |  6 ++-
 .../java/org/apache/solr/core/CoreContainer.java   | 14 +++++-
 .../solr/handler/admin/CollectionsHandler.java     |  3 +-
 .../java/org/apache/solr/util/OrderedExecutor.java |  8 ++++
 .../TestCollectionsAPIViaSolrCloudCluster.java     |  2 -
 .../apache/solr/common/cloud/ZkStateReader.java    | 39 ++++++---------
 .../java/org/apache/solr/common/util/IOUtils.java  | 10 ++++
 11 files changed, 80 insertions(+), 81 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContext.java b/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContext.java
index 4be8259..6028b76 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContext.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContext.java
@@ -112,19 +112,7 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
 
       int leaderVoteWait = cc.getZkController().getLeaderVoteWait();
 
-      log.debug("Running the leader process for shard={} and weAreReplacement={} and leaderVoteWait={}", shardId, weAreReplacement, leaderVoteWait);
-      if (zkController.getClusterState().getCollection(collection).getSlice(shardId).getReplicas().size() > 1) {
-        // Clear the leader in clusterstate. We only need to worry about this if there is actually more than one replica.
-        ZkNodeProps m = ZkNodeProps.fromKeyVals(Overseer.QUEUE_OPERATION, OverseerAction.LEADER.toLower(),
-                ZkStateReader.SHARD_ID_PROP, shardId,
-                ZkStateReader.COLLECTION_PROP, collection,
-                ZkStateReader.BASE_URL_PROP, leaderProps.get(ZkStateReader.BASE_URL_PROP),
-                ZkStateReader.NODE_NAME_PROP, leaderProps.get(ZkStateReader.NODE_NAME_PROP),
-                ZkStateReader.CORE_NODE_NAME_PROP, leaderProps.get(ZkStateReader.CORE_NODE_NAME_PROP),
-                ZkStateReader.CORE_NAME_PROP, leaderProps.get(ZkStateReader.CORE_NAME_PROP));
-
-        zkController.getOverseer().getStateUpdateQueue().offer(Utils.toJSON(m));
-      }
+      log.info("Running the leader process for shard={} and weAreReplacement={} and leaderVoteWait={}", shardId, weAreReplacement, leaderVoteWait);
 
       if (isClosed) {
         // Solr is shutting down or the ZooKeeper session expired while waiting for replicas. If the later,
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkController.java b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
index 1e4db6e..c3d07a6 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
@@ -591,45 +591,13 @@ public class ZkController implements Closeable {
     }
   }
 
-  public void preClose() {
-    this.isClosed = true;
-
-    try {
-      this.removeEphemeralLiveNode();
-    } catch (AlreadyClosedException | SessionExpiredException | KeeperException.ConnectionLossException e) {
-
-    } catch (Exception e) {
-      log.warn("Error removing live node. Continuing to close CoreContainer", e);
-    }
-
-    try {
-      if (getZkClient().getConnectionManager().isConnected()) {
-        log.info("Publish this node as DOWN...");
-        publishNodeAsDown(getNodeName());
-      }
-    } catch (Exception e) {
-      log.warn("Error publishing nodes as down. Continuing to close CoreContainer", e);
-    }
-
-    ExecutorService customThreadPool = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("preCloseThreadPool"));
-
-    try {
-      synchronized (collectionToTerms) {
-        customThreadPool.submit(() -> collectionToTerms.values().parallelStream().forEach(ZkCollectionTerms::close));
-      }
-
-      customThreadPool.submit(() -> replicateFromLeaders.values().parallelStream().forEach(ReplicateFromLeader::stopReplication));
-    } finally {
-      ExecutorUtil.shutdownAndAwaitTermination(customThreadPool);
-    }
-  }
-
   /**
    * Closes the underlying ZooKeeper client.
    */
   public void close() {
-    if (!this.isClosed)
-      preClose();
+    if (this.isClosed) {
+      throw new AlreadyClosedException();
+    }
 
     ExecutorService customThreadPool = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("closeThreadPool"));
 
@@ -641,11 +609,25 @@ public class ZkController implements Closeable {
       customThreadPool.submit(() -> electionContexts.values().parallelStream().forEach(IOUtils::closeQuietly));
 
     } finally {
+      try {
+        if (getZkClient().getConnectionManager().isConnected()) {
+          log.info("Publish this node as DOWN...");
+          publishNodeAsDown(getNodeName());
+        }
+      } catch (Exception e) {
+        if (e instanceof  InterruptedException) {
+          Thread.currentThread().interrupt();
+        }
+        log.warn("Error publishing nodes as down. Continuing to close CoreContainer", e);
+      }
 
-      sysPropsCacher.close();
       customThreadPool.submit(() -> Collections.singleton(cloudSolrClient).parallelStream().forEach(IOUtils::closeQuietly));
       customThreadPool.submit(() -> Collections.singleton(cloudManager).parallelStream().forEach(IOUtils::closeQuietly));
-
+      synchronized (collectionToTerms) {
+        customThreadPool.submit(() -> collectionToTerms.values().parallelStream().forEach(IOUtils::closeQuietly));
+      }
+      customThreadPool.submit(() -> replicateFromLeaders.values().parallelStream().forEach(ReplicateFromLeader::stopReplication));
+      sysPropsCacher.close();
       try {
         try {
           zkStateReader.close();
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkShardTerms.java b/solr/core/src/java/org/apache/solr/cloud/ZkShardTerms.java
index bd446c4..be49409 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkShardTerms.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkShardTerms.java
@@ -111,6 +111,7 @@ public class ZkShardTerms implements AutoCloseable{
    * @param replicasNeedingRecovery set of replicas in which their terms should be lower than leader's term
    */
   public void ensureTermsIsHigher(String leader, Set<String> replicasNeedingRecovery) {
+    log.info("leader={} replicasNeedingRecvoery={}", leader, replicasNeedingRecovery);
     if (replicasNeedingRecovery.isEmpty()) return;
 
     ShardTerms newTerms;
@@ -304,6 +305,7 @@ public class ZkShardTerms implements AutoCloseable{
    * @throws KeeperException.NoNodeException correspond ZK term node is not created
    */
   private boolean saveTerms(ShardTerms newTerms) throws KeeperException.NoNodeException {
+    log.info("Save terms={}", newTerms);
     byte[] znodeData = Utils.toJSON(newTerms);
     try {
       Stat stat = zkClient.setData(znodePath, znodeData, newTerms.getVersion(), true);
@@ -316,6 +318,9 @@ public class ZkShardTerms implements AutoCloseable{
     } catch (KeeperException.NoNodeException e) {
       throw e;
     } catch (Exception e) {
+      if (e instanceof  InterruptedException) {
+        Thread.currentThread().interrupt();
+      }
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error while saving shard term for collection: " + collection, e);
     }
     return false;
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/AliasCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/AliasCmd.java
index 3643d99..6096e89 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/AliasCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/AliasCmd.java
@@ -32,6 +32,7 @@ import org.apache.solr.common.util.NamedList;
 import org.apache.solr.handler.admin.CollectionsHandler;
 import org.apache.solr.request.LocalSolrQueryRequest;
 
+import static org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.CREATE_NODE_SET_EMPTY;
 import static org.apache.solr.cloud.api.collections.RoutedAlias.CREATE_COLLECTION_PREFIX;
 import static org.apache.solr.cloud.api.collections.RoutedAlias.ROUTED_ALIAS_NAME_CORE_PROP;
 import static org.apache.solr.common.params.CollectionAdminParams.COLL_CONF;
@@ -101,7 +102,8 @@ abstract class AliasCmd implements OverseerCollectionMessageHandler.Cmd {
       numShards = shards.split(",").length;
     }
 
-    if ("".equals(zkProps.getStr(OverseerCollectionMessageHandler.CREATE_NODE_SET))) {
+    if (CREATE_NODE_SET_EMPTY.equals(zkProps.getStr(OverseerCollectionMessageHandler.CREATE_NODE_SET))
+            || "".equals(zkProps.getStr(OverseerCollectionMessageHandler.CREATE_NODE_SET))) {
       nrtReplicas = 0;
       pullReplicas = 0;
       tlogReplicas = 0;
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/SliceMutator.java b/solr/core/src/java/org/apache/solr/cloud/overseer/SliceMutator.java
index f63253b..28d3213 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/SliceMutator.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/SliceMutator.java
@@ -112,6 +112,8 @@ public class SliceMutator {
   }
 
   public ZkWriteCommand setShardLeader(ClusterState clusterState, ZkNodeProps message) {
+    log.info("setShardLeader(ClusterState clusterState={}, ZkNodeProps message={}) - start", clusterState, message);
+
     StringBuilder sb = new StringBuilder();
     String baseUrl = message.getStr(ZkStateReader.BASE_URL_PROP);
     String coreName = message.getStr(ZkStateReader.CORE_NAME_PROP);
@@ -139,11 +141,13 @@ public class SliceMutator {
     final Map<String, Replica> newReplicas = new LinkedHashMap<>();
     for (Replica replica : slice.getReplicas()) {
       // TODO: this should only be calculated once and cached somewhere?
-      String coreURL = ZkCoreNodeProps.getCoreUrl(replica.getStr(ZkStateReader.BASE_URL_PROP), replica.getStr(ZkStateReader.CORE_NAME_PROP));
+      log.info("examine for setting or unsetting as leader replica={}", replica);
 
       if (replica == oldLeader && !coreNodeName.equals(replica.getName())) {
+        log.info("Unset leader");
         replica = new ReplicaMutator(cloudManager).unsetLeader(replica);
       } else if (coreNodeName.equals(replica.getName())) {
+        log.info("Set leader");
         replica = new ReplicaMutator(cloudManager).setLeader(replica);
       }
 
diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index c011a64..2ab1ff1 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -965,14 +965,17 @@ public class CoreContainer {
       log.info("Shutting down CoreContainer instance={}", System.identityHashCode(this));
     }
 
-    ExecutorUtil.shutdownAndAwaitTermination(coreContainerAsyncTaskExecutor);
+    // stop accepting new tasks
+    replayUpdatesExecutor.shutdown();
+    coreContainerAsyncTaskExecutor.shutdown();
+    coreContainerWorkExecutor.shutdown();
+
     ExecutorService customThreadPool = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("closeThreadPool"));
 
     isShutDown = true;
     try {
       if (isZooKeeperAware()) {
         cancelCoreRecoveries();
-        zkSys.zkController.preClose();
       }
 
       ExecutorUtil.shutdownAndAwaitTermination(coreContainerWorkExecutor);
@@ -1045,6 +1048,9 @@ public class CoreContainer {
           });
         }
       } catch (Exception e) {
+        if (e instanceof  InterruptedException) {
+          Thread.currentThread().interrupt();
+        }
         log.warn("Error shutting down CoreAdminHandler. Continuing to close CoreContainer.", e);
       }
       if (solrClientCache != null) {
@@ -1071,6 +1077,10 @@ public class CoreContainer {
             zkSys.close();
           } finally {
             ExecutorUtil.shutdownAndAwaitTermination(customThreadPool);
+            replayUpdatesExecutor.awaitTermination();
+            ExecutorUtil.awaitTermination(coreContainerAsyncTaskExecutor);
+            ExecutorUtil.awaitTermination(coreContainerWorkExecutor);
+
           }
         }
 
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
index 384c21b..5ac0038 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
@@ -304,7 +304,8 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
             numShards = shards.split(",").length;
           }
 
-          if ("".equals(zkProps.getStr(OverseerCollectionMessageHandler.CREATE_NODE_SET))) {
+          if (CREATE_NODE_SET_EMPTY.equals(zkProps.getStr(OverseerCollectionMessageHandler.CREATE_NODE_SET))
+                  || "".equals(zkProps.getStr(OverseerCollectionMessageHandler.CREATE_NODE_SET))) {
             nrtReplicas = 0;
             pullReplicas = 0;
             tlogReplicas = 0;
diff --git a/solr/core/src/java/org/apache/solr/util/OrderedExecutor.java b/solr/core/src/java/org/apache/solr/util/OrderedExecutor.java
index 69399c4..fe45aee 100644
--- a/solr/core/src/java/org/apache/solr/util/OrderedExecutor.java
+++ b/solr/core/src/java/org/apache/solr/util/OrderedExecutor.java
@@ -77,10 +77,18 @@ public class OrderedExecutor implements Executor {
     }
   }
 
+  public void shutdown() {
+    delegate.shutdown();
+  }
+
   public void shutdownAndAwaitTermination() {
     ExecutorUtil.shutdownAndAwaitTermination(delegate);
   }
 
+  public void awaitTermination() {
+    ExecutorUtil.awaitTermination(delegate);
+  }
+
   /** A set of locks by a key {@code T}, kind of like Google Striped but the keys are sparse/lazy. */
   private static class SparseStripedLock<T> {
     private ConcurrentHashMap<T, CountDownLatch> map = new ConcurrentHashMap<>();
diff --git a/solr/core/src/test/org/apache/solr/cloud/api/collections/TestCollectionsAPIViaSolrCloudCluster.java b/solr/core/src/test/org/apache/solr/cloud/api/collections/TestCollectionsAPIViaSolrCloudCluster.java
index eed4c64..fcf9779 100644
--- a/solr/core/src/test/org/apache/solr/cloud/api/collections/TestCollectionsAPIViaSolrCloudCluster.java
+++ b/solr/core/src/test/org/apache/solr/cloud/api/collections/TestCollectionsAPIViaSolrCloudCluster.java
@@ -205,8 +205,6 @@ public class TestCollectionsAPIViaSolrCloudCluster extends SolrCloudTestCase {
 
     // delete the collection
     CollectionAdminRequest.deleteCollection(collectionName).process(client);
-    AbstractDistribZkTestBase.waitForCollectionToDisappear
-        (collectionName, client.getZkStateReader(), true, 330);
   }
 
   @Test
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
index 4d50c8e..732853f 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
@@ -173,7 +173,7 @@ public class ZkStateReader implements SolrCloseable {
   /**
    * Last seen ZK version of clusterstate.json.
    */
-  private int legacyClusterStateVersion = 0;
+  private volatile int legacyClusterStateVersion = 0;
 
   /**
    * Collections with format2 state.json, "interesting" and actively watched.
@@ -225,7 +225,7 @@ public class ZkStateReader implements SolrCloseable {
 
   private static final long LAZY_CACHE_TIME = TimeUnit.NANOSECONDS.convert(STATE_UPDATE_DELAY, TimeUnit.MILLISECONDS);
 
-  private Future<?> collectionPropsCacheCleaner; // only kept to identify if the cleaner has already been started.
+  private volatile Future<?> collectionPropsCacheCleaner; // only kept to identify if the cleaner has already been started.
 
   /**
    * Get current {@link AutoScalingConfig}.
@@ -494,12 +494,14 @@ public class ZkStateReader implements SolrCloseable {
       InterruptedException {
     // We need to fetch the current cluster state and the set of live nodes
 
-    log.debug("Updating cluster state from ZooKeeper... ");
+    if (log.isDebugEnabled()) {
+      log.debug("Updating cluster state from ZooKeeper... ");
+    }
 
     // Sanity check ZK structure.
     if (!zkClient.exists(CLUSTER_STATE, true)) {
       throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE,
-          "Cannot connect to cluster at " + zkClient.getZkServerAddress() + ": cluster not found/not ready");
+              "Cannot connect to cluster at " + zkClient.getZkServerAddress() + ": cluster not found/not ready");
     }
 
     // on reconnect of SolrZkClient force refresh and re-add watches.
@@ -556,8 +558,6 @@ public class ZkStateReader implements SolrCloseable {
                   log.error("Error running collections node listener", e);
                 }
               }
-            } catch (KeeperException.ConnectionLossException | KeeperException.SessionExpiredException e) {
-              log.warn("ZooKeeper watch triggered, but Solr cannot talk to ZK: [{}]", e.getMessage());
             } catch (KeeperException e) {
               log.error("A ZK error has occurred", e);
               throw new ZooKeeperException(ErrorCode.SERVER_ERROR, "", e);
@@ -1125,7 +1125,6 @@ public class ZkStateReader implements SolrCloseable {
   @SuppressWarnings("unchecked")
   private void loadClusterProperties() {
     try {
-      while (true) {
         try {
           byte[] data = zkClient.getData(ZkStateReader.CLUSTER_PROPS, clusterPropertiesWatcher, new Stat(), true);
           this.clusterProperties = ClusterProperties.convertCollectionDefaultsToNestedFormat((Map<String, Object>) Utils.fromJSON(data));
@@ -1137,15 +1136,21 @@ public class ZkStateReader implements SolrCloseable {
           return;
         } catch (KeeperException.NoNodeException e) {
           this.clusterProperties = Collections.emptyMap();
-          log.debug("Loaded empty cluster properties");
+          if (log.isDebugEnabled()) {
+            log.debug("Loaded empty cluster properties");
+          }
           // set an exists watch, and if the node has been created since the last call,
           // read the data again
           if (zkClient.exists(ZkStateReader.CLUSTER_PROPS, clusterPropertiesWatcher, true) == null)
             return;
         }
-      }
-    } catch (KeeperException | InterruptedException e) {
+    } catch (KeeperException e) {
       log.error("Error reading cluster properties from zookeeper", SolrZkClient.checkInterrupted(e));
+      if (e instanceof KeeperException.SessionExpiredException) {
+        throw new SolrException(ErrorCode.SERVER_ERROR, e);
+      }
+    } catch (InterruptedException e) {
+      throw new SolrException(ErrorCode.SERVER_ERROR, "Interrupted");
     }
   }
 
@@ -1345,8 +1350,6 @@ public class ZkStateReader implements SolrCloseable {
           constructState(Collections.singleton(coll));
         }
 
-      } catch (KeeperException.SessionExpiredException | KeeperException.ConnectionLossException e) {
-        log.warn("ZooKeeper watch triggered, but Solr cannot talk to ZK: [{}]", e.getMessage());
       } catch (KeeperException e) {
         log.error("Unwatched collection: [{}]", coll, e);
         throw new ZooKeeperException(ErrorCode.SERVER_ERROR, "A ZK error has occurred", e);
@@ -1382,8 +1385,6 @@ public class ZkStateReader implements SolrCloseable {
       } catch (KeeperException.NoNodeException e) {
         throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE,
             "Cannot connect to cluster at " + zkClient.getZkServerAddress() + ": cluster not found/not ready");
-      } catch (KeeperException.SessionExpiredException | KeeperException.ConnectionLossException e) {
-        log.warn("ZooKeeper watch triggered, but Solr cannot talk to ZK: [{}]", e.getMessage());
       } catch (KeeperException e) {
         log.error("A ZK error has occurred", e);
         throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "A ZK error has occurred", e);
@@ -1469,8 +1470,6 @@ public class ZkStateReader implements SolrCloseable {
             }
           }
         }
-      } catch (KeeperException.SessionExpiredException | KeeperException.ConnectionLossException e) {
-        log.warn("ZooKeeper watch triggered, but Solr cannot talk to ZK: [{}]", e.getMessage());
       } catch (KeeperException e) {
         log.error("Lost collection property watcher for {} due to ZK error", coll, e);
         throw new ZooKeeperException(ErrorCode.SERVER_ERROR, "A ZK error has occurred", e);
@@ -1509,8 +1508,6 @@ public class ZkStateReader implements SolrCloseable {
     public void refreshAndWatch() {
       try {
         refreshCollectionList(this);
-      } catch (KeeperException.SessionExpiredException | KeeperException.ConnectionLossException e) {
-        log.warn("ZooKeeper watch triggered, but Solr cannot talk to ZK: [{}]", e.getMessage());
       } catch (KeeperException e) {
         log.error("A ZK error has occurred", e);
         throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "A ZK error has occurred", e);
@@ -1542,8 +1539,6 @@ public class ZkStateReader implements SolrCloseable {
     public void refreshAndWatch() {
       try {
         refreshLiveNodes(this);
-      } catch (KeeperException.SessionExpiredException | KeeperException.ConnectionLossException e) {
-        log.warn("ZooKeeper watch triggered, but Solr cannot talk to ZK: [{}]", e.getMessage());
       } catch (KeeperException e) {
         log.error("A ZK error has occurred", e);
         throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "A ZK error has occurred", e);
@@ -1844,10 +1839,6 @@ public class ZkStateReader implements SolrCloseable {
   public void waitForLiveNodes(long wait, TimeUnit unit, LiveNodesPredicate predicate)
       throws InterruptedException, TimeoutException {
 
-    if (closed) {
-      throw new AlreadyClosedException();
-    }
-
     final CountDownLatch latch = new CountDownLatch(1);
     waitLatches.add(latch);
 
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/IOUtils.java b/solr/solrj/src/java/org/apache/solr/common/util/IOUtils.java
index 198a664..0e7675d 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/IOUtils.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/IOUtils.java
@@ -34,4 +34,14 @@ public class IOUtils {
       log.error("Error while closing", e);
     }
   }
+
+  public static void closeQuietly(AutoCloseable closeable) {
+    try {
+      if (closeable != null) {
+        closeable.close();
+      }
+    } catch (Exception e) {
+      log.error("Error while closing", e);
+    }
+  }
 }