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

svn commit: r903853 - in /lucene/solr/branches/cloud/src: java/org/apache/solr/cloud/ZkController.java test/org/apache/solr/cloud/AbstractZkTestCase.java

Author: markrmiller
Date: Wed Jan 27 22:02:37 2010
New Revision: 903853

URL: http://svn.apache.org/viewvc?rev=903853&view=rev
Log:
bootstrap code for setting collection config

Modified:
    lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/ZkController.java
    lucene/solr/branches/cloud/src/test/org/apache/solr/cloud/AbstractZkTestCase.java

Modified: lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/ZkController.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/ZkController.java?rev=903853&r1=903852&r2=903853&view=diff
==============================================================================
--- lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/ZkController.java (original)
+++ lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/ZkController.java Wed Jan 27 22:02:37 2010
@@ -163,12 +163,35 @@
    * @param shardId
    * @param collection
    * @throws IOException
+   * @throws InterruptedException 
+   * @throws KeeperException 
    */
-  private void addZkShardsNode(String shardId, String collection) throws IOException {
+  private void addZkShardsNode(String shardId, String collection) throws IOException, InterruptedException, KeeperException {
 
+    String collectionPath = COLLECTIONS_ZKNODE + "/" + collection;
     String shardsZkPath = COLLECTIONS_ZKNODE + "/" + collection + SHARDS_ZKNODE + "/" + shardId;
     
+    boolean newCollection = false;
+    boolean newShardId = false;
+    
     try {
+      if(!zkClient.exists(collectionPath)) {
+        try {
+          zkClient.makePath(collectionPath, CreateMode.PERSISTENT, null);
+          String confName = readConfigName(collection);
+          if(confName == null && System.getProperty("bootstrap_confdir") != null) {
+            confName = System.getProperty("bootstrap_confname", "configuration1");
+            zkClient.makePath(COLLECTIONS_ZKNODE + "/" + collection + "/conifg=" + confName);
+          }
+          newCollection = true;
+        } catch (KeeperException e) {
+          // its okay if another beats us creating the node
+          if (e.code() != KeeperException.Code.NODEEXISTS) {
+            throw e;
+          }
+        }
+      }
+      
       // shards node
       if (!zkClient.exists(shardsZkPath)) {
         if (log.isInfoEnabled()) {
@@ -177,23 +200,20 @@
         // makes shards zkNode if it doesn't exist
         zkClient.makePath(shardsZkPath, CreateMode.PERSISTENT, null);
         
-        // nocommit - scrutinize
-        // ping that there is a new collection or a new shardId
-        zkClient.setData(COLLECTIONS_ZKNODE, (byte[])null);
+        newShardId = true;
+
       }
     } catch (KeeperException e) {
       // its okay if another beats us creating the node
       if (e.code() != KeeperException.Code.NODEEXISTS) {
-        log.error("", e);
-        throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
-            "", e);
+        throw e;
       }
-    } catch (InterruptedException e) {
-      // Restore the interrupted status
-      Thread.currentThread().interrupt();
-      log.error("", e);
-      throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
-          "", e);
+    }
+    
+    if(newCollection || newShardId) {
+      // nocommit - scrutinize
+      // ping that there is a new collection or a new shardId
+      zkClient.setData(COLLECTIONS_ZKNODE, (byte[])null);
     }
   }
 
@@ -486,20 +506,11 @@
     try {
       children = zkClient.getChildren(path, null);
     } catch (KeeperException.NoNodeException e) {
-      // no config is set - check if there is only one config
-      // and if there is, use that
-      children = zkClient.getChildren(CONFIGS_ZKNODE, null);
-      if(children.size() == 1) {
-        String config = children.get(0);
-        log.info("No config set for " + collection + ", using single config found:" + config);
-        return config;
-      }
-
       log.error(
-          "Multiple configurations were found, but config name to use for collection:"
+          "Config name to use for collection:"
               + collection + " could not be located", e);
       throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
-          "Multiple configurations were found, but config name to use for collection:"
+          "Config name to use for collection:"
               + collection + " could not be located", e);
     }
     for (String node : children) {
@@ -512,16 +523,11 @@
         // nocommmit : bail or read more?
       }
     }
-
-    if (configName == null) {
-      children = zkClient.getChildren(CONFIGS_ZKNODE, null);
-      if(children.size() == 1) {
-        String config = children.get(0);
-        log.info("No config set for " + collection + ", using single config found:" + config);
-        return config;
-      }
-      throw new IllegalStateException("no config specified for collection:"
-          + collection + " " + children.size() + " configurations exist");
+    
+    if(!zkClient.exists(CONFIGS_ZKNODE + "/" + configName)) {
+      log.error("Specified config does not exist in ZooKeeper:" + configName);
+      throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
+          "Specified config does not exist in ZooKeeper:" + configName);
     }
 
     return configName;
@@ -765,7 +771,9 @@
             // shards node yet -- pause and try again
             madeWatch = false;
             if(i == 4) {
-              throw e;
+              // nocommit:
+              // no shards yet, just bail
+              break;
             }
             Thread.sleep(50);
           }

Modified: lucene/solr/branches/cloud/src/test/org/apache/solr/cloud/AbstractZkTestCase.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/cloud/src/test/org/apache/solr/cloud/AbstractZkTestCase.java?rev=903853&r1=903852&r2=903853&view=diff
==============================================================================
--- lucene/solr/branches/cloud/src/test/org/apache/solr/cloud/AbstractZkTestCase.java (original)
+++ lucene/solr/branches/cloud/src/test/org/apache/solr/cloud/AbstractZkTestCase.java Wed Jan 27 22:02:37 2010
@@ -104,9 +104,9 @@
 
     zkClient = new SolrZkClient(ZOO_KEEPER_ADDRESS, AbstractZkTestCase.TIMEOUT);
     
-    // nocommit : test should work with or without this as we ony have one config
-    //zkClient.makePath("/collections/collection1/config=collection1");
-
+    zkClient.makePath("/collections/collection1/config=conf1");
+    zkClient.makePath("/collections/testcore/config=conf1");
+    
     putConfig(zkClient, config);
     putConfig(zkClient, schema);
     putConfig(zkClient, "stopwords.txt");