You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sl...@apache.org on 2011/07/25 18:11:59 UTC

svn commit: r1150765 - in /cassandra/trunk: src/java/org/apache/cassandra/utils/NodeId.java test/unit/org/apache/cassandra/SchemaLoader.java test/unit/org/apache/cassandra/db/CounterMutationTest.java

Author: slebresne
Date: Mon Jul 25 16:11:58 2011
New Revision: 1150765

URL: http://svn.apache.org/viewvc?rev=1150765&view=rev
Log:
Add unit test for CounterMutationTest
patch by slebresne; reviewed by jbellis for CASSANDRA-2945

Added:
    cassandra/trunk/test/unit/org/apache/cassandra/db/CounterMutationTest.java
Modified:
    cassandra/trunk/src/java/org/apache/cassandra/utils/NodeId.java
    cassandra/trunk/test/unit/org/apache/cassandra/SchemaLoader.java

Modified: cassandra/trunk/src/java/org/apache/cassandra/utils/NodeId.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/utils/NodeId.java?rev=1150765&r1=1150764&r2=1150765&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/utils/NodeId.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/utils/NodeId.java Mon Jul 25 16:11:58 2011
@@ -49,9 +49,14 @@ public class NodeId implements Comparabl
      * To use only when this strictly necessary, as using this will make all
      * counter context grow with time.
      */
-    public static synchronized void renewLocalId()
+    public static void renewLocalId()
     {
-        localIds.renewCurrent();
+        renewLocalId(System.currentTimeMillis());
+    }
+
+    public static synchronized void renewLocalId(long now)
+    {
+        localIds.renewCurrent(now);
     }
 
     /**
@@ -194,13 +199,13 @@ public class NodeId implements Comparabl
             }
         }
 
-        synchronized void renewCurrent()
+        synchronized void renewCurrent(long now)
         {
             NodeId newNodeId = generate();
             NodeId old = current.get();
             SystemTable.writeCurrentLocalNodeId(old, newNodeId);
             current.set(newNodeId);
-            olds.add(new NodeIdRecord(old));
+            olds.add(new NodeIdRecord(old, now));
         }
     }
 
@@ -209,15 +214,15 @@ public class NodeId implements Comparabl
         public final NodeId id;
         public final long timestamp;
 
-        public NodeIdRecord(NodeId id)
-        {
-            this(id, System.currentTimeMillis());
-        }
-
         public NodeIdRecord(NodeId id, long timestamp)
         {
             this.id = id;
             this.timestamp = timestamp;
         }
+
+        public String toString()
+        {
+            return String.format("(%s, %d)", id.toString(), timestamp);
+        }
     }
 }

Modified: cassandra/trunk/test/unit/org/apache/cassandra/SchemaLoader.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/SchemaLoader.java?rev=1150765&r1=1150764&r2=1150765&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/SchemaLoader.java (original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/SchemaLoader.java Mon Jul 25 16:11:58 2011
@@ -143,7 +143,8 @@ public class SchemaLoader
                                                  su,
                                                  bytes,
                                                  bytes)
-                                                 .defaultValidator(CounterColumnType.instance),
+                                                 .defaultValidator(CounterColumnType.instance)
+                                                 .mergeShardsChance(1.0),
                                   jdbcCFMD(ks1, "JdbcInteger", IntegerType.instance).columnMetadata(integerColumn),
                                   jdbcCFMD(ks1, "JdbcUtf8", UTF8Type.instance).columnMetadata(utf8Column),
                                   jdbcCFMD(ks1, "JdbcLong", LongType.instance),

Added: cassandra/trunk/test/unit/org/apache/cassandra/db/CounterMutationTest.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/db/CounterMutationTest.java?rev=1150765&view=auto
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/db/CounterMutationTest.java (added)
+++ cassandra/trunk/test/unit/org/apache/cassandra/db/CounterMutationTest.java Mon Jul 25 16:11:58 2011
@@ -0,0 +1,76 @@
+/**
+ * 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 java.io.IOException;
+
+import org.junit.Test;
+
+import org.apache.cassandra.db.filter.*;
+import org.apache.cassandra.CleanupHelper;
+import org.apache.cassandra.thrift.*;
+import org.apache.cassandra.utils.*;
+import static org.apache.cassandra.db.context.CounterContext.ContextState;
+
+public class CounterMutationTest extends CleanupHelper
+{
+    @Test
+    public void testMergeOldShards() throws IOException
+    {
+        RowMutation rm;
+        CounterMutation cm;
+
+        rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("key1"));
+        rm.addCounter(new QueryPath("Counter1", null, ByteBufferUtil.bytes("Column1")), 3);
+        cm = new CounterMutation(rm, ConsistencyLevel.ONE);
+        cm.apply();
+
+        NodeId.renewLocalId(2L); // faking time of renewal for test
+        NodeId id1 = NodeId.getLocalId();
+
+        rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("key1"));
+        rm.addCounter(new QueryPath("Counter1", null, ByteBufferUtil.bytes("Column1")), 4);
+        cm = new CounterMutation(rm, ConsistencyLevel.ONE);
+        cm.apply();
+
+        NodeId.renewLocalId(4L); // faking time of renewal for test
+        NodeId id2 = NodeId.getLocalId();
+
+        rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("key1"));
+        rm.addCounter(new QueryPath("Counter1", null, ByteBufferUtil.bytes("Column1")), 5);
+        rm.addCounter(new QueryPath("Counter1", null, ByteBufferUtil.bytes("Column2")), 1);
+        cm = new CounterMutation(rm, ConsistencyLevel.ONE);
+        cm.apply();
+
+        RowMutation reprm = cm.makeReplicationMutation();
+        ColumnFamily cf = reprm.getColumnFamilies().iterator().next();
+        CounterColumn.removeOldShards(cf, Integer.MAX_VALUE);
+        IColumn c = cf.getColumn(ByteBufferUtil.bytes("Column1"));
+        assert c != null;
+        assert c instanceof CounterColumn;
+
+        assert ((CounterColumn)c).total() == 12L;
+        ContextState s = new ContextState(c.value());
+        assert s.getNodeId().equals(id1);
+        assert s.getCount() == 7;
+        s.moveToNext();
+        assert s.getNodeId().equals(id2);
+        assert s.getCount() == 5;
+    }
+}
+