You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jb...@apache.org on 2010/10/13 17:43:25 UTC

svn commit: r1022147 [3/3] - in /cassandra/trunk: ./ src/java/org/apache/cassandra/avro/ src/java/org/apache/cassandra/config/ src/java/org/apache/cassandra/db/ src/java/org/apache/cassandra/db/clock/ src/java/org/apache/cassandra/db/filter/ src/java/o...

Modified: cassandra/trunk/test/unit/org/apache/cassandra/db/TimeSortTest.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/db/TimeSortTest.java?rev=1022147&r1=1022146&r2=1022147&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/db/TimeSortTest.java (original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/db/TimeSortTest.java Wed Oct 13 15:43:18 2010
@@ -46,12 +46,12 @@ public class TimeSortTest extends Cleanu
         DecoratedKey key = Util.dk("key0");
 
         rm = new RowMutation("Keyspace1", key.key);
-        rm.add(new QueryPath("StandardLong1", null, getBytes(100)), "a".getBytes(), new TimestampClock(100));
+        rm.add(new QueryPath("StandardLong1", null, getBytes(100)), "a".getBytes(), 100);
         rm.apply();
         cfStore.forceBlockingFlush();
 
         rm = new RowMutation("Keyspace1", key.key);
-        rm.add(new QueryPath("StandardLong1", null, getBytes(0)), "b".getBytes(), new TimestampClock(0));
+        rm.add(new QueryPath("StandardLong1", null, getBytes(0)), "b".getBytes(), 0);
         rm.apply();
 
         ColumnFamily cf = cfStore.getColumnFamily(key, new QueryPath("StandardLong1"), getBytes(10), ArrayUtils.EMPTY_BYTE_ARRAY, false, 1000);
@@ -70,7 +70,7 @@ public class TimeSortTest extends Cleanu
             RowMutation rm = new RowMutation("Keyspace1", Integer.toString(i).getBytes());
             for (int j = 0; j < 8; ++j)
             {
-                rm.add(new QueryPath("StandardLong1", null, getBytes(j * 2)), "a".getBytes(), new TimestampClock(j * 2));
+                rm.add(new QueryPath("StandardLong1", null, getBytes(j * 2)), "a".getBytes(), j * 2);
             }
             rm.apply();
         }
@@ -85,13 +85,13 @@ public class TimeSortTest extends Cleanu
         RowMutation rm = new RowMutation("Keyspace1", key.key);
         for (int j = 0; j < 4; ++j)
         {
-            rm.add(new QueryPath("StandardLong1", null, getBytes(j * 2 + 1)), "b".getBytes(), new TimestampClock(j * 2 + 1));
+            rm.add(new QueryPath("StandardLong1", null, getBytes(j * 2 + 1)), "b".getBytes(), j * 2 + 1);
         }
         rm.apply();
         // and some overwrites
         rm = new RowMutation("Keyspace1", key.key);
-        rm.add(new QueryPath("StandardLong1", null, getBytes(0)), "c".getBytes(), new TimestampClock(100));
-        rm.add(new QueryPath("StandardLong1", null, getBytes(10)), "c".getBytes(), new TimestampClock(100));
+        rm.add(new QueryPath("StandardLong1", null, getBytes(0)), "c".getBytes(), 100);
+        rm.add(new QueryPath("StandardLong1", null, getBytes(10)), "c".getBytes(), 100);
         rm.apply();
 
         // verify
@@ -126,7 +126,7 @@ public class TimeSortTest extends Cleanu
                 int k = j;
                 for (IColumn c : columns)
                 {
-                    assertEquals((k++) * 2, ((TimestampClock)c.clock()).timestamp());
+                    assertEquals((k++) * 2, c.timestamp());
 
                 }
             }

