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/02/19 13:25:30 UTC

cassandra git commit: Select optimal CRC32 implementation at runtime

Repository: cassandra
Updated Branches:
  refs/heads/trunk e034d3fd7 -> dfd695c14


Select optimal CRC32 implementation at runtime

patch by ariel; reviewed by benedict for CASSANDRA-8614


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

Branch: refs/heads/trunk
Commit: dfd695c14b64abc50eb38859f43bd55691207ae9
Parents: e034d3f
Author: Benedict Elliott Smith <be...@apache.org>
Authored: Thu Feb 19 12:24:59 2015 +0000
Committer: Benedict Elliott Smith <be...@apache.org>
Committed: Thu Feb 19 12:24:59 2015 +0000

----------------------------------------------------------------------
 CHANGES.txt                                     |   2 +-
 build.xml                                       |   1 +
 .../apache/cassandra/utils/CRC32Factory.java    | 104 ++++++++++++++++
 src/java/org/apache/cassandra/utils/ICRC32.java |  28 +++++
 .../apache/cassandra/utils/PureJavaCrc32.java   |   2 +-
 src/jdkoverride/java/util/zip/CRC32.java        |  42 +++++++
 .../cassandra/utils/CRC32FactoryTest.java       | 120 +++++++++++++++++++
 7 files changed, 297 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/dfd695c1/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 46f842b..4906a38 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -55,7 +55,7 @@
  * Fix race condition during calculation of pending ranges (CASSANDRA-7390)
  * Fail on very large batch sizes (CASSANDRA-8011)
  * Improve concurrency of repair (CASSANDRA-6455, 8208)
-
+ * Select optimal CRC32 implementation at runtime (CASSANDRA-8614)
 
 2.1.4
  * Fix CommitLog.forceRecycleAllSegments() memory access error (CASSANDRA-8812)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dfd695c1/build.xml
----------------------------------------------------------------------
diff --git a/build.xml b/build.xml
index 6461694..a3e524b 100644
--- a/build.xml
+++ b/build.xml
@@ -34,6 +34,7 @@
     <property name="basedir" value="."/>
     <property name="build.src" value="${basedir}/src"/>
     <property name="build.src.java" value="${basedir}/src/java"/>
+    <property name="build.src.jdkoverride" value="${basedir}/src/jdkoverride" />
     <property name="build.src.resources" value="${basedir}/src/resources"/>
     <property name="build.src.gen-java" value="${basedir}/src/gen-java"/>
     <property name="build.lib" value="${basedir}/lib"/>

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dfd695c1/src/java/org/apache/cassandra/utils/CRC32Factory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/CRC32Factory.java b/src/java/org/apache/cassandra/utils/CRC32Factory.java
new file mode 100644
index 0000000..85b9890
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/CRC32Factory.java
@@ -0,0 +1,104 @@
+/*
+ * 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.utils;
+
+import java.nio.ByteBuffer;
+import java.security.AccessController;
+import java.security.PrivilegedAction;
+import java.util.concurrent.BlockingQueue;
+import java.util.zip.CRC32;
+
+import org.apache.cassandra.utils.PureJavaCrc32;
+
+public class CRC32Factory
+{
+    private static final boolean JDK8;
+    static {
+        boolean jdk8 = false;
+        ClassLoader cl = null;
+        try
+        {
+            if (System.getSecurityManager() == null)
+            {
+                cl = BlockingQueue.class.getClassLoader();
+            }
+            else
+            {
+                cl =  AccessController.doPrivileged(new PrivilegedAction<ClassLoader>()
+                        {
+                    @Override
+                    public ClassLoader run()
+                    {
+                        return BlockingQueue.class.getClassLoader();
+                    }
+                });
+            }
+
+            Class.forName("java.util.concurrent.CompletableFuture", false, cl);
+
+            jdk8 = true;
+        }
+        catch (Exception e)
+        {}
+        JDK8 = jdk8;
+    }
+
+    public static class CRC32Ex extends CRC32 implements ICRC32 {
+
+        @Override
+        public void update(ByteBuffer b, int offset, int length)
+        {
+            final int oldPosition = b.position();
+            final int oldLimit = b.limit();
+            try
+            {
+                b.limit(offset + length);
+                b.position(offset);
+                update(b);
+            }
+            finally
+            {
+                b.position(oldPosition);
+                b.limit(oldLimit);
+            }
+        }
+
+        @Override
+        public void updateInt(int v)
+        {
+            update((v >>> 24) & 0xFF);
+            update((v >>> 16) & 0xFF);
+            update((v >>> 8) & 0xFF);
+            update((v >>> 0) & 0xFF);
+        }
+
+        @Override
+        public int getCrc()
+        {
+            return (int)getValue();
+        }
+    }
+
+    public static ICRC32 create()
+    {
+        if (JDK8)
+            return new CRC32Ex();
+        else
+            return new PureJavaCrc32();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dfd695c1/src/java/org/apache/cassandra/utils/ICRC32.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/ICRC32.java b/src/java/org/apache/cassandra/utils/ICRC32.java
new file mode 100644
index 0000000..3f08ade
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/ICRC32.java
@@ -0,0 +1,28 @@
+/*
+ * 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.utils;
+
+import java.nio.ByteBuffer;
+import java.util.zip.Checksum;
+
+public interface ICRC32 extends Checksum
+{
+    void update(ByteBuffer b, int offset, int length);
+    void updateInt(int v);
+    int getCrc();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dfd695c1/src/java/org/apache/cassandra/utils/PureJavaCrc32.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/PureJavaCrc32.java b/src/java/org/apache/cassandra/utils/PureJavaCrc32.java
index 9a1ac02..bbf8eda 100644
--- a/src/java/org/apache/cassandra/utils/PureJavaCrc32.java
+++ b/src/java/org/apache/cassandra/utils/PureJavaCrc32.java
@@ -34,7 +34,7 @@ import java.util.zip.Checksum;
  * This class is copied from hadoop-commons project and retains that formatting.
  * (The initial patch added PureJavaCrc32 was HADOOP-6148)
  */
