mirror of https://github.com/apache/lucene.git
SOLR-9405: ConcurrentModificationException in ZkStateReader.getStateWatchers
This commit is contained in:
parent
92b5a76b54
commit
f82c3b1206
|
@ -203,6 +203,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
|
||||
----------------------
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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());
|
||||
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue