You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by da...@apache.org on 2018/11/02 11:16:44 UTC

[35/56] lucene-solr:jira/gradle: Add :solr:contrib:analytics module

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/PivotNode.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/PivotNode.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/PivotNode.java
new file mode 100644
index 0000000..c6c0dc4
--- /dev/null
+++ b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/PivotNode.java
@@ -0,0 +1,263 @@
+/*
+ * 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.analytics.facet;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Consumer;
+
+import org.apache.solr.analytics.AnalyticsDriver;
+import org.apache.solr.analytics.function.ExpressionCalculator;
+import org.apache.solr.analytics.function.ReductionCollectionManager;
+import org.apache.solr.analytics.function.ReductionCollectionManager.ReductionDataCollection;
+import org.apache.solr.analytics.function.reduction.data.ReductionData;
+import org.apache.solr.analytics.util.AnalyticsResponseHeadings;
+import org.apache.solr.analytics.value.StringValueStream;
+
+/**
+ * Representation of one layer of a Pivot Facet. A PivotFacet node is individually sortable,
+ * and is collected during the streaming phase of the {@link AnalyticsDriver}.
+ */
+public abstract class PivotNode<T> extends SortableFacet implements Consumer<String> {
+  private StringValueStream expression;
+  protected Map<String,T> currentPivot;
+
+  public PivotNode(String name, StringValueStream expression) {
+    super(name);
+    this.expression = expression;
+  }
+
+  /**
+   * Determine which facet values match the current document. Add the {@link ReductionDataCollection}s of the relevant facet values
+   * to the targets of the streaming {@link ReductionCollectionManager} so that they are updated with the current document's data.
+   */
+  public void addFacetValueCollectionTargets(Map<String,T> pivot) {
+    currentPivot = pivot;
+    expression.streamStrings(this);
+  }
+  
+  /**
+   * Import the shard data from a bit-stream for the given pivot, exported by the {@link #exportPivot} method 
+   * in the each of the collection's shards.
+   * 
+   * @param input The bit-stream to import the data from
+   * @param pivot the values for this pivot node and the pivot children (if they exist)
+   * @throws IOException if an exception occurs while reading from the {@link DataInput}
+   */
+  public void importPivot(DataInput input, Map<String,T> pivot) throws IOException {
+    int size = input.readInt();
+    currentPivot = pivot;
+    for (int i = 0; i < size; ++i) {
+      importPivotValue(input, input.readUTF());
+    }
+  }
+  /**
+   * Import the next pivot value's set of {@link ReductionData} and children's {@link ReductionData} if they exist.
+   * 
+   * @param input the bit-stream to import the reduction data from
+   * @param pivotValue the next pivot value
+   * @throws IOException if an exception occurs while reading from the input
+   */
+  protected abstract void importPivotValue(DataInput input, String pivotValue) throws IOException;
+
+  /**
+   * Export the shard data through a bit-stream for the given pivot, 
+   * to be imported by the {@link #importPivot} method in the originating shard.
+   * 
+   * @param output The bit-stream to output the data through
+   * @param pivot the values for this pivot node and the pivot children (if they exist)
+   * @throws IOException if an exception occurs while writing to the {@link DataOutput}
+   */
+  public void exportPivot(DataOutput output, Map<String,T> pivot) throws IOException {
+    output.writeInt(pivot.size());
+    for (String pivotValue : pivot.keySet()) {
+      output.writeUTF(pivotValue);
+      exportPivotValue(output, pivot.get(pivotValue));
+    }
+  }
+  /**
+   * Export the given pivot data, containing {@link ReductionData} and pivot children if they exist.
+   * 
+   * @param output the bit-stream to output the reduction data to
+   * @param pivotData the next pivot value data
+   * @throws IOException if an exception occurs while reading from the input
+   */
+  protected abstract void exportPivotValue(DataOutput output, T pivotData) throws IOException;
+  
+  /**
+   * Create the response of the facet to be returned in the overall analytics response.
+   * 
+   * @param pivot the pivot to create a response for
+   * @return the response of the facet
+   */
+  public abstract Iterable<Map<String,Object>> getPivotedResponse(Map<String,T> pivot);
+  
+  /**
+   * A pivot node that has no pivot children.
+   */
+  public static class PivotLeaf extends PivotNode<ReductionDataCollection> {
+
+    public PivotLeaf(String name, StringValueStream expression) {
+      super(name, expression);
+    }
+    
+    @Override
+    public void accept(String pivotValue) {
+      ReductionDataCollection collection = currentPivot.get(pivotValue);
+      if (collection == null) {
+        collection = collectionManager.newDataCollectionTarget();
+        currentPivot.put(pivotValue, collection);
+      } else {
+        collectionManager.addCollectTarget(collection);
+      }
+    }
+
+    @Override
+    protected void importPivotValue(DataInput input, String pivotValue) throws IOException {
+      ReductionDataCollection dataCollection = currentPivot.get(pivotValue);
+      if (dataCollection == null) {
+        currentPivot.put(pivotValue, collectionManager.newDataCollectionIO());
+      } else {
+        collectionManager.prepareReductionDataIO(dataCollection);
+      }
+      collectionManager.mergeData();
+    }
+
+    @Override
+    protected void exportPivotValue(DataOutput output, ReductionDataCollection pivotData) throws IOException {
+      collectionManager.prepareReductionDataIO(pivotData);
+      collectionManager.exportData();
+    }
+
+    @Override
+    public Iterable<Map<String,Object>> getPivotedResponse(Map<String,ReductionDataCollection> pivot) {
+      final List<FacetBucket> facetResults = new ArrayList<>();
+      pivot.forEach((facetVal, dataCol) -> {
+        collectionManager.setData(dataCol);
+        facetResults.add(new FacetBucket(facetVal,expressionCalculator.getResults()));
+      });
+
+      Iterable<FacetBucket> facetResultsIter = applyOptions(facetResults);
+      final LinkedList<Map<String,Object>> results = new LinkedList<>();
+      // Export each expression in the bucket.
+      for (FacetBucket bucket : facetResultsIter) {
+        Map<String, Object> bucketMap = new HashMap<>();
+        bucketMap.put(AnalyticsResponseHeadings.PIVOT_NAME, name);
+        bucketMap.put(AnalyticsResponseHeadings.FACET_VALUE, bucket.getFacetValue());
+        bucketMap.put(AnalyticsResponseHeadings.RESULTS, bucket.getResults());
+        results.add(bucketMap);
+      }
+      return results;
+    }
+  }
+  
+  /**
+   * A pivot node that has pivot children.
+   */
+  public static class PivotBranch<T> extends PivotNode<PivotBranch.PivotDataPair<T>> {
+    private final PivotNode<T> childPivot;
+    public PivotBranch(String name, StringValueStream expression, PivotNode<T> childPivot) {
+      super(name, expression);
+      this.childPivot = childPivot;
+    }
+    
+    @Override
+    public void setReductionCollectionManager(ReductionCollectionManager collectionManager) {
+      super.setReductionCollectionManager(collectionManager);
+      childPivot.setReductionCollectionManager(collectionManager);
+    }
+
+    @Override
+    public void setExpressionCalculator(ExpressionCalculator expressionCalculator) {
+      super.setExpressionCalculator(expressionCalculator);
+      childPivot.setExpressionCalculator(expressionCalculator);
+    }
+    
+    @Override
+    public void accept(String pivotValue) {
+      PivotDataPair<T> pivotData = currentPivot.get(pivotValue);
+      if (pivotData == null) {
+        pivotData = new PivotDataPair<>();
+        pivotData.childPivots = new HashMap<>();
+        pivotData.pivotReduction = collectionManager.newDataCollectionTarget();
+        currentPivot.put(pivotValue, pivotData);
+      } else {
+        collectionManager.addCollectTarget(pivotData.pivotReduction);
+      }
+      childPivot.addFacetValueCollectionTargets(pivotData.childPivots);
+    }
+
+    @Override
+    protected void importPivotValue(DataInput input, String pivotValue) throws IOException {
+      PivotDataPair<T> pivotData = currentPivot.get(pivotValue);
+      if (pivotData == null) {
+        pivotData = new PivotDataPair<>();
+        pivotData.childPivots = new HashMap<>();
+        pivotData.pivotReduction = collectionManager.newDataCollectionIO();
+        currentPivot.put(pivotValue, pivotData);
+      } else {
+        collectionManager.prepareReductionDataIO(pivotData.pivotReduction);
+      }
+      collectionManager.mergeData();
+      childPivot.importPivot(input, pivotData.childPivots);
+    }
+
+    @Override
+    protected void exportPivotValue(DataOutput output, PivotDataPair<T> pivotData) throws IOException {
+      collectionManager.prepareReductionDataIO(pivotData.pivotReduction);
+      collectionManager.exportData();
+      
+      childPivot.exportPivot(output, pivotData.childPivots);
+    }
+
+    @Override
+    public Iterable<Map<String,Object>> getPivotedResponse(Map<String,PivotDataPair<T>> pivot) {
+      final List<FacetBucket> facetResults = new ArrayList<>();
+      pivot.forEach((facetVal, dataPair) -> {
+        collectionManager.setData(dataPair.pivotReduction);
+        facetResults.add(new FacetBucket(facetVal,expressionCalculator.getResults()));
+      });
+
+      Iterable<FacetBucket> facetResultsIter = applyOptions(facetResults);
+      final LinkedList<Map<String,Object>> results = new LinkedList<>();
+      // Export each expression in the bucket.
+      for (FacetBucket bucket : facetResultsIter) {
+        Map<String, Object> bucketMap = new HashMap<>();
+        bucketMap.put(AnalyticsResponseHeadings.PIVOT_NAME, name);
+        bucketMap.put(AnalyticsResponseHeadings.FACET_VALUE, bucket.getFacetValue());
+        bucketMap.put(AnalyticsResponseHeadings.RESULTS, bucket.getResults());
+        bucketMap.put(AnalyticsResponseHeadings.PIVOT_CHILDREN, childPivot.getPivotedResponse(pivot.get(bucket.getFacetValue()).childPivots));
+        results.add(bucketMap);
+      }
+      return results;
+    }
+    
+    /**
+     * Contains pivot data for {@link PivotNode.PivotBranch} classes.
+     */
+    protected static class PivotDataPair<T> {
+      ReductionDataCollection pivotReduction;
+      Map<String,T> childPivots;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/QueryFacet.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/QueryFacet.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/QueryFacet.java
new file mode 100644
index 0000000..f880809
--- /dev/null
+++ b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/QueryFacet.java
@@ -0,0 +1,64 @@
+/*
+ * 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.analytics.facet;
+
+import java.util.Map;
+import java.util.function.Consumer;
+
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.solr.analytics.function.ReductionCollectionManager.ReductionDataCollection;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.search.Filter;
+import org.apache.solr.search.QParser;
+
+/**
+ * A facet that breaks down the data by additional Solr Queries.
+ */
+public class QueryFacet extends AbstractSolrQueryFacet {
+  private final Map<String,String> queries;
+  
+  public QueryFacet(String name, Map<String, String> queries) {
+    super(name);
+    this.queries = queries;
+  }
+  
+  @Override
+  public void createFacetValueExecuters(final Filter filter, SolrQueryRequest queryRequest, Consumer<FacetValueQueryExecuter> consumer) {
+    queries.forEach( (queryName, query) -> {
+      final Query q;
+      try {
+        q = QParser.getParser(query, queryRequest).getQuery();
+      } catch( Exception e ){
+        throw new SolrException(ErrorCode.BAD_REQUEST,"Invalid query '"+query+"' in query facet '" + getName() + "'",e);
+      }
+      // The searcher sends docIds to the QueryFacetAccumulator which forwards
+      // them to <code>collectQuery()</code> in this class for collection.
+      Query queryQuery = new BooleanQuery.Builder()
+          .add(q, Occur.MUST)
+          .add(filter, Occur.FILTER)
+          .build();
+
+      ReductionDataCollection dataCol = collectionManager.newDataCollection();
+      reductionData.put(queryName, dataCol);
+      consumer.accept(new FacetValueQueryExecuter(dataCol, queryQuery));
+    });
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/RangeFacet.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/RangeFacet.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/RangeFacet.java
new file mode 100644
index 0000000..80e8d21
--- /dev/null
+++ b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/RangeFacet.java
@@ -0,0 +1,119 @@
+/*
+ * 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.analytics.facet;
+
+import java.util.EnumSet;
+import java.util.List;
+import java.util.function.Consumer;
+
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.solr.analytics.function.ReductionCollectionManager.ReductionDataCollection;
+import org.apache.solr.analytics.util.FacetRangeGenerator;
+import org.apache.solr.analytics.util.FacetRangeGenerator.FacetRange;
+import org.apache.solr.common.params.FacetParams.FacetRangeInclude;
+import org.apache.solr.common.params.FacetParams.FacetRangeOther;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.search.Filter;
+
+/**
+ * A facet that groups data by a discrete set of ranges.
+ */
+public class RangeFacet extends AbstractSolrQueryFacet {
+  protected final SchemaField field;
+  protected final String start;
+  protected final String end;
+  protected final List<String> gaps;
+  protected boolean hardEnd = false;
+  protected EnumSet<FacetRangeInclude> include;
+  protected EnumSet<FacetRangeOther> others;
+  
+  public RangeFacet(String name, SchemaField field, String start, String end, List<String> gaps) {
+    super(name);
+    this.field = field;
+    this.start = start;
+    this.end = end;
+    this.gaps = gaps;
+    include = EnumSet.of(FacetRangeInclude.LOWER);
+    others = EnumSet.of(FacetRangeOther.NONE);
+  }
+
+  @Override
+  public void createFacetValueExecuters(final Filter filter, SolrQueryRequest queryRequest, Consumer<FacetValueQueryExecuter> consumer) {
+    // Computes the end points of the ranges in the rangeFacet
+    final FacetRangeGenerator<? extends Comparable<?>> rec = FacetRangeGenerator.create(this);
+    final SchemaField sf = field;
+    
+    // Create a rangeFacetAccumulator for each range and 
+    // collect the documents for that range.
+    for (FacetRange range : rec.getRanges()) {
+      Query q = sf.getType().getRangeQuery(null, sf, range.lower, range.upper, range.includeLower,range.includeUpper);
+      // The searcher sends docIds to the RangeFacetAccumulator which forwards
+      // them to <code>collectRange()</code> in this class for collection.
+      Query rangeQuery = new BooleanQuery.Builder()
+          .add(q, Occur.MUST)
+          .add(filter, Occur.FILTER)
+          .build();
+      
+      ReductionDataCollection dataCol = collectionManager.newDataCollection();
+      reductionData.put(range.toString(), dataCol);
+      consumer.accept(new FacetValueQueryExecuter(dataCol, rangeQuery));
+    }
+  }
+
+  public String getStart() {
+    return start;
+  }
+
+  public String getEnd() {
+    return end;
+  }
+
+  public EnumSet<FacetRangeInclude> getInclude() {
+    return include;
+  }
+
+  public void setInclude(EnumSet<FacetRangeInclude> include) {
+    this.include = include;
+  }
+
+  public List<String> getGaps() {
+    return gaps;
+  }
+
+  public boolean isHardEnd() {
+    return hardEnd;
+  }
+
+  public void setHardEnd(boolean hardEnd) {
+    this.hardEnd = hardEnd;
+  }
+
+  public EnumSet<FacetRangeOther> getOthers() {
+    return others;
+  }
+
+  public void setOthers(EnumSet<FacetRangeOther> others) {
+    this.others = others;
+  }
+  
+  public SchemaField getField() {
+    return field;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/SortableFacet.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/SortableFacet.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/SortableFacet.java
new file mode 100644
index 0000000..ef1e04b
--- /dev/null
+++ b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/SortableFacet.java
@@ -0,0 +1,178 @@
+/*
+ * 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.analytics.facet;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.solr.analytics.facet.compare.FacetResultsComparator;
+import org.apache.solr.analytics.util.AnalyticsResponseHeadings;
+import org.apache.solr.common.util.NamedList;
+
+import com.google.common.collect.Iterables;
+
+/**
+ * A facet that can be sorted by either the facet value or an expression value.
+ */
+public abstract class SortableFacet extends AnalyticsFacet {
+  protected FacetSortSpecification sort = null;
+  
+  protected SortableFacet(String name) {
+    super(name);
+  }
+  
+  @Override
+  public NamedList<Object> createOldResponse() {
+    final NamedList<Object> results = new NamedList<>();
+    // Export each expression in the bucket.
+    for (FacetBucket bucket : getBuckets()) {
+      results.add(bucket.getFacetValue(), new NamedList<>(bucket.getResults()));
+    }
+    return results;
+  }
+  
+  @Override
+  public Iterable<Map<String,Object>> createResponse() {
+    final LinkedList<Map<String,Object>> results = new LinkedList<>();
+    // Export each expression in the bucket.
+    for (FacetBucket bucket : getBuckets()) {
+      Map<String, Object> bucketMap = new HashMap<>();
+      bucketMap.put(AnalyticsResponseHeadings.FACET_VALUE, bucket.getFacetValue());
+      bucketMap.put(AnalyticsResponseHeadings.RESULTS, bucket.getResults());
+      results.add(bucketMap);
+    }
+    return results;
+  }
+  
+  private Iterable<FacetBucket> getBuckets() {
+    final List<FacetBucket> facetResults = new ArrayList<>();
+    reductionData.forEach((facetVal, dataCol) -> {
+      collectionManager.setData(dataCol);
+      facetResults.add(new FacetBucket(facetVal,expressionCalculator.getResults()));
+    });
+    
+    return applyOptions(facetResults);
+  }
+
+  /**
+   * Apply the sorting options to the given facet results.
+   * 
+   * @param facetResults to apply sorting options to
+   * @return the sorted results
+   */
+  @SuppressWarnings({"unchecked", "rawtypes"})
+  protected Iterable<FacetBucket> applyOptions(List<FacetBucket> facetResults) {
+    // Sorting the buckets if a sort specification is provided
+    if (sort == null || facetResults.isEmpty()) {
+      return facetResults;
+    }
+    Comparator comp = sort.getComparator();
+    Collections.sort(facetResults, comp);
+    
+    Iterable<FacetBucket> facetResultsIter = facetResults;
+    // apply the limit
+    if (sort.getLimit() > 0) {
+      if (sort.getOffset() > 0) {
+        facetResultsIter = Iterables.skip(facetResultsIter, sort.getOffset());
+      }
+      facetResultsIter = Iterables.limit(facetResultsIter, sort.getLimit());
+    } else if (sort.getLimit() == 0) {
+      return new LinkedList<FacetBucket>();
+    }
+    return facetResultsIter;
+  }
+  
+  /**
+   * Specifies how to sort the buckets of a sortable facet.
+   */
+  public static class FacetSortSpecification {
+    private FacetResultsComparator comparator;
+    protected int limit;
+    protected int offset;
+    
+    public FacetSortSpecification(FacetResultsComparator comparator, int limit, int offset) {
+      this.comparator = comparator;
+      this.limit = limit;
+      this.offset = offset;
+    }
+
+    public FacetResultsComparator getComparator() {
+      return comparator;
+    }
+
+    /**
+     * Get the maximum number of buckets to be returned.
+     * 
+     * @return the limit
+     */
+    public int getLimit() {
+      return limit;
+    }
+    /**
+     * Set the maximum number of buckets to be returned.
+     * 
+     * @param limit the maximum number of buckets
+     */
+    public void setLimit(int limit) {
+      this.limit = limit;
+    }
+    
+    /**
+     * Get the first bucket to return, has to be used with the {@code limit} option.
+     * 
+     * @return the bucket offset
+     */
+    public int getOffset() {
+      return offset;
+    }
+  }
+
+  public SortableFacet.FacetSortSpecification getSort() {
+    return sort;
+  }
+
+  public void setSort(SortableFacet.FacetSortSpecification sort) {
+    this.sort = sort;
+  }
+  
+  public static class FacetBucket {
+    private final String facetValue;
+    private final Map<String,Object> expressionResults;
+    
+    public FacetBucket(String facetValue, Map<String,Object> expressionResults) {
+      this.facetValue = facetValue;
+      this.expressionResults = expressionResults;
+    }
+    
+    public Object getResult(String expression) {
+      return expressionResults.get(expression);
+    }
+    
+    public Map<String,Object> getResults() {
+      return expressionResults;
+    }
+    
+    public String getFacetValue() {
+      return facetValue;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/StreamingFacet.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/StreamingFacet.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/StreamingFacet.java
new file mode 100644
index 0000000..6cca041
--- /dev/null
+++ b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/StreamingFacet.java
@@ -0,0 +1,32 @@
+/*
+ * 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.analytics.facet;
+
+import org.apache.solr.analytics.AnalyticsDriver;
+import org.apache.solr.analytics.function.ReductionCollectionManager;
+import org.apache.solr.analytics.function.ReductionCollectionManager.ReductionDataCollection;
+
+/**
+ * A facet that is collected during the streaming phase of the {@link AnalyticsDriver}.
+ */
+public interface StreamingFacet {
+  /**
+   * Determine which facet values match the current document. Add the {@link ReductionDataCollection}s of the relevant facet values
+   * to the targets of the streaming {@link ReductionCollectionManager} so that they are updated with the current document's data.
+   */
+  void addFacetValueCollectionTargets();
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/ValueFacet.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/ValueFacet.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/ValueFacet.java
new file mode 100644
index 0000000..b1d84ba
--- /dev/null
+++ b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/ValueFacet.java
@@ -0,0 +1,60 @@
+/*
+ * 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.analytics.facet;
+
+import java.util.function.Consumer;
+
+import org.apache.solr.analytics.function.ReductionCollectionManager.ReductionDataCollection;
+import org.apache.solr.analytics.value.StringValueStream;
+
+/**
+ * A facet that breaks up data by the values of a mapping expression or field.
+ * The mapping expression must be castable to a {@link StringValueStream}.
+ */
+public class ValueFacet extends SortableFacet implements StreamingFacet, Consumer<String> {
+  private StringValueStream expression;
+
+  public ValueFacet(String name, StringValueStream expression) {
+    super(name);
+    this.expression = expression;
+  }
+
+  @Override
+  public void addFacetValueCollectionTargets() {
+    expression.streamStrings(this);
+  }
+    
+  @Override
+  public void accept(String t) {
+    ReductionDataCollection collection = reductionData.get(t);
+    if (collection == null) {
+      collection = collectionManager.newDataCollectionTarget();
+      reductionData.put(t, collection);
+    } else {
+      collectionManager.addCollectTarget(collection);
+    }
+  }
+  
+  /**
+   * Get the expression used to create the facet values.
+   * 
+   * @return a string mapping expression
+   */
+  public StringValueStream getExpression() {
+    return expression;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/compare/ConstantComparator.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/compare/ConstantComparator.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/compare/ConstantComparator.java
new file mode 100644
index 0000000..ece09f8
--- /dev/null
+++ b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/compare/ConstantComparator.java
@@ -0,0 +1,30 @@
+/*
+ * 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.analytics.facet.compare;
+
+import org.apache.solr.analytics.facet.SortableFacet.FacetBucket;
+
+/**
+ * A results comparator that compares constants.
+ */
+public class ConstantComparator extends FacetResultsComparator {
+
+  @Override
+  public int compare(FacetBucket b1, FacetBucket b2) {
+      return 0;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/compare/DelegatingComparator.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/compare/DelegatingComparator.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/compare/DelegatingComparator.java
new file mode 100644
index 0000000..200e68b
--- /dev/null
+++ b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/compare/DelegatingComparator.java
@@ -0,0 +1,62 @@
+/*
+ * 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.analytics.facet.compare;
+
+import java.util.Collection;
+
+import org.apache.solr.analytics.facet.SortableFacet.FacetBucket;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+
+/**
+ * A comparator used to sort the facet-value buckets of facet, using the delegate comparator if two values are equal.
+ */
+public class DelegatingComparator extends FacetResultsComparator {
+  private final Iterable<FacetResultsComparator> comparators;
+  
+  /**
+   * Create a delegating results comparator. This comparator will in succession use the given comparators, continuing if the values are equal.
+   * Two buckets are considered equal if and only if all comparators find them equal
+   * 
+   * @param comparators the comparators to use in succession
+   */
+  private DelegatingComparator(Iterable<FacetResultsComparator> comparators) {
+    this.comparators = comparators;
+  }
+  
+  public static FacetResultsComparator joinComparators(Collection<FacetResultsComparator> comparators) throws SolrException {
+    if (comparators.size() == 0) {
+      throw new SolrException(ErrorCode.BAD_REQUEST,"A sort must have at least 1 comparator criteria.");
+    } else if (comparators.size() == 1) {
+      return comparators.iterator().next();
+    } else {
+      return new DelegatingComparator(comparators);
+    }
+  }
+
+  @Override
+  public int compare(FacetBucket b1, FacetBucket b2) {
+    int val = 0;
+    for (FacetResultsComparator comparator : comparators) {
+      val = comparator.compare(b1, b2);
+      if (val != 0) {
+        break;
+      }
+    }
+    return val;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/compare/ExpressionComparator.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/compare/ExpressionComparator.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/compare/ExpressionComparator.java
new file mode 100644
index 0000000..e4c1940
--- /dev/null
+++ b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/compare/ExpressionComparator.java
@@ -0,0 +1,46 @@
+/*
+ * 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.analytics.facet.compare;
+
+import org.apache.solr.analytics.facet.SortableFacet.FacetBucket;
+
+/**
+ * A comparator used to sort the facet-value buckets of facet.
+ */
+public class ExpressionComparator<T extends Comparable<T>> extends FacetResultsComparator {
+  private final String expression;
+  
+  /**
+   * Create an entry comparator comparing the given expression.
+   * 
+   * @param expression the name of the expression results to compare
+   */
+  public ExpressionComparator(String expression) {
+    this.expression = expression;
+  }
+  
+  @SuppressWarnings("unchecked")
+  public int compare(FacetBucket b1, FacetBucket b2) {
+    T t1 = (T)b1.getResult(expression);
+    T t2 = (T)b2.getResult(expression);
+    if (t1 == null || t2 == null) {
+      return Boolean.compare(t2 == null, t1 == null) * resultMult;
+    } else {
+      return t1.compareTo(t2) * resultMult;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/compare/FacetResultsComparator.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/compare/FacetResultsComparator.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/compare/FacetResultsComparator.java
new file mode 100644
index 0000000..9303f21
--- /dev/null
+++ b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/compare/FacetResultsComparator.java
@@ -0,0 +1,52 @@
+/*
+ * 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.analytics.facet.compare;
+
+import java.util.Comparator;
+
+import org.apache.solr.analytics.facet.SortableFacet.FacetBucket;
+
+/**
+ * A comparator used to sort the buckets of facet.
+ */
+public abstract class FacetResultsComparator implements Comparator<FacetBucket> {
+  protected int resultMult;
+  
+  /**
+   * Create a results comparator assuming an ascending ordering.
+   */
+  public FacetResultsComparator() {
+    setDirection(true);
+  }
+  
+  /**
+   * Set the order direction for comparison.
+   * 
+   * @param ascending whether to compare using an ascending ordering
+   */
+  public void setDirection(boolean ascending) {
+    this.resultMult = ascending ? 1 : -1;
+  }
+  
+  /**
+   * Compare one facet bucket to another.
+   * 
+   * @param b1 the first bucket to compare
+   * @param b2 the second bucket to compare
+   */
+  public abstract int compare(FacetBucket b1, FacetBucket b2);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/compare/FacetValueComparator.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/compare/FacetValueComparator.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/compare/FacetValueComparator.java
new file mode 100644
index 0000000..a07ac7b
--- /dev/null
+++ b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/compare/FacetValueComparator.java
@@ -0,0 +1,37 @@
+/*
+ * 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.analytics.facet.compare;
+
+import org.apache.solr.analytics.facet.SortableFacet.FacetBucket;
+
+/**
+ * A results comparator that compares the name of facet value buckets, which is the string value of the facet value.
+ */
+public class FacetValueComparator extends FacetResultsComparator {
+  
+  /**
+   * Create a facet value comparator.
+   */
+  public FacetValueComparator() {
+    super();
+  }
+  
+  @Override
+  public int compare(FacetBucket b1, FacetBucket b2) {
+      return b1.getFacetValue().compareTo(b2.getFacetValue()) * resultMult;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/compare/package-info.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/compare/package-info.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/compare/package-info.java
new file mode 100644
index 0000000..c86ad17
--- /dev/null
+++ b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/compare/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+ 
+/** 
+ * Comparators used to sort the buckets of an analytics facet.
+ */
+package org.apache.solr.analytics.facet.compare;
+
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/package-info.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/package-info.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/package-info.java
new file mode 100644
index 0000000..5812e54
--- /dev/null
+++ b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/facet/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+ 
+/** 
+ * Facets supported by the analytics component.
+ */
+package org.apache.solr.analytics.facet;
+
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/ExpressionCalculator.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/ExpressionCalculator.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/ExpressionCalculator.java
new file mode 100644
index 0000000..3c44555
--- /dev/null
+++ b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/ExpressionCalculator.java
@@ -0,0 +1,71 @@
+/*
+ * 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.analytics.function;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.solr.analytics.AnalyticsExpression;
+import org.apache.solr.analytics.function.ReductionCollectionManager.ReductionDataCollection;
+import org.apache.solr.analytics.util.AnalyticsResponseHeadings;
+import org.apache.solr.common.util.NamedList;
+
+/**
+ * A class used to generate results for a list of {@link AnalyticsExpression}s.
+ */
+public class ExpressionCalculator {
+  private final Iterable<AnalyticsExpression> expressions;
+
+  public ExpressionCalculator(Iterable<AnalyticsExpression> expressions) {
+    this.expressions = expressions;
+  }
+
+  /**
+   * Calculate results for the list of {@link AnalyticsExpression}s.
+   * <p>
+   * NOTE: This method can, and is, called multiple times to generate different responses. 
+   * <br>
+   * The results are determined by which {@link ReductionDataCollection} is passed to the {@link ReductionCollectionManager#setData}
+   * method of the {@link ReductionCollectionManager} managing the reduction for the list of {@link AnalyticsExpression}s.
+   * 
+   * @return a {@link NamedList} containing the results
+   */
+  public Map<String,Object> getResults() {
+    Map<String,Object> exprVals = new HashMap<>();
+    expressions.forEach(expr -> {
+      Object obj = expr.toObject();
+      if (expr.exists()) {
+        exprVals.put(expr.getName(), obj);
+      }
+    });
+    return exprVals;
+  }
+
+  /**
+   * Calculate results for the list of {@link AnalyticsExpression}s and add them to the given response.
+   * <p>
+   * NOTE: This method can, and is, called multiple times to generate different responses. 
+   * <br>
+   * The results are determined by which {@link ReductionDataCollection} is passed to the {@link ReductionCollectionManager#setData}
+   * method of the {@link ReductionCollectionManager} managing the reduction for the list of {@link AnalyticsExpression}s.
+   * 
+   * @param response the response to add the results map to.
+   */
+  public void addResults(Map<String,Object> response) {
+    response.put(AnalyticsResponseHeadings.RESULTS, getResults());
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/MergingReductionCollectionManager.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/MergingReductionCollectionManager.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/MergingReductionCollectionManager.java
new file mode 100644
index 0000000..1402a76
--- /dev/null
+++ b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/MergingReductionCollectionManager.java
@@ -0,0 +1,46 @@
+/*
+ * 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.analytics.function;
+
+import org.apache.solr.analytics.function.field.AnalyticsField;
+import org.apache.solr.analytics.function.reduction.data.ReductionDataCollector;
+
+/**
+ * The {@link ReductionCollectionManager} used for distributed requests.
+ */
+public class MergingReductionCollectionManager extends ReductionCollectionManager {
+  
+  public MergingReductionCollectionManager() {
+    super();
+  }
+  
+  public MergingReductionCollectionManager(final ReductionDataCollector<?>[] reductionDataCollectors, final Iterable<AnalyticsField> fields) {
+    super(reductionDataCollectors, fields);
+  }
+  
+  @Override
+  protected ReductionCollectionManager createNewManager(final ReductionDataCollector<?>[] reductionDataCollectors, final Iterable<AnalyticsField> fields) {
+    return new MergingReductionCollectionManager(reductionDataCollectors,fields);
+  }
+  
+  @Override
+  public void setData(ReductionDataCollection dataCollection) {
+    for (int i = 0; i < reductionDataCollectors.length; i++) {
+      reductionDataCollectors[i].setMergedData(dataCollection.dataArr[i]);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/ReductionCollectionManager.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/ReductionCollectionManager.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/ReductionCollectionManager.java
new file mode 100644
index 0000000..b3a178c
--- /dev/null
+++ b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/ReductionCollectionManager.java
@@ -0,0 +1,320 @@
+/*
+ * 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.analytics.function;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.solr.analytics.function.field.AnalyticsField;
+import org.apache.solr.analytics.function.reduction.data.ReductionData;
+import org.apache.solr.analytics.function.reduction.data.ReductionDataCollector;
+import org.apache.solr.analytics.stream.reservation.ReductionDataReservation;
+import org.apache.solr.analytics.stream.reservation.read.ReductionDataReader;
+import org.apache.solr.analytics.stream.reservation.write.ReductionDataWriter;
+import org.apache.solr.analytics.value.AnalyticsValue;
+
+/**
+ * The manager of reduction collection.
+ * Contains a group of {@link ReductionDataCollector}s which will be updated together.
+ * <p>
+ * The manager assumes a non-distributed request. {@link MergingReductionCollectionManager} is used for distributed requests.
+ */
+public class ReductionCollectionManager {
+  protected final ReductionDataCollector<?>[] reductionDataCollectors;
+  private final List<ReductionDataReservation<?,?>> reservations;
+
+  private final List<ReductionDataReader<?>> readers;
+  private final List<ReductionDataWriter<?>> writers;
+  
+  private final Iterable<AnalyticsField> fields;
+  
+  public ReductionCollectionManager() {
+    this(new ReductionDataCollector<?>[0], new ArrayList<>(0));
+  }
+  
+  /**
+   * Create a Manager to oversee the given {@link ReductionDataCollector}s.
+   * 
+   * @param reductionDataCollectors array of collectors that are collecting over the same set of data
+   * @param fields all fields used by the given collectors
+   */
+  public ReductionCollectionManager(final ReductionDataCollector<?>[] reductionDataCollectors, final Iterable<AnalyticsField> fields) {
+    this.reductionDataCollectors = reductionDataCollectors;
+    Arrays.sort(reductionDataCollectors, (a,b) -> a.getExpressionStr().compareTo(b.getExpressionStr()));
+    
+    reservations = new LinkedList<>();
+    for (int i = 0; i < reductionDataCollectors.length; i++) {
+      reductionDataCollectors[i].submitReservations(reservation -> reservations.add(reservation));
+    }
+    
+    this.fields = fields;
+    
+    this.readers = new ArrayList<>();
+    this.writers = new ArrayList<>();
+  }
+  
+  /**
+   * Return whether or not the manager needs collection done, which is false if no collectors are 
+   * being managed and true if at least one is.
+   * 
+   * @return true if at least one collector is being managed
+   */
+  public boolean needsCollection() {
+    return reductionDataCollectors.length > 0;
+  }
+  
+  /**
+   * Merge this collection manager with others.
+   *
+   * @param reductionManagers the collection managers to merge with
+   * @return a collection manager that manages the union of data collectors from this class and the given managers
+   */
+  public ReductionCollectionManager merge(Iterable<ReductionCollectionManager> reductionManagers) {
+    HashMap<String,ReductionDataCollector<?>> mergedCollectors = new HashMap<>();
+    HashMap<String,AnalyticsField> mergedFields = new HashMap<>();
+    
+    for (ReductionDataCollector<?> collector : reductionDataCollectors) {
+      mergedCollectors.put(collector.getExpressionStr(), collector);
+    }
+    fields.forEach( field -> mergedFields.put(field.getExpressionStr(), field) );
+    
+    reductionManagers.forEach( manager -> {
+      for (ReductionDataCollector<?> collector : manager.reductionDataCollectors) {
+        mergedCollectors.put(collector.getExpressionStr(), collector);
+      }
+      manager.fields.forEach( field -> mergedFields.put(field.getExpressionStr(), field) );
+    });
+    ReductionDataCollector<?>[] collectors = new ReductionDataCollector<?>[mergedCollectors.size()];
+    mergedCollectors.values().toArray(collectors);
+    return createNewManager(collectors, mergedFields.values());
+  }
+  
+  /**
+   * Create an {@link ReductionCollectionManager} to manage the given collectors and fields.
+   * 
+   * @param reductionDataCollectors Reduction collectors
+   * @param fields fields used by the reductions
+   * @return a collection manager
+   */
+  protected ReductionCollectionManager createNewManager(final ReductionDataCollector<?>[] reductionDataCollectors, final Iterable<AnalyticsField> fields) {
+    return new ReductionCollectionManager(reductionDataCollectors,fields);
+  }
+  
+  /**
+   * Get the {@link AnalyticsField}s used in the managed expressions.
+   * 
+   * @return the fields used
+   */
+  public Iterable<AnalyticsField> getUsedFields() {
+    return fields;
+  }
+  
+  /**
+   * Set the context of the readers of the used {@link AnalyticsField}s.
+   * 
+   * @param context the reader context
+   * @throws IOException if an error occurs while setting the fields' context
+   */
+  public void doSetNextReader(LeafReaderContext context) throws IOException {
+    for (AnalyticsField field : fields) {
+      field.doSetNextReader(context);
+    }
+  }
+  
+  /**
+   * Collect the values from the used {@link AnalyticsField}s.
+   * 
+   * @param doc the document to collect values for
+   * @throws IOException if an error occurs during field collection
+   */
+  public void collect(int doc) throws IOException {
+    for (AnalyticsField field : fields) {
+      field.collect(doc);
+    }
+  }
+  
+  /**
+   * Add a {@link ReductionDataCollection} to target while collecting documents.
+   * This target is valid until the lasting targets are cleared.
+   * 
+   * @param target data collection to add document data too
+   */
+  public void addLastingCollectTarget(ReductionDataCollection target) {
+    for (int i = 0; i < reductionDataCollectors.length; i++) {
+      reductionDataCollectors[i].addLastingCollectTarget(target.dataArr[i]);
+    }
+  }
+  /**
+   * Clear lasting collection targets.
+   */
+  public void clearLastingCollectTargets() {
+    for (int i = 0; i < reductionDataCollectors.length; i++) {
+      reductionDataCollectors[i].clearLastingCollectTargets();
+    }
+  }
+  
+  /**
+   * Add a new {@link ReductionDataCollection} to target while collecting the next document.
+   * This target is only valid for the next {@link #apply()} call.
+   * 
+   * @return the new data collection being targeted
+   */
+  public ReductionDataCollection newDataCollectionTarget() {
+    ReductionDataCollection newCol = new ReductionDataCollection();
+    newCol.dataArr = new ReductionData[reductionDataCollectors.length];
+    for (int i = 0; i < reductionDataCollectors.length; i++) {
+      newCol.dataArr[i] = reductionDataCollectors[i].newDataTarget();
+    }
+    return newCol;
+  }
+  /**
+   * Add a {@link ReductionDataCollection} to target while collecting the next document.
+   * This target is only valid for the next {@link #apply()} call.
+   * 
+   * @param target data collection to add document data too
+   */
+  public void addCollectTarget(ReductionDataCollection target) {
+    for (int i = 0; i < reductionDataCollectors.length; i++) {
+      reductionDataCollectors[i].addCollectTarget(target.dataArr[i]);
+    }
+  }
+  
+  /**
+   * Apply the values of the collected fields through the expressions' logic to the managed data collectors.
+   * This is called after {@link #collect(int)} has been called and the collection targets have been added.
+   */
+  public void apply() {
+    for (int i = 0; i < reductionDataCollectors.length; i++) {
+      reductionDataCollectors[i].collectAndApply();;
+    }
+  }
+  
+  /**
+   * Finalize the reductions with the collected data stored in the parameter.
+   * Once the data is finalized, the {@link ReductionFunction}s that use these 
+   * {@link ReductionDataCollector}s act like regular {@link AnalyticsValue} classes that 
+   * can be accessed through their {@code get<value-type>} methods.
+   * 
+   * @param dataCollection the collection of reduction data to compute results for
+   */
+  public void setData(ReductionDataCollection dataCollection) {
+    for (int i = 0; i < reductionDataCollectors.length; i++) {
+      reductionDataCollectors[i].setData(dataCollection.dataArr[i]);
+    }
+  }
+  
+  /**
+   * Construct a new data collection holding data for all managed data collectors.
+   * 
+   * @return a new data collection
+   */
+  public ReductionDataCollection newDataCollection() {
+    ReductionDataCollection newCol = new ReductionDataCollection();
+    newCol.dataArr = new ReductionData[reductionDataCollectors.length];
+    for (int i = 0; i < reductionDataCollectors.length; i++) {
+      newCol.dataArr[i] = reductionDataCollectors[i].newData();
+    }
+    return newCol;
+  }
+  
+  /**
+   * Sets the stream of shard data to merge with.
+   * 
+   * @param input the stream of shard data
+   */
+  public void setShardInput(DataInput input) {
+    readers.clear();
+    reservations.forEach( resv -> readers.add(resv.createReadStream(input)));
+  }
+  /**
+   * Merge the data from the given shard input stream into the set IO data collectors.
+   * Should always be called after {@link #setShardInput(DataInput)} and either {@link #prepareReductionDataIO(ReductionDataCollection)}
+   * or {@link #newDataCollectionIO()} have been called.
+   * 
+   * @throws IOException if an error occurs while reading the shard data
+   */
+  public void mergeData() throws IOException {
+    for (ReductionDataReader<?> reader : readers) {
+      reader.read();
+    }
+  }
+  
+  /**
+   * Sets the stream to export shard data to.
+   * 
+   * @param output the stream of shard data
+   */
+  public void setShardOutput(DataOutput output) {
+    writers.clear();
+    reservations.forEach( resv -> writers.add(resv.createWriteStream(output)));
+  }
+  /**
+   * Export the data from the set IO data collectors to the given shard output stream.
+   * Should always be called after {@link #setShardOutput(DataOutput)} and {@link #prepareReductionDataIO(ReductionDataCollection)}.
+   * 
+   * @throws IOException if an error occurs while writing the shard data
+   */
+  public void exportData() throws IOException {
+    for (ReductionDataWriter<?> writer : writers) {
+      writer.write();
+    }
+  }
+  
+  /**
+   * Set the given data collection to be used for either merging or exporting
+   *  
+   * @param col collection to export from or merge to
+   */
+  public void prepareReductionDataIO(ReductionDataCollection col) {
+    for (int i = 0; i < reductionDataCollectors.length; i++) {
+      reductionDataCollectors[i].dataIO(col.dataArr[i]);
+    }
+  }
+  
+  /**
+   * Create a new {@link ReductionDataCollection} to merge to or export from.
+   * Mainly used for creating facet value collectors when merging shard data.
+   *  
+   * @return the new data collection created
+   */
+  public ReductionDataCollection newDataCollectionIO() {
+    ReductionDataCollection newCol = new ReductionDataCollection();
+    newCol.dataArr = new ReductionData[reductionDataCollectors.length];
+    for (int i = 0; i < reductionDataCollectors.length; i++) {
+      newCol.dataArr[i] = reductionDataCollectors[i].newDataIO();
+    }
+    return newCol;
+  }
+  
+  /**
+   * Holds the collection of {@link ReductionData} that will be updated together.
+   * 
+   * For example each grouping will have a separate {@link ReductionDataCollection}, and
+   * ungrouped expressions will have their own as well.
+   */
+  public static class ReductionDataCollection{
+    public ReductionData[] dataArr;
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/ReductionFunction.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/ReductionFunction.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/ReductionFunction.java
new file mode 100644
index 0000000..fea01c2
--- /dev/null
+++ b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/ReductionFunction.java
@@ -0,0 +1,37 @@
+/*
+ * 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.analytics.function;
+
+import java.util.function.UnaryOperator;
+
+import org.apache.solr.analytics.function.reduction.data.ReductionDataCollector;
+import org.apache.solr.analytics.value.AnalyticsValue;
+
+/**
+ * A function that reduces the values of a mapping expression, field or constant.
+ */
+public interface ReductionFunction extends AnalyticsValue {
+
+  /**
+   * Syncs the data collectors with shared versions across the entire Analytics Request
+   * so that as little data as possible is sent across shards.
+   * 
+   * @param sync a function that takes in a {@link ReductionDataCollector} and returns a shared version
+   */
+  void synchronizeDataCollectors(UnaryOperator<ReductionDataCollector<?>> sync);
+}
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/field/AnalyticsField.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/field/AnalyticsField.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/field/AnalyticsField.java
new file mode 100644
index 0000000..e0bbb4b
--- /dev/null
+++ b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/field/AnalyticsField.java
@@ -0,0 +1,74 @@
+/*
+ * 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.analytics.function.field;
+
+import java.io.IOException;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.solr.analytics.value.AnalyticsValueStream;
+
+/**
+ * An analytics wrapper for Solr Fields. 
+ * 
+ * Currently only fields with Doc Values enabled can be used in Analytics queries.
+ */
+public abstract class AnalyticsField implements AnalyticsValueStream {
+  protected static final int initialArrayLength = 20;
+  
+  protected final String fieldName;
+  
+  protected AnalyticsField(String fieldName) {
+    this.fieldName = fieldName;
+  }
+  
+  @Override
+  public String getExpressionStr() {
+    return fieldName;
+  }
+
+  @Override
+  public String getName() {
+    return fieldName;
+  }
+  
+  @Override
+  public ExpressionType getExpressionType() {
+    return ExpressionType.FIELD;
+  }
+  
+  @Override
+  public AnalyticsValueStream convertToConstant() {
+    return this;
+  }
+  
+  /**
+   * Set the segment reader context
+   * 
+   * @param context segment context
+   * @throws IOException if an error occurs while loading the leaf reader
+   */
+  public abstract void doSetNextReader(LeafReaderContext context) throws IOException;
+  
+  /**
+   * Collect the value(s) of the wrapped field for the given document, and store the value.
+   * 
+   * @param doc ID of the document to collect
+   * @throws IOException if an error occurs while reading the document.
+   */
+  public abstract void collect(int doc) throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/field/BooleanField.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/field/BooleanField.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/field/BooleanField.java
new file mode 100644
index 0000000..650bc36
--- /dev/null
+++ b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/field/BooleanField.java
@@ -0,0 +1,111 @@
+/*
+ * 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.analytics.function.field;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.util.BytesRef;
+import org.apache.solr.analytics.facet.compare.ExpressionComparator;
+import org.apache.solr.analytics.util.function.BooleanConsumer;
+import org.apache.solr.analytics.value.BooleanValue.CastingBooleanValue;
+import org.apache.solr.schema.BoolField;
+
+/**
+ * An analytics wrapper for a single-valued {@link BoolField} with DocValues enabled.
+ */
+public class BooleanField extends AnalyticsField implements CastingBooleanValue {
+  private SortedDocValues docValues;
+  boolean value;
+  boolean exists;
+  int trueOrd;
+
+  public BooleanField(String fieldName) {
+    super(fieldName);
+  }
+  
+  @Override
+  public void doSetNextReader(LeafReaderContext context) throws IOException {
+    docValues = DocValues.getSorted(context.reader(), fieldName);
+
+    // figure out what ord maps to true
+    int numOrds = docValues.getValueCount();
+    // if no values in the segment, default trueOrd to something other then -1 (missing)
+    int trueOrd = -2;
+    for (int i=0; i<numOrds; i++) {
+      final BytesRef br = docValues.lookupOrd(i);
+      if (br.length==1 && br.bytes[br.offset]=='T') {
+        trueOrd = i;
+        break;
+      }
+    }
+
+    this.trueOrd = trueOrd;
+  }
+
+  @Override
+  public void collect(int doc) throws IOException {
+    exists = docValues.advanceExact(doc);
+    if (exists) {
+      value = trueOrd ==  docValues.ordValue();
+    }
+  }
+
+  @Override
+  public boolean getBoolean() {
+    return value;
+  }
+  @Override
+  public String getString() {
+    return exists ? Boolean.toString(value) : null;
+  }
+  @Override
+  public Object getObject() {
+    return exists ? value : null;
+  }
+  @Override
+  public boolean exists() {
+    return exists;
+  }
+  
+  @Override
+  public void streamBooleans(BooleanConsumer cons) {
+    if (exists) {
+      cons.accept(value);
+    }
+  }
+  @Override
+  public void streamStrings(Consumer<String> cons) {
+    if (exists) {
+      cons.accept(Boolean.toString(value));
+    }
+  }
+  @Override
+  public void streamObjects(Consumer<Object> cons) {
+    if (exists) {
+      cons.accept(value);
+    }
+  }
+
+  @Override
+  public ExpressionComparator<Boolean> getObjectComparator(String expression) {
+    return new ExpressionComparator<>(expression);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/field/BooleanMultiField.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/field/BooleanMultiField.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/field/BooleanMultiField.java
new file mode 100644
index 0000000..e4eecd3
--- /dev/null
+++ b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/field/BooleanMultiField.java
@@ -0,0 +1,101 @@
+/*
+ * 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.analytics.function.field;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.util.BytesRef;
+import org.apache.solr.analytics.util.function.BooleanConsumer;
+import org.apache.solr.analytics.value.BooleanValueStream.CastingBooleanValueStream;
+import org.apache.solr.schema.BoolField;
+
+
+/**
+ * An analytics wrapper for a multi-valued {@link BoolField} with DocValues enabled.
+ */
+public class BooleanMultiField extends AnalyticsField implements CastingBooleanValueStream {
+  private SortedSetDocValues docValues;
+  private int count;
+  private boolean[] values;
+  
+  private int trueOrd;
+
+  public BooleanMultiField(String fieldName) {
+    super(fieldName);
+    count = 0;
+    values = new boolean[initialArrayLength];
+  }
+  
+  @Override
+  public void doSetNextReader(LeafReaderContext context) throws IOException {
+    docValues = DocValues.getSortedSet(context.reader(), fieldName);
+
+    // figure out what ord maps to true
+    long numOrds = docValues.getValueCount();
+    // if no values in the segment, default trueOrd to something other then -1 (missing)
+    int trueOrd = -2;
+    for (int i=0; i<numOrds; i++) {
+      final BytesRef br = docValues.lookupOrd(i);
+      if (br.length==1 && br.bytes[br.offset]=='T') {
+        trueOrd = i;
+        break;
+      }
+    }
+
+    this.trueOrd = trueOrd;
+  }
+  @Override
+  public void collect(int doc) throws IOException {
+    count = 0;
+    if (docValues.advanceExact(doc)) {
+      int term;
+      while ((term = (int)docValues.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
+        if (count == values.length) {
+          resizeValues();
+        }
+        values[count++] = term == trueOrd;
+      }
+    }
+  }
+  
+  private void resizeValues() {
+    boolean[] newValues = new boolean[values.length*2];
+    for (int i = 0; i < count; ++i) {
+      newValues[i] = values[i];
+    }
+    values = newValues;
+  }
+  
+  @Override
+  public void streamBooleans(BooleanConsumer cons) {
+    for (int i = 0; i < count; ++i) {
+      cons.accept(values[i]);
+    }
+  }
+  @Override
+  public void streamStrings(Consumer<String> cons) {
+    streamBooleans(value -> cons.accept(Boolean.toString(value)));
+  }
+  @Override
+  public void streamObjects(Consumer<Object> cons) {
+    streamBooleans(value -> cons.accept(value));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/field/DateField.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/field/DateField.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/field/DateField.java
new file mode 100644
index 0000000..88e71bc
--- /dev/null
+++ b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/field/DateField.java
@@ -0,0 +1,108 @@
+/*
+ * 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.analytics.function.field;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.util.Date;
+import java.util.function.Consumer;
+import java.util.function.LongConsumer;
+
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.solr.analytics.facet.compare.ExpressionComparator;
+import org.apache.solr.analytics.value.DateValue.CastingDateValue;
+import org.apache.solr.schema.DatePointField;
+import org.apache.solr.schema.TrieDateField;
+
+/**
+ * An analytics wrapper for a single-valued {@link TrieDateField} or {@link DatePointField} with DocValues enabled.
+ */
+public class DateField extends AnalyticsField implements CastingDateValue {
+  private NumericDocValues docValues; 
+  private long value;
+  private boolean exists;
+
+  public DateField(String fieldName) {
+    super(fieldName);
+  }
+
+  @Override
+  public void doSetNextReader(LeafReaderContext context) throws IOException {
+    docValues = DocValues.getNumeric(context.reader(), fieldName);
+  }
+
+  @Override
+  public void collect(int doc) throws IOException {
+    exists = docValues.advanceExact(doc);
+    if (exists) {
+      value = docValues.longValue();
+    }
+  }
+
+  @Override
+  public long getLong() {
+    return value;
+  }
+  @Override
+  public Date getDate() {
+    return exists ? new Date(value) : null;
+  }
+  @Override
+  public String getString() {
+    return exists ? Instant.ofEpochMilli(value).toString() : null;
+  }
+  @Override
+  public Object getObject() {
+    return exists ? value : null;
+  }
+  @Override
+  public boolean exists() {
+    return exists;
+  }
+  
+  @Override
+  public void streamLongs(LongConsumer cons) {
+    if (exists) {
+      cons.accept(value);
+    }
+  }
+  @Override
+  public void streamDates(Consumer<Date> cons) {
+    if (exists) {
+      cons.accept(new Date(value));
+    }
+  }
+  @Override
+  public void streamStrings(Consumer<String> cons) {
+    if (exists) {
+      cons.accept(Instant.ofEpochMilli(value).toString());
+    }
+  }
+  @Override
+  public void streamObjects(Consumer<Object> cons) {
+    if (exists) {
+      cons.accept(new Date(value));
+    }
+  }
+
+  @Override
+  public ExpressionComparator<Date> getObjectComparator(String expression) {
+    return new ExpressionComparator<>(expression);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/field/DateMultiPointField.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/field/DateMultiPointField.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/field/DateMultiPointField.java
new file mode 100644
index 0000000..a1560ef
--- /dev/null
+++ b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/field/DateMultiPointField.java
@@ -0,0 +1,47 @@
+/*
+ * 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.analytics.function.field;
+
+import java.time.Instant;
+import java.util.Date;
+import java.util.function.Consumer;
+
+import org.apache.solr.analytics.value.DateValueStream.CastingDateValueStream;
+import org.apache.solr.schema.DatePointField;
+
+/**
+ * An analytics wrapper for a multi-valued {@link DatePointField} with DocValues enabled.
+ */
+public class DateMultiPointField extends LongMultiPointField implements CastingDateValueStream {
+
+  public DateMultiPointField(String fieldName) {
+    super(fieldName);
+  }
+  
+  @Override
+  public void streamDates(Consumer<Date> cons) {
+    streamLongs(value -> cons.accept(new Date(value)));
+  }
+  @Override
+  public void streamStrings(Consumer<String> cons) {
+    streamLongs(value -> cons.accept(Instant.ofEpochMilli(value).toString()));
+  }
+  @Override
+  public void streamObjects(Consumer<Object> cons) {
+    streamLongs(value -> cons.accept(new Date(value)));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0366b94/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/field/DateMultiTrieField.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/field/DateMultiTrieField.java b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/field/DateMultiTrieField.java
new file mode 100644
index 0000000..cdc77f4
--- /dev/null
+++ b/solr/contrib/analytics/src/main/java/org/apache/solr/analytics/function/field/DateMultiTrieField.java
@@ -0,0 +1,49 @@
+/*
+ * 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.analytics.function.field;
+
+import java.time.Instant;
+import java.util.Date;
+import java.util.function.Consumer;
+
+import org.apache.solr.analytics.value.DateValueStream.CastingDateValueStream;
+import org.apache.solr.schema.TrieDateField;
+
+/**
+ * An analytics wrapper for a multi-valued {@link TrieDateField} with DocValues enabled.
+ * @deprecated Trie fields are deprecated as of Solr 7.0
+ */
+@Deprecated
+public class DateMultiTrieField extends LongMultiTrieField implements CastingDateValueStream {
+
+  public DateMultiTrieField(String fieldName) {
+    super(fieldName);
+  }
+  
+  @Override
+  public void streamDates(Consumer<Date> cons) {
+    streamLongs(value -> cons.accept(new Date(value)));
+  }
+  @Override
+  public void streamStrings(Consumer<String> cons) {
+    streamLongs(value -> cons.accept(Instant.ofEpochMilli(value).toString()));
+  }
+  @Override
+  public void streamObjects(Consumer<Object> cons) {
+    streamLongs(value -> cons.accept(new Date(value)));
+  }
+}