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

svn commit: r1626856 - in /lucene/dev/trunk/solr: ./ core/src/java/org/apache/solr/handler/component/ core/src/java/org/apache/solr/request/ core/src/test/org/apache/solr/ core/src/test/org/apache/solr/handler/component/

Author: hossman
Date: Mon Sep 22 17:45:53 2014
New Revision: 1626856

URL: http://svn.apache.org/r1626856
Log:
SOLR-6354: stats.field can now be used to generate stats over the numeric results of arbitrary functions

Added:
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/StatsField.java   (with props)
Modified:
    lucene/dev/trunk/solr/CHANGES.txt
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/FieldFacetStats.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/StatsComponent.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/request/DocValuesStats.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/request/UnInvertedField.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/TestDistributedSearch.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/component/StatsComponentTest.java

Modified: lucene/dev/trunk/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=1626856&r1=1626855&r2=1626856&view=diff
==============================================================================
--- lucene/dev/trunk/solr/CHANGES.txt (original)
+++ lucene/dev/trunk/solr/CHANGES.txt Mon Sep 22 17:45:53 2014
@@ -131,6 +131,10 @@ New Features
 * SOLR-6482:  Add an onlyIfDown flag for DELETEREPLICA collections API command
   (Erick Erickson)
 
+* SOLR-6354: stats.field can now be used to generate stats over the numeric results
+  of arbitrary functions, ie: stats.field={!func}product(price,popularity)
+  (hossman)
+
 Bug Fixes
 ----------------------
 

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/FieldFacetStats.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/FieldFacetStats.java?rev=1626856&r1=1626855&r2=1626856&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/FieldFacetStats.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/FieldFacetStats.java Mon Sep 22 17:45:53 2014
@@ -46,9 +46,8 @@ import org.apache.solr.search.SolrIndexS
 
 public class FieldFacetStats {
   public final String name;
+  final StatsField statsField;
   final SchemaField facet_sf;
-  final SchemaField field_sf;
-  final boolean calcDistinct;
 
   public final Map<String, StatsValues> facetStatsValues;
   private final Map<Integer, Integer> missingStats;
@@ -62,11 +61,10 @@ public class FieldFacetStats {
 
   SortedDocValues topLevelSortedValues = null;
 
-  public FieldFacetStats(SolrIndexSearcher searcher, String name, SchemaField field_sf, SchemaField facet_sf, boolean calcDistinct) {
-    this.name = name;
-    this.field_sf = field_sf;
+  public FieldFacetStats(SolrIndexSearcher searcher, SchemaField facet_sf, StatsField statsField) {
+    this.statsField = statsField;
     this.facet_sf = facet_sf;
-    this.calcDistinct = calcDistinct;
+    this.name = facet_sf.getName();
 
     topLevelReader = searcher.getAtomicReader();
     valueSource = facet_sf.getType().getValueSource(facet_sf, null);
@@ -79,7 +77,7 @@ public class FieldFacetStats {
   private StatsValues getStatsValues(String key) throws IOException {
     StatsValues stats = facetStatsValues.get(key);
     if (stats == null) {
-      stats = StatsValuesFactory.createStatsValues(field_sf, calcDistinct);
+      stats = StatsValuesFactory.createStatsValues(statsField);
       facetStatsValues.put(key, stats);
       stats.setNextReader(context);
     }
@@ -142,7 +140,7 @@ public class FieldFacetStats {
       String key = (String) pairs.getKey();
       StatsValues facetStats = facetStatsValues.get(key);
       if (facetStats == null) {
-        facetStats = StatsValuesFactory.createStatsValues(field_sf, calcDistinct);
+        facetStats = StatsValuesFactory.createStatsValues(statsField);
         facetStatsValues.put(key, facetStats);
       }
       Integer count = (Integer) pairs.getValue();

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/StatsComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/StatsComponent.java?rev=1626856&r1=1626855&r2=1626856&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/StatsComponent.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/StatsComponent.java Mon Sep 22 17:45:53 2014
@@ -20,36 +20,19 @@ package org.apache.solr.handler.componen
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.Collection;
 import java.util.LinkedHashMap;
-import java.util.IdentityHashMap;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.lucene.search.*;
-import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ShardParams;
 import org.apache.solr.common.params.SolrParams;
-import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.StatsParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
-import org.apache.solr.common.util.StrUtils;
-import org.apache.solr.request.DocValuesStats;
-import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.schema.FieldType;
-import org.apache.solr.schema.IndexSchema;
-import org.apache.solr.schema.SchemaField;
-import org.apache.solr.search.DocIterator;
 import org.apache.solr.search.DocSet;
-import org.apache.solr.search.QParser;
-import org.apache.solr.search.QueryParsing;
-import org.apache.solr.search.SolrIndexSearcher;
-import org.apache.solr.search.SyntaxError;
 
 /**
  * Stats component calculates simple statistics on numeric field values
@@ -198,7 +181,8 @@ class StatsInfo {
     for (String paramValue : statsParams) {
       StatsField current = new StatsField(rb, paramValue);
       statsFields.add(current);
-      distribStatsValues.put(current.getOutputKey(), current.buildNewStatsValues());
+      distribStatsValues.put(current.getOutputKey(), 
+                             StatsValuesFactory.createStatsValues(current));
     }
   }
 
@@ -224,200 +208,3 @@ class StatsInfo {
 
 }
 
-/**
- * Models all of the information associated with a single {@link StatsParams#STATS_FIELD}
- * instance.
- */
-class StatsField {
-
-  private final SolrIndexSearcher searcher;
-  private final ResponseBuilder rb;
-  private final String originalParam; // for error messages
-  private final SolrParams localParams;
-  private final SchemaField sf;
-  private final String fieldName;
-  private final String key;
-  private final boolean calcDistinct;
-  private final String[] facets;
-  private final List<String> excludeTagList;
-
-  /**
-   * @param rb the current request/response
-   * @param statsParam the raw {@link StatsParams#STATS_FIELD} string
-   */
-  public StatsField(ResponseBuilder rb, String statsParam) { 
-    this.rb = rb;
-    this.searcher = rb.req.getSearcher();
-    this.originalParam = statsParam;
-
-    SolrParams params = rb.req.getParams();
-
-    try {
-      SolrParams localParams = QueryParsing.getLocalParams(statsParam, params);
-      if (null == localParams) {
-        localParams = new ModifiableSolrParams();
-      }
-      this.localParams = localParams;
-    } catch (SyntaxError e) {
-      throw new SolrException(ErrorCode.BAD_REQUEST, "Unable to parse " + 
-                              StatsParams.STATS_FIELD + ": " + originalParam + " due to: "
-                              + e.getMessage(), e);
-    }
-
-    // pull fieldName out of localParams, or default to original param value
-    this.fieldName = localParams.get(CommonParams.VALUE, statsParam);
-    // allow explicit set of the key via localparams, default to fieldName
-    this.key = localParams.get(CommonParams.OUTPUT_KEY, fieldName);
-
-    calcDistinct = params.getFieldBool(fieldName, StatsParams.STATS_CALC_DISTINCT, false);
-
-    String[] facets = params.getFieldParams(key, StatsParams.STATS_FACET);
-    this.facets = (null == facets) ? new String[0] : facets;
-
-    // figure out if we need a new base DocSet
-    String excludeStr = localParams.get(CommonParams.EXCLUDE);
-    this.excludeTagList = (null == excludeStr) 
-      ? Collections.<String>emptyList()
-      : StrUtils.splitSmart(excludeStr,',');
-
-    this.sf = searcher.getSchema().getField(fieldName);
-  }
-
-  /** 
-   * The key to be used when refering to this {@link StatsField} instance in the 
-   * response tp clients.
-   */
-  public String getOutputKey() {
-    return key;
-  }
-
-  /**
-   * Returns a new, empty, {@link StatsValues} instance that can be used for
-   * accumulating the appropriate stats from this {@link StatsField}
-   */
-  public StatsValues buildNewStatsValues() {
-    return StatsValuesFactory.createStatsValues(sf, calcDistinct);
-  }
-
-  /**
-   * Computes a base {@link DocSet} for the current request to be used
-   * when computing global stats for the local index.
-   *
-   * This is typically the same as the main DocSet for the {@link ResponseBuilder}
-   * unless {@link CommonParams#TAG tag}ged filter queries have been excluded using 
-   * the {@link CommonParams#EXCLUDE ex} local param
-   */
-  public DocSet computeBaseDocSet() throws IOException {
-
-    DocSet docs = rb.getResults().docSet;
-    Map<?,?> tagMap = (Map<?,?>) rb.req.getContext().get("tags");
-
-    if (excludeTagList.isEmpty() || null == tagMap) {
-      // either the exclude list is empty, or there
-      // aren't any tagged filters to exclude anyway.
-      return docs;
-    }
-
-    IdentityHashMap<Query,Boolean> excludeSet = new IdentityHashMap<Query,Boolean>();
-    for (String excludeTag : excludeTagList) {
-      Object olst = tagMap.get(excludeTag);
-      // tagMap has entries of List<String,List<QParser>>, but subject to change in the future
-      if (!(olst instanceof Collection)) continue;
-      for (Object o : (Collection<?>)olst) {
-        if (!(o instanceof QParser)) continue;
-        QParser qp = (QParser)o;
-        try {
-          excludeSet.put(qp.getQuery(), Boolean.TRUE);
-        } catch (SyntaxError e) {
-          // this shouldn't be possible since the request should have already
-          // failed when attempting to execute the query, but just in case...
-          throw new SolrException(ErrorCode.BAD_REQUEST, "Excluded query can't be parsed: " + 
-                                  originalParam + " due to: " + e.getMessage(), e);
-        }
-      }
-    }
-    if (excludeSet.size() == 0) return docs;
-    
-    List<Query> qlist = new ArrayList<Query>();
-    
-    // add the base query
-    if (!excludeSet.containsKey(rb.getQuery())) {
-      qlist.add(rb.getQuery());
-    }
-    
-    // add the filters
-    if (rb.getFilters() != null) {
-      for (Query q : rb.getFilters()) {
-        if (!excludeSet.containsKey(q)) {
-          qlist.add(q);
-        }
-      }
-    }
-    
-    // get the new base docset for this facet
-    return searcher.getDocSet(qlist);
-  }
-
-  /**
-   * Computes the {@link StatsValues} for this {@link StatsField} relative to the 
-   * specified {@link DocSet} 
-   * @see #computeBaseDocSet
-   */
-  public StatsValues computeLocalStatsValues(DocSet base) throws IOException {
-
-    if (sf.multiValued() || sf.getType().multiValuedFieldCache()) {
-      // TODO: should this also be used for single-valued string fields? (should work fine)
-      return DocValuesStats.getCounts(searcher, fieldName, base, calcDistinct, facets);
-    } else {
-      return getFieldCacheStats(base);
-    }
-  }
-
-  private StatsValues getFieldCacheStats(DocSet base) throws IOException {
-    IndexSchema schema = searcher.getSchema();
-    final StatsValues allstats = StatsValuesFactory.createStatsValues(sf, calcDistinct);
-
-    List<FieldFacetStats> facetStats = new ArrayList<>();
-    for( String facetField : facets ) {
-      SchemaField fsf = schema.getField(facetField);
-
-      if ( fsf.multiValued()) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-          "Stats can only facet on single-valued fields, not: " + facetField );
-      }
-
-      facetStats.add(new FieldFacetStats(searcher, facetField, sf, fsf, calcDistinct));
-    }
-
-    final Iterator<AtomicReaderContext> ctxIt = searcher.getIndexReader().leaves().iterator();
-    AtomicReaderContext ctx = null;
-    for (DocIterator docsIt = base.iterator(); docsIt.hasNext(); ) {
-      final int doc = docsIt.nextDoc();
-      if (ctx == null || doc >= ctx.docBase + ctx.reader().maxDoc()) {
-        // advance
-        do {
-          ctx = ctxIt.next();
-        } while (ctx == null || doc >= ctx.docBase + ctx.reader().maxDoc());
-        assert doc >= ctx.docBase;
-
-        // propagate the context among accumulators.
-        allstats.setNextReader(ctx);
-        for (FieldFacetStats f : facetStats) {
-          f.setNextReader(ctx);
-        }
-      }
-
-      // accumulate
-      allstats.accumulate(doc - ctx.docBase);
-      for (FieldFacetStats f : facetStats) {
-        f.facet(doc - ctx.docBase);
-      }
-    }
-
-    for (FieldFacetStats f : facetStats) {
-      allstats.addFacet(f.name, f.facetStatsValues);
-    }
-    return allstats;
-  }
-
-}

Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/StatsField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/StatsField.java?rev=1626856&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/StatsField.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/StatsField.java Mon Sep 22 17:45:53 2014
@@ -0,0 +1,372 @@
+/*
+ * 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.handler.component;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Collection;
+import java.util.IdentityHashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.lucene.search.*;
+import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.queries.function.FunctionQuery;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.queries.function.valuesource.QueryValueSource;
+import org.apache.lucene.queries.function.valuesource.FieldCacheSource;
+
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.StatsParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.request.SolrQueryRequest; // jdocs
+import org.apache.solr.request.DocValuesStats;
+import org.apache.solr.schema.IndexSchema;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.search.DocIterator;
+import org.apache.solr.search.DocSet;
+import org.apache.solr.search.FunctionQParserPlugin;
+import org.apache.solr.search.QParser;
+import org.apache.solr.search.QParserPlugin;
+import org.apache.solr.search.QueryParsing;
+import org.apache.solr.search.SolrIndexSearcher;
+import org.apache.solr.search.SyntaxError;
+
+/**
+ * Models all of the information associated with a single {@link StatsParams#STATS_FIELD}
+ * instance.
+ *
+ * @see StatsComponent
+ */
+public class StatsField {
+
+  private final SolrIndexSearcher searcher;
+  private final ResponseBuilder rb;
+  private final String originalParam; // for error messages
+  private final SolrParams localParams;
+  private final ValueSource valueSource; // may be null if simple field stats
+  private final SchemaField schemaField; // may be null if function/query stats
+  private final String key;
+  private final boolean calcDistinct; // TODO: put this inside localParams ? SOLR-6349 ?
+  private final String[] facets;
+  private final List<String> excludeTagList;
+
+  /**
+   * @param rb the current request/response
+   * @param statsParam the raw {@link StatsParams#STATS_FIELD} string
+   */
+  public StatsField(ResponseBuilder rb, String statsParam) { 
+    this.rb = rb;
+    this.searcher = rb.req.getSearcher();
+    this.originalParam = statsParam;
+
+    SolrParams params = rb.req.getParams();
+    try {
+      SolrParams localParams = QueryParsing.getLocalParams(originalParam, params);
+      if (null == localParams) {
+        // simplest possible input: bare string (field name)
+        ModifiableSolrParams customParams = new ModifiableSolrParams();
+        customParams.add(QueryParsing.V, originalParam);
+        localParams = customParams;
+      }
+      this.localParams = localParams;
+
+      String parserName = localParams.get(QueryParsing.TYPE);
+      SchemaField sf = null;
+      ValueSource vs = null;
+
+      if ( StringUtils.isBlank(parserName) ) {
+
+        // basic request for field stats
+        sf = searcher.getSchema().getField(localParams.get(QueryParsing.V));
+
+      } else {
+        // we have a non trivial request to compute stats over a query (or function)
+
+        // NOTE we could use QParser.getParser(...) here, but that would redundently
+        // reparse everything.  ( TODO: refactor a common method in QParser ?)
+        QParserPlugin qplug = rb.req.getCore().getQueryPlugin(parserName);
+        QParser qp =  qplug.createParser(localParams.get(QueryParsing.V), 
+                                         localParams, params, rb.req);
+
+        // figure out what type of query we are dealing, get the most direct ValueSource
+        vs = extractValueSource(qp.parse());
+
+        // if this ValueSource directly corrisponds to a SchemaField, act as if
+        // we were asked to compute stats on it directly
+        // ie:  "stats.field={!func key=foo}field(foo)" == "stats.field=foo"
+        sf = extractSchemaField(vs, searcher.getSchema());
+        if (null != sf) {
+          vs = null;
+        }
+      }
+      
+      assert ( (null == vs) ^ (null == sf) ) : "exactly one of vs & sf must be null";
+      
+      this.schemaField = sf;
+      this.valueSource = vs;
+
+    } catch (SyntaxError e) {
+      throw new SolrException(ErrorCode.BAD_REQUEST, "Unable to parse " + 
+                              StatsParams.STATS_FIELD + ": " + originalParam + " due to: "
+                              + e.getMessage(), e);
+    }
+
+    // allow explicit setting of the response key via localparams...
+    this.key = localParams.get(CommonParams.OUTPUT_KEY, 
+                               // default to the main param value...
+                               localParams.get(CommonParams.VALUE, 
+                                               // default to entire original param str.
+                                               originalParam));
+
+    
+    this.calcDistinct = null == schemaField
+      ? params.getBool(StatsParams.STATS_CALC_DISTINCT, false) 
+      : params.getFieldBool(schemaField.getName(), StatsParams.STATS_CALC_DISTINCT, false);
+
+    String[] facets = params.getFieldParams(key, StatsParams.STATS_FACET);
+    this.facets = (null == facets) ? new String[0] : facets;
+
+    // figure out if we need a special base DocSet
+    String excludeStr = localParams.get(CommonParams.EXCLUDE);
+    this.excludeTagList = (null == excludeStr) 
+      ? Collections.<String>emptyList()
+      : StrUtils.splitSmart(excludeStr,',');
+
+    assert ( (null == this.valueSource) ^ (null == this.schemaField) ) 
+      : "exactly one of valueSource & schemaField must be null";
+  }
+
+  /**
+   * Inspects a {@link Query} to see if it directly maps to a {@link ValueSource},
+   * and if so returns it -- otherwise wraps it as needed.
+   *
+   * @param q Query whose scores we have been asked to compute stats of
+   * @returns a ValueSource to use for computing the stats
+   */
+  private static ValueSource extractValueSource(Query q) {
+    return (q instanceof FunctionQuery) ?
+      // Common case: we're wrapping a func, so we can directly pull out ValueSource
+      ((FunctionQuery) q).getValueSource() :
+      // asked to compute stats over a query, wrap it up as a ValueSource
+      new QueryValueSource(q, 0.0F);
+  }
+
+  /**
+   * Inspects a {@link ValueSource} to see if it directly maps to a {@link SchemaField}, 
+   * and if so returns it.
+   *
+   * @param vs ValueSource we've been asked to compute stats of
+   * @param schema The Schema to use
+   * @returns Corrisponding {@link SchemaField} or null if the ValueSource is more complex
+   * @see FieldCacheSource
+   */
+  private static SchemaField extractSchemaField(ValueSource vs, IndexSchema schema) {
+    if (vs instanceof FieldCacheSource) {
+      String fieldName = ((FieldCacheSource)vs).getField();
+      return schema.getField(fieldName);
+    }
+    return null;
+  }
+
+  /** 
+   * The key to be used when refering to this {@link StatsField} instance in the 
+   * response tp clients.
+   */
+  public String getOutputKey() {
+    return key;
+  }
+
+  /**
+   * Computes a base {@link DocSet} for the current request to be used
+   * when computing global stats for the local index.
+   *
+   * This is typically the same as the main DocSet for the {@link ResponseBuilder}
+   * unless {@link CommonParams#TAG tag}ged filter queries have been excluded using 
+   * the {@link CommonParams#EXCLUDE ex} local param
+   */
+  public DocSet computeBaseDocSet() throws IOException {
+
+    DocSet docs = rb.getResults().docSet;
+    Map<?,?> tagMap = (Map<?,?>) rb.req.getContext().get("tags");
+
+    if (excludeTagList.isEmpty() || null == tagMap) {
+      // either the exclude list is empty, or there
+      // aren't any tagged filters to exclude anyway.
+      return docs;
+    }
+
+    IdentityHashMap<Query,Boolean> excludeSet = new IdentityHashMap<Query,Boolean>();
+    for (String excludeTag : excludeTagList) {
+      Object olst = tagMap.get(excludeTag);
+      // tagMap has entries of List<String,List<QParser>>, but subject to change in the future
+      if (!(olst instanceof Collection)) continue;
+      for (Object o : (Collection<?>)olst) {
+        if (!(o instanceof QParser)) continue;
+        QParser qp = (QParser)o;
+        try {
+          excludeSet.put(qp.getQuery(), Boolean.TRUE);
+        } catch (SyntaxError e) {
+          // this shouldn't be possible since the request should have already
+          // failed when attempting to execute the query, but just in case...
+          throw new SolrException(ErrorCode.BAD_REQUEST, "Excluded query can't be parsed: " + 
+                                  originalParam + " due to: " + e.getMessage(), e);
+        }
+      }
+    }
+    if (excludeSet.size() == 0) return docs;
+    
+    List<Query> qlist = new ArrayList<Query>();
+    
+    // add the base query
+    if (!excludeSet.containsKey(rb.getQuery())) {
+      qlist.add(rb.getQuery());
+    }
+    
+    // add the filters
+    if (rb.getFilters() != null) {
+      for (Query q : rb.getFilters()) {
+        if (!excludeSet.containsKey(q)) {
+          qlist.add(q);
+        }
+      }
+    }
+    
+    // get the new base docset for this facet
+    return searcher.getDocSet(qlist);
+  }
+
+  /**
+   * Computes the {@link StatsValues} for this {@link StatsField} relative to the 
+   * specified {@link DocSet} 
+   * @see #computeBaseDocSet
+   */
+  public StatsValues computeLocalStatsValues(DocSet base) throws IOException {
+
+    if (null != schemaField 
+        && (schemaField.multiValued() || schemaField.getType().multiValuedFieldCache())) {
+
+      // TODO: should this also be used for single-valued string fields? (should work fine)
+      return DocValuesStats.getCounts(searcher, this, base, facets);
+    } else {
+      // either a single valued field we pull from FieldCache, or an explicit
+      // function ValueSource
+      return computeLocalValueSourceStats(base);
+    }
+  }
+
+  private StatsValues computeLocalValueSourceStats(DocSet base) throws IOException {
+
+    IndexSchema schema = searcher.getSchema();
+
+    final StatsValues allstats = StatsValuesFactory.createStatsValues(this);
+
+    List<FieldFacetStats> facetStats = new ArrayList<>();
+    for( String facetField : facets ) {
+      SchemaField fsf = schema.getField(facetField);
+
+      if ( fsf.multiValued()) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+          "Stats can only facet on single-valued fields, not: " + facetField );
+      }
+
+      facetStats.add(new FieldFacetStats(searcher, fsf, this));
+    }
+
+    final Iterator<AtomicReaderContext> ctxIt = searcher.getIndexReader().leaves().iterator();
+    AtomicReaderContext ctx = null;
+    for (DocIterator docsIt = base.iterator(); docsIt.hasNext(); ) {
+      final int doc = docsIt.nextDoc();
+      if (ctx == null || doc >= ctx.docBase + ctx.reader().maxDoc()) {
+        // advance
+        do {
+          ctx = ctxIt.next();
+        } while (ctx == null || doc >= ctx.docBase + ctx.reader().maxDoc());
+        assert doc >= ctx.docBase;
+
+        // propagate the context among accumulators.
+        allstats.setNextReader(ctx);
+        for (FieldFacetStats f : facetStats) {
+          f.setNextReader(ctx);
+        }
+      }
+
+      // accumulate
+      allstats.accumulate(doc - ctx.docBase);
+      for (FieldFacetStats f : facetStats) {
+        f.facet(doc - ctx.docBase);
+      }
+    }
+
+    for (FieldFacetStats f : facetStats) {
+      allstats.addFacet(f.name, f.facetStatsValues);
+    }
+    return allstats;
+  }
+
+  /**
+   * The searcher that should be used for processing local stats
+   * @see SolrQueryRequest#getSearcher
+   */
+  public SolrIndexSearcher getSearcher() {
+    // see AbstractStatsValues.setNextReader
+
+    return searcher;
+  }
+
+  /**
+   * The {@link SchemaField} whose results these stats are computed over, may be null 
+   * if the stats are computed over the results of a function or query
+   *
+   * @see #getValueSource
+   */
+  public SchemaField getSchemaField() {
+    return schemaField;
+  }
+
+  /**
+   * The {@link ValueSource} of a function or query whose results these stats are computed 
+   * over, may be null if the stats are directly over a {@link SchemaField}
+   *
+   * @see #getValueSource
+   */
+  public ValueSource getValueSource() {
+    return valueSource;
+  }
+
+  /**
+   * Wether or not the effective value of the {@link StatsParams#STATS_CALC_DISTINCT} param
+   * is true or false for this StatsField
+   */
+  public boolean getCalcDistinct() {
+    return calcDistinct;
+  }
+
+  public String toString() {
+    return "StatsField<" + originalParam + ">";
+  }
+
+}

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java?rev=1626856&r1=1626855&r2=1626856&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java Mon Sep 22 17:45:53 2014
@@ -36,22 +36,30 @@ import org.apache.solr.schema.*;
 public class StatsValuesFactory {
 
   /**
-   * Creates an instance of StatsValues which supports values from a field of the given FieldType
+   * Creates an instance of StatsValues which supports values from the specified {@link StatsField}
    *
-   * @param sf SchemaField for the field whose statistics will be created by the resulting StatsValues
-   * @return Instance of StatsValues that will create statistics from values from a field of the given type
+   * @param statsField {@link StatsField} whose statistics will be created by the resulting {@link StatsValues}
+   * @return Instance of {@link StatsValues} that will create statistics from values from the specified {@link StatsField}
    */
-  public static StatsValues createStatsValues(SchemaField sf, boolean calcDistinct) {
-    // TODO: allow for custom field types
-    FieldType fieldType = sf.getType();
+  public static StatsValues createStatsValues(StatsField statsField) {
+
+    final SchemaField sf = statsField.getSchemaField();
+
+    if (null == sf) {
+      // function stats
+      return new NumericStatsValues(statsField);
+    } 
+
+    final FieldType fieldType = sf.getType(); // TODO: allow FieldType to provide impl.
+    
     if (TrieDateField.class.isInstance(fieldType)) {
-      return new DateStatsValues(sf, calcDistinct);
+      return new DateStatsValues(statsField);
     } else if (TrieField.class.isInstance(fieldType)) {
-      return new NumericStatsValues(sf, calcDistinct);
+      return new NumericStatsValues(statsField);
     } else if (StrField.class.isInstance(fieldType)) {
-      return new StringStatsValues(sf, calcDistinct);
+      return new StringStatsValues(statsField);
     } else if (sf.getType().getClass().equals(EnumField.class)) {
-      return new EnumStatsValues(sf, calcDistinct);
+      return new EnumStatsValues(statsField);
     } else {
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Field type " + fieldType + " is not currently supported");
     }
@@ -59,34 +67,81 @@ public class StatsValuesFactory {
 }
 
 /**
- * Abstract implementation of {@link org.apache.solr.handler.component.StatsValues} that provides the default behavior
- * for most StatsValues implementations.
+ * Abstract implementation of {@link org.apache.solr.handler.component.StatsValues} 
+ * that provides the default behavior for most StatsValues implementations.
  *
- * There are very few requirements placed on what statistics concrete implementations should collect, with the only required
- * statistics being the minimum and maximum values.
+ * There are very few requirements placed on what statistics concrete implementations 
+ * should collect, with the only required statistics being the minimum and maximum values.
  */
 abstract class AbstractStatsValues<T> implements StatsValues {
   private static final String FACETS = "facets";
+
+  /** Tracks all data about tthe stats we need to collect */
+  final protected StatsField statsField;
+
+  /** 
+   * local copy to save method dispatch in tight loops 
+   * @see StatsField#getCalcDistinct
+   */
+  final protected boolean calcDistinct;
+
+  /** may be null if we are collecting stats directly from a function ValueSource */
   final protected SchemaField sf;
+  /** may be null if we are collecting stats directly from a function ValueSource */
   final protected FieldType ft;
+
+  /** 
+   * Either a function value source to collect from, or the ValueSource associated 
+   * with a single valued field we are collecting from.  Will be null until/unless 
+   * {@link #setNextReader} is called at least once
+   */
+  private ValueSource valueSource;
+  /** 
+   * Context to use when retrieving FunctionValues, will be null until/unless 
+   * {@link #setNextReader} is called at least once
+   */
+  private Map vsContext;
+  /** 
+   * Values to collect, will be null until/unless {@link #setNextReader} is called 
+   * at least once 
+   */
+  protected FunctionValues values;
+
   protected T max;
   protected T min;
   protected long missing;
   protected long count;
   protected long countDistinct;
   protected Set<T> distinctValues;
-  private ValueSource valueSource;
-  protected FunctionValues values;
-  protected boolean calcDistinct = false;
   
   // facetField   facetValue
   protected Map<String, Map<String, StatsValues>> facets = new HashMap<>();
 
-  protected AbstractStatsValues(SchemaField sf, boolean calcDistinct) {
-    this.sf = sf;
-    this.ft = sf.getType();
+  protected AbstractStatsValues(StatsField statsField) {
+    this.statsField = statsField;
+    this.calcDistinct = statsField.getCalcDistinct();
     this.distinctValues = new TreeSet<>();
-    this.calcDistinct = calcDistinct;
+
+    // alternatively, we could refactor a common base class that doesn't know/care
+    // about either SchemaField or ValueSource - but then there would be a lot of
+    // duplicate code between "NumericSchemaFieldStatsValues" and 
+    // "NumericValueSourceStatsValues" which would have diff parent classes
+    //
+    // part of the complexity here being that the StatsValues API serves two 
+    // masters: collecting concrete Values from things like DocValuesStats and 
+    // the distributed aggregation logic, but also collecting docIds which it then
+    // uses to go out and pull concreate values from the ValueSource 
+    // (from a func, or single valued field)
+    if (null != statsField.getSchemaField()) {
+      assert null == statsField.getValueSource();
+      this.sf = statsField.getSchemaField();
+      this.ft = sf.getType();
+    } else {
+      assert null != statsField.getValueSource();
+      assert null == statsField.getSchemaField();
+      this.sf = null;
+      this.ft = null;
+    }
   }
 
   /**
@@ -121,7 +176,7 @@ abstract class AbstractStatsValues<T> im
         String val = vals.getName(j);
         StatsValues vvals = addTo.get(val);
         if (vvals == null) {
-          vvals = StatsValuesFactory.createStatsValues(sf, calcDistinct);
+          vvals = StatsValuesFactory.createStatsValues(statsField);
           addTo.put(val, vvals);
         }
         vvals.accumulate((NamedList) vals.getVal(j));
@@ -134,11 +189,14 @@ abstract class AbstractStatsValues<T> im
    */
   @Override
   public void accumulate(BytesRef value, int count) {
+    if (null == ft) {
+      throw new IllegalStateException("Can't collect & convert BytesRefs on stats that do't use a a FieldType: " + statsField);
+    }
     T typedValue = (T)ft.toObject(sf, value);
     accumulate(typedValue, count);
   }
 
-  public void accumulate(T value, int count) {
+  public void accumulate(T value, int count) { 
     this.count += count;
     if (calcDistinct) {
       distinctValues.add(value);
@@ -203,11 +261,18 @@ abstract class AbstractStatsValues<T> im
     return res;
   }
 
-  public void setNextReader(AtomicReaderContext ctx) throws IOException {
+  /**
+   * {@inheritDoc}
+   */
+   public void setNextReader(AtomicReaderContext ctx) throws IOException {
     if (valueSource == null) {
-      valueSource = ft.getValueSource(sf, null);
+      // first time we've collected local values, get the right ValueSource
+      valueSource = (null == ft) 
+        ? statsField.getValueSource()
+        : ft.getValueSource(sf, null);
+      vsContext = ValueSource.newContext(statsField.getSearcher());
     }
-    values = valueSource.getValues(Collections.emptyMap(), ctx);
+    values = valueSource.getValues(vsContext, ctx);
   }
 
   /**
@@ -249,8 +314,8 @@ class NumericStatsValues extends Abstrac
   double sum;
   double sumOfSquares;
 
-  public NumericStatsValues(SchemaField sf, boolean calcDistinct) {
-    super(sf, calcDistinct);
+  public NumericStatsValues(StatsField statsField) {
+    super(statsField);
     min = Double.POSITIVE_INFINITY;
     max = Double.NEGATIVE_INFINITY;
   }
@@ -324,8 +389,8 @@ class NumericStatsValues extends Abstrac
  */
 class EnumStatsValues extends AbstractStatsValues<EnumFieldValue> {
 
-  public EnumStatsValues(SchemaField sf, boolean calcDistinct) {
-    super(sf, calcDistinct);
+  public EnumStatsValues(StatsField statsField) {
+    super(statsField);
   }
 
   /**
@@ -393,8 +458,8 @@ class DateStatsValues extends AbstractSt
   private long sum = 0;
   double sumOfSquares = 0;
 
-  public DateStatsValues(SchemaField sf, boolean calcDistinct) {
-    super(sf, calcDistinct);
+  public DateStatsValues(StatsField statsField) {
+    super(statsField);
   }
 
   @Override
@@ -479,8 +544,8 @@ class DateStatsValues extends AbstractSt
  */
 class StringStatsValues extends AbstractStatsValues<String> {
 
-  public StringStatsValues(SchemaField sf, boolean calcDistinct) {
-    super(sf, calcDistinct);
+  public StringStatsValues(StatsField statsField) {
+    super(statsField);
   }
 
   @Override

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/request/DocValuesStats.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/request/DocValuesStats.java?rev=1626856&r1=1626855&r2=1626856&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/request/DocValuesStats.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/request/DocValuesStats.java Mon Sep 22 17:45:53 2014
@@ -35,6 +35,7 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LongValues;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.handler.component.FieldFacetStats;
+import org.apache.solr.handler.component.StatsField;
 import org.apache.solr.handler.component.StatsValues;
 import org.apache.solr.handler.component.StatsValuesFactory;
 import org.apache.solr.schema.FieldType;
@@ -52,10 +53,17 @@ import org.apache.solr.search.SolrIndexS
 public class DocValuesStats {
   private DocValuesStats() {}
   
-  public static StatsValues getCounts(SolrIndexSearcher searcher, String fieldName, DocSet docs, boolean calcDistinct, String[] facet) throws IOException {
-    SchemaField schemaField = searcher.getSchema().getField(fieldName);
-    FieldType ft = schemaField.getType();
-    StatsValues res = StatsValuesFactory.createStatsValues(schemaField, calcDistinct);
+  public static StatsValues getCounts(SolrIndexSearcher searcher, StatsField statsField, DocSet docs, String[] facet) throws IOException {
+
+    final SchemaField schemaField = statsField.getSchemaField(); 
+    final boolean calcDistinct = statsField.getCalcDistinct();
+
+    assert null != statsField.getSchemaField()
+      : "DocValuesStats requires a StatsField using a SchemaField";
+
+    final String fieldName = schemaField.getName();
+    final FieldType ft = schemaField.getType();
+    final StatsValues res = StatsValuesFactory.createStatsValues(statsField);
     
     //Initialize facetstats, if facets have been passed in
     final FieldFacetStats[] facetStats = new FieldFacetStats[facet.length];
@@ -69,7 +77,7 @@ public class DocValuesStats {
       }
       
       SchemaField facetSchemaField = searcher.getSchema().getField(facetField);
-      facetStats[upto++] = new FieldFacetStats(searcher, facetField, schemaField, facetSchemaField, calcDistinct);
+      facetStats[upto++] = new FieldFacetStats(searcher, facetSchemaField, statsField);
     }
     // TODO: remove multiValuedFieldCache(), check dv type / uninversion type?
     final boolean multiValued = schemaField.multiValued() || ft.multiValuedFieldCache();

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/request/UnInvertedField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/request/UnInvertedField.java?rev=1626856&r1=1626855&r2=1626856&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/request/UnInvertedField.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/request/UnInvertedField.java Mon Sep 22 17:45:53 2014
@@ -40,6 +40,7 @@ import org.apache.solr.common.params.Fac
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.handler.component.FieldFacetStats;
+import org.apache.solr.handler.component.StatsField;
 import org.apache.solr.handler.component.StatsValues;
 import org.apache.solr.handler.component.StatsValuesFactory;
 import org.apache.solr.schema.FieldType;
@@ -467,22 +468,24 @@ public class UnInvertedField extends Doc
    *
    * @param searcher The Searcher to use to gather the statistics
    * @param baseDocs The {@link org.apache.solr.search.DocSet} to gather the stats on
-   * @param calcDistinct whether distinct values should be collected and counted
+   * @param statsField the {@link StatsField} param corrisponding to a real {@link SchemaField} to compute stats over
    * @param facet One or more fields to facet on.
    * @return The {@link org.apache.solr.handler.component.StatsValues} collected
    * @throws IOException If there is a low-level I/O error.
    */
-  public StatsValues getStats(SolrIndexSearcher searcher, DocSet baseDocs, boolean calcDistinct, String[] facet) throws IOException {
+  public StatsValues getStats(SolrIndexSearcher searcher, DocSet baseDocs, StatsField statsField, String[] facet) throws IOException {
     //this function is ripped off nearly wholesale from the getCounts function to use
     //for multiValued fields within the StatsComponent.  may be useful to find common
     //functionality between the two and refactor code somewhat
     use.incrementAndGet();
 
-    SchemaField sf = searcher.getSchema().getField(field);
-   // FieldType ft = sf.getType();
+    assert null != statsField.getSchemaField()
+      : "DocValuesStats requires a StatsField using a SchemaField";
 
-    StatsValues allstats = StatsValuesFactory.createStatsValues(sf, calcDistinct);
+    SchemaField sf = statsField.getSchemaField();
+    // FieldType ft = sf.getType();
 
+    StatsValues allstats = StatsValuesFactory.createStatsValues(statsField);
 
     DocSet docs = baseDocs;
     int baseSize = docs.size();
@@ -498,7 +501,7 @@ public class UnInvertedField extends Doc
     SortedDocValues si;
     for (String f : facet) {
       SchemaField facet_sf = searcher.getSchema().getField(f);
-      finfo[i] = new FieldFacetStats(searcher, f, sf, facet_sf, calcDistinct);
+      finfo[i] = new FieldFacetStats(searcher, facet_sf, statsField);
       i++;
     }
 

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/TestDistributedSearch.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/TestDistributedSearch.java?rev=1626856&r1=1626855&r2=1626856&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/TestDistributedSearch.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/TestDistributedSearch.java Mon Sep 22 17:45:53 2014
@@ -375,6 +375,12 @@ public class TestDistributedSearch exten
           "stats.field", "{!ex=xxx}stats_dt");
 
     query("q","*:*", "sort",i1+" desc", "stats", "true",
+          // do a really simple query so distributed IDF doesn't cause problems
+          // when comparing with control collection
+          "stats.field", "{!lucene key=q_key}" + i1 + "foo_b:true",
+          "stats.field", "{!func key=f_key}sum(" + tlong +","+i1+")");
+
+    query("q","*:*", "sort",i1+" desc", "stats", "true",
           "stats.field", "stats_dt",
           "stats.field", i1,
           "stats.field", tdate_a,

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/component/StatsComponentTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/component/StatsComponentTest.java?rev=1626856&r1=1626855&r2=1626856&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/component/StatsComponentTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/component/StatsComponentTest.java Mon Sep 22 17:45:53 2014
@@ -27,7 +27,11 @@ import java.util.Locale;
 import java.util.Map;
 import java.util.TimeZone;
 
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.queries.function.valuesource.QueryValueSource;
+
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.MapSolrParams;
 import org.apache.solr.common.params.SolrParams;
@@ -35,6 +39,7 @@ import org.apache.solr.common.params.Sta
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.util.AbstractSolrTestCase;
 import org.junit.BeforeClass;
@@ -183,15 +188,72 @@ public class StatsComponentTest extends 
                 , kpre + "double[@name='mean'][.='-25.0']"
                 , kpre + "double[@name='stddev'][.='12.909944487358056']"
 
-                );  
-
-        
+                );
 
       }
     }
 
+    // we should be able to compute exact same stats for a field even
+    // when we specify it using the "field()" function, or use other 
+    // identify equivilent functions
+    for (String param : new String[] {
+        // bare
+        "{!key="+key+" ex=key_ex_tag}" + f,
+        "{!key="+key+" ex=key_ex_tag v="+f+"}",
+        // field func
+        "{!lucene key="+key+" ex=key_ex_tag}_val_:\"field("+f+")\"",
+        "{!func key="+key+" ex=key_ex_tag}field("+f+")",
+        "{!type=func key="+key+" ex=key_ex_tag}field("+f+")",
+        "{!type=func key="+key+" ex=key_ex_tag v=field("+f+")}",
+        "{!type=func key="+key+" ex=key_ex_tag v='field("+f+")'}",
+        // identity math functions
+        "{!type=func key="+key+" ex=key_ex_tag v='sum(0,"+f+")'}",
+        "{!type=func key="+key+" ex=key_ex_tag v='product(1,"+f+")'}",
+      }) {
+      
+      assertQ("test statistics over field specified as a function: " + param,
+              // NOTE: baseParams aren't used, we're looking at the function
+              req("q", "*:*", "stats", "true", "stats.calcdistinct", "true",
+                  "fq", "{!tag=key_ex_tag}-id:4", 
+                  "stats.field", param)
+              
+              , kpre + "double[@name='min'][.='-40.0']"
+              , kpre + "double[@name='max'][.='-10.0']"
+              , kpre + "double[@name='sum'][.='-100.0']"
+              , kpre + "long[@name='count'][.='4']"
+              , kpre + "long[@name='missing'][.='0']"
+              , kpre + "long[@name='countDistinct'][.='4']"
+              , "count(" + kpre + "arr[@name='distinctValues']/*)=4"
+              , kpre + "double[@name='sumOfSquares'][.='3000.0']"
+              , kpre + "double[@name='mean'][.='-25.0']"
+              , kpre + "double[@name='stddev'][.='12.909944487358056']"
+              
+              );
+    }
+    
+    // now get stats over a non-trivial function on our (single) field
+    String func = "product(2, " + f + ")";
+    assertQ("test function statistics & key override", 
+            // NOTE: baseParams aren't used, we're looking at the function
+            req("q", "*:*", "stats", "true", "stats.calcdistinct", "true",
+                "fq", "{!tag=key_ex_tag}-id:4", 
+                "stats.field", "{!func key="+key+" ex=key_ex_tag}"+func)
+
+            , kpre + "double[@name='min'][.='-80.0']"
+            , kpre + "double[@name='max'][.='-20.0']"
+            , kpre + "double[@name='sum'][.='-200.0']"
+            , kpre + "long[@name='count'][.='4']"
+            , kpre + "long[@name='missing'][.='0']"
+            , kpre + "long[@name='countDistinct'][.='4']"
+            , "count(" + kpre + "arr[@name='distinctValues']/*)=4"
+            , kpre + "double[@name='sumOfSquares'][.='12000.0']"
+            , kpre + "double[@name='mean'][.='-50.0']" 
+            , kpre + "double[@name='stddev'][.='25.81988897471611']"
+            );
   }
 
+  
+
 
   public void doTestMVFieldStatisticsResult(String f) throws Exception {
     assertU(adoc("id", "1", f, "-10", f, "-100", "active_s", "true"));
@@ -288,6 +350,7 @@ public class StatsComponentTest extends 
               , "//lst[@name='false']/double[@name='stddev'][.='23.59908190304586']"
               );
     }
+
   }
 
   public void testFieldStatisticsResultsStringField() throws Exception {
@@ -313,6 +376,20 @@ public class StatsComponentTest extends 
             "//long[@name='missing'][.='1']",
             "//long[@name='countDistinct'][.='3']",
             "count(//arr[@name='distinctValues']/str)=3");
+
+    // stats over a string function
+    assertQ("strdist func stats",
+            req("q", "*:*",
+                "fq", "-id:4", // SOLR-6540
+                "stats","true",
+                "stats.field","{!func}strdist('string22',active_s,edit)")
+            , "//double[@name='min'][.='0.75']"
+            , "//double[@name='max'][.='0.875']"
+            , "//double[@name='sum'][.='2.375']"
+            , "//long[@name='count'][.='3']"
+            ,"//long[@name='missing'][.='0']" // SOLR-6540 ==> '1'
+            );
+
   }
 
   public void testFieldStatisticsResultsDateField() throws Exception {
@@ -358,6 +435,10 @@ public class StatsComponentTest extends 
     assertU(adoc("id", "4", f, "-40"));
     assertU(commit());
 
+    final String fpre = XPRE + "lst[@name='stats_fields']/lst[@name='"+f+"']/";
+    final String key = "key_key";
+    final String kpre = XPRE + "lst[@name='stats_fields']/lst[@name='"+key+"']/";
+
     // status should be the same regardless of baseParams
     for (SolrParams baseParams : baseParamsSet) {
 
@@ -376,6 +457,46 @@ public class StatsComponentTest extends 
               , "//double[@name='stddev'][.='15.275252316519467']"
               );
     }
+
+    // we should be able to compute exact same stats for a field even
+    // when we specify it using the "field()" function, or use other 
+    // identify equivilent functions
+    for (String param : new String[] {
+        // bare
+        "{!key="+key+" ex=key_ex_tag}" + f,
+        "{!key="+key+" ex=key_ex_tag v="+f+"}",
+        // field func
+        "{!lucene key="+key+" ex=key_ex_tag}_val_:\"field("+f+")\"",
+        "{!func key="+key+" ex=key_ex_tag}field("+f+")",
+        "{!type=func key="+key+" ex=key_ex_tag}field("+f+")",
+        "{!type=func key="+key+" ex=key_ex_tag v=field("+f+")}",
+        "{!type=func key="+key+" ex=key_ex_tag v='field("+f+")'}",
+
+        // identity math functions don't work as expected due to LUCENE-5961
+        // "{!type=func key="+key+" ex=key_ex_tag v='sum(0,"+f+")'}",
+        // "{!type=func key="+key+" ex=key_ex_tag v='product(1,"+f+")'}",
+      }) {
+      
+      assertQ("test statistics over field specified as a function: " + param,
+              // NOTE: baseParams aren't used, we're looking at the function
+              req("q", "*:*", "stats", "true", "stats.calcdistinct", "true",
+                  "fq", "{!tag=key_ex_tag}-id:4", 
+                  "stats.field", param)
+              
+              , kpre + "double[@name='min'][.='-40.0']"
+              , kpre + "double[@name='max'][.='-10.0']"
+              , kpre + "double[@name='sum'][.='-70.0']"
+              , kpre + "long[@name='count'][.='3']"
+              , kpre + "long[@name='missing'][.='1']"
+              , kpre + "long[@name='countDistinct'][.='3']"
+              , "count(" + kpre + "arr[@name='distinctValues']/*)=3"
+              , kpre + "double[@name='sumOfSquares'][.='2100.0']"
+              , kpre + "double[@name='mean'][.='-23.333333333333332']"
+              , kpre + "double[@name='stddev'][.='15.275252316519467']"
+              
+              );
+    }
+
   }
 
   public void doTestFacetStatisticsResult(String f, SolrParams[] baseParamsSet) throws Exception {
@@ -422,6 +543,54 @@ public class StatsComponentTest extends 
               , pre+"/lst[@name='false']/double[@name='stddev'][.='7.0710678118654755']"
               );
     }
+
+    // we should be able to compute exact same stats & stats.facet for a field even
+    // when we specify it using the "field()" function, or use other 
+    // identify equivilent functions
+    for (String param : new String[] {
+        // bare
+        "{!key="+f+" ex=key_ex_tag}" + f,
+        "{!key="+f+" ex=key_ex_tag v="+f+"}",
+        // field func
+        "{!lucene key="+f+" ex=key_ex_tag}_val_:\"field("+f+")\"",
+        "{!func key="+f+" ex=key_ex_tag}field("+f+")",
+        "{!type=func key="+f+" ex=key_ex_tag}field("+f+")",
+        "{!type=func key="+f+" ex=key_ex_tag v=field("+f+")}",
+        "{!type=func key="+f+" ex=key_ex_tag v='field("+f+")'}",
+
+        // identity math functions don't work as expected due to LUCENE-5961
+        // "{!type=func key="+f+" ex=key_ex_tag v='sum(0,"+f+")'}",
+        // "{!type=func key="+f+" ex=key_ex_tag v='product(1,"+f+")'}",
+      }) {
+      assertQ("test statis & stats.facet over field specified as a function: " + param,
+              req("q", "*:*", "stats", "true", "stats.calcdistinct", "true",
+                  "fq", "{!tag=key_ex_tag}-id:4", 
+                  "stats.field", param,
+                  "stats.facet", "active_s", "stats.facet", "other_s")
+              , "*[count("+pre+")=1]"
+              , pre+"/lst[@name='true']/double[@name='min'][.='10.0']"
+              , pre+"/lst[@name='true']/double[@name='max'][.='20.0']"
+              , pre+"/lst[@name='true']/double[@name='sum'][.='30.0']"
+              , pre+"/lst[@name='true']/long[@name='count'][.='2']"
+              , pre+"/lst[@name='true']/long[@name='missing'][.='0']"
+              , pre + "/lst[@name='true']/long[@name='countDistinct'][.='2']"
+              , "count(" + pre + "/lst[@name='true']/arr[@name='distinctValues']/*)=2"
+              , pre+"/lst[@name='true']/double[@name='sumOfSquares'][.='500.0']"
+              , pre+"/lst[@name='true']/double[@name='mean'][.='15.0']"
+              , pre+"/lst[@name='true']/double[@name='stddev'][.='7.0710678118654755']"
+              //
+              , pre+"/lst[@name='false']/double[@name='min'][.='30.0']"
+              , pre+"/lst[@name='false']/double[@name='max'][.='40.0']"
+              , pre+"/lst[@name='false']/double[@name='sum'][.='70.0']"
+              , pre+"/lst[@name='false']/long[@name='count'][.='2']"
+              , pre+"/lst[@name='false']/long[@name='missing'][.='0']"
+              , pre + "/lst[@name='true']/long[@name='countDistinct'][.='2']"
+              , "count(" + pre + "/lst[@name='true']/arr[@name='distinctValues']/*)=2"
+              , pre+"/lst[@name='false']/double[@name='sumOfSquares'][.='2500.0']"
+              , pre+"/lst[@name='false']/double[@name='mean'][.='35.0']"
+              , pre+"/lst[@name='false']/double[@name='stddev'][.='7.0710678118654755']"
+              );
+    }
   }
   
   public void doTestFacetStatisticsMissingResult(String f, SolrParams[] baseParamsSet) throws Exception {
@@ -463,6 +632,7 @@ public class StatsComponentTest extends 
               , "//lst[@name='false']/double[@name='stddev'][.='0.0']"
               );
     }
+
   }
 
   public void testFieldStatisticsResultsNumericFieldAlwaysMissing() throws Exception {
@@ -704,6 +874,98 @@ public class StatsComponentTest extends 
      }
 
 
+  public void testMiscQueryStats() throws Exception {
+    final String kpre = XPRE + "lst[@name='stats_fields']/lst[@name='k']/";
+
+    assertU(adoc("id", "1", "a_f", "2.3", "b_f", "9.7", "foo_t", "how now brown cow"));
+    assertU(adoc("id", "2", "a_f", "4.5", "b_f", "8.6", "foo_t", "cow cow cow cow"));
+    assertU(adoc("id", "3", "a_f", "5.6", "b_f", "7.5", "foo_t", "red fox"));
+    assertU(adoc("id", "4", "a_f", "6.7", "b_f", "6.3", "foo_t", "red cow"));
+    assertU(commit());
+
+    assertQ("functions over multiple fields",
+            req("q","foo_t:cow", "stats", "true",
+                "stats.field", "{!func key=k}product(a_f,b_f)")
+            
+            , kpre + "double[@name='min'][.='22.309999465942383']"
+            , kpre + "double[@name='max'][.='42.209999084472656']"
+            , kpre + "double[@name='sum'][.='103.21999931335449']"
+            , kpre + "long[@name='count'][.='3']"
+            , kpre + "long[@name='missing'][.='0']"
+            , kpre + "double[@name='sumOfSquares'][.='3777.110157933046']"
+            , kpre + "double[@name='mean'][.='34.40666643778483']"
+            , kpre + "double[@name='stddev'][.='10.622007151430441']"
+            );
+
+    assertQ("functions over a query",
+            req("q","*:*", "stats", "true",
+                "stats.field", "{!lucene key=k}foo_t:cow")
+            // scores are: 1.0, 0.625, 0.5, & "missing"
+            , kpre + "double[@name='min'][.='0.5']"
+            , kpre + "double[@name='max'][.='1.0']"
+            , kpre + "double[@name='sum'][.='2.125']"
+            , kpre + "long[@name='count'][.='3']"
+            , kpre + "long[@name='missing'][.='1']"
+            , kpre + "double[@name='sumOfSquares'][.='1.640625']"
+            , kpre + "double[@name='mean'][.='0.7083333333333334']"
+            , kpre + "double[@name='stddev'][.='0.2602082499332666']"
+            );
+    
+  }
+
+  /**
+   * Whitebox test of {@link StatsField} parsing to ensure expected equivilence 
+   * operations hold up
+   */
+  public void testStatsFieldWhitebox() throws Exception {
+    StatsComponent component = new StatsComponent();
+    List<SearchComponent> components = new ArrayList<>(1);
+    components.add(component);
+    SolrParams common = params("stats", "true", "q", "*:*", "nested","foo_t:cow");
+
+    // all of these should produce the same SchemaField based StatsField
+    for (String param : new String[] { 
+        "foo_i", "{!func}field(\"foo_i\")", "{!lucene}_val_:\"field(foo_i)\""
+      }) {
+      SolrQueryRequest req = req(common);
+      try {
+        ResponseBuilder rb = new ResponseBuilder(req, new SolrQueryResponse(), components);
+        
+        StatsField sf = new StatsField(rb, param);
+        
+        assertNull("value source of: " + param, sf.getValueSource());
+        assertNotNull("schema field of: " + param, sf.getSchemaField());
+
+        assertEquals("field name of: " + param,
+                     "foo_i", sf.getSchemaField().getName());
+      } finally {
+        req.close();
+      }
+    }
+
+    // all of these should produce the same QueryValueSource based StatsField
+    for (String param : new String[] { 
+        "{!lucene}foo_t:cow", "{!func}query($nested)", "{!field f=foo_t}cow", 
+      }) {
+      SolrQueryRequest req = req(common);
+      try {
+        ResponseBuilder rb = new ResponseBuilder(req, new SolrQueryResponse(), components);
+        
+        StatsField sf = new StatsField(rb, param);
+        
+        assertNull("schema field of: " + param, sf.getSchemaField());
+        assertNotNull("value source of: " + param, sf.getValueSource());
+        assertTrue(sf.getValueSource().getClass() + " is vs type of: " + param,
+                   sf.getValueSource() instanceof QueryValueSource);
+        QueryValueSource qvs = (QueryValueSource) sf.getValueSource();
+        assertEquals("query of :" + param,
+                     new TermQuery(new Term("foo_t","cow")),
+                     qvs.getQuery());
+      } finally {
+        req.close();
+      }
+    }
+  }
 
   public void testFieldStatisticsDocValuesAndMultiValuedDouble() throws Exception {
     SolrCore core = h.getCore();