You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by be...@apache.org on 2015/03/10 13:44:55 UTC

cassandra git commit: Optimise (Time)?UUIDType comparisons

Repository: cassandra
Updated Branches:
  refs/heads/trunk dce6033d8 -> 6d266253a


Optimise (Time)?UUIDType comparisons

patch by benedict; reviewed by ariel for CASSANDRA-8730


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/6d266253
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6d266253
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6d266253

Branch: refs/heads/trunk
Commit: 6d266253a5bdaf3a25eef14e54deb56aba9b2944
Parents: dce6033
Author: Benedict Elliott Smith <be...@apache.org>
Authored: Tue Mar 10 12:43:52 2015 +0000
Committer: Benedict Elliott Smith <be...@apache.org>
Committed: Tue Mar 10 12:43:52 2015 +0000

----------------------------------------------------------------------
 CHANGES.txt                                     |   2 +-
 .../cassandra/db/marshal/TimeUUIDType.java      |  72 +++--
 .../apache/cassandra/db/marshal/UUIDType.java   | 113 ++------
 .../cassandra/db/marshal/TimeUUIDTypeTest.java  |  83 ++++--
 .../cassandra/db/marshal/UUIDTypeTest.java      | 267 +++++++++++++++++--
 5 files changed, 388 insertions(+), 149 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d266253/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index b9aa8a8..d326313 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0
+ * Optimise (Time)?UUIDType Comparisons (CASSANDRA-8730)
  * Make CRC32Ex into a separate maven dependency (CASSANDRA-8836)
  * Use preloaded jemalloc w/ Unsafe (CASSANDRA-8714)
  * Add role based access control (CASSANDRA-7653, 8650, 7216, 8760, 8849, 8761)
@@ -60,7 +61,6 @@
  * Select optimal CRC32 implementation at runtime (CASSANDRA-8614)
  * Evaluate MurmurHash of Token once per query (CASSANDRA-7096)
 
-
 2.1.4
  * Add SimpleDate (cql date) and Time (cql time) types (CASSANDRA-7523)
  * Use long for key count in cfstats (CASSANDRA-8913)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d266253/src/java/org/apache/cassandra/db/marshal/TimeUUIDType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/TimeUUIDType.java b/src/java/org/apache/cassandra/db/marshal/TimeUUIDType.java
index 573b368..17c7abc 100644
--- a/src/java/org/apache/cassandra/db/marshal/TimeUUIDType.java
+++ b/src/java/org/apache/cassandra/db/marshal/TimeUUIDType.java
@@ -37,44 +37,58 @@ public class TimeUUIDType extends AbstractType<UUID>
     {
     } // singleton
 
