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 2009/12/10 20:23:41 UTC

svn commit: r889376 - in /lucene/solr/branches/cloud: lib/ src/java/org/apache/solr/core/ src/java/org/apache/solr/handler/component/ src/java/org/apache/solr/util/ src/test/org/apache/solr/

Author: markrmiller
Date: Thu Dec 10 19:23:39 2009
New Revision: 889376

URL: http://svn.apache.org/viewvc?rev=889376&view=rev
Log:
base cloud commit - initial code to play with

Added:
    lucene/solr/branches/cloud/lib/log4j-1.2.15.jar   (with props)
    lucene/solr/branches/cloud/lib/zookeeper-3.2.1.jar   (with props)
    lucene/solr/branches/cloud/src/java/org/apache/solr/core/ZKSolrResourceLoader.java
    lucene/solr/branches/cloud/src/java/org/apache/solr/core/ZooKeeperController.java
    lucene/solr/branches/cloud/src/java/org/apache/solr/util/ZooPut.java
    lucene/solr/branches/cloud/src/test/org/apache/solr/AbstractDistributedZooKeeperTestCase.java
    lucene/solr/branches/cloud/src/test/org/apache/solr/AbstractZooKeeperTestCase.java
    lucene/solr/branches/cloud/src/test/org/apache/solr/BasicDistributedZooKeeperTest.java
    lucene/solr/branches/cloud/src/test/org/apache/solr/BasicZooKeeperTest.java
Modified:
    lucene/solr/branches/cloud/src/java/org/apache/solr/core/CoreContainer.java
    lucene/solr/branches/cloud/src/java/org/apache/solr/core/SolrCore.java
    lucene/solr/branches/cloud/src/java/org/apache/solr/core/SolrResourceLoader.java
    lucene/solr/branches/cloud/src/java/org/apache/solr/handler/component/QueryElevationComponent.java

Added: lucene/solr/branches/cloud/lib/log4j-1.2.15.jar
URL: http://svn.apache.org/viewvc/lucene/solr/branches/cloud/lib/log4j-1.2.15.jar?rev=889376&view=auto
==============================================================================
Binary file - no diff available.

Propchange: lucene/solr/branches/cloud/lib/log4j-1.2.15.jar
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Added: lucene/solr/branches/cloud/lib/zookeeper-3.2.1.jar
URL: http://svn.apache.org/viewvc/lucene/solr/branches/cloud/lib/zookeeper-3.2.1.jar?rev=889376&view=auto
==============================================================================
Binary file - no diff available.

Propchange: lucene/solr/branches/cloud/lib/zookeeper-3.2.1.jar
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Modified: lucene/solr/branches/cloud/src/java/org/apache/solr/core/CoreContainer.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/cloud/src/java/org/apache/solr/core/CoreContainer.java?rev=889376&r1=889375&r2=889376&view=diff
==============================================================================
--- lucene/solr/branches/cloud/src/java/org/apache/solr/core/CoreContainer.java (original)
+++ lucene/solr/branches/cloud/src/java/org/apache/solr/core/CoreContainer.java Thu Dec 10 19:23:39 2009
@@ -75,10 +75,23 @@
   protected String adminHandler;
   protected boolean shareSchema;
   protected String solrHome;
+  protected static final String collection = System.getProperty("collection", "collection1"); //nocommit: default;
+
+  private ZooKeeperController zooKeeperController;
 
   public CoreContainer() {
     solrHome = SolrResourceLoader.locateSolrHome();
   }
