lucene-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From hoss...@apache.org
Subject svn commit: r1636772 [1/2] - in /lucene/dev/trunk/solr: ./ core/src/java/org/apache/solr/handler/component/ core/src/java/org/apache/solr/util/ core/src/test/org/apache/solr/cloud/ core/src/test/org/apache/solr/handler/component/ solrj/src/java/org/apa...
Date Tue, 04 Nov 2014 23:09:42 GMT
Author: hossman
Date: Tue Nov  4 23:09:41 2014
New Revision: 1636772

URL: http://svn.apache.org/r1636772
Log:
SOLR-6351: Stats can now be nested under pivot values by adding a 'stats' local param

Added:
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotSmallAdvancedTest.java   (with props)
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotWhiteBoxTest.java   (with props)
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/component/FacetPivotSmallTest.java   (with props)
Modified:
    lucene/dev/trunk/solr/CHANGES.txt
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/PivotFacetHelper.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/PivotFacetProcessor.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/PivotFacetValue.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/StatsComponent.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/StatsField.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/PivotListEntry.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/TestCloudPivotFacet.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotLargeTest.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotLongTailTest.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotSmallTest.java
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/SolrQuery.java
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/response/FieldStatsInfo.java
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/response/PivotField.java
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/response/QueryResponse.java
    lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTests.java

Modified: lucene/dev/trunk/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=1636772&r1=1636771&r2=1636772&view=diff
==============================================================================
--- lucene/dev/trunk/solr/CHANGES.txt (original)
+++ lucene/dev/trunk/solr/CHANGES.txt Tue Nov  4 23:09:41 2014
@@ -201,6 +201,10 @@ New Features
 * SOLR-6670: change BALANCESLICEUNIQUE to BALANCESHARDUNIQUE. Also, the parameter
   for ADDREPLICAPROP that used to be sliceUnique is now shardUnique. (Erick Erickson)
 
+* SOLR-6351: Stats can now be nested under pivot values by adding a 'stats' local param to 
+  facet.pivot which refers to a 'tag' local param in one or more stats.field params.
+  (hossman, Vitaliy Zhovtyuk)
+
 Bug Fixes
 ----------------------
 

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/PivotFacetHelper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/PivotFacetHelper.java?rev=1636772&r1=1636771&r2=1636772&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/PivotFacetHelper.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/PivotFacetHelper.java Tue Nov  4 23:09:41 2014
@@ -18,15 +18,21 @@
 package org.apache.solr.handler.component;
 
 import org.apache.solr.util.PivotListEntry;
+import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.params.FacetParams;
 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.common.util.StrUtils;
 
 import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Collections;
