You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by us...@apache.org on 2016/02/29 09:26:12 UTC

lucene-solr git commit: LUCENE-7053: Remove custom comparators from BytesRef class and solely use natural byte[] comparator throughout codebase. It also replaces the natural comparator in ArrayUtil by Java 8's Comparator#naturalOrder().

Repository: lucene-solr
Updated Branches:
  refs/heads/master ae4d77ae4 -> f48d23cd1


LUCENE-7053: Remove custom comparators from BytesRef class and solely use natural byte[] comparator throughout codebase. It also replaces the natural comparator in ArrayUtil by Java 8's Comparator#naturalOrder().


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

Branch: refs/heads/master
Commit: f48d23cd1448f20fb1b97ec986ded76a04a7075c
Parents: ae4d77a
Author: Uwe Schindler <us...@apache.org>
Authored: Mon Feb 29 09:25:55 2016 +0100
Committer: Uwe Schindler <us...@apache.org>
Committed: Mon Feb 29 09:25:55 2016 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  10 +-
 .../codecs/blockterms/BlockTermsReader.java     |   4 +-
 .../apache/lucene/search/FuzzyTermsEnum.java    |   5 +-
 .../org/apache/lucene/search/SortField.java     |   2 +-
 .../java/org/apache/lucene/util/ArrayUtil.java  |  20 +--
 .../java/org/apache/lucene/util/BytesRef.java   | 122 ++++---------------
 .../org/apache/lucene/util/BytesRefHash.java    |   6 +-
 .../org/apache/lucene/util/CollectionUtil.java  |   4 +-
 .../apache/lucene/util/LegacyNumericUtils.java  |   4 -
 .../org/apache/lucene/util/OfflineSorter.java   |   2 +-
 .../apache/lucene/index/TestMultiFields.java    |   4 +-
 .../apache/lucene/util/TestBytesRefArray.java   |   5 +-
 .../apache/lucene/util/TestBytesRefHash.java    |  27 +---
 .../lucene/util/TestInPlaceMergeSorter.java     |   4 +-
 .../org/apache/lucene/util/TestIntroSorter.java |   4 +-
 .../org/apache/lucene/util/TestTimSorter.java   |   3 +-
 .../org/apache/lucene/util/TestUnicodeUtil.java |  15 +--
 .../apache/lucene/search/join/TermsQuery.java   |   9 +-
 .../apache/lucene/search/join/TestJoinUtil.java |   2 +-
 .../apache/lucene/index/memory/MemoryIndex.java |  11 +-
 .../lucene/search/DocValuesTermsQuery.java      |   2 +-
 .../search/suggest/SortedInputIterator.java     |   7 +-
 .../suggest/fst/FSTCompletionBuilder.java       |   5 +-
 .../lucene/search/suggest/tst/TSTLookup.java    |  51 +++++++-
 .../search/suggest/TestInputIterator.java       |  21 ++--
 .../search/suggest/fst/BytesRefSortersTest.java |   5 +-
 .../java/org/apache/lucene/util/TestUtil.java   |  21 ++++
 27 files changed, 158 insertions(+), 217 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f48d23cd/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 3e98a45..7060170 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -98,12 +98,18 @@ API Changes
 * LUCENE-6917: Deprecate and rename NumericXXX classes to
   LegacyNumericXXX in favor of points (Mike McCandless)
 
-* LUCENE-6947: SortField.missingValue is now protected. You can read its value
-  using the new SortField.getMissingValue getter. (Adrien Grand)
+* LUCENE-6947: SortField.missingValue is now protected. You can read its
+  value using the new SortField.getMissingValue getter. (Adrien Grand)
 
 * LUCENE-7028: Remove duplicate method in LegacyNumericUtils.
   (Uwe Schindler)
 
+* LUCENE-7052, LUCENE-7053: Remove custom comparators from BytesRef
+  class and solely use natural byte[] comparator throughout codebase.
+  This also simplifies API of BytesRefHash. It also replaces the natural
+  comparator in ArrayUtil by Java 8's Comparator#naturalOrder().
+  (Mike McCandless, Uwe Schindler, Robert Muir)
+
 Optimizations
 
 * LUCENE-6891: Use prefix coding when writing points in 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f48d23cd/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsReader.java