-    public int compare(ByteBuffer o1, ByteBuffer o2)
+    public int compare(ByteBuffer b1, ByteBuffer b2)
     {
-        if (!o1.hasRemaining() || !o2.hasRemaining())
-            return o1.hasRemaining() ? 1 : o2.hasRemaining() ? -1 : 0;
+        // Compare for length
+        int s1 = b1.position(), s2 = b2.position();
+        int l1 = b1.limit(), l2 = b2.limit();
 
-        int res = compareTimestampBytes(o1, o2);
-        if (res != 0)
-            return res;
-        return o1.compareTo(o2);
-    }
-
-    private static int compareTimestampBytes(ByteBuffer o1, ByteBuffer o2)
-    {
-        int o1Pos = o1.position();
-        int o2Pos = o2.position();
-
-        int d = (o1.get(o1Pos + 6) & 0xF) - (o2.get(o2Pos + 6) & 0xF);
-        if (d != 0) return d;
+        // should we assert exactly 16 bytes (or 0)? seems prudent
+        boolean p1 = l1 - s1 == 16, p2 = l2 - s2 == 16;
+        if (!(p1 & p2))
+        {
+            assert p1 | (l1 == s1);
+            assert p2 | (l2 == s2);
+            return p1 ? 1 : p2 ? -1 : 0;
+        }
 
-        d = (o1.get(o1Pos + 7) & 0xFF) - (o2.get(o2Pos + 7) & 0xFF);
-        if (d != 0) return d;
+        long msb1 = b1.getLong(s1);
+        long msb2 = b2.getLong(s2);
+        msb1 = reorderTimestampBytes(msb1);
+        msb2 = reorderTimestampBytes(msb2);
 
-        d = (o1.get(o1Pos + 4) & 0xFF) - (o2.get(o2Pos + 4) & 0xFF);
-        if (d != 0) return d;
+        assert (msb1 & topbyte(0xf0L)) == topbyte(0x10L);
+        assert (msb2 & topbyte(0xf0L)) == topbyte(0x10L);
 
-        d = (o1.get(o1Pos + 5) & 0xFF) - (o2.get(o2Pos + 5) & 0xFF);
-        if (d != 0) return d;
+        int c = Long.compare(msb1, msb2);
+        if (c != 0)
+            return c;
 
-        d = (o1.get(o1Pos) & 0xFF) - (o2.get(o2Pos) & 0xFF);
-        if (d != 0) return d;
+        // this has to be a signed per-byte comparison for compatibility
+        // so we transform the bytes so that a simple long comparison is equivalent
+        long lsb1 = signedBytesToNativeLong(b1.getLong(s1 + 8));
+        long lsb2 = signedBytesToNativeLong(b2.getLong(s2 + 8));
+        return Long.compare(lsb1, lsb2);
+    }
 
-        d = (o1.get(o1Pos + 1) & 0xFF) - (o2.get(o2Pos + 1) & 0xFF);
-        if (d != 0) return d;
+    // takes as input 8 signed bytes in native machine order
+    // returns the first byte unchanged, and the following 7 bytes converted to an unsigned representation
+    // which is the same as a 2's complement long in native format
+    private static long signedBytesToNativeLong(long signedBytes)
+    {
+        return signedBytes ^ 0x0080808080808080L;
+    }
 
-        d = (o1.get(o1Pos + 2) & 0xFF) - (o2.get(o2Pos + 2) & 0xFF);
-        if (d != 0) return d;
+    private static long topbyte(long topbyte)
+    {
+        return topbyte << 56;
+    }
 
-        return (o1.get(o1Pos + 3) & 0xFF) - (o2.get(o2Pos + 3) & 0xFF);
+    protected static long reorderTimestampBytes(long input)
+    {
+        return    (input <<  48)
+                  | ((input <<  16) & 0xFFFF00000000L)
+                  |  (input >>> 32);
     }
 
     public ByteBuffer fromString(String source) throws MarshalException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d266253/src/java/org/apache/cassandra/db/marshal/UUIDType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/UUIDType.java b/src/java/org/apache/cassandra/db/marshal/UUIDType.java
index 83fa2ba..c925ea0 100644
--- a/src/java/org/apache/cassandra/db/marshal/UUIDType.java
+++ b/src/java/org/apache/cassandra/db/marshal/UUIDType.java
@@ -20,6 +20,8 @@ package org.apache.cassandra.db.marshal;
 import java.nio.ByteBuffer;
 import java.util.UUID;
 
+import com.google.common.primitives.UnsignedLongs;
+
 import org.apache.cassandra.cql3.CQL3Type;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.serializers.MarshalException;