+import java.util.Map;
+import java.util.Map.Entry;
 
 public class PivotFacetHelper {
 
@@ -91,31 +97,63 @@ public class PivotFacetHelper {
 
   /** @see PivotListEntry#VALUE */
   public static Comparable getValue(NamedList<Object> pivotList) {
-    return (Comparable) PivotFacetHelper.retrieve(PivotListEntry.VALUE,
-                                                  pivotList);
+    return (Comparable) PivotListEntry.VALUE.extract(pivotList);
   }
 
   /** @see PivotListEntry#FIELD */
   public static String getField(NamedList<Object> pivotList) {
-    return (String) PivotFacetHelper.retrieve(PivotListEntry.FIELD, pivotList);
+    return (String) PivotListEntry.FIELD.extract(pivotList);
   }
   
   /** @see PivotListEntry#COUNT */
   public static Integer getCount(NamedList<Object> pivotList) {
-    return (Integer) PivotFacetHelper.retrieve(PivotListEntry.COUNT, pivotList);
+    return (Integer) PivotListEntry.COUNT.extract(pivotList);
   }
 
   /** @see PivotListEntry#PIVOT */
   public static List<NamedList<Object>> getPivots(NamedList<Object> pivotList) {
-    int pivotIdx = pivotList.indexOf(PivotListEntry.PIVOT.getName(), 0);
-    if (pivotIdx > -1) {
-      return (List<NamedList<Object>>) pivotList.getVal(pivotIdx);
-    }
-    return null;
+    return (List<NamedList<Object>>) PivotListEntry.PIVOT.extract(pivotList);
   }
   
-  private static Object retrieve(PivotListEntry entryToGet, NamedList<Object> pivotList) {
-    return pivotList.get(entryToGet.getName(), entryToGet.getIndex());
+  /** @see PivotListEntry#STATS */
+  public static NamedList<NamedList<NamedList<?>>> getStats(NamedList<Object> pivotList) {
+    return (NamedList<NamedList<NamedList<?>>>) PivotListEntry.STATS.extract(pivotList);
+  }
+
+  /**
+   * Given a mapping of keys to {@link StatsValues} representing the currently 
+   * known "merged" stats (which may be null if none exist yet), and a 
+   * {@link NamedList} containing the "stats" response block returned by an individual 
+   * shard, this method accumulates the stasts for each {@link StatsField} found in 
+   * the shard response with the existing mergeStats
+   *
+   * @return the original <code>merged</code> Map after modifying, or a new Map if the <code>merged</code> param was originally null.
+   * @see StatsInfo#getStatsField
+   * @see StatsValuesFactory#createStatsValues
+   * @see StatsValues#accumulate(NamedList)
+   */
+  public static Map<String,StatsValues> mergeStats
+    (Map<String,StatsValues> merged, 
+     NamedList<NamedList<NamedList<?>>> remoteWrapper, 
+     StatsInfo statsInfo) {
+
+    if (null == merged) merged = new LinkedHashMap<String,StatsValues>();
+
+    NamedList<NamedList<?>> remoteStats = StatsComponent.unwrapStats(remoteWrapper);
+
+    for (Entry<String,NamedList<?>> entry : remoteStats) {
+      StatsValues receivingStatsValues = merged.get(entry.getKey());
+      if (receivingStatsValues == null) {
+        StatsField recievingStatsField = statsInfo.getStatsField(entry.getKey());
+        if (null == recievingStatsField) {
+          throw new SolrException(ErrorCode.SERVER_ERROR , "No stats.field found corrisponding to pivot stats recieved from shard: "+entry.getKey());
+        }
+        receivingStatsValues = StatsValuesFactory.createStatsValues(recievingStatsField);
+        merged.put(entry.getKey(), receivingStatsValues);
+      }
+      receivingStatsValues.accumulate(entry.getValue());
+    }
+    return merged;
   }
 
 }

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/PivotFacetProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/PivotFacetProcessor.java?rev=1636772&r1=1636771&r2=1636772&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/PivotFacetProcessor.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/PivotFacetProcessor.java Tue Nov  4 23:09:41 2014
@@ -23,20 +23,26 @@ import org.apache.solr.schema.FieldType;
 import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.search.DocSet;
 import org.apache.solr.search.SyntaxError;
+import org.apache.solr.util.PivotListEntry;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.params.ShardParams;
 import org.apache.solr.common.params.FacetParams;
+import org.apache.solr.common.params.StatsParams;
 import org.apache.solr.request.SimpleFacets;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.lucene.search.Query;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
@@ -63,9 +69,15 @@ public class PivotFacetProcessor extends
     if (!rb.doFacets || pivots == null) 
       return null;
     
+    // rb._statsInfo may be null if stats=false, ie: refine requests
+    // if that's the case, but we need to refine w/stats, then we'll lazy init our 
+    // own instance of StatsInfo
+    StatsInfo statsInfo = rb._statsInfo; 
+
     SimpleOrderedMap<List<NamedList<Object>>> pivotResponse = new SimpleOrderedMap<>();
     for (String pivotList : pivots) {
       try {
+        // NOTE: this sets localParams (SimpleFacets is stateful)
         this.parseParams(FacetParams.FACET_PIVOT, pivotList);
       } catch (SyntaxError e) {
         throw new SolrException(ErrorCode.BAD_REQUEST, e);
@@ -84,15 +96,37 @@ public class PivotFacetProcessor extends
         }
       } 
 
-      //REFINEMENT
-      String fieldValueKey = localParams == null ? null : localParams.get(PivotFacet.REFINE_PARAM);
-      if(fieldValueKey != null ){
-        String[] refinementValuesByField = params.getParams(PivotFacet.REFINE_PARAM+fieldValueKey);
+      // start by assuing no local params...
+
+      String refineKey = null; // no local => no refinement
+      List<StatsField> statsFields = Collections.emptyList(); // no local => no stats
+      
+      if (null != localParams) {
+        // we might be refining..
+        refineKey = localParams.get(PivotFacet.REFINE_PARAM);
+        
+        String statsLocalParam = localParams.get(StatsParams.STATS);
+        if (null != refineKey
+            && null != statsLocalParam
+            && null == statsInfo) {
+          // we are refining and need to compute stats, 
+          // but stats component hasn't inited StatsInfo (because we
+          // don't need/want top level stats when refining) so we lazy init
+          // our own copy of StatsInfo
+          statsInfo = new StatsInfo(rb);
+        }
+        statsFields = getTaggedStatsFields(statsInfo, statsLocalParam);
+      }
+
+      if (null != refineKey) {
+        String[] refinementValuesByField 
+          = params.getParams(PivotFacet.REFINE_PARAM + refineKey);
+
         for(String refinements : refinementValuesByField){
-          pivotResponse.addAll(processSingle(pivotFields, refinements));
+          pivotResponse.addAll(processSingle(pivotFields, refinements, statsFields));
         }
       } else{
-        pivotResponse.addAll(processSingle(pivotFields, null));
+        pivotResponse.addAll(processSingle(pivotFields, null, statsFields));
       }
     }
     return pivotResponse;
@@ -102,9 +136,13 @@ public class PivotFacetProcessor extends
    * Process a single branch of refinement values for a specific pivot
    * @param pivotFields the ordered list of fields in this pivot
    * @param refinements the comma seperate list of refinement values corrisponding to each field in the pivot, or null if there are no refinements
+   * @param statsFields List of {@link StatsField} instances to compute for each pivot value
    */
-  private SimpleOrderedMap<List<NamedList<Object>>> processSingle(List<String> pivotFields,
-                                                                  String refinements) throws IOException {
+  private SimpleOrderedMap<List<NamedList<Object>>> processSingle
+    (List<String> pivotFields,
+     String refinements,
+     List<StatsField> statsFields) throws IOException {
+
     SolrIndexSearcher searcher = rb.req.getSearcher();
     SimpleOrderedMap<List<NamedList<Object>>> pivotResponse = new SimpleOrderedMap<>();
 
@@ -141,18 +179,54 @@ public class PivotFacetProcessor extends
     if(pivotFields.size() > 1) {
       String subField = pivotFields.get(1);
       pivotResponse.add(key,
-                        doPivots(facetCounts, field, subField, fnames, vnames, this.docs));
+                        doPivots(facetCounts, field, subField, fnames, vnames, this.docs, statsFields));
     } else {
-      pivotResponse.add(key, doPivots(facetCounts, field, null, fnames, vnames, this.docs));
+      pivotResponse.add(key, doPivots(facetCounts, field, null, fnames, vnames, this.docs, statsFields));
     }
     return pivotResponse;
   }
   
   /**
+   * returns the {@link StatsField} instances that should be computed for a pivot
+   * based on the 'stats' local params used.
+   *
+   * @return A list of StatsFields to comput for this pivot, or the empty list if none
+   */
+  private static List<StatsField> getTaggedStatsFields(StatsInfo statsInfo, 
+                                                       String statsLocalParam) {
+    if (null == statsLocalParam || null == statsInfo) {
+      return Collections.emptyList();
+    }
+    
+    List<StatsField> fields = new ArrayList<>(7);
+    List<String> statsAr = StrUtils.splitSmart(statsLocalParam, ',');
+
+    // TODO: for now, we only support a single tag name - we reserve using 
+    // ',' as a possible delimeter for logic related to only computing stats
+    // at certain levels -- see SOLR-6663
+    if (1 < statsAr.size()) {
+      String msg = StatsParams.STATS + " local param of " + FacetParams.FACET_PIVOT + 
+        "may not include tags separated by a comma - please use a common tag on all " + 
+        StatsParams.STATS_FIELD + " params you wish to compute under this pivot";
+      throw new SolrException(ErrorCode.BAD_REQUEST, msg);
+    }
+
+    for(String stat : statsAr) {
+      fields.addAll(statsInfo.getStatsFieldsByTag(stat));
+    }
+    return fields;
+  }
+
+  /**
    * Recursive function to compute all the pivot counts for the values under teh specified field
    */
   protected List<NamedList<Object>> doPivots(NamedList<Integer> superFacets,
-      String field, String subField, Deque<String> fnames,Deque<String> vnames,DocSet docs) throws IOException {
+                                             String field, String subField, 
+                                             Deque<String> fnames, Deque<String> vnames, 
+                                             DocSet docs, List<StatsField> statsFields) 
+    throws IOException {
+
+    boolean isShard = rb.req.getParams().getBool(ShardParams.IS_SHARD, false);
 
     SolrIndexSearcher searcher = rb.req.getSearcher();
     // TODO: optimize to avoid converting to an external string and then having to convert back to internal below
@@ -169,6 +243,7 @@ public class PivotFacetProcessor extends
       // Only sub-facet if parent facet has positive count - still may not be any values for the sub-field though
       if (kv.getValue() >= getMinCountForField(field)) {  
         final String fieldValue = kv.getKey();
+        final int pivotCount = kv.getValue();
 
         SimpleOrderedMap<Object> pivot = new SimpleOrderedMap<>();
         pivot.add( "field", field );
@@ -178,7 +253,7 @@ public class PivotFacetProcessor extends
           ftype.readableToIndexed(fieldValue, termval);
           pivot.add( "value", ftype.toObject(sfield, termval.get()) );
         }
-        pivot.add( "count", kv.getValue() );
+        pivot.add( "count", pivotCount );
 
         DocSet subset = getSubset(docs, sfield, fieldValue);
         
@@ -195,8 +270,16 @@ public class PivotFacetProcessor extends
           }
 
           if (facetCounts.size() >= 1) {
-            pivot.add( "pivot", doPivots( facetCounts, subField, nextField, fnames, vnames, subset) );
+            pivot.add( "pivot", doPivots( facetCounts, subField, nextField, fnames, vnames, subset, statsFields ) );
+          }
+        }
+        if ((isShard || 0 < pivotCount) && ! statsFields.isEmpty()) {
+          Map<String, StatsValues> stv = new LinkedHashMap<>();
+          for (StatsField statsField : statsFields) {
+            stv.put(statsField.getOutputKey(), statsField.computeLocalStatsValues(subset));
           }
+          // for pivots, we *always* include requested stats - even if 'empty'
+          pivot.add("stats", StatsComponent.convertToResponse(true, stv));
         }
         values.add( pivot );
       }

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/PivotFacetValue.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/PivotFacetValue.java?rev=1636772&r1=1636771&r2=1636772&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/PivotFacetValue.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/PivotFacetValue.java Tue Nov  4 23:09:41 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/trunk/solr/core/src/java/org/apache/solr/handler/component/StatsComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/StatsComponent.java?rev=1636772&r1=1636771&r2=1636772&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/StatsComponent.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/StatsComponent.java Tue Nov  4 23:09:41 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/trunk/solr/core/src/java/org/apache/solr/handler/component/StatsField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/StatsField.java?rev=1636772&r1=1636771&r2=1636772&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/StatsField.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/StatsField.java Tue Nov  4 23:09:41 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/trunk/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java?rev=1636772&r1=1636771&r2=1636772&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java Tue Nov  4 23:09:41 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/trunk/solr/core/src/java/org/apache/solr/util/PivotListEntry.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/PivotListEntry.java?rev=1636772&r1=1636771&r2=1636772&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/PivotListEntry.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/PivotListEntry.java Tue Nov  4 23:09:41 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/trunk/solr/core/src/test/org/apache/solr/cloud/TestCloudPivotFacet.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/TestCloudPivotFacet.java?rev=1636772&r1=1636771&r2=1636772&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/TestCloudPivotFacet.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/TestCloudPivotFacet.java Tue Nov  4 23:09:41 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/trunk/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotLargeTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotLargeTest.java?rev=1636772&r1=1636771&r2=1636772&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotLargeTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotLargeTest.java Tue Nov  4 23:09:41 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/trunk/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotLongTailTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotLongTailTest.java?rev=1636772&r1=1636771&r2=1636772&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotLongTailTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotLongTailTest.java Tue Nov  4 23:09:41 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);
   }
 
 }

