Repository: cassandra-dtest Updated Branches: refs/heads/master b8842b979 -> 1a0e26603
update dtests to support netty-based internode messaging/streaming patch by jasobrown, reviewed by Marcus Eriksson for CASSANDRA-13635 Project: http://git-wip-us.apache.org/repos/asf/cassandra-dtest/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra-dtest/commit/1a0e2660 Tree: http://git-wip-us.apache.org/repos/asf/cassandra-dtest/tree/1a0e2660 Diff: http://git-wip-us.apache.org/repos/asf/cassandra-dtest/diff/1a0e2660 Branch: refs/heads/master Commit: 1a0e266038e75930c69842e338c6a6ee196f721c Parents: b8842b9 Author: Jason Brown <jasedbr...@gmail.com> Authored: Fri Jun 16 05:03:36 2017 -0700 Committer: Jason Brown <jasedbr...@gmail.com> Committed: Tue Aug 22 13:56:41 2017 -0700 ---------------------------------------------------------------------- bootstrap_test.py | 11 ++++++++--- byteman/4.0/decommission_failure_inject.btm | 17 +++++++++++++++++ .../4.0/inject_failure_streaming_to_node2.btm | 17 +++++++++++++++++ byteman/4.0/stream_failure.btm | 17 +++++++++++++++++ byteman/decommission_failure_inject.btm | 17 ----------------- byteman/inject_failure_streaming_to_node2.btm | 17 ----------------- byteman/pre4.0/decommission_failure_inject.btm | 17 +++++++++++++++++ .../pre4.0/inject_failure_streaming_to_node2.btm | 17 +++++++++++++++++ byteman/pre4.0/stream_failure.btm | 17 +++++++++++++++++ byteman/stream_failure.btm | 17 ----------------- native_transport_ssl_test.py | 2 +- nodetool_test.py | 8 +++++--- rebuild_test.py | 5 ++++- replace_address_test.py | 10 +++++++--- secondary_indexes_test.py | 13 +++++++++++-- sslnodetonode_test.py | 19 +++++++++---------- topology_test.py | 5 ++++- 17 files changed, 151 insertions(+), 75 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra-dtest/blob/1a0e2660/bootstrap_test.py ---------------------------------------------------------------------- diff --git a/bootstrap_test.py b/bootstrap_test.py index 1d149e6..54c49c1 100644 --- a/bootstrap_test.py +++ b/bootstrap_test.py @@ -148,8 +148,10 @@ class TestBootstrap(BaseBootstrapTest): 2*streaming_keep_alive_period_in_secs to receive a single sstable """ cluster = self.cluster - cluster.set_configuration_options(values={'streaming_socket_timeout_in_ms': 1000, - 'streaming_keep_alive_period_in_secs': 2}) + yaml_opts = {'streaming_keep_alive_period_in_secs': 2} + if cluster.version() < '4.0': + yamp_opts['streaming_socket_timeout_in_ms'] = 1000 + cluster.set_configuration_options(values=yaml_opts) # Create a single node cluster cluster.populate(1) @@ -306,7 +308,10 @@ class TestBootstrap(BaseBootstrapTest): cluster.start(wait_other_notice=True) # kill stream to node3 in the middle of streaming to let it fail - node1.byteman_submit(['./byteman/stream_failure.btm']) + if cluster.version() < '4.0': + node1.byteman_submit(['./byteman/pre4.0/stream_failure.btm']) + else: + node1.byteman_submit(['./byteman/4.0/stream_failure.btm']) node1.stress(['write', 'n=1K', 'no-warmup', 'cl=TWO', '-schema', 'replication(factor=2)', '-rate', 'threads=50']) cluster.flush() http://git-wip-us.apache.org/repos/asf/cassandra-dtest/blob/1a0e2660/byteman/4.0/decommission_failure_inject.btm ---------------------------------------------------------------------- diff --git a/byteman/4.0/decommission_failure_inject.btm b/byteman/4.0/decommission_failure_inject.btm new file mode 100644 index 0000000..a6418fc --- /dev/null +++ b/byteman/4.0/decommission_failure_inject.btm @@ -0,0 +1,17 @@ +# +# Inject decommission failure to fail streaming from 127.0.0.1 +# +# Before start streaming files in `StreamSession#onInitializationComplete()` method, +# interrupt streaming by throwing RuntimeException. +# +RULE inject decommission failure +CLASS org.apache.cassandra.streaming.StreamSession +METHOD prepareSynAck +AT INVOKE startStreamingFiles +BIND peer = $0.peer +# set flag to only run this rule once. +IF peer.equals(InetAddress.getByName("127.0.0.1")) AND NOT flagged("done") +DO + flag("done"); + throw new java.lang.RuntimeException("Triggering network failure") +ENDRULE \ No newline at end of file http://git-wip-us.apache.org/repos/asf/cassandra-dtest/blob/1a0e2660/byteman/4.0/inject_failure_streaming_to_node2.btm ---------------------------------------------------------------------- diff --git a/byteman/4.0/inject_failure_streaming_to_node2.btm b/byteman/4.0/inject_failure_streaming_to_node2.btm new file mode 100644 index 0000000..761950f --- /dev/null +++ b/byteman/4.0/inject_failure_streaming_to_node2.btm @@ -0,0 +1,17 @@ +# +# Inject streaming failure +# +# Before start streaming files in `StreamSession#prepare()` method, +# interrupt streaming by throwing RuntimeException. +# +RULE inject stream failure +CLASS org.apache.cassandra.streaming.StreamSession +METHOD startStreamingFiles +AT ENTRY +BIND peer = $0.peer +# set flag to only run this rule once. +IF peer.equals(InetAddress.getByName("127.0.0.2")) AND NOT flagged("done") +DO + flag("done"); + throw new java.lang.RuntimeException("Triggering network failure") +ENDRULE http://git-wip-us.apache.org/repos/asf/cassandra-dtest/blob/1a0e2660/byteman/4.0/stream_failure.btm ---------------------------------------------------------------------- diff --git a/byteman/4.0/stream_failure.btm b/byteman/4.0/stream_failure.btm new file mode 100644 index 0000000..bf919dd --- /dev/null +++ b/byteman/4.0/stream_failure.btm @@ -0,0 +1,17 @@ +# +# Inject streaming failure +# +# Before start streaming files in `StreamSession#prepare()` method, +# interrupt streaming by throwing RuntimeException. +# +RULE inject stream failure +CLASS org.apache.cassandra.streaming.StreamSession +METHOD prepareAck +AT INVOKE startStreamingFiles +BIND peer = $0.peer +# set flag to only run this rule once. +IF NOT flagged("done") +DO + flag("done"); + throw new java.lang.RuntimeException("Triggering network failure") +ENDRULE http://git-wip-us.apache.org/repos/asf/cassandra-dtest/blob/1a0e2660/byteman/decommission_failure_inject.btm ---------------------------------------------------------------------- diff --git a/byteman/decommission_failure_inject.btm b/byteman/decommission_failure_inject.btm deleted file mode 100644 index 32ef6a2..0000000 --- a/byteman/decommission_failure_inject.btm +++ /dev/null @@ -1,17 +0,0 @@ -# -# Inject decommission failure to fail streaming from 127.0.0.1 -# -# Before start streaming files in `StreamSession#onInitializationComplete()` method, -# interrupt streaming by throwing RuntimeException. -# -RULE inject decommission failure -CLASS org.apache.cassandra.streaming.StreamSession -METHOD onInitializationComplete -AT INVOKE startStreamingFiles -BIND peer = $0.peer -# set flag to only run this rule once. -IF peer.equals(InetAddress.getByName("127.0.0.1")) AND NOT flagged("done") -DO - flag("done"); - throw new java.lang.RuntimeException("Triggering network failure") -ENDRULE \ No newline at end of file http://git-wip-us.apache.org/repos/asf/cassandra-dtest/blob/1a0e2660/byteman/inject_failure_streaming_to_node2.btm ---------------------------------------------------------------------- diff --git a/byteman/inject_failure_streaming_to_node2.btm b/byteman/inject_failure_streaming_to_node2.btm deleted file mode 100644 index 04f996c..0000000 --- a/byteman/inject_failure_streaming_to_node2.btm +++ /dev/null @@ -1,17 +0,0 @@ -# -# Inject streaming failure -# -# Before start streaming files in `StreamSession#prepare()` method, -# interrupt streaming by throwing RuntimeException. -# -RULE inject stream failure -CLASS org.apache.cassandra.streaming.StreamSession -METHOD prepare -AT INVOKE startStreamingFiles -BIND peer = $0.peer -# set flag to only run this rule once. -IF peer.equals(InetAddress.getByName("127.0.0.2")) AND NOT flagged("done") -DO - flag("done"); - throw new java.lang.RuntimeException("Triggering network failure") -ENDRULE http://git-wip-us.apache.org/repos/asf/cassandra-dtest/blob/1a0e2660/byteman/pre4.0/decommission_failure_inject.btm ---------------------------------------------------------------------- diff --git a/byteman/pre4.0/decommission_failure_inject.btm b/byteman/pre4.0/decommission_failure_inject.btm new file mode 100644 index 0000000..32ef6a2 --- /dev/null +++ b/byteman/pre4.0/decommission_failure_inject.btm @@ -0,0 +1,17 @@ +# +# Inject decommission failure to fail streaming from 127.0.0.1 +# +# Before start streaming files in `StreamSession#onInitializationComplete()` method, +# interrupt streaming by throwing RuntimeException. +# +RULE inject decommission failure +CLASS org.apache.cassandra.streaming.StreamSession +METHOD onInitializationComplete +AT INVOKE startStreamingFiles +BIND peer = $0.peer +# set flag to only run this rule once. +IF peer.equals(InetAddress.getByName("127.0.0.1")) AND NOT flagged("done") +DO + flag("done"); + throw new java.lang.RuntimeException("Triggering network failure") +ENDRULE \ No newline at end of file http://git-wip-us.apache.org/repos/asf/cassandra-dtest/blob/1a0e2660/byteman/pre4.0/inject_failure_streaming_to_node2.btm ---------------------------------------------------------------------- diff --git a/byteman/pre4.0/inject_failure_streaming_to_node2.btm b/byteman/pre4.0/inject_failure_streaming_to_node2.btm new file mode 100644 index 0000000..04f996c --- /dev/null +++ b/byteman/pre4.0/inject_failure_streaming_to_node2.btm @@ -0,0 +1,17 @@ +# +# Inject streaming failure +# +# Before start streaming files in `StreamSession#prepare()` method, +# interrupt streaming by throwing RuntimeException. +# +RULE inject stream failure +CLASS org.apache.cassandra.streaming.StreamSession +METHOD prepare +AT INVOKE startStreamingFiles +BIND peer = $0.peer +# set flag to only run this rule once. +IF peer.equals(InetAddress.getByName("127.0.0.2")) AND NOT flagged("done") +DO + flag("done"); + throw new java.lang.RuntimeException("Triggering network failure") +ENDRULE http://git-wip-us.apache.org/repos/asf/cassandra-dtest/blob/1a0e2660/byteman/pre4.0/stream_failure.btm ---------------------------------------------------------------------- diff --git a/byteman/pre4.0/stream_failure.btm b/byteman/pre4.0/stream_failure.btm new file mode 100644 index 0000000..23c63cd --- /dev/null +++ b/byteman/pre4.0/stream_failure.btm @@ -0,0 +1,17 @@ +# +# Inject streaming failure +# +# Before start streaming files in `StreamSession#prepare()` method, +# interrupt streaming by throwing RuntimeException. +# +RULE inject stream failure +CLASS org.apache.cassandra.streaming.StreamSession +METHOD prepare +AT INVOKE startStreamingFiles +BIND peer = $0.peer +# set flag to only run this rule once. +IF NOT flagged("done") +DO + flag("done"); + throw new java.lang.RuntimeException("Triggering network failure") +ENDRULE http://git-wip-us.apache.org/repos/asf/cassandra-dtest/blob/1a0e2660/byteman/stream_failure.btm ---------------------------------------------------------------------- diff --git a/byteman/stream_failure.btm b/byteman/stream_failure.btm deleted file mode 100644 index 23c63cd..0000000 --- a/byteman/stream_failure.btm +++ /dev/null @@ -1,17 +0,0 @@ -# -# Inject streaming failure -# -# Before start streaming files in `StreamSession#prepare()` method, -# interrupt streaming by throwing RuntimeException. -# -RULE inject stream failure -CLASS org.apache.cassandra.streaming.StreamSession -METHOD prepare -AT INVOKE startStreamingFiles -BIND peer = $0.peer -# set flag to only run this rule once. -IF NOT flagged("done") -DO - flag("done"); - throw new java.lang.RuntimeException("Triggering network failure") -ENDRULE http://git-wip-us.apache.org/repos/asf/cassandra-dtest/blob/1a0e2660/native_transport_ssl_test.py ---------------------------------------------------------------------- diff --git a/native_transport_ssl_test.py b/native_transport_ssl_test.py index 67dcfaf..4716c80 100644 --- a/native_transport_ssl_test.py +++ b/native_transport_ssl_test.py @@ -30,7 +30,7 @@ class NativeTransportSSL(Tester): except NoHostAvailable: pass - self.assertGreater(len(node1.grep_log("^io.netty.handler.ssl.NotSslRecordException.*")), 0, "Missing SSL handshake exception while connecting with non-SSL enabled client") + self.assertGreater(len(node1.grep_log("io.netty.handler.ssl.NotSslRecordException.*")), 0, "Missing SSL handshake exception while connecting with non-SSL enabled client") # enabled ssl on the client and try again (this should work) session = self.patient_cql_connection(node1, ssl_opts={'ca_certs': os.path.join(self.test_path, 'ccm_node.cer')}) http://git-wip-us.apache.org/repos/asf/cassandra-dtest/blob/1a0e2660/nodetool_test.py ---------------------------------------------------------------------- diff --git a/nodetool_test.py b/nodetool_test.py index d7ce89a..8d5c776 100644 --- a/nodetool_test.py +++ b/nodetool_test.py @@ -71,9 +71,11 @@ class TestNodetool(Tester): cluster.populate([1]).start() node = cluster.nodelist()[0] - types = ('read', 'range', 'write', 'counterwrite', 'cascontention', - 'truncate', 'streamingsocket', 'misc') - + types = ['read', 'range', 'write', 'counterwrite', 'cascontention', + 'truncate', 'misc'] + if cluster.version() < '4.0': + types.append('streamingsocket') + # read all of the timeouts, make sure we get a sane response for timeout_type in types: out, err, _ = node.nodetool('gettimeout {}'.format(timeout_type)) http://git-wip-us.apache.org/repos/asf/cassandra-dtest/blob/1a0e2660/rebuild_test.py ---------------------------------------------------------------------- diff --git a/rebuild_test.py b/rebuild_test.py index 13d96ba..273a749 100644 --- a/rebuild_test.py +++ b/rebuild_test.py @@ -194,7 +194,10 @@ class TestRebuild(Tester): session.execute("ALTER KEYSPACE system_auth WITH REPLICATION = {'class':'NetworkTopologyStrategy', 'dc1':1, 'dc2':1};") # Path to byteman script which makes the streaming to node2 throw an exception, making rebuild fail - script = ['./byteman/inject_failure_streaming_to_node2.btm'] + if cluster.version() < '4.0': + script = ['./byteman/pre4.0/inject_failure_streaming_to_node2.btm'] + else: + script = ['./byteman/4.0/inject_failure_streaming_to_node2.btm'] node3.byteman_submit(script) # First rebuild must fail and data must be incomplete http://git-wip-us.apache.org/repos/asf/cassandra-dtest/blob/1a0e2660/replace_address_test.py ---------------------------------------------------------------------- diff --git a/replace_address_test.py b/replace_address_test.py index 1da911f..f80af17 100644 --- a/replace_address_test.py +++ b/replace_address_test.py @@ -445,10 +445,14 @@ class TestReplaceAddress(BaseReplaceAddressTest): self._stop_node_to_replace() debug("Submitting byteman script to make stream fail") - self.query_node.byteman_submit(['./byteman/stream_failure.btm']) - self._do_replace(jvm_option='replace_address_first_boot', - opts={'streaming_socket_timeout_in_ms': 1000}) + if self.cluster.version() < '4.0': + self.query_node.byteman_submit(['./byteman/pre4.0/stream_failure.btm']) + self._do_replace(jvm_option='replace_address_first_boot', + opts={'streaming_socket_timeout_in_ms': 1000}) + else: + self.query_node.byteman_submit(['./byteman/4.0/stream_failure.btm']) + self._do_replace(jvm_option='replace_address_first_boot') # Make sure bootstrap did not complete successfully assert_bootstrap_state(self, self.replacement_node, 'IN_PROGRESS') http://git-wip-us.apache.org/repos/asf/cassandra-dtest/blob/1a0e2660/secondary_indexes_test.py ---------------------------------------------------------------------- diff --git a/secondary_indexes_test.py b/secondary_indexes_test.py index 11cd3af..a7580c2 100644 --- a/secondary_indexes_test.py +++ b/secondary_indexes_test.py @@ -1190,10 +1190,19 @@ class TestPreJoinCallback(Tester): node1.byteman_port = '8100' node1.import_config_files() node1.start(wait_for_binary_proto=True) - node1.byteman_submit(['./byteman/inject_failure_streaming_to_node2.btm']) + + if cluster.version() < '4.0': + node1.byteman_submit(['./byteman/pre4.0/inject_failure_streaming_to_node2.btm']) + else: + node1.byteman_submit(['./byteman/4.0/inject_failure_streaming_to_node2.btm']) node2 = new_node(cluster) - node2.set_configuration_options(values={'initial_token': token, 'streaming_socket_timeout_in_ms': 1000}) + + yaml_opts = {'initial_token': token} + if cluster.version() < '4.0': + yaml_opts['streaming_socket_timeout_in_ms'] = 1000 + + node2.set_configuration_options(values=yaml_opts) node2.start(wait_other_notice=False, wait_for_binary_proto=True) assert_bootstrap_state(self, node2, 'IN_PROGRESS') http://git-wip-us.apache.org/repos/asf/cassandra-dtest/blob/1a0e2660/sslnodetonode_test.py ---------------------------------------------------------------------- diff --git a/sslnodetonode_test.py b/sslnodetonode_test.py index a11a3f4..a675985 100644 --- a/sslnodetonode_test.py +++ b/sslnodetonode_test.py @@ -7,11 +7,10 @@ from dtest import Tester from tools import sslkeygen from tools.decorators import since -_LOG_ERR_SIG = "^javax.net.ssl.SSLHandshakeException: sun.security.validator.ValidatorException: Certificate signature validation failed$" -_LOG_ERR_IP = "^javax.net.ssl.SSLHandshakeException: java.security.cert.CertificateException: No subject alternative names matching IP address [0-9.]+ found$" -_LOG_ERR_HOST = "^javax.net.ssl.SSLHandshakeException: java.security.cert.CertificateException: No name matching \S+ found$" -_LOG_ERR_CERT = "^javax.net.ssl.SSLHandshakeException: Received fatal alert: certificate_unknown$" - +# as the error message logged will be different per netty ssl implementation (jdk vs openssl (libre vs boring vs ...)), +# the best we can do is just look for a SSLHandshakeException +_LOG_ERR_HANDSHAKE = "javax.net.ssl.SSLHandshakeException" +_LOG_ERR_GENERAL = "javax.net.ssl.SSLException" @since('3.6') class TestNodeToNodeSSLEncryption(Tester): @@ -60,10 +59,10 @@ class TestNodeToNodeSSLEncryption(Tester): self.allow_log_errors = True self.cluster.start(no_wait=True) - found = self._grep_msg(self.node1, _LOG_ERR_IP, _LOG_ERR_HOST) + found = self._grep_msg(self.node1, _LOG_ERR_HANDSHAKE, _LOG_ERR_GENERAL) self.assertTrue(found) - found = self._grep_msg(self.node2, _LOG_ERR_IP, _LOG_ERR_HOST) + found = self._grep_msg(self.node2, _LOG_ERR_HANDSHAKE, _LOG_ERR_GENERAL) self.assertTrue(found) self.cluster.stop() @@ -81,10 +80,10 @@ class TestNodeToNodeSSLEncryption(Tester): self.cluster.start(no_wait=True) time.sleep(2) - found = self._grep_msg(self.node1, _LOG_ERR_CERT) + found = self._grep_msg(self.node1, _LOG_ERR_HANDSHAKE, _LOG_ERR_GENERAL) self.assertTrue(found) - found = self._grep_msg(self.node2, _LOG_ERR_CERT) + found = self._grep_msg(self.node2, _LOG_ERR_HANDSHAKE, _LOG_ERR_GENERAL) self.assertTrue(found) self.cluster.stop() @@ -114,7 +113,7 @@ class TestNodeToNodeSSLEncryption(Tester): self.allow_log_errors = True self.cluster.start(no_wait=True) - found = self._grep_msg(self.node1, _LOG_ERR_SIG) + found = self._grep_msg(self.node1, _LOG_ERR_HANDSHAKE) self.cluster.stop() self.assertTrue(found) http://git-wip-us.apache.org/repos/asf/cassandra-dtest/blob/1a0e2660/topology_test.py ---------------------------------------------------------------------- diff --git a/topology_test.py b/topology_test.py index 45c1c73..797eca3 100644 --- a/topology_test.py +++ b/topology_test.py @@ -212,7 +212,10 @@ class TestTopology(Tester): # Execute first rebuild, should fail with self.assertRaises(ToolError): - script = ['./byteman/decommission_failure_inject.btm'] + if cluster.version() >= '4.0': + script = ['./byteman/4.0/decommission_failure_inject.btm'] + else: + script = ['./byteman/pre4.0/decommission_failure_inject.btm'] node2.byteman_submit(script) node2.nodetool('decommission') --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org