You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by cm...@apache.org on 2013/08/11 14:19:39 UTC

svn commit: r1512909 [26/38] - in /lucene/dev/branches/lucene4956: ./ dev-tools/ dev-tools/eclipse/ dev-tools/idea/.idea/libraries/ dev-tools/idea/lucene/suggest/ dev-tools/idea/solr/contrib/dataimporthandler/ dev-tools/idea/solr/core/src/test/ dev-too...

Modified: lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/core/SolrCore.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/core/SolrCore.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/core/SolrCore.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/core/SolrCore.java Sun Aug 11 12:19:13 2013
@@ -17,39 +17,6 @@
 
 package org.apache.solr.core;
 
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.Writer;
-import java.lang.reflect.Constructor;
-import java.net.URL;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.IdentityHashMap;
-import java.util.LinkedHashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-import java.util.StringTokenizer;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.locks.ReentrantLock;
-
-import javax.xml.parsers.ParserConfigurationException;
-
 import org.apache.commons.io.IOUtils;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.index.DirectoryReader;
@@ -62,6 +29,7 @@ import org.apache.lucene.store.IndexInpu
 import org.apache.lucene.store.LockObtainFailedException;
 import org.apache.solr.cloud.CloudDescriptor;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.CommonParams.EchoParamStyle;
 import org.apache.solr.common.params.SolrParams;
@@ -124,6 +92,39 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.xml.sax.SAXException;
 
+import javax.xml.parsers.ParserConfigurationException;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.Writer;
+import java.lang.reflect.Constructor;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.IdentityHashMap;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.StringTokenizer;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.ReentrantLock;
+
 
 /**
  *
@@ -141,7 +142,7 @@ public final class SolrCore implements S
 
   private String name;
   private String logid; // used to show what name is set
-  private final CoreDescriptor coreDescriptor;
+  private CoreDescriptor coreDescriptor;
 
   private boolean isReloaded = false;
 
@@ -252,7 +253,7 @@ public final class SolrCore implements S
   
         final InputStream is = new PropertiesInputStream(input);
         try {
-          p.load(is);
+          p.load(new InputStreamReader(is, "UTF-8"));
           
           String s = p.getProperty("index");
           if (s != null && s.trim().length() > 0) {
@@ -301,6 +302,7 @@ public final class SolrCore implements S
   public void setName(String v) {
     this.name = v;
     this.logid = (v==null)?"":("["+v+"] ");
+    this.coreDescriptor = new CoreDescriptor(v, this.coreDescriptor);
   }
 
   public String getLogId()
@@ -521,13 +523,13 @@ public final class SolrCore implements S
     if (msg == null) msg = "SolrCore Object";
     try {
         clazz = getResourceLoader().findClass(className, cast);
-      //most of the classes do not have constructors which takes SolrCore argument. It is recommended to obtain SolrCore by implementing SolrCoreAware.
-      // So invariably always it will cause a  NoSuchMethodException. So iterate though the list of available constructors
-        Constructor[] cons =  clazz.getConstructors();
-        for (Constructor con : cons) {
-          Class[] types = con.getParameterTypes();
+        //most of the classes do not have constructors which takes SolrCore argument. It is recommended to obtain SolrCore by implementing SolrCoreAware.
+        // So invariably always it will cause a  NoSuchMethodException. So iterate though the list of available constructors
+        Constructor<?>[] cons =  clazz.getConstructors();
+        for (Constructor<?> con : cons) {
+          Class<?>[] types = con.getParameterTypes();
           if(types.length == 1 && types[0] == SolrCore.class){
-            return (T)con.newInstance(this);
+            return cast.cast(con.newInstance(this));
           }
         }
         return getResourceLoader().newInstance(className, cast);//use the empty constructor
@@ -550,14 +552,13 @@ public final class SolrCore implements S
     if (msg == null) msg = "SolrCore Object";
     try {
         clazz = getResourceLoader().findClass(className, UpdateHandler.class);
-      //most of the classes do not have constructors which takes SolrCore argument. It is recommended to obtain SolrCore by implementing SolrCoreAware.
-      // So invariably always it will cause a  NoSuchMethodException. So iterate though the list of available constructors
-        Constructor justSolrCoreCon = null;
-        Constructor[] cons =  clazz.getConstructors();
-        for (Constructor con : cons) {
-          Class[] types = con.getParameterTypes();
+        //most of the classes do not have constructors which takes SolrCore argument. It is recommended to obtain SolrCore by implementing SolrCoreAware.
+        // So invariably always it will cause a  NoSuchMethodException. So iterate though the list of available constructors
+        Constructor<?>[] cons =  clazz.getConstructors();
+        for (Constructor<?> con : cons) {
+          Class<?>[] types = con.getParameterTypes();
           if(types.length == 2 && types[0] == SolrCore.class && types[1] == UpdateHandler.class){
-            return (UpdateHandler) con.newInstance(this, updateHandler);
+            return UpdateHandler.class.cast(con.newInstance(this, updateHandler));
           } 
         }
         throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,"Error Instantiating "+msg+", "+className+ " could not find proper constructor for " + UpdateHandler.class.getName());
@@ -669,11 +670,10 @@ public final class SolrCore implements S
     if (dataDir == null) {
       if (cd.usingDefaultDataDir()) dataDir = config.getDataDir();
       if (dataDir == null) {
-        dataDir = cd.getDataDir();
         try {
+          dataDir = cd.getDataDir();
           if (!directoryFactory.isAbsolute(dataDir)) {
-            dataDir = directoryFactory.normalize(SolrResourceLoader
-                .normalizeDir(cd.getInstanceDir()) + dataDir);
+            dataDir = directoryFactory.getDataHome(cd);
           }
         } catch (IOException e) {
           throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, null, e);
@@ -695,7 +695,7 @@ public final class SolrCore implements S
       // mode as well, and can't assert version field support on init.
 
       try {
-        Object ignored = VersionInfo.getAndCheckVersionField(schema);
+        VersionInfo.getAndCheckVersionField(schema);
       } catch (SolrException e) {
         throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
                                 "Schema will not work with SolrCloud mode: " +
@@ -744,7 +744,6 @@ public final class SolrCore implements S
       this.codec = initCodec(solrConfig, schema);
       
       if (updateHandler == null) {
-        initDirectoryFactory();
         solrCoreState = new DefaultSolrCoreState(getDirectoryFactory());
       } else {
         solrCoreState = updateHandler.getSolrCoreState();
@@ -773,9 +772,9 @@ public final class SolrCore implements S
       // cause the executor to stall so firstSearcher events won't fire
       // until after inform() has been called for all components.
       // searchExecutor must be single-threaded for this to work
-      searcherExecutor.submit(new Callable() {
+      searcherExecutor.submit(new Callable<Void>() {
         @Override
-        public Object call() throws Exception {
+        public Void call() throws Exception {
           latch.await();
           return null;
         }
@@ -845,11 +844,21 @@ public final class SolrCore implements S
     // from the core.
     resourceLoader.inform(infoRegistry);
     
+    CoreContainer cc = cd.getCoreContainer();
+
+    if (cc != null && cc.isZooKeeperAware() && Slice.CONSTRUCTION.equals(cd.getCloudDescriptor().getShardState())) {
+      // set update log to buffer before publishing the core
+      getUpdateHandler().getUpdateLog().bufferUpdates();
+      
+      cd.getCloudDescriptor().setShardState(null);
+      cd.getCloudDescriptor().setShardRange(null);
+      
+    }
     // For debugging   
 //    numOpens.incrementAndGet();
 //    openHandles.put(this, new RuntimeException("unclosed core - name:" + getName() + " refs: " + refCount.get()));
   }
-
+    
   private Codec initCodec(SolrConfig solrConfig, final IndexSchema schema) {
     final PluginInfo info = solrConfig.getPluginInfo(CodecFactory.class.getName());
     final CodecFactory factory;
@@ -1120,7 +1129,7 @@ public final class SolrCore implements S
   /**
    * Returns an unmodifiable Map containing the registered handlers of the specified type.
    */
