You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sa...@apache.org on 2018/03/01 22:00:17 UTC

lucene-solr:branch_6_6: SOLR-11503: Collections created with legacyCloud=true cannot be opened if legacyCloud=false

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_6_6 dd3be31f7 -> b4e33a038


SOLR-11503: Collections created with legacyCloud=true cannot be opened if legacyCloud=false


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

Branch: refs/heads/branch_6_6
Commit: b4e33a038569f97752abd61a26e8af0b652e5b44
Parents: dd3be31
Author: Erick Erickson <er...@apache.org>
Authored: Fri Nov 3 19:43:37 2017 -0700
Committer: Steve Rowe <sa...@apache.org>
Committed: Thu Mar 1 17:00:00 2018 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   2 +
 .../org/apache/solr/cloud/ZkController.java     |  16 +-
 .../org/apache/solr/core/CoreContainer.java     |  41 ++++-
 .../solr/cloud/LegacyCloudClusterPropTest.java  | 163 +++++++++++++++++++
 4 files changed, 218 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4e33a03/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 1dda7a3..c376baf 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -35,6 +35,8 @@ Bug Fixes
 * SOLR-11971: Don't allow referal to external resources in DataImportHandler's dataConfig request parameter.
   (麦 香浓郁, Uwe Schindler)
 
+* SOLR-11503: Collections created with legacyCloud=true cannot be opened if legacyCloud=false (Erick Erickson)
+
 ==================  6.6.2 ==================
 
 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/b4e33a03/solr/core/src/java/org/apache/solr/cloud/ZkController.java