+  
+  public CoreContainer(ZooKeeperController zooKeeperController) {
+    this();
+    this.zooKeeperController = zooKeeperController;
+  }
+  
+  public CoreContainer(String solrHome, ZooKeeperController zooKeeperController) {
+    this.solrHome = solrHome;
+    this.zooKeeperController = zooKeeperController;
+  }
 
   public Properties getContainerProperties() {
     return containerProperties;
@@ -88,6 +101,11 @@
   public static class Initializer {
     protected String solrConfigFilename = null;
     protected boolean abortOnConfigurationError = true;
+    protected String dataDir = null; // override datadir for single core mode
+    
+    private ZooKeeperController zooKeeperController;
+    private String zookeeperHost;
+    private String zookeeperSolrPath;
 
     public boolean isAbortOnConfigurationError() {
       return abortOnConfigurationError;
@@ -107,13 +125,29 @@
 
     // core container instantiation
     public CoreContainer initialize() throws IOException, ParserConfigurationException, SAXException {
+      //nocommit: pull zookeeper integration into a new CoreContainer? leaning towards no.
+      zookeeperHost = System.getProperty("zkHost");
+      
+      if (zookeeperHost != null) {
+        zooKeeperController = new ZooKeeperController(zookeeperHost, collection);
+      }
+      
       CoreContainer cores = null;
       String solrHome = SolrResourceLoader.locateSolrHome();
-      File fconf = new File(solrHome, solrConfigFilename == null? "solr.xml": solrConfigFilename);
-      log.info("looking for solr.xml: " + fconf.getAbsolutePath());
+      
+      File fconf = null;
 
+      fconf = new File(solrHome, solrConfigFilename == null ? "solr.xml"
+          : solrConfigFilename);
+      log.info("looking for solr.xml: " + fconf.getAbsolutePath());
+      
       if (fconf.exists()) {
-        cores = new CoreContainer();
+        //nocommit: finish ZooKeeper support in multicore case
+        if (zookeeperHost == null) {
+          cores = new CoreContainer();
+        } else {
+          cores = new CoreContainer(zooKeeperController);
+        }
         cores.load(solrHome, fconf);
         abortOnConfigurationError = false;
         // if any core aborts on startup, then abort
@@ -126,15 +160,37 @@
         solrConfigFilename = cores.getConfigFile().getName();
       } else {
         // perform compatibility init
-        cores = new CoreContainer(solrHome);
-        CoreDescriptor dcore = new CoreDescriptor(cores, "", ".");
-        dcore.setCoreProperties(null);
-        SolrResourceLoader resourceLoader = new SolrResourceLoader(solrHome, null, getCoreProps(solrHome, null,dcore.getCoreProperties()));
+        SolrResourceLoader resourceLoader;
+        CoreDescriptor dcore;
+        if (zookeeperHost == null) {
+          cores = new CoreContainer(solrHome);
+           new CoreDescriptor(cores, "", ".");
+          dcore = new CoreDescriptor(cores, "", ".");
+          dcore.setCoreProperties(null);
+          resourceLoader = new SolrResourceLoader(solrHome, null, getCoreProps(solrHome, null, dcore.getCoreProperties()));
+        } else {
+          cores = new CoreContainer(solrHome, zooKeeperController);
+          dcore = new CoreDescriptor(cores, "", ".");
+          dcore.setCoreProperties(null);
+          resourceLoader = new ZKSolrResourceLoader(solrHome, collection, null, getCoreProps(solrHome, null, dcore.getCoreProperties()), zooKeeperController);
+        }
+       
         cores.loader = resourceLoader;
-        SolrConfig cfg = solrConfigFilename == null ?
-                new SolrConfig(resourceLoader, SolrConfig.DEFAULT_CONF_FILE,null) :
-                new SolrConfig(resourceLoader, solrConfigFilename,null);
-        SolrCore singlecore = new SolrCore(null, null, cfg, null, dcore);
+        SolrConfig cfg;
+        IndexSchema schema = null;
+        String solrConfigName = solrConfigFilename == null ? SolrConfig.DEFAULT_CONF_FILE : solrConfigFilename;
+        if (zookeeperHost == null) {
+          cfg = new SolrConfig(resourceLoader, solrConfigName, null);
+        } else {
+          // found ZooKeeper path, so load config from zookeeper
+          log.info("Loading config from ZooKeeper");
+          // load solrconfig
+          cfg = zooKeeperController.getConfig(solrConfigName, resourceLoader);
+          // load schema
+          schema = zooKeeperController.getSchema(IndexSchema.DEFAULT_SCHEMA_FILE, cfg, resourceLoader);
+        }
+        
+        SolrCore singlecore = new SolrCore(null, dataDir, cfg, schema, dcore, zooKeeperController);
         abortOnConfigurationError = cfg.getBool(
                 "abortOnConfigurationError", abortOnConfigurationError);
         cores.register("", singlecore, false);
@@ -219,7 +275,7 @@
       adminPath  = cfg.get(     "solr/cores/@adminPath", null );
       shareSchema = cfg.getBool("solr/cores/@shareSchema", false );
       if(shareSchema){
-        indexSchemaCache = new ConcurrentHashMap<String ,IndexSchema>();
+        indexSchemaCache = new ConcurrentHashMap<String,IndexSchema>();
       }
       adminHandler  = cfg.get("solr/cores/@adminHandler", null );
       managementPath  = cfg.get("solr/cores/@managementPath", null );
@@ -252,7 +308,10 @@
         try {
           String names = DOMUtil.getAttr(node, "name", null);
           List<String> aliases = StrUtils.splitSmart(names,',');
-          String name = aliases.get(0);
+          String name = "";
+          if(aliases.size() > 0) {
+            name = aliases.get(0);
+          }
           CoreDescriptor p = new CoreDescriptor(this, name, DOMUtil.getAttr(node, "instanceDir", null));
 
           // deal with optional settings
@@ -320,6 +379,9 @@
         }
         cores.clear();
       } finally {
+        if(zooKeeperController != null) {
+          zooKeeperController.close();
+        }
         isShutDown = true;
       }
     }
@@ -394,38 +456,64 @@
    */
   public SolrCore create(CoreDescriptor dcore)  throws ParserConfigurationException, IOException, SAXException {
     // Make the instanceDir relative to the cores instanceDir if not absolute
-    File idir = new File(dcore.getInstanceDir());
-    if (!idir.isAbsolute()) {
-      idir = new File(solrHome, dcore.getInstanceDir());
-    }
-    String instanceDir = idir.getPath();
+    String instanceDir = dcore.getInstanceDir();
+    File idir;
+      idir = new File(instanceDir);
+      if (!idir.isAbsolute()) {
+        idir = new File(solrHome, instanceDir);
+      }
+
+    instanceDir = idir.getPath();
+    System.out.println("instancedir:" + idir.getAbsolutePath());
     
     // Initialize the solr config
-    SolrResourceLoader solrLoader = new SolrResourceLoader(instanceDir, libLoader, getCoreProps(instanceDir, dcore.getPropertiesName(),dcore.getCoreProperties()));
-    SolrConfig config = new SolrConfig(solrLoader, dcore.getConfigName(), null);
+    SolrResourceLoader solrLoader ;
+    
+    SolrConfig config;
+    if(zooKeeperController == null) {
+      solrLoader = new SolrResourceLoader(instanceDir, libLoader, getCoreProps(instanceDir, dcore.getPropertiesName(),dcore.getCoreProperties()));
+      config = new SolrConfig(solrLoader, dcore.getConfigName(), null);
+    } else {
+      solrLoader = new ZKSolrResourceLoader(instanceDir, collection, libLoader, getCoreProps(instanceDir, dcore.getPropertiesName(),dcore.getCoreProperties()), zooKeeperController);
+      config = zooKeeperController.getConfig(dcore.getConfigName(), solrLoader);
+    }
     IndexSchema schema = null;
-    if(indexSchemaCache != null){
-      //schema sharing is enabled. so check if it already is loaded
-      File schemaFile = new File(dcore.getSchemaName());
-      if (!schemaFile.isAbsolute()) {
-        schemaFile = new File(solrLoader.getInstanceDir() + "conf" + File.separator + dcore.getSchemaName());
-      }
-      if(schemaFile. exists()){
-        String key = schemaFile.getAbsolutePath()+":"+new SimpleDateFormat("yyyyMMddhhmmss").format(new Date(schemaFile.lastModified()));
-        schema = indexSchemaCache.get(key);
-        if(schema == null){
-          log.info("creating new schema object for core: " + dcore.name);
-          schema = new IndexSchema(config, dcore.getSchemaName(), null);
-          indexSchemaCache.put(key,schema);
-        } else {
-          log.info("re-using schema object for core: " + dcore.name);
+    if (indexSchemaCache != null) {
+      // nocommit: handle ZooKeeper and schema caching
+      // schema sharing is enabled. so check if it already is loaded
+      if (zooKeeperController != null) {
+        File schemaFile = new File(dcore.getSchemaName());
+        if (!schemaFile.isAbsolute()) {
+          schemaFile = new File(solrLoader.getInstanceDir() + "conf"
+              + File.separator + dcore.getSchemaName());
+        }
+        if (schemaFile.exists()) {
+          String key = schemaFile.getAbsolutePath()
+              + ":"
+              + new SimpleDateFormat("yyyyMMddhhmmss").format(new Date(
+                  schemaFile.lastModified()));
+          schema = indexSchemaCache.get(key);
+          if (schema == null) {
+            log.info("creating new schema object for core: " + dcore.name);
+            schema = new IndexSchema(config, dcore.getSchemaName(), null);
+            indexSchemaCache.put(key, schema);
+          } else {
+            log.info("re-using schema object for core: " + dcore.name);
+          }
         }
+      } else {
+        // nocommit: TODO handle caching from ZooKeeper - perhaps using ZooKeepers versioning
+        // Don't like this cache though - how does it empty as last modified changes?
       }
     }
     if(schema == null){
-      schema = new IndexSchema(config, dcore.getSchemaName(), null);
+      if(zooKeeperController != null) {
+        schema = zooKeeperController.getSchema(dcore.getSchemaName(), config, solrLoader);
+      } else {
+        schema = new IndexSchema(config, dcore.getSchemaName(), null);
+      }
     }
-    SolrCore core = new SolrCore(dcore.getName(), null, config, schema, dcore);
+    SolrCore core = new SolrCore(dcore.getName(), null, config, schema, dcore, zooKeeperController);
     return core;
   }
     
@@ -819,4 +907,9 @@
   public String getSolrHome() {
     return solrHome;
   }
+  
+  //nocommit: consider - used for testing now
+  public boolean isZooKeeperAware() {
+    return zooKeeperController != null;
+  }
 }

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=889376&r1=889375&r2=889376&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 Thu Dec 10 19:23:39 2009
@@ -93,6 +93,9 @@
   private IndexDeletionPolicyWrapper solrDelPolicy;
   private DirectoryFactory directoryFactory;
   private IndexReaderFactory indexReaderFactory;
+  
+  // null if not in use
+  private ZooKeeperController zooKeeperController;
 
   public long getStartTime() { return startTime; }
 
@@ -223,6 +226,14 @@
     return indexReaderFactory;
   }
   
+  //nocommit: consider
+  /**
+   * @return ZooKeeperController for this core or Null if none.
+   */
+  public ZooKeeperController getZooKeeperController() {
+    return zooKeeperController;
+  }
+  
   public String getName() {
     return name;
   }
@@ -497,13 +508,33 @@
   /**
    * Creates a new core and register it in the list of cores.
    * If a core with the same name already exists, it will be stopped and replaced by this one.
-   *@param dataDir the index directory
-   *@param config a solr config instance
-   *@param schema a solr schema instance
-   *
-   *@since solr 1.3
+   * 
+   * @param name name of the core
+   * @param dataDir the index directory
+   * @param config a solr config instance
+   * @param schema a solr schema instance
+   * @param cd descriptor for this core
+   * 
+   * @since solr 1.3
    */
   public SolrCore(String name, String dataDir, SolrConfig config, IndexSchema schema, CoreDescriptor cd) {
+    this(name, dataDir, config, schema, cd, null);
+  }
+  
+
+  /**
+   * Creates a new core and register it in the list of cores.
+   * If a core with the same name already exists, it will be stopped and replaced by this one.
+   * 
+   * @param name name of the core
+   * @param dataDir the index directory
+   * @param config a solr config instance
+   * @param schema a solr schema instance
+   * @param cd descriptor for this core
+   * @param zooKeeperController zooKeeperController for core to use or null
+   */
+  public SolrCore(String name, String dataDir, SolrConfig config, IndexSchema schema, CoreDescriptor cd, ZooKeeperController zooKeeperController) {
+    this.zooKeeperController = zooKeeperController;
     coreDescriptor = cd;
     this.setName( name );
     resourceLoader = config.getResourceLoader();

Modified: lucene/solr/branches/cloud/src/java/org/apache/solr/core/SolrResourceLoader.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/cloud/src/java/org/apache/solr/core/SolrResourceLoader.java?rev=889376&r1=889375&r2=889376&view=diff
==============================================================================
--- lucene/solr/branches/cloud/src/java/org/apache/solr/core/SolrResourceLoader.java (original)
+++ lucene/solr/branches/cloud/src/java/org/apache/solr/core/SolrResourceLoader.java Thu Dec 10 19:23:39 2009
@@ -65,7 +65,7 @@
   static final String base = "org.apache" + "." + project;
   static final String[] packages = {"","analysis.","schema.","handler.","search.","update.","core.","request.","update.processor.","util.", "spelling.", "handler.component.", "handler.dataimport"};
 
-  private URLClassLoader classLoader;
+  protected URLClassLoader classLoader;
   private final String instanceDir;
   private String dataDir;
   

Added: lucene/solr/branches/cloud/src/java/org/apache/solr/core/ZKSolrResourceLoader.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/cloud/src/java/org/apache/solr/core/ZKSolrResourceLoader.java?rev=889376&view=auto
==============================================================================
--- lucene/solr/branches/cloud/src/java/org/apache/solr/core/ZKSolrResourceLoader.java (added)
+++ lucene/solr/branches/cloud/src/java/org/apache/solr/core/ZKSolrResourceLoader.java Thu Dec 10 19:23:39 2009
@@ -0,0 +1,77 @@
+package org.apache.solr.core;
+
+import java.io.ByteArrayInputStream;
+import java.io.InputStream;
+import java.util.Properties;
+
+public class ZKSolrResourceLoader extends SolrResourceLoader {
+
+  private ZooKeeperController zooKeeperController;
+
+ 
+  String collection;
+
+  public ZKSolrResourceLoader(String instanceDir, String collection,
+      ZooKeeperController zooKeeperController) {
+    super(instanceDir);
+    this.zooKeeperController = zooKeeperController;
+    this.collection = collection;
+  }
+
+  /**
+   * <p>
+   * This loader will delegate to the context classloader when possible,
+   * otherwise it will attempt to resolve resources using any jar files found in
+   * the "lib/" directory in the specified instance directory. If the instance
+   * directory is not specified (=null), SolrResourceLoader#locateInstanceDir
+   * will provide one.
+   * <p>
+   */
+  public ZKSolrResourceLoader(String instanceDir, String collection, ClassLoader parent,
+      Properties coreProperties, ZooKeeperController zooKeeperController) {
+    super(instanceDir, parent, coreProperties);
+    this.zooKeeperController = zooKeeperController;
+    this.collection = collection;
+  }
+
+  /**
+   * Opens any resource by its name. By default, this will look in multiple
+   * locations to load the resource: $configDir/$resource (if resource is not
+   * absolute) $CWD/$resource otherwise, it will look for it in any jar
+   * accessible through the class loader. Override this method to customize
+   * loading resources.
+   * 
+   * @return the stream for the named resource
+   */
+  public InputStream openResource(String resource) {
+    InputStream is = null;
+    String file = getConfigDir() + "/" + resource; //nocommit: getConfigDir no longer makes sense here
+    //nocommit:
+    System.out.println("look for:" + file);
+    try {
+      if (zooKeeperController.exists(file)) {
+        byte[] bytes = zooKeeperController.getFile(getConfigDir(), resource);
+        return new ByteArrayInputStream(bytes);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException("Error opening " + file, e);
+    }
+    try {
+      // delegate to the class loader (looking into $INSTANCE_DIR/lib jars)
+      is = classLoader.getResourceAsStream(resource);
+    } catch (Exception e) {
+      throw new RuntimeException("Error opening " + resource, e);
+    }
+    if (is == null) {
+      throw new RuntimeException("Can't find resource '" + resource
+          + "' in classpath or '" + getConfigDir() + "', cwd="
+          + System.getProperty("user.dir"));
+    }
+    return is;
+  }
+
+  // nocommit: deal with code that uses this call to load the file itself (elevation?)
+  public String getConfigDir() {
+    return "/configs/" + collection;
+  }
+}

Added: lucene/solr/branches/cloud/src/java/org/apache/solr/core/ZooKeeperController.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/cloud/src/java/org/apache/solr/core/ZooKeeperController.java?rev=889376&view=auto
==============================================================================
--- lucene/solr/branches/cloud/src/java/org/apache/solr/core/ZooKeeperController.java (added)
+++ lucene/solr/branches/cloud/src/java/org/apache/solr/core/ZooKeeperController.java Thu Dec 10 19:23:39 2009
@@ -0,0 +1,190 @@
+package org.apache.solr.core;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.List;
+
+import javax.xml.parsers.ParserConfigurationException;
+
+import org.apache.solr.common.SolrException;
+import org.apache.solr.schema.IndexSchema;
+import org.apache.solr.util.ZooPut;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooKeeper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.xml.sax.SAXException;
+
+/**
+ * Handle ZooKeeper interactions.
+ */
+public class ZooKeeperController implements Watcher {
+  private static final String CONFIGS_NODE = "configs";
+
+  private static Logger log = LoggerFactory
+      .getLogger(ZooKeeperController.class);
+
+  private ZooKeeper keeper;
+
+  private String configName;
+
+  private String collectionName;
+
+  /**
+   * @param zookeeperHost ZooKeeper host service
+   * @param zkSolrPathPrefix Solr ZooKeeper node (default is /solr)
+   */
+  public ZooKeeperController(String zookeeperHost, String collection) {
+
+
+    this.collectionName = collection;
+    try {
+      keeper = new ZooKeeper(zookeeperHost, 10000, this);
+      loadConfigPath();
+      register();
+    } catch (IOException e) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+          "Can't create ZooKeeper instance", e);
+    }
+
+  }
+
+  // nocommit: fooling around
+  private void register() throws IOException {
+    try {
+      String host = InetAddress.getLocalHost().getHostName();
+      ZooPut zooPut = new ZooPut(keeper);
+      zooPut.makePath("/hosts/" + host);
+    } catch (UnknownHostException e) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+          "Could not determine IP of host", e);
+    } catch (KeeperException e) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+          "ZooKeeper Exception", e);
+    } catch (InterruptedException e) {
+      // nocommit: handle
+    }
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see
+   * org.apache.zookeeper.Watcher#process(org.apache.zookeeper.WatchedEvent)
+   */
+  public void process(WatchedEvent event) {
+    // nocommit
+    System.out.println("ZooKeeper Event:" + event);
+  }
+
+  private void loadConfigPath() {
+    // nocommit: load all config at once or organize differently
+    try {
+      String path = "/collections/" + collectionName;
+      // nocommit
+      System.out.println("look for collection config:" + path);
+      List<String> children = keeper.getChildren(path, null);
+      for (String node : children) {
+        // nocommit
+        System.out.println("check child:" + node);
+        // nocommit: do we actually want to handle settings in the node name?
+        if (node.startsWith("config=")) {
+          configName = node.substring(node.indexOf("=") + 1);
+          // nocommit
+          System.out.println("config:" + configName);
+        }
+      }
+    } catch (KeeperException e) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+          "ZooKeeper Exception", e);
+    } catch (InterruptedException e) {
+      //nocommit
+    }
+    if (configName == null) {
+      throw new IllegalStateException("no config specified for collection:"
+          + collectionName);
+    }
+  }
+
+  /**
+   * Load IndexSchema from ZooKeeper.
+   * 
+   * @param resourceLoader
+   * @param schemaName
+   * @param config
+   * @return
+   */
+  public IndexSchema getSchema(String schemaName, SolrConfig config,
+      SolrResourceLoader resourceLoader) {
+    byte[] configBytes = getFile("/" + CONFIGS_NODE + "/"
+        + configName, schemaName);
+    InputStream is = new ByteArrayInputStream(configBytes);
+    IndexSchema schema = new IndexSchema(config, schemaName, is);
+    return schema;
+  }
+
+  /**
+   * Load SolrConfig from ZooKeeper.
+   * 
+   * @param resourceLoader
+   * @param solrConfigName
+   * @return
+   * @throws IOException
+   * @throws ParserConfigurationException
+   * @throws SAXException
+   */
+  public SolrConfig getConfig(String solrConfigName,
+      SolrResourceLoader resourceLoader) throws IOException,
+      ParserConfigurationException, SAXException {
+    byte[] config = getFile("/" + CONFIGS_NODE + "/"
+        + configName, solrConfigName);
+    InputStream is = new ByteArrayInputStream(config);
+    SolrConfig cfg = solrConfigName == null ? new SolrConfig(resourceLoader,
+        SolrConfig.DEFAULT_CONF_FILE, is) : new SolrConfig(resourceLoader,
+        solrConfigName, is);
+
+    return cfg;
+  }
+
+  public boolean exists(String path) {
+    Object exists = null;
+    try {
+      exists = keeper.exists(path, null);
+    } catch (KeeperException e) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+          "ZooKeeper Exception", e);
+    } catch (InterruptedException e) {
+      // nocommit: handle
+    }
+    return exists != null;
+  }
+
+  public byte[] getFile(String path, String file) {
+    byte[] bytes = null;
+    String configPath = path + "/" + file;
+    try {
+      log.info("Reading " + file + " from zookeeper at " + configPath);
+      bytes = keeper.getData(configPath, false, null);
+    } catch (KeeperException e) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+          "ZooKeeper Exception", e);
+    } catch (InterruptedException e) {
+      // nocommit: handle
+    }
+
+    return bytes;
+  }
+
+  public void close() {
+    try {
+      keeper.close();
+    } catch (InterruptedException e) {
+      // nocommit: handle
+    }
+  }
+}