-  public Map<String,SolrRequestHandler> getRequestHandlers(Class clazz) {
+  public <T extends SolrRequestHandler> Map<String,T> getRequestHandlers(Class<T> clazz) {
     return reqHandlers.getAll(clazz);
   }
   
@@ -1345,6 +1354,10 @@ public final class SolrCore implements S
    * This method acquires openSearcherLock - do not call with searckLock held!
    */
   public RefCounted<SolrIndexSearcher>  openNewSearcher(boolean updateHandlerReopens, boolean realtime) {
+    if (isClosed()) { // catch some errors quicker
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "openNewSearcher called on closed core");
+    }
+
     SolrIndexSearcher tmp;
     RefCounted<SolrIndexSearcher> newestSearcher = null;
     boolean nrt = solrConfig.reopenReaders && updateHandlerReopens;
@@ -2019,6 +2032,7 @@ public final class SolrCore implements S
          QParserPlugin plugin = clazz.newInstance();
          qParserPlugins.put(name, plugin);
          plugin.init(null);
+         infoRegistry.put(name, plugin);
        }
      } catch (Exception e) {
        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);

Modified: lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/core/SolrCores.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/core/SolrCores.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/core/SolrCores.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/core/SolrCores.java Sun Aug 11 12:19:13 2013
@@ -17,31 +17,24 @@ package org.apache.solr.core;
  * limitations under the License.
  */
 
-import java.io.File;
+import com.google.common.collect.Lists;
+import org.apache.solr.common.SolrException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Properties;
 import java.util.Set;
 import java.util.TreeSet;
 import java.util.concurrent.ConcurrentHashMap;
 
