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 2021/02/11 07:56:09 UTC

[lucene-solr] branch reference_impl_dev updated: @1347 Tighten tests, fix noCallerArg that got turned around.

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

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


The following commit(s) were added to refs/heads/reference_impl_dev by this push:
     new a541e8a  @1347 Tighten tests, fix noCallerArg that got turned around.
a541e8a is described below

commit a541e8a0d77a1cc5a332db9e5de6678a5e3b41ce
Author: markrmiller@gmail.com <ma...@gmail.com>
AuthorDate: Thu Feb 11 01:55:43 2021 -0600

    @1347 Tighten tests, fix noCallerArg that got turned around.
---
 .../java/org/apache/solr/cloud/ZkController.java   |  9 ++--
 .../cloud/api/collections/CreateCollectionCmd.java |  5 ++
 .../apache/solr/cloud/ClusterStateUpdateTest.java  |  5 +-
 .../org/apache/solr/cloud/DeleteReplicaTest.java   |  4 --
 .../src/java/org/apache/solr/common/ParWork.java   |  4 +-
 .../org/apache/solr/common/cloud/SolrZkClient.java | 57 +++++++++++++---------
 .../apache/solr/common/cloud/ZkStateReader.java    |  2 +-
 7 files changed, 47 insertions(+), 39 deletions(-)

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 842287c..9d19e28 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
@@ -1365,15 +1365,14 @@ public class ZkController implements Closeable, Runnable {
           leader = replica;
           break;
         }
-
         try {
-          //          if (getCoreContainer().isShutDown() || isDcCalled() || isClosed()) {
-          //            throw new AlreadyClosedException();
-          //          }
-
           leader = zkStateReader.getLeaderRetry(collection, shardId, 3000, true);
 
         } catch (TimeoutException timeoutException) {
+          if (isClosed() || isDcCalled() || cc.isShutDown()) {
+            throw new AlreadyClosedException();
+          }
+
           log.info("Timeout waiting to see leader, retry");
         }
       }
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java
index 62611e0..4b8c057 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java
@@ -27,6 +27,7 @@ import org.apache.solr.cloud.ZkController;
 import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.ShardRequestTracker;
 import org.apache.solr.cloud.overseer.CollectionMutator;
 import org.apache.solr.cloud.overseer.SliceMutator;
+import org.apache.solr.common.AlreadyClosedException;
 import org.apache.solr.common.ParWork;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