Added: lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotSmallAdvancedTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotSmallAdvancedTest.java?rev=1636772&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotSmallAdvancedTest.java (added)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotSmallAdvancedTest.java Tue Nov  4 23:09:41 2014
@@ -0,0 +1,234 @@
+package org.apache.solr.handler.component;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+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.params.FacetParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+/**
+ * tests some edge cases of pivot faceting with stats
+ *
+ * NOTE: This test ignores the control collection (in single node mode, there is no 
+ * need for the overrequesting, all the data is local -- so comparisons with it wouldn't 
+ * be valid in some cases we are testing here)
+ */
+public class DistributedFacetPivotSmallAdvancedTest extends BaseDistributedSearchTestCase {
+
+  public DistributedFacetPivotSmallAdvancedTest() {
+    this.fixShardCount = true;
+    this.shardCount = 2;
+  }
+
+  @Override
+  public void doTest() throws Exception {
+
+    del("*:*");
+    final SolrServer shard0 = clients.get(0);
+    final SolrServer shard1 = clients.get(1);
+
+    // NOTE: we use the literal (4 character) string "null" as a company name
+    // to help ensure there isn't any bugs where the literal string is treated as if it 
+    // were a true NULL value.
+
+    // shard0
+    shard0.add(sdoc(id, 19, "place_t", "cardiff dublin", 
+                    "company_t", "microsoft polecat", 
+                    "price_ti", "15", "foo_s", "aaa", "foo_i", 10));
+    shard0.add(sdoc(id, 20, "place_t", "dublin", 
+                    "company_t", "polecat microsoft null", 
+                    "price_ti", "19", "foo_s", "bbb", "foo_i", 4));
+    shard0.add(sdoc(id, 21, "place_t", "london la dublin", 
+                    "company_t", "microsoft fujitsu null polecat", 
+                    "price_ti", "29", "foo_s", "bbb", "foo_i", 3));
+    shard0.add(sdoc(id, 22, "place_t", "krakow london cardiff", 
+                    "company_t", "polecat null bbc", 
+                    "price_ti", "39", "foo_s", "bbb", "foo_i", 6));
+    shard0.add(sdoc(id, 23, "place_t", "london", 
+                    "company_t", "", 
+                    "price_ti", "29", "foo_s", "bbb", "foo_i", 9));
+    // shard1
+    shard1.add(sdoc(id, 24, "place_t", "la", 
+                    "company_t", "", 
+                    "foo_s", "aaa", "foo_i", 21));
+    shard1.add(sdoc(id, 25, 
+                    "company_t", "microsoft polecat null fujitsu null bbc", 
+                    "price_ti", "59", "foo_s", "aaa", "foo_i", 5));
+    shard1.add(sdoc(id, 26, "place_t", "krakow", 
+                    "company_t", "null", 
+                    "foo_s", "aaa", "foo_i", 23));
+    shard1.add(sdoc(id, 27, "place_t", "krakow cardiff dublin london la", 
+                    "company_t", "null microsoft polecat bbc fujitsu", 
+                    "foo_s", "aaa", "foo_i", 91));
+    shard1.add(sdoc(id, 28, "place_t", "cork", 
+                    "company_t", "fujitsu rte", "foo_s", "aaa", "foo_i", 76));
+    commit();
+
+    handle.clear();
+    handle.put("QTime", SKIPVAL);
+    handle.put("timestamp", SKIPVAL);
+    handle.put("maxScore", SKIPVAL);
+
+    doTestDeepPivotStatsOnString();
+    doTestTopStatsWithRefinement();
+  }
+
+  /**
+   * we need to ensure that stats never "overcount" the values from a single shard
+   * even if we hit that shard with a refinement request 
+   */
+  private void doTestTopStatsWithRefinement() throws Exception {
+    
+    
+    ModifiableSolrParams coreParams = params("q", "*:*", "rows", "0",
+                                             "stats", "true",
+                                             "stats.field", "{!tag=s1}foo_i" );
+    ModifiableSolrParams facetParams = new ModifiableSolrParams(coreParams);
+    facetParams.add(params("facet", "true",
+                           "facet.limit", "1",
+                           "facet.pivot", "{!stats=s1}place_t,company_t"));
+    
+    ModifiableSolrParams facetForceRefineParams = new ModifiableSolrParams(facetParams);
+    facetForceRefineParams.add(params(FacetParams.FACET_OVERREQUEST_COUNT, "0",
+                                      FacetParams.FACET_OVERREQUEST_RATIO, "0"));
+
+    for (ModifiableSolrParams params : new ModifiableSolrParams[] {
+        coreParams, facetParams, facetForceRefineParams }) {
+
+      // for all three sets of these params, the "top level" 
+      // stats in the response of a distributed query should be the same
+      ModifiableSolrParams q = new ModifiableSolrParams(params);
+      q.set("shards", getShardsString());
+
+      QueryResponse rsp = queryServer(q);
+      FieldStatsInfo fieldStatsInfo = rsp.getFieldStatsInfo().get("foo_i");
+
+      String msg = q.toString();
+
+      assertEquals(msg, 3.0, fieldStatsInfo.getMin());
+      assertEquals(msg, 91.0, fieldStatsInfo.getMax());
+      assertEquals(msg, 10, (long) fieldStatsInfo.getCount());
+      assertEquals(msg, 0, (long) fieldStatsInfo.getMissing());
+      assertEquals(msg, 248.0, fieldStatsInfo.getSum());
+      assertEquals(msg, 15294.0, fieldStatsInfo.getSumOfSquares(), 0.1E-7);
+      assertEquals(msg, 24.8, (double) fieldStatsInfo.getMean(), 0.1E-7);
+      assertEquals(msg, 31.87405772027709, fieldStatsInfo.getStddev(), 0.1E-7);
+
+      if (params.getBool("facet", false)) {
+        // if this was a facet request, then the top pivot constraint and pivot 
+        // stats should match what we expect - regardless of wether refine
+        // was used, or if the query was initially satisfied by the default overrequest
+        
+        List<PivotField> placePivots = rsp.getFacetPivot().get("place_t,company_t");
+        assertEquals(1, placePivots.size());
+        
+        PivotField dublinPivotField = placePivots.get(0);
+        assertEquals("dublin", dublinPivotField.getValue());
+        assertEquals(4, dublinPivotField.getCount());
+        assertEquals(1, dublinPivotField.getPivot().size());
+
+        PivotField microsoftPivotField = dublinPivotField.getPivot().get(0);
+        assertEquals("microsoft", microsoftPivotField.getValue());
+        assertEquals(4, microsoftPivotField.getCount());
+        
+        FieldStatsInfo dublinMicrosoftStatsInfo = microsoftPivotField.getFieldStatsInfo().get("foo_i");
+        assertEquals(3.0D, dublinMicrosoftStatsInfo.getMin());
+        assertEquals(91.0D, dublinMicrosoftStatsInfo.getMax());
+        assertEquals(4, (long) dublinMicrosoftStatsInfo.getCount());
+        assertEquals(0, (long) dublinMicrosoftStatsInfo.getMissing());
+        
+      }
+    }
+
+    // sanity check that the top pivot from each shard is diff, to prove to 
+    // ourselves that the above queries really must have involved refinement.
+    Object s0pivValue = clients.get(0)
+      .query(facetParams).getFacetPivot().get("place_t,company_t").get(0).getValue();
+    Object s1pivValue = clients.get(1)
+      .query(facetParams).getFacetPivot().get("place_t,company_t").get(0).getValue();
+    assertFalse("both shards have same top constraint, test is invalid" +
+                "(did someone change the test data?) ==> " + 
+                s0pivValue + "==" + s1pivValue, s0pivValue.equals(s1pivValue));
+    
+  }
+
+  private void doTestDeepPivotStatsOnString() throws Exception {
+    SolrParams params = params("q", "*:*", "rows", "0",
+        "shards", getShardsString(),
+        "facet", "true", "stats", "true",
+        "facet.pivot", "{!stats=s1}place_t,company_t",
+        "stats.field", "{!key=avg_price tag=s1}foo_s");
+    QueryResponse rsp = queryServer(new ModifiableSolrParams(params));
+
+    List<PivotField> placePivots = rsp.getFacetPivot().get("place_t,company_t");
+
+    PivotField dublinPivotField = placePivots.get(0);
+    assertEquals("dublin", dublinPivotField.getValue());
+    assertEquals(4, dublinPivotField.getCount());
+
+    PivotField microsoftPivotField = dublinPivotField.getPivot().get(0);
+    assertEquals("microsoft", microsoftPivotField.getValue());
+    assertEquals(4, microsoftPivotField.getCount());
+
+    FieldStatsInfo dublinMicrosoftStatsInfo = microsoftPivotField.getFieldStatsInfo().get("avg_price");
+    assertEquals("aaa", dublinMicrosoftStatsInfo.getMin());
+    assertEquals("bbb", dublinMicrosoftStatsInfo.getMax());
+    assertEquals(4, (long) dublinMicrosoftStatsInfo.getCount());
+    assertEquals(0, (long) dublinMicrosoftStatsInfo.getMissing());
+
+    PivotField cardiffPivotField = placePivots.get(2);
+    assertEquals("cardiff", cardiffPivotField.getValue());
+    assertEquals(3, cardiffPivotField.getCount());
+
+    PivotField polecatPivotField = cardiffPivotField.getPivot().get(0);
+    assertEquals("polecat", polecatPivotField.getValue());
+    assertEquals(3, polecatPivotField.getCount());
+
+    FieldStatsInfo cardiffPolecatStatsInfo = polecatPivotField.getFieldStatsInfo().get("avg_price");
+    assertEquals("aaa", cardiffPolecatStatsInfo.getMin());
+    assertEquals("bbb", cardiffPolecatStatsInfo.getMax());
+    assertEquals(3, (long) cardiffPolecatStatsInfo.getCount());
+    assertEquals(0, (long) cardiffPolecatStatsInfo.getMissing());
+
+    PivotField krakowPivotField = placePivots.get(3);
+    assertEquals("krakow", krakowPivotField.getValue());
+    assertEquals(3, krakowPivotField.getCount());
+
+    PivotField fujitsuPivotField = krakowPivotField.getPivot().get(3);
+    assertEquals("fujitsu", fujitsuPivotField.getValue());
+    assertEquals(1, fujitsuPivotField.getCount());
+
+    FieldStatsInfo krakowFujitsuStatsInfo = fujitsuPivotField.getFieldStatsInfo().get("avg_price");
+    assertEquals("aaa", krakowFujitsuStatsInfo.getMin());
+    assertEquals("aaa", krakowFujitsuStatsInfo.getMax());
+    assertEquals(1, (long) krakowFujitsuStatsInfo.getCount());
+    assertEquals(0, (long) krakowFujitsuStatsInfo.getMissing());
+  }
+
+}

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotSmallTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotSmallTest.java?rev=1636772&r1=1636771&r2=1636772&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotSmallTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotSmallTest.java Tue Nov  4 23:09:41 2014
@@ -20,9 +20,11 @@ package org.apache.solr.handler.componen
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
+import java.util.Date;
 import java.util.List;
 
 import org.apache.solr.BaseDistributedSearchTestCase;
