diff --git a/repair_tests/preview_repair_test.py b/repair_tests/preview_repair_test.py index df03b3604e..864f05e9b1 100644 --- a/repair_tests/preview_repair_test.py +++ b/repair_tests/preview_repair_test.py @@ -48,6 +48,10 @@ def test_preview(self): assert "Previewed data was in sync" in result.stdout assert_no_repair_history(session) assert preview_failure_count(node1) == 0 + assert bytes_previewed_desynchronized_count(node1) == 0 + assert token_ranges_previewed_desynchronized_count(node1) == 0 + assert bytes_previewed_count(node1) == 0 + assert bytes_previewed_desynchronized_count(node1) == 0 # make data inconsistent between nodes stmt = SimpleStatement("INSERT INTO ks.tbl (k,v) VALUES (%s, %s)") @@ -72,15 +76,24 @@ def test_preview(self): node1.start(wait_for_binary_proto=True) # data should not be in sync for full and unrepaired previews + desynchronized_bytes = bytes_previewed_desynchronized_count(node1) + previewed_bytes = bytes_previewed_count(node1) result = node1.repair(options=['ks', '--preview']) assert "Total estimated streaming" in result.stdout assert "Previewed data was in sync" not in result.stdout assert preview_failure_count(node1) == 1 + assert token_ranges_previewed_desynchronized_count(node1) == 80 + assert bytes_previewed_desynchronized_count(node1) > previewed_bytes + desynchronized_bytes = bytes_previewed_desynchronized_count(node1) + previewed_bytes = bytes_previewed_count(node1) result = node1.repair(options=['ks', '--preview', '--full']) assert "Total estimated streaming" in result.stdout assert "Previewed data was in sync" not in result.stdout assert preview_failure_count(node1) == 2 + assert token_ranges_previewed_desynchronized_count(node1) == 160 + assert bytes_previewed_desynchronized_count(node1) > desynchronized_bytes + assert bytes_previewed_count(node1) > previewed_bytes # repaired data should be in sync anyway result = node1.repair(options=['ks', '--validate']) @@ -94,20 +107,36 @@ def test_preview(self): node.nodetool('compact ks tbl') # ...and everything should be in sync + desynchronized_bytes = bytes_previewed_desynchronized_count(node1) + previewed_bytes = bytes_previewed_count(node1) result = node1.repair(options=['ks', '--preview']) assert "Previewed data was in sync" in result.stdout # data is repaired, previewFailure metric should remain same assert preview_failure_count(node1) == 2 + assert token_ranges_previewed_desynchronized_count(node1) == 160 + assert bytes_previewed_desynchronized_count(node1) == desynchronized_bytes + assert bytes_previewed_count(node1) == previewed_bytes + desynchronized_bytes = bytes_previewed_desynchronized_count(node1) + previewed_bytes = bytes_previewed_count(node1) result = node1.repair(options=['ks', '--preview', '--full']) assert "Previewed data was in sync" in result.stdout assert preview_failure_count(node1) == 2 + assert token_ranges_previewed_desynchronized_count(node1) == 160 + assert bytes_previewed_desynchronized_count(node1) == desynchronized_bytes + assert bytes_previewed_count(node1) > previewed_bytes result = node1.repair(options=['ks', '--validate']) assert "Repaired data is in sync" in result.stdout assert preview_failure_count(node2) == 0 + assert bytes_previewed_desynchronized_count(node2) == 0 + assert token_ranges_previewed_desynchronized_count(node2) == 0 assert preview_failure_count(node3) == 0 + assert bytes_previewed_desynchronized_count(node3) == 0 + assert token_ranges_previewed_desynchronized_count(node3) == 0 + assert bytes_previewed_count(node2) > 0 + assert bytes_previewed_count(node3) > 0 def assert_no_repair_history(session): @@ -121,3 +150,18 @@ def preview_failure_count(node): mbean = make_mbean('metrics', type='Repair', name='PreviewFailures') with JolokiaAgent(node) as jmx: return jmx.read_attribute(mbean, 'Count') + +def token_ranges_previewed_desynchronized_count(node): + mbean = make_mbean('metrics', type='Table', keyspace='ks', scope='tbl', name='TokenRangesPreviewedDesynchronized') + with JolokiaAgent(node) as jmx: + return jmx.read_attribute(mbean, 'Count') + +def bytes_previewed_desynchronized_count(node): + mbean = make_mbean('metrics', type='Table', keyspace='ks', scope='tbl', name='BytesPreviewedDesynchronized') + with JolokiaAgent(node) as jmx: + return jmx.read_attribute(mbean, 'Count') + +def bytes_previewed_count(node): + mbean = make_mbean('metrics', type='Table', keyspace='ks', scope='tbl', name='BytesPreviewed') + with JolokiaAgent(node) as jmx: + return jmx.read_attribute(mbean, 'Count')