You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ro...@apache.org on 2017/01/07 12:04:35 UTC

[1/6] lucene-solr:master: LUCENE-7609: Refactor expressions module to use DoubleValuesSource

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_6x d4d3ede51 -> d268055ca
  refs/heads/master da30f21f5 -> 1a95c5acd


LUCENE-7609: Refactor expressions module to use DoubleValuesSource


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/8b055382
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/8b055382
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/8b055382

Branch: refs/heads/master
Commit: 8b055382d6c88acaed9fe472a038c7ee6b35c016
Parents: da30f21
Author: Alan Woodward <ro...@apache.org>
Authored: Thu Jan 5 13:15:19 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Sat Jan 7 11:05:06 2017 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   5 +
 .../ExpressionAggregationFacetsExample.java     |   2 +-
 lucene/expressions/build.xml                    |  11 --
 .../org/apache/lucene/expressions/Bindings.java |  14 +--
 .../apache/lucene/expressions/Expression.java   |  21 ++--
 .../expressions/ExpressionComparator.java       | 100 -----------------
 .../expressions/ExpressionFunctionValues.java   |  35 +++---
 .../lucene/expressions/ExpressionRescorer.java  |  33 ++++--
 .../lucene/expressions/ExpressionSortField.java |  77 -------------
 .../expressions/ExpressionValueSource.java      |  70 ++++++------
 .../lucene/expressions/ScoreFunctionValues.java |  46 --------
 .../lucene/expressions/ScoreValueSource.java    |  61 ----------
 .../lucene/expressions/SimpleBindings.java      |  30 +++--
 .../expressions/js/JavascriptCompiler.java      |  15 ++-
 .../apache/lucene/expressions/package-info.java |   5 +-
 .../lucene/expressions/TestDemoExpressions.java |  44 ++++++--
 .../expressions/TestExpressionSortField.java    |   2 +-
 .../expressions/TestExpressionValueSource.java  | 111 ++++++-------------
 .../expressions/js/TestCustomFunctions.java     |  20 ++--
 .../expressions/js/TestJavascriptFunction.java  |   2 +-
 .../js/TestJavascriptOperations.java            |   2 +-
 21 files changed, 197 insertions(+), 509 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index fa5cc1c..c667040 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -74,6 +74,11 @@ API Changes
   grouping Collectors are renamed to remove the Abstract* prefix.
   (Alan Woodward, Martijn van Groningen)
 
+* LUCENE-7609: The expressions module now uses the DoubleValuesSource API, and 
+  no longer depends on the queries module.  Expression#getValueSource() is
+  replaced with Expression#getDoubleValuesSource(). (Alan Woodward, Adrien
+  Grand)
+
 New features
 
 * LUCENE-5867: Added BooleanSimilarity. (Robert Muir, Adrien Grand)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/demo/src/java/org/apache/lucene/demo/facet/ExpressionAggregationFacetsExample.java
