You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jb...@apache.org on 2012/01/11 01:18:39 UTC

git commit: r/m unused code, and replace FBUtilities.byteArrayToInt with Guava Ints.fromByteArray

Updated Branches:
  refs/heads/trunk 7c8c57041 -> ea4296932


r/m unused code, and replace FBUtilities.byteArrayToInt with Guava Ints.fromByteArray


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

Branch: refs/heads/trunk
Commit: ea4296932c6be69d7a1a995367bcf7e2130e962b
Parents: 7c8c570
Author: Jonathan Ellis <jb...@apache.org>
Authored: Tue Jan 10 18:17:39 2012 -0600
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Tue Jan 10 18:18:14 2012 -0600

----------------------------------------------------------------------
 .../io/compress/CompressedRandomAccessReader.java  |    4 +-
 .../org/apache/cassandra/utils/FBUtilities.java    |  135 +--------------
 .../apache/cassandra/utils/FBUtilitiesTest.java    |   16 +--
 3 files changed, 7 insertions(+), 148 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/ea429693/src/java/org/apache/cassandra/io/compress/CompressedRandomAccessReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/compress/CompressedRandomAccessReader.java b/src/java/org/apache/cassandra/io/compress/CompressedRandomAccessReader.java
index b7128f3..c83fd90 100644
--- a/src/java/org/apache/cassandra/io/compress/CompressedRandomAccessReader.java
+++ b/src/java/org/apache/cassandra/io/compress/CompressedRandomAccessReader.java
@@ -23,6 +23,8 @@ import java.nio.channels.FileChannel;
 import java.util.zip.CRC32;
 import java.util.zip.Checksum;
 
+import com.google.common.primitives.Ints;
+
 import org.apache.cassandra.io.util.RandomAccessReader;
 import org.apache.cassandra.utils.FBUtilities;
 
@@ -113,7 +115,7 @@ public class CompressedRandomAccessReader extends RandomAccessReader
                                                 chunk.offset,
                                                 chunk.length));
 
-        return FBUtilities.byteArrayToInt(checksumBytes);
+        return Ints.fromByteArray(checksumBytes);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ea429693/src/java/org/apache/cassandra/utils/FBUtilities.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/FBUtilities.java b/src/java/org/apache/cassandra/utils/FBUtilities.java
index 3260f37..777d34f 100644
--- a/src/java/org/apache/cassandra/utils/FBUtilities.java
+++ b/src/java/org/apache/cassandra/utils/FBUtilities.java
@@ -38,6 +38,7 @@ import java.util.concurrent.atomic.AtomicLong;
 
 import com.google.common.base.Joiner;
 import com.google.common.collect.AbstractIterator;
+import com.google.common.primitives.Ints;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -118,21 +119,6 @@ public class FBUtilities
     }
 
     /**
-     * Parses a string representing either a fraction, absolute value or percentage.
-     */
-    public static double parseDoubleOrPercent(String value)
-    {
-        if (value.endsWith("%"))
-        {
-            return Double.parseDouble(value.substring(0, value.length() - 1)) / 100;
-        }
-        else
-        {
-            return Double.parseDouble(value);
-        }
-    }
-
-    /**
      * Please use getBroadcastAddress instead. You need this only when you have to listen/connect.
      */
     public static InetAddress getLocalAddress()
