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 2017/08/14 14:29:34 UTC

[1/6] cassandra git commit: Randomize batchlog endpoint selection with only 1 or 2 racks

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-3.0 fab384560 -> c2b635ac2
  refs/heads/cassandra-3.11 1884dbe28 -> db57cbddc
  refs/heads/trunk ff06424fa -> 99e5f7efc


Randomize batchlog endpoint selection with only 1 or 2 racks

patch by Daniel Cranford; reviewed by Aleksey Yeschenko for
CASSANDRA-12884


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

Branch: refs/heads/cassandra-3.0
Commit: c2b635ac240ae8d9375fd96791a5aba903a94498
Parents: fab3845
Author: dcranford <da...@red-alpha.com>
Authored: Wed Aug 9 10:20:03 2017 -0400
Committer: Aleksey Yeschenko <al...@yeschenko.com>
Committed: Mon Aug 14 15:23:09 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cassandra/batchlog/BatchlogManager.java     | 19 ++++++++---
 .../batchlog/BatchlogEndpointFilterTest.java    | 33 ++++++++++++++++++--
 3 files changed, 47 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/c2b635ac/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 2e9e8ad..358dd04 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.15
+ * Randomize batchlog endpoint selection with only 1 or 2 racks (CASSANDRA-12884)
  * Fix digest calculation for counter cells (CASSANDRA-13750)
  * Fix ColumnDefinition.cellValueType() for non-frozen collection and change SSTabledump to use type.toJSONString() (CASSANDRA-13573)
  * Skip materialized view addition if the base table doesn't exist (CASSANDRA-13737)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c2b635ac/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 f5133bb..b614fc5 100644
--- a/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
+++ b/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
@@ -523,9 +523,14 @@ public class BatchlogManager implements BatchlogManagerMBean
 
             if (validated.keySet().size() == 1)
             {
-                // we have only 1 `other` rack
-                Collection<InetAddress> otherRack = Iterables.getOnlyElement(validated.asMap().values());
-                return Lists.newArrayList(Iterables.limit(otherRack, 2));
+                /*
+                 * we have only 1 `other` rack to select replicas from (whether it be the local rack or a single non-local rack)
+                 * pick two random nodes from there; we are guaranteed to have at least two nodes in the single remaining rack
+                 * because of the preceding if block.
+                 */
+                List<InetAddress> otherRack = Lists.newArrayList(validated.values());
+                shuffle(otherRack);
+                return otherRack.subList(0, 2);
             }
 
             // randomize which racks we pick from if more than 2 remaining
@@ -537,7 +542,7 @@ public class BatchlogManager implements BatchlogManagerMBean
             else
             {
                 racks = Lists.newArrayList(validated.keySet());
-                Collections.shuffle((List<String>) racks);
+                shuffle((List<String>) racks);
             }
 
             // grab a random member of up to two racks
@@ -562,5 +567,11 @@ public class BatchlogManager implements BatchlogManagerMBean
         {
             return ThreadLocalRandom.current().nextInt(bound);
         }
