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

[15/50] lucene-solr:jira/solr-8593: LUCENE-5325: Add LongValuesSource and DoubleValuesSource in core

LUCENE-5325: Add LongValuesSource and DoubleValuesSource in core


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

Branch: refs/heads/jira/solr-8593
Commit: 3f24fd81c836982be96b9b60082b53177fffe504
Parents: 87b6c2c
Author: Alan Woodward <ro...@apache.org>
Authored: Wed Dec 28 20:10:47 2016 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Thu Dec 29 12:22:40 2016 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   5 +
 .../org/apache/lucene/search/DoubleValues.java  |  38 +++
 .../lucene/search/DoubleValuesSource.java       | 313 +++++++++++++++++++
 .../org/apache/lucene/search/LongValues.java    |  38 +++
 .../apache/lucene/search/LongValuesSource.java  | 217 +++++++++++++
 .../lucene/search/TestDoubleValuesSource.java   | 158 ++++++++++
 .../lucene/search/TestLongValuesSource.java     | 140 +++++++++
 7 files changed, 909 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3f24fd81/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 35314db..12b615d 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -89,6 +89,11 @@ New features
   http://blog.mikemccandless.com/2012/04/lucenes-tokenstreams-are-actually.html
   for details. (Mike McCandless)
 
+* LUCENE-5325: Added LongValuesSource and DoubleValuesSource, intended as
+  type-safe replacements for ValueSource in the queries module.  These
+  expose per-segment LongValues or DoubleValues iterators, similar to the
+  existing DocValues iterator API. (Alan Woodward, Adrien Grand)
+
 Bug Fixes
 
 * LUCENE-7547: JapaneseTokenizerFactory was failing to close the

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3f24fd81/lucene/core/src/java/org/apache/lucene/search/DoubleValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/DoubleValues.java b/lucene/core/src/java/org/apache/lucene/search/DoubleValues.java
new file mode 100644
index 0000000..4f12390
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/DoubleValues.java
@@ -0,0 +1,38 @@
+/*
+ * 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.search;
+
+import java.io.IOException;
+
+/**
+ * Per-segment, per-document double values, which can be calculated at search-time
+ */
+public abstract class DoubleValues {
+
+  /**
+   * Get the double value for the current document
+   */
+  public abstract double doubleValue() throws IOException;
+
+  /**
+   * Advance this instance to the given document id
+   * @return true if there is a value for this document
+   */
+  public abstract boolean advanceExact(int doc) throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3f24fd81/lucene/core/src/java/org/apache/lucene/search/DoubleValuesSource.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/DoubleValuesSource.java b/lucene/core/src/java/org/apache/lucene/search/DoubleValuesSource.java
new file mode 100644
index 0000000..4ac8fc1
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/DoubleValuesSource.java
@@ -0,0 +1,313 @@
+/*
+ * 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.search;
+
+import java.io.IOException;
+import java.util.Objects;
+import java.util.function.DoubleToLongFunction;
+import java.util.function.LongToDoubleFunction;
+
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+
+/**
+ * Base class for producing {@link DoubleValues}
+ *
+ * To obtain a {@link DoubleValues} object for a leaf reader, clients should
+ * call {@link #getValues(LeafReaderContext, DoubleValues)}.
+ *
+ * DoubleValuesSource objects for NumericDocValues fields can be obtained by calling
+ * {@link #fromDoubleField(String)}, {@link #fromFloatField(String)}, {@link #fromIntField(String)}
+ * or {@link #fromLongField(String)}, or from {@link #fromField(String, LongToDoubleFunction)} if
+ * special long-to-double encoding is required.
+ *
+ * Scores may be used as a source for value calculations by wrapping a {@link Scorer} using
+ * {@link #fromScorer(Scorer)} and passing the resulting DoubleValues to {@link #getValues(LeafReaderContext, DoubleValues)}.
+ * The scores can then be accessed using the {@link #SCORES} DoubleValuesSource.
+ */
+public abstract class DoubleValuesSource {
+
+  /**
+   * Returns a {@link DoubleValues} instance for the passed-in LeafReaderContext and scores
+   *
+   * If scores are not needed to calculate the values (ie {@link #needsScores() returns false}, callers
+   * may safely pass {@code null} for the {@code scores} parameter.
+   */
+  public abstract DoubleValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException;
+
+  /**
+   * Return true if document scores are needed to calculate values
+   */
+  public abstract boolean needsScores();
+
+  /**
+   * Create a sort field based on the value of this producer
+   * @param reverse true if the sort should be decreasing
+   */
+  public SortField getSortField(boolean reverse) {
+    return new DoubleValuesSortField(this, reverse);
+  }
+
+  /**
+   * Convert to a LongValuesSource by casting the double values to longs
+   */
+  public final LongValuesSource toLongValuesSource() {
+    return new LongValuesSource() {
+      @Override
+      public LongValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
+        DoubleValues in = DoubleValuesSource.this.getValues(ctx, scores);
+        return new LongValues() {
+          @Override
+          public long longValue() throws IOException {
+            return (long) in.doubleValue();
+          }
+
+          @Override
+          public boolean advanceExact(int doc) throws IOException {
+            return in.advanceExact(doc);
+          }
+        };
+      }
+
+      @Override
+      public boolean needsScores() {
+        return DoubleValuesSource.this.needsScores();
+      }
+    };
+  }
+
+  /**
+   * Creates a DoubleValuesSource that wraps a generic NumericDocValues field
+   *
+   * @param field the field to wrap, must have NumericDocValues
+   * @param decoder a function to convert the long-valued doc values to doubles
+   */
+  public static DoubleValuesSource fromField(String field, LongToDoubleFunction decoder) {
+    return new FieldValuesSource(field, decoder);
+  }
+
+  /**
+   * Creates a DoubleValuesSource that wraps a double-valued field
+   */
+  public static DoubleValuesSource fromDoubleField(String field) {
+    return fromField(field, Double::longBitsToDouble);
+  }
+
+  /**
+   * Creates a DoubleValuesSource that wraps a float-valued field
+   */
+  public static DoubleValuesSource fromFloatField(String field) {
+    return fromField(field, (v) -> (double)Float.intBitsToFloat((int)v));
+  }
+
+  /**
+   * Creates a DoubleValuesSource that wraps a long-valued field
+   */
+  public static DoubleValuesSource fromLongField(String field) {
+    return fromField(field, (v) -> (double) v);
+  }
+
+  /**
+   * Creates a DoubleValuesSource that wraps an int-valued field
+   */
+  public static DoubleValuesSource fromIntField(String field) {
+    return fromLongField(field);
+  }
+
+  /**
+   * A DoubleValuesSource that exposes a document's score
+   *
+   * If this source is used as part of a values calculation, then callers must not
+   * pass {@code null} as the {@link DoubleValues} parameter on {@link #getValues(LeafReaderContext, DoubleValues)}
+   */
+  public static final DoubleValuesSource SCORES = new DoubleValuesSource() {
+    @Override
+    public DoubleValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
+      assert scores != null;
+      return scores;
+    }
+
+    @Override
+    public boolean needsScores() {
+      return true;
+    }
+  };
+
+  /**
+   * Returns a DoubleValues instance that wraps scores returned by a Scorer
+   */
+  public static DoubleValues fromScorer(Scorer scorer) {
+    return new DoubleValues() {
+      @Override
+      public double doubleValue() throws IOException {
+        return scorer.score();
+      }
+
+      @Override
+      public boolean advanceExact(int doc) throws IOException {
+        assert scorer.docID() == doc;
+        return true;
+      }
+    };
+  }
+
+  private static class FieldValuesSource extends DoubleValuesSource {
+
+    final String field;
+    final LongToDoubleFunction decoder;
+
+    private FieldValuesSource(String field, LongToDoubleFunction decoder) {
+      this.field = field;
+      this.decoder = decoder;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+      FieldValuesSource that = (FieldValuesSource) o;
+      return Objects.equals(field, that.field) &&
+          Objects.equals(decoder, that.decoder);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(field, decoder);
+    }
+
+    @Override
+    public DoubleValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
+      final NumericDocValues values = DocValues.getNumeric(ctx.reader(), field);
+      return toDoubleValues(values, decoder::applyAsDouble);
+    }
+
+    @Override
+    public boolean needsScores() {
+      return false;
+    }
+  }
+
+  private static class DoubleValuesSortField extends SortField {
+
+    final DoubleValuesSource producer;
+
+    public DoubleValuesSortField(DoubleValuesSource producer, boolean reverse) {
+      super(producer.toString(), new DoubleValuesComparatorSource(producer), reverse);
+      this.producer = producer;
+    }
+
+    @Override
+    public boolean needsScores() {
+      return producer.needsScores();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder buffer = new StringBuilder("<");
+      buffer.append(getField()).append(">");
+      if (reverse)
+        buffer.append("!");
+      return buffer.toString();
+    }
+
+  }
+
+  private static class DoubleValuesHolder {
+    DoubleValues values;
+  }
+
+  private static class DoubleValuesComparatorSource extends FieldComparatorSource {
+    private final DoubleValuesSource producer;
+
+    public DoubleValuesComparatorSource(DoubleValuesSource producer) {
+      this.producer = producer;
+    }
+
+    @Override
+    public FieldComparator<Double> newComparator(String fieldname, int numHits,
+                                               int sortPos, boolean reversed) throws IOException {
+      return new FieldComparator.DoubleComparator(numHits, fieldname, 0.0){
+
+        LeafReaderContext ctx;
+        DoubleValuesHolder holder = new DoubleValuesHolder();
+
+        @Override
+        protected NumericDocValues getNumericDocValues(LeafReaderContext context, String field) throws IOException {
+          ctx = context;
+          return asNumericDocValues(holder, Double::doubleToLongBits);
+        }
+
+        @Override
+        public void setScorer(Scorer scorer) throws IOException {
+          holder.values = producer.getValues(ctx, fromScorer(scorer));
+        }
+      };
+    }
+  }
+
+  private static DoubleValues toDoubleValues(NumericDocValues in, LongToDoubleFunction map) {
+    return new DoubleValues() {
+      @Override
+      public double doubleValue() throws IOException {
+        return map.applyAsDouble(in.longValue());
+      }
+
+      @Override
+      public boolean advanceExact(int target) throws IOException {
+        return in.advanceExact(target);
+      }
+
+    };
+  }
+
+  private static NumericDocValues asNumericDocValues(DoubleValuesHolder in, DoubleToLongFunction converter) {
+    return new NumericDocValues() {
+      @Override
+      public long longValue() throws IOException {
+        return converter.applyAsLong(in.values.doubleValue());
+      }
+
+      @Override
+      public boolean advanceExact(int target) throws IOException {
+        return in.values.advanceExact(target);
+      }
+
+      @Override
+      public int docID() {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public int nextDoc() throws IOException {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public int advance(int target) throws IOException {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public long cost() {
+        throw new UnsupportedOperationException();
+      }
+    };
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3f24fd81/lucene/core/src/java/org/apache/lucene/search/LongValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/LongValues.java b/lucene/core/src/java/org/apache/lucene/search/LongValues.java
new file mode 100644
index 0000000..54d3189
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/LongValues.java
@@ -0,0 +1,38 @@
+/*
+ * 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.search;
+
+import java.io.IOException;
+
+/**
+ * Per-segment, per-document long values, which can be calculated at search-time
+ */
+public abstract class LongValues {
+
+  /**
+   * Get the long value for the current document
+   */
+  public abstract long longValue() throws IOException;
+
+  /**
+   * Advance this instance to the given document id
+   * @return true if there is a value for this document
+   */
+  public abstract boolean advanceExact(int doc) throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3f24fd81/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
new file mode 100644
index 0000000..9d00355
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/LongValuesSource.java
@@ -0,0 +1,217 @@
+/*
+ * 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.search;
+
+import java.io.IOException;
+import java.util.Objects;
+
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+
+/**
+ * Base class for producing {@link LongValues}
+ *
+ * To obtain a {@link LongValues} object for a leaf reader, clients should
+ * call {@link #getValues(LeafReaderContext, DoubleValues)}.
+ *
+ * LongValuesSource objects for long and int-valued NumericDocValues fields can
+ * be obtained by calling {@link #fromLongField(String)} and {@link #fromIntField(String)}.
+ *
+ * To obtain a LongValuesSource from a float or double-valued NumericDocValues field,
+ * use {@link DoubleValuesSource#fromFloatField(String)} or {@link DoubleValuesSource#fromDoubleField(String)}
+ * and then call {@link DoubleValuesSource#toLongValuesSource()}.
+ */
+public abstract class LongValuesSource {
+
+  /**
+   * Returns a {@link LongValues} instance for the passed-in LeafReaderContext and scores
+   *
+   * If scores are not needed to calculate the values (ie {@link #needsScores() returns false}, callers
+   * may safely pass {@code null} for the {@code scores} parameter.
+   */
+  public abstract LongValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException;
+
+  /**
+   * Return true if document scores are needed to calculate values
+   */
+  public abstract boolean needsScores();
+
+  /**
+   * Create a sort field based on the value of this producer
+   * @param reverse true if the sort should be decreasing
+   */
+  public SortField getSortField(boolean reverse) {
+    return new LongValuesSortField(this, reverse);
+  }
+
+  /**
+   * Creates a LongValuesSource that wraps a long-valued field
+   */
+  public static LongValuesSource fromLongField(String field) {
+    return new FieldValuesSource(field);
+  }
+
+  /**
+   * Creates a LongValuesSource that wraps an int-valued field
+   */
+  public static LongValuesSource fromIntField(String field) {
+    return fromLongField(field);
+  }
+
+  private static class FieldValuesSource extends LongValuesSource {
+
+    final String field;
+
+    private FieldValuesSource(String field) {
+      this.field = field;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+      FieldValuesSource that = (FieldValuesSource) o;
+      return Objects.equals(field, that.field);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(field);
+    }
+
+    @Override
+    public LongValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
+      final NumericDocValues values = DocValues.getNumeric(ctx.reader(), field);
+      return toLongValues(values);
+    }
+
+    @Override
+    public boolean needsScores() {
+      return false;
+    }
+  }
+
+  private static class LongValuesSortField extends SortField {
+
+    final LongValuesSource producer;
+
+    public LongValuesSortField(LongValuesSource producer, boolean reverse) {
+      super(producer.toString(), new LongValuesComparatorSource(producer), reverse);
+      this.producer = producer;
+    }
+
+    @Override
+    public boolean needsScores() {
+      return producer.needsScores();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder buffer = new StringBuilder("<");
+      buffer.append(getField()).append(">");
+      if (reverse)
+        buffer.append("!");
+      return buffer.toString();
+    }
+
+  }
+
+  private static class LongValuesHolder {
+    LongValues values;
+  }
+
+  private static class LongValuesComparatorSource extends FieldComparatorSource {
+    private final LongValuesSource producer;
+
+    public LongValuesComparatorSource(LongValuesSource producer) {
+      this.producer = producer;
+    }
+
+    @Override
+    public FieldComparator<Long> newComparator(String fieldname, int numHits,
+                                                 int sortPos, boolean reversed) throws IOException {
+      return new FieldComparator.LongComparator(numHits, fieldname, 0L){
+
+        LeafReaderContext ctx;
+        LongValuesHolder holder = new LongValuesHolder();
+
+        @Override
+        protected NumericDocValues getNumericDocValues(LeafReaderContext context, String field) throws IOException {
+          ctx = context;
+          return asNumericDocValues(holder);
+        }
+
+        @Override
+        public void setScorer(Scorer scorer) throws IOException {
+          holder.values = producer.getValues(ctx, DoubleValuesSource.fromScorer(scorer));
+        }
+      };
+    }
+  }
+
+  private static LongValues toLongValues(NumericDocValues in) {
+    return new LongValues() {
+      @Override
+      public long longValue() throws IOException {
+        return in.longValue();
+      }
+
+      @Override
+      public boolean advanceExact(int target) throws IOException {
+        return in.advanceExact(target);
+      }
+
+    };
+  }
+
+  private static NumericDocValues asNumericDocValues(LongValuesHolder in) {
+    return new NumericDocValues() {
+      @Override
+      public long longValue() throws IOException {
+        return in.values.longValue();
+      }
+
+      @Override
+      public boolean advanceExact(int target) throws IOException {
+        return in.values.advanceExact(target);
+      }
+
+      @Override
+      public int docID() {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public int nextDoc() throws IOException {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public int advance(int target) throws IOException {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public long cost() {
+        throw new UnsupportedOperationException();
+      }
+    };
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3f24fd81/lucene/core/src/test/org/apache/lucene/search/TestDoubleValuesSource.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestDoubleValuesSource.java b/lucene/core/src/test/org/apache/lucene/search/TestDoubleValuesSource.java
new file mode 100644
index 0000000..3860963
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/search/TestDoubleValuesSource.java
@@ -0,0 +1,158 @@
+/*
+ * 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.search;
+
+import java.util.Arrays;
+import java.util.Collections;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.DoubleDocValuesField;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FloatDocValuesField;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.English;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+public class TestDoubleValuesSource extends LuceneTestCase {
+
+  private Directory dir;
+  private IndexReader reader;
+  private IndexSearcher searcher;
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    int numDocs = TestUtil.nextInt(random(), 2049, 4000);
+    for (int i = 0; i < numDocs; i++) {
+      Document document = new Document();
+      document.add(newTextField("english", English.intToEnglish(i), Field.Store.NO));
+      document.add(newTextField("oddeven", (i % 2 == 0) ? "even" : "odd", Field.Store.NO));
+      document.add(new NumericDocValuesField("int", random().nextInt()));
+      document.add(new NumericDocValuesField("long", random().nextLong()));
+      document.add(new FloatDocValuesField("float", random().nextFloat()));
+      document.add(new DoubleDocValuesField("double", random().nextDouble()));
+      iw.addDocument(document);
+    }
+    reader = iw.getReader();
+    iw.close();
+    searcher = newSearcher(reader);
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    reader.close();
+    dir.close();
+    super.tearDown();
+  }
+
+  public void testSimpleFieldEquivalences() throws Exception {
+    checkSorts(new MatchAllDocsQuery(), new Sort(new SortField("int", SortField.Type.INT, random().nextBoolean())));
+    checkSorts(new MatchAllDocsQuery(), new Sort(new SortField("long", SortField.Type.LONG, random().nextBoolean())));
+    checkSorts(new MatchAllDocsQuery(), new Sort(new SortField("float", SortField.Type.FLOAT, random().nextBoolean())));
+    checkSorts(new MatchAllDocsQuery(), new Sort(new SortField("double", SortField.Type.DOUBLE, random().nextBoolean())));
+  }
+
+  public void testHashCodeAndEquals() {
+    DoubleValuesSource vs1 = DoubleValuesSource.fromDoubleField("double");
+    DoubleValuesSource vs2 = DoubleValuesSource.fromDoubleField("double");
+    assertEquals(vs1, vs2);
+    assertEquals(vs1.hashCode(), vs2.hashCode());
+    DoubleValuesSource v3 = DoubleValuesSource.fromLongField("long");
+    assertFalse(vs1.equals(v3));
+  }
+
+  public void testSimpleFieldSortables() throws Exception {
+    int n = atLeast(4);
+    for (int i = 0; i < n; i++) {
+      Sort sort = randomSort();
+      checkSorts(new MatchAllDocsQuery(), sort);
+      checkSorts(new TermQuery(new Term("english", "one")), sort);
+    }
+  }
+
+  Sort randomSort() throws Exception {
+    boolean reversed = random().nextBoolean();
+    SortField fields[] = new SortField[] {
+        new SortField("int", SortField.Type.INT, reversed),
+        new SortField("long", SortField.Type.LONG, reversed),
+        new SortField("float", SortField.Type.FLOAT, reversed),
+        new SortField("double", SortField.Type.DOUBLE, reversed),
+        new SortField("score", SortField.Type.SCORE)
+    };
+    Collections.shuffle(Arrays.asList(fields), random());
+    int numSorts = TestUtil.nextInt(random(), 1, fields.length);
+    return new Sort(Arrays.copyOfRange(fields, 0, numSorts));
+  }
+
+  // Take a Sort, and replace any field sorts with Sortables
+  Sort convertSortToSortable(Sort sort) {
+    SortField original[] = sort.getSort();
+    SortField mutated[] = new SortField[original.length];
+    for (int i = 0; i < mutated.length; i++) {
+      if (random().nextInt(3) > 0) {
+        SortField s = original[i];
+        boolean reverse = s.getType() == SortField.Type.SCORE || s.getReverse();
+        switch (s.getType()) {
+          case INT:
+            mutated[i] = DoubleValuesSource.fromIntField(s.getField()).getSortField(reverse);
+            break;
+          case LONG:
+            mutated[i] = DoubleValuesSource.fromLongField(s.getField()).getSortField(reverse);
+            break;
+          case FLOAT:
+            mutated[i] = DoubleValuesSource.fromFloatField(s.getField()).getSortField(reverse);
+            break;
+          case DOUBLE:
+            mutated[i] = DoubleValuesSource.fromDoubleField(s.getField()).getSortField(reverse);
+            break;
+          case SCORE:
+            mutated[i] = DoubleValuesSource.SCORES.getSortField(reverse);
+            break;
+          default:
+            mutated[i] = original[i];
+        }
+      } else {
+        mutated[i] = original[i];
+      }
+    }
+
+    return new Sort(mutated);
+  }
+
+  void checkSorts(Query query, Sort sort) throws Exception {
+    int size = TestUtil.nextInt(random(), 1, searcher.getIndexReader().maxDoc() / 5);
+    TopDocs expected = searcher.search(query, size, sort, random().nextBoolean(), random().nextBoolean());
+    Sort mutatedSort = convertSortToSortable(sort);
+    TopDocs actual = searcher.search(query, size, mutatedSort, random().nextBoolean(), random().nextBoolean());
+
+    CheckHits.checkEqual(query, expected.scoreDocs, actual.scoreDocs);
+
+    if (size < actual.totalHits) {
+      expected = searcher.searchAfter(expected.scoreDocs[size-1], query, size, sort);
+      actual = searcher.searchAfter(actual.scoreDocs[size-1], query, size, mutatedSort);
+      CheckHits.checkEqual(query, expected.scoreDocs, actual.scoreDocs);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3f24fd81/lucene/core/src/test/org/apache/lucene/search/TestLongValuesSource.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestLongValuesSource.java b/lucene/core/src/test/org/apache/lucene/search/TestLongValuesSource.java
new file mode 100644
index 0000000..9148ad5
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/search/TestLongValuesSource.java
@@ -0,0 +1,140 @@
+/*
+ * 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.search;
+
+import java.util.Arrays;
+import java.util.Collections;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.English;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+public class TestLongValuesSource extends LuceneTestCase {
+
+  private Directory dir;
+  private IndexReader reader;
+  private IndexSearcher searcher;
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    int numDocs = TestUtil.nextInt(random(), 2049, 4000);
+    for (int i = 0; i < numDocs; i++) {
+      Document document = new Document();
+      document.add(newTextField("english", English.intToEnglish(i), Field.Store.NO));
+      document.add(newTextField("oddeven", (i % 2 == 0) ? "even" : "odd", Field.Store.NO));
+      document.add(new NumericDocValuesField("int", random().nextInt()));
+      document.add(new NumericDocValuesField("long", random().nextLong()));
+      iw.addDocument(document);
+    }
+    reader = iw.getReader();
+    iw.close();
+    searcher = newSearcher(reader);
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    reader.close();
+    dir.close();
+    super.tearDown();
+  }
+
+  public void testSimpleFieldEquivalences() throws Exception {
+    checkSorts(new MatchAllDocsQuery(), new Sort(new SortField("int", SortField.Type.INT, random().nextBoolean())));
+    checkSorts(new MatchAllDocsQuery(), new Sort(new SortField("long", SortField.Type.LONG, random().nextBoolean())));
+  }
+
+  public void testHashCodeAndEquals() {
+    LongValuesSource vs1 = LongValuesSource.fromLongField("long");
+    LongValuesSource vs2 = LongValuesSource.fromLongField("long");
+    assertEquals(vs1, vs2);
+    assertEquals(vs1.hashCode(), vs2.hashCode());
+    LongValuesSource v3 = LongValuesSource.fromLongField("int");
+    assertFalse(vs1.equals(v3));
+  }
+
+  public void testSimpleFieldSortables() throws Exception {
+    int n = atLeast(4);
+    for (int i = 0; i < n; i++) {
+      Sort sort = randomSort();
+      checkSorts(new MatchAllDocsQuery(), sort);
+      checkSorts(new TermQuery(new Term("english", "one")), sort);
+    }
+  }
+
+  Sort randomSort() throws Exception {
+    boolean reversed = random().nextBoolean();
+    SortField fields[] = new SortField[] {
+        new SortField("int", SortField.Type.INT, reversed),
+        new SortField("long", SortField.Type.LONG, reversed)
+    };
+    Collections.shuffle(Arrays.asList(fields), random());
+    int numSorts = TestUtil.nextInt(random(), 1, fields.length);
+    return new Sort(Arrays.copyOfRange(fields, 0, numSorts));
+  }
+
+  // Take a Sort, and replace any field sorts with Sortables
+  Sort convertSortToSortable(Sort sort) {
+    SortField original[] = sort.getSort();
+    SortField mutated[] = new SortField[original.length];
+    for (int i = 0; i < mutated.length; i++) {
+      if (random().nextInt(3) > 0) {
+        SortField s = original[i];
+        boolean reverse = s.getType() == SortField.Type.SCORE || s.getReverse();
+        switch (s.getType()) {
+          case INT:
+            mutated[i] = LongValuesSource.fromIntField(s.getField()).getSortField(reverse);
+            break;
+          case LONG:
+            mutated[i] = LongValuesSource.fromLongField(s.getField()).getSortField(reverse);
+            break;
+          default:
+            mutated[i] = original[i];
+        }
+      } else {
+        mutated[i] = original[i];
+      }
+    }
+
+    return new Sort(mutated);
+  }
+
+  void checkSorts(Query query, Sort sort) throws Exception {
+    int size = TestUtil.nextInt(random(), 1, searcher.getIndexReader().maxDoc() / 5);
+    Sort mutatedSort = convertSortToSortable(sort);
+    TopDocs actual = searcher.search(query, size, mutatedSort, random().nextBoolean(), random().nextBoolean());
+    TopDocs expected = searcher.search(query, size, sort, random().nextBoolean(), random().nextBoolean());
+
+    CheckHits.checkEqual(query, expected.scoreDocs, actual.scoreDocs);
+
+    if (size < actual.totalHits) {
+      expected = searcher.searchAfter(expected.scoreDocs[size-1], query, size, sort);
+      actual = searcher.searchAfter(actual.scoreDocs[size-1], query, size, mutatedSort);
+      CheckHits.checkEqual(query, expected.scoreDocs, actual.scoreDocs);
+    }
+  }
+}