You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by al...@apache.org on 2015/11/04 14:13:43 UTC

[1/2] cassandra git commit: Guard batchlog replay against integer division by zero

Repository: cassandra
Updated Branches:
  refs/heads/trunk b14423c8b -> 81e1b15db


Guard batchlog replay against integer division by zero

patch by Branimir Lambov; reviewed by Stefania Alborghetti for
CASSANDRA-9223


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

Branch: refs/heads/trunk
Commit: 1492be07cdf12afbba662645365cb991f223443d
Parents: d2b7864
Author: Branimir Lambov <br...@datastax.com>
Authored: Wed Oct 28 17:11:35 2015 +0200
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Wed Nov 4 13:12:59 2015 +0000

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cassandra/batchlog/BatchlogManager.java     |  8 +++-
 .../cassandra/batchlog/BatchlogManagerTest.java | 40 ++++++++++++++++++++
 3 files changed, 48 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/1492be07/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index dfb0f17..61cc92e 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0
+ * Guard batchlog replay against integer division by zero (CASSANDRA-9223)
  * Fix bug when adding a column to thrift with the same name than a primary key (CASSANDRA-10608)
  * Add client address argument to IAuthenticator::newSaslNegotiator (CASSANDRA-8068)
  * Fix implementation of LegacyLayout.LegacyBoundComparator (CASSANDRA-10602)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1492be07/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/batchlog/BatchlogManager.java b/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
index 8bc4c26..1c98c2a 100644
--- a/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
+++ b/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
@@ -178,7 +178,13 @@ public class BatchlogManager implements BatchlogManagerMBean
 
         // rate limit is in bytes per second. Uses Double.MAX_VALUE if disabled (set to 0 in cassandra.yaml).
         // max rate is scaled by the number of nodes in the cluster (same as for HHOM - see CASSANDRA-5272).
-        int throttleInKB = DatabaseDescriptor.getBatchlogReplayThrottleInKB() / StorageService.instance.getTokenMetadata().getAllEndpoints().size();
+        int endpointsCount = StorageService.instance.getTokenMetadata().getAllEndpoints().size();
+        if (endpointsCount <= 0)
+        {
+            logger.trace("Replay cancelled as there are no peers in the ring.");
+            return;
+        }
+        int throttleInKB = DatabaseDescriptor.getBatchlogReplayThrottleInKB() / endpointsCount;
         RateLimiter rateLimiter = RateLimiter.create(throttleInKB == 0 ? Double.MAX_VALUE : throttleInKB * 1024);
 
         UUID limitUuid = UUIDGen.maxTimeUUID(System.currentTimeMillis() - getBatchlogTimeout());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1492be07/test/unit/org/apache/cassandra/batchlog/BatchlogManagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/batchlog/BatchlogManagerTest.java b/test/unit/org/apache/cassandra/batchlog/BatchlogManagerTest.java
index dfb17c3..dd5444f 100644
--- a/test/unit/org/apache/cassandra/batchlog/BatchlogManagerTest.java
+++ b/test/unit/org/apache/cassandra/batchlog/BatchlogManagerTest.java
@@ -24,6 +24,7 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 
 import com.google.common.collect.Lists;
+
 import org.junit.*;
 
 import org.apache.cassandra.SchemaLoader;
@@ -457,4 +458,43 @@ public class BatchlogManagerTest
         assertNotNull(result);
         assertEquals(0L, result.one().getLong("count"));
     }
+
+    // CASSANRDA-9223
+    @Test
+    public void testReplayWithNoPeers() throws Exception
+    {
+        StorageService.instance.getTokenMetadata().removeEndpoint(InetAddress.getByName("127.0.0.1"));
+
+        long initialAllBatches = BatchlogManager.instance.countAllBatches();
+        long initialReplayedBatches = BatchlogManager.instance.getTotalBatchesReplayed();
+
+        CFMetaData cfm = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD1).metadata;
+
+        long timestamp = (System.currentTimeMillis() - DatabaseDescriptor.getWriteRpcTimeout() * 2) * 1000;
+        UUID uuid = UUIDGen.getTimeUUID();
+
+        // Add a batch with 10 mutations
+        List<Mutation> mutations = new ArrayList<>(10);
+        for (int j = 0; j < 10; j++)
+        {
+            mutations.add(new RowUpdateBuilder(cfm, FBUtilities.timestampMicros(), ByteBufferUtil.bytes(j))
+                          .clustering("name" + j)
+                          .add("val", "val" + j)
+                          .build());
+        }
+        BatchlogManager.store(Batch.createLocal(uuid, timestamp, mutations));
+        assertEquals(1, BatchlogManager.instance.countAllBatches() - initialAllBatches);
+
+        // Flush the batchlog to disk (see CASSANDRA-6822).
+        Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.BATCHES).forceBlockingFlush();
+
+        assertEquals(1, BatchlogManager.instance.countAllBatches() - initialAllBatches);
+        assertEquals(0, BatchlogManager.instance.getTotalBatchesReplayed() - initialReplayedBatches);
+
+        // Force batchlog replay and wait for it to complete.
+        BatchlogManager.instance.startBatchlogReplay().get();
+
+        // Replay should be cancelled as there are no peers in the ring.
+        assertEquals(1, BatchlogManager.instance.countAllBatches() - initialAllBatches);
+    }
 }


[2/2] cassandra git commit: Merge branch 'cassandra-3.0' into trunk

Posted by al...@apache.org.
Merge branch 'cassandra-3.0' into trunk


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

Branch: refs/heads/trunk
Commit: 81e1b15dbd25b6ef8d23ff3b4aba573f4e9efd1a
Parents: b14423c 1492be0
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Wed Nov 4 13:13:32 2015 +0000
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Wed Nov 4 13:13:32 2015 +0000

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cassandra/batchlog/BatchlogManager.java     |  8 +++-
 .../cassandra/batchlog/BatchlogManagerTest.java | 40 ++++++++++++++++++++
 3 files changed, 48 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/81e1b15d/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 6b5984b,61cc92e..1ae41a0
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,10 -1,5 +1,11 @@@
 +3.2
 + * Added graphing option to cassandra-stress (CASSANDRA-7918)
 + * Abort in-progress queries that time out (CASSANDRA-7392)
 + * Add transparent data encryption core classes (CASSANDRA-9945)
 +
 +
  3.0
+  * Guard batchlog replay against integer division by zero (CASSANDRA-9223)
   * Fix bug when adding a column to thrift with the same name than a primary key (CASSANDRA-10608)
   * Add client address argument to IAuthenticator::newSaslNegotiator (CASSANDRA-8068)
   * Fix implementation of LegacyLayout.LegacyBoundComparator (CASSANDRA-10602)