http://git-wip-us.apache.org/repos/asf/cassandra-dtest/blob/49b2dda4/jmx_test.py ---------------------------------------------------------------------- diff --git a/jmx_test.py b/jmx_test.py index 16c1ece..04c05d8 100644 --- a/jmx_test.py +++ b/jmx_test.py @@ -1,24 +1,36 @@ import os import time +import pytest +import parse +import re +import logging import ccmlib.common -import parse from ccmlib.node import ToolError -from dtest import Tester, debug -from tools.decorators import since +from dtest import Tester from tools.jmxutils import (JolokiaAgent, enable_jmx_ssl, make_mbean, remove_perf_disable_shared_mem) from tools.misc import generate_ssl_stores +since = pytest.mark.since +logger = logging.getLogger(__name__) + class TestJMX(Tester): - def netstats_test(self): + + @pytest.fixture(autouse=True) + def fixture_add_additional_log_patterns(self, fixture_dtest_setup): + fixture_dtest_setup.ignore_log_patterns = ( + r'Failed to properly handshake with peer.* Closing the channel' + ) + + def test_netstats(self): """ Check functioning of nodetool netstats, especially with restarts. @jira_ticket CASSANDRA-8122, CASSANDRA-6577 """ - + # cluster = self.cluster cluster.populate(3).start(wait_for_binary_proto=True) node1, node2, node3 = cluster.nodelist() @@ -27,7 +39,7 @@ class TestJMX(Tester): node1.flush() node1.stop(gently=False) - with self.assertRaisesRegexp(ToolError, "ConnectException: 'Connection refused( \(Connection refused\))?'."): + with pytest.raises(ToolError, message="ConnectException: 'Connection refused( \(Connection refused\))?'."): node1.nodetool('netstats') # don't wait; we're testing for when nodetool is called on a node mid-startup @@ -42,17 +54,16 @@ class TestJMX(Tester): try: node1.nodetool('netstats') except Exception as e: - self.assertNotIn('java.lang.reflect.UndeclaredThrowableException', str(e), - 'Netstats failed with UndeclaredThrowableException (CASSANDRA-8122)') + assert 'java.lang.reflect.UndeclaredThrowableException' not in str(e), \ + 'Netstats failed with UndeclaredThrowableException (CASSANDRA-8122)' if not isinstance(e, ToolError): raise else: - self.assertRegexpMatches(str(e), - "ConnectException: 'Connection refused( \(Connection refused\))?'.") + assert re.search("ConnectException: 'Connection refused( \(Connection refused\))?'.", repr(e)) - self.assertTrue(running, msg='node1 never started') + assert running, 'node1 never started' - def table_metric_mbeans_test(self): + def test_table_metric_mbeans(self): """ Test some basic table metric mbeans with simple writes. """ @@ -66,7 +77,7 @@ class TestJMX(Tester): node1.stress(['write', 'n=10K', 'no-warmup', '-schema', 'replication(factor=3)']) typeName = "ColumnFamily" if version <= '2.2.X' else 'Table' - debug('Version {} typeName {}'.format(version, typeName)) + logger.debug('Version {} typeName {}'.format(version, typeName)) # TODO the keyspace and table name are capitalized in 2.0 memtable_size = make_mbean('metrics', type=typeName, keyspace='keyspace1', scope='standard1', @@ -78,21 +89,21 @@ class TestJMX(Tester): with JolokiaAgent(node1) as jmx: mem_size = jmx.read_attribute(memtable_size, "Value") - self.assertGreater(int(mem_size), 10000) + assert int(mem_size) > 10000 on_disk_size = jmx.read_attribute(disk_size, "Count") - self.assertEquals(int(on_disk_size), 0) + assert int(on_disk_size) == 0 node1.flush() on_disk_size = jmx.read_attribute(disk_size, "Count") - self.assertGreater(int(on_disk_size), 10000) + assert int(on_disk_size) > 10000 sstables = jmx.read_attribute(sstable_count, "Value") - self.assertGreaterEqual(int(sstables), 1) + assert int(sstables) >= 1 @since('3.0') - def mv_metric_mbeans_release_test(self): + def test_mv_metric_mbeans_release(self): """ Test that the right mbeans are created and released when creating mvs """ @@ -133,33 +144,33 @@ class TestJMX(Tester): "but wasn't!" with JolokiaAgent(node) as jmx: - self.assertIsNotNone(jmx.read_attribute(table_memtable_size, "Value"), - missing_metric_message.format("AllMemtablesHeapSize", "testtable")) - self.assertIsNotNone(jmx.read_attribute(table_view_read_time, "Count"), - missing_metric_message.format("ViewReadTime", "testtable")) - self.assertIsNotNone(jmx.read_attribute(table_view_lock_time, "Count"), - missing_metric_message.format("ViewLockAcquireTime", "testtable")) - self.assertIsNotNone(jmx.read_attribute(mv_memtable_size, "Value"), - missing_metric_message.format("AllMemtablesHeapSize", "testmv")) - self.assertRaisesRegexp(Exception, ".*InstanceNotFoundException.*", jmx.read_attribute, - mbean=mv_view_read_time, attribute="Count", verbose=False) - self.assertRaisesRegexp(Exception, ".*InstanceNotFoundException.*", jmx.read_attribute, - mbean=mv_view_lock_time, attribute="Count", verbose=False) + assert jmx.read_attribute(table_memtable_size, "Value") is not None, \ + missing_metric_message.format("AllMemtablesHeapSize", "testtable") + assert jmx.read_attribute(table_view_read_time, "Count") is not None, \ + missing_metric_message.format("ViewReadTime", "testtable") + assert jmx.read_attribute(table_view_lock_time, "Count") is not None, \ + missing_metric_message.format("ViewLockAcquireTime", "testtable") + assert jmx.read_attribute(mv_memtable_size, "Value") is not None, \ + missing_metric_message.format("AllMemtablesHeapSize", "testmv") + with pytest.raises(Exception, match=".*InstanceNotFoundException.*"): + jmx.read_attribute(mbean=mv_view_read_time, attribute="Count", verbose=False) + with pytest.raises(Exception, match=".*InstanceNotFoundException.*"): + jmx.read_attribute(mbean=mv_view_lock_time, attribute="Count", verbose=False) node.run_cqlsh(cmds="DROP KEYSPACE mvtest;") with JolokiaAgent(node) as jmx: - self.assertRaisesRegexp(Exception, ".*InstanceNotFoundException.*", jmx.read_attribute, - mbean=table_memtable_size, attribute="Value", verbose=False) - self.assertRaisesRegexp(Exception, ".*InstanceNotFoundException.*", jmx.read_attribute, - mbean=table_view_lock_time, attribute="Count", verbose=False) - self.assertRaisesRegexp(Exception, ".*InstanceNotFoundException.*", jmx.read_attribute, - mbean=table_view_read_time, attribute="Count", verbose=False) - self.assertRaisesRegexp(Exception, ".*InstanceNotFoundException.*", jmx.read_attribute, - mbean=mv_memtable_size, attribute="Value", verbose=False) - self.assertRaisesRegexp(Exception, ".*InstanceNotFoundException.*", jmx.read_attribute, - mbean=mv_view_lock_time, attribute="Count", verbose=False) - self.assertRaisesRegexp(Exception, ".*InstanceNotFoundException.*", jmx.read_attribute, - mbean=mv_view_read_time, attribute="Count", verbose=False) + with pytest.raises(Exception, match=".*InstanceNotFoundException.*"): + jmx.read_attribute(mbean=table_memtable_size, attribute="Value", verbose=False) + with pytest.raises(Exception, match=".*InstanceNotFoundException.*"): + jmx.read_attribute(mbean=table_view_lock_time, attribute="Count", verbose=False) + with pytest.raises(Exception, match=".*InstanceNotFoundException.*"): + jmx.read_attribute(mbean=table_view_read_time, attribute="Count", verbose=False) + with pytest.raises(Exception, match=".*InstanceNotFoundException.*"): + jmx.read_attribute(mbean=mv_memtable_size, attribute="Value", verbose=False) + with pytest.raises(Exception, match=".*InstanceNotFoundException.*"): + jmx.read_attribute(mbean=mv_view_lock_time, attribute="Count", verbose=False) + with pytest.raises(Exception, match=".*InstanceNotFoundException.*"): + jmx.read_attribute(mbean=mv_view_read_time, attribute="Count", verbose=False) def test_compactionstats(self): """ @@ -204,14 +215,14 @@ class TestJMX(Tester): progress = int(parse.search(var, progress_string).named['progress']) updated_progress = int(parse.search(var, updated_progress_string).named['progress']) - debug(progress_string) - debug(updated_progress_string) + logger.debug(progress_string) + logger.debug(updated_progress_string) # We want to make sure that the progress is increasing, # and that values other than zero are displayed. - self.assertGreater(updated_progress, progress) - self.assertGreaterEqual(progress, 0) - self.assertGreater(updated_progress, 0) + assert updated_progress > progress + assert progress >= 0 + assert updated_progress > 0 # Block until the major compaction is complete # Otherwise nodetool will throw an exception @@ -219,41 +230,42 @@ class TestJMX(Tester): # and never ends. start = time.time() max_query_timeout = 600 - debug("Waiting for compaction to finish:") + logger.debug("Waiting for compaction to finish:") while (len(jmx.read_attribute(compaction_manager, 'CompactionSummary')) > 0) and ( time.time() - start < max_query_timeout): - debug(jmx.read_attribute(compaction_manager, 'CompactionSummary')) + logger.debug(jmx.read_attribute(compaction_manager, 'CompactionSummary')) time.sleep(2) @since('2.2') - def phi_test(self): + def test_phi(self): """ Check functioning of nodetool failuredetector. @jira_ticket CASSANDRA-9526 """ - cluster = self.cluster cluster.populate(3).start(wait_for_binary_proto=True) node1, node2, node3 = cluster.nodelist() - phivalues = node1.nodetool("failuredetector").stdout.splitlines() - endpoint1Values = phivalues[1].split() - endpoint2Values = phivalues[2].split() + stdout = node1.nodetool("failuredetector").stdout + phivalues = stdout.splitlines() + endpoint1values = phivalues[1].split() + endpoint2values = phivalues[2].split() - endpoint1 = endpoint1Values[0][1:-1] - endpoint2 = endpoint2Values[0][1:-1] + endpoint1 = endpoint1values[0][1:-1] + endpoint2 = endpoint2values[0][1:-1] - self.assertItemsEqual([endpoint1, endpoint2], ['127.0.0.2', '127.0.0.3']) + assert '127.0.0.2' in [endpoint1, endpoint2] + assert '127.0.0.3' in [endpoint1, endpoint2] - endpoint1Phi = float(endpoint1Values[1]) - endpoint2Phi = float(endpoint2Values[1]) + endpoint1phi = float(endpoint1values[1]) + endpoint2phi = float(endpoint2values[1]) max_phi = 2.0 - self.assertGreater(endpoint1Phi, 0.0) - self.assertLess(endpoint1Phi, max_phi) + assert endpoint1phi > 0.0 + assert endpoint1phi < max_phi - self.assertGreater(endpoint2Phi, 0.0) - self.assertLess(endpoint2Phi, max_phi) + assert endpoint2phi > 0.0 + assert endpoint2phi < max_phi @since('4.0') def test_set_get_batchlog_replay_throttle(self): @@ -272,9 +284,9 @@ class TestJMX(Tester): with JolokiaAgent(node) as jmx: mbean = make_mbean('db', 'StorageService') jmx.write_attribute(mbean, 'BatchlogReplayThrottleInKB', 4096) - self.assertTrue(len(node.grep_log('Updating batchlog replay throttle to 4096 KB/s, 2048 KB/s per endpoint', - filename='debug.log')) > 0) - self.assertEqual(4096, jmx.read_attribute(mbean, 'BatchlogReplayThrottleInKB')) + assert len(node.grep_log('Updating batchlog replay throttle to 4096 KB/s, 2048 KB/s per endpoint', + filename='debug.log')) > 0 + assert 4096 == jmx.read_attribute(mbean, 'BatchlogReplayThrottleInKB') @since('3.9') @@ -283,12 +295,12 @@ class TestJMXSSL(Tester): truststore_password = 'cassandra' def truststore(self): - return os.path.join(self.test_path, 'truststore.jks') + return os.path.join(self.fixture_dtest_setup.test_path, 'truststore.jks') def keystore(self): - return os.path.join(self.test_path, 'keystore.jks') + return os.path.join(self.fixture_dtest_setup.test_path, 'keystore.jks') - def jmx_connection_test(self): + def test_jmx_connection(self): """ Check connecting with a JMX client (via nodetool) where SSL is enabled for JMX @jira_ticket CASSANDRA-12109 @@ -302,7 +314,7 @@ class TestJMXSSL(Tester): node.nodetool("info --ssl -Djavax.net.ssl.trustStore={ts} -Djavax.net.ssl.trustStorePassword={ts_pwd}" .format(ts=self.truststore(), ts_pwd=self.truststore_password)) - def require_client_auth_test(self): + def test_require_client_auth(self): """ Check connecting with a JMX client (via nodetool) where SSL is enabled and client certificate auth is also configured @@ -315,7 +327,7 @@ class TestJMXSSL(Tester): self.assert_insecure_connection_rejected(node) # specifying only the truststore containing the server cert should fail - with self.assertRaisesRegexp(ToolError, ".*SSLHandshakeException.*"): + with pytest.raises(ToolError, match=".*SSLHandshakeException.*"): node.nodetool("info --ssl -Djavax.net.ssl.trustStore={ts} -Djavax.net.ssl.trustStorePassword={ts_pwd}" .format(ts=self.truststore(), ts_pwd=self.truststore_password)) @@ -329,14 +341,14 @@ class TestJMXSSL(Tester): """ Attempts to connect to JMX (via nodetool) without any client side ssl parameters, expecting failure """ - with self.assertRaises(ToolError): + with pytest.raises(ToolError): node.nodetool("info") def _populateCluster(self, require_client_auth=False): cluster = self.cluster cluster.populate(1) - generate_ssl_stores(self.test_path) + generate_ssl_stores(self.fixture_dtest_setup.test_path) if require_client_auth: ts = self.truststore() ts_pwd = self.truststore_password
http://git-wip-us.apache.org/repos/asf/cassandra-dtest/blob/49b2dda4/json_test.py ---------------------------------------------------------------------- diff --git a/json_test.py b/json_test.py index 81590cb..998751a 100644 --- a/json_test.py +++ b/json_test.py @@ -4,13 +4,18 @@ import os import re import subprocess import sys +import pytest +import logging + from distutils.version import LooseVersion from ccmlib import common from ccmlib.common import is_win from dtest import Tester -from tools.decorators import since + +since = pytest.mark.since +logger = logging.getLogger(__name__) def build_doc_context(tester, test_name, prepare=True, connection=None, nodes=None): @@ -79,10 +84,10 @@ def build_doc_context(tester, test_name, prepare=True, connection=None, nodes=No args = [host, str(port)] sys.stdout.flush() p = subprocess.Popen([cli] + args, env=env, stdin=subprocess.PIPE, stderr=subprocess.PIPE, stdout=subprocess.PIPE) - p.stdin.write("USE {};".format(enabled_ks())) + p.stdin.write("USE {};".format(enabled_ks()).encode('utf-8')) for cmd in cmds.split(';'): - p.stdin.write(cmd + ';\n') - p.stdin.write("quit;\n") # may not be necesary, things could simplify a bit if removed + p.stdin.write((cmd + ';\n').encode('utf-8')) + p.stdin.write("quit;\n".encode('utf-8')) # may not be necesary, things could simplify a bit if removed return p.communicate() def cqlsh(cmds, supress_err=False): @@ -96,7 +101,7 @@ def build_doc_context(tester, test_name, prepare=True, connection=None, nodes=No # if output is empty string we want to just return None if output: - return output + return output.decode("utf-8") def cqlsh_print(cmds, supress_err=False): """ @@ -130,13 +135,13 @@ def build_doc_context(tester, test_name, prepare=True, connection=None, nodes=No if not err: raise RuntimeError("Expected cqlsh error but none occurred!") - return err + return err.decode("utf-8") def cqlsh_err_print(cmds): """ Run cqlsh commands expecting error output, and print error output. """ - print(cqlsh_err(cmds)) + print((cqlsh_err(cmds))) def cql(query): """ @@ -193,12 +198,12 @@ def run_func_docstring(tester, test_func, globs=None, verbose=False, compileflag @since('2.2') -class ToJsonSelectTests(Tester): +class TestToJsonSelect(Tester): """ Tests using toJson with a SELECT statement """ - def basic_data_types_test(self): + def test_basic_data_types(self): """ Create our schema: @@ -267,10 +272,10 @@ class ToJsonSelectTests(Tester): (1 rows) <BLANKLINE> """ - run_func_docstring(tester=self, test_func=self.basic_data_types_test) + run_func_docstring(tester=self, test_func=self.test_basic_data_types) # yes, it's probably weird to use json for counter changes - def counters_test(self): + def test_counters(self): """ Add a table with a few counters: @@ -299,9 +304,9 @@ class ToJsonSelectTests(Tester): (1 rows) <BLANKLINE> """ - run_func_docstring(tester=self, test_func=self.counters_test) + run_func_docstring(tester=self, test_func=self.test_counters) - def complex_data_types_test(self): + def test_complex_data_types(self): """ Build some user types and a schema that uses them: @@ -449,16 +454,16 @@ class ToJsonSelectTests(Tester): (1 rows) <BLANKLINE> """ - run_func_docstring(tester=self, test_func=self.complex_data_types_test) + run_func_docstring(tester=self, test_func=self.test_complex_data_types) @since('2.2') -class FromJsonUpdateTests(Tester): +class TestFromJsonUpdate(Tester): """ Tests using fromJson within UPDATE statements. """ - def basic_data_types_test(self): + def test_basic_data_types(self): """ Create a table with the primitive types: @@ -522,9 +527,9 @@ class FromJsonUpdateTests(Tester): (1 rows) <BLANKLINE> """ - run_func_docstring(tester=self, test_func=self.basic_data_types_test) + run_func_docstring(tester=self, test_func=self.test_basic_data_types) - def complex_data_types_test(self): + def test_complex_data_types(self): """" UDT and schema setup: @@ -676,9 +681,9 @@ class FromJsonUpdateTests(Tester): (1 rows) <BLANKLINE> """ - run_func_docstring(tester=self, test_func=self.complex_data_types_test) + run_func_docstring(tester=self, test_func=self.test_complex_data_types) - def collection_update_test(self): + def test_collection_update(self): """ Setup schema, add a row: @@ -750,16 +755,16 @@ class FromJsonUpdateTests(Tester): (1 rows) <BLANKLINE> """ - run_func_docstring(tester=self, test_func=self.collection_update_test) + run_func_docstring(tester=self, test_func=self.test_collection_update) @since('2.2') -class FromJsonSelectTests(Tester): +class TestFromJsonSelect(Tester): """ Tests using fromJson in conjunction with a SELECT statement """ - def selecting_pkey_as_json_test(self): + def test_selecting_pkey_as_json(self): """ Schema setup: @@ -793,9 +798,9 @@ class FromJsonSelectTests(Tester): (1 rows) <BLANKLINE> """ - run_func_docstring(tester=self, test_func=self.selecting_pkey_as_json_test) + run_func_docstring(tester=self, test_func=self.test_selecting_pkey_as_json) - def select_using_secondary_index_test(self): + def test_select_using_secondary_index(self): """ Schema setup and secondary index: @@ -832,16 +837,16 @@ class FromJsonSelectTests(Tester): (1 rows) <BLANKLINE> """ - run_func_docstring(tester=self, test_func=self.select_using_secondary_index_test) + run_func_docstring(tester=self, test_func=self.test_select_using_secondary_index) @since('2.2') -class FromJsonInsertTests(Tester): +class TestFromJsonInsert(Tester): """ Tests using fromJson within INSERT statements. """ - def basic_data_types_test(self): + def test_basic_data_types(self): """ Create a table with the primitive types: @@ -902,9 +907,9 @@ class FromJsonInsertTests(Tester): (1 rows) <BLANKLINE> """ - run_func_docstring(tester=self, test_func=self.basic_data_types_test) + run_func_docstring(tester=self, test_func=self.test_basic_data_types) - def complex_data_types_test(self): + def test_complex_data_types(self): """ Build some user types and a schema that uses them: @@ -1055,16 +1060,16 @@ class FromJsonInsertTests(Tester): (2 rows) <BLANKLINE> """ - run_func_docstring(tester=self, test_func=self.complex_data_types_test) + run_func_docstring(tester=self, test_func=self.test_complex_data_types) @since('2.2') -class FromJsonDeleteTests(Tester): +class TestFromJsonDelete(Tester): """ Tests using fromJson within DELETE statements. """ - def delete_using_pkey_json_test(self): + def test_delete_using_pkey_json(self): """ Schema setup: @@ -1119,16 +1124,16 @@ class FromJsonDeleteTests(Tester): <BLANKLINE> <BLANKLINE> """ - run_func_docstring(tester=self, test_func=self.delete_using_pkey_json_test) + run_func_docstring(tester=self, test_func=self.test_delete_using_pkey_json) @since('2.2') -class JsonFullRowInsertSelect(Tester): +class TestJsonFullRowInsertSelect(Tester): """ Tests for creating full rows from json documents, selecting full rows back as json documents, and related functionality. """ - def simple_schema_test(self): + def test_simple_schema(self): """ Create schema: @@ -1231,9 +1236,9 @@ class JsonFullRowInsertSelect(Tester): (2 rows) <BLANKLINE> """ - run_func_docstring(tester=self, test_func=self.simple_schema_test) + run_func_docstring(tester=self, test_func=self.test_simple_schema) - def pkey_requirement_test(self): + def test_pkey_requirement(self): """ Create schema: @@ -1263,9 +1268,9 @@ class JsonFullRowInsertSelect(Tester): <stdin>:2:InvalidRequest: Error from server: code=2200 [Invalid query] message="Invalid null value in condition for column key1" <BLANKLINE> """ - run_func_docstring(tester=self, test_func=self.pkey_requirement_test) + run_func_docstring(tester=self, test_func=self.test_pkey_requirement) - def null_value_test(self): + def test_null_value(self): """ Create schema: @@ -1306,9 +1311,9 @@ class JsonFullRowInsertSelect(Tester): (1 rows) <BLANKLINE> """ - run_func_docstring(tester=self, test_func=self.null_value_test) + run_func_docstring(tester=self, test_func=self.test_null_value) - def complex_schema_test(self): + def test_complex_schema(self): """ Create some udt's and schema: @@ -1507,4 +1512,4 @@ class JsonFullRowInsertSelect(Tester): <BLANKLINE> """ - run_func_docstring(tester=self, test_func=self.complex_schema_test) + run_func_docstring(tester=self, test_func=self.test_complex_schema) http://git-wip-us.apache.org/repos/asf/cassandra-dtest/blob/49b2dda4/json_tools_test.py ---------------------------------------------------------------------- diff --git a/json_tools_test.py b/json_tools_test.py index cdcb0f3..8065c16 100644 --- a/json_tools_test.py +++ b/json_tools_test.py @@ -1,28 +1,33 @@ import os import tempfile +import pytest +import logging -from dtest import Tester, debug, create_ks +from dtest import Tester, create_ks from tools.data import rows_to_list -from tools.decorators import since +from tools.assertions import assert_lists_equal_ignoring_order +since = pytest.mark.since +logger = logging.getLogger(__name__) -@since('0', '2.2.X') + +@since('0', max_version='2.2.X') class TestJson(Tester): - def json_tools_test(self): + def test_json_tools(self): - debug("Starting cluster...") + logger.debug("Starting cluster...") cluster = self.cluster cluster.set_batch_commitlog(enabled=True) cluster.populate(1).start() - debug("Version: " + cluster.version().vstring) + logger.debug("Version: " + cluster.version().vstring) - debug("Getting CQLSH...") + logger.debug("Getting CQLSH...") [node1] = cluster.nodelist() session = self.patient_cql_connection(node1) - debug("Inserting data...") + logger.debug("Inserting data...") create_ks(session, 'Test', 1) session.execute(""" @@ -35,20 +40,21 @@ class TestJson(Tester): ); """) - session.execute("INSERT INTO Test. users (user_name, password, gender, state, birth_year) VALUES('frodo', 'pass@', 'male', 'CA', 1985);") - session.execute("INSERT INTO Test. users (user_name, password, gender, state, birth_year) VALUES('sam', '@pass', 'male', 'NY', 1980);") + session.execute("INSERT INTO Test. users (user_name, password, gender, state, birth_year) " + "VALUES ('frodo', 'pass@', 'male', 'CA', 1985);") + session.execute("INSERT INTO Test. users (user_name, password, gender, state, birth_year) " + "VALUES ('sam', '@pass', 'male', 'NY', 1980);") res = session.execute("SELECT * FROM Test. users") - self.assertItemsEqual(rows_to_list(res), - [[u'frodo', 1985, u'male', u'pass@', u'CA'], - [u'sam', 1980, u'male', u'@pass', u'NY']]) + assert assert_lists_equal_ignoring_order(rows_to_list(res), [['frodo', 1985, 'male', 'pass@', 'CA'], + ['sam', 1980, 'male', '@pass', 'NY']]) - debug("Flushing and stopping cluster...") + logger.debug("Flushing and stopping cluster...") node1.flush() cluster.stop() - debug("Exporting to JSON file...") + logger.debug("Exporting to JSON file...") json_path = tempfile.mktemp(suffix='.schema.json') with open(json_path, 'w') as f: node1.run_sstable2json(f) @@ -59,11 +65,11 @@ class TestJson(Tester): with open(json_path, 'w') as fout: fout.writelines(data[1:]) - debug("Deleting cluster and creating new...") + logger.debug("Deleting cluster and creating new...") cluster.clear() cluster.start() - debug("Inserting data...") + logger.debug("Inserting data...") session = self.patient_cql_connection(node1) create_ks(session, 'Test', 1) @@ -77,25 +83,25 @@ class TestJson(Tester): ); """) - session.execute("INSERT INTO Test. users (user_name, password, gender, state, birth_year) VALUES('gandalf', 'p@$$', 'male', 'WA', 1955);") + session.execute("INSERT INTO Test. users (user_name, password, gender, state, birth_year) " + "VALUES ('gandalf', 'p@$$', 'male', 'WA', 1955);") node1.flush() cluster.stop() - debug("Importing JSON file...") + logger.debug("Importing JSON file...") with open(json_path) as f: node1.run_json2sstable(f, "test", "users") os.remove(json_path) - debug("Verifying import...") + logger.debug("Verifying import...") cluster.start() [node1] = cluster.nodelist() session = self.patient_cql_connection(node1) res = session.execute("SELECT * FROM Test. users") - debug("data: " + str(res)) + logger.debug("data: " + str(res)) - self.assertItemsEqual(rows_to_list(res), - [[u'frodo', 1985, u'male', u'pass@', u'CA'], - [u'sam', 1980, u'male', u'@pass', u'NY'], - [u'gandalf', 1955, u'male', u'p@$$', u'WA']]) + assert rows_to_list(res) == [['frodo', 1985, 'male', 'pass@', 'CA'], + ['sam', 1980, 'male', '@pass', 'NY'], + ['gandalf', 1955, 'male', 'p@$$', 'WA']] http://git-wip-us.apache.org/repos/asf/cassandra-dtest/blob/49b2dda4/largecolumn_test.py ---------------------------------------------------------------------- diff --git a/largecolumn_test.py b/largecolumn_test.py index d9f3879..261ab75 100644 --- a/largecolumn_test.py +++ b/largecolumn_test.py @@ -1,5 +1,11 @@ -from dtest import Tester, debug -from tools.decorators import since +import pytest +import re +import logging + +from dtest import Tester + +since = pytest.mark.since +logger = logging.getLogger(__name__) @since('2.2') @@ -23,16 +29,16 @@ class TestLargeColumn(Tester): return False output, err, _ = node.nodetool("gcstats") - debug(output) + logger.debug(output) output = output.split("\n") - self.assertRegexpMatches(output[0].strip(), 'Interval') + assert re.search('Interval', output[0].strip()) fields = output[1].split() - self.assertGreaterEqual(len(fields), 6, "Expected output from nodetool gcstats has at least six fields. However, fields is: {}".format(fields)) + assert len(fields) >= 6, "Expected output from nodetool gcstats has at least six fields. However >= fields is: {}".format(fields) for field in fields: - self.assertTrue(is_number(field.strip()) or field == 'NaN', "Expected numeric from fields from nodetool gcstats. However, field.strip() is: {}".format(field.strip())) + assert is_number(field.strip()) or field == 'NaN', "Expected numeric from fields from nodetool gcstats. However, field.strip() is: {}".format(field.strip()) return fields[6] - def cleanup_test(self): + def test_cleanup(self): """ @jira_ticket CASSANDRA-8670 """ @@ -45,12 +51,12 @@ class TestLargeColumn(Tester): node1, node2 = cluster.nodelist() session = self.patient_cql_connection(node1) - debug("Before stress {0}".format(self.directbytes(node1))) - debug("Running stress") + logger.debug("Before stress {0}".format(self.directbytes(node1))) + logger.debug("Running stress") # Run the full stack to see how much memory is utilized for "small" columns self.stress_with_col_size(cluster, node1, 1) beforeStress = self.directbytes(node1) - debug("Ran stress once {0}".format(beforeStress)) + logger.debug("Ran stress once {0}".format(beforeStress)) # Now run the full stack to see how much memory is utilized for "large" columns LARGE_COLUMN_SIZE = 1024 * 1024 * 63 @@ -58,9 +64,9 @@ class TestLargeColumn(Tester): output, err, _ = node1.nodetool("gcstats") afterStress = self.directbytes(node1) - debug("After stress {0}".format(afterStress)) + logger.debug("After stress {0}".format(afterStress)) # Any growth in memory usage should not be proportional column size. Really almost no memory should be used # since Netty was instructed to use a heap allocator diff = int(afterStress) - int(beforeStress) - self.assertLess(diff, LARGE_COLUMN_SIZE) + assert diff < LARGE_COLUMN_SIZE --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org