You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ma...@apache.org on 2012/02/27 15:50:01 UTC

svn commit: r1294169 - in /lucene/dev/trunk/solr: core/src/java/org/apache/solr/cloud/ZkController.java core/src/java/org/apache/solr/core/CoreContainer.java core/src/test/org/apache/solr/cloud/TestMultiCoreConfBootstrap.java example/multicore/zoo.cfg

Author: markrmiller
Date: Mon Feb 27 14:50:01 2012
New Revision: 1294169

URL: http://svn.apache.org/viewvc?rev=1294169&view=rev
Log:
SOLR-3166: Allow bootstrapping multiple config sets from multi-core setups.

Added:
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/TestMultiCoreConfBootstrap.java   (with props)
    lucene/dev/trunk/solr/example/multicore/zoo.cfg
Modified:
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java?rev=1294169&r1=1294168&r2=1294169&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java Mon Feb 27 14:50:01 2012
@@ -878,6 +878,9 @@ public final class ZkController {
             if (!collectionProps.containsKey(CONFIGNAME_PROP))
               collectionProps.put(CONFIGNAME_PROP,  defaultConfigName);
 
+          } else if (Boolean.getBoolean("bootstrap_conf")) {
+            // the conf name should should be the collection name of this core
+            collectionProps.put(CONFIGNAME_PROP,  cd.getCollectionName());
           } else {
             getConfName(collection, collectionPath, collectionProps);
           }
@@ -920,9 +923,15 @@ public final class ZkController {
           break;
         }
       }
+      List<String> configNames = null;
       // if there is only one conf, use that
-      List<String> configNames = zkClient.getChildren(CONFIGS_ZKNODE, null, true);
-      if (configNames.size() == 1) {
+      try {
+        configNames = zkClient.getChildren(CONFIGS_ZKNODE, null,
+            true);
+      } catch (NoNodeException e) {
+        // just keep trying
+      }
+      if (configNames != null && configNames.size() == 1) {
         // no config set named, but there is only 1 - use it
         log.info("Only one config set found in zk - using it:" + configNames.get(0));
         collectionProps.put(CONFIGNAME_PROP,  configNames.get(0));

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java?rev=1294169&r1=1294168&r2=1294169&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java Mon Feb 27 14:50:01 2012
@@ -219,6 +219,12 @@ public class CoreContainer 
           String confName = System.getProperty(ZkController.COLLECTION_PARAM_PREFIX+ZkController.CONFIGNAME_PROP, "configuration1");
           zkController.uploadConfigDir(dir, confName);
         }
+        
+        boolean boostrapConf = Boolean.getBoolean("bootstrap_conf");
+        if(boostrapConf) {
+          bootstrapConf();
+        }
+        
       } catch (InterruptedException e) {
         // Restore the interrupted status
         Thread.currentThread().interrupt();
@@ -242,6 +248,28 @@ public class CoreContainer 
     
   }
 
+  private void bootstrapConf() throws IOException,
+      KeeperException, InterruptedException {
+
+    NodeList nodes = (NodeList)cfg.evaluate("solr/cores/core", XPathConstants.NODESET);
+
+    for (int i=0; i<nodes.getLength(); i++) {
+      Node node = nodes.item(i);
+      String rawName = DOMUtil.getAttr(node, "name", null);
+      String instanceDir = DOMUtil.getAttr(node, "instanceDir", null);
+      File idir = new File(instanceDir);
+      if (!idir.isAbsolute()) {
+        idir = new File(solrHome, instanceDir);
+      }
+      String confName = DOMUtil.getAttr(node, "collection", null);
+      if (confName == null) {
+        confName = rawName;
+      }
+
+      zkController.uploadConfigDir(new File(idir, "conf"), confName);
+    }
+  }
+
   public Properties getContainerProperties() {
     return containerProperties;
   }

