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/07/02 14:02:47 UTC

cassandra git commit: Precompute partial ColumnIdentifier comparison

Repository: cassandra
Updated Branches:
  refs/heads/trunk 07d38b03a -> 8c8103cae


Precompute partial ColumnIdentifier comparison

patch by benedict; reviewed by sylvain for CASSANDRA-9706


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

Branch: refs/heads/trunk
Commit: 8c8103cae53a62251d1d345bf88fd001cdefb92c
Parents: 07d38b0
Author: Benedict Elliott Smith <be...@apache.org>
Authored: Thu Jul 2 12:59:16 2015 +0100
Committer: Benedict Elliott Smith <be...@apache.org>
Committed: Thu Jul 2 13:02:35 2015 +0100

----------------------------------------------------------------------
 .../cassandra/config/ColumnDefinition.java      |  2 +-
 .../apache/cassandra/cql3/ColumnIdentifier.java | 38 +++++++++++-
 src/java/org/apache/cassandra/db/Columns.java   |  2 +-
 .../cassandra/db/filter/ColumnFilter.java       | 11 +---
 .../cassandra/cql3/ColumnIdentifierTest.java    | 61 ++++++++++++++++++++
 5 files changed, 102 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/8c8103ca/src/java/org/apache/cassandra/config/ColumnDefinition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/ColumnDefinition.java b/src/java/org/apache/cassandra/config/ColumnDefinition.java
index d6605a7..8448ca6 100644
--- a/src/java/org/apache/cassandra/config/ColumnDefinition.java
+++ b/src/java/org/apache/cassandra/config/ColumnDefinition.java
@@ -415,7 +415,7 @@ public class ColumnDefinition extends ColumnSpecification implements Comparable<
         if (comparisonOrder != other.comparisonOrder)
             return comparisonOrder - other.comparisonOrder;
 
-        return ByteBufferUtil.compareUnsigned(name.bytes, other.name.bytes);
+        return this.name.compareTo(other.name);
     }
 
     public Comparator<CellPath> cellPathComparator()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8c8103ca/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java b/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
index eafcf8d..47e4384 100644
--- a/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
+++ b/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
@@ -18,6 +18,7 @@
 package org.apache.cassandra.cql3;
 
 import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
 import java.util.List;
 import java.util.Locale;
 import java.nio.ByteBuffer;
@@ -43,20 +44,44 @@ import org.apache.cassandra.utils.memory.AbstractAllocator;
  * Represents an identifer for a CQL column definition.
  * TODO : should support light-weight mode without text representation for when not interned
  */
-public class ColumnIdentifier extends org.apache.cassandra.cql3.selection.Selectable implements IMeasurableMemory
+public class ColumnIdentifier extends org.apache.cassandra.cql3.selection.Selectable implements IMeasurableMemory, Comparable<ColumnIdentifier>
 {
     public final ByteBuffer bytes;
     private final String text;
+    /**
+     * since these objects are compared frequently, we stash an efficiently compared prefix of the bytes, in the expectation
+     * that the majority of comparisons can be answered by this value only
+     */
+    private final long prefixComparison;
     private final boolean interned;
 
     private static final long EMPTY_SIZE = ObjectSizes.measure(new ColumnIdentifier(ByteBufferUtil.EMPTY_BYTE_BUFFER, "", false));
 
     private static final ConcurrentMap<ByteBuffer, ColumnIdentifier> internedInstances = new MapMaker().weakValues().makeMap();
 
+    private static long prefixComparison(ByteBuffer bytes)
+    {
+        long prefix = 0;
+        ByteBuffer read = bytes.duplicate();
+        int i = 0;
+        while (read.hasRemaining() && i < 8)
+        {
+            prefix <<= 8;
+            prefix |= read.get() & 0xFF;
+            i++;
+        }
+        prefix <<= (8 - i) * 8;
+        // by flipping the top bit (==Integer.MIN_VALUE), we ensure that signed comparison gives the same result
+        // as an unsigned without the bit flipped
+        prefix ^= Long.MIN_VALUE;
+        return prefix;
+    }
+
     public ColumnIdentifier(String rawText, boolean keepCase)
     {
         this.text = keepCase ? rawText : rawText.toLowerCase(Locale.US);
         this.bytes = ByteBufferUtil.bytes(this.text);
+        this.prefixComparison = prefixComparison(bytes);
         this.interned = false;
     }
 
@@ -70,6 +95,7 @@ public class ColumnIdentifier extends org.apache.cassandra.cql3.selection.Select
         this.bytes = bytes;
         this.text = text;
         this.interned = interned;
+        this.prefixComparison = prefixComparison(bytes);
     }
 
     public static ColumnIdentifier getInterned(ByteBuffer bytes, AbstractType<?> type)
@@ -152,6 +178,16 @@ public class ColumnIdentifier extends org.apache.cassandra.cql3.selection.Select
         return SimpleSelector.newFactory(def, addAndGetIndex(def, defs));
     }
 
