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/08 12:32:23 UTC

svn commit: r1637544 [5/6] - in /lucene/dev/branches/lucene6005: ./ lucene/ lucene/core/ lucene/core/src/java/org/apache/lucene/codecs/perfield/ lucene/core/src/java/org/apache/lucene/document/ lucene/core/src/java/org/apache/lucene/index/ lucene/core/...

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/PivotFacetValue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/PivotFacetValue.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/PivotFacetValue.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/PivotFacetValue.java Sat Nov  8 11:32:18 2014
@@ -21,11 +21,13 @@ import java.util.BitSet;
 import java.util.Date;
 import java.util.List;
 import java.util.Locale;
+import java.util.Map;
 
 import org.apache.solr.common.params.FacetParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.schema.TrieDateField;
+import org.apache.solr.search.QueryParsing;
 import org.apache.solr.util.PivotListEntry;
 
 /**
@@ -45,6 +47,7 @@ public class PivotFacetValue {
   // child can't be final, circular ref on construction
   private PivotFacetField childPivot = null; 
   private int count; // mutable
+  private Map<String, StatsValues> statsValues = null;
   
   private PivotFacetValue(PivotFacetField parent, Comparable val) { 
     this.parentPivot = parent;
@@ -114,6 +117,7 @@ public class PivotFacetValue {
     Comparable pivotVal = null;
     int pivotCount = 0;
     List<NamedList<Object>> childPivotData = null;
+    NamedList<NamedList<NamedList<?>>> statsValues = null;
     
     for (int i = 0; i < pivotData.size(); i++) {
       String key = pivotData.getName(i);
@@ -135,6 +139,9 @@ public class PivotFacetValue {
       case PIVOT:
         childPivotData = (List<NamedList<Object>>)value;
         break;
+      case STATS:
+        statsValues = (NamedList<NamedList<NamedList<?>>>) value;
+        break;
       default:
         throw new RuntimeException("PivotListEntry contains unaccounted for item: " + entry);
       }
@@ -143,6 +150,9 @@ public class PivotFacetValue {
     PivotFacetValue newPivotFacet = new PivotFacetValue(parentField, pivotVal);
     newPivotFacet.count = pivotCount;
     newPivotFacet.sourceShards.set(shardNumber);
+    if(statsValues != null) {
+      newPivotFacet.statsValues = PivotFacetHelper.mergeStats(null, statsValues, rb._statsInfo);
+    }
     
     newPivotFacet.childPivot = PivotFacetField.createFromListOfNamedLists(shardNumber, rb, newPivotFacet, childPivotData);
     
@@ -171,6 +181,11 @@ public class PivotFacetValue {
     if (childPivot != null && childPivot.convertToListOfNamedLists() != null) {
       newList.add(PivotListEntry.PIVOT.getName(), childPivot.convertToListOfNamedLists());
     }
+    if (null != statsValues) {
+      newList.add(PivotListEntry.STATS.getName(), 
+                  // for pivots, we *always* include requested stats - even if 'empty'
+                  StatsComponent.convertToResponse(true, statsValues));
+    }
     return newList;
   }      
   
@@ -187,6 +202,10 @@ public class PivotFacetValue {
     if (!shardHasContributed(shardNumber)) {
       sourceShards.set(shardNumber);      
       count += PivotFacetHelper.getCount(value);
+      NamedList<NamedList<NamedList<?>>> stats = PivotFacetHelper.getStats(value);
+      if (stats != null) {
+        statsValues = PivotFacetHelper.mergeStats(statsValues, stats, rb._statsInfo);
+      }
     }
     
     List<NamedList<Object>> shardChildPivots = PivotFacetHelper.getPivots(value);
@@ -197,7 +216,7 @@ public class PivotFacetValue {
       childPivot.contributeFromShard(shardNumber, rb, shardChildPivots);
     }
   }
-  
+
   public String toString(){
     return String.format(Locale.ROOT, "F:%s V:%s Co:%d Ch?:%s", 
                          parentPivot.field, value, count, (this.childPivot !=null));

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/StatsComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/StatsComponent.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/StatsComponent.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/StatsComponent.java Sat Nov  8 11:32:18 2014
@@ -25,8 +25,6 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.solr.common.SolrException;
-import org.apache.solr.common.SolrException.ErrorCode;
-import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ShardParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.params.StatsParams;
@@ -56,22 +54,14 @@ public class StatsComponent extends Sear
     if (!rb.doStats) return;
 
     boolean isShard = rb.req.getParams().getBool(ShardParams.IS_SHARD, false);
-    NamedList<Object> out = new SimpleOrderedMap<>();
-    NamedList<Object> stats_fields = new SimpleOrderedMap<>();
+    Map<String, StatsValues> statsValues = new LinkedHashMap<>();
 
     for (StatsField statsField : rb._statsInfo.getStatsFields()) {
       DocSet docs = statsField.computeBaseDocSet();
-      NamedList<?> stv = statsField.computeLocalStatsValues(docs).getStatsValues();
-      
-      if (isShard == true || (Long) stv.get("count") > 0) {
-        stats_fields.add(statsField.getOutputKey(), stv);
-      } else {
-        stats_fields.add(statsField.getOutputKey(), null);
-      }
+      statsValues.put(statsField.getOutputKey(), statsField.computeLocalStatsValues(docs));
     }
     
-    out.add("stats_fields", stats_fields);
-    rb.rsp.add( "stats", out );
+    rb.rsp.add( "stats", convertToResponse(isShard, statsValues) );
   }
 
   @Override
@@ -86,6 +76,8 @@ public class StatsComponent extends Sear
     if ((sreq.purpose & ShardRequest.PURPOSE_GET_TOP_IDS) != 0) {
       sreq.purpose |= ShardRequest.PURPOSE_GET_STATS;
     } else {
+
+
       // turn off stats on other requests
       sreq.params.set(StatsParams.STATS, "false");
       // we could optionally remove stats params
@@ -101,7 +93,8 @@ public class StatsComponent extends Sear
     for (ShardResponse srsp : sreq.responses) {
       NamedList stats = null;
       try {
-        stats = (NamedList) srsp.getSolrResponse().getResponse().get("stats");
+        stats = (NamedList<NamedList<NamedList<?>>>) 
+          srsp.getSolrResponse().getResponse().get("stats");
       } catch (Exception e) {
         if (rb.req.getParams().getBool(ShardParams.SHARDS_TOLERANT, false)) {
           continue; // looks like a shard did not return anything
@@ -110,7 +103,7 @@ public class StatsComponent extends Sear
             "Unable to read stats info for shard: " + srsp.getShard(), e);
       }
 
-      NamedList stats_fields = (NamedList) stats.get("stats_fields");
+      NamedList stats_fields = unwrapStats(stats);
       if (stats_fields != null) {
         for (int i = 0; i < stats_fields.size(); i++) {
           String key = stats_fields.getName(i);
@@ -129,26 +122,44 @@ public class StatsComponent extends Sear
     // so that "result" is already stored in the response (for aesthetics)
 
     Map<String, StatsValues> allStatsValues = rb._statsInfo.getAggregateStatsValues();
+    rb.rsp.add("stats", convertToResponse(false, allStatsValues));
+
+    rb._statsInfo = null; // free some objects 
+  }
+
+  /**
+   * Helper to pull the "stats_fields" out of the extra "stats" wrapper
+   */
+  public static NamedList<NamedList<?>> unwrapStats(NamedList<NamedList<NamedList<?>>> stats) {
+    if (null == stats) return null;
+
+    return stats.get("stats_fields");
+  }
 
