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/09/09 01:27:04 UTC
[lucene-solr] 01/02: @812 Test hardening after the small deluge.
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
commit f50228ad6e88a9e8aa31fbc69343cb3f6aee7576
Author: markrmiller@gmail.com <ma...@gmail.com>
AuthorDate: Tue Sep 8 20:07:19 2020 -0500
@812 Test hardening after the small deluge.
---
.../java/org/apache/solr/cloud/ZkController.java | 1 -
.../solr/cloud/api/collections/AddReplicaCmd.java | 7 +-
.../solr/cloud/api/collections/ReplaceNodeCmd.java | 4 +-
.../src/java/org/apache/solr/core/SolrCore.java | 31 +++-----
.../org/apache/solr/cloud/ReplaceNodeTest.java | 88 ++++++++++------------
.../solrj/io/stream/StreamDecoratorTest.java | 1 -
6 files changed, 56 insertions(+), 76 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 a84dd7d..f957f19 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
@@ -1962,7 +1962,6 @@ public class ZkController implements Closeable {
synchronized (collectionToTerms) {
ZkCollectionTerms ct = collectionToTerms.get(collection);
if (ct != null) {
- ct.close();
ct.remove(cd.getCloudDescriptor().getShardId(), cd);
}
}
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/AddReplicaCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/AddReplicaCmd.java
index d55b0a2..4e04c5c 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/AddReplicaCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/AddReplicaCmd.java
@@ -202,11 +202,8 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
Runnable runnable = () -> {
try {
shardRequestTracker.processResponses(results, shardHandler, true, "ADDREPLICA failed to create replica");
- } catch (KeeperException e) {
- log.error("ZooKeeper exception", e);
- return;
- } catch (InterruptedException e) {
- log.error("Interrupted", e);
+ } catch (Exception e) {
+ ParWork.propegateInterrupt(e);
return;
}
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/ReplaceNodeCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/ReplaceNodeCmd.java
index 620c735..b7c5cb4 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/ReplaceNodeCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/ReplaceNodeCmd.java
@@ -163,8 +163,8 @@ public class ReplaceNodeCmd implements OverseerCollectionMessageHandler.Cmd {
String key = collectionName + "_" + replicaName;
CollectionStateWatcher watcher;
if (waitForFinalState) {
- watcher = new ActiveReplicaWatcher(collectionName, null,
- Collections.singletonList(addedReplica.getStr(ZkStateReader.CORE_NAME_PROP)), replicasToRecover);
+ watcher = new ActiveReplicaWatcher(collectionName, Collections.singletonList(replicaName),
+ null, replicasToRecover);
} else {
watcher = new LeaderRecoveryWatcher(collectionName, shardName, replicaName,
addedReplica.getStr(ZkStateReader.CORE_NAME_PROP), replicasToRecover);
diff --git a/solr/core/src/java/org/apache/solr/core/SolrCore.java b/solr/core/src/java/org/apache/solr/core/SolrCore.java
index 0c7c6e7..4baafcf 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -1539,7 +1539,11 @@ public final class SolrCore implements SolrInfoBean, Closeable {
* expert: increments the core reference count
*/
public void open() {
- refCount.incrementAndGet();
+ if (isClosed()) {
+ throw new AlreadyClosedException();
+ }
+ int cnt = refCount.incrementAndGet();
+ if (log.isDebugEnabled()) log.debug("open refcount {} {}", this, cnt);
MDCLoggingContext.setCore(this);
}
@@ -1571,6 +1575,7 @@ public final class SolrCore implements SolrInfoBean, Closeable {
@Override
public void close() {
int count = refCount.decrementAndGet();
+ if (log.isDebugEnabled()) log.debug("close refcount {} {}", this, count);
if (count > 0) return; // close is called often, and only actually closes if nothing is using it.
if (count < 0) {
log.error("Too many close [count:{}] on {}. Please report this exception to solr-user@lucene.apache.org", count, this);
@@ -1728,6 +1733,7 @@ public final class SolrCore implements SolrInfoBean, Closeable {
infoRegistry.clear();
//areAllSearcherReferencesEmpty();
+ if (log.isDebugEnabled()) log.debug("close done refcount {} {}", this, -1);
refCount.set(-1);
synchronized (closeAndWait) {
closeAndWait.notifyAll();
@@ -1748,7 +1754,7 @@ public final class SolrCore implements SolrInfoBean, Closeable {
* Whether this core is closed.
*/
public boolean isClosed() {
- return refCount.get() <= 0;
+ return refCount.get() < 0;
}
private final Collection<CloseHook> closeHooks = ConcurrentHashMap.newKeySet(128);
@@ -3173,26 +3179,11 @@ public final class SolrCore implements SolrInfoBean, Closeable {
if (cfg != null) {
cfg.refreshRequestParams();
}
- if (checkStale(zkClient, overlayPath, solrConfigversion) ||
- checkStale(zkClient, solrConfigPath, overlayVersion) ||
- checkStale(zkClient, managedSchmaResourcePath, managedSchemaVersion)) {
+ if (checkStale(zkClient, overlayPath, solrConfigversion) || checkStale(zkClient, solrConfigPath, overlayVersion) || checkStale(zkClient, managedSchmaResourcePath, managedSchemaVersion)) {
log.info("core reload {}", coreName);
- SolrConfigHandler configHandler = ((SolrConfigHandler) core.getRequestHandler("/config"));
- // if (configHandler.getReloadLock().tryLock()) {
-
- try {
- cc.reload(coreName);
- } catch (SolrCoreState.CoreIsClosedException e) {
- /*no problem this core is already closed*/
- } finally {
- // configHandler.getReloadLock().unlock();
- }
-
-// } else {
-// log.info("Another reload is in progress. Not doing anything.");
-// }
- return;
+ cc.reload(coreName);
}
+
//some files in conf directory may have other than managedschema, overlay, params
try (ParWork worker = new ParWork("ConfListeners")) {
diff --git a/solr/core/src/test/org/apache/solr/cloud/ReplaceNodeTest.java b/solr/core/src/test/org/apache/solr/cloud/ReplaceNodeTest.java
index 34edcad..4ce912f 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ReplaceNodeTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ReplaceNodeTest.java
@@ -25,6 +25,7 @@ import java.util.EnumSet;
import java.util.Iterator;
import java.util.List;
import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.client.solrj.impl.CloudHttp2SolrClient;
@@ -37,6 +38,7 @@ import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.params.CollectionParams;
+import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.StrUtils;
@@ -48,6 +50,9 @@ import org.slf4j.LoggerFactory;
public class ReplaceNodeTest extends SolrCloudTestCase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+ private static AtomicInteger asyncId = new AtomicInteger();
+
@BeforeClass
public static void setupCluster() throws Exception {
configureCluster(6)
@@ -93,10 +98,11 @@ public class ReplaceNodeTest extends SolrCloudTestCase {
DocCollection collection = cloudClient.getZkStateReader().getClusterState().getCollection(coll);
log.debug("### Before decommission: {}", collection);
log.info("excluded_node : {} ", emptyNode);
- createReplaceNodeRequest(node2bdecommissioned, emptyNode, null).processAsync("000", cloudClient);
- CollectionAdminRequest.RequestStatus requestStatus = CollectionAdminRequest.requestStatus("000");
+ String asyncId0 = Integer.toString(asyncId.incrementAndGet());
+ createReplaceNodeRequest(node2bdecommissioned, emptyNode, null).processAsync(asyncId0, cloudClient);
+ CollectionAdminRequest.RequestStatus requestStatus = CollectionAdminRequest.requestStatus(asyncId0);
boolean success = false;
- for (int i = 0; i < 10; i++) {
+ for (int i = 0; i < 20; i++) {
CollectionAdminRequest.RequestStatusResponse rsp = requestStatus.process(cloudClient);
if (rsp.getRequestStatus() == RequestStatusState.COMPLETED) {
success = true;
@@ -106,12 +112,18 @@ public class ReplaceNodeTest extends SolrCloudTestCase {
Thread.sleep(500);
}
assertTrue(success);
- try (Http2SolrClient coreclient = SolrTestCaseJ4.getHttpSolrClient(cloudClient.getZkStateReader().getBaseUrlForNodeName(node2bdecommissioned))) {
- CoreAdminResponse status = CoreAdminRequest.getStatus(null, coreclient);
- assertTrue(status.getCoreStatus().size() == 0);
- }
+ Http2SolrClient coreclient = cloudClient.getHttpClient();
+
+ String url = cloudClient.getZkStateReader().getBaseUrlForNodeName(node2bdecommissioned);
+ CoreAdminRequest req = new CoreAdminRequest();
+ req.setBasePath(url);
+ req.setCoreName(null);
+ req.setAction(CoreAdminParams.CoreAdminAction.STATUS);
+
+ CoreAdminResponse status = req.process(coreclient);
+ assertTrue(status.getCoreStatus().size() == 0);
+
- Thread.sleep(1000);
collection = cloudClient.getZkStateReader().getClusterState().getCollection(coll);
log.debug("### After decommission: {}", collection);
// check what are replica states on the decommissioned node
@@ -122,33 +134,29 @@ public class ReplaceNodeTest extends SolrCloudTestCase {
log.debug("### Existing replicas on decommissioned node: {}", replicas);
//let's do it back - this time wait for recoveries
- CollectionAdminRequest.AsyncCollectionAdminRequest replaceNodeRequest = createReplaceNodeRequest(emptyNode, node2bdecommissioned, Boolean.TRUE);
- replaceNodeRequest.setWaitForFinalState(true);
- replaceNodeRequest.processAsync("001", cloudClient);
- requestStatus = CollectionAdminRequest.requestStatus("001");
+ CollectionAdminRequest replaceNodeRequest = createReplaceNodeRequest(emptyNode, node2bdecommissioned, Boolean.TRUE);
+ replaceNodeRequest.process(cloudClient);
- for (int i = 0; i < 10; i++) {
- CollectionAdminRequest.RequestStatusResponse rsp = requestStatus.process(cloudClient);
- if (rsp.getRequestStatus() == RequestStatusState.COMPLETED) {
- success = true;
- break;
- }
- assertFalse(rsp.getRequestStatus() == RequestStatusState.FAILED);
- Thread.sleep(500);
- }
- assertTrue(success);
- try (Http2SolrClient coreclient = SolrTestCaseJ4.getHttpSolrClient(cloudClient.getZkStateReader().getBaseUrlForNodeName(emptyNode))) {
- CoreAdminResponse status = CoreAdminRequest.getStatus(null, coreclient);
- assertEquals("Expecting no cores but found some: " + status.getCoreStatus(), 0, status.getCoreStatus().size());
- }
+ coreclient = cloudClient.getHttpClient();
+ url = cloudClient.getZkStateReader().getBaseUrlForNodeName(emptyNode);
+ req = new CoreAdminRequest();
+ req.setBasePath(url);
+ req.setCoreName(null);
+ req.setAction(CoreAdminParams.CoreAdminAction.STATUS);
+ status = req.process(coreclient);
+
+ assertEquals("Expecting no cores but found some: " + status.getCoreStatus(), 0, status.getCoreStatus().size());
+
+ cluster.waitForActiveCollection(coll, 5, create.getNumNrtReplicas().intValue() + create.getNumTlogReplicas().intValue() + create.getNumPullReplicas().intValue());
collection = cloudClient.getZkStateReader().getClusterState().getCollection(coll);
assertEquals(create.getNumShards().intValue(), collection.getSlices().size());
- for (Slice s:collection.getSlices()) {
- assertEquals(create.getNumNrtReplicas().intValue(), s.getReplicas(EnumSet.of(Replica.Type.NRT)).size());
- assertEquals(create.getNumTlogReplicas().intValue(), s.getReplicas(EnumSet.of(Replica.Type.TLOG)).size());
- assertEquals(create.getNumPullReplicas().intValue(), s.getReplicas(EnumSet.of(Replica.Type.PULL)).size());
- }
+ // is this a good check? we are moving replicas between nodes ...
+// for (Slice s:collection.getSlices()) {
+// assertEquals(create.getNumNrtReplicas().intValue(), s.getReplicas(EnumSet.of(Replica.Type.NRT)).size());
+// assertEquals(create.getNumTlogReplicas().intValue(), s.getReplicas(EnumSet.of(Replica.Type.TLOG)).size());
+// assertEquals(create.getNumPullReplicas().intValue(), s.getReplicas(EnumSet.of(Replica.Type.PULL)).size());
+// }
// make sure all newly created replicas on node are active
List<Replica> newReplicas = collection.getReplicas(node2bdecommissioned);
replicas.forEach(r -> {
@@ -170,24 +178,10 @@ public class ReplaceNodeTest extends SolrCloudTestCase {
assertFalse(r.toString(), Replica.State.ACTIVE.equals(r.getState()));
}
}
+ CollectionAdminRequest.deleteCollection(coll).process(cluster.getSolrClient());
}
public static CollectionAdminRequest.AsyncCollectionAdminRequest createReplaceNodeRequest(String sourceNode, String targetNode, Boolean parallel) {
- if (random().nextBoolean()) {
- return new CollectionAdminRequest.ReplaceNode(sourceNode, targetNode).setParallel(parallel);
- } else {
- // test back compat with old param names
- // todo remove in solr 8.0
- return new CollectionAdminRequest.AsyncCollectionAdminRequest(CollectionParams.CollectionAction.REPLACENODE) {
- @Override
- public SolrParams getParams() {
- ModifiableSolrParams params = (ModifiableSolrParams) super.getParams();
- params.set("source", sourceNode);
- params.setNonNull("target", targetNode);
- if (parallel != null) params.set("parallel", parallel.toString());
- return params;
- }
- };
- }
+ return new CollectionAdminRequest.ReplaceNode(sourceNode, targetNode).setParallel(parallel);
}
}
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamDecoratorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamDecoratorTest.java
index ba3f4ec..6790679 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamDecoratorTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamDecoratorTest.java
@@ -2615,7 +2615,6 @@ public class StreamDecoratorTest extends SolrCloudTestCase {
assertList(tuple.getLongs("i_multi"), Long.parseLong("4444"), Long.parseLong("7777"));
stream.close();
} finally {
- System.out.println("DELETECOLL");
CollectionAdminRequest.deleteCollection("destinationCollection").process(cluster.getSolrClient());
}
}