+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.params.FacetParams;
@@ -46,20 +48,22 @@ public class DistributedFacetPivotSmallT
     // NOTE: we use the literal (4 character) string "null" as a company name
     // to help ensure there isn't any bugs where the literal string is treated as if it 
     // were a true NULL value.
-    index(id, 19, "place_t", "cardiff dublin", "company_t", "microsoft polecat");
-    index(id, 20, "place_t", "dublin", "company_t", "polecat microsoft null");
+    index(id, 19, "place_t", "cardiff dublin", "company_t", "microsoft polecat", "price_ti", "15");
+    index(id, 20, "place_t", "dublin", "company_t", "polecat microsoft null", "price_ti", "19",
+          // this is the only doc to have solo_* fields, therefore only 1 shard has them
+          // TODO: add enum field - blocked by SOLR-6682
+          "solo_i", 42, "solo_s", "lonely", "solo_dt", "1976-03-06T01:23:45Z");
     index(id, 21, "place_t", "london la dublin", "company_t",
-        "microsoft fujitsu null polecat");
+        "microsoft fujitsu null polecat", "price_ti", "29");
     index(id, 22, "place_t", "krakow london cardiff", "company_t",
-        "polecat null bbc");
-    index(id, 23, "place_t", "london", "company_t", "");
+        "polecat null bbc", "price_ti", "39");
+    index(id, 23, "place_t", "london", "company_t", "", "price_ti", "29");
     index(id, 24, "place_t", "la", "company_t", "");
