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/03 16:40:06 UTC

svn commit: r895421 - in /lucene/solr/branches/cloud/src: java/org/apache/solr/cloud/ java/org/apache/solr/core/ test/org/apache/solr/cloud/

Author: markrmiller
Date: Sun Jan  3 15:40:05 2010
New Revision: 895421

URL: http://svn.apache.org/viewvc?rev=895421&view=rev
Log:
some clean up and support for a core to de-register its zk node on close

Added:
    lucene/solr/branches/cloud/src/test/org/apache/solr/cloud/ZkSolrClientTest.java
Modified:
    lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/SolrZkClient.java
    lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/ZkController.java
    lucene/solr/branches/cloud/src/java/org/apache/solr/core/SolrCore.java
    lucene/solr/branches/cloud/src/test/org/apache/solr/cloud/AbstractDistributedZkTestCase.java
    lucene/solr/branches/cloud/src/test/org/apache/solr/cloud/AbstractZkTestCase.java
    lucene/solr/branches/cloud/src/test/org/apache/solr/cloud/ZkControllerTest.java

Modified: lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/SolrZkClient.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/SolrZkClient.java?rev=895421&r1=895420&r2=895421&view=diff
==============================================================================
--- lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/SolrZkClient.java (original)
+++ lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/SolrZkClient.java Sun Jan  3 15:40:05 2010
@@ -35,6 +35,14 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+/**
+ * 
+ * All Solr ZooKeeper interactions should go through this class rather than
+ * ZooKeeper. This class handles synchronous connects and reconnections.
+ * 
+ * nocommit: and may end up optionally queuing commands when zk is down
+ *
+ */
 public class SolrZkClient {
   static final String NEWL = System.getProperty("line.separator");
 
@@ -49,10 +57,25 @@
 
   private volatile ZooKeeper keeper;
   
+  /**
+   * @param zkServerAddress
+   * @param zkClientTimeout
+   * @throws InterruptedException
+   * @throws TimeoutException
+   * @throws IOException
+   */
   public SolrZkClient(String zkServerAddress, int zkClientTimeout) throws InterruptedException, TimeoutException, IOException {
     this(zkServerAddress, zkClientTimeout, new DefaultConnectionStrategy());
   }
 
+  /**
+   * @param zkServerAddress
+   * @param zkClientTimeout
+   * @param strat
+   * @throws InterruptedException
+   * @throws TimeoutException
+   * @throws IOException
+   */
   public SolrZkClient(String zkServerAddress, int zkClientTimeout,
       ZkClientConnectionStrategy strat) throws InterruptedException,
       TimeoutException, IOException {
@@ -67,35 +90,88 @@
     connManager.waitForConnected(CONNECT_TIMEOUT);
   }
 
+  /**
+   * @return true if client is connected
+   */
   public boolean isConnected() {
     return keeper != null && keeper.getState() == ZooKeeper.States.CONNECTED;
   }
+  
+  /**
+   * @param path
+   * @param version
+   * @throws InterruptedException
+   * @throws KeeperException
+   */
+  public void delete(final String path, int version)
+      throws InterruptedException, KeeperException {
+    keeper.delete(path, version);
+  }
 
+  /**
+   * @param path
+   * @param watcher
+   * @return
+   * @throws KeeperException
+   * @throws InterruptedException
+   */
   public Stat exists(final String path, Watcher watcher)
       throws KeeperException, InterruptedException {
     return keeper.exists(path, watcher);
   }
 
+  /**
+   * @param path
+   * @param data
+   * @param acl
+   * @param createMode
+   * @return
+   * @throws KeeperException
+   * @throws InterruptedException
+   */
   public String create(final String path, byte data[], List<ACL> acl,
       CreateMode createMode) throws KeeperException, InterruptedException {
     return keeper.create(path, data, acl, createMode);
   }
 
+  /**
+   * @param path
+   * @param watcher
+   * @return
+   * @throws KeeperException
+   * @throws InterruptedException
+   */
   public List<String> getChildren(final String path, Watcher watcher)
       throws KeeperException, InterruptedException {
     return keeper.getChildren(path, watcher);
   }
 
+  /**
+   * @param path
+   * @param watcher
+   * @param stat
+   * @return
+   * @throws KeeperException
+   * @throws InterruptedException
+   */
   public byte[] getData(final String path, Watcher watcher, Stat stat)
       throws KeeperException, InterruptedException {
     return keeper.getData(path, watcher, stat);
   }
 
+  /**
+   * @param path
+   * @param data
+   * @param version
+   * @return
+   * @throws KeeperException
+   * @throws InterruptedException
+   */
   public Stat setData(final String path, byte data[], int version)
       throws KeeperException, InterruptedException {
     return keeper.setData(path, data, version);
   }
-
+  
   /**
    * 
    * @param path
@@ -105,13 +181,9 @@
    * @throws KeeperException
    * @throws InterruptedException
    */
-  public String create(String path, byte[] data, CreateMode createMode,
-      Watcher watcher) throws KeeperException, InterruptedException {
+  public String create(String path, byte[] data, CreateMode createMode) throws KeeperException, InterruptedException {
 
-    String zkPath = keeper.create(path, data, ZooDefs.Ids.OPEN_ACL_UNSAFE,
-        createMode);
-    // nocommit : race issue on keeper switch
-    exists(zkPath, watcher);
+    String zkPath = keeper.create(path, data, ZooDefs.Ids.OPEN_ACL_UNSAFE, createMode);
 
     return zkPath;
   }
@@ -321,7 +393,17 @@
     keeper.close();
   }
 
+  /**
+   * Allows package private classes to update volatile ZooKeeper.
+   * 
+   * @param keeper
+   */
   void updateKeeper(ZooKeeper keeper) {
    this.keeper = keeper;
   }
+
+  // for testing
+  void dissconect() throws InterruptedException {
+    this.keeper.close();
+  }
 }

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=895421&r1=895420&r2=895421&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 Sun Jan  3 15:40:05 2010
@@ -142,13 +142,13 @@
 
   private String zooKeeperHostName;
 
