You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by yu...@apache.org on 2015/07/23 17:28:02 UTC

[3/6] cassandra git commit: Use random nodes for batch log when only 2 racks

Use random nodes for batch log when only 2 racks

patch by Mihai Suteu and yukim; reviewed by Jeremiah Jordan for CASSANDRA-8735


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

Branch: refs/heads/trunk
Commit: 1c80b04be1d47d03bbde888cea960f5ff8a95d58
Parents: c2142e6
Author: Yuki Morishita <yu...@apache.org>
Authored: Thu Jul 23 10:24:23 2015 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Thu Jul 23 10:24:23 2015 -0500

----------------------------------------------------------------------
 CHANGES.txt                                                 | 1 +
 src/java/org/apache/cassandra/db/BatchlogManager.java       | 4 +++-
 .../cassandra/service/BatchlogEndpointFilterTest.java       | 9 +++++----
 3 files changed, 9 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/1c80b04b/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 5d142cc..69a7b31 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -6,6 +6,7 @@
  * (cqlsh) Allow the SSL protocol version to be specified through the
    config file or environment variables (CASSANDRA-9544)
  * Remove repair snapshot leftover on startup (CASSANDRA-7357)
+ * Use random nodes for batch log when only 2 racks (CASSANDRA-8735)
 Merged from 2.0:
  * checkForEndpointCollision fails for legitimate collisions (CASSANDRA-9765)
  * Complete CASSANDRA-8448 fix (CASSANDRA-9519)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1c80b04b/src/java/org/apache/cassandra/db/BatchlogManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/BatchlogManager.java b/src/java/org/apache/cassandra/db/BatchlogManager.java
index 20f134d..4588156 100644
--- a/src/java/org/apache/cassandra/db/BatchlogManager.java
+++ b/src/java/org/apache/cassandra/db/BatchlogManager.java
@@ -495,7 +495,9 @@ public class BatchlogManager implements BatchlogManagerMBean
             if (validated.keySet().size() == 1)
             {
                 // we have only 1 `other` rack
-                Collection<InetAddress> otherRack = Iterables.getOnlyElement(validated.asMap().values());
+                // pick up to two random nodes from there
+                List<InetAddress> otherRack = validated.get(validated.keySet().iterator().next());
+                Collections.shuffle(otherRack);
                 return Lists.newArrayList(Iterables.limit(otherRack, 2));
             }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1c80b04b/test/unit/org/apache/cassandra/service/BatchlogEndpointFilterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/BatchlogEndpointFilterTest.java b/test/unit/org/apache/cassandra/service/BatchlogEndpointFilterTest.java
index 72e8df5..3a19b75 100644
--- a/test/unit/org/apache/cassandra/service/BatchlogEndpointFilterTest.java
+++ b/test/unit/org/apache/cassandra/service/BatchlogEndpointFilterTest.java
@@ -20,6 +20,8 @@ package org.apache.cassandra.service;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.Collection;
+import java.util.HashSet;
+
 import org.junit.Test;
 import org.junit.matchers.JUnitMatchers;
 
@@ -78,7 +80,7 @@ public class BatchlogEndpointFilterTest
     }
 
     @Test
-    public void shouldSelectTwoFirstHostsFromSingleOtherRack() throws UnknownHostException
+    public void shouldSelectTwoRandomHostsFromSingleOtherRack() throws UnknownHostException
     {
         Multimap<String, InetAddress> endpoints = ImmutableMultimap.<String, InetAddress> builder()
                 .put(LOCAL, InetAddress.getByName("0"))
@@ -88,9 +90,8 @@ public class BatchlogEndpointFilterTest
                 .put("1", InetAddress.getByName("111"))
                 .build();
         Collection<InetAddress> result = new TestEndpointFilter(LOCAL, endpoints).filter();
-        assertThat(result.size(), is(2));
-        assertThat(result, JUnitMatchers.hasItem(InetAddress.getByName("1")));
-        assertThat(result, JUnitMatchers.hasItem(InetAddress.getByName("11")));
+        // result should contain random two distinct values
+        assertThat(new HashSet<>(result).size(), is(2));
     }
 
     private static class TestEndpointFilter extends BatchlogManager.EndpointFilter