-    index(id, 25, "company_t", "microsoft polecat null fujitsu null bbc");
+    index(id, 25, "company_t", "microsoft polecat null fujitsu null bbc", "price_ti", "59");
     index(id, 26, "place_t", "krakow", "company_t", "null");
-    index(id, 27, "place_t", "krakow cardiff dublin london la", "company_t",
-        "null microsoft polecat bbc fujitsu");
-    index(id, 28, "place_t", "cork", "company_t",
-        "fujitsu rte");
+    index(id, 27, "place_t", "krakow cardiff dublin london la", 
+          "company_t", "null microsoft polecat bbc fujitsu");
+    index(id, 28, "place_t", "cork", "company_t", "fujitsu rte");
     commit();
     
     handle.clear();
@@ -332,6 +336,76 @@ public class DistributedFacetPivotSmallT
         throw new AssertionError(ae.getMessage() + " <== " + p.toString(), ae);
       }
     }
+
+    doTestDeepPivotStats();
+
+    doTestPivotStatsFromOneShard();
+  }
+
+  private void doTestDeepPivotStats() throws Exception {
+    SolrParams params = params("q", "*:*", "rows", "0", 
+                               "facet", "true", "stats", "true", 
+                               "facet.pivot", "{!stats=s1}place_t,company_t", 
+                               "stats.field", "{!key=avg_price tag=s1}price_ti");
+    QueryResponse rsp = query(params);
+
+    List<PivotField> placePivots = rsp.getFacetPivot().get("place_t,company_t");
+
+    PivotField dublinPivotField = placePivots.get(0);
+    assertEquals("dublin", dublinPivotField.getValue());
+    assertEquals(4, dublinPivotField.getCount());
+
+    PivotField microsoftPivotField = dublinPivotField.getPivot().get(0);
+    assertEquals("microsoft", microsoftPivotField.getValue());
+    assertEquals(4, microsoftPivotField.getCount());
+
+    FieldStatsInfo dublinMicrosoftStatsInfo = microsoftPivotField.getFieldStatsInfo().get("avg_price");
+    assertEquals(15.0, dublinMicrosoftStatsInfo.getMin());
+    assertEquals(29.0, dublinMicrosoftStatsInfo.getMax());
+    assertEquals(3, (long) dublinMicrosoftStatsInfo.getCount());
+    assertEquals(1, (long) dublinMicrosoftStatsInfo.getMissing());
+    assertEquals(63.0, dublinMicrosoftStatsInfo.getSum());
+    assertEquals(1427.0, dublinMicrosoftStatsInfo.getSumOfSquares(), 0.1E-7);
+    assertEquals(21.0, (double) dublinMicrosoftStatsInfo.getMean(), 0.1E-7);
+    assertEquals(7.211102550927978, dublinMicrosoftStatsInfo.getStddev(), 0.1E-7);
+
+
+    PivotField cardiffPivotField = placePivots.get(2);
+    assertEquals("cardiff", cardiffPivotField.getValue());
+    assertEquals(3, cardiffPivotField.getCount());
+
+    PivotField polecatPivotField = cardiffPivotField.getPivot().get(0);
+    assertEquals("polecat", polecatPivotField.getValue());
+    assertEquals(3, polecatPivotField.getCount());
+
+    FieldStatsInfo cardiffPolecatStatsInfo = polecatPivotField.getFieldStatsInfo().get("avg_price");
+    assertEquals(15.0, cardiffPolecatStatsInfo.getMin());
+    assertEquals(39.0, cardiffPolecatStatsInfo.getMax());
+    assertEquals(2, (long) cardiffPolecatStatsInfo.getCount());
+    assertEquals(1, (long) cardiffPolecatStatsInfo.getMissing());
+    assertEquals(54.0, cardiffPolecatStatsInfo.getSum());
+    assertEquals(1746.0, cardiffPolecatStatsInfo.getSumOfSquares(), 0.1E-7);
+    assertEquals(27.0, (double) cardiffPolecatStatsInfo.getMean(), 0.1E-7);
+    assertEquals(16.97056274847714, cardiffPolecatStatsInfo.getStddev(), 0.1E-7);
+
+
+    PivotField krakowPivotField = placePivots.get(3);
+    assertEquals("krakow", krakowPivotField.getValue());
+    assertEquals(3, krakowPivotField.getCount());
+
+    PivotField fujitsuPivotField = krakowPivotField.getPivot().get(3);
+    assertEquals("fujitsu", fujitsuPivotField.getValue());
+    assertEquals(1, fujitsuPivotField.getCount());
+
+    FieldStatsInfo krakowFujitsuStatsInfo = fujitsuPivotField.getFieldStatsInfo().get("avg_price");
+    assertEquals(null, krakowFujitsuStatsInfo.getMin());
+    assertEquals(null, krakowFujitsuStatsInfo.getMax());
+    assertEquals(0, (long) krakowFujitsuStatsInfo.getCount());
+    assertEquals(1, (long) krakowFujitsuStatsInfo.getMissing());
+    assertEquals(0.0, krakowFujitsuStatsInfo.getSum());
+    assertEquals(0.0, krakowFujitsuStatsInfo.getSumOfSquares(), 0.1E-7);
+    assertEquals(Double.NaN, (double) krakowFujitsuStatsInfo.getMean(), 0.1E-7);
+    assertEquals(0.0, krakowFujitsuStatsInfo.getStddev(), 0.1E-7);
   }
 
   // Useful to check for errors, orders lists and does toString() equality check
