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 2015/07/16 15:29:09 UTC

[1/3] cassandra git commit: Move CASSANDRA-9519 test in long tests (and reduce the size of the list used)

Repository: cassandra
Updated Branches:
  refs/heads/trunk b968ba4e6 -> 4feaa7a3d


Move CASSANDRA-9519 test in long tests (and reduce the size of the list used)


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

Branch: refs/heads/trunk
Commit: 4fcd7d4d366d001cf5f1f7d926c608c902e3f0af
Parents: 9ff5d44
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Thu Jul 16 15:14:54 2015 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Thu Jul 16 15:21:26 2015 +0200

----------------------------------------------------------------------
 .../locator/DynamicEndpointSnitchLongTest.java  | 104 +++++++++++++++++++
 .../locator/DynamicEndpointSnitchTest.java      |  64 ------------
 2 files changed, 104 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/4fcd7d4d/test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java b/test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java
new file mode 100644
index 0000000..1c628fa
--- /dev/null
+++ b/test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java
@@ -0,0 +1,104 @@
+/*
+* 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.locator;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.util.*;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.service.StorageService;
+import org.junit.Test;
+
+import org.apache.cassandra.utils.FBUtilities;
+
+import static org.junit.Assert.assertEquals;
+
+public class DynamicEndpointSnitchLongTest
+{
+    @Test
+    public void testConcurrency() throws InterruptedException, IOException, ConfigurationException
+    {
+        // The goal of this test is to check for CASSANDRA-8448/CASSANDRA-9519
+        double badness = DatabaseDescriptor.getDynamicBadnessThreshold();
+        DatabaseDescriptor.setDynamicBadnessThreshold(0.0);
+
+        try
+        {
+            final int ITERATIONS = 10000;
+
+            // do this because SS needs to be initialized before DES can work properly.
+            StorageService.instance.initClient(0);
+            SimpleSnitch ss = new SimpleSnitch();
+            DynamicEndpointSnitch dsnitch = new DynamicEndpointSnitch(ss, String.valueOf(ss.hashCode()));
+            InetAddress self = FBUtilities.getBroadcastAddress();
+
+            List<InetAddress> hosts = new ArrayList<>();
+            // We want a big list of hosts so  sorting takes time, making it much more likely to reproduce the
+            // problem we're looking for.
+            for (int i = 0; i < 100; i++)
+                for (int j = 0; j < 256; j++)
+                    hosts.add(InetAddress.getByAddress(new byte[]{127, 0, (byte)i, (byte)j}));
+
+            ScoreUpdater updater = new ScoreUpdater(dsnitch, hosts);
+            updater.start();
+
+            List<InetAddress> result = null;
+            for (int i = 0; i < ITERATIONS; i++)
+                result = dsnitch.getSortedListByProximity(self, hosts);
+
+            updater.stopped = true;
+            updater.join();
+        }
+        finally
+        {
+            DatabaseDescriptor.setDynamicBadnessThreshold(badness);
+        }
+    }
+
+    public static class ScoreUpdater extends Thread
+    {
+        private static final int SCORE_RANGE = 100;
+
+        public volatile boolean stopped;
+
+        private final DynamicEndpointSnitch dsnitch;
+        private final List<InetAddress> hosts;
+        private final Random random = new Random();
+
+        public ScoreUpdater(DynamicEndpointSnitch dsnitch, List<InetAddress> hosts)
+        {
+            this.dsnitch = dsnitch;
+            this.hosts = hosts;
+        }
+
+        public void run()
+        {
+            while (!stopped)
+            {
+                InetAddress host = hosts.get(random.nextInt(hosts.size()));
+                int score = random.nextInt(SCORE_RANGE);
+                dsnitch.receiveTiming(host, score);
+            }
+        }
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4fcd7d4d/test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java b/test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java
index 3f90532..c1928d8 100644
--- a/test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java
@@ -23,7 +23,6 @@ import java.io.IOException;
 import java.net.InetAddress;
 import java.util.*;
 
-import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.service.StorageService;
 import org.junit.Test;
@@ -90,67 +89,4 @@ public class DynamicEndpointSnitchTest
         order = Arrays.asList(host1, host3, host2);
         assertEquals(order, dsnitch.getSortedListByProximity(self, Arrays.asList(host1, host2, host3)));
     }
-
-    @Test
-    public void testConcurrency() throws InterruptedException, IOException, ConfigurationException
-    {
-        // The goal of this test is to check for CASSANDRA-8448/CASSANDRA-9519
-        double badness = DatabaseDescriptor.getDynamicBadnessThreshold();
-        DatabaseDescriptor.setDynamicBadnessThreshold(0.0);
-
-        final int ITERATIONS = 10;
-
-        // do this because SS needs to be initialized before DES can work properly.
-        StorageService.instance.initClient(0);
-        SimpleSnitch ss = new SimpleSnitch();
-        DynamicEndpointSnitch dsnitch = new DynamicEndpointSnitch(ss, String.valueOf(ss.hashCode()));
-        InetAddress self = FBUtilities.getBroadcastAddress();
-
-        List<InetAddress> hosts = new ArrayList<>();
-        // We want a giant list of hosts so that sorting it takes time, making it much more likely to reproduce the
-        // problem we're looking for.
-        for (int i = 0; i < 10; i++)
-            for (int j = 0; j < 256; j++)
-                for (int k = 0; k < 256; k++)
-                    hosts.add(InetAddress.getByAddress(new byte[]{127, (byte)i, (byte)j, (byte)k}));
-
-        ScoreUpdater updater = new ScoreUpdater(dsnitch, hosts);
-        updater.start();
-
-        List<InetAddress> result = null;
-        for (int i = 0; i < ITERATIONS; i++)
-            result = dsnitch.getSortedListByProximity(self, hosts);
-
-        updater.stopped = true;
-        updater.join();
-
-        DatabaseDescriptor.setDynamicBadnessThreshold(badness);
-    }
-
-    public static class ScoreUpdater extends Thread
-    {
-        private static final int SCORE_RANGE = 100;
-
-        public volatile boolean stopped;
-
-        private final DynamicEndpointSnitch dsnitch;
-        private final List<InetAddress> hosts;
-        private final Random random = new Random();
-
-        public ScoreUpdater(DynamicEndpointSnitch dsnitch, List<InetAddress> hosts)
-        {
-            this.dsnitch = dsnitch;
-            this.hosts = hosts;
-        }
-
-        public void run()
-        {
-            while (!stopped)
-            {
-                InetAddress host = hosts.get(random.nextInt(hosts.size()));
-                int score = random.nextInt(SCORE_RANGE);
-                dsnitch.receiveTiming(host, score);
-            }
-        }
-    }
 }


[3/3] cassandra git commit: Merge branch 'cassandra-2.2' into trunk

Posted by sl...@apache.org.
Merge branch 'cassandra-2.2' into trunk

Conflicts:
	test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java


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

Branch: refs/heads/trunk
Commit: 4feaa7a3dbc1058a800f2ea825fd7a54c92a7ef5
Parents: b968ba4 d97fc9b
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Thu Jul 16 15:29:02 2015 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Thu Jul 16 15:29:02 2015 +0200

----------------------------------------------------------------------
 .../locator/DynamicEndpointSnitchLongTest.java  | 105 +++++++++++++++++++
 .../locator/DynamicEndpointSnitchTest.java      |  64 -----------
 2 files changed, 105 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/4feaa7a3/test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java
----------------------------------------------------------------------
diff --cc test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java
index 0000000,841f73e..0d66fa9
mode 000000,100644..100644
--- a/test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java
+++ b/test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java
@@@ -1,0 -1,104 +1,105 @@@
+ /*
+ * 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.locator;
+ 
+ import java.io.IOException;
+ import java.net.InetAddress;
+ import java.util.*;
+ 
++import org.junit.Test;
++
+ import org.apache.cassandra.config.DatabaseDescriptor;
+ import org.apache.cassandra.exceptions.ConfigurationException;
+ import org.apache.cassandra.service.StorageService;
 -import org.junit.Test;
+ 
+ import org.apache.cassandra.utils.FBUtilities;
+ 
+ import static org.junit.Assert.assertEquals;
+ 
+ public class DynamicEndpointSnitchLongTest
+ {
+     @Test
+     public void testConcurrency() throws InterruptedException, IOException, ConfigurationException
+     {
+         // The goal of this test is to check for CASSANDRA-8448/CASSANDRA-9519
+         double badness = DatabaseDescriptor.getDynamicBadnessThreshold();
+         DatabaseDescriptor.setDynamicBadnessThreshold(0.0);
+ 
+         try
+         {
+             final int ITERATIONS = 10000;
+ 
+             // do this because SS needs to be initialized before DES can work properly.
+             StorageService.instance.unsafeInitialize();
+             SimpleSnitch ss = new SimpleSnitch();
+             DynamicEndpointSnitch dsnitch = new DynamicEndpointSnitch(ss, String.valueOf(ss.hashCode()));
+             InetAddress self = FBUtilities.getBroadcastAddress();
+ 
+             List<InetAddress> hosts = new ArrayList<>();
+             // We want a big list of hosts so  sorting takes time, making it much more likely to reproduce the
+             // problem we're looking for.
+             for (int i = 0; i < 100; i++)
+                 for (int j = 0; j < 256; j++)
+                     hosts.add(InetAddress.getByAddress(new byte[]{127, 0, (byte)i, (byte)j}));
+ 
+             ScoreUpdater updater = new ScoreUpdater(dsnitch, hosts);
+             updater.start();
+ 
+             List<InetAddress> result = null;
+             for (int i = 0; i < ITERATIONS; i++)
+                 result = dsnitch.getSortedListByProximity(self, hosts);
+ 
+             updater.stopped = true;
+             updater.join();
+         }
+         finally
+         {
+             DatabaseDescriptor.setDynamicBadnessThreshold(badness);
+         }
+     }
+ 
+     public static class ScoreUpdater extends Thread
+     {
+         private static final int SCORE_RANGE = 100;
+ 
+         public volatile boolean stopped;
+ 
+         private final DynamicEndpointSnitch dsnitch;
+         private final List<InetAddress> hosts;
+         private final Random random = new Random();
+ 
+         public ScoreUpdater(DynamicEndpointSnitch dsnitch, List<InetAddress> hosts)
+         {
+             this.dsnitch = dsnitch;
+             this.hosts = hosts;
+         }
+ 
+         public void run()
+         {
+             while (!stopped)
+             {
+                 InetAddress host = hosts.get(random.nextInt(hosts.size()));
+                 int score = random.nextInt(SCORE_RANGE);
+                 dsnitch.receiveTiming(host, score);
+             }
+         }
+     }
+ }
+ 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4feaa7a3/test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java
index e5f156f,100cd25..a0b1863
--- a/test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java
@@@ -22,11 -23,10 +22,10 @@@ import java.io.IOException
  import java.net.InetAddress;
  import java.util.*;
  
 -import org.apache.cassandra.exceptions.ConfigurationException;
 -import org.apache.cassandra.service.StorageService;
  import org.junit.Test;
  
- import org.apache.cassandra.config.DatabaseDescriptor;
 +import org.apache.cassandra.exceptions.ConfigurationException;
 +import org.apache.cassandra.service.StorageService;
  import org.apache.cassandra.utils.FBUtilities;
  
  import static org.junit.Assert.assertEquals;


[2/3] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2

Posted by sl...@apache.org.
Merge branch 'cassandra-2.1' into cassandra-2.2

Conflicts:
	test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java


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

Branch: refs/heads/trunk
Commit: d97fc9b77f9ea332f9a908ee6c47326334a1211d
Parents: 5e930e4 4fcd7d4
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Thu Jul 16 15:26:17 2015 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Thu Jul 16 15:26:17 2015 +0200

----------------------------------------------------------------------
 .../locator/DynamicEndpointSnitchLongTest.java  | 104 +++++++++++++++++++
 .../locator/DynamicEndpointSnitchTest.java      |  64 ------------
 2 files changed, 104 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/d97fc9b7/test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java
----------------------------------------------------------------------
diff --cc test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java
index 0000000,1c628fa..841f73e
mode 000000,100644..100644
--- a/test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java
+++ b/test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java
@@@ -1,0 -1,104 +1,104 @@@
+ /*
+ * 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.locator;
+ 
+ import java.io.IOException;
+ import java.net.InetAddress;
+ import java.util.*;
+ 
+ import org.apache.cassandra.config.DatabaseDescriptor;
+ import org.apache.cassandra.exceptions.ConfigurationException;
+ import org.apache.cassandra.service.StorageService;
+ import org.junit.Test;
+ 
+ import org.apache.cassandra.utils.FBUtilities;
+ 
+ import static org.junit.Assert.assertEquals;
+ 
+ public class DynamicEndpointSnitchLongTest
+ {
+     @Test
+     public void testConcurrency() throws InterruptedException, IOException, ConfigurationException
+     {
+         // The goal of this test is to check for CASSANDRA-8448/CASSANDRA-9519
+         double badness = DatabaseDescriptor.getDynamicBadnessThreshold();
+         DatabaseDescriptor.setDynamicBadnessThreshold(0.0);
+ 
+         try
+         {
+             final int ITERATIONS = 10000;
+ 
+             // do this because SS needs to be initialized before DES can work properly.
 -            StorageService.instance.initClient(0);
++            StorageService.instance.unsafeInitialize();
+             SimpleSnitch ss = new SimpleSnitch();
+             DynamicEndpointSnitch dsnitch = new DynamicEndpointSnitch(ss, String.valueOf(ss.hashCode()));
+             InetAddress self = FBUtilities.getBroadcastAddress();
+ 
+             List<InetAddress> hosts = new ArrayList<>();
+             // We want a big list of hosts so  sorting takes time, making it much more likely to reproduce the
+             // problem we're looking for.
+             for (int i = 0; i < 100; i++)
+                 for (int j = 0; j < 256; j++)
+                     hosts.add(InetAddress.getByAddress(new byte[]{127, 0, (byte)i, (byte)j}));
+ 
+             ScoreUpdater updater = new ScoreUpdater(dsnitch, hosts);
+             updater.start();
+ 
+             List<InetAddress> result = null;
+             for (int i = 0; i < ITERATIONS; i++)
+                 result = dsnitch.getSortedListByProximity(self, hosts);
+ 
+             updater.stopped = true;
+             updater.join();
+         }
+         finally
+         {
+             DatabaseDescriptor.setDynamicBadnessThreshold(badness);
+         }
+     }
+ 
+     public static class ScoreUpdater extends Thread
+     {
+         private static final int SCORE_RANGE = 100;
+ 
+         public volatile boolean stopped;
+ 
+         private final DynamicEndpointSnitch dsnitch;
+         private final List<InetAddress> hosts;
+         private final Random random = new Random();
+ 
+         public ScoreUpdater(DynamicEndpointSnitch dsnitch, List<InetAddress> hosts)
+         {
+             this.dsnitch = dsnitch;
+             this.hosts = hosts;
+         }
+ 
+         public void run()
+         {
+             while (!stopped)
+             {
+                 InetAddress host = hosts.get(random.nextInt(hosts.size()));
+                 int score = random.nextInt(SCORE_RANGE);
+                 dsnitch.receiveTiming(host, score);
+             }
+         }
+     }
+ }
+ 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d97fc9b7/test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java
----------------------------------------------------------------------