@@ -47,110 +49,47 @@ public class UUIDType extends AbstractType<UUID>
 
     public int compare(ByteBuffer b1, ByteBuffer b2)
     {
-
         // Compare for length
+        int s1 = b1.position(), s2 = b2.position();
+        int l1 = b1.limit(), l2 = b2.limit();
 
-        if ((b1 == null) || (b1.remaining() < 16))
+        // should we assert exactly 16 bytes (or 0)? seems prudent
+        boolean p1 = l1 - s1 == 16, p2 = l2 - s2 == 16;
+        if (!(p1 & p2))
         {
-            return ((b2 == null) || (b2.remaining() < 16)) ? 0 : -1;
+            assert p1 | (l1 == s1);
+            assert p2 | (l2 == s2);
+            return p1 ? 1 : p2 ? -1 : 0;
         }
-        if ((b2 == null) || (b2.remaining() < 16))
-        {
-            return 1;
-        }
-
-        int s1 = b1.position();
-        int s2 = b2.position();
 
         // Compare versions
+        long msb1 = b1.getLong(s1);
+        long msb2 = b2.getLong(s2);
 
-        int v1 = (b1.get(s1 + 6) >> 4) & 0x0f;
-        int v2 = (b2.get(s2 + 6) >> 4) & 0x0f;
+        int version1 = (int) ((msb1 >>> 12) & 0xf);
+        int version2 = (int) ((msb2 >>> 12) & 0xf);
+        if (version1 != version2)
+            return version1 - version2;
 
-        if (v1 != v2)
+        // bytes: version is top 4 bits of byte 6
+        // then: [6.5-8), [4-6), [0-4)
+        if (version1 == 1)
         {
-            return v1 - v2;
-        }
-
-        // Compare timestamps for version 1
-
-        if (v1 == 1)
-        {
-            // if both time-based, compare as timestamps
-            int c = compareTimestampBytes(b1, b2);
+            long reorder1 = TimeUUIDType.reorderTimestampBytes(msb1);
+            long reorder2 = TimeUUIDType.reorderTimestampBytes(msb2);
+            // we know this is >= 0, since the top 3 bits will be 0
+            int c = Long.compare(reorder1, reorder2);
             if (c != 0)
-            {
                 return c;
-            }
         }
-
-        // Compare the two byte arrays starting from the first
-        // byte in the sequence until an inequality is
-        // found. This should provide equivalent results
-        // to the comparison performed by the RFC 4122
-        // Appendix A - Sample Implementation.
-        // Note: java.util.UUID.compareTo is not a lexical
-        // comparison
-        for (int i = 0; i < 16; i++)
+        else
         {
-            int c = ((b1.get(s1 + i)) & 0xFF) - ((b2.get(s2 + i)) & 0xFF);
+            int c = UnsignedLongs.compare(msb1, msb2);
             if (c != 0)
-            {
                 return c;
-            }
-        }
-
-        return 0;
-    }
-
-    private static int compareTimestampBytes(ByteBuffer o1, ByteBuffer o2)
-    {
-        int o1Pos = o1.position();
-        int o2Pos = o2.position();
-
-        int d = (o1.get(o1Pos + 6) & 0xF) - (o2.get(o2Pos + 6) & 0xF);
-        if (d != 0)
-        {
-            return d;
-        }
-
-        d = (o1.get(o1Pos + 7) & 0xFF) - (o2.get(o2Pos + 7) & 0xFF);
-        if (d != 0)
-        {
-            return d;
-        }
-
-        d = (o1.get(o1Pos + 4) & 0xFF) - (o2.get(o2Pos + 4) & 0xFF);
-        if (d != 0)
-        {
-            return d;
-        }
-
-        d = (o1.get(o1Pos + 5) & 0xFF) - (o2.get(o2Pos + 5) & 0xFF);
-        if (d != 0)
-        {
-            return d;
-        }
-
-        d = (o1.get(o1Pos) & 0xFF) - (o2.get(o2Pos) & 0xFF);
-        if (d != 0)
-        {
-            return d;
-        }
-
-        d = (o1.get(o1Pos + 1) & 0xFF) - (o2.get(o2Pos + 1) & 0xFF);
-        if (d != 0)
-        {
-            return d;
-        }
-
-        d = (o1.get(o1Pos + 2) & 0xFF) - (o2.get(o2Pos + 2) & 0xFF);
-        if (d != 0)
-        {
-            return d;
         }
 
-        return (o1.get(o1Pos + 3) & 0xFF) - (o2.get(o2Pos + 3) & 0xFF);
+        return UnsignedLongs.compare(b1.getLong(s1 + 8), b2.getLong(s2 + 8));
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d266253/test/unit/org/apache/cassandra/db/marshal/TimeUUIDTypeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/marshal/TimeUUIDTypeTest.java b/test/unit/org/apache/cassandra/db/marshal/TimeUUIDTypeTest.java
index 4e22df8..0054163 100644
--- a/test/unit/org/apache/cassandra/db/marshal/TimeUUIDTypeTest.java
+++ b/test/unit/org/apache/cassandra/db/marshal/TimeUUIDTypeTest.java
@@ -22,7 +22,9 @@ import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.Random;
 import java.util.UUID;
+import java.util.concurrent.ThreadLocalRandom;
 
+import junit.framework.Assert;
 import org.apache.cassandra.serializers.MarshalException;
 import org.junit.Test;
 import static org.junit.Assert.assertEquals;
@@ -79,25 +81,76 @@ public class TimeUUIDTypeTest
     @Test
     public void testTimestampComparison()
     {
-        Random rng = new Random();
-        ByteBuffer[] uuids = new ByteBuffer[100];
-        for (int i = 0; i < uuids.length; i++)
-        {
-            uuids[i] = ByteBuffer.allocate(16);
-            rng.nextBytes(uuids[i].array());
-            // set version to 1
-            uuids[i].array()[6] &= 0x0F;
-            uuids[i].array()[6] |= 0x10;
-        }
-        Arrays.sort(uuids, timeUUIDType);
-        for (int i = 1; i < uuids.length; i++)
+        compareAll(UUIDTypeTest.random(1000, (byte) 0x10));
+        for (ByteBuffer[] permutations : UUIDTypeTest.permutations(100, (byte) 0x10))
+            compareAll(permutations);
+    }
+
+    private void compareAll(ByteBuffer[] uuids)
+    {
+        for (int i = 0 ; i < uuids.length ; i++)
         {
-            long i0 = UUIDGen.getUUID(uuids[i - 1]).timestamp();
-            long i1 = UUIDGen.getUUID(uuids[i]).timestamp();
-            assert i0 <= i1;
+            for (int j = i + 1 ; j < uuids.length ; j++)
+            {
+                ByteBuffer bi = uuids[i];
+                ByteBuffer bj = uuids[j];
+                long i0 = UUIDGen.getUUID(bi).timestamp();
+                long i1 = UUIDGen.getUUID(bj).timestamp();
+                int c = timeUUIDType.compare(bi, bj);
+                if (i0 == i1) Assert.assertTrue(isComparisonEquivalent(bi.compareTo(bj), c));
+                else Assert.assertTrue(isComparisonEquivalent(Long.compare(i0, i1), c));
+                Assert.assertTrue(isComparisonEquivalent(compareV1(bi, bj), c));
+            }
         }
     }
 
+    private static int compareV1(ByteBuffer o1, ByteBuffer o2)
+    {
+        if (!o1.hasRemaining() || !o2.hasRemaining())
+            return o1.hasRemaining() ? 1 : o2.hasRemaining() ? -1 : 0;
+
+        int res = compareTimestampBytes(o1, o2);
+        if (res != 0)
+            return res;
+        return o1.compareTo(o2);
+    }
+
+    private static int compareTimestampBytes(ByteBuffer o1, ByteBuffer o2)
+    {
+        int o1Pos = o1.position();
+        int o2Pos = o2.position();
+
+        int d = (o1.get(o1Pos + 6) & 0xF) - (o2.get(o2Pos + 6) & 0xF);
+        if (d != 0) return d;
+
+        d = (o1.get(o1Pos + 7) & 0xFF) - (o2.get(o2Pos + 7) & 0xFF);
+        if (d != 0) return d;
+
+        d = (o1.get(o1Pos + 4) & 0xFF) - (o2.get(o2Pos + 4) & 0xFF);
+        if (d != 0) return d;
+
+        d = (o1.get(o1Pos + 5) & 0xFF) - (o2.get(o2Pos + 5) & 0xFF);
+        if (d != 0) return d;
+
+        d = (o1.get(o1Pos) & 0xFF) - (o2.get(o2Pos) & 0xFF);
+        if (d != 0) return d;
+
+        d = (o1.get(o1Pos + 1) & 0xFF) - (o2.get(o2Pos + 1) & 0xFF);
+        if (d != 0) return d;
+
+        d = (o1.get(o1Pos + 2) & 0xFF) - (o2.get(o2Pos + 2) & 0xFF);
+        if (d != 0) return d;
+
+        return (o1.get(o1Pos + 3) & 0xFF) - (o2.get(o2Pos + 3) & 0xFF);
+    }
+
+    private static boolean isComparisonEquivalent(int c1, int c2)
+    {
+        c1 = c1 < -1 ? -1 : c1 > 1 ? 1 : c1;
+        c2 = c2 < -1 ? -1 : c2 > 1 ? 1 : c2;
+        return c1 == c2;
+    }
+
     @Test
     public void testValidTimeVersion()
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d266253/test/unit/org/apache/cassandra/db/marshal/UUIDTypeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/marshal/UUIDTypeTest.java b/test/unit/org/apache/cassandra/db/marshal/UUIDTypeTest.java
index 1ecacf3..335860c 100644
--- a/test/unit/org/apache/cassandra/db/marshal/UUIDTypeTest.java
+++ b/test/unit/org/apache/cassandra/db/marshal/UUIDTypeTest.java
@@ -24,15 +24,17 @@ package org.apache.cassandra.db.marshal;
 import static org.junit.Assert.assertEquals;
 
 import java.nio.ByteBuffer;
-import java.util.Arrays;
+import java.util.Iterator;
 import java.util.Random;
 import java.util.UUID;
 
+import org.junit.Test;
+
+import junit.framework.Assert;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.UUIDGen;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.junit.Test;
 
 public class UUIDTypeTest
 {
@@ -42,7 +44,7 @@ public class UUIDTypeTest
     UUIDType uuidType = new UUIDType();
 
     @Test
-    public void testCompare()
+    public void testRandomCompare()
     {
 
         UUID t1 = UUIDGen.getTimeUUID();
@@ -201,24 +203,255 @@ public class UUIDTypeTest
     }
 
     @Test
-    public void testTimestampComparison()
+    public void testPermutations()
+    {
+        compareAll(random(1000, (byte) 0x00, (byte) 0x10, (byte) 0x20));
+        for (ByteBuffer[] permutations : permutations(10,  (byte) 0x00, (byte) 0x10, (byte) 0x20))
+            compareAll(permutations);
+    }
+
+    private void compareAll(ByteBuffer[] uuids)
     {
-        Random rng = new Random();
-        ByteBuffer[] uuids = new ByteBuffer[100];
-        for (int i = 0; i < uuids.length; i++)
+        for (int i = 0 ; i < uuids.length ; i++)
         {
-            uuids[i] = ByteBuffer.allocate(16);
-            rng.nextBytes(uuids[i].array());
-            // set version to 1
-            uuids[i].array()[6] &= 0x0F;
-            uuids[i].array()[6] |= 0x10;
+            for (int j = i + 1 ; j < uuids.length ; j++)
+            {
+                ByteBuffer bi = uuids[i];
+                ByteBuffer bj = uuids[j];
+                UUID ui = UUIDGen.getUUID(bi);
+                UUID uj = UUIDGen.getUUID(bj);
+                int c = uuidType.compare(bi, bj);
+                if (ui.version() != uj.version())
+                {
+                    Assert.assertTrue(isComparisonEquivalent(ui.version() - uj.version(), c));
+                }
+                else if (ui.version() == 1)
+                {
+                    long i0 = ui.timestamp();
+                    long i1 = uj.timestamp();
+                    if (i0 == i1) Assert.assertTrue(isComparisonEquivalent(ByteBufferUtil.compareUnsigned(bi, bj), c));
+                    else Assert.assertTrue(isComparisonEquivalent(Long.compare(i0, i1), c));
+                }
+                else
+                {
+                    Assert.assertTrue(isComparisonEquivalent(ByteBufferUtil.compareUnsigned(bi, bj), c));
+                }
+                Assert.assertTrue(isComparisonEquivalent(compareV1(bi, bj), c));
+            }
         }
-        Arrays.sort(uuids, uuidType);
-        for (int i = 1; i < uuids.length; i++)
+    }
+
+    private static boolean isComparisonEquivalent(int c1, int c2)
+    {
+        c1 = c1 < -1 ? -1 : c1 > 1 ? 1 : c1;
+        c2 = c2 < -1 ? -1 : c2 > 1 ? 1 : c2;
+        return c1 == c2;
+    }
+
+    // produce randomCount random byte strings, and permute every possible byte within each
+    // for all provided types, using permute()
+    static Iterable<ByteBuffer[]> permutations(final int randomCount, final byte ... types)
+    {
+        final Random random = new Random();
+        long seed = random.nextLong();
+        random.setSeed(seed);
+        System.out.println("UUIDTypeTest.permutations.seed=" + seed);
+        return new Iterable<ByteBuffer[]>()
         {
-            long i0 = UUIDGen.getUUID(uuids[i - 1]).timestamp();
-            long i1 = UUIDGen.getUUID(uuids[i]).timestamp();
-            assert i0 <= i1;
+            public Iterator<ByteBuffer[]> iterator()
+            {
+                return new Iterator<ByteBuffer[]>()
+                {
+                    byte[] bytes = new byte[16];
+                    int c = -1, i = 16;
+                    public boolean hasNext()
+                    {
+                        return i < 16 || c < randomCount - 1;
+                    }
+
+                    public ByteBuffer[] next()
+                    {
+                        if (i == 16)
+                        {
+                            random.nextBytes(bytes);
+                            i = 0;
+                            c++;
+                        }
+                        return permute(bytes, i++, types);
+                    }
+                    public void remove()
+                    {
+                    }
+                };
+            }
+        };
+    }
+
+    // for each of the given UUID types provided, produce every possible
+    // permutation of the provided byte[] for the given index
+    static ByteBuffer[] permute(byte[] src, int byteIndex, byte ... types)
+    {
+        assert src.length == 16;
+        assert byteIndex < 16;
+        byte[] bytes = src.clone();
+        ByteBuffer[] permute;
+        if (byteIndex == 6)
+        {
+            permute = new ByteBuffer[16 * types.length];
+            for (int i = 0 ; i < types.length ; i++)
+            {
+                for (int j = 0 ; j < 16 ; j++)
+                {
+                    int k = i * 16 + j;
+                    bytes[6] = (byte)(types[i] | j);
+                    permute[k] = ByteBuffer.wrap(bytes.clone());
+                }
+            }
+        }
+        else
+        {
+            permute = new ByteBuffer[256 * types.length];
+            for (int i = 0 ; i < types.length ; i++)
+            {
+                bytes[6] = types[i];
+                for (int j = 0 ; j < 256 ; j++)
+                {
+                    int k = i * 256 + j;
+                    bytes[byteIndex] = (byte) ((bytes[byteIndex] & 0x0F) | i);
+                    permute[k] = ByteBuffer.wrap(bytes.clone());
+                }
+            }
         }
+        return permute;
+    }
+
+    static ByteBuffer[] random(int count, byte ... types)
+    {
+        Random random = new Random();
+        long seed = random.nextLong();
+        random.setSeed(seed);
+        System.out.println("UUIDTypeTest.random.seed=" + seed);
+        ByteBuffer[] uuids = new ByteBuffer[count * types.length];
+        for (int i = 0 ; i < types.length ; i++)
+        {
+            for (int j = 0; j < count; j++)
+            {
+                int k = (i * count) + j;
+                uuids[k] = ByteBuffer.allocate(16);
+                random.nextBytes(uuids[k].array());
+                // set version to 1
+                uuids[k].array()[6] &= 0x0F;
+                uuids[k].array()[6] |= types[i];
+            }
+        }
+        return uuids;
+    }
+
+    private static int compareV1(ByteBuffer b1, ByteBuffer b2)
+    {
+
+        // Compare for length
+
+        if ((b1 == null) || (b1.remaining() < 16))
+        {
+            return ((b2 == null) || (b2.remaining() < 16)) ? 0 : -1;
+        }
+        if ((b2 == null) || (b2.remaining() < 16))
+        {
+            return 1;
+        }
+
+        int s1 = b1.position();
+        int s2 = b2.position();
+
+        // Compare versions
+
+        int v1 = (b1.get(s1 + 6) >> 4) & 0x0f;
+        int v2 = (b2.get(s2 + 6) >> 4) & 0x0f;
+
+        if (v1 != v2)
+        {
+            return v1 - v2;
+        }
+
+        // Compare timestamps for version 1
+
+        if (v1 == 1)
+        {
+            // if both time-based, compare as timestamps
+            int c = compareTimestampBytes(b1, b2);
+            if (c != 0)
+            {
+                return c;
+            }
+        }
+
+        // Compare the two byte arrays starting from the first
+        // byte in the sequence until an inequality is
+        // found. This should provide equivalent results
+        // to the comparison performed by the RFC 4122
+        // Appendix A - Sample Implementation.
+        // Note: java.util.UUID.compareTo is not a lexical
+        // comparison
+        for (int i = 0; i < 16; i++)
+        {
+            int c = ((b1.get(s1 + i)) & 0xFF) - ((b2.get(s2 + i)) & 0xFF);
+            if (c != 0)
+            {
+                return c;
+            }
+        }
+
+        return 0;
+    }
+
+    private static int compareTimestampBytes(ByteBuffer o1, ByteBuffer o2)
+    {
+        int o1Pos = o1.position();
+        int o2Pos = o2.position();
+
+        int d = (o1.get(o1Pos + 6) & 0xF) - (o2.get(o2Pos + 6) & 0xF);
+        if (d != 0)
+        {
+            return d;
+        }
+
+        d = (o1.get(o1Pos + 7) & 0xFF) - (o2.get(o2Pos + 7) & 0xFF);
+        if (d != 0)
+        {
+            return d;
+        }
+
+        d = (o1.get(o1Pos + 4) & 0xFF) - (o2.get(o2Pos + 4) & 0xFF);
+        if (d != 0)
+        {
+            return d;
+        }
+
+        d = (o1.get(o1Pos + 5) & 0xFF) - (o2.get(o2Pos + 5) & 0xFF);
+        if (d != 0)
+        {
+            return d;
+        }
+
+        d = (o1.get(o1Pos) & 0xFF) - (o2.get(o2Pos) & 0xFF);
+        if (d != 0)
+        {
+            return d;
+        }
+
+        d = (o1.get(o1Pos + 1) & 0xFF) - (o2.get(o2Pos + 1) & 0xFF);
+        if (d != 0)
+        {
+            return d;
+        }
+
+        d = (o1.get(o1Pos + 2) & 0xFF) - (o2.get(o2Pos + 2) & 0xFF);
+        if (d != 0)
+        {
+            return d;
+        }
+
+        return (o1.get(o1Pos + 3) & 0xFF) - (o2.get(o2Pos + 3) & 0xFF);
     }
 }