@@ -351,6 +425,46 @@ public class DistributedFacetPivotSmallT
     }
     assertEquals(expectedPlacePivots.toString(), placePivots.toString());
   }
+
+  /**
+   * sanity check the stat values nested under a pivot when at least one shard
+   * has nothing but missing values for the stat
+   */
+  private void doTestPivotStatsFromOneShard() throws Exception {
+    SolrParams params = params("q", "*:*", "rows", "0", 
+                               "facet", "true", "stats", "true", 
+                               "facet.pivot", "{!stats=s1}place_t,company_t", 
+                               "stats.field", "{!tag=s1}solo_i",
+                               "stats.field", "{!tag=s1}solo_s",
+                               "stats.field", "{!tag=s1}solo_dt");
+                               
+    QueryResponse rsp = query(params);
+
+    List<PivotField> placePivots = rsp.getFacetPivot().get("place_t,company_t");
+
+    PivotField placePivot = placePivots.get(0);
+    assertEquals("dublin", placePivot.getValue());
+    assertEquals(4, placePivot.getCount());
+
+    PivotField companyPivot = placePivot.getPivot().get(2);
+    assertEquals("null", companyPivot.getValue());
+    assertEquals(3, companyPivot.getCount());
+
+    for (PivotField pf : new PivotField[] { placePivot, companyPivot }) {
+      assertThereCanBeOnlyOne(pf, pf.getFieldStatsInfo().get("solo_s"), "lonely");
+
+      assertThereCanBeOnlyOne(pf, pf.getFieldStatsInfo().get("solo_i"), 42.0D);
+      assertEquals(pf.getField()+":"+pf.getValue()+": int mean",
+                   42.0D, pf.getFieldStatsInfo().get("solo_i").getMean());
+
+      Object expected = new Date(194923425000L); // 1976-03-06T01:23:45Z
+      assertThereCanBeOnlyOne(pf, pf.getFieldStatsInfo().get("solo_dt"), expected);
+      assertEquals(pf.getField()+":"+pf.getValue()+": date mean",
+                   expected, pf.getFieldStatsInfo().get("solo_dt").getMean());
+
+      // TODO: add enum field asserts - blocked by SOLR-6682
+    }
+  }
   
   private void testCountSorting(List<PivotField> pivots) {
     Integer lastCount = null;
@@ -365,12 +479,27 @@ public class DistributedFacetPivotSmallT
     }
   }
   
+  /**
+   * given a PivotField, a FieldStatsInfo, and a value; asserts that:
+   * <ul>
+   *  <li>stat count == 1</li>
+   *  <li>stat missing == pivot count - 1</li>
+   *  <li>stat min == stat max == value</li>
+   * </ul>
+   */
+  private void assertThereCanBeOnlyOne(PivotField pf, FieldStatsInfo stats, Object val) {
+    String msg = pf.getField() + ":" + pf.getValue();
+    assertEquals(msg + " stats count", 1L, (long) stats.getCount());
+    assertEquals(msg + " stats missing", pf.getCount()-1L, (long) stats.getMissing());
+    assertEquals(msg + " stats min", val, stats.getMin());
+    assertEquals(msg + " stats max", val, stats.getMax());
+  }
+
   public static class ComparablePivotField extends PivotField {
     
 
-    public ComparablePivotField(String f, Object v, int count,
-        List<PivotField> pivot) {
-      super(f,v,count,pivot);
+    public ComparablePivotField(String f, Object v, int count, List<PivotField> pivot) {
+      super(f,v,count,pivot, null);
     }
 
     @Override



Mime
View raw message