You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by no...@apache.org on 2017/01/24 03:33:36 UTC

[40/50] [abbrv] lucene-solr:apiv2: LUCENE-7643: Move IndexOrDocValuesQuery to core.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71ca2a84/solr/core/src/java/org/apache/solr/schema/EnumField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/EnumField.java b/solr/core/src/java/org/apache/solr/schema/EnumField.java
index 967070c..5723206 100644
--- a/solr/core/src/java/org/apache/solr/schema/EnumField.java
+++ b/solr/core/src/java/org/apache/solr/schema/EnumField.java
@@ -43,7 +43,6 @@ import org.apache.lucene.legacy.LegacyNumericUtils;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.valuesource.EnumFieldSource;
 import org.apache.lucene.search.ConstantScoreQuery;
-import org.apache.lucene.search.DocValuesRangeQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.util.BytesRef;
@@ -253,10 +252,21 @@ public class EnumField extends PrimitiveFieldType {
     Query query = null;
     final boolean matchOnly = field.hasDocValues() && !field.indexed();
     if (matchOnly) {
-      query = new ConstantScoreQuery(DocValuesRangeQuery.newLongRange(field.getName(),
-              min == null ? null : minValue.longValue(),
-              max == null ? null : maxValue.longValue(),
-              minInclusive, maxInclusive));
+      long lowerValue = Long.MIN_VALUE;
+      long upperValue = Long.MAX_VALUE;
+      if (minValue != null) {
+        lowerValue = minValue.longValue();
+        if (minInclusive == false) {
+          ++lowerValue;
+        }
+      }
+      if (maxValue != null) {
+        upperValue = maxValue.longValue();
+        if (maxInclusive == false) {
+          --upperValue;
+        }
+      }
+      query = new ConstantScoreQuery(NumericDocValuesField.newRangeQuery(field.getName(), lowerValue, upperValue));
     } else {
       query = LegacyNumericRangeQuery.newIntRange(field.getName(), DEFAULT_PRECISION_STEP,
           min == null ? null : minValue,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71ca2a84/solr/core/src/java/org/apache/solr/schema/FieldType.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/FieldType.java b/solr/core/src/java/org/apache/solr/schema/FieldType.java
index 3922edc..54f882f 100644
--- a/solr/core/src/java/org/apache/solr/schema/FieldType.java
+++ b/solr/core/src/java/org/apache/solr/schema/FieldType.java
@@ -36,13 +36,13 @@ import org.apache.lucene.analysis.util.CharFilterFactory;
 import org.apache.lucene.analysis.util.TokenFilterFactory;
 import org.apache.lucene.analysis.util.TokenizerFactory;
 import org.apache.lucene.document.Field;
+import org.apache.lucene.document.SortedSetDocValuesField;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.legacy.LegacyNumericType;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
-import org.apache.lucene.search.DocValuesRangeQuery;
 import org.apache.lucene.search.DocValuesRewriteMethod;
 import org.apache.lucene.search.MultiTermQuery;
 import org.apache.lucene.search.PrefixQuery;
@@ -720,17 +720,17 @@ public abstract class FieldType extends FieldProperties {
    */
   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)
+    final BytesRef miValue = part1 == null ? null : new BytesRef(toInternal(part1));
+    final BytesRef maxValue = part2 == null ? null : new BytesRef(toInternal(part2));
     if (field.hasDocValues() && !field.indexed()) {
-      return DocValuesRangeQuery.newBytesRefRange(
-          field.getName(),
-          part1 == null ? null : new BytesRef(toInternal(part1)),
-          part2 == null ? null : new BytesRef(toInternal(part2)),
-          minInclusive, maxInclusive);
+      return SortedSetDocValuesField.newRangeQuery(
+            field.getName(),
+            miValue, maxValue,
+            minInclusive, maxInclusive);
     } else {
       SolrRangeQuery rangeQuery = new SolrRangeQuery(
             field.getName(),
-            part1 == null ? null : new BytesRef(toInternal(part1)),
-            part2 == null ? null : new BytesRef(toInternal(part2)),
+            miValue, maxValue,
             minInclusive, maxInclusive);
       return rangeQuery;
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71ca2a84/solr/core/src/java/org/apache/solr/schema/TrieField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/TrieField.java b/solr/core/src/java/org/apache/solr/schema/TrieField.java
index 0e8324c..57dbeff 100644
--- a/solr/core/src/java/org/apache/solr/schema/TrieField.java
+++ b/solr/core/src/java/org/apache/solr/schema/TrieField.java
@@ -43,7 +43,7 @@ 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.DocValuesRangeQuery;
+import org.apache.lucene.search.MatchNoDocsQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.SortedSetSelector;
@@ -376,9 +376,9 @@ public class TrieField extends PrimitiveFieldType {
     switch (type) {
       case INTEGER:
         if (matchOnly) {
-          query = DocValuesRangeQuery.newLongRange(field.getName(),
-                min == null ? null : (long) Integer.parseInt(min),
-                max == null ? null : (long) Integer.parseInt(max),
+          query = numericDocValuesRangeQuery(field.getName(),
+                min == null ? null : Integer.parseInt(min),
+                max == null ? null : Integer.parseInt(max),
                 minInclusive, maxInclusive);
         } else {
           query = LegacyNumericRangeQuery.newIntRange(field.getName(), ps,
@@ -399,7 +399,7 @@ public class TrieField extends PrimitiveFieldType {
         break;
       case LONG:
         if (matchOnly) {
-          query = DocValuesRangeQuery.newLongRange(field.getName(),
+          query = numericDocValuesRangeQuery(field.getName(),
                 min == null ? null : Long.parseLong(min),
                 max == null ? null : Long.parseLong(max),
                 minInclusive, maxInclusive);
@@ -422,7 +422,7 @@ public class TrieField extends PrimitiveFieldType {
         break;
       case DATE:
         if (matchOnly) {
-          query = DocValuesRangeQuery.newLongRange(field.getName(),
+          query = numericDocValuesRangeQuery(field.getName(),
                 min == null ? null : DateMathParser.parseMath(null, min).getTime(),
                 max == null ? null : DateMathParser.parseMath(null, max).getTime(),
                 minInclusive, maxInclusive);
@@ -440,6 +440,35 @@ public class TrieField extends PrimitiveFieldType {
     return query;
   }
 
+  private static Query numericDocValuesRangeQuery(
+      String field,
+      Number lowerValue, Number upperValue,
+      boolean lowerInclusive, boolean upperInclusive) {
+
+    long actualLowerValue = Long.MIN_VALUE;
+    if (lowerValue != null) {
+      actualLowerValue = lowerValue.longValue();
+      if (lowerInclusive == false) {
+        if (actualLowerValue == Long.MAX_VALUE) {
+          return new MatchNoDocsQuery();
+        }
+        ++actualLowerValue;
+      }
+    }
+
+    long actualUpperValue = Long.MAX_VALUE;
+    if (upperValue != null) {
+      actualUpperValue = upperValue.longValue();
+      if (upperInclusive == false) {
+        if (actualUpperValue == Long.MIN_VALUE) {
+          return new MatchNoDocsQuery();
+        }
+        --actualUpperValue;
+      }
+    }
+    return NumericDocValuesField.newRangeQuery(field, actualLowerValue, actualUpperValue);
+  }
+
   private static long FLOAT_NEGATIVE_INFINITY_BITS = (long)Float.floatToIntBits(Float.NEGATIVE_INFINITY);
   private static long DOUBLE_NEGATIVE_INFINITY_BITS = Double.doubleToLongBits(Double.NEGATIVE_INFINITY);
   private static long FLOAT_POSITIVE_INFINITY_BITS = (long)Float.floatToIntBits(Float.POSITIVE_INFINITY);
@@ -476,10 +505,10 @@ public class TrieField extends PrimitiveFieldType {
     } else { // If both max and min are negative (or -0d), then issue range query with max and min reversed
       if ((minVal == null || minVal.doubleValue() < 0d || minBits == minusZeroBits) &&
           (maxVal != null && (maxVal.doubleValue() < 0d || maxBits == minusZeroBits))) {
-        query = DocValuesRangeQuery.newLongRange
+        query = numericDocValuesRangeQuery
             (fieldName, maxBits, (min == null ? negativeInfinityBits : minBits), maxInclusive, minInclusive);
       } else { // If both max and min are positive, then issue range query
-        query = DocValuesRangeQuery.newLongRange
+        query = numericDocValuesRangeQuery
             (fieldName, minBits, (max == null ? positiveInfinityBits : maxBits), minInclusive, maxInclusive);
       }
     }