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 2014/11/30 12:27:23 UTC

svn commit: r1642537 [6/6] - in /lucene/dev/branches/lucene6005: ./ dev-tools/ dev-tools/idea/.idea/ dev-tools/maven/solr/webapp/ lucene/ lucene/analysis/ lucene/analysis/common/ lucene/analysis/common/src/java/org/apache/lucene/analysis/br/ lucene/ana...

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java?rev=1642537&r1=1642536&r2=1642537&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java Sun Nov 30 11:27:18 2014
@@ -68,6 +68,8 @@ import org.apache.solr.client.solrj.resp
 import org.apache.solr.cloud.Assign.Node;
 import org.apache.solr.cloud.DistributedQueue.QueueEvent;
 import org.apache.solr.cloud.Overseer.LeaderStatus;
+import org.apache.solr.cloud.overseer.ClusterStateMutator;
+import org.apache.solr.cloud.overseer.OverseerAction;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.Aliases;
@@ -467,7 +469,7 @@ public class OverseerCollectionProcessor
     }
     //now ask the current leader to QUIT , so that the designate can takeover
     Overseer.getInQueue(zkStateReader.getZkClient()).offer(
-        ZkStateReader.toJSON(new ZkNodeProps(Overseer.QUEUE_OPERATION, Overseer.OverseerAction.QUIT.toLower(),
+        ZkStateReader.toJSON(new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.QUIT.toLower(),
             "id",getLeaderId(zkStateReader.getZkClient()))));
 
   }
@@ -698,7 +700,7 @@ public class OverseerCollectionProcessor
     SolrZkClient zkClient = zkStateReader.getZkClient();
     DistributedQueue inQueue = Overseer.getInQueue(zkClient);
     Map<String, Object> propMap = new HashMap<>();
-    propMap.put(Overseer.QUEUE_OPERATION, Overseer.OverseerAction.LEADER.toLower());
+    propMap.put(Overseer.QUEUE_OPERATION, OverseerAction.LEADER.toLower());
     propMap.put(COLLECTION_PROP, collectionName);
     propMap.put(SHARD_ID_PROP, shardId);
     propMap.put(BASE_URL_PROP, baseURL);
