You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by dp...@apache.org on 2017/06/28 18:17:42 UTC

[19/20] lucene-solr:master: 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/ExpressionFactory.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/ExpressionFactory.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/ExpressionFactory.java
new file mode 100644
index 0000000..b59469f
--- /dev/null
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/ExpressionFactory.java
@@ -0,0 +1,821 @@
+/*
+ * 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;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.Locale;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.solr.analytics.function.MergingReductionCollectionManager;
+import org.apache.solr.analytics.function.ReductionCollectionManager;
+import org.apache.solr.analytics.function.mapping.*;
+import org.apache.solr.analytics.function.mapping.CompareFunction.EqualFunction;
+import org.apache.solr.analytics.function.mapping.CompareFunction.GTEFunction;
+import org.apache.solr.analytics.function.mapping.CompareFunction.GTFunction;
+import org.apache.solr.analytics.function.mapping.CompareFunction.LTEFunction;
+import org.apache.solr.analytics.function.mapping.CompareFunction.LTFunction;
+import org.apache.solr.analytics.function.mapping.ConcatFunction.ConcatSeparatedFunction;
+import org.apache.solr.analytics.function.mapping.NumericConvertFunction.CeilingFunction;
+import org.apache.solr.analytics.function.mapping.NumericConvertFunction.FloorFunction;
+import org.apache.solr.analytics.function.mapping.NumericConvertFunction.RoundFunction;
+import org.apache.solr.analytics.function.mapping.LogicFunction.AndFunction;
+import org.apache.solr.analytics.function.mapping.LogicFunction.OrFunction;
+import org.apache.solr.analytics.function.reduction.*;
+import org.apache.solr.analytics.function.reduction.data.ReductionDataCollector;
+import org.apache.solr.analytics.value.*;
+import org.apache.solr.analytics.value.AnalyticsValueStream.ExpressionType;
+import org.apache.solr.analytics.value.constant.ConstantValue;
+import org.apache.solr.analytics.function.field.*;
+import org.apache.solr.analytics.function.ReductionFunction;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.schema.BoolField;
+import org.apache.solr.schema.DatePointField;
+import org.apache.solr.schema.DoublePointField;
+import org.apache.solr.schema.FieldType;
+import org.apache.solr.schema.FloatPointField;
+import org.apache.solr.schema.IndexSchema;
+import org.apache.solr.schema.IntPointField;
+import org.apache.solr.schema.LongPointField;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.schema.StrField;
+import org.apache.solr.schema.TrieDateField;
+import org.apache.solr.schema.TrieDoubleField;
+import org.apache.solr.schema.TrieFloatField;
+import org.apache.solr.schema.TrieIntField;
+import org.apache.solr.schema.TrieLongField;
+
+/**
+ * A factory to parse and create expressions, and capture information about those expressions along the way.
+ * 
+ * <p>
+ * In order to use, first call {@link #startRequest()} and create all ungrouped expressions,
+ * then call {@link #createReductionManager} to get the ungrouped reduction manager.
+ * <br>
+ * Then for each grouping call {@link #startGrouping()} first then create all expressions within that grouping,
+ * finally calling {@link #createGroupingReductionManager}  to get the reduction manager for that grouping.
+ */
+public class ExpressionFactory {
+  private static final Pattern functionNamePattern = Pattern.compile("^\\s*([^().\\s]+)\\s*(?:\\(.*\\)\\s*)?$", Pattern.CASE_INSENSITIVE);
+  private static final Pattern functionParamsPattern = Pattern.compile("^\\s*(?:[^(.)]+)\\s*\\(\\s*(.+)\\s*\\)\\s*$", Pattern.CASE_INSENSITIVE);
+  private static final String funtionVarParamUniqueName = ".%s_(%d)";
+
+  /**
+   * Used to denote a variable length parameter.
+   */
+  public final static String variableLengthParamSuffix = "..";
+  /**
+   * The character used to denote the start of a for each lambda expression
+   */
+  public final static char variableForEachSep = ':';
+  /**
+   * The character used to denote the looped parameter in the for each lambda expression
+   */
+  public final static char variableForEachParam = '_';
+
+  private HashMap<String, VariableFunctionInfo> systemVariableFunctions;
+  private HashMap<String, VariableFunctionInfo> variableFunctions;
+  private HashSet<String> variableFunctionNameHistory;
+  
+  private HashMap<String, CreatorFunction> expressionCreators;
+  private final ConstantFunction constantCreator;
+  
+  private LinkedHashMap<String, ReductionFunction> reductionFunctions;
+  private LinkedHashMap<String, ReductionDataCollector<?>> collectors;
+  private LinkedHashMap<String, AnalyticsField> fields;
+  private HashMap<String, AnalyticsValueStream> expressions;
+  
+  private IndexSchema schema;
+  
+  private Map<String, ReductionDataCollector<?>> groupedCollectors;
+  private Map<String, AnalyticsField> groupedFields;
+  private boolean isGrouped;
+
+  public ExpressionFactory(IndexSchema schema) {
+    this.schema = schema;
+    
+    expressionCreators = new HashMap<>();
+    systemVariableFunctions = new HashMap<>();
+    
+    constantCreator = ConstantValue.creatorFunction;
+    addSystemFunctions();
+  }
+  
+  /**
+   * Get the index schema used by this factory.
+   * 
+   * @return the index schema
+   */
+  public IndexSchema getSchema() {
+    return schema;
+  }
+  
+  /**
+   * Prepare the factory to start building the request.
+   */
+  public void startRequest() {
+    reductionFunctions = new LinkedHashMap<>();
+    collectors = new LinkedHashMap<>();
+    fields = new LinkedHashMap<>();
+    expressions = new HashMap<>();
+
+    variableFunctions = new HashMap<>();
+    variableFunctions.putAll(systemVariableFunctions);
+    variableFunctionNameHistory = new HashSet<>();
+    
+    isGrouped = false;
+  }
+  
+  /**
+   * Prepare the factory to start building the next grouping.
+   * <br>
+   * NOTE: MUST be called before each new grouping.
+   */
+  public void startGrouping() {
+    groupedCollectors = new HashMap<>();
+    groupedFields = new HashMap<>();
+    
+    isGrouped = true;
+  }
+
+  /**
+   * Add a system function to the expression factory.
+   * This will be treated as a native function and not a variable function.
+   * 
+   * @param functionName the unique name for the function
+   * @param functionCreator the creator function to generate an expression
+   * @return this factory, to easily chain function adds
+   * @throws SolrException if the functionName is not unique
+   */
+  public ExpressionFactory addSystemFunction(final String functionName, final CreatorFunction functionCreator) throws SolrException {
+    if (expressionCreators.containsKey(functionName)) {
+      throw new SolrException(ErrorCode.BAD_REQUEST,"System function " + functionName + " defined twice.");
+    }
+    expressionCreators.put(CountFunction.name, CountFunction.creatorFunction);
+    return this;
+  }
+
+  /**
+   * Add a variable function that will be treated like a system function.
+   * 
+   * @param functionSignature the function signature of the variable function (e.g. {@code func(a,b)} )
+   * @param returnSignature the return signature of the variable function (e.g. {@code div(sum(a,b),count(b))} )
+   * @return this factory, to easily chain function adds
+   * @throws SolrException if the name of the function is not unique or the syntax of either signature is incorrect
+   */
+  public ExpressionFactory addSystemVariableFunction(final String functionSignature, final String returnSignature) throws SolrException {
+    return addVariableFunction(functionSignature, returnSignature, systemVariableFunctions);
+  }
+
+  /**
+   * Add a variable function that was defined in an analytics request.
+   * 
+   * @param functionSignature the function signature of the variable function (e.g. {@code func(a,b)} )
+   * @param returnSignature the return signature of the variable function (e.g. {@code div(sum(a,b),count(b))} )
+   * @return this factory, to easily chain function adds
+   * @throws SolrException if the name of the function is not unique or the syntax of either signature is incorrect
+   */
+  public ExpressionFactory addUserDefinedVariableFunction(final String functionSignature, final String returnSignature) throws SolrException {
+    return addVariableFunction(functionSignature, returnSignature, variableFunctions);
+  }
+  
+  /**
+   * Add a variable function to the given map of variable functions. 
+   * 
+   * @param functionSignature the function signature of the variable function (e.g. {@code func(a,b)} )
+   * @param returnSignature the return signature of the variable function (e.g. {@code div(sum(a,b),count(b))} )
+   * @param variableFunctions the map of variable functions to add the new function to
+   * @return this factory, to easily chain function adds
+   * @throws SolrException if the name of the function is not unique or the syntax of either signature is incorrect
+   */
+  private ExpressionFactory addVariableFunction(final String functionSignature,
+                                                final String returnSignature,
+                                                Map<String,VariableFunctionInfo> variableFunctions) throws SolrException {
+    String functionName = getFunctionName(functionSignature);
+    if (expressionCreators.containsKey(functionName)) {
+      throw new SolrException(ErrorCode.BAD_REQUEST,"Users cannot define a variable function with the same name as an existing function: " + functionName);
+    }
+    VariableFunctionInfo varFuncInfo = new VariableFunctionInfo();
+    varFuncInfo.params = getParams(functionSignature, null, null);
+    varFuncInfo.returnSignature = returnSignature;
+    variableFunctions.put(functionName, varFuncInfo);
+    return this;
+  }
+
+  /**
+   * Create a reduction manager to manage the collection of all expressions that have been created since
+   * {@link #startRequest()} was called.
+   * 
+   * @param isCloudCollection whether the request is a distributed request
+   * @return a reduction manager
+   */
+  public ReductionCollectionManager createReductionManager(boolean isCloudCollection) {
+    ReductionDataCollector<?>[] collectorsArr = new ReductionDataCollector<?>[collectors.size()];
+    collectors.values().toArray(collectorsArr);
+    if (isCloudCollection) {
+      return new MergingReductionCollectionManager(collectorsArr, fields.values());
+    } else {
+      return new ReductionCollectionManager(collectorsArr, fields.values());
+    }
+  }
+
+  /**
+   * Create a reduction manager to manage the collection of all expressions that have been created since
+   * {@link #startGrouping()} was called.
+   * 
+   * @param isCloudCollection whether the request is a distributed request
+   * @return a reduction manager
+   */
+  public ReductionCollectionManager createGroupingReductionManager(boolean isCloudCollection) {
+    ReductionDataCollector<?>[] collectorsArr = new ReductionDataCollector<?>[groupedCollectors.size()];
+    groupedCollectors.values().toArray(collectorsArr);
+    if (isCloudCollection) {
+      return new MergingReductionCollectionManager(collectorsArr, groupedFields.values());
+    } else {
+      return new ReductionCollectionManager(collectorsArr, groupedFields.values());
+    }
+  }
+  
+  /**
+   * Parse and build an expression from the given expression string.
+   * 
+   * @param expressionStr string that represents the desired expression
+   * @return the object representation of the expression
+   * @throws SolrException if an error occurs while constructing the expression
+   */
+  public AnalyticsValueStream createExpression(String expressionStr) throws SolrException {
+    return createExpression(expressionStr, new HashMap<>(), null, null);
+  }
+
+  /**
+   * Create an expression from the given expression string, with the given variable function information.
+   * 
+   * @param expressionStr string that represents the desired expression
+   * @param varFuncParams the current set of variable function parameters and their values. If this expression is not a variable function
+   * return signature, the map should be empty.
+   * @param varFuncVarParamName if the current expression is a variable function return signature, this must be the name of the variable length
+   * parameter if it is included in the function signature.
+   * @param varFuncVarParamValues if the current expression is a variable function return signature, this must be the array values of the variable length
+   * parameter if they are included when calling the function.
+   * @return the object representation of the expression
+   * @throws SolrException if an error occurs while constructing the expression
+   */
+  private AnalyticsValueStream createExpression(String expressionStr, Map<String,AnalyticsValueStream> varFuncParams,
+                                                String varFuncVarParamName, String[] varFuncVarParamValues) throws SolrException {
+    AnalyticsValueStream expression;
+    expressionStr = expressionStr.trim();
+    
+    boolean isField = false;
+    try {
+      // Try to make a constant value
+      expression = constantCreator.apply(expressionStr);
+    } catch (SolrException e1) {
+      // Not a constant
+      // If the expression has parens, it is an expression otherwise it is a field
+      if (!expressionStr.contains("(")) {
+        try {
+          // Try to make a field out of it
+          expression = createField(schema.getField(expressionStr));
+          isField = true;
+        } catch (SolrException e2) {
+          throw new SolrException(ErrorCode.BAD_REQUEST,"The following is not a field, constant or function : " + expressionStr); 
+        }  
+      } else {
+        // Must be a function
+        expression = createFunction(expressionStr, varFuncParams, varFuncVarParamName, varFuncVarParamValues);
+      }
+    }
+
+    // Try to use an already made expression instead of the new one.
+    // This will decrease the amount of collection needed to be done.
+    if (expressions.containsKey(expression.getExpressionStr())) {
+      expression = expressions.get(expression.getExpressionStr());
+      // If this is a grouped expression, make sure that the reduction info for the expression is included in the grouped reduction manager.
+      if (expression.getExpressionType() == ExpressionType.REDUCTION && isGrouped) {
+        ((ReductionFunction)expression).synchronizeDataCollectors( collector -> {
+          groupedCollectors.put(collector.getExpressionStr(), collector);
+          return collector;
+        });
+      }
+    }
+    else {
+      expressions.put(expression.getExpressionStr(), expression);
+      // Make sure that the reduction info for the expression is included in the reduction manager and grouped reduction manager if necessary.
+      if (expression.getExpressionType() == ExpressionType.REDUCTION) {
+        reductionFunctions.put(expression.getExpressionStr(), (ReductionFunction)expression);
+        ((ReductionFunction)expression).synchronizeDataCollectors( collector -> {
+          String collectorStr = collector.getExpressionStr();
+          ReductionDataCollector<?> usedCollector = collectors.get(collectorStr);
+          if (usedCollector == null) {
+            usedCollector = collector;
+            collectors.put(collectorStr, collector);
+          }
+          if (isGrouped) {
+            groupedCollectors.put(collectorStr, usedCollector);
+          }
+          return usedCollector;
+        });
+      }
+      // Add the field info to the reduction manager
+      if (isField) {
+        fields.put(expression.getExpressionStr(), (AnalyticsField)expression);
+      }
+    }
+    // If this is a grouped expression, make sure that the field info is included in the grouped reduction manager.
+    if (isField && isGrouped) {
+      groupedFields.put(expression.getExpressionStr(), (AnalyticsField)expression);
+    }
+    return expression;
+  }
+  
+  /**
+   * Create a function expression from the given expression string, with the given variable function information.
+   * 
+   * @param expressionStr string that represents the desired expression
+   * @param varFuncParams the current set of variable function parameters and their values. If this expression is not a variable function
+   * return signature, the map should be empty.
+   * @param varFuncVarParamName if the current expression is a variable function return signature, this must be the name of the variable length
+   * parameter if it is included in the function signature.
+   * @param varFuncVarParamValues if the current expression is a variable function return signature, this must be the array values of the variable length
+   * parameter if they are included when calling the function.
+   * @return the object representation of the expression
+   * @throws SolrException if an error occurs while constructing the expression
+   */
+  private AnalyticsValueStream createFunction(String expressionStr, Map<String,AnalyticsValueStream> varFuncParams,
+                                              String varFuncVarParamName, String[] varFuncVarParamValues) throws SolrException {
+    AnalyticsValueStream expression = null;
+    String name = getFunctionName(expressionStr);
+
+    final String[] params = getParams(expressionStr, varFuncVarParamName, varFuncVarParamValues);
+    AnalyticsValueStream[] paramStreams = new AnalyticsValueStream[params.length];
+    for (int i = 0; i < params.length; i++) {
+      // First check if the parameter is a variable function variable otherwise create the expression
+      if (varFuncParams.containsKey(params[i])) {
+        paramStreams[i] = varFuncParams.get(params[i]);
+      } else {
+        paramStreams[i] = createExpression(params[i], varFuncParams, varFuncVarParamName, varFuncVarParamValues);
+      }
+    }
+    // Check to see if the function name is a variable function name, if so apply the variables to the return signature
+    if (variableFunctions.containsKey(name)) {
+      if (variableFunctionNameHistory.contains(name)) {
+        throw new SolrException(ErrorCode.BAD_REQUEST,"The following variable function is self referencing : " + name); 
+      }
+      variableFunctionNameHistory.add(name);
+      VariableFunctionInfo newVarFunc = variableFunctions.get(name);
+      Map<String, AnalyticsValueStream> newVarFuncParams = new HashMap<>();
+      
+      boolean varLenEnd = false;
+      
+      for (int i = 0; i < newVarFunc.params.length; ++i) {
+        String variable = newVarFunc.params[i];
+        if (variable.endsWith(variableLengthParamSuffix)) {
+          if (i != newVarFunc.params.length - 1) {
+            throw new SolrException(ErrorCode.BAD_REQUEST,"The following invocation of a variable function has the incorrect number of arguments : " + expressionStr);
+          }
+          variable = variable.substring(0, variable.length() - variableLengthParamSuffix.length()).trim();
+          int numVars = paramStreams.length - i;
+          String[] newVarFuncVarParamValues = new String[numVars];
+          for (int j = 0; j < numVars; ++j) {
+            // Create a new name for each variable length parameter value
+            String paramName = String.format(Locale.ROOT, funtionVarParamUniqueName, variable, j);
+            newVarFuncVarParamValues[j] = paramName;
+            newVarFuncParams.put(paramName, paramStreams[i + j]);
+          }
+          expression = createFunction(newVarFunc.returnSignature, newVarFuncParams, variable, newVarFuncVarParamValues);
+          varLenEnd = true;
+        } else {
+          newVarFuncParams.put(variable, paramStreams[i]);
+        }
+      }
+      if (!varLenEnd) {
+        expression = createExpression(newVarFunc.returnSignature, newVarFuncParams, null, null);
+      }
+      variableFunctionNameHistory.remove(name);
+      return expression;
+    } else if (expressionCreators.containsKey(name)) {
+      // It is a regular system function
+      expression = expressionCreators.get(name).apply(paramStreams);
+    } else {
+      throw new SolrException(ErrorCode.BAD_REQUEST,"The following function does not exist: " + name);
+    }
+    return expression;
+  }
+
+  /**
+   * Create an {@link AnalyticsField} out of the given {@link SchemaField}.
+   * <p>
+   * Currently only fields with doc-values enabled are supported.
+   * 
+   * @param field the field to convert for analytics
+   * @return an analytics representation of the field
+   * @throws SolrException if the field is not supported by the analytics framework
+   */
+  private AnalyticsField createField(SchemaField field) throws SolrException {
+    String fieldName = field.getName();
+    if (fields.containsKey(fieldName)) {
+      return fields.get(fieldName);
+    }
+    if (!field.hasDocValues()) {
+      throw new SolrException(ErrorCode.BAD_REQUEST,"The field "+fieldName+" does not have docValues enabled.");
+    }
+    boolean multivalued = field.multiValued();
+    FieldType fieldType = field.getType();
+    AnalyticsField aField;
+    if (fieldType instanceof BoolField) {
+      if (multivalued) {
+        aField = new BooleanMultiField(fieldName);
+      } else {
+        aField = new BooleanField(fieldName);
+      }
+    } else if (fieldType instanceof TrieIntField) {
+      if (multivalued) {
+        aField = new IntMultiField(fieldName);
+      } else {
+        aField = new IntField(fieldName);
+      }
+    } else if (fieldType instanceof IntPointField) {
+      if (multivalued) {
+        aField = new IntMultiPointField(fieldName);
+      } else {
+        aField = new IntField(fieldName);
+      }
+    } else if (fieldType instanceof TrieLongField) {
+      if (multivalued) {
+        aField = new LongMultiField(fieldName);
+      } else {
+        aField = new LongField(fieldName);
+      }
+    } else if (fieldType instanceof LongPointField) {
+      if (multivalued) {
+        aField = new LongMultiPointField(fieldName);
+      } else {
+        aField = new LongField(fieldName);
+      }
+    } else if (fieldType instanceof TrieFloatField) {
+      if (multivalued) {
+        aField = new FloatMultiField(fieldName);
+      } else {
+        aField = new FloatField(fieldName);
+      }
+    } else if (fieldType instanceof FloatPointField) {
+      if (multivalued) {
+        aField = new FloatMultiPointField(fieldName);
+      } else {
+        aField = new FloatField(fieldName);
+      }
+    } else if (fieldType instanceof TrieDoubleField) {
+      if (multivalued) {
+        aField = new DoubleMultiField(fieldName);
+      } else {
+        aField = new DoubleField(fieldName);
+      }
+    } else if (fieldType instanceof DoublePointField) {
+      if (multivalued) {
+        aField = new DoubleMultiPointField(fieldName);
+      } else {
+        aField = new DoubleField(fieldName);
+      }
+    } else if (fieldType instanceof TrieDateField) {
+      if (multivalued) {
+        aField = new DateMultiField(fieldName);
+      } else {
+        aField = new DateField(fieldName);
+      }
+    } else if (fieldType instanceof DatePointField) {
+      if (multivalued) {
+        aField = new DateMultiPointField(fieldName);
+      } else {
+        aField = new DateField(fieldName);
+      }
+    } else if (fieldType instanceof StrField) {
+      if (multivalued) {
+        aField = new StringMultiField(fieldName);
+      } else {
+        aField = new StringField(fieldName);
+      }
+    } else {
+      throw new SolrException(ErrorCode.BAD_REQUEST,"FieldType of the following field not supported by analytics: "+fieldName);
+    }
+    return aField;
+  }
+
+  /**
+   * Get the name of the top function used in the given expression.
+   * 
+   * @param expression the expression to find the function name of
+   * @return the name of the function 
+   * @throws SolrException if the expression has incorrect syntax
+   */
+  private static String getFunctionName(String expression) throws SolrException {
+    Matcher m = functionNamePattern.matcher(expression);
+    if (!m.matches()) {
+      throw new SolrException(ErrorCode.BAD_REQUEST,"The following function has no name: " + expression);
+    }
+    String name = m.group(1);
+    return name;
+  }
+  
+  /**
+   * Get the params of a function.
+   * 
+   * @param function the function to parse
+   * @return an array of param strings
+   * @throws SolrException if the function has incorrect syntax
+   */
+  private static String[] getFunctionParams(String function) throws SolrException {
+    return getParams(function, null, null);
+  }
+
+  /**
+   * Parse a function expression string and break up the parameters of the function into separate strings.
+   * <p>
+   * The parsing replaces the variable length parameter, and lambda for-each's using the variable length parameter,
+   * with the parameter values in the returned parameter string.
+   * <p>
+   * Parsing breaks up parameters by commas (',') and ignores ',' inside of extra parens and quotes (both ' and "), since these commas are either
+   * splitting up the parameters of nested functions or are apart of strings.
+   * <br>
+   * The only escaping that needs to be done is " within a double quote string and ' within a single quote string and \ within any string.
+   * For example\:
+   * <ul>
+   * <li> {@code func("This is \" the \\ escaping ' example")} will be treated as {@code func(This is " the \ escaping ' example)}
+   * <li> {@code func('This is " the \\ escaping \' example')} will be treated as {@code func(This is " the \ escaping ' example)}
+   * </ul>
+   * In string constants the \ character is used to escape quotes, so it can never be used alone. in order to write a \ you must write \\
+   * 
+   * @param expression the function expression to parse
+   * @param varLengthParamName the name of the variable length parameter that is used in the expression, pass null if none is used.
+   * @param varLengthParamValues the values of the variable length parameter that are used in the expression, pass null if none are used.
+   * @return the parsed and split arguments to the function
+   * @throws SolrException if the expression has incorrect syntax.
+   */
+  private static String[] getParams(String expression, String varLengthParamName, String[] varLengthParamValues) throws SolrException {
+    Matcher m = functionParamsPattern.matcher(expression);
+    if (!m.matches()) {
+      return new String[0];
+    }
+    String paramsStr = m.group(1);
+    
+    ArrayList<String> paramsList = new ArrayList<String>();
+    StringBuilder param = new StringBuilder();
+    
+    // Variables to help while filling out the values of for-each lambda functions.
+    boolean inForEach = false;
+    int forEachStart = -1;
+    int forEachIter = -1;
+    int forEachLevel = -1;
+    
+    // The current level of nested parenthesis, 0 means the iteration is in no nested parentheses
+    int parenCount = 0;
+    // If the iteration is currently in a single-quote string constant
+    boolean singleQuoteOn = false;
+    // If the iteration is currently in a double-quote string constant
+    boolean doubleQuoteOn = false;
+    // If the iteration is currently in any kind of string constant
+    boolean quoteOn = false;
+    // Is the next character escaped.
+    boolean escaped = false;
+    
+    char[] chars = paramsStr.toCharArray();
+    
+    // Iterate through every character, building the params one at a time
+    for (int i = 0; i < chars.length; ++i) {
+      char c = chars[i];
+      
+      if (c == ' ' && !quoteOn) {
+        // Ignore white space that is not in string constants
+        continue;
+      } else if (c == ',' && parenCount == 0 && !quoteOn) {
+        // This signifies the end of one parameter and the start of another, since we are not in a nested parenthesis or a string constant
+        String paramStr = param.toString();
+        if (paramStr.length() == 0) {
+          throw new SolrException(ErrorCode.BAD_REQUEST,"Empty parameter in expression: " + expression);
+        }
+        // check to see if the parameter is a variable length parameter
+        if (paramStr.equals(varLengthParamName)) {
+          // Add every variable length parameter value, since there are a variable amount
+          for (String paramName : varLengthParamValues) {
+            paramsList.add(paramName);
+          }
+        } else {
+          paramsList.add(paramStr);
+        }
+        
+        param.setLength(0);
+        continue;
+      } else if (c == ',' && !quoteOn && inForEach) {
+        // separate the for each parameters, so they can be replaced with the result of the for each
+        if (param.charAt(param.length()-1) == variableForEachParam && 
+            (param.charAt(param.length()-2) == '(' || param.charAt(param.length()-2) == ',')) {
+          param.setLength(param.length()-1);
+          param.append(varLengthParamValues[forEachIter++]);
+        }
+      } else if (c == '"' && !singleQuoteOn) {
+        // Deal with escaping, or ending string constants
+        if (doubleQuoteOn && !escaped) {
+          doubleQuoteOn = false;
+          quoteOn = false;
+        } else if (!quoteOn) {
+          doubleQuoteOn = true;
+          quoteOn = true;
+        } else {
+          // only happens if escaped is true
+          escaped = false;
+        }
+      }  else if (c== '\'' && !doubleQuoteOn) {
+        // Deal with escaping, or ending string constants
+        if (singleQuoteOn && !escaped) {
+          singleQuoteOn = false;
+          quoteOn = false;
+        } else if (!singleQuoteOn) {
+          singleQuoteOn = true;
+          quoteOn = true;
+        } else {
+          // only happens if escaped is true
+          escaped = false;
+        }
+      } else if (c == '(' && !quoteOn) {
+        // Reached a further level of nested parentheses
+        parenCount++;
+      } else if (c == ')' && !quoteOn) {
+        // Returned from a level of nested parentheses
+        parenCount--;
+        if (parenCount < 0) {
+          throw new SolrException(ErrorCode.BAD_REQUEST,"The following expression has extra end parens: " + param.toString());
+        }
+        if (inForEach) {
+          if (param.charAt(param.length()-1) == variableForEachParam && 
+              (param.charAt(param.length()-2) == '(' || param.charAt(param.length()-2) == ',')) {
+            param.setLength(param.length()-1);
+            param.append(varLengthParamValues[forEachIter++]);
+          }
+          if (forEachLevel == parenCount) {
+            // at the end of the for-each start the parsing of the for-each again, with the next value of the variable length parameter
+            if (forEachIter == 0) {
+              throw new SolrException(ErrorCode.BAD_REQUEST,"For each statement for variable '" + varLengthParamName + "' has no use of lambda variable " + variableForEachParam);
+            } else if (forEachIter < varLengthParamValues.length) {
+              if (parenCount == 0) {
+                param.append(')');
+                paramsList.add(param.toString());
+                param.setLength(0);
+              } else {
+                param.append(')');
+                param.append(',');
+              }
+              i = forEachStart;
+              continue;
+            } else {
+              inForEach = false;
+            }
+          }
+        }
+      }
+      if (c == '\\') {
+        // Escaping or escaped backslash
+        if (!quoteOn) {
+          throw new SolrException(ErrorCode.BAD_REQUEST,"The following expression has escaped characters outside of quotation marks: " + expression.toString());
+        }
+        if (escaped) {
+          escaped = false;
+        } else {
+          escaped = true;
+          if (parenCount == 0) {
+            continue;
+          }
+        }
+      } else if (escaped) {
+        throw new SolrException(ErrorCode.BAD_REQUEST,"Invalid escape character '" + c + "' used in the following expression: " + expression.toString());
+      }
+      if (c == variableForEachSep && !quoteOn && varLengthParamName != null) {
+        int varStart = param.length()-varLengthParamName.length();
+        if (param.subSequence(varStart, param.length()).equals(varLengthParamName)) {
+          inForEach = true;
+          forEachStart = i;
+          forEachIter = 0;
+          forEachLevel = parenCount;
+          param.setLength(varStart);
+          continue;
+        }
+        throw new SolrException(ErrorCode.BAD_REQUEST,"For-each called on invalid parameter '" + param.toString().trim());
+      }
+      param.append(c);
+    }
+    String paramStr = param.toString().trim();
+    if (paramStr.length() == 0) {
+      throw new SolrException(ErrorCode.BAD_REQUEST,"Empty parameter in expression: " + expression);
+    }
+    if (parenCount > 0) {
+      throw new SolrException(ErrorCode.BAD_REQUEST,"The following expression needs more end parens: " + param.toString());
+    }
+    if (quoteOn) {
+      throw new SolrException(ErrorCode.BAD_REQUEST,"Misplaced quotation marks in expression: " + expression);
+    }
+    if (paramStr.equals(varLengthParamName)) {
+      for (String paramName : varLengthParamValues) {
+        paramsList.add(paramName);
+      }
+    } else {
+      paramsList.add(paramStr);
+    }
+    return paramsList.toArray(new String[paramsList.size()]);
+  }
+  
+  /**
+   * Add the natively supported functionality.
+   */
+  public void addSystemFunctions() {
+    // Mapping Functions
+    expressionCreators.put(AbsoluteValueFunction.name, AbsoluteValueFunction.creatorFunction);
+    expressionCreators.put(AndFunction.name, AndFunction.creatorFunction);
+    expressionCreators.put(AddFunction.name, AddFunction.creatorFunction);
+    expressionCreators.put(BottomFunction.name, BottomFunction.creatorFunction);
+    expressionCreators.put(CeilingFunction.name, CeilingFunction.creatorFunction);
+    expressionCreators.put(ConcatFunction.name, ConcatFunction.creatorFunction);
+    expressionCreators.put(ConcatSeparatedFunction.name, ConcatSeparatedFunction.creatorFunction);
+    expressionCreators.put(DateMathFunction.name, DateMathFunction.creatorFunction);
+    expressionCreators.put(DateParseFunction.name, DateParseFunction.creatorFunction);
+    expressionCreators.put(DivideFunction.name, DivideFunction.creatorFunction);
+    expressionCreators.put(DocCountFunction.name, DocCountFunction.creatorFunction);
+    expressionCreators.put(EqualFunction.name,EqualFunction.creatorFunction);
+    expressionCreators.put(FillMissingFunction.name, FillMissingFunction.creatorFunction);
+    expressionCreators.put(FilterFunction.name, FilterFunction.creatorFunction);
+    expressionCreators.put(FloorFunction.name, FloorFunction.creatorFunction);
+    expressionCreators.put(GTFunction.name,GTFunction.creatorFunction);
+    expressionCreators.put(GTEFunction.name,GTEFunction.creatorFunction);
+    expressionCreators.put(IfFunction.name, IfFunction.creatorFunction);
+    expressionCreators.put(JoinFunction.name, JoinFunction.creatorFunction);
+    expressionCreators.put(LogFunction.name,LogFunction.creatorFunction);
+    expressionCreators.put(LTFunction.name,LTFunction.creatorFunction);
+    expressionCreators.put(LTEFunction.name,LTEFunction.creatorFunction);
+    expressionCreators.put(MultFunction.name, MultFunction.creatorFunction);
+    expressionCreators.put(NegateFunction.name, NegateFunction.creatorFunction);
+    expressionCreators.put(OrFunction.name, OrFunction.creatorFunction);
+    expressionCreators.put(PowerFunction.name, PowerFunction.creatorFunction);
+    expressionCreators.put(ReplaceFunction.name, ReplaceFunction.creatorFunction);
+    expressionCreators.put(RemoveFunction.name, RemoveFunction.creatorFunction);
+    expressionCreators.put(RoundFunction.name, RoundFunction.creatorFunction);
+    expressionCreators.put(StringCastFunction.name, StringCastFunction.creatorFunction);
+    expressionCreators.put(SubtractFunction.name, SubtractFunction.creatorFunction);
+    expressionCreators.put(TopFunction.name, TopFunction.creatorFunction);
+    
+    // Reduction Functions
+    expressionCreators.put(CountFunction.name, CountFunction.creatorFunction);
+    expressionCreators.put(MaxFunction.name, MaxFunction.creatorFunction);
+    expressionCreators.put(MedianFunction.name, MedianFunction.creatorFunction);
+    expressionCreators.put(MinFunction.name, MinFunction.creatorFunction);
+    expressionCreators.put(MissingFunction.name, MissingFunction.creatorFunction);
+    expressionCreators.put(OrdinalFunction.name, OrdinalFunction.creatorFunction);
+    expressionCreators.put(PercentileFunction.name, PercentileFunction.creatorFunction);
+    expressionCreators.put(SumFunction.name, SumFunction.creatorFunction);
+    expressionCreators.put(UniqueFunction.name, UniqueFunction.creatorFunction);
+    
+    // Variables
+    addSystemVariableFunction("wmean(a,b)","div(mean(prod(a,b)),sum(b))");
+    addSystemVariableFunction("mean(a)","div(sum(a),count(a))");
+    addSystemVariableFunction("sumofsquares(a)","sum(pow(a,2))");
+    addSystemVariableFunction("sqrt(a)","pow(a,0.5)");
+    addSystemVariableFunction("variance(a)","sub(mean(pow(a,2)),pow(mean(a),2))");
+    addSystemVariableFunction("stddev(a)","sqrt(variance(a))");
+    addSystemVariableFunction("csv(a..)","concatsep(',',a)");
+    addSystemVariableFunction("csv_output(a..)","concatsep(',',a:fillmissing(join(_,';'),''))");
+  }
+
+  /**
+   * Used for system analytics functions for initialization. Should take in a list of expression parameters and return an expression.
+   */
+  @FunctionalInterface
+  public static interface CreatorFunction {
+    AnalyticsValueStream apply(AnalyticsValueStream[] t) throws SolrException;
+  }
+  /**
+   * Used to initialize analytics constants.
+   */
+  @FunctionalInterface
+  public static interface ConstantFunction {
+    AnalyticsValueStream apply(String t) throws SolrException;
+  }
+}
+class VariableFunctionInfo {
+  public String[] params;
+  public String returnSignature;
+}
\ 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/accumulator/BasicAccumulator.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/accumulator/BasicAccumulator.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/accumulator/BasicAccumulator.java
deleted file mode 100644
index cbd8078..0000000
--- a/solr/contrib/analytics/src/java/org/apache/solr/analytics/accumulator/BasicAccumulator.java
+++ /dev/null
@@ -1,173 +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.accumulator;
-
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Date;
-import java.util.List;
-import java.util.Set;
-import java.util.function.Supplier;
-
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.solr.analytics.expression.Expression;
-import org.apache.solr.analytics.expression.ExpressionFactory;
-import org.apache.solr.analytics.request.AnalyticsRequest;
-import org.apache.solr.analytics.request.ExpressionRequest;
-import org.apache.solr.analytics.statistics.StatsCollector;
-import org.apache.solr.analytics.statistics.StatsCollectorSupplierFactory;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.SolrException.ErrorCode;
-import org.apache.solr.common.util.NamedList;
-import org.apache.solr.search.DocSet;
-import org.apache.solr.search.SolrIndexSearcher;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A <code>BasicAccumulator</code> manages the ValueCounters and Expressions without regard to Facets.
- */
-public class BasicAccumulator extends ValueAccumulator {
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  protected final SolrIndexSearcher searcher;
-  protected final AnalyticsRequest request;
-  protected final DocSet docs;
-  protected final Supplier<StatsCollector[]> statsCollectorArraySupplier;
-  protected final StatsCollector[] statsCollectors;
-  protected final Expression[] expressions;
-  protected final String[] expressionNames;
-  protected final String[] expressionStrings;
-  protected final Set<String> hiddenExpressions;
-  protected LeafReaderContext context = null;
-  
-  public BasicAccumulator(SolrIndexSearcher searcher, DocSet docs, AnalyticsRequest request) throws IOException {
-    this.searcher = searcher;
-    this.docs = docs;
-    this.request = request;
-    final List<ExpressionRequest> exRequests = new ArrayList<ExpressionRequest>(request.getExpressions()); // make a copy here
-    Collections.sort(exRequests);
-    log.info("Processing request '"+request.getName()+"'");
-    statsCollectorArraySupplier = StatsCollectorSupplierFactory.create(searcher.getSchema(), exRequests);
-    statsCollectors = statsCollectorArraySupplier.get();
-    int size = exRequests.size();
-    expressionNames = new String[size];
-    expressionStrings = new String[size];
-    int count = 0;
-    for (ExpressionRequest expRequest : exRequests) {
-      expressionNames[count] = expRequest.getName();
-      expressionStrings[count++] = expRequest.getExpressionString();
-    }
-    expressions = makeExpressions(statsCollectors);
-    hiddenExpressions = request.getHiddenExpressions();
-  }
-  
-  @Override
-  protected void doSetNextReader(LeafReaderContext context) throws IOException {
-    this.context = context;
-    for (StatsCollector counter : statsCollectors) {
-      counter.setNextReader(context);
-    }
-  }
- 
-  public static BasicAccumulator create(SolrIndexSearcher searcher, DocSet docs, AnalyticsRequest request) throws IOException {
-    return new BasicAccumulator(searcher,docs,request);
-  }
-  
-  /**
-   * Passes the documents on to the {@link StatsCollector}s to be collected.
-   * @param doc Document to collect from
-   */
-  @Override
-  public void collect(int doc) throws IOException {
-    for (StatsCollector statsCollector : statsCollectors) {
-      statsCollector.collect(doc);
-    }
-  }
-  
-  @Override
-  public void compute() {
-    for (StatsCollector statsCollector : statsCollectors) {
-      statsCollector.compute();
-    }
-  }
-  
-  public NamedList<?> export(){
-    NamedList<Object> base = new NamedList<>();
-    for (int count = 0; count < expressions.length; count++) {
-      if (!hiddenExpressions.contains(expressionNames[count])) {
-        base.add(expressionNames[count], expressions[count].getValue());
-      }
-    }
-    return base;
-  }
-  
-  /**
-   * Builds an array of Expressions with the given list of counters
-   * @param statsCollectors the stats collectors
-   * @return The array of Expressions
-   */
-  public Expression[] makeExpressions(StatsCollector[] statsCollectors) {
-   Expression[] expressions = new Expression[expressionStrings.length];
-    for (int count = 0; count < expressionStrings.length; count++) {
-      expressions[count] = ExpressionFactory.create(expressionStrings[count], statsCollectors);
-    }
-    return expressions;
-  }
-  
-  /**
-   * Returns the value of an expression to use in a field or query facet.
-   * @param expressionName the name of the expression
-   * @return String String representation of pivot value
-   */
-  @SuppressWarnings({ "deprecation", "rawtypes" })
-  public String getResult(String expressionName) {
-    for (int count = 0; count < expressionNames.length; count++) {
-      if (expressionName.equals(expressionNames[count])) {
-        Comparable value = expressions[count].getValue();
-        if (value.getClass().equals(Date.class)) {
-          return ((Date)value).toInstant().toString();
-        } else {
-          return value.toString();
-        }
-      }
-    }
-    throw new SolrException(ErrorCode.BAD_REQUEST, "Pivot expression "+expressionName+" not found.");
-  }
-
-  /**
-   * Used for JMX stats collecting. Counts the number of stats requests
-   * @return number of unique stats collectors
-   */
-  public long getNumStatsCollectors() {
-    return statsCollectors.length;
-  }
-
-  /**
-   * Used for JMX stats collecting. Counts the number of queries in all query facets
-   * @return number of queries requested in all query facets.
-   */
-  public long getNumQueries() {
-    return 0l;
-  }
-  
-  @Override
-  public boolean needsScores() {
-    return true; // TODO: is this true?
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/accumulator/FacetingAccumulator.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/accumulator/FacetingAccumulator.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/accumulator/FacetingAccumulator.java
deleted file mode 100644
index d8828a6..0000000
--- a/solr/contrib/analytics/src/java/org/apache/solr/analytics/accumulator/FacetingAccumulator.java
+++ /dev/null
@@ -1,730 +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.accumulator;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Date;
-import java.util.HashSet;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.TreeMap;
-
-import com.google.common.collect.Iterables;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.search.BooleanClause.Occur;
-import org.apache.lucene.search.BooleanQuery;
-import org.apache.lucene.search.Query;
-import org.apache.solr.analytics.accumulator.facet.FacetValueAccumulator;
-import org.apache.solr.analytics.accumulator.facet.FieldFacetAccumulator;
-import org.apache.solr.analytics.accumulator.facet.QueryFacetAccumulator;
-import org.apache.solr.analytics.accumulator.facet.RangeFacetAccumulator;
-import org.apache.solr.analytics.expression.Expression;
-import org.apache.solr.analytics.expression.ExpressionFactory;
-import org.apache.solr.analytics.request.AnalyticsContentHandler;
-import org.apache.solr.analytics.request.AnalyticsRequest;
-import org.apache.solr.analytics.request.FieldFacetRequest;
-import org.apache.solr.analytics.request.FieldFacetRequest.FacetSortSpecification;
-import org.apache.solr.analytics.request.QueryFacetRequest;
-import org.apache.solr.analytics.request.RangeFacetRequest;
-import org.apache.solr.analytics.statistics.StatsCollector;
-import org.apache.solr.analytics.util.AnalyticsParams;
-import org.apache.solr.analytics.util.RangeEndpointCalculator;
-import org.apache.solr.analytics.util.RangeEndpointCalculator.FacetRange;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.SolrException.ErrorCode;
-import org.apache.solr.common.util.NamedList;
-import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.schema.SchemaField;
-import org.apache.solr.search.DocSet;
-import org.apache.solr.search.Filter;
-import org.apache.solr.search.QParser;
-import org.apache.solr.search.SolrIndexSearcher;
-import org.apache.solr.search.SyntaxError;
-
-/**
- * A <code>FacetingAccumulator</code> manages the StatsCollectors and Expressions for facets.
- */
-public class FacetingAccumulator extends BasicAccumulator implements FacetValueAccumulator {
-  public static final String MISSING_VALUE = "(MISSING)";
-  protected boolean basicsAndFieldFacetsComputed;
-  protected int leafNum;
-  protected LeafReaderContext leaf;
-  protected final AnalyticsRequest analyticsRequest;
-  protected final Map<String,Map<String,Expression[]>> fieldFacetExpressions;
-  protected final Map<String,Map<String,Expression[]>> rangeFacetExpressions;
-  protected final Map<String,Map<String,Expression[]>> queryFacetExpressions;
-  protected final Map<String,Map<String,StatsCollector[]>> fieldFacetCollectors;
-  protected final Map<String,Map<String,StatsCollector[]>> rangeFacetCollectors;
-  protected final Map<String,Map<String,StatsCollector[]>> queryFacetCollectors;
-  protected final List<FieldFacetAccumulator> facetAccumulators;
-  protected final Set<String> hiddenFieldFacets;
-  /** the current value of this stat field */
-  protected final SolrQueryRequest queryRequest;
-  
-  protected List<RangeFacetRequest> rangeFacets = null;
-  protected List<QueryFacetRequest> queryFacets = null;
-  
-  protected long queryCount;
-  
-  public FacetingAccumulator(SolrIndexSearcher searcher, DocSet docs, AnalyticsRequest request, SolrQueryRequest queryRequest) throws IOException {
-    // The parent Basic Accumulator keeps track of overall stats while
-    // the Faceting Accumulator only manages the facet stats
-    super(searcher, docs, request);
-    this.analyticsRequest = request;
-    this.queryRequest = queryRequest;
-    basicsAndFieldFacetsComputed = false;
-    List<FieldFacetRequest> fieldFreqs = request.getFieldFacets();
-    List<RangeFacetRequest> rangeFreqs = request.getRangeFacets();
-    List<QueryFacetRequest> queryFreqs = request.getQueryFacets();
-
-    this.fieldFacetExpressions = new TreeMap<>();
-    this.rangeFacetExpressions = new LinkedHashMap<>(rangeFreqs.size());
-    this.queryFacetExpressions = new LinkedHashMap<>(queryFreqs.size());
-    this.fieldFacetCollectors = new LinkedHashMap<>(fieldFreqs.size());
-    this.rangeFacetCollectors = new LinkedHashMap<>(rangeFreqs.size());
-    this.queryFacetCollectors = new LinkedHashMap<>(queryFreqs.size());
-    this.facetAccumulators = new ArrayList<>();
-    this.hiddenFieldFacets = new HashSet<>();
-    
-    /**
-     * For each field facet request add a bucket to the {@link Expression} map and {@link StatsCollector} map.
-     * Field facets are computed during the initial collection of documents, therefore
-     * the FieldFacetAccumulators are created initially.
-     */
-    for( FieldFacetRequest freq : fieldFreqs ){
-      final FieldFacetRequest fr = (FieldFacetRequest) freq;
-      if (fr.isHidden()) {
-        hiddenFieldFacets.add(fr.getName());
-      }
-      final SchemaField ff = fr.getField();
-      final FieldFacetAccumulator facc = FieldFacetAccumulator.create(searcher, this, ff);
-      facetAccumulators.add(facc);
-      fieldFacetExpressions.put(freq.getName(), new TreeMap<String, Expression[]>() );
-      fieldFacetCollectors.put(freq.getName(), new TreeMap<String,StatsCollector[]>());
-    }
-    /**
-     * For each range and query facet request add a bucket to the corresponding
-     * {@link Expression} map and {@link StatsCollector} map.
-     * Range and Query Facets are computed in the post processing, so the accumulators
-     * are not created initially.
-     */
-    for( RangeFacetRequest freq : rangeFreqs ){
-      if( rangeFacets == null ) rangeFacets = new ArrayList<>();
-      rangeFacets.add(freq);
-      rangeFacetExpressions.put(freq.getName(), new LinkedHashMap<String,Expression[]>() );
-      rangeFacetCollectors.put(freq.getName(), new LinkedHashMap<String,StatsCollector[]>());
-    }
-    for( QueryFacetRequest freq : queryFreqs ){
-      if( queryFacets == null ) queryFacets = new ArrayList<>();
-      queryFacets.add(freq);
-      queryFacetExpressions.put(freq.getName(), new LinkedHashMap<String,Expression[]>() );
-      queryFacetCollectors.put(freq.getName(), new LinkedHashMap<String,StatsCollector[]>());
-    }
-    this.queryCount = 0l;
-  }
-  
-  public static FacetingAccumulator create(SolrIndexSearcher searcher, DocSet docs, AnalyticsRequest request, SolrQueryRequest queryRequest) throws IOException {
-    return new FacetingAccumulator(searcher,docs,request,queryRequest);
-  }
-
-  /**
-   * Update the readers for the {@link BasicAccumulator}, field facets and field facet {@link StatsCollector}s.
-   * @param context The context to read documents from.
-   * @throws IOException if there is an error setting the next reader
-   */
-  @Override
-  protected void doSetNextReader(LeafReaderContext context) throws IOException {
-    super.doSetNextReader(context);
-    for( Map<String,StatsCollector[]> valueList : fieldFacetCollectors.values() ){
-      for (StatsCollector[] statsCollectorList : valueList.values()) {
-        for (StatsCollector statsCollector : statsCollectorList) {
-          statsCollector.setNextReader(context);
-        }
-      }
-    }
-    for (FieldFacetAccumulator fa : facetAccumulators) {
-      fa.getLeafCollector(context);
-    }
-  }
-  
-  /**
-   * Updates the reader for all of the range facet {@link StatsCollector}s.
-   * @param context The context to read documents from.
-   * @throws IOException if there is an error setting the next reader
-   */
-  public void setRangeStatsCollectorReaders(LeafReaderContext context) throws IOException {
-    super.getLeafCollector(context);
-    for( Map<String,StatsCollector[]> rangeList : rangeFacetCollectors.values() ){
-      for (StatsCollector[] statsCollectorList : rangeList.values()) {
-        for (StatsCollector statsCollector : statsCollectorList) {
-          statsCollector.setNextReader(context);
-        }
-      }
-    }
-  }
-
-  
-  /**
-   * Updates the reader for all of the query facet {@link StatsCollector}s.
-   * @param context The context to read documents from.
-   * @throws IOException if there is an error setting the next reader
-   */
-  public void setQueryStatsCollectorReaders(LeafReaderContext context) throws IOException {
-    super.getLeafCollector(context);
-    for( Map<String,StatsCollector[]> queryList : queryFacetCollectors.values() ){
-      for (StatsCollector[] statsCollectorList : queryList.values()) {
-        for (StatsCollector statsCollector : statsCollectorList) {
-          statsCollector.setNextReader(context);
-        }
-      }
-    }
-  }
-
-  /**
-   * Called from Analytics stats, adds documents to the field 
-   * facets and the super {@link BasicAccumulator}.
-   */
-  @Override
-  public void collect(int doc) throws IOException {
-    for( FieldFacetAccumulator fa : facetAccumulators ){
-      fa.collect(doc);
-    }
-    super.collect(doc);
-  }
-  
-  /**
-   * Given a document, fieldFacet field and facetValue, adds the document to the
-   * {@link StatsCollector}s held in the bucket corresponding to the fieldFacet field and facetValue.
-   * Called during initial document collection.
-   */
-  @Override
-  public void collectField(int doc, String facetField, String facetValue) throws IOException {
-    Map<String,StatsCollector[]> map = fieldFacetCollectors.get(facetField);
-    StatsCollector[] statsCollectors = map.get(facetValue);
-    // If the facetValue has not been seen yet, a StatsCollector array is
-    // created and associated with that bucket.
-    if( statsCollectors == null ){
-      statsCollectors = statsCollectorArraySupplier.get();
-      map.put(facetValue,statsCollectors);
-      fieldFacetExpressions.get(facetField).put(facetValue,makeExpressions(statsCollectors));
-      for (StatsCollector statsCollector : statsCollectors) {
-        statsCollector.setNextReader(context);
-      }
-    }
-    for (StatsCollector statsCollector : statsCollectors) {
-      statsCollector.collect(doc);
-    }
-  }
-  
-  /**
-   * Given a document, rangeFacet field and range, adds the document to the
-   * {@link StatsCollector}s held in the bucket corresponding to the rangeFacet field and range.
-   * Called during post processing.
-   */
-  @Override
-  public void collectRange(int doc, String facetField, String range) throws IOException {
-    Map<String,StatsCollector[]> map = rangeFacetCollectors.get(facetField);
-    StatsCollector[] statsCollectors = map.get(range);
-    // If the range has not been seen yet, a StatsCollector array is
-    // created and associated with that bucket.
-    if( statsCollectors == null ){
-      statsCollectors = statsCollectorArraySupplier.get();
-      map.put(range,statsCollectors);
-      rangeFacetExpressions.get(facetField).put(range,makeExpressions(statsCollectors));
-      for (StatsCollector statsCollector : statsCollectors) {
-        statsCollector.setNextReader(context);
-      }
-    }
-    for (StatsCollector statsCollector : statsCollectors) {
-      statsCollector.collect(doc);
-    }
-  }
-  
-  /**
-   * Given a document, queryFacet name and query, adds the document to the
-   * {@link StatsCollector}s held in the bucket corresponding to the queryFacet name and query.
-   * Called during post processing.
-   */
-  @Override
-  public void collectQuery(int doc, String facetName, String query) throws IOException {
-    Map<String,StatsCollector[]> map = queryFacetCollectors.get(facetName);
-    StatsCollector[] statsCollectors = map.get(query);
-    // If the query has not been seen yet, a StatsCollector array is
-    // created and associated with that bucket.
-    if( statsCollectors == null ){
-      statsCollectors = statsCollectorArraySupplier.get();
-      map.put(query,statsCollectors);
-      queryFacetExpressions.get(facetName).put(query,makeExpressions(statsCollectors));
-      for (StatsCollector statsCollector : statsCollectors) {
-        statsCollector.setNextReader(context);
-      }
-    }
-    for (StatsCollector statsCollector : statsCollectors) {
-      statsCollector.collect(doc);
-    }
-  }
-
-  /**
-   * A comparator to compare expression values for field facet sorting.
-   */
-  public static class EntryComparator implements Comparator<Entry<String,Expression[]>> {
-    private final Comparator<Expression> comp;
-    private final int comparatorExpressionPlace;
-   
-    public EntryComparator(Comparator<Expression> comp, int comparatorExpressionPlace) {
-      this.comp = comp;
-      this.comparatorExpressionPlace = comparatorExpressionPlace;
-    }
-
-    @Override
-    public int compare(Entry<String,Expression[]> o1, Entry<String,Expression[]> o2) {
-      return comp.compare(o1.getValue()[comparatorExpressionPlace], o2.getValue()[comparatorExpressionPlace]);
-    }
-  }
-  
-  /**
-   * Finalizes the statistics within the each facet bucket before exporting;
-   */
-  @Override
-  public void compute() {
-    if (!basicsAndFieldFacetsComputed) {
-      super.compute();
-      for( Map<String, StatsCollector[]> f : fieldFacetCollectors.values() ){
-        for( StatsCollector[] arr : f.values() ){
-          for( StatsCollector b : arr ){
-            b.compute();
-          }
-        }
-      }
-      basicsAndFieldFacetsComputed = true;
-    }
-  }
-  
-  /**
-   * Finalizes the statistics within the a specific query facet before exporting;
-   */
-  public void computeQueryFacet(String facet) {
-    Map<String, StatsCollector[]> f = queryFacetCollectors.get(facet);
-    for( StatsCollector[] arr : f.values() ){
-      for( StatsCollector b : arr ){
-        b.compute();
-      }
-    }
-  }
-  
-  /**
-   * Finalizes the statistics within the a specific range facet before exporting;
-   */
-  public void computeRangeFacet(String facet) {
-    Map<String, StatsCollector[]> f = rangeFacetCollectors.get(facet);
-    for( StatsCollector[] arr : f.values() ){
-      for( StatsCollector b : arr ){
-        b.compute();
-      }
-    }
-  }
-  
-  /**
-   * Returns the value of an expression to use in a range or query facet.
-   * @param expressionName the name of the expression
-   * @param fieldFacet the facet field
-   * @param facetValue the facet value
-   * @return String String representation of pivot value
-   */
-  @SuppressWarnings({ "deprecation", "rawtypes" })
-  public String getResult(String expressionName, String fieldFacet, String facetValue) {
-    if (facetValue.contains(AnalyticsParams.RESULT) && !facetValue.contains(AnalyticsParams.QUERY_RESULT)) {
-      try {
-        String[] pivotStr = ExpressionFactory.getArguments(facetValue.substring(facetValue.indexOf('(')+1,facetValue.lastIndexOf(')')).trim());
-        if (pivotStr.length==1) {
-          facetValue = getResult(pivotStr[0]);
-        } else if (pivotStr.length==3) {
-          facetValue = getResult(pivotStr[0],pivotStr[1],pivotStr[2]);
-        } else {
-          throw new SolrException(ErrorCode.BAD_REQUEST, "Result request "+facetValue+" has an invalid amount of arguments.");
-        }
-      } catch (IndexOutOfBoundsException e) {
-        throw new SolrException(ErrorCode.BAD_REQUEST, "Result request "+facetValue+" is invalid. Lacks parentheses.",e);
-      }
-    } 
-    if (fieldFacetExpressions.get(fieldFacet)!=null) {
-      Expression[] facetExpressions = fieldFacetExpressions.get(fieldFacet).get(facetValue);
-      for (int count = 0; count < expressionNames.length; count++) {
-        if (expressionName.equals(expressionNames[count])) {
-          Comparable value = facetExpressions[count].getValue();
-          if (value.getClass().equals(Date.class)) {
-            return ((Date)value).toInstant().toString();
-          } else {
-            return value.toString();
-          }
-        }
-      }
-    }
-    throw new SolrException(ErrorCode.BAD_REQUEST,"Field Facet Pivot expression "+expressionName+" not found.");
-  }
-  
-  /**
-   * Returns the value of an expression to use in a range or query facet.
-   * @param currentFacet the name of the current facet
-   * @param expressionName the name of the expression
-   * @param queryFacet the facet query
-   * @param facetValue the field value
-   * @return String String representation of pivot value
-   */
-  @SuppressWarnings({ "deprecation", "rawtypes" })
-  public String getQueryResult(String currentFacet, String expressionName, String queryFacet, String facetValue) {
-    if (facetValue.contains(AnalyticsParams.RESULT) && !facetValue.contains(AnalyticsParams.QUERY_RESULT)) {
-      try {
-        String[] pivotStr = ExpressionFactory.getArguments(facetValue.substring(facetValue.indexOf('(')+1,facetValue.lastIndexOf(')')).trim());
-        if (pivotStr.length==1) {
-          facetValue = getResult(pivotStr[0]);
-        } else if (pivotStr.length==3) {
-          facetValue = getResult(pivotStr[0],pivotStr[1],pivotStr[2]);
-        } else {
-          throw new SolrException(ErrorCode.BAD_REQUEST, "Result request "+facetValue+" has an invalid amount of arguments.");
-        }
-      } catch (IndexOutOfBoundsException e) {
-        throw new SolrException(ErrorCode.BAD_REQUEST,"Result request "+facetValue+" is invalid. Lacks parentheses.",e);
-      }
-    } 
-    if (facetValue.contains(AnalyticsParams.QUERY_RESULT)) {
-      try {
-        String[] pivotStr = ExpressionFactory.getArguments(facetValue.substring(facetValue.indexOf('(')+1,facetValue.lastIndexOf(')')).trim());
-        if (pivotStr.length==1) {
-          facetValue = getResult(pivotStr[0]);
-        } else if (pivotStr.length==3) {
-          facetValue = getQueryResult(currentFacet,pivotStr[0],pivotStr[1],pivotStr[2]);
-        } else {
-          throw new SolrException(ErrorCode.BAD_REQUEST,"Result request "+facetValue+" has an invalid amount of arguments.");
-        }
-      } catch (IndexOutOfBoundsException e) {
-        throw new SolrException(ErrorCode.BAD_REQUEST,"Result request "+facetValue+" is invalid. Lacks parentheses.",e);
-      }
-    } 
-    if (queryFacetExpressions.get(queryFacet)!=null) {
-      Expression[] facetExpressions = queryFacetExpressions.get(queryFacet).get(facetValue);
-      for (int count = 0; count < expressionNames.length; count++) {
-        if (expressionName.equals(expressionNames[count])) {
-          Comparable value = facetExpressions[count].getValue();
-          if (value.getClass().equals(Date.class)) {
-            return ((Date)value).toInstant().toString();
-          } else {
-            return value.toString();
-          }
-        }
-      }
-    }
-    throw new SolrException(ErrorCode.BAD_REQUEST,"Field Facet Pivot expression "+expressionName+" not found.");
-  }
-  
-  @Override
-  @SuppressWarnings("unchecked")
-  public NamedList<?> export() {
-    final NamedList<Object> base = (NamedList<Object>)super.export();
-    NamedList<NamedList<?>> facetList = new NamedList<>();
-    
-    // Add the field facet buckets to the output
-    base.add("fieldFacets",facetList);
-    for( FieldFacetRequest freq : request.getFieldFacets() ){
-      final String name = freq.getName();
-      if (hiddenFieldFacets.contains(name)) {
-        continue;
-      }
-      final Map<String,Expression[]> buckets = fieldFacetExpressions.get(name);
-      final NamedList<Object> bucketBase = new NamedList<>();
-
-      Iterable<Entry<String,Expression[]>> iter = buckets.entrySet();
-      
-      final FieldFacetRequest fr = (FieldFacetRequest) freq;
-     
-      final FacetSortSpecification sort = fr.getSort();
-      final int limit = fr.getLimit();
-      final int offset = fr.getOffset();
-      final boolean showMissing = fr.showsMissing();
-      if (!showMissing) {
-        buckets.remove(MISSING_VALUE);
-      }
-      // Sorting the buckets if a sort specification is provided
-      if( sort != null && buckets.values().iterator().hasNext()){
-        int sortPlace = Arrays.binarySearch(expressionNames, sort.getStatistic());
-        final Expression first = buckets.values().iterator().next()[sortPlace];
-        final Comparator<Expression> comp = (Comparator<Expression>) first.comparator(sort.getDirection());
-        
-        final List<Entry<String,Expression[]>> sorted = new ArrayList<>(buckets.size());
-        Iterables.addAll(sorted, iter);
-        Collections.sort(sorted, new EntryComparator(comp,sortPlace));
-        iter = sorted;
-      }
-      // apply the limit
-      if( limit > AnalyticsContentHandler.DEFAULT_FACET_LIMIT ){
-        if( offset > 0 ){
-          iter = Iterables.skip(iter, offset);
-        }
-        iter = Iterables.limit(iter, limit);
-      }
-      
-      // Export each expression in the bucket.
-      for( Entry<String,Expression[]> bucket : iter ){
-        bucketBase.add(bucket.getKey(),export(bucket.getValue()));
-      }
-      
-      facetList.add(name, bucketBase);
-    }
-
-    // Add the range facet buckets to the output
-    facetList = new NamedList<>();
-    base.add("rangeFacets",facetList);
-    for( RangeFacetRequest freq : request.getRangeFacets() ){
-      final String name = freq.getName();
-      final Map<String,Expression[]> buckets = rangeFacetExpressions.get(name);
-      final NamedList<Object> bucketBase = new NamedList<>();
-
-      Iterable<Entry<String,Expression[]>> iter = buckets.entrySet();
-      
-      for( Entry<String,Expression[]> bucket : iter ){
-        bucketBase.add(bucket.getKey(),export(bucket.getValue()));
-      }
-      
-      facetList.add(name, bucketBase);
-    }
-    
-    // Add the query facet buckets to the output
-    facetList = new NamedList<>();
-    base.add("queryFacets",facetList);
-    for( QueryFacetRequest freq : request.getQueryFacets() ){
-      final String name = freq.getName();
-      final Map<String,Expression[]> buckets = queryFacetExpressions.get(name);
-      final NamedList<Object> bucketBase = new NamedList<>();
-
-      Iterable<Entry<String,Expression[]>> iter = buckets.entrySet();
-      
-      for( Entry<String,Expression[]> bucket : iter ){
-        bucketBase.add(bucket.getKey(),export(bucket.getValue()));
-      }
-      
-      facetList.add(name, bucketBase);
-    }
-
-    return base;
-  }
-  
-  /**
-   * Exports a list of expressions as a NamedList
-   * @param expressionArr an array of expressions
-   * @return named list of expressions
-   */
-  public NamedList<?> export(Expression[] expressionArr) {
-    NamedList<Object> base = new NamedList<>();
-    for (int count = 0; count < expressionArr.length; count++) {
-      if (!hiddenExpressions.contains(expressionNames[count])) {
-        base.add(expressionNames[count], expressionArr[count].getValue());
-      }
-    }
-    return base;
-  }
-
-  /**
-   * Processes the query and range facets.
-   * Must be called if range and/or query facets are supported.
-   */
-  @Override
-  public void postProcess() throws IOException {
-    super.compute();
-    for( Map<String, StatsCollector[]> f : fieldFacetCollectors.values() ){
-      for( StatsCollector[] arr : f.values() ){
-        for( StatsCollector b : arr ){
-          b.compute();
-        }
-      }
-    }
-    basicsAndFieldFacetsComputed = true;
-    final Filter filter = docs.getTopFilter();
-    if( rangeFacets != null ){
-      processRangeFacets(filter); 
-    }
-    if( queryFacets != null ){
-      processQueryFacets(filter); 
-    }
-  }
-  
-  /**
-   * Initiates the collecting of query facets
-   * @param filter the base filter to work against
-   * @throws IOException if searching failed
-   */
-  public void processQueryFacets(final Filter filter) throws IOException {
-    for( QueryFacetRequest qfr : queryFacets ){
-      for( String query : qfr.getQueries() ){
-        if (query.contains(AnalyticsParams.RESULT) && !query.contains(AnalyticsParams.QUERY_RESULT)) {
-          try {
-            String[] pivotStr = ExpressionFactory.getArguments(query.substring(query.indexOf('(')+1,query.lastIndexOf(')')).trim());
-            if (pivotStr.length==1) {
-              query = getResult(pivotStr[0]);
-            } else if (pivotStr.length==3) {
-              query = getResult(pivotStr[0],pivotStr[1],pivotStr[2]);
-            } else {
-              throw new SolrException(ErrorCode.BAD_REQUEST,"Result request "+query+" has an invalid amount of arguments.");
-            }
-          } catch (IndexOutOfBoundsException e) {
-            throw new SolrException(ErrorCode.BAD_REQUEST,"Result request "+query+" is invalid. Lacks parentheses.",e);
-          }
-        } else if (query.contains(AnalyticsParams.QUERY_RESULT)) {
-          try {
-            String[] pivotStr = ExpressionFactory.getArguments(query.substring(query.indexOf('(')+1,query.lastIndexOf(')')).trim());
-            if (pivotStr.length==3) {
-              query = getQueryResult(qfr.getName(),pivotStr[0],pivotStr[1],pivotStr[2]);
-            } else {
-              throw new SolrException(ErrorCode.BAD_REQUEST,"Result request "+query+" has an invalid amount of arguments.");
-            }
-          } catch (IndexOutOfBoundsException e) {
-            throw new SolrException(ErrorCode.BAD_REQUEST,"Result request "+query+" is invalid. Lacks parentheses.",e);
-          }
-        }
-        QueryFacetAccumulator qAcc = new QueryFacetAccumulator(this,qfr.getName(),query);
-        final Query q;
-        try {
-          q = QParser.getParser(query, queryRequest).getQuery();
-        } catch( SyntaxError e ){
-          throw new SolrException(ErrorCode.BAD_REQUEST,"Invalid query '"+query+"'",e);
-        }
-        // The searcher sends docIds to the QueryFacetAccumulator which forwards
-        // them to <code>collectQuery()</code> in this class for collection.
-        Query filtered = new BooleanQuery.Builder()
-            .add(q, Occur.MUST)
-            .add(filter, Occur.FILTER)
-            .build();
-        searcher.search(filtered, qAcc);
-        computeQueryFacet(qfr.getName());
-        queryCount++;
-      }
-    }
-  }
-  
-  @Override
-  public long getNumQueries() {
-    return queryCount;
-  }
-
-  /**
-   * Initiates the collecting of range facets
-   * @param filter the base filter to use
-   * @throws IOException if searching fails
-   */
-  public void processRangeFacets(final Filter filter) throws IOException {
-    for( RangeFacetRequest rfr : rangeFacets ){
-      String[] pivotStr;
-      String start = rfr.getStart();
-      if (start.contains(AnalyticsParams.QUERY_RESULT)) {
-        throw new SolrException(ErrorCode.BAD_REQUEST,"Query result requests can not be used in Range Facets");
-      } else if (start.contains(AnalyticsParams.RESULT)) {
-        try {
-          pivotStr = ExpressionFactory.getArguments(start.substring(start.indexOf('(')+1,start.indexOf(')')).trim());
-          if (pivotStr.length==1) {
-            rfr.setStart(getResult(pivotStr[0]));
-          } else if (pivotStr.length==3) {
-            rfr.setStart(getResult(pivotStr[0],pivotStr[1],pivotStr[2]));
-          } else {
-            throw new SolrException(ErrorCode.BAD_REQUEST, "Result request "+start+" has an invalid amount of arguments.");
-          }
-        } catch (IndexOutOfBoundsException e) {
-          throw new SolrException(ErrorCode.BAD_REQUEST, "Result request "+start+" is invalid. Lacks parentheses.",e);
-        }
-      }
-      String end = rfr.getEnd();
-      if (end.contains(AnalyticsParams.QUERY_RESULT)) {
-        throw new SolrException(ErrorCode.BAD_REQUEST, "Query result requests can not be used in Range Facets");
-      } else if (end.contains(AnalyticsParams.RESULT)) {
-        try {
-          pivotStr = ExpressionFactory.getArguments(end.substring(end.indexOf('(')+1,end.indexOf(')')).trim());
-          if (pivotStr.length==1) {
-            rfr.setEnd(getResult(pivotStr[0]));
-          } else if (pivotStr.length==3) {
-            rfr.setEnd(getResult(pivotStr[0],pivotStr[1],pivotStr[2]));
-          } else {
-            throw new SolrException(ErrorCode.BAD_REQUEST, "Result request "+end+" has an invalid amount of arguments.");
-          }
-        } catch (IndexOutOfBoundsException e) {
-          throw new SolrException(ErrorCode.BAD_REQUEST, "Result request "+end+" is invalid. Lacks parentheses.",e);
-        }
-      }
-      String[] gaps = rfr.getGaps();
-      for (int count = 0; count<gaps.length; count++){
-        String gap = gaps[count];
-        if (gap.contains(AnalyticsParams.QUERY_RESULT)) {
-          throw new SolrException(ErrorCode.BAD_REQUEST, "Query result requests can not be used in Range Facets");
-        } else if (gap.contains(AnalyticsParams.RESULT)) {
-          try {
-            pivotStr = ExpressionFactory.getArguments(gap.substring(gap.indexOf('(')+1,gap.indexOf(')')).trim());
-            if (pivotStr.length==1) {
-              gaps[count]=getResult(pivotStr[0]);
-            } else if (pivotStr.length==3) {
-              gaps[count]=getResult(pivotStr[0],pivotStr[1],pivotStr[2]);
-            } else {
-              throw new SolrException(ErrorCode.BAD_REQUEST, "Result request "+gap+" has an invalid amount of arguments.");
-            }
-          } catch (IndexOutOfBoundsException e) {
-            throw new SolrException(ErrorCode.BAD_REQUEST, "Result request "+gap+" is invalid. Lacks parentheses.",e);
-          }
-        }
-      }
-      // Computes the end points of the ranges in the rangeFacet
-      final RangeEndpointCalculator<? extends Comparable<?>> rec = RangeEndpointCalculator.create(rfr);
-      final SchemaField sf = rfr.getField();
-      
-      // Create a rangeFacetAccumulator for each range and 
-      // collect the documents for that range.
-      for( FacetRange range : rec.getRanges() ){
-        final String upper;
-        final String lower;
-        String facetValue = "";
-        if( range.lower == null ){
-          facetValue = "(*";
-          lower = null;
-        } else {
-          lower = range.lower;
-          facetValue = ((range.includeLower)?"[":"(") + range.lower;
-        }
-        facetValue+=" TO ";
-        if( range.upper == null ){
-          upper = null;
-          facetValue += "*)";
-        } else {
-          upper = range.upper;
-          facetValue += range.upper + ((range.includeUpper)?"]":")");
-        }
-        
-        Query q = sf.getType().getRangeQuery(null, sf, lower, upper, range.includeLower,range.includeUpper);
-        RangeFacetAccumulator rAcc = new RangeFacetAccumulator(this,rfr.getName(),facetValue);
-        // The searcher sends docIds to the RangeFacetAccumulator which forwards
-        // them to <code>collectRange()</code> in this class for collection.
-        Query filtered = new BooleanQuery.Builder()
-            .add(q, Occur.MUST)
-            .add(filter, Occur.FILTER)
-            .build();
-        searcher.search(filtered, rAcc);
-        computeRangeFacet(sf.getName());
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/accumulator/ValueAccumulator.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/accumulator/ValueAccumulator.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/accumulator/ValueAccumulator.java
deleted file mode 100644
index 489d3de..0000000
--- a/solr/contrib/analytics/src/java/org/apache/solr/analytics/accumulator/ValueAccumulator.java
+++ /dev/null
@@ -1,40 +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.accumulator;
-
-import java.io.IOException;
-
-import org.apache.lucene.search.SimpleCollector;
-import org.apache.solr.common.util.NamedList;
-
-/**
- * Abstract Collector that manages all StatsCollectors, Expressions and Facets.
- */
-public abstract class ValueAccumulator extends SimpleCollector {
-  
-  /**
-   * Finalizes the statistics within each StatsCollector.
-   * Must be called before <code>export()</code>.
-   */
-  public abstract void compute();
-  public abstract NamedList<?> export();
-  
-  public void postProcess() throws IOException {
-    // NOP
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5963beb/solr/contrib/analytics/src/java/org/apache/solr/analytics/accumulator/facet/FacetValueAccumulator.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/accumulator/facet/FacetValueAccumulator.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/accumulator/facet/FacetValueAccumulator.java
deleted file mode 100644
index 1b6fbb7..0000000
--- a/solr/contrib/analytics/src/java/org/apache/solr/analytics/accumulator/facet/FacetValueAccumulator.java
+++ /dev/null
@@ -1,35 +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.accumulator.facet;
-
-import java.io.IOException;
-
-import org.apache.lucene.index.LeafReaderContext;
-
-/**
- * Interface that describes the methods needed for an Accumulator to be able to handle 
- * fieldFacets, rangeFacets and queryFacets.
- */
-public interface FacetValueAccumulator {
-
-  void collectField(int doc, String facetName, String facetValue) throws IOException;
-  void collectQuery(int doc, String facetName, String facetValue) throws IOException;
-  void collectRange(int doc, String facetName, String facetValue) throws IOException;
-  void setQueryStatsCollectorReaders(LeafReaderContext context) throws IOException;
-  void setRangeStatsCollectorReaders(LeafReaderContext context) throws IOException;
-
-}