You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by db...@apache.org on 2016/07/04 23:29:49 UTC

cassandra git commit: combine all the architecture alignment checks into one place

Repository: cassandra
Updated Branches:
  refs/heads/trunk 6c16d2745 -> 73b5cabbc


combine all the architecture alignment checks into one place


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

Branch: refs/heads/trunk
Commit: 73b5cabbcdf476632300c8d73fa5edf3fb447bdc
Parents: 6c16d27
Author: Dave Brosius <db...@mebigfatguy.com>
Authored: Mon Jul 4 19:29:26 2016 -0400
Committer: Dave Brosius <db...@mebigfatguy.com>
Committed: Mon Jul 4 19:29:26 2016 -0400

----------------------------------------------------------------------
 .../org/apache/cassandra/io/util/Memory.java    | 20 ++++-----
 .../apache/cassandra/utils/Architecture.java    | 44 ++++++++++++++++++++
 .../cassandra/utils/FastByteOperations.java     |  5 +--
 .../cassandra/utils/memory/MemoryUtil.java      | 22 ++++------
 4 files changed, 61 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/73b5cabb/src/java/org/apache/cassandra/io/util/Memory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/Memory.java b/src/java/org/apache/cassandra/io/util/Memory.java
index 78950ce..bd87f5e 100644
--- a/src/java/org/apache/cassandra/io/util/Memory.java
+++ b/src/java/org/apache/cassandra/io/util/Memory.java
@@ -22,6 +22,8 @@ import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
 
 import net.nicoulaj.compilecommand.annotations.Inline;
+
+import org.apache.cassandra.utils.Architecture;
 import org.apache.cassandra.utils.FastByteOperations;
 import org.apache.cassandra.utils.concurrent.Ref;
 import org.apache.cassandra.utils.memory.MemoryUtil;
@@ -51,17 +53,9 @@ public class Memory implements AutoCloseable
     private static final long BYTE_ARRAY_BASE_OFFSET = unsafe.arrayBaseOffset(byte[].class);
 
     private static final boolean bigEndian = ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN);
-    private static final boolean unaligned;
 
     public static final ByteBuffer[] NO_BYTE_BUFFERS = new ByteBuffer[0];
 
-    static
-    {
-        String arch = System.getProperty("os.arch");
-        unaligned = arch.equals("i386") || arch.equals("x86")
-                    || arch.equals("amd64") || arch.equals("x86_64") || arch.equals("s390x");
-    }
-
     protected long peer;
     // size of the memory region
     protected final long size;