-import javax.xml.xpath.XPathExpressionException;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.solr.cloud.CloudDescriptor;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.core.SolrXMLSerializer.SolrCoreXMLDef;
-import org.apache.solr.util.DOMUtil;
-import org.w3c.dom.Node;
-
 
 class SolrCores {
-  private static SolrXMLSerializer SOLR_XML_SERIALIZER = new SolrXMLSerializer();
+
   private static Object modifyLock = new Object(); // for locking around manipulating any of the core maps.
   private final Map<String, SolrCore> cores = new LinkedHashMap<String, SolrCore>(); // For "permanent" cores
 
@@ -56,6 +49,8 @@ class SolrCores {
 
   private final CoreContainer container;
 
+  private static final Logger logger = LoggerFactory.getLogger(SolrCores.class);
+
   // This map will hold objects that are being currently operated on. The core (value) may be null in the case of
   // initial load. The rule is, never to any operation on a core that is currently being operated upon.
   private static final Set<String> pendingCoreOps = new HashSet<String>();
@@ -70,16 +65,17 @@ class SolrCores {
 
   // Trivial helper method for load, note it implements LRU on transient cores. Also note, if
   // there is no setting for max size, nothing is done and all cores go in the regular "cores" list
-  protected void allocateLazyCores(final ConfigSolr cfg, final SolrResourceLoader loader) {
-    final int transientCacheSize = cfg.getInt(ConfigSolr.CfgProp.SOLR_TRANSIENTCACHESIZE, Integer.MAX_VALUE);
-    if (transientCacheSize != Integer.MAX_VALUE) {
-      CoreContainer.log.info("Allocating transient cache for {} transient cores", transientCacheSize);
-      transientCores = new LinkedHashMap<String, SolrCore>(transientCacheSize, 0.75f, true) {
+  protected void allocateLazyCores(final int cacheSize, final SolrResourceLoader loader) {
+    if (cacheSize != Integer.MAX_VALUE) {
+      CoreContainer.log.info("Allocating transient cache for {} transient cores", cacheSize);
+      transientCores = new LinkedHashMap<String, SolrCore>(cacheSize, 0.75f, true) {
         @Override
         protected boolean removeEldestEntry(Map.Entry<String, SolrCore> eldest) {
-          if (size() > transientCacheSize) {
+          if (size() > cacheSize) {
             synchronized (modifyLock) {
-              pendingCloses.add(eldest.getValue()); // Essentially just queue this core up for closing.
+              SolrCore coreToClose = eldest.getValue();
+              logger.info("Closing transient core [{}]", coreToClose.getName());
+              pendingCloses.add(coreToClose); // Essentially just queue this core up for closing.
               modifyLock.notifyAll(); // Wakes up closer thread too
             }
             return true;
@@ -99,70 +95,34 @@ class SolrCores {
   // We are shutting down. You can't hold the lock on the various lists of cores while they shut down, so we need to
   // make a temporary copy of the names and shut them down outside the lock.
   protected void close() {
-    List<String> coreNames;
-    List<String> transientNames;
-    List<SolrCore> pendingToClose;
+    Collection<SolrCore> coreList = new ArrayList<SolrCore>();
 
     // It might be possible for one of the cores to move from one list to another while we're closing them. So
     // loop through the lists until they're all empty. In particular, the core could have moved from the transient
     // list to the pendingCloses list.
 
-    while (true) {
+    do {
+      coreList.clear();
       synchronized (modifyLock) {
-        coreNames = new ArrayList<String>(cores.keySet());
-        transientNames = new ArrayList<String>(transientCores.keySet());
-        pendingToClose = new ArrayList<SolrCore>(pendingCloses);
-      }
+        // make a copy of the cores then clear the map so the core isn't handed out to a request again
+        coreList.addAll(cores.values());
+        cores.clear();
 
-      if (coreNames.size() == 0 && transientNames.size() == 0 && pendingToClose.size() == 0) break;
-
-      for (String coreName : coreNames) {
-        SolrCore core = cores.get(coreName);
-        if (core == null) {
-          CoreContainer.log.info("Core " + coreName + " moved from core container list before closing.");
-        } else {
-          try {
-            core.close();
-          } catch (Throwable t) {
-            SolrException.log(CoreContainer.log, "Error shutting down core", t);
-          } finally {
-            synchronized (modifyLock) {
-              cores.remove(coreName);
-            }
-          }
-        }
-      }
+        coreList.addAll(transientCores.values());
+        transientCores.clear();
 
-      for (String coreName : transientNames) {
-        SolrCore core = transientCores.get(coreName);
-        if (core == null) {
-          CoreContainer.log.info("Core " + coreName + " moved from transient core container list before closing.");
-        } else {
-          try {
-            core.close();
-          } catch (Throwable t) {
-            SolrException.log(CoreContainer.log, "Error shutting down core", t);
-          } finally {
-            synchronized (modifyLock) {
-              transientCores.remove(coreName);
-            }
-          }
-        }
+        coreList.addAll(pendingCloses);
+        pendingCloses.clear();
       }
 
-      // We might have some cores that we were _thinking_ about shutting down, so take care of those too.
-      for (SolrCore core : pendingToClose) {
+      for (SolrCore core : coreList) {
         try {
           core.close();
         } catch (Throwable t) {
           SolrException.log(CoreContainer.log, "Error shutting down core", t);
-        } finally {
-          synchronized (modifyLock) {
-            pendingCloses.remove(core);
-          }
         }
       }
-    }
+    } while (coreList.size() > 0);
   }
 
   //WARNING! This should be the _only_ place you put anything into the list of transient cores!
@@ -246,17 +206,23 @@ class SolrCores {
     synchronized (modifyLock) {
       SolrCore c0 = cores.get(n0);
       SolrCore c1 = cores.get(n1);
-      if (c0 == null)
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No such core: " + n0);
-      if (c1 == null)
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No such core: " + n1);
+      if (c0 == null) { // Might be an unloaded transient core
+        c0 = container.getCore(n0);
+        if (c0 == null) {
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No such core: " + n0);
+        }
+      }
+      if (c1 == null) { // Might be an unloaded transient core
+        c1 = container.getCore(n1);
+        if (c1 == null) {
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No such core: " + n1);
+        }
+      }
       cores.put(n0, c1);
       cores.put(n1, c0);
 
       c0.setName(n1);
-      c0.getCoreDescriptor().putProperty(CoreDescriptor.CORE_NAME, n1);
       c1.setName(n0);
-      c1.getCoreDescriptor().putProperty(CoreDescriptor.CORE_NAME, n0);
     }
 
   }
@@ -299,20 +265,20 @@ class SolrCores {
     }
   }
 
-  protected SolrCore getCoreFromAnyList(String name) {
-    SolrCore core;
-
+  /* If you don't increment the reference count, someone could close the core before you use it. */
+  protected SolrCore  getCoreFromAnyList(String name, boolean incRefCount) {
     synchronized (modifyLock) {
-      core = cores.get(name);
-      if (core != null) {
-        return core;
+      SolrCore core = cores.get(name);
+
+      if (core == null) {
+        core = transientCores.get(name);
       }
 
-      if (dynamicDescriptors.size() == 0) {
-        return null; // Nobody even tried to define any transient cores, so we're done.
+      if (core != null && incRefCount) {
+        core.open();
       }
-      // Now look for already loaded transient cores.
-      return transientCores.get(name);
+
+      return core;
     }
   }
 
@@ -341,7 +307,7 @@ class SolrCores {
       if (desc == null) {
         return null;
       }
-      return new CoreDescriptor(desc);
+      return new CoreDescriptor(cname, desc);
     }
 
   }
@@ -351,43 +317,7 @@ class SolrCores {
       return coreToOrigName.get(solrCore);
     }
   }
-  
-  public void persistCores(Config cfg, Properties containerProperties,
-      Map<String,String> rootSolrAttribs, Map<String,String> coresAttribs,
-      File file, File configFile, SolrResourceLoader loader) throws XPathExpressionException {
-
-    
-    List<SolrXMLSerializer.SolrCoreXMLDef> solrCoreXMLDefs = new ArrayList<SolrXMLSerializer.SolrCoreXMLDef>();
-    synchronized (modifyLock) {
-      
-      persistCores(cfg, cores, loader, solrCoreXMLDefs);
-      persistCores(cfg, transientCores, loader, solrCoreXMLDefs);
-      // add back all the cores that aren't loaded, either in cores or transient
-      // cores
-      for (Map.Entry<String,CoreDescriptor> ent : dynamicDescriptors.entrySet()) {
-        if (!cores.containsKey(ent.getKey())
-            && !transientCores.containsKey(ent.getKey())) {
-          addCoreToPersistList(cfg, loader, ent.getValue(), null, solrCoreXMLDefs);
-        }
-      }
-      for (Map.Entry<String,SolrCore> ent : createdCores.entrySet()) {
-        if (!cores.containsKey(ent.getKey())
-            && !transientCores.containsKey(ent.getKey())
-            && !dynamicDescriptors.containsKey(ent.getKey())) {
-          addCoreToPersistList(cfg, loader, ent.getValue().getCoreDescriptor(),
-              null, solrCoreXMLDefs);
-        }
-      }
 
-      SolrXMLSerializer.SolrXMLDef solrXMLDef = new SolrXMLSerializer.SolrXMLDef();
-      solrXMLDef.coresDefs = solrCoreXMLDefs;
-      solrXMLDef.containerProperties = containerProperties;
-      solrXMLDef.solrAttribs = rootSolrAttribs;
-      solrXMLDef.coresAttribs = coresAttribs;
-      SOLR_XML_SERIALIZER.persistFile(file, solrXMLDef);
-    }
-    
-  }
   // Wait here until any pending operations (load, unload or reload) are completed on this core.
   protected SolrCore waitAddPendingCoreOps(String name) {
 
@@ -419,7 +349,7 @@ class SolrCores {
         if (! pendingCoreOps.add(name)) {
           CoreContainer.log.warn("Replaced an entry in pendingCoreOps {}, we should not be doing this", name);
         }
-        return getCoreFromAnyList(name); // we might have been _unloading_ the core, so return the core if it was loaded.
+        return getCoreFromAnyList(name, false); // we might have been _unloading_ the core, so return the core if it was loaded.
       }
     }
     return null;
@@ -436,103 +366,6 @@ class SolrCores {
     }
   }
 
-
-  protected void persistCores(Config cfg, Map<String, SolrCore> whichCores, SolrResourceLoader loader, List<SolrCoreXMLDef> solrCoreXMLDefs) throws XPathExpressionException {
-    for (SolrCore solrCore : whichCores.values()) {
-      addCoreToPersistList(cfg, loader, solrCore.getCoreDescriptor(), getCoreToOrigName(solrCore), solrCoreXMLDefs);
-    }
-  }
-  
-  private void addCoreProperty(Map<String,String> coreAttribs, SolrResourceLoader loader, Node node, String name,
-      String value, String defaultValue) {
-    
-    if (node == null) {
-      coreAttribs.put(name, value);
-      return;
-    }
-    
-    if (node != null) {
-      String rawAttribValue = DOMUtil.getAttr(node, name, null);
-
-      if (value == null) {
-        coreAttribs.put(name, rawAttribValue);
-        return;
-      }
-      if (rawAttribValue == null && defaultValue != null && value.equals(defaultValue)) {
-        return;
-      }
-      if (rawAttribValue != null && value.equals(DOMUtil.substituteProperty(rawAttribValue, loader.getCoreProperties()))){
-        coreAttribs.put(name, rawAttribValue);
-      } else {
-        coreAttribs.put(name, value);
-      }
-    }
-
-  }
-
-  protected void addCoreToPersistList(Config cfg, SolrResourceLoader loader,
-      CoreDescriptor dcore, String origCoreName,
-      List<SolrCoreXMLDef> solrCoreXMLDefs) throws XPathExpressionException {
-    
-    String coreName = dcore.getProperty(CoreDescriptor.CORE_NAME);
-    
-    Map<String,String> coreAttribs = new HashMap<String,String>();
-
-    CloudDescriptor cd = dcore.getCloudDescriptor();
-    String collection = null;
-    if (cd != null) collection = cd.getCollectionName();
-
-    if (origCoreName == null) {
-      origCoreName = coreName;
-    }
-    
-    Properties properties = dcore.getCoreProperties();
-    Node node = null;
-    if (cfg != null) {
-      node = cfg.getNode("/solr/cores/core[@name='" + origCoreName + "']",
-          false);
-    }
-    
-    coreAttribs.put(CoreDescriptor.CORE_NAME, coreName);
-    
-    addCoreProperty(coreAttribs, loader, node, CoreDescriptor.CORE_INSTDIR, dcore.getRawInstanceDir(), null);
-
-    coreAttribs.put(CoreDescriptor.CORE_COLLECTION,
-        StringUtils.isNotBlank(collection) ? collection : dcore.getName());
-    
-    addCoreProperty(coreAttribs, loader, node, CoreDescriptor.CORE_DATADIR, dcore.getDataDir(), null);
-    addCoreProperty(coreAttribs, loader, node, CoreDescriptor.CORE_ULOGDIR, dcore.getUlogDir(), null);
-    addCoreProperty(coreAttribs, loader, node, CoreDescriptor.CORE_TRANSIENT, Boolean.toString(dcore.isTransient()), null);
-    addCoreProperty(coreAttribs, loader, node, CoreDescriptor.CORE_LOADONSTARTUP, Boolean.toString(dcore.isLoadOnStartup()), null);
-    
-    addCoreProperty(coreAttribs, loader, node, CoreDescriptor.CORE_COLLECTION,
-        collection, dcore.getName());
-    
-    String shard = null;
-    String roles = null;
-    if (cd != null) {
-      shard = cd.getShardId();
-      roles = cd.getRoles();
-    }
-    addCoreProperty(coreAttribs, loader, node, CoreDescriptor.CORE_SHARD,
-        shard, null);
-    
-    addCoreProperty(coreAttribs, loader, node, CoreDescriptor.CORE_ROLES,
-        roles, null);
-
-    coreAttribs.put(CoreDescriptor.CORE_LOADONSTARTUP,
-        Boolean.toString(dcore.isLoadOnStartup()));
-    coreAttribs.put(CoreDescriptor.CORE_TRANSIENT,
-        Boolean.toString(dcore.isTransient()));
-    
-
-    SolrXMLSerializer.SolrCoreXMLDef solrCoreXMLDef = new SolrXMLSerializer.SolrCoreXMLDef();
-    solrCoreXMLDef.coreAttribs = coreAttribs;
-    solrCoreXMLDef.coreProperties = properties;
-    solrCoreXMLDefs.add(solrCoreXMLDef);
-
-  }
-
   protected Object getModifyLock() {
     return modifyLock;
   }
@@ -559,4 +392,37 @@ class SolrCores {
       createdCores.put(core.getName(), core);
     }
   }
+
+  /**
+   * Return the CoreDescriptor corresponding to a given core name.
+   * @param coreName the name of the core
+   * @return the CoreDescriptor
+   */
+  public CoreDescriptor getCoreDescriptor(String coreName) {
+    synchronized (modifyLock) {
+      if (cores.containsKey(coreName))
+        return cores.get(coreName).getCoreDescriptor();
+      if (dynamicDescriptors.containsKey(coreName))
+        return dynamicDescriptors.get(coreName);
+      return null;
+    }
+  }
+
+  /**
+   * Get the CoreDescriptors for every SolrCore managed here
+   * @return a List of CoreDescriptors
+   */
+  public List<CoreDescriptor> getCoreDescriptors() {
+    List<CoreDescriptor> cds = Lists.newArrayList();
+    synchronized (modifyLock) {
+      for (String coreName : getAllCoreNames()) {
+        // TODO: This null check is a bit suspicious - it seems that
+        // getAllCoreNames might return deleted cores as well?
+        CoreDescriptor cd = getCoreDescriptor(coreName);
+        if (cd != null)
+          cds.add(cd);
+      }
+    }
+    return cds;
+  }
 }

Modified: lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/core/SolrDeletionPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/core/SolrDeletionPolicy.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/core/SolrDeletionPolicy.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/core/SolrDeletionPolicy.java Sun Aug 11 12:19:13 2013
@@ -77,7 +77,7 @@ public class SolrDeletionPolicy extends 
    * Internal use for Lucene... do not explicitly call.
    */
   @Override
-  public void onInit(List commits) throws IOException {
+  public void onInit(List<? extends IndexCommit> commits) throws IOException {
     // SOLR-4547: log basic data at INFO, add filenames at DEBUG.
     if (commits.isEmpty()) {
       return;
@@ -86,26 +86,26 @@ public class SolrDeletionPolicy extends 
         new CommitsLoggingInfo(commits));
     log.debug("SolrDeletionPolicy.onInit: commits: {}",
         new CommitsLoggingDebug(commits));
-    updateCommits((List<IndexCommit>) commits);
+    updateCommits(commits);
   }
 
   /**
    * Internal use for Lucene... do not explicitly call.
    */
   @Override
-  public void onCommit(List commits) throws IOException {
+  public void onCommit(List<? extends IndexCommit> commits) throws IOException {
     // SOLR-4547: log basic data at INFO, add filenames at DEBUG.
     log.info("SolrDeletionPolicy.onCommit: commits: {}",
         new CommitsLoggingInfo(commits));
     log.debug("SolrDeletionPolicy.onCommit: commits: {}",
         new CommitsLoggingDebug(commits));
-    updateCommits((List<IndexCommit>) commits);
+    updateCommits(commits);
   }
 
   private static class CommitsLoggingInfo {
-    private List<IndexCommit> commits;
+    private List<? extends IndexCommit> commits;
 
-    public CommitsLoggingInfo(List<IndexCommit> commits) {
+    public CommitsLoggingInfo(List<? extends IndexCommit> commits) {
       this.commits = commits;
     }
 
@@ -135,7 +135,7 @@ public class SolrDeletionPolicy extends 
   }
 
   private static class CommitsLoggingDebug extends CommitsLoggingInfo {
-    public CommitsLoggingDebug(List<IndexCommit> commits) {
+    public CommitsLoggingDebug(List<? extends IndexCommit> commits) {
       super(commits);
     }
 
@@ -150,7 +150,7 @@ public class SolrDeletionPolicy extends 
     }
   }
 
-  private void updateCommits(List<IndexCommit> commits) {
+  private void updateCommits(List<? extends IndexCommit> commits) {
     // to be safe, we should only call delete on a commit point passed to us
     // in this specific call (may be across diff IndexWriter instances).
     // this will happen rarely, so just synchronize everything

Modified: lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/core/SolrInfoMBean.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/core/SolrInfoMBean.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/core/SolrInfoMBean.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/core/SolrInfoMBean.java Sun Aug 11 12:19:13 2013
@@ -30,7 +30,7 @@ import org.apache.solr.common.util.Named
  */
 public interface SolrInfoMBean {
 
-  public enum Category { CORE, QUERYHANDLER, UPDATEHANDLER, CACHE, HIGHLIGHTING, OTHER };
+  public enum Category { CORE, QUERYHANDLER, UPDATEHANDLER, CACHE, HIGHLIGHTING, QUERYPARSER, OTHER };
 
   /**
    * Simple common usage name, e.g. BasicQueryHandler,

Modified: lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java Sun Aug 11 12:19:13 2013
@@ -17,55 +17,57 @@
 
 package org.apache.solr.core;
 
-import java.io.Closeable;
-import java.io.File;
-import java.io.FileFilter;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.net.URLClassLoader;
-import java.util.*;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
 import org.apache.lucene.analysis.util.CharFilterFactory;
 import org.apache.lucene.analysis.util.ResourceLoaderAware;
 import org.apache.lucene.analysis.util.TokenFilterFactory;
 import org.apache.lucene.analysis.util.TokenizerFactory;
+import org.apache.lucene.analysis.util.WordlistLoader;
 import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.analysis.util.WordlistLoader;
 import org.apache.solr.common.ResourceLoader;
+import org.apache.solr.common.SolrException;
 import org.apache.solr.handler.admin.CoreAdminHandler;
+import org.apache.solr.handler.component.SearchComponent;
 import org.apache.solr.handler.component.ShardHandlerFactory;
+import org.apache.solr.request.SolrRequestHandler;
+import org.apache.solr.response.QueryResponseWriter;
+import org.apache.solr.schema.FieldType;
 import org.apache.solr.schema.ManagedIndexSchemaFactory;
 import org.apache.solr.schema.SimilarityFactory;
+import org.apache.solr.search.QParserPlugin;
+import org.apache.solr.update.processor.UpdateRequestProcessorFactory;
+import org.apache.solr.util.FileUtils;
+import org.apache.solr.util.plugin.SolrCoreAware;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.nio.charset.CharacterCodingException;
-import java.nio.charset.Charset;
-import java.lang.reflect.Constructor;
-
 import javax.naming.Context;
 import javax.naming.InitialContext;
 import javax.naming.NamingException;
 import javax.naming.NoInitialContextException;
-
-import org.apache.solr.util.FileUtils;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.handler.component.SearchComponent;
-import org.apache.solr.request.SolrRequestHandler;
-import org.apache.solr.response.QueryResponseWriter;
-import org.apache.solr.schema.FieldType;
-import org.apache.solr.update.processor.UpdateRequestProcessorFactory;
-import org.apache.solr.util.plugin.SolrCoreAware;
-import org.apache.solr.search.QParserPlugin;
+import java.io.Closeable;
+import java.io.File;
+import java.io.FileFilter;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.lang.reflect.Constructor;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 
 /**
  * @since solr 1.3

Modified: lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/core/SolrXMLSerializer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/core/SolrXMLSerializer.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/core/SolrXMLSerializer.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/core/SolrXMLSerializer.java Sun Aug 11 12:19:13 2013
@@ -17,12 +17,24 @@ package org.apache.solr.core;
  * limitations under the License.
  */
 
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.util.XML;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.w3c.dom.Node;
+
+import javax.xml.transform.OutputKeys;
+import javax.xml.transform.Transformer;
+import javax.xml.transform.TransformerFactory;
+import javax.xml.transform.dom.DOMSource;
+import javax.xml.transform.stream.StreamResult;
 import java.io.BufferedWriter;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.OutputStreamWriter;
+import java.io.StringWriter;
 import java.io.Writer;
 import java.nio.channels.FileChannel;
 import java.util.List;
@@ -30,11 +42,6 @@ import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
 
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.util.XML;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 public class SolrXMLSerializer {
   protected static Logger log = LoggerFactory
       .getLogger(SolrXMLSerializer.class);
@@ -62,6 +69,25 @@ public class SolrXMLSerializer {
     if (containerProperties != null && !containerProperties.isEmpty()) {
       writeProperties(w, containerProperties, "  ");
     }
+
+    // Output logging section if any
+    if (solrXMLDef.loggingAttribs.size() > 0 || solrXMLDef.watcherAttribs.size() > 0) {
+      w.write(INDENT + "<logging");
+      for (Map.Entry<String, String> ent : solrXMLDef.loggingAttribs.entrySet()) {
+        writeAttribute(w, ent.getKey(), ent.getValue());
+      }
+      w.write(">\n");
+
+      if (solrXMLDef.watcherAttribs.size() > 0) {
+        w.write(INDENT + INDENT + "<watcher");
+        for (Map.Entry<String, String> ent : solrXMLDef.watcherAttribs.entrySet()) {
+          writeAttribute(w, ent.getKey(), ent.getValue());
+        }
+        w.write("/>\n");
+      }
+      w.write(INDENT + "</logging>\n");
+    }
+
     w.write(INDENT + "<cores");
     Map<String,String> coresAttribs = solrXMLDef.coresAttribs;
     Set<String> coreAttribKeys = coresAttribs.keySet();
@@ -75,9 +101,28 @@ public class SolrXMLSerializer {
       persist(w, coreDef);
     }
 
+    // Shard handler section
+    if (solrXMLDef.shardHandlerNode != null) {
+      w.write(nodeToXML(solrXMLDef.shardHandlerNode));
+    }
+
     w.write(INDENT + "</cores>\n");
     w.write("</solr>\n");
   }
+
+  private String nodeToXML(Node node) {
+    try {
+      TransformerFactory tfactory = TransformerFactory.newInstance();
+      Transformer tx = tfactory.newTransformer();
+      StringWriter buffer = new StringWriter();
+      tx.setOutputProperty(OutputKeys.OMIT_XML_DECLARATION, "yes");
+      tx.transform(new DOMSource(node), new StreamResult(buffer));
+      return buffer.toString();
+    }
+    catch (Exception e) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error transforming XML: " + e.getMessage());
+    }
+  }
   
   /** Writes the cores configuration node for a given core. */
   private void persist(Writer w, SolrCoreXMLDef coreDef) throws IOException {
@@ -116,7 +161,7 @@ public class SolrXMLSerializer {
   }
   
   void persistFile(File file, SolrXMLDef solrXMLDef) {
-    log.info("Persisting cores config to " + file);
+    log.info("Persisting cores config to " + file.getAbsolutePath());
     
     File tmpFile = null;
     try {
@@ -198,6 +243,9 @@ public class SolrXMLSerializer {
     Properties containerProperties;
     Map<String,String> solrAttribs;
     Map<String,String> coresAttribs;
+    Map<String, String> loggingAttribs;
+    Map<String, String> watcherAttribs;
+    Node shardHandlerNode;
     List<SolrCoreXMLDef> coresDefs;
   }
   

Modified: lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/core/StandardIndexReaderFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/core/StandardIndexReaderFactory.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/core/StandardIndexReaderFactory.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/core/StandardIndexReaderFactory.java Sun Aug 11 12:19:13 2013
@@ -31,6 +31,6 @@ public class StandardIndexReaderFactory 
   
   @Override
   public DirectoryReader newReader(Directory indexDir, SolrCore core) throws IOException {
-    return DirectoryReader.open(indexDir, termInfosIndexDivisor);
+    return DirectoryReader.open(indexDir);
   }
 }

Modified: lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/core/ZkContainer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/core/ZkContainer.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/core/ZkContainer.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/core/ZkContainer.java Sun Aug 11 12:19:13 2013
@@ -17,14 +17,6 @@ package org.apache.solr.core;
  * limitations under the License.
  */
 
-import java.io.ByteArrayInputStream;
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.concurrent.TimeoutException;
-
 import org.apache.solr.cloud.CurrentCoreDescriptorProvider;
 import org.apache.solr.cloud.SolrZkServer;
 import org.apache.solr.cloud.ZkController;
@@ -40,6 +32,14 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.xml.sax.InputSource;
 
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.TimeoutException;
+
 public class ZkContainer {
   protected static Logger log = LoggerFactory.getLogger(ZkContainer.class);
   
@@ -49,48 +49,31 @@ public class ZkContainer {
   private String hostPort;
   private String hostContext;
   private String host;
-  private String leaderVoteWait;
+  private int leaderVoteWait;
+  private Boolean genericCoreNodeNames;
   private int distribUpdateConnTimeout;
-
-  public SolrZkServer getZkServer() {
-    return zkServer;
-  }
-
-  public int getZkClientTimeout() {
-    return zkClientTimeout;
-  }
-
-  public String getHostPort() {
-    return hostPort;
-  }
-
-  public String getHostContext() {
-    return hostContext;
-  }
-
-  public String getHost() {
-    return host;
-  }
-
-  public String getLeaderVoteWait() {
-    return leaderVoteWait;
-  }
-
-  public int getDistribUpdateConnTimeout() {
-    return distribUpdateConnTimeout;
-  }
-
-  public int getDistribUpdateSoTimeout() {
-    return distribUpdateSoTimeout;
-  }
-
   private int distribUpdateSoTimeout;
   
   public ZkContainer() {
     
   }
+
+  public void initZooKeeper(final CoreContainer cc, String solrHome, ConfigSolr config) {
+
+    if (config.getCoreLoadThreadCount() <= 1) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+          "SolrCloud requires a value of at least 2 in solr.xml for coreLoadThreads");
+    }
+
+    initZooKeeper(cc, solrHome,
+        config.getZkHost(), config.getZkClientTimeout(), config.getZkHostPort(), config.getZkHostContext(),
+        config.getHost(), config.getLeaderVoteWait(), config.getGenericCoreNodeNames(),
+        config.getDistributedConnectionTimeout(), config.getDistributedSocketTimeout());
+  }
   
-  public void initZooKeeper(final CoreContainer cc, String solrHome, String zkHost, int zkClientTimeout, String hostPort, String hostContext, String host, String leaderVoteWait, int distribUpdateConnTimeout, int distribUpdateSoTimeout) {
+  public void initZooKeeper(final CoreContainer cc, String solrHome, String zkHost, int zkClientTimeout, String hostPort,
+                            String hostContext, String host, int leaderVoteWait, boolean genericCoreNodeNames,
+                            int distribUpdateConnTimeout, int distribUpdateSoTimeout) {
     ZkController zkController = null;
     
     // if zkHost sys property is not set, we are not using ZooKeeper
@@ -108,6 +91,7 @@ public class ZkContainer {
     this.hostContext = hostContext;
     this.host = host;
     this.leaderVoteWait = leaderVoteWait;
+    this.genericCoreNodeNames = genericCoreNodeNames;
     this.distribUpdateConnTimeout = distribUpdateConnTimeout;
     this.distribUpdateSoTimeout = distribUpdateSoTimeout;
     
@@ -163,7 +147,7 @@ public class ZkContainer {
         }
         zkController = new ZkController(cc, zookeeperHost, zkClientTimeout,
             zkClientConnectTimeout, host, hostPort, hostContext,
-            leaderVoteWait, distribUpdateConnTimeout, distribUpdateSoTimeout,
+            leaderVoteWait, genericCoreNodeNames, distribUpdateConnTimeout, distribUpdateSoTimeout,
             new CurrentCoreDescriptorProvider() {
 
               @Override
@@ -197,7 +181,7 @@ public class ZkContainer {
 
         
         if(boostrapConf) {
-          ZkController.bootstrapConf(zkController.getZkClient(), cc.cfg, solrHome);
+          ZkController.bootstrapConf(zkController.getZkClient(), cc, solrHome);
         }
         
       } catch (InterruptedException e) {
@@ -243,8 +227,7 @@ public class ZkContainer {
             "Could not find config name for collection:" + collection);
       }
       solrLoader = new ZkSolrResourceLoader(instanceDir, zkConfigName,
-          loader.getClassLoader(), ConfigSolrXml.getCoreProperties(instanceDir,
-              dcore), zkController);
+          loader.getClassLoader(), dcore.getCoreProperties(), zkController);
       config = getSolrConfigFromZk(zkConfigName, dcore.getConfigName(),
           solrLoader);
       schema = IndexSchemaFactory.buildIndexSchema(dcore.getSchemaName(),

Modified: lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/handler/AnalysisRequestHandlerBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/handler/AnalysisRequestHandlerBase.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/handler/AnalysisRequestHandlerBase.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/handler/AnalysisRequestHandlerBase.java Sun Aug 11 12:19:13 2013
@@ -86,7 +86,7 @@ public abstract class AnalysisRequestHan
 
       TokenStream tokenStream = null;
       try {
-        tokenStream = analyzer.tokenStream(context.getFieldName(), new StringReader(value));
+        tokenStream = analyzer.tokenStream(context.getFieldName(), value);
       } catch (IOException e) {
         throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
       }
@@ -140,7 +140,7 @@ public abstract class AnalysisRequestHan
   protected Set<BytesRef> getQueryTokenSet(String query, Analyzer analyzer) {
     try {
       final Set<BytesRef> tokens = new HashSet<BytesRef>();
-      final TokenStream tokenStream = analyzer.tokenStream("", new StringReader(query));
+      final TokenStream tokenStream = analyzer.tokenStream("", query);
       final TermToBytesRefAttribute bytesAtt = tokenStream.getAttribute(TermToBytesRefAttribute.class);
       final BytesRef bytes = bytesAtt.getBytesRef();
 

Modified: lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/handler/DocumentAnalysisRequestHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/handler/DocumentAnalysisRequestHandler.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/handler/DocumentAnalysisRequestHandler.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/handler/DocumentAnalysisRequestHandler.java Sun Aug 11 12:19:13 2013
@@ -35,6 +35,7 @@ import org.apache.solr.request.SolrQuery
 import org.apache.solr.schema.FieldType;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.SchemaField;
+import org.apache.solr.util.EmptyEntityResolver;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -90,6 +91,8 @@ public class DocumentAnalysisRequestHand
     super.init(args);
 
     inputFactory = XMLInputFactory.newInstance();
+    EmptyEntityResolver.configureXMLInputFactory(inputFactory);
+    inputFactory.setXMLReporter(xmllog);
     try {
       // The java 1.6 bundled stax parser (sjsxp) does not currently have a thread-safe
       // XMLInputFactory, as that implementation tries to cache and reuse the
@@ -103,7 +106,6 @@ public class DocumentAnalysisRequestHand
       // isimplementation specific.
       log.debug("Unable to set the 'reuse-instance' property for the input factory: " + inputFactory);
     }
-    inputFactory.setXMLReporter(xmllog);
   }
 
   /**

Modified: lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java Sun Aug 11 12:19:13 2013
@@ -20,6 +20,7 @@ import java.io.File;
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.InputStreamReader;
 import java.io.OutputStream;
 import java.io.Writer;
 import java.nio.ByteBuffer;
@@ -48,6 +49,7 @@ import org.apache.lucene.index.IndexWrit
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
+import static org.apache.lucene.util.IOUtils.CHARSET_UTF_8;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.params.CommonParams;
@@ -813,7 +815,7 @@ public class ReplicationHandler extends 
         try {
           final InputStream is = new PropertiesInputStream(input);
           Properties props = new Properties();
-          props.load(is);
+          props.load(new InputStreamReader(is, CHARSET_UTF_8));
           return props;
         } finally {
           input.close();

Modified: lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/handler/SnapPuller.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/handler/SnapPuller.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/handler/SnapPuller.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/handler/SnapPuller.java Sun Aug 11 12:19:13 2013
@@ -40,7 +40,9 @@ import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
-import java.io.OutputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.io.Writer;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
 import java.text.SimpleDateFormat;
@@ -73,6 +75,7 @@ import org.apache.lucene.index.IndexWrit
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
+import static org.apache.lucene.util.IOUtils.CHARSET_UTF_8;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.impl.HttpClientUtil;
 import org.apache.solr.client.solrj.impl.HttpSolrServer;
@@ -318,20 +321,25 @@ public class SnapPuller {
       long latestVersion = (Long) response.get(CMD_INDEX_VERSION);
       long latestGeneration = (Long) response.get(GENERATION);
 
-      IndexCommit commit;
-      RefCounted<SolrIndexSearcher> searcherRefCounted = null;
-      try {
-        searcherRefCounted = core.getNewestSearcher(false);
-        if (searcherRefCounted == null) {
-          SolrException.log(LOG, "No open searcher found - fetch aborted");
-          return false;
+      // TODO: make sure that getLatestCommit only returns commit points for the main index (i.e. no side-car indexes)
+      IndexCommit commit = core.getDeletionPolicy().getLatestCommit();
+      if (commit == null) {
+        // Presumably the IndexWriter hasn't been opened yet, and hence the deletion policy hasn't been updated with commit points
+        RefCounted<SolrIndexSearcher> searcherRefCounted = null;
+        try {
+          searcherRefCounted = core.getNewestSearcher(false);
+          if (searcherRefCounted == null) {
+            LOG.warn("No open searcher found - fetch aborted");
+            return false;
+          }
+          commit = searcherRefCounted.get().getIndexReader().getIndexCommit();
+        } finally {
+          if (searcherRefCounted != null)
+            searcherRefCounted.decref();
         }
-        commit = searcherRefCounted.get().getIndexReader().getIndexCommit();
-      } finally {
-        if (searcherRefCounted != null)
-          searcherRefCounted.decref();
       }
-      
+
+
       if (latestVersion == 0L) {
         if (forceReplication && commit.getGeneration() != 0) {
           // since we won't get the files for an empty index,
@@ -403,7 +411,14 @@ public class SnapPuller {
             successfulInstall = modifyIndexProps(tmpIdxDirName);
             deleteTmpIdxDir  =  false;
           } else {
-            successfulInstall = moveIndexFiles(tmpIndexDir, indexDir);
+            solrCore.getUpdateHandler().getSolrCoreState()
+                .closeIndexWriter(core, true);
+            try {
+              successfulInstall = moveIndexFiles(tmpIndexDir, indexDir);
+            } finally {
+              solrCore.getUpdateHandler().getSolrCoreState()
+                  .openIndexWriter(core);
+            }
           }
           if (successfulInstall) {
             if (isFullCopyNeeded) {
@@ -426,7 +441,12 @@ public class SnapPuller {
             successfulInstall = modifyIndexProps(tmpIdxDirName);
             deleteTmpIdxDir =  false;
           } else {
-            successfulInstall = moveIndexFiles(tmpIndexDir, indexDir);
+            solrCore.getUpdateHandler().getSolrCoreState().closeIndexWriter(core, true);
+            try {
+              successfulInstall = moveIndexFiles(tmpIndexDir, indexDir);
+            } finally {
+              solrCore.getUpdateHandler().getSolrCoreState().openIndexWriter(core);
+            }
           }
           if (successfulInstall) {
             logReplicationTimeAndConfFiles(modifiedConfFiles, successfulInstall);
@@ -443,7 +463,11 @@ public class SnapPuller {
               core.getDirectoryFactory().remove(indexDir);
             }
           }
-          openNewWriterAndSearcher(isFullCopyNeeded);
+          if (isFullCopyNeeded) {
+            solrCore.getUpdateHandler().newIndexWriter(isFullCopyNeeded);
+          }
+          
+          openNewSearcherAndUpdateCommitPoint(isFullCopyNeeded);
         }
         
         replicationStartTime = 0;
@@ -557,7 +581,7 @@ public class SnapPuller {
       }
 
       final IndexOutput out = dir.createOutput(REPLICATION_PROPERTIES, DirectoryFactory.IOCONTEXT_NO_CACHE);
-      OutputStream outFile = new PropertiesOutputStream(out);
+      Writer outFile = new OutputStreamWriter(new PropertiesOutputStream(out), CHARSET_UTF_8);
       try {
         props.store(outFile, "Replication details");
         dir.sync(Collections.singleton(REPLICATION_PROPERTIES));
@@ -615,11 +639,9 @@ public class SnapPuller {
     return sb;
   }
 
-  private void openNewWriterAndSearcher(boolean isFullCopyNeeded) throws IOException {
+  private void openNewSearcherAndUpdateCommitPoint(boolean isFullCopyNeeded) throws IOException {
     SolrQueryRequest req = new LocalSolrQueryRequest(solrCore,
         new ModifiableSolrParams());
-    // reboot the writer on the new index and get a new searcher
-    solrCore.getUpdateHandler().newIndexWriter(isFullCopyNeeded);
     
     RefCounted<SolrIndexSearcher> searcher = null;
     IndexCommit commitPoint;
@@ -871,7 +893,7 @@ public class SnapPuller {
   
         final InputStream is = new PropertiesInputStream(input);
         try {
-          p.load(is);
+          p.load(new InputStreamReader(is, CHARSET_UTF_8));
         } catch (Exception e) {
           LOG.error("Unable to load " + SnapPuller.INDEX_PROPERTIES, e);
         } finally {
@@ -885,9 +907,9 @@ public class SnapPuller {
       }
       final IndexOutput out = dir.createOutput(SnapPuller.INDEX_PROPERTIES, DirectoryFactory.IOCONTEXT_NO_CACHE);
       p.put("index", tmpIdxDirName);
-      OutputStream os = null;
+      Writer os = null;
       try {
-        os = new PropertiesOutputStream(out);
+        os = new OutputStreamWriter(new PropertiesOutputStream(out), CHARSET_UTF_8);
         p.store(os, SnapPuller.INDEX_PROPERTIES);
         dir.sync(Collections.singleton(INDEX_PROPERTIES));
       } catch (Exception e) {

Modified: lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java Sun Aug 11 12:19:13 2013
@@ -17,10 +17,6 @@ package org.apache.solr.handler.admin;
  * limitations under the License.
  */
 
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-
 import org.apache.solr.client.solrj.SolrResponse;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.impl.HttpSolrServer;
@@ -32,6 +28,7 @@ import org.apache.solr.cloud.OverseerCol
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.ImplicitDocRouter;
 import org.apache.solr.common.cloud.ZkCoreNodeProps;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
@@ -49,6 +46,22 @@ import org.apache.zookeeper.KeeperExcept
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.solr.cloud.Overseer.QUEUE_OPERATION;
+import static org.apache.solr.cloud.OverseerCollectionProcessor.COLL_CONF;
+import static org.apache.solr.cloud.OverseerCollectionProcessor.CREATESHARD;
+import static org.apache.solr.cloud.OverseerCollectionProcessor.CREATE_NODE_SET;
+import static org.apache.solr.cloud.OverseerCollectionProcessor.MAX_SHARDS_PER_NODE;
+import static org.apache.solr.cloud.OverseerCollectionProcessor.NUM_SLICES;
+import static org.apache.solr.cloud.OverseerCollectionProcessor.REPLICATION_FACTOR;
+import static org.apache.solr.cloud.OverseerCollectionProcessor.ROUTER;
+import static org.apache.solr.cloud.OverseerCollectionProcessor.SHARDS_PROP;
+import static org.apache.solr.common.cloud.DocRouter.ROUTE_FIELD;
+import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
 
 public class CollectionsHandler extends RequestHandlerBase {
   protected static Logger log = LoggerFactory.getLogger(CollectionsHandler.class);
@@ -132,10 +145,17 @@ public class CollectionsHandler extends 
         this.handleDeleteAliasAction(req, rsp);
         break;
       }
-        case SPLITSHARD:  {
-          this.handleSplitShardAction(req, rsp);
-          break;
-        }
+      case SPLITSHARD:  {
+        this.handleSplitShardAction(req, rsp);
+        break;
+      }
+      case DELETESHARD: {
+        this.handleDeleteShardAction(req, rsp);
+        break;
+      }case CREATESHARD: {
+        this.handleCreateShard(req, rsp);
+        break;
+      }
 
       default: {
           throw new RuntimeException("Unknown action: " + action);
@@ -146,13 +166,18 @@ public class CollectionsHandler extends 
   }
   
   public static long DEFAULT_ZK_TIMEOUT = 60*1000;
+
+  private void handleResponse(String operation, ZkNodeProps m,
+                              SolrQueryResponse rsp) throws KeeperException, InterruptedException {
+    handleResponse(operation, m, rsp, DEFAULT_ZK_TIMEOUT);
+  }
   
   private void handleResponse(String operation, ZkNodeProps m,
-      SolrQueryResponse rsp) throws KeeperException, InterruptedException {
+      SolrQueryResponse rsp, long timeout) throws KeeperException, InterruptedException {
     long time = System.currentTimeMillis();
     QueueEvent event = coreContainer.getZkController()
         .getOverseerCollectionQueue()
-        .offer(ZkStateReader.toJSON(m), DEFAULT_ZK_TIMEOUT);
+        .offer(ZkStateReader.toJSON(m), timeout);
     if (event.getBytes() != null) {
       SolrResponse response = SolrResponse.deserialize(event.getBytes());
       rsp.getValues().addAll(response.getResponse());
@@ -162,9 +187,9 @@ public class CollectionsHandler extends 
         rsp.setException(new SolrException(code != null && code != -1 ? ErrorCode.getErrorCode(code) : ErrorCode.SERVER_ERROR, (String)exp.get("msg")));
       }
     } else {
-      if (System.currentTimeMillis() - time >= DEFAULT_ZK_TIMEOUT) {
+      if (System.currentTimeMillis() - time >= timeout) {
         throw new SolrException(ErrorCode.SERVER_ERROR, operation
-            + " the collection time out:" + DEFAULT_ZK_TIMEOUT / 1000 + "s");
+            + " the collection time out:" + timeout / 1000 + "s");
       } else if (event.getWatchedEvent() != null) {
         throw new SolrException(ErrorCode.SERVER_ERROR, operation
             + " the collection error [Watcher fired on path: "
@@ -251,13 +276,7 @@ public class CollectionsHandler extends 
   private void handleCreateAction(SolrQueryRequest req,
       SolrQueryResponse rsp) throws InterruptedException, KeeperException {
     log.info("Creating Collection : " + req.getParamString());
-    Integer numReplicas = req.getParams().getInt(OverseerCollectionProcessor.REPLICATION_FACTOR, 1);
     String name = req.getParams().required().get("name");
-    String configName = req.getParams().get("collection.configName");
-    String numShards = req.getParams().get(OverseerCollectionProcessor.NUM_SLICES);
-    String maxShardsPerNode = req.getParams().get(OverseerCollectionProcessor.MAX_SHARDS_PER_NODE);
-    String createNodeSetStr = req.getParams().get(OverseerCollectionProcessor.CREATE_NODE_SET);
-    
     if (name == null) {
       log.error("Collection name is required to create a new collection");
       throw new SolrException(ErrorCode.BAD_REQUEST,
@@ -267,18 +286,59 @@ public class CollectionsHandler extends 
     Map<String,Object> props = new HashMap<String,Object>();
     props.put(Overseer.QUEUE_OPERATION,
         OverseerCollectionProcessor.CREATECOLLECTION);
-    props.put(OverseerCollectionProcessor.REPLICATION_FACTOR, numReplicas.toString());
-    props.put("name", name);
-    if (configName != null) {
-      props.put("collection.configName", configName);
+
+    copyIfNotNull(req.getParams(),props,
+        "name",
+        REPLICATION_FACTOR,
+         COLL_CONF,
+         NUM_SLICES,
+         MAX_SHARDS_PER_NODE,
+        CREATE_NODE_SET ,
+        ROUTER,
+        SHARDS_PROP,
+        ROUTE_FIELD);
+
+
+    ZkNodeProps m = new ZkNodeProps(props);
+    handleResponse(OverseerCollectionProcessor.CREATECOLLECTION, m, rsp);
+  }
+
+  private void handleCreateShard(SolrQueryRequest req, SolrQueryResponse rsp) throws KeeperException, InterruptedException {
+    log.info("Create shard: " + req.getParamString());
+    req.getParams().required().check(COLLECTION_PROP, SHARD_ID_PROP);
+    ClusterState clusterState = coreContainer.getZkController().getClusterState();
+    if(!ImplicitDocRouter.NAME.equals( clusterState.getCollection(req.getParams().get(COLLECTION_PROP)).getStr(ROUTER)))
+      throw new SolrException(ErrorCode.BAD_REQUEST, "shards can be added only to 'implicit' collections" );
+
+    Map<String, Object> map = OverseerCollectionProcessor.asMap(QUEUE_OPERATION, CREATESHARD);
+    copyIfNotNull(req.getParams(),map,COLLECTION_PROP, SHARD_ID_PROP, REPLICATION_FACTOR);
+    ZkNodeProps m = new ZkNodeProps(map);
+    handleResponse(CREATESHARD, m, rsp);
+  }
+
+  private static void copyIfNotNull(SolrParams params, Map<String, Object> props, String... keys) {
+    if(keys !=null){
+      for (String key : keys) {
+        String v = params.get(key);
+        if(v != null) props.put(key,v);
+      }
     }
-    props.put(OverseerCollectionProcessor.NUM_SLICES, numShards);
-    props.put(OverseerCollectionProcessor.MAX_SHARDS_PER_NODE, maxShardsPerNode);
-    props.put(OverseerCollectionProcessor.CREATE_NODE_SET, createNodeSetStr);
+
+  }
+  
+  private void handleDeleteShardAction(SolrQueryRequest req,
+      SolrQueryResponse rsp) throws InterruptedException, KeeperException {
+    log.info("Deleting Shard : " + req.getParamString());
+    String name = req.getParams().required().get("collection");
+    String shard = req.getParams().required().get("shard");
     
-    ZkNodeProps m = new ZkNodeProps(props);
+    Map<String,Object> props = new HashMap<String,Object>();
+    props.put("collection", name);
+    props.put(Overseer.QUEUE_OPERATION, OverseerCollectionProcessor.DELETESHARD);
+    props.put(ZkStateReader.SHARD_ID_PROP, shard);
 
-    handleResponse(OverseerCollectionProcessor.CREATECOLLECTION, m, rsp);
+    ZkNodeProps m = new ZkNodeProps(props);
+    handleResponse(OverseerCollectionProcessor.DELETESHARD, m, rsp);
   }
 
   private void handleSplitShardAction(SolrQueryRequest req, SolrQueryResponse rsp) throws KeeperException, InterruptedException {
@@ -295,10 +355,7 @@ public class CollectionsHandler extends 
 
     ZkNodeProps m = new ZkNodeProps(props);
 
-    // todo remove this hack
-    DEFAULT_ZK_TIMEOUT *= 5;
-    handleResponse(OverseerCollectionProcessor.SPLITSHARD, m, rsp);
-    DEFAULT_ZK_TIMEOUT /= 5;
+    handleResponse(OverseerCollectionProcessor.SPLITSHARD, m, rsp, DEFAULT_ZK_TIMEOUT * 5);
   }
 
   public static ModifiableSolrParams params(String... params) {