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 2016/12/21 19:45:16 UTC

[1/3] lucene-solr:master: LUCENE-7600: Simplify DocIdMerger.

Repository: lucene-solr
Updated Branches:
  refs/heads/master 7e03427fa -> 0d3c64ab0


LUCENE-7600: Simplify DocIdMerger.


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

Branch: refs/heads/master
Commit: 0d3c64ab099a1ddd168971e05e6199894fe5dfe7
Parents: 18d53a4
Author: Adrien Grand <jp...@gmail.com>
Authored: Wed Dec 21 19:34:19 2016 +0100
Committer: Adrien Grand <jp...@gmail.com>
Committed: Wed Dec 21 20:44:56 2016 +0100

----------------------------------------------------------------------
 .../apache/lucene/codecs/DocValuesConsumer.java |  10 +-
 .../org/apache/lucene/codecs/NormsConsumer.java |   2 +-
 .../lucene/codecs/StoredFieldsWriter.java       |   2 +-
 .../apache/lucene/codecs/TermVectorsWriter.java |   2 +-
 .../CompressingStoredFieldsWriter.java          |   2 +-
 .../org/apache/lucene/index/DocIDMerger.java    | 213 ++++++++++---------
 .../lucene/index/MappingMultiPostingsEnum.java  |   2 +-
 .../apache/lucene/index/TestDocIDMerger.java    |   4 +-
 8 files changed, 130 insertions(+), 107 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0d3c64ab/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
index ba2f2aa..3d06b51 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
@@ -198,7 +198,7 @@ public abstract class DocValuesConsumer implements Closeable {
                           }
                         }
 
-                        final DocIDMerger<NumericDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.needsIndexSort);
+                        final DocIDMerger<NumericDocValuesSub> docIDMerger = DocIDMerger.of(subs, mergeState.needsIndexSort);
 
                         final long finalCost = cost;
                         
@@ -296,7 +296,7 @@ public abstract class DocValuesConsumer implements Closeable {
                          }
                        }
 
-                       final DocIDMerger<BinaryDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.needsIndexSort);
+                       final DocIDMerger<BinaryDocValuesSub> docIDMerger = DocIDMerger.of(subs, mergeState.needsIndexSort);
                        final long finalCost = cost;
                        
                        return new BinaryDocValues() {
@@ -397,7 +397,7 @@ public abstract class DocValuesConsumer implements Closeable {
 
                               final long finalCost = cost;
 
-                              final DocIDMerger<SortedNumericDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.needsIndexSort);
+                              final DocIDMerger<SortedNumericDocValuesSub> docIDMerger = DocIDMerger.of(subs, mergeState.needsIndexSort);
 
                               return new SortedNumericDocValues() {
 
@@ -555,7 +555,7 @@ public abstract class DocValuesConsumer implements Closeable {
 
                        final long finalCost = cost;
 
-                       final DocIDMerger<SortedDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.needsIndexSort);
+                       final DocIDMerger<SortedDocValuesSub> docIDMerger = DocIDMerger.of(subs, mergeState.needsIndexSort);
                        
                        return new SortedDocValues() {
                          private int docID = -1;
@@ -721,7 +721,7 @@ public abstract class DocValuesConsumer implements Closeable {
                             subs.add(new SortedSetDocValuesSub(mergeState.docMaps[i], values, map.getGlobalOrds(i)));
                           }
             
-                          final DocIDMerger<SortedSetDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.needsIndexSort);
+                          final DocIDMerger<SortedSetDocValuesSub> docIDMerger = DocIDMerger.of(subs, mergeState.needsIndexSort);
                           
                           final long finalCost = cost;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0d3c64ab/lucene/core/src/java/org/apache/lucene/codecs/NormsConsumer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/NormsConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/NormsConsumer.java
index 7ad7a7c..c21fc01 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/NormsConsumer.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/NormsConsumer.java
@@ -130,7 +130,7 @@ public abstract class NormsConsumer implements Closeable {
                           }
                         }
 
-                        final DocIDMerger<NumericDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.needsIndexSort);
+                        final DocIDMerger<NumericDocValuesSub> docIDMerger = DocIDMerger.of(subs, mergeState.needsIndexSort);
 
                         return new NumericDocValues() {
                           private int docID = -1;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0d3c64ab/lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsWriter.java
index 80a9c49..0540f4f 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsWriter.java
@@ -117,7 +117,7 @@ public abstract class StoredFieldsWriter implements Closeable {
       subs.add(new StoredFieldsMergeSub(new MergeVisitor(mergeState, i), mergeState.docMaps[i], storedFieldsReader, mergeState.maxDocs[i]));
     }
 
-    final DocIDMerger<StoredFieldsMergeSub> docIDMerger = new DocIDMerger<>(subs, mergeState.needsIndexSort);
+    final DocIDMerger<StoredFieldsMergeSub> docIDMerger = DocIDMerger.of(subs, mergeState.needsIndexSort);
 
     int docCount = 0;
     while (true) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0d3c64ab/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java
index c8ad9f6..b84065a 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java
@@ -205,7 +205,7 @@ public abstract class TermVectorsWriter implements Closeable {
       subs.add(new TermVectorsMergeSub(mergeState.docMaps[i], reader, mergeState.maxDocs[i]));
     }
 
-    final DocIDMerger<TermVectorsMergeSub> docIDMerger = new DocIDMerger<>(subs, mergeState.needsIndexSort);
+    final DocIDMerger<TermVectorsMergeSub> docIDMerger = DocIDMerger.of(subs, mergeState.needsIndexSort);
 
     int docCount = 0;
     while (true) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0d3c64ab/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java
index c775e1c..7ab20af 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java
@@ -514,7 +514,7 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
       }
 
       final DocIDMerger<CompressingStoredFieldsMergeSub> docIDMerger =