Added: lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/TestMultiCoreConfBootstrap.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/TestMultiCoreConfBootstrap.java?rev=1294169&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/TestMultiCoreConfBootstrap.java (added)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/TestMultiCoreConfBootstrap.java Mon Feb 27 14:50:01 2012
@@ -0,0 +1,118 @@
+/**
+ * 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.File;
+import java.io.IOException;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.util.AbstractSolrTestCase;
+import org.apache.solr.util.ExternalPaths;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestMultiCoreConfBootstrap extends SolrTestCaseJ4 {
+  protected static Logger log = LoggerFactory.getLogger(TestMultiCoreConfBootstrap.class);
+  protected CoreContainer cores = null;
+  private String home;
+
+
+  protected static ZkTestServer zkServer;
+  protected static String zkDir;
+  
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    createTempDir();
+  }
+  
+  @AfterClass
+  public static void afterClass() throws IOException {
+
+  }
+  
+  @Override
+  @Before
+  public void setUp() throws Exception {
+    super.setUp();
+    home = ExternalPaths.EXAMPLE_MULTICORE_HOME;
+    System.setProperty("solr.solr.home", home);
+    
+    zkDir = dataDir.getAbsolutePath() + File.separator
+        + "zookeeper/server1/data";
+    zkServer = new ZkTestServer(zkDir);
+    zkServer.run();
+    
+    SolrZkClient zkClient = new SolrZkClient(zkServer.getZkHost(), AbstractZkTestCase.TIMEOUT);
+    zkClient.makePath("/solr", false, true);
+    zkClient.close();
+    
+    System.setProperty("zkHost", zkServer.getZkAddress());
+  }
+
+  @Override
+  @After
+  public void tearDown() throws Exception {
+    System.clearProperty("bootstrap_confdir");
+    System.clearProperty("zkHost");
+    System.clearProperty("solr.solr.home");
+    
+    if (cores != null)
+      cores.shutdown();
+    
+    zkServer.shutdown();
+    
+    File dataDir1 = new File(home + File.separator + "core0","data");
+    File dataDir2 = new File(home + File.separator + "core1","data");
+
+    String skip = System.getProperty("solr.test.leavedatadir");
+    if (null != skip && 0 != skip.trim().length()) {
+      log.info("NOTE: per solr.test.leavedatadir, dataDir will not be removed: " + dataDir.getAbsolutePath());
+    } else {
+      if (!AbstractSolrTestCase.recurseDelete(dataDir1)) {
+        log.warn("!!!! WARNING: best effort to remove " + dataDir.getAbsolutePath() + " FAILED !!!!!");
+      }
+      if (!AbstractSolrTestCase.recurseDelete(dataDir2)) {
+        log.warn("!!!! WARNING: best effort to remove " + dataDir.getAbsolutePath() + " FAILED !!!!!");
+      }
+    }
+
+    super.tearDown();
+  }
+
+
+  @Test
+  public void testMultiCoreConfBootstrap() throws Exception {
+    System.setProperty("bootstrap_conf", "true");
+    cores = new CoreContainer(home, new File(home, "solr.xml"));
+    SolrZkClient zkclient = cores.getZkController().getZkClient();
+    // zkclient.printLayoutToStdOut();
+    
+    assertTrue(zkclient.exists("/configs/core1/solrconfig.xml", true));
+    assertTrue(zkclient.exists("/configs/core1/schema.xml", true));
+    assertTrue(zkclient.exists("/configs/core0/solrconfig.xml", true));
+    assertTrue(zkclient.exists("/configs/core1/schema.xml", true));
+  }
+
+}

Added: lucene/dev/trunk/solr/example/multicore/zoo.cfg
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/example/multicore/zoo.cfg?rev=1294169&view=auto
==============================================================================
--- lucene/dev/trunk/solr/example/multicore/zoo.cfg (added)
+++ lucene/dev/trunk/solr/example/multicore/zoo.cfg Mon Feb 27 14:50:01 2012
@@ -0,0 +1,17 @@
+# The number of milliseconds of each tick
+tickTime=2000
+# The number of ticks that the initial
+# synchronization phase can take
+initLimit=10
+# The number of ticks that can pass between
+# sending a request and getting an acknowledgement
+syncLimit=5
+
+# the directory where the snapshot is stored.
+# dataDir=/opt/zookeeper/data
+# NOTE: Solr defaults the dataDir to <solrHome>/zoo_data
+
+# the port at which the clients will connect
+# clientPort=2181
+# NOTE: Solr sets this based on zkRun / zkHost params
+