@@ -384,6 +385,10 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
           if (createNodeSet == null || !createNodeSet.equals(ZkStateReader.CREATE_NODE_SET_EMPTY)) {
             try {
               zkStateReader.waitForState(collectionName, 10, TimeUnit.SECONDS, (l, c) -> {
+                if (ocmh.isClosed()) {
+                  throw new AlreadyClosedException();
+                }
+
                 if (c == null) {
                   return false;
                 }
diff --git a/solr/core/src/test/org/apache/solr/cloud/ClusterStateUpdateTest.java b/solr/core/src/test/org/apache/solr/cloud/ClusterStateUpdateTest.java
index 1184200..971e96b 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ClusterStateUpdateTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ClusterStateUpdateTest.java
@@ -55,8 +55,7 @@ public class ClusterStateUpdateTest extends SolrCloudTestCase  {
 
   @AfterClass
   public static void afterClass() throws InterruptedException, IOException {
-    System.clearProperty("solrcloud.skip.autorecovery");
-    System.clearProperty("genericCoreNodeNames");
+
   }
   
   @Test
@@ -84,7 +83,7 @@ public class ClusterStateUpdateTest extends SolrCloudTestCase  {
           && slices.get("s1").getReplicasMap().size() > 0) {
         break;
       }
-      Thread.sleep(500);
+      Thread.sleep(200);
     }
 
     assertNotNull(slices);
diff --git a/solr/core/src/test/org/apache/solr/cloud/DeleteReplicaTest.java b/solr/core/src/test/org/apache/solr/cloud/DeleteReplicaTest.java
index 3552bf4..92e73b1 100644
--- a/solr/core/src/test/org/apache/solr/cloud/DeleteReplicaTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/DeleteReplicaTest.java
@@ -31,7 +31,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.solr.SolrTestCase;
 import org.apache.solr.SolrTestUtil;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
@@ -61,9 +60,6 @@ import org.slf4j.LoggerFactory;
 
 import static org.apache.solr.common.cloud.Replica.State.DOWN;
 
-// TODO: this is flakey, can rarely leak a Directory
-// The UnloadCoreOnDeletedWatcher has been removed
-@SolrTestCase.SuppressObjectReleaseTracker(object = "NRTCachingDirectory")
 public class DeleteReplicaTest extends SolrCloudTestCase {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
diff --git a/solr/solrj/src/java/org/apache/solr/common/ParWork.java b/solr/solrj/src/java/org/apache/solr/common/ParWork.java
index 422d9d3..ee8e605 100644
--- a/solr/solrj/src/java/org/apache/solr/common/ParWork.java
+++ b/solr/solrj/src/java/org/apache/solr/common/ParWork.java
@@ -474,8 +474,8 @@ public class ParWork implements Closeable {
     return new PerThreadExecService(getRootSharedExecutor(), maximumPoolSize);
   }
 
-  public static ExecutorService getExecutorService(int maximumPoolSize, boolean noCallerRunsAllowed, boolean noCallerRunsAvailableLimit) {
-    return new PerThreadExecService(getRootSharedExecutor(), maximumPoolSize, noCallerRunsAllowed, noCallerRunsAvailableLimit);
+  public static ExecutorService getExecutorService(int maximumPoolSize, boolean callerThreadAllowed, boolean noCallerRunsAvailableLimit) {
+    return new PerThreadExecService(getRootSharedExecutor(), maximumPoolSize, callerThreadAllowed, noCallerRunsAvailableLimit);
   }
 
   private void handleObject(AtomicReference<Throwable> exception, final TimeTracker workUnitTracker, ParObject ob) {
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java b/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
index 99e1282..fde3094 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
@@ -24,6 +24,8 @@ import org.apache.solr.common.StringUtils;
 import org.apache.solr.common.cloud.ConnectionManager.IsClosed;
 import org.apache.solr.common.util.CloseTracker;
 import org.apache.solr.common.util.ObjectReleaseTracker;
+import org.apache.solr.common.util.TimeOut;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.zookeeper.AsyncCallback;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
@@ -100,9 +102,9 @@ public class SolrZkClient implements Closeable {
 
   private ZkCmdExecutor zkCmdExecutor;
 
-  protected final ExecutorService zkCallbackExecutor = ParWork.getExecutorService(Integer.MAX_VALUE, true, false);
+  protected final ExecutorService zkCallbackExecutor = ParWork.getExecutorService(Integer.MAX_VALUE, false, false);
      // ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("zkCallback"));
-  protected final ExecutorService zkConnManagerCallbackExecutor = ParWork.getExecutorService(Integer.MAX_VALUE, true, false);
+  protected final ExecutorService zkConnManagerCallbackExecutor = ParWork.getExecutorService(Integer.MAX_VALUE, false, false);
      // ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("zkConnectionManagerCallback"));
 
   private volatile boolean isClosed = false;
@@ -789,38 +791,45 @@ public class SolrZkClient implements Closeable {
     for (String path : paths) {
       if (log.isDebugEnabled()) log.debug("process path={} connManager={}", path, connManager);
       ZooKeeper keeper = connManager.getKeeper();
-      if (log.isDebugEnabled()) log.debug("keeper={}", keeper);
 
       CountDownLatch finalLatch = latch;
       keeper.delete(path, -1, (rc, path1, ctx) -> {
-        if (log.isDebugEnabled()) log.debug("async delete resp rc={}, path1={}, ctx={}", rc, path1, ctx);
-        if (rc != 0) {
-          log.error("got zk error {}", rc);
-          final KeeperException.Code keCode = KeeperException.Code.get(rc);
-
-          if (keCode == KeeperException.Code.NONODE) {
-            if (log.isDebugEnabled()) log.debug("Problem removing zk node {}", path1);
+        try {
+          // MRM TODO:
+          if (log.isDebugEnabled()) log.debug("async delete resp rc={}, path1={}, ctx={}", rc, path1, ctx);
+          if (rc != 0) {
+            log.error("got zk error deleting paths {}", rc);
+            final KeeperException.Code keCode = KeeperException.Code.get(rc);
+
+            if (keCode == KeeperException.Code.NONODE) {
+              if (log.isDebugEnabled()) log.debug("Problem removing zk node {}", path1);
+            }
+          }
+        } finally {
+          if (wait) {
+            finalLatch.countDown();
           }
         }
-        if (log.isDebugEnabled()) log.debug("done with latch countdown wait={}", wait);
-        if (wait) {
-          if (log.isDebugEnabled()) log.debug("latch countdown");
-          finalLatch.countDown();
-        }
-        if (log.isDebugEnabled()) log.debug("after wait &&  latch countdown");
       }, null);
+
     }
 
     if (log.isDebugEnabled()) log.debug("done with all paths, see if wait ... wait={}", wait);
     if (wait) {
-      boolean success;
-      try {
-        success = latch.await(15, TimeUnit.SECONDS);
-        if (log.isDebugEnabled()) log.debug("done waiting on latch, success={}", success);
-      } catch (InterruptedException e) {
-        ParWork.propagateInterrupt(e);
-        log.error("", e);
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+      TimeOut timeout = new TimeOut(15, TimeUnit.SECONDS, TimeSource.NANO_TIME);
+      boolean success = false;
+      while (!timeout.hasTimedOut() && !isClosed) {
+        try {
+          success = latch.await(3, TimeUnit.SECONDS);
+          if (log.isDebugEnabled()) log.debug("done waiting on latch, success={}", success);
+          if (success) {
+            break;
+          }
+        } catch (InterruptedException e) {
+          ParWork.propagateInterrupt(e);
+          log.error("", e);
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+        }
       }
 
       if (!success) {
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 1318413..3176dba 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
@@ -228,7 +228,7 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
 
   private Set<CloudCollectionsListener> cloudCollectionsListeners = ConcurrentHashMap.newKeySet();
 
-  private final ExecutorService notifications = ParWork.getExecutorService(Integer.MAX_VALUE, true, false);
+  private final ExecutorService notifications = ParWork.getExecutorService(Integer.MAX_VALUE, false, false);
 
   private final Set<LiveNodesListener> liveNodesListeners = ConcurrentHashMap.newKeySet();