+
+        @VisibleForTesting
+        protected void shuffle(List<?> list)
+        {
+            Collections.shuffle(list);
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c2b635ac/test/unit/org/apache/cassandra/batchlog/BatchlogEndpointFilterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/batchlog/BatchlogEndpointFilterTest.java b/test/unit/org/apache/cassandra/batchlog/BatchlogEndpointFilterTest.java
index 23aeaaa..7db1cfa 100644
--- a/test/unit/org/apache/cassandra/batchlog/BatchlogEndpointFilterTest.java
+++ b/test/unit/org/apache/cassandra/batchlog/BatchlogEndpointFilterTest.java
@@ -20,7 +20,9 @@ package org.apache.cassandra.batchlog;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashSet;
+import java.util.List;
 
 import com.google.common.collect.ImmutableMultimap;
 import com.google.common.collect.Multimap;
@@ -87,8 +89,28 @@ public class BatchlogEndpointFilterTest
                 .put("1", InetAddress.getByName("111"))
                 .build();
         Collection<InetAddress> result = new TestEndpointFilter(LOCAL, endpoints).filter();
-        // result should contain random two distinct values
-        assertThat(new HashSet<>(result).size(), is(2));
+        // result should be the last two non-local replicas
+        // (Collections.shuffle has been replaced with Collections.reverse for testing)
+        assertThat(result.size(), is(2));
+        assertThat(result, JUnitMatchers.hasItem(InetAddress.getByName("11")));
+        assertThat(result, JUnitMatchers.hasItem(InetAddress.getByName("111")));
+    }
+
+    @Test
+    public void shouldSelectTwoRandomHostsFromSingleRack() throws UnknownHostException
+    {
+        Multimap<String, InetAddress> endpoints = ImmutableMultimap.<String, InetAddress> builder()
+                .put(LOCAL, InetAddress.getByName("1"))
+                .put(LOCAL, InetAddress.getByName("11"))
+                .put(LOCAL, InetAddress.getByName("111"))
+                .put(LOCAL, InetAddress.getByName("1111"))
+                .build();
+        Collection<InetAddress> result = new TestEndpointFilter(LOCAL, endpoints).filter();
+        // result should be the last two non-local replicas
+        // (Collections.shuffle has been replaced with Collections.reverse for testing)
+        assertThat(result.size(), is(2));
+        assertThat(result, JUnitMatchers.hasItem(InetAddress.getByName("111")));
+        assertThat(result, JUnitMatchers.hasItem(InetAddress.getByName("1111")));
     }
 
     private static class TestEndpointFilter extends BatchlogManager.EndpointFilter
@@ -111,5 +133,12 @@ public class BatchlogEndpointFilterTest
             // We don't need random behavior here
             return bound - 1;
         }
+
+        @Override
+        protected void shuffle(List<?> list)
+        {
+            // We don't need random behavior here
+            Collections.reverse(list);
+        }
     }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[6/6] cassandra git commit: Merge branch 'cassandra-3.11' into trunk

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


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

Branch: refs/heads/trunk
Commit: 99e5f7efc33fb3672e11dfba9f2521d09473dddf
Parents: ff06424 db57cbd
Author: Aleksey Yeschenko <al...@yeschenko.com>
Authored: Mon Aug 14 15:28:22 2017 +0100
Committer: Aleksey Yeschenko <al...@yeschenko.com>
Committed: Mon Aug 14 15:28:22 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cassandra/batchlog/BatchlogManager.java     | 19 ++++++++---
 .../batchlog/BatchlogEndpointFilterTest.java    | 33 ++++++++++++++++++--
 3 files changed, 47 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/99e5f7ef/CHANGES.txt
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/99e5f7ef/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
----------------------------------------------------------------------


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[5/6] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11

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


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

Branch: refs/heads/cassandra-3.11
Commit: db57cbddc390c7af4d824962686ab6f6a0b3d079
Parents: 1884dbe c2b635a
Author: Aleksey Yeschenko <al...@yeschenko.com>
Authored: Mon Aug 14 15:25:59 2017 +0100
Committer: Aleksey Yeschenko <al...@yeschenko.com>
Committed: Mon Aug 14 15:25:59 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cassandra/batchlog/BatchlogManager.java     | 19 ++++++++---
 .../batchlog/BatchlogEndpointFilterTest.java    | 33 ++++++++++++++++++--
 3 files changed, 47 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/db57cbdd/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index c672675,358dd04..5403812
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,9 -1,5 +1,10 @@@
 -3.0.15
 +3.11.1
 + * "ignore" option is ignored in sstableloader (CASSANDRA-13721)
 + * Deadlock in AbstractCommitLogSegmentManager (CASSANDRA-13652)
 + * Duplicate the buffer before passing it to analyser in SASI operation (CASSANDRA-13512)
 + * Properly evict pstmts from prepared statements cache (CASSANDRA-13641)
 +Merged from 3.0:
+  * Randomize batchlog endpoint selection with only 1 or 2 racks (CASSANDRA-12884)
   * Fix digest calculation for counter cells (CASSANDRA-13750)
   * Fix ColumnDefinition.cellValueType() for non-frozen collection and change SSTabledump to use type.toJSONString() (CASSANDRA-13573)
   * Skip materialized view addition if the base table doesn't exist (CASSANDRA-13737)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/db57cbdd/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
----------------------------------------------------------------------


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[3/6] cassandra git commit: Randomize batchlog endpoint selection with only 1 or 2 racks

Posted by al...@apache.org.
Randomize batchlog endpoint selection with only 1 or 2 racks