----------------------------------------------------------------------
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 02ed406..182af5f 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
@@ -1405,9 +1405,19 @@ public class ZkController {
   private void checkStateInZk(CoreDescriptor cd) throws InterruptedException {
     if (!Overseer.isLegacy(zkStateReader)) {
       CloudDescriptor cloudDesc = cd.getCloudDescriptor();
-      String coreNodeName = cloudDesc.getCoreNodeName();
-      if (coreNodeName == null)
-        throw new SolrException(ErrorCode.SERVER_ERROR, "No coreNodeName for " + cd);
+      String nodeName = cloudDesc.getCoreNodeName();
+      if (nodeName == null) {
+        if (cc.repairCoreProperty(cd, CoreDescriptor.CORE_NODE_NAME) == false) {
+          throw new SolrException(ErrorCode.SERVER_ERROR, "No coreNodeName for " + cd);
+        }
+        nodeName = cloudDesc.getCoreNodeName();
+        // verify that the repair worked.
+        if (nodeName == null) {
+          throw new SolrException(ErrorCode.SERVER_ERROR, "No coreNodeName for " + cd);
+        }
+      }
+      final String coreNodeName = nodeName;
+
       if (cloudDesc.getShardId() == null) {
         throw new SolrException(ErrorCode.SERVER_ERROR, "No shard id for " + cd);
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4e33a03/solr/core/src/java/org/apache/solr/core/CoreContainer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index 0c77a7e..3b6710d 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -860,6 +860,7 @@ public class CoreContainer {
       try {
         solrCores.waitAddPendingCoreOps(cd.getName());
         core = createFromDescriptor(cd, true, newCollection);
+        coresLocator.persist(this, cd); // Write out the current core properties in case anything changed when the core was created
       } finally {
         solrCores.removeFromPendingOps(cd.getName());
       }
@@ -1529,7 +1530,45 @@ public class CoreContainer {
   public TransientSolrCoreCache getTransientCache() {
     return solrCores.getTransientCacheHandler();
   }
-  
+
+
+  /**
+   *
+   * @param cd CoreDescriptor, presumably a deficient one
+   * @param prop The property that needs to be repaired.
+   * @return true if we were able to successfuly perisist the repaired coreDescriptor, false otherwise.
+   *
+   * See SOLR-11503, This can be removed when there's no chance we'll need to upgrade a
+   * Solr isntallation createged with legacyCloud=true from 6.6.1 through 7.1
+   */
+  public boolean repairCoreProperty(CoreDescriptor cd, String prop) {
+    // So far, coreNodeName is the only property that we need to repair, this may get more complex as other properties
+    // are added.
+
+    if (CoreDescriptor.CORE_NODE_NAME.equals(prop) == false) {
+      throw new SolrException(ErrorCode.SERVER_ERROR,
+          String.format(Locale.ROOT,"The only supported property for repair is currently [%s]",
+              CoreDescriptor.CORE_NODE_NAME));
+    }
+
+    // Try to read the coreNodeName from the cluster state.
+
+    String coreName = cd.getName();
+    DocCollection coll = getZkController().getZkStateReader().getClusterState().getCollection(cd.getCollectionName());
+    for (Replica rep : coll.getReplicas()) {
+      if (coreName.equals(rep.getCoreName())) {
+        log.warn("Core properties file for node {} found with no coreNodeName, attempting to repair with value {}. See SOLR-11503. " +
+            "This message should only appear if upgrading from collections created Solr 6.6.1 through 7.1.",
+            rep.getCoreName(), rep.getName());
+        cd.getCloudDescriptor().setCoreNodeName(rep.getName());
+        coresLocator.persist(this, cd);
+        return true;
+      }
+    }
+    log.error("Could not repair coreNodeName in core.properties file for core {}", coreName);
+    return false;
+  }
+
 }
 
 class CloserThread extends Thread {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4e33a03/solr/core/src/test/org/apache/solr/cloud/LegacyCloudClusterPropTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/LegacyCloudClusterPropTest.java b/solr/core/src/test/org/apache/solr/cloud/LegacyCloudClusterPropTest.java
new file mode 100644
index 0000000..72e8e6d
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/LegacyCloudClusterPropTest.java
@@ -0,0 +1,163 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.cloud;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.io.Writer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Properties;
+
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.common.cloud.ClusterProperties;
+import org.apache.solr.common.cloud.ClusterStateUtil;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.core.CorePropertiesLocator;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+
+public class LegacyCloudClusterPropTest extends SolrCloudTestCase {
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+
+    // currently this test is fine with a single shard with a single replica and it's simpler. Could easily be
+    // extended to multiple shards/replicas, but there's no particular need.
+    configureCluster(1)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
+  }
+
+
+  // Are all these required?
+  private static String[] requiredProps = {
+      "numShards",
+      "collection.configName",
+      "name",
+      "replicaType",
+      "shard",
+      "collection",
+      "coreNodeName"
+  };
+
+  @Test
+  public void testCreateCollectionSwitchLegacyCloud() throws Exception {
+    createAndTest("legacyTrue", true);
+    createAndTest("legacyFalse", false);
+  }
+
+  private void createAndTest(final String coll, final boolean legacy) throws Exception {
+
+    // First, just insure that core.properties file gets created with coreNodeName and all other mandatory parameters.
+    final String legacyString = Boolean.toString(legacy);
+    final String legacyAnti = Boolean.toString(!legacy);
+    CollectionAdminRequest.setClusterProperty(ZkStateReader.LEGACY_CLOUD, legacyString).process(cluster.getSolrClient());
+    ClusterProperties props = new ClusterProperties(zkClient());
+
+    assertEquals("Value of legacyCloud cluster prop unexpected", legacyString,
+        props.getClusterProperty(ZkStateReader.LEGACY_CLOUD, legacyAnti));
+
+    CollectionAdminRequest.createCollection(coll, "conf", 1, 1)
+        .setMaxShardsPerNode(1)
+        .process(cluster.getSolrClient());
+    assertTrue(ClusterStateUtil.waitForAllActiveAndLiveReplicas(cluster.getSolrClient().getZkStateReader(), 120000));
+    
+    // Insure all mandatory properties are there.
+    checkMandatoryProps(coll);
+
+    checkCollectionActive(coll);
+    // The fixes for SOLR-11503 insure that creating a collection has coreNodeName whether legacyCloud is true or false,
+    // we still need to test repairing a properties file that does _not_ have coreNodeName set, the second part of
+    // the fix.
+
+    // First, remove the coreNodeName from cluster.properties and write it out it.
+    removePropertyFromAllReplicas(coll, "coreNodeName");
+
+    // Now restart Solr, this should repair the removal on core load no matter the value of legacyCloud
+    JettySolrRunner jetty = cluster.getJettySolrRunner(0);
+    jetty.stop();
+    jetty.start();
+    checkMandatoryProps(coll);
+    checkCollectionActive(coll);
+  }
+
+  private void checkCollectionActive(String coll) {
+    assertTrue(ClusterStateUtil.waitForAllActiveAndLiveReplicas(cluster.getSolrClient().getZkStateReader(), 120000));
+    DocCollection docColl = getCollectionState(coll);
+    for (Replica rep : docColl.getReplicas()) {
+      if (rep.getState() == Replica.State.ACTIVE) return;
+    }
+    fail("Replica was not active for collection " + coll);
+  }
+  private void removePropertyFromAllReplicas(String coll, String propDel) throws IOException {
+    DocCollection docColl = getCollectionState(coll);
+
+    // First remove the property from all core.properties files
+    for (Replica rep : docColl.getReplicas()) {
+      final String coreName = rep.getCoreName();
+      Properties prop = loadPropFileForReplica(coreName);
+      prop.remove(propDel);
+      JettySolrRunner jetty = cluster.getJettySolrRunner(0);
+      Path expected = Paths.get(jetty.getSolrHome()).toAbsolutePath().resolve(coreName);
+      Path corePropFile = Paths.get(expected.toString(), CorePropertiesLocator.PROPERTIES_FILENAME);
+
+      try (Writer os = new OutputStreamWriter(Files.newOutputStream(corePropFile), StandardCharsets.UTF_8)) {
+        prop.store(os, "");
+      }
+    }
+
+    // Now insure it's really gone
+    for (Replica rep : docColl.getReplicas()) {
+      Properties prop = loadPropFileForReplica(rep.getCoreName());
+      assertEquals("Property " + propDel + " should have been deleted",
+          "bogus", prop.getProperty(propDel, "bogus"));
+    }
+  }
+
+  private Properties loadPropFileForReplica(String coreName) throws IOException {
+    JettySolrRunner jetty = cluster.getJettySolrRunner(0);
+    Path expected = Paths.get(jetty.getSolrHome()).toAbsolutePath().resolve(coreName);
+    Path corePropFile = Paths.get(expected.toString(), CorePropertiesLocator.PROPERTIES_FILENAME);
+    Properties props = new Properties();
+    try (InputStream fis = Files.newInputStream(corePropFile)) {
+      props.load(new InputStreamReader(fis, StandardCharsets.UTF_8));
+    }
+    return props;
+  }
+
+  private void checkMandatoryProps(String coll) throws IOException {
+    DocCollection docColl = getCollectionState(coll);
+    for (Replica rep : docColl.getReplicas()) {
+      Properties prop = loadPropFileForReplica(rep.getCoreName());      for (String testProp : requiredProps) {
+        String propVal = prop.getProperty(testProp, "bogus");
+        if ("bogus".equals(propVal)) {
+          fail("Should have found property " + testProp + " in properties file");
+        }
+      }
+    }
+  }
+}