-  private int zkClientTimeout;
+  private String hostName;
 
   /**
    * 
    * @param zkServerAddress ZooKeeper server host address
    * @param collection
-   * @param hostUrl
+   * @param hostName
    * @param hostPort
    * @param hostContext
    * @param zkClientTimeout
@@ -157,13 +157,13 @@
    * @throws InterruptedException 
    */
   public ZkController(String zkServerAddress, String collection,
-      String hostUrl, String hostPort, String hostContext, int zkClientTimeout) throws InterruptedException, TimeoutException, IOException {
+      String hostName, String hostPort, String hostContext, int zkClientTimeout) throws InterruptedException, TimeoutException, IOException {
 
     this.collectionName = collection;
     this.zkServerAddress = zkServerAddress;
-    this.zkClientTimeout = zkClientTimeout;
     this.hostPort = hostPort;
     this.hostContext = hostContext;
+    this.hostName = hostName;
     zkClient = new SolrZkClient(zkServerAddress, zkClientTimeout);
 
     shardsZkPath = COLLECTIONS_ZKNODE + collectionName + SHARDS_ZKNODE;
@@ -178,7 +178,6 @@
       Matcher m = URL_POST.matcher(zooKeeperHostName);
       if (m.matches()) {
         String hostName = m.group(1);
-
         // register host
         zkClient.makePath(hostName);
       } else {
@@ -307,8 +306,9 @@
    * ZooKeeper.
    * 
    * @param core
+   * @return
    */
-  public void registerShard(SolrCore core) {
+  public String registerShard(SolrCore core) {
     String coreName = core.getCoreDescriptor().getName();
     String shardUrl = zooKeeperHostName + ":" + hostPort + "/" + hostContext
         + "/" + coreName;
@@ -319,6 +319,7 @@
           + shardUrl);
     }
 
+    String nodePath = null;
     try {
       // create node
       ByteArrayOutputStream baos = new ByteArrayOutputStream();
@@ -331,38 +332,67 @@
       props.put(CollectionInfo.SHARD_LIST_PROP, shardList == null ? "" : shardList);
       props.store(baos, PROPS_DESC);
 
-      zkClient.create(shardsZkPath + NODE_ZKPREFIX, baos
-          .toByteArray(), CreateMode.EPHEMERAL_SEQUENTIAL, SHARD_WATCHER);
+      
+      nodePath = zkClient.create(shardsZkPath + NODE_ZKPREFIX, baos
+          .toByteArray(), CreateMode.EPHEMERAL_SEQUENTIAL);
+      // nocommit
+      zkClient.exists(shardsZkPath, SHARD_WATCHER);
 
     } catch (InterruptedException e) {
       // Restore the interrupted status
       Thread.currentThread().interrupt();
-    } catch (Exception e) {
+    } catch (KeeperException e) {
       log.error("ZooKeeper Exception", e);
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
           "ZooKeeper Exception", e);
-    }
+    } catch (IOException e) {
+      log.error("", e);
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+          "", e);
+    } 
 