-          new DocIDMerger<>(subs, true);
+          DocIDMerger.of(subs, true);
       while (true) {
         CompressingStoredFieldsMergeSub sub = docIDMerger.next();
         if (sub == null) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0d3c64ab/lucene/core/src/java/org/apache/lucene/index/DocIDMerger.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/DocIDMerger.java b/lucene/core/src/java/org/apache/lucene/index/DocIDMerger.java
index cd4726b..dd0c9b7 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DocIDMerger.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DocIDMerger.java
@@ -29,17 +29,7 @@ import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
  *  concatenated (unsorted) order, or by a specified index-time sort, skipping
  *  deleted documents and remapping non-deleted documents. */
 
-public class DocIDMerger<T extends DocIDMerger.Sub> {
-
-  private final List<T> subs;
-
-  // Used when indexSort != null:
-  private final PriorityQueue<T> queue;
-  private boolean first;
-
-  // Used when indexIsSorted
-  private T current;
-  private int nextIndex;
+public abstract class DocIDMerger<T extends DocIDMerger.Sub> {
 
   /** Represents one sub-reader being merged */
   public static abstract class Sub {
@@ -58,10 +48,84 @@ public class DocIDMerger<T extends DocIDMerger.Sub> {
   }
 
   /** Construct this from the provided subs, specifying the maximum sub count */
-  public DocIDMerger(List<T> subs, int maxCount, boolean indexIsSorted) throws IOException {
-    this.subs = subs;
-
+  public static <T extends DocIDMerger.Sub> DocIDMerger<T> of(List<T> subs, int maxCount, boolean indexIsSorted) throws IOException {
     if (indexIsSorted && maxCount > 1) {
+      return new SortedDocIDMerger<>(subs, maxCount);
+    } else {
+      return new SequentialDocIDMerger<>(subs);
+    }
+  }
+
+  /** Construct this from the provided subs */
+  public static <T extends DocIDMerger.Sub> DocIDMerger<T> of(List<T> subs, boolean indexIsSorted) throws IOException {
+    return of(subs, subs.size(), indexIsSorted);
+  }
+
+  /** Reuse API, currently only used by postings during merge */
+  public abstract void reset() throws IOException;
+
+  /** Returns null when done */
+  public abstract T next() throws IOException;
+
+  private DocIDMerger() {}
+
+  private static class SequentialDocIDMerger<T extends DocIDMerger.Sub> extends DocIDMerger<T> {
+
+    private final List<T> subs;
+    private T current;
+    private int nextIndex;
+
+    private SequentialDocIDMerger(List<T> subs) throws IOException {
+      this.subs = subs;
+      reset();
+    }
+
+    @Override
+    public void reset() throws IOException {
+      if (subs.size() > 0) {
+        current = subs.get(0);
+        nextIndex = 1;
+      } else {
+        current = null;
+        nextIndex = 0;
+      }
+    }
+
+    @Override
+    public T next() throws IOException {
+      if (current == null) {
+        // NOTE: it's annoying that caller is allowed to call us again even after we returned null before
+        return null;
+      }
+      while (true) {
+        int docID = current.nextDoc();
+        if (docID == NO_MORE_DOCS) {
+          if (nextIndex == subs.size()) {
+            current = null;
+            return null;
+          }
+          current = subs.get(nextIndex);
+          nextIndex++;
+          continue;
+        }
+
+        int mappedDocID = current.docMap.get(docID);
+        if (mappedDocID != -1) {
+          current.mappedDocID = mappedDocID;
+          return current;
+        }
+      }
+    }
+
+  }
+
+  private static class SortedDocIDMerger<T extends DocIDMerger.Sub> extends DocIDMerger<T> {
+
+    private final List<T> subs;
+    private final PriorityQueue<T> queue;
+
+    private SortedDocIDMerger(List<T> subs, int maxCount) throws IOException {
+      this.subs = subs;
       queue = new PriorityQueue<T>(maxCount) {
         @Override
         protected boolean lessThan(Sub a, Sub b) {
@@ -69,112 +133,71 @@ public class DocIDMerger<T extends DocIDMerger.Sub> {
           return a.mappedDocID < b.mappedDocID;
         }
       };
-    } else {
-      // We simply concatentate
-      queue = null;
+      reset();
     }
 
-    reset();
-  }
-
-  /** Construct this from the provided subs */
-  public DocIDMerger(List<T> subs, boolean indexIsSorted) throws IOException {
-    this(subs, subs.size(), indexIsSorted);
-  }
-
-  /** Reuse API, currently only used by postings during merge */
-  public void reset() throws IOException {
-    if (queue != null) {
+    @Override
+    public void reset() throws IOException {
       // caller may not have fully consumed the queue:
       queue.clear();
+      boolean first = true;
       for(T sub : subs) {
-        while (true) {
-          int docID = sub.nextDoc();
-          if (docID == NO_MORE_DOCS) {
-            // all docs in this sub were deleted; do not add it to the queue!
-            break;
+        if (first) {
+          // by setting mappedDocID = -1, this entry is guaranteed to be the top of the queue
+          // so the first call to next() will advance it
+          sub.mappedDocID = -1;
+          first = false;
+        } else {
+          int mappedDocID;
+          while (true) {
+            int docID = sub.nextDoc();
+            if (docID == NO_MORE_DOCS) {
+              mappedDocID = NO_MORE_DOCS;
+              break;
+            }
+            mappedDocID = sub.docMap.get(docID);
+            if (mappedDocID != -1) {
+              break;
+            }
           }
-
-          int mappedDocID = sub.docMap.get(docID);
-          if (mappedDocID == -1) {
-            // doc was deleted
+          if (mappedDocID == NO_MORE_DOCS) {
+            // all docs in this sub were deleted; do not add it to the queue!
             continue;
-          } else {
-            sub.mappedDocID = mappedDocID;
-            queue.add(sub);
-            break;
           }
+          sub.mappedDocID = mappedDocID;
         }
+        queue.add(sub);
       }
-      first = true;
-    } else if (subs.size() > 0) {
-      current = subs.get(0);
-      nextIndex = 1;
-    } else {
-      current = null;
-      nextIndex = 0;
     }
-  }
 
-  /** Returns null when done */
-  public T next() throws IOException {
-    // Loop until we find a non-deleted document
-    if (queue != null) {
+    @Override
+    public T next() throws IOException {
       T top = queue.top();
       if (top == null) {
         // NOTE: it's annoying that caller is allowed to call us again even after we returned null before
         return null;
       }
 
-      if (first == false) {
-        while (true) {
-          int docID = top.nextDoc();
-          if (docID == NO_MORE_DOCS) {
-            queue.pop();
-            top = queue.top();
-            break;
-          }
-          int mappedDocID = top.docMap.get(docID);
-          if (mappedDocID == -1) {
-            // doc was deleted
-            continue;
-          } else {
-            top.mappedDocID = mappedDocID;
-            top = queue.updateTop();
-            break;
-          }
-        }
-      }
-
-      first = false;
-
-      return top;
-
-    } else {
       while (true) {
-        if (current == null) {
-          // NOTE: it's annoying that caller is allowed to call us again even after we returned null before
-          return null;
-        }
-        int docID = current.nextDoc();
+        int docID = top.nextDoc();
         if (docID == NO_MORE_DOCS) {
-          if (nextIndex == subs.size()) {
-            current = null;
-            return null;
-          }
-          current = subs.get(nextIndex);
-          nextIndex++;
-          continue;
+          queue.pop();
+          top = queue.top();
+          break;
         }
-        int mappedDocID = current.docMap.get(docID);
+        int mappedDocID = top.docMap.get(docID);
         if (mappedDocID == -1) {
-          // doc is deleted
+          // doc was deleted
           continue;
+        } else {
+          top.mappedDocID = mappedDocID;
+          top = queue.updateTop();
+          break;
         }
-
-        current.mappedDocID = mappedDocID;
-        return current;
       }
+
+      return top;
     }
   }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0d3c64ab/lucene/core/src/java/org/apache/lucene/index/MappingMultiPostingsEnum.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/MappingMultiPostingsEnum.java b/lucene/core/src/java/org/apache/lucene/index/MappingMultiPostingsEnum.java
index d93c771..6672d64 100644
--- a/lucene/core/src/java/org/apache/lucene/index/MappingMultiPostingsEnum.java
+++ b/lucene/core/src/java/org/apache/lucene/index/MappingMultiPostingsEnum.java
@@ -62,7 +62,7 @@ final class MappingMultiPostingsEnum extends PostingsEnum {
     for(int i=0;i<allSubs.length;i++) {
       allSubs[i] = new MappingPostingsSub(mergeState.docMaps[i]);
     }
-    this.docIDMerger = new DocIDMerger<MappingPostingsSub>(subs, allSubs.length, mergeState.needsIndexSort);
+    this.docIDMerger = DocIDMerger.of(subs, allSubs.length, mergeState.needsIndexSort);
   }
 
   MappingMultiPostingsEnum reset(MultiPostingsEnum postingsEnum) throws IOException {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0d3c64ab/lucene/core/src/test/org/apache/lucene/index/TestDocIDMerger.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDocIDMerger.java b/lucene/core/src/test/org/apache/lucene/index/TestDocIDMerger.java
index d957646..5e19bbb 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestDocIDMerger.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestDocIDMerger.java
@@ -70,7 +70,7 @@ public class TestDocIDMerger extends LuceneTestCase {
       valueStart += maxDoc;
     }
 
-    DocIDMerger<TestSubUnsorted> merger = new DocIDMerger<>(subs, false);
+    DocIDMerger<TestSubUnsorted> merger = DocIDMerger.of(subs, false);
 
     int count = 0;
     while (true) {
@@ -175,7 +175,7 @@ public class TestDocIDMerger extends LuceneTestCase {
         }, docMap.length, i));
     }
 
-    DocIDMerger<TestSubSorted> merger = new DocIDMerger<>(subs, true);
+    DocIDMerger<TestSubSorted> merger = DocIDMerger.of(subs, true);
 
     int count = 0;
     while (true) {


[2/3] lucene-solr:master: LUCENE-7594: Fixed point range queries on floating-point types to recommend using helpers for exclusive bounds that are consistent with Double.compare.

Posted by jp...@apache.org.
LUCENE-7594: Fixed point range queries on floating-point types to recommend using helpers for exclusive bounds that are consistent with Double.compare.


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

Branch: refs/heads/master
Commit: 18d53a43f7b4536572ef98a94ec1d9b529084d1b
Parents: 5020ea2
Author: Adrien Grand <jp...@gmail.com>
Authored: Wed Dec 21 19:33:52 2016 +0100
Committer: Adrien Grand <jp...@gmail.com>
Committed: Wed Dec 21 20:44:56 2016 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  4 +++
 .../org/apache/lucene/document/DoublePoint.java | 30 ++++++++++++++++++--
 .../org/apache/lucene/document/FloatPoint.java  | 30 ++++++++++++++++++--
 .../apache/lucene/search/TestPointQueries.java  | 28 ++++++++++++++++++
 .../lucene/document/TestHalfFloatPoint.java     |  2 ++
 5 files changed, 90 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/18d53a43/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 2e2f9ab..912974d 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -109,6 +109,10 @@ Bug Fixes
   there are too many merges running and one of the merges hits a
   tragic exception (Joey Echeverria via Mike McCandless)
 
+* LUCENE-7594: Fixed point range queries on floating-point types to recommend
+  using helpers for exclusive bounds that are consistent with Double.compare.
+  (Adrien Grand, Dawid Weiss)
+
 Improvements
 
 * LUCENE-6824: TermAutomatonQuery now rewrites to TermQuery,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/18d53a43/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java b/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java
index 9a383a4..6547402 100644
--- a/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java
@@ -45,6 +45,32 @@ import org.apache.lucene.util.NumericUtils;
  */
 public final class DoublePoint extends Field {
 
+  /**
+   * Return the least double that compares greater than {@code d} consistently
+   * with {@link Double#compare}. The only difference with
+   * {@link Math#nextUp(double)} is that this method returns {@code +0d} when
+   * the argument is {@code -0d}.
+   */
+  public static double nextUp(double d) {
+    if (Double.doubleToLongBits(d) == 0x8000_0000_0000_0000L) { // -0d
+      return +0d;
+    }
+    return Math.nextUp(d);
+  }
+
+  /**
+   * Return the greatest double that compares less than {@code d} consistently
+   * with {@link Double#compare}. The only difference with
+   * {@link Math#nextDown(double)} is that this method returns {@code -0d} when
+   * the argument is {@code +0d}.
+   */
+  public static double nextDown(double d) {
+    if (Double.doubleToLongBits(d) == 0L) { // +0d
+      return -0f;
+    }
+    return Math.nextDown(d);
+  }
+
   private static FieldType getType(int numDims) {
     FieldType type = new FieldType();
     type.setDimensions(numDims, Double.BYTES);
@@ -164,8 +190,8 @@ public final class DoublePoint extends Field {
    * <p>
    * You can have half-open ranges (which are in fact &lt;/&le; or &gt;/&ge; queries)
    * by setting {@code lowerValue = Double.NEGATIVE_INFINITY} or {@code upperValue = Double.POSITIVE_INFINITY}.
-   * <p> Ranges are inclusive. For exclusive ranges, pass {@code Math#nextUp(lowerValue)}
-   * or {@code Math.nextDown(upperValue)}.
+   * <p> Ranges are inclusive. For exclusive ranges, pass {@link #nextUp(double) nextUp(lowerValue)}
+   * or {@link #nextUp(double) nextDown(upperValue)}.
    * <p>
    * Range comparisons are consistent with {@link Double#compareTo(Double)}.
    *

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/18d53a43/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java b/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java
index 8d84269..0ec67fd 100644
--- a/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java
@@ -45,6 +45,32 @@ import org.apache.lucene.util.NumericUtils;
  */
 public final class FloatPoint extends Field {
 
+  /**
+   * Return the least float that compares greater than {@code f} consistently
+   * with {@link Float#compare}. The only difference with
+   * {@link Math#nextUp(float)} is that this method returns {@code +0f} when
+   * the argument is {@code -0f}.
+   */
+  public static float nextUp(float f) {
+    if (Float.floatToIntBits(f) == 0x8000_0000) { // -0f
+      return +0f;
+    }
+    return Math.nextUp(f);
+  }
+
+  /**
+   * Return the greatest float that compares less than {@code f} consistently
+   * with {@link Float#compare}. The only difference with
+   * {@link Math#nextDown(float)} is that this method returns {@code -0f} when
+   * the argument is {@code +0f}.
+   */
+  public static float nextDown(float f) {
+    if (Float.floatToIntBits(f) == 0) { // +0f
+      return -0f;
+    }
+    return Math.nextDown(f);
+  }
+
   private static FieldType getType(int numDims) {
     FieldType type = new FieldType();
     type.setDimensions(numDims, Float.BYTES);
@@ -164,8 +190,8 @@ public final class FloatPoint extends Field {
    * <p>
    * You can have half-open ranges (which are in fact &lt;/&le; or &gt;/&ge; queries)
    * by setting {@code lowerValue = Float.NEGATIVE_INFINITY} or {@code upperValue = Float.POSITIVE_INFINITY}.
-   * <p> Ranges are inclusive. For exclusive ranges, pass {@code Math#nextUp(lowerValue)}
-   * or {@code Math.nextDown(upperValue)}.
+   * <p> Ranges are inclusive. For exclusive ranges, pass {@link #nextUp(float) nextUp(lowerValue)}
+   * or {@link #nextUp(float) nextDown(upperValue)}.
    * <p>
    * Range comparisons are consistent with {@link Float#compareTo(Float)}.
    *

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/18d53a43/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java b/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
index 73b2813..5c66478 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
@@ -2052,4 +2052,32 @@ public class TestPointQueries extends LuceneTestCase {
                                               });
     assertEquals("lowerPoint has length=4 but upperPoint has different length=8", e.getMessage());
   }
+
+  public void testNextUp() {
+    assertTrue(Double.compare(0d, DoublePoint.nextUp(-0d)) == 0);
+    assertTrue(Double.compare(Double.MIN_VALUE, DoublePoint.nextUp(0d)) == 0);
+    assertTrue(Double.compare(Double.POSITIVE_INFINITY, DoublePoint.nextUp(Double.MAX_VALUE)) == 0);
+    assertTrue(Double.compare(Double.POSITIVE_INFINITY, DoublePoint.nextUp(Double.POSITIVE_INFINITY)) == 0);
+    assertTrue(Double.compare(-Double.MAX_VALUE, DoublePoint.nextUp(Double.NEGATIVE_INFINITY)) == 0);
+
+    assertTrue(Float.compare(0f, FloatPoint.nextUp(-0f)) == 0);
+    assertTrue(Float.compare(Float.MIN_VALUE, FloatPoint.nextUp(0f)) == 0);
+    assertTrue(Float.compare(Float.POSITIVE_INFINITY, FloatPoint.nextUp(Float.MAX_VALUE)) == 0);
+    assertTrue(Float.compare(Float.POSITIVE_INFINITY, FloatPoint.nextUp(Float.POSITIVE_INFINITY)) == 0);
+    assertTrue(Float.compare(-Float.MAX_VALUE, FloatPoint.nextUp(Float.NEGATIVE_INFINITY)) == 0);
+  }
+
+  public void testNextDown() {
+    assertTrue(Double.compare(-0d, DoublePoint.nextDown(0d)) == 0);
+    assertTrue(Double.compare(-Double.MIN_VALUE, DoublePoint.nextDown(-0d)) == 0);
+    assertTrue(Double.compare(Double.NEGATIVE_INFINITY, DoublePoint.nextDown(-Double.MAX_VALUE)) == 0);
+    assertTrue(Double.compare(Double.NEGATIVE_INFINITY, DoublePoint.nextDown(Double.NEGATIVE_INFINITY)) == 0);
+    assertTrue(Double.compare(Double.MAX_VALUE, DoublePoint.nextDown(Double.POSITIVE_INFINITY)) == 0);
+
+    assertTrue(Float.compare(-0f, FloatPoint.nextDown(0f)) == 0);
+    assertTrue(Float.compare(-Float.MIN_VALUE, FloatPoint.nextDown(-0f)) == 0);
+    assertTrue(Float.compare(Float.NEGATIVE_INFINITY, FloatPoint.nextDown(-Float.MAX_VALUE)) == 0);
+    assertTrue(Float.compare(Float.NEGATIVE_INFINITY, FloatPoint.nextDown(Float.NEGATIVE_INFINITY)) == 0);
+    assertTrue(Float.compare(Float.MAX_VALUE, FloatPoint.nextDown(Float.POSITIVE_INFINITY)) == 0);
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/18d53a43/lucene/sandbox/src/test/org/apache/lucene/document/TestHalfFloatPoint.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestHalfFloatPoint.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestHalfFloatPoint.java
index a24d992..0bcb3f8 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestHalfFloatPoint.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestHalfFloatPoint.java
@@ -229,6 +229,7 @@ public class TestHalfFloatPoint extends LuceneTestCase {
     // values that cannot be exactly represented as a half float
     assertEquals(HalfFloatPoint.nextUp(0f), HalfFloatPoint.nextUp(Float.MIN_VALUE), 0f);
     assertEquals(Float.floatToIntBits(-0f), Float.floatToIntBits(HalfFloatPoint.nextUp(-Float.MIN_VALUE)));
+    assertEquals(Float.floatToIntBits(0f), Float.floatToIntBits(HalfFloatPoint.nextUp(-0f)));
   }
 
   public void testNextDown() {
@@ -239,5 +240,6 @@ public class TestHalfFloatPoint extends LuceneTestCase {
     // values that cannot be exactly represented as a half float
     assertEquals(Float.floatToIntBits(0f), Float.floatToIntBits(HalfFloatPoint.nextDown(Float.MIN_VALUE)));
     assertEquals(HalfFloatPoint.nextDown(-0f), HalfFloatPoint.nextDown(-Float.MIN_VALUE), 0f);
+    assertEquals(Float.floatToIntBits(-0f), Float.floatToIntBits(HalfFloatPoint.nextDown(+0f)));
   }
 }


[3/3] lucene-solr:master: LUCENE-7599: Simplify TestRandomChains using Java's built-in Predicate and Function interfaces.

Posted by jp...@apache.org.
LUCENE-7599: Simplify TestRandomChains using Java's built-in Predicate and Function interfaces.


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

Branch: refs/heads/master
Commit: 5020ea28bc4255de473e795a6638ae67f2720396
Parents: 7e03427
Author: Adrien Grand <jp...@gmail.com>
Authored: Wed Dec 21 19:25:54 2016 +0100
Committer: Adrien Grand <jp...@gmail.com>
Committed: Wed Dec 21 20:44:56 2016 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   3 +
 .../lucene/analysis/core/TestRandomChains.java  | 177 +++++--------------
 2 files changed, 45 insertions(+), 135 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5020ea28/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 618f73a..2e2f9ab 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -190,6 +190,9 @@ Other
 * LUCENE-7559: UnifiedHighlighter: Make Passage more exposed to allow passage creation to
   be customized. (David Smiley)
 
+* LUCENE-7599: Simplify TestRandomChains using Java's built-in Predicate and
+  Function interfaces. (Ahmet Arslan via Adrien Grand)
+
 Build
 
 * LUCENE-7387: fix defaultCodec in build.xml to account for the line ending (hossman)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5020ea28/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestRandomChains.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestRandomChains.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestRandomChains.java
index 94924d3..0bd5e0a 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestRandomChains.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestRandomChains.java
@@ -45,6 +45,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.Random;
 import java.util.Set;
+import java.util.function.Function;
+import java.util.function.Predicate;
 import java.util.regex.Pattern;
 
 import org.apache.lucene.analysis.Analyzer;
@@ -106,15 +108,7 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
   static List<Constructor<? extends TokenFilter>> tokenfilters;
   static List<Constructor<? extends CharFilter>> charfilters;
 
-  private static interface Predicate<T> {
-    boolean apply(T o);
-  }
-
-  private static final Predicate<Object[]> ALWAYS = new Predicate<Object[]>() {
-    public boolean apply(Object[] args) {
-      return true;
-    };
-  };
+  private static final Predicate<Object[]> ALWAYS = (objects -> true);
 
   private static final Map<Constructor<?>,Predicate<Object[]>> brokenConstructors = new HashMap<>();
   static {
@@ -124,36 +118,27 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
           ALWAYS);
       brokenConstructors.put(
           LimitTokenCountFilter.class.getConstructor(TokenStream.class, int.class, boolean.class),
-          new Predicate<Object[]>() {
-            @Override
-            public boolean apply(Object[] args) {
+          args -> {
               assert args.length == 3;
               return !((Boolean) args[2]); // args are broken if consumeAllTokens is false
-            }
           });
       brokenConstructors.put(
           LimitTokenOffsetFilter.class.getConstructor(TokenStream.class, int.class),
           ALWAYS);
       brokenConstructors.put(
           LimitTokenOffsetFilter.class.getConstructor(TokenStream.class, int.class, boolean.class),
-          new Predicate<Object[]>() {
-            @Override
-            public boolean apply(Object[] args) {
+          args -> {
               assert args.length == 3;
               return !((Boolean) args[2]); // args are broken if consumeAllTokens is false
-            }
           });
       brokenConstructors.put(
           LimitTokenPositionFilter.class.getConstructor(TokenStream.class, int.class),
           ALWAYS);
       brokenConstructors.put(
           LimitTokenPositionFilter.class.getConstructor(TokenStream.class, int.class, boolean.class),
-          new Predicate<Object[]>() {
-            @Override
-            public boolean apply(Object[] args) {
+          args -> {
               assert args.length == 3;
               return !((Boolean) args[2]); // args are broken if consumeAllTokens is false
-            }
           });
       for (Class<?> c : Arrays.<Class<?>>asList(
           // TODO: can we promote some of these to be only
@@ -247,12 +232,7 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
       }
     }
     
-    final Comparator<Constructor<?>> ctorComp = new Comparator<Constructor<?>>() {
-      @Override
-      public int compare(Constructor<?> arg0, Constructor<?> arg1) {
-        return arg0.toGenericString().compareTo(arg1.toGenericString());
-      }
-    };
+    final Comparator<Constructor<?>> ctorComp = (arg0, arg1) -> arg0.toGenericString().compareTo(arg1.toGenericString());
     Collections.sort(tokenizers, ctorComp);
     Collections.sort(tokenfilters, ctorComp);
     Collections.sort(charfilters, ctorComp);
@@ -318,21 +298,14 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
     }
   }
   
-  private static interface ArgProducer {
-    Object create(Random random);
-  }
-  
-  private static final Map<Class<?>,ArgProducer> argProducers = new IdentityHashMap<Class<?>,ArgProducer>() {{
-    put(int.class, new ArgProducer() {
-      @Override public Object create(Random random) {
+  private static final Map<Class<?>,Function<Random,Object>> argProducers = new IdentityHashMap<Class<?>,Function<Random,Object>>() {{
+    put(int.class, random ->  {
         // TODO: could cause huge ram usage to use full int range for some filters
         // (e.g. allocate enormous arrays)
         // return Integer.valueOf(random.nextInt());
         return Integer.valueOf(TestUtil.nextInt(random, -50, 50));
-      }
     });
-    put(char.class, new ArgProducer() {
-      @Override public Object create(Random random) {
+    put(char.class, random ->  {
         // TODO: fix any filters that care to throw IAE instead.
         // also add a unicode validating filter to validate termAtt?
         // return Character.valueOf((char)random.nextInt(65536));
@@ -342,49 +315,19 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
             return Character.valueOf(c);
           }
         }
-      }
     });
-    put(float.class, new ArgProducer() {
-      @Override public Object create(Random random) {
-        return Float.valueOf(random.nextFloat());
-      }
-    });
-    put(boolean.class, new ArgProducer() {
-      @Override public Object create(Random random) {
-        return Boolean.valueOf(random.nextBoolean());
-      }
-    });
-    put(byte.class, new ArgProducer() {
-      @Override public Object create(Random random) {
-        // this wraps to negative when casting to byte
-        return Byte.valueOf((byte) random.nextInt(256));
-      }
-    });
-    put(byte[].class, new ArgProducer() {
-      @Override public Object create(Random random) {
+    put(float.class, Random::nextFloat);
+    put(boolean.class, Random::nextBoolean);
+    put(byte.class, random -> (byte) random.nextInt(256));
+    put(byte[].class, random ->  {
         byte bytes[] = new byte[random.nextInt(256)];
         random.nextBytes(bytes);
         return bytes;
-      }
-    });
-    put(Random.class, new ArgProducer() {
-      @Override public Object create(Random random) {
-        return new Random(random.nextLong());
-      }
-    });
-    put(Version.class, new ArgProducer() {
-      @Override public Object create(Random random) {
-        // we expect bugs in emulating old versions
-        return Version.LATEST;
-      }
     });
-    put(AttributeFactory.class, new ArgProducer() {
-      @Override public Object create(Random random) {
-        return newAttributeFactory(random);
-      }
-    });
-    put(Set.class, new ArgProducer() {
-      @Override public Object create(Random random) {
+    put(Random.class, random ->  new Random(random.nextLong()));
+    put(Version.class, random -> Version.LATEST);
+    put(AttributeFactory.class, BaseTokenStreamTestCase::newAttributeFactory);
+    put(Set.class,random ->  {
         // TypeTokenFilter
         Set<String> set = new HashSet<>();
         int num = random.nextInt(5);
@@ -392,10 +335,8 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
           set.add(StandardTokenizer.TOKEN_TYPES[random.nextInt(StandardTokenizer.TOKEN_TYPES.length)]);
         }
         return set;
-      }
     });
-    put(Collection.class, new ArgProducer() {
-      @Override public Object create(Random random) {
+    put(Collection.class, random ->  {
         // CapitalizationFilter
         Collection<char[]> col = new ArrayList<>();
         int num = random.nextInt(5);
@@ -403,10 +344,8 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
           col.add(TestUtil.randomSimpleString(random).toCharArray());
         }
         return col;
-      }
     });
-    put(CharArraySet.class, new ArgProducer() {
-      @Override public Object create(Random random) {
+    put(CharArraySet.class, random ->  {
         int num = random.nextInt(10);
         CharArraySet set = new CharArraySet(num, random.nextBoolean());
         for (int i = 0; i < num; i++) {
@@ -414,28 +353,13 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
           set.add(TestUtil.randomSimpleString(random));
         }
         return set;
-      }
     });
-    put(Pattern.class, new ArgProducer() {
-      @Override public Object create(Random random) {
-        // TODO: don't want to make the exponentially slow ones Dawid documents
-        // in TestPatternReplaceFilter, so dont use truly random patterns (for now)
-        return Pattern.compile("a");
-      }
-    });
-    
-    put(Pattern[].class, new ArgProducer() {
-      @Override public Object create(Random random) {
-        return new Pattern[] {Pattern.compile("([a-z]+)"), Pattern.compile("([0-9]+)")};
-      }
-    });
-    put(PayloadEncoder.class, new ArgProducer() {
-      @Override public Object create(Random random) {
-        return new IdentityEncoder(); // the other encoders will throw exceptions if tokens arent numbers?
-      }
-    });
-    put(Dictionary.class, new ArgProducer() {
-      @Override public Object create(Random random) {
+    // TODO: don't want to make the exponentially slow ones Dawid documents
+    // in TestPatternReplaceFilter, so dont use truly random patterns (for now)
+    put(Pattern.class, random ->  Pattern.compile("a"));
+    put(Pattern[].class, random -> new Pattern[] {Pattern.compile("([a-z]+)"), Pattern.compile("([0-9]+)")});
+    put(PayloadEncoder.class, random -> new IdentityEncoder()); // the other encoders will throw exceptions if tokens arent numbers?
+    put(Dictionary.class, random -> {
         // TODO: make nastier
         InputStream affixStream = TestHunspellStemFilter.class.getResourceAsStream("simple.aff");
         InputStream dictStream = TestHunspellStemFilter.class.getResourceAsStream("simple.dic");
@@ -445,10 +369,8 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
           Rethrow.rethrow(ex);
           return null; // unreachable code
         }
-      }
     });
-    put(HyphenationTree.class, new ArgProducer() {
-      @Override public Object create(Random random) {
+    put(HyphenationTree.class, random -> {
         // TODO: make nastier
         try {
           InputSource is = new InputSource(TestCompoundWordTokenFilter.class.getResource("da_UTF8.xml").toExternalForm());
@@ -458,10 +380,8 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
           Rethrow.rethrow(ex);
           return null; // unreachable code
         }
-      }
     });
-    put(SnowballProgram.class, new ArgProducer() {
-      @Override public Object create(Random random) {
+    put(SnowballProgram.class, random ->  {
         try {
           String lang = TestSnowball.SNOWBALL_LANGS[random.nextInt(TestSnowball.SNOWBALL_LANGS.length)];
           Class<? extends SnowballProgram> clazz = Class.forName("org.tartarus.snowball.ext." + lang + "Stemmer").asSubclass(SnowballProgram.class);
@@ -470,10 +390,8 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
           Rethrow.rethrow(ex);
           return null; // unreachable code
         }
-      }
     });
-    put(String.class, new ArgProducer() {
-      @Override public Object create(Random random) {
+    put(String.class, random ->  {
         // TODO: make nastier
         if (random.nextBoolean()) {
           // a token type
@@ -481,10 +399,8 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
         } else {
           return TestUtil.randomSimpleString(random);
         }
-      }
     });
-    put(NormalizeCharMap.class, new ArgProducer() {
-      @Override public Object create(Random random) {
+    put(NormalizeCharMap.class, random -> {
         NormalizeCharMap.Builder builder = new NormalizeCharMap.Builder();
         // we can't add duplicate keys, or NormalizeCharMap gets angry
         Set<String> keys = new HashSet<>();
@@ -500,10 +416,8 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
           }
         }
         return builder.build();
-      }
     });
-    put(CharacterRunAutomaton.class, new ArgProducer() {
-      @Override public Object create(Random random) {
+    put(CharacterRunAutomaton.class, random -> {
         // TODO: could probably use a purely random automaton
         switch(random.nextInt(5)) {
           case 0: return MockTokenizer.KEYWORD;
@@ -512,10 +426,8 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
           case 3: return MockTokenFilter.EMPTY_STOPSET;
           default: return MockTokenFilter.ENGLISH_STOPSET;
         }
-      }
     });
-    put(CharArrayMap.class, new ArgProducer() {
-      @Override public Object create(Random random) {
+    put(CharArrayMap.class, random -> {
         int num = random.nextInt(10);
         CharArrayMap<String> map = new CharArrayMap<>(num, random.nextBoolean());
         for (int i = 0; i < num; i++) {
@@ -523,10 +435,8 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
           map.put(TestUtil.randomSimpleString(random), TestUtil.randomSimpleString(random));
         }
         return map;
-      }
     });
-    put(StemmerOverrideMap.class, new ArgProducer() {
-      @Override public Object create(Random random) {
+    put(StemmerOverrideMap.class, random -> {
         int num = random.nextInt(10);
         StemmerOverrideFilter.Builder builder = new StemmerOverrideFilter.Builder(random.nextBoolean());
         for (int i = 0; i < num; i++) {
@@ -545,11 +455,10 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
         } catch (Exception ex) {
           Rethrow.rethrow(ex);
           return null; // unreachable code
-        }
       }
     });
-    put(SynonymMap.class, new ArgProducer() {
-      @Override public Object create(Random random) {
+    put(SynonymMap.class, new Function<Random, Object>() {
+      @Override public Object apply(Random random) {
         SynonymMap.Builder b = new SynonymMap.Builder(random.nextBoolean());
         final int numEntries = atLeast(10);
         for (int j = 0; j < numEntries; j++) {
@@ -578,12 +487,9 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
         }
       }    
     });
-    put(DateFormat.class, new ArgProducer() {
-      @Override
-      public Object create(Random random) {
+    put(DateFormat.class, random -> {
         if (random.nextBoolean()) return null;
         return DateFormat.getDateInstance(DateFormat.DEFAULT, randomLocale(random));
-      }
     });
   }};
   
@@ -608,9 +514,9 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
   
   @SuppressWarnings("unchecked")
   static <T> T newRandomArg(Random random, Class<T> paramType) {
-    final ArgProducer producer = argProducers.get(paramType);
+    final Function<Random,Object> producer = argProducers.get(paramType);
     assertNotNull("No producer for arguments of type " + paramType.getName() + " found", producer);
-    return (T) producer.create(random);
+    return (T) producer.apply(random);
   }
   
   static Object[] newTokenizerArgs(Random random, Class<?>[] paramTypes) {
@@ -707,7 +613,8 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
       sb.append("filters=");
       sb.append(tokenFilterSpec.toString);
       sb.append("\n");
-      sb.append("offsetsAreCorrect=" + tokenFilterSpec.offsetsAreCorrect);
+      sb.append("offsetsAreCorrect=");
+      sb.append(tokenFilterSpec.offsetsAreCorrect);
       return sb.toString();
     }
     
@@ -745,12 +652,12 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
 
     private boolean broken(Constructor<?> ctor, Object[] args) {
       final Predicate<Object[]> pred = brokenConstructors.get(ctor);
-      return pred != null && pred.apply(args);
+      return pred != null && pred.test(args);
     }
 
     private boolean brokenOffsets(Constructor<?> ctor, Object[] args) {
       final Predicate<Object[]> pred = brokenOffsetsConstructors.get(ctor);
-      return pred != null && pred.apply(args);
+      return pred != null && pred.test(args);
     }
 
     // create a new random tokenizer from classpath