You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ho...@apache.org on 2014/08/13 20:23:54 UTC

svn commit: r1617789 [2/3] - in /lucene/dev/trunk/solr: ./ core/src/java/org/apache/solr/handler/component/ core/src/java/org/apache/solr/request/ core/src/java/org/apache/solr/util/ core/src/test/org/apache/solr/ core/src/test/org/apache/solr/cloud/ c...

Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/PivotFacetFieldValueCollection.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/PivotFacetFieldValueCollection.java?rev=1617789&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/PivotFacetFieldValueCollection.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/PivotFacetFieldValueCollection.java Wed Aug 13 18:23:53 2014
@@ -0,0 +1,342 @@
+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 java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+
+import org.apache.solr.common.params.FacetParams;
+
+/**
+ * Emcapsulates a collection of {@link PivotFacetValue}s associated with a 
+ * {@link PivotFacetField} withs pecial tracking of a {@link PivotFacetValue} 
+ * corrisponding to the <code>null</code> value when {@link FacetParams#FACET_MISSING} 
+ * is used.
+ *
+ * @see #markDirty
+ * @see PivotFacetField
+ * @see PivotFacetValue
+ */
+@SuppressWarnings("rawtypes")
+public class PivotFacetFieldValueCollection implements Iterable<PivotFacetValue> {
+  private List<PivotFacetValue> explicitValues;  
+  private PivotFacetValue missingValue;
+  private Map<Comparable, PivotFacetValue> valuesMap;
+  private boolean dirty = true;
+
+  //Facet parameters relating to this field
+  private final int facetFieldMinimumCount;
+  private final int facetFieldOffset;  
+  private final int facetFieldLimit;
+  private final String facetFieldSort;
+  
+  
+  public PivotFacetFieldValueCollection(int minCount, int offset, int limit, String fieldSort){
+    this.explicitValues = new ArrayList<>();
+    this.valuesMap = new HashMap<>(); 
+    this.facetFieldMinimumCount = minCount;
+    this.facetFieldOffset = offset;
+    this.facetFieldLimit = limit;
+    this.facetFieldSort = fieldSort;
+  }
+
+  /**
+   * Indicates that the values in this collection have been modified by the caller.
+   *
+   * Any caller that manipulates the {@link PivotFacetValue}s contained in this collection
+   * must call this method after doing so.
+   */
+  public void markDirty() {
+    dirty = true;
+  }
+
+  /**
+   * The {@link PivotFacetValue} with corisponding to a a value of 
+   * <code>null</code> when {@link FacetParams#FACET_MISSING} is used.
+   * 
+   * @return the appropriate <code>PivotFacetValue</code> object, may be null 
+   *         if we "missing" is not in use, or if it does not meat the mincount.
+   */
+  public PivotFacetValue getMissingValue(){
+    return missingValue;
+  }
+
+  /** 
+   * Read-Only access to the Collection of {@link PivotFacetValue}s corrisponding to 
+   * non-missing values.
+   *
+   * @see #getMissingValue
+   */
+  public List<PivotFacetValue> getExplicitValuesList() {
+    return Collections.unmodifiableList(explicitValues);
+  }
+
+  /** 
+   * Size of {@link #getExplicitValuesList}
+   */
+  public int getExplicitValuesListSize() {
+    return this.explicitValues.size();
+  }
+  
+  /** 
+   * Total number of {@link PivotFacetValue}s, including the "missing" value if used.
+   *
+   * @see #getMissingValue
+   * @see #getExplicitValuesList
+   */
+  public int size() {
+    return this.getExplicitValuesListSize() + (this.missingValue == null ? 0 : 1);
+  }
+  
+  /**
+   * Returns the appropriate sub-list of the explicit values that need to be refined, 
+   * based on the {@link FacetParams#FACET_OFFSET} &amp; {@link FacetParams#FACET_LIMIT} 
+   * for this field.
+   *
+   * @see #getExplicitValuesList
+   * @see List#subList
+   */
+  public List<PivotFacetValue> getNextLevelValuesToRefine() {
+    final int numRefinableValues = getExplicitValuesListSize();
+    if (facetFieldOffset < numRefinableValues) {
+      final int offsetPlusCount = (facetFieldLimit >= 0) 
+        ? Math.min(facetFieldLimit + facetFieldOffset, numRefinableValues) 
+        : numRefinableValues;
+      return getExplicitValuesList().subList(facetFieldOffset,  offsetPlusCount);
+    } else {
+      return Collections.<PivotFacetValue>emptyList();
+    }
+  }
+  
+  /**
+   * Fast lookup to retrieve a {@link PivotFacetValue} from this collection if it 
+   * exists
+   *
+   * @param value of the <code>PivotFacetValue</code> to lookup, if 
+   *        <code>null</code> this returns the same as {@link #getMissingValue}
+   * @return the corrisponding <code>PivotFacetValue</code> or null if there is 
+   *        no <code>PivotFacetValue</code> in this collection corrisponding to 
+   *        the specified value.
+   */
+  public PivotFacetValue get(Comparable value){
+    return valuesMap.get(value);
+  }
+  
+  /**
+   * Fetchs a {@link PivotFacetValue} from this collection via the index, may not 
+   * be used to fetch the <code>PivotFacetValue</code> corrisponding to the missing-value.
+   *
+   * @see #getExplicitValuesList
+   * @see List#get(int)
+   * @see #getMissingValue
+   */
+  public PivotFacetValue getAt(int index){
+    return explicitValues.get(index);
+  }
+  
+  /**
+   * Adds a {@link PivotFacetValue} to this collection -- callers must not use this 
+   * method if a {@link PivotFacetValue} with the same value already exists in this collection
+   */
+  public void add(PivotFacetValue pfValue) {
+    Comparable val = pfValue.getValue();
+    assert ! this.valuesMap.containsKey(val) 
+      : "Must not add duplicate PivotFacetValue with redundent inner value";
+
+    dirty = true;
+    if(null == val) {
+      this.missingValue = pfValue;
+    } else {
+      this.explicitValues.add(pfValue);
+    }
+    this.valuesMap.put(val, pfValue);
+  }
+
+
+  /**
+   * Destructive method that recursively prunes values from the data structure 
+   * based on the counts for those values and the effective sort, mincount, limit, 
+   * and offset being used for each field.
+   * <p>
+   * This method should only be called after all refinement is completed.
+   * </p>
+   *
+   * @see PivotFacetField#trim
+   * @see PivotFacet#getTrimmedPivotsAsListOfNamedLists
+   */
+  public void trim() {   // NOTE: destructive
+    // TODO: see comment in PivotFacetField about potential optimization
+    // (ie: trim as we refine)
+    trimNonNullValues(); 
+    trimNullValue();
+  }
+  
+  private void trimNullValue(){
+    if (missingValue == null) {
+      return;
+    }
+
+    if (missingValue.getCount() >= facetFieldMinimumCount){
+      if (null != missingValue.getChildPivot()) {
+        missingValue.getChildPivot().trim();
+      }
+    } else { // missing count less than mincount
+      missingValue = null;
+    }
+  }
+  
+  private void trimNonNullValues(){
+    if (explicitValues != null && explicitValues.size() > 0) {
+      
+      sort();
+      
+      ArrayList<PivotFacetValue> trimmedValues = new ArrayList<>();
+      
+      int facetsSkipped = 0;
+      
+      for (PivotFacetValue pivotValue : explicitValues) {
+        
+        if (pivotValue.getCount() >= facetFieldMinimumCount) {
+          if (facetsSkipped >= facetFieldOffset) {
+            trimmedValues.add(pivotValue);
+            if (pivotValue.getChildPivot() != null) {
+              pivotValue.getChildPivot().trim();
+            }
+            if (facetFieldLimit > 0 && trimmedValues.size() >= facetFieldLimit) {
+              break;
+            }
+          } else {
+            facetsSkipped++;
+          }
+        }
+      }
+      
+      explicitValues = trimmedValues;
+      valuesMap.clear();
+    }
+  }
+  
+  /**
+   * Sorts the collection and recursively sorts the collections assocaited with 
+   * any sub-pivots.
+   *
+   * @see FacetParams#FACET_SORT
+   * @see PivotFacetField#sort
+   */
+  public void sort() {
+    
+    if (dirty) {
+      if (facetFieldSort.equals(FacetParams.FACET_SORT_COUNT)) {
+        Collections.sort(this.explicitValues, new PivotFacetCountComparator());
+      } else if (facetFieldSort.equals(FacetParams.FACET_SORT_INDEX)) {
+        Collections.sort(this.explicitValues, new PivotFacetValueComparator());
+      }
+      dirty = false;
+    }
+    
+    for (PivotFacetValue value : this.explicitValues)
+      if (value.getChildPivot() != null) {
+        value.getChildPivot().sort();
+      }
+   
+    if (missingValue != null && missingValue.getChildPivot() != null) {
+      missingValue.getChildPivot().sort();
+    }
+  }
+
+  /**
+   * Iterator over all elements in this Collection, including the result of 
+   * {@link #getMissingValue} as the last element (if it exists)
+   */
+  @Override
+  public Iterator<PivotFacetValue> iterator() {
+    Iterator<PivotFacetValue> it = new Iterator<PivotFacetValue>() {
+      private final Iterator valuesIterator = explicitValues.iterator();
+      private boolean shouldGiveMissingValue = (missingValue != null);
+      
+      @Override
+      public boolean hasNext() {
+        return valuesIterator.hasNext() || shouldGiveMissingValue;
+      }
+      
+      @Override
+      public PivotFacetValue next() {
+        while(valuesIterator.hasNext()){
+          return (PivotFacetValue) valuesIterator.next();
+        }
+        //else
+        if(shouldGiveMissingValue){
+          shouldGiveMissingValue = false;
+          return missingValue;
+        }
+        return null;
+      }
+      
+      @Override
+      public void remove() {
+        throw new UnsupportedOperationException("Can't remove from this iterator");
+      }
+    };
+    return it;
+  }
+    
+  /** Sorts {@link PivotFacetValue} instances by their count */
+  public class PivotFacetCountComparator implements Comparator<PivotFacetValue> {    
+    public int compare(PivotFacetValue left, PivotFacetValue right) {
+      int countCmp = right.getCount() - left.getCount();
+      return (0 != countCmp) ? countCmp : 
+        compareWithNullLast(left.getValue(), right.getValue());
+    }    
+  }
+  
+  /** Sorts {@link PivotFacetValue} instances by their value */
+  public class PivotFacetValueComparator implements Comparator<PivotFacetValue> {
+    public int compare(PivotFacetValue left, PivotFacetValue right) {
+      return compareWithNullLast(left.getValue(), right.getValue());
+    }
+  }
+  
+  /**
+   * A helper method for use in <code>Comparator</code> classes where object properties 
+   * are <code>Comparable</code> but may be null.
+   */
+  static int compareWithNullLast(final Comparable o1, final Comparable o2) {
+    if (null == o1) {
+      if (null == o2) {
+        return 0;
+      }
+      return 1; // o1 is null, o2 is not
+    }
+    if (null == o2) {
+      return -1; // o2 is null, o1 is not
+    }
+    return o1.compareTo(o2);
+  }
+  
+  public String toString(){
+    return String.format(Locale.ROOT, "Values:%s | Missing:%s ", explicitValues, missingValue);
+  }
+}
+
+

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=1617789&r1=1617788&r2=1617789&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 Wed Aug 13 18:23:53 2014
@@ -17,254 +17,105 @@
 
 package org.apache.solr.handler.component;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Deque;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.lucene.index.Term;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.search.TermRangeQuery;
