You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2012/07/19 17:59:32 UTC

svn commit: r1363400 [24/31] - in /lucene/dev/branches/pforcodec_3892: ./ dev-tools/ dev-tools/eclipse/ dev-tools/idea/.idea/ dev-tools/idea/.idea/copyright/ dev-tools/idea/.idea/libraries/ dev-tools/idea/lucene/ dev-tools/maven/ dev-tools/maven/lucene...

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/analysis/TrieTokenizerFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/analysis/TrieTokenizerFactory.java?rev=1363400&r1=1363399&r2=1363400&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/analysis/TrieTokenizerFactory.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/analysis/TrieTokenizerFactory.java Thu Jul 19 15:58:54 2012
@@ -73,17 +73,13 @@ final class TrieTokenizer extends Tokeni
     this.precisionStep = precisionStep;
     this.ts = ts;
 
-   try {
-     reset(input);
-   } catch (IOException e) {
-     throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unable to create TrieIndexTokenizer", e);
-   }
+    setReader(input);
   }
 
   @Override
-  public void reset(Reader input) throws IOException {
+  public void setReader(Reader input) {
    try {
-      super.reset(input);
+      super.setReader(input);
       input = super.input;
       char[] buf = new char[32];
       int len = input.read(buf);
@@ -132,7 +128,7 @@ final class TrieTokenizer extends Tokeni
   }
 
   @Override
-  public boolean incrementToken() throws IOException {
+  public boolean incrementToken() {
     if (ts.incrementToken()) {
       ofsAtt.setOffset(startOfs, endOfs);
       return true;

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/client/solrj/embedded/EmbeddedSolrServer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/client/solrj/embedded/EmbeddedSolrServer.java?rev=1363400&r1=1363399&r2=1363400&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/client/solrj/embedded/EmbeddedSolrServer.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/client/solrj/embedded/EmbeddedSolrServer.java Thu Jul 19 15:58:54 2012
@@ -21,9 +21,7 @@ import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
-import java.util.Set;
 
-import org.apache.lucene.document.Document;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrServer;
 import org.apache.solr.client.solrj.SolrServerException;
@@ -45,9 +43,6 @@ import org.apache.solr.request.SolrReque
 import org.apache.solr.response.BinaryResponseWriter;
 import org.apache.solr.response.ResultContext;
 import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.response.transform.DocTransformer;
-import org.apache.solr.search.DocIterator;
-import org.apache.solr.search.DocList;
 import org.apache.solr.servlet.SolrRequestParsers;
 
 /**
@@ -187,7 +182,7 @@ public class EmbeddedSolrServer extends 
           new JavaBinCodec(resolver) {
 
             @Override
-            public void writeSolrDocument(SolrDocument doc) throws IOException {
+            public void writeSolrDocument(SolrDocument doc) {
               callback.streamSolrDocument( doc );
               //super.writeSolrDocument( doc, fields );
             }

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/DistributedQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/DistributedQueue.java?rev=1363400&r1=1363399&r2=1363400&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/DistributedQueue.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/DistributedQueue.java Thu Jul 19 15:58:54 2012
@@ -1,4 +1,4 @@
-/**
+/*
  *
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -266,6 +266,45 @@ public class DistributedQueue {
   }
   
   /**
+   * Returns the data at the first element of the queue, or null if the queue is
+   * empty.
+   * 
+   * @return data at the first element of the queue, or null.
+   * @throws KeeperException
+   * @throws InterruptedException
+   */
+  public byte[] peek(boolean block) throws KeeperException, InterruptedException {
+    if (!block) {
+      return peek();
+    }
+    
+    TreeMap<Long,String> orderedChildren;
+    while (true) {
+      LatchChildWatcher childWatcher = new LatchChildWatcher();
+      try {
+        orderedChildren = orderedChildren(childWatcher);
+      } catch (KeeperException.NoNodeException e) {
+        zookeeper.create(dir, new byte[0], acl, CreateMode.PERSISTENT);
+        continue;
+      }
+      if (orderedChildren.size() == 0) {
+        childWatcher.await();
+        continue;
+      }
+      
+      for (String headNode : orderedChildren.values()) {
+        String path = dir + "/" + headNode;
+        try {
+          byte[] data = zookeeper.getData(path, false, null);
+          return data;
+        } catch (KeeperException.NoNodeException e) {
+          // Another client deleted the node first.
+        }
+      }
+    }
+  }
+  
+  /**
    * Attempts to remove the head of the queue and return it. Returns null if the
    * queue is empty.
    * 

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java?rev=1363400&r1=1363399&r2=1363400&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java Thu Jul 19 15:58:54 2012
@@ -13,6 +13,7 @@ import org.apache.solr.common.cloud.ZkNo
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.SolrCore;
+import org.apache.solr.handler.component.ShardHandler;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.NodeExistsException;
@@ -96,6 +97,8 @@ class ShardLeaderElectionContextBase ext
           CreateMode.EPHEMERAL, true);
     }
     
+    // TODO: above we make it looks like leaderProps could be true, but here
+    // you would get an NPE if it was.
     ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION,
         "leader", ZkStateReader.SHARD_ID_PROP, shardId,
         ZkStateReader.COLLECTION_PROP, collection, ZkStateReader.BASE_URL_PROP,
@@ -245,10 +248,14 @@ final class OverseerElectionContext exte
   
   private final SolrZkClient zkClient;
   private final ZkStateReader stateReader;
+  private ShardHandler shardHandler;
+  private String adminPath;
 
-  public OverseerElectionContext(final String zkNodeName, ZkStateReader stateReader) {
+  public OverseerElectionContext(ShardHandler shardHandler, String adminPath, final String zkNodeName, ZkStateReader stateReader) {
     super(zkNodeName, "/overseer_elect", "/overseer_elect/leader", null, stateReader.getZkClient());
     this.stateReader = stateReader;
+    this.shardHandler = shardHandler;
+    this.adminPath = adminPath;
     this.zkClient = stateReader.getZkClient();
   }
 
@@ -271,7 +278,7 @@ final class OverseerElectionContext exte
           CreateMode.EPHEMERAL, true);
     }
   
-    new Overseer(stateReader, id);
+    new Overseer(shardHandler, adminPath, stateReader, id);
   }
   
 }

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java?rev=1363400&r1=1363399&r2=1363400&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java Thu Jul 19 15:58:54 2012
@@ -30,12 +30,12 @@ import org.apache.solr.common.SolrExcept
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkCmdExecutor;
 import org.apache.solr.common.cloud.ZooKeeperException;
-import org.apache.solr.core.SolrCore;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.ConnectionLossException;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.Watcher.Event.EventType;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -115,6 +115,11 @@ public  class LeaderElector {
               
               @Override
               public void process(WatchedEvent event) {
+                // session events are not change events,
+                // and do not remove the watcher
+                if (EventType.None.equals(event.getType())) {
+                  return;
+                }
                 // am I the next leader?
                 try {
                   checkIfIamLeader(seq, context, true);

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/Overseer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/Overseer.java?rev=1363400&r1=1363399&r2=1363400&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/Overseer.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/Overseer.java Thu Jul 19 15:58:54 2012
@@ -31,6 +31,7 @@ import org.apache.solr.common.cloud.ZkCo
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.cloud.ZooKeeperException;
+import org.apache.solr.handler.component.ShardHandler;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
@@ -48,6 +49,7 @@ public class Overseer {
   
   private static class CloudStateUpdater implements Runnable {
     
+    private static final String DELETECORE = "deletecore";
     private final ZkStateReader reader;
     private final SolrZkClient zkClient;
     private final String myId;
@@ -156,11 +158,10 @@ public class Overseer {
     }
 
     private CloudState processMessage(CloudState cloudState,
-        final ZkNodeProps message, final String operation)
-        throws KeeperException, InterruptedException {
+        final ZkNodeProps message, final String operation) {
       if ("state".equals(operation)) {
         cloudState = updateState(cloudState, message);
-      } else if ("deletecore".equals(operation)) {
+      } else if (DELETECORE.equals(operation)) {
         cloudState = removeCore(cloudState, message);
       } else if (ZkStateReader.LEADER_PROP.equals(operation)) {
         StringBuilder sb = new StringBuilder();
@@ -197,10 +198,8 @@ public class Overseer {
       }
       /**
        * Try to assign core to the cluster. 
-       * @throws KeeperException 
-       * @throws InterruptedException 
        */
-      private CloudState updateState(CloudState state, final ZkNodeProps message) throws KeeperException, InterruptedException {
+      private CloudState updateState(CloudState state, final ZkNodeProps message) {
         final String collection = message.get(ZkStateReader.COLLECTION_PROP);
         final String zkCoreNodeName = message.get(ZkStateReader.NODE_NAME_PROP) + "_" + message.get(ZkStateReader.CORE_NAME_PROP);
         final Integer numShards = message.get(ZkStateReader.NUM_SHARDS_PROP)!=null?Integer.parseInt(message.get(ZkStateReader.NUM_SHARDS_PROP)):null;
@@ -363,13 +362,36 @@ public class Overseer {
                 LinkedHashMap<String, ZkNodeProps> newShards = new LinkedHashMap<String, ZkNodeProps>();
                 newShards.putAll(slice.getShards());
                 newShards.remove(coreNodeName);
+                
                 Slice newSlice = new Slice(slice.getName(), newShards);
                 newSlices.put(slice.getName(), newSlice);
+
               } else {
                 newSlices.put(slice.getName(), slice);
               }
             }
-            newStates.put(collectionName, newSlices);
+            int cnt = 0;
+            for (Slice slice : newSlices.values()) {
+              cnt+=slice.getShards().size();
+            }
+            // TODO: if no nodes are left after this unload
+            // remove from zk - do we have a race where Overseer
+            // see's registered nodes and publishes though?
+            if (cnt > 0) {
+              newStates.put(collectionName, newSlices);
+            } else {
+              // TODO: it might be better logically to have this in ZkController
+              // but for tests (it's easier) it seems better for the moment to leave CoreContainer and/or
+              // ZkController out of the Overseer.
+              try {
+                zkClient.clean("/collections/" + collectionName);
+              } catch (InterruptedException e) {
+                SolrException.log(log, "Cleaning up collection in zk was interrupted:" + collectionName, e);
+                Thread.currentThread().interrupt();
+              } catch (KeeperException e) {
+                SolrException.log(log, "Problem cleaning up collection in zk:" + collectionName, e);
+              }
+            }
           } else {
             newStates.put(collectionName, cloudState.getSlices(collectionName));
           }
@@ -377,9 +399,10 @@ public class Overseer {
         CloudState newState = new CloudState(cloudState.getLiveNodes(), newStates);
         return newState;
      }
+    
   }
   
-  public Overseer(final ZkStateReader reader, final String id) throws KeeperException, InterruptedException {
+  public Overseer(ShardHandler shardHandler, String adminPath, final ZkStateReader reader, final String id) throws KeeperException, InterruptedException {
     log.info("Overseer (id=" + id + ") starting");
     createOverseerNode(reader.getZkClient());
     //launch cluster state updater thread
@@ -387,6 +410,11 @@ public class Overseer {
     Thread updaterThread = new Thread(tg, new CloudStateUpdater(reader, id));
     updaterThread.setDaemon(true);
     updaterThread.start();
+    
+    ThreadGroup ccTg = new ThreadGroup("Overseer collection creation process.");
+    Thread ccThread = new Thread(ccTg, new OverseerCollectionProcessor(reader, id, shardHandler, adminPath));
+    ccThread.setDaemon(true);
+    ccThread.start();
   }
 
   /**
@@ -403,6 +431,12 @@ public class Overseer {
     return new DistributedQueue(zkClient.getSolrZooKeeper(), "/overseer/queue-work", null);
   }
   
+  /* Collection creation queue */
+  static DistributedQueue getCollectionQueue(final SolrZkClient zkClient) {
+    createOverseerNode(zkClient);
+    return new DistributedQueue(zkClient.getSolrZooKeeper(), "/overseer/collection-queue-work", null);
+  }
+  
   private static void createOverseerNode(final SolrZkClient zkClient) {
     try {
       zkClient.create("/overseer", new byte[0], CreateMode.PERSISTENT, true);

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java?rev=1363400&r1=1363399&r2=1363400&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java Thu Jul 19 15:58:54 2012
@@ -18,12 +18,10 @@ package org.apache.solr.cloud;
  */
 
 import java.io.IOException;
-import java.net.MalformedURLException;
 import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
-import java.util.concurrent.TimeoutException;
 
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.impl.HttpSolrServer;
@@ -137,7 +135,7 @@ public class RecoveryStrategy extends Th
       }
       
       ModifiableSolrParams solrParams = new ModifiableSolrParams();
-      solrParams.set(ReplicationHandler.MASTER_URL, leaderUrl + "replication");
+      solrParams.set(ReplicationHandler.MASTER_URL, leaderUrl);
       
       if (isClosed()) retries = INTERRUPTED;
       boolean success = replicationHandler.doFetch(solrParams, true); // TODO: look into making sure force=true does not download files we already have
@@ -162,8 +160,7 @@ public class RecoveryStrategy extends Th
     }
   }
 
-  private void commitOnLeader(String leaderUrl) throws MalformedURLException,
-      SolrServerException, IOException {
+  private void commitOnLeader(String leaderUrl) throws SolrServerException, IOException {
     HttpSolrServer server = new HttpSolrServer(leaderUrl);
     server.setConnectionTimeout(30000);
     server.setSoTimeout(30000);
@@ -176,7 +173,7 @@ public class RecoveryStrategy extends Th
   }
 
   private void sendPrepRecoveryCmd(String leaderBaseUrl,
-      String leaderCoreName) throws MalformedURLException, SolrServerException,
+      String leaderCoreName) throws SolrServerException,
       IOException {
     HttpSolrServer server = new HttpSolrServer(leaderBaseUrl);
     server.setConnectionTimeout(45000);
@@ -430,7 +427,7 @@ public class RecoveryStrategy extends Th
   }
 
   private Future<RecoveryInfo> replay(UpdateLog ulog)
-      throws InterruptedException, ExecutionException, TimeoutException {
+      throws InterruptedException, ExecutionException {
     Future<RecoveryInfo> future = ulog.applyBufferedUpdates();
     if (future == null) {
       // no replay needed\

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/SolrZkServer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/SolrZkServer.java?rev=1363400&r1=1363399&r2=1363400&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/SolrZkServer.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/SolrZkServer.java Thu Jul 19 15:58:54 2012
@@ -20,8 +20,8 @@ package org.apache.solr.cloud;
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileInputStream;
-import java.io.FileReader;
 import java.io.IOException;
+import java.io.InputStreamReader;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
@@ -29,6 +29,7 @@ import java.util.Map;
 import java.util.Properties;
 import java.util.Map.Entry;
 
+import org.apache.lucene.util.IOUtils;
 import org.apache.solr.common.SolrException;
 import org.apache.zookeeper.server.ServerConfig;
 import org.apache.zookeeper.server.ZooKeeperServerMain;
@@ -462,7 +463,7 @@ class SolrZkServerProps extends QuorumPe
             + " file is missing");
       }
 
-      BufferedReader br = new BufferedReader(new FileReader(myIdFile));
+      BufferedReader br = new BufferedReader(new InputStreamReader(new FileInputStream(myIdFile), IOUtils.CHARSET_UTF_8));
       String myIdString;
       try {
         myIdString = br.readLine();

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java?rev=1363400&r1=1363399&r2=1363400&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java Thu Jul 19 15:58:54 2012
@@ -17,14 +17,11 @@ package org.apache.solr.cloud;
  * limitations under the License.
  */
 
-import java.io.IOException;
-import java.net.MalformedURLException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
 import org.apache.http.client.HttpClient;
-import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.impl.HttpClientUtil;
 import org.apache.solr.client.solrj.impl.HttpSolrServer;
 import org.apache.solr.client.solrj.request.CoreAdminRequest.RequestRecovery;
@@ -160,8 +157,7 @@ public class SyncStrategy {
   }
   
   private boolean syncWithReplicas(ZkController zkController, SolrCore core,
-      ZkNodeProps props, String collection, String shardId)
-      throws MalformedURLException, SolrServerException, IOException {
+      ZkNodeProps props, String collection, String shardId) {
     List<ZkCoreNodeProps> nodes = zkController.getZkStateReader()
         .getReplicaProps(collection, shardId,
             props.get(ZkStateReader.NODE_NAME_PROP),
@@ -191,8 +187,7 @@ public class SyncStrategy {
   }
   
   private void syncToMe(ZkController zkController, String collection,
-      String shardId, ZkNodeProps leaderProps) throws MalformedURLException,
-      SolrServerException, IOException {
+      String shardId, ZkNodeProps leaderProps) {
     
     // sync everyone else
     // TODO: we should do this in parallel at least

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/ZkController.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/ZkController.java?rev=1363400&r1=1363399&r2=1363400&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/ZkController.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/ZkController.java Thu Jul 19 15:58:54 2012
@@ -26,7 +26,6 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
-import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeoutException;
 import java.util.regex.Matcher;
@@ -34,6 +33,7 @@ import java.util.regex.Pattern;
 
 import javax.xml.xpath.XPathConstants;
 
+import org.apache.commons.io.FileUtils;
 import org.apache.solr.client.solrj.impl.HttpSolrServer;
 import org.apache.solr.client.solrj.request.CoreAdminRequest.WaitForState;
 import org.apache.solr.common.SolrException;
@@ -51,7 +51,9 @@ import org.apache.solr.core.Config;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.core.SolrCore;
-import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.handler.component.HttpShardHandlerFactory;
+
+import org.apache.solr.handler.component.ShardHandler;
 import org.apache.solr.update.UpdateLog;
 import org.apache.solr.util.DOMUtil;
 import org.apache.zookeeper.CreateMode;
@@ -62,7 +64,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.w3c.dom.Node;
 import org.w3c.dom.NodeList;
-import org.xml.sax.InputSource;
 
 /**
  * Handle ZooKeeper interactions.
@@ -84,7 +85,9 @@ public final class ZkController {
   private final static Pattern URL_PREFIX = Pattern.compile("(https?://).*");
 
   private final boolean SKIP_AUTO_RECOVERY = Boolean.getBoolean("solrcloud.skip.autorecovery");
-  private final DistributedQueue overseerStatusQueue;
+  
+  private final DistributedQueue overseerJobQueue;
+  private final DistributedQueue overseerCollectionQueue;
   
   // package private for tests
 
@@ -114,57 +117,11 @@ public final class ZkController {
   private LeaderElector overseerElector;
   
 
-  // this can be null in which case recovery will be inactive
+  // for now, this can be null in tests, in which case recovery will be inactive, and other features
+  // may accept defaults or use mocks rather than pulling things from a CoreContainer
   private CoreContainer cc;
 
   /**
-   * Bootstraps the current configs for all collections in solr.xml.
-   * Takes two params - the zkhost to connect to and the solrhome location
-   * to find solr.xml.
-   *
-   * If you also pass a solrPort, it will be used to start
-   * an embedded zk useful for single machine, multi node tests.
-   * 
-   * @param args
-   * @throws Exception
-   */
-  public static void main(String[] args) throws Exception {
-    // start up a tmp zk server first
-    String zkServerAddress = args[0];
-    
-    String solrHome = args[1];
-   
-    String solrPort = null;
-    if (args.length > 2) {
-      solrPort = args[2];
-    }
-    
-
-    SolrZkServer zkServer = null;
-    if (solrPort != null) {
-      zkServer = new SolrZkServer("true", null, solrHome + "/zoo_data", solrHome, solrPort);
-      zkServer.parseConfig();
-      zkServer.start();
-    }
-    
-    SolrZkClient zkClient = new SolrZkClient(zkServerAddress, 15000, 5000,
-        new OnReconnect() {
-          @Override
-          public void command() {
-          }});
-    
-    SolrResourceLoader loader = new SolrResourceLoader(solrHome);
-    solrHome = loader.getInstanceDir();
-    
-    InputSource cfgis = new InputSource(new File(solrHome, "solr.xml").toURI().toASCIIString());
-    Config cfg = new Config(loader, null, cfgis , null, false);
-    bootstrapConf(zkClient, cfg, solrHome);
-    if (solrPort != null) {
-      zkServer.stop();
-    }
-  }
-
-  /**
    * @param cc if null, recovery will not be enabled
    * @param zkServerAddress
    * @param zkClientTimeout
@@ -177,7 +134,7 @@ public final class ZkController {
    * @throws TimeoutException
    * @throws IOException
    */
-  public ZkController(CoreContainer cc, String zkServerAddress, int zkClientTimeout, int zkClientConnectTimeout, String localHost, String locaHostPort,
+  public ZkController(final CoreContainer cc, String zkServerAddress, int zkClientTimeout, int zkClientConnectTimeout, String localHost, String locaHostPort,
       String localHostContext, final CurrentCoreDescriptorProvider registerOnReconnect) throws InterruptedException,
       TimeoutException, IOException {
     this.cc = cc;
@@ -204,8 +161,19 @@ public final class ZkController {
               
               // seems we dont need to do this again...
               //Overseer.createClientNodes(zkClient, getNodeName());
-
-              ElectionContext context = new OverseerElectionContext(getNodeName(), zkStateReader);
+              ShardHandler shardHandler;
+              String adminPath;
+              if (cc == null) {
+                shardHandler = new HttpShardHandlerFactory().getShardHandler();
+                adminPath = "/admin/cores";
+              } else {
+                shardHandler = cc.getShardHandlerFactory().getShardHandler();
+                adminPath = cc.getAdminPath();
+              }
+              
+              ElectionContext context = new OverseerElectionContext(
+                  shardHandler, adminPath,
+                  getNodeName(), zkStateReader);
               overseerElector.joinElection(context);
               zkStateReader.createClusterStateWatchersAndUpdate();
               
@@ -255,7 +223,8 @@ public final class ZkController {
 
  
         });
-    this.overseerStatusQueue = Overseer.getInQueue(zkClient);
+    this.overseerJobQueue = Overseer.getInQueue(zkClient);
+    this.overseerCollectionQueue = Overseer.getCollectionQueue(zkClient);
     cmdExecutor = new ZkCmdExecutor();
     leaderElector = new LeaderElector(zkClient);
     zkStateReader = new ZkStateReader(zkClient);
@@ -378,8 +347,19 @@ public final class ZkController {
       createEphemeralLiveNode();
       cmdExecutor.ensureExists(ZkStateReader.COLLECTIONS_ZKNODE, zkClient);
 
+      ShardHandler shardHandler;
+      String adminPath;
+      if (cc == null) {
+        shardHandler = new HttpShardHandlerFactory().getShardHandler();
+        adminPath = "/admin/cores";
+      } else {
+        shardHandler = cc.getShardHandlerFactory().getShardHandler();
+        adminPath = cc.getAdminPath();
+      }
+      
       overseerElector = new LeaderElector(zkClient);
-      ElectionContext context = new OverseerElectionContext(getNodeName(), zkStateReader);
+      ElectionContext context = new OverseerElectionContext(shardHandler,
+          adminPath, getNodeName(), zkStateReader);
       overseerElector.setup(context);
       overseerElector.joinElection(context);
       zkStateReader.createClusterStateWatchersAndUpdate();
@@ -460,10 +440,9 @@ public final class ZkController {
    * @return config value
    * @throws KeeperException
    * @throws InterruptedException
-   * @throws IOException 
    */
   public String readConfigName(String collection) throws KeeperException,
-      InterruptedException, IOException {
+      InterruptedException {
 
     String configName = null;
 
@@ -685,17 +664,12 @@ public final class ZkController {
    * @param core
    * @param cc
    * @return whether or not a recovery was started
-   * @throws InterruptedException
-   * @throws KeeperException
-   * @throws IOException
-   * @throws ExecutionException
    */
   private boolean checkRecovery(String coreName, final CoreDescriptor desc,
       boolean recoverReloadedCores, final boolean isLeader,
       final CloudDescriptor cloudDesc, final String collection,
       final String shardZkNodeName, String shardId, ZkNodeProps leaderProps,
-      SolrCore core, CoreContainer cc) throws InterruptedException,
-      KeeperException, IOException, ExecutionException {
+      SolrCore core, CoreContainer cc) {
     if (SKIP_AUTO_RECOVERY) {
       log.warn("Skipping recovery according to sys prop solrcloud.skip.autorecovery");
       return false;
@@ -749,7 +723,7 @@ public final class ZkController {
             .getCollectionName(), ZkStateReader.STATE_PROP, state,
         ZkStateReader.NUM_SHARDS_PROP, numShards != null ? numShards.toString()
             : null);
-    overseerStatusQueue.offer(ZkStateReader.toJSON(m));
+    overseerJobQueue.offer(ZkStateReader.toJSON(m));
   }
 
   private boolean needsToBeAssignedShardId(final CoreDescriptor desc,
@@ -778,7 +752,7 @@ public final class ZkController {
         "deletecore", ZkStateReader.CORE_NAME_PROP, coreName,
         ZkStateReader.NODE_NAME_PROP, getNodeName(),
         ZkStateReader.COLLECTION_PROP, cloudDesc.getCollectionName());
-    overseerStatusQueue.offer(ZkStateReader.toJSON(m));
+    overseerJobQueue.offer(ZkStateReader.toJSON(m));
 
     final String zkNodeName = getNodeName() + "_" + coreName;
     ElectionContext context = electionContexts.remove(zkNodeName);
@@ -786,6 +760,14 @@ public final class ZkController {
       context.cancelElection();
     }
   }
+  
+  public void createCollection(String collection) throws KeeperException,
+      InterruptedException {
+    ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION,
+        "createcollection", ZkStateReader.NODE_NAME_PROP, getNodeName(),
+        ZkStateReader.COLLECTION_PROP, collection);
+    overseerJobQueue.offer(ZkStateReader.toJSON(m));
+  }
 
   /**
    * @param dir
@@ -814,7 +796,7 @@ public final class ZkController {
     zkClient.printLayoutToStdOut();
   }
 
-  public void createCollectionZkNode(CloudDescriptor cd) throws KeeperException, InterruptedException, IOException {
+  public void createCollectionZkNode(CloudDescriptor cd) throws KeeperException, InterruptedException {
     String collection = cd.getCollectionName();
     
     log.info("Check for collection zkNode:" + collection);
@@ -869,9 +851,7 @@ public final class ZkController {
           
           ZkNodeProps zkProps = new ZkNodeProps(collectionProps);
           zkClient.makePath(collectionPath, ZkStateReader.toJSON(zkProps), CreateMode.PERSISTENT, null, true);
-         
-          // ping that there is a new collection
-          zkClient.setData(ZkStateReader.COLLECTIONS_ZKNODE, (byte[])null, true);
+
         } catch (KeeperException e) {
           // its okay if the node already exists
           if (e.code() != KeeperException.Code.NODEEXISTS) {
@@ -943,8 +923,7 @@ public final class ZkController {
     return zkStateReader;
   }
 
-  private String doGetShardIdProcess(String coreName, CloudDescriptor descriptor)
-      throws InterruptedException {
+  private String doGetShardIdProcess(String coreName, CloudDescriptor descriptor) {
     final String shardZkNodeName = getNodeName() + "_" + coreName;
     int retryCount = 120;
     while (retryCount-- > 0) {
@@ -980,6 +959,24 @@ public final class ZkController {
     }
   }
   
+  public static void downloadFromZK(SolrZkClient zkClient, String zkPath,
+      File dir) throws IOException, KeeperException, InterruptedException {
+    List<String> files = zkClient.getChildren(zkPath, null, true);
+    
+    for (String file : files) {
+      List<String> children = zkClient.getChildren(zkPath + "/" + file, null, true);
+      if (children.size() == 0) {
+        byte[] data = zkClient.getData(zkPath + "/" + file, null, null, true);
+        dir.mkdirs(); 
+        log.info("Write file " + new File(dir, file));
+        FileUtils.writeStringToFile(new File(dir, file), new String(data, "UTF-8"), "UTF-8");
+      } else {
+        downloadFromZK(zkClient, zkPath + "/" + file, new File(dir, file));
+      }
+    }
+  }
+  
+  
   private String getCoreNodeName(CoreDescriptor descriptor){
     return getNodeName() + "_"
         + descriptor.getName();
@@ -988,6 +985,10 @@ public final class ZkController {
   public static void uploadConfigDir(SolrZkClient zkClient, File dir, String configName) throws IOException, KeeperException, InterruptedException {
     uploadToZK(zkClient, dir, ZkController.CONFIGS_ZKNODE + "/" + configName);
   }
+  
+  public static void downloadConfigDir(SolrZkClient zkClient, String configName, File dir) throws IOException, KeeperException, InterruptedException {
+    downloadFromZK(zkClient, ZkController.CONFIGS_ZKNODE + "/" + configName, dir);
+  }
 
   public void preRegister(CoreDescriptor cd) throws KeeperException, InterruptedException {
     // before becoming available, make sure we are not live and active
@@ -996,11 +997,7 @@ public final class ZkController {
     String shardZkNodeName = getCoreNodeName(cd);
     if (cd.getCloudDescriptor().getShardId() == null && needsToBeAssignedShardId(cd, zkStateReader.getCloudState(), shardZkNodeName)) {
       String shardId;
-      try {
-        shardId = doGetShardIdProcess(cd.getName(), cd.getCloudDescriptor());
-      } catch (InterruptedException e) {
-        throw new SolrException(ErrorCode.SERVER_ERROR, "Interrupted");
-      }
+      shardId = doGetShardIdProcess(cd.getName(), cd.getCloudDescriptor());
       cd.getCloudDescriptor().setShardId(shardId);
     }
 
@@ -1076,6 +1073,50 @@ public final class ZkController {
     return leaderProps;
   }
   
+  public static void linkConfSet(SolrZkClient zkClient, String collection, String confSetName) throws KeeperException, InterruptedException {
+    String path = ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection;
+    if (log.isInfoEnabled()) {
+      log.info("Load collection config from:" + path);
+    }
+    byte[] data;
+    try {
+      data = zkClient.getData(path, null, null, true);
+    } catch (NoNodeException e) {
+      // if there is no node, we will try and create it
+      // first try to make in case we are pre configuring
+      ZkNodeProps props = new ZkNodeProps(CONFIGNAME_PROP, confSetName);
+      try {
+
+        zkClient.makePath(path, ZkStateReader.toJSON(props),
+            CreateMode.PERSISTENT, null, true);
+      } catch (KeeperException e2) {
+        // its okay if the node already exists
+        if (e2.code() != KeeperException.Code.NODEEXISTS) {
+          throw e;
+        }
+        // if we fail creating, setdata
+        // TODO: we should consider using version
+        zkClient.setData(path, ZkStateReader.toJSON(props), true);
+      }
+      return;
+    }
+    // we found existing data, let's update it
+    ZkNodeProps props = null;
+    if(data != null) {
+      props = ZkNodeProps.load(data);
+      Map<String,String> newProps = new HashMap<String,String>();
+      newProps.putAll(props.getProperties());
+      newProps.put(CONFIGNAME_PROP, confSetName);
+      props = new ZkNodeProps(newProps);
+    } else {
+      props = new ZkNodeProps(CONFIGNAME_PROP, confSetName);
+    }
+    
+    // TODO: we should consider using version
+    zkClient.setData(path, ZkStateReader.toJSON(props), true);
+
+  }
+  
   /**
    * If in SolrCloud mode, upload config sets for each SolrCore in solr.xml.
    * 
@@ -1090,19 +1131,29 @@ public final class ZkController {
 
     for (int i=0; i<nodes.getLength(); i++) {
       Node node = nodes.item(i);
-      String rawName = DOMUtil.getAttr(node, "name", null);
+      String rawName = DOMUtil.substituteProperty(DOMUtil.getAttr(node, "name", null), new Properties());
+
       String instanceDir = DOMUtil.getAttr(node, "instanceDir", null);
       File idir = new File(instanceDir);
       if (!idir.isAbsolute()) {
         idir = new File(solrHome, instanceDir);
       }
-      String confName = DOMUtil.getAttr(node, "collection", null);
+      String confName = DOMUtil.substituteProperty(DOMUtil.getAttr(node, "collection", null), new Properties());
       if (confName == null) {
         confName = rawName;
       }
-
-      ZkController.uploadConfigDir(zkClient, new File(idir, "conf"), confName);
+      File udir = new File(idir, "conf");
+      SolrException.log(log, "Uploading directory " + udir + " with name " + confName + " for SolrCore " + rawName);
+      ZkController.uploadConfigDir(zkClient, udir, confName);
     }
   }
 
+  public DistributedQueue getOverseerJobQueue() {
+    return overseerJobQueue;
+  }
+
+  public DistributedQueue getOverseerCollectionQueue() {
+    return overseerCollectionQueue;
+  }
+
 }

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/core/CachingDirectoryFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/core/CachingDirectoryFactory.java?rev=1363400&r1=1363399&r2=1363400&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/core/CachingDirectoryFactory.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/core/CachingDirectoryFactory.java Thu Jul 19 15:58:54 2012
@@ -19,7 +19,9 @@ package org.apache.solr.core;
 
 import java.io.File;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Locale;
 import java.util.Map;
 
@@ -54,6 +56,46 @@ public abstract class CachingDirectoryFa
   
   protected Map<Directory,CacheValue> byDirectoryCache = new HashMap<Directory,CacheValue>();
   
+  protected Map<Directory,List<CloseListener>> closeListeners = new HashMap<Directory,List<CloseListener>>();
+  
+  public interface CloseListener {
+    public void onClose();
+  }
+  
+  @Override
+  public void addCloseListener(Directory dir, CloseListener closeListener) {
+    synchronized (this) {
+      if (!byDirectoryCache.containsKey(dir)) {
+        throw new IllegalArgumentException("Unknown directory: " + dir
+            + " " + byDirectoryCache);
+      }
+      List<CloseListener> listeners = closeListeners.get(dir);
+      if (listeners == null) {
+        listeners = new ArrayList<CloseListener>();
+        closeListeners.put(dir, listeners);
+      }
+      listeners.add(closeListener);
+      
+      closeListeners.put(dir, listeners);
+    }
+  }
+  
+  @Override
+  public void doneWithDirectory(Directory directory) throws IOException {
+    synchronized (this) {
+      CacheValue cacheValue = byDirectoryCache.get(directory);
+      if (cacheValue == null) {
+        throw new IllegalArgumentException("Unknown directory: " + directory
+            + " " + byDirectoryCache);
+      }
+      cacheValue.doneWithDir = true;
+      if (cacheValue.refCnt == 0) {
+        cacheValue.refCnt++; // this will go back to 0 in close
+        close(directory);
+      }
+    }
+  }
+  
   /*
    * (non-Javadoc)
    * 
@@ -82,6 +124,13 @@ public abstract class CachingDirectoryFa
         directory.close();
         byDirectoryCache.remove(directory);
         byPathCache.remove(cacheValue.path);
+        List<CloseListener> listeners = closeListeners.remove(directory);
+        if (listeners != null) {
+          for (CloseListener listener : listeners) {
+            listener.onClose();
+          }
+          closeListeners.remove(directory);
+        }
       }
     }
   }
@@ -198,7 +247,7 @@ public abstract class CachingDirectoryFa
       log.warn("No lockType configured for " + dir + " assuming 'simple'");
       rawLockType = "simple";
     }
-    final String lockType = rawLockType.toLowerCase(Locale.ENGLISH).trim();
+    final String lockType = rawLockType.toLowerCase(Locale.ROOT).trim();
     
     if ("simple".equals(lockType)) {
       // multiple SimpleFSLockFactory instances should be OK

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/core/CodecFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/core/CodecFactory.java?rev=1363400&r1=1363399&r2=1363400&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/core/CodecFactory.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/core/CodecFactory.java Thu Jul 19 15:58:54 2012
@@ -19,7 +19,6 @@ package org.apache.solr.core;
 
 import org.apache.lucene.codecs.Codec;
 import org.apache.solr.common.util.NamedList;
-import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.util.plugin.NamedListInitializedPlugin;
 
 /**
@@ -29,5 +28,5 @@ public abstract class CodecFactory imple
   public void init(NamedList args) {  
   }
   
-  public abstract Codec create(IndexSchema Schema);
+  public abstract Codec getCodec();
 }

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/core/CoreContainer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/core/CoreContainer.java?rev=1363400&r1=1363399&r2=1363400&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/core/CoreContainer.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/core/CoreContainer.java Thu Jul 19 15:58:54 2012
@@ -62,6 +62,7 @@ import org.apache.solr.util.FileUtils;
 import org.apache.solr.util.SystemIdResolver;
 import org.apache.solr.core.SolrXMLSerializer.SolrCoreXMLDef;
 import org.apache.solr.core.SolrXMLSerializer.SolrXMLDef;
+import org.apache.solr.handler.admin.CollectionsHandler;
 import org.apache.solr.handler.admin.CoreAdminHandler;
 import org.apache.solr.handler.component.HttpShardHandlerFactory;
 import org.apache.solr.handler.component.ShardHandlerFactory;
@@ -89,12 +90,24 @@ public class CoreContainer 
 {
   private static final String DEFAULT_HOST_CONTEXT = "solr";
   private static final String DEFAULT_HOST_PORT = "8983";
-  private static final int DEFAULT_ZK_CLIENT_TIMEOUT = 10000;
+  private static final int DEFAULT_ZK_CLIENT_TIMEOUT = 15000;
   public static final String DEFAULT_DEFAULT_CORE_NAME = "collection1";
   private static final boolean DEFAULT_SHARE_SCHEMA = false;
   
   protected static Logger log = LoggerFactory.getLogger(CoreContainer.class);
   
+  // solr.xml node constants
+  private static final String CORE_NAME = "name";
+  private static final String CORE_CONFIG = "config";
+  private static final String CORE_INSTDIR = "instanceDir";
+  private static final String CORE_DATADIR = "dataDir";
+  private static final String CORE_SCHEMA = "schema";
+  private static final String CORE_SHARD = "shard";
+  private static final String CORE_COLLECTION = "collection";
+  private static final String CORE_ROLES = "roles";
+  private static final String CORE_PROPERTIES = "properties";
+
+
   protected final Map<String, SolrCore> cores = new LinkedHashMap<String, SolrCore>();
   protected boolean persistent = false;
   protected String adminPath = null;
@@ -103,6 +116,7 @@ public class CoreContainer 
   protected String hostContext;
   protected String host;
   protected CoreAdminHandler coreAdminHandler = null;
+  protected CollectionsHandler collectionsHandler = null;
   protected File configFile = null;
   protected String libDir = null;
   protected ClassLoader libLoader = null;
@@ -219,9 +233,17 @@ public class CoreContainer 
             }
             return descriptors;
           }
-        });
-        
+        });        
+
         String confDir = System.getProperty("bootstrap_confdir");
+        boolean boostrapConf = Boolean.getBoolean("bootstrap_conf");
+        
+        if (zkRun != null && zkServer.getServers().size() > 1 && confDir == null && boostrapConf == false) {
+          // we are part of an ensemble and we are not uploading the config - pause to give the config time
+          // to get up
+          Thread.sleep(10000);
+        }
+        
         if(confDir != null) {
           File dir = new File(confDir);
           if(!dir.isDirectory()) {
@@ -230,8 +252,9 @@ public class CoreContainer 
           String confName = System.getProperty(ZkController.COLLECTION_PARAM_PREFIX+ZkController.CONFIGNAME_PROP, "configuration1");
           zkController.uploadConfigDir(dir, confName);
         }
+
+
         
-        boolean boostrapConf = Boolean.getBoolean("bootstrap_conf");
         if(boostrapConf) {
           ZkController.bootstrapConf(zkController.getZkClient(), cfg, solrHome);
         }
@@ -453,6 +476,8 @@ public class CoreContainer 
       }
     }
 
+    collectionsHandler = new CollectionsHandler(this);
+    
     try {
       containerProperties = readProperties(cfg, ((NodeList) cfg.evaluate(DEFAULT_HOST_CONTEXT, XPathConstants.NODESET)).item(0));
     } catch (Throwable e) {
@@ -464,43 +489,43 @@ public class CoreContainer 
     for (int i=0; i<nodes.getLength(); i++) {
       Node node = nodes.item(i);
       try {
-        String rawName = DOMUtil.getAttr(node, "name", null);
+        String rawName = DOMUtil.getAttr(node, CORE_NAME, null);
         if (null == rawName) {
           throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
                                   "Each core in solr.xml must have a 'name'");
         }
         String name = rawName;
-        CoreDescriptor p = new CoreDescriptor(this, name, DOMUtil.getAttr(node, "instanceDir", null));
+        CoreDescriptor p = new CoreDescriptor(this, name, DOMUtil.getAttr(node, CORE_INSTDIR, null));
 
         // deal with optional settings
-        String opt = DOMUtil.getAttr(node, "config", null);
+        String opt = DOMUtil.getAttr(node, CORE_CONFIG, null);
 
         if (opt != null) {
           p.setConfigName(opt);
         }
-        opt = DOMUtil.getAttr(node, "schema", null);
+        opt = DOMUtil.getAttr(node, CORE_SCHEMA, null);
         if (opt != null) {
           p.setSchemaName(opt);
         }
         if (zkController != null) {
-          opt = DOMUtil.getAttr(node, "shard", null);
+          opt = DOMUtil.getAttr(node, CORE_SHARD, null);
           if (opt != null && opt.length() > 0) {
             p.getCloudDescriptor().setShardId(opt);
           }
-          opt = DOMUtil.getAttr(node, "collection", null);
+          opt = DOMUtil.getAttr(node, CORE_COLLECTION, null);
           if (opt != null) {
             p.getCloudDescriptor().setCollectionName(opt);
           }
-          opt = DOMUtil.getAttr(node, "roles", null);
+          opt = DOMUtil.getAttr(node, CORE_ROLES, null);
           if(opt != null){
         	  p.getCloudDescriptor().setRoles(opt);
           }
         }
-        opt = DOMUtil.getAttr(node, "properties", null);
+        opt = DOMUtil.getAttr(node, CORE_PROPERTIES, null);
         if (opt != null) {
           p.setPropertiesName(opt);
         }
-        opt = DOMUtil.getAttr(node, CoreAdminParams.DATA_DIR, null);
+        opt = DOMUtil.getAttr(node, CORE_DATADIR, null);
         if (opt != null) {
           p.setDataDir(opt);
         }
@@ -552,6 +577,8 @@ public class CoreContainer 
    */
   public void shutdown() {
     log.info("Shutting down CoreContainer instance="+System.identityHashCode(this));
+    isShutDown = true;
+    
     if (isZooKeeperAware()) {
       cancelCoreRecoveries();
     }
@@ -576,7 +603,6 @@ public class CoreContainer 
         if (shardHandlerFactory != null) {
           shardHandlerFactory.close();
         }
-        isShutDown = true;
       }
     }
   }
@@ -777,7 +803,7 @@ public class CoreContainer 
         if (schemaFile.exists()) {
           String key = schemaFile.getAbsolutePath()
               + ":"
-              + new SimpleDateFormat("yyyyMMddHHmmss", Locale.US).format(new Date(
+              + new SimpleDateFormat("yyyyMMddHHmmss", Locale.ROOT).format(new Date(
                   schemaFile.lastModified()));
           schema = indexSchemaCache.get(key);
           if (schema == null) {
@@ -1023,6 +1049,10 @@ public class CoreContainer 
     return coreAdminHandler;
   }
   
+  public CollectionsHandler getCollectionsHandler() {
+    return collectionsHandler;
+  }
+  
   /**
    * the default core name, or null if there is no default core name
    */
@@ -1136,7 +1166,7 @@ public class CoreContainer 
           for (int i = 0; i < nodes.getLength(); i++) {
             Node node = nodes.item(i);
             
-            String name = DOMUtil.getAttr(node, "name", null);
+            String name = DOMUtil.getAttr(node, CORE_NAME, null);
             if (origCoreName.equals(name)) {
               coreNode = node;
               if (coreName.equals(origCoreName)) {
@@ -1150,7 +1180,7 @@ public class CoreContainer 
             // see if we match with substitution
             for (int i = 0; i < nodes.getLength(); i++) {
               Node node = nodes.item(i);
-              String name = DOMUtil.getAttr(node, "name", null);
+              String name = DOMUtil.getAttr(node, CORE_NAME, null);
               if (origCoreName.equals(DOMUtil.substituteProperty(name,
                   loader.getCoreProperties()))) {
                 coreNode = node;
@@ -1163,28 +1193,28 @@ public class CoreContainer 
           }
         }
 
-        coreAttribs.put("name", coreName);
+        coreAttribs.put(CORE_NAME, coreName);
         
         String instanceDir = dcore.getInstanceDir();
-        addCoreProperty(coreAttribs, coreNode, "instanceDir", instanceDir, null);
+        addCoreProperty(coreAttribs, coreNode, CORE_INSTDIR, instanceDir, null);
         
         // write config 
         String configName = dcore.getConfigName();
-        addCoreProperty(coreAttribs, coreNode, "conf", configName, dcore.getDefaultConfigName());
+        addCoreProperty(coreAttribs, coreNode, CORE_CONFIG, configName, dcore.getDefaultConfigName());
         
         // write schema
         String schema = dcore.getSchemaName();
-        addCoreProperty(coreAttribs, coreNode, "schema", schema, dcore.getDefaultSchemaName());
+        addCoreProperty(coreAttribs, coreNode, CORE_SCHEMA, schema, dcore.getDefaultSchemaName());
         
         String dataDir = dcore.dataDir;
-        addCoreProperty(coreAttribs, coreNode, "dataDir", dataDir, null);
+        addCoreProperty(coreAttribs, coreNode, CORE_DATADIR, dataDir, null);
         
         CloudDescriptor cd = dcore.getCloudDescriptor();
         String shard = null;
         if (cd != null) {
           shard = cd.getShardId();
         }
-        addCoreProperty(coreAttribs, coreNode, "shard", shard, null);
+        addCoreProperty(coreAttribs, coreNode, CORE_SHARD, shard, null);
         
         String collection = null;
         // only write out the collection name if it's not the default (the
@@ -1194,12 +1224,12 @@ public class CoreContainer 
           collection = cd.getCollectionName();
         }
         
-        addCoreProperty(coreAttribs, coreNode, "collection", collection, dcore.name);
+        addCoreProperty(coreAttribs, coreNode, CORE_COLLECTION, collection, dcore.name);
         
         // we don't try and preserve sys prop defs in these
         String opt = dcore.getPropertiesName();
         if (opt != null) {
-          coreAttribs.put("properties", opt);
+          coreAttribs.put(CORE_PROPERTIES, opt);
         }
         
         SolrCoreXMLDef solrCoreXMLDef = new SolrCoreXMLDef();
@@ -1321,7 +1351,7 @@ public class CoreContainer 
   private static final String DEF_SOLR_XML ="<?xml version=\"1.0\" encoding=\"UTF-8\" ?>\n" +
           "<solr persistent=\"false\">\n" +
           "  <cores adminPath=\"/admin/cores\" defaultCoreName=\"" + DEFAULT_DEFAULT_CORE_NAME + "\">\n" +
-          "    <core name=\""+ DEFAULT_DEFAULT_CORE_NAME + "\" shard=\"${shard:}\" instanceDir=\".\" />\n" +
+          "    <core name=\""+ DEFAULT_DEFAULT_CORE_NAME + "\" shard=\"${shard:}\" instanceDir=\"collection1\" />\n" +
           "  </cores>\n" +
           "</solr>";
 }

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java?rev=1363400&r1=1363399&r2=1363400&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java Thu Jul 19 15:58:54 2012
@@ -21,6 +21,7 @@ import java.io.Closeable;
 import java.io.IOException;
 
 import org.apache.lucene.store.Directory;
+import org.apache.solr.core.CachingDirectoryFactory.CloseListener;
 import org.apache.solr.util.plugin.NamedListInitializedPlugin;
 
 /**
@@ -31,6 +32,24 @@ public abstract class DirectoryFactory i
     Closeable {
   
   /**
+   * Indicates a Directory will no longer be used, and when it's ref count
+   * hits 0, it can be closed. On shutdown all directories will be closed
+   * with this has been called or not. This is simply to allow early cleanup.
+   * 
+   * @param directory
+   * @throws IOException 
+   */
+  public abstract void doneWithDirectory(Directory directory) throws IOException;
+  
+  /**
+   * Adds a close listener for a Directory.
+   * 
+   * @param dir
+   * @param closeListener
+   */
+  public abstract void addCloseListener(Directory dir, CloseListener closeListener);
+  
+  /**
    * Close the this and all of the Directories it contains.
    * 
    * @throws IOException
@@ -62,7 +81,9 @@ public abstract class DirectoryFactory i
   /**
    * Returns the Directory for a given path, using the specified rawLockType.
    * Will return the same Directory instance for the same path unless forceNew,
-   * in which case a new Directory is returned.
+   * in which case a new Directory is returned. There is no need to call
+   * {@link #doneWithDirectory(Directory)} in this case - the old Directory
+   * will be closed when it's ref count hits 0.
    * 
    * @throws IOException
    */

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/core/JmxMonitoredMap.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/core/JmxMonitoredMap.java?rev=1363400&r1=1363399&r2=1363400&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/core/JmxMonitoredMap.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/core/JmxMonitoredMap.java Thu Jul 19 15:58:54 2012
@@ -281,7 +281,7 @@ public class JmxMonitoredMap<K, V> exten
       } else if (staticStats.contains(attribute) && attribute != null
               && attribute.length() > 0) {
         try {
-          String getter = "get" + attribute.substring(0, 1).toUpperCase(Locale.ENGLISH)
+          String getter = "get" + attribute.substring(0, 1).toUpperCase(Locale.ROOT)
                   + attribute.substring(1);
           Method meth = infoBean.getClass().getMethod(getter);
           val = meth.invoke(infoBean);

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/core/SolrConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/core/SolrConfig.java?rev=1363400&r1=1363399&r2=1363400&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/core/SolrConfig.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/core/SolrConfig.java Thu Jul 19 15:58:54 2012
@@ -112,7 +112,7 @@ public class SolrConfig extends Config {
    *@param name the configuration name
    *@param is the configuration stream
    */
-  SolrConfig(SolrResourceLoader loader, String name, InputSource is)
+  public SolrConfig(SolrResourceLoader loader, String name, InputSource is)
   throws ParserConfigurationException, IOException, SAXException {
     super(loader, name, is, "/config/");
     initLibs();
@@ -221,7 +221,7 @@ public class SolrConfig extends Config {
 
      loadPluginInfo(DirectoryFactory.class,"directoryFactory",false, true);
      loadPluginInfo(IndexDeletionPolicy.class,indexConfigPrefix+"/deletionPolicy",false, true);
-     loadPluginInfo(CodecFactory.class,"mainIndex/codecFactory",false, false);
+     loadPluginInfo(CodecFactory.class,"codecFactory",false, false);
      loadPluginInfo(IndexReaderFactory.class,"indexReaderFactory",false, true);
      loadPluginInfo(UpdateRequestProcessorChain.class,"updateRequestProcessorChain",false, false);
      loadPluginInfo(UpdateLog.class,"updateHandler/updateLog",false, false);
@@ -344,7 +344,7 @@ public class SolrConfig extends Config {
       /** Input must not be null */
       public static LastModFrom parse(final String s) {
         try {
-          return valueOf(s.toUpperCase(Locale.ENGLISH));
+          return valueOf(s.toUpperCase(Locale.ROOT));
         } catch (Exception e) {
           log.warn( "Unrecognized value for lastModFrom: " + s, e);
           return BOGUS;

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/core/SolrCore.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/core/SolrCore.java?rev=1363400&r1=1363399&r2=1363400&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/core/SolrCore.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/core/SolrCore.java Thu Jul 19 15:58:54 2012
@@ -25,20 +25,39 @@ import org.apache.lucene.index.IndexWrit
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.LockObtainFailedException;
+import org.apache.solr.cloud.CloudDescriptor;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.CommonParams.EchoParamStyle;
-import org.apache.solr.common.params.ShardParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.handler.admin.ShowFileRequestHandler;
-import org.apache.solr.handler.component.*;
+import org.apache.solr.handler.component.DebugComponent;
+import org.apache.solr.handler.component.FacetComponent;
+import org.apache.solr.handler.component.HighlightComponent;
+import org.apache.solr.handler.component.MoreLikeThisComponent;
+import org.apache.solr.handler.component.QueryComponent;
+import org.apache.solr.handler.component.RealTimeGetComponent;
+import org.apache.solr.handler.component.SearchComponent;
+import org.apache.solr.handler.component.StatsComponent;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrRequestHandler;
-import org.apache.solr.response.*;
+import org.apache.solr.response.BinaryResponseWriter;
+import org.apache.solr.response.CSVResponseWriter;
+import org.apache.solr.response.JSONResponseWriter;
+import org.apache.solr.response.PHPResponseWriter;
+import org.apache.solr.response.PHPSerializedResponseWriter;
+import org.apache.solr.response.PythonResponseWriter;
+import org.apache.solr.response.QueryResponseWriter;
+import org.apache.solr.response.RawResponseWriter;
+import org.apache.solr.response.RubyResponseWriter;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.response.XMLResponseWriter;
 import org.apache.solr.response.transform.TransformerFactory;
+import org.apache.solr.schema.FieldType;
 import org.apache.solr.schema.IndexSchema;
+import org.apache.solr.schema.SchemaAware;
 import org.apache.solr.search.QParserPlugin;
 import org.apache.solr.search.SolrFieldCacheMBean;
 import org.apache.solr.search.SolrIndexSearcher;
@@ -46,7 +65,11 @@ import org.apache.solr.search.ValueSourc
 import org.apache.solr.update.DirectUpdateHandler2;
 import org.apache.solr.update.SolrIndexWriter;
 import org.apache.solr.update.UpdateHandler;
-import org.apache.solr.update.processor.*;
+import org.apache.solr.update.processor.DistributedUpdateProcessorFactory;
+import org.apache.solr.update.processor.LogUpdateProcessorFactory;
+import org.apache.solr.update.processor.RunUpdateProcessorFactory;
+import org.apache.solr.update.processor.UpdateRequestProcessorChain;
+import org.apache.solr.update.processor.UpdateRequestProcessorFactory;
 import org.apache.solr.util.RefCounted;
 import org.apache.solr.util.plugin.NamedListInitializedPlugin;
 import org.apache.solr.util.plugin.PluginInfoInitialized;
@@ -56,11 +79,35 @@ import org.slf4j.LoggerFactory;
 import org.xml.sax.SAXException;
 
 import javax.xml.parsers.ParserConfigurationException;
-import java.io.*;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.Writer;
 import java.lang.reflect.Constructor;
 import java.net.URL;
-import java.util.*;
-import java.util.concurrent.*;
+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.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.ReentrantLock;
@@ -75,7 +122,7 @@ public final class SolrCore implements S
   // These should *only* be used for debugging or monitoring purposes
   public static final AtomicLong numOpens = new AtomicLong();
   public static final AtomicLong numCloses = new AtomicLong();
-  public static Map<SolrCore,Exception> openHandles = Collections.synchronizedMap(new IdentityHashMap<SolrCore,Exception>());
+  public static Map<SolrCore,Exception> openHandles = Collections.synchronizedMap(new IdentityHashMap<SolrCore, Exception>());
 
   
   public static Logger log = LoggerFactory.getLogger(SolrCore.class);
@@ -340,7 +387,7 @@ public final class SolrCore implements S
       dirFactory = getResourceLoader().newInstance(info.className, DirectoryFactory.class);
       dirFactory.init(info.initArgs);
     } else {
-      dirFactory = new StandardDirectoryFactory();
+      dirFactory = new NRTCachingDirectoryFactory();
     }
     // And set it
     directoryFactory = dirFactory;
@@ -617,7 +664,8 @@ public final class SolrCore implements S
       latch.countDown();//release the latch, otherwise we block trying to do the close.  This should be fine, since counting down on a latch of 0 is still fine
       //close down the searcher and any other resources, if it exists, as this is not recoverable
       close();
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, null, e);
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, 
+                              e.getMessage(), e);
     } finally {
       // allow firstSearcher events to fire and make sure it is released
       latch.countDown();
@@ -645,9 +693,25 @@ public final class SolrCore implements S
       factory = schema.getResourceLoader().newInstance(info.className, CodecFactory.class);
       factory.init(info.initArgs);
     } else {
-      factory = new DefaultCodecFactory();
+      factory = new CodecFactory() {
+        @Override
+        public Codec getCodec() {
+          return Codec.getDefault();
+        }
+      };
     }
-    return factory.create(schema);
+    if (factory instanceof SchemaAware) {
+      ((SchemaAware)factory).inform(schema);
+    } else {
+      for (FieldType ft : schema.getFieldTypes().values()) {
+        if (null != ft.getPostingsFormat()) {
+          String msg = "FieldType '" + ft.getTypeName() + "' is configured with a postings format, but the codec does not support it: " + factory.getClass();
+          log.error(msg);
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, msg);
+        }
+      }
+    }
+    return factory.getCodec();
   }
 
   /**
@@ -837,13 +901,14 @@ public final class SolrCore implements S
   public static void verbose(Object... args) {
     if (!VERBOSE) return;
     StringBuilder sb = new StringBuilder("VERBOSE:");
-    sb.append(Thread.currentThread().getName());
-    sb.append(':');
+//    sb.append(Thread.currentThread().getName());
+//    sb.append(':');
     for (Object o : args) {
       sb.append(' ');
       sb.append(o==null ? "(null)" : o.toString());
     }
-    System.out.println(sb.toString());
+    // System.out.println(sb.toString());
+    log.info(sb.toString());
   }
 
 
@@ -1003,12 +1068,7 @@ public final class SolrCore implements S
    * {@link org.apache.solr.request.SolrQueryRequest#getSearcher()} instead.
   */
   public RefCounted<SolrIndexSearcher> getSearcher() {
-    try {
-      return getSearcher(false,true,null);
-    } catch (IOException e) {
-      SolrException.log(log,null,e);
-      return null;
-    }
+    return getSearcher(false,true,null);
   }
 
   /**
@@ -1072,7 +1132,7 @@ public final class SolrCore implements S
   }
 
 
-  public RefCounted<SolrIndexSearcher> getSearcher(boolean forceNew, boolean returnSearcher, final Future[] waitSearcher) throws IOException {
+  public RefCounted<SolrIndexSearcher> getSearcher(boolean forceNew, boolean returnSearcher, final Future[] waitSearcher) {
     return getSearcher(forceNew, returnSearcher, waitSearcher, false);
   }
 
@@ -1120,8 +1180,12 @@ public final class SolrCore implements S
 
         if (updateHandlerReopens) {
           // SolrCore.verbose("start reopen from",previousSearcher,"writer=",writer);
-          IndexWriter writer = getUpdateHandler().getSolrCoreState().getIndexWriter(this);
-          newReader = DirectoryReader.openIfChanged(currentReader, writer, true);
+          RefCounted<IndexWriter> writer = getUpdateHandler().getSolrCoreState().getIndexWriter(this);
+          try {
+            newReader = DirectoryReader.openIfChanged(currentReader, writer.get(), true);
+          } finally {
+            writer.decref();
+          }
 
         } else {
           // verbose("start reopen without writer, reader=", currentReader);
@@ -1217,9 +1281,8 @@ public final class SolrCore implements S
    * @param returnSearcher       if true, returns a {@link SolrIndexSearcher} holder with the refcount already incremented.
    * @param waitSearcher         if non-null, will be filled in with a {@link Future} that will return after the new searcher is registered.
    * @param updateHandlerReopens if true, the UpdateHandler will be used when reopening a {@link SolrIndexSearcher}.
-   * @throws IOException
    */
-  public RefCounted<SolrIndexSearcher> getSearcher(boolean forceNew, boolean returnSearcher, final Future[] waitSearcher, boolean updateHandlerReopens) throws IOException {
+  public RefCounted<SolrIndexSearcher> getSearcher(boolean forceNew, boolean returnSearcher, final Future[] waitSearcher, boolean updateHandlerReopens) {
     // it may take some time to open an index.... we may need to make
     // sure that two threads aren't trying to open one at the same time
     // if it isn't necessary.
@@ -1476,7 +1539,7 @@ public final class SolrCore implements S
   //
   // onDeckSearchers will also be decremented (it should have been incremented
   // as a result of opening a new searcher).
-  private void registerSearcher(RefCounted<SolrIndexSearcher> newSearcherHolder) throws IOException {
+  private void registerSearcher(RefCounted<SolrIndexSearcher> newSearcherHolder) {
     synchronized (searcherLock) {
       try {
         if (_searcher != null) {
@@ -1753,7 +1816,7 @@ public final class SolrCore implements S
   public QParserPlugin getQueryPlugin(String parserName) {
     QParserPlugin plugin = qParserPlugins.get(parserName);
     if (plugin != null) return plugin;
-    throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown query type '"+parserName+"'");
+    throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown query parser '"+parserName+"'");
   }
   
   private final HashMap<String, ValueSourceParser> valueSourceParsers = new HashMap<String, ValueSourceParser>();
@@ -1873,13 +1936,13 @@ public final class SolrCore implements S
         Set<String> hide = new HashSet<String>();
 
         for (String file : solrConfig.getResourceLoader().listConfigDir()) {
-          hide.add(file.toUpperCase(Locale.ENGLISH));
+          hide.add(file.toUpperCase(Locale.ROOT));
         }    
         
         // except the "gettable" list
         StringTokenizer st = new StringTokenizer( gettable );
         while( st.hasMoreTokens() ) {
-          hide.remove( st.nextToken().toUpperCase(Locale.ENGLISH) );
+          hide.remove( st.nextToken().toUpperCase(Locale.ROOT) );
         }
         for( String s : hide ) {
           invariants.add( ShowFileRequestHandler.HIDDEN, s );
@@ -1941,9 +2004,26 @@ public final class SolrCore implements S
     lst.add("startTime", new Date(startTime));
     lst.add("refCount", getOpenCount());
 
-    if (null != getCoreDescriptor() && null != getCoreDescriptor().getCoreContainer()) {
-      lst.add("aliases", getCoreDescriptor().getCoreContainer().getCoreNames(this));
+    CoreDescriptor cd = getCoreDescriptor();
+    if (cd != null) {
+      if (null != cd && cd.getCoreContainer() != null) {
+        lst.add("aliases", getCoreDescriptor().getCoreContainer().getCoreNames(this));
+      }
+      CloudDescriptor cloudDesc = cd.getCloudDescriptor();
+      if (cloudDesc != null) {
+        String collection = cloudDesc.getCollectionName();
+        if (collection == null) {
+          collection = "_notset_";
+        }
+        lst.add("collection", collection);
+        String shard = cloudDesc.getShardId();
+        if (shard == null) {
+          shard = "_auto_";
+        }
+        lst.add("shard", shard);
+      }
     }
+    
     return lst;
   }
   

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/core/SolrDeletionPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/core/SolrDeletionPolicy.java?rev=1363400&r1=1363399&r2=1363400&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/core/SolrDeletionPolicy.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/core/SolrDeletionPolicy.java Thu Jul 19 15:58:54 2012
@@ -145,7 +145,7 @@ public class SolrDeletionPolicy implemen
         try {
           if (maxCommitAge != null) {
             if (maxCommitAgeTimeStamp==-1) {
-              DateMathParser dmp = new DateMathParser(DateField.UTC, Locale.US);
+              DateMathParser dmp = new DateMathParser(DateField.UTC, Locale.ROOT);
               maxCommitAgeTimeStamp = dmp.parseMath(maxCommitAge).getTime();
             }
             if (IndexDeletionPolicyWrapper.getCommitTimestamp(commit) < maxCommitAgeTimeStamp) {

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/handler/AnalysisRequestHandlerBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/handler/AnalysisRequestHandlerBase.java?rev=1363400&r1=1363399&r2=1363400&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/handler/AnalysisRequestHandlerBase.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/handler/AnalysisRequestHandlerBase.java Thu Jul 19 15:58:54 2012
@@ -18,8 +18,6 @@
 package org.apache.solr.handler;
 
 import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.analysis.CharReader;
-import org.apache.lucene.analysis.CharStream;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.*;
 import org.apache.lucene.analysis.util.CharFilterFactory;
@@ -41,6 +39,7 @@ import org.apache.solr.response.SolrQuer
 import org.apache.solr.schema.FieldType;
 
 import java.io.IOException;
+import java.io.Reader;
 import java.io.StringReader;
 import java.util.*;
 import org.apache.commons.lang.ArrayUtils;
@@ -106,7 +105,7 @@ public abstract class AnalysisRequestHan
     if( cfiltfacs != null ){
       String source = value;
       for(CharFilterFactory cfiltfac : cfiltfacs ){
-        CharStream reader = CharReader.get(new StringReader(source));
+        Reader reader = new StringReader(source);
         reader = cfiltfac.create(reader);
         source = writeCharStream(namedList, reader);
       }
@@ -287,7 +286,7 @@ public abstract class AnalysisRequestHan
     return tokensNamedLists;
   }
   
-  private String writeCharStream(NamedList<Object> out, CharStream input ){
+  private String writeCharStream(NamedList<Object> out, Reader input ){
     final int BUFFER_SIZE = 1024;
     char[] buf = new char[BUFFER_SIZE];
     int len = 0;
@@ -326,7 +325,7 @@ public abstract class AnalysisRequestHan
     }
 
     @Override
-    public boolean incrementToken() throws IOException {
+    public boolean incrementToken() {
       if (tokenIterator.hasNext()) {
         clearAttributes();
         AttributeSource next = tokenIterator.next();

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/handler/PingRequestHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/handler/PingRequestHandler.java?rev=1363400&r1=1363399&r2=1363400&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/handler/PingRequestHandler.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/handler/PingRequestHandler.java Thu Jul 19 15:58:54 2012
@@ -191,7 +191,7 @@ public class PingRequestHandler extends 
     }
     else {
       try {
-        action = ACTIONS.valueOf(actionParam.toUpperCase(Locale.ENGLISH));
+        action = ACTIONS.valueOf(actionParam.toUpperCase(Locale.ROOT));
       }
       catch (IllegalArgumentException iae){
         throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, 

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java?rev=1363400&r1=1363399&r2=1363400&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java Thu Jul 19 15:58:54 2012
@@ -42,7 +42,6 @@ import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.IOUtils;
 import org.apache.lucene.index.IndexCommit;
 import org.apache.lucene.index.IndexDeletionPolicy;
-import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
@@ -602,10 +601,7 @@ public class ReplicationHandler extends 
     if (showSlaveDetails && snapPuller != null) {
       Properties props = loadReplicationProperties();
       try {
-        NamedList<String> command = new NamedList<String>();
-        command.add(COMMAND, CMD_DETAILS);
-        command.add("slave", "false");
-        NamedList nl = snapPuller.getCommandResponse(command);
+        NamedList nl = snapPuller.getDetails();
         slave.add("masterDetails", nl.get(CMD_DETAILS));
       } catch (Exception e) {
         LOG.warn("Exception while invoking 'details' method for replication on master ", e);
@@ -946,7 +942,7 @@ public class ReplicationHandler extends 
         stream.write(out);
       }
 
-      public void write(Writer writer, SolrQueryRequest request, SolrQueryResponse response) throws IOException {
+      public void write(Writer writer, SolrQueryRequest request, SolrQueryResponse response) {
         throw new RuntimeException("This is a binary writer , Cannot write to a characterstream");
       }
 

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/handler/RequestHandlerUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/handler/RequestHandlerUtils.java?rev=1363400&r1=1363399&r2=1363400&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/handler/RequestHandlerUtils.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/handler/RequestHandlerUtils.java Thu Jul 19 15:58:54 2012
@@ -90,8 +90,7 @@ public class RequestHandlerUtils
   /**
    * Modify UpdateCommand based on request parameters
    */
-  public static void updateCommit(CommitUpdateCommand cmd, SolrParams params) throws IOException
-  {
+  public static void updateCommit(CommitUpdateCommand cmd, SolrParams params) {
     if( params == null ) return;
 
     cmd.openSearcher = params.getBool( UpdateParams.OPEN_SEARCHER, cmd.openSearcher );