patch by Daniel Cranford; reviewed by Aleksey Yeschenko for
CASSANDRA-12884


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

Branch: refs/heads/trunk
Commit: c2b635ac240ae8d9375fd96791a5aba903a94498
Parents: fab3845
Author: dcranford <da...@red-alpha.com>
Authored: Wed Aug 9 10:20:03 2017 -0400
Committer: Aleksey Yeschenko <al...@yeschenko.com>
Committed: Mon Aug 14 15:23:09 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cassandra/batchlog/BatchlogManager.java     | 19 ++++++++---
 .../batchlog/BatchlogEndpointFilterTest.java    | 33 ++++++++++++++++++--
 3 files changed, 47 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/c2b635ac/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 2e9e8ad..358dd04 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.15
+ * Randomize batchlog endpoint selection with only 1 or 2 racks (CASSANDRA-12884)
  * Fix digest calculation for counter cells (CASSANDRA-13750)
  * Fix ColumnDefinition.cellValueType() for non-frozen collection and change SSTabledump to use type.toJSONString() (CASSANDRA-13573)
  * Skip materialized view addition if the base table doesn't exist (CASSANDRA-13737)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c2b635ac/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 f5133bb..b614fc5 100644
--- a/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
+++ b/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
@@ -523,9 +523,14 @@ public class BatchlogManager implements BatchlogManagerMBean
 
             if (validated.keySet().size() == 1)
             {
-                // we have only 1 `other` rack
-                Collection<InetAddress> otherRack = Iterables.getOnlyElement(validated.asMap().values());
-                return Lists.newArrayList(Iterables.limit(otherRack, 2));
+                /*
+                 * we have only 1 `other` rack to select replicas from (whether it be the local rack or a single non-local rack)
+                 * pick two random nodes from there; we are guaranteed to have at least two nodes in the single remaining rack
+                 * because of the preceding if block.
+                 */
+                List<InetAddress> otherRack = Lists.newArrayList(validated.values());
+                shuffle(otherRack);
+                return otherRack.subList(0, 2);
             }
 
             // randomize which racks we pick from if more than 2 remaining
@@ -537,7 +542,7 @@ public class BatchlogManager implements BatchlogManagerMBean
             else
             {
                 racks = Lists.newArrayList(validated.keySet());
-                Collections.shuffle((List<String>) racks);
+                shuffle((List<String>) racks);
             }
 
             // grab a random member of up to two racks
@@ -562,5 +567,11 @@ public class BatchlogManager implements BatchlogManagerMBean
         {
             return ThreadLocalRandom.current().nextInt(bound);
         }
+
+        @VisibleForTesting
+        protected void shuffle(List<?> list)
+        {
+            Collections.shuffle(list);
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c2b635ac/test/unit/org/apache/cassandra/batchlog/BatchlogEndpointFilterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/batchlog/BatchlogEndpointFilterTest.java b/test/unit/org/apache/cassandra/batchlog/BatchlogEndpointFilterTest.java
index 23aeaaa..7db1cfa 100644
--- a/test/unit/org/apache/cassandra/batchlog/BatchlogEndpointFilterTest.java
+++ b/test/unit/org/apache/cassandra/batchlog/BatchlogEndpointFilterTest.java
@@ -20,7 +20,9 @@ package org.apache.cassandra.batchlog;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashSet;
+import java.util.List;
 
 import com.google.common.collect.ImmutableMultimap;
 import com.google.common.collect.Multimap;
@@ -87,8 +89,28 @@ public class BatchlogEndpointFilterTest
                 .put("1", InetAddress.getByName("111"))
                 .build();
         Collection<InetAddress> result = new TestEndpointFilter(LOCAL, endpoints).filter();
-        // result should contain random two distinct values
-        assertThat(new HashSet<>(result).size(), is(2));
+        // result should be the last two non-local replicas
+        // (Collections.shuffle has been replaced with Collections.reverse for testing)
+        assertThat(result.size(), is(2));
+        assertThat(result, JUnitMatchers.hasItem(InetAddress.getByName("11")));
+        assertThat(result, JUnitMatchers.hasItem(InetAddress.getByName("111")));
+    }
+
+    @Test
+    public void shouldSelectTwoRandomHostsFromSingleRack() throws UnknownHostException
+    {
+        Multimap<String, InetAddress> endpoints = ImmutableMultimap.<String, InetAddress> builder()
+                .put(LOCAL, InetAddress.getByName("1"))
+                .put(LOCAL, InetAddress.getByName("11"))
+                .put(LOCAL, InetAddress.getByName("111"))
+                .put(LOCAL, InetAddress.getByName("1111"))
+                .build();
+        Collection<InetAddress> result = new TestEndpointFilter(LOCAL, endpoints).filter();
+        // result should be the last two non-local replicas
+        // (Collections.shuffle has been replaced with Collections.reverse for testing)
+        assertThat(result.size(), is(2));
+        assertThat(result, JUnitMatchers.hasItem(InetAddress.getByName("111")));
+        assertThat(result, JUnitMatchers.hasItem(InetAddress.getByName("1111")));
     }
 
     private static class TestEndpointFilter extends BatchlogManager.EndpointFilter