+    return nodePath;
+  }
+  
+  /**
+   * @param core
+   * @param zkNodePath
+   */
+  public void unRegisterShard(SolrCore core, String zkNodePath) {
+    // nocommit : version?
+    try {
+      zkClient.delete(zkNodePath, -1);
+    } catch (InterruptedException e) {
+      // Restore the interrupted status
+      Thread.currentThread().interrupt();
+    } catch (KeeperException.NoNodeException e) {
+      // nocommit - this is okay - for some reason the node is already gone
+      log.warn("Unregistering core: " + core.getName()
+          + " but core's ZooKeeper node has already been removed");
+    } catch (KeeperException e) {
+      log.error("ZooKeeper Exception", e);
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+          "ZooKeeper Exception", e);
+    }
   }
 
   // nocommit: fooling around
   private String getHostAddress() throws IOException {
-    String host = null;
 
-    if (host == null) {
-      host = "http://" + InetAddress.getLocalHost().getHostName();
+    if (hostName == null) {
+      hostName = "http://" + InetAddress.getLocalHost().getHostName();
     } else {
-      Matcher m = URL_PREFIX.matcher(host);
+      Matcher m = URL_PREFIX.matcher(hostName);
       if (m.matches()) {
         String prefix = m.group(1);
-        host = prefix + host;
+        hostName = prefix + hostName;
       }
     }
     if (log.isInfoEnabled()) {
-      log.info("Register host with ZooKeeper:" + host);
+      log.info("Register host with ZooKeeper:" + hostName);
     }
 
-    return host;
+    return hostName;
   }
 
   /**
@@ -414,28 +444,6 @@
     return zkClient.getData(CONFIGS_ZKNODE + zkConfigName, null, null);
   }
 
-  // /**
-  // * Get data at zkNode path/fileName.
-  // *
-  // * @param path to zkNode
-  // * @param fileName name of zkNode
-  // * @return data at path/file
-  // * @throws InterruptedException
-  // * @throws KeeperException
-  // */
-  // public byte[] getFile(String path, String fileName) throws KeeperException,
-  // InterruptedException {
-  // byte[] bytes = null;
-  // String configPath = path + "/" + fileName;
-  //
-  // if (log.isInfoEnabled()) {
-  // log.info("Reading " + fileName + " from zookeeper at " + configPath);
-  // }
-  // bytes = keeperConnection.getData(configPath, null, null);
-  //
-  // return bytes;
-  // }
-
   /**
    * Load IndexSchema from ZooKeeper.
    * 

Modified: lucene/solr/branches/cloud/src/java/org/apache/solr/core/SolrCore.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/cloud/src/java/org/apache/solr/core/SolrCore.java?rev=895421&r1=895420&r2=895421&view=diff
==============================================================================
--- lucene/solr/branches/cloud/src/java/org/apache/solr/core/SolrCore.java (original)
+++ lucene/solr/branches/cloud/src/java/org/apache/solr/core/SolrCore.java Sun Jan  3 15:40:05 2010
@@ -285,6 +285,9 @@
   List<SolrEventListener> newSearcherListeners;
 
   private ZkController zooKeeperComponent;
+  
+  // path of node that represents this core in ZooKeeper
+  private String zkNodePath;
 
   /**
    * NOTE: this function is not thread safe.  However, it is safe to call within the
@@ -528,7 +531,7 @@
     
     zooKeeperComponent = cd.getCoreContainer().getZooKeeperController();
     if(zooKeeperComponent != null) {
-      zooKeeperComponent.registerShard(this);
+      this.zkNodePath = zooKeeperComponent.registerShard(this);
     }
 
     //Initialize JMX
@@ -701,6 +704,7 @@
    * @see #isClosed() 
    */
   public void close() {
+    
     int count = refCount.decrementAndGet();
     if (count > 0) return; // close is called often, and only actually closes if nothing is using it.
     if (count < 0) {
@@ -708,6 +712,10 @@
       return;
     }
     log.info(logid+" CLOSING SolrCore " + this);
+    // nocommit : if ZooKeeper, unregister core
+    if(zooKeeperComponent != null) {
+      zooKeeperComponent.unRegisterShard(this, zkNodePath);
+    }
     try {
       infoRegistry.clear();
     } catch (Exception e) {

Modified: lucene/solr/branches/cloud/src/test/org/apache/solr/cloud/AbstractDistributedZkTestCase.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/cloud/src/test/org/apache/solr/cloud/AbstractDistributedZkTestCase.java?rev=895421&r1=895420&r2=895421&view=diff
==============================================================================
--- lucene/solr/branches/cloud/src/test/org/apache/solr/cloud/AbstractDistributedZkTestCase.java (original)
+++ lucene/solr/branches/cloud/src/test/org/apache/solr/cloud/AbstractDistributedZkTestCase.java Sun Jan  3 15:40:05 2010
@@ -37,7 +37,7 @@
     log.info("####SETUP_START " + getName());
     portSeed = 13000;
     
-    System.setProperty("zkHost", AbstractZkTestCase.ZOO_KEEPER_HOST);
+    System.setProperty("zkHost", AbstractZkTestCase.ZOO_KEEPER_ADDRESS);
     String zkDir = tmpDir.getAbsolutePath() + File.separator
     + "zookeeper/server1/data";
     zkServer = new ZkTestServer(zkDir);

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=895421&r1=895420&r2=895421&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 Sun Jan  3 15:40:05 2010
@@ -30,7 +30,8 @@
  */
 public abstract class AbstractZkTestCase extends AbstractSolrTestCase {
 
-  static final String ZOO_KEEPER_HOST = "localhost:2181/solr";
+  static final String ZOO_KEEPER_ADDRESS = "localhost:2181/solr";
+  static final String ZOO_KEEPER_SERVER = "localhost:2181";
   static final int TIMEOUT = 10000;
 
   protected static Logger log = LoggerFactory
@@ -57,7 +58,7 @@
 
   public void setUp() throws Exception {
     try {
-      System.setProperty("zkHost", ZOO_KEEPER_HOST);
+      System.setProperty("zkHost", ZOO_KEEPER_ADDRESS);
       String zkDir = tmpDir.getAbsolutePath() + File.separator
       + "zookeeper/server1/data";
       zkServer = new ZkTestServer(zkDir);
@@ -90,8 +91,8 @@
 
   }
 
-  final static String JUST_HOST_NAME = AbstractZkTestCase.ZOO_KEEPER_HOST.substring(0,
-      AbstractZkTestCase.ZOO_KEEPER_HOST.indexOf('/'));
+  final static String JUST_HOST_NAME = AbstractZkTestCase.ZOO_KEEPER_ADDRESS.substring(0,
+      AbstractZkTestCase.ZOO_KEEPER_ADDRESS.indexOf('/'));
   
   // static to share with distrib test
   static void buildZooKeeper(String config, String schema)
@@ -100,7 +101,7 @@
     zkClient.makePath("/solr");
     zkClient.close();
 
-    zkClient = new SolrZkClient(ZOO_KEEPER_HOST, AbstractZkTestCase.TIMEOUT);
+    zkClient = new SolrZkClient(ZOO_KEEPER_ADDRESS, AbstractZkTestCase.TIMEOUT);
     
     zkClient.makePath("/collections/collection1/config=collection1");
 
@@ -130,10 +131,15 @@
 
   private void printLayout() throws Exception {
     SolrZkClient zkClient = new SolrZkClient(
-        AbstractZkTestCase.ZOO_KEEPER_HOST.substring(0,
-            AbstractZkTestCase.ZOO_KEEPER_HOST.indexOf('/')),
+        AbstractZkTestCase.ZOO_KEEPER_SERVER,
         AbstractZkTestCase.TIMEOUT);
     zkClient.printLayoutToStdOut();
     zkClient.close();
   }
+  
+  static void makeSolrZkNode() throws Exception {
+    SolrZkClient zkClient = new SolrZkClient(ZOO_KEEPER_SERVER, TIMEOUT);
+    zkClient.makePath("/solr");
+    zkClient.close();
+  }
 }

Modified: lucene/solr/branches/cloud/src/test/org/apache/solr/cloud/ZkControllerTest.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/cloud/src/test/org/apache/solr/cloud/ZkControllerTest.java?rev=895421&r1=895420&r2=895421&view=diff
==============================================================================
--- lucene/solr/branches/cloud/src/test/org/apache/solr/cloud/ZkControllerTest.java (original)
+++ lucene/solr/branches/cloud/src/test/org/apache/solr/cloud/ZkControllerTest.java Sun Jan  3 15:40:05 2010
@@ -39,6 +39,7 @@
   private static final String SHARD1 = "shard1";
 
   static final String ZOO_KEEPER_ADDRESS = "localhost:2181/solr";
+
   static final String ZOO_KEEPER_HOST = "localhost:2181";
 
   static final int TIMEOUT = 10000;
@@ -55,62 +56,65 @@
     String zkDir = tmpDir.getAbsolutePath() + File.separator
         + "zookeeper/server1/data";
     ZkTestServer server = null;
-      SolrZkClient zkClient = null;
+    SolrZkClient zkClient = null;
     try {
-    server = new ZkTestServer(zkDir);
-    server.run();
+      server = new ZkTestServer(zkDir);
+      server.run();
 
-    makeSolrZkNode();
-    
-    zkClient = new SolrZkClient(ZOO_KEEPER_ADDRESS, TIMEOUT);
-    String shardsPath = "/collections/collection1/shards";
-    zkClient.makePath(shardsPath);
-    
-    zkClient.makePath("collections/collection1/config=collection1");
+      AbstractZkTestCase.makeSolrZkNode();
 
-    addShardToZk(zkClient, shardsPath, URL1, SHARD1 + "," + SHARD2);
-    addShardToZk(zkClient, shardsPath, "http://localhost:3123/solr/core1", SHARD1);
-    addShardToZk(zkClient, shardsPath, "http://localhost:3133/solr/core1", SHARD1);
+      zkClient = new SolrZkClient(ZOO_KEEPER_ADDRESS, TIMEOUT);
+      String shardsPath = "/collections/collection1/shards";
+      zkClient.makePath(shardsPath);
 
+      zkClient.makePath("collections/collection1/config=collection1");
 
-    if (DEBUG) {
-      zkClient.printLayoutToStdOut();
-    }
+      addShardToZk(zkClient, shardsPath, URL1, SHARD1 + "," + SHARD2);
+      addShardToZk(zkClient, shardsPath, "http://localhost:3123/solr/core1",
+          SHARD1);
+      addShardToZk(zkClient, shardsPath, "http://localhost:3133/solr/core1",
+          SHARD1);
 
-    ZkController zkController = new ZkController(ZOO_KEEPER_ADDRESS, "collection1", "localhost", "8983", "/solr", TIMEOUT);
-    Map<String,ShardInfoList> shardInfoMap = zkController.readShardInfo(shardsPath);
-    assertTrue(shardInfoMap.size() > 0);
-    
-    Set<Entry<String,ShardInfoList>> entries = shardInfoMap.entrySet();
+      if (DEBUG) {
+        zkClient.printLayoutToStdOut();
+      }
 
-    if (DEBUG) {
-      for (Entry<String,ShardInfoList> entry : entries) {
-        System.out.println("shard:" + entry.getKey() + " value:"
-            + entry.getValue().toString());
+      ZkController zkController = new ZkController(ZOO_KEEPER_ADDRESS,
+          "collection1", "localhost", "8983", "/solr", TIMEOUT);
+      Map<String,ShardInfoList> shardInfoMap = zkController
+          .readShardInfo(shardsPath);
+      assertTrue(shardInfoMap.size() > 0);
+
+      Set<Entry<String,ShardInfoList>> entries = shardInfoMap.entrySet();
+
+      if (DEBUG) {
+        for (Entry<String,ShardInfoList> entry : entries) {
+          System.out.println("shard:" + entry.getKey() + " value:"
+              + entry.getValue().toString());
+        }
       }
-    }
 
-    Set<String> keys = shardInfoMap.keySet();
+      Set<String> keys = shardInfoMap.keySet();
 
-    assertTrue(keys.size() == 2);
+      assertTrue(keys.size() == 2);
 
-    assertTrue(keys.contains(SHARD1));
-    assertTrue(keys.contains(SHARD2));
+      assertTrue(keys.contains(SHARD1));
+      assertTrue(keys.contains(SHARD2));
 
-    ShardInfoList shardInfoList = shardInfoMap.get(SHARD1);
+      ShardInfoList shardInfoList = shardInfoMap.get(SHARD1);
 
-    assertEquals(3, shardInfoList.getShards().size());
+      assertEquals(3, shardInfoList.getShards().size());
 
-    shardInfoList = shardInfoMap.get(SHARD2);
+      shardInfoList = shardInfoMap.get(SHARD2);
 
-    assertEquals(1, shardInfoList.getShards().size());
+      assertEquals(1, shardInfoList.getShards().size());
 
-    assertEquals(URL1, shardInfoList.getShards().get(0).getUrl());
+      assertEquals(URL1, shardInfoList.getShards().get(0).getUrl());
     } finally {
-      if(zkClient != null) {
+      if (zkClient != null) {
         zkClient.close();
       }
-      if(server != null) {
+      if (server != null) {
         server.shutdown();
       }
     }
@@ -123,25 +127,27 @@
     ZkTestServer server = new ZkTestServer(zkDir);
     server.run();
 
-    makeSolrZkNode();
+    AbstractZkTestCase.makeSolrZkNode();
 
     SolrZkClient zkClient = new SolrZkClient(ZOO_KEEPER_ADDRESS, TIMEOUT);
     String actualConfigName = "firstConfig";
-      
-    String shardsPath = "/collections/" + COLLECTION_NAME + "/config=" + actualConfigName;
+
+    String shardsPath = "/collections/" + COLLECTION_NAME + "/config="
+        + actualConfigName;
     zkClient.makePath(shardsPath);
 
     if (DEBUG) {
       zkClient.printLayoutToStdOut();
     }
-    
-    ZkController zkController = new ZkController(ZOO_KEEPER_ADDRESS, "collection1", "localhost", "8983", "/solr", TIMEOUT);
+
+    ZkController zkController = new ZkController(ZOO_KEEPER_ADDRESS,
+        "collection1", "localhost", "8983", "/solr", TIMEOUT);
     String configName = zkController.readConfigName(COLLECTION_NAME);
     assertEquals(configName, actualConfigName);
-    
+
     zkClient.close();
     server.shutdown();
-    
+
   }
 
   private void addShardToZk(SolrZkClient zkClient, String shardsPath,
@@ -154,13 +160,9 @@
     props.put(CollectionInfo.SHARD_LIST_PROP, shardList);
     props.store(baos, ZkController.PROPS_DESC);
 
-    zkClient.create(shardsPath
-        + ZkController.NODE_ZKPREFIX, baos.toByteArray(), CreateMode.EPHEMERAL_SEQUENTIAL, null);
+    zkClient.create(shardsPath + ZkController.NODE_ZKPREFIX,
+        baos.toByteArray(), CreateMode.EPHEMERAL_SEQUENTIAL);
   }
 
-  private void makeSolrZkNode() throws Exception {
-    SolrZkClient zkClient = new SolrZkClient(ZOO_KEEPER_HOST, TIMEOUT);
-    zkClient.makePath("/solr");
-    zkClient.close();
-  }
+
 }

Added: lucene/solr/branches/cloud/src/test/org/apache/solr/cloud/ZkSolrClientTest.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/cloud/src/test/org/apache/solr/cloud/ZkSolrClientTest.java?rev=895421&view=auto
==============================================================================
--- lucene/solr/branches/cloud/src/test/org/apache/solr/cloud/ZkSolrClientTest.java (added)
+++ lucene/solr/branches/cloud/src/test/org/apache/solr/cloud/ZkSolrClientTest.java Sun Jan  3 15:40:05 2010
@@ -0,0 +1,43 @@
+package org.apache.solr.cloud;
+
+import java.io.File;
+
+import junit.framework.TestCase;
+
+public class ZkSolrClientTest extends TestCase {
+  protected File tmpDir = new File(System.getProperty("java.io.tmpdir")
+      + System.getProperty("file.separator") + getClass().getName() + "-"
+      + System.currentTimeMillis());
+  
+  public void testBasic() throws Exception {
+    String zkDir = tmpDir.getAbsolutePath() + File.separator
+        + "zookeeper/server1/data";
+    ZkTestServer server = null;
+    SolrZkClient zkClient = null;
+    try {
+      server = new ZkTestServer(zkDir);
+      server.run();
+
+      AbstractZkTestCase.makeSolrZkNode();
+
+      zkClient = new SolrZkClient(AbstractZkTestCase.ZOO_KEEPER_ADDRESS,
+          AbstractZkTestCase.TIMEOUT);
+      String shardsPath = "/collections/collection1/shards";
+      zkClient.makePath(shardsPath);
+
+      zkClient.makePath("collections/collection1/config=collection1");
+      
+      zkClient.dissconect();
+      
+      zkClient.makePath("collections/collection1/config=collection2");
+
+    } finally {
+      if (zkClient != null) {
+        zkClient.close();
+      }
+      if (server != null) {
+        server.shutdown();
+      }
+    }
+  }
+}