You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jp...@apache.org on 2015/02/20 17:36:48 UTC

svn commit: r1661156 - in /lucene/dev/trunk: lucene/ lucene/analysis/common/src/java/org/apache/lucene/collation/ lucene/analysis/common/src/test/org/apache/lucene/collation/ lucene/analysis/icu/src/java/org/apache/lucene/collation/ lucene/analysis/icu...

Author: jpountz
Date: Fri Feb 20 16:36:46 2015
New Revision: 1661156

URL: http://svn.apache.org/r1661156
Log:
LUCENE-6268: Replace FieldValueFilter and DocValuesRangeFilter with equivalent queries that support approximations.

Added:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocValuesRangeQuery.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldValueQuery.java   (with props)
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestDocValuesRangeQuery.java   (with props)
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFieldValueQuery.java   (with props)
Removed:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocTermOrdsRangeFilter.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocValuesRangeFilter.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldValueFilter.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestDocTermOrdsRangeFilter.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFieldCacheRangeFilter.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFieldValueFilter.java
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/collation/CollationDocValuesField.java
    lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/collation/TestCollationDocValuesField.java
    lucene/dev/trunk/lucene/analysis/icu/src/java/org/apache/lucene/collation/ICUCollationDocValuesField.java
    lucene/dev/trunk/lucene/analysis/icu/src/test/org/apache/lucene/collation/TestICUCollationDocValuesField.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocIdSetIterator.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TermRangeFilter.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestCachingWrapperFilter.java
    lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/SlowCollatedTermRangeFilter.java
    lucene/dev/trunk/solr/contrib/analysis-extras/src/java/org/apache/solr/schema/ICUCollationField.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/CollationField.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/CurrencyField.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/EnumField.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/FieldType.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/TrieField.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1661156&r1=1661155&r2=1661156&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Fri Feb 20 16:36:46 2015
@@ -131,6 +131,9 @@ API Changes
   positions are not available.
   (Ryan Ernst)
 
+* LUCENE-6268: Replace FieldValueFilter and DocValuesRangeFilter with equivalent
+  queries that support approximations. (Adrien Grand)
+
 Other
 
 * LUCENE-6248: Remove unused odd constants from StandardSyntaxParser.jj

Modified: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/collation/CollationDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/collation/CollationDocValuesField.java?rev=1661156&r1=1661155&r2=1661156&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/collation/CollationDocValuesField.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/collation/CollationDocValuesField.java Fri Feb 20 16:36:46 2015
@@ -21,7 +21,7 @@ import java.text.Collator;
 
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.SortedDocValuesField;
-import org.apache.lucene.search.DocValuesRangeFilter;
+import org.apache.lucene.search.DocValuesRangeQuery;
 import org.apache.lucene.util.BytesRef;
 
 /**
@@ -29,7 +29,7 @@ import org.apache.lucene.util.BytesRef;
  * <p>
  * This is more efficient that {@link CollationKeyAnalyzer} if the field 
  * only has one value: no uninversion is necessary to sort on the field, 
- * locale-sensitive range queries can still work via {@link DocValuesRangeFilter}, 
+ * locale-sensitive range queries can still work via {@link DocValuesRangeQuery}, 
  * and the underlying data structures built at index-time are likely more efficient 
  * and use less memory than FieldCache.
  */

Modified: lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/collation/TestCollationDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/collation/TestCollationDocValuesField.java?rev=1661156&r1=1661155&r2=1661156&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/collation/TestCollationDocValuesField.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/collation/TestCollationDocValuesField.java Fri Feb 20 16:36:46 2015
@@ -25,9 +25,10 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.ConstantScoreQuery;
-import org.apache.lucene.search.DocValuesRangeFilter;
+import org.apache.lucene.search.DocValuesRangeQuery;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.Query;
@@ -36,7 +37,6 @@ import org.apache.lucene.search.ScoreDoc
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.TopDocs;
-import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
@@ -111,7 +111,7 @@ public class TestCollationDocValuesField
         String end = TestUtil.randomSimpleString(random());
         BytesRef lowerVal = new BytesRef(collator.getCollationKey(start).toByteArray());
         BytesRef upperVal = new BytesRef(collator.getCollationKey(end).toByteArray());
-        Query query = new ConstantScoreQuery(DocValuesRangeFilter.newBytesRefRange("collated", lowerVal, upperVal, true, true));
+        Query query = DocValuesRangeQuery.newBytesRefRange("collated", lowerVal, upperVal, true, true);
         doTestRanges(is, start, end, query, collator);
       }
     } finally {

Modified: lucene/dev/trunk/lucene/analysis/icu/src/java/org/apache/lucene/collation/ICUCollationDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/icu/src/java/org/apache/lucene/collation/ICUCollationDocValuesField.java?rev=1661156&r1=1661155&r2=1661156&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/icu/src/java/org/apache/lucene/collation/ICUCollationDocValuesField.java (original)
+++ lucene/dev/trunk/lucene/analysis/icu/src/java/org/apache/lucene/collation/ICUCollationDocValuesField.java Fri Feb 20 16:36:46 2015
@@ -19,7 +19,7 @@ package org.apache.lucene.collation;
 
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.SortedDocValuesField;
-import org.apache.lucene.search.DocValuesRangeFilter;
+import org.apache.lucene.search.DocValuesRangeQuery;
 import org.apache.lucene.util.BytesRef;
 
 import com.ibm.icu.text.Collator;
@@ -30,7 +30,7 @@ import com.ibm.icu.text.RawCollationKey;
  * <p>
  * This is more efficient that {@link ICUCollationKeyAnalyzer} if the field 
  * only has one value: no uninversion is necessary to sort on the field, 
- * locale-sensitive range queries can still work via {@link DocValuesRangeFilter}, 
+ * locale-sensitive range queries can still work via {@link DocValuesRangeQuery}, 
  * and the underlying data structures built at index-time are likely more efficient 
  * and use less memory than FieldCache.
  */

Modified: lucene/dev/trunk/lucene/analysis/icu/src/test/org/apache/lucene/collation/TestICUCollationDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/icu/src/test/org/apache/lucene/collation/TestICUCollationDocValuesField.java?rev=1661156&r1=1661155&r2=1661156&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/icu/src/test/org/apache/lucene/collation/TestICUCollationDocValuesField.java (original)
+++ lucene/dev/trunk/lucene/analysis/icu/src/test/org/apache/lucene/collation/TestICUCollationDocValuesField.java Fri Feb 20 16:36:46 2015
@@ -22,9 +22,9 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.BooleanQuery;
-import org.apache.lucene.search.ConstantScoreQuery;
-import org.apache.lucene.search.DocValuesRangeFilter;
+import org.apache.lucene.search.DocValuesRangeQuery;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.Query;
@@ -33,7 +33,6 @@ import org.apache.lucene.search.ScoreDoc
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.TopDocs;
-import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
@@ -109,7 +108,7 @@ public class TestICUCollationDocValuesFi
       String end = TestUtil.randomSimpleString(random());
       BytesRef lowerVal = new BytesRef(collator.getCollationKey(start).toByteArray());
       BytesRef upperVal = new BytesRef(collator.getCollationKey(end).toByteArray());
-      Query query = new ConstantScoreQuery(DocValuesRangeFilter.newBytesRefRange("collated", lowerVal, upperVal, true, true));
+      Query query = DocValuesRangeQuery.newBytesRefRange("collated", lowerVal, upperVal, true, true);
       doTestRanges(is, start, end, query, collator);
     }
     

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocIdSetIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocIdSetIterator.java?rev=1661156&r1=1661155&r2=1661156&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocIdSetIterator.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocIdSetIterator.java Fri Feb 20 16:36:46 2015
@@ -58,7 +58,39 @@ public abstract class DocIdSetIterator {
       }
     };
   }