----------------------------------------------------------------------
diff --git a/lucene/demo/src/java/org/apache/lucene/demo/facet/ExpressionAggregationFacetsExample.java b/lucene/demo/src/java/org/apache/lucene/demo/facet/ExpressionAggregationFacetsExample.java
index fca7b6c..7f29b38 100644
--- a/lucene/demo/src/java/org/apache/lucene/demo/facet/ExpressionAggregationFacetsExample.java
+++ b/lucene/demo/src/java/org/apache/lucene/demo/facet/ExpressionAggregationFacetsExample.java
@@ -103,7 +103,7 @@ public class ExpressionAggregationFacetsExample {
     FacetsCollector.search(searcher, new MatchAllDocsQuery(), 10, fc);
 
     // Retrieve results
-    Facets facets = new TaxonomyFacetSumValueSource(taxoReader, config, fc, expr.getValueSource(bindings));
+    Facets facets = new TaxonomyFacetSumValueSource(taxoReader, config, fc, expr.getDoubleValuesSource(bindings));
     FacetResult result = facets.getTopChildren(10, "A");
     
     indexReader.close();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/build.xml
----------------------------------------------------------------------
diff --git a/lucene/expressions/build.xml b/lucene/expressions/build.xml
index 1dddc6d..61ae64f 100644
--- a/lucene/expressions/build.xml
+++ b/lucene/expressions/build.xml
@@ -26,7 +26,6 @@
   <path id="classpath">
     <path refid="base.classpath"/>
     <fileset dir="lib"/>
-    <pathelement path="${queries.jar}"/>
   </path>
 
   <path id="test.classpath">
@@ -35,16 +34,6 @@
     <pathelement path="src/test-files"/>
   </path>
 
-  <target name="compile-core" depends="jar-queries,common.compile-core" />
-
-  <target name="javadocs" depends="javadocs-queries,compile-core,check-javadocs-uptodate" unless="javadocs-uptodate-${name}">
-    <invoke-module-javadoc>
-      <links>
-        <link href="../queries"/>
-      </links>
-    </invoke-module-javadoc>
-  </target>
-
   <target name="regenerate" depends="run-antlr"/>
 
   <target name="resolve-antlr" xmlns:ivy="antlib:org.apache.ivy.ant">

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/src/java/org/apache/lucene/expressions/Bindings.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/java/org/apache/lucene/expressions/Bindings.java b/lucene/expressions/src/java/org/apache/lucene/expressions/Bindings.java
index 5ec2edb..91a22f5 100644
--- a/lucene/expressions/src/java/org/apache/lucene/expressions/Bindings.java
+++ b/lucene/expressions/src/java/org/apache/lucene/expressions/Bindings.java
@@ -16,7 +16,7 @@
  */
 package org.apache.lucene.expressions;
 
-import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.search.DoubleValuesSource;
 
 /**
  * Binds variable names in expressions to actual data.
@@ -31,14 +31,10 @@ public abstract class Bindings {
   /** Sole constructor. (For invocation by subclass 
    *  constructors, typically implicit.) */
   protected Bindings() {}
-  
+
   /**
-   * Returns a ValueSource bound to the variable name.
+   * Returns a DoubleValuesSource bound to the variable name
    */
-  public abstract ValueSource getValueSource(String name);
-  
-  /** Returns a {@code ValueSource} over relevance scores */
-  protected final ValueSource getScoreValueSource() {
-    return new ScoreValueSource();
-  }
+  public abstract DoubleValuesSource getDoubleValuesSource(String name);
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/src/java/org/apache/lucene/expressions/Expression.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/java/org/apache/lucene/expressions/Expression.java b/lucene/expressions/src/java/org/apache/lucene/expressions/Expression.java
index 02be23b..c92e21b 100644
--- a/lucene/expressions/src/java/org/apache/lucene/expressions/Expression.java
+++ b/lucene/expressions/src/java/org/apache/lucene/expressions/Expression.java
@@ -16,9 +16,9 @@
  */
 package org.apache.lucene.expressions;
 
-import org.apache.lucene.expressions.js.JavascriptCompiler; // javadocs
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.expressions.js.JavascriptCompiler;
+import org.apache.lucene.search.DoubleValues;
+import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.Rescorer;
 import org.apache.lucene.search.SortField;
 
@@ -63,26 +63,25 @@ public abstract class Expression {
   }
 
   /**
-   * Evaluates the expression for the given document.
+   * Evaluates the expression for the current document.
    *
-   * @param document <code>docId</code> of the document to compute a value for
-   * @param functionValues {@link FunctionValues} for each element of {@link #variables}.
+   * @param functionValues {@link DoubleValues} for each element of {@link #variables}.
    * @return The computed value of the expression for the given document.
    */
-  public abstract double evaluate(int document, FunctionValues[] functionValues);
+  public abstract double evaluate(DoubleValues[] functionValues);
 
   /**
-   * Get a value source which can compute the value of this expression in the context of the given bindings.
+   * Get a DoubleValuesSource which can compute the value of this expression in the context of the given bindings.
    * @param bindings Bindings to use for external values in this expression
-   * @return A value source which will evaluate this expression when used
+   * @return A DoubleValuesSource which will evaluate this expression when used
    */
-  public ValueSource getValueSource(Bindings bindings) {
+  public DoubleValuesSource getDoubleValuesSource(Bindings bindings) {
     return new ExpressionValueSource(bindings, this);
   }
   
   /** Get a sort field which can be used to rank documents by this expression. */
   public SortField getSortField(Bindings bindings, boolean reverse) {
-    return getValueSource(bindings).getSortField(reverse);
+    return getDoubleValuesSource(bindings).getSortField(reverse);
   }
 
   /** Get a {@link Rescorer}, to rescore first-pass hits

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionComparator.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionComparator.java b/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionComparator.java
deleted file mode 100644
index eabf6dd..0000000
--- a/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionComparator.java
+++ /dev/null
@@ -1,100 +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.lucene.expressions;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.search.FieldComparator;
-import org.apache.lucene.search.LeafFieldComparator;
-import org.apache.lucene.search.Scorer;
-
-/** A custom comparator for sorting documents by an expression */
-class ExpressionComparator extends FieldComparator<Double> implements LeafFieldComparator {
-  private final double[] values;
-  private double bottom;
-  private double topValue;
-  
-  private ValueSource source;
-  private FunctionValues scores;
-  private LeafReaderContext readerContext;
-  
-  public ExpressionComparator(ValueSource source, int numHits) {
-    values = new double[numHits];
-    this.source = source;
-  }
-  
-  // TODO: change FieldComparator.setScorer to throw IOException and remove this try-catch
-  @Override
-  public void setScorer(Scorer scorer) {
-    // TODO: might be cleaner to lazy-init 'source' and set scorer after?
-    assert readerContext != null;
-    try {
-      Map<String,Object> context = new HashMap<>();
-      assert scorer != null;
-      context.put("scorer", scorer);
-      scores = source.getValues(context, readerContext);
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-  }
-  
-  @Override
-  public int compare(int slot1, int slot2) {
-    return Double.compare(values[slot1], values[slot2]);
-  }
-  
-  @Override
-  public void setBottom(int slot) {
-    bottom = values[slot];
-  }
-  
-  @Override
-  public void setTopValue(Double value) {
-    topValue = value.doubleValue();
-  }
-  
-  @Override
-  public int compareBottom(int doc) throws IOException {
-    return Double.compare(bottom, scores.doubleVal(doc));
-  }
-  
-  @Override
-  public void copy(int slot, int doc) throws IOException {
-    values[slot] = scores.doubleVal(doc);
-  }
-  
-  @Override
-  public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
-    this.readerContext = context;
-    return this;
-  }
-  
-  @Override
-  public Double value(int slot) {
-    return Double.valueOf(values[slot]);
-  }
-  
-  @Override
-  public int compareTop(int doc) throws IOException {
-    return Double.compare(topValue, scores.doubleVal(doc));
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionFunctionValues.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionFunctionValues.java b/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionFunctionValues.java
index ad195cd..2e6f7c4 100644
--- a/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionFunctionValues.java
+++ b/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionFunctionValues.java
@@ -16,20 +16,16 @@
  */
 package org.apache.lucene.expressions;
 
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
+import java.io.IOException;
 
-/** A {@link FunctionValues} which evaluates an expression */
-class ExpressionFunctionValues extends DoubleDocValues {
+import org.apache.lucene.search.DoubleValues;
+
+/** A {@link DoubleValues} which evaluates an expression */
+class ExpressionFunctionValues extends DoubleValues {
   final Expression expression;
-  final FunctionValues[] functionValues;
-  
-  int currentDocument = -1;
-  double currentValue;
+  final DoubleValues[] functionValues;
   
-  ExpressionFunctionValues(ValueSource parent, Expression expression, FunctionValues[] functionValues) {
-    super(parent);
+  ExpressionFunctionValues(Expression expression, DoubleValues[] functionValues) {
     if (expression == null) {
       throw new NullPointerException();
     }
@@ -39,14 +35,17 @@ class ExpressionFunctionValues extends DoubleDocValues {
     this.expression = expression;
     this.functionValues = functionValues;
   }
-  
+
   @Override
-  public double doubleVal(int document) {
-    if (currentDocument != document) {
-      currentDocument = document;
-      currentValue = expression.evaluate(document, functionValues);
+  public boolean advanceExact(int doc) throws IOException {
+    for (DoubleValues v : functionValues) {
+      v.advanceExact(doc);
     }
-    
-    return currentValue;
+    return true;
+  }
+  
+  @Override
+  public double doubleValue() {
+    return expression.evaluate(functionValues);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionRescorer.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionRescorer.java b/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionRescorer.java
index 33e8428..e3e7a4e 100644
--- a/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionRescorer.java
+++ b/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionRescorer.java
@@ -20,13 +20,11 @@ package org.apache.lucene.expressions;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.ReaderUtil;
-import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.search.DoubleValues;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Rescorer;
@@ -49,7 +47,7 @@ class ExpressionRescorer extends SortRescorer {
   private final Expression expression;
   private final Bindings bindings;
 
-  /** Uses the provided {@link ValueSource} to assign second
+  /** Uses the provided {@link Expression} to assign second
    *  pass scores. */
   public ExpressionRescorer(Expression expression, Bindings bindings) {
     super(new Sort(expression.getSortField(bindings, true)));
@@ -57,6 +55,21 @@ class ExpressionRescorer extends SortRescorer {
     this.bindings = bindings;
   }
 
+  private static DoubleValues scores(int doc, float score) {
+    return new DoubleValues() {
+      @Override
+      public double doubleValue() throws IOException {
+        return score;
+      }
+
+      @Override
+      public boolean advanceExact(int target) throws IOException {
+        assert doc == target;
+        return true;
+      }
+    };
+  }
+
   @Override
   public Explanation explain(IndexSearcher searcher, Explanation firstPassExplanation, int docID) throws IOException {
     Explanation superExpl = super.explain(searcher, firstPassExplanation, docID);
@@ -65,18 +78,14 @@ class ExpressionRescorer extends SortRescorer {
     int subReader = ReaderUtil.subIndex(docID, leaves);
     LeafReaderContext readerContext = leaves.get(subReader);
     int docIDInSegment = docID - readerContext.docBase;
-    Map<String,Object> context = new HashMap<>();
-
-    FakeScorer fakeScorer = new FakeScorer();
-    fakeScorer.score = firstPassExplanation.getValue();
-    fakeScorer.doc = docIDInSegment;
 
-    context.put("scorer", fakeScorer);
+    DoubleValues scores = scores(docIDInSegment, firstPassExplanation.getValue());
 
     List<Explanation> subs = new ArrayList<>(Arrays.asList(superExpl.getDetails()));
     for(String variable : expression.variables) {
-      subs.add(Explanation.match((float) bindings.getValueSource(variable).getValues(context, readerContext).doubleVal(docIDInSegment),
-                                       "variable \"" + variable + "\""));
+      DoubleValues dv = bindings.getDoubleValuesSource(variable).getValues(readerContext, scores);
+      if (dv.advanceExact(docIDInSegment))
+        subs.add(Explanation.match((float) dv.doubleValue(), "variable \"" + variable + "\""));
     }
 
     return Explanation.match(superExpl.getValue(), superExpl.getDescription(), subs);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionSortField.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionSortField.java b/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionSortField.java
deleted file mode 100644
index 2b39834..0000000
--- a/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionSortField.java
+++ /dev/null
@@ -1,77 +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.lucene.expressions;
-
-import java.io.IOException;
-
-import org.apache.lucene.search.FieldComparator;
-import org.apache.lucene.search.SortField;
-
-/** A {@link SortField} which sorts documents by the evaluated value of an expression for each document */
-class ExpressionSortField extends SortField {
-  private final ExpressionValueSource source;
-
-  ExpressionSortField(String name, ExpressionValueSource source, boolean reverse) {
-    super(name, Type.CUSTOM, reverse);
-    this.source = source;
-  }
-  
-  @Override
-  public FieldComparator<?> getComparator(final int numHits, final int sortPos) throws IOException {
-    return new ExpressionComparator(source, numHits);
-  }
-
-  @Override
-  public int hashCode() {
-    final int prime = 31;
-    int result = super.hashCode();
-    result = prime * result + ((source == null) ? 0 : source.hashCode());
-    return result;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj) return true;
-    if (!super.equals(obj)) return false;
-    if (getClass() != obj.getClass()) return false;
-    ExpressionSortField other = (ExpressionSortField) obj;
-    if (source == null) {
-      if (other.source != null) return false;
-    } else if (!source.equals(other.source)) return false;
-    return true;
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder buffer = new StringBuilder();
-    
-    buffer.append("<expr \"");
-    buffer.append(getField());
-    buffer.append("\">");
-    
-    if (getReverse()) {
-      buffer.append('!');
-    }
-
-    return buffer.toString();
-  }
-
-  @Override
-  public boolean needsScores() {
-    return source.needsScores();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionValueSource.java b/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionValueSource.java
index fcba455..7842de9 100644
--- a/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionValueSource.java
+++ b/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionValueSource.java
@@ -20,76 +20,77 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Objects;
 
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.search.SortField;
+import org.apache.lucene.search.DoubleValues;
+import org.apache.lucene.search.DoubleValuesSource;
 
 /**
- * A {@link ValueSource} which evaluates a {@link Expression} given the context of an {@link Bindings}.
+ * A {@link DoubleValuesSource} which evaluates a {@link Expression} given the context of an {@link Bindings}.
  */
 @SuppressWarnings({"rawtypes", "unchecked"})
-final class ExpressionValueSource extends ValueSource {
-  final ValueSource variables[];
+final class ExpressionValueSource extends DoubleValuesSource {
+  final DoubleValuesSource variables[];
   final Expression expression;
   final boolean needsScores;
 
   ExpressionValueSource(Bindings bindings, Expression expression) {
     if (bindings == null) throw new NullPointerException();
-    if (expression == null) throw new NullPointerException();
-    this.expression = expression;
-    variables = new ValueSource[expression.variables.length];
+    this.expression = Objects.requireNonNull(expression);
+    variables = new DoubleValuesSource[expression.variables.length];
     boolean needsScores = false;
     for (int i = 0; i < variables.length; i++) {
-      ValueSource source = bindings.getValueSource(expression.variables[i]);
-      if (source instanceof ScoreValueSource) {
-        needsScores = true;
-      } else if (source instanceof ExpressionValueSource) {
-        if (((ExpressionValueSource)source).needsScores()) {
-          needsScores = true;
-        }
-      } else if (source == null) {
+      DoubleValuesSource source = bindings.getDoubleValuesSource(expression.variables[i]);
+      if (source == null) {
         throw new RuntimeException("Internal error. Variable (" + expression.variables[i] + ") does not exist.");
       }
+      needsScores |= source.needsScores();
       variables[i] = source;
     }
     this.needsScores = needsScores;
   }
 
   @Override
-  public FunctionValues getValues(Map context, LeafReaderContext readerContext) throws IOException {
-    Map<String, FunctionValues> valuesCache = (Map<String, FunctionValues>)context.get("valuesCache");
-    if (valuesCache == null) {
-      valuesCache = new HashMap<>();
-      context = new HashMap(context);
-      context.put("valuesCache", valuesCache);
-    }
-    FunctionValues[] externalValues = new FunctionValues[expression.variables.length];
+  public DoubleValues getValues(LeafReaderContext readerContext, DoubleValues scores) throws IOException {
+    Map<String, DoubleValues> valuesCache = new HashMap<>();
+    DoubleValues[] externalValues = new DoubleValues[expression.variables.length];
 
     for (int i = 0; i < variables.length; ++i) {
       String externalName = expression.variables[i];
-      FunctionValues values = valuesCache.get(externalName);
+      DoubleValues values = valuesCache.get(externalName);
       if (values == null) {
-        values = variables[i].getValues(context, readerContext);
+        values = variables[i].getValues(readerContext, scores);
         if (values == null) {
           throw new RuntimeException("Internal error. External (" + externalName + ") does not exist.");
         }
         valuesCache.put(externalName, values);
       }
-      externalValues[i] = values;
+      externalValues[i] = zeroWhenUnpositioned(values);
     }
 
-    return new ExpressionFunctionValues(this, expression, externalValues);
+    return new ExpressionFunctionValues(expression, externalValues);
   }
 
-  @Override
-  public SortField getSortField(boolean reverse) {
-    return new ExpressionSortField(expression.sourceText, this, reverse);
+  private static DoubleValues zeroWhenUnpositioned(DoubleValues in) {
+    return new DoubleValues() {
+
+      boolean positioned = false;
+
+      @Override
+      public double doubleValue() throws IOException {
+        return positioned ? in.doubleValue() : 0;
+      }
+
+      @Override
+      public boolean advanceExact(int doc) throws IOException {
+        return positioned = in.advanceExact(doc);
+      }
+    };
   }
 
   @Override
-  public String description() {
+  public String toString() {
     return "expr(" + expression.sourceText + ")";
   }
   
@@ -132,7 +133,8 @@ final class ExpressionValueSource extends ValueSource {
     return true;
   }
 
-  boolean needsScores() {
+  @Override
+  public boolean needsScores() {
     return needsScores;
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/src/java/org/apache/lucene/expressions/ScoreFunctionValues.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/java/org/apache/lucene/expressions/ScoreFunctionValues.java b/lucene/expressions/src/java/org/apache/lucene/expressions/ScoreFunctionValues.java
deleted file mode 100644
index e310c06..0000000
--- a/lucene/expressions/src/java/org/apache/lucene/expressions/ScoreFunctionValues.java
+++ /dev/null
@@ -1,46 +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.lucene.expressions;
-
-import java.io.IOException;
-
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
-import org.apache.lucene.search.Scorer;
-
-/**
- * A utility class to allow expressions to access the score as a {@link FunctionValues}.
- */
-class ScoreFunctionValues extends DoubleDocValues {
-  final Scorer scorer;
-
-  ScoreFunctionValues(ValueSource parent, Scorer scorer) {
-    super(parent);
-    this.scorer = scorer;
-  }
-  
-  @Override
-  public double doubleVal(int document) {
-    try {
-      assert document == scorer.docID();
-      return scorer.score();
-    } catch (IOException exception) {
-      throw new RuntimeException(exception);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/src/java/org/apache/lucene/expressions/ScoreValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/java/org/apache/lucene/expressions/ScoreValueSource.java b/lucene/expressions/src/java/org/apache/lucene/expressions/ScoreValueSource.java
deleted file mode 100644
index ea1669c..0000000
--- a/lucene/expressions/src/java/org/apache/lucene/expressions/ScoreValueSource.java
+++ /dev/null
@@ -1,61 +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.lucene.expressions;
-
-
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.search.Scorer;
-
-import java.io.IOException;
-import java.util.Map;
-
-/**
- * A {@link ValueSource} which uses the {@link Scorer} passed through
- * the context map by {@link ExpressionComparator}.
- */
-@SuppressWarnings({"rawtypes"})
-class ScoreValueSource extends ValueSource {
-
-  /**
-   * <code>context</code> must contain a key "scorer" which is a {@link Scorer}.
-   */
-  @Override
-  public FunctionValues getValues(Map context, LeafReaderContext readerContext) throws IOException {
-    Scorer v = (Scorer) context.get("scorer");
-    if (v == null) {
-      throw new IllegalStateException("Expressions referencing the score can only be used for sorting");
-    }
-    return new ScoreFunctionValues(this, v);
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    return o == this;
-  }
-
-  @Override
-  public int hashCode() {
-    return System.identityHashCode(this);
-  }
-
-  @Override
-  public String description() {
-    return "score()";
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/src/java/org/apache/lucene/expressions/SimpleBindings.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/java/org/apache/lucene/expressions/SimpleBindings.java b/lucene/expressions/src/java/org/apache/lucene/expressions/SimpleBindings.java
index 1c11cb2..6276055 100644
--- a/lucene/expressions/src/java/org/apache/lucene/expressions/SimpleBindings.java
+++ b/lucene/expressions/src/java/org/apache/lucene/expressions/SimpleBindings.java
@@ -20,11 +20,7 @@ package org.apache.lucene.expressions;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.queries.function.valuesource.DoubleFieldSource;
-import org.apache.lucene.queries.function.valuesource.FloatFieldSource;
-import org.apache.lucene.queries.function.valuesource.IntFieldSource;
-import org.apache.lucene.queries.function.valuesource.LongFieldSource;
+import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.SortField;
 
 /**
@@ -64,9 +60,9 @@ public final class SimpleBindings extends Bindings {
   }
 
   /**
-   * Bind a {@link ValueSource} directly to the given name.
+   * Bind a {@link DoubleValuesSource} directly to the given name.
    */
-  public void add(String name, ValueSource source) { map.put(name, source); }
+  public void add(String name, DoubleValuesSource source) { map.put(name, source); }
   
   /** 
    * Adds an Expression to the bindings.
@@ -78,27 +74,27 @@ public final class SimpleBindings extends Bindings {
   }
   
   @Override
-  public ValueSource getValueSource(String name) {
+  public DoubleValuesSource getDoubleValuesSource(String name) {
     Object o = map.get(name);
     if (o == null) {
       throw new IllegalArgumentException("Invalid reference '" + name + "'");
     } else if (o instanceof Expression) {
-      return ((Expression)o).getValueSource(this);
-    } else if (o instanceof ValueSource) {
-      return ((ValueSource)o);
+      return ((Expression)o).getDoubleValuesSource(this);
+    } else if (o instanceof DoubleValuesSource) {
+      return ((DoubleValuesSource) o);
     }
     SortField field = (SortField) o;
     switch(field.getType()) {
       case INT:
-        return new IntFieldSource(field.getField());
+        return DoubleValuesSource.fromIntField(field.getField());
       case LONG:
-        return new LongFieldSource(field.getField());
+        return DoubleValuesSource.fromLongField(field.getField());
       case FLOAT:
-        return new FloatFieldSource(field.getField());
+        return DoubleValuesSource.fromFloatField(field.getField());
       case DOUBLE:
-        return new DoubleFieldSource(field.getField());
+        return DoubleValuesSource.fromDoubleField(field.getField());
       case SCORE:
-        return getScoreValueSource();
+        return DoubleValuesSource.SCORES;
       default:
         throw new UnsupportedOperationException(); 
     }
@@ -113,7 +109,7 @@ public final class SimpleBindings extends Bindings {
       if (o instanceof Expression) {
         Expression expr = (Expression) o;
         try {
-          expr.getValueSource(this);
+          expr.getDoubleValuesSource(this);
         } catch (StackOverflowError e) {
           throw new IllegalArgumentException("Recursion Error: Cycle detected originating in (" + expr.sourceText + ")");
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/src/java/org/apache/lucene/expressions/js/JavascriptCompiler.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/java/org/apache/lucene/expressions/js/JavascriptCompiler.java b/lucene/expressions/src/java/org/apache/lucene/expressions/js/JavascriptCompiler.java
index 13174bc..87e41c0 100644
--- a/lucene/expressions/src/java/org/apache/lucene/expressions/js/JavascriptCompiler.java
+++ b/lucene/expressions/src/java/org/apache/lucene/expressions/js/JavascriptCompiler.java
@@ -39,7 +39,7 @@ import org.antlr.v4.runtime.CommonTokenStream;
 import org.antlr.v4.runtime.tree.ParseTree;
 import org.apache.lucene.expressions.Expression;
 import org.apache.lucene.expressions.js.JavascriptParser.ExpressionContext;
-import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.search.DoubleValues;
 import org.apache.lucene.util.IOUtils;
 import org.objectweb.asm.ClassWriter;
 import org.objectweb.asm.Label;
@@ -93,13 +93,13 @@ public final class JavascriptCompiler {
   private static final String COMPILED_EXPRESSION_INTERNAL = COMPILED_EXPRESSION_CLASS.replace('.', '/');
   
   static final Type EXPRESSION_TYPE = Type.getType(Expression.class);
-  static final Type FUNCTION_VALUES_TYPE = Type.getType(FunctionValues.class);
+  static final Type FUNCTION_VALUES_TYPE = Type.getType(DoubleValues.class);
 
   private static final org.objectweb.asm.commons.Method
     EXPRESSION_CTOR = getAsmMethod(void.class, "<init>", String.class, String[].class),
-    EVALUATE_METHOD = getAsmMethod(double.class, "evaluate", int.class, FunctionValues[].class);
+    EVALUATE_METHOD = getAsmMethod(double.class, "evaluate", DoubleValues[].class);
 
-  static final org.objectweb.asm.commons.Method DOUBLE_VAL_METHOD = getAsmMethod(double.class, "doubleVal", int.class);
+  static final org.objectweb.asm.commons.Method DOUBLE_VAL_METHOD = getAsmMethod(double.class, "doubleValue");
   
   /** create an ASM Method object from return type, method name, and parameters. */
   private static org.objectweb.asm.commons.Method getAsmMethod(Class<?> rtype, String name, Class<?>... ptypes) {
@@ -155,8 +155,8 @@ public final class JavascriptCompiler {
    */
   @SuppressWarnings({"unused", "null"})
   private static void unusedTestCompile() throws IOException {
-    FunctionValues f = null;
-    double ret = f.doubleVal(2);
+    DoubleValues f = null;
+    double ret = f.doubleValue();
   }
   
   /**
@@ -325,10 +325,9 @@ public final class JavascriptCompiler {
             externalsMap.put(text, index);
           }
 
-          gen.loadArg(1);
+          gen.loadArg(0);
           gen.push(index);
           gen.arrayLoad(FUNCTION_VALUES_TYPE);
-          gen.loadArg(0);
           gen.invokeVirtual(FUNCTION_VALUES_TYPE, DOUBLE_VAL_METHOD);
           gen.cast(Type.DOUBLE_TYPE, typeStack.peek());
         } else {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/src/java/org/apache/lucene/expressions/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/java/org/apache/lucene/expressions/package-info.java b/lucene/expressions/src/java/org/apache/lucene/expressions/package-info.java
index 62a519b..4a1eadf 100644
--- a/lucene/expressions/src/java/org/apache/lucene/expressions/package-info.java
+++ b/lucene/expressions/src/java/org/apache/lucene/expressions/package-info.java
@@ -25,11 +25,12 @@
  * 
  * <p>
  * {@link org.apache.lucene.expressions.Bindings} - abstraction for binding external variables
- * to a way to get a value for those variables for a particular document (ValueSource).
+ * to a way to get a value for those variables for a particular document (DoubleValuesSource).
  * </p>
  * 
  * <p>
- * {@link org.apache.lucene.expressions.SimpleBindings} - default implementation of bindings which provide easy ways to bind sort fields and other expressions to external variables
+ * {@link org.apache.lucene.expressions.SimpleBindings} - default implementation of bindings which
+ * provide easy ways to bind sort fields and other expressions to external variables
  * </p>
  */
 package org.apache.lucene.expressions;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/src/test/org/apache/lucene/expressions/TestDemoExpressions.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/test/org/apache/lucene/expressions/TestDemoExpressions.java b/lucene/expressions/src/test/org/apache/lucene/expressions/TestDemoExpressions.java
index 01b3394..d76ef1c 100644
--- a/lucene/expressions/src/test/org/apache/lucene/expressions/TestDemoExpressions.java
+++ b/lucene/expressions/src/test/org/apache/lucene/expressions/TestDemoExpressions.java
@@ -16,18 +16,20 @@
  */
 package org.apache.lucene.expressions;
 
+import java.io.IOException;
+
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.expressions.js.JavascriptCompiler;
 import org.apache.lucene.expressions.js.VariableContext;
 import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.queries.function.valuesource.DoubleConstValueSource;
-import org.apache.lucene.queries.function.valuesource.IntFieldSource;
 import org.apache.lucene.search.CheckHits;
+import org.apache.lucene.search.DoubleValues;
+import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.FieldDoc;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
@@ -39,9 +41,9 @@ import org.apache.lucene.search.TopFieldDocs;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
 
+import static org.apache.lucene.expressions.js.VariableContext.Type.INT_INDEX;
 import static org.apache.lucene.expressions.js.VariableContext.Type.MEMBER;
 import static org.apache.lucene.expressions.js.VariableContext.Type.STR_INDEX;
-import static org.apache.lucene.expressions.js.VariableContext.Type.INT_INDEX;
 
 
 /** simple demo of using expressions */
@@ -236,7 +238,7 @@ public class  TestDemoExpressions extends LuceneTestCase {
   public void testStaticExtendedVariableExample() throws Exception {
     Expression popularity = JavascriptCompiler.compile("doc[\"popularity\"].value");
     SimpleBindings bindings = new SimpleBindings();
-    bindings.add("doc['popularity'].value", new IntFieldSource("popularity"));
+    bindings.add("doc['popularity'].value", DoubleValuesSource.fromIntField("popularity"));
     Sort sort = new Sort(popularity.getSortField(bindings, true));
     TopFieldDocs td = searcher.search(new MatchAllDocsQuery(), 3, sort);
 
@@ -250,6 +252,30 @@ public class  TestDemoExpressions extends LuceneTestCase {
     assertEquals(2D, (Double)d.fields[0], 1E-4);
   }
 
+  private static DoubleValuesSource constant(double value) {
+    return new DoubleValuesSource() {
+      @Override
+      public DoubleValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
+        return new DoubleValues() {
+          @Override
+          public double doubleValue() throws IOException {
+            return value;
+          }
+
+          @Override
+          public boolean advanceExact(int doc) throws IOException {
+            return true;
+          }
+        };
+      }
+
+      @Override
+      public boolean needsScores() {
+        return false;
+      }
+    };
+  }
+
   public void testDynamicExtendedVariableExample() throws Exception {
     Expression popularity = JavascriptCompiler.compile("doc['popularity'].value + magicarray[0] + fourtytwo");
 
@@ -258,7 +284,7 @@ public class  TestDemoExpressions extends LuceneTestCase {
     // filled in with proper error messages for a real use case.
     Bindings bindings = new Bindings() {
       @Override
-      public ValueSource getValueSource(String name) {
+      public DoubleValuesSource getDoubleValuesSource(String name) {
         VariableContext[] var = VariableContext.parse(name);
         assert var[0].type == MEMBER;
         String base = var[0].text;
@@ -266,7 +292,7 @@ public class  TestDemoExpressions extends LuceneTestCase {
           if (var.length > 1 && var[1].type == STR_INDEX) {
             String field = var[1].text;
             if (var.length > 2 && var[2].type == MEMBER && var[2].text.equals("value")) {
-              return new IntFieldSource(field);
+              return DoubleValuesSource.fromIntField(field);
             } else {
               fail("member: " + var[2].text);// error case, non/missing "value" member access
             }
@@ -275,12 +301,12 @@ public class  TestDemoExpressions extends LuceneTestCase {
           }
         } else if (base.equals("magicarray")) {
           if (var.length > 1 && var[1].type == INT_INDEX) {
-            return new DoubleConstValueSource(2048);
+            return constant(2048);
           } else {
             fail();// error case, magic array isn't an array
           }
         } else if (base.equals("fourtytwo")) {
-          return new DoubleConstValueSource(42);
+          return constant(42);
         } else {
           fail();// error case (variable doesn't exist)
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/src/test/org/apache/lucene/expressions/TestExpressionSortField.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/test/org/apache/lucene/expressions/TestExpressionSortField.java b/lucene/expressions/src/test/org/apache/lucene/expressions/TestExpressionSortField.java
index ec6ea11..73e7b8b 100644
--- a/lucene/expressions/src/test/org/apache/lucene/expressions/TestExpressionSortField.java
+++ b/lucene/expressions/src/test/org/apache/lucene/expressions/TestExpressionSortField.java
@@ -31,7 +31,7 @@ public class TestExpressionSortField extends LuceneTestCase {
     bindings.add(new SortField("popularity", SortField.Type.INT));
     
     SortField sf = expr.getSortField(bindings, true);
-    assertEquals("<expr \"sqrt(_score) + ln(popularity)\">!", sf.toString());
+    assertEquals("<expr(sqrt(_score) + ln(popularity))>!", sf.toString());
   }
   
   public void testEquals() throws Exception {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/src/test/org/apache/lucene/expressions/TestExpressionValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/test/org/apache/lucene/expressions/TestExpressionValueSource.java b/lucene/expressions/src/test/org/apache/lucene/expressions/TestExpressionValueSource.java
index eeb3c9c..39217d8 100644
--- a/lucene/expressions/src/test/org/apache/lucene/expressions/TestExpressionValueSource.java
+++ b/lucene/expressions/src/test/org/apache/lucene/expressions/TestExpressionValueSource.java
@@ -17,21 +17,17 @@
 package org.apache.lucene.expressions;
 
 
-import java.util.HashMap;
-
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.expressions.js.JavascriptCompiler;
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.queries.function.ValueSourceScorer;
-import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.DoubleValues;
+import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
@@ -47,7 +43,7 @@ public class TestExpressionValueSource extends LuceneTestCase {
     IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()));
     iwc.setMergePolicy(newLogMergePolicy());
     RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
-    
+
     Document doc = new Document();
     doc.add(newStringField("id", "1", Field.Store.YES));
     doc.add(newTextField("body", "some contents and more contents", Field.Store.NO));
@@ -58,6 +54,7 @@ public class TestExpressionValueSource extends LuceneTestCase {
     doc.add(newStringField("id", "2", Field.Store.YES));
     doc.add(newTextField("body", "another document with different contents", Field.Store.NO));
     doc.add(new NumericDocValuesField("popularity", 20));
+    doc.add(new NumericDocValuesField("count", 1));
     iw.addDocument(doc);
     
     doc = new Document();
@@ -77,81 +74,34 @@ public class TestExpressionValueSource extends LuceneTestCase {
     dir.close();
     super.tearDown();
   }
-  
-  public void testTypes() throws Exception {
-    Expression expr = JavascriptCompiler.compile("2*popularity");
-    SimpleBindings bindings = new SimpleBindings();
-    bindings.add(new SortField("popularity", SortField.Type.LONG));
-    ValueSource vs = expr.getValueSource(bindings);
-    
-    assertEquals(1, reader.leaves().size());
-    LeafReaderContext leaf = reader.leaves().get(0);
-    FunctionValues values = vs.getValues(new HashMap<String,Object>(), leaf);
-    
-    assertEquals(10, values.doubleVal(0), 0);
-    assertEquals(10, values.floatVal(0), 0);
-    assertEquals(10, values.longVal(0));
-    assertEquals(10, values.intVal(0));
-    assertEquals(10, values.shortVal(0));
-    assertEquals(10, values.byteVal(0));
-    assertEquals("10.0", values.strVal(0));
-    assertEquals(new Double(10), values.objectVal(0));
-    
-    assertEquals(40, values.doubleVal(1), 0);
-    assertEquals(40, values.floatVal(1), 0);
-    assertEquals(40, values.longVal(1));
-    assertEquals(40, values.intVal(1));
-    assertEquals(40, values.shortVal(1));
-    assertEquals(40, values.byteVal(1));
-    assertEquals("40.0", values.strVal(1));
-    assertEquals(new Double(40), values.objectVal(1));
-    
-    assertEquals(4, values.doubleVal(2), 0);
-    assertEquals(4, values.floatVal(2), 0);
-    assertEquals(4, values.longVal(2));
-    assertEquals(4, values.intVal(2));
-    assertEquals(4, values.shortVal(2));
-    assertEquals(4, values.byteVal(2));
-    assertEquals("4.0", values.strVal(2));
-    assertEquals(new Double(4), values.objectVal(2));    
-  }
-  
-  public void testRangeScorer() throws Exception {
-    Expression expr = JavascriptCompiler.compile("2*popularity");
+
+  public void testDoubleValuesSourceTypes() throws Exception {
+    Expression expr = JavascriptCompiler.compile("2*popularity + count");
     SimpleBindings bindings = new SimpleBindings();
     bindings.add(new SortField("popularity", SortField.Type.LONG));
-    ValueSource vs = expr.getValueSource(bindings);
-    
+    bindings.add(new SortField("count", SortField.Type.LONG));
+    DoubleValuesSource vs = expr.getDoubleValuesSource(bindings);
+
     assertEquals(1, reader.leaves().size());
     LeafReaderContext leaf = reader.leaves().get(0);
-    FunctionValues values = vs.getValues(new HashMap<String,Object>(), leaf);
-    
-    // everything
-    ValueSourceScorer scorer = values.getRangeScorer(leaf, "4", "40", true, true);
-    DocIdSetIterator iter = scorer.iterator();
-    assertEquals(-1, iter.docID());
-    assertEquals(0, iter.nextDoc());
-    assertEquals(1, iter.nextDoc());
-    assertEquals(2, iter.nextDoc());
-    assertEquals(DocIdSetIterator.NO_MORE_DOCS, iter.nextDoc());
+    DoubleValues values = vs.getValues(leaf, null);
 
-    // just the first doc
-    values = vs.getValues(new HashMap<String,Object>(), leaf);
-    scorer = values.getRangeScorer(leaf, "4", "40", false, false);
-    iter = scorer.iterator();
-    assertEquals(-1, scorer.docID());
-    assertEquals(0, iter.nextDoc());
-    assertEquals(DocIdSetIterator.NO_MORE_DOCS, iter.nextDoc());
+    assertTrue(values.advanceExact(0));
+    assertEquals(10, values.doubleValue(), 0);
+    assertTrue(values.advanceExact(1));
+    assertEquals(41, values.doubleValue(), 0);
+    assertTrue(values.advanceExact(2));
+    assertEquals(4, values.doubleValue(), 0);
   }
-  
-  public void testEquals() throws Exception {
+
+  public void testDoubleValuesSourceEquals() throws Exception {
     Expression expr = JavascriptCompiler.compile("sqrt(a) + ln(b)");
-    
-    SimpleBindings bindings = new SimpleBindings();    
+
+    SimpleBindings bindings = new SimpleBindings();
     bindings.add(new SortField("a", SortField.Type.INT));
     bindings.add(new SortField("b", SortField.Type.INT));
-    
-    ValueSource vs1 = expr.getValueSource(bindings);
+
+    DoubleValuesSource vs1 = expr.getDoubleValuesSource(bindings);
     // same instance
     assertEquals(vs1, vs1);
     // null
@@ -159,20 +109,21 @@ public class TestExpressionValueSource extends LuceneTestCase {
     // other object
     assertFalse(vs1.equals("foobar"));
     // same bindings and expression instances
-    ValueSource vs2 = expr.getValueSource(bindings);
+    DoubleValuesSource vs2 = expr.getDoubleValuesSource(bindings);
     assertEquals(vs1.hashCode(), vs2.hashCode());
     assertEquals(vs1, vs2);
     // equiv bindings (different instance)
-    SimpleBindings bindings2 = new SimpleBindings();    
+    SimpleBindings bindings2 = new SimpleBindings();
     bindings2.add(new SortField("a", SortField.Type.INT));
     bindings2.add(new SortField("b", SortField.Type.INT));
-    ValueSource vs3 = expr.getValueSource(bindings2);
+    DoubleValuesSource vs3 = expr.getDoubleValuesSource(bindings2);
     assertEquals(vs1, vs3);
     // different bindings (same names, different types)
-    SimpleBindings bindings3 = new SimpleBindings();    
+    SimpleBindings bindings3 = new SimpleBindings();
     bindings3.add(new SortField("a", SortField.Type.LONG));
-    bindings3.add(new SortField("b", SortField.Type.INT));
-    ValueSource vs4 = expr.getValueSource(bindings3);
+    bindings3.add(new SortField("b", SortField.Type.FLOAT));
+    DoubleValuesSource vs4 = expr.getDoubleValuesSource(bindings3);
     assertFalse(vs1.equals(vs4));
   }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestCustomFunctions.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestCustomFunctions.java b/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestCustomFunctions.java
index 7c46b05..9df5d74 100644
--- a/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestCustomFunctions.java
+++ b/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestCustomFunctions.java
@@ -50,7 +50,7 @@ public class TestCustomFunctions extends LuceneTestCase {
   public void testDefaultList() throws Exception {
     Map<String,Method> functions = JavascriptCompiler.DEFAULT_FUNCTIONS;
     Expression expr = JavascriptCompiler.compile("sqrt(20)", functions, getClass().getClassLoader());
-    assertEquals(Math.sqrt(20), expr.evaluate(0, null), DELTA);
+    assertEquals(Math.sqrt(20), expr.evaluate(null), DELTA);
   }
   
   public static double zeroArgMethod() { return 5; }
@@ -60,7 +60,7 @@ public class TestCustomFunctions extends LuceneTestCase {
     Map<String,Method> functions = new HashMap<>();
     functions.put("foo", getClass().getMethod("zeroArgMethod"));
     Expression expr = JavascriptCompiler.compile("foo()", functions, getClass().getClassLoader());
-    assertEquals(5, expr.evaluate(0, null), DELTA);
+    assertEquals(5, expr.evaluate(null), DELTA);
   }
 
   public static double oneArgMethod(double arg1) { return 3 + arg1; }
@@ -70,7 +70,7 @@ public class TestCustomFunctions extends LuceneTestCase {
     Map<String,Method> functions = new HashMap<>();
     functions.put("foo", getClass().getMethod("oneArgMethod", double.class));
     Expression expr = JavascriptCompiler.compile("foo(3)", functions, getClass().getClassLoader());
-    assertEquals(6, expr.evaluate(0, null), DELTA);
+    assertEquals(6, expr.evaluate(null), DELTA);
   }
   
   public static double threeArgMethod(double arg1, double arg2, double arg3) { return arg1 + arg2 + arg3; }
@@ -80,7 +80,7 @@ public class TestCustomFunctions extends LuceneTestCase {
     Map<String,Method> functions = new HashMap<>();
     functions.put("foo", getClass().getMethod("threeArgMethod", double.class, double.class, double.class));
     Expression expr = JavascriptCompiler.compile("foo(3, 4, 5)", functions, getClass().getClassLoader());
-    assertEquals(12, expr.evaluate(0, null), DELTA);
+    assertEquals(12, expr.evaluate(null), DELTA);
   }
   
   /** tests a map with 2 functions */
@@ -89,7 +89,7 @@ public class TestCustomFunctions extends LuceneTestCase {
     functions.put("foo", getClass().getMethod("zeroArgMethod"));
     functions.put("bar", getClass().getMethod("oneArgMethod", double.class));
     Expression expr = JavascriptCompiler.compile("foo() + bar(3)", functions, getClass().getClassLoader());
-    assertEquals(11, expr.evaluate(0, null), DELTA);
+    assertEquals(11, expr.evaluate(null), DELTA);
   }
 
   /** tests invalid methods that are not allowed to become variables to be mapped */
@@ -220,7 +220,7 @@ public class TestCustomFunctions extends LuceneTestCase {
     
     // this should pass:
     Expression expr = JavascriptCompiler.compile("bar()", functions, childLoader);
-    assertEquals(2.0, expr.evaluate(0, null), DELTA);
+    assertEquals(2.0, expr.evaluate(null), DELTA);
     
     // use our classloader, not the foreign one, which should fail!
     IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
@@ -232,9 +232,9 @@ public class TestCustomFunctions extends LuceneTestCase {
     Map<String,Method> mixedFunctions = new HashMap<>(JavascriptCompiler.DEFAULT_FUNCTIONS);
     mixedFunctions.putAll(functions);
     expr = JavascriptCompiler.compile("bar()", mixedFunctions, childLoader);
-    assertEquals(2.0, expr.evaluate(0, null), DELTA);
+    assertEquals(2.0, expr.evaluate(null), DELTA);
     expr = JavascriptCompiler.compile("sqrt(20)", mixedFunctions, childLoader);
-    assertEquals(Math.sqrt(20), expr.evaluate(0, null), DELTA);
+    assertEquals(Math.sqrt(20), expr.evaluate(null), DELTA);
     
     // use our classloader, not the foreign one, which should fail!
     expected = expectThrows(IllegalArgumentException.class, () -> {
@@ -256,7 +256,7 @@ public class TestCustomFunctions extends LuceneTestCase {
     String source = "3 * foo() / 5";
     Expression expr = JavascriptCompiler.compile(source, functions, getClass().getClassLoader());
     ArithmeticException expected = expectThrows(ArithmeticException.class, () -> {
-      expr.evaluate(0, null);
+      expr.evaluate(null);
     });
     assertEquals(MESSAGE, expected.getMessage());
     StringWriter sw = new StringWriter();
@@ -272,6 +272,6 @@ public class TestCustomFunctions extends LuceneTestCase {
     functions.put("foo.bar", getClass().getMethod("zeroArgMethod"));
     String source = "foo.bar()";
     Expression expr = JavascriptCompiler.compile(source, functions, getClass().getClassLoader());
-    assertEquals(5, expr.evaluate(0, null), DELTA);
+    assertEquals(5, expr.evaluate(null), DELTA);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestJavascriptFunction.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestJavascriptFunction.java b/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestJavascriptFunction.java
index 81362a6..ed68a5f 100644
--- a/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestJavascriptFunction.java
+++ b/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestJavascriptFunction.java
@@ -24,7 +24,7 @@ public class TestJavascriptFunction extends LuceneTestCase {
   
   private void assertEvaluatesTo(String expression, double expected) throws Exception {
     Expression evaluator = JavascriptCompiler.compile(expression);
-    double actual = evaluator.evaluate(0, null);
+    double actual = evaluator.evaluate(null);
     assertEquals(expected, actual, DELTA);
   }
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestJavascriptOperations.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestJavascriptOperations.java b/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestJavascriptOperations.java
index 82d5056..fd098c5 100644
--- a/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestJavascriptOperations.java
+++ b/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestJavascriptOperations.java
@@ -22,7 +22,7 @@ import org.apache.lucene.util.LuceneTestCase;
 public class TestJavascriptOperations extends LuceneTestCase {
   private void assertEvaluatesTo(String expression, long expected) throws Exception {
     Expression evaluator = JavascriptCompiler.compile(expression);
-    long actual = (long)evaluator.evaluate(0, null);
+    long actual = (long)evaluator.evaluate(null);
     assertEquals(expected, actual);
   }
   


[4/6] lucene-solr:branch_6x: LUCENE-7610: Deprecate ValueSource methods in facets module

Posted by ro...@apache.org.
LUCENE-7610: Deprecate ValueSource methods in facets module


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/a238610b
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/a238610b
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/a238610b

Branch: refs/heads/branch_6x
Commit: a238610bab1499b340fde8e120f02b33233b40e1
Parents: 776087e
Author: Alan Woodward <ro...@apache.org>
Authored: Fri Jan 6 10:41:12 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Sat Jan 7 11:51:50 2017 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   3 +
 .../demo/facet/DistanceFacetsExample.java       |  18 +--
 .../org/apache/lucene/facet/package-info.java   |   2 +-
 .../apache/lucene/facet/range/DoubleRange.java  |  33 ++++--
 .../facet/range/DoubleRangeFacetCounts.java     |  80 ++++++++-----
 .../apache/lucene/facet/range/LongRange.java    |  34 ++++--
 .../facet/range/LongRangeFacetCounts.java       |  47 +++++---
 .../org/apache/lucene/facet/range/Range.java    |  31 ------
 .../lucene/facet/taxonomy/FakeScorer.java       |  53 ---------
 .../taxonomy/TaxonomyFacetSumValueSource.java   |  99 ++++++++++++-----
 .../facet/range/TestRangeFacetCounts.java       |  53 +++++----
 .../TestTaxonomyFacetSumValueSource.java        |   5 +-
 .../lucene/queries/function/ValueSource.java    | 111 +++++++++++++++++++
 13 files changed, 361 insertions(+), 208 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a238610b/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 74ee5ef..826c86a 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -24,6 +24,9 @@ API Changes
   replaced with Expression#getDoubleValuesSource(). (Alan Woodward, Adrien
   Grand)
 
+* LUCENE-7610: The facets module now uses the DoubleValuesSource API, and
+  methods that take ValueSource parameters are deprecated (Alan Woodward)
+
 New features
 
 * LUCENE-5867: Added BooleanSimilarity. (Robert Muir, Adrien Grand)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a238610b/lucene/demo/src/java/org/apache/lucene/demo/facet/DistanceFacetsExample.java
----------------------------------------------------------------------
diff --git a/lucene/demo/src/java/org/apache/lucene/demo/facet/DistanceFacetsExample.java b/lucene/demo/src/java/org/apache/lucene/demo/facet/DistanceFacetsExample.java
index 7d029ee..fa7ce83 100644
--- a/lucene/demo/src/java/org/apache/lucene/demo/facet/DistanceFacetsExample.java
+++ b/lucene/demo/src/java/org/apache/lucene/demo/facet/DistanceFacetsExample.java
@@ -16,9 +16,13 @@
  */
 package org.apache.lucene.demo.facet;
 
+import java.io.Closeable;
+import java.io.IOException;
+import java.text.ParseException;
+
 import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
-import org.apache.lucene.document.DoublePoint;
 import org.apache.lucene.document.Document;
+import org.apache.lucene.document.DoublePoint;
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.expressions.Expression;
 import org.apache.lucene.expressions.SimpleBindings;
@@ -36,9 +40,9 @@ import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
-import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.Query;
@@ -48,10 +52,6 @@ import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.RAMDirectory;
 import org.apache.lucene.util.SloppyMath;
 
-import java.io.Closeable;
-import java.io.IOException;
-import java.text.ParseException;
-
 /** Shows simple usage of dynamic range faceting, using the
  *  expressions module to calculate distance. */
 public class DistanceFacetsExample implements Closeable {
@@ -117,7 +117,7 @@ public class DistanceFacetsExample implements Closeable {
     writer.close();
   }
 
-  private ValueSource getDistanceValueSource() {
+  private DoubleValuesSource getDistanceValueSource() {
     Expression distance;
     try {
       distance = JavascriptCompiler.compile(
@@ -130,7 +130,7 @@ public class DistanceFacetsExample implements Closeable {
     bindings.add(new SortField("latitude", SortField.Type.DOUBLE));
     bindings.add(new SortField("longitude", SortField.Type.DOUBLE));
 
-    return distance.getValueSource(bindings);
+    return distance.getDoubleValuesSource(bindings);
   }
 
   /** Given a latitude and longitude (in degrees) and the
@@ -224,7 +224,7 @@ public class DistanceFacetsExample implements Closeable {
     // Passing no baseQuery means we drill down on all
     // documents ("browse only"):
     DrillDownQuery q = new DrillDownQuery(null);
-    final ValueSource vs = getDistanceValueSource();
+    final DoubleValuesSource vs = getDistanceValueSource();
     q.add("field", range.getQuery(getBoundingBoxQuery(ORIGIN_LATITUDE, ORIGIN_LONGITUDE, range.max), vs));
     DrillSideways ds = new DrillSideways(searcher, config, (TaxonomyReader) null) {
         @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a238610b/lucene/facet/src/java/org/apache/lucene/facet/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/package-info.java b/lucene/facet/src/java/org/apache/lucene/facet/package-info.java
index 0501d6a..acbdd44 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/package-info.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/package-info.java
@@ -40,7 +40,7 @@
  * 
  *  <li> Range faceting {@link org.apache.lucene.facet.range.LongRangeFacetCounts}, {@link
  *       org.apache.lucene.facet.range.DoubleRangeFacetCounts} compute counts for a dynamic numeric
- *       range from a provided {@link org.apache.lucene.queries.function.ValueSource} (previously indexed
+ *       range from a provided {@link org.apache.lucene.search.LongValuesSource} (previously indexed
  *       numeric field, or a dynamic expression such as distance).
  * </ul>
  * <p>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a238610b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java
index 8893c65..c893db9 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java
@@ -17,16 +17,16 @@
 package org.apache.lucene.facet.range;
 
 import java.io.IOException;
-import java.util.Collections;
 import java.util.Objects;
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.ConstantScoreScorer;
 import org.apache.lucene.search.ConstantScoreWeight;
 import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.DoubleValues;
+import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Scorer;
@@ -95,9 +95,9 @@ public final class DoubleRange extends Range {
   private static class ValueSourceQuery extends Query {
     private final DoubleRange range;
     private final Query fastMatchQuery;
-    private final ValueSource valueSource;
+    private final DoubleValuesSource valueSource;
 
-    ValueSourceQuery(DoubleRange range, Query fastMatchQuery, ValueSource valueSource) {
+    ValueSourceQuery(DoubleRange range, Query fastMatchQuery, DoubleValuesSource valueSource) {
       this.range = range;
       this.fastMatchQuery = fastMatchQuery;
       this.valueSource = valueSource;
@@ -158,11 +158,11 @@ public final class DoubleRange extends Range {
             approximation = s.iterator();
           }
 
-          final FunctionValues values = valueSource.getValues(Collections.emptyMap(), context);
+          final DoubleValues values = valueSource.getValues(context, null);
           final TwoPhaseIterator twoPhase = new TwoPhaseIterator(approximation) {
             @Override
             public boolean matches() throws IOException {
-              return range.accept(values.doubleVal(approximation.docID()));
+              return values.advanceExact(approximation.docID()) && range.accept(values.doubleValue());
             }
 
             @Override
@@ -177,8 +177,27 @@ public final class DoubleRange extends Range {
 
   }
 
-  @Override
+  /**
+   * @deprecated Use {@link #getQuery(Query, DoubleValuesSource)}
+   */
+  @Deprecated
   public Query getQuery(final Query fastMatchQuery, final ValueSource valueSource) {
+    return new ValueSourceQuery(this, fastMatchQuery, valueSource.asDoubleValuesSource());
+  }
+
+  /**
+   * Create a Query that matches documents in this range
+   *
+   * The query will check all documents that match the provided match query,
+   * or every document in the index if the match query is null.
+   *
+   * If the value source is static, eg an indexed numeric field, it may be
+   * faster to use {@link org.apache.lucene.search.PointRangeQuery}
+   *
+   * @param fastMatchQuery a query to use as a filter
+   * @param valueSource    the source of values for the range check
+   */
+  public Query getQuery(Query fastMatchQuery, DoubleValuesSource valueSource) {
     return new ValueSourceQuery(this, fastMatchQuery, valueSource);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a238610b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java
index 2d0ba5c..63fc935 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java
@@ -17,22 +17,18 @@
 package org.apache.lucene.facet.range;
 
 import java.io.IOException;
-import java.util.Collections;
 import java.util.List;
 
-import org.apache.lucene.document.DoubleDocValuesField; // javadocs
-import org.apache.lucene.document.FloatDocValuesField; // javadocs
+import org.apache.lucene.document.FloatDocValuesField;
 import org.apache.lucene.facet.Facets;
-import org.apache.lucene.facet.FacetsCollector.MatchingDocs;
 import org.apache.lucene.facet.FacetsCollector;
+import org.apache.lucene.facet.FacetsCollector.MatchingDocs;
 import org.apache.lucene.index.IndexReaderContext;
 import org.apache.lucene.index.ReaderUtil;
-import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.queries.function.valuesource.DoubleFieldSource;
-import org.apache.lucene.queries.function.valuesource.FloatFieldSource; // javadocs
-import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.DoubleValues;
+import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Scorer;
@@ -41,46 +37,70 @@ import org.apache.lucene.util.NumericUtils;
 
 /** {@link Facets} implementation that computes counts for
  *  dynamic double ranges from a provided {@link
- *  ValueSource}, using {@link FunctionValues#doubleVal}.  Use
- *  this for dimensions that change in real-time (e.g. a
+ *  DoubleValuesSource}.  Use this for dimensions that change in real-time (e.g. a
  *  relative time based dimension like "Past day", "Past 2
  *  days", etc.) or that change for each request (e.g.
  *  distance from the user's location, "&lt; 1 km", "&lt; 2 km",
  *  etc.).
  *
- *  <p> If you had indexed your field using {@link
- *  FloatDocValuesField} then pass {@link FloatFieldSource}
- *  as the {@link ValueSource}; if you used {@link
- *  DoubleDocValuesField} then pass {@link
- *  DoubleFieldSource} (this is the default used when you
- *  pass just a the field name).
+ *  If you have indexed your field using {@link
+ *  FloatDocValuesField}, then you should use a DoubleValuesSource
+ *  generated from {@link DoubleValuesSource#fromFloatField(String)}.
  *
  *  @lucene.experimental */
 public class DoubleRangeFacetCounts extends RangeFacetCounts {
 
-  /** Create {@code RangeFacetCounts}, using {@link
-   *  DoubleFieldSource} from the specified field. */
+  /**
+   * Create {@code RangeFacetCounts}, using {@link DoubleValues} from the specified field.
+   *
+   * N.B This assumes that the field was indexed with {@link org.apache.lucene.document.DoubleDocValuesField}.
+   * For float-valued fields, use {@link #DoubleRangeFacetCounts(String, DoubleValuesSource, FacetsCollector, DoubleRange...)}
+   */
   public DoubleRangeFacetCounts(String field, FacetsCollector hits, DoubleRange... ranges) throws IOException {
-    this(field, new DoubleFieldSource(field), hits, ranges);
+    this(field, DoubleValuesSource.fromDoubleField(field), hits, ranges);
   }
 
-  /** Create {@code RangeFacetCounts}, using the provided
-   *  {@link ValueSource}. */
+  /**
+   * Create {@code RangeFacetCounts}, using the provided {@link ValueSource}.
+   *
+   * @deprecated Use {@link #DoubleRangeFacetCounts(String, DoubleValuesSource, FacetsCollector, DoubleRange...)}
+   * */
   public DoubleRangeFacetCounts(String field, ValueSource valueSource, FacetsCollector hits, DoubleRange... ranges) throws IOException {
     this(field, valueSource, hits, null, ranges);
   }
 
-  /** Create {@code RangeFacetCounts}, using the provided
-   *  {@link ValueSource}, and using the provided Query as
-   *  a fastmatch: only documents passing the filter are
-   *  checked for the matching ranges.  The filter must be
-   *  random access (implement {@link DocIdSet#bits}). */
+  /**
+   * Create {@code RangeFacetCounts} using the provided {@link DoubleValuesSource}
+   */
+  public DoubleRangeFacetCounts(String field, DoubleValuesSource valueSource, FacetsCollector hits, DoubleRange... ranges) throws IOException {
+    this(field, valueSource, hits, null, ranges);
+  }
+
+  /**
+   * Create {@code RangeFacetCounts}, using the provided
+   * {@link ValueSource}, and using the provided Query as
+   * a fastmatch: only documents matching the query are
+   * checked for the matching ranges.
+   *
+   * @deprecated Use ({@link #DoubleRangeFacetCounts(String, DoubleValuesSource, FacetsCollector, Query, DoubleRange...)}
+   */
+  @Deprecated
   public DoubleRangeFacetCounts(String field, ValueSource valueSource, FacetsCollector hits, Query fastMatchQuery, DoubleRange... ranges) throws IOException {
+    this(field, valueSource.asDoubleValuesSource(), hits, fastMatchQuery, ranges);
+  }
+
+  /**
+   * Create {@code RangeFacetCounts}, using the provided
+   * {@link DoubleValuesSource}, and using the provided Query as
+   * a fastmatch: only documents matching the query are
+   * checked for the matching ranges.
+   */
+ public DoubleRangeFacetCounts(String field, DoubleValuesSource valueSource, FacetsCollector hits, Query fastMatchQuery, DoubleRange... ranges) throws IOException {
     super(field, ranges, fastMatchQuery);
     count(valueSource, hits.getMatchingDocs());
   }
 
-  private void count(ValueSource valueSource, List<MatchingDocs> matchingDocs) throws IOException {
+  private void count(DoubleValuesSource valueSource, List<MatchingDocs> matchingDocs) throws IOException {
 
     DoubleRange[] ranges = (DoubleRange[]) this.ranges;
 
@@ -96,7 +116,7 @@ public class DoubleRangeFacetCounts extends RangeFacetCounts {
 
     int missingCount = 0;
     for (MatchingDocs hits : matchingDocs) {
-      FunctionValues fv = valueSource.getValues(Collections.emptyMap(), hits.context);
+      DoubleValues fv = valueSource.getValues(hits.context, null);
       
       totCount += hits.totalHits;
       final DocIdSetIterator fastMatchDocs;
@@ -129,8 +149,8 @@ public class DoubleRangeFacetCounts extends RangeFacetCounts {
           }
         }
         // Skip missing docs:
-        if (fv.exists(doc)) {
-          counter.add(NumericUtils.doubleToSortableLong(fv.doubleVal(doc)));
+        if (fv.advanceExact(doc)) {
+          counter.add(NumericUtils.doubleToSortableLong(fv.doubleValue()));
         } else {
           missingCount++;
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a238610b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java
index 46ee00b..2e7978b 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java
@@ -17,17 +17,17 @@
 package org.apache.lucene.facet.range;
 
 import java.io.IOException;
-import java.util.Collections;
 import java.util.Objects;
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.ConstantScoreScorer;
 import org.apache.lucene.search.ConstantScoreWeight;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.LongValues;
+import org.apache.lucene.search.LongValuesSource;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.TwoPhaseIterator;
@@ -87,9 +87,9 @@ public final class LongRange extends Range {
   private static class ValueSourceQuery extends Query {
     private final LongRange range;
     private final Query fastMatchQuery;
-    private final ValueSource valueSource;
+    private final LongValuesSource valueSource;
 
-    ValueSourceQuery(LongRange range, Query fastMatchQuery, ValueSource valueSource) {
+    ValueSourceQuery(LongRange range, Query fastMatchQuery, LongValuesSource valueSource) {
       this.range = range;
       this.fastMatchQuery = fastMatchQuery;
       this.valueSource = valueSource;
@@ -150,11 +150,11 @@ public final class LongRange extends Range {
             approximation = s.iterator();
           }
 
-          final FunctionValues values = valueSource.getValues(Collections.emptyMap(), context);
+          final LongValues values = valueSource.getValues(context, null);
           final TwoPhaseIterator twoPhase = new TwoPhaseIterator(approximation) {
             @Override
             public boolean matches() throws IOException {
-              return range.accept(values.longVal(approximation.docID()));
+              return values.advanceExact(approximation.docID()) && range.accept(values.longValue());
             }
 
             @Override
@@ -169,8 +169,28 @@ public final class LongRange extends Range {
 
   }
 
-  @Override
+
+  /**
+   * @deprecated Use {@link #getQuery(Query, LongValuesSource)}
+   */
+  @Deprecated
   public Query getQuery(final Query fastMatchQuery, final ValueSource valueSource) {
+    return new ValueSourceQuery(this, fastMatchQuery, valueSource.asLongValuesSource());
+  }
+
+  /**
+   * Create a Query that matches documents in this range
+   *
+   * The query will check all documents that match the provided match query,
+   * or every document in the index if the match query is null.
+   *
+   * If the value source is static, eg an indexed numeric field, it may be
+   * faster to use {@link org.apache.lucene.search.PointRangeQuery}
+   *
+   * @param fastMatchQuery a query to use as a filter
+   * @param valueSource    the source of values for the range check
+   */
+  public Query getQuery(Query fastMatchQuery, LongValuesSource valueSource) {
     return new ValueSourceQuery(this, fastMatchQuery, valueSource);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a238610b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java
index 0512ab3..a3cfc71 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java
@@ -17,27 +17,25 @@
 package org.apache.lucene.facet.range;
 
 import java.io.IOException;
-import java.util.Collections;
 import java.util.List;
 
 import org.apache.lucene.facet.Facets;
-import org.apache.lucene.facet.FacetsCollector.MatchingDocs;
 import org.apache.lucene.facet.FacetsCollector;
+import org.apache.lucene.facet.FacetsCollector.MatchingDocs;
 import org.apache.lucene.index.IndexReaderContext;
 import org.apache.lucene.index.ReaderUtil;
-import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.queries.function.valuesource.LongFieldSource;
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.LongValues;
+import org.apache.lucene.search.LongValuesSource;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Weight;
 
 /** {@link Facets} implementation that computes counts for
- *  dynamic long ranges from a provided {@link ValueSource},
- *  using {@link FunctionValues#longVal}.  Use
+ *  dynamic long ranges from a provided {@link LongValuesSource}.  Use
  *  this for dimensions that change in real-time (e.g. a
  *  relative time based dimension like "Past day", "Past 2
  *  days", etc.) or that change for each request (e.g. 
@@ -48,28 +46,49 @@ import org.apache.lucene.search.Weight;
 public class LongRangeFacetCounts extends RangeFacetCounts {
 
   /** Create {@code LongRangeFacetCounts}, using {@link
-   *  LongFieldSource} from the specified field. */
+   *  LongValuesSource} from the specified field. */
   public LongRangeFacetCounts(String field, FacetsCollector hits, LongRange... ranges) throws IOException {
-    this(field, new LongFieldSource(field), hits, ranges);
+    this(field, LongValuesSource.fromLongField(field), hits, ranges);
+  }
+
+  /**
+   * Create {@code RangeFacetCounts}, using the provided {@link ValueSource}.
+   *
+   * @deprecated Use {@link #LongRangeFacetCounts(String, LongValuesSource, FacetsCollector, LongRange...)}
+   */
+  @Deprecated
+  public LongRangeFacetCounts(String field, ValueSource valueSource, FacetsCollector hits, LongRange... ranges) throws IOException {
+    this(field, valueSource.asLongValuesSource(), hits, null, ranges);
   }
 
   /** Create {@code RangeFacetCounts}, using the provided
    *  {@link ValueSource}. */
-  public LongRangeFacetCounts(String field, ValueSource valueSource, FacetsCollector hits, LongRange... ranges) throws IOException {
+  public LongRangeFacetCounts(String field, LongValuesSource valueSource, FacetsCollector hits, LongRange... ranges) throws IOException {
     this(field, valueSource, hits, null, ranges);
   }
 
+  /**
+   * Create {@code RangeFacetCounts}, using the provided {@link ValueSource}.
+   *
+   * @deprecated Use {@link #LongRangeFacetCounts(String, LongValuesSource, FacetsCollector, Query, LongRange...)}
+   */
+  @Deprecated
+  public LongRangeFacetCounts(String field, ValueSource valueSource, FacetsCollector hits, Query fastMatchQuery, LongRange... ranges) throws IOException {
+    this(field, valueSource.asLongValuesSource(), hits, fastMatchQuery, ranges);
+  }
+
+
   /** Create {@code RangeFacetCounts}, using the provided
    *  {@link ValueSource}, and using the provided Filter as
    *  a fastmatch: only documents passing the filter are
    *  checked for the matching ranges.  The filter must be
    *  random access (implement {@link DocIdSet#bits}). */
-  public LongRangeFacetCounts(String field, ValueSource valueSource, FacetsCollector hits, Query fastMatchQuery, LongRange... ranges) throws IOException {
+  public LongRangeFacetCounts(String field, LongValuesSource valueSource, FacetsCollector hits, Query fastMatchQuery, LongRange... ranges) throws IOException {
     super(field, ranges, fastMatchQuery);
     count(valueSource, hits.getMatchingDocs());
   }
 
-  private void count(ValueSource valueSource, List<MatchingDocs> matchingDocs) throws IOException {
+  private void count(LongValuesSource valueSource, List<MatchingDocs> matchingDocs) throws IOException {
 
     LongRange[] ranges = (LongRange[]) this.ranges;
 
@@ -77,7 +96,7 @@ public class LongRangeFacetCounts extends RangeFacetCounts {
 
     int missingCount = 0;
     for (MatchingDocs hits : matchingDocs) {
-      FunctionValues fv = valueSource.getValues(Collections.emptyMap(), hits.context);
+      LongValues fv = valueSource.getValues(hits.context, null);
       
       totCount += hits.totalHits;
       final DocIdSetIterator fastMatchDocs;
@@ -109,8 +128,8 @@ public class LongRangeFacetCounts extends RangeFacetCounts {
           }
         }
         // Skip missing docs:
-        if (fv.exists(doc)) {
-          counter.add(fv.longVal(doc));
+        if (fv.advanceExact(doc)) {
+          counter.add(fv.longValue());
         } else {
           missingCount++;
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a238610b/lucene/facet/src/java/org/apache/lucene/facet/range/Range.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/Range.java b/lucene/facet/src/java/org/apache/lucene/facet/range/Range.java
index 5f6de98..82b8088 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/range/Range.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/range/Range.java
@@ -16,10 +16,6 @@
  */
 package org.apache.lucene.facet.range;
 
-import org.apache.lucene.facet.DrillDownQuery; // javadocs
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.search.Query;
-
 /** Base class for a single labeled range.
  *
  *  @lucene.experimental */
@@ -36,33 +32,6 @@ public abstract class Range {
     this.label = label;
   }
 
-  /** Returns a new {@link Query} accepting only documents
-   *  in this range.  This query might not be very efficient
-   *  when run on its own since it is optimized towards
-   *  random-access, so it is best used either with
-   *  {@link DrillDownQuery#add(String, Query) DrillDownQuery}
-   *  or when intersected with another query that can lead the
-   *  iteration.  If the {@link ValueSource} is static, e.g. an
-   *  indexed numeric field, then it may be more efficient to use
-   *  {@link org.apache.lucene.search.PointRangeQuery}. The provided fastMatchQuery,
-   *  if non-null, will first be consulted, and only if
-   *  that is set for each document will the range then be
-   *  checked. */
-  public abstract Query getQuery(Query fastMatchQuery, ValueSource valueSource);
-
-  /** Returns a new {@link Query} accepting only documents
-   *  in this range.  This query might not be very efficient
-   *  when run on its own since it is optimized towards
-   *  random-access, so it is best used either with
-   *  {@link DrillDownQuery#add(String, Query) DrillDownQuery}
-   *  or when intersected with another query that can lead the
-   *  iteration.  If the {@link ValueSource} is static, e.g. an
-   *  indexed numeric field, then it may be more efficient to
-   *  use {@link org.apache.lucene.search.PointRangeQuery}. */
-  public Query getQuery(ValueSource valueSource) {
-    return getQuery(null, valueSource);
-  }
-
   /** Invoke this for a useless range. */
   protected void failNoMatch() {
     throw new IllegalArgumentException("range \"" + label + "\" matches nothing");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a238610b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FakeScorer.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FakeScorer.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FakeScorer.java
deleted file mode 100644
index 238b74c..0000000
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FakeScorer.java
+++ /dev/null
@@ -1,53 +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.lucene.facet.taxonomy;
-
-import java.io.IOException;
-
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.Scorer;
-
-class FakeScorer extends Scorer {
-
-  float score;
-  int doc = -1;
-  int freq = 1;
-
-  FakeScorer() {
-    super(null);
-  }
-
-  @Override
-  public int docID() {
-    return doc;
-  }
-
-  @Override
-  public DocIdSetIterator iterator() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public int freq() throws IOException {
-    return freq;
-  }
-
-  @Override
-  public float score() throws IOException {
-    return score;
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a238610b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java
index 4010c81..0a73ae5 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java
@@ -17,7 +17,6 @@
 package org.apache.lucene.facet.taxonomy;
 
 import java.io.IOException;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
@@ -29,6 +28,8 @@ import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
 import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.DoubleValues;
+import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.util.IntsRef;
 
@@ -39,52 +40,94 @@ import org.apache.lucene.util.IntsRef;
 public class TaxonomyFacetSumValueSource extends FloatTaxonomyFacets {
   private final OrdinalsReader ordinalsReader;
 
-  /** Aggreggates float facet values from the provided
+  /**
+   * Aggreggates double facet values from the provided
    *  {@link ValueSource}, pulling ordinals using {@link
    *  DocValuesOrdinalsReader} against the default indexed
    *  facet field {@link
-   *  FacetsConfig#DEFAULT_INDEX_FIELD_NAME}. */
+   *  FacetsConfig#DEFAULT_INDEX_FIELD_NAME}.
+   *
+   *  @deprecated {@link #TaxonomyFacetSumValueSource(TaxonomyReader, FacetsConfig, FacetsCollector, DoubleValuesSource)}
+   */
+  @Deprecated
   public TaxonomyFacetSumValueSource(TaxonomyReader taxoReader, FacetsConfig config,
                                      FacetsCollector fc, ValueSource valueSource) throws IOException {
     this(new DocValuesOrdinalsReader(FacetsConfig.DEFAULT_INDEX_FIELD_NAME), taxoReader, config, fc, valueSource);
   }
 
-  /** Aggreggates float facet values from the provided
+  /**
+   * Aggreggates double facet values from the provided
+   * {@link DoubleValuesSource}, pulling ordinals using {@link
+   * DocValuesOrdinalsReader} against the default indexed
+   * facet field {@link FacetsConfig#DEFAULT_INDEX_FIELD_NAME}.
+   */
+   public TaxonomyFacetSumValueSource(TaxonomyReader taxoReader, FacetsConfig config,
+                                     FacetsCollector fc, DoubleValuesSource valueSource) throws IOException {
+    this(new DocValuesOrdinalsReader(FacetsConfig.DEFAULT_INDEX_FIELD_NAME), taxoReader, config, fc, valueSource);
+  }
+
+  /**
+   * Aggreggates float facet values from the provided
    *  {@link ValueSource}, and pulls ordinals from the
-   *  provided {@link OrdinalsReader}. */
+   *  provided {@link OrdinalsReader}.
+   *
+   *  @deprecated use {@link #TaxonomyFacetSumValueSource(OrdinalsReader, TaxonomyReader, FacetsConfig, FacetsCollector, DoubleValuesSource)}
+   */
+  @Deprecated
   public TaxonomyFacetSumValueSource(OrdinalsReader ordinalsReader, TaxonomyReader taxoReader,
                                      FacetsConfig config, FacetsCollector fc, ValueSource valueSource) throws IOException {
     super(ordinalsReader.getIndexFieldName(), taxoReader, config);
     this.ordinalsReader = ordinalsReader;
-    sumValues(fc.getMatchingDocs(), fc.getKeepScores(), valueSource);
+    sumValues(fc.getMatchingDocs(), fc.getKeepScores(), valueSource.asDoubleValuesSource());
   }
 
-  private final void sumValues(List<MatchingDocs> matchingDocs, boolean keepScores, ValueSource valueSource) throws IOException {
-    final FakeScorer scorer = new FakeScorer();
-    Map<String, Scorer> context = new HashMap<>();
-    if (keepScores) {
-      context.put("scorer", scorer);
-    }
+  /**
+   * Aggreggates float facet values from the provided
+   *  {@link DoubleValuesSource}, and pulls ordinals from the
+   *  provided {@link OrdinalsReader}.
+   */
+   public TaxonomyFacetSumValueSource(OrdinalsReader ordinalsReader, TaxonomyReader taxoReader,
+                                     FacetsConfig config, FacetsCollector fc, DoubleValuesSource vs) throws IOException {
+    super(ordinalsReader.getIndexFieldName(), taxoReader, config);
+    this.ordinalsReader = ordinalsReader;
+    sumValues(fc.getMatchingDocs(), fc.getKeepScores(), vs);
+  }
+
+  private static DoubleValues scores(MatchingDocs hits) {
+    return new DoubleValues() {
+
+      int index = -1;
+
+      @Override
+      public double doubleValue() throws IOException {
+        return hits.scores[index];
+      }
+
+      @Override
+      public boolean advanceExact(int doc) throws IOException {
+        index++;
+        return true;
+      }
+    };
+  }
+
+  private void sumValues(List<MatchingDocs> matchingDocs, boolean keepScores, DoubleValuesSource valueSource) throws IOException {
+
     IntsRef scratch = new IntsRef();
     for(MatchingDocs hits : matchingDocs) {
       OrdinalsReader.OrdinalsSegmentReader ords = ordinalsReader.getReader(hits.context);
-      
-      int scoresIdx = 0;
-      float[] scores = hits.scores;
-
-      FunctionValues functionValues = valueSource.getValues(context, hits.context);
+      DoubleValues scores = keepScores ? scores(hits) : null;
+      DoubleValues functionValues = valueSource.getValues(hits.context, scores);
       DocIdSetIterator docs = hits.bits.iterator();
       
       int doc;
       while ((doc = docs.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
         ords.get(doc, scratch);
-        if (keepScores) {
-          scorer.doc = doc;
-          scorer.score = scores[scoresIdx++];
-        }
-        float value = (float) functionValues.doubleVal(doc);
-        for(int i=0;i<scratch.length;i++) {
-          values[scratch.ints[i]] += value;
+        if (functionValues.advanceExact(doc)) {
+          float value = (float) functionValues.doubleValue();
+          for (int i = 0; i < scratch.length; i++) {
+            values[scratch.ints[i]] += value;
+          }
         }
       }
     }
@@ -92,9 +135,13 @@ public class TaxonomyFacetSumValueSource extends FloatTaxonomyFacets {
     rollup();
   }
 
-  /** {@link ValueSource} that returns the score for each
+  /**
+   * {@link ValueSource} that returns the score for each
    *  hit; use this to aggregate the sum of all hit scores
-   *  for each facet label.  */
+   *  for each facet label.
+   *
+   * @deprecated Use {@link DoubleValuesSource#SCORES}
+   */
   public static class ScoreValueSource extends ValueSource {
 
     /** Sole constructor. */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a238610b/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java b/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
index b9ff1ce..c5734c6 100644
--- a/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
+++ b/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
@@ -28,8 +28,8 @@ import org.apache.lucene.document.DoublePoint;
 import org.apache.lucene.document.LongPoint;
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.facet.DrillDownQuery;
-import org.apache.lucene.facet.DrillSideways.DrillSidewaysResult;
 import org.apache.lucene.facet.DrillSideways;
+import org.apache.lucene.facet.DrillSideways.DrillSidewaysResult;
 import org.apache.lucene.facet.FacetField;
 import org.apache.lucene.facet.FacetResult;
 import org.apache.lucene.facet.FacetTestCase;
@@ -46,11 +46,11 @@ import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
 import org.apache.lucene.queries.function.valuesource.DoubleFieldSource;
 import org.apache.lucene.queries.function.valuesource.LongFieldSource;
+import org.apache.lucene.search.DoubleValues;
+import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
@@ -718,7 +718,7 @@ public class TestRangeFacetCounts extends FacetTestCase {
 
   }
 
-  public void testCustomDoublesValueSource() throws Exception {
+  public void testCustomDoubleValuesSource() throws Exception {
     Directory dir = newDirectory();
     RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
     
@@ -730,33 +730,30 @@ public class TestRangeFacetCounts extends FacetTestCase {
     // Test wants 3 docs in one segment:
     writer.forceMerge(1);
 
-    final ValueSource vs = new ValueSource() {
-        @SuppressWarnings("rawtypes")
-        @Override
-        public FunctionValues getValues(Map ignored, LeafReaderContext ignored2) {
-          return new DoubleDocValues(null) {
-            @Override
-            public double doubleVal(int doc) {
-              return doc+1;
-            }
-          };
-        }
+    final DoubleValuesSource vs = new DoubleValuesSource() {
 
-        @Override
-        public boolean equals(Object o) {
-          return o != null && getClass() == o.getClass();
-        }
+      @Override
+      public DoubleValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
+        return new DoubleValues() {
+          int doc = -1;
+          @Override
+          public double doubleValue() throws IOException {
+            return doc + 1;
+          }
 
-        @Override
-        public int hashCode() {
-          return getClass().hashCode();
-        }
+          @Override
+          public boolean advanceExact(int doc) throws IOException {
+            this.doc = doc;
+            return true;
+          }
+        };
+      }
 
-        @Override
-        public String description() {
-          throw new UnsupportedOperationException();
-        }
-      };
+      @Override
+      public boolean needsScores() {
+        return false;
+      }
+    };
 
     FacetsConfig config = new FacetsConfig();
     

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a238610b/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetSumValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetSumValueSource.java b/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetSumValueSource.java
index 0ad90ba..31bf6e1 100644
--- a/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetSumValueSource.java
+++ b/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetSumValueSource.java
@@ -52,6 +52,7 @@ import org.apache.lucene.queries.function.valuesource.IntFieldSource;
 import org.apache.lucene.queries.function.valuesource.LongFieldSource;
 import org.apache.lucene.search.BoostQuery;
 import org.apache.lucene.search.ConstantScoreQuery;
+import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.Query;
@@ -266,7 +267,7 @@ public class TestTaxonomyFacetSumValueSource extends FacetTestCase {
     
     TopDocs td = FacetsCollector.search(newSearcher(r), csq, 10, fc);
 
-    Facets facets = new TaxonomyFacetSumValueSource(taxoReader, config, fc, new TaxonomyFacetSumValueSource.ScoreValueSource());
+    Facets facets = new TaxonomyFacetSumValueSource(taxoReader, config, fc, DoubleValuesSource.SCORES);
     
     int expected = (int) (td.getMaxScore() * td.totalHits);
     assertEquals(expected, facets.getSpecificValue("dim", "a").intValue());
@@ -408,7 +409,7 @@ public class TestTaxonomyFacetSumValueSource extends FacetTestCase {
     FacetsCollector.search(newSearcher(r), new MatchAllDocsQuery(), 10, fc);
     
     Facets facets1 = getTaxonomyFacetCounts(taxoReader, config, fc);
-    Facets facets2 = new TaxonomyFacetSumValueSource(new DocValuesOrdinalsReader("$b"), taxoReader, config, fc, new TaxonomyFacetSumValueSource.ScoreValueSource());
+    Facets facets2 = new TaxonomyFacetSumValueSource(new DocValuesOrdinalsReader("$b"), taxoReader, config, fc, DoubleValuesSource.SCORES);
 
     assertEquals(r.maxDoc(), facets1.getTopChildren(10, "a").value.intValue());
     assertEquals(r.maxDoc(), facets2.getTopChildren(10, "b").value.doubleValue(), 1E-10);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a238610b/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSource.java b/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSource.java
index 49d4b77..ce804d1 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSource.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSource.java
@@ -17,13 +17,20 @@
 package org.apache.lucene.queries.function;
 
 import java.io.IOException;
+import java.util.HashMap;
 import java.util.IdentityHashMap;
 import java.util.Map;
 
 import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.DoubleValues;
+import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.FieldComparator;
 import org.apache.lucene.search.FieldComparatorSource;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.LongValues;
+import org.apache.lucene.search.LongValuesSource;
+import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.SimpleFieldComparator;
 import org.apache.lucene.search.SortField;
 
@@ -76,6 +83,110 @@ public abstract class ValueSource {
     return context;
   }
 
+  private static class FakeScorer extends Scorer {
+
+    int current = -1;
+    float score = 0;
+
+    FakeScorer() {
+      super(null);
+    }
+
+    @Override
+    public int docID() {
+      return current;
+    }
+
+    @Override
+    public float score() throws IOException {
+      return score;
+    }
+
+    @Override
+    public int freq() throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public DocIdSetIterator iterator() {
+      throw new UnsupportedOperationException();
+    }
+  }
+
+  /**
+   * Expose this ValueSource as a LongValuesSource
+   */
+  public LongValuesSource asLongValuesSource() {
+    return new LongValuesSource() {
+      @Override
+      public LongValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
+        Map context = new IdentityHashMap<>();
+        FakeScorer scorer = new FakeScorer();
+        context.put("scorer", scorer);
+        final FunctionValues fv = ValueSource.this.getValues(context, ctx);
+        return new LongValues() {
+
+          @Override
+          public long longValue() throws IOException {
+            return fv.longVal(scorer.current);
+          }
+
+          @Override
+          public boolean advanceExact(int doc) throws IOException {
+            scorer.current = doc;
+            if (scores != null && scores.advanceExact(doc))
+              scorer.score = (float) scores.doubleValue();
+            else
+              scorer.score = 0;
+            return fv.exists(doc);
+          }
+        };
+      }
+
+      @Override
+      public boolean needsScores() {
+        return false;
+      }
+    };
+  }
+
+  /**
+   * Expose this ValueSource as a DoubleValuesSource
+   */
+  public DoubleValuesSource asDoubleValuesSource() {
+    return new DoubleValuesSource() {
+      @Override
+      public DoubleValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
+        Map context = new HashMap<>();
+        FakeScorer scorer = new FakeScorer();
+        context.put("scorer", scorer);
+        FunctionValues fv = ValueSource.this.getValues(context, ctx);
+        return new DoubleValues() {
+
+          @Override
+          public double doubleValue() throws IOException {
+            return fv.doubleVal(scorer.current);
+          }
+
+          @Override
+          public boolean advanceExact(int doc) throws IOException {
+            scorer.current = doc;
+            if (scores != null && scores.advanceExact(doc)) {
+              scorer.score = (float) scores.doubleValue();
+            }
+            else
+              scorer.score = 0;
+            return fv.exists(doc);
+          }
+        };
+      }
+
+      @Override
+      public boolean needsScores() {
+        return true;  // be on the safe side
+      }
+    };
+  }
 
   //
   // Sorting by function


[2/6] lucene-solr:branch_6x: LUCENE-7609: Refactor expressions module to use DoubleValuesSource

Posted by ro...@apache.org.
LUCENE-7609: Refactor expressions module to use DoubleValuesSource


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/776087ee
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/776087ee
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/776087ee

Branch: refs/heads/branch_6x
Commit: 776087eef48dbeba639b94b574f806b7265a7ffe
Parents: d4d3ede
Author: Alan Woodward <ro...@apache.org>
Authored: Thu Jan 5 13:15:19 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Sat Jan 7 11:08:51 2017 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   5 +
 .../ExpressionAggregationFacetsExample.java     |   2 +-
 lucene/expressions/build.xml                    |  11 --
 .../org/apache/lucene/expressions/Bindings.java |  14 +--
 .../apache/lucene/expressions/Expression.java   |  21 ++--
 .../expressions/ExpressionComparator.java       | 100 -----------------
 .../expressions/ExpressionFunctionValues.java   |  35 +++---
 .../lucene/expressions/ExpressionRescorer.java  |  33 ++++--
 .../lucene/expressions/ExpressionSortField.java |  77 -------------
 .../expressions/ExpressionValueSource.java      |  70 ++++++------
 .../lucene/expressions/ScoreFunctionValues.java |  46 --------
 .../lucene/expressions/ScoreValueSource.java    |  61 ----------
 .../lucene/expressions/SimpleBindings.java      |  30 +++--
 .../expressions/js/JavascriptCompiler.java      |  17 ++-
 .../apache/lucene/expressions/package-info.java |   5 +-
 .../lucene/expressions/TestDemoExpressions.java |  44 ++++++--
 .../expressions/TestExpressionSortField.java    |   2 +-
 .../expressions/TestExpressionValueSource.java  | 110 ++++++-------------
 .../expressions/js/TestCustomFunctions.java     |  20 ++--
 .../expressions/js/TestJavascriptFunction.java  |   2 +-
 .../js/TestJavascriptOperations.java            |   2 +-
 21 files changed, 198 insertions(+), 509 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/776087ee/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 9d7b99b..74ee5ef 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -19,6 +19,11 @@ API Changes
   grouping Collectors are renamed to remove the Abstract* prefix.
   (Alan Woodward, Martijn van Groningen)
 
+* LUCENE-7609: The expressions module now uses the DoubleValuesSource API, and 
+  no longer depends on the queries module.  Expression#getValueSource() is
+  replaced with Expression#getDoubleValuesSource(). (Alan Woodward, Adrien
+  Grand)
+
 New features
 
 * LUCENE-5867: Added BooleanSimilarity. (Robert Muir, Adrien Grand)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/776087ee/lucene/demo/src/java/org/apache/lucene/demo/facet/ExpressionAggregationFacetsExample.java
----------------------------------------------------------------------
diff --git a/lucene/demo/src/java/org/apache/lucene/demo/facet/ExpressionAggregationFacetsExample.java b/lucene/demo/src/java/org/apache/lucene/demo/facet/ExpressionAggregationFacetsExample.java
index fca7b6c..7f29b38 100644
--- a/lucene/demo/src/java/org/apache/lucene/demo/facet/ExpressionAggregationFacetsExample.java
+++ b/lucene/demo/src/java/org/apache/lucene/demo/facet/ExpressionAggregationFacetsExample.java
@@ -103,7 +103,7 @@ public class ExpressionAggregationFacetsExample {
     FacetsCollector.search(searcher, new MatchAllDocsQuery(), 10, fc);
 
     // Retrieve results
-    Facets facets = new TaxonomyFacetSumValueSource(taxoReader, config, fc, expr.getValueSource(bindings));
+    Facets facets = new TaxonomyFacetSumValueSource(taxoReader, config, fc, expr.getDoubleValuesSource(bindings));
     FacetResult result = facets.getTopChildren(10, "A");
     
     indexReader.close();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/776087ee/lucene/expressions/build.xml
----------------------------------------------------------------------
diff --git a/lucene/expressions/build.xml b/lucene/expressions/build.xml
index 1dddc6d..61ae64f 100644
--- a/lucene/expressions/build.xml
+++ b/lucene/expressions/build.xml
@@ -26,7 +26,6 @@
   <path id="classpath">
     <path refid="base.classpath"/>
     <fileset dir="lib"/>
-    <pathelement path="${queries.jar}"/>
   </path>
 
   <path id="test.classpath">
@@ -35,16 +34,6 @@
     <pathelement path="src/test-files"/>
   </path>
 
-  <target name="compile-core" depends="jar-queries,common.compile-core" />
-
-  <target name="javadocs" depends="javadocs-queries,compile-core,check-javadocs-uptodate" unless="javadocs-uptodate-${name}">
-    <invoke-module-javadoc>
-      <links>
-        <link href="../queries"/>
-      </links>
-    </invoke-module-javadoc>
-  </target>
-
   <target name="regenerate" depends="run-antlr"/>
 
   <target name="resolve-antlr" xmlns:ivy="antlib:org.apache.ivy.ant">

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/776087ee/lucene/expressions/src/java/org/apache/lucene/expressions/Bindings.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/java/org/apache/lucene/expressions/Bindings.java b/lucene/expressions/src/java/org/apache/lucene/expressions/Bindings.java
index 5ec2edb..91a22f5 100644
--- a/lucene/expressions/src/java/org/apache/lucene/expressions/Bindings.java
+++ b/lucene/expressions/src/java/org/apache/lucene/expressions/Bindings.java
@@ -16,7 +16,7 @@
  */
 package org.apache.lucene.expressions;
 
-import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.search.DoubleValuesSource;
 
 /**
  * Binds variable names in expressions to actual data.
@@ -31,14 +31,10 @@ public abstract class Bindings {
   /** Sole constructor. (For invocation by subclass 
    *  constructors, typically implicit.) */
   protected Bindings() {}
-  
+
   /**
-   * Returns a ValueSource bound to the variable name.
+   * Returns a DoubleValuesSource bound to the variable name
    */
-  public abstract ValueSource getValueSource(String name);
-  
-  /** Returns a {@code ValueSource} over relevance scores */
-  protected final ValueSource getScoreValueSource() {
-    return new ScoreValueSource();
-  }
+  public abstract DoubleValuesSource getDoubleValuesSource(String name);
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/776087ee/lucene/expressions/src/java/org/apache/lucene/expressions/Expression.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/java/org/apache/lucene/expressions/Expression.java b/lucene/expressions/src/java/org/apache/lucene/expressions/Expression.java
index 02be23b..c92e21b 100644
--- a/lucene/expressions/src/java/org/apache/lucene/expressions/Expression.java
+++ b/lucene/expressions/src/java/org/apache/lucene/expressions/Expression.java
@@ -16,9 +16,9 @@
  */
 package org.apache.lucene.expressions;
 
-import org.apache.lucene.expressions.js.JavascriptCompiler; // javadocs
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.expressions.js.JavascriptCompiler;
+import org.apache.lucene.search.DoubleValues;
+import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.Rescorer;
 import org.apache.lucene.search.SortField;
 
@@ -63,26 +63,25 @@ public abstract class Expression {
   }
 
   /**
-   * Evaluates the expression for the given document.
+   * Evaluates the expression for the current document.
    *
-   * @param document <code>docId</code> of the document to compute a value for
-   * @param functionValues {@link FunctionValues} for each element of {@link #variables}.
+   * @param functionValues {@link DoubleValues} for each element of {@link #variables}.
    * @return The computed value of the expression for the given document.
    */
-  public abstract double evaluate(int document, FunctionValues[] functionValues);
+  public abstract double evaluate(DoubleValues[] functionValues);
 
   /**
-   * Get a value source which can compute the value of this expression in the context of the given bindings.
+   * Get a DoubleValuesSource which can compute the value of this expression in the context of the given bindings.
    * @param bindings Bindings to use for external values in this expression
-   * @return A value source which will evaluate this expression when used
+   * @return A DoubleValuesSource which will evaluate this expression when used
    */
-  public ValueSource getValueSource(Bindings bindings) {
+  public DoubleValuesSource getDoubleValuesSource(Bindings bindings) {
     return new ExpressionValueSource(bindings, this);
   }
   
   /** Get a sort field which can be used to rank documents by this expression. */
   public SortField getSortField(Bindings bindings, boolean reverse) {
-    return getValueSource(bindings).getSortField(reverse);
+    return getDoubleValuesSource(bindings).getSortField(reverse);
   }
 
   /** Get a {@link Rescorer}, to rescore first-pass hits

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/776087ee/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionComparator.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionComparator.java b/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionComparator.java
deleted file mode 100644
index eabf6dd..0000000
--- a/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionComparator.java
+++ /dev/null
@@ -1,100 +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.lucene.expressions;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.search.FieldComparator;
-import org.apache.lucene.search.LeafFieldComparator;
-import org.apache.lucene.search.Scorer;
-
-/** A custom comparator for sorting documents by an expression */
-class ExpressionComparator extends FieldComparator<Double> implements LeafFieldComparator {
-  private final double[] values;
-  private double bottom;
-  private double topValue;
-  
-  private ValueSource source;
-  private FunctionValues scores;
-  private LeafReaderContext readerContext;
-  
-  public ExpressionComparator(ValueSource source, int numHits) {
-    values = new double[numHits];
-    this.source = source;
-  }
-  
-  // TODO: change FieldComparator.setScorer to throw IOException and remove this try-catch
-  @Override
-  public void setScorer(Scorer scorer) {
-    // TODO: might be cleaner to lazy-init 'source' and set scorer after?
-    assert readerContext != null;
-    try {
-      Map<String,Object> context = new HashMap<>();
-      assert scorer != null;
-      context.put("scorer", scorer);
-      scores = source.getValues(context, readerContext);
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-  }
-  
-  @Override
-  public int compare(int slot1, int slot2) {
-    return Double.compare(values[slot1], values[slot2]);
-  }
-  
-  @Override
-  public void setBottom(int slot) {
-    bottom = values[slot];
-  }
-  
-  @Override
-  public void setTopValue(Double value) {
-    topValue = value.doubleValue();
-  }
-  
-  @Override
-  public int compareBottom(int doc) throws IOException {
-    return Double.compare(bottom, scores.doubleVal(doc));
-  }
-  
-  @Override
-  public void copy(int slot, int doc) throws IOException {
-    values[slot] = scores.doubleVal(doc);
-  }
-  
-  @Override
-  public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
-    this.readerContext = context;
-    return this;
-  }
-  
-  @Override
-  public Double value(int slot) {
-    return Double.valueOf(values[slot]);
-  }
-  
-  @Override
-  public int compareTop(int doc) throws IOException {
-    return Double.compare(topValue, scores.doubleVal(doc));
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/776087ee/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionFunctionValues.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionFunctionValues.java b/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionFunctionValues.java
index ad195cd..2e6f7c4 100644
--- a/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionFunctionValues.java
+++ b/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionFunctionValues.java
@@ -16,20 +16,16 @@
  */
 package org.apache.lucene.expressions;
 
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
+import java.io.IOException;
 
-/** A {@link FunctionValues} which evaluates an expression */
-class ExpressionFunctionValues extends DoubleDocValues {
+import org.apache.lucene.search.DoubleValues;
+
+/** A {@link DoubleValues} which evaluates an expression */
+class ExpressionFunctionValues extends DoubleValues {
   final Expression expression;
-  final FunctionValues[] functionValues;
-  
-  int currentDocument = -1;
-  double currentValue;
+  final DoubleValues[] functionValues;
   
-  ExpressionFunctionValues(ValueSource parent, Expression expression, FunctionValues[] functionValues) {
-    super(parent);
+  ExpressionFunctionValues(Expression expression, DoubleValues[] functionValues) {
     if (expression == null) {
       throw new NullPointerException();
     }
@@ -39,14 +35,17 @@ class ExpressionFunctionValues extends DoubleDocValues {
     this.expression = expression;
     this.functionValues = functionValues;
   }
-  
+
   @Override
-  public double doubleVal(int document) {
-    if (currentDocument != document) {
-      currentDocument = document;
-      currentValue = expression.evaluate(document, functionValues);
+  public boolean advanceExact(int doc) throws IOException {
+    for (DoubleValues v : functionValues) {
+      v.advanceExact(doc);
     }
-    
-    return currentValue;
+    return true;
+  }
+  
+  @Override
+  public double doubleValue() {
+    return expression.evaluate(functionValues);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/776087ee/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionRescorer.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionRescorer.java b/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionRescorer.java
index 33e8428..e3e7a4e 100644
--- a/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionRescorer.java
+++ b/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionRescorer.java
@@ -20,13 +20,11 @@ package org.apache.lucene.expressions;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.ReaderUtil;
-import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.search.DoubleValues;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Rescorer;
@@ -49,7 +47,7 @@ class ExpressionRescorer extends SortRescorer {
   private final Expression expression;
   private final Bindings bindings;
 
-  /** Uses the provided {@link ValueSource} to assign second
+  /** Uses the provided {@link Expression} to assign second
    *  pass scores. */
   public ExpressionRescorer(Expression expression, Bindings bindings) {
     super(new Sort(expression.getSortField(bindings, true)));
@@ -57,6 +55,21 @@ class ExpressionRescorer extends SortRescorer {
     this.bindings = bindings;
   }
 
+  private static DoubleValues scores(int doc, float score) {
+    return new DoubleValues() {
+      @Override
+      public double doubleValue() throws IOException {
+        return score;
+      }
+
+      @Override
+      public boolean advanceExact(int target) throws IOException {
+        assert doc == target;
+        return true;
+      }
+    };
+  }
+
   @Override
   public Explanation explain(IndexSearcher searcher, Explanation firstPassExplanation, int docID) throws IOException {
     Explanation superExpl = super.explain(searcher, firstPassExplanation, docID);
@@ -65,18 +78,14 @@ class ExpressionRescorer extends SortRescorer {
     int subReader = ReaderUtil.subIndex(docID, leaves);
     LeafReaderContext readerContext = leaves.get(subReader);
     int docIDInSegment = docID - readerContext.docBase;
-    Map<String,Object> context = new HashMap<>();
-
-    FakeScorer fakeScorer = new FakeScorer();
-    fakeScorer.score = firstPassExplanation.getValue();
-    fakeScorer.doc = docIDInSegment;
 
-    context.put("scorer", fakeScorer);
+    DoubleValues scores = scores(docIDInSegment, firstPassExplanation.getValue());
 
     List<Explanation> subs = new ArrayList<>(Arrays.asList(superExpl.getDetails()));
     for(String variable : expression.variables) {
-      subs.add(Explanation.match((float) bindings.getValueSource(variable).getValues(context, readerContext).doubleVal(docIDInSegment),
-                                       "variable \"" + variable + "\""));
+      DoubleValues dv = bindings.getDoubleValuesSource(variable).getValues(readerContext, scores);
+      if (dv.advanceExact(docIDInSegment))
+        subs.add(Explanation.match((float) dv.doubleValue(), "variable \"" + variable + "\""));
     }
 
     return Explanation.match(superExpl.getValue(), superExpl.getDescription(), subs);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/776087ee/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionSortField.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionSortField.java b/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionSortField.java
deleted file mode 100644
index 2b39834..0000000
--- a/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionSortField.java
+++ /dev/null
@@ -1,77 +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.lucene.expressions;
-
-import java.io.IOException;
-
-import org.apache.lucene.search.FieldComparator;
-import org.apache.lucene.search.SortField;
-
-/** A {@link SortField} which sorts documents by the evaluated value of an expression for each document */
-class ExpressionSortField extends SortField {
-  private final ExpressionValueSource source;
-
-  ExpressionSortField(String name, ExpressionValueSource source, boolean reverse) {
-    super(name, Type.CUSTOM, reverse);
-    this.source = source;
-  }
-  
-  @Override
-  public FieldComparator<?> getComparator(final int numHits, final int sortPos) throws IOException {
-    return new ExpressionComparator(source, numHits);
-  }
-
-  @Override
-  public int hashCode() {
-    final int prime = 31;
-    int result = super.hashCode();
-    result = prime * result + ((source == null) ? 0 : source.hashCode());
-    return result;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj) return true;
-    if (!super.equals(obj)) return false;
-    if (getClass() != obj.getClass()) return false;
-    ExpressionSortField other = (ExpressionSortField) obj;
-    if (source == null) {
-      if (other.source != null) return false;
-    } else if (!source.equals(other.source)) return false;
-    return true;
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder buffer = new StringBuilder();
-    
-    buffer.append("<expr \"");
-    buffer.append(getField());
-    buffer.append("\">");
-    
-    if (getReverse()) {
-      buffer.append('!');
-    }
-
-    return buffer.toString();
-  }
-
-  @Override
-  public boolean needsScores() {
-    return source.needsScores();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/776087ee/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionValueSource.java b/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionValueSource.java
index fcba455..7842de9 100644
--- a/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionValueSource.java
+++ b/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionValueSource.java
@@ -20,76 +20,77 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Objects;
 
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.search.SortField;
+import org.apache.lucene.search.DoubleValues;
+import org.apache.lucene.search.DoubleValuesSource;
 
 /**
- * A {@link ValueSource} which evaluates a {@link Expression} given the context of an {@link Bindings}.
+ * A {@link DoubleValuesSource} which evaluates a {@link Expression} given the context of an {@link Bindings}.
  */
 @SuppressWarnings({"rawtypes", "unchecked"})
-final class ExpressionValueSource extends ValueSource {
-  final ValueSource variables[];
+final class ExpressionValueSource extends DoubleValuesSource {
+  final DoubleValuesSource variables[];
   final Expression expression;
   final boolean needsScores;
 
   ExpressionValueSource(Bindings bindings, Expression expression) {
     if (bindings == null) throw new NullPointerException();
-    if (expression == null) throw new NullPointerException();
-    this.expression = expression;
-    variables = new ValueSource[expression.variables.length];
+    this.expression = Objects.requireNonNull(expression);
+    variables = new DoubleValuesSource[expression.variables.length];
     boolean needsScores = false;
     for (int i = 0; i < variables.length; i++) {
-      ValueSource source = bindings.getValueSource(expression.variables[i]);
-      if (source instanceof ScoreValueSource) {
-        needsScores = true;
-      } else if (source instanceof ExpressionValueSource) {
-        if (((ExpressionValueSource)source).needsScores()) {
-          needsScores = true;
-        }
-      } else if (source == null) {
+      DoubleValuesSource source = bindings.getDoubleValuesSource(expression.variables[i]);
+      if (source == null) {
         throw new RuntimeException("Internal error. Variable (" + expression.variables[i] + ") does not exist.");
       }
+      needsScores |= source.needsScores();
       variables[i] = source;
     }
     this.needsScores = needsScores;
   }
 
   @Override
-  public FunctionValues getValues(Map context, LeafReaderContext readerContext) throws IOException {
-    Map<String, FunctionValues> valuesCache = (Map<String, FunctionValues>)context.get("valuesCache");
-    if (valuesCache == null) {
-      valuesCache = new HashMap<>();
-      context = new HashMap(context);
-      context.put("valuesCache", valuesCache);
-    }
-    FunctionValues[] externalValues = new FunctionValues[expression.variables.length];
+  public DoubleValues getValues(LeafReaderContext readerContext, DoubleValues scores) throws IOException {
+    Map<String, DoubleValues> valuesCache = new HashMap<>();
+    DoubleValues[] externalValues = new DoubleValues[expression.variables.length];
 
     for (int i = 0; i < variables.length; ++i) {
       String externalName = expression.variables[i];
-      FunctionValues values = valuesCache.get(externalName);
+      DoubleValues values = valuesCache.get(externalName);
       if (values == null) {
-        values = variables[i].getValues(context, readerContext);
+        values = variables[i].getValues(readerContext, scores);
         if (values == null) {
           throw new RuntimeException("Internal error. External (" + externalName + ") does not exist.");
         }
         valuesCache.put(externalName, values);
       }
-      externalValues[i] = values;
+      externalValues[i] = zeroWhenUnpositioned(values);
     }
 
-    return new ExpressionFunctionValues(this, expression, externalValues);
+    return new ExpressionFunctionValues(expression, externalValues);
   }
 
-  @Override
-  public SortField getSortField(boolean reverse) {
-    return new ExpressionSortField(expression.sourceText, this, reverse);
+  private static DoubleValues zeroWhenUnpositioned(DoubleValues in) {
+    return new DoubleValues() {
+
+      boolean positioned = false;
+
+      @Override
+      public double doubleValue() throws IOException {
+        return positioned ? in.doubleValue() : 0;
+      }
+
+      @Override
+      public boolean advanceExact(int doc) throws IOException {
+        return positioned = in.advanceExact(doc);
+      }
+    };
   }
 
   @Override
-  public String description() {
+  public String toString() {
     return "expr(" + expression.sourceText + ")";
   }
   
@@ -132,7 +133,8 @@ final class ExpressionValueSource extends ValueSource {
     return true;
   }
 
-  boolean needsScores() {
+  @Override
+  public boolean needsScores() {
     return needsScores;
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/776087ee/lucene/expressions/src/java/org/apache/lucene/expressions/ScoreFunctionValues.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/java/org/apache/lucene/expressions/ScoreFunctionValues.java b/lucene/expressions/src/java/org/apache/lucene/expressions/ScoreFunctionValues.java
deleted file mode 100644
index e310c06..0000000
--- a/lucene/expressions/src/java/org/apache/lucene/expressions/ScoreFunctionValues.java
+++ /dev/null
@@ -1,46 +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.lucene.expressions;
-
-import java.io.IOException;
-
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
-import org.apache.lucene.search.Scorer;
-
-/**
- * A utility class to allow expressions to access the score as a {@link FunctionValues}.
- */
-class ScoreFunctionValues extends DoubleDocValues {
-  final Scorer scorer;
-
-  ScoreFunctionValues(ValueSource parent, Scorer scorer) {
-    super(parent);
-    this.scorer = scorer;
-  }
-  
-  @Override
-  public double doubleVal(int document) {
-    try {
-      assert document == scorer.docID();
-      return scorer.score();
-    } catch (IOException exception) {
-      throw new RuntimeException(exception);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/776087ee/lucene/expressions/src/java/org/apache/lucene/expressions/ScoreValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/java/org/apache/lucene/expressions/ScoreValueSource.java b/lucene/expressions/src/java/org/apache/lucene/expressions/ScoreValueSource.java
deleted file mode 100644
index ea1669c..0000000
--- a/lucene/expressions/src/java/org/apache/lucene/expressions/ScoreValueSource.java
+++ /dev/null
@@ -1,61 +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.lucene.expressions;
-
-
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.search.Scorer;
-
-import java.io.IOException;
-import java.util.Map;
-
-/**
- * A {@link ValueSource} which uses the {@link Scorer} passed through
- * the context map by {@link ExpressionComparator}.
- */
-@SuppressWarnings({"rawtypes"})
-class ScoreValueSource extends ValueSource {
-
-  /**
-   * <code>context</code> must contain a key "scorer" which is a {@link Scorer}.
-   */
-  @Override
-  public FunctionValues getValues(Map context, LeafReaderContext readerContext) throws IOException {
-    Scorer v = (Scorer) context.get("scorer");
-    if (v == null) {
-      throw new IllegalStateException("Expressions referencing the score can only be used for sorting");
-    }
-    return new ScoreFunctionValues(this, v);
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    return o == this;
-  }
-
-  @Override
-  public int hashCode() {
-    return System.identityHashCode(this);
-  }
-
-  @Override
-  public String description() {
-    return "score()";
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/776087ee/lucene/expressions/src/java/org/apache/lucene/expressions/SimpleBindings.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/java/org/apache/lucene/expressions/SimpleBindings.java b/lucene/expressions/src/java/org/apache/lucene/expressions/SimpleBindings.java
index e64249e..e0b4882 100644
--- a/lucene/expressions/src/java/org/apache/lucene/expressions/SimpleBindings.java
+++ b/lucene/expressions/src/java/org/apache/lucene/expressions/SimpleBindings.java
@@ -20,11 +20,7 @@ package org.apache.lucene.expressions;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.queries.function.valuesource.DoubleFieldSource;
-import org.apache.lucene.queries.function.valuesource.FloatFieldSource;
-import org.apache.lucene.queries.function.valuesource.IntFieldSource;
-import org.apache.lucene.queries.function.valuesource.LongFieldSource;
+import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.SortField;
 
 /**
@@ -64,9 +60,9 @@ public final class SimpleBindings extends Bindings {
   }
 
   /**
-   * Bind a {@link ValueSource} directly to the given name.
+   * Bind a {@link DoubleValuesSource} directly to the given name.
    */
-  public void add(String name, ValueSource source) { map.put(name, source); }
+  public void add(String name, DoubleValuesSource source) { map.put(name, source); }
   
   /** 
    * Adds an Expression to the bindings.
@@ -78,27 +74,27 @@ public final class SimpleBindings extends Bindings {
   }
   
   @Override
-  public ValueSource getValueSource(String name) {
+  public DoubleValuesSource getDoubleValuesSource(String name) {
     Object o = map.get(name);
     if (o == null) {
       throw new IllegalArgumentException("Invalid reference '" + name + "'");
     } else if (o instanceof Expression) {
-      return ((Expression)o).getValueSource(this);
-    } else if (o instanceof ValueSource) {
-      return ((ValueSource)o);
+      return ((Expression)o).getDoubleValuesSource(this);
+    } else if (o instanceof DoubleValuesSource) {
+      return ((DoubleValuesSource) o);
     }
     SortField field = (SortField) o;
     switch(field.getType()) {
       case INT:
-        return new IntFieldSource(field.getField());
+        return DoubleValuesSource.fromIntField(field.getField());
       case LONG:
-        return new LongFieldSource(field.getField());
+        return DoubleValuesSource.fromLongField(field.getField());
       case FLOAT:
-        return new FloatFieldSource(field.getField());
+        return DoubleValuesSource.fromFloatField(field.getField());
       case DOUBLE:
-        return new DoubleFieldSource(field.getField());
+        return DoubleValuesSource.fromDoubleField(field.getField());
       case SCORE:
-        return getScoreValueSource();
+        return DoubleValuesSource.SCORES;
       default:
         throw new UnsupportedOperationException(); 
     }
@@ -113,7 +109,7 @@ public final class SimpleBindings extends Bindings {
       if (o instanceof Expression) {
         Expression expr = (Expression) o;
         try {
-          expr.getValueSource(this);
+          expr.getDoubleValuesSource(this);
         } catch (StackOverflowError e) {
           throw new IllegalArgumentException("Recursion Error: Cycle detected originating in (" + expr.sourceText + ")");
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/776087ee/lucene/expressions/src/java/org/apache/lucene/expressions/js/JavascriptCompiler.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/java/org/apache/lucene/expressions/js/JavascriptCompiler.java b/lucene/expressions/src/java/org/apache/lucene/expressions/js/JavascriptCompiler.java
index c78f6a97..87e41c0 100644
--- a/lucene/expressions/src/java/org/apache/lucene/expressions/js/JavascriptCompiler.java
+++ b/lucene/expressions/src/java/org/apache/lucene/expressions/js/JavascriptCompiler.java
@@ -39,7 +39,7 @@ import org.antlr.v4.runtime.CommonTokenStream;
 import org.antlr.v4.runtime.tree.ParseTree;
 import org.apache.lucene.expressions.Expression;
 import org.apache.lucene.expressions.js.JavascriptParser.ExpressionContext;
-import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.search.DoubleValues;
 import org.apache.lucene.util.IOUtils;
 import org.objectweb.asm.ClassWriter;
 import org.objectweb.asm.Label;
@@ -93,13 +93,13 @@ public final class JavascriptCompiler {
   private static final String COMPILED_EXPRESSION_INTERNAL = COMPILED_EXPRESSION_CLASS.replace('.', '/');
   
   static final Type EXPRESSION_TYPE = Type.getType(Expression.class);
-  static final Type FUNCTION_VALUES_TYPE = Type.getType(FunctionValues.class);
+  static final Type FUNCTION_VALUES_TYPE = Type.getType(DoubleValues.class);
 
   private static final org.objectweb.asm.commons.Method
     EXPRESSION_CTOR = getAsmMethod(void.class, "<init>", String.class, String[].class),
-    EVALUATE_METHOD = getAsmMethod(double.class, "evaluate", int.class, FunctionValues[].class);
+    EVALUATE_METHOD = getAsmMethod(double.class, "evaluate", DoubleValues[].class);
 
-  static final org.objectweb.asm.commons.Method DOUBLE_VAL_METHOD = getAsmMethod(double.class, "doubleVal", int.class);
+  static final org.objectweb.asm.commons.Method DOUBLE_VAL_METHOD = getAsmMethod(double.class, "doubleValue");
   
   /** create an ASM Method object from return type, method name, and parameters. */
   private static org.objectweb.asm.commons.Method getAsmMethod(Class<?> rtype, String name, Class<?>... ptypes) {
@@ -154,9 +154,9 @@ public final class JavascriptCompiler {
    * use the FunctionValues class.
    */
   @SuppressWarnings({"unused", "null"})
-  private static void unusedTestCompile() {
-    FunctionValues f = null;
-    double ret = f.doubleVal(2);
+  private static void unusedTestCompile() throws IOException {
+    DoubleValues f = null;
+    double ret = f.doubleValue();
   }
   
   /**
@@ -325,10 +325,9 @@ public final class JavascriptCompiler {
             externalsMap.put(text, index);
           }
 
-          gen.loadArg(1);
+          gen.loadArg(0);
           gen.push(index);
           gen.arrayLoad(FUNCTION_VALUES_TYPE);
-          gen.loadArg(0);
           gen.invokeVirtual(FUNCTION_VALUES_TYPE, DOUBLE_VAL_METHOD);
           gen.cast(Type.DOUBLE_TYPE, typeStack.peek());
         } else {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/776087ee/lucene/expressions/src/java/org/apache/lucene/expressions/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/java/org/apache/lucene/expressions/package-info.java b/lucene/expressions/src/java/org/apache/lucene/expressions/package-info.java
index 62a519b..4a1eadf 100644
--- a/lucene/expressions/src/java/org/apache/lucene/expressions/package-info.java
+++ b/lucene/expressions/src/java/org/apache/lucene/expressions/package-info.java
@@ -25,11 +25,12 @@
  * 
  * <p>
  * {@link org.apache.lucene.expressions.Bindings} - abstraction for binding external variables
- * to a way to get a value for those variables for a particular document (ValueSource).
+ * to a way to get a value for those variables for a particular document (DoubleValuesSource).
  * </p>
  * 
  * <p>
- * {@link org.apache.lucene.expressions.SimpleBindings} - default implementation of bindings which provide easy ways to bind sort fields and other expressions to external variables
+ * {@link org.apache.lucene.expressions.SimpleBindings} - default implementation of bindings which
+ * provide easy ways to bind sort fields and other expressions to external variables
  * </p>
  */
 package org.apache.lucene.expressions;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/776087ee/lucene/expressions/src/test/org/apache/lucene/expressions/TestDemoExpressions.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/test/org/apache/lucene/expressions/TestDemoExpressions.java b/lucene/expressions/src/test/org/apache/lucene/expressions/TestDemoExpressions.java
index 01b3394..d76ef1c 100644
--- a/lucene/expressions/src/test/org/apache/lucene/expressions/TestDemoExpressions.java
+++ b/lucene/expressions/src/test/org/apache/lucene/expressions/TestDemoExpressions.java
@@ -16,18 +16,20 @@
  */
 package org.apache.lucene.expressions;
 
+import java.io.IOException;
+
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.expressions.js.JavascriptCompiler;
 import org.apache.lucene.expressions.js.VariableContext;
 import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.queries.function.valuesource.DoubleConstValueSource;
-import org.apache.lucene.queries.function.valuesource.IntFieldSource;
 import org.apache.lucene.search.CheckHits;
+import org.apache.lucene.search.DoubleValues;
+import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.FieldDoc;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
@@ -39,9 +41,9 @@ import org.apache.lucene.search.TopFieldDocs;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
 
+import static org.apache.lucene.expressions.js.VariableContext.Type.INT_INDEX;
 import static org.apache.lucene.expressions.js.VariableContext.Type.MEMBER;
 import static org.apache.lucene.expressions.js.VariableContext.Type.STR_INDEX;
-import static org.apache.lucene.expressions.js.VariableContext.Type.INT_INDEX;
 
 
 /** simple demo of using expressions */
@@ -236,7 +238,7 @@ public class  TestDemoExpressions extends LuceneTestCase {
   public void testStaticExtendedVariableExample() throws Exception {
     Expression popularity = JavascriptCompiler.compile("doc[\"popularity\"].value");
     SimpleBindings bindings = new SimpleBindings();
-    bindings.add("doc['popularity'].value", new IntFieldSource("popularity"));
+    bindings.add("doc['popularity'].value", DoubleValuesSource.fromIntField("popularity"));
     Sort sort = new Sort(popularity.getSortField(bindings, true));
     TopFieldDocs td = searcher.search(new MatchAllDocsQuery(), 3, sort);
 
@@ -250,6 +252,30 @@ public class  TestDemoExpressions extends LuceneTestCase {
     assertEquals(2D, (Double)d.fields[0], 1E-4);
   }
 
+  private static DoubleValuesSource constant(double value) {
+    return new DoubleValuesSource() {
+      @Override
+      public DoubleValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
+        return new DoubleValues() {
+          @Override
+          public double doubleValue() throws IOException {
+            return value;
+          }
+
+          @Override
+          public boolean advanceExact(int doc) throws IOException {
+            return true;
+          }
+        };
+      }
+
+      @Override
+      public boolean needsScores() {
+        return false;
+      }
+    };
+  }
+
   public void testDynamicExtendedVariableExample() throws Exception {
     Expression popularity = JavascriptCompiler.compile("doc['popularity'].value + magicarray[0] + fourtytwo");
 
@@ -258,7 +284,7 @@ public class  TestDemoExpressions extends LuceneTestCase {
     // filled in with proper error messages for a real use case.
     Bindings bindings = new Bindings() {
       @Override
-      public ValueSource getValueSource(String name) {
+      public DoubleValuesSource getDoubleValuesSource(String name) {
         VariableContext[] var = VariableContext.parse(name);
         assert var[0].type == MEMBER;
         String base = var[0].text;
@@ -266,7 +292,7 @@ public class  TestDemoExpressions extends LuceneTestCase {
           if (var.length > 1 && var[1].type == STR_INDEX) {
             String field = var[1].text;
             if (var.length > 2 && var[2].type == MEMBER && var[2].text.equals("value")) {
-              return new IntFieldSource(field);
+              return DoubleValuesSource.fromIntField(field);
             } else {
               fail("member: " + var[2].text);// error case, non/missing "value" member access
             }
@@ -275,12 +301,12 @@ public class  TestDemoExpressions extends LuceneTestCase {
           }
         } else if (base.equals("magicarray")) {
           if (var.length > 1 && var[1].type == INT_INDEX) {
-            return new DoubleConstValueSource(2048);
+            return constant(2048);
           } else {
             fail();// error case, magic array isn't an array
           }
         } else if (base.equals("fourtytwo")) {
-          return new DoubleConstValueSource(42);
+          return constant(42);
         } else {
           fail();// error case (variable doesn't exist)
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/776087ee/lucene/expressions/src/test/org/apache/lucene/expressions/TestExpressionSortField.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/test/org/apache/lucene/expressions/TestExpressionSortField.java b/lucene/expressions/src/test/org/apache/lucene/expressions/TestExpressionSortField.java
index ec6ea11..73e7b8b 100644
--- a/lucene/expressions/src/test/org/apache/lucene/expressions/TestExpressionSortField.java
+++ b/lucene/expressions/src/test/org/apache/lucene/expressions/TestExpressionSortField.java
@@ -31,7 +31,7 @@ public class TestExpressionSortField extends LuceneTestCase {
     bindings.add(new SortField("popularity", SortField.Type.INT));
     
     SortField sf = expr.getSortField(bindings, true);
-    assertEquals("<expr \"sqrt(_score) + ln(popularity)\">!", sf.toString());
+    assertEquals("<expr(sqrt(_score) + ln(popularity))>!", sf.toString());
   }
   
   public void testEquals() throws Exception {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/776087ee/lucene/expressions/src/test/org/apache/lucene/expressions/TestExpressionValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/test/org/apache/lucene/expressions/TestExpressionValueSource.java b/lucene/expressions/src/test/org/apache/lucene/expressions/TestExpressionValueSource.java
index 3129d8c..39217d8 100644
--- a/lucene/expressions/src/test/org/apache/lucene/expressions/TestExpressionValueSource.java
+++ b/lucene/expressions/src/test/org/apache/lucene/expressions/TestExpressionValueSource.java
@@ -17,21 +17,17 @@
 package org.apache.lucene.expressions;
 
 
-import java.util.HashMap;
-
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.expressions.js.JavascriptCompiler;
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.queries.function.ValueSourceScorer;
-import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.DoubleValues;
+import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
@@ -47,7 +43,7 @@ public class TestExpressionValueSource extends LuceneTestCase {
     IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()));
     iwc.setMergePolicy(newLogMergePolicy());
     RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
-    
+
     Document doc = new Document();
     doc.add(newStringField("id", "1", Field.Store.YES));
     doc.add(newTextField("body", "some contents and more contents", Field.Store.NO));
@@ -58,6 +54,7 @@ public class TestExpressionValueSource extends LuceneTestCase {
     doc.add(newStringField("id", "2", Field.Store.YES));
     doc.add(newTextField("body", "another document with different contents", Field.Store.NO));
     doc.add(new NumericDocValuesField("popularity", 20));
+    doc.add(new NumericDocValuesField("count", 1));
     iw.addDocument(doc);
     
     doc = new Document();
@@ -77,80 +74,34 @@ public class TestExpressionValueSource extends LuceneTestCase {
     dir.close();
     super.tearDown();
   }
-  
-  public void testTypes() throws Exception {
-    Expression expr = JavascriptCompiler.compile("2*popularity");
-    SimpleBindings bindings = new SimpleBindings();
-    bindings.add(new SortField("popularity", SortField.Type.LONG));
-    ValueSource vs = expr.getValueSource(bindings);
-    
-    assertEquals(1, reader.leaves().size());
-    LeafReaderContext leaf = reader.leaves().get(0);
-    FunctionValues values = vs.getValues(new HashMap<String,Object>(), leaf);
-    
-    assertEquals(10, values.doubleVal(0), 0);
-    assertEquals(10, values.floatVal(0), 0);
-    assertEquals(10, values.longVal(0));
-    assertEquals(10, values.intVal(0));
-    assertEquals(10, values.shortVal(0));
-    assertEquals(10, values.byteVal(0));
-    assertEquals("10.0", values.strVal(0));
-    assertEquals(new Double(10), values.objectVal(0));
-    
-    assertEquals(40, values.doubleVal(1), 0);
-    assertEquals(40, values.floatVal(1), 0);
-    assertEquals(40, values.longVal(1));
-    assertEquals(40, values.intVal(1));
-    assertEquals(40, values.shortVal(1));
-    assertEquals(40, values.byteVal(1));
-    assertEquals("40.0", values.strVal(1));
-    assertEquals(new Double(40), values.objectVal(1));
-    
-    assertEquals(4, values.doubleVal(2), 0);
-    assertEquals(4, values.floatVal(2), 0);
-    assertEquals(4, values.longVal(2));
-    assertEquals(4, values.intVal(2));
-    assertEquals(4, values.shortVal(2));
-    assertEquals(4, values.byteVal(2));
-    assertEquals("4.0", values.strVal(2));
-    assertEquals(new Double(4), values.objectVal(2));    
-  }
-  
-  public void testRangeScorer() throws Exception {
-    Expression expr = JavascriptCompiler.compile("2*popularity");
+
+  public void testDoubleValuesSourceTypes() throws Exception {
+    Expression expr = JavascriptCompiler.compile("2*popularity + count");
     SimpleBindings bindings = new SimpleBindings();
     bindings.add(new SortField("popularity", SortField.Type.LONG));
-    ValueSource vs = expr.getValueSource(bindings);
-    
+    bindings.add(new SortField("count", SortField.Type.LONG));
+    DoubleValuesSource vs = expr.getDoubleValuesSource(bindings);
+
     assertEquals(1, reader.leaves().size());
     LeafReaderContext leaf = reader.leaves().get(0);
-    FunctionValues values = vs.getValues(new HashMap<String,Object>(), leaf);
-    
-    // everything
-    ValueSourceScorer scorer = values.getRangeScorer(leaf, "4", "40", true, true);
-    DocIdSetIterator iter = scorer.iterator();
-    assertEquals(-1, iter.docID());
-    assertEquals(0, iter.nextDoc());
-    assertEquals(1, iter.nextDoc());
-    assertEquals(2, iter.nextDoc());
-    assertEquals(DocIdSetIterator.NO_MORE_DOCS, iter.nextDoc());
+    DoubleValues values = vs.getValues(leaf, null);
 
-    // just the first doc
-    scorer = values.getRangeScorer(leaf, "4", "40", false, false);
-    iter = scorer.iterator();
-    assertEquals(-1, scorer.docID());
-    assertEquals(0, iter.nextDoc());
-    assertEquals(DocIdSetIterator.NO_MORE_DOCS, iter.nextDoc());
+    assertTrue(values.advanceExact(0));
+    assertEquals(10, values.doubleValue(), 0);
+    assertTrue(values.advanceExact(1));
+    assertEquals(41, values.doubleValue(), 0);
+    assertTrue(values.advanceExact(2));
+    assertEquals(4, values.doubleValue(), 0);
   }
-  
-  public void testEquals() throws Exception {
+
+  public void testDoubleValuesSourceEquals() throws Exception {
     Expression expr = JavascriptCompiler.compile("sqrt(a) + ln(b)");
-    
-    SimpleBindings bindings = new SimpleBindings();    
+
+    SimpleBindings bindings = new SimpleBindings();
     bindings.add(new SortField("a", SortField.Type.INT));
     bindings.add(new SortField("b", SortField.Type.INT));
-    
-    ValueSource vs1 = expr.getValueSource(bindings);
+
+    DoubleValuesSource vs1 = expr.getDoubleValuesSource(bindings);
     // same instance
     assertEquals(vs1, vs1);
     // null
@@ -158,20 +109,21 @@ public class TestExpressionValueSource extends LuceneTestCase {
     // other object
     assertFalse(vs1.equals("foobar"));
     // same bindings and expression instances
-    ValueSource vs2 = expr.getValueSource(bindings);
+    DoubleValuesSource vs2 = expr.getDoubleValuesSource(bindings);
     assertEquals(vs1.hashCode(), vs2.hashCode());
     assertEquals(vs1, vs2);
     // equiv bindings (different instance)
-    SimpleBindings bindings2 = new SimpleBindings();    
+    SimpleBindings bindings2 = new SimpleBindings();
     bindings2.add(new SortField("a", SortField.Type.INT));
     bindings2.add(new SortField("b", SortField.Type.INT));
-    ValueSource vs3 = expr.getValueSource(bindings2);
+    DoubleValuesSource vs3 = expr.getDoubleValuesSource(bindings2);
     assertEquals(vs1, vs3);
     // different bindings (same names, different types)
-    SimpleBindings bindings3 = new SimpleBindings();    
+    SimpleBindings bindings3 = new SimpleBindings();
     bindings3.add(new SortField("a", SortField.Type.LONG));
-    bindings3.add(new SortField("b", SortField.Type.INT));
-    ValueSource vs4 = expr.getValueSource(bindings3);
+    bindings3.add(new SortField("b", SortField.Type.FLOAT));
+    DoubleValuesSource vs4 = expr.getDoubleValuesSource(bindings3);
     assertFalse(vs1.equals(vs4));
   }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/776087ee/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestCustomFunctions.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestCustomFunctions.java b/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestCustomFunctions.java
index 7c46b05..9df5d74 100644
--- a/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestCustomFunctions.java
+++ b/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestCustomFunctions.java
@@ -50,7 +50,7 @@ public class TestCustomFunctions extends LuceneTestCase {
   public void testDefaultList() throws Exception {
     Map<String,Method> functions = JavascriptCompiler.DEFAULT_FUNCTIONS;
     Expression expr = JavascriptCompiler.compile("sqrt(20)", functions, getClass().getClassLoader());
-    assertEquals(Math.sqrt(20), expr.evaluate(0, null), DELTA);
+    assertEquals(Math.sqrt(20), expr.evaluate(null), DELTA);
   }
   
   public static double zeroArgMethod() { return 5; }
@@ -60,7 +60,7 @@ public class TestCustomFunctions extends LuceneTestCase {
     Map<String,Method> functions = new HashMap<>();
     functions.put("foo", getClass().getMethod("zeroArgMethod"));
     Expression expr = JavascriptCompiler.compile("foo()", functions, getClass().getClassLoader());
-    assertEquals(5, expr.evaluate(0, null), DELTA);
+    assertEquals(5, expr.evaluate(null), DELTA);
   }
 
   public static double oneArgMethod(double arg1) { return 3 + arg1; }
@@ -70,7 +70,7 @@ public class TestCustomFunctions extends LuceneTestCase {
     Map<String,Method> functions = new HashMap<>();
     functions.put("foo", getClass().getMethod("oneArgMethod", double.class));
     Expression expr = JavascriptCompiler.compile("foo(3)", functions, getClass().getClassLoader());
-    assertEquals(6, expr.evaluate(0, null), DELTA);
+    assertEquals(6, expr.evaluate(null), DELTA);
   }
   
   public static double threeArgMethod(double arg1, double arg2, double arg3) { return arg1 + arg2 + arg3; }
@@ -80,7 +80,7 @@ public class TestCustomFunctions extends LuceneTestCase {
     Map<String,Method> functions = new HashMap<>();
     functions.put("foo", getClass().getMethod("threeArgMethod", double.class, double.class, double.class));
     Expression expr = JavascriptCompiler.compile("foo(3, 4, 5)", functions, getClass().getClassLoader());
-    assertEquals(12, expr.evaluate(0, null), DELTA);
+    assertEquals(12, expr.evaluate(null), DELTA);
   }
   
   /** tests a map with 2 functions */
@@ -89,7 +89,7 @@ public class TestCustomFunctions extends LuceneTestCase {
     functions.put("foo", getClass().getMethod("zeroArgMethod"));
     functions.put("bar", getClass().getMethod("oneArgMethod", double.class));
     Expression expr = JavascriptCompiler.compile("foo() + bar(3)", functions, getClass().getClassLoader());
-    assertEquals(11, expr.evaluate(0, null), DELTA);
+    assertEquals(11, expr.evaluate(null), DELTA);
   }
 
   /** tests invalid methods that are not allowed to become variables to be mapped */
@@ -220,7 +220,7 @@ public class TestCustomFunctions extends LuceneTestCase {
     
     // this should pass:
     Expression expr = JavascriptCompiler.compile("bar()", functions, childLoader);
-    assertEquals(2.0, expr.evaluate(0, null), DELTA);
+    assertEquals(2.0, expr.evaluate(null), DELTA);
     
     // use our classloader, not the foreign one, which should fail!
     IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
@@ -232,9 +232,9 @@ public class TestCustomFunctions extends LuceneTestCase {
     Map<String,Method> mixedFunctions = new HashMap<>(JavascriptCompiler.DEFAULT_FUNCTIONS);
     mixedFunctions.putAll(functions);
     expr = JavascriptCompiler.compile("bar()", mixedFunctions, childLoader);
-    assertEquals(2.0, expr.evaluate(0, null), DELTA);
+    assertEquals(2.0, expr.evaluate(null), DELTA);
     expr = JavascriptCompiler.compile("sqrt(20)", mixedFunctions, childLoader);
-    assertEquals(Math.sqrt(20), expr.evaluate(0, null), DELTA);
+    assertEquals(Math.sqrt(20), expr.evaluate(null), DELTA);
     
     // use our classloader, not the foreign one, which should fail!
     expected = expectThrows(IllegalArgumentException.class, () -> {
@@ -256,7 +256,7 @@ public class TestCustomFunctions extends LuceneTestCase {
     String source = "3 * foo() / 5";
     Expression expr = JavascriptCompiler.compile(source, functions, getClass().getClassLoader());
     ArithmeticException expected = expectThrows(ArithmeticException.class, () -> {
-      expr.evaluate(0, null);
+      expr.evaluate(null);
     });
     assertEquals(MESSAGE, expected.getMessage());
     StringWriter sw = new StringWriter();
@@ -272,6 +272,6 @@ public class TestCustomFunctions extends LuceneTestCase {
     functions.put("foo.bar", getClass().getMethod("zeroArgMethod"));
     String source = "foo.bar()";
     Expression expr = JavascriptCompiler.compile(source, functions, getClass().getClassLoader());
-    assertEquals(5, expr.evaluate(0, null), DELTA);
+    assertEquals(5, expr.evaluate(null), DELTA);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/776087ee/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestJavascriptFunction.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestJavascriptFunction.java b/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestJavascriptFunction.java
index 81362a6..ed68a5f 100644
--- a/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestJavascriptFunction.java
+++ b/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestJavascriptFunction.java
@@ -24,7 +24,7 @@ public class TestJavascriptFunction extends LuceneTestCase {
   
   private void assertEvaluatesTo(String expression, double expected) throws Exception {
     Expression evaluator = JavascriptCompiler.compile(expression);
-    double actual = evaluator.evaluate(0, null);
+    double actual = evaluator.evaluate(null);
     assertEquals(expected, actual, DELTA);
   }
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/776087ee/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestJavascriptOperations.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestJavascriptOperations.java b/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestJavascriptOperations.java
index 82d5056..fd098c5 100644
--- a/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestJavascriptOperations.java
+++ b/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestJavascriptOperations.java
@@ -22,7 +22,7 @@ import org.apache.lucene.util.LuceneTestCase;
 public class TestJavascriptOperations extends LuceneTestCase {
   private void assertEvaluatesTo(String expression, long expected) throws Exception {
     Expression evaluator = JavascriptCompiler.compile(expression);
-    long actual = (long)evaluator.evaluate(0, null);
+    long actual = (long)evaluator.evaluate(null);
     assertEquals(expected, actual);
   }
   


[3/6] lucene-solr:branch_6x: LUCENE-7611: Suggester uses LongValuesSource in place of ValueSource

Posted by ro...@apache.org.
LUCENE-7611: Suggester uses LongValuesSource in place of ValueSource


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/d268055c
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/d268055c
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/d268055c

Branch: refs/heads/branch_6x
Commit: d268055ca3f6fc6885940bdca39bed36b8f558fc
Parents: a238610
Author: Alan Woodward <ro...@apache.org>
Authored: Fri Jan 6 11:03:09 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Sat Jan 7 11:51:50 2017 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   3 +
 .../apache/lucene/search/LongValuesSource.java  |  27 ++
 .../suggest/DocumentValueSourceDictionary.java  |  76 +++++-
 .../DocumentValueSourceDictionaryTest.java      | 264 ++++++++++++++++++-
 .../DocumentExpressionDictionaryFactory.java    |   6 +-
 5 files changed, 356 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d268055c/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 826c86a..52462dc 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -27,6 +27,9 @@ API Changes
 * LUCENE-7610: The facets module now uses the DoubleValuesSource API, and
   methods that take ValueSource parameters are deprecated (Alan Woodward)
 
+* LUCENE-7611: DocumentValueSourceDictionary now takes a LongValuesSource
+  as a parameter, and the ValueSource equivalent is deprecated (Alan Woodward)
+
 New features
 
 * LUCENE-5867: Added BooleanSimilarity. (Robert Muir, Adrien Grand)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d268055c/lucene/core/src/java/org/apache/lucene/search/LongValuesSource.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/LongValuesSource.java b/lucene/core/src/java/org/apache/lucene/search/LongValuesSource.java
index 9599f8d..9becac8 100644
--- a/lucene/core/src/java/org/apache/lucene/search/LongValuesSource.java
+++ b/lucene/core/src/java/org/apache/lucene/search/LongValuesSource.java
@@ -75,6 +75,33 @@ public abstract class LongValuesSource {
     return fromLongField(field);
   }
 
+  /**
+   * Creates a LongValuesSource that always returns a constant value
+   */
+  public static LongValuesSource constant(long value) {
+    return new LongValuesSource() {
+      @Override
+      public LongValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
+        return new LongValues() {
+          @Override
+          public long longValue() throws IOException {
+            return value;
+          }
+
+          @Override
+          public boolean advanceExact(int doc) throws IOException {
+            return true;
+          }
+        };
+      }
+
+      @Override
+      public boolean needsScores() {
+        return false;
+      }
+    };
+  }
+
   private static class FieldValuesSource extends LongValuesSource {
 
     final String field;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d268055c/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java
index 8affcc5..3496aad 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java
@@ -17,15 +17,15 @@
 package org.apache.lucene.search.suggest;
 
 import java.io.IOException;
-import java.util.HashMap;
 import java.util.List;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.ReaderUtil;
-import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.search.LongValues;
+import org.apache.lucene.search.LongValuesSource;
 
 
 /**
@@ -34,7 +34,7 @@ import org.apache.lucene.queries.function.ValueSource;
  * optionally contexts information
  * taken from stored fields in a Lucene index. Similar to 
  * {@link DocumentDictionary}, except it obtains the weight
- * of the terms in a document based on a {@link ValueSource}.
+ * of the terms in a document based on a {@link LongValuesSource}.
  * </p>
  * <b>NOTE:</b> 
  *  <ul>
@@ -46,44 +46,75 @@ import org.apache.lucene.queries.function.ValueSource;
  *    </li>
  *  </ul>
  *  <p>
- *  In practice the {@link ValueSource} will likely be obtained
+ *  In practice the {@link LongValuesSource} will likely be obtained
  *  using the lucene expression module. The following example shows
- *  how to create a {@link ValueSource} from a simple addition of two
+ *  how to create a {@link LongValuesSource} from a simple addition of two
  *  fields:
  *  <code>
  *    Expression expression = JavascriptCompiler.compile("f1 + f2");
  *    SimpleBindings bindings = new SimpleBindings();
  *    bindings.add(new SortField("f1", SortField.Type.LONG));
  *    bindings.add(new SortField("f2", SortField.Type.LONG));
- *    ValueSource valueSource = expression.getValueSource(bindings);
+ *    LongValuesSource valueSource = expression.getDoubleValuesSource(bindings).toLongValuesSource();
  *  </code>
  *  </p>
  *
  */
 public class DocumentValueSourceDictionary extends DocumentDictionary {
   
-  private final ValueSource weightsValueSource;
+  private final LongValuesSource weightsValueSource;
   
   /**
    * Creates a new dictionary with the contents of the fields named <code>field</code>
    * for the terms, <code>payload</code> for the corresponding payloads, <code>contexts</code>
    * for the associated contexts and uses the <code>weightsValueSource</code> supplied 
    * to determine the score.
+   *
+   * @deprecated Use {@link #DocumentValueSourceDictionary(IndexReader, String, LongValuesSource, String, String)}
    */
+  @Deprecated
   public DocumentValueSourceDictionary(IndexReader reader, String field,
                                        ValueSource weightsValueSource, String payload, String contexts) {
     super(reader, field, null, payload, contexts);
+    this.weightsValueSource = weightsValueSource.asLongValuesSource();
+  }
+
+  /**
+   * Creates a new dictionary with the contents of the fields named <code>field</code>
+   * for the terms, <code>payload</code> for the corresponding payloads, <code>contexts</code>
+   * for the associated contexts and uses the <code>weightsValueSource</code> supplied
+   * to determine the score.
+   */
+  public DocumentValueSourceDictionary(IndexReader reader, String field,
+                                       LongValuesSource weightsValueSource, String payload, String contexts) {
+    super(reader, field, null, payload, contexts);
     this.weightsValueSource = weightsValueSource;
   }
+
   /**
    * Creates a new dictionary with the contents of the fields named <code>field</code>
    * for the terms, <code>payloadField</code> for the corresponding payloads
    * and uses the <code>weightsValueSource</code> supplied to determine the 
    * score.
+   *
+   * @deprecated Use {@link #DocumentValueSourceDictionary(IndexReader, String, LongValuesSource, String)}
    */
+  @Deprecated
   public DocumentValueSourceDictionary(IndexReader reader, String field,
                                        ValueSource weightsValueSource, String payload) {
     super(reader, field, null, payload);
+    this.weightsValueSource = weightsValueSource.asLongValuesSource();
+  }
+
+  /**
+   * Creates a new dictionary with the contents of the fields named <code>field</code>
+   * for the terms, <code>payloadField</code> for the corresponding payloads
+   * and uses the <code>weightsValueSource</code> supplied to determine the
+   * score.
+   */
+  public DocumentValueSourceDictionary(IndexReader reader, String field,
+                                       LongValuesSource weightsValueSource, String payload) {
+    super(reader, field, null, payload);
     this.weightsValueSource = weightsValueSource;
   }
   
@@ -91,11 +122,25 @@ public class DocumentValueSourceDictionary extends DocumentDictionary {
    * Creates a new dictionary with the contents of the fields named <code>field</code>
    * for the terms and uses the <code>weightsValueSource</code> supplied to determine the 
    * score.
+   *
+   * @deprecated Use {@link #DocumentValueSourceDictionary(IndexReader, String, LongValuesSource)}
    */
+  @Deprecated
   public DocumentValueSourceDictionary(IndexReader reader, String field,
                                        ValueSource weightsValueSource) {
     super(reader, field, null, null);
-    this.weightsValueSource = weightsValueSource;  
+    this.weightsValueSource = weightsValueSource.asLongValuesSource();
+  }
+
+  /**
+   * Creates a new dictionary with the contents of the fields named <code>field</code>
+   * for the terms and uses the <code>weightsValueSource</code> supplied to determine the
+   * score.
+   */
+  public DocumentValueSourceDictionary(IndexReader reader, String field,
+                                       LongValuesSource weightsValueSource) {
+    super(reader, field, null, null);
+    this.weightsValueSource = weightsValueSource;
   }
   
   @Override
@@ -105,7 +150,7 @@ public class DocumentValueSourceDictionary extends DocumentDictionary {
   
   final class DocumentValueSourceInputIterator extends DocumentDictionary.DocumentInputIterator {
     
-    private FunctionValues currentWeightValues;
+    private LongValues currentWeightValues;
     /** leaves of the reader */
     private final List<LeafReaderContext> leaves;
     /** starting docIds of all the leaves */
@@ -123,7 +168,7 @@ public class DocumentValueSourceDictionary extends DocumentDictionary {
       }
       starts[leaves.size()] = reader.maxDoc();
       currentWeightValues = (leaves.size() > 0) 
-          ? weightsValueSource.getValues(new HashMap<String, Object>(), leaves.get(currentLeafIndex))
+          ? weightsValueSource.getValues(leaves.get(currentLeafIndex), null)
           : null;
     }
     
@@ -140,12 +185,19 @@ public class DocumentValueSourceDictionary extends DocumentDictionary {
       if (subIndex != currentLeafIndex) {
         currentLeafIndex = subIndex;
         try {
-          currentWeightValues = weightsValueSource.getValues(new HashMap<String, Object>(), leaves.get(currentLeafIndex));
+          currentWeightValues = weightsValueSource.getValues(leaves.get(currentLeafIndex), null);
         } catch (IOException e) {
           throw new RuntimeException(e);
         }
       }
-      return currentWeightValues.longVal(docId - starts[subIndex]);
+      try {
+        if (currentWeightValues.advanceExact(docId - starts[subIndex]))
+          return currentWeightValues.longValue();
+        else
+          return 0;
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
     }
 
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d268055c/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentValueSourceDictionaryTest.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentValueSourceDictionaryTest.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentValueSourceDictionaryTest.java
index 9e58a4e..92799cd 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentValueSourceDictionaryTest.java
+++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentValueSourceDictionaryTest.java
@@ -36,12 +36,16 @@ import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.valuesource.DoubleConstValueSource;
 import org.apache.lucene.queries.function.valuesource.LongFieldSource;
 import org.apache.lucene.queries.function.valuesource.SumFloatFunction;
+import org.apache.lucene.search.DoubleValues;
+import org.apache.lucene.search.LongValues;
+import org.apache.lucene.search.LongValuesSource;
 import org.apache.lucene.search.spell.Dictionary;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
@@ -59,7 +63,7 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
   static final String CONTEXTS_FIELD_NAME = "c1";
   
   @Test
-  public void testEmptyReader() throws IOException {
+  public void testValueSourceEmptyReader() throws IOException {
     Directory dir = newDirectory();
     Analyzer analyzer = new MockAnalyzer(random());
     IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
@@ -78,9 +82,30 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
 
     IOUtils.close(ir, analyzer, dir);
   }
+
+  @Test
+  public void testLongValuesSourceEmptyReader() throws IOException {
+    Directory dir = newDirectory();
+    Analyzer analyzer = new MockAnalyzer(random());
+    IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
+    iwc.setMergePolicy(newLogMergePolicy());
+    // Make sure the index is created?
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
+    writer.commit();
+    writer.close();
+    IndexReader ir = DirectoryReader.open(dir);
+    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, LongValuesSource.constant(10), PAYLOAD_FIELD_NAME);
+    InputIterator inputIterator = dictionary.getEntryIterator();
+
+    assertNull(inputIterator.next());
+    assertEquals(inputIterator.weight(), 0);
+    assertNull(inputIterator.payload());
+
+    IOUtils.close(ir, analyzer, dir);
+  }
   
   @Test
-  public void testBasic() throws IOException {
+  public void testValueSourceBasic() throws IOException {
     Directory dir = newDirectory();
     Analyzer analyzer = new MockAnalyzer(random());
     IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
@@ -112,9 +137,83 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
     assertTrue(docs.isEmpty());
     IOUtils.close(ir, analyzer, dir);
   }
+
+  private static LongValuesSource sum(String... fields) {
+    LongValuesSource[] sources = new LongValuesSource[fields.length];
+    for (int i = 0; i < fields.length; i++) {
+      sources[i] = LongValuesSource.fromLongField(fields[i]);
+    }
+    return new LongValuesSource() {
+      @Override
+      public LongValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
+        LongValues[] values = new LongValues[fields.length];
+        for (int i = 0; i < sources.length; i++) {
+          values[i] = sources[i].getValues(ctx, scores);
+        }
+        return new LongValues() {
+          @Override
+          public long longValue() throws IOException {
+            long v = 0;
+            for (LongValues value : values) {
+              v += value.longValue();
+            }
+            return v;
+          }
+
+          @Override
+          public boolean advanceExact(int doc) throws IOException {
+            boolean v = true;
+            for (LongValues value : values) {
+              v &= value.advanceExact(doc);
+            }
+            return v;
+          }
+        };
+      }
+
+      @Override
+      public boolean needsScores() {
+        return false;
+      }
+    };
+  }
+
+  @Test
+  public void testLongValuesSourceBasic() throws IOException {
+    Directory dir = newDirectory();
+    Analyzer analyzer = new MockAnalyzer(random());
+    IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
+    Map<String, Document> docs = generateIndexDocuments(atLeast(100));
+    for(Document doc: docs.values()) {
+      writer.addDocument(doc);
+    }
+    writer.commit();
+    writer.close();
+
+    IndexReader ir = DirectoryReader.open(dir);
+    LongValuesSource sumValueSource = sum(WEIGHT_FIELD_NAME_1, WEIGHT_FIELD_NAME_2, WEIGHT_FIELD_NAME_3);
+    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, sumValueSource, PAYLOAD_FIELD_NAME);
+    InputIterator inputIterator = dictionary.getEntryIterator();
+    BytesRef f;
+    while((f = inputIterator.next())!=null) {
+      Document doc = docs.remove(f.utf8ToString());
+      long w1 = doc.getField(WEIGHT_FIELD_NAME_1).numericValue().longValue();
+      long w2 = doc.getField(WEIGHT_FIELD_NAME_2).numericValue().longValue();
+      long w3 = doc.getField(WEIGHT_FIELD_NAME_3).numericValue().longValue();
+      assertTrue(f.equals(new BytesRef(doc.get(FIELD_NAME))));
+      assertEquals(inputIterator.weight(), (w1 + w2 + w3));
+      IndexableField payloadField = doc.getField(PAYLOAD_FIELD_NAME);
+      if (payloadField == null) assertTrue(inputIterator.payload().length == 0);
+      else assertEquals(inputIterator.payload(), payloadField.binaryValue());
+    }
+    assertTrue(docs.isEmpty());
+    IOUtils.close(ir, analyzer, dir);
+  }
   
   @Test
-  public void testWithContext() throws IOException {
+  public void testValueSourceWithContext() throws IOException {
     Directory dir = newDirectory();
     Analyzer analyzer = new MockAnalyzer(random());
     IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
@@ -153,7 +252,46 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
   }
 
   @Test
-  public void testWithoutPayload() throws IOException {
+  public void testLongValuesSourceWithContext() throws IOException {
+    Directory dir = newDirectory();
+    Analyzer analyzer = new MockAnalyzer(random());
+    IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
+    Map<String, Document> docs = generateIndexDocuments(atLeast(100));
+    for(Document doc: docs.values()) {
+      writer.addDocument(doc);
+    }
+    writer.commit();
+    writer.close();
+
+    IndexReader ir = DirectoryReader.open(dir);
+    LongValuesSource sumValues = sum(WEIGHT_FIELD_NAME_1, WEIGHT_FIELD_NAME_2, WEIGHT_FIELD_NAME_3);
+    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, sumValues, PAYLOAD_FIELD_NAME, CONTEXTS_FIELD_NAME);
+    InputIterator inputIterator = dictionary.getEntryIterator();
+    BytesRef f;
+    while((f = inputIterator.next())!=null) {
+      Document doc = docs.remove(f.utf8ToString());
+      long w1 = doc.getField(WEIGHT_FIELD_NAME_1).numericValue().longValue();
+      long w2 = doc.getField(WEIGHT_FIELD_NAME_2).numericValue().longValue();
+      long w3 = doc.getField(WEIGHT_FIELD_NAME_3).numericValue().longValue();
+      assertTrue(f.equals(new BytesRef(doc.get(FIELD_NAME))));
+      assertEquals(inputIterator.weight(), (w1 + w2 + w3));
+      IndexableField payloadField = doc.getField(PAYLOAD_FIELD_NAME);
+      if (payloadField == null) assertTrue(inputIterator.payload().length == 0);
+      else assertEquals(inputIterator.payload(), payloadField.binaryValue());
+      Set<BytesRef> originalCtxs = new HashSet<>();
+      for (IndexableField ctxf: doc.getFields(CONTEXTS_FIELD_NAME)) {
+        originalCtxs.add(ctxf.binaryValue());
+      }
+      assertEquals(originalCtxs, inputIterator.contexts());
+    }
+    assertTrue(docs.isEmpty());
+    IOUtils.close(ir, analyzer, dir);
+  }
+
+  @Test
+  public void testValueSourceWithoutPayload() throws IOException {
     Directory dir = newDirectory();
     Analyzer analyzer = new MockAnalyzer(random());
     IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
@@ -183,9 +321,41 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
     assertTrue(docs.isEmpty());
     IOUtils.close(ir, analyzer, dir);
   }
+
+  @Test
+  public void testLongValuesSourceWithoutPayload() throws IOException {
+    Directory dir = newDirectory();
+    Analyzer analyzer = new MockAnalyzer(random());
+    IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
+    Map<String, Document> docs = generateIndexDocuments(atLeast(100));
+    for(Document doc: docs.values()) {
+      writer.addDocument(doc);
+    }
+    writer.commit();
+    writer.close();
+
+    IndexReader ir = DirectoryReader.open(dir);
+    LongValuesSource sumValues = sum(WEIGHT_FIELD_NAME_1, WEIGHT_FIELD_NAME_2, WEIGHT_FIELD_NAME_3);
+    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, sumValues);
+    InputIterator inputIterator = dictionary.getEntryIterator();
+    BytesRef f;
+    while((f = inputIterator.next())!=null) {
+      Document doc = docs.remove(f.utf8ToString());
+      long w1 = doc.getField(WEIGHT_FIELD_NAME_1).numericValue().longValue();
+      long w2 = doc.getField(WEIGHT_FIELD_NAME_2).numericValue().longValue();
+      long w3 = doc.getField(WEIGHT_FIELD_NAME_3).numericValue().longValue();
+      assertTrue(f.equals(new BytesRef(doc.get(FIELD_NAME))));
+      assertEquals(inputIterator.weight(), (w1 + w2 + w3));
+      assertNull(inputIterator.payload());
+    }
+    assertTrue(docs.isEmpty());
+    IOUtils.close(ir, analyzer, dir);
+  }
   
   @Test
-  public void testWithDeletions() throws IOException {
+  public void testValueSourceWithDeletions() throws IOException {
     Directory dir = newDirectory();
     Analyzer analyzer = new MockAnalyzer(random());
     IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
@@ -238,6 +408,60 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
     assertTrue(docs.isEmpty());
     IOUtils.close(ir, analyzer, dir);
   }
+
+  @Test
+  public void testLongValuesSourceWithDeletions() throws IOException {
+    Directory dir = newDirectory();
+    Analyzer analyzer = new MockAnalyzer(random());
+    IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
+    Map<String, Document> docs = generateIndexDocuments(atLeast(100));
+    Random rand = random();
+    List<String> termsToDel = new ArrayList<>();
+    for(Document doc : docs.values()) {
+      if(rand.nextBoolean() && termsToDel.size() < docs.size()-1) {
+        termsToDel.add(doc.get(FIELD_NAME));
+      }
+      writer.addDocument(doc);
+    }
+    writer.commit();
+
+    Term[] delTerms = new Term[termsToDel.size()];
+    for(int i=0; i < termsToDel.size() ; i++) {
+      delTerms[i] = new Term(FIELD_NAME, termsToDel.get(i));
+    }
+
+    for(Term delTerm: delTerms) {
+      writer.deleteDocuments(delTerm);
+    }
+    writer.commit();
+    writer.close();
+
+    for(String termToDel: termsToDel) {
+      assertTrue(null!=docs.remove(termToDel));
+    }
+
+    IndexReader ir = DirectoryReader.open(dir);
+    assertTrue("NumDocs should be > 0 but was " + ir.numDocs(), ir.numDocs() > 0);
+    assertEquals(ir.numDocs(), docs.size());
+    LongValuesSource sumValues = sum(WEIGHT_FIELD_NAME_1, WEIGHT_FIELD_NAME_2);
+    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, sumValues, PAYLOAD_FIELD_NAME);
+    InputIterator inputIterator = dictionary.getEntryIterator();
+    BytesRef f;
+    while((f = inputIterator.next())!=null) {
+      Document doc = docs.remove(f.utf8ToString());
+      long w1 = doc.getField(WEIGHT_FIELD_NAME_1).numericValue().longValue();
+      long w2 = doc.getField(WEIGHT_FIELD_NAME_2).numericValue().longValue();
+      assertTrue(f.equals(new BytesRef(doc.get(FIELD_NAME))));
+      assertEquals(inputIterator.weight(), w2+w1);
+      IndexableField payloadField = doc.getField(PAYLOAD_FIELD_NAME);
+      if (payloadField == null) assertTrue(inputIterator.payload().length == 0);
+      else assertEquals(inputIterator.payload(), payloadField.binaryValue());
+    }
+    assertTrue(docs.isEmpty());
+    IOUtils.close(ir, analyzer, dir);
+  }
   
   @Test
   public void testWithValueSource() throws IOException {
@@ -269,6 +493,36 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
     IOUtils.close(ir, analyzer, dir);
   }
 
+  @Test
+  public void testWithLongValuesSource() throws IOException {
+    Directory dir = newDirectory();
+    Analyzer analyzer = new MockAnalyzer(random());
+    IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
+    Map<String, Document> docs = generateIndexDocuments(atLeast(100));
+    for(Document doc: docs.values()) {
+      writer.addDocument(doc);
+    }
+    writer.commit();
+    writer.close();
+
+    IndexReader ir = DirectoryReader.open(dir);
+    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, LongValuesSource.constant(10), PAYLOAD_FIELD_NAME);
+    InputIterator inputIterator = dictionary.getEntryIterator();
+    BytesRef f;
+    while((f = inputIterator.next())!=null) {
+      Document doc = docs.remove(f.utf8ToString());
+      assertTrue(f.equals(new BytesRef(doc.get(FIELD_NAME))));
+      assertEquals(inputIterator.weight(), 10);
+      IndexableField payloadField = doc.getField(PAYLOAD_FIELD_NAME);
+      if (payloadField == null) assertTrue(inputIterator.payload().length == 0);
+      else assertEquals(inputIterator.payload(), payloadField.binaryValue());
+    }
+    assertTrue(docs.isEmpty());
+    IOUtils.close(ir, analyzer, dir);
+  }
+
   private Map<String, Document> generateIndexDocuments(int ndocs) {
     Map<String, Document> docs = new HashMap<>();
     for(int i = 0; i < ndocs ; i++) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d268055c/solr/core/src/java/org/apache/solr/spelling/suggest/DocumentExpressionDictionaryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/spelling/suggest/DocumentExpressionDictionaryFactory.java b/solr/core/src/java/org/apache/solr/spelling/suggest/DocumentExpressionDictionaryFactory.java
index 24f1553..b0d7007 100644
--- a/solr/core/src/java/org/apache/solr/spelling/suggest/DocumentExpressionDictionaryFactory.java
+++ b/solr/core/src/java/org/apache/solr/spelling/suggest/DocumentExpressionDictionaryFactory.java
@@ -23,7 +23,7 @@ import java.util.Set;
 import org.apache.lucene.expressions.Expression;
 import org.apache.lucene.expressions.SimpleBindings;
 import org.apache.lucene.expressions.js.JavascriptCompiler;
-import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.search.LongValuesSource;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.spell.Dictionary;
 import org.apache.lucene.search.suggest.DocumentValueSourceDictionary;
@@ -93,7 +93,7 @@ public class DocumentExpressionDictionaryFactory extends DictionaryFactory {
         sortFields), payloadField);
   }
 
-  public ValueSource fromExpression(String weightExpression, Set<SortField> sortFields) {
+  public LongValuesSource fromExpression(String weightExpression, Set<SortField> sortFields) {
     Expression expression = null;
     try {
       expression = JavascriptCompiler.compile(weightExpression);
@@ -104,7 +104,7 @@ public class DocumentExpressionDictionaryFactory extends DictionaryFactory {
     for (SortField sortField : sortFields) {
       bindings.add(sortField);
     }
-    return expression.getValueSource(bindings);
+    return expression.getDoubleValuesSource(bindings).toLongValuesSource();
   }
   
   private SortField.Type getSortFieldType(SolrCore core, String sortFieldName) {


[5/6] lucene-solr:master: LUCENE-7610: Deprecate ValueSource methods in facets module

Posted by ro...@apache.org.
LUCENE-7610: Deprecate ValueSource methods in facets module


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/713b65d1
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/713b65d1
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/713b65d1

Branch: refs/heads/master
Commit: 713b65d1dcc80c1fe147a5bf999e1a88b63b9dce
Parents: 8b05538
Author: Alan Woodward <ro...@apache.org>
Authored: Fri Jan 6 10:41:12 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Sat Jan 7 11:52:24 2017 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   3 +
 .../demo/facet/DistanceFacetsExample.java       |  18 +--
 .../org/apache/lucene/facet/package-info.java   |   2 +-
 .../apache/lucene/facet/range/DoubleRange.java  |  33 ++++--
 .../facet/range/DoubleRangeFacetCounts.java     |  80 ++++++++-----
 .../apache/lucene/facet/range/LongRange.java    |  34 ++++--
 .../facet/range/LongRangeFacetCounts.java       |  47 +++++---
 .../org/apache/lucene/facet/range/Range.java    |  31 ------
 .../lucene/facet/taxonomy/FakeScorer.java       |  53 ---------
 .../taxonomy/TaxonomyFacetSumValueSource.java   |  99 ++++++++++++-----
 .../facet/range/TestRangeFacetCounts.java       |  53 +++++----
 .../TestTaxonomyFacetSumValueSource.java        |   5 +-
 .../lucene/queries/function/ValueSource.java    | 111 +++++++++++++++++++
 13 files changed, 361 insertions(+), 208 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/713b65d1/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index c667040..d0bedb7 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -79,6 +79,9 @@ API Changes
   replaced with Expression#getDoubleValuesSource(). (Alan Woodward, Adrien
   Grand)
 
+* LUCENE-7610: The facets module now uses the DoubleValuesSource API, and
+  methods that take ValueSource parameters are deprecated (Alan Woodward)
+
 New features
 
 * LUCENE-5867: Added BooleanSimilarity. (Robert Muir, Adrien Grand)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/713b65d1/lucene/demo/src/java/org/apache/lucene/demo/facet/DistanceFacetsExample.java
----------------------------------------------------------------------
diff --git a/lucene/demo/src/java/org/apache/lucene/demo/facet/DistanceFacetsExample.java b/lucene/demo/src/java/org/apache/lucene/demo/facet/DistanceFacetsExample.java
index 7d029ee..fa7ce83 100644
--- a/lucene/demo/src/java/org/apache/lucene/demo/facet/DistanceFacetsExample.java
+++ b/lucene/demo/src/java/org/apache/lucene/demo/facet/DistanceFacetsExample.java
@@ -16,9 +16,13 @@
  */
 package org.apache.lucene.demo.facet;
 
+import java.io.Closeable;
+import java.io.IOException;
+import java.text.ParseException;
+
 import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
-import org.apache.lucene.document.DoublePoint;
 import org.apache.lucene.document.Document;
+import org.apache.lucene.document.DoublePoint;
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.expressions.Expression;
 import org.apache.lucene.expressions.SimpleBindings;
@@ -36,9 +40,9 @@ import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
-import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.Query;
@@ -48,10 +52,6 @@ import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.RAMDirectory;
 import org.apache.lucene.util.SloppyMath;
 
-import java.io.Closeable;
-import java.io.IOException;
-import java.text.ParseException;
-
 /** Shows simple usage of dynamic range faceting, using the
  *  expressions module to calculate distance. */
 public class DistanceFacetsExample implements Closeable {
@@ -117,7 +117,7 @@ public class DistanceFacetsExample implements Closeable {
     writer.close();
   }
 
-  private ValueSource getDistanceValueSource() {
+  private DoubleValuesSource getDistanceValueSource() {
     Expression distance;
     try {
       distance = JavascriptCompiler.compile(
@@ -130,7 +130,7 @@ public class DistanceFacetsExample implements Closeable {
     bindings.add(new SortField("latitude", SortField.Type.DOUBLE));
     bindings.add(new SortField("longitude", SortField.Type.DOUBLE));
 
-    return distance.getValueSource(bindings);
+    return distance.getDoubleValuesSource(bindings);
   }
 
   /** Given a latitude and longitude (in degrees) and the
@@ -224,7 +224,7 @@ public class DistanceFacetsExample implements Closeable {
     // Passing no baseQuery means we drill down on all
     // documents ("browse only"):
     DrillDownQuery q = new DrillDownQuery(null);
-    final ValueSource vs = getDistanceValueSource();
+    final DoubleValuesSource vs = getDistanceValueSource();
     q.add("field", range.getQuery(getBoundingBoxQuery(ORIGIN_LATITUDE, ORIGIN_LONGITUDE, range.max), vs));
     DrillSideways ds = new DrillSideways(searcher, config, (TaxonomyReader) null) {
         @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/713b65d1/lucene/facet/src/java/org/apache/lucene/facet/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/package-info.java b/lucene/facet/src/java/org/apache/lucene/facet/package-info.java
index 0501d6a..acbdd44 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/package-info.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/package-info.java
@@ -40,7 +40,7 @@
  * 
  *  <li> Range faceting {@link org.apache.lucene.facet.range.LongRangeFacetCounts}, {@link
  *       org.apache.lucene.facet.range.DoubleRangeFacetCounts} compute counts for a dynamic numeric
- *       range from a provided {@link org.apache.lucene.queries.function.ValueSource} (previously indexed
+ *       range from a provided {@link org.apache.lucene.search.LongValuesSource} (previously indexed
  *       numeric field, or a dynamic expression such as distance).
  * </ul>
  * <p>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/713b65d1/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java
index 2203be3..ce377f5 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java
@@ -17,16 +17,16 @@
 package org.apache.lucene.facet.range;
 
 import java.io.IOException;
-import java.util.Collections;
 import java.util.Objects;
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.ConstantScoreScorer;
 import org.apache.lucene.search.ConstantScoreWeight;
 import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.DoubleValues;
+import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Scorer;
@@ -95,9 +95,9 @@ public final class DoubleRange extends Range {
   private static class ValueSourceQuery extends Query {
     private final DoubleRange range;
     private final Query fastMatchQuery;
-    private final ValueSource valueSource;
+    private final DoubleValuesSource valueSource;
 
-    ValueSourceQuery(DoubleRange range, Query fastMatchQuery, ValueSource valueSource) {
+    ValueSourceQuery(DoubleRange range, Query fastMatchQuery, DoubleValuesSource valueSource) {
       this.range = range;
       this.fastMatchQuery = fastMatchQuery;
       this.valueSource = valueSource;
@@ -158,11 +158,11 @@ public final class DoubleRange extends Range {
             approximation = s.iterator();
           }
 
-          final FunctionValues values = valueSource.getValues(Collections.emptyMap(), context);
+          final DoubleValues values = valueSource.getValues(context, null);
           final TwoPhaseIterator twoPhase = new TwoPhaseIterator(approximation) {
             @Override
             public boolean matches() throws IOException {
-              return range.accept(values.doubleVal(approximation.docID()));
+              return values.advanceExact(approximation.docID()) && range.accept(values.doubleValue());
             }
 
             @Override
@@ -177,8 +177,27 @@ public final class DoubleRange extends Range {
 
   }
 
-  @Override
+  /**
+   * @deprecated Use {@link #getQuery(Query, DoubleValuesSource)}
+   */
+  @Deprecated
   public Query getQuery(final Query fastMatchQuery, final ValueSource valueSource) {
+    return new ValueSourceQuery(this, fastMatchQuery, valueSource.asDoubleValuesSource());
+  }
+
+  /**
+   * Create a Query that matches documents in this range
+   *
+   * The query will check all documents that match the provided match query,
+   * or every document in the index if the match query is null.
+   *
+   * If the value source is static, eg an indexed numeric field, it may be
+   * faster to use {@link org.apache.lucene.search.PointRangeQuery}
+   *
+   * @param fastMatchQuery a query to use as a filter
+   * @param valueSource    the source of values for the range check
+   */
+  public Query getQuery(Query fastMatchQuery, DoubleValuesSource valueSource) {
     return new ValueSourceQuery(this, fastMatchQuery, valueSource);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/713b65d1/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java
index 2d0ba5c..63fc935 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java
@@ -17,22 +17,18 @@
 package org.apache.lucene.facet.range;
 
 import java.io.IOException;
-import java.util.Collections;
 import java.util.List;
 
-import org.apache.lucene.document.DoubleDocValuesField; // javadocs
-import org.apache.lucene.document.FloatDocValuesField; // javadocs
+import org.apache.lucene.document.FloatDocValuesField;
 import org.apache.lucene.facet.Facets;
-import org.apache.lucene.facet.FacetsCollector.MatchingDocs;
 import org.apache.lucene.facet.FacetsCollector;
+import org.apache.lucene.facet.FacetsCollector.MatchingDocs;
 import org.apache.lucene.index.IndexReaderContext;
 import org.apache.lucene.index.ReaderUtil;
-import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.queries.function.valuesource.DoubleFieldSource;
-import org.apache.lucene.queries.function.valuesource.FloatFieldSource; // javadocs
-import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.DoubleValues;
+import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Scorer;
@@ -41,46 +37,70 @@ import org.apache.lucene.util.NumericUtils;
 
 /** {@link Facets} implementation that computes counts for
  *  dynamic double ranges from a provided {@link
- *  ValueSource}, using {@link FunctionValues#doubleVal}.  Use
- *  this for dimensions that change in real-time (e.g. a
+ *  DoubleValuesSource}.  Use this for dimensions that change in real-time (e.g. a
  *  relative time based dimension like "Past day", "Past 2
  *  days", etc.) or that change for each request (e.g.
  *  distance from the user's location, "&lt; 1 km", "&lt; 2 km",
  *  etc.).
  *
- *  <p> If you had indexed your field using {@link
- *  FloatDocValuesField} then pass {@link FloatFieldSource}
- *  as the {@link ValueSource}; if you used {@link
- *  DoubleDocValuesField} then pass {@link
- *  DoubleFieldSource} (this is the default used when you
- *  pass just a the field name).
+ *  If you have indexed your field using {@link
+ *  FloatDocValuesField}, then you should use a DoubleValuesSource
+ *  generated from {@link DoubleValuesSource#fromFloatField(String)}.
  *
  *  @lucene.experimental */
 public class DoubleRangeFacetCounts extends RangeFacetCounts {
 
-  /** Create {@code RangeFacetCounts}, using {@link
-   *  DoubleFieldSource} from the specified field. */
+  /**
+   * Create {@code RangeFacetCounts}, using {@link DoubleValues} from the specified field.
+   *
+   * N.B This assumes that the field was indexed with {@link org.apache.lucene.document.DoubleDocValuesField}.
+   * For float-valued fields, use {@link #DoubleRangeFacetCounts(String, DoubleValuesSource, FacetsCollector, DoubleRange...)}
+   */
   public DoubleRangeFacetCounts(String field, FacetsCollector hits, DoubleRange... ranges) throws IOException {
-    this(field, new DoubleFieldSource(field), hits, ranges);
+    this(field, DoubleValuesSource.fromDoubleField(field), hits, ranges);
   }
 
-  /** Create {@code RangeFacetCounts}, using the provided
-   *  {@link ValueSource}. */
+  /**
+   * Create {@code RangeFacetCounts}, using the provided {@link ValueSource}.
+   *
+   * @deprecated Use {@link #DoubleRangeFacetCounts(String, DoubleValuesSource, FacetsCollector, DoubleRange...)}
+   * */
   public DoubleRangeFacetCounts(String field, ValueSource valueSource, FacetsCollector hits, DoubleRange... ranges) throws IOException {
     this(field, valueSource, hits, null, ranges);
   }
 
-  /** Create {@code RangeFacetCounts}, using the provided
-   *  {@link ValueSource}, and using the provided Query as
-   *  a fastmatch: only documents passing the filter are
-   *  checked for the matching ranges.  The filter must be
-   *  random access (implement {@link DocIdSet#bits}). */
+  /**
+   * Create {@code RangeFacetCounts} using the provided {@link DoubleValuesSource}
+   */
+  public DoubleRangeFacetCounts(String field, DoubleValuesSource valueSource, FacetsCollector hits, DoubleRange... ranges) throws IOException {
+    this(field, valueSource, hits, null, ranges);
+  }
+
+  /**
+   * Create {@code RangeFacetCounts}, using the provided
+   * {@link ValueSource}, and using the provided Query as
+   * a fastmatch: only documents matching the query are
+   * checked for the matching ranges.
+   *
+   * @deprecated Use ({@link #DoubleRangeFacetCounts(String, DoubleValuesSource, FacetsCollector, Query, DoubleRange...)}
+   */
+  @Deprecated
   public DoubleRangeFacetCounts(String field, ValueSource valueSource, FacetsCollector hits, Query fastMatchQuery, DoubleRange... ranges) throws IOException {
+    this(field, valueSource.asDoubleValuesSource(), hits, fastMatchQuery, ranges);
+  }
+
+  /**
+   * Create {@code RangeFacetCounts}, using the provided
+   * {@link DoubleValuesSource}, and using the provided Query as
+   * a fastmatch: only documents matching the query are
+   * checked for the matching ranges.
+   */
+ public DoubleRangeFacetCounts(String field, DoubleValuesSource valueSource, FacetsCollector hits, Query fastMatchQuery, DoubleRange... ranges) throws IOException {
     super(field, ranges, fastMatchQuery);
     count(valueSource, hits.getMatchingDocs());
   }
 
-  private void count(ValueSource valueSource, List<MatchingDocs> matchingDocs) throws IOException {
+  private void count(DoubleValuesSource valueSource, List<MatchingDocs> matchingDocs) throws IOException {
 
     DoubleRange[] ranges = (DoubleRange[]) this.ranges;
 
@@ -96,7 +116,7 @@ public class DoubleRangeFacetCounts extends RangeFacetCounts {
 
     int missingCount = 0;
     for (MatchingDocs hits : matchingDocs) {
-      FunctionValues fv = valueSource.getValues(Collections.emptyMap(), hits.context);
+      DoubleValues fv = valueSource.getValues(hits.context, null);
       
       totCount += hits.totalHits;
       final DocIdSetIterator fastMatchDocs;
@@ -129,8 +149,8 @@ public class DoubleRangeFacetCounts extends RangeFacetCounts {
           }
         }
         // Skip missing docs:
-        if (fv.exists(doc)) {
-          counter.add(NumericUtils.doubleToSortableLong(fv.doubleVal(doc)));
+        if (fv.advanceExact(doc)) {
+          counter.add(NumericUtils.doubleToSortableLong(fv.doubleValue()));
         } else {
           missingCount++;
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/713b65d1/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java
index 9c12ecd..20c408d 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java
@@ -17,17 +17,17 @@
 package org.apache.lucene.facet.range;
 
 import java.io.IOException;
-import java.util.Collections;
 import java.util.Objects;
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.ConstantScoreScorer;
 import org.apache.lucene.search.ConstantScoreWeight;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.LongValues;
+import org.apache.lucene.search.LongValuesSource;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.TwoPhaseIterator;
@@ -87,9 +87,9 @@ public final class LongRange extends Range {
   private static class ValueSourceQuery extends Query {
     private final LongRange range;
     private final Query fastMatchQuery;
-    private final ValueSource valueSource;
+    private final LongValuesSource valueSource;
 
-    ValueSourceQuery(LongRange range, Query fastMatchQuery, ValueSource valueSource) {
+    ValueSourceQuery(LongRange range, Query fastMatchQuery, LongValuesSource valueSource) {
       this.range = range;
       this.fastMatchQuery = fastMatchQuery;
       this.valueSource = valueSource;
@@ -150,11 +150,11 @@ public final class LongRange extends Range {
             approximation = s.iterator();
           }
 
-          final FunctionValues values = valueSource.getValues(Collections.emptyMap(), context);
+          final LongValues values = valueSource.getValues(context, null);
           final TwoPhaseIterator twoPhase = new TwoPhaseIterator(approximation) {
             @Override
             public boolean matches() throws IOException {
-              return range.accept(values.longVal(approximation.docID()));
+              return values.advanceExact(approximation.docID()) && range.accept(values.longValue());
             }
 
             @Override
@@ -169,8 +169,28 @@ public final class LongRange extends Range {
 
   }
 
-  @Override
+
+  /**
+   * @deprecated Use {@link #getQuery(Query, LongValuesSource)}
+   */
+  @Deprecated
   public Query getQuery(final Query fastMatchQuery, final ValueSource valueSource) {
+    return new ValueSourceQuery(this, fastMatchQuery, valueSource.asLongValuesSource());
+  }
+
+  /**
+   * Create a Query that matches documents in this range
+   *
+   * The query will check all documents that match the provided match query,
+   * or every document in the index if the match query is null.
+   *
+   * If the value source is static, eg an indexed numeric field, it may be
+   * faster to use {@link org.apache.lucene.search.PointRangeQuery}
+   *
+   * @param fastMatchQuery a query to use as a filter
+   * @param valueSource    the source of values for the range check
+   */
+  public Query getQuery(Query fastMatchQuery, LongValuesSource valueSource) {
     return new ValueSourceQuery(this, fastMatchQuery, valueSource);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/713b65d1/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java
index 0512ab3..a3cfc71 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java
@@ -17,27 +17,25 @@
 package org.apache.lucene.facet.range;
 
 import java.io.IOException;
-import java.util.Collections;
 import java.util.List;
 
 import org.apache.lucene.facet.Facets;
-import org.apache.lucene.facet.FacetsCollector.MatchingDocs;
 import org.apache.lucene.facet.FacetsCollector;
+import org.apache.lucene.facet.FacetsCollector.MatchingDocs;
 import org.apache.lucene.index.IndexReaderContext;
 import org.apache.lucene.index.ReaderUtil;
-import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.queries.function.valuesource.LongFieldSource;
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.LongValues;
+import org.apache.lucene.search.LongValuesSource;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Weight;
 
 /** {@link Facets} implementation that computes counts for
- *  dynamic long ranges from a provided {@link ValueSource},
- *  using {@link FunctionValues#longVal}.  Use
+ *  dynamic long ranges from a provided {@link LongValuesSource}.  Use
  *  this for dimensions that change in real-time (e.g. a
  *  relative time based dimension like "Past day", "Past 2
  *  days", etc.) or that change for each request (e.g. 
@@ -48,28 +46,49 @@ import org.apache.lucene.search.Weight;
 public class LongRangeFacetCounts extends RangeFacetCounts {
 
   /** Create {@code LongRangeFacetCounts}, using {@link
-   *  LongFieldSource} from the specified field. */
+   *  LongValuesSource} from the specified field. */
   public LongRangeFacetCounts(String field, FacetsCollector hits, LongRange... ranges) throws IOException {
-    this(field, new LongFieldSource(field), hits, ranges);
+    this(field, LongValuesSource.fromLongField(field), hits, ranges);
+  }
+
+  /**
+   * Create {@code RangeFacetCounts}, using the provided {@link ValueSource}.
+   *
+   * @deprecated Use {@link #LongRangeFacetCounts(String, LongValuesSource, FacetsCollector, LongRange...)}
+   */
+  @Deprecated
+  public LongRangeFacetCounts(String field, ValueSource valueSource, FacetsCollector hits, LongRange... ranges) throws IOException {
+    this(field, valueSource.asLongValuesSource(), hits, null, ranges);
   }
 
   /** Create {@code RangeFacetCounts}, using the provided
    *  {@link ValueSource}. */
-  public LongRangeFacetCounts(String field, ValueSource valueSource, FacetsCollector hits, LongRange... ranges) throws IOException {
+  public LongRangeFacetCounts(String field, LongValuesSource valueSource, FacetsCollector hits, LongRange... ranges) throws IOException {
     this(field, valueSource, hits, null, ranges);
   }
 
+  /**
+   * Create {@code RangeFacetCounts}, using the provided {@link ValueSource}.
+   *
+   * @deprecated Use {@link #LongRangeFacetCounts(String, LongValuesSource, FacetsCollector, Query, LongRange...)}
+   */
+  @Deprecated
+  public LongRangeFacetCounts(String field, ValueSource valueSource, FacetsCollector hits, Query fastMatchQuery, LongRange... ranges) throws IOException {
+    this(field, valueSource.asLongValuesSource(), hits, fastMatchQuery, ranges);
+  }
+
+
   /** Create {@code RangeFacetCounts}, using the provided
    *  {@link ValueSource}, and using the provided Filter as
    *  a fastmatch: only documents passing the filter are
    *  checked for the matching ranges.  The filter must be
    *  random access (implement {@link DocIdSet#bits}). */
-  public LongRangeFacetCounts(String field, ValueSource valueSource, FacetsCollector hits, Query fastMatchQuery, LongRange... ranges) throws IOException {
+  public LongRangeFacetCounts(String field, LongValuesSource valueSource, FacetsCollector hits, Query fastMatchQuery, LongRange... ranges) throws IOException {
     super(field, ranges, fastMatchQuery);
     count(valueSource, hits.getMatchingDocs());
   }
 
-  private void count(ValueSource valueSource, List<MatchingDocs> matchingDocs) throws IOException {
+  private void count(LongValuesSource valueSource, List<MatchingDocs> matchingDocs) throws IOException {
 
     LongRange[] ranges = (LongRange[]) this.ranges;
 
@@ -77,7 +96,7 @@ public class LongRangeFacetCounts extends RangeFacetCounts {
 
     int missingCount = 0;
     for (MatchingDocs hits : matchingDocs) {
-      FunctionValues fv = valueSource.getValues(Collections.emptyMap(), hits.context);
+      LongValues fv = valueSource.getValues(hits.context, null);
       
       totCount += hits.totalHits;
       final DocIdSetIterator fastMatchDocs;
@@ -109,8 +128,8 @@ public class LongRangeFacetCounts extends RangeFacetCounts {
           }
         }
         // Skip missing docs:
-        if (fv.exists(doc)) {
-          counter.add(fv.longVal(doc));
+        if (fv.advanceExact(doc)) {
+          counter.add(fv.longValue());
         } else {
           missingCount++;
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/713b65d1/lucene/facet/src/java/org/apache/lucene/facet/range/Range.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/Range.java b/lucene/facet/src/java/org/apache/lucene/facet/range/Range.java
index 5f6de98..82b8088 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/range/Range.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/range/Range.java
@@ -16,10 +16,6 @@
  */
 package org.apache.lucene.facet.range;
 
-import org.apache.lucene.facet.DrillDownQuery; // javadocs
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.search.Query;
-
 /** Base class for a single labeled range.
  *
  *  @lucene.experimental */
@@ -36,33 +32,6 @@ public abstract class Range {
     this.label = label;
   }
 
-  /** Returns a new {@link Query} accepting only documents
-   *  in this range.  This query might not be very efficient
-   *  when run on its own since it is optimized towards
-   *  random-access, so it is best used either with
-   *  {@link DrillDownQuery#add(String, Query) DrillDownQuery}
-   *  or when intersected with another query that can lead the
-   *  iteration.  If the {@link ValueSource} is static, e.g. an
-   *  indexed numeric field, then it may be more efficient to use
-   *  {@link org.apache.lucene.search.PointRangeQuery}. The provided fastMatchQuery,
-   *  if non-null, will first be consulted, and only if
-   *  that is set for each document will the range then be
-   *  checked. */
-  public abstract Query getQuery(Query fastMatchQuery, ValueSource valueSource);
-
-  /** Returns a new {@link Query} accepting only documents
-   *  in this range.  This query might not be very efficient
-   *  when run on its own since it is optimized towards
-   *  random-access, so it is best used either with
-   *  {@link DrillDownQuery#add(String, Query) DrillDownQuery}
-   *  or when intersected with another query that can lead the
-   *  iteration.  If the {@link ValueSource} is static, e.g. an
-   *  indexed numeric field, then it may be more efficient to
-   *  use {@link org.apache.lucene.search.PointRangeQuery}. */
-  public Query getQuery(ValueSource valueSource) {
-    return getQuery(null, valueSource);
-  }
-
   /** Invoke this for a useless range. */
   protected void failNoMatch() {
     throw new IllegalArgumentException("range \"" + label + "\" matches nothing");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/713b65d1/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FakeScorer.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FakeScorer.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FakeScorer.java
deleted file mode 100644
index 238b74c..0000000
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FakeScorer.java
+++ /dev/null
@@ -1,53 +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.lucene.facet.taxonomy;
-
-import java.io.IOException;
-
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.Scorer;
-
-class FakeScorer extends Scorer {
-
-  float score;
-  int doc = -1;
-  int freq = 1;
-
-  FakeScorer() {
-    super(null);
-  }
-
-  @Override
-  public int docID() {
-    return doc;
-  }
-
-  @Override
-  public DocIdSetIterator iterator() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public int freq() throws IOException {
-    return freq;
-  }
-
-  @Override
-  public float score() throws IOException {
-    return score;
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/713b65d1/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java
index 4010c81..0a73ae5 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java
@@ -17,7 +17,6 @@
 package org.apache.lucene.facet.taxonomy;
 
 import java.io.IOException;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
@@ -29,6 +28,8 @@ import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
 import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.DoubleValues;
+import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.util.IntsRef;
 
@@ -39,52 +40,94 @@ import org.apache.lucene.util.IntsRef;
 public class TaxonomyFacetSumValueSource extends FloatTaxonomyFacets {
   private final OrdinalsReader ordinalsReader;
 
-  /** Aggreggates float facet values from the provided
+  /**
+   * Aggreggates double facet values from the provided
    *  {@link ValueSource}, pulling ordinals using {@link
    *  DocValuesOrdinalsReader} against the default indexed
    *  facet field {@link
-   *  FacetsConfig#DEFAULT_INDEX_FIELD_NAME}. */
+   *  FacetsConfig#DEFAULT_INDEX_FIELD_NAME}.
+   *
+   *  @deprecated {@link #TaxonomyFacetSumValueSource(TaxonomyReader, FacetsConfig, FacetsCollector, DoubleValuesSource)}
+   */
+  @Deprecated
   public TaxonomyFacetSumValueSource(TaxonomyReader taxoReader, FacetsConfig config,
                                      FacetsCollector fc, ValueSource valueSource) throws IOException {
     this(new DocValuesOrdinalsReader(FacetsConfig.DEFAULT_INDEX_FIELD_NAME), taxoReader, config, fc, valueSource);
   }
 
-  /** Aggreggates float facet values from the provided
+  /**
+   * Aggreggates double facet values from the provided
+   * {@link DoubleValuesSource}, pulling ordinals using {@link
+   * DocValuesOrdinalsReader} against the default indexed
+   * facet field {@link FacetsConfig#DEFAULT_INDEX_FIELD_NAME}.
+   */
+   public TaxonomyFacetSumValueSource(TaxonomyReader taxoReader, FacetsConfig config,
+                                     FacetsCollector fc, DoubleValuesSource valueSource) throws IOException {
+    this(new DocValuesOrdinalsReader(FacetsConfig.DEFAULT_INDEX_FIELD_NAME), taxoReader, config, fc, valueSource);
+  }
+
+  /**
+   * Aggreggates float facet values from the provided
    *  {@link ValueSource}, and pulls ordinals from the
-   *  provided {@link OrdinalsReader}. */
+   *  provided {@link OrdinalsReader}.
+   *
+   *  @deprecated use {@link #TaxonomyFacetSumValueSource(OrdinalsReader, TaxonomyReader, FacetsConfig, FacetsCollector, DoubleValuesSource)}
+   */
+  @Deprecated
   public TaxonomyFacetSumValueSource(OrdinalsReader ordinalsReader, TaxonomyReader taxoReader,
                                      FacetsConfig config, FacetsCollector fc, ValueSource valueSource) throws IOException {
     super(ordinalsReader.getIndexFieldName(), taxoReader, config);
     this.ordinalsReader = ordinalsReader;
-    sumValues(fc.getMatchingDocs(), fc.getKeepScores(), valueSource);
+    sumValues(fc.getMatchingDocs(), fc.getKeepScores(), valueSource.asDoubleValuesSource());
   }
 
-  private final void sumValues(List<MatchingDocs> matchingDocs, boolean keepScores, ValueSource valueSource) throws IOException {
-    final FakeScorer scorer = new FakeScorer();
-    Map<String, Scorer> context = new HashMap<>();
-    if (keepScores) {
-      context.put("scorer", scorer);
-    }
+  /**
+   * Aggreggates float facet values from the provided
+   *  {@link DoubleValuesSource}, and pulls ordinals from the
+   *  provided {@link OrdinalsReader}.
+   */
+   public TaxonomyFacetSumValueSource(OrdinalsReader ordinalsReader, TaxonomyReader taxoReader,
+                                     FacetsConfig config, FacetsCollector fc, DoubleValuesSource vs) throws IOException {
+    super(ordinalsReader.getIndexFieldName(), taxoReader, config);
+    this.ordinalsReader = ordinalsReader;
+    sumValues(fc.getMatchingDocs(), fc.getKeepScores(), vs);
+  }
+
+  private static DoubleValues scores(MatchingDocs hits) {
+    return new DoubleValues() {
+
+      int index = -1;
+
+      @Override
+      public double doubleValue() throws IOException {
+        return hits.scores[index];
+      }
+
+      @Override
+      public boolean advanceExact(int doc) throws IOException {
+        index++;
+        return true;
+      }
+    };
+  }
+
+  private void sumValues(List<MatchingDocs> matchingDocs, boolean keepScores, DoubleValuesSource valueSource) throws IOException {
+
     IntsRef scratch = new IntsRef();
     for(MatchingDocs hits : matchingDocs) {
       OrdinalsReader.OrdinalsSegmentReader ords = ordinalsReader.getReader(hits.context);
-      
-      int scoresIdx = 0;
-      float[] scores = hits.scores;
-
-      FunctionValues functionValues = valueSource.getValues(context, hits.context);
+      DoubleValues scores = keepScores ? scores(hits) : null;
+      DoubleValues functionValues = valueSource.getValues(hits.context, scores);
       DocIdSetIterator docs = hits.bits.iterator();
       
       int doc;
       while ((doc = docs.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
         ords.get(doc, scratch);
-        if (keepScores) {
-          scorer.doc = doc;
-          scorer.score = scores[scoresIdx++];
-        }
-        float value = (float) functionValues.doubleVal(doc);
-        for(int i=0;i<scratch.length;i++) {
-          values[scratch.ints[i]] += value;
+        if (functionValues.advanceExact(doc)) {
+          float value = (float) functionValues.doubleValue();
+          for (int i = 0; i < scratch.length; i++) {
+            values[scratch.ints[i]] += value;
+          }
         }
       }
     }
@@ -92,9 +135,13 @@ public class TaxonomyFacetSumValueSource extends FloatTaxonomyFacets {
     rollup();
   }
 
-  /** {@link ValueSource} that returns the score for each
+  /**
+   * {@link ValueSource} that returns the score for each
    *  hit; use this to aggregate the sum of all hit scores
-   *  for each facet label.  */
+   *  for each facet label.
+   *
+   * @deprecated Use {@link DoubleValuesSource#SCORES}
+   */
   public static class ScoreValueSource extends ValueSource {
 
     /** Sole constructor. */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/713b65d1/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java b/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
index 7250ef4..ff207d3 100644
--- a/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
+++ b/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
@@ -28,8 +28,8 @@ import org.apache.lucene.document.DoublePoint;
 import org.apache.lucene.document.LongPoint;
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.facet.DrillDownQuery;
-import org.apache.lucene.facet.DrillSideways.DrillSidewaysResult;
 import org.apache.lucene.facet.DrillSideways;
+import org.apache.lucene.facet.DrillSideways.DrillSidewaysResult;
 import org.apache.lucene.facet.FacetField;
 import org.apache.lucene.facet.FacetResult;
 import org.apache.lucene.facet.FacetTestCase;
@@ -46,11 +46,11 @@ import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
 import org.apache.lucene.queries.function.valuesource.DoubleFieldSource;
 import org.apache.lucene.queries.function.valuesource.LongFieldSource;
+import org.apache.lucene.search.DoubleValues;
+import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
@@ -708,7 +708,7 @@ public class TestRangeFacetCounts extends FacetTestCase {
 
   }
 
-  public void testCustomDoublesValueSource() throws Exception {
+  public void testCustomDoubleValuesSource() throws Exception {
     Directory dir = newDirectory();
     RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
     
@@ -720,33 +720,30 @@ public class TestRangeFacetCounts extends FacetTestCase {
     // Test wants 3 docs in one segment:
     writer.forceMerge(1);
 
-    final ValueSource vs = new ValueSource() {
-        @SuppressWarnings("rawtypes")
-        @Override
-        public FunctionValues getValues(Map ignored, LeafReaderContext ignored2) {
-          return new DoubleDocValues(null) {
-            @Override
-            public double doubleVal(int doc) {
-              return doc+1;
-            }
-          };
-        }
+    final DoubleValuesSource vs = new DoubleValuesSource() {
 
-        @Override
-        public boolean equals(Object o) {
-          return o != null && getClass() == o.getClass();
-        }
+      @Override
+      public DoubleValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
+        return new DoubleValues() {
+          int doc = -1;
+          @Override
+          public double doubleValue() throws IOException {
+            return doc + 1;
+          }
 
-        @Override
-        public int hashCode() {
-          return getClass().hashCode();
-        }
+          @Override
+          public boolean advanceExact(int doc) throws IOException {
+            this.doc = doc;
+            return true;
+          }
+        };
+      }
 
-        @Override
-        public String description() {
-          throw new UnsupportedOperationException();
-        }
-      };
+      @Override
+      public boolean needsScores() {
+        return false;
+      }
+    };
 
     FacetsConfig config = new FacetsConfig();
     

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/713b65d1/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetSumValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetSumValueSource.java b/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetSumValueSource.java
index 0ad90ba..31bf6e1 100644
--- a/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetSumValueSource.java
+++ b/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetSumValueSource.java
@@ -52,6 +52,7 @@ import org.apache.lucene.queries.function.valuesource.IntFieldSource;
 import org.apache.lucene.queries.function.valuesource.LongFieldSource;
 import org.apache.lucene.search.BoostQuery;
 import org.apache.lucene.search.ConstantScoreQuery;
+import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.Query;
@@ -266,7 +267,7 @@ public class TestTaxonomyFacetSumValueSource extends FacetTestCase {
     
     TopDocs td = FacetsCollector.search(newSearcher(r), csq, 10, fc);
 
-    Facets facets = new TaxonomyFacetSumValueSource(taxoReader, config, fc, new TaxonomyFacetSumValueSource.ScoreValueSource());
+    Facets facets = new TaxonomyFacetSumValueSource(taxoReader, config, fc, DoubleValuesSource.SCORES);
     
     int expected = (int) (td.getMaxScore() * td.totalHits);
     assertEquals(expected, facets.getSpecificValue("dim", "a").intValue());
@@ -408,7 +409,7 @@ public class TestTaxonomyFacetSumValueSource extends FacetTestCase {
     FacetsCollector.search(newSearcher(r), new MatchAllDocsQuery(), 10, fc);
     
     Facets facets1 = getTaxonomyFacetCounts(taxoReader, config, fc);
-    Facets facets2 = new TaxonomyFacetSumValueSource(new DocValuesOrdinalsReader("$b"), taxoReader, config, fc, new TaxonomyFacetSumValueSource.ScoreValueSource());
+    Facets facets2 = new TaxonomyFacetSumValueSource(new DocValuesOrdinalsReader("$b"), taxoReader, config, fc, DoubleValuesSource.SCORES);
 
     assertEquals(r.maxDoc(), facets1.getTopChildren(10, "a").value.intValue());
     assertEquals(r.maxDoc(), facets2.getTopChildren(10, "b").value.doubleValue(), 1E-10);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/713b65d1/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSource.java b/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSource.java
index 4064fc9..5bf6324 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSource.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSource.java
@@ -17,13 +17,20 @@
 package org.apache.lucene.queries.function;
 
 import java.io.IOException;
+import java.util.HashMap;
 import java.util.IdentityHashMap;
 import java.util.Map;
 
 import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.DoubleValues;
+import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.FieldComparator;
 import org.apache.lucene.search.FieldComparatorSource;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.LongValues;
+import org.apache.lucene.search.LongValuesSource;
+import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.SimpleFieldComparator;
 import org.apache.lucene.search.SortField;
 
@@ -78,6 +85,110 @@ public abstract class ValueSource {
     return context;
   }
 
+  private static class FakeScorer extends Scorer {
+
+    int current = -1;
+    float score = 0;
+
+    FakeScorer() {
+      super(null);
+    }
+
+    @Override
+    public int docID() {
+      return current;
+    }
+
+    @Override
+    public float score() throws IOException {
+      return score;
+    }
+
+    @Override
+    public int freq() throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public DocIdSetIterator iterator() {
+      throw new UnsupportedOperationException();
+    }
+  }
+
+  /**
+   * Expose this ValueSource as a LongValuesSource
+   */
+  public LongValuesSource asLongValuesSource() {
+    return new LongValuesSource() {
+      @Override
+      public LongValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
+        Map context = new IdentityHashMap<>();
+        FakeScorer scorer = new FakeScorer();
+        context.put("scorer", scorer);
+        final FunctionValues fv = ValueSource.this.getValues(context, ctx);
+        return new LongValues() {
+
+          @Override
+          public long longValue() throws IOException {
+            return fv.longVal(scorer.current);
+          }
+
+          @Override
+          public boolean advanceExact(int doc) throws IOException {
+            scorer.current = doc;
+            if (scores != null && scores.advanceExact(doc))
+              scorer.score = (float) scores.doubleValue();
+            else
+              scorer.score = 0;
+            return fv.exists(doc);
+          }
+        };
+      }
+
+      @Override
+      public boolean needsScores() {
+        return false;
+      }
+    };
+  }
+
+  /**
+   * Expose this ValueSource as a DoubleValuesSource
+   */
+  public DoubleValuesSource asDoubleValuesSource() {
+    return new DoubleValuesSource() {
+      @Override
+      public DoubleValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
+        Map context = new HashMap<>();
+        FakeScorer scorer = new FakeScorer();
+        context.put("scorer", scorer);
+        FunctionValues fv = ValueSource.this.getValues(context, ctx);
+        return new DoubleValues() {
+
+          @Override
+          public double doubleValue() throws IOException {
+            return fv.doubleVal(scorer.current);
+          }
+
+          @Override
+          public boolean advanceExact(int doc) throws IOException {
+            scorer.current = doc;
+            if (scores != null && scores.advanceExact(doc)) {
+              scorer.score = (float) scores.doubleValue();
+            }
+            else
+              scorer.score = 0;
+            return fv.exists(doc);
+          }
+        };
+      }
+
+      @Override
+      public boolean needsScores() {
+        return true;  // be on the safe side
+      }
+    };
+  }
 
   //
   // Sorting by function


[6/6] lucene-solr:master: LUCENE-7611: Suggester uses LongValuesSource in place of ValueSource

Posted by ro...@apache.org.
LUCENE-7611: Suggester uses LongValuesSource in place of ValueSource


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/1a95c5ac
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/1a95c5ac
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/1a95c5ac

Branch: refs/heads/master
Commit: 1a95c5acd0f69efb1a24b2c980a289289e703758
Parents: 713b65d
Author: Alan Woodward <ro...@apache.org>
Authored: Fri Jan 6 11:03:09 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Sat Jan 7 11:52:25 2017 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   3 +
 .../apache/lucene/search/LongValuesSource.java  |  27 ++
 .../suggest/DocumentValueSourceDictionary.java  |  72 ++++-
 .../DocumentValueSourceDictionaryTest.java      | 264 ++++++++++++++++++-
 .../DocumentExpressionDictionaryFactory.java    |   6 +-
 5 files changed, 352 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1a95c5ac/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index d0bedb7..30c9ab0 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -82,6 +82,9 @@ API Changes
 * LUCENE-7610: The facets module now uses the DoubleValuesSource API, and
   methods that take ValueSource parameters are deprecated (Alan Woodward)
 
+* LUCENE-7611: DocumentValueSourceDictionary now takes a LongValuesSource
+  as a parameter, and the ValueSource equivalent is deprecated (Alan Woodward)
+
 New features
 
 * LUCENE-5867: Added BooleanSimilarity. (Robert Muir, Adrien Grand)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1a95c5ac/lucene/core/src/java/org/apache/lucene/search/LongValuesSource.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/LongValuesSource.java b/lucene/core/src/java/org/apache/lucene/search/LongValuesSource.java
index 9d00355..524822c 100644
--- a/lucene/core/src/java/org/apache/lucene/search/LongValuesSource.java
+++ b/lucene/core/src/java/org/apache/lucene/search/LongValuesSource.java
@@ -74,6 +74,33 @@ public abstract class LongValuesSource {
     return fromLongField(field);
   }
 
+  /**
+   * Creates a LongValuesSource that always returns a constant value
+   */
+  public static LongValuesSource constant(long value) {
+    return new LongValuesSource() {
+      @Override
+      public LongValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
+        return new LongValues() {
+          @Override
+          public long longValue() throws IOException {
+            return value;
+          }
+
+          @Override
+          public boolean advanceExact(int doc) throws IOException {
+            return true;
+          }
+        };
+      }
+
+      @Override
+      public boolean needsScores() {
+        return false;
+      }
+    };
+  }
+
   private static class FieldValuesSource extends LongValuesSource {
 
     final String field;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1a95c5ac/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java
index 2c0b8f4..656dc04 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java
@@ -17,15 +17,15 @@
 package org.apache.lucene.search.suggest;
 
 import java.io.IOException;
-import java.util.HashMap;
 import java.util.List;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.ReaderUtil;
-import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.search.LongValues;
+import org.apache.lucene.search.LongValuesSource;
 
 
 /**
@@ -34,7 +34,7 @@ import org.apache.lucene.queries.function.ValueSource;
  * optionally contexts information
  * taken from stored fields in a Lucene index. Similar to 
  * {@link DocumentDictionary}, except it obtains the weight
- * of the terms in a document based on a {@link ValueSource}.
+ * of the terms in a document based on a {@link LongValuesSource}.
  * </p>
  * <b>NOTE:</b> 
  *  <ul>
@@ -46,44 +46,75 @@ import org.apache.lucene.queries.function.ValueSource;
  *    </li>
  *  </ul>
  *  <p>
- *  In practice the {@link ValueSource} will likely be obtained
+ *  In practice the {@link LongValuesSource} will likely be obtained
  *  using the lucene expression module. The following example shows
- *  how to create a {@link ValueSource} from a simple addition of two
+ *  how to create a {@link LongValuesSource} from a simple addition of two
  *  fields:
  *  <code>
  *    Expression expression = JavascriptCompiler.compile("f1 + f2");
  *    SimpleBindings bindings = new SimpleBindings();
  *    bindings.add(new SortField("f1", SortField.Type.LONG));
  *    bindings.add(new SortField("f2", SortField.Type.LONG));
- *    ValueSource valueSource = expression.getValueSource(bindings);
+ *    LongValuesSource valueSource = expression.getDoubleValuesSource(bindings).toLongValuesSource();
  *  </code>
  *  </p>
  *
  */
 public class DocumentValueSourceDictionary extends DocumentDictionary {
   
-  private final ValueSource weightsValueSource;
+  private final LongValuesSource weightsValueSource;
   
   /**
    * Creates a new dictionary with the contents of the fields named <code>field</code>
    * for the terms, <code>payload</code> for the corresponding payloads, <code>contexts</code>
    * for the associated contexts and uses the <code>weightsValueSource</code> supplied 
    * to determine the score.
+   *
+   * @deprecated Use {@link #DocumentValueSourceDictionary(IndexReader, String, LongValuesSource, String, String)}
    */
+  @Deprecated
   public DocumentValueSourceDictionary(IndexReader reader, String field,
                                        ValueSource weightsValueSource, String payload, String contexts) {
     super(reader, field, null, payload, contexts);
+    this.weightsValueSource = weightsValueSource.asLongValuesSource();
+  }
+
+  /**
+   * Creates a new dictionary with the contents of the fields named <code>field</code>
+   * for the terms, <code>payload</code> for the corresponding payloads, <code>contexts</code>
+   * for the associated contexts and uses the <code>weightsValueSource</code> supplied
+   * to determine the score.
+   */
+  public DocumentValueSourceDictionary(IndexReader reader, String field,
+                                       LongValuesSource weightsValueSource, String payload, String contexts) {
+    super(reader, field, null, payload, contexts);
     this.weightsValueSource = weightsValueSource;
   }
+
   /**
    * Creates a new dictionary with the contents of the fields named <code>field</code>
    * for the terms, <code>payloadField</code> for the corresponding payloads
    * and uses the <code>weightsValueSource</code> supplied to determine the 
    * score.
+   *
+   * @deprecated Use {@link #DocumentValueSourceDictionary(IndexReader, String, LongValuesSource, String)}
    */
+  @Deprecated
   public DocumentValueSourceDictionary(IndexReader reader, String field,
                                        ValueSource weightsValueSource, String payload) {
     super(reader, field, null, payload);
+    this.weightsValueSource = weightsValueSource.asLongValuesSource();
+  }
+
+  /**
+   * Creates a new dictionary with the contents of the fields named <code>field</code>
+   * for the terms, <code>payloadField</code> for the corresponding payloads
+   * and uses the <code>weightsValueSource</code> supplied to determine the
+   * score.
+   */
+  public DocumentValueSourceDictionary(IndexReader reader, String field,
+                                       LongValuesSource weightsValueSource, String payload) {
+    super(reader, field, null, payload);
     this.weightsValueSource = weightsValueSource;
   }
   
@@ -91,11 +122,25 @@ public class DocumentValueSourceDictionary extends DocumentDictionary {
    * Creates a new dictionary with the contents of the fields named <code>field</code>
    * for the terms and uses the <code>weightsValueSource</code> supplied to determine the 
    * score.
+   *
+   * @deprecated Use {@link #DocumentValueSourceDictionary(IndexReader, String, LongValuesSource)}
    */
+  @Deprecated
   public DocumentValueSourceDictionary(IndexReader reader, String field,
                                        ValueSource weightsValueSource) {
     super(reader, field, null, null);
-    this.weightsValueSource = weightsValueSource;  
+    this.weightsValueSource = weightsValueSource.asLongValuesSource();
+  }
+
+  /**
+   * Creates a new dictionary with the contents of the fields named <code>field</code>
+   * for the terms and uses the <code>weightsValueSource</code> supplied to determine the
+   * score.
+   */
+  public DocumentValueSourceDictionary(IndexReader reader, String field,
+                                       LongValuesSource weightsValueSource) {
+    super(reader, field, null, null);
+    this.weightsValueSource = weightsValueSource;
   }
   
   @Override
@@ -105,7 +150,7 @@ public class DocumentValueSourceDictionary extends DocumentDictionary {
   
   final class DocumentValueSourceInputIterator extends DocumentDictionary.DocumentInputIterator {
     
-    private FunctionValues currentWeightValues;
+    private LongValues currentWeightValues;
     /** leaves of the reader */
     private final List<LeafReaderContext> leaves;
     /** starting docIds of all the leaves */
@@ -123,7 +168,7 @@ public class DocumentValueSourceDictionary extends DocumentDictionary {
       }
       starts[leaves.size()] = reader.maxDoc();
       currentWeightValues = (leaves.size() > 0) 
-          ? weightsValueSource.getValues(new HashMap<String, Object>(), leaves.get(currentLeafIndex))
+          ? weightsValueSource.getValues(leaves.get(currentLeafIndex), null)
           : null;
     }
     
@@ -140,13 +185,16 @@ public class DocumentValueSourceDictionary extends DocumentDictionary {
       if (subIndex != currentLeafIndex) {
         currentLeafIndex = subIndex;
         try {
-          currentWeightValues = weightsValueSource.getValues(new HashMap<String, Object>(), leaves.get(currentLeafIndex));
+          currentWeightValues = weightsValueSource.getValues(leaves.get(currentLeafIndex), null);
         } catch (IOException e) {
           throw new RuntimeException(e);
         }
       }
       try {
-        return currentWeightValues.longVal(docId - starts[subIndex]);
+        if (currentWeightValues.advanceExact(docId - starts[subIndex]))
+          return currentWeightValues.longValue();
+        else
+          return 0;
       } catch (IOException e) {
         throw new RuntimeException(e);
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1a95c5ac/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentValueSourceDictionaryTest.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentValueSourceDictionaryTest.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentValueSourceDictionaryTest.java
index 9e58a4e..92799cd 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentValueSourceDictionaryTest.java
+++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentValueSourceDictionaryTest.java
@@ -36,12 +36,16 @@ import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.valuesource.DoubleConstValueSource;
 import org.apache.lucene.queries.function.valuesource.LongFieldSource;
 import org.apache.lucene.queries.function.valuesource.SumFloatFunction;
+import org.apache.lucene.search.DoubleValues;
+import org.apache.lucene.search.LongValues;
+import org.apache.lucene.search.LongValuesSource;
 import org.apache.lucene.search.spell.Dictionary;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
@@ -59,7 +63,7 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
   static final String CONTEXTS_FIELD_NAME = "c1";
   
   @Test
-  public void testEmptyReader() throws IOException {
+  public void testValueSourceEmptyReader() throws IOException {
     Directory dir = newDirectory();
     Analyzer analyzer = new MockAnalyzer(random());
     IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
@@ -78,9 +82,30 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
 
     IOUtils.close(ir, analyzer, dir);
   }
+
+  @Test
+  public void testLongValuesSourceEmptyReader() throws IOException {
+    Directory dir = newDirectory();
+    Analyzer analyzer = new MockAnalyzer(random());
+    IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
+    iwc.setMergePolicy(newLogMergePolicy());
+    // Make sure the index is created?
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
+    writer.commit();
+    writer.close();
+    IndexReader ir = DirectoryReader.open(dir);
+    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, LongValuesSource.constant(10), PAYLOAD_FIELD_NAME);
+    InputIterator inputIterator = dictionary.getEntryIterator();
+
+    assertNull(inputIterator.next());
+    assertEquals(inputIterator.weight(), 0);
+    assertNull(inputIterator.payload());
+
+    IOUtils.close(ir, analyzer, dir);
+  }
   
   @Test
-  public void testBasic() throws IOException {
+  public void testValueSourceBasic() throws IOException {
     Directory dir = newDirectory();
     Analyzer analyzer = new MockAnalyzer(random());
     IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
@@ -112,9 +137,83 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
     assertTrue(docs.isEmpty());
     IOUtils.close(ir, analyzer, dir);
   }
+
+  private static LongValuesSource sum(String... fields) {
+    LongValuesSource[] sources = new LongValuesSource[fields.length];
+    for (int i = 0; i < fields.length; i++) {
+      sources[i] = LongValuesSource.fromLongField(fields[i]);
+    }
+    return new LongValuesSource() {
+      @Override
+      public LongValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
+        LongValues[] values = new LongValues[fields.length];
+        for (int i = 0; i < sources.length; i++) {
+          values[i] = sources[i].getValues(ctx, scores);
+        }
+        return new LongValues() {
+          @Override
+          public long longValue() throws IOException {
+            long v = 0;
+            for (LongValues value : values) {
+              v += value.longValue();
+            }
+            return v;
+          }
+
+          @Override
+          public boolean advanceExact(int doc) throws IOException {
+            boolean v = true;
+            for (LongValues value : values) {
+              v &= value.advanceExact(doc);
+            }
+            return v;
+          }
+        };
+      }
+
+      @Override
+      public boolean needsScores() {
+        return false;
+      }
+    };
+  }
+
+  @Test
+  public void testLongValuesSourceBasic() throws IOException {
+    Directory dir = newDirectory();
+    Analyzer analyzer = new MockAnalyzer(random());
+    IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
+    Map<String, Document> docs = generateIndexDocuments(atLeast(100));
+    for(Document doc: docs.values()) {
+      writer.addDocument(doc);
+    }
+    writer.commit();
+    writer.close();
+
+    IndexReader ir = DirectoryReader.open(dir);
+    LongValuesSource sumValueSource = sum(WEIGHT_FIELD_NAME_1, WEIGHT_FIELD_NAME_2, WEIGHT_FIELD_NAME_3);
+    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, sumValueSource, PAYLOAD_FIELD_NAME);
+    InputIterator inputIterator = dictionary.getEntryIterator();
+    BytesRef f;
+    while((f = inputIterator.next())!=null) {
+      Document doc = docs.remove(f.utf8ToString());
+      long w1 = doc.getField(WEIGHT_FIELD_NAME_1).numericValue().longValue();
+      long w2 = doc.getField(WEIGHT_FIELD_NAME_2).numericValue().longValue();
+      long w3 = doc.getField(WEIGHT_FIELD_NAME_3).numericValue().longValue();
+      assertTrue(f.equals(new BytesRef(doc.get(FIELD_NAME))));
+      assertEquals(inputIterator.weight(), (w1 + w2 + w3));
+      IndexableField payloadField = doc.getField(PAYLOAD_FIELD_NAME);
+      if (payloadField == null) assertTrue(inputIterator.payload().length == 0);
+      else assertEquals(inputIterator.payload(), payloadField.binaryValue());
+    }
+    assertTrue(docs.isEmpty());
+    IOUtils.close(ir, analyzer, dir);
+  }
   
   @Test
-  public void testWithContext() throws IOException {
+  public void testValueSourceWithContext() throws IOException {
     Directory dir = newDirectory();
     Analyzer analyzer = new MockAnalyzer(random());
     IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
@@ -153,7 +252,46 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
   }
 
   @Test
-  public void testWithoutPayload() throws IOException {
+  public void testLongValuesSourceWithContext() throws IOException {
+    Directory dir = newDirectory();
+    Analyzer analyzer = new MockAnalyzer(random());
+    IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
+    Map<String, Document> docs = generateIndexDocuments(atLeast(100));
+    for(Document doc: docs.values()) {
+      writer.addDocument(doc);
+    }
+    writer.commit();
+    writer.close();
+
+    IndexReader ir = DirectoryReader.open(dir);
+    LongValuesSource sumValues = sum(WEIGHT_FIELD_NAME_1, WEIGHT_FIELD_NAME_2, WEIGHT_FIELD_NAME_3);
+    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, sumValues, PAYLOAD_FIELD_NAME, CONTEXTS_FIELD_NAME);
+    InputIterator inputIterator = dictionary.getEntryIterator();
+    BytesRef f;
+    while((f = inputIterator.next())!=null) {
+      Document doc = docs.remove(f.utf8ToString());
+      long w1 = doc.getField(WEIGHT_FIELD_NAME_1).numericValue().longValue();
+      long w2 = doc.getField(WEIGHT_FIELD_NAME_2).numericValue().longValue();
+      long w3 = doc.getField(WEIGHT_FIELD_NAME_3).numericValue().longValue();
+      assertTrue(f.equals(new BytesRef(doc.get(FIELD_NAME))));
+      assertEquals(inputIterator.weight(), (w1 + w2 + w3));
+      IndexableField payloadField = doc.getField(PAYLOAD_FIELD_NAME);
+      if (payloadField == null) assertTrue(inputIterator.payload().length == 0);
+      else assertEquals(inputIterator.payload(), payloadField.binaryValue());
+      Set<BytesRef> originalCtxs = new HashSet<>();
+      for (IndexableField ctxf: doc.getFields(CONTEXTS_FIELD_NAME)) {
+        originalCtxs.add(ctxf.binaryValue());
+      }
+      assertEquals(originalCtxs, inputIterator.contexts());
+    }
+    assertTrue(docs.isEmpty());
+    IOUtils.close(ir, analyzer, dir);
+  }
+
+  @Test
+  public void testValueSourceWithoutPayload() throws IOException {
     Directory dir = newDirectory();
     Analyzer analyzer = new MockAnalyzer(random());
     IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
@@ -183,9 +321,41 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
     assertTrue(docs.isEmpty());
     IOUtils.close(ir, analyzer, dir);
   }
+
+  @Test
+  public void testLongValuesSourceWithoutPayload() throws IOException {
+    Directory dir = newDirectory();
+    Analyzer analyzer = new MockAnalyzer(random());
+    IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
+    Map<String, Document> docs = generateIndexDocuments(atLeast(100));
+    for(Document doc: docs.values()) {
+      writer.addDocument(doc);
+    }
+    writer.commit();
+    writer.close();
+
+    IndexReader ir = DirectoryReader.open(dir);
+    LongValuesSource sumValues = sum(WEIGHT_FIELD_NAME_1, WEIGHT_FIELD_NAME_2, WEIGHT_FIELD_NAME_3);
+    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, sumValues);
+    InputIterator inputIterator = dictionary.getEntryIterator();
+    BytesRef f;
+    while((f = inputIterator.next())!=null) {
+      Document doc = docs.remove(f.utf8ToString());
+      long w1 = doc.getField(WEIGHT_FIELD_NAME_1).numericValue().longValue();
+      long w2 = doc.getField(WEIGHT_FIELD_NAME_2).numericValue().longValue();
+      long w3 = doc.getField(WEIGHT_FIELD_NAME_3).numericValue().longValue();
+      assertTrue(f.equals(new BytesRef(doc.get(FIELD_NAME))));
+      assertEquals(inputIterator.weight(), (w1 + w2 + w3));
+      assertNull(inputIterator.payload());
+    }
+    assertTrue(docs.isEmpty());
+    IOUtils.close(ir, analyzer, dir);
+  }
   
   @Test
-  public void testWithDeletions() throws IOException {
+  public void testValueSourceWithDeletions() throws IOException {
     Directory dir = newDirectory();
     Analyzer analyzer = new MockAnalyzer(random());
     IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
@@ -238,6 +408,60 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
     assertTrue(docs.isEmpty());
     IOUtils.close(ir, analyzer, dir);
   }
+
+  @Test
+  public void testLongValuesSourceWithDeletions() throws IOException {
+    Directory dir = newDirectory();
+    Analyzer analyzer = new MockAnalyzer(random());
+    IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
+    Map<String, Document> docs = generateIndexDocuments(atLeast(100));
+    Random rand = random();
+    List<String> termsToDel = new ArrayList<>();
+    for(Document doc : docs.values()) {
+      if(rand.nextBoolean() && termsToDel.size() < docs.size()-1) {
+        termsToDel.add(doc.get(FIELD_NAME));
+      }
+      writer.addDocument(doc);
+    }
+    writer.commit();
+
+    Term[] delTerms = new Term[termsToDel.size()];
+    for(int i=0; i < termsToDel.size() ; i++) {
+      delTerms[i] = new Term(FIELD_NAME, termsToDel.get(i));
+    }
+
+    for(Term delTerm: delTerms) {
+      writer.deleteDocuments(delTerm);
+    }
+    writer.commit();
+    writer.close();
+
+    for(String termToDel: termsToDel) {
+      assertTrue(null!=docs.remove(termToDel));
+    }
+
+    IndexReader ir = DirectoryReader.open(dir);
+    assertTrue("NumDocs should be > 0 but was " + ir.numDocs(), ir.numDocs() > 0);
+    assertEquals(ir.numDocs(), docs.size());
+    LongValuesSource sumValues = sum(WEIGHT_FIELD_NAME_1, WEIGHT_FIELD_NAME_2);
+    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, sumValues, PAYLOAD_FIELD_NAME);
+    InputIterator inputIterator = dictionary.getEntryIterator();
+    BytesRef f;
+    while((f = inputIterator.next())!=null) {
+      Document doc = docs.remove(f.utf8ToString());
+      long w1 = doc.getField(WEIGHT_FIELD_NAME_1).numericValue().longValue();
+      long w2 = doc.getField(WEIGHT_FIELD_NAME_2).numericValue().longValue();
+      assertTrue(f.equals(new BytesRef(doc.get(FIELD_NAME))));
+      assertEquals(inputIterator.weight(), w2+w1);
+      IndexableField payloadField = doc.getField(PAYLOAD_FIELD_NAME);
+      if (payloadField == null) assertTrue(inputIterator.payload().length == 0);
+      else assertEquals(inputIterator.payload(), payloadField.binaryValue());
+    }
+    assertTrue(docs.isEmpty());
+    IOUtils.close(ir, analyzer, dir);
+  }
   
   @Test
   public void testWithValueSource() throws IOException {
@@ -269,6 +493,36 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
     IOUtils.close(ir, analyzer, dir);
   }
 
+  @Test
+  public void testWithLongValuesSource() throws IOException {
+    Directory dir = newDirectory();
+    Analyzer analyzer = new MockAnalyzer(random());
+    IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
+    Map<String, Document> docs = generateIndexDocuments(atLeast(100));
+    for(Document doc: docs.values()) {
+      writer.addDocument(doc);
+    }
+    writer.commit();
+    writer.close();
+
+    IndexReader ir = DirectoryReader.open(dir);
+    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, LongValuesSource.constant(10), PAYLOAD_FIELD_NAME);
+    InputIterator inputIterator = dictionary.getEntryIterator();
+    BytesRef f;
+    while((f = inputIterator.next())!=null) {
+      Document doc = docs.remove(f.utf8ToString());
+      assertTrue(f.equals(new BytesRef(doc.get(FIELD_NAME))));
+      assertEquals(inputIterator.weight(), 10);
+      IndexableField payloadField = doc.getField(PAYLOAD_FIELD_NAME);
+      if (payloadField == null) assertTrue(inputIterator.payload().length == 0);
+      else assertEquals(inputIterator.payload(), payloadField.binaryValue());
+    }
+    assertTrue(docs.isEmpty());
+    IOUtils.close(ir, analyzer, dir);
+  }
+
   private Map<String, Document> generateIndexDocuments(int ndocs) {
     Map<String, Document> docs = new HashMap<>();
     for(int i = 0; i < ndocs ; i++) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1a95c5ac/solr/core/src/java/org/apache/solr/spelling/suggest/DocumentExpressionDictionaryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/spelling/suggest/DocumentExpressionDictionaryFactory.java b/solr/core/src/java/org/apache/solr/spelling/suggest/DocumentExpressionDictionaryFactory.java
index 24f1553..b0d7007 100644
--- a/solr/core/src/java/org/apache/solr/spelling/suggest/DocumentExpressionDictionaryFactory.java
+++ b/solr/core/src/java/org/apache/solr/spelling/suggest/DocumentExpressionDictionaryFactory.java
@@ -23,7 +23,7 @@ import java.util.Set;
 import org.apache.lucene.expressions.Expression;
 import org.apache.lucene.expressions.SimpleBindings;
 import org.apache.lucene.expressions.js.JavascriptCompiler;
-import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.search.LongValuesSource;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.spell.Dictionary;
 import org.apache.lucene.search.suggest.DocumentValueSourceDictionary;
@@ -93,7 +93,7 @@ public class DocumentExpressionDictionaryFactory extends DictionaryFactory {
         sortFields), payloadField);
   }
 
-  public ValueSource fromExpression(String weightExpression, Set<SortField> sortFields) {
+  public LongValuesSource fromExpression(String weightExpression, Set<SortField> sortFields) {
     Expression expression = null;
     try {
       expression = JavascriptCompiler.compile(weightExpression);
@@ -104,7 +104,7 @@ public class DocumentExpressionDictionaryFactory extends DictionaryFactory {
     for (SortField sortField : sortFields) {
       bindings.add(sortField);
     }
-    return expression.getValueSource(bindings);
+    return expression.getDoubleValuesSource(bindings).toLongValuesSource();
   }
   
   private SortField.Type getSortFieldType(SolrCore core, String sortFieldName) {