@@ -113,7 +107,7 @@ public class Memory implements AutoCloseable
     public void setLong(long offset, long l)
     {
         checkBounds(offset, offset + 8);
-        if (unaligned)
+        if (Architecture.IS_UNALIGNED)
         {
             unsafe.putLong(peer + offset, l);
         }
@@ -152,7 +146,7 @@ public class Memory implements AutoCloseable
     public void setInt(long offset, int l)
     {
         checkBounds(offset, offset + 4);
-        if (unaligned)
+        if (Architecture.IS_UNALIGNED)
         {
             unsafe.putInt(peer + offset, l);
         }
@@ -183,7 +177,7 @@ public class Memory implements AutoCloseable
     public void setShort(long offset, short l)
     {
         checkBounds(offset, offset + 2);
-        if (unaligned)
+        if (Architecture.IS_UNALIGNED)
         {
             unsafe.putShort(peer + offset, l);
         }
@@ -258,7 +252,7 @@ public class Memory implements AutoCloseable
     public long getLong(long offset)
     {
         checkBounds(offset, offset + 8);
-        if (unaligned) {
+        if (Architecture.IS_UNALIGNED) {
             return unsafe.getLong(peer + offset);
         } else {
             return getLongByByte(peer + offset);
@@ -290,7 +284,7 @@ public class Memory implements AutoCloseable
     public int getInt(long offset)
     {
         checkBounds(offset, offset + 4);
-        if (unaligned) {
+        if (Architecture.IS_UNALIGNED) {
             return unsafe.getInt(peer + offset);
         } else {
             return getIntByByte(peer + offset);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/73b5cabb/src/java/org/apache/cassandra/utils/Architecture.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/Architecture.java b/src/java/org/apache/cassandra/utils/Architecture.java
new file mode 100644
index 0000000..3173f8e
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/Architecture.java
@@ -0,0 +1,44 @@
+/*
+* 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.util.Collections;
+import java.util.Set;
+
+import com.google.common.collect.Sets;
+
+public final class Architecture
+{
+    // Note that s390x architecture are not officially supported and adding it here is only done out of convenience
+    // for those that want to run C* on this architecture at their own risk (see #11214)
+    private static final Set<String> UNALIGNED_ARCH = Collections.unmodifiableSet(Sets.newHashSet(
+        "i386",
+        "x86",
+        "amd64",
+        "x86_64",
+        "s390x"
+    ));
+    
+    public static final boolean IS_UNALIGNED = UNALIGNED_ARCH.contains(System.getProperty("os.arch"));
+    
+    private Architecture()
+    {
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/73b5cabb/src/java/org/apache/cassandra/utils/FastByteOperations.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/FastByteOperations.java b/src/java/org/apache/cassandra/utils/FastByteOperations.java
index cf8d305..02c0dbb 100644
--- a/src/java/org/apache/cassandra/utils/FastByteOperations.java
+++ b/src/java/org/apache/cassandra/utils/FastByteOperations.java
@@ -102,10 +102,7 @@ public class FastByteOperations
          */
         static ByteOperations getBest()
         {
-            String arch = System.getProperty("os.arch");
-            boolean unaligned = arch.equals("i386") || arch.equals("x86")
-                                || arch.equals("amd64") || arch.equals("x86_64") || arch.equals("s390x");
-            if (!unaligned)
+            if (!Architecture.IS_UNALIGNED)
                 return new PureJavaOperations();
             try
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/73b5cabb/src/java/org/apache/cassandra/utils/memory/MemoryUtil.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/memory/MemoryUtil.java b/src/java/org/apache/cassandra/utils/memory/MemoryUtil.java
index 3a18964..6c2e6fd 100644
--- a/src/java/org/apache/cassandra/utils/memory/MemoryUtil.java
+++ b/src/java/org/apache/cassandra/utils/memory/MemoryUtil.java
@@ -23,6 +23,9 @@ import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
 
 import com.sun.jna.Native;
+
+import org.apache.cassandra.utils.Architecture;
+
 import sun.misc.Unsafe;
 import sun.nio.ch.DirectBuffer;
 
@@ -44,17 +47,10 @@ public abstract class MemoryUtil
 
     private static final boolean BIG_ENDIAN = ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN);
 
-    private static final boolean UNALIGNED;
-    public static final boolean INVERTED_ORDER;
+    public static final boolean INVERTED_ORDER = Architecture.IS_UNALIGNED && !BIG_ENDIAN;
 
     static
     {
-        String arch = System.getProperty("os.arch");
-        // Note that s390x architecture are not officially supported and adding it here is only done out of convenience
-        // for those that want to run C* on this architecture at their own risk (see #11214)
-        UNALIGNED = arch.equals("i386") || arch.equals("x86")
-                || arch.equals("amd64") || arch.equals("x86_64") || arch.equals("s390x");
-        INVERTED_ORDER = UNALIGNED && !BIG_ENDIAN;
         try
         {
             Field field = sun.misc.Unsafe.class.getDeclaredField("theUnsafe");
@@ -120,7 +116,7 @@ public abstract class MemoryUtil
 
     public static void setInt(long address, int l)
     {
-        if (UNALIGNED)
+        if (Architecture.IS_UNALIGNED)
             unsafe.putInt(address, l);
         else
             putIntByByte(address, l);
@@ -128,7 +124,7 @@ public abstract class MemoryUtil
 
     public static void setLong(long address, long l)
     {
-        if (UNALIGNED)
+        if (Architecture.IS_UNALIGNED)
             unsafe.putLong(address, l);
         else
             putLongByByte(address, l);
@@ -141,17 +137,17 @@ public abstract class MemoryUtil
 
     public static int getShort(long address)
     {
-        return (UNALIGNED ? unsafe.getShort(address) : getShortByByte(address)) & 0xffff;
+        return (Architecture.IS_UNALIGNED ? unsafe.getShort(address) : getShortByByte(address)) & 0xffff;
     }
 
     public static int getInt(long address)
     {
-        return UNALIGNED ? unsafe.getInt(address) : getIntByByte(address);
+        return Architecture.IS_UNALIGNED ? unsafe.getInt(address) : getIntByByte(address);
     }
 
     public static long getLong(long address)
     {
-        return UNALIGNED ? unsafe.getLong(address) : getLongByByte(address);
+        return Architecture.IS_UNALIGNED ? unsafe.getLong(address) : getLongByByte(address);
     }
 
     public static ByteBuffer getByteBuffer(long address, int length)