-  
+
+  /** A {@link DocIdSetIterator} that matches all documents up to
+   *  {@code maxDoc - 1}. */
+  public static final DocIdSetIterator all(int maxDoc) {
+    return new DocIdSetIterator() {
+      int doc = -1;
+
+      @Override
+      public int docID() {
+        return doc;
+      }
+
+      @Override
+      public int nextDoc() throws IOException {
+        return advance(doc + 1);
+      }
+
+      @Override
+      public int advance(int target) throws IOException {
+        doc = target;
+        if (doc >= maxDoc) {
+          doc = NO_MORE_DOCS;
+        }
+        return doc;
+      }
+
+      @Override
+      public long cost() {
+        return maxDoc;
+      }
+    };
+  }
+
   /**
    * When returned by {@link #nextDoc()}, {@link #advance(int)} and
    * {@link #docID()} it means there are no more docs in the iterator.

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocValuesRangeQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocValuesRangeQuery.java?rev=1661156&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocValuesRangeQuery.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocValuesRangeQuery.java Fri Feb 20 16:36:46 2015
@@ -0,0 +1,393 @@
+package org.apache.lucene.search;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+import java.util.Objects;
+
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.DocValuesType;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.SortedNumericDocValues;
+import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.Bits.MatchNoBits;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.ToStringUtils;
+
+/**
+ * A range query that works on top of the doc values APIs. Such queries are
+ * usually slow since they do not use an inverted index. However, in the
+ * dense case where most documents match this query, it <b>might</b> be as
+ * fast or faster than a regular {@link NumericRangeQuery}.
+ * @lucene.experimental
+ */
+public final class DocValuesRangeQuery extends Query {
+
+  /** Create a new numeric range query on a numeric doc-values field. The field
+   *  must has been indexed with either {@link DocValuesType#NUMERIC} or
+   *  {@link DocValuesType#SORTED_NUMERIC} doc values. */
+  public static Query newLongRange(String field, Long lowerVal, Long upperVal, boolean includeLower, boolean includeUpper) {
+    return new DocValuesRangeQuery(field, lowerVal, upperVal, includeLower, includeUpper);
+  }
+
+  /** Create a new numeric range query on a numeric doc-values field. The field
+   *  must has been indexed with {@link DocValuesType#SORTED} or
+   *  {@link DocValuesType#SORTED_SET} doc values. */
+  public static Query newBytesRefRange(String field, BytesRef lowerVal, BytesRef upperVal, boolean includeLower, boolean includeUpper) {
+    return new DocValuesRangeQuery(field, deepCopyOf(lowerVal), deepCopyOf(upperVal), includeLower, includeUpper);
+  }
+
+  private static BytesRef deepCopyOf(BytesRef b) {
+    if (b == null) {
+      return null;
+    } else {
+      return BytesRef.deepCopyOf(b);
+    }
+  }
+
+  private final String field;
+  private final Object lowerVal, upperVal;
+  private final boolean includeLower, includeUpper;
+
+  private DocValuesRangeQuery(String field, Object lowerVal, Object upperVal, boolean includeLower, boolean includeUpper) {
+    this.field = Objects.requireNonNull(field);
+    this.lowerVal = lowerVal;
+    this.upperVal = upperVal;
+    this.includeLower = includeLower;
+    this.includeUpper = includeUpper;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof DocValuesRangeQuery == false) {
+      return false;
+    }
+    final DocValuesRangeQuery that = (DocValuesRangeQuery) obj;
+    return field.equals(that.field)
+        && Objects.equals(lowerVal, that.lowerVal)
+        && Objects.equals(upperVal, that.upperVal)
+        && includeLower == that.includeLower
+        && includeUpper == that.includeUpper
+        && getBoost() == that.getBoost();
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(field, lowerVal, upperVal, includeLower, includeUpper, getBoost());
+  }
+
+  @Override
+  public String toString(String field) {
+    StringBuilder sb = new StringBuilder();
+    if (this.field.equals(field) == false) {
+      sb.append(this.field).append(':');
+    }
+    sb.append(includeLower ? '[' : '{');
+    sb.append(lowerVal == null ? "*" : lowerVal.toString());
+    sb.append(" TO ");
+    sb.append(upperVal == null ? "*" : upperVal.toString());
+    sb.append(includeUpper ? ']' : '}');
+    sb.append(ToStringUtils.boost(getBoost()));
+    return sb.toString();
+  }
+
+  @Override
+  public Query rewrite(IndexReader reader) throws IOException {
+    if (lowerVal == null && upperVal == null) {
+      final FieldValueQuery rewritten = new FieldValueQuery(field);
+      rewritten.setBoost(getBoost());
+      return rewritten;
+    }
+    return this;
+  }
+
+  @Override
+  public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
+    if (lowerVal == null && upperVal == null) {
+      throw new IllegalStateException("Both min and max values cannot be null, call rewrite first");
+    }
+    return new Weight(DocValuesRangeQuery.this) {
+
+      private float queryNorm;
+      private float queryWeight;
+
+      @Override
+      public float getValueForNormalization() throws IOException {
+        queryWeight = getBoost();
+        return queryWeight * queryWeight;
+      }
+
+      @Override
+      public void normalize(float norm, float topLevelBoost) {
+        queryNorm = norm * topLevelBoost;
+        queryWeight *= queryNorm;
+      }
+
+      @Override
+      public Explanation explain(LeafReaderContext context, int doc) throws IOException {
+        final Scorer s = scorer(context, context.reader().getLiveDocs());
+        final boolean exists = (s != null && s.advance(doc) == doc);
+
+        final ComplexExplanation result = new ComplexExplanation();
+        if (exists) {
+          result.setDescription(DocValuesRangeQuery.this.toString() + ", product of:");
+          result.setValue(queryWeight);
+          result.setMatch(Boolean.TRUE);
+          result.addDetail(new Explanation(getBoost(), "boost"));
+          result.addDetail(new Explanation(queryNorm, "queryNorm"));
+        } else {
+          result.setDescription(DocValuesRangeQuery.this.toString() + " doesn't match id " + doc);
+          result.setValue(0);
+          result.setMatch(Boolean.FALSE);
+        }
+        return result;
+      }
+
+      @Override
+      public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
+
+        final Bits docsWithField = context.reader().getDocsWithField(field);
+        if (docsWithField == null || docsWithField instanceof MatchNoBits) {
+          return null;
+        }
+
+        final DocIdSetIterator approximation = DocIdSetIterator.all(context.reader().maxDoc());
+        final TwoPhaseDocIdSetIterator twoPhaseRange;
+        if (lowerVal instanceof Long || upperVal instanceof Long) {
+
+          final SortedNumericDocValues values = DocValues.getSortedNumeric(context.reader(), field);
+
+          final long min;
+          if (lowerVal == null) {
+            min = Long.MIN_VALUE;
+          } else if (includeLower) {
+            min = (long) lowerVal;
+          } else {
+            min = 1 + (long) lowerVal;
+          }
+
+          final long max;
+          if (upperVal == null) {
+            max = Long.MAX_VALUE;
+          } else if (includeUpper) {
+            max = (long) upperVal;
+          } else {
+            max = -1 + (long) upperVal;
+          }
+
+          if (min > max) {
+            return null;
+          }
+
+          twoPhaseRange = new TwoPhaseNumericRange(values, min, max, approximation, acceptDocs);
+
+        } else if (lowerVal instanceof BytesRef || upperVal instanceof BytesRef) {
+
+          final SortedSetDocValues values = DocValues.getSortedSet(context.reader(), field);
+
+          final long minOrd;
+          if (lowerVal == null) {
+            minOrd = 0;
+          } else {
+            final long ord = values.lookupTerm((BytesRef) lowerVal);
+            if (ord < 0) {
+              minOrd = -1 - ord;
+            } else if (includeLower) {
+              minOrd = ord;
+            } else {
+              minOrd = ord + 1;
+            }
+          }
+
+          final long maxOrd;
+          if (upperVal == null) {
+            maxOrd = values.getValueCount() - 1;
+          } else {
+            final long ord = values.lookupTerm((BytesRef) upperVal);
+            if (ord < 0) {
+              maxOrd = -2 - ord;
+            } else if (includeUpper) {
+              maxOrd = ord;
+            } else {
+              maxOrd = ord - 1;
+            }
+          }
+
+          if (minOrd > maxOrd) {
+            return null;
+          }
+
+          twoPhaseRange = new TwoPhaseOrdRange(values, minOrd, maxOrd, approximation, acceptDocs);
+
+        } else {
+          throw new AssertionError();
+        }
+
+        return new RangeScorer(this, twoPhaseRange, queryWeight);
+      }
+
+    };
+  }
+
+  private static class TwoPhaseNumericRange extends TwoPhaseDocIdSetIterator {
+
+    private final DocIdSetIterator approximation;
+    private final SortedNumericDocValues values;
+    private final long min, max;
+    private final Bits acceptDocs;
+
+    TwoPhaseNumericRange(SortedNumericDocValues values, long min, long max, DocIdSetIterator approximation, Bits acceptDocs) {
+      this.values = values;
+      this.min = min;
+      this.max = max;
+      this.approximation = approximation;
+      this.acceptDocs = acceptDocs;
+    }
+
+    @Override
+    public DocIdSetIterator approximation() {
+      return approximation;
+    }
+
+    @Override
+    public boolean matches() throws IOException {
+      final int doc = approximation.docID();
+      if (acceptDocs == null || acceptDocs.get(doc)) {
+        values.setDocument(doc);
+        final int count = values.count();
+        for (int i = 0; i < count; ++i) {
+          final long value = values.valueAt(i);
+          if (value >= min && value <= max) {
+            return true;
+          }
+        }
+      }
+      return false;
+    }
+
+  }
+
+  private static class TwoPhaseOrdRange extends TwoPhaseDocIdSetIterator {
+
+    private final DocIdSetIterator approximation;
+    private final SortedSetDocValues values;
+    private final long minOrd, maxOrd;
+    private final Bits acceptDocs;
+
+    TwoPhaseOrdRange(SortedSetDocValues values, long minOrd, long maxOrd, DocIdSetIterator approximation, Bits acceptDocs) {
+      this.values = values;
+      this.minOrd = minOrd;
+      this.maxOrd = maxOrd;
+      this.approximation = approximation;
+      this.acceptDocs = acceptDocs;
+    }
+
+    @Override
+    public DocIdSetIterator approximation() {
+      return approximation;
+    }
+
+    @Override
+    public boolean matches() throws IOException {
+      final int doc = approximation.docID();
+      if (acceptDocs == null || acceptDocs.get(doc)) {
+        values.setDocument(doc);
+        for (long ord = values.nextOrd(); ord != SortedSetDocValues.NO_MORE_ORDS; ord = values.nextOrd()) {
+          if (ord >= minOrd && ord <= maxOrd) {
+            return true;
+          }
+        }
+      }
+      return false;
+    }
+
+  }
+
+  private static class RangeScorer extends Scorer {
+
+    private final TwoPhaseDocIdSetIterator twoPhaseRange;
+    private final DocIdSetIterator disi;
+    private final float score;
+
+    RangeScorer(Weight weight, TwoPhaseDocIdSetIterator twoPhaseRange, float score) {
+      super(weight);
+      this.twoPhaseRange = twoPhaseRange;
+      this.disi = TwoPhaseDocIdSetIterator.asDocIdSetIterator(twoPhaseRange);
+      this.score = score;
+    }
+
+    @Override
+    public TwoPhaseDocIdSetIterator asTwoPhaseIterator() {
+      return twoPhaseRange;
+    }
+
+    @Override
+    public float score() throws IOException {
+      return score;
+    }
+
+    @Override
+    public int freq() throws IOException {
+      return 1;
+    }
+
+    @Override
+    public int nextPosition() throws IOException {
+      return -1;
+    }
+
+    @Override
+    public int startOffset() throws IOException {
+      return -1;
+    }
+
+    @Override
+    public int endOffset() throws IOException {
+      return -1;
+    }
+
+    @Override
+    public BytesRef getPayload() throws IOException {
+      return null;
+    }
+
+    @Override
+    public int docID() {
+      return disi.docID();
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      return disi.nextDoc();
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      return disi.advance(target);
+    }
+
+    @Override
+    public long cost() {
+      return disi.cost();
+    }
+
+  }
+
+}

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldValueQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldValueQuery.java?rev=1661156&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldValueQuery.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldValueQuery.java Fri Feb 20 16:36:46 2015
@@ -0,0 +1,192 @@
+package org.apache.lucene.search;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+import java.util.Objects;
+
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.Bits.MatchNoBits;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.ToStringUtils;
+
+/**
+ * A {@link Query} that matches documents that have a value for a given field
+ * as reported by {@link LeafReader#getDocsWithField(String)}.
+ */
+public final class FieldValueQuery extends Query {
+
+  private final String field;
+
+  /** Create a query that will match that have a value for the given
+   *  {@code field}. */
+  public FieldValueQuery(String field) {
+    this.field = Objects.requireNonNull(field);
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof FieldValueQuery == false) {
+      return false;
+    }
+    final FieldValueQuery that = (FieldValueQuery) obj;
+    return field.equals(that.field) && getBoost() == that.getBoost();
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(getClass(), field, getBoost());
+  }
+
+  @Override
+  public String toString(String field) {
+    return "FieldValueQuery [field=" + this.field + "]" + ToStringUtils.boost(getBoost());
+  }
+
+  @Override
+  public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
+    return new Weight(this) {
+
+      private float queryNorm;
+      private float queryWeight;
+
+      @Override
+      public float getValueForNormalization() throws IOException {
+        queryWeight = getBoost();
+        return queryWeight * queryWeight;
+      }
+
+      @Override
+      public void normalize(float norm, float topLevelBoost) {
+        queryNorm = norm * topLevelBoost;
+        queryWeight *= queryNorm;
+      }
+
+      @Override
+      public Explanation explain(LeafReaderContext context, int doc) throws IOException {
+        final Scorer s = scorer(context, context.reader().getLiveDocs());
+        final boolean exists = (s != null && s.advance(doc) == doc);
+
+        final ComplexExplanation result = new ComplexExplanation();
+        if (exists) {
+          result.setDescription(FieldValueQuery.this.toString() + ", product of:");
+          result.setValue(queryWeight);
+          result.setMatch(Boolean.TRUE);
+          result.addDetail(new Explanation(getBoost(), "boost"));
+          result.addDetail(new Explanation(queryNorm, "queryNorm"));
+        } else {
+          result.setDescription(FieldValueQuery.this.toString() + " doesn't match id " + doc);
+          result.setValue(0);
+          result.setMatch(Boolean.FALSE);
+        }
+        return result;
+      }
+
+      @Override
+      public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
+        final Bits docsWithField = context.reader().getDocsWithField(field);
+        if (docsWithField == null || docsWithField instanceof MatchNoBits) {
+          return null;
+        }
+
+        final DocIdSetIterator approximation = DocIdSetIterator.all(context.reader().maxDoc());
+        final TwoPhaseDocIdSetIterator twoPhaseIterator = new TwoPhaseDocIdSetIterator() {
+
+          @Override
+          public boolean matches() throws IOException {
+            final int doc = approximation.docID();
+            if (acceptDocs != null && acceptDocs.get(doc) == false) {
+              return false;
+            }
+            if (docsWithField.get(doc) == false) {
+              return false;
+            }
+            return true;
+          }
+
+          @Override
+          public DocIdSetIterator approximation() {
+            return approximation;
+          }
+        };
+        final DocIdSetIterator disi = TwoPhaseDocIdSetIterator.asDocIdSetIterator(twoPhaseIterator);
+
+        return new Scorer(this) {
+
+          @Override
+          public TwoPhaseDocIdSetIterator asTwoPhaseIterator() {
+            return twoPhaseIterator;
+          }
+
+          @Override
+          public int nextDoc() throws IOException {
+            return disi.nextDoc();
+          }
+
+          @Override
+          public int docID() {
+            return disi.docID();
+          }
+
+          @Override
+          public long cost() {
+            return disi.cost();
+          }
+
+          @Override
+          public int advance(int target) throws IOException {
+            return disi.advance(target);
+          }
+
+          @Override
+          public int startOffset() throws IOException {
+            return -1;
+          }
+
+          @Override
+          public int nextPosition() throws IOException {
+            return -1;
+          }
+
+          @Override
+          public BytesRef getPayload() throws IOException {
+            return null;
+          }
+
+          @Override
+          public int freq() throws IOException {
+            return 1;
+          }
+
+          @Override
+          public int endOffset() throws IOException {
+            return -1;
+          }
+
+          @Override
+          public float score() throws IOException {
+            return queryWeight;
+          }
+        };
+      }
+    };
+  }
+
+}

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TermRangeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TermRangeFilter.java?rev=1661156&r1=1661155&r2=1661156&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TermRangeFilter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TermRangeFilter.java Fri Feb 20 16:36:46 2015
@@ -27,9 +27,6 @@ import org.apache.lucene.util.BytesRef;
  * supplied range according to {@link
  * Byte#compareTo(Byte)},  It is not intended
  * for numerical ranges; use {@link NumericRangeFilter} instead.
- *
- * <p>If you construct a large number of range filters with different ranges but on the 
- * same field, {@link DocValuesRangeFilter} may have significantly better performance. 
  * @since 2.9
  */
 public class TermRangeFilter extends MultiTermQueryWrapperFilter<TermRangeQuery> {

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestCachingWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestCachingWrapperFilter.java?rev=1661156&r1=1661155&r2=1661156&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestCachingWrapperFilter.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestCachingWrapperFilter.java Fri Feb 20 16:36:46 2015
@@ -268,7 +268,7 @@ public class TestCachingWrapperFilter ex
     // returns default empty docidset, always cacheable:
     assertDocIdSetCacheable(reader, NumericRangeFilter.newIntRange("test", Integer.valueOf(10000), Integer.valueOf(-10000), true, true), true);
     // is cacheable:
-    assertDocIdSetCacheable(reader, DocValuesRangeFilter.newIntRange("test", Integer.valueOf(10), Integer.valueOf(20), true, true), false);
+    assertDocIdSetCacheable(reader, NumericRangeFilter.newIntRange("test", 10, 20, true, true), false);
     // a fixedbitset filter is always cacheable
     assertDocIdSetCacheable(reader, new Filter() {
       @Override

Added: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestDocValuesRangeQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestDocValuesRangeQuery.java?rev=1661156&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestDocValuesRangeQuery.java (added)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestDocValuesRangeQuery.java Fri Feb 20 16:36:46 2015
@@ -0,0 +1,283 @@
+package org.apache.lucene.search;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.LongField;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.SortedDocValuesField;
+import org.apache.lucene.document.SortedNumericDocValuesField;
+import org.apache.lucene.document.SortedSetDocValuesField;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.NumericUtils;
+import org.apache.lucene.util.TestUtil;
+
+public class TestDocValuesRangeQuery extends LuceneTestCase {
+
+  public void testDuelNumericRangeQuery() throws IOException {
+    final int iters = atLeast(10);
+      for (int iter = 0; iter < iters; ++iter) {
+      Directory dir = newDirectory();
+      RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+      final int numDocs = atLeast(100);
+      for (int i = 0; i < numDocs; ++i) {
+        Document doc = new Document();
+        final int numValues = random().nextInt(2);
+        for (int j = 0; j < numValues; ++j) {
+          final long value = TestUtil.nextLong(random(), -100, 10000);
+          doc.add(new SortedNumericDocValuesField("dv", value));
+          doc.add(new LongField("idx", value, Store.NO));
+        }
+        iw.addDocument(doc);
+      }
+      if (random().nextBoolean()) {
+        iw.deleteDocuments(NumericRangeQuery.newLongRange("idx", 0L, 10L, true, true));
+      }
+      iw.commit();
+      final IndexReader reader = iw.getReader();
+      final IndexSearcher searcher = newSearcher(reader);
+      iw.close();
+
+      for (int i = 0; i < 100; ++i) {
+        final Long min = random().nextBoolean() ? null : TestUtil.nextLong(random(), -100, 1000);
+        final Long max = random().nextBoolean() ? null : TestUtil.nextLong(random(), -100, 1000);
+        final boolean minInclusive = random().nextBoolean();
+        final boolean maxInclusive = random().nextBoolean();
+        final Query q1 = NumericRangeQuery.newLongRange("idx", min, max, minInclusive, maxInclusive);
+        final Query q2 = DocValuesRangeQuery.newLongRange("dv", min, max, minInclusive, maxInclusive);
+        assertSameMatches(searcher, q1, q2, false);
+      }
+
+      reader.close();
+      dir.close();
+    }
+  }
+
+  private static BytesRef toSortableBytes(Long l) {
+    if (l == null) {
+      return null;
+    } else {
+      final BytesRefBuilder bytes = new BytesRefBuilder();
+      NumericUtils.longToPrefixCoded(l, 0, bytes);
+      return bytes.get();
+    }
+  }
+
+  public void testDuelNumericSorted() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    final int numDocs = atLeast(100);
+    for (int i = 0; i < numDocs; ++i) {
+      Document doc = new Document();
+      final int numValues = random().nextInt(3);
+      for (int j = 0; j < numValues; ++j) {
+        final long value = TestUtil.nextLong(random(), -100, 10000);
+        doc.add(new SortedNumericDocValuesField("dv1", value));
+        doc.add(new SortedSetDocValuesField("dv2", toSortableBytes(value)));
+      }
+      iw.addDocument(doc);
+    }
+    if (random().nextBoolean()) {
+      iw.deleteDocuments(DocValuesRangeQuery.newLongRange("dv1", 0L, 10L, true, true));
+    }
+    iw.commit();
+    final IndexReader reader = iw.getReader();
+    final IndexSearcher searcher = newSearcher(reader);
+    iw.close();
+
+    for (int i = 0; i < 100; ++i) {
+      final Long min = random().nextBoolean() ? null : TestUtil.nextLong(random(), -100, 1000);
+      final Long max = random().nextBoolean() ? null : TestUtil.nextLong(random(), -100, 1000);
+      final boolean minInclusive = random().nextBoolean();
+      final boolean maxInclusive = random().nextBoolean();
+      final Query q1 = DocValuesRangeQuery.newLongRange("dv1", min, max, minInclusive, maxInclusive);
+      final Query q2 = DocValuesRangeQuery.newBytesRefRange("dv2", toSortableBytes(min), toSortableBytes(max), minInclusive, maxInclusive);
+      assertSameMatches(searcher, q1, q2, true);
+    }
+
+    reader.close();
+    dir.close();
+  }
+
+  public void testScore() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    final int numDocs = atLeast(100);
+    for (int i = 0; i < numDocs; ++i) {
+      Document doc = new Document();
+      final int numValues = random().nextInt(3);
+      for (int j = 0; j < numValues; ++j) {
+        final long value = TestUtil.nextLong(random(), -100, 10000);
+        doc.add(new SortedNumericDocValuesField("dv1", value));
+        doc.add(new SortedSetDocValuesField("dv2", toSortableBytes(value)));
+      }
+      iw.addDocument(doc);
+    }
+    if (random().nextBoolean()) {
+      iw.deleteDocuments(DocValuesRangeQuery.newLongRange("dv1", 0L, 10L, true, true));
+    }
+    iw.commit();
+    final IndexReader reader = iw.getReader();
+    final IndexSearcher searcher = newSearcher(reader);
+    iw.close();
+
+    for (int i = 0; i < 100; ++i) {
+      final Long min = random().nextBoolean() ? null : TestUtil.nextLong(random(), -100, 1000);
+      final Long max = random().nextBoolean() ? null : TestUtil.nextLong(random(), -100, 1000);
+      final boolean minInclusive = random().nextBoolean();
+      final boolean maxInclusive = random().nextBoolean();
+
+      final float boost = random().nextFloat() * 10;
+
+      final Query q1 = DocValuesRangeQuery.newLongRange("dv1", min, max, minInclusive, maxInclusive);
+      q1.setBoost(boost);
+      final ConstantScoreQuery csq1 = new ConstantScoreQuery(DocValuesRangeQuery.newLongRange("dv1", min, max, minInclusive, maxInclusive));
+      csq1.setBoost(boost);
+      assertSameMatches(searcher, q1, csq1, true);
+
+      final Query q2 = DocValuesRangeQuery.newBytesRefRange("dv2", toSortableBytes(min), toSortableBytes(max), minInclusive, maxInclusive);
+      q2.setBoost(boost);
+      final ConstantScoreQuery csq2 = new ConstantScoreQuery(DocValuesRangeQuery.newBytesRefRange("dv2", toSortableBytes(min), toSortableBytes(max), minInclusive, maxInclusive));
+      csq2.setBoost(boost);
+      assertSameMatches(searcher, q2, csq2, true);
+    }
+
+    reader.close();
+    dir.close();
+  }
+
+  public void testApproximation() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    final int numDocs = atLeast(100);
+    for (int i = 0; i < numDocs; ++i) {
+      Document doc = new Document();
+      final int numValues = random().nextInt(3);
+      for (int j = 0; j < numValues; ++j) {
+        final long value = TestUtil.nextLong(random(), -100, 10000);
+        doc.add(new SortedNumericDocValuesField("dv1", value));
+        doc.add(new SortedSetDocValuesField("dv2", toSortableBytes(value)));
+        doc.add(new LongField("idx", value, Store.NO));
+        doc.add(new StringField("f", random().nextBoolean() ? "a" : "b", Store.NO));
+      }
+      iw.addDocument(doc);
+    }
+    if (random().nextBoolean()) {
+      iw.deleteDocuments(NumericRangeQuery.newLongRange("idx", 0L, 10L, true, true));
+    }
+    iw.commit();
+    final IndexReader reader = iw.getReader();
+    final IndexSearcher searcher = newSearcher(reader);
+    iw.close();
+
+    for (int i = 0; i < 100; ++i) {
+      final Long min = random().nextBoolean() ? null : TestUtil.nextLong(random(), -100, 1000);
+      final Long max = random().nextBoolean() ? null : TestUtil.nextLong(random(), -100, 1000);
+      final boolean minInclusive = random().nextBoolean();
+      final boolean maxInclusive = random().nextBoolean();
+
+      BooleanQuery ref = new BooleanQuery();
+      ref.add(NumericRangeQuery.newLongRange("idx", min, max, minInclusive, maxInclusive), Occur.FILTER);
+      ref.add(new TermQuery(new Term("f", "a")), Occur.MUST);
+
+      BooleanQuery bq1 = new BooleanQuery();
+      bq1.add(DocValuesRangeQuery.newLongRange("dv1", min, max, minInclusive, maxInclusive), Occur.FILTER);
+      bq1.add(new TermQuery(new Term("f", "a")), Occur.MUST);
+
+      assertSameMatches(searcher, ref, bq1, true);
+
+      BooleanQuery bq2 = new BooleanQuery();
+      bq2.add(DocValuesRangeQuery.newBytesRefRange("dv2", toSortableBytes(min), toSortableBytes(max), minInclusive, maxInclusive), Occur.FILTER);
+      bq2.add(new TermQuery(new Term("f", "a")), Occur.MUST);
+
+      assertSameMatches(searcher, ref, bq2, true);
+    }
+
+    reader.close();
+    dir.close();
+  }
+
+  private void assertSameMatches(IndexSearcher searcher, Query q1, Query q2, boolean scores) throws IOException {
+    final int maxDoc = searcher.getIndexReader().maxDoc();
+    final TopDocs td1 = searcher.search(q1, maxDoc, scores ? Sort.RELEVANCE : Sort.INDEXORDER);
+    final TopDocs td2 = searcher.search(q2, maxDoc, scores ? Sort.RELEVANCE : Sort.INDEXORDER);
+    assertEquals(td1.totalHits, td2.totalHits);
+    for (int i = 0; i < td1.scoreDocs.length; ++i) {
+      assertEquals(td1.scoreDocs[i].doc, td2.scoreDocs[i].doc);
+      if (scores) {
+        assertEquals(td1.scoreDocs[i].score, td2.scoreDocs[i].score, 10e-7);
+      }
+    }
+  }
+
+  public void testToString() {
+    assertEquals("f:[2 TO 5]", DocValuesRangeQuery.newLongRange("f", 2L, 5L, true, true).toString());
+    assertEquals("f:{2 TO 5]", DocValuesRangeQuery.newLongRange("f", 2L, 5L, false, true).toString());
+    assertEquals("f:{2 TO 5}", DocValuesRangeQuery.newLongRange("f", 2L, 5L, false, false).toString());
+    assertEquals("f:{* TO 5}", DocValuesRangeQuery.newLongRange("f", null, 5L, false, false).toString());
+    assertEquals("f:[2 TO *}", DocValuesRangeQuery.newLongRange("f", 2L, null, true, false).toString());
+
+    BytesRef min = new BytesRef("a");
+    BytesRef max = new BytesRef("b");
+    assertEquals("f:[[61] TO [62]]", DocValuesRangeQuery.newBytesRefRange("f", min, max, true, true).toString());
+    assertEquals("f:{[61] TO [62]]", DocValuesRangeQuery.newBytesRefRange("f", min, max, false, true).toString());
+    assertEquals("f:{[61] TO [62]}", DocValuesRangeQuery.newBytesRefRange("f", min, max, false, false).toString());
+    assertEquals("f:{* TO [62]}", DocValuesRangeQuery.newBytesRefRange("f", null, max, false, false).toString());
+    assertEquals("f:[[61] TO *}", DocValuesRangeQuery.newBytesRefRange("f", min, null, true, false).toString());
+  }
+
+  public void testDocValuesRangeSupportsApproximation() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    Document doc = new Document();
+    doc.add(new NumericDocValuesField("dv1", 5L));
+    doc.add(new SortedDocValuesField("dv2", toSortableBytes(42L)));
+    iw.addDocument(doc);
+    iw.commit();
+    final IndexReader reader = iw.getReader();
+    final LeafReaderContext ctx = reader.leaves().get(0);
+    final IndexSearcher searcher = newSearcher(reader);
+    iw.close();
+
+    Query q1 = DocValuesRangeQuery.newLongRange("dv1", 0L, 100L, random().nextBoolean(), random().nextBoolean());
+    Weight w = searcher.createNormalizedWeight(q1, random().nextBoolean());
+    Scorer s = w.scorer(ctx, null);
+    assertNotNull(s.asTwoPhaseIterator());
+
+    Query q2 = DocValuesRangeQuery.newBytesRefRange("dv2", toSortableBytes(0L), toSortableBytes(100L), random().nextBoolean(), random().nextBoolean());
+    w = searcher.createNormalizedWeight(q2, random().nextBoolean());
+    s = w.scorer(ctx, null);
+    assertNotNull(s.asTwoPhaseIterator());
+
+    reader.close();
+    dir.close();
+  }
+
+}

