mirror of https://github.com/apache/lucene.git
SOLR-13678: Harden CollectionPropsTest.testReadWriteCached to work around removeCollectionPropsWatcher() deadlock bug
(cherry picked from commit a052fb5436
)
This commit is contained in:
parent
445f4bf5c2
commit
b18041476f
|
@ -25,6 +25,9 @@ import java.util.Collections;
|
|||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.concurrent.Semaphore;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.solr.SolrTestCaseJ4;
|
||||
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
|
||||
|
@ -95,10 +98,22 @@ public class CollectionPropsTest extends SolrCloudTestCase {
|
|||
@Test
|
||||
public void testReadWriteCached() throws InterruptedException, IOException {
|
||||
CollectionProperties collectionProps = new CollectionProperties(zkClient());
|
||||
|
||||
// NOTE: Using a semaphore to ensure we wait for Watcher to fire before proceeding with
|
||||
// test logic, to prevent triggering SOLR-13678
|
||||
final Semaphore sawExpectedProps = new Semaphore(0);
|
||||
final AtomicReference<Map<String,String>> expectedProps
|
||||
= new AtomicReference<Map<String,String>>(null);
|
||||
|
||||
final CollectionPropsWatcher w = new CollectionPropsWatcher() {
|
||||
@Override
|
||||
public boolean onStateChanged(Map<String,String> collectionProperties) {
|
||||
log.info("collection properties changed. Now: {}", collectionProperties);
|
||||
final Map<String,String> expected = expectedProps.get();
|
||||
if (expected != null && expected.equals(collectionProperties)) {
|
||||
log.info("...new props match expected");
|
||||
sawExpectedProps.release();
|
||||
}
|
||||
return false;
|
||||
}
|
||||
};
|
||||
|
@ -109,7 +124,11 @@ public class CollectionPropsTest extends SolrCloudTestCase {
|
|||
collectionProps.setCollectionProperty(collectionName, "property2", "value2");
|
||||
waitForValue("property1", "value1", 5000);
|
||||
waitForValue("property2", "value2", 5000);
|
||||
|
||||
|
||||
// HACK: don't let our watcher be removed until we're sure it's "up to date"
|
||||
// with the final prop values expected below...
|
||||
expectedProps.set(new HashMap<>());
|
||||
|
||||
collectionProps.setCollectionProperty(collectionName, "property1", "value1"); // no change
|
||||
checkValue("property1", "value1");
|
||||
|
||||
|
@ -123,11 +142,14 @@ public class CollectionPropsTest extends SolrCloudTestCase {
|
|||
|
||||
collectionProps.setCollectionProperty(collectionName, "property2", null); // no change
|
||||
checkValue("property2", null);
|
||||
|
||||
|
||||
assertTrue("Gave up waitng an excessive amount of time for watcher to see final expected props",
|
||||
sawExpectedProps.tryAcquire(1, 120, TimeUnit.SECONDS));
|
||||
cluster.getSolrClient().getZkStateReader().removeCollectionPropsWatcher(collectionName, w);
|
||||
|
||||
collectionProps.setCollectionProperty(collectionName, "property1", "value1");
|
||||
checkValue("property1", "value1"); //Should be no cache, so the change should take effect immediately
|
||||
|
||||
}
|
||||
|
||||
private void checkValue(String propertyName, String expectedValue) throws InterruptedException {
|
||||
|
|
Loading…
Reference in New Issue