----------------------------------------------------------------------
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsReader.java b/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsReader.java
index 6c9ddd2..f363cd3 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsReader.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsReader.java
@@ -372,7 +372,7 @@ public class BlockTermsReader extends FieldsProducer {
         // is after current term but before next index term:
         if (indexIsCurrent) {
 
-          final int cmp = BytesRef.getUTF8SortedAsUnicodeComparator().compare(term.get(), target);
+          final int cmp = term.get().compareTo(target);
 
           if (cmp == 0) {
             // Already at the requested term
@@ -390,7 +390,7 @@ public class BlockTermsReader extends FieldsProducer {
               didIndexNext = true;
             }
 
-            if (nextIndexTerm == null || BytesRef.getUTF8SortedAsUnicodeComparator().compare(target, nextIndexTerm) < 0) {
+            if (nextIndexTerm == null || target.compareTo(nextIndexTerm) < 0) {
               // Optimization: requested term is within the
               // same term block we are now in; skip seeking
               // (but do scanning):

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f48d23cd/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java b/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
index f934d80..8331d0f 100644
--- a/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
+++ b/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
@@ -59,9 +59,6 @@ public class FuzzyTermsEnum extends TermsEnum {
   
   private float bottom;
   private BytesRef bottomTerm;
-
-  // TODO: chicken-and-egg
-  private final Comparator<BytesRef> termComparator = BytesRef.getUTF8SortedAsUnicodeComparator();
   
   protected final float minSimilarity;
   protected final float scale_factor;
@@ -193,7 +190,7 @@ public class FuzzyTermsEnum extends TermsEnum {
     int oldMaxEdits = maxEdits;
     
     // true if the last term encountered is lexicographically equal or after the bottom term in the PQ
-    boolean termAfter = bottomTerm == null || (lastTerm != null && termComparator.compare(lastTerm, bottomTerm) >= 0);
+    boolean termAfter = bottomTerm == null || (lastTerm != null && lastTerm.compareTo(bottomTerm) >= 0);
 
     // as long as the max non-competitive boost is >= the max boost
     // for some edit distance, keep dropping the max edit distance.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f48d23cd/lucene/core/src/java/org/apache/lucene/search/SortField.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/SortField.java b/lucene/core/src/java/org/apache/lucene/search/SortField.java
index 3c5974f..880697b 100644
--- a/lucene/core/src/java/org/apache/lucene/search/SortField.java
+++ b/lucene/core/src/java/org/apache/lucene/search/SortField.java
@@ -316,7 +316,7 @@ public class SortField {
     return Objects.hash(field, type, reverse, comparatorSource, missingValue);
   }
 
-  private Comparator<BytesRef> bytesComparator = BytesRef.getUTF8SortedAsUnicodeComparator();
+  private Comparator<BytesRef> bytesComparator = Comparator.naturalOrder();
 
   public void setBytesComparator(Comparator<BytesRef> b) {
     bytesComparator = b;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f48d23cd/lucene/core/src/java/org/apache/lucene/util/ArrayUtil.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/ArrayUtil.java b/lucene/core/src/java/org/apache/lucene/util/ArrayUtil.java
index 3ab5da2..70feff1 100644
--- a/lucene/core/src/java/org/apache/lucene/util/ArrayUtil.java
+++ b/lucene/core/src/java/org/apache/lucene/util/ArrayUtil.java
@@ -620,22 +620,6 @@ public final class ArrayUtil {
     return result;
   }
 
-  private static class NaturalComparator<T extends Comparable<? super T>> implements Comparator<T> {
-    NaturalComparator() {}
-    @Override
-    public int compare(T o1, T o2) {
-      return o1.compareTo(o2);
-    }
-  }
-
-  private static final Comparator<?> NATURAL_COMPARATOR = new NaturalComparator<>();
-
-  /** Get the natural {@link Comparator} for the provided object class. */
-  @SuppressWarnings("unchecked")
-  public static <T extends Comparable<? super T>> Comparator<T> naturalComparator() {
-    return (Comparator<T>) NATURAL_COMPARATOR;
-  }
-
   /** Swap values stored in slots <code>i</code> and <code>j</code> */
   public static <T> void swap(T[] arr, int i, int j) {
     final T tmp = arr[i];
@@ -672,7 +656,7 @@ public final class ArrayUtil {
    */
   public static <T extends Comparable<? super T>> void introSort(T[] a, int fromIndex, int toIndex) {
     if (toIndex-fromIndex <= 1) return;
-    introSort(a, fromIndex, toIndex, ArrayUtil.<T>naturalComparator());
+    introSort(a, fromIndex, toIndex, Comparator.naturalOrder());
   }
   
   /**
@@ -712,7 +696,7 @@ public final class ArrayUtil {
    */
   public static <T extends Comparable<? super T>> void timSort(T[] a, int fromIndex, int toIndex) {
     if (toIndex-fromIndex <= 1) return;
-    timSort(a, fromIndex, toIndex, ArrayUtil.<T>naturalComparator());
+    timSort(a, fromIndex, toIndex, Comparator.naturalOrder());
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f48d23cd/lucene/core/src/java/org/apache/lucene/util/BytesRef.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/BytesRef.java b/lucene/core/src/java/org/apache/lucene/util/BytesRef.java
index c30c0db..c62d639 100644
--- a/lucene/core/src/java/org/apache/lucene/util/BytesRef.java
+++ b/lucene/core/src/java/org/apache/lucene/util/BytesRef.java
@@ -18,7 +18,6 @@ package org.apache.lucene.util;
 
 
 import java.util.Arrays;
-import java.util.Comparator;
 
 /** Represents byte[], as a slice (offset + length) into an
  *  existing byte[].  The {@link #bytes} member should never be null;
@@ -30,6 +29,10 @@ import java.util.Comparator;
  * Using code like {@code new String(bytes, offset, length)} to do this
  * is <b>wrong</b>, as it does not respect the correct character set
  * and may return wrong results (depending on the platform's defaults)!
+ * 
+ * <p>{@code BytesRef} implements {@link Comparable}. The underlying byte arrays
+ * are sorted lexicographically, numerically treating elements as unsigned.
+ * This is identical to Unicode codepoint order.
  */
 public final class BytesRef implements Comparable<BytesRef>,Cloneable {
   /** An empty byte array for convenience */
@@ -169,106 +172,29 @@ public final class BytesRef implements Comparable<BytesRef>,Cloneable {
   /** Unsigned byte order comparison */
   @Override
   public int compareTo(BytesRef other) {
-    return utf8SortedAsUnicodeSortOrder.compare(this, other);
-  }
-  
-  private final static Comparator<BytesRef> utf8SortedAsUnicodeSortOrder = new UTF8SortedAsUnicodeComparator();
-
-  public static Comparator<BytesRef> getUTF8SortedAsUnicodeComparator() {
-    return utf8SortedAsUnicodeSortOrder;
-  }
-
-  private static class UTF8SortedAsUnicodeComparator implements Comparator<BytesRef> {
-    // Only singleton
-    private UTF8SortedAsUnicodeComparator() {};
-
-    @Override
-    public int compare(BytesRef a, BytesRef b) {
-      final byte[] aBytes = a.bytes;
-      int aUpto = a.offset;
-      final byte[] bBytes = b.bytes;
-      int bUpto = b.offset;
-      
-      final int aStop = aUpto + Math.min(a.length, b.length);
-      while(aUpto < aStop) {
-        int aByte = aBytes[aUpto++] & 0xff;
-        int bByte = bBytes[bUpto++] & 0xff;
-
-        int diff = aByte - bByte;
-        if (diff != 0) {
-          return diff;
-        }
-      }
-
-      // One is a prefix of the other, or, they are equal:
-      return a.length - b.length;
-    }    
-  }
-
-  /** @deprecated This comparator is only a transition mechanism */
-  @Deprecated
-  private final static Comparator<BytesRef> utf8SortedAsUTF16SortOrder = new UTF8SortedAsUTF16Comparator();
-
-  /** @deprecated This comparator is only a transition mechanism */
-  @Deprecated
-  public static Comparator<BytesRef> getUTF8SortedAsUTF16Comparator() {
-    return utf8SortedAsUTF16SortOrder;
-  }
-
-  /** @deprecated This comparator is only a transition mechanism */
-  @Deprecated
-  private static class UTF8SortedAsUTF16Comparator implements Comparator<BytesRef> {
-    // Only singleton
-    private UTF8SortedAsUTF16Comparator() {};
-
-    @Override
-    public int compare(BytesRef a, BytesRef b) {
-
-      final byte[] aBytes = a.bytes;
-      int aUpto = a.offset;
-      final byte[] bBytes = b.bytes;
-      int bUpto = b.offset;
-      
-      final int aStop;
-      if (a.length < b.length) {
-        aStop = aUpto + a.length;
-      } else {
-        aStop = aUpto + b.length;
+    // TODO: Once we are on Java 9 replace this by java.util.Arrays#compareUnsigned()
+    // which is implemented by a Hotspot intrinsic! Also consider building a
+    // Multi-Release-JAR!
+    final byte[] aBytes = this.bytes;
+    int aUpto = this.offset;
+    final byte[] bBytes = other.bytes;
+    int bUpto = other.offset;
+    
+    final int aStop = aUpto + Math.min(this.length, other.length);
+    while(aUpto < aStop) {
+      int aByte = aBytes[aUpto++] & 0xff;
+      int bByte = bBytes[bUpto++] & 0xff;
+
+      int diff = aByte - bByte;
+      if (diff != 0) {
+        return diff;
       }
-
-      while(aUpto < aStop) {
-        int aByte = aBytes[aUpto++] & 0xff;
-        int bByte = bBytes[bUpto++] & 0xff;
-
-        if (aByte != bByte) {
-
-          // See http://icu-project.org/docs/papers/utf16_code_point_order.html#utf-8-in-utf-16-order
-
-          // We know the terms are not equal, but, we may
-          // have to carefully fixup the bytes at the
-          // difference to match UTF16's sort order:
-          
-          // NOTE: instead of moving supplementary code points (0xee and 0xef) to the unused 0xfe and 0xff, 
-          // we move them to the unused 0xfc and 0xfd [reserved for future 6-byte character sequences]
-          // this reserves 0xff for preflex's term reordering (surrogate dance), and if unicode grows such
-          // that 6-byte sequences are needed we have much bigger problems anyway.
-          if (aByte >= 0xee && bByte >= 0xee) {
-            if ((aByte & 0xfe) == 0xee) {
-              aByte += 0xe;
-            }
-            if ((bByte&0xfe) == 0xee) {
-              bByte += 0xe;
-            }
-          }
-          return aByte - bByte;
-        }
-      }
-
-      // One is a prefix of the other, or, they are equal:
-      return a.length - b.length;
     }
+
+    // One is a prefix of the other, or, they are equal:
+    return this.length - other.length;
   }
-  
+    
   /**
    * Creates a new BytesRef that points to a copy of the bytes from 
    * <code>other</code>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f48d23cd/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java b/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java
index 82cce03..20d9c4d 100644
--- a/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java
+++ b/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java
@@ -18,7 +18,6 @@ package org.apache.lucene.util;
 
 
 import java.util.Arrays;
-import java.util.Comparator;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.util.ByteBlockPool.DirectAllocator;
@@ -158,7 +157,6 @@ public final class BytesRefHash {
    * </p>
    */
   public int[] sort() {
-    final Comparator<BytesRef> comp = BytesRef.getUTF8SortedAsUnicodeComparator();
     final int[] compact = compact();
     new IntroSorter() {
       @Override
@@ -174,7 +172,7 @@ public final class BytesRefHash {
         assert bytesStart.length > id1 && bytesStart.length > id2;
         pool.setBytesRef(scratch1, bytesStart[id1]);
         pool.setBytesRef(scratch2, bytesStart[id2]);
-        return comp.compare(scratch1, scratch2);
+        return scratch1.compareTo(scratch2);
       }
 
       @Override
@@ -189,7 +187,7 @@ public final class BytesRefHash {
         final int id = compact[j];
         assert bytesStart.length > id;
         pool.setBytesRef(scratch2, bytesStart[id]);
-        return comp.compare(pivot, scratch2);
+        return pivot.compareTo(scratch2);
       }
       
       private final BytesRef pivot = new BytesRef(),

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f48d23cd/lucene/core/src/java/org/apache/lucene/util/CollectionUtil.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/CollectionUtil.java b/lucene/core/src/java/org/apache/lucene/util/CollectionUtil.java
index 0fc6cc8..ec0c53a 100644
--- a/lucene/core/src/java/org/apache/lucene/util/CollectionUtil.java
+++ b/lucene/core/src/java/org/apache/lucene/util/CollectionUtil.java
@@ -146,7 +146,7 @@ public final class CollectionUtil {
   public static <T extends Comparable<? super T>> void introSort(List<T> list) {
     final int size = list.size();
     if (size <= 1) return;
-    introSort(list, ArrayUtil.<T>naturalComparator());
+    introSort(list, Comparator.naturalOrder());
   }
 
   // Tim sorts:
@@ -172,7 +172,7 @@ public final class CollectionUtil {
   public static <T extends Comparable<? super T>> void timSort(List<T> list) {
     final int size = list.size();
     if (size <= 1) return;
-    timSort(list, ArrayUtil.<T>naturalComparator());
+    timSort(list, Comparator.naturalOrder());
   }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f48d23cd/lucene/core/src/java/org/apache/lucene/util/LegacyNumericUtils.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/LegacyNumericUtils.java b/lucene/core/src/java/org/apache/lucene/util/LegacyNumericUtils.java
index b41b5f0..f338d25 100644
--- a/lucene/core/src/java/org/apache/lucene/util/LegacyNumericUtils.java
+++ b/lucene/core/src/java/org/apache/lucene/util/LegacyNumericUtils.java
@@ -53,10 +53,6 @@ import org.apache.lucene.index.TermsEnum;
  * {@link org.apache.lucene.search.LegacyNumericRangeQuery} implements the query part
  * for the same data types.
  *
- * <p>This class can also be used, to generate lexicographically sortable (according to
- * {@link BytesRef#getUTF8SortedAsUTF16Comparator()}) representations of numeric data
- * types for other usages (e.g. sorting).
- *
  * @lucene.internal
  *
  * @deprecated Please use {@link org.apache.lucene.index.PointValues} instead.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f48d23cd/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java b/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java
index 8516ec7..b146102 100644
--- a/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java
+++ b/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java
@@ -174,7 +174,7 @@ public class OfflineSorter {
   private final Comparator<BytesRef> comparator;
   
   /** Default comparator: sorts in binary (codepoint) order */
-  public static final Comparator<BytesRef> DEFAULT_COMPARATOR = BytesRef.getUTF8SortedAsUnicodeComparator();
+  public static final Comparator<BytesRef> DEFAULT_COMPARATOR = Comparator.naturalOrder();
 
   /**
    * Defaults constructor.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f48d23cd/lucene/core/src/test/org/apache/lucene/index/TestMultiFields.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestMultiFields.java b/lucene/core/src/test/org/apache/lucene/index/TestMultiFields.java
index f0954f3..27f2f1a 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestMultiFields.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestMultiFields.java
@@ -104,8 +104,8 @@ public class TestMultiFields extends LuceneTestCase {
 
       if (VERBOSE) {
         List<BytesRef> termsList = new ArrayList<>(uniqueTerms);
-        Collections.sort(termsList, BytesRef.getUTF8SortedAsUTF16Comparator());
-        System.out.println("TEST: terms in UTF16 order:");
+        Collections.sort(termsList);
+        System.out.println("TEST: terms in UTF-8 order:");
         for(BytesRef b : termsList) {
           System.out.println("  " + UnicodeUtil.toHexString(b.utf8ToString()) + " " + b);
           for(int docID : docs.get(b)) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f48d23cd/lucene/core/src/test/org/apache/lucene/util/TestBytesRefArray.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/TestBytesRefArray.java b/lucene/core/src/test/org/apache/lucene/util/TestBytesRefArray.java
index f2e7b60..048eae0 100644
--- a/lucene/core/src/test/org/apache/lucene/util/TestBytesRefArray.java
+++ b/lucene/core/src/test/org/apache/lucene/util/TestBytesRefArray.java
@@ -90,9 +90,8 @@ public class TestBytesRefArray extends LuceneTestCase {
         stringList.add(randomRealisticUnicodeString);
       }
       
-      Collections.sort(stringList);
-      BytesRefIterator iter = list.iterator(BytesRef
-          .getUTF8SortedAsUTF16Comparator());
+      Collections.sort(stringList, TestUtil.STRING_CODEPOINT_COMPARATOR);
+      BytesRefIterator iter = list.iterator(Comparator.naturalOrder());
       int i = 0;
       BytesRef next;
       while ((next = iter.next()) != null) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f48d23cd/lucene/core/src/test/org/apache/lucene/util/TestBytesRefHash.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/TestBytesRefHash.java b/lucene/core/src/test/org/apache/lucene/util/TestBytesRefHash.java
index 50d921b..f80aea4 100644
--- a/lucene/core/src/test/org/apache/lucene/util/TestBytesRefHash.java
+++ b/lucene/core/src/test/org/apache/lucene/util/TestBytesRefHash.java
@@ -17,9 +17,7 @@
 package org.apache.lucene.util;
 
 
-import java.util.Arrays;
 import java.util.BitSet;
-import java.util.Comparator;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map.Entry;
@@ -168,15 +166,6 @@ public class TestBytesRefHash extends LuceneTestCase {
     }
   }
 
-  private static int[] codePoints(String input) {
-    int length = Character.codePointCount(input, 0, input.length());
-    int word[] = new int[length];
-    for (int i = 0, j = 0, cp = 0; i < input.length(); i += Character.charCount(cp)) {
-      word[j++] = cp = input.codePointAt(i);
-    }
-    return word;
-  }
-
   /**
    * Test method for
    * {@link org.apache.lucene.util.BytesRefHash#sort()}.
@@ -188,21 +177,7 @@ public class TestBytesRefHash extends LuceneTestCase {
     for (int j = 0; j < num; j++) {
 
       // Sorts by unicode code point order (is there a simple way, e.g. a Collator?)
-      SortedSet<String> strings = new TreeSet<>(new Comparator<String>() {
-          @Override
-          public int compare(String a, String b) {
-            int[] aCodePoints = codePoints(a);
-            int[] bCodePoints = codePoints(b);
-            for(int i=0;i<Math.min(aCodePoints.length, bCodePoints.length);i++) {
-              if (aCodePoints[i] < bCodePoints[i]) {
-                return -1;
-              } else if (aCodePoints[i] > bCodePoints[i]) {
-                return 1;
-              }
-            }
-            return aCodePoints.length - bCodePoints.length;
-          }
-        });
+      SortedSet<String> strings = new TreeSet<>(TestUtil.STRING_CODEPOINT_COMPARATOR);
       for (int i = 0; i < 797; i++) {
         String str;
         do {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f48d23cd/lucene/core/src/test/org/apache/lucene/util/TestInPlaceMergeSorter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/TestInPlaceMergeSorter.java b/lucene/core/src/test/org/apache/lucene/util/TestInPlaceMergeSorter.java
index ed8e030..3158f7c 100644
--- a/lucene/core/src/test/org/apache/lucene/util/TestInPlaceMergeSorter.java
+++ b/lucene/core/src/test/org/apache/lucene/util/TestInPlaceMergeSorter.java
@@ -17,6 +17,8 @@
 package org.apache.lucene.util;
 
 
+import java.util.Comparator;
+
 import org.junit.runner.RunWith;
 
 import com.carrotsearch.randomizedtesting.RandomizedRunner;
@@ -30,7 +32,7 @@ public class TestInPlaceMergeSorter extends BaseSortTestCase {
 
   @Override
   public Sorter newSorter(Entry[] arr) {
-    return new ArrayInPlaceMergeSorter<>(arr, ArrayUtil.<Entry>naturalComparator());
+    return new ArrayInPlaceMergeSorter<>(arr, Comparator.naturalOrder());
   }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f48d23cd/lucene/core/src/test/org/apache/lucene/util/TestIntroSorter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/TestIntroSorter.java b/lucene/core/src/test/org/apache/lucene/util/TestIntroSorter.java
index 56022d2..070f445 100644
--- a/lucene/core/src/test/org/apache/lucene/util/TestIntroSorter.java
+++ b/lucene/core/src/test/org/apache/lucene/util/TestIntroSorter.java
@@ -16,7 +16,7 @@
  */
 package org.apache.lucene.util;
 
-
+import java.util.Comparator;
 
 public class TestIntroSorter extends BaseSortTestCase {
 
@@ -26,7 +26,7 @@ public class TestIntroSorter extends BaseSortTestCase {
 
   @Override
   public Sorter newSorter(Entry[] arr) {
-    return new ArrayIntroSorter<>(arr, ArrayUtil.<Entry>naturalComparator());
+    return new ArrayIntroSorter<>(arr, Comparator.naturalOrder());
   }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f48d23cd/lucene/core/src/test/org/apache/lucene/util/TestTimSorter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/TestTimSorter.java b/lucene/core/src/test/org/apache/lucene/util/TestTimSorter.java
index 59c0b91..5a28659 100644
--- a/lucene/core/src/test/org/apache/lucene/util/TestTimSorter.java
+++ b/lucene/core/src/test/org/apache/lucene/util/TestTimSorter.java
@@ -16,6 +16,7 @@
  */
 package org.apache.lucene.util;
 
+import java.util.Comparator;
 
 public class TestTimSorter extends BaseSortTestCase {
 
@@ -25,6 +26,6 @@ public class TestTimSorter extends BaseSortTestCase {
 
   @Override
   public Sorter newSorter(Entry[] arr) {
-    return new ArrayTimSorter<>(arr, ArrayUtil.<Entry>naturalComparator(), TestUtil.nextInt(random(), 0, arr.length));
+    return new ArrayTimSorter<>(arr, Comparator.naturalOrder(), TestUtil.nextInt(random(), 0, arr.length));
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f48d23cd/lucene/core/src/test/org/apache/lucene/util/TestUnicodeUtil.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/TestUnicodeUtil.java b/lucene/core/src/test/org/apache/lucene/util/TestUnicodeUtil.java
index d86cdf2..7bbd606 100644
--- a/lucene/core/src/test/org/apache/lucene/util/TestUnicodeUtil.java
+++ b/lucene/core/src/test/org/apache/lucene/util/TestUnicodeUtil.java
@@ -134,7 +134,6 @@ public class TestUnicodeUtil extends LuceneTestCase {
 
   public void testUTF8toUTF32() {
     int[] utf32 = new int[0];
-    int[] codePoints = new int[20];
     int num = atLeast(50000);
     for (int i = 0; i < num; i++) {
       final String s = TestUtil.randomUnicodeString(random());
@@ -143,21 +142,15 @@ public class TestUnicodeUtil extends LuceneTestCase {
       utf32 = ArrayUtil.grow(utf32, utf8Len);
       final int utf32Len = UnicodeUtil.UTF8toUTF32(new BytesRef(utf8, 0, utf8Len), utf32);
       
-      int charUpto = 0;
-      int intUpto = 0;
-      while(charUpto < s.length()) {
-        final int cp = s.codePointAt(charUpto);
-        codePoints[intUpto++] = cp;
-        charUpto += Character.charCount(cp);
-      }
-      if (!ArrayUtil.equals(codePoints, 0, utf32, 0, intUpto)) {
+      int[] codePoints = s.codePoints().toArray();
+      if (!ArrayUtil.equals(codePoints, 0, utf32, 0, codePoints.length)) {
         System.out.println("FAILED");
         for(int j=0;j<s.length();j++) {
           System.out.println("  char[" + j + "]=" + Integer.toHexString(s.charAt(j)));
         }
         System.out.println();
-        assertEquals(intUpto, utf32Len);
-        for(int j=0;j<intUpto;j++) {
+        assertEquals(codePoints.length, utf32Len);
+        for(int j=0;j<codePoints.length;j++) {
           System.out.println("  " + Integer.toHexString(utf32[j]) + " vs " + Integer.toHexString(codePoints[j]));
         }
         fail("mismatch");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f48d23cd/lucene/join/src/java/org/apache/lucene/search/join/TermsQuery.java
----------------------------------------------------------------------
diff --git a/lucene/join/src/java/org/apache/lucene/search/join/TermsQuery.java b/lucene/join/src/java/org/apache/lucene/search/join/TermsQuery.java
index 11b201d..63561c3 100644
--- a/lucene/join/src/java/org/apache/lucene/search/join/TermsQuery.java
+++ b/lucene/join/src/java/org/apache/lucene/search/join/TermsQuery.java
@@ -26,7 +26,6 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefHash;
 
 import java.io.IOException;
-import java.util.Comparator;
 
 /**
  * A query that has an array of terms from a specific field. This query will match documents have one or more terms in
@@ -100,7 +99,6 @@ class TermsQuery extends MultiTermQuery {
 
     private final BytesRef lastTerm;
     private final BytesRef spare = new BytesRef();
-    private final Comparator<BytesRef> comparator;
 
     private BytesRef seekTerm;
     private int upto = 0;
@@ -109,7 +107,6 @@ class TermsQuery extends MultiTermQuery {
       super(tenum);
       this.terms = terms;
       this.ords = ords;
-      comparator = BytesRef.getUTF8SortedAsUnicodeComparator();
       lastElement = terms.size() - 1;
       lastTerm = terms.get(ords[lastElement], new BytesRef());
       seekTerm = terms.get(ords[upto], spare);
@@ -124,12 +121,12 @@ class TermsQuery extends MultiTermQuery {
 
     @Override
     protected AcceptStatus accept(BytesRef term) throws IOException {
-      if (comparator.compare(term, lastTerm) > 0) {
+      if (term.compareTo(lastTerm) > 0) {
         return AcceptStatus.END;
       }
 
       BytesRef currentTerm = terms.get(ords[upto], spare);
-      if (comparator.compare(term, currentTerm) == 0) {
+      if (term.compareTo(currentTerm) == 0) {
         if (upto == lastElement) {
           return AcceptStatus.YES;
         } else {
@@ -148,7 +145,7 @@ class TermsQuery extends MultiTermQuery {
             // typically the terms dict is a superset of query's terms so it's unusual that we have to skip many of
             // our terms so we don't do a binary search here
             seekTerm = terms.get(ords[++upto], spare);
-          } while ((cmp = comparator.compare(seekTerm, term)) < 0);
+          } while ((cmp = seekTerm.compareTo(term)) < 0);
           if (cmp == 0) {
             if (upto == lastElement) {
               return AcceptStatus.YES;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f48d23cd/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java
----------------------------------------------------------------------
diff --git a/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java b/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java
index 44fd401..9ab886b 100644
--- a/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java
+++ b/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java
@@ -1130,7 +1130,7 @@ public class TestJoinUtil extends LuceneTestCase {
         Terms terms = slowCompositeReader.terms(toField);
         if (terms != null) {
           PostingsEnum postingsEnum = null;
-          SortedSet<BytesRef> joinValues = new TreeSet<>(BytesRef.getUTF8SortedAsUnicodeComparator());
+          SortedSet<BytesRef> joinValues = new TreeSet<>();
           joinValues.addAll(joinValueToJoinScores.keySet());
           for (BytesRef joinValue : joinValues) {
             TermsEnum termsEnum = terms.iterator();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f48d23cd/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
----------------------------------------------------------------------
diff --git a/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java b/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
index c666724..849cd63 100644
--- a/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
+++ b/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
@@ -19,7 +19,6 @@ package org.apache.lucene.index.memory;
 import java.io.IOException;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.Comparator;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.SortedMap;
@@ -950,12 +949,12 @@ public class MemoryIndex {
       }
       
       private final int binarySearch(BytesRef b, BytesRef bytesRef, int low,
-          int high, BytesRefHash hash, int[] ords, Comparator<BytesRef> comparator) {
+          int high, BytesRefHash hash, int[] ords) {
         int mid = 0;
         while (low <= high) {
           mid = (low + high) >>> 1;
           hash.get(ords[mid], bytesRef);
-          final int cmp = comparator.compare(bytesRef, b);
+          final int cmp = bytesRef.compareTo(b);
           if (cmp < 0) {
             low = mid + 1;
           } else if (cmp > 0) {
@@ -964,20 +963,20 @@ public class MemoryIndex {
             return mid;
           }
         }
-        assert comparator.compare(bytesRef, b) != 0;
+        assert bytesRef.compareTo(b) != 0;
         return -(low + 1);
       }
     
 
       @Override
       public boolean seekExact(BytesRef text) {
-        termUpto = binarySearch(text, br, 0, info.terms.size()-1, info.terms, info.sortedTerms, BytesRef.getUTF8SortedAsUnicodeComparator());
+        termUpto = binarySearch(text, br, 0, info.terms.size()-1, info.terms, info.sortedTerms);
         return termUpto >= 0;
       }
 
       @Override
       public SeekStatus seekCeil(BytesRef text) {
-        termUpto = binarySearch(text, br, 0, info.terms.size()-1, info.terms, info.sortedTerms, BytesRef.getUTF8SortedAsUnicodeComparator());
+        termUpto = binarySearch(text, br, 0, info.terms.size()-1, info.terms, info.sortedTerms);
         if (termUpto < 0) { // not found; choose successor
           termUpto = -termUpto-1;
           if (termUpto >= info.terms.size()) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f48d23cd/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesTermsQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesTermsQuery.java b/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesTermsQuery.java
index 5faddc1..4dd13cb 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesTermsQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesTermsQuery.java
@@ -98,7 +98,7 @@ public class DocValuesTermsQuery extends Query {
     this.field = Objects.requireNonNull(field);
     Objects.requireNonNull(terms, "Collection of terms must not be null");
     this.terms = terms.toArray(new BytesRef[terms.size()]);
-    ArrayUtil.timSort(this.terms, BytesRef.getUTF8SortedAsUnicodeComparator());
+    ArrayUtil.timSort(this.terms);
   }
 
   public DocValuesTermsQuery(String field, BytesRef... terms) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f48d23cd/lucene/suggest/src/java/org/apache/lucene/search/suggest/SortedInputIterator.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/SortedInputIterator.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/SortedInputIterator.java
index 854a481..2be1759 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/SortedInputIterator.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/SortedInputIterator.java
@@ -57,11 +57,10 @@ public class SortedInputIterator implements InputIterator {
   private Set<BytesRef> contexts = null;
   
   /**
-   * Creates a new sorted wrapper, using {@link
-   * BytesRef#getUTF8SortedAsUnicodeComparator} for
-   * sorting. */
+   * Creates a new sorted wrapper, using {@linkplain Comparator#naturalOrder() natural order}
+   * for sorting. */
   public SortedInputIterator(Directory tempDir, String tempFileNamePrefix, InputIterator source) throws IOException {
-    this(tempDir, tempFileNamePrefix, source, BytesRef.getUTF8SortedAsUnicodeComparator());
+    this(tempDir, tempFileNamePrefix, source, Comparator.naturalOrder());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f48d23cd/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java
index 28670af..241525d 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java
@@ -18,6 +18,7 @@ package org.apache.lucene.search.suggest.fst;
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.util.Comparator;
 
 import org.apache.lucene.search.suggest.InMemorySorter;
 import org.apache.lucene.util.BytesRef;
@@ -148,10 +149,10 @@ public class FSTCompletionBuilder {
   /**
    * Creates an {@link FSTCompletion} with default options: 10 buckets, exact match
    * promoted to first position and {@link InMemorySorter} with a comparator obtained from
-   * {@link BytesRef#getUTF8SortedAsUnicodeComparator()}.
+   * {@link Comparator#naturalOrder()}.
    */
   public FSTCompletionBuilder() {
-    this(DEFAULT_BUCKETS, new InMemorySorter(BytesRef.getUTF8SortedAsUnicodeComparator()), Integer.MAX_VALUE);
+    this(DEFAULT_BUCKETS, new InMemorySorter(Comparator.naturalOrder()), Integer.MAX_VALUE);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f48d23cd/lucene/suggest/src/java/org/apache/lucene/search/suggest/tst/TSTLookup.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/tst/TSTLookup.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/tst/TSTLookup.java
index 1196917..888882a 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/tst/TSTLookup.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/tst/TSTLookup.java
@@ -18,6 +18,7 @@ package org.apache.lucene.search.suggest.tst;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Comparator;
 import java.util.List;
 import java.util.Set;
 
@@ -63,6 +64,54 @@ public class TSTLookup extends Lookup {
     this.tempDir = tempDir;
     this.tempFileNamePrefix = tempFileNamePrefix;
   }
+  
+  // TODO: Review if this comparator is really needed for TST to work correctly!!!
+
+  /** TST uses UTF-16 sorting, so we need a suitable BytesRef comparator to do this. */
+  private final static Comparator<BytesRef> utf8SortedAsUTF16SortOrder = (a, b) -> {
+    final byte[] aBytes = a.bytes;
+    int aUpto = a.offset;
+    final byte[] bBytes = b.bytes;
+    int bUpto = b.offset;
+    
+    final int aStop;
+    if (a.length < b.length) {
+      aStop = aUpto + a.length;
+    } else {
+      aStop = aUpto + b.length;
+    }
+
+    while(aUpto < aStop) {
+      int aByte = aBytes[aUpto++] & 0xff;
+      int bByte = bBytes[bUpto++] & 0xff;
+
+      if (aByte != bByte) {
+
+        // See http://icu-project.org/docs/papers/utf16_code_point_order.html#utf-8-in-utf-16-order
+
+        // We know the terms are not equal, but, we may
+        // have to carefully fixup the bytes at the
+        // difference to match UTF16's sort order:
+        
+        // NOTE: instead of moving supplementary code points (0xee and 0xef) to the unused 0xfe and 0xff, 
+        // we move them to the unused 0xfc and 0xfd [reserved for future 6-byte character sequences]
+        // this reserves 0xff for preflex's term reordering (surrogate dance), and if unicode grows such
+        // that 6-byte sequences are needed we have much bigger problems anyway.
+        if (aByte >= 0xee && bByte >= 0xee) {
+          if ((aByte & 0xfe) == 0xee) {
+            aByte += 0xe;
+          }
+          if ((bByte&0xfe) == 0xee) {
+            bByte += 0xe;
+          }
+        }
+        return aByte - bByte;
+      }
+    }
+
+    // One is a prefix of the other, or, they are equal:
+    return a.length - b.length;
+  };
 
   @Override
   public void build(InputIterator iterator) throws IOException {
@@ -75,7 +124,7 @@ public class TSTLookup extends Lookup {
     root = new TernaryTreeNode();
 
     // make sure it's sorted and the comparator uses UTF16 sort order
-    iterator = new SortedInputIterator(tempDir, tempFileNamePrefix, iterator, BytesRef.getUTF8SortedAsUTF16Comparator());
+    iterator = new SortedInputIterator(tempDir, tempFileNamePrefix, iterator, utf8SortedAsUTF16SortOrder);
     count = 0;
     ArrayList<String> tokens = new ArrayList<>();
     ArrayList<Number> vals = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f48d23cd/lucene/suggest/src/test/org/apache/lucene/search/suggest/TestInputIterator.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/TestInputIterator.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/TestInputIterator.java
index 0c80bb3..ecb77bd 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/TestInputIterator.java
+++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/TestInputIterator.java
@@ -17,7 +17,6 @@
 package org.apache.lucene.search.suggest;
 
 import java.util.AbstractMap.SimpleEntry;
-import java.util.Comparator;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Map;
@@ -26,7 +25,6 @@ import java.util.Set;
 import java.util.TreeMap;
 
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
@@ -36,7 +34,7 @@ public class TestInputIterator extends LuceneTestCase {
   public void testEmpty() throws Exception {
     InputArrayIterator iterator = new InputArrayIterator(new Input[0]);
     try (Directory dir = getDirectory()) {
-      InputIterator wrapper = new SortedInputIterator(dir, "sorted", iterator, BytesRef.getUTF8SortedAsUnicodeComparator());
+      InputIterator wrapper = new SortedInputIterator(dir, "sorted", iterator);
       assertNull(wrapper.next());
       wrapper = new UnsortedInputIterator(iterator);
       assertNull(wrapper.next());
@@ -47,11 +45,10 @@ public class TestInputIterator extends LuceneTestCase {
     Random random = random();
     int num = atLeast(10000);
     
-    Comparator<BytesRef> comparator = random.nextBoolean() ? BytesRef.getUTF8SortedAsUnicodeComparator() : BytesRef.getUTF8SortedAsUTF16Comparator();
-    TreeMap<BytesRef, SimpleEntry<Long, BytesRef>> sorted = new TreeMap<>(comparator);
-    TreeMap<BytesRef, Long> sortedWithoutPayload = new TreeMap<>(comparator);
-    TreeMap<BytesRef, SimpleEntry<Long, Set<BytesRef>>> sortedWithContext = new TreeMap<>(comparator);
-    TreeMap<BytesRef, SimpleEntry<Long, SimpleEntry<BytesRef, Set<BytesRef>>>> sortedWithPayloadAndContext = new TreeMap<>(comparator);
+    TreeMap<BytesRef, SimpleEntry<Long, BytesRef>> sorted = new TreeMap<>();
+    TreeMap<BytesRef, Long> sortedWithoutPayload = new TreeMap<>();
+    TreeMap<BytesRef, SimpleEntry<Long, Set<BytesRef>>> sortedWithContext = new TreeMap<>();
+    TreeMap<BytesRef, SimpleEntry<Long, SimpleEntry<BytesRef, Set<BytesRef>>>> sortedWithPayloadAndContext = new TreeMap<>();
     Input[] unsorted = new Input[num];
     Input[] unsortedWithoutPayload = new Input[num];
     Input[] unsortedWithContexts = new Input[num];
@@ -81,7 +78,7 @@ public class TestInputIterator extends LuceneTestCase {
     
     // test the sorted iterator wrapper with payloads
     try (Directory tempDir = getDirectory()) {
-      InputIterator wrapper = new SortedInputIterator(tempDir, "sorted", new InputArrayIterator(unsorted), comparator);
+      InputIterator wrapper = new SortedInputIterator(tempDir, "sorted", new InputArrayIterator(unsorted));
       Iterator<Map.Entry<BytesRef, SimpleEntry<Long, BytesRef>>> expected = sorted.entrySet().iterator();
       while (expected.hasNext()) {
         Map.Entry<BytesRef,SimpleEntry<Long, BytesRef>> entry = expected.next();
@@ -95,7 +92,7 @@ public class TestInputIterator extends LuceneTestCase {
     
     // test the sorted iterator wrapper with contexts
     try (Directory tempDir = getDirectory()) {
-      InputIterator wrapper = new SortedInputIterator(tempDir, "sorted", new InputArrayIterator(unsortedWithContexts), comparator);
+      InputIterator wrapper = new SortedInputIterator(tempDir, "sorted", new InputArrayIterator(unsortedWithContexts));
       Iterator<Map.Entry<BytesRef, SimpleEntry<Long, Set<BytesRef>>>> actualEntries = sortedWithContext.entrySet().iterator();
       while (actualEntries.hasNext()) {
         Map.Entry<BytesRef, SimpleEntry<Long, Set<BytesRef>>> entry = actualEntries.next();
@@ -109,7 +106,7 @@ public class TestInputIterator extends LuceneTestCase {
 
     // test the sorted iterator wrapper with contexts and payload
     try (Directory tempDir = getDirectory()) {
-      InputIterator wrapper = new SortedInputIterator(tempDir, "sorter", new InputArrayIterator(unsortedWithPayloadAndContext), comparator);
+      InputIterator wrapper = new SortedInputIterator(tempDir, "sorter", new InputArrayIterator(unsortedWithPayloadAndContext));
       Iterator<Map.Entry<BytesRef, SimpleEntry<Long, SimpleEntry<BytesRef, Set<BytesRef>>>>> expectedPayloadContextEntries = sortedWithPayloadAndContext.entrySet().iterator();
       while (expectedPayloadContextEntries.hasNext()) {
         Map.Entry<BytesRef, SimpleEntry<Long, SimpleEntry<BytesRef, Set<BytesRef>>>> entry = expectedPayloadContextEntries.next();
@@ -136,7 +133,7 @@ public class TestInputIterator extends LuceneTestCase {
 
     // test the sorted iterator wrapper without payloads
     try (Directory tempDir = getDirectory()) {
-      InputIterator wrapperWithoutPayload = new SortedInputIterator(tempDir, "sorted", new InputArrayIterator(unsortedWithoutPayload), comparator);
+      InputIterator wrapperWithoutPayload = new SortedInputIterator(tempDir, "sorted", new InputArrayIterator(unsortedWithoutPayload));
       Iterator<Map.Entry<BytesRef, Long>> expectedWithoutPayload = sortedWithoutPayload.entrySet().iterator();
       while (expectedWithoutPayload.hasNext()) {
         Map.Entry<BytesRef, Long> entry = expectedWithoutPayload.next();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f48d23cd/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/BytesRefSortersTest.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/BytesRefSortersTest.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/BytesRefSortersTest.java
index 1006700..b64d283 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/BytesRefSortersTest.java
+++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/BytesRefSortersTest.java
@@ -16,9 +16,10 @@
  */
 package org.apache.lucene.search.suggest.fst;
 
+import java.util.Comparator;
+
 import org.apache.lucene.search.suggest.InMemorySorter;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefIterator;
 import org.apache.lucene.util.IOUtils;
@@ -37,7 +38,7 @@ public class BytesRefSortersTest extends LuceneTestCase {
 
   @Test
   public void testInMemorySorter() throws Exception {
-    check(new InMemorySorter(BytesRef.getUTF8SortedAsUnicodeComparator()));
+    check(new InMemorySorter(Comparator.naturalOrder()));
   }
 
   private void check(BytesRefSorter sorter) throws Exception {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f48d23cd/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java b/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
index efbed63..15580d8 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
@@ -32,6 +32,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Comparator;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
@@ -769,6 +770,26 @@ public final class TestUtil {
     0x1D24F, 0x1D35F, 0x1D37F, 0x1D7FF, 0x1F02F, 0x1F09F, 0x1F1FF, 0x1F2FF, 
     0x2A6DF, 0x2B73F, 0x2FA1F, 0xE007F, 0xE01EF, 0xFFFFF, 0x10FFFF
   };
+
+  /** 
+   * A comparator that compares strings according to Unicode code point order.
+   * This can be used to verify {@link BytesRef} order. 
+   * <p>
+   * <b>Warning:</b> This comparator is rather inefficient, because
+   * it converts the strings to a {@code int[]} array on each invocation.
+   * */
+  public static final Comparator<String> STRING_CODEPOINT_COMPARATOR = (a, b) -> {
+    final int[] aCodePoints = a.codePoints().toArray();
+    final int[] bCodePoints = b.codePoints().toArray();
+    for(int i = 0, c = Math.min(aCodePoints.length, bCodePoints.length); i < c; i++) {
+      if (aCodePoints[i] < bCodePoints[i]) {
+        return -1;
+      } else if (aCodePoints[i] > bCodePoints[i]) {
+        return 1;
+      }
+    }
+    return aCodePoints.length - bCodePoints.length;
+  };
   
   /** Returns random string of length between 0-20 codepoints, all codepoints within the same unicode block. */
   public static String randomRealisticUnicodeString(Random r) {