Author: markrmiller
Date: Mon Jan 25 19:05:38 2010
New Revision: 902932

URL: http://svn.apache.org/viewvc?rev=902932&view=rev
Log:
watch should be getChildren not exists

Modified:
    lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/ZkController.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=902932&r1=902931&r2=902932&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 
Mon Jan 25 19:05:38 2010
@@ -418,7 +418,13 @@
     Map<String,Map<String,Slice>> collectionStates = new 
HashMap<String,Map<String,Slice>>();
     for (String collection : collections) {
       String shardIdPaths = COLLECTIONS_ZKNODE + "/" + collection + 
SHARDS_ZKNODE;
-      List<String> shardIdNames = zkClient.getChildren(shardIdPaths, null);
+      List<String> shardIdNames;
+      try {
+        shardIdNames = zkClient.getChildren(shardIdPaths, null);
+      } catch(KeeperException.NoNodeException e) {
+        // node is not valid currently
+        continue;
+      }
       Map<String,Slice> slices = new HashMap<String,Slice>();
       for(String shardIdZkPath : shardIdNames) {
         Map<String,ZkNodeProps> shardsMap = readShards(shardIdPaths + "/" + 
shardIdZkPath);
@@ -695,16 +701,17 @@
     }
     
     log.info("Start watching collections node for changes");
-    zkClient.exists(COLLECTIONS_ZKNODE, new Watcher(){
+    zkClient.getChildren(COLLECTIONS_ZKNODE, new Watcher(){
 
       public void process(WatchedEvent event) {
           try {
             // TODO: fine grained - just reload what's changed
-            
+            // nocommit
+            log.info("children changed");
             // something changed, reload cloud state
             updateCloudState();
             // re-watch
-            zkClient.exists(event.getPath(), this);
+            zkClient.getChildren(event.getPath(), this);
           } catch (KeeperException e) {
             log.error("", e);
             throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,


Reply via email to