Added: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFieldValueQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFieldValueQuery.java?rev=1661156&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFieldValueQuery.java (added)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFieldValueQuery.java Fri Feb 20 16:36:46 2015
@@ -0,0 +1,216 @@
+package org.apache.lucene.search;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.SortedNumericDocValuesField;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+
+public class TestFieldValueQuery extends LuceneTestCase {
+
+  public void testRandom() throws IOException {
+    final int iters = atLeast(10);
+    for (int iter = 0; iter < iters; ++iter) {
+      Directory dir = newDirectory();
+      RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+      final int numDocs = atLeast(100);
+      for (int i = 0; i < numDocs; ++i) {
+        Document doc = new Document();
+        final boolean hasValue = random().nextBoolean();
+        if (hasValue) {
+          doc.add(new NumericDocValuesField("dv1", 1));
+          doc.add(new SortedNumericDocValuesField("dv2", 1));
+          doc.add(new SortedNumericDocValuesField("dv2", 2));
+          doc.add(new StringField("has_value", "yes", Store.NO));
+        }
+        doc.add(new StringField("f", random().nextBoolean() ? "yes" : "no", Store.NO));
+        iw.addDocument(doc);
+      }
+      if (random().nextBoolean()) {
+        iw.deleteDocuments(new TermQuery(new Term("f", "no")));
+      }
+      iw.commit();
+      final IndexReader reader = iw.getReader();
+      final IndexSearcher searcher = newSearcher(reader);
+      iw.close();
+
+      assertSameMatches(searcher, new TermQuery(new Term("has_value", "yes")), new FieldValueQuery("dv1"), false);
+      assertSameMatches(searcher, new TermQuery(new Term("has_value", "yes")), new FieldValueQuery("dv2"), false);
+
+      reader.close();
+      dir.close();
+    }
+  }
+
+  public void testApproximation() throws IOException {
+    final int iters = atLeast(10);
+    for (int iter = 0; iter < iters; ++iter) {
+      Directory dir = newDirectory();
+      RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+      final int numDocs = atLeast(100);
+      for (int i = 0; i < numDocs; ++i) {
+        Document doc = new Document();
+        final boolean hasValue = random().nextBoolean();
+        if (hasValue) {
+          doc.add(new NumericDocValuesField("dv1", 1));
+          doc.add(new SortedNumericDocValuesField("dv2", 1));
+          doc.add(new SortedNumericDocValuesField("dv2", 2));
+          doc.add(new StringField("has_value", "yes", Store.NO));
+        }
+        doc.add(new StringField("f", random().nextBoolean() ? "yes" : "no", Store.NO));
+        iw.addDocument(doc);
+      }
+      if (random().nextBoolean()) {
+        iw.deleteDocuments(new TermQuery(new Term("f", "no")));
+      }
+      iw.commit();
+      final IndexReader reader = iw.getReader();
+      final IndexSearcher searcher = newSearcher(reader);
+      iw.close();
+
+      BooleanQuery ref = new BooleanQuery();
+      ref.add(new TermQuery(new Term("f", "yes")), Occur.MUST);
+      ref.add(new TermQuery(new Term("has_value", "yes")), Occur.FILTER);
+
+      BooleanQuery bq1 = new BooleanQuery();
+      bq1.add(new TermQuery(new Term("f", "yes")), Occur.MUST);
+      bq1.add(new FieldValueQuery("dv1"), Occur.FILTER);
+      assertSameMatches(searcher, ref, bq1, true);
+
+      BooleanQuery bq2 = new BooleanQuery();
+      bq2.add(new TermQuery(new Term("f", "yes")), Occur.MUST);
+      bq2.add(new FieldValueQuery("dv2"), Occur.FILTER);
+      assertSameMatches(searcher, ref, bq2, true);
+
+      reader.close();
+      dir.close();
+    }
+  }
+
+  public void testScore() throws IOException {
+    final int iters = atLeast(10);
+    for (int iter = 0; iter < iters; ++iter) {
+      Directory dir = newDirectory();
+      RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+      final int numDocs = atLeast(100);
+      for (int i = 0; i < numDocs; ++i) {
+        Document doc = new Document();
+        final boolean hasValue = random().nextBoolean();
+        if (hasValue) {
+          doc.add(new NumericDocValuesField("dv1", 1));
+          doc.add(new SortedNumericDocValuesField("dv2", 1));
+          doc.add(new SortedNumericDocValuesField("dv2", 2));
+          doc.add(new StringField("has_value", "yes", Store.NO));
+        }
+        doc.add(new StringField("f", random().nextBoolean() ? "yes" : "no", Store.NO));
+        iw.addDocument(doc);
+      }
+      if (random().nextBoolean()) {
+        iw.deleteDocuments(new TermQuery(new Term("f", "no")));
+      }
+      iw.commit();
+      final IndexReader reader = iw.getReader();
+      final IndexSearcher searcher = newSearcher(reader);
+      iw.close();
+
+      final float boost = random().nextFloat() * 10;
+      final Query ref = new ConstantScoreQuery(new TermQuery(new Term("has_value", "yes")));
+      ref.setBoost(boost);
+
+      final Query q1 = new FieldValueQuery("dv1");
+      q1.setBoost(boost);
+      assertSameMatches(searcher, ref, q1, true);
+
+      final Query q2 = new FieldValueQuery("dv2");
+      q2.setBoost(boost);
+      assertSameMatches(searcher, ref, q2, true);
+
+      reader.close();
+      dir.close();
+    }
+  }
+
+  public void testMissingField() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    iw.addDocument(new Document());
+    iw.commit();
+    final IndexReader reader = iw.getReader();
+    final IndexSearcher searcher = newSearcher(reader);
+    iw.close();
+    assertEquals(0, searcher.search(new FieldValueQuery("f"), 1).totalHits);
+    reader.close();
+    dir.close();
+  }
+
+  public void testAllDocsHaveField() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    Document doc = new Document();
+    doc.add(new NumericDocValuesField("f", 1));
+    iw.addDocument(doc);
+    iw.commit();
+    final IndexReader reader = iw.getReader();
+    final IndexSearcher searcher = newSearcher(reader);
+    iw.close();
+    assertEquals(1, searcher.search(new FieldValueQuery("f"), 1).totalHits);
+    reader.close();
+    dir.close();
+  }
+
+  public void testFieldExistsButNoDocsHaveField() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    // 1st segment has the field, but 2nd one does not
+    Document doc = new Document();
+    doc.add(new NumericDocValuesField("f", 1));
+    iw.addDocument(doc);
+    iw.commit();
+    iw.addDocument(new Document());
+    iw.commit();
+    final IndexReader reader = iw.getReader();
+    final IndexSearcher searcher = newSearcher(reader);
+    iw.close();
+    assertEquals(1, searcher.search(new FieldValueQuery("f"), 1).totalHits);
+    reader.close();
+    dir.close();
+  }
+
+  private void assertSameMatches(IndexSearcher searcher, Query q1, Query q2, boolean scores) throws IOException {
+    final int maxDoc = searcher.getIndexReader().maxDoc();
+    final TopDocs td1 = searcher.search(q1, maxDoc, scores ? Sort.RELEVANCE : Sort.INDEXORDER);
+    final TopDocs td2 = searcher.search(q2, maxDoc, scores ? Sort.RELEVANCE : Sort.INDEXORDER);
+    assertEquals(td1.totalHits, td2.totalHits);
+    for (int i = 0; i < td1.scoreDocs.length; ++i) {
+      assertEquals(td1.scoreDocs[i].doc, td2.scoreDocs[i].doc);
+      if (scores) {
+        assertEquals(td1.scoreDocs[i].score, td2.scoreDocs[i].score, 10e-7);
+      }
+    }
+  }
+}