-    NamedList<NamedList<Object>> stats = new SimpleOrderedMap<>();
-    NamedList<Object> stats_fields = new SimpleOrderedMap<>();
+  /**
+   * Given a map of {@link StatsValues} using the appropriate response key,
+   * builds up the neccessary "stats" data structure for including in the response -- 
+   * including the esoteric "stats_fields" wrapper.
+   */
+  public static NamedList<NamedList<NamedList<?>>> convertToResponse
+    (boolean force, Map<String,StatsValues> statsValues) {
+
+    NamedList<NamedList<NamedList<?>>> stats = new SimpleOrderedMap<>();
+    NamedList<NamedList<?>> stats_fields = new SimpleOrderedMap<>();
     stats.add("stats_fields", stats_fields);
     
-    for (Map.Entry<String,StatsValues> entry : allStatsValues.entrySet()) {
+    for (Map.Entry<String,StatsValues> entry : statsValues.entrySet()) {
       String key = entry.getKey();
       NamedList stv = entry.getValue().getStatsValues();
-      if ((Long) stv.get("count") != 0) {
+      if (force || ((Long) stv.get("count") != 0)) {
         stats_fields.add(key, stv);
       } else {
         stats_fields.add(key, null);
       }
     }
-
-    rb.rsp.add("stats", stats);
-    rb._statsInfo = null; // free some objects 
+    return stats;
   }
 
-
   /////////////////////////////////////////////
   ///  SolrInfoMBean
   ////////////////////////////////////////////
@@ -168,6 +179,8 @@ class StatsInfo {
   private final ResponseBuilder rb;
   private final List<StatsField> statsFields = new ArrayList<>(7);
   private final Map<String, StatsValues> distribStatsValues = new LinkedHashMap<>();
+  private final Map<String, StatsField> statsFieldMap = new LinkedHashMap<>();
+  private final Map<String, List<StatsField>> tagToStatsFields = new LinkedHashMap<>();
 
   public StatsInfo(ResponseBuilder rb) { 
     this.rb = rb;
@@ -177,10 +190,19 @@ class StatsInfo {
       // no stats.field params, nothing to parse.
       return;
     }
-
+    
     for (String paramValue : statsParams) {
       StatsField current = new StatsField(rb, paramValue);
       statsFields.add(current);
+      for (String tag : current.getTagList()) {
+        List<StatsField> fieldList = tagToStatsFields.get(tag);
+        if (fieldList == null) {
+          fieldList = new ArrayList<>();
+        }
+        fieldList.add(current);
+        tagToStatsFields.put(tag, fieldList);
+      }
+      statsFieldMap.put(current.getOutputKey(), current);
       distribStatsValues.put(current.getOutputKey(), 
                              StatsValuesFactory.createStatsValues(current));
     }
@@ -192,7 +214,31 @@ class StatsInfo {
    * as part of this request
    */
   public List<StatsField> getStatsFields() {
-    return Collections.<StatsField>unmodifiableList(statsFields);
+    return Collections.unmodifiableList(statsFields);
+  }
+
+  /**
+   * Returns the {@link StatsField} associated with the specified (effective) 
+   * outputKey, or null if there was no {@link StatsParams#STATS_FIELD} param
+   * that would corrispond with that key.
+   */
+  public StatsField getStatsField(String outputKey) {
+    return statsFieldMap.get(outputKey);
+  }
+
+  /**
+   * Return immutable list of {@link StatsField} instances by string tag local parameter.
+   *
+   * @param tag tag local parameter
+   * @return list of stats fields
+   */
+  public List<StatsField> getStatsFieldsByTag(String tag) {
+    List<StatsField> raw = tagToStatsFields.get(tag);
+    if (null == raw) {
+      return Collections.emptyList();
+    } else {
+      return Collections.unmodifiableList(raw);
+    }
   }
 
   /**
@@ -203,7 +249,7 @@ class StatsInfo {
    * will never be null.
    */
   public Map<String, StatsValues> getAggregateStatsValues() {
-    return Collections.<String, StatsValues>unmodifiableMap(distribStatsValues);
+    return Collections.unmodifiableMap(distribStatsValues);
   }
 
 }

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/StatsField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/StatsField.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/StatsField.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/StatsField.java Sat Nov  8 11:32:18 2014
@@ -70,6 +70,7 @@ public class StatsField {
   private final String key;
   private final boolean calcDistinct; // TODO: put this inside localParams ? SOLR-6349 ?
   private final String[] facets;
+  private final List<String> tagList;
   private final List<String> excludeTagList;
 
   /**
@@ -147,6 +148,10 @@ public class StatsField {
 
     String[] facets = params.getFieldParams(key, StatsParams.STATS_FACET);
     this.facets = (null == facets) ? new String[0] : facets;
+    String tagStr = localParams.get(CommonParams.TAG);
+    this.tagList = (null == tagStr)
+        ? Collections.<String>emptyList()
+        : StrUtils.splitSmart(tagStr,',');
 
     // figure out if we need a special base DocSet
     String excludeStr = localParams.get(CommonParams.EXCLUDE);
@@ -363,6 +368,11 @@ public class StatsField {
     return calcDistinct;
   }
 
+
+  public List<String> getTagList() {
+    return tagList;
+  }
+
   public String toString() {
     return "StatsField<" + originalParam + ">";
   }

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java Sat Nov  8 11:32:18 2014
@@ -316,8 +316,6 @@ class NumericStatsValues extends Abstrac
 
   public NumericStatsValues(StatsField statsField) {
     super(statsField);
-    min = Double.POSITIVE_INFINITY;
-    max = Double.NEGATIVE_INFINITY;
   }
 
   @Override
@@ -353,8 +351,22 @@ class NumericStatsValues extends Abstrac
    */
   @Override
   protected void updateMinMax(Number min, Number max) {
-    this.min = Math.min(this.min.doubleValue(), min.doubleValue());
-    this.max = Math.max(this.max.doubleValue(), max.doubleValue());
+    if (null == min) {
+      assert null == max : "min is null but max isn't ? ==> " + max;
+      return; // No-Op
+    }
+
+    assert null != max : "max is null but min isn't ? ==> " + min;
+
+    // we always use the double value, because that way the response Object class is 
+    // consistent regardless of wether we only have 1 value or many that we min/max
+    //
+    // TODO: would be nice to have subclasses for each type of Number ... breaks backcompat
+    double minD = min.doubleValue();
+    double maxD = max.doubleValue();
+
+    this.min = (null == this.min) ? minD : Math.min(this.min.doubleValue(), minD);
+    this.max = (null == this.max) ? maxD : Math.max(this.max.doubleValue(), maxD);
   }
 
   /**
@@ -594,7 +606,7 @@ class StringStatsValues extends Abstract
     // Add no statistics
   }
 
-  /**
+  /** 
    * Determines which of the given Strings is the maximum, as computed by {@link String#compareTo(String)}
    *
    * @param str1 String to compare against b

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java Sat Nov  8 11:32:18 2014
@@ -41,8 +41,8 @@ import org.apache.lucene.util.QueryBuild
 import org.apache.lucene.util.ToStringUtils;
 import org.apache.lucene.util.Version;
 import org.apache.lucene.util.automaton.Automata;
-import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.Automaton;
+import org.apache.lucene.util.automaton.Operations;
 import org.apache.solr.analysis.ReversedWildcardFilterFactory;
 import org.apache.solr.analysis.TokenizerChain;
 import org.apache.solr.common.SolrException;
@@ -788,7 +788,7 @@ public abstract class SolrQueryParserBas
             Automata.makeChar(factory.getMarkerChar()),
             Automata.makeAnyString());
         // subtract these away
-        automaton = Operations.minus(automaton, falsePositives);
+        automaton = Operations.minus(automaton, falsePositives, Operations.DEFAULT_MAX_DETERMINIZED_STATES);
       }
       return new AutomatonQuery(term, automaton) {
         // override toString so its completely transparent

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/response/SortingResponseWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/response/SortingResponseWriter.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/response/SortingResponseWriter.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/response/SortingResponseWriter.java Sat Nov  8 11:32:18 2014
@@ -118,7 +118,7 @@ public class SortingResponseWriter imple
     }
 
     FieldWriter[] fieldWriters = getFieldWriters(fields, req.getSearcher());
-    writer.write("{\"numFound\":"+totalHits+", \"docs\":[");
+    writer.write("{\"responseHeader\": {\"status\": 0}, \"response\":{\"numFound\":"+totalHits+", \"docs\":[");
 
     //Write the data.
     List<LeafReaderContext> leaves = req.getSearcher().getTopReaderContext().leaves();
@@ -128,9 +128,9 @@ public class SortingResponseWriter imple
     SortQueue queue = new SortQueue(queueSize, sortDoc);
     SortDoc[] outDocs = new SortDoc[queueSize];
 
+    boolean commaNeeded = false;
     while(count < totalHits) {
       //long begin = System.nanoTime();
-      boolean commaNeeded = false;
       queue.reset();
       SortDoc top = queue.top();
       for(int i=0; i<leaves.size(); i++) {
@@ -188,7 +188,7 @@ public class SortingResponseWriter imple
     }
 
     //System.out.println("Sort Time 2:"+Long.toString(total/1000000));
-    writer.write("]}");
+    writer.write("]}}");
     writer.flush();
   }
 

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/rest/SolrConfigRestApi.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/rest/SolrConfigRestApi.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/rest/SolrConfigRestApi.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/rest/SolrConfigRestApi.java Sat Nov  8 11:32:18 2014
@@ -58,7 +58,7 @@ public class SolrConfigRestApi extends A
    */
   @Override
   public synchronized Restlet createInboundRoot() {
-
+/*
     log.info("createInboundRoot started for /config");
     
     router.attachDefault(RestManager.ManagedEndpoint.class);
@@ -70,6 +70,7 @@ public class SolrConfigRestApi extends A
 
     log.info("createInboundRoot complete for /config");
 
-    return router;
+    return router;*/
+    return null;
   }  
 }

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/BBoxField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/BBoxField.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/BBoxField.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/BBoxField.java Sat Nov  8 11:32:18 2014
@@ -103,7 +103,7 @@ public class BBoxField extends AbstractS
     //and annoyingly this field isn't going to have a docValues format because Solr uses a separate Field for that
     if (field.hasDocValues()) {
       luceneType = new org.apache.lucene.document.FieldType(luceneType);
-      luceneType.setDocValueType(DocValuesType.NUMERIC);
+      luceneType.setDocValuesType(DocValuesType.NUMERIC);
     }
     strategy.setFieldType(luceneType);
     return strategy;

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/FieldType.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/FieldType.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/FieldType.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/FieldType.java Sat Nov  8 11:32:18 2014
@@ -303,7 +303,7 @@ public abstract class FieldType extends 
    */
   public List<StorableField> createFields(SchemaField field, Object value, float boost) {
     StorableField f = createField( field, value, boost);
-    if (field.hasDocValues() && f.fieldType().docValueType() == null) {
+    if (field.hasDocValues() && f.fieldType().docValuesType() == null) {
       // field types that support doc values should either override createField
       // to return a field with doc values or extend createFields if this can't
       // be done in a single field instance (see StrField for example)

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/IndexSchema.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/IndexSchema.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/IndexSchema.java Sat Nov  8 11:32:18 2014
@@ -17,9 +17,34 @@
 
 package org.apache.solr.schema;
 
+import java.io.IOException;
+import java.io.Writer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.SortedSet;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.regex.Pattern;
+
+import javax.xml.xpath.XPath;
+import javax.xml.xpath.XPathConstants;
+import javax.xml.xpath.XPathExpressionException;
+
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.DelegatingAnalyzerWrapper;
+import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.MultiFields;
@@ -34,14 +59,14 @@ import org.apache.solr.common.SolrExcept
 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.core.Config;
+import org.apache.solr.core.SolrConfig;
+import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.response.SchemaXmlWriter;
 import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.util.DOMUtil;
-import org.apache.solr.core.SolrConfig;
-import org.apache.solr.core.Config;
-import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.search.similarities.DefaultSimilarityFactory;
+import org.apache.solr.util.DOMUtil;
 import org.apache.solr.util.plugin.SolrCoreAware;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -52,29 +77,6 @@ import org.w3c.dom.Node;
 import org.w3c.dom.NodeList;
 import org.xml.sax.InputSource;
 
-import javax.xml.xpath.XPath;
-import javax.xml.xpath.XPathConstants;
-import javax.xml.xpath.XPathExpressionException;
-
-import java.io.IOException;
-import java.io.Writer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Set;
-import java.util.SortedMap;
-import java.util.SortedSet;
-import java.util.TreeMap;
-import java.util.TreeSet;
-import java.util.regex.Pattern;
-
 import static java.util.Collections.singletonList;
 import static java.util.Collections.singletonMap;
 
@@ -369,7 +371,7 @@ public class IndexSchema {
   public Map<String,UninvertingReader.Type> getUninversionMap(IndexReader reader) {
     Map<String,UninvertingReader.Type> map = new HashMap<>();
     for (FieldInfo f : MultiFields.getMergedFieldInfos(reader)) {
-      if (f.hasDocValues() == false && f.isIndexed()) {
+      if (f.getDocValuesType() == DocValuesType.NONE && f.getIndexOptions() != IndexOptions.NONE) {
         SchemaField sf = getFieldOrNull(f.name);
         if (sf != null) {
           UninvertingReader.Type type = sf.getType().getUninversionType(sf);

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java Sat Nov  8 11:32:18 2014
@@ -165,7 +165,7 @@ public final class ManagedIndexSchema ex
     final ZkSolrResourceLoader zkLoader = (ZkSolrResourceLoader)loader;
     final ZkController zkController = zkLoader.getZkController();
     final SolrZkClient zkClient = zkController.getZkClient();
-    final String managedSchemaPath = zkLoader.getCollectionZkPath() + "/" + managedSchemaResourceName;
+    final String managedSchemaPath = zkLoader.getConfigSetZkPath() + "/" + managedSchemaResourceName;
     boolean success = true;
     boolean schemaChangedInZk = false;
     try {

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchemaFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchemaFactory.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchemaFactory.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchemaFactory.java Sat Nov  8 11:32:18 2014
@@ -97,7 +97,7 @@ public class ManagedIndexSchemaFactory e
    * the instantiated IndexSchema is persisted to the managed schema file named in the
    * managedSchemaResourceName param, in the directory given by 
    * {@link org.apache.solr.core.SolrResourceLoader#getConfigDir()}, or if configs are
-   * in ZooKeeper, under {@link org.apache.solr.cloud.ZkSolrResourceLoader#collectionZkPath}.
+   * in ZooKeeper, under {@link org.apache.solr.cloud.ZkSolrResourceLoader#getConfigSetZkPath()}.
    *
    * After the managed schema file is persisted, the original schema file is
    * renamed by appending the extension named in {@link #UPGRADED_SCHEMA_EXTENSION}.
@@ -119,7 +119,7 @@ public class ManagedIndexSchemaFactory e
     } else { // ZooKeeper
       final ZkSolrResourceLoader zkLoader = (ZkSolrResourceLoader)loader;
       final SolrZkClient zkClient = zkLoader.getZkController().getZkClient();
-      final String managedSchemaPath = zkLoader.getCollectionZkPath() + "/" + managedSchemaResourceName;
+      final String managedSchemaPath = zkLoader.getConfigSetZkPath() + "/" + managedSchemaResourceName;
       Stat stat = new Stat();
       try {
         // Attempt to load the managed schema
@@ -224,7 +224,7 @@ public class ManagedIndexSchemaFactory e
       SolrResourceLoader loader = config.getResourceLoader();
       if (loader instanceof ZkSolrResourceLoader) {
         ZkSolrResourceLoader zkLoader = (ZkSolrResourceLoader)loader;
-        String nonManagedSchemaPath = zkLoader.getCollectionZkPath() + "/" + resourceName;
+        String nonManagedSchemaPath = zkLoader.getConfigSetZkPath() + "/" + resourceName;
         try {
           exists = zkLoader.getZkController().pathExists(nonManagedSchemaPath);
         } catch (InterruptedException e) {
@@ -349,7 +349,7 @@ public class ManagedIndexSchemaFactory e
     } else {
       // Rename the non-managed schema znode in ZooKeeper
       ZkSolrResourceLoader zkLoader = (ZkSolrResourceLoader)loader;
-      final String nonManagedSchemaPath = zkLoader.getCollectionZkPath() + "/" + resourceName;
+      final String nonManagedSchemaPath = zkLoader.getConfigSetZkPath() + "/" + resourceName;
       try {
         ZkController zkController = zkLoader.getZkController();
         ZkCmdExecutor zkCmdExecutor = new ZkCmdExecutor(zkController.getClientTimeout());

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/ZkIndexSchemaReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/ZkIndexSchemaReader.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/ZkIndexSchemaReader.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/ZkIndexSchemaReader.java Sat Nov  8 11:32:18 2014
@@ -43,7 +43,7 @@ public class ZkIndexSchemaReader impleme
     this.managedIndexSchemaFactory = managedIndexSchemaFactory;
     ZkSolrResourceLoader zkLoader = (ZkSolrResourceLoader)managedIndexSchemaFactory.getResourceLoader();
     this.zkClient = zkLoader.getZkController().getZkClient();
-    managedSchemaPath = zkLoader.getCollectionZkPath() + "/" + managedIndexSchemaFactory.getManagedSchemaResourceName();
+    managedSchemaPath = zkLoader.getConfigSetZkPath() + "/" + managedIndexSchemaFactory.getManagedSchemaResourceName();
     createSchemaWatcher();
     zkLoader.getZkController().addOnReconnectListener(this);
   }

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/search/CacheConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/search/CacheConfig.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/search/CacheConfig.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/search/CacheConfig.java Sat Nov  8 11:32:18 2014
@@ -17,9 +17,14 @@
 
 package org.apache.solr.search;
 
+import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.core.MapSerializable;
 import org.w3c.dom.Node;
 import org.w3c.dom.NodeList;
 
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 
 import org.apache.solr.common.SolrException;
@@ -36,7 +41,7 @@ import javax.xml.xpath.XPathConstants;
  *
  *
  */
-public class CacheConfig {
+public class CacheConfig implements MapSerializable{
   private String nodeName;
 
   private Class<? extends SolrCache> clazz;
@@ -70,7 +75,7 @@ public class CacheConfig {
     if (nodes==null || nodes.getLength()==0) return null;
     CacheConfig[] configs = new CacheConfig[nodes.getLength()];
     for (int i=0; i<nodes.getLength(); i++) {
-      configs[i] = getConfig(solrConfig, nodes.item(i));
+      configs[i] = getConfig(solrConfig, nodes.item(i).getNodeName(), DOMUtil.toMap(nodes.item(i).getAttributes()), configPath);
     }
     return configs;
   }
@@ -78,15 +83,29 @@ public class CacheConfig {
 
   public static CacheConfig getConfig(SolrConfig solrConfig, String xpath) {
     Node node = solrConfig.getNode(xpath, false);
-    return getConfig(solrConfig, node);
+    if(node == null) {
+      Map<String, String> m = solrConfig.getOverlay().getEditableSubProperties(xpath);
+      if(m==null) return null;
+      List<String> parts = StrUtils.splitSmart(xpath, '/');
+      return getConfig(solrConfig,parts.get(parts.size()-1) , Collections.EMPTY_MAP,xpath);
+    }
+    return getConfig(solrConfig, node.getNodeName(),DOMUtil.toMap(node.getAttributes()), xpath);
   }
 
 
-  public static CacheConfig getConfig(SolrConfig solrConfig, Node node) {
-    if (node==null) return null;
+  public static CacheConfig getConfig(SolrConfig solrConfig, String nodeName, Map<String,String> attrs, String xpath) {
     CacheConfig config = new CacheConfig();
-    config.nodeName = node.getNodeName();
-    config.args = DOMUtil.toMap(node.getAttributes());
+    config.nodeName = nodeName;
+    config.args = attrs;
+
+    Map<String, String> map = solrConfig.getOverlay().getEditableSubProperties(xpath);
+    if(map != null){
+      HashMap<String, String> mapCopy = new HashMap<>(config.args);
+      for (Map.Entry<String, String> e : map.entrySet()) {
+        mapCopy.put(e.getKey(),String.valueOf(e.getValue()));
+      }
+      config.args = mapCopy;
+    }
     String nameAttr = config.args.get("name");  // OPTIONAL
     if (nameAttr==null) {
       config.args.put("name",config.nodeName);
@@ -94,6 +113,7 @@ public class CacheConfig {
 
     SolrResourceLoader loader = solrConfig.getResourceLoader();
     config.cacheImpl = config.args.get("class");
+    if(config.cacheImpl == null) config.cacheImpl = "solr.LRUCache";
     config.regenImpl = config.args.get("regenerator");
     config.clazz = loader.findClass(config.cacheImpl, SolrCache.class);
     if (config.regenImpl != null) {
@@ -116,4 +136,15 @@ public class CacheConfig {
     }
   }
 
+  @Override
+  public Map<String, Object> toMap() {
+    Map result = Collections.unmodifiableMap(args);
+    return result;
+  }
+
+  public String getNodeName() {
+    return nodeName;
+  }
+
+
 }

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/search/mlt/CloudMLTQParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/search/mlt/CloudMLTQParser.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/search/mlt/CloudMLTQParser.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/search/mlt/CloudMLTQParser.java Sat Nov  8 11:32:18 2014
@@ -27,11 +27,15 @@ import org.apache.solr.core.SolrCore;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequestBase;
 import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.schema.SchemaField;
 import org.apache.solr.search.QParser;
 import org.apache.solr.search.QueryParsing;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -42,6 +46,8 @@ public class CloudMLTQParser extends QPa
     super(qstr, localParams, params, req);
   }
 
+  private static Logger log = LoggerFactory
+      .getLogger(CloudMLTQParser.class);
   public Query parse() {
     String id = localParams.get(QueryParsing.V);
     // Do a Real Time Get for the document
@@ -61,17 +67,27 @@ public class CloudMLTQParser extends QPa
     mlt.setAnalyzer(req.getSchema().getIndexAnalyzer());
 
     String[] qf = localParams.getParams("qf");
-    Map<String, ArrayList<String>> filteredDocument = new HashMap();
+    Map<String, Collection<Object>> filteredDocument = new HashMap();
 
     if (qf != null) {
       mlt.setFieldNames(qf);
       for (String field : qf) {
-        filteredDocument.put(field, (ArrayList<String>) doc.get(field));
+        filteredDocument.put(field, doc.getFieldValues(field));
       }
     } else {
+      Map<String, SchemaField> fields = req.getSchema().getFields();
+      ArrayList<String> fieldNames = new ArrayList();
       for (String field : doc.getFieldNames()) {
-        filteredDocument.put(field, (ArrayList<String>) doc.get(field));
+        // Only use fields that are stored and have an explicit analyzer.
+        // This makes sense as the query uses tf/idf/.. for query construction.
+        // We might want to relook and change this in the future though.
+        if(fields.get(field).stored() 
+            && fields.get(field).getType().isExplicitAnalyzer()) {
+          fieldNames.add(field);
+          filteredDocument.put(field, doc.getFieldValues(field));
+        }
       }
+      mlt.setFieldNames(fieldNames.toArray(new String[fieldNames.size()]));
     }
 
     try {

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java Sat Nov  8 11:32:18 2014
@@ -344,7 +344,7 @@ public class SolrDispatchFilter extends 
 
           // Handle /schema/* and /config/* paths via Restlet
           if( path.equals("/schema") || path.startsWith("/schema/")
-              || path.equals("/config") || path.startsWith("/config/")) {
+              /*|| path.equals("/config") || path.startsWith("/config/")*/) {
             solrReq = parser.parse(core, path, req);
             SolrRequestInfo.setRequestInfo(new SolrRequestInfo(solrReq, new SolrQueryResponse()));
             if( path.equals(req.getServletPath()) ) {

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java Sat Nov  8 11:32:18 2014
@@ -24,7 +24,9 @@ import org.apache.lucene.util.PrintStrea
 import org.apache.lucene.util.Version;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.MapSerializable;
 import org.apache.solr.core.SolrConfig;
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.schema.IndexSchema;
@@ -36,12 +38,13 @@ import java.io.File;
 import java.io.FileOutputStream;
 import java.io.PrintStream;
 import java.util.List;
+import java.util.Map;
 
 /**
  * This config object encapsulates IndexWriter config params,
  * defined in the &lt;indexConfig&gt; section of solrconfig.xml
  */
-public class SolrIndexConfig {
+public class SolrIndexConfig implements MapSerializable {
   public static final Logger log = LoggerFactory.getLogger(SolrIndexConfig.class);
   
   final String defaultMergePolicyClassName;
@@ -173,6 +176,19 @@ public class SolrIndexConfig {
 
     checkIntegrityAtMerge = solrConfig.getBool(prefix + "/checkIntegrityAtMerge", def.checkIntegrityAtMerge);
   }
+  @Override
+  public Map<String, Object> toMap() {
+    Map<String, Object> m = ZkNodeProps.makeMap("maxBufferedDocs", maxBufferedDocs,
+        "maxMergeDocs", maxMergeDocs,
+        "maxIndexingThreads", maxIndexingThreads,
+        "mergeFactor", mergeFactor,
+        "ramBufferSizeMB", ramBufferSizeMB,
+        "writeLockTimeout", writeLockTimeout,
+        "lockType", lockType);
+    if(mergeSchedulerInfo != null) m.put("mergeScheduler",mergeSchedulerInfo.toMap());
+    if(mergePolicyInfo != null) m.put("mergeScheduler",mergePolicyInfo.toMap());
+    return m;
+  }
 
   /*
    * Assert that assertCondition is true.

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/util/CommandOperation.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/util/CommandOperation.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/util/CommandOperation.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/util/CommandOperation.java Sat Nov  8 11:32:18 2014
@@ -56,13 +56,13 @@ public  class CommandOperation {
     if (commandData instanceof Map) {
       return (Map) commandData;
     }
-    addError(MessageFormat.format("The command {0} should have the values as a json object {key:val} format", name));
+    addError(MessageFormat.format("The command ''{0}'' should have the values as a json object {key:val} format", name));
     return Collections.EMPTY_MAP;
   }
 
   private Object getRootPrimitive(){
     if (commandData instanceof Map) {
-      errors.add(MessageFormat.format("The value has to be a string for command : {1}",name));
+      errors.add(MessageFormat.format("The value has to be a string for command : ''{0}'' ",name));
       return null;
     }
     return commandData;
@@ -99,7 +99,12 @@ public  class CommandOperation {
    * single value collection is returned
    */
   public List<String> getStrs(String key, List<String> def){
-    Object v = getMapVal(key);
+    Object v = null;
+    if(ROOT_OBJ.equals(key)) {
+      v = getRootPrimitive();
+    } else {
+      v = getMapVal(key);
+    }
     if(v == null){
       return def;
     } else {

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/util/PivotListEntry.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/util/PivotListEntry.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/util/PivotListEntry.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/util/PivotListEntry.java Sat Nov  8 11:32:18 2014
@@ -17,6 +17,10 @@ package org.apache.solr.util;
  * limitations under the License.
  */
 
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.util.NamedList;
+
 import java.util.Locale;
 
 /**
@@ -24,16 +28,28 @@ import java.util.Locale;
  */
 public enum PivotListEntry {
   
-  FIELD(0), 
+  // mandatory entries with exact indexes
+  FIELD(0),
   VALUE(1),
   COUNT(2),
-  PIVOT(3);
+  // optional entries
+  PIVOT,
+  STATS;
   
-  // we could just use the ordinal(), but safer to be very explicit
-  private final int index;
+  private static final int MIN_INDEX_OF_OPTIONAL = 3;
+
+  /** 
+   * Given a NamedList representing a Pivot Value, this is Minimum Index at 
+   * which this PivotListEntry may exist 
+   */
+  private final int minIndex;
   
-  private PivotListEntry(int index) {
-    this.index = index;
+  private PivotListEntry() {
+    this.minIndex = MIN_INDEX_OF_OPTIONAL;
+  }
+  private PivotListEntry(int minIndex) {
+    assert minIndex < MIN_INDEX_OF_OPTIONAL;
+    this.minIndex = minIndex;
   }
   
   /**
@@ -53,10 +69,19 @@ public enum PivotListEntry {
   }
   
   /**
-   * Indec of this entry when used in response
+   * Given a {@link NamedList} representing a Pivot Value, extracts the Object 
+   * which corrisponds to this {@link PivotListEntry}, or returns null if not found.
    */
-  public int getIndex() {
-    return index;
+  public Object extract(NamedList<Object> pivotList) {
+    if (this.minIndex < MIN_INDEX_OF_OPTIONAL) {
+      // a mandatory entry at an exact index.
+      assert this.getName().equals(pivotList.getName(this.minIndex));
+      assert this.minIndex < pivotList.size();
+      return pivotList.getVal(this.minIndex);
+    }
+    // otherweise...
+    // scan starting at the min/optional index
+    return pivotList.get(this.getName(), this.minIndex);
   }
 
 }

Modified: lucene/dev/branches/lucene6005/solr/core/src/test-files/solr/collection1/conf/schema.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/test-files/solr/collection1/conf/schema.xml?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/test-files/solr/collection1/conf/schema.xml (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/test-files/solr/collection1/conf/schema.xml Sat Nov  8 11:32:18 2014
@@ -83,6 +83,7 @@
       -->
     <fieldtype name="date" class="solr.TrieDateField" precisionStep="0"/>
     <fieldtype name="tdate" class="solr.TrieDateField" precisionStep="6"/>
+    <fieldtype name="tdatedv" class="solr.TrieDateField" precisionStep="6" docValues="true" multiValued="true"/>
 
     <fieldtype name="dateRange" class="solr.DateRangeField" />
 
@@ -573,6 +574,7 @@
    <dynamicField name="*_tds" type="tdouble" indexed="true" stored="true" multiValued="false"/>
    <dynamicField name="*_tdt" type="tdate"  indexed="true"  stored="true"/>
    <dynamicField name="*_tdt1" type="tdate"  indexed="true"  stored="true" multiValued="false"/>
+   <dynamicField name="*_tdtdv" type="tdatedv"  indexed="true"  stored="true"/>
 
    <dynamicField name="*_sI" type="string"  indexed="true"  stored="false"/>
    <dynamicField name="*_sS" type="string"  indexed="false" stored="true"/>

Modified: lucene/dev/branches/lucene6005/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml Sat Nov  8 11:32:18 2014
@@ -48,4 +48,11 @@
   <requestHandler name="/update" class="solr.UpdateRequestHandler"/>
   <requestHandler name="/replication" class="solr.ReplicationHandler" startup="lazy" />
 
+  <requestHandler name="/dump" class="DumpRequestHandler" initParams="a">
+    <lst name="defaults">
+      <str name="a">${my.custom.variable.a:A}</str>
+      <str name="b">${my.custom.variable.b:B}</str>
+    </lst>
+  </requestHandler>
+
 </config>

Modified: lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/analysis/TestReversedWildcardFilterFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/analysis/TestReversedWildcardFilterFactory.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/analysis/TestReversedWildcardFilterFactory.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/analysis/TestReversedWildcardFilterFactory.java Sat Nov  8 11:32:18 2014
@@ -16,9 +16,7 @@ package org.apache.solr.analysis;
  * limitations under the License.
  */
 
-
 import java.io.IOException;
-import java.lang.reflect.Field;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -26,8 +24,8 @@ import org.apache.lucene.analysis.Analyz
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.search.AutomatonQuery;
 import org.apache.lucene.search.Query;
-import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.Automaton;
+import org.apache.lucene.util.automaton.Operations;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.schema.IndexSchema;
@@ -161,7 +159,8 @@ public class TestReversedWildcardFilterF
       return false;
     }
     Automaton automaton = ((AutomatonQuery) q).getAutomaton();
-    String prefix = Operations.getCommonPrefix(Operations.determinize(automaton));
+    String prefix = Operations.getCommonPrefix(Operations.determinize(automaton,
+      Operations.DEFAULT_MAX_DETERMINIZED_STATES));
     return prefix.length() > 0 && prefix.charAt(0) == '\u0001';
   }
 

Modified: lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java Sat Nov  8 11:32:18 2014
@@ -356,7 +356,7 @@ public class CollectionsAPIDistributedZk
     assertTrue(response.isSuccess());
     coresStatus = response.getCollectionCoresStatus();
     assertEquals(0, (int) coresStatus.get("conf1_shard1_0_replica1").get("status"));
-    assertEquals(0, (int) coresStatus.get("conf1_shard1_0_replica1").get("status"));
+    assertEquals(0, (int) coresStatus.get("conf1_shard1_1_replica1").get("status"));
 
     deleteCollectionRequest = new CollectionAdminRequest.Delete();
     deleteCollectionRequest.setCollectionName("conf1");

Modified: lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/cloud/TestCloudPivotFacet.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/cloud/TestCloudPivotFacet.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/cloud/TestCloudPivotFacet.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/cloud/TestCloudPivotFacet.java Sat Nov  8 11:32:18 2014
@@ -16,17 +16,22 @@
  */
 package org.apache.solr.cloud;
 
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.lucene.util.CollectionUtil;
 import org.apache.lucene.util.TestUtil;
 import org.apache.solr.SolrTestCaseJ4.SuppressSSL;
 import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.response.FieldStatsInfo;
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.client.solrj.response.PivotField;
 import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.params.StatsParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.schema.TrieDateField;
 
+import org.apache.solr.common.params.FacetParams; // jdoc lint
 import static org.apache.solr.common.params.FacetParams.*;
 
 import org.apache.commons.lang.StringUtils;
@@ -92,6 +97,9 @@ public class TestCloudPivotFacet extends
 
   @Override
   public void doTest() throws Exception {
+
+    sanityCheckAssertDoubles();
+
     waitForThingsToLevelOut(30000); // TODO: why whould we have to wait?
     // 
     handle.clear();
@@ -107,7 +115,7 @@ public class TestCloudPivotFacet extends
     for (int i = 1; i <= numDocs; i++) {
       SolrInputDocument doc = buildRandomDocument(i);
 
-      // not efficient, but it garuntees that even if people change buildRandomDocument
+      // not efficient, but it guarantees that even if people change buildRandomDocument
       // we'll always have the full list of fields w/o needing to keep code in sync
       fieldNameSet.addAll(doc.getFieldNames());
 
@@ -119,7 +127,7 @@ public class TestCloudPivotFacet extends
     assertTrue("WTF, bogus field exists?", fieldNameSet.add("bogus_not_in_any_doc_s"));
 
     final String[] fieldNames = fieldNameSet.toArray(new String[fieldNameSet.size()]);
-    Arrays.sort(fieldNames); // need determinism for buildRandomPivot calls
+    Arrays.sort(fieldNames); // need determinism when picking random fields
 
 
     for (int i = 0; i < 5; i++) {
@@ -134,10 +142,28 @@ public class TestCloudPivotFacet extends
         baseP.add("fq", "id:[* TO " + TestUtil.nextInt(random(),200,numDocs) + "]");
       }
 
-      ModifiableSolrParams pivotP = params(FACET,"true",
-                                           FACET_PIVOT, buildRandomPivot(fieldNames));
+      final boolean stats = random().nextBoolean();
+      if (stats) {
+        baseP.add(StatsParams.STATS, "true");
+        
+        // if we are doing stats, then always generated the same # of STATS_FIELD
+        // params, using multiple tags from a fixed set, but with diff fieldName values.
+        // later, each pivot will randomly pick a tag.
+        baseP.add(StatsParams.STATS_FIELD, "{!key=sk1 tag=st1,st2}" +
+                  pickRandomStatsFields(fieldNames));
+        baseP.add(StatsParams.STATS_FIELD, "{!key=sk2 tag=st2,st3}" +
+                  pickRandomStatsFields(fieldNames));
+        baseP.add(StatsParams.STATS_FIELD, "{!key=sk3 tag=st3,st4}" +
+                  pickRandomStatsFields(fieldNames));
+        // NOTE: there's a chance that some of those stats field names
+        // will be the same, but if so, all the better to test that edge case
+      }
+      
+      ModifiableSolrParams pivotP = params(FACET,"true");
+      pivotP.add(FACET_PIVOT, buildPivotParamValue(buildRandomPivot(fieldNames)));
+                 
       if (random().nextBoolean()) {
-        pivotP.add(FACET_PIVOT, buildRandomPivot(fieldNames));
+        pivotP.add(FACET_PIVOT, buildPivotParamValue(buildRandomPivot(fieldNames)));
       }
 
       // keep limit low - lots of unique values, and lots of depth in pivots
@@ -268,7 +294,7 @@ public class TestCloudPivotFacet extends
                                            params("fq", buildFilter(constraint)));
     List<PivotField> subPivots = null;
     try {
-      assertNumFound(pivotName, constraint.getCount(), p);
+      assertPivotData(pivotName, constraint, p); 
       subPivots = constraint.getPivot();
     } catch (Exception e) {
       throw new RuntimeException(pivotName + ": count query failed: " + p + ": " + 
@@ -286,6 +312,97 @@ public class TestCloudPivotFacet extends
   }
 
   /**
+   * Executes a query and compares the results with the data available in the 
+   * {@link PivotField} constraint -- this method is not recursive, and doesn't 
+   * check anything about the sub-pivots (if any).
+   *
+   * @param pivotName pivot name
+   * @param constraint filters on pivot
+   * @param params base solr parameters
+   */
+  private void assertPivotData(String pivotName, PivotField constraint, SolrParams params) 
+    throws SolrServerException {
+    
+    SolrParams p = SolrParams.wrapDefaults(params("rows","0"), params);
+    QueryResponse res = cloudClient.query(p);
+    String msg = pivotName + ": " + p;
+
+    assertNumFound(msg, constraint.getCount(), res);
+
+    if ( p.getBool(StatsParams.STATS, false) ) {
+      // only check stats if stats expected
+      assertPivotStats(msg, constraint, res);
+    }
+  }
+
+  /**
+   * Compare top level stats in response with stats from pivot constraint
+   */
+  private void assertPivotStats(String message, PivotField constraint, QueryResponse response) throws SolrServerException {
+
+    if (null == constraint.getFieldStatsInfo()) {
+      // no stats for this pivot, nothing to check
+
+      // TODO: use a trace param to know if/how-many to expect ?
+      log.info("No stats to check for => " + message);
+      return;
+    }
+    
+    Map<String, FieldStatsInfo> actualFieldStatsInfoMap = response.getFieldStatsInfo();
+
+    for (FieldStatsInfo pivotStats : constraint.getFieldStatsInfo().values()) {
+      String statsKey = pivotStats.getName();
+
+      FieldStatsInfo actualStats = actualFieldStatsInfoMap.get(statsKey);
+
+      if (actualStats == null) {
+        // handle case for not found stats (using stats query)
+        //
+        // these has to be a special case check due to the legacy behavior of "top level" 
+        // StatsComponent results being "null" (and not even included in the 
+        // getFieldStatsInfo() Map due to specila SolrJ logic) 
+
+        log.info("Requested stats missing in verification query, pivot stats: " + pivotStats);
+        assertEquals("Special Count", 0L, pivotStats.getCount().longValue());
+        assertEquals("Special Missing", 
+                     constraint.getCount(), pivotStats.getMissing().longValue());
+
+      } else {
+        // regular stats, compare everything...
+
+        assert actualStats != null;
+        String msg = " of " + statsKey + " => " + message;
+        
+        assertEquals("Min" + msg, pivotStats.getMin(), actualStats.getMin());
+        assertEquals("Max" + msg, pivotStats.getMax(), actualStats.getMax());
+        assertEquals("Mean" + msg, pivotStats.getMean(), actualStats.getMean());
+        assertEquals("Sum" + msg, pivotStats.getSum(), actualStats.getSum());
+        assertEquals("Count" + msg, pivotStats.getCount(), actualStats.getCount());
+        assertEquals("Missing" + msg, pivotStats.getMissing(), actualStats.getMissing());
+        
+        assertDoubles("Stddev" + msg, pivotStats.getStddev(), actualStats.getStddev());
+        assertDoubles("SumOfSquares" + msg, 
+                      pivotStats.getSumOfSquares(), actualStats.getSumOfSquares());
+      }
+    }
+
+    if (constraint.getFieldStatsInfo().containsKey("sk2")) { // cheeseball hack
+      // if "sk2" was one of hte stats we computed, then we must have also seen
+      // sk1 or sk3 because of the way the tags are fixed
+      assertEquals("had stats sk2, but not another stat?", 
+                   2, constraint.getFieldStatsInfo().size());
+    } else {
+      // if we did not see "sk2", then 1 of the others must be alone
+      assertEquals("only expected 1 stat",
+                   1, constraint.getFieldStatsInfo().size());
+      assertTrue("not sk1 or sk3", 
+                 constraint.getFieldStatsInfo().containsKey("sk1") ||
+                 constraint.getFieldStatsInfo().containsKey("sk3"));
+    }
+
+  }
+
+  /**
    * Verify that the PivotFields we're lookin at doesn't violate any of the expected 
    * behaviors based on the <code>TRACE_*</code> params found in the base params
    */
@@ -364,6 +481,39 @@ public class TestCloudPivotFacet extends
     return StringUtils.join(fields, ",");
   }
 
+  /**
+   * Picks a random field to use for Stats
+   */
+  private static String pickRandomStatsFields(String[] fieldNames) {
+    // we need to skip boolean fields when computing stats
+    String fieldName;
+    do {
+      fieldName = fieldNames[TestUtil.nextInt(random(),0,fieldNames.length-1)];
+    }
+    while(fieldName.endsWith("_b") || fieldName.endsWith("_b1")) ;
+          
+    return fieldName;
+  }
+
+  /**
+   * Generates a random {@link FacetParams#FACET_PIVOT} value w/ local params 
+   * using the specified pivotValue.
+   */
+  private static String buildPivotParamValue(String pivotValue) {
+    // randomly decide which stat tag to use
+
+    // if this is 0, or stats aren't enabled, we'll be asking for a tag that doesn't exist
+    // ...which should be fine (just like excluding a taged fq that doesn't exist)
+    final int statTag = TestUtil.nextInt(random(), -1, 4);
+      
+    if (0 <= statTag) {
+      // only use 1 tag name in the 'stats' localparam - see SOLR-6663
+      return "{!stats=st"+statTag+"}" + pivotValue;
+    } else {
+      // statTag < 0 == sanity check the case of a pivot w/o any stats
+      return pivotValue;
+    }
+  }
 
   /**
    * Creates a document with randomized field values, some of which be missing values, 
@@ -512,16 +662,80 @@ public class TestCloudPivotFacet extends
   }
   
   /**
-   * Asserts the number of docs matching the SolrParams aganst the cloudClient
+   * Asserts the number of docs found in the response
    */
-  private void assertNumFound(String msg, int expected, SolrParams p) 
+  private void assertNumFound(String msg, int expected, QueryResponse response) 
     throws SolrServerException {
 
     countNumFoundChecks++;
 
-    SolrParams params = SolrParams.wrapDefaults(params("rows","0"), p);
-    assertEquals(msg + ": " + params, 
-                 expected, cloudClient.query(params).getResults().getNumFound());
+    assertEquals(msg, expected, response.getResults().getNumFound());
+  }
+
+  /**
+   * Given two objects, asserts that they are either both null, or both Numbers
+   * with double values that are equally-ish with a "small" epsilon (relative to the 
+   * scale of the expected value)
+   *
+   * @see Number#doubleValue
+   */
+  private void assertDoubles(String msg, Object expected, Object actual) {
+    if (null == expected || null == actual) {
+      assertEquals(msg, expected, actual);
+    } else {
+      assertTrue(msg + " ... expected not a double: " + 
+                 expected + "=>" + expected.getClass(),
+                 expected instanceof Number);
+      assertTrue(msg + " ... actual not a double: " + 
+                 actual + "=>" + actual.getClass(),
+                 actual instanceof Number);
+
+      // compute an epsilon relative to the size of the expected value
+      double expect = ((Number)expected).doubleValue();
+      double epsilon = expect * 0.1E-7D;
+
+      assertEquals(msg, expect, ((Number)actual).doubleValue(), epsilon);
+                   
+    }
+  }
+
+  /**
+   * test the test
+   */
+  private void sanityCheckAssertDoubles() {
+    assertDoubles("Null?", null, null);
+    assertDoubles("big", 
+                  new Double(2.3005390038169265E9), 
+                  new Double(2.300539003816927E9));
+    assertDoubles("small", 
+                  new Double(2.3005390038169265E-9), 
+                  new Double(2.300539003816927E-9));
+    try {
+      assertDoubles("non-null", null, 42);
+      fail("expected was null");
+    } catch (AssertionError e) {}
+    try {
+      assertDoubles("non-null", 42, null);
+      fail("actual was null");
+    } catch (AssertionError e) {}
+    try {
+      assertDoubles("non-number", 42, "foo");
+      fail("actual was non-number");
+    } catch (AssertionError e) {}
+    try {
+      assertDoubles("diff", 
+                    new Double(2.3005390038169265E9), 
+                    new Double(2.267272520100462E9));
+      fail("big & diff");
+    } catch (AssertionError e) {}
+    try {
+      assertDoubles("diff", 
+                    new Double(2.3005390038169265E-9), 
+                    new Double(2.267272520100462E-9));
+      fail("small & diff");
+    } catch (AssertionError e) {}
+
+
   }
 
   /**
@@ -529,4 +743,5 @@ public class TestCloudPivotFacet extends
    * @see #assertPivotCountsAreCorrect(SolrParams,SolrParams)
    */
   private int countNumFoundChecks = 0;
+
 }

Modified: lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/cloud/TestCollectionAPI.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/cloud/TestCollectionAPI.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/cloud/TestCollectionAPI.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/cloud/TestCollectionAPI.java Sat Nov  8 11:32:18 2014
@@ -35,7 +35,7 @@ import org.apache.solr.common.util.Named
 import org.apache.zookeeper.KeeperException;
 import org.junit.Before;
 
-import static org.apache.solr.cloud.OverseerCollectionProcessor.SLICE_UNIQUE;
+import static org.apache.solr.cloud.OverseerCollectionProcessor.SHARD_UNIQUE;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -429,7 +429,7 @@ public class TestCollectionAPI extends R
           "replica", c1_s1_r1,
           "property", "testprop",
           "property.value", "nonsense",
-          SLICE_UNIQUE, "true");
+          SHARD_UNIQUE, "true");
 
       verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "property.preferredleader", "true");
       verifyPropertyVal(client, COLLECTION_NAME, c1_s2_r1, "property.preferredleader", "true");
@@ -446,7 +446,7 @@ public class TestCollectionAPI extends R
           "replica", c1_s1_r1,
           "property", "property.testprop",
           "property.value", "true",
-          SLICE_UNIQUE, "false");
+          SHARD_UNIQUE, "false");
 
       verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "property.preferredleader", "true");
       verifyPropertyVal(client, COLLECTION_NAME, c1_s2_r1, "property.preferredleader", "true");
@@ -477,11 +477,11 @@ public class TestCollectionAPI extends R
             "replica", c1_s1_r1,
             "property", "preferredLeader",
             "property.value", "true",
-            SLICE_UNIQUE, "false");
-        fail("Should have thrown an exception, setting sliceUnique=false is not allowed for 'preferredLeader'.");
+            SHARD_UNIQUE, "false");
+        fail("Should have thrown an exception, setting shardUnique=false is not allowed for 'preferredLeader'.");
       } catch (SolrException se) {
         assertTrue("Should have received a specific error message",
-            se.getMessage().contains("with the sliceUnique parameter set to something other than 'true'"));
+            se.getMessage().contains("with the shardUnique parameter set to something other than 'true'"));
       }
 
       verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "property.preferredleader", "true");

Modified: lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/cloud/TestReplicaProperties.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/cloud/TestReplicaProperties.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/cloud/TestReplicaProperties.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/cloud/TestReplicaProperties.java Sat Nov  8 11:32:18 2014
@@ -106,7 +106,7 @@ public class TestReplicaProperties exten
       client.connect();
       try {
         doPropertyAction(client,
-            "action", CollectionParams.CollectionAction.BALANCESLICEUNIQUE.toString(),
+            "action", CollectionParams.CollectionAction.BALANCESHARDUNIQUE.toString(),
             "property", "preferredLeader");
       } catch (SolrException se) {
         assertTrue("Should have seen missing required parameter 'collection' error",
@@ -114,30 +114,30 @@ public class TestReplicaProperties exten
       }
 
       doPropertyAction(client,
-          "action", CollectionParams.CollectionAction.BALANCESLICEUNIQUE.toString(),
+          "action", CollectionParams.CollectionAction.BALANCESHARDUNIQUE.toString(),
           "collection", COLLECTION_NAME,
           "property", "preferredLeader");
 
       verifyUniqueAcrossCollection(client, COLLECTION_NAME, "property.preferredleader");
 
       doPropertyAction(client,
-          "action", CollectionParams.CollectionAction.BALANCESLICEUNIQUE.toString(),
+          "action", CollectionParams.CollectionAction.BALANCESHARDUNIQUE.toString(),
           "collection", COLLECTION_NAME,
           "property", "property.newunique",
-          "sliceUnique", "true");
+          "shardUnique", "true");
       verifyUniqueAcrossCollection(client, COLLECTION_NAME, "property.newunique");
 
       try {
         doPropertyAction(client,
-            "action", CollectionParams.CollectionAction.BALANCESLICEUNIQUE.toString(),
+            "action", CollectionParams.CollectionAction.BALANCESHARDUNIQUE.toString(),
             "collection", COLLECTION_NAME,
             "property", "whatever",
-            "sliceUnique", "false");
+            "shardUnique", "false");
         fail("Should have thrown an exception here.");
       } catch (SolrException se) {
         assertTrue("Should have gotten a specific error message here",
             se.getMessage().contains("Balancing properties amongst replicas in a slice requires that the " +
-                "property be pre-defined as a unique property (e.g. 'preferredLeader') or that 'sliceUnique' be set to 'true'"));
+                "property be pre-defined as a unique property (e.g. 'preferredLeader') or that 'shardUnique' be set to 'true'"));
       }
       // Should be able to set non-unique-per-slice values in several places.
       Map<String, Slice> slices = client.getZkStateReader().getClusterState().getCollection(COLLECTION_NAME).getSlicesMap();
@@ -165,24 +165,24 @@ public class TestReplicaProperties exten
 
       try {
         doPropertyAction(client,
-            "action", CollectionParams.CollectionAction.BALANCESLICEUNIQUE.toString(),
+            "action", CollectionParams.CollectionAction.BALANCESHARDUNIQUE.toString(),
             "collection", COLLECTION_NAME,
             "property", "bogus1",
-            "sliceUnique", "false");
+            "shardUnique", "false");
         fail("Should have thrown parameter error here");
       } catch (SolrException se) {
         assertTrue("Should have caught specific exception ",
             se.getMessage().contains("Balancing properties amongst replicas in a slice requires that the property be " +
-                "pre-defined as a unique property (e.g. 'preferredLeader') or that 'sliceUnique' be set to 'true'"));
+                "pre-defined as a unique property (e.g. 'preferredLeader') or that 'shardUnique' be set to 'true'"));
       }
 
-      // Should have no effect despite the "sliceUnique" param being set.
+      // Should have no effect despite the "shardUnique" param being set.
 
       doPropertyAction(client,
-          "action", CollectionParams.CollectionAction.BALANCESLICEUNIQUE.toString(),
+          "action", CollectionParams.CollectionAction.BALANCESHARDUNIQUE.toString(),
           "collection", COLLECTION_NAME,
           "property", "property.bogus1",
-          "sliceUnique", "true");
+          "shardUnique", "true");
 
       verifyPropertyVal(client, COLLECTION_NAME,
           c1_s1_r1, "property.bogus1", "true");

Modified: lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/core/TestInitParams.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/core/TestInitParams.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/core/TestInitParams.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/core/TestInitParams.java Sat Nov  8 11:32:18 2014
@@ -32,7 +32,7 @@ public class TestInitParams extends Solr
     initCore("solrconfig-paramset.xml","schema.xml");
   }
   @Test
-  public void testComponentWithParamSet(){
+  public void testComponentWithInitParams(){
 
     for (String s : Arrays.asList("/dump1", "/dump3","/root/dump5" , "/root1/anotherlevel/dump6")) {
       SolrRequestHandler handler = h.getCore().getRequestHandler(s);
@@ -49,7 +49,7 @@ public class TestInitParams extends Solr
   }
 
   @Test
-  public void testMultiParamSet(){
+  public void testMultiInitParams(){
     SolrRequestHandler handler = h.getCore().getRequestHandler("/dump6");
     SolrQueryResponse rsp = new SolrQueryResponse();
     handler.handleRequest(req("initArgs", "true"), rsp);
@@ -65,7 +65,7 @@ public class TestInitParams extends Solr
   }
 
   @Test
-  public void testComponentWithParamSetRequestParam(){
+  public void testComponentWithInitParamAndRequestParam(){
     for (String s : Arrays.asList("/dump4")) {
       SolrRequestHandler handler = h.getCore().getRequestHandler(s);
       SolrQueryResponse rsp = new SolrQueryResponse();
@@ -77,13 +77,13 @@ public class TestInitParams extends Solr
     }
   }
   @Test
-  public void testComponentWithConflictingParamSet(){
+  public void testComponentWithConflictingInitParams(){
     SolrRequestHandler handler = h.getCore().getRequestHandler("/dump2");
     SolrQueryResponse rsp = new SolrQueryResponse();
     handler.handleRequest(req("initArgs", "true"), rsp);
     NamedList nl = (NamedList) rsp.getValues().get("initArgs");
     NamedList def = (NamedList) nl.get(PluginInfo.DEFAULTS);
-    assertEquals("A" ,def.get("a"));
+    assertEquals("A1" ,def.get("a"));
     def = (NamedList) nl.get(PluginInfo.INVARIANTS);
     assertEquals("B1" ,def.get("b"));
     def = (NamedList) nl.get(PluginInfo.APPENDS);

Modified: lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotLargeTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotLargeTest.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotLargeTest.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotLargeTest.java Sat Nov  8 11:32:18 2014
@@ -24,6 +24,7 @@ import java.io.IOException;
 import org.apache.solr.BaseDistributedSearchTestCase;
 import org.apache.solr.client.solrj.SolrServer;
 import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.response.FieldStatsInfo;
 import org.apache.solr.client.solrj.response.PivotField;
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.common.SolrInputDocument;
@@ -665,7 +666,80 @@ public class DistributedFacetPivotLargeT
                 "facet.pivot","place_s,company_t",
                 FacetParams.FACET_OVERREQUEST_RATIO, "0",
                 FacetParams.FACET_OVERREQUEST_COUNT, "0");
-    
+
+    doTestDeepPivotStats();
+  }
+
+  private void doTestDeepPivotStats() throws Exception {
+
+    QueryResponse rsp = query("q", "*:*",
+                              "rows", "0",
+                              "facet", "true",
+                              "facet.pivot","{!stats=s1}place_s,company_t",
+                              "stats", "true",
+                              "stats.field", "{!key=avg_price tag=s1}pay_i");
+
+    List<PivotField> pivots = rsp.getFacetPivot().get("place_s,company_t");
+
+    PivotField cardiffPivotField = pivots.get(0);
+    assertEquals("cardiff", cardiffPivotField.getValue());
+    assertEquals(257, cardiffPivotField.getCount());
+
+    FieldStatsInfo cardiffStatsInfo = cardiffPivotField.getFieldStatsInfo().get("avg_price");
+    assertEquals("avg_price", cardiffStatsInfo.getName());
+    assertEquals(0.0, cardiffStatsInfo.getMin());
+    assertEquals(8742.0, cardiffStatsInfo.getMax());
+    assertEquals(257, (long) cardiffStatsInfo.getCount());
+    assertEquals(0, (long) cardiffStatsInfo.getMissing());
+    assertEquals(347554.0, cardiffStatsInfo.getSum());
+    assertEquals(8.20968772E8, cardiffStatsInfo.getSumOfSquares(), 0.1E-7);
+    assertEquals(1352.35019455253, (double) cardiffStatsInfo.getMean(), 0.1E-7);
+    assertEquals(1170.86048165857, cardiffStatsInfo.getStddev(), 0.1E-7);
+
+    PivotField bbcCardifftPivotField = cardiffPivotField.getPivot().get(0);
+    assertEquals("bbc", bbcCardifftPivotField.getValue());
+    assertEquals(101, bbcCardifftPivotField.getCount());
+
+    FieldStatsInfo bbcCardifftPivotFieldStatsInfo = bbcCardifftPivotField.getFieldStatsInfo().get("avg_price");
+    assertEquals(2400.0, bbcCardifftPivotFieldStatsInfo.getMin());
+    assertEquals(8742.0, bbcCardifftPivotFieldStatsInfo.getMax());
+    assertEquals(101, (long) bbcCardifftPivotFieldStatsInfo.getCount());
+    assertEquals(0, (long) bbcCardifftPivotFieldStatsInfo.getMissing());
+    assertEquals(248742.0, bbcCardifftPivotFieldStatsInfo.getSum());
+    assertEquals(6.52422564E8, bbcCardifftPivotFieldStatsInfo.getSumOfSquares(), 0.1E-7);
+    assertEquals(2462.792079208, (double) bbcCardifftPivotFieldStatsInfo.getMean(), 0.1E-7);
+    assertEquals(631.0525860312, bbcCardifftPivotFieldStatsInfo.getStddev(), 0.1E-7);
+
+
+    PivotField placeholder0PivotField = pivots.get(2);
+    assertEquals("0placeholder", placeholder0PivotField.getValue());
+    assertEquals(6, placeholder0PivotField.getCount());
+
+    FieldStatsInfo placeholder0PivotFieldStatsInfo = placeholder0PivotField.getFieldStatsInfo().get("avg_price");
+    assertEquals("avg_price", placeholder0PivotFieldStatsInfo.getName());
+    assertEquals(2000.0, placeholder0PivotFieldStatsInfo.getMin());
+    assertEquals(6400.0, placeholder0PivotFieldStatsInfo.getMax());
+    assertEquals(6, (long) placeholder0PivotFieldStatsInfo.getCount());
+    assertEquals(0, (long) placeholder0PivotFieldStatsInfo.getMissing());
+    assertEquals(22700.0, placeholder0PivotFieldStatsInfo.getSum());
+    assertEquals(1.0105E8, placeholder0PivotFieldStatsInfo.getSumOfSquares(), 0.1E-7);
+    assertEquals(3783.333333333, (double) placeholder0PivotFieldStatsInfo.getMean(), 0.1E-7);
+    assertEquals(1741.742422595, placeholder0PivotFieldStatsInfo.getStddev(), 0.1E-7);
+
+    PivotField microsoftPlaceholder0PivotField = placeholder0PivotField.getPivot().get(1);
+    assertEquals("microsoft", microsoftPlaceholder0PivotField.getValue());
+    assertEquals(6, microsoftPlaceholder0PivotField.getCount());
+
+    FieldStatsInfo microsoftPlaceholder0PivotFieldStatsInfo = microsoftPlaceholder0PivotField.getFieldStatsInfo().get("avg_price");
+    assertEquals("avg_price", microsoftPlaceholder0PivotFieldStatsInfo.getName());
+    assertEquals(2000.0, microsoftPlaceholder0PivotFieldStatsInfo.getMin());
+    assertEquals(6400.0, microsoftPlaceholder0PivotFieldStatsInfo.getMax());
+    assertEquals(6, (long) microsoftPlaceholder0PivotFieldStatsInfo.getCount());
+    assertEquals(0, (long) microsoftPlaceholder0PivotFieldStatsInfo.getMissing());
+    assertEquals(22700.0, microsoftPlaceholder0PivotFieldStatsInfo.getSum());
+    assertEquals(1.0105E8, microsoftPlaceholder0PivotFieldStatsInfo.getSumOfSquares(), 0.1E-7);
+    assertEquals(3783.333333333, (double) microsoftPlaceholder0PivotFieldStatsInfo.getMean(), 0.1E-7);
+    assertEquals(1741.742422595, microsoftPlaceholder0PivotFieldStatsInfo.getStddev(), 0.1E-7);
   }
 
   /**

Modified: lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotLongTailTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotLongTailTest.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotLongTailTest.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotLongTailTest.java Sat Nov  8 11:32:18 2014
@@ -27,9 +27,8 @@ import java.io.IOException;
 import org.apache.solr.BaseDistributedSearchTestCase;
 import org.apache.solr.client.solrj.SolrServer;
 import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.response.FieldStatsInfo;
 import org.apache.solr.client.solrj.response.PivotField;
-import org.apache.solr.client.solrj.response.QueryResponse;
-import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.params.FacetParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
@@ -68,9 +67,9 @@ public class DistributedFacetPivotLongTa
     // the 5 top foo_s terms have 100 docs each on every shard
     for (int i = 0; i < 100; i++) {
       for (int j = 0; j < 5; j++) {
-        shard0.add(sdoc("id", getDocNum(), "foo_s", "aaa"+j));
-        shard1.add(sdoc("id", getDocNum(), "foo_s", "aaa"+j));
-        shard2.add(sdoc("id", getDocNum(), "foo_s", "aaa"+j));
+        shard0.add(sdoc("id", getDocNum(), "foo_s", "aaa"+j, "stat_i", j * 13 - i));
+        shard1.add(sdoc("id", getDocNum(), "foo_s", "aaa"+j, "stat_i", j * 3 + i));
+        shard2.add(sdoc("id", getDocNum(), "foo_s", "aaa"+j, "stat_i", i * 7 + j));
       }
     }
 
@@ -78,14 +77,14 @@ public class DistributedFacetPivotLongTa
     // on both shard0 & shard1 ("bbb_")
     for (int i = 0; i < 50; i++) {
       for (int j = 0; j < 20; j++) {
-        shard0.add(sdoc("id", getDocNum(), "foo_s", "bbb"+j));
-        shard1.add(sdoc("id", getDocNum(), "foo_s", "bbb"+j));
+        shard0.add(sdoc("id", getDocNum(), "foo_s", "bbb"+j, "stat_i", 0));
+        shard1.add(sdoc("id", getDocNum(), "foo_s", "bbb"+j, "stat_i", 1));
       }
       // distracting term appears on only on shard2 50 times
       shard2.add(sdoc("id", getDocNum(), "foo_s", "junkA"));
     }
     // put "bbb0" on shard2 exactly once to sanity check refinement
-    shard2.add(sdoc("id", getDocNum(), "foo_s", "bbb0"));
+    shard2.add(sdoc("id", getDocNum(), "foo_s", "bbb0", "stat_i", -2));
 
     // long 'tail' foo_s term appears in 45 docs on every shard
     // foo_s:tail is the only term with bar_s sub-pivot terms
@@ -95,11 +94,12 @@ public class DistributedFacetPivotLongTa
       // but the top 5 terms are ccc(0-4) -- 7 on each shard
       // (4 docs each have junk terms)
       String sub_term = (i < 35) ? "ccc"+(i % 5) : ((i < 41) ? "tailB" : "junkA");
-      shard0.add(sdoc("id", getDocNum(), "foo_s", "tail", "bar_s", sub_term));
-      shard1.add(sdoc("id", getDocNum(), "foo_s", "tail", "bar_s", sub_term));
+      shard0.add(sdoc("id", getDocNum(), "foo_s", "tail", "bar_s", sub_term, "stat_i", i));
+      shard1.add(sdoc("id", getDocNum(), "foo_s", "tail", "bar_s", sub_term, "stat_i", i));
 
       // shard2's top 5 sub-pivot terms are junk only it has with 8 docs each
       // and 5 docs that use "tailB"
+      // NOTE: none of these get stat_i ! !
       sub_term = (i < 40) ? "junkB"+(i % 5) : "tailB";
       shard2.add(sdoc("id", getDocNum(), "foo_s", "tail", "bar_s", sub_term));
     }
@@ -175,7 +175,9 @@ public class DistributedFacetPivotLongTa
                                   FacetParams.FACET_OVERREQUEST_RATIO, "0",
                                   "facet", "true",
                                   "facet.limit", "6",
-                                  "facet.pivot", "foo_s,bar_s" ) 
+                                  "facet.pivot", "{!stats=sxy}foo_s,bar_s",
+                                  "stats", "true",
+                                  "stats.field", "{!tag=sxy}stat_i")
                           ).getFacetPivot().get("foo_s,bar_s");
     assertEquals(6, pivots.size());
     for (int i = 0; i < 5; i++) {
@@ -183,9 +185,23 @@ public class DistributedFacetPivotLongTa
       assertTrue(pivot.toString(), pivot.getValue().toString().startsWith("aaa"));
       assertEquals(pivot.toString(), 300, pivot.getCount());
     }
-    // even w/o the long tail, we should have still asked shard2 to refine bbb0
-    assertTrue(pivots.get(5).toString(), pivots.get(5).getValue().equals("bbb0"));
-    assertEquals(pivots.get(5).toString(), 101, pivots.get(5).getCount());
+    { // even w/o the long tail, we should have still asked shard2 to refine bbb0
+      pivot = pivots.get(5);
+      assertTrue(pivot.toString(), pivot.getValue().equals("bbb0"));
+      assertEquals(pivot.toString(), 101, pivot.getCount());
+      // basic check of refined stats
+      FieldStatsInfo bbb0Stats = pivot.getFieldStatsInfo().get("stat_i");
+      assertEquals("stat_i", bbb0Stats.getName());
+      assertEquals(-2.0, bbb0Stats.getMin());
+      assertEquals(1.0, bbb0Stats.getMax());
+      assertEquals(101, (long) bbb0Stats.getCount());
+      assertEquals(0, (long) bbb0Stats.getMissing());
+      assertEquals(48.0, bbb0Stats.getSum());
+      assertEquals(0.475247524752475, (double) bbb0Stats.getMean(), 0.1E-7);
+      assertEquals(54.0, bbb0Stats.getSumOfSquares(), 0.1E-7);
+      assertEquals(0.55846323792, bbb0Stats.getStddev(), 0.1E-7);
+    }
+
 
     // with default overrequesting, we should find the correct top 6 including 
     // long tail and top sub-pivots
@@ -284,6 +300,65 @@ public class DistributedFacetPivotLongTa
       assertTrue(pivot.toString(), pivot.getValue().toString().startsWith("ccc"));
       assertEquals(pivot.toString(), 14, pivot.getCount());
     }
+    
+    doTestDeepPivotStats();
+  }
+
+  public void doTestDeepPivotStats() throws Exception {
+    // Deep checking of some Facet stats - no refinement involved here
+
+    List<PivotField> pivots = 
+      query("q", "*:*",
+            "shards", getShardsString(),
+            "facet", "true",
+            "rows" , "0",
+            "facet.pivot","{!stats=s1}foo_s,bar_s",
+            "stats", "true",
+            "stats.field", "{!key=avg_price tag=s1}stat_i").getFacetPivot().get("foo_s,bar_s");
+    PivotField aaa0PivotField = pivots.get(0);
+    assertEquals("aaa0", aaa0PivotField.getValue());
+    assertEquals(300, aaa0PivotField.getCount());
+
+    FieldStatsInfo aaa0StatsInfo = aaa0PivotField.getFieldStatsInfo().get("avg_price");
+    assertEquals("avg_price", aaa0StatsInfo.getName());
+    assertEquals(-99.0, aaa0StatsInfo.getMin());
+    assertEquals(693.0, aaa0StatsInfo.getMax());
+    assertEquals(300, (long) aaa0StatsInfo.getCount());
+    assertEquals(0, (long) aaa0StatsInfo.getMissing());
+    assertEquals(34650.0, aaa0StatsInfo.getSum());
+    assertEquals(1.674585E7, aaa0StatsInfo.getSumOfSquares(), 0.1E-7);
+    assertEquals(115.5, (double) aaa0StatsInfo.getMean(), 0.1E-7);
+    assertEquals(206.4493184076, aaa0StatsInfo.getStddev(), 0.1E-7);
+
+    PivotField tailPivotField = pivots.get(5);
+    assertEquals("tail", tailPivotField.getValue());
+    assertEquals(135, tailPivotField.getCount());
+
+    FieldStatsInfo tailPivotFieldStatsInfo = tailPivotField.getFieldStatsInfo().get("avg_price");
+    assertEquals("avg_price", tailPivotFieldStatsInfo.getName());
+    assertEquals(0.0, tailPivotFieldStatsInfo.getMin());
+    assertEquals(44.0, tailPivotFieldStatsInfo.getMax());
+    assertEquals(90, (long) tailPivotFieldStatsInfo.getCount());
+    assertEquals(45, (long) tailPivotFieldStatsInfo.getMissing());
+    assertEquals(1980.0, tailPivotFieldStatsInfo.getSum());
+    assertEquals(22.0, (double) tailPivotFieldStatsInfo.getMean(), 0.1E-7);
+    assertEquals(58740.0, tailPivotFieldStatsInfo.getSumOfSquares(), 0.1E-7);
+    assertEquals(13.0599310011, tailPivotFieldStatsInfo.getStddev(), 0.1E-7);
+
+    PivotField tailBPivotField = tailPivotField.getPivot().get(0);
+    assertEquals("tailB", tailBPivotField.getValue());
+    assertEquals(17, tailBPivotField.getCount());
+
+    FieldStatsInfo tailBPivotFieldStatsInfo = tailBPivotField.getFieldStatsInfo().get("avg_price");
+    assertEquals("avg_price", tailBPivotFieldStatsInfo.getName());
+    assertEquals(35.0, tailBPivotFieldStatsInfo.getMin());
+    assertEquals(40.0, tailBPivotFieldStatsInfo.getMax());
+    assertEquals(12, (long) tailBPivotFieldStatsInfo.getCount());
+    assertEquals(5, (long) tailBPivotFieldStatsInfo.getMissing());
+    assertEquals(450.0, tailBPivotFieldStatsInfo.getSum());
+    assertEquals(37.5, (double) tailBPivotFieldStatsInfo.getMean(), 0.1E-7);
+    assertEquals(16910.0, tailBPivotFieldStatsInfo.getSumOfSquares(), 0.1E-7);
+    assertEquals(1.78376517, tailBPivotFieldStatsInfo.getStddev(), 0.1E-7);
   }
 
 }