Return-Path: Delivered-To: apmail-cassandra-commits-archive@www.apache.org Received: (qmail 73240 invoked from network); 26 Oct 2010 03:37:00 -0000 Received: from unknown (HELO mail.apache.org) (140.211.11.3) by 140.211.11.9 with SMTP; 26 Oct 2010 03:37:00 -0000 Received: (qmail 30866 invoked by uid 500); 26 Oct 2010 03:37:00 -0000 Delivered-To: apmail-cassandra-commits-archive@cassandra.apache.org Received: (qmail 30835 invoked by uid 500); 26 Oct 2010 03:37:00 -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 30827 invoked by uid 99); 26 Oct 2010 03:37:00 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 26 Oct 2010 03:37:00 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=10.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 26 Oct 2010 03:36:57 +0000 Received: by eris.apache.org (Postfix, from userid 65534) id EA7362388A3D; Tue, 26 Oct 2010 03:36:00 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1027353 [3/4] - in /cassandra/trunk: ./ contrib/client_only/ contrib/pig/src/java/org/apache/cassandra/hadoop/pig/ contrib/word_count/src/ interface/ interface/thrift/gen-java/org/apache/cassandra/thrift/ src/java/org/apache/cassandra/avro... Date: Tue, 26 Oct 2010 03:35:59 -0000 To: commits@cassandra.apache.org From: jbellis@apache.org X-Mailer: svnmailer-1.0.8 Message-Id: <20101026033600.EA7362388A3D@eris.apache.org> Modified: cassandra/trunk/src/java/org/apache/cassandra/dht/Range.java URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/dht/Range.java?rev=1027353&r1=1027352&r2=1027353&view=diff ============================================================================== --- cassandra/trunk/src/java/org/apache/cassandra/dht/Range.java (original) +++ cassandra/trunk/src/java/org/apache/cassandra/dht/Range.java Tue Oct 26 03:35:58 2010 @@ -230,7 +230,7 @@ public class Range extends AbstractBound { l = ((ByteBuffer)left.token).array(); lo = ((ByteBuffer)left.token).position()+((ByteBuffer)left.token).arrayOffset(); - ll = ((ByteBuffer)left.token).limit(); + ll = ((ByteBuffer)left.token).limit()+((ByteBuffer)left.token).arrayOffset(); }else{ //Handles other token types return left.compareTo(right); @@ -246,7 +246,7 @@ public class Range extends AbstractBound { r = ((ByteBuffer)right.token).array(); ro = ((ByteBuffer)right.token).position()+((ByteBuffer)right.token).arrayOffset(); - rl = ((ByteBuffer)right.token).limit(); + rl = ((ByteBuffer)right.token).limit()+((ByteBuffer)right.token).arrayOffset(); } Modified: cassandra/trunk/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java?rev=1027353&r1=1027352&r2=1027353&view=diff ============================================================================== --- cassandra/trunk/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java (original) +++ cassandra/trunk/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java Tue Oct 26 03:35:58 2010 @@ -142,19 +142,19 @@ public class NetworkTopologyStrategy ext /** * This method will generate the QRH object and returns. If the Consistency - * level is DCQUORUM then it will return a DCQRH with a map of local rep - * factor alone. If the consistency level is DCQUORUMSYNC then it will + * level is LOCAL_QUORUM then it will return a DCQRH with a map of local rep + * factor alone. If the consistency level is EACH_QUORUM then it will * return a DCQRH with a map of all the DC rep factor. */ @Override public IWriteResponseHandler getWriteResponseHandler(Collection writeEndpoints, Multimap hintedEndpoints, ConsistencyLevel consistency_level) { - if (consistency_level == ConsistencyLevel.DCQUORUM) + if (consistency_level == ConsistencyLevel.LOCAL_QUORUM) { // block for in this context will be localnodes block. return DatacenterWriteResponseHandler.create(writeEndpoints, hintedEndpoints, consistency_level, table); } - else if (consistency_level == ConsistencyLevel.DCQUORUMSYNC) + else if (consistency_level == ConsistencyLevel.EACH_QUORUM) { return DatacenterSyncWriteResponseHandler.create(writeEndpoints, hintedEndpoints, consistency_level, table); } @@ -163,12 +163,12 @@ public class NetworkTopologyStrategy ext /** * This method will generate the WRH object and returns. If the Consistency - * level is DCQUORUM/DCQUORUMSYNC then it will return a DCQRH. + * level is LOCAL_QUORUM/EACH_QUORUM then it will return a DCQRH. */ @Override public QuorumResponseHandler getQuorumResponseHandler(IResponseResolver responseResolver, ConsistencyLevel consistencyLevel) { - if (consistencyLevel.equals(ConsistencyLevel.DCQUORUM) || consistencyLevel.equals(ConsistencyLevel.DCQUORUMSYNC)) + if (consistencyLevel.equals(ConsistencyLevel.LOCAL_QUORUM) || consistencyLevel.equals(ConsistencyLevel.EACH_QUORUM)) { return new DatacenterQuorumResponseHandler(responseResolver, consistencyLevel, table); } Modified: cassandra/trunk/src/java/org/apache/cassandra/net/OutboundTcpConnection.java URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/net/OutboundTcpConnection.java?rev=1027353&r1=1027352&r2=1027353&view=diff ============================================================================== --- cassandra/trunk/src/java/org/apache/cassandra/net/OutboundTcpConnection.java (original) +++ cassandra/trunk/src/java/org/apache/cassandra/net/OutboundTcpConnection.java Tue Oct 26 03:35:58 2010 @@ -104,7 +104,7 @@ public class OutboundTcpConnection exten { try { - output.write(bb.array(), 0, bb.limit()); + output.write(bb.array(), bb.position()+bb.arrayOffset(), bb.limit()+bb.arrayOffset()); if (queue.peek() == null) { output.flush(); Modified: cassandra/trunk/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java?rev=1027353&r1=1027352&r2=1027353&view=diff ============================================================================== --- cassandra/trunk/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java (original) +++ cassandra/trunk/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java Tue Oct 26 03:35:58 2010 @@ -42,7 +42,7 @@ import com.google.common.collect.Multima import org.apache.cassandra.utils.FBUtilities; /** - * This class blocks for a quorum of responses _in all datacenters_ (CL.DCQUORUMSYNC). + * This class blocks for a quorum of responses _in all datacenters_ (CL.EACH_QUORUM). */ public class DatacenterSyncWriteResponseHandler extends AbstractWriteResponseHandler { @@ -61,7 +61,7 @@ public class DatacenterSyncWriteResponse { // Response is been managed by the map so make it 1 for the superclass. super(writeEndpoints, hintedEndpoints, consistencyLevel); - assert consistencyLevel == ConsistencyLevel.DCQUORUM; + assert consistencyLevel == ConsistencyLevel.LOCAL_QUORUM; strategy = (NetworkTopologyStrategy) Table.open(table).replicationStrategy; Modified: cassandra/trunk/src/java/org/apache/cassandra/service/DatacenterWriteResponseHandler.java URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/service/DatacenterWriteResponseHandler.java?rev=1027353&r1=1027352&r2=1027353&view=diff ============================================================================== --- cassandra/trunk/src/java/org/apache/cassandra/service/DatacenterWriteResponseHandler.java (original) +++ cassandra/trunk/src/java/org/apache/cassandra/service/DatacenterWriteResponseHandler.java Tue Oct 26 03:35:58 2010 @@ -39,7 +39,7 @@ import org.apache.cassandra.utils.FBUtil import com.google.common.collect.Multimap; /** - * This class blocks for a quorum of responses _in the local datacenter only_ (CL.DCQUORUM). + * This class blocks for a quorum of responses _in the local datacenter only_ (CL.LOCAL_QUORUM). */ public class DatacenterWriteResponseHandler extends WriteResponseHandler { @@ -54,7 +54,7 @@ public class DatacenterWriteResponseHand protected DatacenterWriteResponseHandler(Collection writeEndpoints, Multimap hintedEndpoints, ConsistencyLevel consistencyLevel, String table) { super(writeEndpoints, hintedEndpoints, consistencyLevel, table); - assert consistencyLevel == ConsistencyLevel.DCQUORUM; + assert consistencyLevel == ConsistencyLevel.LOCAL_QUORUM; } public static IWriteResponseHandler create(Collection writeEndpoints, Multimap hintedEndpoints, ConsistencyLevel consistencyLevel, String table) Modified: cassandra/trunk/src/java/org/apache/cassandra/utils/ByteBufferUtil.java URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/utils/ByteBufferUtil.java?rev=1027353&r1=1027352&r2=1027353&view=diff ============================================================================== --- cassandra/trunk/src/java/org/apache/cassandra/utils/ByteBufferUtil.java (original) +++ cassandra/trunk/src/java/org/apache/cassandra/utils/ByteBufferUtil.java Tue Oct 26 03:35:58 2010 @@ -18,28 +18,63 @@ */ package org.apache.cassandra.utils; +import java.io.UnsupportedEncodingException; import java.nio.ByteBuffer; import java.nio.charset.Charset; /** * Utility methods to make ByteBuffers less painful + * The following should illustrate the different ways byte buffers can be used + * + * public void testArrayOffet() + * { + * + * byte[] b = "test_slice_array".getBytes(); + * ByteBuffer bb = ByteBuffer.allocate(1024); + * + * assert bb.position() == 0; + * assert bb.limit() == 1024; + * assert bb.capacity() == 1024; + * + * bb.put(b); + * + * assert bb.position() == b.length; + * assert bb.remaining() == bb.limit() - bb.position(); + * + * ByteBuffer bb2 = bb.slice(); + * + * assert bb2.position() == 0; + * + * //slice should begin at other buffers current position + * assert bb2.arrayOffset() == bb.position(); + * + * //to match the position in the underlying array one needs to + * //track arrayOffset + * assert bb2.limit()+bb2.arrayOffset() == bb.limit(); + * + * + * assert bb2.remaining() == bb.remaining(); + * + * } + * + * } * */ public class ByteBufferUtil { public static int compareUnsigned(ByteBuffer o1, ByteBuffer o2) { - return FBUtilities.compareUnsigned(o1.array(), o2.array(), o1.arrayOffset()+o1.position(), o2.arrayOffset()+o2.position(), o1.limit(), o2.limit()); + return FBUtilities.compareUnsigned(o1.array(), o2.array(), o1.arrayOffset()+o1.position(), o2.arrayOffset()+o2.position(), o1.limit()+o1.arrayOffset(), o2.limit()+o2.arrayOffset()); } public static int compare(byte[] o1, ByteBuffer o2) { - return FBUtilities.compareUnsigned(o1, o2.array(), 0, o2.arrayOffset()+o2.position(), o1.length, o2.limit()); + return FBUtilities.compareUnsigned(o1, o2.array(), 0, o2.arrayOffset()+o2.position(), o1.length, o2.limit()+o2.arrayOffset()); } public static int compare(ByteBuffer o1, byte[] o2) { - return FBUtilities.compareUnsigned(o1.array(), o2, o1.arrayOffset()+o1.position(), 0, o1.limit(), o2.length); + return FBUtilities.compareUnsigned(o1.array(), o2, o1.arrayOffset()+o1.position(), 0, o1.limit()+o1.arrayOffset(), o2.length); } public static String string(ByteBuffer b, Charset charset) @@ -51,4 +86,16 @@ public class ByteBufferUtil { { return new String(b.array(), b.arrayOffset() + b.position(), b.remaining()); } + + public static ByteBuffer bytes(String s) + { + try + { + return ByteBuffer.wrap(s.getBytes("UTF-8")); + } + catch (UnsupportedEncodingException e) + { + throw new RuntimeException(e); + } + } } Modified: cassandra/trunk/src/java/org/apache/cassandra/utils/FBUtilities.java URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/utils/FBUtilities.java?rev=1027353&r1=1027352&r2=1027353&view=diff ============================================================================== --- cassandra/trunk/src/java/org/apache/cassandra/utils/FBUtilities.java (original) +++ cassandra/trunk/src/java/org/apache/cassandra/utils/FBUtilities.java Tue Oct 26 03:35:58 2010 @@ -67,14 +67,6 @@ public class FBUtilities public static final int MAX_UNSIGNED_SHORT = 0xFFFF; - /*public static final Comparator byteArrayComparator = new Comparator() - { - public int compare(byte[] o1, byte[] o2) - { - return compareByteArrays(o1, o2); - } - };*/ - /** * Parses a string representing either a fraction, absolute value or percentage. */ @@ -264,12 +256,13 @@ public class FBUtilities { throw new IOException("Corrupt (negative) value length encountered"); } - byte[] value = new byte[length]; + + ByteBuffer bb = ByteBuffer.allocate(length); if (length > 0) { - in.readFully(value); + in.readFully(bb.array(),bb.position(),bb.remaining()); } - return ByteBuffer.wrap(value); + return bb; } public static void writeShortByteArray(ByteBuffer name, DataOutput out) @@ -299,9 +292,10 @@ public class FBUtilities public static ByteBuffer readShortByteArray(DataInput in) throws IOException { - byte[] bytes = new byte[readShortLength(in)]; - in.readFully(bytes); - return ByteBuffer.wrap(bytes); + int length = readShortLength(in); + ByteBuffer bb = ByteBuffer.allocate(length); + in.readFully(bb.array(),bb.position(),bb.remaining()); + return bb; } /** @return null */ @@ -346,7 +340,7 @@ public class FBUtilities public static String bytesToHex(ByteBuffer bytes) { StringBuilder sb = new StringBuilder(); - for (int i=bytes.position()+bytes.arrayOffset(); i rms = new LinkedList(); RowMutation rm; - rm = new RowMutation("Keyspace1", ByteBuffer.wrap("key1".getBytes())); - rm.add(new QueryPath("Standard1", null, ByteBuffer.wrap("Column1".getBytes())), ByteBuffer.wrap("asdf".getBytes()), 0); - rm.add(new QueryPath("Standard1", null, ByteBuffer.wrap("Column2".getBytes())), ByteBuffer.wrap("asdf".getBytes()), 0); + rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("key1")); + rm.add(new QueryPath("Standard1", null, ByteBufferUtil.bytes("Column1")), ByteBufferUtil.bytes("asdf"), 0); + rm.add(new QueryPath("Standard1", null, ByteBufferUtil.bytes("Column2")), ByteBufferUtil.bytes("asdf"), 0); rms.add(rm); ColumnFamilyStore store = Util.writeColumnFamily(rms); @@ -78,7 +79,7 @@ public class ColumnFamilyStoreTest exten List ssTables = table.getAllSSTables(); assertEquals(1, ssTables.size()); ssTables.get(0).forceFilterFailures(); - ColumnFamily cf = store.getColumnFamily(QueryFilter.getIdentityFilter(Util.dk("key2"), new QueryPath("Standard1", null, ByteBuffer.wrap("Column1".getBytes())))); + ColumnFamily cf = store.getColumnFamily(QueryFilter.getIdentityFilter(Util.dk("key2"), new QueryPath("Standard1", null, ByteBufferUtil.bytes("Column1")))); assertNull(cf); } @@ -89,7 +90,7 @@ public class ColumnFamilyStoreTest exten final ColumnFamilyStore store = table.getColumnFamilyStore("Standard2"); RowMutation rm; - rm = new RowMutation("Keyspace1", ByteBuffer.wrap("key1".getBytes())); + rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("key1")); rm.delete(new QueryPath("Standard2", null, null), System.currentTimeMillis()); rm.apply(); @@ -102,7 +103,7 @@ public class ColumnFamilyStoreTest exten assert cf.isMarkedForDelete(); assert cf.getColumnsMap().isEmpty(); - QueryFilter namesFilter = QueryFilter.getNamesFilter(Util.dk("key1"), new QueryPath("Standard2", null, null), ByteBuffer.wrap("a".getBytes())); + QueryFilter namesFilter = QueryFilter.getNamesFilter(Util.dk("key1"), new QueryPath("Standard2", null, null), ByteBufferUtil.bytes("a")); cf = store.getColumnFamily(namesFilter); assert cf.isMarkedForDelete(); assert cf.getColumnsMap().isEmpty(); @@ -121,9 +122,9 @@ public class ColumnFamilyStoreTest exten List result = cfs.getRangeSlice(FBUtilities.EMPTY_BYTE_BUFFER, Util.range(p, "key1", "key2"), 10, - new NamesQueryFilter(ByteBuffer.wrap("asdf".getBytes()))); + new NamesQueryFilter(ByteBufferUtil.bytes("asdf"))); assertEquals(1, result.size()); - assert result.get(0).key.key.equals(ByteBuffer.wrap("key2".getBytes())); + assert result.get(0).key.key.equals(ByteBufferUtil.bytes("key2")); } @Test @@ -131,28 +132,28 @@ public class ColumnFamilyStoreTest exten { RowMutation rm; - rm = new RowMutation("Keyspace1", ByteBuffer.wrap("k1".getBytes())); - rm.add(new QueryPath("Indexed1", null, ByteBuffer.wrap("notbirthdate".getBytes("UTF8"))), FBUtilities.toByteBuffer(1L), 0); - rm.add(new QueryPath("Indexed1", null, ByteBuffer.wrap("birthdate".getBytes("UTF8"))), FBUtilities.toByteBuffer(1L), 0); + rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("k1")); + rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("notbirthdate")), FBUtilities.toByteBuffer(1L), 0); + rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")), FBUtilities.toByteBuffer(1L), 0); rm.apply(); - rm = new RowMutation("Keyspace1", ByteBuffer.wrap("k2".getBytes())); - rm.add(new QueryPath("Indexed1", null, ByteBuffer.wrap("notbirthdate".getBytes("UTF8"))), FBUtilities.toByteBuffer(2L), 0); - rm.add(new QueryPath("Indexed1", null, ByteBuffer.wrap("birthdate".getBytes("UTF8"))), FBUtilities.toByteBuffer(2L), 0); + rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("k2")); + rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("notbirthdate")), FBUtilities.toByteBuffer(2L), 0); + rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")), FBUtilities.toByteBuffer(2L), 0); rm.apply(); - rm = new RowMutation("Keyspace1", ByteBuffer.wrap("k3".getBytes())); - rm.add(new QueryPath("Indexed1", null, ByteBuffer.wrap("notbirthdate".getBytes("UTF8"))), FBUtilities.toByteBuffer(2L), 0); - rm.add(new QueryPath("Indexed1", null, ByteBuffer.wrap("birthdate".getBytes("UTF8"))), FBUtilities.toByteBuffer(1L), 0); + rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("k3")); + rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("notbirthdate")), FBUtilities.toByteBuffer(2L), 0); + rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")), FBUtilities.toByteBuffer(1L), 0); rm.apply(); - rm = new RowMutation("Keyspace1", ByteBuffer.wrap("k4aaaa".getBytes())); - rm.add(new QueryPath("Indexed1", null, ByteBuffer.wrap("notbirthdate".getBytes("UTF8"))), FBUtilities.toByteBuffer(2L), 0); - rm.add(new QueryPath("Indexed1", null, ByteBuffer.wrap("birthdate".getBytes("UTF8"))), FBUtilities.toByteBuffer(3L), 0); + rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("k4aaaa")); + rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("notbirthdate")), FBUtilities.toByteBuffer(2L), 0); + rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")), FBUtilities.toByteBuffer(3L), 0); rm.apply(); // basic single-expression query - IndexExpression expr = new IndexExpression(ByteBuffer.wrap("birthdate".getBytes("UTF8")), IndexOperator.EQ, FBUtilities.toByteBuffer(1L)); + IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexOperator.EQ, FBUtilities.toByteBuffer(1L)); IndexClause clause = new IndexClause(Arrays.asList(expr),FBUtilities.EMPTY_BYTE_BUFFER, 100); IFilter filter = new IdentityQueryFilter(); IPartitioner p = StorageService.getPartitioner(); @@ -161,24 +162,32 @@ public class ColumnFamilyStoreTest exten assert rows != null; assert rows.size() == 2 : StringUtils.join(rows, ","); - assert Arrays.equals("k1".getBytes(), rows.get(0).key.key.array()); - assert Arrays.equals("k3".getBytes(), rows.get(1).key.key.array()); - assert FBUtilities.toByteBuffer(1L).equals( rows.get(0).cf.getColumn(ByteBuffer.wrap("birthdate".getBytes("UTF8"))).value()); - assert FBUtilities.toByteBuffer(1L).equals( rows.get(1).cf.getColumn(ByteBuffer.wrap("birthdate".getBytes("UTF8"))).value()); + + String key = new String(rows.get(0).key.key.array(),rows.get(0).key.key.position(),rows.get(0).key.key.remaining()); + assert "k1".equals( key ); + + key = new String(rows.get(1).key.key.array(),rows.get(1).key.key.position(),rows.get(1).key.key.remaining()); + assert "k3".equals(key); + + assert FBUtilities.toByteBuffer(1L).equals( rows.get(0).cf.getColumn(ByteBufferUtil.bytes("birthdate")).value()); + assert FBUtilities.toByteBuffer(1L).equals( rows.get(1).cf.getColumn(ByteBufferUtil.bytes("birthdate")).value()); // add a second expression - IndexExpression expr2 = new IndexExpression(ByteBuffer.wrap("notbirthdate".getBytes("UTF8")), IndexOperator.GTE, FBUtilities.toByteBuffer(2L)); + IndexExpression expr2 = new IndexExpression(ByteBufferUtil.bytes("notbirthdate"), IndexOperator.GTE, FBUtilities.toByteBuffer(2L)); clause = new IndexClause(Arrays.asList(expr, expr2), FBUtilities.EMPTY_BYTE_BUFFER, 100); rows = Table.open("Keyspace1").getColumnFamilyStore("Indexed1").scan(clause, range, filter); assert rows.size() == 1 : StringUtils.join(rows, ","); - assert Arrays.equals("k3".getBytes(), rows.get(0).key.key.array()); - + key = new String(rows.get(0).key.key.array(),rows.get(0).key.key.position(),rows.get(0).key.key.remaining()); + assert "k3".equals( key ); + // same query again, but with resultset not including the subordinate expression - rows = Table.open("Keyspace1").getColumnFamilyStore("Indexed1").scan(clause, range, new NamesQueryFilter(ByteBuffer.wrap("birthdate".getBytes("UTF8")))); + rows = Table.open("Keyspace1").getColumnFamilyStore("Indexed1").scan(clause, range, new NamesQueryFilter(ByteBufferUtil.bytes("birthdate"))); assert rows.size() == 1 : StringUtils.join(rows, ","); - assert Arrays.equals("k3".getBytes(), rows.get(0).key.key.array()); + key = new String(rows.get(0).key.key.array(),rows.get(0).key.key.position(),rows.get(0).key.key.remaining()); + assert "k3".equals( key ); + assert rows.get(0).cf.getColumnCount() == 1 : rows.get(0).cf; // once more, this time with a slice rowset that needs to be expanded @@ -186,12 +195,14 @@ public class ColumnFamilyStoreTest exten rows = Table.open("Keyspace1").getColumnFamilyStore("Indexed1").scan(clause, range, emptyFilter); assert rows.size() == 1 : StringUtils.join(rows, ","); - assert Arrays.equals("k3".getBytes(), rows.get(0).key.key.array()); + key = new String(rows.get(0).key.key.array(),rows.get(0).key.key.position(),rows.get(0).key.key.remaining()); + assert "k3".equals( key ); + assert rows.get(0).cf.getColumnCount() == 0; // query with index hit but rejected by secondary clause, with a small enough count that just checking count // doesn't tell the scan loop that it's done - IndexExpression expr3 = new IndexExpression(ByteBuffer.wrap("notbirthdate".getBytes("UTF8")), IndexOperator.EQ, FBUtilities.toByteBuffer(-1L)); + IndexExpression expr3 = new IndexExpression(ByteBufferUtil.bytes("notbirthdate"), IndexOperator.EQ, FBUtilities.toByteBuffer(-1L)); clause = new IndexClause(Arrays.asList(expr, expr3), FBUtilities.EMPTY_BYTE_BUFFER, 1); rows = Table.open("Keyspace1").getColumnFamilyStore("Indexed1").scan(clause, range, filter); @@ -204,43 +215,45 @@ public class ColumnFamilyStoreTest exten ColumnFamilyStore cfs = Table.open("Keyspace3").getColumnFamilyStore("Indexed1"); RowMutation rm; - rm = new RowMutation("Keyspace3", ByteBuffer.wrap("k1".getBytes())); - rm.add(new QueryPath("Indexed1", null, ByteBuffer.wrap("birthdate".getBytes("UTF8"))), FBUtilities.toByteBuffer(1L), 0); + rm = new RowMutation("Keyspace3", ByteBufferUtil.bytes("k1")); + rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")), FBUtilities.toByteBuffer(1L), 0); rm.apply(); - IndexExpression expr = new IndexExpression(ByteBuffer.wrap("birthdate".getBytes("UTF8")), IndexOperator.EQ, FBUtilities.toByteBuffer(1L)); + IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexOperator.EQ, FBUtilities.toByteBuffer(1L)); IndexClause clause = new IndexClause(Arrays.asList(expr), FBUtilities.EMPTY_BYTE_BUFFER, 100); IFilter filter = new IdentityQueryFilter(); IPartitioner p = StorageService.getPartitioner(); Range range = new Range(p.getMinimumToken(), p.getMinimumToken()); List rows = cfs.scan(clause, range, filter); assert rows.size() == 1 : StringUtils.join(rows, ","); - assert Arrays.equals("k1".getBytes(), rows.get(0).key.key.array()); - + String key = new String(rows.get(0).key.key.array(),rows.get(0).key.key.position(),rows.get(0).key.key.remaining()); + assert "k1".equals( key ); + // delete the column directly - rm = new RowMutation("Keyspace3", ByteBuffer.wrap("k1".getBytes())); - rm.delete(new QueryPath("Indexed1", null, ByteBuffer.wrap("birthdate".getBytes("UTF8"))), 1); + rm = new RowMutation("Keyspace3", ByteBufferUtil.bytes("k1")); + rm.delete(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")), 1); rm.apply(); rows = cfs.scan(clause, range, filter); assert rows.isEmpty(); // verify that it's not being indexed under the deletion column value either IColumn deletion = rm.getColumnFamilies().iterator().next().iterator().next(); - IndexExpression expr0 = new IndexExpression(ByteBuffer.wrap("birthdate".getBytes("UTF8")), IndexOperator.EQ, deletion.value()); + IndexExpression expr0 = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexOperator.EQ, deletion.value()); IndexClause clause0 = new IndexClause(Arrays.asList(expr0), FBUtilities.EMPTY_BYTE_BUFFER, 100); rows = cfs.scan(clause0, range, filter); assert rows.isEmpty(); // resurrect w/ a newer timestamp - rm = new RowMutation("Keyspace3", ByteBuffer.wrap("k1".getBytes())); - rm.add(new QueryPath("Indexed1", null, ByteBuffer.wrap("birthdate".getBytes("UTF8"))), FBUtilities.toByteBuffer(1L), 2); + rm = new RowMutation("Keyspace3", ByteBufferUtil.bytes("k1")); + rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")), FBUtilities.toByteBuffer(1L), 2); rm.apply(); rows = cfs.scan(clause, range, filter); assert rows.size() == 1 : StringUtils.join(rows, ","); - assert Arrays.equals("k1".getBytes(), rows.get(0).key.key.array()); + key = new String(rows.get(0).key.key.array(),rows.get(0).key.key.position(),rows.get(0).key.key.remaining()); + assert "k1".equals( key ); // delete the entire row - rm = new RowMutation("Keyspace3", ByteBuffer.wrap("k1".getBytes())); + rm = new RowMutation("Keyspace3", ByteBufferUtil.bytes("k1")); rm.delete(new QueryPath("Indexed1"), 3); rm.apply(); rows = cfs.scan(clause, range, filter); @@ -254,14 +267,14 @@ public class ColumnFamilyStoreTest exten // create a row and update the birthdate value, test that the index query fetches the new version RowMutation rm; - rm = new RowMutation("Keyspace2", ByteBuffer.wrap("k1".getBytes())); - rm.add(new QueryPath("Indexed1", null, ByteBuffer.wrap("birthdate".getBytes("UTF8"))), FBUtilities.toByteBuffer(1L), 1); + rm = new RowMutation("Keyspace2", ByteBufferUtil.bytes("k1")); + rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")), FBUtilities.toByteBuffer(1L), 1); rm.apply(); - rm = new RowMutation("Keyspace2", ByteBuffer.wrap("k1".getBytes())); - rm.add(new QueryPath("Indexed1", null, ByteBuffer.wrap("birthdate".getBytes("UTF8"))), FBUtilities.toByteBuffer(2L), 2); + rm = new RowMutation("Keyspace2", ByteBufferUtil.bytes("k1")); + rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")), FBUtilities.toByteBuffer(2L), 2); rm.apply(); - IndexExpression expr = new IndexExpression(ByteBuffer.wrap("birthdate".getBytes("UTF8")), IndexOperator.EQ, FBUtilities.toByteBuffer(1L)); + IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexOperator.EQ, FBUtilities.toByteBuffer(1L)); IndexClause clause = new IndexClause(Arrays.asList(expr), FBUtilities.EMPTY_BYTE_BUFFER, 100); IFilter filter = new IdentityQueryFilter(); IPartitioner p = StorageService.getPartitioner(); @@ -269,18 +282,21 @@ public class ColumnFamilyStoreTest exten List rows = table.getColumnFamilyStore("Indexed1").scan(clause, range, filter); assert rows.size() == 0; - expr = new IndexExpression(ByteBuffer.wrap("birthdate".getBytes("UTF8")), IndexOperator.EQ, FBUtilities.toByteBuffer(2L)); + expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexOperator.EQ, FBUtilities.toByteBuffer(2L)); clause = new IndexClause(Arrays.asList(expr), FBUtilities.EMPTY_BYTE_BUFFER, 100); rows = table.getColumnFamilyStore("Indexed1").scan(clause, range, filter); - assert Arrays.equals("k1".getBytes(), rows.get(0).key.key.array()); - + String key = new String(rows.get(0).key.key.array(),rows.get(0).key.key.position(),rows.get(0).key.key.remaining()); + assert "k1".equals( key ); + // update the birthdate value with an OLDER timestamp, and test that the index ignores this - rm = new RowMutation("Keyspace2", ByteBuffer.wrap("k1".getBytes())); - rm.add(new QueryPath("Indexed1", null, ByteBuffer.wrap("birthdate".getBytes("UTF8"))), FBUtilities.toByteBuffer(3L), 0); + rm = new RowMutation("Keyspace2", ByteBufferUtil.bytes("k1")); + rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")), FBUtilities.toByteBuffer(3L), 0); rm.apply(); rows = table.getColumnFamilyStore("Indexed1").scan(clause, range, filter); - assert Arrays.equals("k1".getBytes(), rows.get(0).key.key.array()); + key = new String(rows.get(0).key.key.array(),rows.get(0).key.key.position(),rows.get(0).key.key.remaining()); + assert "k1".equals( key ); + } @Test @@ -290,38 +306,39 @@ public class ColumnFamilyStoreTest exten // create a row and update the birthdate value, test that the index query fetches the new version RowMutation rm; - rm = new RowMutation("Keyspace1", ByteBuffer.wrap("k1".getBytes())); - rm.add(new QueryPath("Indexed2", null, ByteBuffer.wrap("birthdate".getBytes("UTF8"))), FBUtilities.toByteBuffer(1L), 1); + rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("k1")); + rm.add(new QueryPath("Indexed2", null, ByteBufferUtil.bytes("birthdate")), FBUtilities.toByteBuffer(1L), 1); rm.apply(); ColumnFamilyStore cfs = table.getColumnFamilyStore("Indexed2"); - ColumnDefinition old = cfs.metadata.column_metadata.get(ByteBuffer.wrap("birthdate".getBytes("UTF8"))); + ColumnDefinition old = cfs.metadata.column_metadata.get(ByteBufferUtil.bytes("birthdate")); ColumnDefinition cd = new ColumnDefinition(old.name, old.validator.getClass().getName(), IndexType.KEYS, "birthdate_index"); cfs.addIndex(cd); - while (!SystemTable.isIndexBuilt("Keyspace1", cfs.getIndexedColumnFamilyStore(ByteBuffer.wrap("birthdate".getBytes("UTF8"))).columnFamily)) + while (!SystemTable.isIndexBuilt("Keyspace1", cfs.getIndexedColumnFamilyStore(ByteBufferUtil.bytes("birthdate")).columnFamily)) TimeUnit.MILLISECONDS.sleep(100); - IndexExpression expr = new IndexExpression(ByteBuffer.wrap("birthdate".getBytes("UTF8")), IndexOperator.EQ, FBUtilities.toByteBuffer(1L)); + IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexOperator.EQ, FBUtilities.toByteBuffer(1L)); IndexClause clause = new IndexClause(Arrays.asList(expr), FBUtilities.EMPTY_BYTE_BUFFER, 100); IFilter filter = new IdentityQueryFilter(); IPartitioner p = StorageService.getPartitioner(); Range range = new Range(p.getMinimumToken(), p.getMinimumToken()); List rows = table.getColumnFamilyStore("Indexed2").scan(clause, range, filter); assert rows.size() == 1 : StringUtils.join(rows, ","); - assert Arrays.equals("k1".getBytes(), rows.get(0).key.key.array()); + String key = new String(rows.get(0).key.key.array(),rows.get(0).key.key.position(),rows.get(0).key.key.remaining()); + assert "k1".equals( key ); } private ColumnFamilyStore insertKey1Key2() throws IOException, ExecutionException, InterruptedException { List rms = new LinkedList(); RowMutation rm; - rm = new RowMutation("Keyspace2", ByteBuffer.wrap("key1".getBytes())); - rm.add(new QueryPath("Standard1", null, ByteBuffer.wrap("Column1".getBytes())), ByteBuffer.wrap("asdf".getBytes()), 0); + rm = new RowMutation("Keyspace2", ByteBufferUtil.bytes("key1")); + rm.add(new QueryPath("Standard1", null, ByteBufferUtil.bytes("Column1")), ByteBufferUtil.bytes("asdf"), 0); rms.add(rm); Util.writeColumnFamily(rms); - rm = new RowMutation("Keyspace2", ByteBuffer.wrap("key2".getBytes())); - rm.add(new QueryPath("Standard1", null, ByteBuffer.wrap("Column1".getBytes())), ByteBuffer.wrap("asdf".getBytes()), 0); + rm = new RowMutation("Keyspace2", ByteBufferUtil.bytes("key2")); + rm.add(new QueryPath("Standard1", null, ByteBufferUtil.bytes("Column1")), ByteBufferUtil.bytes("asdf"), 0); rms.add(rm); return Util.writeColumnFamily(rms); } Modified: cassandra/trunk/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java?rev=1027353&r1=1027352&r2=1027353&view=diff ============================================================================== --- cassandra/trunk/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java (original) +++ cassandra/trunk/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java Tue Oct 26 03:35:58 2010 @@ -31,6 +31,8 @@ import org.junit.Test; import org.apache.cassandra.io.util.DataOutputBuffer; import org.apache.cassandra.db.filter.QueryPath; import static org.apache.cassandra.Util.column; +import org.apache.cassandra.utils.ByteBufferUtil; + public class ColumnFamilyTest extends SchemaLoader { @@ -78,7 +80,8 @@ public class ColumnFamilyTest extends Sc cf = ColumnFamily.serializer().deserialize(new DataInputStream(bufIn)); for (String cName : map.navigableKeySet()) { - assert new String(cf.getColumn(ByteBuffer.wrap(cName.getBytes())).value().array()).equals(map.get(cName)); + ByteBuffer val = cf.getColumn(ByteBuffer.wrap(cName.getBytes())).value(); + assert new String(val.array(),val.position(),val.remaining()).equals(map.get(cName)); } assert cf.getColumnNames().size() == map.size(); } @@ -105,7 +108,7 @@ public class ColumnFamilyTest extends Sc cf.addColumn(column("col1", "val2", 2)); // same timestamp, new value cf.addColumn(column("col1", "val3", 1)); // older timestamp -- should be ignored - assert Arrays.equals("val2".getBytes(), cf.getColumn(ByteBuffer.wrap("col1".getBytes())).value().array()); + assert ByteBufferUtil.bytes("val2").equals(cf.getColumn(ByteBufferUtil.bytes("col1")).value()); } @Test @@ -114,33 +117,33 @@ public class ColumnFamilyTest extends Sc ColumnFamily cf_new = ColumnFamily.create("Keyspace1", "Standard1"); ColumnFamily cf_old = ColumnFamily.create("Keyspace1", "Standard1"); ColumnFamily cf_result = ColumnFamily.create("Keyspace1", "Standard1"); - ByteBuffer val = ByteBuffer.wrap("sample value".getBytes()); - ByteBuffer val2 = ByteBuffer.wrap("x value ".getBytes()); + ByteBuffer val = ByteBufferUtil.bytes("sample value"); + ByteBuffer val2 = ByteBufferUtil.bytes("x value "); // exercise addColumn(QueryPath, ...) - cf_new.addColumn(QueryPath.column(ByteBuffer.wrap("col1".getBytes())), val, 3); - cf_new.addColumn(QueryPath.column(ByteBuffer.wrap("col2".getBytes())), val, 4); + cf_new.addColumn(QueryPath.column(ByteBufferUtil.bytes("col1")), val, 3); + cf_new.addColumn(QueryPath.column(ByteBufferUtil.bytes("col2")), val, 4); - cf_old.addColumn(QueryPath.column(ByteBuffer.wrap("col2".getBytes())), val2, 1); - cf_old.addColumn(QueryPath.column(ByteBuffer.wrap("col3".getBytes())), val2, 2); + cf_old.addColumn(QueryPath.column(ByteBufferUtil.bytes("col2")), val2, 1); + cf_old.addColumn(QueryPath.column(ByteBufferUtil.bytes("col3")), val2, 2); cf_result.addAll(cf_new); cf_result.addAll(cf_old); assert 3 == cf_result.getColumnCount() : "Count is " + cf_new.getColumnCount(); //addcolumns will only add if timestamp >= old timestamp - assert val.equals(cf_result.getColumn(ByteBuffer.wrap("col2".getBytes())).value()); + assert val.equals(cf_result.getColumn(ByteBufferUtil.bytes("col2")).value()); // check that tombstone wins timestamp ties - cf_result.addTombstone(ByteBuffer.wrap("col1".getBytes()), 0, 3); - assert cf_result.getColumn(ByteBuffer.wrap("col1".getBytes())).isMarkedForDelete(); - cf_result.addColumn(QueryPath.column(ByteBuffer.wrap("col1".getBytes())), val2, 3); - assert cf_result.getColumn(ByteBuffer.wrap("col1".getBytes())).isMarkedForDelete(); + cf_result.addTombstone(ByteBufferUtil.bytes("col1"), 0, 3); + assert cf_result.getColumn(ByteBufferUtil.bytes("col1")).isMarkedForDelete(); + cf_result.addColumn(QueryPath.column(ByteBufferUtil.bytes("col1")), val2, 3); + assert cf_result.getColumn(ByteBufferUtil.bytes("col1")).isMarkedForDelete(); // check that column value wins timestamp ties in absence of tombstone - cf_result.addColumn(QueryPath.column(ByteBuffer.wrap("col3".getBytes())), val, 2); - assert cf_result.getColumn(ByteBuffer.wrap("col3".getBytes())).value().equals(val2); - cf_result.addColumn(QueryPath.column(ByteBuffer.wrap("col3".getBytes())), ByteBuffer.wrap("z".getBytes()), 2); - assert cf_result.getColumn(ByteBuffer.wrap("col3".getBytes())).value().equals(ByteBuffer.wrap("z".getBytes())); + cf_result.addColumn(QueryPath.column(ByteBufferUtil.bytes("col3")), val, 2); + assert cf_result.getColumn(ByteBufferUtil.bytes("col3")).value().equals(val2); + cf_result.addColumn(QueryPath.column(ByteBufferUtil.bytes("col3")), ByteBufferUtil.bytes("z"), 2); + assert cf_result.getColumn(ByteBufferUtil.bytes("col3")).value().equals(ByteBufferUtil.bytes("z")); } } Modified: cassandra/trunk/test/unit/org/apache/cassandra/db/CompactionsPurgeTest.java URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/db/CompactionsPurgeTest.java?rev=1027353&r1=1027352&r2=1027353&view=diff ============================================================================== --- cassandra/trunk/test/unit/org/apache/cassandra/db/CompactionsPurgeTest.java (original) +++ cassandra/trunk/test/unit/org/apache/cassandra/db/CompactionsPurgeTest.java Tue Oct 26 03:35:58 2010 @@ -35,6 +35,8 @@ import org.apache.cassandra.Util; import static junit.framework.Assert.assertEquals; import static org.apache.cassandra.db.TableTest.assertColumns; +import org.apache.cassandra.utils.ByteBufferUtil; + public class CompactionsPurgeTest extends CleanupHelper { @@ -209,18 +211,18 @@ public class CompactionsPurgeTest extend // inserts rm = new RowMutation(TABLE1, key1.key); - rm.add(new QueryPath(cfName, null, ByteBuffer.wrap("1".getBytes())), FBUtilities.EMPTY_BYTE_BUFFER, 0); + rm.add(new QueryPath(cfName, null, ByteBufferUtil.bytes("1")), FBUtilities.EMPTY_BYTE_BUFFER, 0); rm.apply(); rm = new RowMutation(TABLE1, key2.key); - rm.add(new QueryPath(cfName, null, ByteBuffer.wrap("2".getBytes())), FBUtilities.EMPTY_BYTE_BUFFER, 0); + rm.add(new QueryPath(cfName, null, ByteBufferUtil.bytes("2")), FBUtilities.EMPTY_BYTE_BUFFER, 0); rm.apply(); // deletes rm = new RowMutation(TABLE1, key1.key); - rm.delete(new QueryPath(cfName, null, ByteBuffer.wrap("1".getBytes())), 1); + rm.delete(new QueryPath(cfName, null, ByteBufferUtil.bytes("1")), 1); rm.apply(); rm = new RowMutation(TABLE1, key2.key); - rm.delete(new QueryPath(cfName, null, ByteBuffer.wrap("2".getBytes())), 1); + rm.delete(new QueryPath(cfName, null, ByteBufferUtil.bytes("2")), 1); rm.apply(); // After a flush, the cache should expand to be X% of indices * INDEX_INTERVAL. Modified: cassandra/trunk/test/unit/org/apache/cassandra/db/DefsTest.java URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/db/DefsTest.java?rev=1027353&r1=1027352&r2=1027353&view=diff ============================================================================== --- cassandra/trunk/test/unit/org/apache/cassandra/db/DefsTest.java (original) +++ cassandra/trunk/test/unit/org/apache/cassandra/db/DefsTest.java Tue Oct 26 03:35:58 2010 @@ -18,29 +18,50 @@ package org.apache.cassandra.db; +import static org.junit.Assert.assertEquals; + import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; -import java.util.*; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.SortedSet; +import java.util.TreeSet; +import java.util.UUID; import java.util.concurrent.ExecutionException; -import org.apache.cassandra.io.SerDeUtils; -import org.apache.cassandra.locator.OldNetworkTopologyStrategy; -import org.apache.cassandra.thrift.CfDef; -import org.apache.cassandra.thrift.ColumnDef; -import org.junit.Test; - import org.apache.cassandra.CleanupHelper; import org.apache.cassandra.Util; -import org.apache.cassandra.config.*; +import org.apache.cassandra.config.CFMetaData; +import org.apache.cassandra.config.ColumnDefinition; +import org.apache.cassandra.config.ConfigurationException; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.KSMetaData; import org.apache.cassandra.db.filter.QueryFilter; import org.apache.cassandra.db.filter.QueryPath; import org.apache.cassandra.db.marshal.BytesType; import org.apache.cassandra.db.marshal.UTF8Type; -import org.apache.cassandra.db.migration.*; +import org.apache.cassandra.db.migration.AddColumnFamily; +import org.apache.cassandra.db.migration.AddKeyspace; +import org.apache.cassandra.db.migration.DropColumnFamily; +import org.apache.cassandra.db.migration.DropKeyspace; +import org.apache.cassandra.db.migration.Migration; +import org.apache.cassandra.db.migration.RenameColumnFamily; +import org.apache.cassandra.db.migration.RenameKeyspace; +import org.apache.cassandra.db.migration.UpdateColumnFamily; +import org.apache.cassandra.db.migration.UpdateKeyspace; +import org.apache.cassandra.locator.OldNetworkTopologyStrategy; import org.apache.cassandra.locator.SimpleStrategy; +import org.apache.cassandra.thrift.CfDef; +import org.apache.cassandra.thrift.ColumnDef; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.UUIDGen; +import org.junit.Test; +import org.apache.cassandra.utils.ByteBufferUtil; + public class DefsTest extends CleanupHelper { @@ -172,16 +193,16 @@ public class DefsTest extends CleanupHel // now read and write to it. DecoratedKey dk = Util.dk("key0"); RowMutation rm = new RowMutation(ks, dk.key); - rm.add(new QueryPath(cf, null, ByteBuffer.wrap("col0".getBytes())), ByteBuffer.wrap("value0".getBytes()), 1L); + rm.add(new QueryPath(cf, null, ByteBufferUtil.bytes("col0")), ByteBufferUtil.bytes("value0"), 1L); rm.apply(); ColumnFamilyStore store = Table.open(ks).getColumnFamilyStore(cf); assert store != null; store.forceBlockingFlush(); - ColumnFamily cfam = store.getColumnFamily(QueryFilter.getNamesFilter(dk, new QueryPath(cf), ByteBuffer.wrap("col0".getBytes()))); - assert cfam.getColumn(ByteBuffer.wrap("col0".getBytes())) != null; - IColumn col = cfam.getColumn(ByteBuffer.wrap("col0".getBytes())); - assert ByteBuffer.wrap("value0".getBytes()).equals(col.value()); + ColumnFamily cfam = store.getColumnFamily(QueryFilter.getNamesFilter(dk, new QueryPath(cf), ByteBufferUtil.bytes("col0"))); + assert cfam.getColumn(ByteBufferUtil.bytes("col0")) != null; + IColumn col = cfam.getColumn(ByteBufferUtil.bytes("col0")); + assert ByteBufferUtil.bytes("value0").equals(col.value()); } @Test @@ -197,7 +218,7 @@ public class DefsTest extends CleanupHel // write some data, force a flush, then verify that files exist on disk. RowMutation rm = new RowMutation(ks.name, dk.key); for (int i = 0; i < 100; i++) - rm.add(new QueryPath(cfm.cfName, null, ByteBuffer.wrap(("col" + i).getBytes())), ByteBuffer.wrap("anyvalue".getBytes()), 1L); + rm.add(new QueryPath(cfm.cfName, null, ByteBuffer.wrap(("col" + i).getBytes())), ByteBufferUtil.bytes("anyvalue"), 1L); rm.apply(); ColumnFamilyStore store = Table.open(cfm.tableName).getColumnFamilyStore(cfm.cfName); assert store != null; @@ -214,7 +235,7 @@ public class DefsTest extends CleanupHel boolean success = true; try { - rm.add(new QueryPath("Standard1", null, ByteBuffer.wrap("col0".getBytes())), ByteBuffer.wrap("value0".getBytes()), 1L); + rm.add(new QueryPath("Standard1", null, ByteBufferUtil.bytes("col0")), ByteBufferUtil.bytes("value0"), 1L); rm.apply(); } catch (Throwable th) @@ -243,7 +264,7 @@ public class DefsTest extends CleanupHel // write some data, force a flush, then verify that files exist on disk. RowMutation rm = new RowMutation(ks.name, dk.key); for (int i = 0; i < 100; i++) - rm.add(new QueryPath(oldCfm.cfName, null, ByteBuffer.wrap(("col" + i).getBytes())), ByteBuffer.wrap("anyvalue".getBytes()), 1L); + rm.add(new QueryPath(oldCfm.cfName, null, ByteBuffer.wrap(("col" + i).getBytes())), ByteBufferUtil.bytes("anyvalue"), 1L); rm.apply(); ColumnFamilyStore store = Table.open(oldCfm.tableName).getColumnFamilyStore(oldCfm.cfName); assert store != null; @@ -268,13 +289,13 @@ public class DefsTest extends CleanupHel // do some writes rm = new RowMutation(ks.name, dk.key); - rm.add(new QueryPath(cfName, null, ByteBuffer.wrap("col5".getBytes())), ByteBuffer.wrap("updated".getBytes()), 2L); + rm.add(new QueryPath(cfName, null, ByteBufferUtil.bytes("col5")), ByteBufferUtil.bytes("updated"), 2L); rm.apply(); store.forceBlockingFlush(); - cfam = store.getColumnFamily(QueryFilter.getNamesFilter(dk, new QueryPath(cfName), ByteBuffer.wrap("col5".getBytes()))); + cfam = store.getColumnFamily(QueryFilter.getNamesFilter(dk, new QueryPath(cfName), ByteBufferUtil.bytes("col5"))); assert cfam.getColumnCount() == 1; - assert cfam.getColumn(ByteBuffer.wrap("col5".getBytes())).value().equals( ByteBuffer.wrap("updated".getBytes())); + assert cfam.getColumn(ByteBufferUtil.bytes("col5")).value().equals( ByteBufferUtil.bytes("updated")); } @Test @@ -292,16 +313,16 @@ public class DefsTest extends CleanupHel // test reads and writes. RowMutation rm = new RowMutation(newCf.tableName, dk.key); - rm.add(new QueryPath(newCf.cfName, null, ByteBuffer.wrap("col0".getBytes())), ByteBuffer.wrap("value0".getBytes()), 1L); + rm.add(new QueryPath(newCf.cfName, null, ByteBufferUtil.bytes("col0")), ByteBufferUtil.bytes("value0"), 1L); rm.apply(); ColumnFamilyStore store = Table.open(newCf.tableName).getColumnFamilyStore(newCf.cfName); assert store != null; store.forceBlockingFlush(); - ColumnFamily cfam = store.getColumnFamily(QueryFilter.getNamesFilter(dk, new QueryPath(newCf.cfName), ByteBuffer.wrap("col0".getBytes()))); - assert cfam.getColumn(ByteBuffer.wrap("col0".getBytes())) != null; - IColumn col = cfam.getColumn(ByteBuffer.wrap("col0".getBytes())); - assert ByteBuffer.wrap("value0".getBytes()).equals(col.value()); + ColumnFamily cfam = store.getColumnFamily(QueryFilter.getNamesFilter(dk, new QueryPath(newCf.cfName), ByteBufferUtil.bytes("col0"))); + assert cfam.getColumn(ByteBufferUtil.bytes("col0")) != null; + IColumn col = cfam.getColumn(ByteBufferUtil.bytes("col0")); + assert ByteBufferUtil.bytes("value0").equals(col.value()); } @Test @@ -317,7 +338,7 @@ public class DefsTest extends CleanupHel // write some data, force a flush, then verify that files exist on disk. RowMutation rm = new RowMutation(ks.name, dk.key); for (int i = 0; i < 100; i++) - rm.add(new QueryPath(cfm.cfName, null, ByteBuffer.wrap(("col" + i).getBytes())), ByteBuffer.wrap("anyvalue".getBytes()), 1L); + rm.add(new QueryPath(cfm.cfName, null, ByteBuffer.wrap(("col" + i).getBytes())), ByteBufferUtil.bytes("anyvalue"), 1L); rm.apply(); ColumnFamilyStore store = Table.open(cfm.tableName).getColumnFamilyStore(cfm.cfName); assert store != null; @@ -333,7 +354,7 @@ public class DefsTest extends CleanupHel boolean success = true; try { - rm.add(new QueryPath("Standard1", null, ByteBuffer.wrap("col0".getBytes())), ByteBuffer.wrap("value0".getBytes()), 1L); + rm.add(new QueryPath("Standard1", null, ByteBufferUtil.bytes("col0")), ByteBufferUtil.bytes("value0"), 1L); rm.apply(); } catch (Throwable th) @@ -367,7 +388,7 @@ public class DefsTest extends CleanupHel // write some data that we hope to read back later. RowMutation rm = new RowMutation(oldKs.name, dk.key); for (int i = 0; i < 10; i++) - rm.add(new QueryPath(cfName, null, ByteBuffer.wrap(("col" + i).getBytes())), ByteBuffer.wrap("value".getBytes()), 1L); + rm.add(new QueryPath(cfName, null, ByteBuffer.wrap(("col" + i).getBytes())), ByteBufferUtil.bytes("value"), 1L); rm.apply(); ColumnFamilyStore store = Table.open(oldKs.name).getColumnFamilyStore(cfName); assert store != null; @@ -396,11 +417,11 @@ public class DefsTest extends CleanupHel } // write on old should fail. - rm = new RowMutation(oldKs.name, ByteBuffer.wrap("any key will do".getBytes())); + rm = new RowMutation(oldKs.name, ByteBufferUtil.bytes("any key will do")); boolean success = true; try { - rm.add(new QueryPath(cfName, null, ByteBuffer.wrap("col0".getBytes())), ByteBuffer.wrap("value0".getBytes()), 1L); + rm.add(new QueryPath(cfName, null, ByteBufferUtil.bytes("col0")), ByteBufferUtil.bytes("value0"), 1L); rm.apply(); } catch (Throwable th) @@ -411,7 +432,7 @@ public class DefsTest extends CleanupHel // write on new should work. rm = new RowMutation(newKsName, dk.key); - rm.add(new QueryPath(cfName, null, ByteBuffer.wrap("col0".getBytes())), ByteBuffer.wrap("newvalue".getBytes()), 2L); + rm.add(new QueryPath(cfName, null, ByteBufferUtil.bytes("col0")), ByteBufferUtil.bytes("newvalue"), 2L); rm.apply(); store = Table.open(newKs.name).getColumnFamilyStore(cfName); assert store != null; @@ -419,14 +440,17 @@ public class DefsTest extends CleanupHel // read on new should work. SortedSet cols = new TreeSet(BytesType.instance); - cols.add(ByteBuffer.wrap("col0".getBytes())); - cols.add(ByteBuffer.wrap("col1".getBytes())); + cols.add(ByteBufferUtil.bytes("col0")); + cols.add(ByteBufferUtil.bytes("col1")); ColumnFamily cfam = store.getColumnFamily(QueryFilter.getNamesFilter(dk, new QueryPath(cfName), cols)); assert cfam.getColumnCount() == cols.size(); // tests new write. - assert Arrays.equals(cfam.getColumn(ByteBuffer.wrap("col0".getBytes())).value().array(), "newvalue".getBytes()); + + ByteBuffer val = cfam.getColumn(ByteBufferUtil.bytes("col0")).value(); + assertEquals( new String(val.array(),val.position(),val.remaining()), "newvalue"); // tests old write. - assert Arrays.equals(cfam.getColumn(ByteBuffer.wrap("col1".getBytes())).value().array(), "value".getBytes()); + val = cfam.getColumn(ByteBufferUtil.bytes("col1")).value(); + assertEquals( new String(val.array(),val.position(),val.remaining()), "value"); } @Test @@ -453,16 +477,16 @@ public class DefsTest extends CleanupHel // now read and write to it. DecoratedKey dk = Util.dk("key0"); RowMutation rm = new RowMutation(newKs.name, dk.key); - rm.add(new QueryPath(newCf.cfName, null, ByteBuffer.wrap("col0".getBytes())), ByteBuffer.wrap("value0".getBytes()), 1L); + rm.add(new QueryPath(newCf.cfName, null, ByteBufferUtil.bytes("col0")), ByteBufferUtil.bytes("value0"), 1L); rm.apply(); ColumnFamilyStore store = Table.open(newKs.name).getColumnFamilyStore(newCf.cfName); assert store != null; store.forceBlockingFlush(); - ColumnFamily cfam = store.getColumnFamily(QueryFilter.getNamesFilter(dk, new QueryPath(newCf.cfName), ByteBuffer.wrap("col0".getBytes()))); - assert cfam.getColumn(ByteBuffer.wrap("col0".getBytes())) != null; - IColumn col = cfam.getColumn(ByteBuffer.wrap("col0".getBytes())); - assert ByteBuffer.wrap("value0".getBytes()).equals(col.value()); + ColumnFamily cfam = store.getColumnFamily(QueryFilter.getNamesFilter(dk, new QueryPath(newCf.cfName), ByteBufferUtil.bytes("col0"))); + assert cfam.getColumn(ByteBufferUtil.bytes("col0")) != null; + IColumn col = cfam.getColumn(ByteBufferUtil.bytes("col0")); + assert ByteBufferUtil.bytes("value0").equals(col.value()); } @Test Modified: cassandra/trunk/test/unit/org/apache/cassandra/db/NameSortTest.java URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/db/NameSortTest.java?rev=1027353&r1=1027352&r2=1027353&view=diff ============================================================================== --- cassandra/trunk/test/unit/org/apache/cassandra/db/NameSortTest.java (original) +++ cassandra/trunk/test/unit/org/apache/cassandra/db/NameSortTest.java Tue Oct 26 03:35:58 2010 @@ -1,40 +1,35 @@ /* -* 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. -*/ + * 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.db; +import static junit.framework.Assert.assertEquals; +import static org.apache.cassandra.Util.addMutation; + import java.io.IOException; -import java.util.concurrent.ExecutionException; -import java.util.Collection; -import java.util.Arrays; import java.nio.ByteBuffer; - -import org.apache.cassandra.Util; -import org.junit.Test; +import java.util.Collection; +import java.util.concurrent.ExecutionException; import org.apache.cassandra.CleanupHelper; import org.apache.cassandra.Util; -import static org.apache.cassandra.Util.addMutation; - import org.apache.cassandra.db.filter.QueryPath; -import org.apache.cassandra.utils.ByteBufferUtil; - -import static junit.framework.Assert.assertEquals; +import org.junit.Test; public class NameSortTest extends CleanupHelper { @@ -59,7 +54,6 @@ public class NameSortTest extends Cleanu testNameSort(100); } - private void testNameSort(int N) throws IOException, ExecutionException, InterruptedException { Table table = Table.open("Keyspace1"); @@ -109,9 +103,10 @@ public class NameSortTest extends Cleanu Collection columns = cf.getSortedColumns(); for (IColumn column : columns) { - int j = Integer.valueOf(new String(column.name().array()).split("-")[1]); + int j = Integer.valueOf(new String(column.name().array(),column.name().position(),column.name().remaining()).split("-")[1]); byte[] bytes = j % 2 == 0 ? "a".getBytes() : "b".getBytes(); - assert Arrays.equals(bytes, column.value().array()); + assertEquals(new String(bytes), new String(column.value().array(), column.value().position(), column + .value().remaining())); } cf = Util.getColumnFamily(table, key, "Super1"); @@ -120,14 +115,15 @@ public class NameSortTest extends Cleanu assert superColumns.size() == 8 : cf; for (IColumn superColumn : superColumns) { - int j = Integer.valueOf(new String(superColumn.name().array()).split("-")[1]); + int j = Integer.valueOf(new String(superColumn.name().array(),superColumn.name().position(),superColumn.name().remaining()).split("-")[1]); Collection subColumns = superColumn.getSubColumns(); assert subColumns.size() == 4; for (IColumn subColumn : subColumns) { - long k = subColumn.name().getLong(subColumn.name().position()+subColumn.name().arrayOffset()); + long k = subColumn.name().getLong(subColumn.name().position() + subColumn.name().arrayOffset()); byte[] bytes = (j + k) % 2 == 0 ? "a".getBytes() : "b".getBytes(); - assert Arrays.equals(bytes, subColumn.value().array()); + assertEquals(new String(bytes), new String(subColumn.value().array(), subColumn.value().position(), + subColumn.value().remaining())); } } } Modified: cassandra/trunk/test/unit/org/apache/cassandra/db/OneCompactionTest.java URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/db/OneCompactionTest.java?rev=1027353&r1=1027352&r2=1027353&view=diff ============================================================================== --- cassandra/trunk/test/unit/org/apache/cassandra/db/OneCompactionTest.java (original) +++ cassandra/trunk/test/unit/org/apache/cassandra/db/OneCompactionTest.java Tue Oct 26 03:35:58 2010 @@ -32,6 +32,8 @@ import static junit.framework.Assert.ass import org.apache.cassandra.db.filter.QueryPath; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.CleanupHelper; +import org.apache.cassandra.utils.ByteBufferUtil; + public class OneCompactionTest extends CleanupHelper { @@ -46,7 +48,7 @@ public class OneCompactionTest extends C for (int j = 0; j < insertsPerTable; j++) { DecoratedKey key = Util.dk(String.valueOf(j)); RowMutation rm = new RowMutation("Keyspace1", key.key); - rm.add(new QueryPath(columnFamilyName, null, ByteBuffer.wrap("0".getBytes())), FBUtilities.EMPTY_BYTE_BUFFER, j); + rm.add(new QueryPath(columnFamilyName, null, ByteBufferUtil.bytes("0")), FBUtilities.EMPTY_BYTE_BUFFER, j); rm.apply(); inserted.add(key); store.forceBlockingFlush(); Modified: cassandra/trunk/test/unit/org/apache/cassandra/db/ReadMessageTest.java URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/db/ReadMessageTest.java?rev=1027353&r1=1027352&r2=1027353&view=diff ============================================================================== --- cassandra/trunk/test/unit/org/apache/cassandra/db/ReadMessageTest.java (original) +++ cassandra/trunk/test/unit/org/apache/cassandra/db/ReadMessageTest.java Tue Oct 26 03:35:58 2010 @@ -34,6 +34,8 @@ import org.apache.cassandra.io.util.Data import org.apache.cassandra.utils.FBUtilities; import org.apache.commons.lang.ArrayUtils; import org.junit.Test; +import org.apache.cassandra.utils.ByteBufferUtil; + public class ReadMessageTest extends SchemaLoader { @@ -41,8 +43,8 @@ public class ReadMessageTest extends Sch public void testMakeReadMessage() throws IOException { ArrayList colList = new ArrayList(); - colList.add(ByteBuffer.wrap("col1".getBytes())); - colList.add(ByteBuffer.wrap("col2".getBytes())); + colList.add(ByteBufferUtil.bytes("col1")); + colList.add(ByteBufferUtil.bytes("col2")); ReadCommand rm, rm2; DecoratedKey dk = Util.dk("row1"); @@ -55,7 +57,7 @@ public class ReadMessageTest extends Sch rm2 = serializeAndDeserializeReadMessage(rm); assert rm2.toString().equals(rm.toString()); - rm = new SliceFromReadCommand("Keyspace1", dk.key, new QueryPath("Standard1"), ByteBuffer.wrap("a".getBytes()), ByteBuffer.wrap("z".getBytes()), true, 5); + rm = new SliceFromReadCommand("Keyspace1", dk.key, new QueryPath("Standard1"), ByteBufferUtil.bytes("a"), ByteBufferUtil.bytes("z"), true, 5); rm2 = serializeAndDeserializeReadMessage(rm); assertEquals(rm2.toString(), rm.toString()); @@ -63,7 +65,7 @@ public class ReadMessageTest extends Sch rm2 = serializeAndDeserializeReadMessage(rm); assert rm2.toString().equals(rm.toString()); - rm = new SliceFromReadCommand("Keyspace1", dk.key, new QueryPath("Standard1"), ByteBuffer.wrap("a".getBytes()), ByteBuffer.wrap("z".getBytes()), true, 5); + rm = new SliceFromReadCommand("Keyspace1", dk.key, new QueryPath("Standard1"), ByteBufferUtil.bytes("a"), ByteBufferUtil.bytes("z"), true, 5); rm2 = serializeAndDeserializeReadMessage(rm); assertEquals(rm2.toString(), rm.toString()); } @@ -88,12 +90,12 @@ public class ReadMessageTest extends Sch // add data rm = new RowMutation("Keyspace1", dk.key); - rm.add(new QueryPath("Standard1", null, ByteBuffer.wrap("Column1".getBytes())), ByteBuffer.wrap("abcd".getBytes()), 0); + rm.add(new QueryPath("Standard1", null, ByteBufferUtil.bytes("Column1")), ByteBufferUtil.bytes("abcd"), 0); rm.apply(); - ReadCommand command = new SliceByNamesReadCommand("Keyspace1", dk.key, new QueryPath("Standard1"), Arrays.asList(ByteBuffer.wrap("Column1".getBytes()))); + ReadCommand command = new SliceByNamesReadCommand("Keyspace1", dk.key, new QueryPath("Standard1"), Arrays.asList(ByteBufferUtil.bytes("Column1"))); Row row = command.getRow(table); - IColumn col = row.cf.getColumn(ByteBuffer.wrap("Column1".getBytes())); + IColumn col = row.cf.getColumn(ByteBufferUtil.bytes("Column1")); assert Arrays.equals(col.value().array(), "abcd".getBytes()); } } Modified: cassandra/trunk/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java?rev=1027353&r1=1027352&r2=1027353&view=diff ============================================================================== --- cassandra/trunk/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java (original) +++ cassandra/trunk/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java Tue Oct 26 03:35:58 2010 @@ -32,6 +32,7 @@ import org.apache.cassandra.db.commitlog import org.apache.cassandra.db.filter.QueryFilter; import org.apache.cassandra.db.filter.QueryPath; import org.junit.Test; +import org.apache.cassandra.utils.ByteBufferUtil; /** * Test for the truncate operation. @@ -51,7 +52,7 @@ public class RecoveryManagerTruncateTest ColumnFamily cf; // trucate clears memtable - rm = new RowMutation("Keyspace1", ByteBuffer.wrap("keymulti".getBytes())); + rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("keymulti")); cf = ColumnFamily.create("Keyspace1", "Standard1"); cf.addColumn(column("col1", "val1", 1L)); rm.add(cf); @@ -68,7 +69,7 @@ public class RecoveryManagerTruncateTest assertNull(getFromTable(table, "Standard1", "keymulti", "col1")); // truncate clears sstable - rm = new RowMutation("Keyspace1", ByteBuffer.wrap("keymulti".getBytes())); + rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("keymulti")); cf = ColumnFamily.create("Keyspace1", "Standard1"); cf.addColumn(column("col1", "val1", 1L)); rm.add(cf); Modified: cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java?rev=1027353&r1=1027352&r2=1027353&view=diff ============================================================================== --- cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java (original) +++ cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java Tue Oct 26 03:35:58 2010 @@ -30,6 +30,8 @@ import org.apache.cassandra.db.filter.Qu import org.apache.cassandra.CleanupHelper; import org.apache.cassandra.Util; +import org.apache.cassandra.utils.ByteBufferUtil; + public class RemoveColumnFamilyTest extends CleanupHelper { @@ -43,7 +45,7 @@ public class RemoveColumnFamilyTest exte // add data rm = new RowMutation("Keyspace1", dk.key); - rm.add(new QueryPath("Standard1", null, ByteBuffer.wrap("Column1".getBytes())), ByteBuffer.wrap("asdf".getBytes()), 0); + rm.add(new QueryPath("Standard1", null, ByteBufferUtil.bytes("Column1")), ByteBufferUtil.bytes("asdf"), 0); rm.apply(); // remove @@ -51,9 +53,9 @@ public class RemoveColumnFamilyTest exte rm.delete(new QueryPath("Standard1"), 1); rm.apply(); - ColumnFamily retrieved = store.getColumnFamily(QueryFilter.getIdentityFilter(dk, new QueryPath("Standard1", null, ByteBuffer.wrap("Column1".getBytes())))); + ColumnFamily retrieved = store.getColumnFamily(QueryFilter.getIdentityFilter(dk, new QueryPath("Standard1", null, ByteBufferUtil.bytes("Column1")))); assert retrieved.isMarkedForDelete(); - assertNull(retrieved.getColumn(ByteBuffer.wrap("Column1".getBytes()))); + assertNull(retrieved.getColumn(ByteBufferUtil.bytes("Column1"))); assertNull(Util.cloneAndRemoveDeleted(retrieved, Integer.MAX_VALUE)); } } Modified: cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java?rev=1027353&r1=1027352&r2=1027353&view=diff ============================================================================== --- cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java (original) +++ cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java Tue Oct 26 03:35:58 2010 @@ -30,6 +30,8 @@ import org.apache.cassandra.db.filter.Qu import org.apache.cassandra.CleanupHelper; import org.apache.cassandra.Util; +import org.apache.cassandra.utils.ByteBufferUtil; + public class RemoveColumnFamilyWithFlush1Test extends CleanupHelper { @@ -43,8 +45,8 @@ public class RemoveColumnFamilyWithFlush // add data rm = new RowMutation("Keyspace1", dk.key); - rm.add(new QueryPath("Standard1", null, ByteBuffer.wrap("Column1".getBytes())), ByteBuffer.wrap("asdf".getBytes()), 0); - rm.add(new QueryPath("Standard1", null, ByteBuffer.wrap("Column2".getBytes())), ByteBuffer.wrap("asdf".getBytes()), 0); + rm.add(new QueryPath("Standard1", null, ByteBufferUtil.bytes("Column1")), ByteBufferUtil.bytes("asdf"), 0); + rm.add(new QueryPath("Standard1", null, ByteBufferUtil.bytes("Column2")), ByteBufferUtil.bytes("asdf"), 0); rm.apply(); store.forceBlockingFlush(); @@ -55,7 +57,7 @@ public class RemoveColumnFamilyWithFlush ColumnFamily retrieved = store.getColumnFamily(QueryFilter.getIdentityFilter(dk, new QueryPath("Standard1"))); assert retrieved.isMarkedForDelete(); - assertNull(retrieved.getColumn(ByteBuffer.wrap("Column1".getBytes()))); + assertNull(retrieved.getColumn(ByteBufferUtil.bytes("Column1"))); assertNull(Util.cloneAndRemoveDeleted(retrieved, Integer.MAX_VALUE)); } } Modified: cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java?rev=1027353&r1=1027352&r2=1027353&view=diff ============================================================================== --- cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java (original) +++ cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java Tue Oct 26 03:35:58 2010 @@ -30,6 +30,8 @@ import org.apache.cassandra.db.filter.Qu import org.apache.cassandra.CleanupHelper; import org.apache.cassandra.Util; +import org.apache.cassandra.utils.ByteBufferUtil; + public class RemoveColumnFamilyWithFlush2Test extends CleanupHelper { @@ -43,7 +45,7 @@ public class RemoveColumnFamilyWithFlush // add data rm = new RowMutation("Keyspace1", dk.key); - rm.add(new QueryPath("Standard1", null, ByteBuffer.wrap("Column1".getBytes())), ByteBuffer.wrap("asdf".getBytes()), 0); + rm.add(new QueryPath("Standard1", null, ByteBufferUtil.bytes("Column1")), ByteBufferUtil.bytes("asdf"), 0); rm.apply(); // remove rm = new RowMutation("Keyspace1", dk.key); @@ -51,9 +53,9 @@ public class RemoveColumnFamilyWithFlush rm.apply(); store.forceBlockingFlush(); - ColumnFamily retrieved = store.getColumnFamily(QueryFilter.getIdentityFilter(dk, new QueryPath("Standard1", null, ByteBuffer.wrap("Column1".getBytes())))); + ColumnFamily retrieved = store.getColumnFamily(QueryFilter.getIdentityFilter(dk, new QueryPath("Standard1", null, ByteBufferUtil.bytes("Column1")))); assert retrieved.isMarkedForDelete(); - assertNull(retrieved.getColumn(ByteBuffer.wrap("Column1".getBytes()))); + assertNull(retrieved.getColumn(ByteBufferUtil.bytes("Column1"))); assertNull(Util.cloneAndRemoveDeleted(retrieved, Integer.MAX_VALUE)); } } Modified: cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveColumnTest.java URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveColumnTest.java?rev=1027353&r1=1027352&r2=1027353&view=diff ============================================================================== --- cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveColumnTest.java (original) +++ cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveColumnTest.java Tue Oct 26 03:35:58 2010 @@ -31,6 +31,8 @@ import org.apache.cassandra.db.filter.Qu import org.apache.cassandra.CleanupHelper; import org.apache.cassandra.Util; +import org.apache.cassandra.utils.ByteBufferUtil; + public class RemoveColumnTest extends CleanupHelper { @@ -44,17 +46,17 @@ public class RemoveColumnTest extends Cl // add data rm = new RowMutation("Keyspace1", dk.key); - rm.add(new QueryPath("Standard1", null, ByteBuffer.wrap("Column1".getBytes())), ByteBuffer.wrap("asdf".getBytes()), 0); + rm.add(new QueryPath("Standard1", null, ByteBufferUtil.bytes("Column1")), ByteBufferUtil.bytes("asdf"), 0); rm.apply(); store.forceBlockingFlush(); // remove rm = new RowMutation("Keyspace1", dk.key); - rm.delete(new QueryPath("Standard1", null, ByteBuffer.wrap("Column1".getBytes())), 1); + rm.delete(new QueryPath("Standard1", null, ByteBufferUtil.bytes("Column1")), 1); rm.apply(); - ColumnFamily retrieved = store.getColumnFamily(QueryFilter.getNamesFilter(dk, new QueryPath("Standard1"), ByteBuffer.wrap("Column1".getBytes()))); - assert retrieved.getColumn(ByteBuffer.wrap("Column1".getBytes())).isMarkedForDelete(); + ColumnFamily retrieved = store.getColumnFamily(QueryFilter.getNamesFilter(dk, new QueryPath("Standard1"), ByteBufferUtil.bytes("Column1"))); + assert retrieved.getColumn(ByteBufferUtil.bytes("Column1")).isMarkedForDelete(); assertNull(Util.cloneAndRemoveDeleted(retrieved, Integer.MAX_VALUE)); assertNull(Util.cloneAndRemoveDeleted(store.getColumnFamily(QueryFilter.getIdentityFilter(dk, new QueryPath("Standard1"))), Integer.MAX_VALUE)); } Modified: cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveSubColumnTest.java URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveSubColumnTest.java?rev=1027353&r1=1027352&r2=1027353&view=diff ============================================================================== --- cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveSubColumnTest.java (original) +++ cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveSubColumnTest.java Tue Oct 26 03:35:58 2010 @@ -30,6 +30,8 @@ import org.apache.cassandra.db.filter.Qu import static org.apache.cassandra.Util.getBytes; import org.apache.cassandra.Util; import org.apache.cassandra.CleanupHelper; +import org.apache.cassandra.utils.ByteBufferUtil; + public class RemoveSubColumnTest extends CleanupHelper { @@ -49,11 +51,11 @@ public class RemoveSubColumnTest extends // remove rm = new RowMutation("Keyspace1", dk.key); - rm.delete(new QueryPath("Super1", ByteBuffer.wrap("SC1".getBytes()), getBytes(1)), 1); + rm.delete(new QueryPath("Super1", ByteBufferUtil.bytes("SC1"), getBytes(1)), 1); rm.apply(); - ColumnFamily retrieved = store.getColumnFamily(QueryFilter.getIdentityFilter(dk, new QueryPath("Super1", ByteBuffer.wrap("SC1".getBytes())))); - assert retrieved.getColumn(ByteBuffer.wrap("SC1".getBytes())).getSubColumn(getBytes(1)).isMarkedForDelete(); + ColumnFamily retrieved = store.getColumnFamily(QueryFilter.getIdentityFilter(dk, new QueryPath("Super1", ByteBufferUtil.bytes("SC1")))); + assert retrieved.getColumn(ByteBufferUtil.bytes("SC1")).getSubColumn(getBytes(1)).isMarkedForDelete(); assertNull(Util.cloneAndRemoveDeleted(retrieved, Integer.MAX_VALUE)); } } Modified: cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveSuperColumnTest.java URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveSuperColumnTest.java?rev=1027353&r1=1027352&r2=1027353&view=diff ============================================================================== --- cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveSuperColumnTest.java (original) +++ cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveSuperColumnTest.java Tue Oct 26 03:35:58 2010 @@ -35,6 +35,8 @@ import static org.apache.cassandra.Util. import org.apache.cassandra.CleanupHelper; import static junit.framework.Assert.assertNotNull; +import org.apache.cassandra.utils.ByteBufferUtil; + public class RemoveSuperColumnTest extends CleanupHelper { @@ -53,7 +55,7 @@ public class RemoveSuperColumnTest exten // remove rm = new RowMutation("Keyspace1", dk.key); - rm.delete(new QueryPath("Super1", ByteBuffer.wrap("SC1".getBytes())), 1); + rm.delete(new QueryPath("Super1", ByteBufferUtil.bytes("SC1")), 1); rm.apply(); validateRemoveTwoSources(dk); @@ -82,7 +84,7 @@ public class RemoveSuperColumnTest exten // remove rm = new RowMutation("Keyspace1", dk.key); - rm.delete(new QueryPath("Super3", ByteBuffer.wrap("SC1".getBytes()), Util.getBytes(1)), 1); + rm.delete(new QueryPath("Super3", ByteBufferUtil.bytes("SC1"), Util.getBytes(1)), 1); rm.apply(); validateRemoveSubColumn(dk); @@ -94,20 +96,20 @@ public class RemoveSuperColumnTest exten private void validateRemoveSubColumn(DecoratedKey dk) throws IOException { ColumnFamilyStore store = Table.open("Keyspace1").getColumnFamilyStore("Super3"); - ColumnFamily cf = store.getColumnFamily(QueryFilter.getNamesFilter(dk, new QueryPath("Super3", ByteBuffer.wrap("SC1".getBytes())), Util.getBytes(1))); + ColumnFamily cf = store.getColumnFamily(QueryFilter.getNamesFilter(dk, new QueryPath("Super3", ByteBufferUtil.bytes("SC1")), Util.getBytes(1))); assertNull(Util.cloneAndRemoveDeleted(cf, Integer.MAX_VALUE)); - cf = store.getColumnFamily(QueryFilter.getNamesFilter(dk, new QueryPath("Super3", ByteBuffer.wrap("SC1".getBytes())), Util.getBytes(2))); + cf = store.getColumnFamily(QueryFilter.getNamesFilter(dk, new QueryPath("Super3", ByteBufferUtil.bytes("SC1")), Util.getBytes(2))); assertNotNull(Util.cloneAndRemoveDeleted(cf, Integer.MAX_VALUE)); } private void validateRemoveTwoSources(DecoratedKey dk) throws IOException { ColumnFamilyStore store = Table.open("Keyspace1").getColumnFamilyStore("Super1"); - ColumnFamily cf = store.getColumnFamily(QueryFilter.getNamesFilter(dk, new QueryPath("Super1"), ByteBuffer.wrap("SC1".getBytes()))); + ColumnFamily cf = store.getColumnFamily(QueryFilter.getNamesFilter(dk, new QueryPath("Super1"), ByteBufferUtil.bytes("SC1"))); assert cf.getSortedColumns().iterator().next().getMarkedForDeleteAt() == 1 : cf; assert cf.getSortedColumns().iterator().next().getSubColumns().size() == 0 : cf; assertNull(Util.cloneAndRemoveDeleted(cf, Integer.MAX_VALUE)); - cf = store.getColumnFamily(QueryFilter.getNamesFilter(dk, new QueryPath("Super1"), ByteBuffer.wrap("SC1".getBytes()))); + cf = store.getColumnFamily(QueryFilter.getNamesFilter(dk, new QueryPath("Super1"), ByteBufferUtil.bytes("SC1"))); assertNull(Util.cloneAndRemoveDeleted(cf, Integer.MAX_VALUE)); cf = store.getColumnFamily(QueryFilter.getIdentityFilter(dk, new QueryPath("Super1"))); assertNull(Util.cloneAndRemoveDeleted(cf, Integer.MAX_VALUE)); @@ -117,7 +119,7 @@ public class RemoveSuperColumnTest exten private void validateRemoveCompacted(DecoratedKey dk) throws IOException { ColumnFamilyStore store = Table.open("Keyspace1").getColumnFamilyStore("Super1"); - ColumnFamily resolved = store.getColumnFamily(QueryFilter.getNamesFilter(dk, new QueryPath("Super1"), ByteBuffer.wrap("SC1".getBytes()))); + ColumnFamily resolved = store.getColumnFamily(QueryFilter.getNamesFilter(dk, new QueryPath("Super1"), ByteBufferUtil.bytes("SC1"))); assert resolved.getSortedColumns().iterator().next().getMarkedForDeleteAt() == 1; Collection subColumns = resolved.getSortedColumns().iterator().next().getSubColumns(); assert subColumns.size() == 0; @@ -138,7 +140,7 @@ public class RemoveSuperColumnTest exten // remove rm = new RowMutation("Keyspace1", dk.key); - rm.delete(new QueryPath("Super2", ByteBuffer.wrap("SC1".getBytes())), 1); + rm.delete(new QueryPath("Super2", ByteBufferUtil.bytes("SC1")), 1); rm.apply(); // new data @@ -159,7 +161,7 @@ public class RemoveSuperColumnTest exten private void validateRemoveWithNewData(DecoratedKey dk) throws IOException { ColumnFamilyStore store = Table.open("Keyspace1").getColumnFamilyStore("Super2"); - ColumnFamily cf = store.getColumnFamily(QueryFilter.getNamesFilter(dk, new QueryPath("Super2", ByteBuffer.wrap("SC1".getBytes())), getBytes(2))); + ColumnFamily cf = store.getColumnFamily(QueryFilter.getNamesFilter(dk, new QueryPath("Super2", ByteBufferUtil.bytes("SC1")), getBytes(2))); Collection subColumns = cf.getSortedColumns().iterator().next().getSubColumns(); assert subColumns.size() == 1; assert subColumns.iterator().next().timestamp() == 2; @@ -179,9 +181,9 @@ public class RemoveSuperColumnTest exten // remove rm = new RowMutation("Keyspace1", key.key); - rm.delete(new QueryPath("Super2", ByteBuffer.wrap("SC1".getBytes())), 1); + rm.delete(new QueryPath("Super2", ByteBufferUtil.bytes("SC1")), 1); rm.apply(); - assertNull(Util.cloneAndRemoveDeleted(store.getColumnFamily(QueryFilter.getNamesFilter(key, new QueryPath("Super2"), ByteBuffer.wrap("SC1".getBytes()))), Integer.MAX_VALUE)); + assertNull(Util.cloneAndRemoveDeleted(store.getColumnFamily(QueryFilter.getNamesFilter(key, new QueryPath("Super2"), ByteBufferUtil.bytes("SC1"))), Integer.MAX_VALUE)); // resurrect rm = new RowMutation("Keyspace1", key.key); @@ -189,7 +191,7 @@ public class RemoveSuperColumnTest exten rm.apply(); // validate - ColumnFamily cf = store.getColumnFamily(QueryFilter.getNamesFilter(key, new QueryPath("Super2"), ByteBuffer.wrap("SC1".getBytes()))); + ColumnFamily cf = store.getColumnFamily(QueryFilter.getNamesFilter(key, new QueryPath("Super2"), ByteBufferUtil.bytes("SC1"))); cf = Util.cloneAndRemoveDeleted(cf, Integer.MAX_VALUE); Collection subColumns = cf.getSortedColumns().iterator().next().getSubColumns(); assert subColumns.size() == 1;