Modified: lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/SlowCollatedTermRangeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/SlowCollatedTermRangeFilter.java?rev=1661156&r1=1661155&r2=1661156&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/SlowCollatedTermRangeFilter.java (original)
+++ lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/SlowCollatedTermRangeFilter.java Fri Feb 20 16:36:46 2015
@@ -19,9 +19,10 @@ package org.apache.lucene.sandbox.querie
 
 import java.text.Collator;
 
+import org.apache.lucene.search.DocValuesRangeQuery;
 import org.apache.lucene.search.MultiTermQueryWrapperFilter;
 import org.apache.lucene.search.NumericRangeFilter; // javadoc
-import org.apache.lucene.search.DocValuesRangeFilter; // javadoc
+// javadoc
 
 /**
  * A Filter that restricts search results to a range of term
@@ -33,7 +34,7 @@ import org.apache.lucene.search.DocValue
  * for numerical ranges; use {@link NumericRangeFilter} instead.
  *
  * <p>If you construct a large number of range filters with different ranges but on the 
- * same field, {@link DocValuesRangeFilter} may have significantly better performance. 
+ * same field, {@link DocValuesRangeQuery} may have significantly better performance. 
  * @deprecated Index collation keys with CollationKeyAnalyzer or ICUCollationKeyAnalyzer instead.
  * This class will be removed in Lucene 5.0
  */

