Return-Path: X-Original-To: apmail-cassandra-commits-archive@www.apache.org Delivered-To: apmail-cassandra-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 7060D10A83 for ; Wed, 19 Mar 2014 17:02:14 +0000 (UTC) Received: (qmail 78625 invoked by uid 500); 19 Mar 2014 17:02:14 -0000 Delivered-To: apmail-cassandra-commits-archive@cassandra.apache.org Received: (qmail 78579 invoked by uid 500); 19 Mar 2014 17:02:13 -0000 Mailing-List: contact commits-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@cassandra.apache.org Delivered-To: mailing list commits@cassandra.apache.org Received: (qmail 78560 invoked by uid 99); 19 Mar 2014 17:02:12 -0000 Received: from tyr.zones.apache.org (HELO tyr.zones.apache.org) (140.211.11.114) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 19 Mar 2014 17:02:12 +0000 Received: by tyr.zones.apache.org (Postfix, from userid 65534) id 81E4F9847EE; Wed, 19 Mar 2014 17:02:12 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: marcuse@apache.org To: commits@cassandra.apache.org Date: Wed, 19 Mar 2014 17:02:12 -0000 Message-Id: X-Mailer: ASF-Git Admin Mailer Subject: [1/4] ByteBuffer write() methods for serializing sstables Repository: cassandra Updated Branches: refs/heads/cassandra-2.1 2a2141ce6 -> 75508ec89 http://git-wip-us.apache.org/repos/asf/cassandra/blob/75508ec8/test/unit/org/apache/cassandra/utils/FastByteOperationsTest.java ---------------------------------------------------------------------- diff --git a/test/unit/org/apache/cassandra/utils/FastByteOperationsTest.java b/test/unit/org/apache/cassandra/utils/FastByteOperationsTest.java new file mode 100644 index 0000000..4f6d652 --- /dev/null +++ b/test/unit/org/apache/cassandra/utils/FastByteOperationsTest.java @@ -0,0 +1,162 @@ +/* + * 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.Arrays; +import java.util.Random; + +import org.junit.Test; + +public class FastByteOperationsTest +{ + + private static final FastByteOperations.PureJavaOperations PJO = new FastByteOperations.PureJavaOperations(); + private static final FastByteOperations.UnsafeOperations UO = new FastByteOperations.UnsafeOperations(); + private static final Random rand = new Random(0); + private static final ByteBuffer dbuf1 = ByteBuffer.allocateDirect(150); + private static final ByteBuffer dbuf2 = ByteBuffer.allocateDirect(150); + private static final ByteBuffer hbuf1 = ByteBuffer.allocate(150); + private static final ByteBuffer hbuf2 = ByteBuffer.allocate(150); + + @Test + public void testFastByteCopy() + { + byte[] bytes1 = new byte[128]; + byte[] empty = new byte[128]; + rand.nextBytes(bytes1); + testCopy(bytes1, wrap1(bytes1, true), wrap2(empty, true), PJO); + testCopy(bytes1, wrap1(bytes1, true), wrap2(empty, false), PJO); + testCopy(bytes1, wrap1(bytes1, false), wrap2(empty, true), PJO); + testCopy(bytes1, wrap1(bytes1, false), wrap2(empty, false), PJO); + testCopy(bytes1, wrap1(bytes1, true), wrap2(empty, true), UO); + testCopy(bytes1, wrap1(bytes1, true), wrap2(empty, false), UO); + testCopy(bytes1, wrap1(bytes1, false), wrap2(empty, true), UO); + testCopy(bytes1, wrap1(bytes1, false), wrap2(empty, false), UO); + } + + private void testCopy(byte[] canon, ByteBuffer src, ByteBuffer trg, FastByteOperations.ByteOperations ops) + { + byte[] result = new byte[src.remaining()]; + ops.copy(src, src.position(), trg, trg.position(), src.remaining()); + ops.copy(trg, trg.position(), result, 0, trg.remaining()); + assert firstdiff(canon, result) < 0; + } + + private static int firstdiff(byte[] canon, byte[] test) + { + for (int i = 0 ; i < canon.length ; i++) + if (canon[i] != test[i]) + return i; + return -1; + } + + @Test + public void testFastByteComparisons() + { + byte[] bytes1 = new byte[128]; + for (int i = 0 ; i < 1000 ; i++) + { + rand.nextBytes(bytes1); + for (int j = 0 ; j < 16 ; j++) + { + byte[] bytes2 = Arrays.copyOf(bytes1, bytes1.length - j); + testTwiddleOneByteComparisons(bytes1, bytes2, 16, true, 1); + testTwiddleOneByteComparisons(bytes1, bytes2, 16, true, -1); + testTwiddleOneByteComparisons(bytes1, bytes2, 16, false, 1); + testTwiddleOneByteComparisons(bytes1, bytes2, 16, false, -1); + testTwiddleOneByteComparisons(bytes1, bytes2, 16, true, 128); + testTwiddleOneByteComparisons(bytes1, bytes2, 16, false, 128); + } + } + } + + private void testTwiddleOneByteComparisons(byte[] bytes1, byte[] bytes2, int count, boolean start, int inc) + { + for (int j = 0 ; j < count ; j++) + { + int index = start ? j : bytes2.length - (j + 1); + bytes2[index] += inc; + testComparisons(bytes1, bytes2); + bytes2[index] -= inc; + } + } + + private static ByteBuffer wrap1(byte[] bytes, boolean direct) + { + return slice(bytes, direct ? dbuf1 : hbuf1); + } + + private static ByteBuffer wrap2(byte[] bytes, boolean direct) + { + return slice(bytes, direct ? dbuf2 : hbuf2); + } + + private static ByteBuffer slice(byte[] bytes, ByteBuffer buf) + { + buf = buf.duplicate(); + buf.position((buf.limit() - bytes.length) / 2); + buf.limit(buf.position() + bytes.length); + buf.duplicate().put(bytes); + return buf; + } + + private void testComparisons(byte[] bytes1, byte[] bytes2) + { + testComparison(bytes1, bytes2); + testComparison(bytes2, bytes1); + testComparison(wrap1(bytes1, false), bytes2); + testComparison(wrap2(bytes2, false), bytes1); + testComparison(wrap1(bytes1, false), wrap2(bytes2, false)); + testComparison(wrap2(bytes2, false), wrap1(bytes1, false)); + testComparison(wrap1(bytes1, true), bytes2); + testComparison(wrap2(bytes2, true), bytes1); + testComparison(wrap1(bytes1, true), wrap2(bytes2, true)); + testComparison(wrap2(bytes2, true), wrap1(bytes1, true)); + testComparison(wrap1(bytes1, true), wrap2(bytes2, false)); + testComparison(wrap1(bytes1, false), wrap2(bytes2, true)); + testComparison(wrap2(bytes2, true), wrap1(bytes1, false)); + testComparison(wrap2(bytes2, false), wrap1(bytes1, true)); + } + + private void testComparison(byte[] bytes1, byte[] bytes2) + { + assert sameComparisonResult(PJO.compare(bytes1, 0, bytes1.length, bytes2, 0, bytes2.length), UO.compare(bytes1, 0, bytes1.length, bytes2, 0, bytes2.length)); + assert sameComparisonResult(PJO.compare(bytes1, 10, bytes1.length - 10, bytes2, 10, bytes2.length - 10), UO.compare(bytes1, 10, bytes1.length - 10, bytes2, 10, bytes2.length - 10)); + } + + private void testComparison(ByteBuffer bytes1, byte[] bytes2) + { + assert sameComparisonResult(PJO.compare(bytes1, bytes2, 0, bytes2.length), UO.compare(bytes1, bytes2, 0, bytes2.length)); + assert sameComparisonResult(PJO.compare(bytes1, bytes2, 10, bytes2.length - 10), UO.compare(bytes1, bytes2, 10, bytes2.length - 10)); + } + + private void testComparison(ByteBuffer bytes1, ByteBuffer bytes2) + { + assert sameComparisonResult(PJO.compare(bytes1, bytes2), UO.compare(bytes1, bytes2)); + } + + static boolean sameComparisonResult(int exp, int act) + { + if (exp < 0) + return act < 0; + if (exp > 0) + return act > 0; + return act == 0; + } +} http://git-wip-us.apache.org/repos/asf/cassandra/blob/75508ec8/test/unit/org/apache/cassandra/utils/IntervalTreeTest.java ---------------------------------------------------------------------- diff --git a/test/unit/org/apache/cassandra/utils/IntervalTreeTest.java b/test/unit/org/apache/cassandra/utils/IntervalTreeTest.java index 55e047a..ea88092 100644 --- a/test/unit/org/apache/cassandra/utils/IntervalTreeTest.java +++ b/test/unit/org/apache/cassandra/utils/IntervalTreeTest.java @@ -23,6 +23,7 @@ package org.apache.cassandra.utils; import org.junit.Test; +import java.lang.reflect.Constructor; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -33,6 +34,8 @@ import static org.junit.Assert.*; import org.apache.cassandra.db.TypeSizes; import org.apache.cassandra.io.ISerializer; import org.apache.cassandra.io.IVersionedSerializer; +import org.apache.cassandra.io.util.DataOutputBuffer; +import org.apache.cassandra.io.util.DataOutputPlus; public class IntervalTreeTest { @@ -144,25 +147,24 @@ public class IntervalTreeTest IVersionedSerializer>> serializer = IntervalTree.serializer( new ISerializer() { - public void serialize(Integer i, DataOutput out) throws IOException { out.writeInt(i); } + public void serialize(Integer i, DataOutputPlus out) throws IOException { out.writeInt(i); } public Integer deserialize(DataInput in) throws IOException { return in.readInt(); } public long serializedSize(Integer i, TypeSizes s) { return 4; } }, new ISerializer() { - public void serialize(String v, DataOutput out) throws IOException { out.writeUTF(v); } + public void serialize(String v, DataOutputPlus out) throws IOException { out.writeUTF(v); } public String deserialize(DataInput in) throws IOException { return in.readUTF(); } public long serializedSize(String v, TypeSizes s) { return v.length(); } }, - Interval.class.getConstructor(Object.class, Object.class, Object.class) + (Constructor>) (Object) Interval.class.getConstructor(Object.class, Object.class, Object.class) ); - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - DataOutputStream out = new DataOutputStream(baos); + DataOutputBuffer out = new DataOutputBuffer(); serializer.serialize(it, out, 0); - DataInputStream in = new DataInputStream(new ByteArrayInputStream(baos.toByteArray())); + DataInputStream in = new DataInputStream(new ByteArrayInputStream(out.toByteArray())); IntervalTree> it2 = serializer.deserialize(in, 0); List> intervals2 = new ArrayList>(); http://git-wip-us.apache.org/repos/asf/cassandra/blob/75508ec8/test/unit/org/apache/cassandra/utils/MerkleTreeTest.java ---------------------------------------------------------------------- diff --git a/test/unit/org/apache/cassandra/utils/MerkleTreeTest.java b/test/unit/org/apache/cassandra/utils/MerkleTreeTest.java index 864b5b8..116e194 100644 --- a/test/unit/org/apache/cassandra/utils/MerkleTreeTest.java +++ b/test/unit/org/apache/cassandra/utils/MerkleTreeTest.java @@ -30,6 +30,7 @@ import org.junit.Before; import org.junit.Test; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.dht.*; +import org.apache.cassandra.io.util.DataOutputBuffer; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.utils.MerkleTree.Hashable; import org.apache.cassandra.utils.MerkleTree.RowHash; @@ -392,7 +393,7 @@ public class MerkleTreeTest byte[] initialhash = mt.hash(full); - ByteArrayDataOutput out = ByteStreams.newDataOutput(); + DataOutputBuffer out = new DataOutputBuffer(); MerkleTree.serializer.serialize(mt, out, MessagingService.current_version); byte[] serialized = out.toByteArray(); http://git-wip-us.apache.org/repos/asf/cassandra/blob/75508ec8/test/unit/org/apache/cassandra/utils/SerializationsTest.java ---------------------------------------------------------------------- diff --git a/test/unit/org/apache/cassandra/utils/SerializationsTest.java b/test/unit/org/apache/cassandra/utils/SerializationsTest.java index f2112c2..976a3eb 100644 --- a/test/unit/org/apache/cassandra/utils/SerializationsTest.java +++ b/test/unit/org/apache/cassandra/utils/SerializationsTest.java @@ -19,6 +19,7 @@ package org.apache.cassandra.utils; import org.apache.cassandra.AbstractSerializationsTester; +import org.apache.cassandra.io.util.DataOutputStreamAndChannel; import org.apache.cassandra.service.StorageService; import org.junit.Test; @@ -35,7 +36,7 @@ public class SerializationsTest extends AbstractSerializationsTester IFilter bf = FilterFactory.getFilter(1000000, 0.0001, offheap); for (int i = 0; i < 100; i++) bf.add(StorageService.getPartitioner().getTokenFactory().toByteArray(StorageService.getPartitioner().getRandomToken())); - DataOutputStream out = getOutput("utils.BloomFilter.bin"); + DataOutputStreamAndChannel out = getOutput("utils.BloomFilter.bin"); FilterFactory.serialize(bf, out); out.close(); } @@ -65,7 +66,7 @@ public class SerializationsTest extends AbstractSerializationsTester data[offsets.length] = 100000; EstimatedHistogram hist2 = new EstimatedHistogram(offsets, data); - DataOutputStream out = getOutput("utils.EstimatedHistogram.bin"); + DataOutputStreamAndChannel out = getOutput("utils.EstimatedHistogram.bin"); EstimatedHistogram.serializer.serialize(hist0, out); EstimatedHistogram.serializer.serialize(hist1, out); EstimatedHistogram.serializer.serialize(hist2, out); http://git-wip-us.apache.org/repos/asf/cassandra/blob/75508ec8/test/unit/org/apache/cassandra/utils/StreamingHistogramTest.java ---------------------------------------------------------------------- diff --git a/test/unit/org/apache/cassandra/utils/StreamingHistogramTest.java b/test/unit/org/apache/cassandra/utils/StreamingHistogramTest.java index 8f75935..13828eb 100644 --- a/test/unit/org/apache/cassandra/utils/StreamingHistogramTest.java +++ b/test/unit/org/apache/cassandra/utils/StreamingHistogramTest.java @@ -25,6 +25,8 @@ import java.io.DataInputStream; import java.io.DataOutputStream; import java.util.*; +import org.apache.cassandra.io.util.DataOutputBuffer; + import static org.junit.Assert.assertEquals; public class StreamingHistogramTest @@ -97,8 +99,8 @@ public class StreamingHistogramTest hist.update(samples[i]); } - ByteArrayOutputStream out = new ByteArrayOutputStream(); - StreamingHistogram.serializer.serialize(hist, new DataOutputStream(out)); + DataOutputBuffer out = new DataOutputBuffer(); + StreamingHistogram.serializer.serialize(hist, out); byte[] bytes = out.toByteArray(); StreamingHistogram deserialized = StreamingHistogram.serializer.deserialize(new DataInputStream(new ByteArrayInputStream(bytes)));