+    public int compareTo(ColumnIdentifier that)
+    {
+        int c = Long.compare(this.prefixComparison, that.prefixComparison);
+        if (c != 0)
+            return c;
+        if (this == that)
+            return 0;
+        return ByteBufferUtil.compareUnsigned(this.bytes, that.bytes);
+    }
+
     /**
      * Because Thrift-created tables may have a non-text comparator, we cannot determine the proper 'key' until
      * we know the comparator. ColumnIdentifier.Raw is a placeholder that can be converted to a real ColumnIdentifier

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8c8103ca/src/java/org/apache/cassandra/db/Columns.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Columns.java b/src/java/org/apache/cassandra/db/Columns.java
index 94e45dc..055624b 100644
--- a/src/java/org/apache/cassandra/db/Columns.java
+++ b/src/java/org/apache/cassandra/db/Columns.java
@@ -410,7 +410,7 @@ public class Columns implements Iterable<ColumnDefinition>
                 if (regular >= complexIdx)
                     return columns[complex++];
 
-                return ByteBufferUtil.compareUnsigned(columns[regular].name.bytes, columns[complex].name.bytes) < 0
+                return columns[regular].name.compareTo(columns[complex].name) < 0
                      ? columns[regular++]
                      : columns[complex++];
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8c8103ca/src/java/org/apache/cassandra/db/filter/ColumnFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/ColumnFilter.java b/src/java/org/apache/cassandra/db/filter/ColumnFilter.java
index 1dc239f..b98108d 100644
--- a/src/java/org/apache/cassandra/db/filter/ColumnFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/ColumnFilter.java
@@ -52,13 +52,6 @@ public class ColumnFilter
 {
     public static final Serializer serializer = new Serializer();
 
-    private static final Comparator<ColumnIdentifier> keyComparator = new Comparator<ColumnIdentifier>()
-    {
-        public int compare(ColumnIdentifier id1, ColumnIdentifier id2)
-        {
-            return ByteBufferUtil.compareUnsigned(id1.bytes, id2.bytes);
-        }
-    };
     private static final Comparator<ColumnSubselection> valueComparator = new Comparator<ColumnSubselection>()
     {
         public int compare(ColumnSubselection s1, ColumnSubselection s2)
@@ -309,7 +302,7 @@ public class ColumnFilter
             SortedSetMultimap<ColumnIdentifier, ColumnSubselection> s = null;
             if (subSelections != null)
             {
-                s = TreeMultimap.create(keyComparator, valueComparator);
+                s = TreeMultimap.create(Comparator.<ColumnIdentifier>naturalOrder(), valueComparator);
                 for (ColumnSubselection subSelection : subSelections)
                     s.put(subSelection.column().name, subSelection);
             }
@@ -401,7 +394,7 @@ public class ColumnFilter
             SortedSetMultimap<ColumnIdentifier, ColumnSubselection> subSelections = null;
             if (hasSubSelections)
             {
-                subSelections = TreeMultimap.create(keyComparator, valueComparator);
+                subSelections = TreeMultimap.create(Comparator.<ColumnIdentifier>naturalOrder(), valueComparator);
                 int size = in.readUnsignedShort();
                 for (int i = 0; i < size; i++)
                 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8c8103ca/test/unit/org/apache/cassandra/cql3/ColumnIdentifierTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/ColumnIdentifierTest.java b/test/unit/org/apache/cassandra/cql3/ColumnIdentifierTest.java
new file mode 100644
index 0000000..c287883
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/ColumnIdentifierTest.java
@@ -0,0 +1,61 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*    http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied.  See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+package org.apache.cassandra.cql3;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.ThreadLocalRandom;
+
+import org.junit.Test;
+
+import junit.framework.Assert;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class ColumnIdentifierTest
+{
+
+    @Test
+    public void testComparisonMethod()
+    {
+        ThreadLocalRandom random = ThreadLocalRandom.current();
+        byte[] commonBytes = new byte[10];
+        byte[] aBytes = new byte[16];
+        byte[] bBytes = new byte[16];
+        for (int i = 0 ; i < 100000 ; i++)
+        {
+            int commonLength = random.nextInt(0, 10);
+            random.nextBytes(commonBytes);
+            random.nextBytes(aBytes);
+            random.nextBytes(bBytes);
+            System.arraycopy(commonBytes, 0, aBytes, 0, commonLength);
+            System.arraycopy(commonBytes, 0, bBytes, 0, commonLength);
+            int aLength = random.nextInt(commonLength, 16);
+            int bLength = random.nextInt(commonLength, 16);
+            ColumnIdentifier a = new ColumnIdentifier(ByteBuffer.wrap(aBytes, 0, aLength), BytesType.instance);
+            ColumnIdentifier b = new ColumnIdentifier(ByteBuffer.wrap(bBytes, 0, bLength), BytesType.instance);
+            Assert.assertEquals("" + i, compareResult(a.compareTo(b)), compareResult(ByteBufferUtil.compareUnsigned(a.bytes, b.bytes)));
+        }
+    }
+
+    private static int compareResult(int v)
+    {
+        return v < 0 ? -1 : v > 0 ? 1 : 0;
+    }
+
+}