@@ -1148,7 +1150,7 @@ public class OverseerCollectionProcessor
 
   private void deleteCoreNode(String collectionName, String replicaName, Replica replica, String core) throws KeeperException, InterruptedException {
     ZkNodeProps m = new ZkNodeProps(
-        Overseer.QUEUE_OPERATION, Overseer.OverseerAction.DELETECORE.toLower(),
+        Overseer.QUEUE_OPERATION, OverseerAction.DELETECORE.toLower(),
         ZkStateReader.CORE_NAME_PROP, core,
         ZkStateReader.NODE_NAME_PROP, replica.getStr(ZkStateReader.NODE_NAME_PROP),
         ZkStateReader.COLLECTION_PROP, collectionName,
@@ -1760,7 +1762,7 @@ public class OverseerCollectionProcessor
         log.info("Replication factor is 1 so switching shard states");
         DistributedQueue inQueue = Overseer.getInQueue(zkStateReader.getZkClient());
         Map<String, Object> propMap = new HashMap<>();
-        propMap.put(Overseer.QUEUE_OPERATION, Overseer.OverseerAction.UPDATESHARDSTATE.toLower());
+        propMap.put(Overseer.QUEUE_OPERATION, OverseerAction.UPDATESHARDSTATE.toLower());
         propMap.put(slice, Slice.INACTIVE);
         for (String subSlice : subSlices) {
           propMap.put(subSlice, Slice.ACTIVE);
@@ -1772,7 +1774,7 @@ public class OverseerCollectionProcessor
         log.info("Requesting shard state be set to 'recovery'");
         DistributedQueue inQueue = Overseer.getInQueue(zkStateReader.getZkClient());
         Map<String, Object> propMap = new HashMap<>();
-        propMap.put(Overseer.QUEUE_OPERATION, Overseer.OverseerAction.UPDATESHARDSTATE.toLower());
+        propMap.put(Overseer.QUEUE_OPERATION, OverseerAction.UPDATESHARDSTATE.toLower());
         for (String subSlice : subSlices) {
           propMap.put(subSlice, Slice.RECOVERY);
         }
@@ -2062,7 +2064,7 @@ public class OverseerCollectionProcessor
     completeAsyncRequest(asyncId, requestMap, results);
 
     ZkNodeProps m = new ZkNodeProps(
-        Overseer.QUEUE_OPERATION, Overseer.OverseerAction.ADDROUTINGRULE.toLower(),
+        Overseer.QUEUE_OPERATION, OverseerAction.ADDROUTINGRULE.toLower(),
         COLLECTION_PROP, sourceCollection.getName(),
         SHARD_ID_PROP, sourceSlice.getName(),
         "routeKey", SolrIndexSplitter.getRouteKey(splitKey) + "!",
@@ -2315,10 +2317,10 @@ public class OverseerCollectionProcessor
       String router = message.getStr("router.name", DocRouter.DEFAULT_NAME);
       List<String> shardNames = new ArrayList<>();
       if(ImplicitDocRouter.NAME.equals(router)){
-        Overseer.getShardNames(shardNames, message.getStr("shards",null));
+        ClusterStateMutator.getShardNames(shardNames, message.getStr("shards", null));
         numSlices = shardNames.size();
       } else {
-        Overseer.getShardNames(numSlices,shardNames);
+        ClusterStateMutator.getShardNames(numSlices, shardNames);
       }
 
       if (numSlices == null ) {

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/cloud/ZkController.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/cloud/ZkController.java?rev=1642537&r1=1642536&r2=1642537&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/cloud/ZkController.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/cloud/ZkController.java Sun Nov 30 11:27:18 2014
@@ -46,6 +46,8 @@ import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.solr.client.solrj.impl.HttpSolrServer;
 import org.apache.solr.client.solrj.request.CoreAdminRequest.WaitForState;
+import org.apache.solr.cloud.overseer.OverseerAction;
+import org.apache.solr.cloud.overseer.SliceMutator;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.BeforeReconnect;
@@ -838,7 +840,7 @@ public final class ZkController {
       boolean joinAtHead = false;
       Replica replica = zkStateReader.getClusterState().getReplica(desc.getCloudDescriptor().getCollectionName(), coreZkNodeName);
       if (replica != null) {
-        joinAtHead = replica.getBool(Overseer.preferredLeaderProp, false);
+        joinAtHead = replica.getBool(SliceMutator.PREFERRED_LEADER_PROP, false);
       }
       joinElection(desc, afterExpiration, joinAtHead);
     } catch (InterruptedException e) {
@@ -1214,7 +1216,7 @@ public final class ZkController {
     }
     if (removeWatch) zkStateReader.removeZKWatch(collection);
     ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION,
-        Overseer.OverseerAction.DELETECORE.toLower(), ZkStateReader.CORE_NAME_PROP, coreName,
+        OverseerAction.DELETECORE.toLower(), ZkStateReader.CORE_NAME_PROP, coreName,
         ZkStateReader.NODE_NAME_PROP, getNodeName(),
         ZkStateReader.COLLECTION_PROP, cloudDescriptor.getCollectionName(),
         ZkStateReader.CORE_NODE_NAME_PROP, coreNodeName);

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java?rev=1642537&r1=1642536&r2=1642537&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java Sun Nov 30 11:27:18 2014
@@ -83,6 +83,7 @@ import org.apache.solr.cloud.Distributed
 import org.apache.solr.cloud.Overseer;
 import org.apache.solr.cloud.OverseerCollectionProcessor;
 import org.apache.solr.cloud.OverseerSolrResponse;
+import org.apache.solr.cloud.overseer.SliceMutator;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.ClusterState;
@@ -301,7 +302,7 @@ public class CollectionsHandler extends 
     for (Slice slice : dc.getSlices()) {
       for (Replica replica : slice.getReplicas()) {
         // Tell the replica to become the leader if we're the preferred leader AND active AND not the leader already
-        if (replica.getBool(Overseer.preferredLeaderProp, false) == false) {
+        if (replica.getBool(SliceMutator.PREFERRED_LEADER_PROP, false) == false) {
           continue;
         }
         if (StringUtils.equalsIgnoreCase(replica.getStr(STATE_PROP), ACTIVE) == false) {
@@ -442,7 +443,7 @@ public class CollectionsHandler extends 
     // Check if we're trying to set a property with parameters that allow us to set the property on multiple replicas
     // in a slice on properties that are known to only be one-per-slice and error out if so.
     if (StringUtils.isNotBlank((String)map.get(SHARD_UNIQUE)) &&
-        Overseer.sliceUniqueBooleanProperties.contains(property.toLowerCase(Locale.ROOT)) &&
+        SliceMutator.SLICE_UNIQUE_BOOLEAN_PROPERTIES.contains(property.toLowerCase(Locale.ROOT)) &&
         uniquePerSlice == false) {
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
           "Overseer replica property command received for property " + property +
@@ -472,7 +473,7 @@ public class CollectionsHandler extends 
     }
 
     if (shardUnique == false &&
-        Overseer.sliceUniqueBooleanProperties.contains(prop) == false) {
+        SliceMutator.SLICE_UNIQUE_BOOLEAN_PROPERTIES.contains(prop) == false) {
       throw new SolrException(ErrorCode.BAD_REQUEST, "Balancing properties amongst replicas in a slice requires that"
       + " the property be pre-defined as a unique property (e.g. 'preferredLeader') or that 'shardUnique' be set to 'true'. " +
       " Property: " + prop + " shardUnique: " + Boolean.toString(shardUnique));

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java?rev=1642537&r1=1642536&r2=1642537&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java Sun Nov 30 11:27:18 2014
@@ -176,14 +176,13 @@ public class DefaultSolrHighlighter exte
    * @param query The current Query
    * @param fieldName The name of the field
    * @param request The current SolrQueryRequest
-   * @param tokenStream document text CachingTokenStream
+   * @param tokenStream document text tokenStream that implements reset() efficiently (e.g. CachingTokenFilter).
+   *                    If it's used, call reset() first.
    * @throws IOException If there is a low-level I/O error.
    */
-  protected Highlighter getPhraseHighlighter(Query query, String fieldName, SolrQueryRequest request, CachingTokenFilter tokenStream) throws IOException {
+  protected Highlighter getPhraseHighlighter(Query query, String fieldName, SolrQueryRequest request, TokenStream tokenStream) throws IOException {
     SolrParams params = request.getParams();
-    Highlighter highlighter = null;
-    
-    highlighter = new Highlighter(
+    Highlighter highlighter = new Highlighter(
         getFormatter(fieldName, params),
         getEncoder(fieldName, params),
         getSpanQueryScorer(query, fieldName, tokenStream, request));
@@ -212,16 +211,14 @@ public class DefaultSolrHighlighter exte
   /**
    * Return a {@link org.apache.lucene.search.highlight.QueryScorer} suitable for this Query and field.
    * @param query The current query
-   * @param tokenStream document text CachingTokenStream
+   * @param tokenStream document text tokenStream that implements reset() efficiently (e.g. CachingTokenFilter).
+   *                    If it's used, call reset() first.
    * @param fieldName The name of the field
    * @param request The SolrQueryRequest
    */
   private QueryScorer getSpanQueryScorer(Query query, String fieldName, TokenStream tokenStream, SolrQueryRequest request) {
     boolean reqFieldMatch = request.getParams().getFieldBool(fieldName, HighlightParams.FIELD_MATCH, false);
-    Boolean highlightMultiTerm = request.getParams().getBool(HighlightParams.HIGHLIGHT_MULTI_TERM, true);
-    if(highlightMultiTerm == null) {
-      highlightMultiTerm = false;
-    }
+    boolean highlightMultiTerm = request.getParams().getBool(HighlightParams.HIGHLIGHT_MULTI_TERM, true);
     QueryScorer scorer;
     if (reqFieldMatch) {
       scorer = new QueryScorer(query, fieldName);
@@ -435,6 +432,7 @@ public class DefaultSolrHighlighter exte
     return termPosOff;
   }
   
+  @SuppressWarnings("unchecked")
   private void doHighlightingByHighlighter( Query query, SolrQueryRequest req, NamedList docSummaries,
       int docId, StoredDocument doc, String fieldName ) throws IOException {
     final SolrIndexSearcher searcher = req.getSearcher();
@@ -444,10 +442,7 @@ public class DefaultSolrHighlighter exte
     // so we disable them until fixed (see LUCENE-3080)!
     // BEGIN: Hack
     final SchemaField schemaField = schema.getFieldOrNull(fieldName);
-    if (schemaField != null && (
-      (schemaField.getType() instanceof org.apache.solr.schema.TrieField) ||
-      (schemaField.getType() instanceof org.apache.solr.schema.TrieDateField)
-    )) return;
+    if (schemaField != null && schemaField.getType() instanceof org.apache.solr.schema.TrieField) return;
     // END: Hack
     
     SolrParams params = req.getParams();
@@ -456,19 +451,18 @@ public class DefaultSolrHighlighter exte
     boolean preserveMulti = params.getFieldBool(fieldName, HighlightParams.PRESERVE_MULTI, false);
 
     List<StorableField> allFields = doc.getFields();
-    if (allFields != null && allFields.size() == 0) return; // No explicit contract that getFields returns != null,
+    if (allFields == null || allFields.isEmpty()) return; // No explicit contract that getFields returns != null,
                                                             // although currently it can't.
 
-    TokenStream tstream = null;
     int numFragments = getMaxSnippets(fieldName, params);
     boolean mergeContiguousFragments = isMergeContiguousFragments(fieldName, params);
 
     String[] summaries = null;
     List<TextFragment> frags = new ArrayList<>();
 
-    TermOffsetsTokenStream tots = null; // to be non-null iff we're using TermOffsets optimization
+    TermOffsetsTokenStream tots = null; // to be non-null iff we're using TermOffsets optimization (multi-valued)
     TokenStream tvStream = TokenSources.getTokenStreamWithOffsets(searcher.getIndexReader(), docId, fieldName);
-    if (tvStream != null) {
+    if (tvStream != null && schemaField.multiValued() && isActuallyMultiValued(allFields, fieldName)) {
       tots = new TermOffsetsTokenStream(tvStream);
     }
     int mvToExamine = Integer.parseInt(req.getParams().get(HighlightParams.MAX_MULTIVALUED_TO_EXAMINE,
@@ -483,10 +477,13 @@ public class DefaultSolrHighlighter exte
 
       --mvToExamine;
       String thisText = thisField.stringValue();
-      if( tots != null ) {
-        // if we're using TermOffsets optimization, then get the next
+      TokenStream tstream;
+      if (tots != null) {
+        // if we're using TermOffsets optimization (multi-valued field with term vectors), then get the next
         // field value's TokenStream (i.e. get field j's TokenStream) from tots:
-        tstream = tots.getMultiValuedTokenStream( thisText.length() );
+        tstream = tots.getMultiValuedTokenStream(thisText.length());
+      } else if (tvStream != null) {
+        tstream = tvStream; // single-valued with term vectors
       } else {
         // fall back to analyzer
         tstream = createAnalyzerTStream(schema, fieldName, thisText);
@@ -498,17 +495,30 @@ public class DefaultSolrHighlighter exte
       
       Highlighter highlighter;
       if (Boolean.valueOf(req.getParams().get(HighlightParams.USE_PHRASE_HIGHLIGHTER, "true"))) {
-        if (maxCharsToAnalyze < 0) {
-          tstream = new CachingTokenFilter(tstream);
+        // We're going to call getPhraseHighlighter and it might consume the tokenStream. If it does, the tokenStream
+        // needs to implement reset() efficiently.
+
+        //If the tokenStream is right from the term vectors, then CachingTokenFilter is unnecessary.
+        //  It should be okay if OffsetLimit won't get applied in this case.
+        final TokenStream tempTokenStream;
+        if (tstream != tvStream) {
+          if (maxCharsToAnalyze < 0) {
+            tempTokenStream = new CachingTokenFilter(tstream);
+          } else {
+            tempTokenStream = new CachingTokenFilter(new OffsetLimitTokenFilter(tstream, maxCharsToAnalyze));
+          }
         } else {
-          tstream = new CachingTokenFilter(new OffsetLimitTokenFilter(tstream, maxCharsToAnalyze));
+          tempTokenStream = tstream;
         }
-        
+
         // get highlighter
-        highlighter = getPhraseHighlighter(query, fieldName, req, (CachingTokenFilter) tstream);
+        highlighter = getPhraseHighlighter(query, fieldName, req, tempTokenStream);
          
-        // after highlighter initialization, reset tstream since construction of highlighter already used it
-        tstream.reset();
+        // if the CachingTokenFilter was consumed then use it going forward.
+        if (tempTokenStream instanceof CachingTokenFilter && ((CachingTokenFilter)tempTokenStream).isCached()) {
+          tstream = tempTokenStream;
+        }
+        //tstream.reset(); not needed; getBestTextFragments will reset it.
       }
       else {
         // use "the old way"
@@ -523,15 +533,15 @@ public class DefaultSolrHighlighter exte
 
       try {
         TextFragment[] bestTextFragments = highlighter.getBestTextFragments(tstream, thisText, mergeContiguousFragments, numFragments);
-        for (int k = 0; k < bestTextFragments.length; k++) {
+        for (TextFragment bestTextFragment : bestTextFragments) {
           if (preserveMulti) {
-            if (bestTextFragments[k] != null) {
-              frags.add(bestTextFragments[k]);
+            if (bestTextFragment != null) {
+              frags.add(bestTextFragment);
               --mvToMatch;
             }
           } else {
-            if ((bestTextFragments[k] != null) && (bestTextFragments[k].getScore() > 0)) {
-              frags.add(bestTextFragments[k]);
+            if ((bestTextFragment != null) && (bestTextFragment.getScore() > 0)) {
+              frags.add(bestTextFragment);
               --mvToMatch;
             }
           }
@@ -539,19 +549,20 @@ public class DefaultSolrHighlighter exte
       } catch (InvalidTokenOffsetsException e) {
         throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
       }
-    }
+    }//end field value loop
+
     // sort such that the fragments with the highest score come first
-     if(!preserveMulti){
-        Collections.sort(frags, new Comparator<TextFragment>() {
-                @Override
-                public int compare(TextFragment arg0, TextFragment arg1) {
-                 return Math.round(arg1.getScore() - arg0.getScore());
+    if (!preserveMulti) {
+      Collections.sort(frags, new Comparator<TextFragment>() {
+        @Override
+        public int compare(TextFragment arg0, TextFragment arg1) {
+          return Math.round(arg1.getScore() - arg0.getScore());
         }
-        });
-     }
+      });
+    }
 
-     // convert fragments back into text
-     // TODO: we can include score and position information in output as snippet attributes
+    // convert fragments back into text
+    // TODO: we can include score and position information in output as snippet attributes
     if (frags.size() > 0) {
       ArrayList<String> fragTexts = new ArrayList<>();
       for (TextFragment fragment: frags) {
@@ -577,6 +588,22 @@ public class DefaultSolrHighlighter exte
     }
   }
 
+  /** Is this field *actually* multi-valued for this document's fields? */
+  private boolean isActuallyMultiValued(List<StorableField> allFields, String fieldName) {
+    boolean foundFirst = false;
+    for (StorableField field : allFields) {
+      if (field.name().equals(fieldName)) {
+        if (foundFirst) {
+          return true;//we found another
+        } else {
+          foundFirst = true;
+        }
+      }
+    }
+    return false;//0 or 1 value
+  }
+
+  @SuppressWarnings("unchecked")
   private void doHighlightingByFastVectorHighlighter( FastVectorHighlighter highlighter, FieldQuery fieldQuery,
       SolrQueryRequest req, NamedList docSummaries, int docId, StoredDocument doc,
       String fieldName ) throws IOException {
@@ -596,6 +623,7 @@ public class DefaultSolrHighlighter exte
       alternateField( docSummaries, params, doc, fieldName );
   }
   
+  @SuppressWarnings("unchecked")
   private void alternateField( NamedList docSummaries, SolrParams params, StoredDocument doc, String fieldName ){
     String alternateField = params.getFieldParam(fieldName, HighlightParams.ALTERNATE_FIELD);
     if (alternateField != null && alternateField.length() > 0) {
@@ -612,7 +640,7 @@ public class DefaultSolrHighlighter exte
 
       String[] altTexts = listFields.toArray(new String[listFields.size()]);
 
-      if (altTexts != null && altTexts.length > 0){
+      if (altTexts.length > 0){
         Encoder encoder = getEncoder(fieldName, params);
         int alternateFieldLen = params.getFieldInt(fieldName, HighlightParams.ALTERNATE_FIELD_LENGTH,0);
         List<String> altList = new ArrayList<>();
@@ -622,6 +650,7 @@ public class DefaultSolrHighlighter exte
             altList.add(encoder.encodeText(altText));
           }
           else{
+            //note: seemingly redundant new String(...) releases memory to the larger text
             altList.add( len + altText.length() > alternateFieldLen ?
                 encoder.encodeText(new String(altText.substring( 0, alternateFieldLen - len ))) :
                 encoder.encodeText(altText) );
@@ -635,12 +664,7 @@ public class DefaultSolrHighlighter exte
   }
   
   private TokenStream createAnalyzerTStream(IndexSchema schema, String fieldName, String docText) throws IOException {
-
-    TokenStream tstream;
-    TokenStream ts = schema.getIndexAnalyzer().tokenStream(fieldName, docText);
-    ts.reset();
-    tstream = new TokenOrderingFilter(ts, 10);
-    return tstream;
+    return new TokenOrderingFilter(schema.getIndexAnalyzer().tokenStream(fieldName, docText), 10);
   }
 }
 
@@ -651,7 +675,7 @@ public class DefaultSolrHighlighter exte
  */
 final class TokenOrderingFilter extends TokenFilter {
   private final int windowSize;
-  private final LinkedList<OrderedToken> queue = new LinkedList<>();
+  private final LinkedList<OrderedToken> queue = new LinkedList<>(); //TODO replace with Deque, Array impl
   private boolean done=false;
   private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
   
@@ -694,10 +718,6 @@ final class TokenOrderingFilter extends 
     }
   }
 
-  @Override
-  public void reset() throws IOException {
-    // this looks wrong: but its correct.
-  }
 }
 
 // for TokenOrderingFilter, so it can easily sort by startOffset
@@ -708,19 +728,18 @@ class OrderedToken {
 
 class TermOffsetsTokenStream {
 
-  TokenStream bufferedTokenStream = null;
-  OffsetAttribute bufferedOffsetAtt;
+  final TokenStream bufferedTokenStream;
+  final OffsetAttribute bufferedOffsetAtt;
   State bufferedToken;
   int bufferedStartOffset;
   int bufferedEndOffset;
-  int startOffset;
+  int startOffset = 0;
   int endOffset;
+  boolean bufferedTokenStreamWasReset = false;
 
   public TermOffsetsTokenStream( TokenStream tstream ){
     bufferedTokenStream = tstream;
     bufferedOffsetAtt = bufferedTokenStream.addAttribute(OffsetAttribute.class);
-    startOffset = 0;
-    bufferedToken = null;
   }
 
   public TokenStream getMultiValuedTokenStream( final int length ){
@@ -730,38 +749,53 @@ class TermOffsetsTokenStream {
   
   final class MultiValuedStream extends TokenStream {
     private final int length;
+    private boolean incrementTokenWasCalled = false;
     OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
 
-      MultiValuedStream(int length) { 
-        super(bufferedTokenStream.cloneAttributes());
-        this.length = length;
-      }
-      
-      @Override
-      public boolean incrementToken() throws IOException {
-        while( true ){
-          if( bufferedToken == null ) {
-            if (!bufferedTokenStream.incrementToken())
-              return false;
-            bufferedToken = bufferedTokenStream.captureState();
-            bufferedStartOffset = bufferedOffsetAtt.startOffset();
-            bufferedEndOffset = bufferedOffsetAtt.endOffset();
-          }
-          
-          if( startOffset <= bufferedStartOffset &&
-              bufferedEndOffset <= endOffset ){
-            restoreState(bufferedToken);
-            bufferedToken = null;
-            offsetAtt.setOffset( offsetAtt.startOffset() - startOffset, offsetAtt.endOffset() - startOffset );
-            return true;
-          }
-          else if( bufferedEndOffset > endOffset ){
-            startOffset += length + 1;
+    MultiValuedStream(int length) {
+      super(bufferedTokenStream.cloneAttributes());//clone so we don't manipulate the buffered offsets
+      this.length = length;
+    }
+
+    @Override
+    public void reset() throws IOException {
+      //this flag allows reset() to be called multiple times up-front without a problem
+      if (incrementTokenWasCalled) {
+        throw new IllegalStateException("This TokenStream does not support being subsequently reset()");
+      }
+      if (!bufferedTokenStreamWasReset) {
+        bufferedTokenStream.reset();
+        bufferedTokenStreamWasReset = true;
+      }
+      super.reset();
+    }
+
+    @Override
+    public boolean incrementToken() throws IOException {
+      incrementTokenWasCalled = true;
+      while( true ){
+        if( bufferedToken == null ) {
+          if (!bufferedTokenStream.incrementToken())
             return false;
-          }
+          bufferedToken = bufferedTokenStream.captureState();
+          bufferedStartOffset = bufferedOffsetAtt.startOffset();
+          bufferedEndOffset = bufferedOffsetAtt.endOffset();
+        }
+
+        if( startOffset <= bufferedStartOffset &&
+            bufferedEndOffset <= endOffset ){
+          restoreState(bufferedToken);
           bufferedToken = null;
+          offsetAtt.setOffset( offsetAtt.startOffset() - startOffset, offsetAtt.endOffset() - startOffset );
+          return true;
         }
+        else if( bufferedEndOffset > endOffset ){
+          startOffset += length + 1;
+          return false;
+        }
+        bufferedToken = null;
       }
+    }
 
-  };
-};
+  }
+}

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/spelling/suggest/fst/AnalyzingInfixLookupFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/spelling/suggest/fst/AnalyzingInfixLookupFactory.java?rev=1642537&r1=1642536&r2=1642537&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/spelling/suggest/fst/AnalyzingInfixLookupFactory.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/spelling/suggest/fst/AnalyzingInfixLookupFactory.java Sun Nov 30 11:27:18 2014
@@ -96,8 +96,7 @@ public class AnalyzingInfixLookupFactory
     : AnalyzingInfixSuggester.DEFAULT_MIN_PREFIX_CHARS;
 
     try {
-      return new AnalyzingInfixSuggester(core.getSolrConfig().luceneMatchVersion, 
-                                         FSDirectory.open(new File(indexPath).toPath()), indexAnalyzer,
+      return new AnalyzingInfixSuggester(FSDirectory.open(new File(indexPath).toPath()), indexAnalyzer,
                                          queryAnalyzer, minPrefixChars, true) {
         @Override
         public List<LookupResult> lookup(CharSequence key, Set<BytesRef> contexts, int num, boolean allTermsRequired, boolean doHighlight) throws IOException {

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/spelling/suggest/fst/BlendedInfixLookupFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/spelling/suggest/fst/BlendedInfixLookupFactory.java?rev=1642537&r1=1642536&r2=1642537&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/spelling/suggest/fst/BlendedInfixLookupFactory.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/spelling/suggest/fst/BlendedInfixLookupFactory.java Sun Nov 30 11:27:18 2014
@@ -101,8 +101,7 @@ public class BlendedInfixLookupFactory e
     : BlendedInfixSuggester.DEFAULT_NUM_FACTOR;
     
     try {
-      return new BlendedInfixSuggester(core.getSolrConfig().luceneMatchVersion, 
-                                       FSDirectory.open(new File(indexPath).toPath()),
+      return new BlendedInfixSuggester(FSDirectory.open(new File(indexPath).toPath()),
                                        indexAnalyzer, queryAnalyzer, minPrefixChars,
                                        blenderType, numFactor, true) {
         @Override

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java?rev=1642537&r1=1642536&r2=1642537&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java Sun Nov 30 11:27:18 2014
@@ -47,6 +47,7 @@ import org.apache.solr.cloud.Distributed
 import org.apache.solr.cloud.LeaderInitiatedRecoveryThread;
 import org.apache.solr.cloud.Overseer;
 import org.apache.solr.cloud.ZkController;
+import org.apache.solr.cloud.overseer.OverseerAction;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.SolrInputDocument;
@@ -77,7 +78,6 @@ import org.apache.solr.request.SolrReque
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.SchemaField;
-import org.apache.solr.schema.TrieDateField;
 import org.apache.solr.update.AddUpdateCommand;
 import org.apache.solr.update.CommitUpdateCommand;
 import org.apache.solr.update.DeleteUpdateCommand;
@@ -542,7 +542,7 @@ public class DistributedUpdateProcessor 
                   if (ruleExpiryLock.tryLock(10, TimeUnit.MILLISECONDS)) {
                     log.info("Going to expire routing rule");
                     try {
-                      Map<String, Object> map = ZkNodeProps.makeMap(Overseer.QUEUE_OPERATION, Overseer.OverseerAction.REMOVEROUTINGRULE.toLower(),
+                      Map<String, Object> map = ZkNodeProps.makeMap(Overseer.QUEUE_OPERATION, OverseerAction.REMOVEROUTINGRULE.toLower(),
                           ZkStateReader.COLLECTION_PROP, collection,
                           ZkStateReader.SHARD_ID_PROP, myShardId,
                           "routeKey", routeKey + "!");

Modified: lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/cloud/DeleteShardTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/cloud/DeleteShardTest.java?rev=1642537&r1=1642536&r2=1642537&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/cloud/DeleteShardTest.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/cloud/DeleteShardTest.java Sun Nov 30 11:27:18 2014
@@ -21,6 +21,7 @@ import org.apache.solr.client.solrj.Solr
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.impl.HttpSolrServer;
 import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.cloud.overseer.OverseerAction;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.Slice;
@@ -157,7 +158,7 @@ public class DeleteShardTest extends Abs
       KeeperException, InterruptedException {
     DistributedQueue inQueue = Overseer.getInQueue(cloudClient.getZkStateReader().getZkClient());
     Map<String, Object> propMap = new HashMap<>();
-    propMap.put(Overseer.QUEUE_OPERATION, Overseer.OverseerAction.UPDATESHARDSTATE.toLower());
+    propMap.put(Overseer.QUEUE_OPERATION, OverseerAction.UPDATESHARDSTATE.toLower());
     propMap.put(slice, state);
     propMap.put(ZkStateReader.COLLECTION_PROP, "collection1");
     ZkNodeProps m = new ZkNodeProps(propMap);

Modified: lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java?rev=1642537&r1=1642536&r2=1642537&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java Sun Nov 30 11:27:18 2014
@@ -38,6 +38,7 @@ import org.apache.solr.client.solrj.Solr
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.client.solrj.impl.CloudSolrServer;
 import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.cloud.overseer.OverseerAction;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
@@ -106,7 +107,7 @@ public class OverseerRolesTest  extends 
     Map m = (Map) ZkStateReader.fromJSON(data);
     String s = (String) m.get("id");
     String leader = LeaderElector.getNodeName(s);
-    Overseer.getInQueue(zk).offer(ZkStateReader.toJSON(new ZkNodeProps(Overseer.QUEUE_OPERATION, Overseer.OverseerAction.QUIT.toLower())));
+    Overseer.getInQueue(zk).offer(ZkStateReader.toJSON(new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.QUIT.toLower())));
     long timeout = System.currentTimeMillis()+10000;
     String newLeader=null;
     for(;System.currentTimeMillis() < timeout;){

Modified: lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java?rev=1642537&r1=1642536&r2=1642537&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java Sun Nov 30 11:27:18 2014
@@ -36,6 +36,7 @@ import javax.xml.parsers.ParserConfigura
 
 import org.apache.lucene.util.LuceneTestCase.Slow;
 import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.cloud.overseer.OverseerAction;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
@@ -43,10 +44,14 @@ import org.apache.solr.common.cloud.Slic
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.handler.component.HttpShardHandlerFactory;
 import org.apache.solr.update.UpdateShardHandler;
 import org.apache.solr.util.DefaultSolrThreadFactory;
 import org.apache.solr.util.MockConfigSolr;
+import org.apache.solr.util.stats.Snapshot;
+import org.apache.solr.util.stats.Timer;
+import org.apache.solr.util.stats.TimerContext;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.NoNodeException;
@@ -54,6 +59,7 @@ import org.apache.zookeeper.KeeperExcept
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.xml.sax.SAXException;
 
@@ -112,7 +118,7 @@ public class OverseerTest extends SolrTe
         if (ec != null) {
           ec.cancelElection();
         }
-        ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, Overseer.OverseerAction.DELETECORE.toLower(),
+        ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.DELETECORE.toLower(),
             ZkStateReader.NODE_NAME_PROP, nodeName,
             ZkStateReader.CORE_NAME_PROP, coreName,
             ZkStateReader.CORE_NODE_NAME_PROP, coreNodeName,
@@ -121,7 +127,7 @@ public class OverseerTest extends SolrTe
             q.offer(ZkStateReader.toJSON(m));
          return null;
       } else {
-        ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, Overseer.OverseerAction.STATE.toLower(),
+        ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.STATE.toLower(),
         ZkStateReader.STATE_PROP, stateName,
         ZkStateReader.NODE_NAME_PROP, nodeName,
         ZkStateReader.CORE_NAME_PROP, coreName,
@@ -525,7 +531,7 @@ public class OverseerTest extends SolrTe
 
       DistributedQueue q = Overseer.getInQueue(zkClient);
       
-      ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, Overseer.OverseerAction.STATE.toLower(),
+      ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.STATE.toLower(),
           ZkStateReader.BASE_URL_PROP, "http://127.0.0.1/solr",
           ZkStateReader.NODE_NAME_PROP, "node1",
           ZkStateReader.COLLECTION_PROP, "collection1",
@@ -878,6 +884,133 @@ public class OverseerTest extends SolrTe
     }
   }
 
+  @Test
+  @Ignore
+  public void testPerformance() throws Exception {
+    String zkDir = createTempDir("OverseerTest.testPerformance").toFile().getAbsolutePath();
+
+    ZkTestServer server = new ZkTestServer(zkDir);
+
+    SolrZkClient controllerClient = null;
+    SolrZkClient overseerClient = null;
+    ZkStateReader reader = null;
+    MockZKController mockController = null;
+
+    try {
+      server.run();
+      controllerClient = new SolrZkClient(server.getZkAddress(), TIMEOUT);
+
+      AbstractZkTestCase.tryCleanSolrZkNode(server.getZkHost());
+      AbstractZkTestCase.makeSolrZkNode(server.getZkHost());
+      controllerClient.makePath(ZkStateReader.LIVE_NODES_ZKNODE, true);
+
+      reader = new ZkStateReader(controllerClient);
+      reader.createClusterStateWatchersAndUpdate();
+
+      mockController = new MockZKController(server.getZkAddress(), "node1");
+
+      for (int i=0; i<100; i++)  {
+        ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, CollectionParams.CollectionAction.CREATE.toLower(),
+            "name", "perf" + i,
+            ZkStateReader.NUM_SHARDS_PROP, "1",
+            "stateFormat", "2",
+            ZkStateReader.REPLICATION_FACTOR, "1",
+            ZkStateReader.MAX_SHARDS_PER_NODE, "1"
+            );
+        DistributedQueue q = Overseer.getInQueue(controllerClient);
+        q.offer(ZkStateReader.toJSON(m));
+        controllerClient.makePath("/collections/perf" + i, true);
+      }
+
+      for (int i = 0, j = 0, k = 0; i < 20000; i++, j++, k++) {
+        ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.STATE.toLower(),
+            ZkStateReader.STATE_PROP, ZkStateReader.RECOVERING,
+            ZkStateReader.NODE_NAME_PROP,  "node1",
+            ZkStateReader.CORE_NAME_PROP, "core" + k,
+            ZkStateReader.CORE_NODE_NAME_PROP, "node1",
+            ZkStateReader.COLLECTION_PROP, "perf" + j,
+            ZkStateReader.NUM_SHARDS_PROP, "1",
+            ZkStateReader.BASE_URL_PROP, "http://" +  "node1"
+            + "/solr/");
+        DistributedQueue q = Overseer.getInQueue(controllerClient);
+        q.offer(ZkStateReader.toJSON(m));
+        if (j == 99) j = 0;
+        if (k == 9) k = 0;
+        if (i > 0 && i % 100 == 0) log.info("Published {} items", i);
+      }
+
+      // let's publish a sentinel collection which we'll use to wait for overseer to complete operations
+      ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.STATE.toLower(),
+          ZkStateReader.STATE_PROP, ZkStateReader.ACTIVE,
+          ZkStateReader.NODE_NAME_PROP, "node1",
+          ZkStateReader.CORE_NAME_PROP, "core1",
+          ZkStateReader.CORE_NODE_NAME_PROP, "node1",
+          ZkStateReader.COLLECTION_PROP, "perf_sentinel",
+          ZkStateReader.NUM_SHARDS_PROP, "1",
+          ZkStateReader.BASE_URL_PROP, "http://" + "node1"
+          + "/solr/");
+      DistributedQueue q = Overseer.getInQueue(controllerClient);
+      q.offer(ZkStateReader.toJSON(m));
+
+      Timer t = new Timer();
+      TimerContext context = t.time();
+      try {
+        overseerClient = electNewOverseer(server.getZkAddress());
+        assertTrue(overseers.size() > 0);
+
+        while (true)  {
+          reader.updateClusterState(true);
+          ClusterState state = reader.getClusterState();
+          if (state.hasCollection("perf_sentinel")) {
+            break;
+          }
+          Thread.sleep(1000);
+        }
+      } finally {
+        context.stop();
+      }
+
+      log.info("Overseer loop finished processing: ");
+      printTimingStats(t);
+
+      Overseer overseer = overseers.get(0);
+      Overseer.Stats stats = overseer.getStats();
+
+      String[] interestingOps = {"state", "update_state", "am_i_leader", ""};
+      Arrays.sort(interestingOps);
+      for (Map.Entry<String, Overseer.Stat> entry : stats.getStats().entrySet()) {
+        String op = entry.getKey();
+        if (Arrays.binarySearch(interestingOps, op) < 0)
+          continue;
+        Overseer.Stat stat = entry.getValue();
+        log.info("op: {}, success: {}, failure: {}", op, stat.success.get(), stat.errors.get());
+        Timer timer = stat.requestTime;
+        printTimingStats(timer);
+      }
+
+    } finally {
+      close(overseerClient);
+      close(mockController);
+      close(controllerClient);
+      close(reader);
+      server.shutdown();
+    }
+  }
+
+  private void printTimingStats(Timer timer) {
+    Snapshot snapshot = timer.getSnapshot();
+    log.info("\t totalTime: {}", timer.getSum());
+    log.info("\t avgRequestsPerMinute: {}", timer.getMeanRate());
+    log.info("\t 5minRateRequestsPerMinute: {}", timer.getFiveMinuteRate());
+    log.info("\t 15minRateRequestsPerMinute: {}", timer.getFifteenMinuteRate());
+    log.info("\t avgTimePerRequest: {}", timer.getMean());
+    log.info("\t medianRequestTime: {}", snapshot.getMedian());
+    log.info("\t 75thPctlRequestTime: {}", snapshot.get75thPercentile());
+    log.info("\t 95thPctlRequestTime: {}", snapshot.get95thPercentile());
+    log.info("\t 99thPctlRequestTime: {}", snapshot.get99thPercentile());
+    log.info("\t 999thPctlRequestTime: {}", snapshot.get999thPercentile());
+  }
+
   private void close(MockZKController mockController) {
     if (mockController != null) {
       mockController.close();
@@ -928,7 +1061,7 @@ public class OverseerTest extends SolrTe
       
       //submit to proper queue
       queue = Overseer.getInQueue(zkClient);
-      m = new ZkNodeProps(Overseer.QUEUE_OPERATION, Overseer.OverseerAction.STATE.toLower(),
+      m = new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.STATE.toLower(),
           ZkStateReader.BASE_URL_PROP, "http://127.0.0.1/solr",
           ZkStateReader.NODE_NAME_PROP, "node1",
           ZkStateReader.SHARD_ID_PROP, "s1",

Modified: lucene/dev/branches/lucene6005/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java?rev=1642537&r1=1642536&r2=1642537&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java (original)
+++ lucene/dev/branches/lucene6005/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java Sun Nov 30 11:27:18 2014
@@ -40,7 +40,7 @@ import org.slf4j.LoggerFactory;
 public class ClusterState implements JSONWriter.Writable {
   private static Logger log = LoggerFactory.getLogger(ClusterState.class);
   
-  private Integer znodeVersion;
+  private final Integer znodeVersion;
   
   private final Map<String, CollectionRef> collectionStates;
   private Set<String> liveNodes;
@@ -84,15 +84,19 @@ public class ClusterState implements JSO
   }
 
 
-  public ClusterState copyWith(Map<String,DocCollection> modified){
+  /**
+   * Returns a new cluster state object modified with the given collection.
+   *
+   * @param collectionName the name of the modified (or deleted) collection
+   * @param collection     the collection object. A null value deletes the collection from the state
+   * @return the updated cluster state which preserves the current live nodes and zk node version
+   */
+  public ClusterState copyWith(String collectionName, DocCollection collection) {
     ClusterState result = new ClusterState(liveNodes, new LinkedHashMap<>(collectionStates), znodeVersion);
-    for (Entry<String, DocCollection> e : modified.entrySet()) {
-      DocCollection c = e.getValue();
-      if(c == null) {
-        result.collectionStates.remove(e.getKey());
-        continue;
-      }
-      result.collectionStates.put(c.getName(), new CollectionRef(c));
+    if (collection == null) {
+      result.collectionStates.remove(collectionName);
+    } else {
+      result.collectionStates.put(collectionName, new CollectionRef(collection));
     }
     return result;
   }

Modified: lucene/dev/branches/lucene6005/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java?rev=1642537&r1=1642536&r2=1642537&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java (original)
+++ lucene/dev/branches/lucene6005/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java Sun Nov 30 11:27:18 2014
@@ -35,7 +35,7 @@ public class DocCollection extends ZkNod
   public static final String DOC_ROUTER = "router";
   public static final String SHARDS = "shards";
   public static final String STATE_FORMAT = "stateFormat";
-  private int znodeVersion;
+  private int znodeVersion = -1; // sentinel
 
   private final String name;
   private final Map<String, Slice> slices;

Modified: lucene/dev/branches/lucene6005/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java?rev=1642537&r1=1642536&r2=1642537&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java (original)
+++ lucene/dev/branches/lucene6005/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java Sun Nov 30 11:27:18 2014
@@ -873,8 +873,7 @@ public class ZkStateReader implements Cl
     log.info("Updating data for {} to ver {} ", newState.getName(),
         newState.getZNodeVersion());
     
-    this.clusterState = clusterState.copyWith(Collections.singletonMap(
-        newState.getName(), newState));
+    this.clusterState = clusterState.copyWith(newState.getName(), newState);
   }
   
   /** This is not a public API. Only used by ZkController */