Modified: lucene/solr/branches/cloud/src/java/org/apache/solr/handler/component/QueryElevationComponent.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/cloud/src/java/org/apache/solr/handler/component/QueryElevationComponent.java?rev=889376&r1=889375&r2=889376&view=diff
==============================================================================
--- lucene/solr/branches/cloud/src/java/org/apache/solr/handler/component/QueryElevationComponent.java (original)
+++ lucene/solr/branches/cloud/src/java/org/apache/solr/handler/component/QueryElevationComponent.java Thu Dec 10 19:23:39 2009
@@ -169,6 +169,7 @@
               "QueryElevationComponent must specify argument: '"+CONFIG_FILE
               +"' -- path to elevate.xml" );
         }
+        //nocommit: add resourceloader.exists? needs to be handled with zookeeper
         File fC = new File( core.getResourceLoader().getConfigDir(), f );
         File fD = new File( core.getDataDir(), f );
         if( fC.exists() == fD.exists() ) {

Added: lucene/solr/branches/cloud/src/java/org/apache/solr/util/ZooPut.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/cloud/src/java/org/apache/solr/util/ZooPut.java?rev=889376&view=auto
==============================================================================
--- lucene/solr/branches/cloud/src/java/org/apache/solr/util/ZooPut.java (added)
+++ lucene/solr/branches/cloud/src/java/org/apache/solr/util/ZooPut.java Thu Dec 10 19:23:39 2009
@@ -0,0 +1,136 @@
+package org.apache.solr.util;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.ZooKeeper;
+
+/**
+ * Util for uploading and updating files in ZooKeeper.
+ * 
+ */
+public class ZooPut {
+
+  private ZooKeeper keeper;
+
+  private boolean closeKeeper = true;
+
+  public ZooPut(String host) throws IOException {
+    keeper = new ZooKeeper(host, 10000, null);
+  }
+
+  public ZooPut(ZooKeeper keeper) throws IOException {
+    this.closeKeeper = false;
+    this.keeper = keeper;
+  }
+
+  public void close() throws InterruptedException {
+    if (closeKeeper) {
+      keeper.close();
+    }
+  }
+
+  public void makePath(String path) throws KeeperException,
+      InterruptedException {
+    makePath(path, CreateMode.PERSISTENT);
+  }
+
+  public void makePath(String path, CreateMode createMode)
+      throws KeeperException, InterruptedException {
+    // nocommit
+    System.out.println("make:" + path);
+
+    if (path.startsWith("/")) {
+      path = path.substring(1, path.length());
+    }
+    String[] paths = path.split("/");
+    StringBuilder sbPath = new StringBuilder();
+    for (int i = 0; i < paths.length; i++) {
+      String pathPiece = paths[i];
+      sbPath.append("/" + pathPiece);
+      String currentPath = sbPath.toString();
+      Object exists = keeper.exists(currentPath, null);
+      if (exists == null) {
+        CreateMode mode = CreateMode.PERSISTENT;
+        if (i == paths.length - 1) {
+          mode = createMode;
+        }
+        keeper.create(currentPath, null, ZooDefs.Ids.OPEN_ACL_UNSAFE, mode);
+      }
+    }
+  }
+
+  /**
+   * Upload file to ZooKeeper.
+   * 
+   * @param path path to upload file to e.g. /solr/conf/solrconfig.xml
+   * @param file path to file to be uploaded
+   * @throws IOException
+   * @throws KeeperException
+   * @throws InterruptedException
+   */
+  public void putFile(String path, File file) throws IOException,
+      KeeperException, InterruptedException {
+    // nocommit:
+    System.out.println("put:" + path + " " + file);
+
+    makePath(path);
+
+    String fdata = readFileAsString(file);
+
+    Object exists = keeper.exists(path, null);
+    if (exists != null) {
+      keeper.setData(path, fdata.getBytes(), -1);
+    } else {
+      keeper.create(path, fdata.getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE,
+          CreateMode.PERSISTENT);
+    }
+
+  }
+
+  private static String readFileAsString(File file) throws java.io.IOException {
+    byte[] buffer = new byte[(int) file.length()];
+    FileInputStream f = new FileInputStream(file);
+    try {
+      f.read(buffer);
+    } finally {
+      f.close();
+    }
+    return new String(buffer);
+  }
+
+  /**
+   * 
+   * TODO: ACL's ??
+   * 
+   * @param args
+   * @throws Exception
+   */
+  public static void main(String[] args) throws Exception {
+    if (args.length < 2 || args.length > 3) {
+      System.out.println("usage: zoopath solrconfig.xml {host}");
+      return;
+    }
+
+    String path = args[0];
+    String file = args[1];
+
+    String host;
+
+    if (args.length > 2) {
+      host = args[2];
+    } else {
+      // nocommit:
+      host = "localhost:2181";
+    }
+
+    ZooPut zooPut = new ZooPut(host);
+    zooPut.putFile(path, new File(file));
+    zooPut.close();
+  }
+
+}

Added: lucene/solr/branches/cloud/src/test/org/apache/solr/AbstractDistributedZooKeeperTestCase.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/cloud/src/test/org/apache/solr/AbstractDistributedZooKeeperTestCase.java?rev=889376&view=auto
==============================================================================
--- lucene/solr/branches/cloud/src/test/org/apache/solr/AbstractDistributedZooKeeperTestCase.java (added)
+++ lucene/solr/branches/cloud/src/test/org/apache/solr/AbstractDistributedZooKeeperTestCase.java Thu Dec 10 19:23:39 2009
@@ -0,0 +1,77 @@
+package org.apache.solr;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.util.TestHarness;
+import org.apache.zookeeper.server.ServerConfig;
+import org.apache.zookeeper.server.ZooKeeperServerMain;
+
+public abstract class AbstractDistributedZooKeeperTestCase extends BaseDistributedSearchTestCase {
+  protected ZooKeeperServerMain zkServer = new ZooKeeperServerMain();
+
+  protected File tmpDir = new File(System.getProperty("java.io.tmpdir")
+      + System.getProperty("file.separator") + getClass().getName() + "-"
+      + System.currentTimeMillis());
+  
+  @Override
+  public void setUp() throws Exception {
+    // we don't call super.setUp
+    log.info("####SETUP_START " + getName());
+    
+    System.setProperty("zkHost", AbstractZooKeeperTestCase.ZOO_KEEPER_HOST);
+    Thread zooThread = new Thread() {
+      @Override
+      public void run() {
+        ServerConfig config = new ServerConfig() {
+          {
+            this.clientPort = 2181;
+            this.dataDir = tmpDir.getAbsolutePath() + File.separator
+                + "zookeeper/server1/data";
+            this.dataLogDir = this.dataDir;
+          }
+        };
+
+        try {
+          zkServer.runFromConfig(config);
+        } catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+
+      }
+    };
+    zooThread.setDaemon(true);
+    zooThread.start();
+    Thread.sleep(500); // pause for ZooKeeper to start
+    AbstractZooKeeperTestCase.buildZooKeeper(getSolrConfigFile(), getSchemaFile());
+
+    dataDir = tmpDir;
+    dataDir.mkdirs();
+
+    // set some system properties for use by tests
+    System.setProperty("solr.test.sys.prop1", "propone");
+    System.setProperty("solr.test.sys.prop2", "proptwo");
+
+    CoreContainer.Initializer init = new CoreContainer.Initializer() {
+      {
+        this.dataDir = super.dataDir;
+      }
+    };
+
+    h = new TestHarness("", init);
+    lrf = h.getRequestFactory("standard", 0, 20, "version", "2.2");
+
+    log.info("####SETUP_END " + getName());
+
+    testDir = new File(System.getProperty("java.io.tmpdir")
+            + System.getProperty("file.separator")
+            + getClass().getName() + "-" + System.currentTimeMillis());
+    testDir.mkdirs();
+    postSetUp();
+  }
+
+  public void tearDown() throws Exception {
+    super.tearDown();
+  }
+}

Added: lucene/solr/branches/cloud/src/test/org/apache/solr/AbstractZooKeeperTestCase.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/cloud/src/test/org/apache/solr/AbstractZooKeeperTestCase.java?rev=889376&view=auto
==============================================================================
--- lucene/solr/branches/cloud/src/test/org/apache/solr/AbstractZooKeeperTestCase.java (added)
+++ lucene/solr/branches/cloud/src/test/org/apache/solr/AbstractZooKeeperTestCase.java Thu Dec 10 19:23:39 2009
@@ -0,0 +1,113 @@
+package org.apache.solr;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.util.AbstractSolrTestCase;
+import org.apache.solr.util.TestHarness;
+import org.apache.solr.util.ZooPut;
+import org.apache.zookeeper.server.ServerConfig;
+import org.apache.zookeeper.server.ZooKeeperServerMain;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Base test class for ZooKeeper tests.
+ */
+public abstract class AbstractZooKeeperTestCase extends AbstractSolrTestCase {
+  public static final String ZOO_KEEPER_HOST = "localhost:2181/solr";
+  protected static Logger log = LoggerFactory.getLogger(AbstractZooKeeperTestCase.class);
+  protected ZooKeeperServerMain zkServer = new ZooKeeperServerMain();
+
+  protected File tmpDir = new File(System.getProperty("java.io.tmpdir")
+      + System.getProperty("file.separator") + getClass().getName() + "-"
+      + System.currentTimeMillis());
+
+  public AbstractZooKeeperTestCase() {
+
+  }
+
+  public String getSchemaFile() {
+    return "schema.xml";
+  }
+
+  public String getSolrConfigFile() {
+    return "solrconfig.xml";
+  }
+
+  public void setUp() throws Exception {
+    // we don't call super.setUp
+    System.setProperty("zkHost", ZOO_KEEPER_HOST);
+    Thread zooThread = new Thread() {
+      @Override
+      public void run() {
+        ServerConfig config = new ServerConfig() {
+          {
+            this.clientPort = 2181;
+            this.dataDir = tmpDir.getAbsolutePath() + File.separator
+                + "zookeeper/server1/data";
+            this.dataLogDir = this.dataDir;
+          }
+        };
+
+        try {
+          zkServer.runFromConfig(config);
+        } catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+
+      }
+    };
+    zooThread.setDaemon(true);
+    zooThread.start();
+    Thread.sleep(500); // pause for ZooKeeper to start
+    buildZooKeeper(getSolrConfigFile(), getSchemaFile());
+
+    log.info("####SETUP_START " + getName());
+    dataDir = tmpDir;
+    dataDir.mkdirs();
+
+    // set some system properties for use by tests
+    System.setProperty("solr.test.sys.prop1", "propone");
+    System.setProperty("solr.test.sys.prop2", "proptwo");
+
+    CoreContainer.Initializer init = new CoreContainer.Initializer() {
+      {
+        this.dataDir = AbstractZooKeeperTestCase.this.dataDir.getAbsolutePath();
+      }
+    };
+
+    h = new TestHarness("", init);
+    lrf = h.getRequestFactory("standard", 0, 20, "version", "2.2");
+
+    log.info("####SETUP_END " + getName());
+  }
+
+  public static void buildZooKeeper(String config, String schema) throws Exception {
+    ZooPut zooPut = new ZooPut(ZOO_KEEPER_HOST.substring(0, ZOO_KEEPER_HOST.indexOf('/')));
+    zooPut.makePath("/solr");
+    zooPut.close();
+    
+    zooPut = new ZooPut(ZOO_KEEPER_HOST);
+    
+    zooPut.makePath("/collections/collection1/config=collection1");
+    
+    putConfig(zooPut, config);
+    putConfig(zooPut, schema);
+    putConfig(zooPut, "stopwords.txt");
+    putConfig(zooPut, "protwords.txt");
+    putConfig(zooPut, "mapping-ISOLatin1Accent.txt");
+    putConfig(zooPut, "old_synonyms.txt");
+    zooPut.close();
+  }
+  
+  private static void putConfig(ZooPut zooPut, String name) throws Exception {
+    zooPut.putFile("/configs/collection1/" + name, new File("solr" + File.separator
+        + "conf" + File.separator + name));
+  }
+
+  public void tearDown() throws Exception {
+    super.tearDown();
+  }
+}

Added: lucene/solr/branches/cloud/src/test/org/apache/solr/BasicDistributedZooKeeperTest.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/cloud/src/test/org/apache/solr/BasicDistributedZooKeeperTest.java?rev=889376&view=auto
==============================================================================
--- lucene/solr/branches/cloud/src/test/org/apache/solr/BasicDistributedZooKeeperTest.java (added)
+++ lucene/solr/branches/cloud/src/test/org/apache/solr/BasicDistributedZooKeeperTest.java Thu Dec 10 19:23:39 2009
@@ -0,0 +1,188 @@
+package org.apache.solr;
+
+import org.apache.solr.client.solrj.SolrServerException;
+
+/**
+ * nocommit: 
+ *
+ */
+public class BasicDistributedZooKeeperTest extends AbstractDistributedZooKeeperTestCase {
+  @Override
+  public String getSchemaFile() {
+    return "schema.xml";
+  }
+
+  @Override
+  public String getSolrConfigFile() {
+    return "solrconfig.xml";
+  }
+  
+  
+  String t1="a_t";
+  String i1="a_si";
+  String nint = "n_i";
+  String tint = "n_ti";
+  String nfloat = "n_f";
+  String tfloat = "n_tf";
+  String ndouble = "n_d";
+  String tdouble = "n_td";
+  String nlong = "n_l";
+  String tlong = "n_tl";
+  String ndate = "n_dt";
+  String tdate = "n_tdt";
+  
+  String oddField="oddField_s";
+  String missingField="missing_but_valid_field_t";
+  String invalidField="invalid_field_not_in_schema";
+
+  @Override
+  public void doTest() throws Exception {
+    del("*:*");
+    indexr(id,1, i1, 100, tlong, 100,t1,"now is the time for all good men"
+            ,"foo_f", 1.414f, "foo_b", "true", "foo_d", 1.414d);
+    indexr(id,2, i1, 50 , tlong, 50,t1,"to come to the aid of their country."
+    );
+    indexr(id,3, i1, 2, tlong, 2,t1,"how now brown cow"
+    );
+    indexr(id,4, i1, -100 ,tlong, 101,t1,"the quick fox jumped over the lazy dog"
+    );
+    indexr(id,5, i1, 500, tlong, 500 ,t1,"the quick fox jumped way over the lazy dog"
+    );
+    indexr(id,6, i1, -600, tlong, 600 ,t1,"humpty dumpy sat on a wall");
+    indexr(id,7, i1, 123, tlong, 123 ,t1,"humpty dumpy had a great fall");
+    indexr(id,8, i1, 876, tlong, 876,t1,"all the kings horses and all the kings men");
+    indexr(id,9, i1, 7, tlong, 7,t1,"couldn't put humpty together again");
+    indexr(id,10, i1, 4321, tlong, 4321,t1,"this too shall pass");
+    indexr(id,11, i1, -987, tlong, 987,t1,"An eye for eye only ends up making the whole world blind.");
+    indexr(id,12, i1, 379, tlong, 379,t1,"Great works are performed, not by strength, but by perseverance.");
+    indexr(id,13, i1, 232, tlong, 232,t1,"no eggs on wall, lesson learned", oddField, "odd man out");
+
+    indexr(id, 14, "SubjectTerms_mfacet", new String[]  {"mathematical models", "mathematical analysis"});
+    indexr(id, 15, "SubjectTerms_mfacet", new String[]  {"test 1", "test 2", "test3"});
+    indexr(id, 16, "SubjectTerms_mfacet", new String[]  {"test 1", "test 2", "test3"});
+    String[] vals = new String[100];
+    for (int i=0; i<100; i++) {
+      vals[i] = "test " + i;
+    }
+    indexr(id, 17, "SubjectTerms_mfacet", vals);
+
+    for (int i=100; i<150; i++) {
+      indexr(id, i);      
+    }
+
+    commit();
+
+    handle.clear();
+    handle.put("QTime", SKIPVAL);
+    handle.put("timestamp", SKIPVAL);
+
+    // random value sort
+    for (String f : fieldNames) {
+      query("q","*:*", "sort",f+" desc");
+      query("q","*:*", "sort",f+" asc");
+    }
+
+
+    // these queries should be exactly ordered and scores should exactly match
+    query("q","*:*", "sort",i1+" desc");
+    query("q","*:*", "sort",i1+" asc");
+    query("q","*:*", "sort",i1+" desc", "fl","*,score");
+    query("q","*:*", "sort",tlong+" asc", "fl","score");  // test legacy behavior - "score"=="*,score"
+    query("q","*:*", "sort",tlong+" desc");
+    handle.put("maxScore", SKIPVAL);
+    query("q","{!func}"+i1);// does not expect maxScore. So if it comes ,ignore it. JavaBinCodec.writeSolrDocumentList()
+    //is agnostic of request params.
+    handle.remove("maxScore");
+    query("q","{!func}"+i1, "fl","*,score");  // even scores should match exactly here
+
+    handle.put("highlighting", UNORDERED);
+    handle.put("response", UNORDERED);
+
+    handle.put("maxScore", SKIPVAL);
+    query("q","quick");
+    query("q","all","fl","id","start","0");
+    query("q","all","fl","foofoofoo","start","0");  // no fields in returned docs
+    query("q","all","fl","id","start","100");
+
+    handle.put("score", SKIPVAL);
+    query("q","quick","fl","*,score");
+    query("q","all","fl","*,score","start","1");
+    query("q","all","fl","*,score","start","100");
+
+    query("q","now their fox sat had put","fl","*,score",
+            "hl","true","hl.fl",t1);
+
+    query("q","now their fox sat had put","fl","foofoofoo",
+            "hl","true","hl.fl",t1);
+
+
+    handle.put("debug", UNORDERED);
+    handle.put("time", SKIPVAL);
+
+    query("q","now their fox sat had put","fl","*,score",
+            "debugQuery", "true");
+
+    // TODO: This test currently fails because debug info is obtained only
+    // on shards with matches.
+    /***
+    query("q","matchesnothing","fl","*,score",
+            "debugQuery", "true");    
+    ***/
+    query("q","matchesnothing","fl","*,score");  
+
+
+    query("q","*:*", "rows",100, "facet","true", "facet.field",t1);
+    query("q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","count");
+    query("q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","count", "facet.mincount",2);
+    query("q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","index");
+    query("q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","index", "facet.mincount",2);
+    query("q","*:*", "rows",100, "facet","true", "facet.field",t1,"facet.limit",1);
+    query("q","*:*", "rows",100, "facet","true", "facet.query","quick", "facet.query","all", "facet.query","*:*");
+    query("q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.offset",1);
+    query("q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.mincount",2);
+
+    // test faceting multiple things at once
+    query("q","*:*", "rows",100, "facet","true", "facet.query","quick", "facet.query","all", "facet.query","*:*"
+    ,"facet.field",t1);
+
+    // test filter tagging, facet exclusion, and naming (multi-select facet support)
+    query("q","*:*", "rows",100, "facet","true", "facet.query","{!key=myquick}quick", "facet.query","{!key=myall ex=a}all", "facet.query","*:*"
+    ,"facet.field","{!key=mykey ex=a}"+t1
+    ,"facet.field","{!key=other ex=b}"+t1
+    ,"facet.field","{!key=again ex=a,b}"+t1
+    ,"facet.field",t1
+    ,"fq","{!tag=a}id:[1 TO 7]", "fq","{!tag=b}id:[3 TO 9]"
+    );
+    query("q", "*:*", "facet", "true", "facet.field", "{!ex=t1}SubjectTerms_mfacet", "fq", "{!tag=t1}SubjectTerms_mfacet:(test 1)", "facet.limit", "10", "facet.mincount", "1");
+
+    // test field that is valid in schema but missing in all shards
+    query("q","*:*", "rows",100, "facet","true", "facet.field",missingField, "facet.mincount",2);
+    // test field that is valid in schema and missing in some shards
+    query("q","*:*", "rows",100, "facet","true", "facet.field",oddField, "facet.mincount",2);
+
+    query("q","*:*", "sort",i1+" desc", "stats", "true", "stats.field", i1);
+
+    try {
+      // test error produced for field that is invalid for schema
+      query("q","*:*", "rows",100, "facet","true", "facet.field",invalidField, "facet.mincount",2);
+      fail("SolrServerException expected for invalid field that is not in schema");
+    } catch (SolrServerException ex) {
+      // expected
+    }
+
+    // index the same document to two servers and make sure things
+    // don't blow up.
+    if (clients.size()>=2) {
+      index(id,100, i1, 107 ,t1,"oh no, a duplicate!");
+      for (int i=0; i<clients.size(); i++) {
+        index_specific(i, id,100, i1, 107 ,t1,"oh no, a duplicate!");
+      }
+      commit();
+      query("q","duplicate", "hl","true", "hl.fl", t1);
+      query("q","fox duplicate horses", "hl","true", "hl.fl", t1);
+      query("q","*:*", "rows",100);
+    }
+
+    // Thread.sleep(10000000000L);
+  }
+}

Added: lucene/solr/branches/cloud/src/test/org/apache/solr/BasicZooKeeperTest.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/cloud/src/test/org/apache/solr/BasicZooKeeperTest.java?rev=889376&view=auto
==============================================================================
--- lucene/solr/branches/cloud/src/test/org/apache/solr/BasicZooKeeperTest.java (added)
+++ lucene/solr/branches/cloud/src/test/org/apache/solr/BasicZooKeeperTest.java Thu Dec 10 19:23:39 2009
@@ -0,0 +1,86 @@
+package org.apache.solr;
+
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.update.SolrIndexWriter;
+
+/**
+ * TODO: assert config came from ZooKeeper
+ *
+ */
+public class BasicZooKeeperTest extends AbstractZooKeeperTestCase {
+  public void testBasic() throws Exception {
+    // test using ZooKeeper
+    assertTrue("Not using ZooKeeper", h.getCoreContainer().isZooKeeperAware());
+    
+    // test merge factor picked up
+    SolrCore core = h.getCore();
+    SolrIndexWriter writer = new SolrIndexWriter("testWriter", core
+        .getNewIndexDir(), core.getDirectoryFactory(), false, core.getSchema(),
+        core.getSolrConfig().mainIndexConfig, core.getDeletionPolicy());
+    assertEquals("Mergefactor was not picked up", writer.getMergeFactor(), 8);
+
+    lrf.args.put("version", "2.0");
+    assertQ("test query on empty index", req("qlkciyopsbgzyvkylsjhchghjrdf"),
+        "//result[@numFound='0']");
+
+    // test escaping of ";"
+    assertU("deleting 42 for no reason at all", delI("42"));
+    assertU("adding doc#42", adoc("id", "42", "val_s", "aa;bb"));
+    assertU("does commit work?", commit());
+
+    assertQ("backslash escaping semicolon", req("id:42 AND val_s:aa\\;bb"),
+        "//*[@numFound='1']", "//int[@name='id'][.='42']");
+
+    assertQ("quote escaping semicolon", req("id:42 AND val_s:\"aa;bb\""),
+        "//*[@numFound='1']", "//int[@name='id'][.='42']");
+
+    assertQ("no escaping semicolon", req("id:42 AND val_s:aa"),
+        "//*[@numFound='0']");
+
+    assertU(delI("42"));
+    assertU(commit());
+    assertQ(req("id:42"), "//*[@numFound='0']");
+
+    // test allowDups default of false
+
+    assertU(adoc("id", "42", "val_s", "AAA"));
+    assertU(adoc("id", "42", "val_s", "BBB"));
+    assertU(commit());
+    assertQ(req("id:42"), "//*[@numFound='1']", "//str[.='BBB']");
+    assertU(adoc("id", "42", "val_s", "CCC"));
+    assertU(adoc("id", "42", "val_s", "DDD"));
+    assertU(commit());
+    assertQ(req("id:42"), "//*[@numFound='1']", "//str[.='DDD']");
+
+    // test deletes
+    String[] adds = new String[] { add(doc("id", "101"), "allowDups", "false"),
+        add(doc("id", "101"), "allowDups", "false"),
+        add(doc("id", "105"), "allowDups", "true"),
+        add(doc("id", "102"), "allowDups", "false"),
+        add(doc("id", "103"), "allowDups", "true"),
+        add(doc("id", "101"), "allowDups", "false"), };
+    for (String a : adds) {
+      assertU(a, a);
+    }
+    assertU(commit());
+
+    // test maxint
+    assertQ(req("q", "id:[100 TO 110]", "rows", "2147483647"),
+        "//*[@numFound='4']");
+
+    // test big limit
+    assertQ(req("q", "id:[100 TO 111]", "rows", "1147483647"),
+        "//*[@numFound='4']");
+
+    assertQ(req("id:[100 TO 110]"), "//*[@numFound='4']");
+    assertU(delI("102"));
+    assertU(commit());
+    assertQ(req("id:[100 TO 110]"), "//*[@numFound='3']");
+    assertU(delI("105"));
+    assertU(commit());
+    assertQ(req("id:[100 TO 110]"), "//*[@numFound='2']");
+    assertU(delQ("id:[100 TO 110]"));
+    assertU(commit());
+    assertQ(req("id:[100 TO 110]"), "//*[@numFound='0']");
+  }
+}