You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ds...@apache.org on 2018/04/20 20:22:21 UTC

lucene-solr:master: SOLR-12256: AliasesManager.update() should call ZooKeeper.sync() * SetAliasPropCmd now calls AliasesManager.update() first. * SetAliasPropCmd now more efficiently updates multiple values. * Tests: Commented out BadApple annotations on

Repository: lucene-solr
Updated Branches:
  refs/heads/master 22c4b9c36 -> 8f296d0cc


SOLR-12256: AliasesManager.update() should call ZooKeeper.sync()
* SetAliasPropCmd now calls AliasesManager.update() first.
* SetAliasPropCmd now more efficiently updates multiple values.
* Tests: Commented out BadApple annotations on alias related stuff.


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/8f296d0c
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/8f296d0c
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/8f296d0c

Branch: refs/heads/master
Commit: 8f296d0ccf82174f9c612920ce25b928196a1fa8
Parents: 22c4b9c
Author: David Smiley <ds...@apache.org>
Authored: Fri Apr 20 16:22:16 2018 -0400
Committer: David Smiley <ds...@apache.org>
Committed: Fri Apr 20 16:22:16 2018 -0400

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  2 +
 .../cloud/api/collections/SetAliasPropCmd.java  | 45 +++++++++++---------
 .../apache/solr/cloud/AliasIntegrationTest.java | 33 +++++++-------
 .../solr/cloud/CreateRoutedAliasTest.java       | 10 +++--
 .../TimeRoutedAliasUpdateProcessorTest.java     | 31 ++++++++------
 .../apache/solr/common/cloud/ZkStateReader.java | 22 ++++++----
 6 files changed, 78 insertions(+), 65 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8f296d0c/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index ed36d79..efa6000 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -259,6 +259,8 @@ Bug Fixes
 
 * SOLR-12204: Upgrade commons-fileupload dependency to 1.3.3 to address CVE-2016-1000031.  (Steve Rowe)
 
+* SOLR-12256: Fixed some eventual-consistency issues with collection aliases by using ZooKeeper.sync(). (David Smiley)
+
 ==================  7.3.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8f296d0c/solr/core/src/java/org/apache/solr/cloud/api/collections/SetAliasPropCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/SetAliasPropCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/SetAliasPropCmd.java
index 618b72d..fdee1d1 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/SetAliasPropCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/SetAliasPropCmd.java
@@ -18,6 +18,7 @@
 package org.apache.solr.cloud.api.collections;
 
 import java.lang.invoke.MethodHandles;
+import java.util.LinkedHashMap;
 import java.util.Locale;
 import java.util.Map;
 
@@ -29,7 +30,7 @@ import org.apache.solr.common.util.NamedList;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.*;
+import static org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.Cmd;
 import static org.apache.solr.common.SolrException.ErrorCode.BAD_REQUEST;
 import static org.apache.solr.common.params.CommonParams.NAME;
 