-import org.apache.lucene.util.BytesRefBuilder;
-import org.apache.solr.common.SolrException;
+import org.apache.solr.util.PivotListEntry;
 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.request.SimpleFacets;
-import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.schema.FieldType;
-import org.apache.solr.schema.SchemaField;
-import org.apache.solr.search.DocSet;
-import org.apache.solr.search.SolrIndexSearcher;
-import org.apache.solr.search.SyntaxError;
-
-/**
- * @since solr 4.0
- */
-public class PivotFacetHelper extends SimpleFacets
-{
-
-  protected int minMatch;
-
-  public PivotFacetHelper(SolrQueryRequest req, DocSet docs, SolrParams params, ResponseBuilder rb) {
-    super(req, docs, params, rb);
-    minMatch = params.getInt( FacetParams.FACET_PIVOT_MINCOUNT, 1 );
-  }
-
-  public SimpleOrderedMap<List<NamedList<Object>>> process(String[] pivots) throws IOException {
-    if (!rb.doFacets || pivots == null) 
-      return null;
-
-    SimpleOrderedMap<List<NamedList<Object>>> pivotResponse = new SimpleOrderedMap<>();
-    for (String pivot : pivots) {
-      //ex: pivot == "features,cat" or even "{!ex=mytag}features,cat"
-      try {
-        this.parseParams(FacetParams.FACET_PIVOT, pivot);
-      } catch (SyntaxError e) {
-        throw new SolrException(ErrorCode.BAD_REQUEST, e);
-      }
-      pivot = facetValue;//facetValue potentially modified from parseParams()
+import org.apache.solr.common.util.StrUtils;
 
-      String[] fields = pivot.split(",");
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Collections;
 
-      if( fields.length < 2 ) {
-        throw new SolrException( ErrorCode.BAD_REQUEST,
-            "Pivot Facet needs at least two fields: "+pivot );
-      }
+public class PivotFacetHelper {
 
-      String field = fields[0];
-      String subField = fields[1];
-      Deque<String> fnames = new LinkedList<>();
-      for( int i=fields.length-1; i>1; i-- ) {
-        fnames.push( fields[i] );
-      }
+  /**
+   * Encodes a value path as a string for the purposes of a refinement request
+   *
+   * @see PivotFacetValue#getValuePath
+   * @see #decodeRefinementValuePath
+   */
+  public static String encodeRefinementValuePath(List<String> values) {
+    // HACK: prefix flag every value to account for empty string vs null
+    // NOTE: even if we didn't have to worry about null's smartSplit is stupid about
+    // pruning empty strings from list
+    // "^" prefix = null
+    // "~" prefix = not null, may be empty string
 
-      NamedList<Integer> superFacets = this.getTermCounts(field);
+    assert null != values;
 
-      //super.key usually == pivot unless local-param 'key' used
-      pivotResponse.add(key, doPivots(superFacets, field, subField, fnames, docs));
+    // special case: empty list => empty string
+    if (values.isEmpty()) { return ""; }
+
+    
+    StringBuilder out = new StringBuilder();
+    for (String val : values) {
+      if (null == val) {
+        out.append('^');
+      } else {
+        out.append('~');
+        StrUtils.appendEscapedTextToBuilder(out, val, ',');
+      }
+      out.append(',');
     }
-    return pivotResponse;
+    out.deleteCharAt(out.length()-1);  // prune the last seperator
+    return out.toString();
+    // return StrUtils.join(values, ',');
   }
 
   /**
-   * Recursive function to do all the pivots
+   * Decodes a value path string specified for refinement.
+   *
+   * @see #encodeRefinementValuePath
    */
-  protected List<NamedList<Object>> doPivots(NamedList<Integer> superFacets,
-                                             String field, String subField, Deque<String> fnames,
-                                             DocSet docs) throws IOException
-  {
-    SolrIndexSearcher searcher = rb.req.getSearcher();
-    // TODO: optimize to avoid converting to an external string and then having to convert back to internal below
-    SchemaField sfield = searcher.getSchema().getField(field);
-    FieldType ftype = sfield.getType();
-
-    String nextField = fnames.poll();
-
-    List<NamedList<Object>> values = new ArrayList<>( superFacets.size() );
-    for (Map.Entry<String, Integer> kv : superFacets) {
-      // Only sub-facet if parent facet has positive count - still may not be any values for the sub-field though
-      if (kv.getValue() >= minMatch) {
-
-        // may be null when using facet.missing
-        final String fieldValue = kv.getKey(); 
-
-        // don't reuse the same BytesRef each time since we will be 
-        // constructing Term objects used in TermQueries that may be cached.
-        BytesRefBuilder termval = null;
-
-        SimpleOrderedMap<Object> pivot = new SimpleOrderedMap<>();
-        pivot.add( "field", field );
-        if (null == fieldValue) {
-          pivot.add( "value", null );
-        } else {
-          termval = new BytesRefBuilder();
-          ftype.readableToIndexed(fieldValue, termval);
-          pivot.add( "value", ftype.toObject(sfield, termval.get()) );
-        }
-        pivot.add( "count", kv.getValue() );
-        
-        if( subField == null ) {
-          values.add( pivot );
-        }
-        else {
-          DocSet subset = null;
-          if ( null == termval ) {
-            DocSet hasVal = searcher.getDocSet
-              (new TermRangeQuery(field, null, null, false, false));
-            subset = docs.andNot(hasVal);
-          } else {
-            Query query = new TermQuery(new Term(field, termval.get()));
-            subset = searcher.getDocSet(query, docs);
-          }
-          super.docs = subset;//used by getTermCounts()
-
-          NamedList<Integer> nl = this.getTermCounts(subField);
-          if (nl.size() >= minMatch) {
-            pivot.add( "pivot", doPivots( nl, subField, nextField, fnames, subset) );
-            values.add( pivot ); // only add response if there are some counts
-          }
-        }
+  public static List<String> decodeRefinementValuePath(String valuePath) {
+    List <String> rawvals = StrUtils.splitSmart(valuePath, ",", true);
+    // special case: empty list => empty string
+    if (rawvals.isEmpty()) return rawvals;
+
+    List<String> out = new ArrayList<String>(rawvals.size());
+    for (String raw : rawvals) {
+      assert 0 < raw.length();
+      if ('^' == raw.charAt(0)) {
+        assert 1 == raw.length();
+        out.add(null);
+      } else {
+        assert '~' == raw.charAt(0);
+        out.add(raw.substring(1));
       }
     }
-    
-    // put the field back on the list
-    fnames.push( nextField );
-    return values;
-  }
-
-// TODO: This is code from various patches to support distributed search.
-//  Some parts may be helpful for whoever implements distributed search.
-//
-//  @Override
-//  public int distributedProcess(ResponseBuilder rb) throws IOException {
-//    if (!rb.doFacets) {
-//      return ResponseBuilder.STAGE_DONE;
-//    }
-//
-//    if (rb.stage == ResponseBuilder.STAGE_GET_FIELDS) {
-//      SolrParams params = rb.req.getParams();
-//      String[] pivots = params.getParams(FacetParams.FACET_PIVOT);
-//      for ( ShardRequest sreq : rb.outgoing ) {
-//        if (( sreq.purpose & ShardRequest.PURPOSE_GET_FIELDS ) != 0
-//            && sreq.shards != null && sreq.shards.length == 1 ) {
-//          sreq.params.set( FacetParams.FACET, "true" );
-//          sreq.params.set( FacetParams.FACET_PIVOT, pivots );
-//          sreq.params.set( FacetParams.FACET_PIVOT_MINCOUNT, 1 ); // keep this at 1 regardless so that it accumulates everything
-//            }
-//      }
-//    }
-//    return ResponseBuilder.STAGE_DONE;
-//  }
-//
-//  @Override
-//  public void handleResponses(ResponseBuilder rb, ShardRequest sreq) {
-//    if (!rb.doFacets) return;
-//
-//
-//    if ((sreq.purpose & ShardRequest.PURPOSE_GET_FACETS)!=0) {
-//      SimpleOrderedMap<List<NamedList<Object>>> tf = rb._pivots;
-//      if ( null == tf ) {
-//        tf = new SimpleOrderedMap<List<NamedList<Object>>>();
-//        rb._pivots = tf;
-//      }
-//      for (ShardResponse srsp: sreq.responses) {
-//        int shardNum = rb.getShardNum(srsp.getShard());
-//
-//        NamedList facet_counts = (NamedList)srsp.getSolrResponse().getResponse().get("facet_counts");
-//
-//        // handle facet trees from shards
-//        SimpleOrderedMap<List<NamedList<Object>>> shard_pivots = 
-//          (SimpleOrderedMap<List<NamedList<Object>>>)facet_counts.get( PIVOT_KEY );
-//        
-//        if ( shard_pivots != null ) {
-//          for (int j=0; j< shard_pivots.size(); j++) {
-//            // TODO -- accumulate the results from each shard
-//            // The following code worked to accumulate facets for an previous 
-//            // two level patch... it is here for reference till someone can upgrade
-//            /**
-//            String shard_tree_name = (String) shard_pivots.getName( j );
-//            SimpleOrderedMap<NamedList> shard_tree = (SimpleOrderedMap<NamedList>)shard_pivots.getVal( j );
-//            SimpleOrderedMap<NamedList> facet_tree = tf.get( shard_tree_name );
-//            if ( null == facet_tree) { 
-//              facet_tree = new SimpleOrderedMap<NamedList>(); 
-//              tf.add( shard_tree_name, facet_tree );
-//            }
-//
-//            for( int o = 0; o < shard_tree.size() ; o++ ) {
-//              String shard_outer = (String) shard_tree.getName( o );
-//              NamedList shard_innerList = (NamedList) shard_tree.getVal( o );
-//              NamedList tree_innerList  = (NamedList) facet_tree.get( shard_outer );
-//              if ( null == tree_innerList ) { 
-//                tree_innerList = new NamedList();
-//                facet_tree.add( shard_outer, tree_innerList );
-//              }
-//
-//              for ( int i = 0 ; i < shard_innerList.size() ; i++ ) {
-//                String shard_term = (String) shard_innerList.getName( i );
-//                long shard_count  = ((Number) shard_innerList.getVal(i)).longValue();
-//                int tree_idx      = tree_innerList.indexOf( shard_term, 0 );
-//
-//                if ( -1 == tree_idx ) {
-//                  tree_innerList.add( shard_term, shard_count );
-//                } else {
-//                  long tree_count = ((Number) tree_innerList.getVal( tree_idx )).longValue();
-//                  tree_innerList.setVal( tree_idx, shard_count + tree_count );
-//                }
-//              } // innerList loop
-//            } // outer loop
-//              **/
-//          } // each tree loop
-//        }
-//      }
-//    } 
-//    return ;
-//  }
-//
-//  @Override
-//  public void finishStage(ResponseBuilder rb) {
-//    if (!rb.doFacets || rb.stage != ResponseBuilder.STAGE_GET_FIELDS) return;
-//    // wait until STAGE_GET_FIELDS
-//    // so that "result" is already stored in the response (for aesthetics)
-//
-//    SimpleOrderedMap<List<NamedList<Object>>> tf = rb._pivots;
-//
-//    // get 'facet_counts' from the response
-//    NamedList facetCounts = (NamedList) rb.rsp.getValues().get("facet_counts");
-//    if (facetCounts == null) {
-//      facetCounts = new NamedList();
-//      rb.rsp.add("facet_counts", facetCounts);
-//    }
-//    facetCounts.add( PIVOT_KEY, tf );
-//    rb._pivots = null;
-//  }
-//
-//  public String getDescription() {
-//    return "Handle Pivot (multi-level) Faceting";
-//  }
+
+    return out;
+  }
+
+  /** @see PivotListEntry#VALUE */
+  public static Comparable getValue(NamedList<Object> pivotList) {
+    return (Comparable) PivotFacetHelper.retrieve(PivotListEntry.VALUE,
+                                                  pivotList);
+  }
+
+  /** @see PivotListEntry#FIELD */
+  public static String getField(NamedList<Object> pivotList) {
+    return (String) PivotFacetHelper.retrieve(PivotListEntry.FIELD, pivotList);
+  }
+  
+  /** @see PivotListEntry#COUNT */
+  public static Integer getCount(NamedList<Object> pivotList) {
+    return (Integer) PivotFacetHelper.retrieve(PivotListEntry.COUNT, 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;
+  }
+  
+  private static Object retrieve(PivotListEntry entryToGet, NamedList<Object> pivotList) {
+    return pivotList.get(entryToGet.getName(), entryToGet.getIndex());
+  }
+
 }

Added: 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=1617789&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/PivotFacetProcessor.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/PivotFacetProcessor.java Wed Aug 13 18:23:53 2014
@@ -0,0 +1,252 @@
+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.lucene.util.BytesRefBuilder;
+import org.apache.solr.schema.SchemaField;
+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.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.FacetParams;
+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.Deque;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Processes all Pivot facet logic for a single node -- both non-distrib, and per-shard
+ */
+public class PivotFacetProcessor extends SimpleFacets
+{
+  protected SolrParams params;
+    
+  public PivotFacetProcessor(SolrQueryRequest req, DocSet docs, SolrParams params, ResponseBuilder rb) {
+    super(req, docs, params, rb);
+    this.params = params;
+  }
+  
+  /**
+   * Processes all of the specified {@link FacetParams#FACET_PIVOT} strings, generating 
+   * a completel response tree for each pivot.  The values in this response will either 
+   * be the complete tree of fields and values for the specified pivot in the local index, 
+   * or the requested refinements if the pivot params include the {@link PivotFacet#REFINE_PARAM}
+   */
+  public SimpleOrderedMap<List<NamedList<Object>>> process(String[] pivots) throws IOException {
+    if (!rb.doFacets || pivots == null) 
+      return null;
+    
+    SimpleOrderedMap<List<NamedList<Object>>> pivotResponse = new SimpleOrderedMap<>();
+    for (String pivotList : pivots) {
+      try {
+        this.parseParams(FacetParams.FACET_PIVOT, pivotList);
+      } catch (SyntaxError e) {
+        throw new SolrException(ErrorCode.BAD_REQUEST, e);
+      }
+      List<String> pivotFields = StrUtils.splitSmart(facetValue, ",", true);
+      if( pivotFields.size() < 1 ) {
+        throw new SolrException( ErrorCode.BAD_REQUEST,
+                                 "Pivot Facet needs at least one field name: " + pivotList);
+      } else {
+        SolrIndexSearcher searcher = rb.req.getSearcher();
+        for (String fieldName : pivotFields) {
+          SchemaField sfield = searcher.getSchema().getField(fieldName);
+          if (sfield == null) {
+            throw new SolrException(ErrorCode.BAD_REQUEST, "\"" + fieldName + "\" is not a valid field name in pivot: " + pivotList);
+          }
+        }
+      } 
+
+      //REFINEMENT
+      String fieldValueKey = localParams == null ? null : localParams.get(PivotFacet.REFINE_PARAM);
+      if(fieldValueKey != null ){
+        String[] refinementValuesByField = params.getParams(PivotFacet.REFINE_PARAM+fieldValueKey);
+        for(String refinements : refinementValuesByField){
+          pivotResponse.addAll(processSingle(pivotFields, refinements));
+        }
+      } else{
+        pivotResponse.addAll(processSingle(pivotFields, null));
+      }
+    }
+    return pivotResponse;
+  }
+
+  /**
+   * 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
+   */
+  private SimpleOrderedMap<List<NamedList<Object>>> processSingle(List<String> pivotFields,
+                                                                  String refinements) throws IOException {
+    SolrIndexSearcher searcher = rb.req.getSearcher();
+    SimpleOrderedMap<List<NamedList<Object>>> pivotResponse = new SimpleOrderedMap<>();
+
+    String field = pivotFields.get(0);
+    SchemaField sfield = searcher.getSchema().getField(field);
+      
+    Deque<String> fnames = new LinkedList<>();
+    for( int i = pivotFields.size()-1; i>1; i-- ) {
+      fnames.push( pivotFields.get(i) );
+    }
+    
+    NamedList<Integer> facetCounts;
+    Deque<String> vnames = new LinkedList<>();
+
+    if (null != refinements) {
+      // All values, split by the field they should go to
+      List<String> refinementValuesByField
+        = PivotFacetHelper.decodeRefinementValuePath(refinements);
+
+      for( int i=refinementValuesByField.size()-1; i>0; i-- ) {
+        vnames.push(refinementValuesByField.get(i));//Only for [1] and on
+      }
+
+      String firstFieldsValues = refinementValuesByField.get(0);
+
+      facetCounts = new NamedList<Integer>();
+      facetCounts.add(firstFieldsValues,
+                      getSubsetSize(this.docs, sfield, firstFieldsValues));
+    } else {
+      // no refinements needed
+      facetCounts = this.getTermCountsForPivots(field, this.docs);
+    }
+    
+    if(pivotFields.size() > 1) {
+      String subField = pivotFields.get(1);
+      pivotResponse.add(key,
+                        doPivots(facetCounts, field, subField, fnames, vnames, this.docs));
+    } else {
+      pivotResponse.add(key, doPivots(facetCounts, field, null, fnames, vnames, this.docs));
+    }
+    return pivotResponse;
+  }
+  
+  /**
+   * 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 {
+
+    SolrIndexSearcher searcher = rb.req.getSearcher();
+    // TODO: optimize to avoid converting to an external string and then having to convert back to internal below
+    SchemaField sfield = searcher.getSchema().getField(field);
+    FieldType ftype = sfield.getType();
+
+    String nextField = fnames.poll();
+
+    // re-useable BytesRefBuilder for conversion of term values to Objects
+    BytesRefBuilder termval = new BytesRefBuilder(); 
+
+    List<NamedList<Object>> values = new ArrayList<>( superFacets.size() );
+    for (Map.Entry<String, Integer> kv : superFacets) {
+      // 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();
+
+        SimpleOrderedMap<Object> pivot = new SimpleOrderedMap<>();
+        pivot.add( "field", field );
+        if (null == fieldValue) {
+          pivot.add( "value", null );
+        } else {
+          ftype.readableToIndexed(fieldValue, termval);
+          pivot.add( "value", ftype.toObject(sfield, termval.get()) );
+        }
+        pivot.add( "count", kv.getValue() );
+
+        DocSet subset = getSubset(docs, sfield, fieldValue);
+        
+        if( subField != null )  {
+          NamedList<Integer> facetCounts;
+          if(!vnames.isEmpty()){
+            String val = vnames.pop();
+            facetCounts = new NamedList<Integer>();
+            facetCounts.add(val, getSubsetSize(subset,
+                                               searcher.getSchema().getField(subField),
+                                               val));
+          } else {
+            facetCounts = this.getTermCountsForPivots(subField, subset);
+          }
+
+          if (facetCounts.size() >= 1) {
+            pivot.add( "pivot", doPivots( facetCounts, subField, nextField, fnames, vnames, subset) );
+          }
+        }
+        values.add( pivot );
+      }
+
+    }
+    // put the field back on the list
+    fnames.push( nextField );
+    return values;
+  }
+  
+  /**
+   * Given a base docset, computes the size of the subset of documents corrisponding to the specified pivotValue
+   *
+   * @param base the set of documents to evalute relative to
+   * @param field the field type used by the pivotValue
+   * @param pivotValue String representation of the value, may be null (ie: "missing")
+   */
+  private int getSubsetSize(DocSet base, SchemaField field, String pivotValue) throws IOException {
+    FieldType ft = field.getType();
+    if ( null == pivotValue ) {
+      Query query = ft.getRangeQuery(null, field, null, null, false, false);
+      DocSet hasVal = searcher.getDocSet(query);
+      return base.andNotSize(hasVal);
+    } else {
+      Query query = ft.getFieldQuery(null, field, pivotValue);
+      return searcher.numDocs(query, base);
+    }
+  }
+
+  /**
+   * Given a base docset, computes the subset of documents corrisponding to the specified pivotValue
+   *
+   * @param base the set of documents to evalute relative to
+   * @param field the field type used by the pivotValue
+   * @param pivotValue String representation of the value, may be null (ie: "missing")
+   */
+  private DocSet getSubset(DocSet base, SchemaField field, String pivotValue) throws IOException {
+    FieldType ft = field.getType();
+    if ( null == pivotValue ) {
+      Query query = ft.getRangeQuery(null, field, null, null, false, false);
+      DocSet hasVal = searcher.getDocSet(query);
+      return base.andNot(hasVal);
+    } else {
+      Query query = ft.getFieldQuery(null, field, pivotValue);
+      return searcher.getDocSet(query, base);
+    }
+  }
+
+  private int getMinCountForField(String fieldname){
+    return params.getFieldInt(fieldname, FacetParams.FACET_PIVOT_MINCOUNT, 1);
+  }
+  
+}

Added: 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=1617789&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/PivotFacetValue.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/PivotFacetValue.java Wed Aug 13 18:23:53 2014
@@ -0,0 +1,206 @@
+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 java.util.BitSet;
+import java.util.Date;
+import java.util.List;
+import java.util.Locale;
+
+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.util.PivotListEntry;
+
+/**
+ * Models a single (value, count) pair that will exist in the collection of values for a 
+ * {@link PivotFacetField} parent.  This <code>PivotFacetValue</code> may itself have a 
+ * nested {@link PivotFacetField} child
+ *
+ * @see PivotFacetField
+ * @see PivotFacetFieldValueCollection
+ */
+@SuppressWarnings("rawtypes")
+public class PivotFacetValue {
+    
+  private final BitSet sourceShards = new BitSet();
+  private final PivotFacetField parentPivot;
+  private final Comparable value;
+  // child can't be final, circular ref on construction
+  private PivotFacetField childPivot = null; 
+  private int count; // mutable
+  
+  private PivotFacetValue(PivotFacetField parent, Comparable val) { 
+    this.parentPivot = parent;
+    this.value = val;
+  }
+
+  /** 
+   * The value of the asssocated field modeled by this <code>PivotFacetValue</code>. 
+   * May be null if this <code>PivotFacetValue</code> models the count for docs 
+   * "missing" the field value.
+   *
+   * @see FacetParams#FACET_MISSING
+   */
+  public Comparable getValue() { return value; }
+
+  /** The count corrisponding to the value modeled by this <code>PivotFacetValue</code> */
+  public int getCount() { return count; }
+
+  /** 
+   * The {@link PivotFacetField} corrisponding to the nested child pivot for this 
+   * <code>PivotFacetValue</code>. May be null if this object is the leaf of a pivot.
+   */
+  public PivotFacetField getChildPivot() { return childPivot; }
+
+
+  /** 
+   * A recursive method that walks up the tree of pivot fields/values to build 
+   * a list of the String representations of the values that lead down to this 
+   * PivotFacetValue.
+   *
+   * @return a mutable List of the pivot value Strings leading down to and including 
+   *      this pivot value, will never be null but may contain nulls
+   * @see PivotFacetField#getValuePath
+   */
+  public List<String> getValuePath() {
+    List<String> out = parentPivot.getValuePath();
+
+    // Note: this code doesn't play nice with custom FieldTypes -- see SOLR-6330
+
+    if (null == value) {
+      out.add(null);
+    } else if (value instanceof Date) {
+      out.add(TrieDateField.formatExternal((Date) value));
+    } else {
+      out.add(value.toString());
+    }
+    return out;
+  }
+
+  /**
+   * A recursive method to construct a new <code>PivotFacetValue</code> object from 
+   * the contents of the {@link NamedList} provided by the specified shard, relative 
+   * to the specified field.  
+   *
+   * If the <code>NamedList</code> contains data for a child {@link PivotFacetField} 
+   * that will be recursively built as well.
+   *
+   * @see PivotFacetField#createFromListOfNamedLists
+   * @param shardNumber the id of the shard that provided this data
+   * @param rb The response builder of the current request
+   * @param parentField the parent field in the current pivot associated with this value
+   * @param pivotData the data from the specified shard for this pivot value
+   */
+  @SuppressWarnings("unchecked")
+  public static PivotFacetValue createFromNamedList(int shardNumber, ResponseBuilder rb, PivotFacetField parentField, NamedList<Object> pivotData) {
+    
+    Comparable pivotVal = null;
+    int pivotCount = 0;
+    List<NamedList<Object>> childPivotData = null;
+    
+    for (int i = 0; i < pivotData.size(); i++) {
+      String key = pivotData.getName(i);
+      Object value = pivotData.getVal(i);
+      PivotListEntry entry = PivotListEntry.get(key);
+      
+      switch (entry) {
+
+      case VALUE: 
+        pivotVal = (Comparable)value;
+        break;
+      case FIELD:
+        assert parentField.field.equals(value) 
+          : "Parent Field mismatch: " + parentField.field + "!=" + value;
+        break;
+      case COUNT:
+        pivotCount = (Integer)value;
+        break;
+      case PIVOT:
+        childPivotData = (List<NamedList<Object>>)value;
+        break;
+      default:
+        throw new RuntimeException("PivotListEntry contains unaccounted for item: " + entry);
+      }
+    }    
+
+    PivotFacetValue newPivotFacet = new PivotFacetValue(parentField, pivotVal);
+    newPivotFacet.count = pivotCount;
+    newPivotFacet.sourceShards.set(shardNumber);
+    
+    newPivotFacet.childPivot = PivotFacetField.createFromListOfNamedLists(shardNumber, rb, newPivotFacet, childPivotData);
+    
+    return newPivotFacet;
+  }
+
+  /** 
+   * A <b>NON-Recursive</b> method indicating if the specified shard has already
+   * contributed to the count for this value.
+   */
+  public boolean shardHasContributed(int shardNum) {
+    return sourceShards.get(shardNum);
+  }
+  
+  /** 
+   * A recursive method for generating a NamedList from this value suitable for 
+   * including in a pivot facet response to the original distributed request.
+   *
+   * @see PivotFacetField#convertToListOfNamedLists
+   */
+  public NamedList<Object> convertToNamedList() {
+    NamedList<Object> newList = new SimpleOrderedMap<>();
+    newList.add(PivotListEntry.FIELD.getName(), parentPivot.field);
+    newList.add(PivotListEntry.VALUE.getName(), value);    
+    newList.add(PivotListEntry.COUNT.getName(), count);      
+    if (childPivot != null && childPivot.convertToListOfNamedLists() != null) {
+      newList.add(PivotListEntry.PIVOT.getName(), childPivot.convertToListOfNamedLists());
+    }
+    return newList;
+  }      
+  
+  /**
+   * Merges in the count contributions from the specified shard for each.
+   * This method is recursive if the shard data includes sub-pivots
+   *
+   * @see PivotFacetField#contributeFromShard
+   * @see PivotFacetField#createFromListOfNamedLists
+   */
+  public void mergeContributionFromShard(int shardNumber, ResponseBuilder rb, NamedList<Object> value) {
+    assert null != value : "can't merge in null data";
+    
+    if (!shardHasContributed(shardNumber)) {
+      sourceShards.set(shardNumber);      
+      count += PivotFacetHelper.getCount(value);
+    }
+    
+    List<NamedList<Object>> shardChildPivots = PivotFacetHelper.getPivots(value);
+    // sub pivot -- we may not have seen this yet depending on refinement
+    if (null == childPivot) {
+      childPivot = PivotFacetField.createFromListOfNamedLists(shardNumber, rb,  this,  shardChildPivots);
+    } else {
+      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/ShardRequest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/ShardRequest.java?rev=1617789&r1=1617788&r2=1617789&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/ShardRequest.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/ShardRequest.java Wed Aug 13 18:23:53 2014
@@ -39,6 +39,7 @@ public class ShardRequest {
   public final static int PURPOSE_GET_TERMS       =0x400;
   public final static int PURPOSE_GET_TOP_GROUPS  =0x800;
   public final static int PURPOSE_GET_MLT_RESULTS =0x1000;
+  public final static int PURPOSE_REFINE_PIVOT_FACETS =0x2000;
 
   public int purpose;  // the purpose of this request
 

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/request/SimpleFacets.java?rev=1617789&r1=1617788&r2=1617789&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/request/SimpleFacets.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/request/SimpleFacets.java Wed Aug 13 18:23:53 2014
@@ -332,15 +332,45 @@ public class SimpleFacets {
     ENUM, FC, FCS;
   }
 
+  /**
+   * Term counts for use in pivot faceting that resepcts the appropriate mincount
+   * @see FacetParams#FACET_PIVOT_MINCOUNT
+   */
+  public NamedList<Integer> getTermCountsForPivots(String field, DocSet docs) throws IOException {
+    Integer mincount = params.getFieldInt(field, FacetParams.FACET_PIVOT_MINCOUNT, 1);
+    return getTermCounts(field, mincount, docs);
+  }
+
+  /**
+   * Term counts for use in field faceting that resepects the appropriate mincount
+   *
+   * @see FacetParams#FACET_MINCOUNT
+   */
   public NamedList<Integer> getTermCounts(String field) throws IOException {
     return getTermCounts(field, this.docs);
   }
 
+  /**
+   * Term counts for use in field faceting that resepects the appropriate mincount
+   *
+   * @see FacetParams#FACET_MINCOUNT
+   */
   public NamedList<Integer> getTermCounts(String field, DocSet base) throws IOException {
+    Integer mincount = params.getFieldInt(field, FacetParams.FACET_MINCOUNT);
+    return getTermCounts(field, mincount, base);
+  }
+
+  /**
+   * Term counts for use in field faceting that resepcts the specified mincount - 
+   * if mincount is null, the "zeros" param is consulted for the appropriate backcompat 
+   * default
+   *
+   * @see FacetParams#FACET_ZEROS
+   */
+  private NamedList<Integer> getTermCounts(String field, Integer mincount, DocSet base) throws IOException {
     int offset = params.getFieldInt(field, FacetParams.FACET_OFFSET, 0);
     int limit = params.getFieldInt(field, FacetParams.FACET_LIMIT, 100);
     if (limit == 0) return new NamedList<>();
-    Integer mincount = params.getFieldInt(field, FacetParams.FACET_MINCOUNT);
     if (mincount==null) {
       Boolean zeros = params.getFieldBool(field, FacetParams.FACET_ZEROS);
       // mincount = (zeros!=null && zeros) ? 0 : 1;
@@ -554,7 +584,8 @@ public class SimpleFacets {
             try {
               NamedList<Object> result = new SimpleOrderedMap<>();
               if(termList != null) {
-                result.add(workerKey, getListedTermCounts(workerFacetValue, termList, workerBase));
+                List<String> terms = StrUtils.splitSmart(termList, ",", true);
+                result.add(workerKey, getListedTermCounts(workerFacetValue, workerBase, terms));
               } else {
                 result.add(workerKey, getTermCounts(workerFacetValue, workerBase));
               }
@@ -597,13 +628,25 @@ public class SimpleFacets {
   }
 
 
+  /**
+   * Computes the term-&gt;count counts for the specified termList relative to the 
+   * @param field the name of the field to compute term counts against
+   * @param termList a comma seperated (and backslash escaped) list of term values (in the specified field) to compute the counts for
+   * @see StrUtils#splitSmart
+   */
   private NamedList<Integer> getListedTermCounts(String field, String termList) throws IOException {
-    return getListedTermCounts(field, termList, this.docs);
+    List<String> terms = StrUtils.splitSmart(termList, ",", true);
+    return getListedTermCounts(field, this.docs, terms);
   }
 
-  private NamedList getListedTermCounts(String field, String termList, DocSet base) throws IOException {
+  /**
+   * Computes the term-&gt;count counts for the specified term values relative to the 
+   * @param field the name of the field to compute term counts against
+   * @param base the docset to compute term counts relative to
+   * @param terms a list of term values (in the specified field) to compute the counts for 
+   */
+  protected NamedList<Integer> getListedTermCounts(String field, DocSet base, List<String> terms) throws IOException {
     FieldType ft = searcher.getSchema().getFieldType(field);
-    List<String> terms = StrUtils.splitSmart(termList, ",", true);
     NamedList<Integer> res = new NamedList<>();
     for (String term : terms) {
       String internal = ft.toInternal(term);

Added: 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=1617789&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/PivotListEntry.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/PivotListEntry.java Wed Aug 13 18:23:53 2014
@@ -0,0 +1,62 @@
+package org.apache.solr.util;
+
+/*
+ * 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 java.util.Locale;
+
+/**
+ * Enum for modeling the elements of a (nested) pivot entry as expressed in a NamedList
+ */
+public enum PivotListEntry {
+  
+  FIELD(0), 
+  VALUE(1),
+  COUNT(2),
+  PIVOT(3);
+  
+  // we could just use the ordinal(), but safer to be very explicit
+  private final int index;
+  
+  private PivotListEntry(int index) {
+    this.index = index;
+  }
+  
+  /**
+   * Case-insensitive lookup of PivotListEntry by name
+   * @see #getName
+   */
+  public static PivotListEntry get(String name) {
+    return PivotListEntry.valueOf(name.toUpperCase(Locale.ROOT));
+  }
+
+  /**
+   * Name of this entry when used in response
+   * @see #get
+   */
+  public String getName() {
+    return name().toLowerCase(Locale.ROOT);
+  }
+  
+  /**
+   * Indec of this entry when used in response
+   */
+  public int getIndex() {
+    return index;
+  }
+
+}

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/CursorPagingTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/CursorPagingTest.java?rev=1617789&r1=1617788&r2=1617789&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/CursorPagingTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/CursorPagingTest.java Wed Aug 13 18:23:53 2014
@@ -31,7 +31,6 @@ import static org.apache.solr.common.par
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.schema.TrieDateField;
 import org.apache.solr.search.CursorMark; //jdoc
 
 import org.noggit.ObjectBuilder;
@@ -39,12 +38,10 @@ import org.noggit.ObjectBuilder;
 import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.ArrayList;
-import java.util.Date;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.Locale;
 import java.util.Map;
 import java.util.UUID;
 
@@ -619,11 +616,6 @@ public class CursorPagingTest extends So
     return 0 != TestUtil.nextInt(random(), 0, 30);
   }
   
-  /** returns likely most (1/10) of the time, otherwise unlikely */
-  private static Object skewed(Object likely, Object unlikely) {
-    return (0 == TestUtil.nextInt(random(), 0, 9)) ? unlikely : likely;
-  }
-  
   /**
    * An immutable list of the fields in the schema that can be used for sorting,
    * deterministically random order.
@@ -898,7 +890,7 @@ public class CursorPagingTest extends So
                                     1.0D / random().nextInt(37)));
     }
     if (useField()) {
-      doc.addField("str", skewed(randomUsableUnicodeString(),
+      doc.addField("str", skewed(randomXmlUsableUnicodeString(),
                                  TestUtil.randomSimpleString(random(), 1, 1)));
     }
     if (useField()) {
@@ -908,8 +900,7 @@ public class CursorPagingTest extends So
       doc.addField("bin", ByteBuffer.wrap(randBytes));
     }
     if (useField()) {
-      doc.addField("date", skewed(randomDate(),
-                                  dateWithRandomSecondOn2010_10_31_at_10_31()));
+      doc.addField("date", skewed(randomDate(), randomSkewedDate()));
     }
     if (useField()) {
       doc.addField("uuid", UUID.randomUUID().toString());
@@ -949,28 +940,6 @@ public class CursorPagingTest extends So
     }
   }
 
-  /**
-   * We want "realistic" unicode strings beyond simple ascii, but because our
-   * updates use XML we need to ensure we don't get "special" code block.
-   */
-  private static String randomUsableUnicodeString() {
-    String result = TestUtil.randomRealisticUnicodeString(random());
-    if (result.matches(".*\\p{InSpecials}.*")) {
-      // oh well
-      result = TestUtil.randomSimpleString(random());
-    }
-    return result;
-  }
-
-  private static String randomDate() {
-    return TrieDateField.formatExternal(new Date(random().nextLong()));
-  }
-
-  private static String dateWithRandomSecondOn2010_10_31_at_10_31() {
-    return String.format(Locale.ROOT, "2010-10-31T10:31:%02d.000Z",
-                         TestUtil.nextInt(random(), 0, 59));
-  }
-
   private static final String[] currencies = { "USD", "EUR", "NOK" };
 
   public static String randomCurrency() {

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/TestDistributedSearch.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/TestDistributedSearch.java?rev=1617789&r1=1617788&r2=1617789&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/TestDistributedSearch.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/TestDistributedSearch.java Wed Aug 13 18:23:53 2014
@@ -61,6 +61,7 @@ public class TestDistributedSearch exten
 
   @Override
   public void doTest() throws Exception {
+    QueryResponse rsp = null;
     int backupStress = stress; // make a copy so we can restore
 
 
@@ -174,6 +175,13 @@ public class TestDistributedSearch exten
 
     // a facet query to test out chars out of the ascii range
     query("q","*:*", "rows",0, "facet","true", "facet.query","{!term f=foo_s}international\u00ff\u01ff\u2222\u3333");
+    
+    // simple field facet on date fields
+    rsp = query("q","*:*", "rows",0, "facet","true", "facet.field", tdate_a);
+    assertEquals(1, rsp.getFacetFields().size());
+    rsp = query("q","*:*", "rows",0, "facet","true", 
+                "facet.field", tdate_b, "facet.field", tdate_a);
+    assertEquals(2, rsp.getFacetFields().size());
 
     // simple date facet on one field
     query("q","*:*", "rows",100, "facet","true", 
@@ -337,7 +345,7 @@ public class TestDistributedSearch exten
     q.set("q", "*:*");
     q.set(ShardParams.SHARDS_INFO, true);
     setDistributedParams(q);
-    QueryResponse rsp = queryServer(q);
+    rsp = queryServer(q);
     NamedList<?> sinfo = (NamedList<?>) rsp.getResponse().get(ShardParams.SHARDS_INFO);
     String shards = getShardsString();
     int cnt = StringUtils.countMatches(shards, ",")+1;

Added: 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=1617789&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/TestCloudPivotFacet.java (added)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/TestCloudPivotFacet.java Wed Aug 13 18:23:53 2014
@@ -0,0 +1,530 @@
+/*
+ * 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.
+ */
+package org.apache.solr.cloud;
+
+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.QueryResponse;
+import org.apache.solr.client.solrj.response.PivotField;
+import org.apache.solr.common.SolrInputDocument;
+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 static org.apache.solr.common.params.FacetParams.*;
+
+import org.apache.commons.lang.StringUtils;
+
+import org.junit.BeforeClass;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Date;
+
+/**
+ * <p>
+ * Randomized testing of Pivot Faceting using SolrCloud.
+ * </p>
+ * <p>
+ * After indexing a bunch of random docs, picks some random fields to pivot facet on, 
+ * and then confirms that the resulting counts match the results of filtering on those 
+ * values.  This gives us strong assertions on the correctness of the total counts for 
+ * each pivot value, but no assertions that the correct "top" counts were chosen.
+ * </p>
+ * <p>
+ * NOTE: this test ignores the control collection and only deals with the 
+ * CloudSolrServer - this is because the randomized field values make it very easy for 
+ * the term stats to miss values even with the overrequest.
+ * (because so many values will tie for "1").  What we care about here is 
+ * that the counts we get back are correct and match what we get when filtering on those 
+ * constraints.
+ * </p>
+ *
+ *
+ *
+ */
+@SuppressSSL // Too Slow
+public class TestCloudPivotFacet extends AbstractFullDistribZkTestBase {
+
+  public static Logger log = LoggerFactory.getLogger(TestCloudPivotFacet.class);
+
+  // param used by test purely for tracing & validation
+  private static String TRACE_MIN = "_test_min";
+  // param used by test purely for tracing & validation
+  private static String TRACE_MISS = "_test_miss";
+  // param used by test purely for tracing & validation
+  private static String TRACE_SORT = "_test_sort";
+
+  /** 
+   * Controls the odds of any given doc having a value in any given field -- as this gets lower, 
+   * the counts for "facet.missing" pivots should increase.
+   * @see #useField()
+   */
+  private static int useFieldRandomizedFactor = -1;
+
+  @BeforeClass
+  public static void initUseFieldRandomizedFactor() {
+    useFieldRandomizedFactor = TestUtil.nextInt(random(), 2, 30);
+    log.info("init'ing useFieldRandomizedFactor = {}", useFieldRandomizedFactor);
+  }
+
+  @Override
+  public void doTest() throws Exception {
+    handle.clear();
+    handle.put("QTime", SKIPVAL);
+    handle.put("timestamp", SKIPVAL);
+    
+    final Set<String> fieldNameSet = new HashSet<>();
+    
+    // build up a randomized index
+    final int numDocs = atLeast(500);
+    log.info("numDocs: {}", numDocs);
+
+    for (int i = 1; i <= numDocs; i++) {
+      SolrInputDocument doc = buildRandomDocument(i);
+
+      // not efficient, but it garuntees 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());
+
+      cloudClient.add(doc);
+    }
+    cloudClient.commit();
+
+    fieldNameSet.remove("id");
+    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
+
+
+    for (int i = 0; i < 5; i++) {
+
+      String q = "*:*";
+      if (random().nextBoolean()) {
+        q = "id:[* TO " + TestUtil.nextInt(random(),300,numDocs) + "]";
+      }
+      ModifiableSolrParams baseP = params("rows", "0", "q", q);
+      
+      if (random().nextBoolean()) {
+        baseP.add("fq", "id:[* TO " + TestUtil.nextInt(random(),200,numDocs) + "]");
+      }
+
+      ModifiableSolrParams pivotP = params(FACET,"true",
+                                           FACET_PIVOT, buildRandomPivot(fieldNames));
+      if (random().nextBoolean()) {
+        pivotP.add(FACET_PIVOT, buildRandomPivot(fieldNames));
+      }
+
+      // keep limit low - lots of unique values, and lots of depth in pivots
+      pivotP.add(FACET_LIMIT, ""+TestUtil.nextInt(random(),1,17));
+
+      // sometimes use an offset
+      if (random().nextBoolean()) {
+        pivotP.add(FACET_OFFSET, ""+TestUtil.nextInt(random(),0,7));
+      }
+
+      if (random().nextBoolean()) {
+        String min = ""+TestUtil.nextInt(random(),0,numDocs+10);
+        pivotP.add(FACET_PIVOT_MINCOUNT, min);
+        // trace param for validation
+        baseP.add(TRACE_MIN, min);
+      }
+
+      if (random().nextBoolean()) {
+        String missing = ""+random().nextBoolean();
+        pivotP.add(FACET_MISSING, missing);
+        // trace param for validation
+        baseP.add(TRACE_MISS, missing);
+      }
+
+      if (random().nextBoolean()) {
+        String sort = random().nextBoolean() ? "index" : "count";
+        pivotP.add(FACET_SORT, sort);
+        // trace param for validation
+        baseP.add(TRACE_SORT, sort);
+      }
+
+      // overrequest
+      //
+      // NOTE: since this test focuses on accuracy of refinement, and doesn't do 
+      // control collection comparisons, there isn't a lot of need for excessive
+      // overrequesting -- we focus here on trying to exercise the various edge cases
+      // involved as different values are used with overrequest
+      if (0 == TestUtil.nextInt(random(),0,4)) {
+        // we want a decent chance of no overrequest at all
+        pivotP.add(FACET_OVERREQUEST_COUNT, "0");
+        pivotP.add(FACET_OVERREQUEST_RATIO, "0");
+      } else {
+        if (random().nextBoolean()) {
+          pivotP.add(FACET_OVERREQUEST_COUNT, ""+TestUtil.nextInt(random(),0,5));
+        }
+        if (random().nextBoolean()) {
+          // sometimes give a ratio less then 1, code should be smart enough to deal
+          float ratio = 0.5F + random().nextFloat();
+          // sometimes go negative
+          if (random().nextBoolean()) {
+            ratio *= -1;
+          }
+          pivotP.add(FACET_OVERREQUEST_RATIO, ""+ratio);
+        }
+      }
+      
+      assertPivotCountsAreCorrect(baseP, pivotP);
+    }
+  }
+
+  /**
+   * Given some query params, executes the request against the cloudClient and 
+   * then walks the pivot facet values in the response, treating each one as a 
+   * filter query to assert the pivot counts are correct.
+   */
+  private void assertPivotCountsAreCorrect(SolrParams baseParams, 
+                                           SolrParams pivotParams) 
+    throws SolrServerException {
+    
+    SolrParams initParams = SolrParams.wrapAppended(pivotParams, baseParams);
+
+    log.info("Doing full run: {}", initParams);
+    countNumFoundChecks = 0;
+
+    NamedList<List<PivotField>> pivots = null;
+    try {
+      QueryResponse initResponse = cloudClient.query(initParams);
+      pivots = initResponse.getFacetPivot();
+      assertNotNull(initParams + " has null pivots?", pivots);
+      assertEquals(initParams + " num pivots", 
+                   initParams.getParams("facet.pivot").length, pivots.size());
+    } catch (Exception e) {
+      throw new RuntimeException("init query failed: " + initParams + ": " + 
+                                 e.getMessage(), e);
+    }
+    try {
+      for (Map.Entry<String,List<PivotField>> pivot : pivots) {
+        final String pivotKey = pivot.getKey();
+        // :HACK: for counting the max possible pivot depth
+        final int maxDepth = 1 + pivotKey.length() - pivotKey.replace(",","").length();
+
+        assertTraceOk(pivotKey, baseParams, pivot.getValue());
+
+        // NOTE: we can't make any assumptions/assertions about the number of
+        // constraints here because of the random data - which means if pivotting is
+        // completely broken and there are no constrains this loop could be a No-Op
+        // but in that case we just have to trust that DistributedFacetPivotTest
+        // will catch it.
+        for (PivotField constraint : pivot.getValue()) {
+          int depth = assertPivotCountsAreCorrect(pivotKey, baseParams, constraint);
+          
+          // we can't assert that the depth reached is the same as the depth requested
+          // because the fq and/or mincount may have pruned the tree too much
+          assertTrue("went too deep: "+depth+": " + pivotKey + " ==> " + pivot,
+                     depth <= maxDepth);
+
+        }
+      }
+    } catch (AssertionError e) {
+      throw new AssertionError(initParams + " ==> " + e.getMessage(), e);
+    } finally {
+      log.info("Ending full run (countNumFoundChecks={}): {}", 
+               countNumFoundChecks, initParams);
+    }
+  }
+  
+  /**
+   * Recursive Helper method for asserting that pivot constraint counds match
+   * results when filtering on those constraints. Returns the recursive depth reached 
+   * (for sanity checking)
+   */
+  private int assertPivotCountsAreCorrect(String pivotName,
+                                          SolrParams baseParams, 
+                                          PivotField constraint) 
+    throws SolrServerException {
+
+    SolrParams p = SolrParams.wrapAppended(baseParams,
+                                           params("fq", buildFilter(constraint)));
+    List<PivotField> subPivots = null;
+    try {
+      assertNumFound(pivotName, constraint.getCount(), p);
+      subPivots = constraint.getPivot();
+    } catch (Exception e) {
+      throw new RuntimeException(pivotName + ": count query failed: " + p + ": " + 
+                                 e.getMessage(), e);
+    }
+    int depth = 0;
+    if (null != subPivots) {
+      assertTraceOk(pivotName, baseParams, subPivots);
+
+      for (PivotField subPivot : subPivots) {
+        depth = assertPivotCountsAreCorrect(pivotName, p, subPivot);
+      }
+    }
+    return depth + 1;
+  }
+
+  /**
+   * 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
+   */
+  private void assertTraceOk(String pivotName, SolrParams baseParams, List<PivotField> constraints) {
+    if (null == constraints || 0 == constraints.size()) {
+      return;
+    }
+    final int maxIdx = constraints.size() - 1;
+      
+    final int min = baseParams.getInt(TRACE_MIN, -1);
+    final boolean expectMissing = baseParams.getBool(TRACE_MISS, false);
+    final boolean checkCount = "count".equals(baseParams.get(TRACE_SORT, "count"));
+
+    int prevCount = Integer.MAX_VALUE;
+
+    for (int i = 0; i <= maxIdx; i++) {
+      final PivotField constraint = constraints.get(i);
+      final int count = constraint.getCount();
+
+      if (0 < min) {
+        assertTrue(pivotName + ": val #"+i +" of " + maxIdx + 
+                   ": count("+count+") < facet.mincount("+min+"): " + constraint,
+                   min <= count);
+      }
+      // missing value must always come last, but only if facet.missing was used
+      // and may not exist at all (mincount, none missing for this sub-facet, etc...)
+      if ((i < maxIdx) || (!expectMissing)) {
+        assertNotNull(pivotName + ": val #"+i +" of " + maxIdx + 
+                      " has null value: " + constraint,
+                      constraint.getValue());
+      }
+      // if we are expecting count based sort, then the count of each constraint 
+      // must be lt-or-eq the count that came before -- or it must be the last value and 
+      // be "missing"
+      if (checkCount) {
+        assertTrue(pivotName + ": val #"+i +" of" + maxIdx + 
+                   ": count("+count+") > prevCount("+prevCount+"): " + constraint,
+                   ((count <= prevCount)
+                    || (expectMissing && i == maxIdx && null == constraint.getValue())));
+        prevCount = count;
+      }
+    }
+  }
+
+  /**
+   * Given a PivotField constraint, generate a query for the field+value
+   * for use in an <code>fq</code> to verify the constraint count
+   */
+  private static String buildFilter(PivotField constraint) {
+    Object value = constraint.getValue();
+    if (null == value) {
+      // facet.missing, exclude any indexed term
+      return "-" + constraint.getField() + ":[* TO *]";
+    }
+    // otherwise, build up a term filter...
+    String prefix = "{!term f=" + constraint.getField() + "}";
+    if (value instanceof Date) {
+      return prefix + TrieDateField.formatExternal((Date)value);
+    } else {
+      return prefix + value;
+    }
+  }
+
+
+  /**
+   * Creates a random facet.pivot param string using some of the specified fieldNames
+   */
+  private static String buildRandomPivot(String[] fieldNames) {
+    final int depth = TestUtil.nextInt(random(), 1, 3);
+    String [] fields = new String[depth];
+    for (int i = 0; i < depth; i++) {
+      // yes this means we might use the same field twice
+      // makes it a robust test (especially for multi-valued fields)
+      fields[i] = fieldNames[TestUtil.nextInt(random(),0,fieldNames.length-1)];
+    }
+    return StringUtils.join(fields, ",");
+  }
+
+
+  /**
+   * Creates a document with randomized field values, some of which be missing values, 
+   * some of which will be multi-valued (per the schema) and some of which will be 
+   * skewed so that small subsets of the ranges will be more common (resulting in an 
+   * increased likelihood of duplicate values)
+   * 
+   * @see #buildRandomPivot
+   */
+  private static SolrInputDocument buildRandomDocument(int id) {
+    SolrInputDocument doc = sdoc("id", id);
+    // most fields are in most docs
+    // if field is in a doc, then "skewed" chance val is from a dense range
+    // (hopefully with lots of duplication)
+    for (String prefix : new String[] { "pivot_i", "pivot_ti" }) {
+      if (useField()) {
+        doc.addField(prefix+"1", skewed(TestUtil.nextInt(random(), 20, 50),
+                                        random().nextInt()));
+                                        
+      }
+      if (useField()) {
+        int numMulti = atLeast(1);
+        while (0 < numMulti--) {
+          doc.addField(prefix, skewed(TestUtil.nextInt(random(), 20, 50), 
+                                      random().nextInt()));
+        }
+      }
+    }
+    for (String prefix : new String[] { "pivot_l", "pivot_tl" }) {
+      if (useField()) {
+        doc.addField(prefix+"1", skewed(TestUtil.nextInt(random(), 5000, 5100),
+                                        random().nextLong()));
+      }
+      if (useField()) {
+        int numMulti = atLeast(1);
+        while (0 < numMulti--) {
+          doc.addField(prefix, skewed(TestUtil.nextInt(random(), 5000, 5100), 
+                                      random().nextLong()));
+        }
+      }
+    }
+    for (String prefix : new String[] { "pivot_f", "pivot_tf" }) {
+      if (useField()) {
+        doc.addField(prefix+"1", skewed(1.0F / random().nextInt(13),
+                                        random().nextFloat() * random().nextInt()));
+      }
+      if (useField()) {
+        int numMulti = atLeast(1);
+        while (0 < numMulti--) {
+          doc.addField(prefix, skewed(1.0F / random().nextInt(13),
+                                      random().nextFloat() * random().nextInt()));
+        }
+      }
+    }
+    for (String prefix : new String[] { "pivot_d", "pivot_td" }) {
+      if (useField()) {
+        doc.addField(prefix+"1", skewed(1.0D / random().nextInt(19),
+                                        random().nextDouble() * random().nextInt()));
+      }
+      if (useField()) {
+        int numMulti = atLeast(1);
+        while (0 < numMulti--) {
+          doc.addField(prefix, skewed(1.0D / random().nextInt(19),
+                                      random().nextDouble() * random().nextInt()));
+        }
+      }
+    }
+    for (String prefix : new String[] { "pivot_dt", "pivot_tdt" }) {
+      if (useField()) {
+        doc.addField(prefix+"1", skewed(randomSkewedDate(), randomDate()));
+                                        
+      }
+      if (useField()) {
+        int numMulti = atLeast(1);
+        while (0 < numMulti--) {
+          doc.addField(prefix, skewed(randomSkewedDate(), randomDate()));
+                                      
+        }
+      }
+    }
+    {
+      String prefix = "pivot_b";
+      if (useField()) {
+        doc.addField(prefix+"1", random().nextBoolean() ? "t" : "f");
+      }
+      if (useField()) {
+        int numMulti = atLeast(1);
+        while (0 < numMulti--) {
+          doc.addField(prefix, random().nextBoolean() ? "t" : "f");
+        }
+      }
+    }
+    for (String prefix : new String[] { "pivot_x_s", "pivot_y_s", "pivot_z_s"}) {
+      if (useField()) {
+        doc.addField(prefix+"1", skewed(TestUtil.randomSimpleString(random(), 1, 1),
+                                        randomXmlUsableUnicodeString()));
+      }
+      if (useField()) {
+        int numMulti = atLeast(1);
+        while (0 < numMulti--) {
+          doc.addField(prefix, skewed(TestUtil.randomSimpleString(random(), 1, 1),
+                                      randomXmlUsableUnicodeString()));
+        }
+      }
+    }
+
+    //
+    // for the remaining fields, make every doc have a value in a dense range
+    //
+
+    for (String prefix : new String[] { "dense_pivot_x_s", "dense_pivot_y_s" }) {
+      if (useField()) {
+        doc.addField(prefix+"1", TestUtil.randomSimpleString(random(), 1, 1));
+      }
+      if (useField()) {
+        int numMulti = atLeast(1);
+        while (0 < numMulti--) {
+          doc.addField(prefix, TestUtil.randomSimpleString(random(), 1, 1));
+        }
+      }
+    }
+    for (String prefix : new String[] { "dense_pivot_i", "dense_pivot_ti" }) {
+      if (useField()) {
+        doc.addField(prefix+"1", TestUtil.nextInt(random(), 20, 50));
+      }
+      if (useField()) {
+        int numMulti = atLeast(1);
+        while (0 < numMulti--) {
+          doc.addField(prefix, TestUtil.nextInt(random(), 20, 50));
+        }
+      }
+    }
+
+    return doc;
+  }
+
+  /** 
+   * Similar to usually() but we want it to happen just as often regardless
+   * of test multiplier and nightly status
+   *
+   * @see #useFieldRandomizedFactor
+   */
+  private static boolean useField() {
+    assert 0 < useFieldRandomizedFactor;
+    return 0 != TestUtil.nextInt(random(), 0, useFieldRandomizedFactor);
+  }
+  
+  /**
+   * Asserts the number of docs matching the SolrParams aganst the cloudClient
+   */
+  private void assertNumFound(String msg, int expected, SolrParams p) 
+    throws SolrServerException {
+
+    countNumFoundChecks++;
+
+    SolrParams params = SolrParams.wrapDefaults(params("rows","0"), p);
+    assertEquals(msg + ": " + params, 
+                 expected, cloudClient.query(params).getResults().getNumFound());
+  }
+
+  /**
+   * @see #assertNumFound
+   * @see #assertPivotCountsAreCorrect(SolrParams,SolrParams)
+   */
+  private int countNumFoundChecks = 0;
+}