@@ -111,5 +133,12 @@ public class BatchlogEndpointFilterTest
             // We don't need random behavior here
             return bound - 1;
         }
+
+        @Override
+        protected void shuffle(List<?> list)
+        {
+            // We don't need random behavior here
+            Collections.reverse(list);
+        }
     }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[2/6] cassandra git commit: Randomize batchlog endpoint selection with only 1 or 2 racks

Posted by al...@apache.org.
Randomize batchlog endpoint selection with only 1 or 2 racks

patch by Daniel Cranford; reviewed by Aleksey Yeschenko for
CASSANDRA-12884


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

Branch: refs/heads/cassandra-3.11
Commit: c2b635ac240ae8d9375fd96791a5aba903a94498
Parents: fab3845
Author: dcranford <da...@red-alpha.com>
Authored: Wed Aug 9 10:20:03 2017 -0400
Committer: Aleksey Yeschenko <al...@yeschenko.com>
Committed: Mon Aug 14 15:23:09 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cassandra/batchlog/BatchlogManager.java     | 19 ++++++++---
 .../batchlog/BatchlogEndpointFilterTest.java    | 33 ++++++++++++++++++--
 3 files changed, 47 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/c2b635ac/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 2e9e8ad..358dd04 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.15
+ * Randomize batchlog endpoint selection with only 1 or 2 racks (CASSANDRA-12884)
  * Fix digest calculation for counter cells (CASSANDRA-13750)
  * Fix ColumnDefinition.cellValueType() for non-frozen collection and change SSTabledump to use type.toJSONString() (CASSANDRA-13573)
  * Skip materialized view addition if the base table doesn't exist (CASSANDRA-13737)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c2b635ac/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 f5133bb..b614fc5 100644
--- a/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
+++ b/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
@@ -523,9 +523,14 @@ public class BatchlogManager implements BatchlogManagerMBean
 
             if (validated.keySet().size() == 1)
             {
-                // we have only 1 `other` rack
-                Collection<InetAddress> otherRack = Iterables.getOnlyElement(validated.asMap().values());
-                return Lists.newArrayList(Iterables.limit(otherRack, 2));
+                /*
+                 * we have only 1 `other` rack to select replicas from (whether it be the local rack or a single non-local rack)
+                 * pick two random nodes from there; we are guaranteed to have at least two nodes in the single remaining rack
+                 * because of the preceding if block.
+                 */
+                List<InetAddress> otherRack = Lists.newArrayList(validated.values());
+                shuffle(otherRack);
+                return otherRack.subList(0, 2);
             }
 
             // randomize which racks we pick from if more than 2 remaining
@@ -537,7 +542,7 @@ public class BatchlogManager implements BatchlogManagerMBean
             else
             {
                 racks = Lists.newArrayList(validated.keySet());
-                Collections.shuffle((List<String>) racks);
+                shuffle((List<String>) racks);
             }
 
             // grab a random member of up to two racks
@@ -562,5 +567,11 @@ public class BatchlogManager implements BatchlogManagerMBean
         {
             return ThreadLocalRandom.current().nextInt(bound);
         }