-public class PureJavaCrc32 implements Checksum {
+public class PureJavaCrc32 implements ICRC32 {
 
   /** the current CRC value, bit-flipped */
   private int crc;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dfd695c1/src/jdkoverride/java/util/zip/CRC32.java
----------------------------------------------------------------------
diff --git a/src/jdkoverride/java/util/zip/CRC32.java b/src/jdkoverride/java/util/zip/CRC32.java
new file mode 100644
index 0000000..93e1e6f
--- /dev/null
+++ b/src/jdkoverride/java/util/zip/CRC32.java
@@ -0,0 +1,42 @@
+/*
+ * 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 java.util.zip;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A fake implementation of java.util.zip.CRC32 with the additonal JDK 8 methods so
+ * that when compiling using Java 7 we can link against those new methods and then
+ * avoid calling them at runtime if running with Java 7.
+ */
+public class CRC32 implements Checksum
+{
+    public CRC32() {}
+
+    public void update(int b) {}
+
+    public void update(byte[] b, int off, int len) {}
+
+    public void update(byte[] b) {}
+
+    public void update(ByteBuffer buffer) {}
+
+    public void reset() {}
+
+    public long getValue() { return 0L; }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dfd695c1/test/unit/org/apache/cassandra/utils/CRC32FactoryTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/CRC32FactoryTest.java b/test/unit/org/apache/cassandra/utils/CRC32FactoryTest.java
new file mode 100644
index 0000000..a55fbf0
--- /dev/null
+++ b/test/unit/org/apache/cassandra/utils/CRC32FactoryTest.java
@@ -0,0 +1,120 @@
+/**
+ * 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.utils;
+
+import org.apache.cassandra.utils.CRC32Factory.CRC32Ex;
+
+import java.nio.ByteBuffer;
+import java.util.Random;
+
+import static org.junit.Assert.*;
+
+import org.junit.Test;
+
+public class CRC32FactoryTest
+{
+
+    @Test
+    public void updateInt()
+    {
+        ICRC32 crcA = new CRC32Ex();
+        PureJavaCrc32 crcB = new PureJavaCrc32();
+
+        crcA.updateInt(42);
+        crcB.updateInt(42);
+
+        assertEquals(crcA.getCrc(), crcB.getCrc());
+        assertEquals(crcA.getValue(), crcB.getValue());
+    }
+
+    @Test
+    public void testFuzzz()
+    {
+        for (int ii = 0; ii < 100; ii++)
+        {
+            testOnce();
+        }
+    }
+
+    private void testOnce()
+    {
+        final long seed = System.nanoTime();
+        System.out.println("Seed is " + seed);
+        Random r = new java.util.Random(seed);
+
+        ByteBuffer source = null;
+        int nextSize = r.nextDouble() < .9 ? r.nextInt(1024 * 1024) : r.nextInt(16);
+
+        if (r.nextDouble() > .5)
+        {
+            source = ByteBuffer.allocate(nextSize);
+            r.nextBytes(source.array());
+        }
+        else
+        {
+            source = ByteBuffer.allocateDirect(nextSize);
+            while (source.hasRemaining())
+            {
+                source.put((byte)(r.nextInt() % 127));
+            }
+            source.clear();
+        }
+
+        ICRC32 crcA = new CRC32Ex();
+        PureJavaCrc32 crcB = new PureJavaCrc32();
+        if (source.hasArray())
+        {
+            if (r.nextDouble() > 0.5)
+            {
+                crcA.update(source.array(), 0, source.remaining());
+                crcB.update(source.array(), 0, source.remaining());
+            }
+            else
+            {
+                crcA.update(source, 0, source.remaining());
+                assertEquals(0, source.position());
+                assertEquals(source.capacity(), source.limit());
+                crcB.update(source, 0, source.remaining());
+                assertEquals(0, source.position());
+                assertEquals(source.capacity(), source.limit());
+            }
+        }
+        else
+        {
+            crcA.update(source, 0, source.remaining());
+            assertEquals(0, source.position());
+            assertEquals(source.capacity(), source.limit());
+            crcB.update(source, 0, source.remaining());
+            assertEquals(0, source.position());
+            assertEquals(source.capacity(), source.limit());
+        }
+        assertEquals(crcA.getCrc(), crcB.getCrc());
+        assertEquals(crcA.getValue(), crcB.getValue());
+    }
+
+    @Test
+    public void jdkDetection()
+    {
+        if (System.getProperty("java.version").startsWith("1.7"))
+            assertFalse(CRC32Factory.create() instanceof CRC32Factory.CRC32Ex);
+        else
+            assertTrue(CRC32Factory.create() instanceof CRC32Factory.CRC32Ex);
+    }
+}