@@ -49,33 +50,35 @@ public class SetAliasPropCmd implements Cmd {
   public void call(ClusterState state, ZkNodeProps message, NamedList results) throws Exception {
     String aliasName = message.getStr(NAME);
 
+    final ZkStateReader.AliasesManager aliasesManager = messageHandler.zkStateReader.aliasesManager;
 
-    ZkStateReader zkStateReader = messageHandler.zkStateReader;
-    if (zkStateReader.getAliases().getCollectionAliasMap().get(aliasName) == null) {
+    // Ensure we see the alias.  This may be redundant but SetAliasPropCmd isn't expected to be called very frequently
+    aliasesManager.update();
+
+    if (aliasesManager.getAliases().getCollectionAliasMap().get(aliasName) == null) {
       // nicer than letting aliases object throw later on...
       throw new SolrException(BAD_REQUEST,
           String.format(Locale.ROOT,  "Can't modify non-existent alias %s", aliasName));
     }
 
     @SuppressWarnings("unchecked")
-    Map<String, String> properties = (Map<String, String>) message.get(PROPERTIES);
-
-    zkStateReader.aliasesManager.applyModificationAndExportToZk(aliases1 -> {
-      for (Map.Entry<String, String> entry : properties.entrySet()) {
-        String key = entry.getKey();
-        if ("".equals(key.trim())) {
-          throw new SolrException(BAD_REQUEST, "property keys must not be pure whitespace");
-        }
-        if (!key.equals(key.trim())) {
-          throw new SolrException(BAD_REQUEST, "property keys should not begin or end with whitespace");
-        }
-        String value = entry.getValue();
-        if ("".equals(value)) {
-          value = null;
-        }
-        aliases1 = aliases1.cloneWithCollectionAliasProperties(aliasName, key, value);
+    Map<String, String> properties = new LinkedHashMap<>((Map<String, String>) message.get(PROPERTIES));
+
+    // check & cleanup properties.  It's a mutable copy.
+    for (Map.Entry<String, String> entry : properties.entrySet()) {
+      String key = entry.getKey();
+      if ("".equals(key.trim())) {
+        throw new SolrException(BAD_REQUEST, "property keys must not be pure whitespace");
+      }
+      if (!key.equals(key.trim())) {
+        throw new SolrException(BAD_REQUEST, "property keys should not begin or end with whitespace");
       }
-      return aliases1;
-    });
+      String value = entry.getValue();
+      if ("".equals(value)) {
+        entry.setValue(null);
+      }
+    }
+
+    aliasesManager.applyModificationAndExportToZk(aliases1 -> aliases1.cloneWithCollectionAliasProperties(aliasName, properties));
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8f296d0c/solr/core/src/test/org/apache/solr/cloud/AliasIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/AliasIntegrationTest.java b/solr/core/src/test/org/apache/solr/cloud/AliasIntegrationTest.java
index 9858ea7..2a82894 100644
--- a/solr/core/src/test/org/apache/solr/cloud/AliasIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/AliasIntegrationTest.java
@@ -93,7 +93,7 @@ public class AliasIntegrationTest extends SolrCloudTestCase {
   }
 
   @Test
-  @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028")
+  //@BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028")
   public void testProperties() throws Exception {
     CollectionAdminRequest.createCollection("collection1meta", "conf", 2, 1).process(cluster.getSolrClient());
     CollectionAdminRequest.createCollection("collection2meta", "conf", 1, 1).process(cluster.getSolrClient());
@@ -118,16 +118,16 @@ public class AliasIntegrationTest extends SolrCloudTestCase {
     assertTrue(((Map<String,Map<String,?>>)Utils.fromJSON(rawBytes)).get("collection").get("meta1") instanceof String);
 
     // set properties
-    UnaryOperator<Aliases> op5 = a -> a.cloneWithCollectionAliasProperties("meta1", "foo", "bar");
-    aliasesManager.applyModificationAndExportToZk(op5);
+    aliasesManager.applyModificationAndExportToZk(a1 ->
+        a1.cloneWithCollectionAliasProperties("meta1", "foo", "bar"));
     Map<String, String> meta = zkStateReader.getAliases().getCollectionAliasProperties("meta1");
     assertNotNull(meta);
     assertTrue(meta.containsKey("foo"));
     assertEquals("bar", meta.get("foo"));
 
     // set more properties
-    UnaryOperator<Aliases> op4 = a -> a.cloneWithCollectionAliasProperties("meta1", "foobar", "bazbam");
-    aliasesManager.applyModificationAndExportToZk(op4);
+    aliasesManager.applyModificationAndExportToZk( a1 ->
+        a1.cloneWithCollectionAliasProperties("meta1", "foobar", "bazbam"));
     meta = zkStateReader.getAliases().getCollectionAliasProperties("meta1");
     assertNotNull(meta);
 
@@ -140,8 +140,8 @@ public class AliasIntegrationTest extends SolrCloudTestCase {
     assertEquals("bazbam", meta.get("foobar"));
 
     // remove properties
-    UnaryOperator<Aliases> op3 = a -> a.cloneWithCollectionAliasProperties("meta1", "foo", null);
-    aliasesManager.applyModificationAndExportToZk(op3);
+    aliasesManager.applyModificationAndExportToZk(a1 ->
+        a1.cloneWithCollectionAliasProperties("meta1", "foo", null));
     meta = zkStateReader.getAliases().getCollectionAliasProperties("meta1");
     assertNotNull(meta);
 
@@ -153,18 +153,17 @@ public class AliasIntegrationTest extends SolrCloudTestCase {
     assertEquals("bazbam", meta.get("foobar"));
 
     // removal of non existent key should succeed.
-    UnaryOperator<Aliases> op2 = a -> a.cloneWithCollectionAliasProperties("meta1", "foo", null);
-    aliasesManager.applyModificationAndExportToZk(op2);
+    aliasesManager.applyModificationAndExportToZk(a2 ->
+        a2.cloneWithCollectionAliasProperties("meta1", "foo", null));
 
     // chained invocations
-    UnaryOperator<Aliases> op1 = a ->
-        a.cloneWithCollectionAliasProperties("meta1", "foo2", "bazbam")
-        .cloneWithCollectionAliasProperties("meta1", "foo3", "bazbam2");
-    aliasesManager.applyModificationAndExportToZk(op1);
+    aliasesManager.applyModificationAndExportToZk(a1 ->
+        a1.cloneWithCollectionAliasProperties("meta1", "foo2", "bazbam")
+        .cloneWithCollectionAliasProperties("meta1", "foo3", "bazbam2"));
 
     // some other independent update (not overwritten)
-    UnaryOperator<Aliases> op = a -> a.cloneWithCollectionAlias("meta3", "collection1meta,collection2meta");
-    aliasesManager.applyModificationAndExportToZk(op);
+    aliasesManager.applyModificationAndExportToZk(a1 ->
+        a1.cloneWithCollectionAlias("meta3", "collection1meta,collection2meta"));
 
     // competing went through
     assertEquals("collection1meta,collection2meta", zkStateReader.getAliases().getCollectionAliasMap().get("meta3"));
@@ -240,7 +239,7 @@ public class AliasIntegrationTest extends SolrCloudTestCase {
   }
 
   @Test
-  @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 09-Apr-2018
+  //@BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 09-Apr-2018
   public void testModifyPropertiesV1() throws Exception {
     // note we don't use TZ in this test, thus it's UTC
     final String aliasName = getTestName();
@@ -256,7 +255,7 @@ public class AliasIntegrationTest extends SolrCloudTestCase {
   }
 
   @Test
-  @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028")
+  //@BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028")
   public void testModifyPropertiesCAR() throws Exception {
     // note we don't use TZ in this test, thus it's UTC
     final String aliasName = getTestName();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8f296d0c/solr/core/src/test/org/apache/solr/cloud/CreateRoutedAliasTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/CreateRoutedAliasTest.java b/solr/core/src/test/org/apache/solr/cloud/CreateRoutedAliasTest.java
index 92135d6..4b81445 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CreateRoutedAliasTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/CreateRoutedAliasTest.java
@@ -99,7 +99,7 @@ public class CreateRoutedAliasTest extends SolrCloudTestCase {
   // This is a fairly complete test where we set many options and see that it both affected the created
   //  collection and that the alias metadata was saved accordingly
   @Test
-  @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 09-Apr-2018
+  //@BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 09-Apr-2018
   public void testV2() throws Exception {
     // note we don't use TZ in this test, thus it's UTC
     final String aliasName = getTestName();
@@ -181,7 +181,7 @@ public class CreateRoutedAliasTest extends SolrCloudTestCase {
   }
 
   @Test
-  @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 09-Apr-2018
+  //@BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 09-Apr-2018
   public void testV1() throws Exception {
     final String aliasName = getTestName();
     final String baseUrl = cluster.getRandomJetty(random()).getBaseUrl().toString();
@@ -225,7 +225,7 @@ public class CreateRoutedAliasTest extends SolrCloudTestCase {
 
   // TZ should not affect the first collection name if absolute date given for start
   @Test
-  @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 09-Apr-2018
+  //@BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 09-Apr-2018
   public void testTimezoneAbsoluteDate() throws Exception {
     final String aliasName = getTestName();
     try (SolrClient client = getCloudSolrClient(cluster)) {
@@ -244,7 +244,7 @@ public class CreateRoutedAliasTest extends SolrCloudTestCase {
   }
 
   @Test
-  @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 09-Apr-2018
+  //@BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 09-Apr-2018
   public void testCollectionNamesMustBeAbsent() throws Exception {
     CollectionAdminRequest.createCollection("collection1meta", "_default", 2, 1).process(cluster.getSolrClient());
     CollectionAdminRequest.createCollection("collection2meta", "_default", 1, 1).process(cluster.getSolrClient());
@@ -330,6 +330,7 @@ public class CreateRoutedAliasTest extends SolrCloudTestCase {
         "&create-collection.numShards=1");
     assertFailure(get, "Unit not recognized");
   }
+
   @Test
   public void testNegativeFutureFails() throws Exception {
     final String aliasName = getTestName();
@@ -346,6 +347,7 @@ public class CreateRoutedAliasTest extends SolrCloudTestCase {
         "&create-collection.numShards=1");
     assertFailure(get, "must be >= 0");
   }
+
   @Test
   public void testUnParseableFutureFails() throws Exception {
     final String aliasName = "testAlias";

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8f296d0c/solr/core/src/test/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessorTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessorTest.java b/solr/core/src/test/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessorTest.java
index ef8e1a5..cc7b7ce 100644
--- a/solr/core/src/test/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessorTest.java
+++ b/solr/core/src/test/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessorTest.java
@@ -37,7 +37,6 @@ import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.ConfigSetAdminRequest;
 import org.apache.solr.client.solrj.request.V2Request;
-import org.apache.solr.client.solrj.response.ConfigSetAdminResponse;
 import org.apache.solr.client.solrj.response.FieldStatsInfo;
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.client.solrj.response.UpdateResponse;
@@ -71,6 +70,11 @@ public class TimeRoutedAliasUpdateProcessorTest extends SolrCloudTestCase {
   public static void setupCluster() throws Exception {
     configureCluster(2).configure();
     solrClient = getCloudSolrClient(cluster);
+    //log this to help debug potential causes of problems
+    System.out.println("SolrClient: " + solrClient);
+    if (solrClient instanceof CloudSolrClient) {
+      System.out.println(((CloudSolrClient)solrClient).getClusterStateProvider());
+    }
   }
 
   @AfterClass
@@ -85,16 +89,17 @@ public class TimeRoutedAliasUpdateProcessorTest extends SolrCloudTestCase {
     // Then we create a collection with the name of the eventual config.
     // We configure it, and ultimately delete the collection, leaving a modified config-set behind.
     // Then when we create the "real" collections referencing this modified config-set.
-    final ConfigSetAdminRequest.Create adminRequest = new ConfigSetAdminRequest.Create();
-        adminRequest.setConfigSetName(configName);
-        adminRequest.setBaseConfigSetName("_default");
-        ConfigSetAdminResponse adminResponse = adminRequest.process(solrClient);
-        assertEquals(adminResponse.getStatus(), 0);
+    assertEquals(0, new ConfigSetAdminRequest.Create()
+        .setConfigSetName(configName)
+        .setBaseConfigSetName("_default")
+        .process(solrClient).getStatus());
 
-    CollectionAdminRequest.createCollection(configName, configName,1, 1).process(solrClient);
-    // manipulate the config...
+    CollectionAdminRequest.createCollection(configName, configName, 1, 1).process(solrClient);
 
-        String conf = "{" +
+    // manipulate the config...
+    checkNoError(solrClient.request(new V2Request.Builder("/collections/" + configName + "/config")
+        .withMethod(SolrRequest.METHOD.POST)
+        .withPayload("{" +
             "  'set-user-property' : {'update.autoCreateFields':false}," + // no data driven
             "  'add-updateprocessor' : {" +
             "    'name':'tolerant', 'class':'solr.TolerantUpdateProcessorFactory'" +
@@ -103,10 +108,8 @@ public class TimeRoutedAliasUpdateProcessorTest extends SolrCloudTestCase {
             "    'name':'inc', 'class':'" + IncrementURPFactory.class.getName() + "'," +
             "    'fieldName':'" + intField + "'" +
             "  }," +
-            "}";
-    checkNoError(solrClient.request(new V2Request.Builder("/collections/" + configName + "/config")
-        .withMethod(SolrRequest.METHOD.POST)
-        .withPayload(conf).build()));    // only sometimes test with "tolerant" URP
+            "}").build()));
+    // only sometimes test with "tolerant" URP:
     final String urpNames = "inc" + (random().nextBoolean() ? ",tolerant" : "");
     checkNoError(solrClient.request(new V2Request.Builder("/collections/" + configName + "/config/params")
         .withMethod(SolrRequest.METHOD.POST)
@@ -115,8 +118,8 @@ public class TimeRoutedAliasUpdateProcessorTest extends SolrCloudTestCase {
             "    '_UPDATE' : {'processor':'" + urpNames + "'}" +
             "  }" +
             "}").build()));
-    CollectionAdminRequest.deleteCollection(configName).process(solrClient);
 
+    CollectionAdminRequest.deleteCollection(configName).process(solrClient);
     assertTrue(
         new ConfigSetAdminRequest.List().process(solrClient).getConfigSets()
             .contains(configName)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8f296d0c/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 cfae849..9f1ddc6 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
@@ -16,14 +16,6 @@
  */
 package org.apache.solr.common.cloud;
 
-import static java.util.Arrays.asList;
-import static java.util.Collections.EMPTY_MAP;
-import static java.util.Collections.emptyMap;
-import static java.util.Collections.emptySet;
-import static java.util.Collections.emptySortedSet;
-import static java.util.Collections.unmodifiableSet;
-import static org.apache.solr.common.util.Utils.fromJSON;
-
 import java.io.Closeable;
 import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
@@ -51,6 +43,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.UnaryOperator;
 import java.util.stream.Collectors;
+
 import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
 import org.apache.solr.common.Callable;
 import org.apache.solr.common.SolrException;
@@ -69,6 +62,14 @@ import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static java.util.Arrays.asList;
+import static java.util.Collections.EMPTY_MAP;
+import static java.util.Collections.emptyMap;
+import static java.util.Collections.emptySet;
+import static java.util.Collections.emptySortedSet;
+import static java.util.Collections.unmodifiableSet;
+import static org.apache.solr.common.util.Utils.fromJSON;
+
 public class ZkStateReader implements Closeable {
   public static final int STATE_UPDATE_DELAY = Integer.getInteger("solr.OverseerStateUpdateDelay", 2000);  // delay between cloud state updates
   private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@@ -1705,7 +1706,7 @@ public class ZkStateReader implements Closeable {
             LOG.debug(e.toString(), e);
             LOG.warn("Couldn't save aliases due to race with another modification; will update and retry until timeout");
             // considered a backoff here, but we really do want to compete strongly since the normal case is
-            // that we will do one update and succeed. This is left as a hot loop for 5 tries intentionally.
+            // that we will do one update and succeed. This is left as a hot loop for limited tries intentionally.
             // More failures than that here probably indicate a bug or a very strange high write frequency usage for
             // aliases.json, timeouts mean zk is being very slow to respond, or this node is being crushed
             // by other processing and just can't find any cpu cycles at all.
@@ -1733,6 +1734,9 @@ public class ZkStateReader implements Closeable {
      * @return true if an update was performed
      */
     public boolean update() throws KeeperException, InterruptedException {
+      LOG.debug("Checking ZK for most up to date Aliases " + ALIASES);
+      // Call sync() first to ensure the subsequent read (getData) is up to date.
+      zkClient.getSolrZooKeeper().sync(ALIASES, null, null);
       Stat stat = new Stat();
       final byte[] data = zkClient.getData(ALIASES, null, stat, true);
       return setIfNewer(Aliases.fromJSON(data, stat.getVersion()));