+
+        @VisibleForTesting
+        protected void shuffle(List<?> list)
+        {
+            Collections.shuffle(list);
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c2b635ac/test/unit/org/apache/cassandra/batchlog/BatchlogEndpointFilterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/batchlog/BatchlogEndpointFilterTest.java b/test/unit/org/apache/cassandra/batchlog/BatchlogEndpointFilterTest.java
index 23aeaaa..7db1cfa 100644
--- a/test/unit/org/apache/cassandra/batchlog/BatchlogEndpointFilterTest.java
+++ b/test/unit/org/apache/cassandra/batchlog/BatchlogEndpointFilterTest.java
@@ -20,7 +20,9 @@ package org.apache.cassandra.batchlog;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashSet;
+import java.util.List;
 
 import com.google.common.collect.ImmutableMultimap;
 import com.google.common.collect.Multimap;
@@ -87,8 +89,28 @@ public class BatchlogEndpointFilterTest
                 .put("1", InetAddress.getByName("111"))
                 .build();
         Collection<InetAddress> result = new TestEndpointFilter(LOCAL, endpoints).filter();
-        // result should contain random two distinct values
-        assertThat(new HashSet<>(result).size(), is(2));
+        // result should be the last two non-local replicas
+        // (Collections.shuffle has been replaced with Collections.reverse for testing)
+        assertThat(result.size(), is(2));
+        assertThat(result, JUnitMatchers.hasItem(InetAddress.getByName("11")));
+        assertThat(result, JUnitMatchers.hasItem(InetAddress.getByName("111")));
+    }
+
+    @Test
+    public void shouldSelectTwoRandomHostsFromSingleRack() throws UnknownHostException
+    {
+        Multimap<String, InetAddress> endpoints = ImmutableMultimap.<String, InetAddress> builder()
+                .put(LOCAL, InetAddress.getByName("1"))
+                .put(LOCAL, InetAddress.getByName("11"))
+                .put(LOCAL, InetAddress.getByName("111"))
+                .put(LOCAL, InetAddress.getByName("1111"))
+                .build();
+        Collection<InetAddress> result = new TestEndpointFilter(LOCAL, endpoints).filter();
+        // result should be the last two non-local replicas
+        // (Collections.shuffle has been replaced with Collections.reverse for testing)
+        assertThat(result.size(), is(2));
+        assertThat(result, JUnitMatchers.hasItem(InetAddress.getByName("111")));
+        assertThat(result, JUnitMatchers.hasItem(InetAddress.getByName("1111")));
     }
 
     private static class TestEndpointFilter extends BatchlogManager.EndpointFilter
@@ -111,5 +133,12 @@ public class BatchlogEndpointFilterTest
             // We don't need random behavior here
             return bound - 1;
         }
+
+        @Override
+        protected void shuffle(List<?> list)
+        {
+            // We don't need random behavior here
+            Collections.reverse(list);
+        }
     }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[4/6] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11

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


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

Branch: refs/heads/trunk
Commit: db57cbddc390c7af4d824962686ab6f6a0b3d079
Parents: 1884dbe c2b635a
Author: Aleksey Yeschenko <al...@yeschenko.com>
Authored: Mon Aug 14 15:25:59 2017 +0100
Committer: Aleksey Yeschenko <al...@yeschenko.com>
Committed: Mon Aug 14 15:25:59 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cassandra/batchlog/BatchlogManager.java     | 19 ++++++++---
 .../batchlog/BatchlogEndpointFilterTest.java    | 33 ++++++++++++++++++--
 3 files changed, 47 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/db57cbdd/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index c672675,358dd04..5403812
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,9 -1,5 +1,10 @@@
 -3.0.15
 +3.11.1
 + * "ignore" option is ignored in sstableloader (CASSANDRA-13721)
 + * Deadlock in AbstractCommitLogSegmentManager (CASSANDRA-13652)
 + * Duplicate the buffer before passing it to analyser in SASI operation (CASSANDRA-13512)
 + * Properly evict pstmts from prepared statements cache (CASSANDRA-13641)
 +Merged from 3.0:
+  * Randomize batchlog endpoint selection with only 1 or 2 racks (CASSANDRA-12884)
   * Fix digest calculation for counter cells (CASSANDRA-13750)
   * Fix ColumnDefinition.cellValueType() for non-frozen collection and change SSTabledump to use type.toJSONString() (CASSANDRA-13573)
   * Skip materialized view addition if the base table doesn't exist (CASSANDRA-13737)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/db57cbdd/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
----------------------------------------------------------------------


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org