You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ab...@apache.org on 2017/07/03 11:57:51 UTC

[11/58] [abbrv] lucene-solr:jira/solr-10879: SOLR-10123: Upgraded the Analytics Component to version 2.0

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/UniqueFunction.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/UniqueFunction.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/UniqueFunction.java
new file mode 100644
index 0000000..f62f7d9
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/UniqueFunction.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.reduction;
+
+import java.util.function.UnaryOperator;
+
+import org.apache.solr.analytics.ExpressionFactory.CreatorFunction;
+import org.apache.solr.analytics.function.ReductionFunction;
+import org.apache.solr.analytics.function.reduction.data.ReductionDataCollector;
+import org.apache.solr.analytics.function.reduction.data.UniqueCollector;
+import org.apache.solr.analytics.function.reduction.data.UniqueCollector.UniqueDoubleCollector;
+import org.apache.solr.analytics.function.reduction.data.UniqueCollector.UniqueFloatCollector;
+import org.apache.solr.analytics.function.reduction.data.UniqueCollector.UniqueIntCollector;
+import org.apache.solr.analytics.function.reduction.data.UniqueCollector.UniqueLongCollector;
+import org.apache.solr.analytics.function.reduction.data.UniqueCollector.UniqueStringCollector;
+import org.apache.solr.analytics.value.AnalyticsValueStream;
+import org.apache.solr.analytics.value.DoubleValueStream;
+import org.apache.solr.analytics.value.FloatValueStream;
+import org.apache.solr.analytics.value.IntValueStream;
+import org.apache.solr.analytics.value.LongValueStream;
+import org.apache.solr.analytics.value.StringValueStream;
+import org.apache.solr.analytics.value.LongValue.AbstractLongValue;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+
+/**
+ * A reduction function which returns the number of unique values of the given expression.
+ */
+public class UniqueFunction extends AbstractLongValue implements ReductionFunction {
+  private UniqueCollector<?> collector;
+  public static final String name = "unique";
+  private final String exprStr;
+  public static final CreatorFunction creatorFunction = (params -> {
+    if (params.length != 1) {
+      throw new SolrException(ErrorCode.BAD_REQUEST,"The "+name+" function requires 1 paramater, " + params.length + " found.");
+    }
+    AnalyticsValueStream param = params[0];
+    UniqueCollector<?> collector;
+    if (param instanceof IntValueStream) {
+      collector = new UniqueIntCollector((IntValueStream)param);
+    } else if (param instanceof LongValueStream) {
+      collector = new UniqueLongCollector((LongValueStream)param);
+    } else if (param instanceof FloatValueStream) {
+      collector = new UniqueFloatCollector((FloatValueStream)param);
+    } else if (param instanceof DoubleValueStream) {
+      collector = new UniqueDoubleCollector((DoubleValueStream)param);
+    } else if (param instanceof StringValueStream) {
+      collector = new UniqueStringCollector((StringValueStream)param);
+    } else {
+      throw new SolrException(ErrorCode.BAD_REQUEST,"The "+name+" function requires a comparable parameter.");
+    }
+    return new UniqueFunction(param, collector);
+  });
+  
+  public UniqueFunction(AnalyticsValueStream param, UniqueCollector<?> collector) {
+    this.collector = collector;
+    this.exprStr = AnalyticsValueStream.createExpressionString(name,param);
+  }
+
+  @Override
+  public long getLong() {
+    return collector.count();
+  }
+  @Override
+  public boolean exists() {
+    return true;
+  }
+
+  @Override
+  public void synchronizeDataCollectors(UnaryOperator<ReductionDataCollector<?>> sync) {
+    collector = (UniqueCollector<?>)sync.apply(collector);
+  }
+
+  @Override
+  public String getName() {
+    return name;
+  }
+  @Override
+  public String getExpressionStr() {
+    return exprStr;
+  }
+
+  @Override
+  public ExpressionType getExpressionType() {
+    return ExpressionType.REDUCTION;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/data/CountCollector.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/data/CountCollector.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/data/CountCollector.java
new file mode 100644
index 0000000..135b587
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/data/CountCollector.java
@@ -0,0 +1,188 @@
+/*
+ * 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.reduction.data;
+
+import java.util.function.Consumer;
+
+import org.apache.solr.analytics.function.reduction.data.ReductionData;
+import org.apache.solr.analytics.stream.reservation.LongReservation;
+import org.apache.solr.analytics.stream.reservation.ReductionDataReservation;
+import org.apache.solr.analytics.value.AnalyticsValueStream;
+
+public abstract class CountCollector extends ReductionDataCollector<CountCollector.CountData> {
+  public static final String name = "count";
+  private final String exprStr;
+  
+  public CountCollector(String exprStr) {
+    this.exprStr = exprStr;
+  }
+
+  private long count;
+  private long docCount;
+
+  /**
+   * The number of Solr Documents for which the given analytics expression exists.
+   * 
+   * @return the count
+   */
+  public long count() {
+    return count;
+  }
+  /**
+   * The number of Solr Documents used in this reduction.
+   * 
+   * @return the number of documents
+   */
+  public long docCount() {
+    return docCount;
+  }
+
+  @Override
+  public CountData newData() {
+    CountData data = new CountData();
+    data.count = 0;
+    data.missing = 0;
+    data.docCount = 0;
+    return data;
+  }
+
+  @Override
+  public void submitReservations(Consumer<ReductionDataReservation<?,?>> consumer) {
+    // Count
+    consumer.accept(new LongReservation(
+        value -> ioData.count += value,
+        () -> ioData.count
+      ));
+    // DocCount
+    consumer.accept(new LongReservation(
+        value -> ioData.docCount += value,
+        () -> ioData.docCount
+      ));
+  }
+
+  @Override
+  public void setMergedData(ReductionData data) {
+    count = ((CountData)data).count;
+    docCount = ((CountData)data).docCount;
+  }
+
+  @Override
+  public void setData(ReductionData data) {
+    count = ((CountData)data).count;
+    docCount = ((CountData)data).docCount;
+  }
+
+  @Override
+  public String getName() {
+    return name;
+  }
+  @Override
+  public String getExpressionStr() {
+    return exprStr;
+  }
+  
+  public static class CountData extends ReductionData {
+    long count;
+    long missing;
+    long docCount;
+  }
+
+  /**
+   * Represents a {@code count(expr)} expression. This collects 3 values:
+   * 
+   * docCount - The number of Solr Documents for which the wrapped expression exists.
+   * count - The number of values which wrapped expression contains.
+   * missing - The number of Solr Documents for which the wrapped expression does not exist.
+   */
+  public static class ExpressionCountCollector extends CountCollector {
+    private final AnalyticsValueStream param;
+    
+    public ExpressionCountCollector(AnalyticsValueStream param) {
+      super(AnalyticsValueStream.createExpressionString(name, param));
+      this.param = param;
+    }
+    
+    private long missing;
+    
+    /**
+     * The number of Solr Documents for which the given analytics expression does not exist.
+     * 
+     * @return the number of missing values
+     */
+    public long missing() {
+      return missing;
+    }
+
+    @Override
+    public void setMergedData(ReductionData data) {
+      super.setMergedData(data);
+      missing = ((CountData)data).missing;
+    }
+
+    @Override
+    public void setData(ReductionData data) {
+      super.setData(data);
+      missing = ((CountData)data).missing;
+    }
+    
+    long tempCount;
+    int tempMissing;
+    int tempDocCount;
+    @Override
+    public void collect() {
+      tempCount = 0;
+      param.streamObjects( obj -> {
+        ++tempCount;
+      });
+      tempMissing = tempCount == 0 ? 1 : 0;
+      tempDocCount = tempCount > 0 ? 1 : 0;
+    }
+    
+    @Override
+    protected void apply(CountData data) {
+      data.count += tempCount;
+      data.missing += tempMissing;
+      data.docCount += tempDocCount;
+    }
+
+    @Override
+    public void submitReservations(Consumer<ReductionDataReservation<?,?>> consumer) {
+      super.submitReservations(consumer);
+      // Missing
+      consumer.accept(new LongReservation(
+          value -> ioData.missing += value,
+          () -> ioData.missing
+        ));
+    }
+  }
+  
+  /**
+   * Represents a {@code count()} expression. This collects the number of Solr Documents used in a result set.
+   */
+  public static class TotalCountCollector extends CountCollector {
+    
+    public TotalCountCollector() {
+      super(AnalyticsValueStream.createExpressionString(name));
+    }
+    
+    @Override
+    protected void apply(CountData data) {
+      data.count += 1;
+      data.docCount += 1;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/data/MaxCollector.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/data/MaxCollector.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/data/MaxCollector.java
new file mode 100644
index 0000000..bf49907
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/data/MaxCollector.java
@@ -0,0 +1,476 @@
+/*
+ * 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.reduction.data;
+
+import java.util.function.Consumer;
+
+import org.apache.solr.analytics.function.reduction.data.ReductionData;
+import org.apache.solr.analytics.stream.reservation.DoubleCheckedReservation;
+import org.apache.solr.analytics.stream.reservation.FloatCheckedReservation;
+import org.apache.solr.analytics.stream.reservation.IntCheckedReservation;
+import org.apache.solr.analytics.stream.reservation.LongCheckedReservation;
+import org.apache.solr.analytics.stream.reservation.ReductionDataReservation;
+import org.apache.solr.analytics.stream.reservation.StringCheckedReservation;
+import org.apache.solr.analytics.value.AnalyticsValueStream;
+import org.apache.solr.analytics.value.DoubleValueStream;
+import org.apache.solr.analytics.value.FloatValueStream;
+import org.apache.solr.analytics.value.IntValueStream;
+import org.apache.solr.analytics.value.LongValueStream;
+import org.apache.solr.analytics.value.StringValueStream;
+
+/**
+ * Collector of max values.
+ * <p>
+ * Supported types are:
+ * <ul>
+ * <li>Int
+ * <li>Long
+ * <li>Float
+ * <li>Double
+ * <li>Date (through longs)
+ * <li>String
+ * </ul>
+ *
+ * @param <T> The type of data being processed.
+ */
+public abstract class MaxCollector<T extends ReductionData> extends ReductionDataCollector<T> {
+  public static final String name = "max";
+  private final String exprStr;
+  
+  protected MaxCollector(AnalyticsValueStream param) {
+    this.exprStr = AnalyticsValueStream.createExpressionString(name,param);
+  }
+  
+  private boolean exists;
+  
+  /**
+   * Returns true if any of the values being reduce exist, and false if none of them do.
+   * 
+   * @return whether a max value exists
+   */
+  public boolean exists() {
+    return exists;
+  }
+
+  @Override
+  public void setMergedData(ReductionData data) {
+    exists = data.exists;
+  }
+
+  @Override
+  public void setData(ReductionData data) {
+    exists = data.exists;
+  }
+
+  @Override
+  public String getName() {
+    return name;
+  }
+  @Override
+  public String getExpressionStr() {
+    return exprStr;
+  }
+
+  public static class IntMaxCollector extends MaxCollector<IntMaxCollector.MaxData> {
+    private IntValueStream param;
+    
+    public IntMaxCollector(IntValueStream param) {
+      super(param);
+      this.param = param;
+    }
+
+    @Override
+    public MaxData newData() {
+      MaxData data = new MaxData();
+      data.exists = false;
+      return data;
+    }
+
+    int max;
+    
+    /**
+     * Returns the max value of the set data.
+     * 
+     * @return the max
+     */
+    public int max() {
+      return max;
+    }
+
+    int tempMax;
+    boolean tempExists;
+    @Override
+    public void collect() {
+      tempExists = false;
+      param.streamInts( val -> {
+        if (!tempExists || val > tempMax) {
+          tempMax = val;
+          tempExists = true;
+        }
+      });
+    }
+    @Override
+    protected void apply(MaxData data) {
+      if (tempExists && (!data.exists || tempMax > data.val)) {
+        data.val = tempMax;
+        data.exists = true;
+      }
+    }
+    
+    @Override
+    public void submitReservations(Consumer<ReductionDataReservation<?,?>> consumer) {
+      consumer.accept(new IntCheckedReservation(
+          value -> {
+            if (!ioData.exists || value > ioData.val) {
+              ioData.val = value;
+              ioData.exists = true;
+            }
+          },
+          ()-> ioData.val,
+          ()-> ioData.exists
+        ));
+    }
+
+    @Override
+    public void setMergedData(ReductionData data) {
+      super.setData(data);
+      max = ((MaxData)data).val;
+    }
+
+    @Override
+    public void setData(ReductionData data) {
+      super.setData(data);
+      max = ((MaxData)data).val;
+    }
+    
+    public static class MaxData extends ReductionData {
+      int val;
+    }
+  }
+  
+
+
+  public static class LongMaxCollector extends MaxCollector<LongMaxCollector.MaxData> {
+    private LongValueStream param;
+    
+    public LongMaxCollector(LongValueStream param) {
+      super(param);
+      this.param = param;
+    }
+
+    @Override
+    public MaxData newData() {
+      MaxData data = new MaxData();
+      data.exists = false;
+      return data;
+    }
+
+    long max;
+    
+    /**
+     * Returns the max value of the set data.
+     * 
+     * @return the max
+     */
+    public long max() {
+      return max;
+    }
+
+    long tempMax;
+    boolean tempExists;
+    @Override
+    public void collect() {
+      tempExists = false;
+      param.streamLongs( val -> {
+        if (!tempExists || val > tempMax) {
+          tempMax = val;
+          tempExists = true;
+        }
+      });
+    }
+    @Override
+    protected void apply(MaxData data) {
+      if (tempExists && (!data.exists || tempMax > data.val)) {
+        data.val = tempMax;
+        data.exists = true;
+      }
+    }
+    
+    @Override
+    public void submitReservations(Consumer<ReductionDataReservation<?,?>> consumer) {
+      consumer.accept(new LongCheckedReservation(
+          value -> {
+            if (!ioData.exists || value > ioData.val) {
+              ioData.val = value;
+              ioData.exists = true;
+            }
+          },
+          ()-> ioData.val,
+          ()-> ioData.exists
+        ));
+    }
+
+    @Override
+    public void setMergedData(ReductionData data) {
+      super.setData(data);
+      max = ((MaxData)data).val;
+    }
+
+    @Override
+    public void setData(ReductionData data) {
+      super.setData(data);
+      max = ((MaxData)data).val;
+    }
+    
+    public static class MaxData extends ReductionData {
+      long val;
+    }
+  }
+
+  public static class FloatMaxCollector extends MaxCollector<FloatMaxCollector.MaxData> {
+    private FloatValueStream param;
+    
+    public FloatMaxCollector(FloatValueStream param) {
+      super(param);
+      this.param = param;
+    }
+
+    @Override
+    public MaxData newData() {
+      MaxData data = new MaxData();
+      data.exists = false;
+      return data;
+    }
+
+    float max;
+    
+    /**
+     * Returns the max value of the set data.
+     * 
+     * @return the max
+     */
+    public float max() {
+      return max;
+    }
+
+    float tempMax;
+    boolean tempExists;
+    @Override
+    public void collect() {
+      tempExists = false;
+      param.streamFloats( val -> {
+        if (!tempExists || val > tempMax) {
+          tempMax = val;
+          tempExists = true;
+        }
+      });
+    }
+    @Override
+    protected void apply(MaxData data) {
+      if (tempExists && (!data.exists || tempMax > data.val)) {
+        data.val = tempMax;
+        data.exists = true;
+      }
+    }
+    
+    @Override
+    public void submitReservations(Consumer<ReductionDataReservation<?,?>> consumer) {
+      consumer.accept(new FloatCheckedReservation(
+          value -> {
+            if (!ioData.exists || value > ioData.val) {
+              ioData.val = value;
+              ioData.exists = true;
+            }
+          },
+          ()-> ioData.val,
+          ()-> ioData.exists
+        ));
+    }
+
+    @Override
+    public void setMergedData(ReductionData data) {
+      super.setData(data);
+      max = ((MaxData)data).val;
+    }
+
+    @Override
+    public void setData(ReductionData data) {
+      super.setData(data);
+      max = ((MaxData)data).val;
+    }
+    
+    public static class MaxData extends ReductionData {
+      float val;
+    }
+  }
+
+  public static class DoubleMaxCollector extends MaxCollector<DoubleMaxCollector.MaxData> {
+    private DoubleValueStream param;
+    
+    public DoubleMaxCollector(DoubleValueStream param) {
+      super(param);
+      this.param = param;
+    }
+
+    @Override
+    public MaxData newData() {
+      MaxData data = new MaxData();
+      data.exists = false;
+      return data;
+    }
+
+    double max;
+    
+    /**
+     * Returns the max value of the set data.
+     * 
+     * @return the max
+     */
+    public double max() {
+      return max;
+    }
+
+    double tempMax;
+    boolean tempExists;
+    @Override
+    public void collect() {
+      tempExists = false;
+      param.streamDoubles( val -> {
+        if (!tempExists || val > tempMax) {
+          tempMax = val;
+          tempExists = true;
+        }
+      });
+    }
+    @Override
+    protected void apply(MaxData data) {
+      if (tempExists && (!data.exists || tempMax > data.val)) {
+        data.val = tempMax;
+        data.exists = true;
+      }
+    }
+    
+    @Override
+    public void submitReservations(Consumer<ReductionDataReservation<?,?>> consumer) {
+      consumer.accept(new DoubleCheckedReservation(
+          value -> {
+            if (!ioData.exists || value > ioData.val) {
+              ioData.val = value;
+              ioData.exists = true;
+            }
+          },
+          ()-> ioData.val,
+          ()-> ioData.exists
+        ));
+    }
+
+    @Override
+    public void setMergedData(ReductionData data) {
+      super.setData(data);
+      max = ((MaxData)data).val;
+    }
+
+    @Override
+    public void setData(ReductionData data) {
+      super.setData(data);
+      max = ((MaxData)data).val;
+    }
+    
+    public static class MaxData extends ReductionData {
+      double val;
+    }
+  }
+  
+
+
+  public static class StringMaxCollector extends MaxCollector<StringMaxCollector.MaxData> {
+    private StringValueStream param;
+    
+    public StringMaxCollector(StringValueStream param) {
+      super(param);
+      this.param = param;
+    }
+
+    @Override
+    public MaxData newData() {
+      MaxData data = new MaxData();
+      data.exists = false;
+      return data;
+    }
+
+    String max;
+    
+    /**
+     * Returns the max value of the set data.
+     * 
+     * @return the max
+     */
+    public String max() {
+      return max;
+    }
+
+    String tempMax;
+    boolean tempExists;
+    @Override
+    public void collect() {
+      tempExists = false;
+      param.streamStrings( val -> {
+        if (!tempExists || val.compareTo(tempMax) > 0) {
+          tempMax = val;
+          tempExists = true;
+        }
+      });
+    }
+    @Override
+    protected void apply(MaxData data) {
+      if (tempExists && (!data.exists || tempMax.compareTo(data.val) > 0)) {
+        data.val = tempMax;
+        data.exists = true;
+      }
+    }
+    
+    @Override
+    public void submitReservations(Consumer<ReductionDataReservation<?,?>> consumer) {
+      consumer.accept(new StringCheckedReservation(
+          value -> {
+            if (!ioData.exists || value.compareTo(ioData.val) > 0) {
+              ioData.val = value;
+              ioData.exists = true;
+            }
+          },
+          ()-> ioData.val,
+          ()-> ioData.exists
+        ));
+    }
+
+    @Override
+    public void setMergedData(ReductionData data) {
+      super.setData(data);
+      max = ((MaxData)data).val;
+    }
+
+    @Override
+    public void setData(ReductionData data) {
+      super.setData(data);
+      max = ((MaxData)data).val;
+    }
+    
+    public static class MaxData extends ReductionData {
+      String val;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/data/MinCollector.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/data/MinCollector.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/data/MinCollector.java
new file mode 100644
index 0000000..3a33660
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/data/MinCollector.java
@@ -0,0 +1,476 @@
+/*
+ * 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.reduction.data;
+
+import java.util.function.Consumer;
+
+import org.apache.solr.analytics.function.reduction.data.ReductionData;
+import org.apache.solr.analytics.stream.reservation.DoubleCheckedReservation;
+import org.apache.solr.analytics.stream.reservation.FloatCheckedReservation;
+import org.apache.solr.analytics.stream.reservation.IntCheckedReservation;
+import org.apache.solr.analytics.stream.reservation.LongCheckedReservation;
+import org.apache.solr.analytics.stream.reservation.ReductionDataReservation;
+import org.apache.solr.analytics.stream.reservation.StringCheckedReservation;
+import org.apache.solr.analytics.value.AnalyticsValueStream;
+import org.apache.solr.analytics.value.DoubleValueStream;
+import org.apache.solr.analytics.value.FloatValueStream;
+import org.apache.solr.analytics.value.IntValueStream;
+import org.apache.solr.analytics.value.LongValueStream;
+import org.apache.solr.analytics.value.StringValueStream;
+
+/**
+ * Collector of min values.
+ * <p>
+ * Supported types are:
+ * <ul>
+ * <li>Int
+ * <li>Long
+ * <li>Float
+ * <li>Double
+ * <li>Date (through longs)
+ * <li>String
+ * </ul>
+ *
+ * @param <T> The type of data being processed.
+ */
+public abstract class MinCollector<T extends ReductionData> extends ReductionDataCollector<T> {
+  public static final String name = "min";
+  private final String exprStr;
+  
+  protected MinCollector(AnalyticsValueStream param) {
+    this.exprStr = AnalyticsValueStream.createExpressionString(name,param);
+  }
+  
+  private boolean exists;
+  
+  /**
+   * Returns true if any of the values being reduce exist, and false if none of them do.
+   * 
+   * @return whether a min value exists
+   */
+  public boolean exists() {
+    return exists;
+  }
+
+  @Override
+  public void setMergedData(ReductionData data) {
+    exists = data.exists;
+  }
+
+  @Override
+  public void setData(ReductionData data) {
+    exists = data.exists;
+  }
+
+  @Override
+  public String getName() {
+    return name;
+  }
+  @Override
+  public String getExpressionStr() {
+    return exprStr;
+  }
+
+  public static class IntMinCollector extends MinCollector<IntMinCollector.MinData> {
+    private IntValueStream param;
+    
+    public IntMinCollector(IntValueStream param) {
+      super(param);
+      this.param = param;
+    }
+
+    @Override
+    public MinData newData() {
+      MinData data = new MinData();
+      data.exists = false;
+      return data;
+    }
+
+    int min;
+    
+    /**
+     * Returns the min value of the set data.
+     * 
+     * @return the min
+     */
+    public int min() {
+      return min;
+    }
+
+    int tempMin;
+    boolean tempExists;
+    @Override
+    public void collect() {
+      tempExists = false;
+      param.streamInts( val -> {
+        if (!tempExists || val < tempMin) {
+          tempMin = val;
+          tempExists = true;
+        }
+      });
+    }
+    @Override
+    protected void apply(MinData data) {
+      if (tempExists && (!data.exists || tempMin < data.val)) {
+        data.val = tempMin;
+        data.exists = true;
+      }
+    }
+    
+    @Override
+    public void submitReservations(Consumer<ReductionDataReservation<?,?>> consumer) {
+      consumer.accept(new IntCheckedReservation(
+          value -> {
+            if (!ioData.exists || value < ioData.val) {
+              ioData.val = value;
+              ioData.exists = true;
+            }
+          },
+          ()-> ioData.val,
+          ()-> ioData.exists
+        ));
+    }
+
+    @Override
+    public void setMergedData(ReductionData data) {
+      super.setData(data);
+      min = ((MinData)data).val;
+    }
+
+    @Override
+    public void setData(ReductionData data) {
+      super.setData(data);
+      min = ((MinData)data).val;
+    }
+    
+    public static class MinData extends ReductionData {
+      int val;
+    }
+  }
+  
+
+
+  public static class LongMinCollector extends MinCollector<LongMinCollector.MinData> {
+    private LongValueStream param;
+    
+    public LongMinCollector(LongValueStream param) {
+      super(param);
+      this.param = param;
+    }
+
+    @Override
+    public MinData newData() {
+      MinData data = new MinData();
+      data.exists = false;
+      return data;
+    }
+
+    long min;
+    
+    /**
+     * Returns the min value of the set data.
+     * 
+     * @return the min
+     */
+    public long min() {
+      return min;
+    }
+
+    long tempMin;
+    boolean tempExists;
+    @Override
+    public void collect() {
+      tempExists = false;
+      param.streamLongs( val -> {
+        if (!tempExists || val < tempMin) {
+          tempMin = val;
+          tempExists = true;
+        }
+      });
+    }
+    @Override
+    protected void apply(MinData data) {
+      if (tempExists && (!data.exists || tempMin < data.val)) {
+        data.val = tempMin;
+        data.exists = true;
+      }
+    }
+    
+    @Override
+    public void submitReservations(Consumer<ReductionDataReservation<?,?>> consumer) {
+      consumer.accept(new LongCheckedReservation(
+          value -> {
+            if (!ioData.exists || value < ioData.val) {
+              ioData.val = value;
+              ioData.exists = true;
+            }
+          },
+          ()-> ioData.val,
+          ()-> ioData.exists
+        ));
+    }
+
+    @Override
+    public void setMergedData(ReductionData data) {
+      super.setData(data);
+      min = ((MinData)data).val;
+    }
+
+    @Override
+    public void setData(ReductionData data) {
+      super.setData(data);
+      min = ((MinData)data).val;
+    }
+    
+    public static class MinData extends ReductionData {
+      long val;
+    }
+  }
+
+  public static class FloatMinCollector extends MinCollector<FloatMinCollector.MinData> {
+    private FloatValueStream param;
+    
+    public FloatMinCollector(FloatValueStream param) {
+      super(param);
+      this.param = param;
+    }
+
+    @Override
+    public MinData newData() {
+      MinData data = new MinData();
+      data.exists = false;
+      return data;
+    }
+
+    float min;
+    
+    /**
+     * Returns the min value of the set data.
+     * 
+     * @return the min
+     */
+    public float min() {
+      return min;
+    }
+
+    float tempMin;
+    boolean tempExists;
+    @Override
+    public void collect() {
+      tempExists = false;
+      param.streamFloats( val -> {
+        if (!tempExists || val < tempMin) {
+          tempMin = val;
+          tempExists = true;
+        }
+      });
+    }
+    @Override
+    protected void apply(MinData data) {
+      if (tempExists && (!data.exists || tempMin < data.val)) {
+        data.val = tempMin;
+        data.exists = true;
+      }
+    }
+    
+    @Override
+    public void submitReservations(Consumer<ReductionDataReservation<?,?>> consumer) {
+      consumer.accept(new FloatCheckedReservation(
+          value -> {
+            if (!ioData.exists || value < ioData.val) {
+              ioData.val = value;
+              ioData.exists = true;
+            }
+          },
+          ()-> ioData.val,
+          ()-> ioData.exists
+        ));
+    }
+
+    @Override
+    public void setMergedData(ReductionData data) {
+      super.setData(data);
+      min = ((MinData)data).val;
+    }
+
+    @Override
+    public void setData(ReductionData data) {
+      super.setData(data);
+      min = ((MinData)data).val;
+    }
+    
+    public static class MinData extends ReductionData {
+      float val;
+    }
+  }
+
+  public static class DoubleMinCollector extends MinCollector<DoubleMinCollector.MinData> {
+    private DoubleValueStream param;
+    
+    public DoubleMinCollector(DoubleValueStream param) {
+      super(param);
+      this.param = param;
+    }
+
+    @Override
+    public MinData newData() {
+      MinData data = new MinData();
+      data.exists = false;
+      return data;
+    }
+
+    double min;
+    
+    /**
+     * Returns the min value of the set data.
+     * 
+     * @return the min
+     */
+    public double min() {
+      return min;
+    }
+
+    double tempMin;
+    boolean tempExists;
+    @Override
+    public void collect() {
+      tempExists = false;
+      param.streamDoubles( val -> {
+        if (!tempExists || val < tempMin) {
+          tempMin = val;
+          tempExists = true;
+        }
+      });
+    }
+    @Override
+    protected void apply(MinData data) {
+      if (tempExists && (!data.exists || tempMin < data.val)) {
+        data.val = tempMin;
+        data.exists = true;
+      }
+    }
+    
+    @Override
+    public void submitReservations(Consumer<ReductionDataReservation<?,?>> consumer) {
+      consumer.accept(new DoubleCheckedReservation(
+          value -> {
+            if (!ioData.exists || value < ioData.val) {
+              ioData.val = value;
+              ioData.exists = true;
+            }
+          },
+          ()-> ioData.val,
+          ()-> ioData.exists
+        ));
+    }
+
+    @Override
+    public void setMergedData(ReductionData data) {
+      super.setData(data);
+      min = ((MinData)data).val;
+    }
+
+    @Override
+    public void setData(ReductionData data) {
+      super.setData(data);
+      min = ((MinData)data).val;
+    }
+    
+    public static class MinData extends ReductionData {
+      double val;
+    }
+  }
+  
+
+
+  public static class StringMinCollector extends MinCollector<StringMinCollector.MinData> {
+    private StringValueStream param;
+    
+    public StringMinCollector(StringValueStream param) {
+      super(param);
+      this.param = param;
+    }
+
+    @Override
+    public MinData newData() {
+      MinData data = new MinData();
+      data.exists = false;
+      return data;
+    }
+
+    String min;
+    
+    /**
+     * Returns the min value of the set data.
+     * 
+     * @return the min
+     */
+    public String min() {
+      return min;
+    }
+
+    String tempMin;
+    boolean tempExists;
+    @Override
+    public void collect() {
+      tempExists = false;
+      param.streamStrings( val -> {
+        if (!tempExists || val.compareTo(tempMin) < 0) {
+          tempMin = val;
+          tempExists = true;
+        }
+      });
+    }
+    @Override
+    protected void apply(MinData data) {
+      if (tempExists && (!data.exists || tempMin.compareTo(data.val) < 0)) {
+        data.val = tempMin;
+        data.exists = true;
+      }
+    }
+    
+    @Override
+    public void submitReservations(Consumer<ReductionDataReservation<?,?>> consumer) {
+      consumer.accept(new StringCheckedReservation(
+          value -> {
+            if (!ioData.exists || value.compareTo(ioData.val) < 0) {
+              ioData.val = value;
+              ioData.exists = true;
+            }
+          },
+          ()-> ioData.val,
+          ()-> ioData.exists
+        ));
+    }
+
+    @Override
+    public void setMergedData(ReductionData data) {
+      super.setData(data);
+      min = ((MinData)data).val;
+    }
+
+    @Override
+    public void setData(ReductionData data) {
+      super.setData(data);
+      min = ((MinData)data).val;
+    }
+    
+    public static class MinData extends ReductionData {
+      String val;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/data/ReductionData.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/data/ReductionData.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/data/ReductionData.java
new file mode 100644
index 0000000..8c265fd
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/data/ReductionData.java
@@ -0,0 +1,24 @@
+/*
+ * 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.reduction.data;
+
+/**
+ * Base class to store data for {@link ReductionDataCollector}s
+ */
+public class ReductionData {
+  public boolean exists;
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/data/ReductionDataCollector.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/data/ReductionDataCollector.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/data/ReductionDataCollector.java
new file mode 100644
index 0000000..9674d98
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/data/ReductionDataCollector.java
@@ -0,0 +1,183 @@
+/*
+ * 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.reduction.data;
+
+import java.util.ArrayList;
+import java.util.function.Consumer;
+
+import org.apache.solr.analytics.function.ReductionFunction;
+import org.apache.solr.analytics.stream.reservation.ReductionDataReservation;
+import org.apache.solr.analytics.value.AnalyticsValue;
+
+/**
+ * Manager of a specific instance of {@link ReductionData} collection. 
+ * 
+ * @param <T> the type of reduction data being collected
+ */
+public abstract class ReductionDataCollector<T extends ReductionData> {
+
+  protected ArrayList<T> lastingTargets;
+  protected ArrayList<T> collectionTargets;
+  protected T ioData;
+  
+  protected ReductionDataCollector() {
+    lastingTargets = new ArrayList<>();
+    collectionTargets = new ArrayList<>();
+  }
+  
+  /**
+   * Submits the data reservations needed for this data collector.
+   * 
+   * @param consumer the consumer which the reservations are submitted to
+   */
+  public abstract void submitReservations(Consumer<ReductionDataReservation<?,?>> consumer);
+
+  /**
+   * A clean slate to start a new reduction.
+   * 
+   * @return the new reduction data
+   */
+  public abstract T newData();
+  
+  /**
+   * Add a reduction data to target during collection.
+   * The given target is valid until the lasting targets are cleared.
+   * 
+   * @param data the data to target
+   */
+  @SuppressWarnings("unchecked")
+  public void addLastingCollectTarget(ReductionData data) {
+    lastingTargets.add((T) data);
+  }
+  
+  /**
+   * Clear the lasting collection targets. After this is called the current lasting
+   * targets will not be affected by future {@link #collectAndApply()} calls. 
+   */
+  public void clearLastingCollectTargets() {
+    lastingTargets.clear();
+  }
+  
+  /**
+   * Create a new reduction data to target during collection.
+   * The given target is only valid for one call to {@link #collectAndApply()}.
+   * 
+   * @return the reduction data created
+   */
+  public T newDataTarget() {
+    T data = newData();
+    collectionTargets.add(data);
+    return data;
+  }
+  
+  /**
+   * Add a reduction data to target during collection.
+   * The given target is only valid for one call to {@link #collectAndApply()}.
+   * 
+   * @param data the data to target
+   */
+  @SuppressWarnings("unchecked")
+  public void addCollectTarget(ReductionData data) {
+    collectionTargets.add((T)data);
+  }
+
+  /**
+   * Collect the info for the current Solr Document and apply the results to the
+   * given collection targets.
+   * 
+   * After application, all non-lasting targets are removed.
+   */
+  public void collectAndApply() {
+    collect();
+    lastingTargets.forEach( target -> apply(target) );
+    collectionTargets.forEach( target -> apply(target) );
+    collectionTargets.clear();
+  }
+  
+  /**
+   * Collect the information from current Solr Document.
+   */
+  protected void collect() { }
+  
+  /**
+   * Apply the collected info to the given reduction data.
+   * Should always be called after a {@link #collect()} call.
+   * 
+   * @param data reduction data to apply collected info to
+   */
+  protected abstract void apply(T data);
+  
+  /**
+   * Create a new reduction data to use in exporting and merging.
+   * 
+   * @return the created reduction data
+   */
+  public T newDataIO() {
+    ioData = newData();
+    return ioData;
+  }
+  
+  /**
+   * Set the reduction data to use in exporting and merging.
+   * 
+   * @param data the data to use
+   */
+  @SuppressWarnings("unchecked")
+  public void dataIO(ReductionData data) {
+    ioData = (T)data;
+  }
+
+  /**
+   * Finalize the reduction with the merged data stored in the parameter.
+   * Once the reduction is finalized, the {@link ReductionFunction}s that use this 
+   * data collector act like regular {@link AnalyticsValue} classes that 
+   * can be accessed through their {@code get<value-type>} methods.
+   * 
+   * (FOR CLOUD)
+   * 
+   * @param data the merged data to compute a reduction for
+   */
+  public abstract void setMergedData(ReductionData data);
+
+  /**
+   * Finalize the reduction with the collected data stored in the parameter.
+   * Once the reduction is finalized, the {@link ReductionFunction}s that use this 
+   * data collector act like regular {@link AnalyticsValue} classes that 
+   * can be accessed through their {@code get<value-type>} methods.
+   * 
+   * (FOR SINGLE-SHARD)
+   * 
+   * @param data the collected data to compute a reduction for
+   */
+  public abstract void setData(ReductionData data);
+  
+  /**
+   * Get the name of the reduction data collector. This is the same across all instances of the data collector.
+   * 
+   * @return the name
+   */
+  public abstract String getName();
+  
+  /**
+   * The unique expression string of the reduction data collector, given all inputs and parameters.
+   * Used during {@link ReductionDataCollector} syncing. Since the string should be unique,
+   * only one of expression is kept.
+   * 
+   * @return the expression string 
+   */
+  public abstract String getExpressionStr();
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/data/SortedListCollector.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/data/SortedListCollector.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/data/SortedListCollector.java
new file mode 100644
index 0000000..c0de54b
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/data/SortedListCollector.java
@@ -0,0 +1,363 @@
+/*
+ * 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.reduction.data;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Consumer;
+
+import org.apache.solr.analytics.function.reduction.data.ReductionData;
+import org.apache.solr.analytics.stream.reservation.DoubleArrayReservation;
+import org.apache.solr.analytics.stream.reservation.FloatArrayReservation;
+import org.apache.solr.analytics.stream.reservation.IntArrayReservation;
+import org.apache.solr.analytics.stream.reservation.LongArrayReservation;
+import org.apache.solr.analytics.stream.reservation.ReductionDataReservation;
+import org.apache.solr.analytics.stream.reservation.StringArrayReservation;
+import org.apache.solr.analytics.util.OrdinalCalculator;
+import org.apache.solr.analytics.value.AnalyticsValueStream;
+import org.apache.solr.analytics.value.DoubleValueStream;
+import org.apache.solr.analytics.value.FloatValueStream;
+import org.apache.solr.analytics.value.IntValueStream;
+import org.apache.solr.analytics.value.LongValueStream;
+import org.apache.solr.analytics.value.StringValueStream;
+
+/**
+ * Collector of sorted lists.
+ * 
+ * Once the sorted list has been collected, it can be reduced by calculating a median, percentiles, or ordinals.
+ * All of the above reductions over the same data share one {@link SortedListCollector}.
+ * <p>
+ * Supported types are:
+ * <ul>
+ * <li>Int
+ * <li>Long
+ * <li>Float
+ * <li>Double
+ * <li>Date (through longs)
+ * <li>String
+ * </ul>
+ *
+ * @param <T> The type of data being processed.
+ */
+public abstract class SortedListCollector<T extends Comparable<T>> extends ReductionDataCollector<SortedListCollector.SortedListData<T>> {
+  public static final String name = "sorted";
+  private final String exprStr;
+  
+  protected SortedListCollector(AnalyticsValueStream param, String specificationName) {
+    this.exprStr = AnalyticsValueStream.createExpressionString(name + "_" + specificationName,param);
+    
+    tempList = new ArrayList<>();
+    
+    calcMedian = false;
+    calcPercs = new HashSet<>();
+    calcOrds = new HashSet<>();
+    calcRevOrds = new HashSet<>();
+  }
+  
+  private List<T> list;
+  
+  private boolean calcMedian;
+  private Set<Double> calcPercs;
+  private Set<Integer> calcOrds;
+  private Set<Integer> calcRevOrds;
+
+  public int size() {
+    return list.size();
+  }
+
+  /**
+   * Informs the collector that the median needs to be computed.
+   */
+  public void calcMedian() {
+    calcMedian = true;
+  }
+  
+  /**
+   * Informs the collector that the following percentile needs to be computed.
+   * 
+   * @param percentile requested percentile
+   */
+  public void calcPercentile(double percentile) {
+    calcPercs.add(percentile);
+  }
+
+  /**
+   * Informs the collector that the following ordinal needs to be computed.
+   * 
+   * @param ordinal requested ordinal
+   */
+  public void calcOrdinal(int ordinal) {
+    calcOrds.add(ordinal);
+  }
+
+  /**
+   * Informs the collector that the following reverse ordinal needs to be computed.
+   * Reverse ordinals are ordinals that start at the end of the list.
+   * 
+   * @param reverseOrdinal requested reverseOrdinal
+   */
+  public void calcReverseOrdinal(int reverseOrdinal) {
+    calcRevOrds.add(reverseOrdinal);
+  }
+
+  /**
+   * Once the data has been set by either {@link #setData} or {@link #setMergedData},
+   * this returns the value at the given sorted index. 
+   * 
+   * Only the indices specified by {@link #calcMedian}, {@link #calcPercentile}, {@link #calcOrdinal}, and {@link #calcReverseOrdinal}
+   * will contain valid data. All other indices may return unsorted data.
+   * 
+   * @param index the index of the sorted data to return
+   */
+  public T get(int index) {
+    return list.get(index);
+  }
+
+  @Override
+  public SortedListData<T> newData() {
+    SortedListData<T> data = new SortedListData<>();
+    data.list = new ArrayList<T>();
+    data.exists = false;
+    return data;
+  }
+  
+  ArrayList<T> tempList; 
+  @Override
+  protected void apply(SortedListData<T> data) {
+    data.list.addAll(tempList);
+  }
+  
+  /**
+   * Starts the import of the shard data.
+   * 
+   * @param size the size of the incoming shard list
+   */
+  protected void startImport(int size) {
+    ioData.list.ensureCapacity(ioData.list.size() + size);
+  }
+  
+  /**
+   * Merges the current list with the incoming value.
+   * 
+   * @param value the next imported value to add
+   */
+  protected void importNext(T value) {
+    ioData.list.add(value);
+  }
+  
+  Iterator<T> iter;
+  /**
+   * The list to be exported is unsorted.
+   * The lists of all shards will be combined with the {@link #startImport} and {@link #importNext} methods.
+   * 
+   * @return the size of the list being exported.
+   */
+  public int startExport() {
+    iter = ioData.list.iterator();
+    return ioData.list.size();
+  }
+  /**
+   * Return the next value in the list.
+   * 
+   * @return the next sorted value
+   */
+  public T exportNext() {
+    return iter.next();
+  }
+
+  /**
+   * Put the given indices in their sorted positions
+   */
+  @Override
+  public void setMergedData(ReductionData data) {
+    setData(data);
+  }
+
+  /**
+   * This is where the given indices are put in their sorted positions.
+   * 
+   * Only the given indices are guaranteed to be in sorted order.
+   */
+  @SuppressWarnings("unchecked")
+  @Override
+  public void setData(ReductionData data) {
+    list = ((SortedListData<T>)data).list;
+    int size = list.size();
+    if (size <= 1) {
+      return;
+    }
+    
+    // Ordinals start at 0 and end at size-1
+    Set<Integer> ordinals = new HashSet<>();
+    for (double percentile : calcPercs) {
+      ordinals.add((int) Math.ceil(percentile * size) - 1);
+    }
+    ordinals.addAll(calcOrds);
+    for (int reverseOrdinal : calcRevOrds) {
+      ordinals.add(size + reverseOrdinal);
+    }
+    if (calcMedian) {
+      int mid = list.size() / 2;
+      ordinals.add(mid);
+      if (list.size() % 2 == 0) {
+        ordinals.add(mid - 1);
+      }
+    }
+    OrdinalCalculator.putOrdinalsInPosition(list, ordinals);
+  }
+
+  @Override
+  public String getName() {
+    return name;
+  }
+  @Override
+  public String getExpressionStr() {
+    return exprStr;
+  }
+  
+  public static class SortedListData<D extends Comparable<D>> extends ReductionData {
+    ArrayList<D> list;
+  }
+
+  public static class SortedIntListCollector extends SortedListCollector<Integer> {
+    private IntValueStream param;
+    
+    public SortedIntListCollector(IntValueStream param) {
+      super(param, "int");
+      this.param = param;
+    }
+
+    @Override
+    public void collect() {
+      tempList.clear();
+      param.streamInts( val -> tempList.add(val) );
+    }
+    
+    @Override
+    public void submitReservations(Consumer<ReductionDataReservation<?,?>> consumer) {
+      consumer.accept(new IntArrayReservation(
+          value -> importNext(value),
+          importSize -> startImport(importSize),
+          () -> exportNext(),
+          () -> startExport()
+        ));
+    }
+  }
+  
+  public static class SortedLongListCollector extends SortedListCollector<Long> {
+    private LongValueStream param;
+    
+    public SortedLongListCollector(LongValueStream param) {
+      super(param, "long");
+      this.param = param;
+    }
+
+    @Override
+    public void collect() {
+      tempList.clear();
+      param.streamLongs( val -> tempList.add(val) );
+    }
+    
+    @Override
+    public void submitReservations(Consumer<ReductionDataReservation<?,?>> consumer) {
+      consumer.accept(new LongArrayReservation(
+          value -> importNext(value),
+          importSize -> startImport(importSize),
+          () -> exportNext(),
+          () -> startExport()
+        ));
+    }
+  }
+  
+  public static class SortedFloatListCollector extends SortedListCollector<Float> {
+    private FloatValueStream param;
+    
+    public SortedFloatListCollector(FloatValueStream param) {
+      super(param, "float");
+      this.param = param;
+    }
+
+    @Override
+    public void collect() {
+      tempList.clear();
+      param.streamFloats( val -> tempList.add(val) );
+    }
+    
+    @Override
+    public void submitReservations(Consumer<ReductionDataReservation<?,?>> consumer) {
+      consumer.accept(new FloatArrayReservation(
+          value -> importNext(value),
+          importSize -> startImport(importSize),
+          () -> exportNext(),
+          () -> startExport()
+        ));
+    }
+  }
+  
+  public static class SortedDoubleListCollector extends SortedListCollector<Double> {
+    private DoubleValueStream param;
+    
+    public SortedDoubleListCollector(DoubleValueStream param) {
+      super(param, "double");
+      this.param = param;
+    }
+
+    @Override
+    public void collect() {
+      tempList.clear();
+      param.streamDoubles( val -> tempList.add(val) );
+    }
+    
+    @Override
+    public void submitReservations(Consumer<ReductionDataReservation<?,?>> consumer) {
+      consumer.accept(new DoubleArrayReservation(
+          value -> importNext(value),
+          importSize -> startImport(importSize),
+          () -> exportNext(),
+          () -> startExport()
+        ));
+    }
+  }
+  
+  public static class SortedStringListCollector extends SortedListCollector<String> {
+    private StringValueStream param;
+    
+    public SortedStringListCollector(StringValueStream param) {
+      super(param, "string");
+      this.param = param;
+    }
+
+    @Override
+    public void collect() {
+      tempList.clear();
+      param.streamStrings( val -> tempList.add(val) );
+    }
+    
+    @Override
+    public void submitReservations(Consumer<ReductionDataReservation<?,?>> consumer) {
+      consumer.accept(new StringArrayReservation(
+          value -> importNext(value),
+          importSize -> startImport(importSize),
+          () -> exportNext(),
+          () -> startExport()
+        ));
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/data/SumCollector.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/data/SumCollector.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/data/SumCollector.java
new file mode 100644
index 0000000..6f4fc18
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/data/SumCollector.java
@@ -0,0 +1,124 @@
+/*
+ * 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.reduction.data;
+
+import java.util.function.Consumer;
+
+import org.apache.solr.analytics.function.reduction.data.ReductionData;
+import org.apache.solr.analytics.stream.reservation.DoubleCheckedReservation;
+import org.apache.solr.analytics.stream.reservation.ReductionDataReservation;
+import org.apache.solr.analytics.value.AnalyticsValueStream;
+import org.apache.solr.analytics.value.DoubleValueStream;
+
+/**
+ * Collects the sum of the given {@link DoubleValueStream} parameter. 
+ */
+public class SumCollector extends ReductionDataCollector<SumCollector.SumData> {
+  private final DoubleValueStream param;
+  public static final String name = "sum";
+  private final String exprStr;
+  
+  public SumCollector(DoubleValueStream param) {
+    this.param = param;
+    this.exprStr = AnalyticsValueStream.createExpressionString(name,param);
+  }
+  
+  private double sum;
+  private boolean exists;
+
+  /**
+   * Return the sum of the set data
+   * 
+   * @return the sum
+   */
+  public double sum() {
+    return sum;
+  }
+  
+  /**
+   * Return whether a sum exists.
+   * A sum will always exist if there is at least one existing value for the parameter, 
+   * otherwise the sum does not exist.
+   * 
+   * @return whether a sum exists
+   */
+  public boolean exists() {
+    return exists;
+  }
+
+  @Override
+  public SumData newData() {
+    SumData data = new SumData();
+    data.sum = 0;
+    data.exists = false;
+    return data;
+  }
+
+  double tempSum;
+  boolean tempExists;
+  @Override
+  public void collect() {
+    tempSum = 0;
+    tempExists = false;
+    param.streamDoubles( val -> {
+      tempSum += val;
+      tempExists = true;
+    });
+  }
+  @Override
+  protected void apply(SumData data) {
+    data.sum += tempSum;
+    data.exists |= tempExists;
+  }
+
+  @Override
+  public void submitReservations(Consumer<ReductionDataReservation<?,?>> consumer) {
+    consumer.accept(new DoubleCheckedReservation(
+        value -> {
+          ioData.sum += value;
+          ioData.exists = true;
+        },
+        ()-> ioData.sum,
+        ()-> ioData.exists
+      ));
+  }
+
+  @Override
+  public void setMergedData(ReductionData data) {
+    sum = ((SumData)data).sum;
+    exists = data.exists;
+  }
+
+  @Override
+  public void setData(ReductionData data) {
+    sum = ((SumData)data).sum;
+    exists = data.exists;
+  }
+
+  @Override
+  public String getName() {
+    return name;
+  }
+  @Override
+  public String getExpressionStr() {
+    return exprStr;
+  }
+  
+  public static class SumData extends ReductionData {
+    double sum;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/data/UniqueCollector.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/data/UniqueCollector.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/data/UniqueCollector.java
new file mode 100644
index 0000000..3e30114
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/data/UniqueCollector.java
@@ -0,0 +1,241 @@
+/*
+ * 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.reduction.data;
+
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+import java.util.function.Consumer;
+
+import org.apache.solr.analytics.function.reduction.data.ReductionData;
+import org.apache.solr.analytics.stream.reservation.DoubleArrayReservation;
+import org.apache.solr.analytics.stream.reservation.FloatArrayReservation;
+import org.apache.solr.analytics.stream.reservation.IntArrayReservation;
+import org.apache.solr.analytics.stream.reservation.LongArrayReservation;
+import org.apache.solr.analytics.stream.reservation.ReductionDataReservation;
+import org.apache.solr.analytics.stream.reservation.StringArrayReservation;
+import org.apache.solr.analytics.value.AnalyticsValueStream;
+import org.apache.solr.analytics.value.DoubleValueStream;
+import org.apache.solr.analytics.value.FloatValueStream;
+import org.apache.solr.analytics.value.IntValueStream;
+import org.apache.solr.analytics.value.LongValueStream;
+import org.apache.solr.analytics.value.StringValueStream;
+
+/**
+ * Collects the number of unique values that exist for the given parameter. 
+ * <p>
+ * Supported types are:
+ * <ul>
+ * <li>Int
+ * <li>Long
+ * <li>Float
+ * <li>Double
+ * <li>Date (through longs)
+ * <li>String
+ * </ul>
+ */
+public abstract class UniqueCollector<T> extends ReductionDataCollector<UniqueCollector.UniqueData<T>> {
+  public static final String name = "unique";
+  private final String exprStr;
+  
+  public UniqueCollector(AnalyticsValueStream param) {
+    this.exprStr = AnalyticsValueStream.createExpressionString(name,param);
+    this.tempSet = new HashSet<T>();
+  }
+  
+  private long count;
+
+  /**
+   * Get the count of unique values in the set data.
+   * 
+   * @return the count of unique values
+   */
+  public long count() {
+    return count;
+  }
+
+  @Override
+  public UniqueData<T> newData() {
+    UniqueData<T> data = new UniqueData<T>();
+    data.set = new HashSet<>();
+    data.exists = false;
+    return data;
+  }
+  
+  Set<T> tempSet;
+  @Override
+  protected void apply(UniqueData<T> data) {
+    data.set.addAll(tempSet);
+  }
+  
+  Iterator<T> iter;
+  public int startExport() {
+    iter = ioData.set.iterator();
+    return ioData.set.size();
+  }
+  public T exportNext() {
+    return iter.next();
+  }
+
+  @Override
+  public void setMergedData(ReductionData data) {
+    count = ((UniqueData<?>)data).set.size();
+  }
+
+  @Override
+  public void setData(ReductionData data) {
+    count = ((UniqueData<?>)data).set.size();
+  }
+
+  @Override
+  public String getName() {
+    return name;
+  }
+  @Override
+  public String getExpressionStr() {
+    return exprStr;
+  }
+  
+  public static class UniqueData<T> extends ReductionData {
+    Set<T> set;
+  }
+
+  public static class UniqueIntCollector extends UniqueCollector<Integer> {
+    private IntValueStream param;
+    
+    public UniqueIntCollector(IntValueStream param) {
+      super(param);
+      this.param = param;
+    }
+
+    @Override
+    public void collect() {
+      tempSet.clear();
+      param.streamInts( val -> tempSet.add(val) );
+    }
+    
+    @Override
+    public void submitReservations(Consumer<ReductionDataReservation<?,?>> consumer) {
+      consumer.accept(new IntArrayReservation(
+          value -> ioData.set.add(value),
+          size -> {},
+          () -> exportNext(),
+          () -> startExport()
+        ));
+    }
+  }
+  
+  public static class UniqueLongCollector extends UniqueCollector<Long> {
+    private LongValueStream param;
+    
+    public UniqueLongCollector(LongValueStream param) {
+      super(param);
+      this.param = param;
+    }
+
+    @Override
+    public void collect() {
+      tempSet.clear();
+      param.streamLongs( val -> tempSet.add(val) );
+    }
+    
+    @Override
+    public void submitReservations(Consumer<ReductionDataReservation<?,?>> consumer) {
+      consumer.accept(new LongArrayReservation(
+          value -> ioData.set.add(value),
+          size -> {},
+          () -> exportNext(),
+          () -> startExport()
+        ));
+    }
+  }
+  
+  public static class UniqueFloatCollector extends UniqueCollector<Float> {
+    private FloatValueStream param;
+    
+    public UniqueFloatCollector(FloatValueStream param) {
+      super(param);
+      this.param = param;
+    }
+
+    @Override
+    public void collect() {
+      tempSet.clear();
+      param.streamFloats( val -> tempSet.add(val) );
+    }
+    
+    @Override
+    public void submitReservations(Consumer<ReductionDataReservation<?,?>> consumer) {
+      consumer.accept(new FloatArrayReservation(
+          value -> ioData.set.add(value),
+          size -> {},
+          () -> exportNext(),
+          () -> startExport()
+        ));
+    }
+  }
+  
+  public static class UniqueDoubleCollector extends UniqueCollector<Double> {
+    private DoubleValueStream param;
+    
+    public UniqueDoubleCollector(DoubleValueStream param) {
+      super(param);
+      this.param = param;
+    }
+
+    @Override
+    public void collect() {
+      tempSet.clear();
+      param.streamDoubles( val -> tempSet.add(val) );
+    }
+    
+    @Override
+    public void submitReservations(Consumer<ReductionDataReservation<?,?>> consumer) {
+      consumer.accept(new DoubleArrayReservation(
+          value -> ioData.set.add(value),
+          size -> {},
+          () -> exportNext(),
+          () -> startExport()
+        ));
+    }
+  }
+  
+  public static class UniqueStringCollector extends UniqueCollector<String> {
+    private StringValueStream param;
+    
+    public UniqueStringCollector(StringValueStream param) {
+      super(param);
+      this.param = param;
+    }
+
+    @Override
+    public void collect() {
+      tempSet.clear();
+      param.streamStrings( val -> tempSet.add(val) );
+    }
+    
+    @Override
+    public void submitReservations(Consumer<ReductionDataReservation<?,?>> consumer) {
+      consumer.accept(new StringArrayReservation(
+          value -> ioData.set.add(value),
+          size -> {},
+          () -> exportNext(),
+          () -> startExport()
+        ));
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/data/package-info.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/data/package-info.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/data/package-info.java
new file mode 100644
index 0000000..15b4d18
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/data/package-info.java
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+ 
+/** 
+ * Reduction data collectors to use while computing analytics expressions.
+ * For multi-sharded collections, this is the data that is sent from shard to shard.
+ */
+package org.apache.solr.analytics.function.reduction.data;
+
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/package-info.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/package-info.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/package-info.java
new file mode 100644
index 0000000..ae45ef1
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/function/reduction/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.
+ */
+ 
+/** 
+ * Reduction functions to use for analytics expressions.
+ */
+package org.apache.solr.analytics.function.reduction;
+
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/package-info.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/package-info.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/package-info.java
new file mode 100644
index 0000000..e5bd519
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/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.
+ */
+ 
+/** 
+ * A solr component to compute complex analytics over search results. 
+ */
+package org.apache.solr.analytics;
+
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/request/AbstractFieldFacetRequest.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/request/AbstractFieldFacetRequest.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/request/AbstractFieldFacetRequest.java
deleted file mode 100644
index 8121f56..0000000
--- a/solr/contrib/analytics/src/java/org/apache/solr/analytics/request/AbstractFieldFacetRequest.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * 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.request;
-
-import org.apache.solr.schema.SchemaField;
-
-/**
- * An abstract request for a facet over a single field, such as a field or range facet.
- */
-public abstract class AbstractFieldFacetRequest implements FacetRequest {
-  protected SchemaField field = null;
-  
-  public AbstractFieldFacetRequest(SchemaField field) {
-    this.field = field;
-  }
-
-  public SchemaField getField() {
-    return field;
-  }
-
-  public void setField(SchemaField field) {
-    this.field = field;
-  }
-
-  public String getName() {
-    return field.getName();
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/request/AnalyticsContentHandler.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/request/AnalyticsContentHandler.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/request/AnalyticsContentHandler.java
deleted file mode 100644
index b93a59e..0000000
--- a/solr/contrib/analytics/src/java/org/apache/solr/analytics/request/AnalyticsContentHandler.java
+++ /dev/null
@@ -1,314 +0,0 @@
-/*
- * 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.request;
-
-import java.util.ArrayList;
-import java.util.EnumSet;
-import java.util.List;
-
-import org.apache.solr.analytics.request.FieldFacetRequest.FacetSortDirection;
-import org.apache.solr.analytics.request.FieldFacetRequest.FacetSortSpecification;
-import org.apache.solr.common.params.FacetParams.FacetRangeInclude;
-import org.apache.solr.common.params.FacetParams.FacetRangeOther;
-import org.apache.solr.schema.IndexSchema;
-import org.xml.sax.Attributes;
-import org.xml.sax.ContentHandler;
-import org.xml.sax.Locator;
-import org.xml.sax.SAXException;
-
-/**
- * Handles the parsing of the AnalysisRequestEnvelope elements if passed in through XML.
- */
-public class AnalyticsContentHandler implements ContentHandler {
-  // XML Element/Attribute Name Constants
-  public static final String ANALYTICS_REQUEST_ENVELOPE="analyticsRequestEnvelope";
-  
-  public static final String ANALYTICS_REQUEST="analyticsRequest";
-  public static final String NAME="name";
-  
-  public static final String STATISTIC="statistic";
-  public static final String EXPRESSION="expression";
-  
-  public static final String FIELD_FACET="fieldFacet";
-  public static final String FIELD="field";
-  public static final String SHOW_MISSING="showMissing";
-  public static final String LIMIT="limit";
-  public static final String MIN_COUNT="minCount";
-  
-  public static final String SORT_SPECIFICATION="sortSpecification";
-  public static final String STAT_NAME="statName";
-  public static final String DIRECTION="direction";
-  
-  public static final String RANGE_FACET="rangeFacet";
-  public static final String START="start";
-  public static final String END="end";
-  public static final String GAP="gap";
-  public static final String INCLUDE_BOUNDARY="includeBoundary";
-  public static final String OTHER_RANGE="otherRange";
-  public static final String HARD_END="hardend";
-  
-  public static final String QUERY_FACET="queryFacet";
-  public static final String QUERY="query";
-  
-  // Default Values
-  public static final int DEFAULT_FACET_LIMIT = -1;
-  public static final boolean DEFAULT_FACET_HARDEND = false;
-  public static final int DEFAULT_FACET_MINCOUNT = 0;
-  public static final boolean DEFAULT_FACET_FIELD_SHOW_MISSING = false;
-
-  boolean inEnvelope = false;
-  boolean inRequest = false;
-  boolean inStatistic = false;
-  boolean inFieldFacet = false;
-  boolean inSortSpecification = false;
-  boolean inQueryFacet = false;
-  boolean inRangeFacet = false;
-  
-  private final IndexSchema schema;
-  
-  // Objects to use while building the Analytics Requests
-  
-  String currentElementText;
-  
-  List<AnalyticsRequest> requests;
-  
-  AnalyticsRequest analyticsRequest;
-  List<ExpressionRequest> expressionList;
-  List<FieldFacetRequest> fieldFacetList;
-  List<RangeFacetRequest> rangeFacetList;
-  List<QueryFacetRequest> queryFacetList;
-  
-  ExpressionRequest expression;
-  
-  FieldFacetRequest fieldFacet;
-  int limit;
-  int minCount;
-  boolean showMissing;
-  FacetSortSpecification sortSpecification;
-  
-  RangeFacetRequest rangeFacet;
-  boolean hardend;
-  List<String> gaps;
-  EnumSet<FacetRangeInclude> includeBoundaries;
-  EnumSet<FacetRangeOther> otherRanges;
-  
-  String queryName;
-  List<String> queries;
-  
-  public AnalyticsContentHandler(IndexSchema schema) {
-    this.schema = schema;
-  }
-
-  @Override
-  public void setDocumentLocator(Locator locator) { }
-
-  @Override
-  public void startDocument() throws SAXException { }
-
-  @Override
-  public void endDocument() throws SAXException { }
-
-  @Override
-  public void startPrefixMapping(String prefix, String uri) throws SAXException { }
-
-  @Override
-  public void endPrefixMapping(String prefix) throws SAXException { }
-
-  @Override
-  public void startElement(String uri, String localName, String qName, Attributes atts) throws SAXException {
-    currentElementText = "";
-    if (inEnvelope) {
-      if (inRequest) {
-        if (localName.equals(STATISTIC)) {
-          // Start a Statistic Request
-          inStatistic = true;
-        } else if (inFieldFacet) {
-          if (localName.equals(SORT_SPECIFICATION)) {
-            // Start a Sort Specification
-            inSortSpecification = true;
-            sortSpecification = new FacetSortSpecification();
-          }
-        } else if (localName.equals(FIELD_FACET)) {
-          // Start a Field Facet Request
-          // Get attributes (limit, minCount, showMissing)
-          String att = atts.getValue(uri,LIMIT);
-          if (att!=null) {
-            limit = Integer.parseInt(att);
-          } else {
-            limit = DEFAULT_FACET_LIMIT;
-          }
-          att = atts.getValue(uri,MIN_COUNT);
-          if (att!=null) {
-            minCount = Integer.parseInt(att);
-          } else {
-            minCount = DEFAULT_FACET_MINCOUNT;
-          }
-          att = atts.getValue(uri,SHOW_MISSING);
-          if (att!=null) {
-            showMissing = Boolean.parseBoolean(att);
-          } else {
-            showMissing = DEFAULT_FACET_FIELD_SHOW_MISSING;
-          }
-          
-          inFieldFacet = true;
-        } else if (localName.equals(RANGE_FACET)) {
-          // Start a Range Facet Request
-          // Get attributes (hardEnd)
-          String att = atts.getValue(uri,HARD_END);
-          if (att!=null) {
-            hardend = Boolean.parseBoolean(att);
-          } else {
-            hardend = false;
-          }
-          
-          // Initiate Range Facet classes
-          gaps = new ArrayList<>();
-          includeBoundaries = EnumSet.noneOf(FacetRangeInclude.class);
-          otherRanges = EnumSet.noneOf(FacetRangeOther.class);
-          inRangeFacet = true;
-        } else if (localName.equals(QUERY_FACET)) {
-          // Start a Query Facet Request
-          queries = new ArrayList<>();
-          inQueryFacet = true;
-        }
-      } else if (localName.equals(ANALYTICS_REQUEST)){
-        // Start an Analytics Request
-        
-        // Renew each list.
-        fieldFacetList = new ArrayList<>();
-        rangeFacetList = new ArrayList<>();
-        queryFacetList = new ArrayList<>();
-        expressionList = new ArrayList<>();
-        inRequest = true;
-      }
-    } else if (localName.equals(ANALYTICS_REQUEST_ENVELOPE)){
-      //Begin the parsing of the Analytics Requests
-      requests = new ArrayList<>();
-      inEnvelope = true;
-    }
-  }
-
-  @Override
-  public void endElement(String uri, String localName, String qName) throws SAXException {
-    if (inEnvelope) {
-      if (inRequest) {
-        if (inStatistic) {
-          if (localName.equals(EXPRESSION)) {
-            expression = new ExpressionRequest(currentElementText,currentElementText);
-          } else if (localName.equals(NAME)) {
-            expression.setName(currentElementText);
-          } else if (localName.equals(STATISTIC)) {
-            // Finished Parsing the Statistic Request
-            expressionList.add(expression);
-            inStatistic = false;
-          } 
-        } else if (inFieldFacet) {
-          if (inSortSpecification) {
-            if (localName.equals(STAT_NAME)) {
-              sortSpecification.setStatistic(currentElementText);
-            } else if (localName.equals(DIRECTION)) {
-              sortSpecification.setDirection(FacetSortDirection.fromExternal(currentElementText));
-            } else if (localName.equals(SORT_SPECIFICATION)) {
-              // Finished Parsing the Sort Specification
-              fieldFacet.setSort(sortSpecification);
-              inSortSpecification = false;
-            } 
-          } else if (localName.equals(FIELD)) {
-            fieldFacet = new FieldFacetRequest(schema.getField(currentElementText));
-          } else if (localName.equals(FIELD_FACET)) {
-            // Finished Parsing the Field Facet Request
-            fieldFacet.setLimit(limit);
-            fieldFacet.showMissing(showMissing);
-            fieldFacetList.add(fieldFacet);
-            inFieldFacet = false;
-          } 
-        } else if (inRangeFacet) {
-          if (localName.equals(FIELD)) {
-            rangeFacet = new RangeFacetRequest(schema.getField(currentElementText), "", "", new String[1]);
-          } else if (localName.equals(START)) {
-            rangeFacet.setStart(currentElementText);
-          } else if (localName.equals(END)) {
-            rangeFacet.setEnd(currentElementText);
-          } else if (localName.equals(GAP)) {
-            gaps.add(currentElementText);
-          } else if (localName.equals(INCLUDE_BOUNDARY)) {
-            includeBoundaries.add(FacetRangeInclude.get(currentElementText));
-          } else if (localName.equals(OTHER_RANGE)) {
-            otherRanges.add(FacetRangeOther.get(currentElementText));
-          } else if (localName.equals(RANGE_FACET)) {
-            // Finished Parsing the Range Facet Request
-            rangeFacet.setHardEnd(hardend);
-            rangeFacet.setGaps(gaps.toArray(new String[1]));
-            rangeFacet.setInclude(includeBoundaries);
-            rangeFacet.setOthers(otherRanges);
-            inRangeFacet = false;
-            rangeFacetList.add(rangeFacet);
-          } 
-        } else if (inQueryFacet) {
-          if (localName.equals(NAME)) {
-            queryName = currentElementText;
-          } else if (localName.equals(QUERY)) {
-            queries.add(currentElementText);
-          } else if (localName.equals(QUERY_FACET)) {
-            // Finished Parsing the Query Facet Request
-            QueryFacetRequest temp = new QueryFacetRequest(queryName);
-            temp.setQueries(queries);
-            queryFacetList.add(temp);
-            inQueryFacet = false;
-          }
-        } else if (localName.equals(NAME)) {
-          analyticsRequest = new AnalyticsRequest(currentElementText);
-        } else if (localName.equals(ANALYTICS_REQUEST)){
-          // Finished Parsing the Analytics Request
-          analyticsRequest.setExpressions(expressionList);
-          analyticsRequest.setFieldFacets(fieldFacetList);
-          analyticsRequest.setRangeFacets(rangeFacetList);
-          analyticsRequest.setQueryFacets(queryFacetList);
-          requests.add(analyticsRequest);
-          inRequest = false;
-        }
-      } else if (localName.equals(ANALYTICS_REQUEST_ENVELOPE)){
-        // Finished Parsing
-        inEnvelope = false;
-      }
-    }
-  }
-
-  @Override
-  public void characters(char[] ch, int start, int length) throws SAXException {
-    currentElementText += new String(ch,start,length);
-  }
-
-  @Override
-  public void ignorableWhitespace(char[] ch, int start, int length) throws SAXException { }
-
-  @Override
-  public void processingInstruction(String target, String data) throws SAXException { }
-
-  @Override
-  public void skippedEntity(String name) throws SAXException { }
-  
-  /**
-   * Returns the list of Analytics Requests built during parsing.
-   * 
-   * @return List of {@link AnalyticsRequest} objects specified by the given XML file
-   */
-  public List<AnalyticsRequest> getAnalyticsRequests() {
-    return requests;
-  }
-
-}