You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sh...@apache.org on 2016/08/12 12:10:49 UTC
lucene-solr:branch_6x: SOLR-9405: ConcurrentModificationException in
ZkStateReader.getStateWatchers (cherry picked from commit f82c3b1)
Repository: lucene-solr
Updated Branches:
refs/heads/branch_6x 025279096 -> 42254c2d9
SOLR-9405: ConcurrentModificationException in ZkStateReader.getStateWatchers
(cherry picked from commit f82c3b1)
Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/42254c2d
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/42254c2d
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/42254c2d
Branch: refs/heads/branch_6x
Commit: 42254c2d9b0e93a96e280b44b2fcbd4b9b6693af
Parents: 0252790
Author: Shalin Shekhar Mangar <sh...@apache.org>
Authored: Fri Aug 12 17:33:32 2016 +0530
Committer: Shalin Shekhar Mangar <sh...@apache.org>
Committed: Fri Aug 12 17:39:30 2016 +0530
----------------------------------------------------------------------
solr/CHANGES.txt | 3 ++
.../apache/solr/common/cloud/ZkStateReader.java | 14 ++++--
.../cloud/TestCollectionStateWatchers.java | 51 ++++++++++++++------
3 files changed, 49 insertions(+), 19 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/42254c2d/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 36eeac9..3626da3 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -168,6 +168,9 @@ Bug Fixes
* SOLR-9397: Config API does not support adding caches (noble)
+* SOLR-9405: ConcurrentModificationException in ZkStateReader.getStateWatchers.
+ (Alan Woodward, Edward Ribeiro, shalin)
+
Optimizations
----------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/42254c2d/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
----------------------------------------------------------------------
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 9df4a76..a3de324 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
@@ -147,7 +147,7 @@ public class ZkStateReader implements Closeable {
private class CollectionWatch {
int coreRefCount = 0;
- Set<CollectionStateWatcher> stateWatchers = new HashSet<>();
+ Set<CollectionStateWatcher> stateWatchers = ConcurrentHashMap.newKeySet();
public boolean canBeRemoved() {
return coreRefCount + stateWatchers.size() == 0;
@@ -1273,10 +1273,14 @@ public class ZkStateReader implements Closeable {
/* package-private for testing */
Set<CollectionStateWatcher> getStateWatchers(String collection) {
- CollectionWatch watch = collectionWatches.get(collection);
- if (watch == null)
- return null;
- return new HashSet<>(watch.stateWatchers);
+ final Set<CollectionStateWatcher> watchers = new HashSet<>();
+ collectionWatches.compute(collection, (k, v) -> {
+ if (v != null) {
+ watchers.addAll(v.stateWatchers);
+ }
+ return v;
+ });
+ return watchers;
}
// returns true if the state has changed
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/42254c2d/solr/solrj/src/test/org/apache/solr/common/cloud/TestCollectionStateWatchers.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/common/cloud/TestCollectionStateWatchers.java b/solr/solrj/src/test/org/apache/solr/common/cloud/TestCollectionStateWatchers.java
index d959aa8..fca0e35 100644
--- a/solr/solrj/src/test/org/apache/solr/common/cloud/TestCollectionStateWatchers.java
+++ b/solr/solrj/src/test/org/apache/solr/common/cloud/TestCollectionStateWatchers.java
@@ -19,8 +19,9 @@ package org.apache.solr.common.cloud;
import java.lang.invoke.MethodHandles;
import java.util.HashMap;
-import java.util.Set;
+import java.util.concurrent.Callable;
import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
@@ -81,6 +82,31 @@ public class TestCollectionStateWatchers extends SolrCloudTestCase {
});
}
+ private static void waitFor(String message, long timeout, TimeUnit unit, Callable<Boolean> predicate)
+ throws InterruptedException, ExecutionException {
+ Future<Boolean> future = executor.submit(() -> {
+ try {
+ while (true) {
+ if (predicate.call())
+ return true;
+ TimeUnit.MILLISECONDS.sleep(10);
+ }
+ }
+ catch (InterruptedException e) {
+ return false;
+ }
+ });
+ try {
+ if (future.get(timeout, unit) == true) {
+ return;
+ }
+ }
+ catch (TimeoutException e) {
+ // pass failure message on
+ }
+ future.cancel(true);
+ fail(message);
+ }
@Test
public void testSimpleCollectionWatch() throws Exception {
@@ -113,9 +139,8 @@ public class TestCollectionStateWatchers extends SolrCloudTestCase {
cluster.stopJettySolrRunner(random().nextInt(cluster.getJettySolrRunners().size()));
assertTrue("CollectionStateWatcher was never notified of cluster change", latch.await(MAX_WAIT_TIMEOUT, TimeUnit.SECONDS));
- Set<CollectionStateWatcher> watchers = client.getZkStateReader().getStateWatchers("testcollection");
- assertTrue("CollectionStateWatcher wasn't cleared after completion",
- watchers == null || watchers.size() == 0);
+ waitFor("CollectionStateWatcher wasn't cleared after completion", 1, TimeUnit.SECONDS,
+ () -> client.getZkStateReader().getStateWatchers("testcollection").isEmpty());
}
@@ -144,8 +169,8 @@ public class TestCollectionStateWatchers extends SolrCloudTestCase {
assertTrue("CollectionStateWatcher isn't called when registering for already-watched collection",
latch.await(MAX_WAIT_TIMEOUT, TimeUnit.SECONDS));
- assertEquals("CollectionStateWatcher should be removed",
- 1, client.getZkStateReader().getStateWatchers("currentstate").size());
+ waitFor("CollectionStateWatcher should be removed", 1, TimeUnit.SECONDS,
+ () -> client.getZkStateReader().getStateWatchers("currentstate").size() == 1);
}
@Test
@@ -189,9 +214,8 @@ public class TestCollectionStateWatchers extends SolrCloudTestCase {
expectThrows(TimeoutException.class, () -> {
client.waitForState("nosuchcollection", 1, TimeUnit.SECONDS, ((liveNodes, collectionState) -> false));
});
- Set<CollectionStateWatcher> watchers = client.getZkStateReader().getStateWatchers("nosuchcollection");
- assertTrue("Watchers for collection should be removed after timeout",
- watchers == null || watchers.size() == 0);
+ waitFor("Watchers for collection should be removed after timeout", 1, TimeUnit.SECONDS,
+ () -> client.getZkStateReader().getStateWatchers("nosuchcollection").isEmpty());
}
@@ -229,18 +253,17 @@ public class TestCollectionStateWatchers extends SolrCloudTestCase {
}
@Test
- public void testWatcherIsRemovedAfterTimeout() {
+ public void testWatcherIsRemovedAfterTimeout() throws Exception {
CloudSolrClient client = cluster.getSolrClient();
assertTrue("There should be no watchers for a non-existent collection!",
- client.getZkStateReader().getStateWatchers("no-such-collection") == null);
+ client.getZkStateReader().getStateWatchers("no-such-collection").isEmpty());
expectThrows(TimeoutException.class, () -> {
client.waitForState("no-such-collection", 10, TimeUnit.MILLISECONDS, (n, c) -> DocCollection.isFullyActive(n, c, 1, 1));
});
- Set<CollectionStateWatcher> watchers = client.getZkStateReader().getStateWatchers("no-such-collection");
- assertTrue("Watchers for collection should be removed after timeout",
- watchers == null || watchers.size() == 0);
+ waitFor("Watchers for collection should be removed after timeout", 1, TimeUnit.SECONDS,
+ () -> client.getZkStateReader().getStateWatchers("no-such-collection").isEmpty());
}