Add checks to test the RepairMetrics.previewFailures counter

patch by Yasar Arafath Baigh; reviewed by Adam Holmberg, Andrés de la Peña and Benjamin
Lerer for CASSANDRA-16191
diff --git a/repair_tests/preview_repair_test.py b/repair_tests/preview_repair_test.py
index 5333315..df03b36 100644
--- a/repair_tests/preview_repair_test.py
+++ b/repair_tests/preview_repair_test.py
@@ -7,6 +7,7 @@
 from dtest import Tester, create_ks
 from repair_tests.incremental_repair_test import assert_parent_repair_session_count
 from tools.data import create_c1c2_table
+from tools.jmxutils import make_mbean, JolokiaAgent
 
 since = pytest.mark.since
 
@@ -14,12 +15,6 @@
 @since('4.0')
 class TestPreviewRepair(Tester):
 
-    def assert_no_repair_history(self, session):
-        rows = session.execute("select * from system_distributed.repair_history")
-        assert rows.current_rows == []
-        rows = session.execute("select * from system_distributed.parent_repair_history")
-        assert rows.current_rows == []
-
     @since('4.0')
     def test_parent_repair_session_cleanup(self):
         """
@@ -51,7 +46,8 @@
         # everything should be in sync
         result = node1.repair(options=['ks', '--preview'])
         assert "Previewed data was in sync" in result.stdout
-        self.assert_no_repair_history(session)
+        assert_no_repair_history(session)
+        assert preview_failure_count(node1) == 0
 
         # make data inconsistent between nodes
         stmt = SimpleStatement("INSERT INTO ks.tbl (k,v) VALUES (%s, %s)")
@@ -79,16 +75,18 @@
         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
 
         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
 
         # repaired data should be in sync anyway
         result = node1.repair(options=['ks', '--validate'])
         assert "Repaired data is in sync" in result.stdout
 
-        self.assert_no_repair_history(session)
+        assert_no_repair_history(session)
 
         # repair the data...
         node1.repair(options=['ks'])
@@ -98,9 +96,28 @@
         # ...and everything should be in sync
         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
 
         result = node1.repair(options=['ks', '--preview', '--full'])
         assert "Previewed data was in sync" in result.stdout
+        assert preview_failure_count(node1) == 2
 
         result = node1.repair(options=['ks', '--validate'])
         assert "Repaired data is in sync" in result.stdout
+
+        assert preview_failure_count(node2) == 0
+        assert preview_failure_count(node3) == 0
+
+
+def assert_no_repair_history(session):
+    rows = session.execute("select * from system_distributed.repair_history")
+    assert rows.current_rows == []
+    rows = session.execute("select * from system_distributed.parent_repair_history")
+    assert rows.current_rows == []
+
+
+def preview_failure_count(node):
+    mbean = make_mbean('metrics', type='Repair', name='PreviewFailures')
+    with JolokiaAgent(node) as jmx:
+        return jmx.read_attribute(mbean, 'Count')