Modified: cassandra/trunk/test/unit/org/apache/cassandra/io/LazilyCompactedRowTest.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/io/LazilyCompactedRowTest.java?rev=1022147&r1=1022146&r2=1022147&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/io/LazilyCompactedRowTest.java (original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/io/LazilyCompactedRowTest.java Wed Oct 13 15:43:18 2010
@@ -91,7 +91,7 @@ public class LazilyCompactedRowTest exte
             ColumnFamily.serializer().deserializeFromSSTableNoColumns(cf1, in1);
             ColumnFamily.serializer().deserializeFromSSTableNoColumns(cf2, in2);
             assert cf1.getLocalDeletionTime() == cf2.getLocalDeletionTime();
-            assert cf1.getMarkedForDeleteAt().equals(cf2.getMarkedForDeleteAt());   
+            assert cf1.getMarkedForDeleteAt() == cf2.getMarkedForDeleteAt();   
             // columns
             int columns = in1.readInt();
             assert columns == in2.readInt();
@@ -117,7 +117,7 @@ public class LazilyCompactedRowTest exte
 
         byte[] key = "k".getBytes();
         RowMutation rm = new RowMutation("Keyspace1", key);
-        rm.add(new QueryPath("Standard1", null, "c".getBytes()), new byte[0], new TimestampClock(0));
+        rm.add(new QueryPath("Standard1", null, "c".getBytes()), new byte[0], 0);
         rm.apply();
         cfs.forceBlockingFlush();
 
@@ -134,8 +134,8 @@ public class LazilyCompactedRowTest exte
 
         byte[] key = "k".getBytes();
         RowMutation rm = new RowMutation("Keyspace1", key);
-        rm.add(new QueryPath("Standard1", null, "c".getBytes()), new byte[0], new TimestampClock(0));
-        rm.add(new QueryPath("Standard1", null, "d".getBytes()), new byte[0], new TimestampClock(0));
+        rm.add(new QueryPath("Standard1", null, "c".getBytes()), new byte[0], 0);
+        rm.add(new QueryPath("Standard1", null, "d".getBytes()), new byte[0], 0);
         rm.apply();
         cfs.forceBlockingFlush();
 
@@ -152,7 +152,7 @@ public class LazilyCompactedRowTest exte
 
         byte[] key = "k".getBytes();
         RowMutation rm = new RowMutation("Keyspace1", key);
-        rm.add(new QueryPath("Standard1", null, "c".getBytes()), new byte[0], new TimestampClock(0));
+        rm.add(new QueryPath("Standard1", null, "c".getBytes()), new byte[0], 0);
         rm.apply();
         cfs.forceBlockingFlush();
 
@@ -172,8 +172,8 @@ public class LazilyCompactedRowTest exte
 
         byte[] key = "k".getBytes();
         RowMutation rm = new RowMutation("Keyspace1", key);
-        rm.add(new QueryPath("Standard1", null, "c".getBytes()), new byte[0], new TimestampClock(0));
-        rm.add(new QueryPath("Standard1", null, "d".getBytes()), new byte[0], new TimestampClock(0));
+        rm.add(new QueryPath("Standard1", null, "c".getBytes()), new byte[0], 0);
+        rm.add(new QueryPath("Standard1", null, "d".getBytes()), new byte[0], 0);
         rm.apply();
         cfs.forceBlockingFlush();
 
@@ -196,7 +196,7 @@ public class LazilyCompactedRowTest exte
             for (int i = 0; i < ROWS_PER_SSTABLE; i++) {
                 byte[] key = String.valueOf(i % 2).getBytes();
                 RowMutation rm = new RowMutation("Keyspace1", key);
-                rm.add(new QueryPath("Standard1", null, String.valueOf(i / 2).getBytes()), new byte[0], new TimestampClock(j * ROWS_PER_SSTABLE + i));
+                rm.add(new QueryPath("Standard1", null, String.valueOf(i / 2).getBytes()), new byte[0], j * ROWS_PER_SSTABLE + i);
                 rm.apply();
             }
             cfs.forceBlockingFlush();

Modified: cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java?rev=1022147&r1=1022146&r2=1022147&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java (original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java Wed Oct 13 15:43:18 2010
@@ -63,7 +63,7 @@ public class SSTableReaderTest extends C
         {
             byte[] key = String.valueOf(j).getBytes();
             RowMutation rm = new RowMutation("Keyspace1", key);
-            rm.add(new QueryPath("Standard2", null, "0".getBytes()), new byte[0], new TimestampClock(j));
+            rm.add(new QueryPath("Standard2", null, "0".getBytes()), new byte[0], j);
             rm.apply();
         }
         store.forceBlockingFlush();
@@ -104,7 +104,7 @@ public class SSTableReaderTest extends C
         {
             byte[] key = String.valueOf(j).getBytes();
             RowMutation rm = new RowMutation("Keyspace1", key);
-            rm.add(new QueryPath("Standard1", null, "0".getBytes()), new byte[0], new TimestampClock(j));
+            rm.add(new QueryPath("Standard1", null, "0".getBytes()), new byte[0], j);
             rm.apply();
         }
         store.forceBlockingFlush();
@@ -141,7 +141,7 @@ public class SSTableReaderTest extends C
         {
             byte[] key = String.valueOf(j).getBytes();
             RowMutation rm = new RowMutation("Keyspace1", key);
-            rm.add(new QueryPath("Standard1", null, "0".getBytes()), new byte[0], new TimestampClock(j));
+            rm.add(new QueryPath("Standard1", null, "0".getBytes()), new byte[0], j);
             rm.apply();
         }
         store.forceBlockingFlush();

Modified: cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java?rev=1022147&r1=1022146&r2=1022147&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java (original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java Wed Oct 13 15:43:18 2010
@@ -32,7 +32,7 @@ public class SSTableUtils
     public static String TABLENAME = "Keyspace1";
     public static String CFNAME = "Standard1";
 
-    public static ColumnFamily createCF(IClock mfda, int ldt, IColumn... cols)
+    public static ColumnFamily createCF(long mfda, int ldt, IColumn... cols)
     {
         ColumnFamily cf = ColumnFamily.create(TABLENAME, CFNAME);
         cf.delete(ldt, mfda);
@@ -63,7 +63,7 @@ public class SSTableUtils
         for (String key : keys)
         {
             ColumnFamily cf = ColumnFamily.create(TABLENAME, CFNAME);
-            cf.addColumn(new Column(key.getBytes(), key.getBytes(), new TimestampClock(0)));
+            cf.addColumn(new Column(key.getBytes(), key.getBytes(), 0));
             map.put(key, cf);
         }
         return writeSSTable(map);

Modified: cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTest.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTest.java?rev=1022147&r1=1022146&r2=1022147&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTest.java (original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTest.java Wed Oct 13 15:43:18 2010
@@ -54,12 +54,12 @@ public class SSTableWriterTest extends C
         RowMutation rm;
 
         rm = new RowMutation("Keyspace1", "k1".getBytes());
-        rm.add(new QueryPath("Indexed1", null, "birthdate".getBytes("UTF8")), FBUtilities.toByteArray(1L), new TimestampClock(0));
+        rm.add(new QueryPath("Indexed1", null, "birthdate".getBytes("UTF8")), FBUtilities.toByteArray(1L), 0);
         rm.apply();
         
         ColumnFamily cf = ColumnFamily.create("Keyspace1", "Indexed1");        
-        cf.addColumn(new Column("birthdate".getBytes(), FBUtilities.toByteArray(1L), new TimestampClock(0)));
-        cf.addColumn(new Column("anydate".getBytes(), FBUtilities.toByteArray(1L), new TimestampClock(0)));
+        cf.addColumn(new Column("birthdate".getBytes(), FBUtilities.toByteArray(1L), 0));
+        cf.addColumn(new Column("anydate".getBytes(), FBUtilities.toByteArray(1L), 0));
         
         Map<byte[], byte[]> entries = new HashMap<byte[], byte[]>();
         
@@ -68,7 +68,7 @@ public class SSTableWriterTest extends C
         entries.put("k2".getBytes(), Arrays.copyOf(buffer.getData(), buffer.getLength()));        
         cf.clear();
         
-        cf.addColumn(new Column("anydate".getBytes(), FBUtilities.toByteArray(1L), new TimestampClock(0)));
+        cf.addColumn(new Column("anydate".getBytes(), FBUtilities.toByteArray(1L), 0));
         buffer = new DataOutputBuffer();
         ColumnFamily.serializer().serializeWithIndexes(cf, buffer);               
         entries.put("k3".getBytes(), Arrays.copyOf(buffer.getData(), buffer.getLength()));

Modified: cassandra/trunk/test/unit/org/apache/cassandra/service/AntiEntropyServiceTest.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/service/AntiEntropyServiceTest.java?rev=1022147&r1=1022146&r2=1022147&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/service/AntiEntropyServiceTest.java (original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/service/AntiEntropyServiceTest.java Wed Oct 13 15:43:18 2010
@@ -103,7 +103,7 @@ public class AntiEntropyServiceTest exte
         List<RowMutation> rms = new LinkedList<RowMutation>();
         RowMutation rm;
         rm = new RowMutation(tablename, "key1".getBytes());
-        rm.add(new QueryPath(cfname, null, "Column1".getBytes()), "asdf".getBytes(), new TimestampClock(0));
+        rm.add(new QueryPath(cfname, null, "Column1".getBytes()), "asdf".getBytes(), 0);
         rms.add(rm);
         Util.writeColumnFamily(rms);
 

Modified: cassandra/trunk/test/unit/org/apache/cassandra/service/ReadResponseResolverTest.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/service/ReadResponseResolverTest.java?rev=1022147&r1=1022146&r2=1022147&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/service/ReadResponseResolverTest.java (original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/service/ReadResponseResolverTest.java Wed Oct 13 15:43:18 2010
@@ -27,7 +27,6 @@ import org.apache.cassandra.SchemaLoader
 import org.junit.Test;
 
 import org.apache.cassandra.db.ColumnFamily;
-import org.apache.cassandra.db.TimestampClock;
 
 import static org.apache.cassandra.db.TableTest.assertColumns;
 import static org.apache.cassandra.Util.column;
@@ -39,10 +38,10 @@ public class ReadResponseResolverTest ex
     public void testResolveSupersetNewer()
     {
         ColumnFamily cf1 = ColumnFamily.create("Keyspace1", "Standard1");
-        cf1.addColumn(column("c1", "v1", new TimestampClock(0)));
+        cf1.addColumn(column("c1", "v1", 0));
 
         ColumnFamily cf2 = ColumnFamily.create("Keyspace1", "Standard1");
-        cf2.addColumn(column("c1", "v2", new TimestampClock(1)));
+        cf2.addColumn(column("c1", "v2", 1));
 
         ColumnFamily resolved = ReadResponseResolver.resolveSuperset(Arrays.asList(cf1, cf2));
         assertColumns(resolved, "c1");
@@ -54,10 +53,10 @@ public class ReadResponseResolverTest ex
     public void testResolveSupersetDisjoint()
     {
         ColumnFamily cf1 = ColumnFamily.create("Keyspace1", "Standard1");
-        cf1.addColumn(column("c1", "v1", new TimestampClock(0)));
+        cf1.addColumn(column("c1", "v1", 0));
 
         ColumnFamily cf2 = ColumnFamily.create("Keyspace1", "Standard1");
-        cf2.addColumn(column("c2", "v2", new TimestampClock(1)));
+        cf2.addColumn(column("c2", "v2", 1));
 
         ColumnFamily resolved = ReadResponseResolver.resolveSuperset(Arrays.asList(cf1, cf2));
         assertColumns(resolved, "c1", "c2");
@@ -69,7 +68,7 @@ public class ReadResponseResolverTest ex
     public void testResolveSupersetNullOne()
     {
         ColumnFamily cf2 = ColumnFamily.create("Keyspace1", "Standard1");
-        cf2.addColumn(column("c2", "v2", new TimestampClock(1)));
+        cf2.addColumn(column("c2", "v2", 1));
 
         ColumnFamily resolved = ReadResponseResolver.resolveSuperset(Arrays.asList(null, cf2));
         assertColumns(resolved, "c2");
@@ -81,7 +80,7 @@ public class ReadResponseResolverTest ex
     public void testResolveSupersetNullTwo()
     {
         ColumnFamily cf1 = ColumnFamily.create("Keyspace1", "Standard1");
-        cf1.addColumn(column("c1", "v1", new TimestampClock(0)));
+        cf1.addColumn(column("c1", "v1", 0));
 
         ColumnFamily resolved = ReadResponseResolver.resolveSuperset(Arrays.asList(cf1, null));
         assertColumns(resolved, "c1");

Modified: cassandra/trunk/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java?rev=1022147&r1=1022146&r2=1022147&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java (original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java Wed Oct 13 15:43:18 2010
@@ -69,8 +69,8 @@ public class StreamingTransferTest exten
             String key = "key" + i;
             RowMutation rm = new RowMutation("Keyspace1", key.getBytes());
             ColumnFamily cf = ColumnFamily.create(table.name, cfs.columnFamily);
-            cf.addColumn(column(key, "v", new TimestampClock(0)));
-            cf.addColumn(new Column("birthdate".getBytes("UTF8"), FBUtilities.toByteArray((long) i), new TimestampClock(0)));
+            cf.addColumn(column(key, "v", 0));
+            cf.addColumn(new Column("birthdate".getBytes("UTF8"), FBUtilities.toByteArray((long) i), 0));
             rm.add(cf);
             rm.apply();
         }

Modified: cassandra/trunk/test/unit/org/apache/cassandra/tools/SSTableExportTest.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/tools/SSTableExportTest.java?rev=1022147&r1=1022146&r2=1022147&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/tools/SSTableExportTest.java (original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/tools/SSTableExportTest.java Wed Oct 13 15:43:18 2010
@@ -27,7 +27,6 @@ import java.util.Arrays;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamily;
-import org.apache.cassandra.db.TimestampClock;
 import org.apache.cassandra.db.filter.QueryFilter;
 import org.apache.cassandra.db.filter.QueryPath;
 import org.apache.cassandra.dht.IPartitioner;
@@ -63,12 +62,12 @@ public class SSTableExportTest extends S
         SSTableWriter writer = new SSTableWriter(tempSS.getPath(), 2);
         
         // Add rowA
-        cfamily.addColumn(new QueryPath("Standard1", null, "colA".getBytes()), "valA".getBytes(), new TimestampClock(1));
+        cfamily.addColumn(new QueryPath("Standard1", null, "colA".getBytes()), "valA".getBytes(), 1);
         writer.append(Util.dk("rowA"), cfamily);
         cfamily.clear();
         
         // Add rowB
-        cfamily.addColumn(new QueryPath("Standard1", null, "colB".getBytes()), "valB".getBytes(), new TimestampClock(1));
+        cfamily.addColumn(new QueryPath("Standard1", null, "colB".getBytes()), "valB".getBytes(), 1);
         writer.append(Util.dk("rowB"), cfamily);
         cfamily.clear();
      
@@ -95,17 +94,17 @@ public class SSTableExportTest extends S
         SSTableWriter writer = new SSTableWriter(tempSS.getPath(), 2);
         
         // Add rowA
-        cfamily.addColumn(new QueryPath("Standard1", null, "colA".getBytes()), "valA".getBytes(), new TimestampClock(1));
+        cfamily.addColumn(new QueryPath("Standard1", null, "colA".getBytes()), "valA".getBytes(), 1);
         writer.append(Util.dk("rowA"), cfamily);
         cfamily.clear();
         
         // Add rowB
-        cfamily.addColumn(new QueryPath("Standard1", null, "colB".getBytes()), "valB".getBytes(), new TimestampClock(1));
+        cfamily.addColumn(new QueryPath("Standard1", null, "colB".getBytes()), "valB".getBytes(), 1);
         writer.append(Util.dk("rowB"), cfamily);
         cfamily.clear();
 
         // Add rowExclude
-        cfamily.addColumn(new QueryPath("Standard1", null, "colX".getBytes()), "valX".getBytes(), new TimestampClock(1));
+        cfamily.addColumn(new QueryPath("Standard1", null, "colX".getBytes()), "valX".getBytes(), 1);
         writer.append(Util.dk("rowExclude"), cfamily);
         cfamily.clear();
 
@@ -137,17 +136,17 @@ public class SSTableExportTest extends S
         SSTableWriter writer = new SSTableWriter(tempSS.getPath(), 2);
         
         // Add rowA
-        cfamily.addColumn(new QueryPath("Super4", "superA".getBytes(), "colA".getBytes()), "valA".getBytes(), new TimestampClock(1));
+        cfamily.addColumn(new QueryPath("Super4", "superA".getBytes(), "colA".getBytes()), "valA".getBytes(), 1);
         writer.append(Util.dk("rowA"), cfamily);
         cfamily.clear();
         
         // Add rowB
-        cfamily.addColumn(new QueryPath("Super4", "superB".getBytes(), "colB".getBytes()), "valB".getBytes(), new TimestampClock(1));
+        cfamily.addColumn(new QueryPath("Super4", "superB".getBytes(), "colB".getBytes()), "valB".getBytes(), 1);
         writer.append(Util.dk("rowB"), cfamily);
         cfamily.clear();
 
         // Add rowExclude
-        cfamily.addColumn(new QueryPath("Super4", "superX".getBytes(), "colX".getBytes()), "valX".getBytes(), new TimestampClock(1));
+        cfamily.addColumn(new QueryPath("Super4", "superX".getBytes(), "colX".getBytes()), "valX".getBytes(), 1);
         writer.append(Util.dk("rowExclude"), cfamily);
         cfamily.clear();
 
@@ -177,12 +176,12 @@ public class SSTableExportTest extends S
         SSTableWriter writer = new SSTableWriter(tempSS.getPath(), 2);
         
         // Add rowA
-        cfamily.addColumn(new QueryPath("Standard1", null, "name".getBytes()), "val".getBytes(), new TimestampClock(1));
+        cfamily.addColumn(new QueryPath("Standard1", null, "name".getBytes()), "val".getBytes(), 1);
         writer.append(Util.dk("rowA"), cfamily);
         cfamily.clear();
 
         // Add rowExclude
-        cfamily.addColumn(new QueryPath("Standard1", null, "name".getBytes()), "val".getBytes(), new TimestampClock(1));
+        cfamily.addColumn(new QueryPath("Standard1", null, "name".getBytes()), "val".getBytes(), 1);
         writer.append(Util.dk("rowExclude"), cfamily);
         cfamily.clear();
 

Modified: cassandra/trunk/test/unit/org/apache/cassandra/utils/FBUtilitiesTest.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/utils/FBUtilitiesTest.java?rev=1022147&r1=1022146&r2=1022147&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/utils/FBUtilitiesTest.java (original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/utils/FBUtilitiesTest.java Wed Oct 13 15:43:18 2010
@@ -24,11 +24,6 @@ import static org.junit.Assert.assertEqu
 import java.io.IOException;
 import java.nio.charset.CharacterCodingException;
 import java.util.Arrays;
-import java.util.concurrent.atomic.AtomicReference;
-
-import org.apache.cassandra.db.IClock;
-import org.apache.cassandra.db.TimestampClock;
-
 
 import org.junit.Test;
 
@@ -88,27 +83,6 @@ public class FBUtilitiesTest 
             assertEquals(i, actual);
         }
     }
-    
-    @Test
-    public void testAtomicSetMaxIClock()
-    {
-        AtomicReference<IClock> atomicClock = new AtomicReference<IClock>(null);
-        
-        // atomic < new
-        atomicClock.set(TimestampClock.MIN_VALUE);
-        FBUtilities.atomicSetMax(atomicClock, new TimestampClock(1L));
-        assert ((TimestampClock)atomicClock.get()).timestamp() == 1L;
-        
-        // atomic == new
-        atomicClock.set(new TimestampClock(3L));
-        FBUtilities.atomicSetMax(atomicClock, new TimestampClock(3L));
-        assert ((TimestampClock)atomicClock.get()).timestamp() == 3L;
-
-        // atomic > new
-        atomicClock.set(new TimestampClock(9L));
-        FBUtilities.atomicSetMax(atomicClock, new TimestampClock(3L));
-        assert ((TimestampClock)atomicClock.get()).timestamp() == 9L;
-    } 
 
     @Test(expected=CharacterCodingException.class)
     public void testDecode() throws IOException