Modified: lucene/dev/trunk/solr/contrib/analysis-extras/src/java/org/apache/solr/schema/ICUCollationField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/analysis-extras/src/java/org/apache/solr/schema/ICUCollationField.java?rev=1661156&r1=1661155&r2=1661156&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/analysis-extras/src/java/org/apache/solr/schema/ICUCollationField.java (original)
+++ lucene/dev/trunk/solr/contrib/analysis-extras/src/java/org/apache/solr/schema/ICUCollationField.java Fri Feb 20 16:36:46 2015
@@ -28,20 +28,17 @@ import org.apache.commons.io.IOUtils;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
+import org.apache.lucene.analysis.util.ResourceLoader;
 import org.apache.lucene.collation.ICUCollationKeyAnalyzer;
 import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.document.SortedSetDocValuesField;
 import org.apache.lucene.index.StorableField;
-import org.apache.lucene.search.ConstantScoreQuery;
-import org.apache.lucene.search.DocTermOrdsRangeFilter;
-import org.apache.lucene.search.DocValuesRangeFilter;
+import org.apache.lucene.search.DocValuesRangeQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.TermRangeQuery;
 import org.apache.lucene.uninverting.UninvertingReader.Type;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.Version;
-import org.apache.lucene.analysis.util.ResourceLoader;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.response.TextResponseWriter;
@@ -277,11 +274,11 @@ public class ICUCollationField extends F
     BytesRef high = part2 == null ? null : getCollationKey(f, part2);
     if (!field.indexed() && field.hasDocValues()) {
       if (field.multiValued()) {
-          return new ConstantScoreQuery(DocTermOrdsRangeFilter.newBytesRefRange(
-              field.getName(), low, high, minInclusive, maxInclusive));
+          return DocValuesRangeQuery.newBytesRefRange(
+              field.getName(), low, high, minInclusive, maxInclusive);
         } else {
-          return new ConstantScoreQuery(DocValuesRangeFilter.newBytesRefRange(
-              field.getName(), low, high, minInclusive, maxInclusive));
+          return DocValuesRangeQuery.newBytesRefRange(
+              field.getName(), low, high, minInclusive, maxInclusive);
         } 
     } else {
       return new TermRangeQuery(field.getName(), low, high, minInclusive, maxInclusive);

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/CollationField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/CollationField.java?rev=1661156&r1=1661155&r2=1661156&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/CollationField.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/CollationField.java Fri Feb 20 16:36:46 2015
@@ -32,20 +32,17 @@ import org.apache.commons.io.IOUtils;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
+import org.apache.lucene.analysis.util.ResourceLoader;
 import org.apache.lucene.collation.CollationKeyAnalyzer;
 import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.document.SortedSetDocValuesField;
 import org.apache.lucene.index.StorableField;
-import org.apache.lucene.search.ConstantScoreQuery;
-import org.apache.lucene.search.DocTermOrdsRangeFilter;
-import org.apache.lucene.search.DocValuesRangeFilter;
+import org.apache.lucene.search.DocValuesRangeQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.TermRangeQuery;
 import org.apache.lucene.uninverting.UninvertingReader.Type;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.Version;
-import org.apache.lucene.analysis.util.ResourceLoader;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.response.TextResponseWriter;
@@ -248,13 +245,8 @@ public class CollationField extends Fiel
     BytesRef low = part1 == null ? null : getCollationKey(f, part1);
     BytesRef high = part2 == null ? null : getCollationKey(f, part2);
     if (!field.indexed() && field.hasDocValues()) {
-      if (field.multiValued()) {
-          return new ConstantScoreQuery(DocTermOrdsRangeFilter.newBytesRefRange(
-              field.getName(), low, high, minInclusive, maxInclusive));
-        } else {
-          return new ConstantScoreQuery(DocValuesRangeFilter.newBytesRefRange(
-              field.getName(), low, high, minInclusive, maxInclusive));
-        } 
+      return DocValuesRangeQuery.newBytesRefRange(
+          field.getName(), low, high, minInclusive, maxInclusive);
     } else {
       return new TermRangeQuery(field.getName(), low, high, minInclusive, maxInclusive);
     }

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/CurrencyField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/CurrencyField.java?rev=1661156&r1=1661155&r2=1661156&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/CurrencyField.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/CurrencyField.java Fri Feb 20 16:36:46 2015
@@ -16,19 +16,37 @@ package org.apache.solr.schema;
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Currency;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.xpath.XPath;
+import javax.xml.xpath.XPathConstants;
+import javax.xml.xpath.XPathExpressionException;
+import javax.xml.xpath.XPathFactory;
+
 import org.apache.lucene.analysis.util.ResourceLoader;
 import org.apache.lucene.analysis.util.ResourceLoaderAware;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.StorableField;
+import org.apache.lucene.queries.BooleanFilter;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.search.FieldValueQuery;
+import org.apache.lucene.search.Filter;
 import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryWrapperFilter;
 import org.apache.lucene.search.SortField;
-import org.apache.lucene.search.Filter;
-import org.apache.lucene.search.FieldValueFilter;
 import org.apache.lucene.uninverting.UninvertingReader.Type;
-import org.apache.lucene.queries.BooleanFilter;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.response.TextResponseWriter;
@@ -43,23 +61,6 @@ import org.w3c.dom.Node;
 import org.w3c.dom.NodeList;
 import org.xml.sax.SAXException;
 
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.parsers.ParserConfigurationException;
-import javax.xml.xpath.XPath;
-import javax.xml.xpath.XPathConstants;
-import javax.xml.xpath.XPathExpressionException;
-import javax.xml.xpath.XPathFactory;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.ArrayList;
-import java.util.Currency;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
 /**
  * Field type for support of monetary values.
  * <p>
@@ -328,7 +329,7 @@ public class CurrencyField extends Field
                           (p2 != null) ? p2.getCurrencyCode() : defaultCurrency;
 
     // ValueSourceRangeFilter doesn't check exists(), so we have to
-    final Filter docsWithValues = new FieldValueFilter(getAmountField(field).getName());
+    final Filter docsWithValues = new QueryWrapperFilter(new FieldValueQuery(getAmountField(field).getName()));
     final Filter vsRangeFilter = new ValueSourceRangeFilter
       (new RawCurrencyValueSource(field, currencyCode, parser),
        p1 == null ? null : p1.getAmount() + "", 

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/EnumField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/EnumField.java?rev=1661156&r1=1661155&r2=1661156&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/EnumField.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/EnumField.java Fri Feb 20 16:36:46 2015
@@ -237,9 +237,9 @@ public class EnumField extends Primitive
     Query query = null;
     final boolean matchOnly = field.hasDocValues() && !field.indexed();
     if (matchOnly) {
-      query = new ConstantScoreQuery(DocValuesRangeFilter.newIntRange(field.getName(),
-              min == null ? null : minValue,
-              max == null ? null : maxValue,
+      query = new ConstantScoreQuery(DocValuesRangeQuery.newLongRange(field.getName(),
+              min == null ? null : minValue.longValue(),
+              max == null ? null : maxValue.longValue(),
               minInclusive, maxInclusive));
     } else {
       query = NumericRangeQuery.newIntRange(field.getName(), DEFAULT_PRECISION_STEP,

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/FieldType.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/FieldType.java?rev=1661156&r1=1661155&r2=1661156&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/FieldType.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/FieldType.java Fri Feb 20 16:36:46 2015
@@ -17,6 +17,17 @@
 
 package org.apache.solr.schema;
 
+import static org.apache.lucene.analysis.util.AbstractAnalysisFactory.LUCENE_MATCH_VERSION_PARAM;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
@@ -29,10 +40,8 @@ import org.apache.lucene.index.IndexOpti
 import org.apache.lucene.index.StorableField;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.search.ConstantScoreQuery;
-import org.apache.lucene.search.DocTermOrdsRangeFilter;
 import org.apache.lucene.search.DocTermOrdsRewriteMethod;
-import org.apache.lucene.search.DocValuesRangeFilter;
+import org.apache.lucene.search.DocValuesRangeQuery;
 import org.apache.lucene.search.DocValuesRewriteMethod;
 import org.apache.lucene.search.MultiTermQuery;
 import org.apache.lucene.search.PrefixQuery;
@@ -46,7 +55,6 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.CharsRef;
 import org.apache.lucene.util.CharsRefBuilder;
-import org.apache.lucene.util.UnicodeUtil;
 import org.apache.solr.analysis.SolrAnalyzer;
 import org.apache.solr.analysis.TokenizerChain;
 import org.apache.solr.common.SolrException;
@@ -60,17 +68,6 @@ import org.apache.solr.search.Sorting;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import static org.apache.lucene.analysis.util.AbstractAnalysisFactory.LUCENE_MATCH_VERSION_PARAM;
-
 /**
  * Base class for all field types used by an index schema.
  *
@@ -692,19 +689,11 @@ public abstract class FieldType extends
   public Query getRangeQuery(QParser parser, SchemaField field, String part1, String part2, boolean minInclusive, boolean maxInclusive) {
     // TODO: change these all to use readableToIndexed/bytes instead (e.g. for unicode collation)
     if (field.hasDocValues() && !field.indexed()) {
-      if (field.multiValued()) {
-        return new ConstantScoreQuery(DocTermOrdsRangeFilter.newBytesRefRange(
-            field.getName(),
-            part1 == null ? null : new BytesRef(toInternal(part1)),
-            part2 == null ? null : new BytesRef(toInternal(part2)),
-            minInclusive, maxInclusive));
-      } else {
-        return new ConstantScoreQuery(DocValuesRangeFilter.newStringRange(
-            field.getName(), 
-            part1 == null ? null : toInternal(part1),
-            part2 == null ? null : toInternal(part2),
-            minInclusive, maxInclusive));
-      }
+      return DocValuesRangeQuery.newBytesRefRange(
+          field.getName(),
+          part1 == null ? null : new BytesRef(toInternal(part1)),
+          part2 == null ? null : new BytesRef(toInternal(part2)),
+          minInclusive, maxInclusive);
     } else {
       MultiTermQuery rangeQuery = TermRangeQuery.newStringRange(
             field.getName(),

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/TrieField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/TrieField.java?rev=1661156&r1=1661155&r2=1661156&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/TrieField.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/TrieField.java Fri Feb 20 16:36:46 2015
@@ -39,8 +39,7 @@ import org.apache.lucene.queries.functio
 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.ConstantScoreQuery;
-import org.apache.lucene.search.DocValuesRangeFilter;
+import org.apache.lucene.search.DocValuesRangeQuery;
 import org.apache.lucene.search.NumericRangeQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.SortField;
@@ -310,10 +309,10 @@ public class TrieField extends Primitive
     switch (type) {
       case INTEGER:
         if (matchOnly) {
-          query = new ConstantScoreQuery(DocValuesRangeFilter.newIntRange(field.getName(),
-                min == null ? null : Integer.parseInt(min),
-                max == null ? null : Integer.parseInt(max),
-                minInclusive, maxInclusive));
+          query = DocValuesRangeQuery.newLongRange(field.getName(),
+                min == null ? null : (long) Integer.parseInt(min),
+                max == null ? null : (long) Integer.parseInt(max),
+                minInclusive, maxInclusive);
         } else {
           query = NumericRangeQuery.newIntRange(field.getName(), ps,
                 min == null ? null : Integer.parseInt(min),
@@ -323,10 +322,10 @@ public class TrieField extends Primitive
         break;
       case FLOAT:
         if (matchOnly) {
-          query = new ConstantScoreQuery(DocValuesRangeFilter.newFloatRange(field.getName(),
-                min == null ? null : Float.parseFloat(min),
-                max == null ? null : Float.parseFloat(max),
-                minInclusive, maxInclusive));
+          query = DocValuesRangeQuery.newLongRange(field.getName(),
+                min == null ? null : (long) NumericUtils.floatToSortableInt(Float.parseFloat(min)),
+                max == null ? null : (long) NumericUtils.floatToSortableInt(Float.parseFloat(max)),
+                minInclusive, maxInclusive);
         } else {
           query = NumericRangeQuery.newFloatRange(field.getName(), ps,
                 min == null ? null : Float.parseFloat(min),
@@ -336,10 +335,10 @@ public class TrieField extends Primitive
         break;
       case LONG:
         if (matchOnly) {
-          query = new ConstantScoreQuery(DocValuesRangeFilter.newLongRange(field.getName(),
+          query = DocValuesRangeQuery.newLongRange(field.getName(),
                 min == null ? null : Long.parseLong(min),
                 max == null ? null : Long.parseLong(max),
-                minInclusive, maxInclusive));
+                minInclusive, maxInclusive);
         } else {
           query = NumericRangeQuery.newLongRange(field.getName(), ps,
                 min == null ? null : Long.parseLong(min),
@@ -349,10 +348,10 @@ public class TrieField extends Primitive
         break;
       case DOUBLE:
         if (matchOnly) {
-          query = new ConstantScoreQuery(DocValuesRangeFilter.newDoubleRange(field.getName(),
-                min == null ? null : Double.parseDouble(min),
-                max == null ? null : Double.parseDouble(max),
-                minInclusive, maxInclusive));
+          query = DocValuesRangeQuery.newLongRange(field.getName(),
+                min == null ? null : NumericUtils.doubleToSortableLong(Double.parseDouble(min)),
+                max == null ? null : NumericUtils.doubleToSortableLong(Double.parseDouble(max)),
+                minInclusive, maxInclusive);
         } else {
           query = NumericRangeQuery.newDoubleRange(field.getName(), ps,
                 min == null ? null : Double.parseDouble(min),
@@ -362,10 +361,10 @@ public class TrieField extends Primitive
         break;
       case DATE:
         if (matchOnly) {
-          query = new ConstantScoreQuery(DocValuesRangeFilter.newLongRange(field.getName(),
+          query = DocValuesRangeQuery.newLongRange(field.getName(),
                 min == null ? null : dateField.parseMath(null, min).getTime(),
                 max == null ? null : dateField.parseMath(null, max).getTime(),
-                minInclusive, maxInclusive));
+                minInclusive, maxInclusive);
         } else {
           query = NumericRangeQuery.newLongRange(field.getName(), ps,
                 min == null ? null : dateField.parseMath(null, min).getTime(),