@@ -161,27 +147,6 @@ public class FBUtilities
     }
 
     /**
-     * @param fractOrAbs A double that may represent a fraction or absolute value.
-     * @param total If fractionOrAbs is a fraction, the total to take the fraction from
-     * @return An absolute value which may be larger than the total.
-     */
-    public static long absoluteFromFraction(double fractOrAbs, long total)
-    {
-        if (fractOrAbs < 0)
-            throw new UnsupportedOperationException("unexpected negative value " + fractOrAbs);
-
-        if (0 < fractOrAbs && fractOrAbs <= 1)
-        {
-            // fraction
-            return Math.max(1, (long)(fractOrAbs * total));
-        }
-
-        // absolute
-        assert fractOrAbs >= 1 || fractOrAbs == 0;
-        return (long)fractOrAbs;
-    }
-
-    /**
      * Given two bit arrays represented as BigIntegers, containing the given
      * number of significant bits, calculate a midpoint.
      *
@@ -212,80 +177,6 @@ public class FBUtilities
         return new Pair<BigInteger, Boolean>(midpoint, remainder);
     }
 
-    /**
-     * Copy bytes from int into bytes starting from offset.
-     * @param bytes Target array
-     * @param offset Offset into the array
-     * @param i Value to write
-     */
-    public static void copyIntoBytes(byte[] bytes, int offset, int i)
-    {
-        bytes[offset]   = (byte)( ( i >>> 24 ) & 0xFF );
-        bytes[offset+1] = (byte)( ( i >>> 16 ) & 0xFF );
-        bytes[offset+2] = (byte)( ( i >>> 8  ) & 0xFF );
-        bytes[offset+3] = (byte)(   i          & 0xFF );
-    }
-
-    /**
-     * @param i Write this int to an array
-     * @return 4-byte array containing the int
-     */
-    public static byte[] toByteArray(int i)
-    {
-        byte[] bytes = new byte[4];
-        copyIntoBytes(bytes, 0, i);
-        return bytes;
-    }
-
-    /**
-     * Copy bytes from long into bytes starting from offset.
-     * @param bytes Target array
-     * @param offset Offset into the array
-     * @param l Value to write
-     */
-    public static void copyIntoBytes(byte[] bytes, int offset, long l)
-    {
-        bytes[offset]   = (byte)( ( l >>> 56 ) & 0xFF );
-        bytes[offset+1] = (byte)( ( l >>> 48 ) & 0xFF );
-        bytes[offset+2] = (byte)( ( l >>> 40 ) & 0xFF );
-        bytes[offset+3] = (byte)( ( l >>> 32 ) & 0xFF );
-        bytes[offset+4] = (byte)( ( l >>> 24 ) & 0xFF );
-        bytes[offset+5] = (byte)( ( l >>> 16 ) & 0xFF );
-        bytes[offset+6] = (byte)( ( l >>> 8  ) & 0xFF );
-        bytes[offset+7] = (byte)(   l          & 0xFF );
-    }
-
-    /**
-     * @param l Write this long to an array
-     * @return 8-byte array containing the long
-     */
-    public static byte[] toByteArray(long l)
-    {
-        byte[] bytes = new byte[8];
-        copyIntoBytes(bytes, 0, l);
-        return bytes;
-    }
-
-    /**
-     * Convert the byte array to an int starting from the given offset.
-     *
-     * @param b The byte array
-     *
-     * @return The integer
-     */
-    public static int byteArrayToInt(byte[] b)
-    {
-        int value = 0;
-
-        for (int i = 0; i < 4; i++)
-        {
-            int shift = (4 - 1 - i) * 8;
-            value += (b[i] & 0x000000FF) << shift;
-        }
-
-        return value;
-    }
-
     public static int compareUnsigned(byte[] bytes1, byte[] bytes2, int offset1, int offset2, int len1, int len2)
     {
         return FastByteComparisons.compareTo(bytes1, offset1, len1, bytes2, offset2, len2);
@@ -341,26 +232,6 @@ public class FBUtilities
         }
     }
 
-    public static void atomicSetMax(AtomicInteger atomic, int i)
-    {
-        while (true)
-        {
-            int j = atomic.get();
-            if (j >= i || atomic.compareAndSet(j, i))
-                break;
-        }
-    }
-
-    public static void atomicSetMax(AtomicLong atomic, long i)
-    {
-        while (true)
-        {
-            long j = atomic.get();
-            if (j >= i || atomic.compareAndSet(j, i))
-                break;
-        }
-    }
-
     public static void serialize(TSerializer serializer, TBase struct, DataOutput out)
     throws IOException
     {
@@ -528,10 +399,6 @@ public class FBUtilities
             {
                 throw new RuntimeException(e);
             }
-            catch (TimeoutException e)
-            {
-               throw e;
-            }
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ea429693/test/unit/org/apache/cassandra/utils/FBUtilitiesTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/FBUtilitiesTest.java b/test/unit/org/apache/cassandra/utils/FBUtilitiesTest.java
index 88a49b8..67dd0ac 100644
--- a/test/unit/org/apache/cassandra/utils/FBUtilitiesTest.java
+++ b/test/unit/org/apache/cassandra/utils/FBUtilitiesTest.java
@@ -24,25 +24,15 @@ import static org.junit.Assert.fail;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.charset.CharacterCodingException;
+import java.util.Arrays;
 
 import com.google.common.base.Charsets;
+import com.google.common.primitives.Ints;
 import org.junit.Test;
 
 public class FBUtilitiesTest 
 {
     @Test
-    public void testCopyIntoBytes()
-    {
-        int i = 300;
-        long l = 1000;
-        ByteBuffer b = ByteBuffer.allocate(20);
-        FBUtilities.copyIntoBytes(b.array(), 0, i);
-        FBUtilities.copyIntoBytes(b.array(), 4, l);
-        assertEquals(i, b.getInt(0));
-        assertEquals(l, b.getLong(4));
-    }
-    
-    @Test
     public void testCompareByteSubArrays()
     {
         ByteBuffer bytes = ByteBuffer.allocate(16);
@@ -56,7 +46,7 @@ public class FBUtilitiesTest
                 ByteBufferUtil.bytes(524255231), 0, null, 0, 4) == 1;
 
         // handle comparisons
-        FBUtilities.copyIntoBytes(bytes.array(), 3, 524255231);
+        System.arraycopy(Ints.toByteArray(524255231), 0, bytes.array(), 3, 4);
         assert ByteBufferUtil.compareSubArrays(
                 bytes, 3, ByteBufferUtil.bytes(524255231), 0, 4) == 0;
         assert ByteBufferUtil.compareSubArrays(