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);
+ }
+ }
}