This is an automated email from the ASF dual-hosted git repository.

brandonwilliams pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 6d343c5  Add AssertJ test dependency.
6d343c5 is described below

commit 6d343c59f275b1165be08e2256be878ac9b95fec
Author: Kevin Gallardo <[email protected]>
AuthorDate: Mon Mar 23 16:42:00 2020 -0400

    Add AssertJ test dependency.
    
    Patch by Kevin Gallardo, reviewed by David Capwell and brandon.williams
    for CASSANDRA-15631
---
 build.xml                                          |   8 +-
 .../org/apache/cassandra/repair/LocalSyncTask.java |   4 +-
 src/java/org/apache/cassandra/repair/SyncTask.java |   4 +-
 .../org/apache/cassandra/repair/RepairJobTest.java | 359 ++++++++++-----------
 .../utils/asserts/SizeableObjectAssert.java        |  47 +++
 .../cassandra/utils/asserts/SyncTaskAssert.java    |  92 ++++++
 .../utils/asserts/SyncTaskListAssert.java          |  62 ++++
 7 files changed, 378 insertions(+), 198 deletions(-)

diff --git a/build.xml b/build.xml
index 3be96e4..f22f4a9 100644
--- a/build.xml
+++ b/build.xml
@@ -626,8 +626,9 @@
           <dependency groupId="org.gridkit.jvmtool" artifactId="mxdump" 
version="0.14"/>
           <dependency groupId="org.gridkit.lab" artifactId="jvm-attach-api" 
version="1.5"/>
           <dependency groupId="org.gridkit.jvmtool" artifactId="sjk-json" 
version="0.14"/>
-
           <dependency groupId="com.beust" artifactId="jcommander" 
version="1.30"/>
+          <!-- when updating assertj, make sure to also update the 
corresponding junit-bom dependency -->
+          <dependency groupId="org.assertj" artifactId="assertj-core" 
version="3.15.0"/>
 
         </dependencyManagement>
         <developer id="adelapena" name="Andres de la Peña"/>
@@ -709,6 +710,10 @@
         <dependency groupId="org.gridkit.lab" artifactId="jvm-attach-api" />
         <dependency groupId="com.beust" artifactId="jcommander" />
         <dependency groupId="org.gridkit.jvmtool" artifactId="sjk-json"/>
+        <!-- adding this dependency is necessary for assertj. When updating 
assertj, need to also update the version of
+             this that the new assertj's `assertj-parent-pom` depends on. -->
+        <dependency groupId="org.junit" artifactId="junit-bom" version="5.6.0" 
type="pom"/>
+        <dependency groupId="org.assertj" artifactId="assertj-core"/>
       </artifact:pom>
       <!-- this build-deps-pom-sources "artifact" is the same as 
build-deps-pom but only with those
            artifacts that have "-source.jar" files -->
@@ -727,6 +732,7 @@
         <dependency groupId="org.openjdk.jmh" 
artifactId="jmh-generator-annprocess"/>
         <dependency groupId="net.ju-n.compile-command-annotations" 
artifactId="compile-command-annotations"/>
         <dependency groupId="org.apache.ant" artifactId="ant-junit" 
version="1.9.7" />
+        <dependency groupId="org.assertj" artifactId="assertj-core"/>
       </artifact:pom>
 
       <artifact:pom id="coverage-deps-pom"
diff --git a/src/java/org/apache/cassandra/repair/LocalSyncTask.java 
b/src/java/org/apache/cassandra/repair/LocalSyncTask.java
index f9b5765..5916401 100644
--- a/src/java/org/apache/cassandra/repair/LocalSyncTask.java
+++ b/src/java/org/apache/cassandra/repair/LocalSyncTask.java
@@ -54,9 +54,9 @@ public class LocalSyncTask extends SyncTask implements 
StreamEventHandler
     private final UUID pendingRepair;
 
     @VisibleForTesting
-    final boolean requestRanges;
+    public final boolean requestRanges;
     @VisibleForTesting
-    final boolean transferRanges;
+    public final boolean transferRanges;
 
     public LocalSyncTask(RepairJobDesc desc, InetAddressAndPort local, 
InetAddressAndPort remote,
                          List<Range<Token>> diff, UUID pendingRepair,
diff --git a/src/java/org/apache/cassandra/repair/SyncTask.java 
b/src/java/org/apache/cassandra/repair/SyncTask.java
index 8b622c4..d0f1eca 100644
--- a/src/java/org/apache/cassandra/repair/SyncTask.java
+++ b/src/java/org/apache/cassandra/repair/SyncTask.java
@@ -21,6 +21,7 @@ package org.apache.cassandra.repair;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.util.concurrent.AbstractFuture;
 
@@ -39,7 +40,8 @@ public abstract class SyncTask extends 
AbstractFuture<SyncStat> implements Runna
     private static Logger logger = LoggerFactory.getLogger(SyncTask.class);
 
     protected final RepairJobDesc desc;
-    protected final List<Range<Token>> rangesToSync;
+    @VisibleForTesting
+    public final List<Range<Token>> rangesToSync;
     protected final PreviewKind previewKind;
     protected final SyncNodePair nodePair;
 
diff --git a/test/unit/org/apache/cassandra/repair/RepairJobTest.java 
b/test/unit/org/apache/cassandra/repair/RepairJobTest.java
index 068544d..5713898 100644
--- a/test/unit/org/apache/cassandra/repair/RepairJobTest.java
+++ b/test/unit/org/apache/cassandra/repair/RepairJobTest.java
@@ -21,7 +21,6 @@ package org.apache.cassandra.repair;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -35,10 +34,9 @@ import java.util.concurrent.TimeoutException;
 import java.util.function.Predicate;
 import java.util.stream.Collectors;
 
-import com.google.common.collect.Sets;
+import com.google.common.collect.ImmutableMap;
 import com.google.common.util.concurrent.ListenableFuture;
 import org.junit.After;
-import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -66,11 +64,11 @@ import org.apache.cassandra.utils.MerkleTree;
 import org.apache.cassandra.utils.MerkleTrees;
 import org.apache.cassandra.utils.ObjectSizes;
 import org.apache.cassandra.utils.UUIDGen;
+import org.apache.cassandra.utils.asserts.SyncTaskListAssert;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
+import static org.apache.cassandra.utils.asserts.SyncTaskAssert.assertThat;
+import static org.apache.cassandra.utils.asserts.SyncTaskListAssert.assertThat;
+import static org.assertj.core.api.Assertions.assertThat;
 
 public class RepairJobTest
 {
@@ -80,14 +78,14 @@ public class RepairJobTest
     private static final IPartitioner MURMUR3_PARTITIONER = 
Murmur3Partitioner.instance;
     private static final String KEYSPACE = "RepairJobTest";
     private static final String CF = "Standard1";
-    private static final Object messageLock = new Object();
-
-    private static final Range<Token> range1 = range(0, 1);
-    private static final Range<Token> range2 = range(2, 3);
-    private static final Range<Token> range3 = range(4, 5);
-    private static final RepairJobDesc desc = new 
RepairJobDesc(UUID.randomUUID(), UUID.randomUUID(), KEYSPACE, CF, 
Arrays.asList());
-    private static final List<Range<Token>> fullRange = 
Collections.singletonList(new Range<>(MURMUR3_PARTITIONER.getMinimumToken(),
-                                                                               
               MURMUR3_PARTITIONER.getMaximumToken()));
+    private static final Object MESSAGE_LOCK = new Object();
+
+    private static final Range<Token> RANGE_1 = range(0, 1);
+    private static final Range<Token> RANGE_2 = range(2, 3);
+    private static final Range<Token> RANGE_3 = range(4, 5);
+    private static final RepairJobDesc JOB_DESC = new 
RepairJobDesc(UUID.randomUUID(), UUID.randomUUID(), KEYSPACE, CF, 
Collections.emptyList());
+    private static final List<Range<Token>> FULL_RANGE = 
Collections.singletonList(new Range<>(MURMUR3_PARTITIONER.getMinimumToken(),
+                                                                               
                MURMUR3_PARTITIONER.getMaximumToken()));
     private static InetAddressAndPort addr1;
     private static InetAddressAndPort addr2;
     private static InetAddressAndPort addr3;
@@ -135,11 +133,11 @@ public class RepairJobTest
 
         UUID parentRepairSession = UUID.randomUUID();
         
ActiveRepairService.instance.registerParentRepairSession(parentRepairSession, 
FBUtilities.getBroadcastAddressAndPort(),
-                                                                 
Collections.singletonList(Keyspace.open(KEYSPACE).getColumnFamilyStore(CF)), 
fullRange, false,
+                                                                 
Collections.singletonList(Keyspace.open(KEYSPACE).getColumnFamilyStore(CF)), 
FULL_RANGE, false,
                                                                  
ActiveRepairService.UNREPAIRED_SSTABLE, false, PreviewKind.NONE);
 
         this.session = new MeasureableRepairSession(parentRepairSession, 
UUIDGen.getTimeUUID(),
-                                                    new CommonRange(neighbors, 
Collections.emptySet(), fullRange),
+                                                    new CommonRange(neighbors, 
Collections.emptySet(), FULL_RANGE),
                                                     KEYSPACE, 
RepairParallelism.SEQUENTIAL,
                                                     false, false, false,
                                                     PreviewKind.NONE, false, 
CF);
@@ -179,7 +177,7 @@ public class RepairJobTest
         RepairResult result = job.get(TEST_TIMEOUT_S, TimeUnit.SECONDS);
 
         // Since there are no differences, there should be nothing to sync.
-        assertEquals(0, result.stats.size());
+        assertThat(result.stats).hasSize(0);
 
         // RepairJob should send out SNAPSHOTS -> VALIDATIONS -> done
         List<Verb> expectedTypes = new ArrayList<>();
@@ -188,9 +186,7 @@ public class RepairJobTest
         for (int i = 0; i < 3; i++)
             expectedTypes.add(Verb.VALIDATION_REQ);
 
-        assertEquals(expectedTypes, observedMessages.stream()
-                                                    .map(Message::verb)
-                                                    
.collect(Collectors.toList()));
+        
assertThat(observedMessages).extracting(Message::verb).containsExactlyElementsOf(expectedTypes);
     }
 
     /**
@@ -223,16 +219,16 @@ public class RepairJobTest
                                                                      
session.previewKind);
 
         // SyncTasks themselves should not contain significant memory
-        assertTrue(ObjectSizes.measureDeep(syncTasks) < 0.2 * singleTreeSize);
+        SyncTaskListAssert.assertThat(syncTasks).hasSizeLessThan(0.2 * 
singleTreeSize);
 
         ListenableFuture<List<SyncStat>> syncResults = 
job.executeTasks(syncTasks);
 
         // Immediately following execution the internal execution queue should 
still retain the trees
-        assertTrue(ObjectSizes.measureDeep(session) > singleTreeSize);
+        
assertThat(ObjectSizes.measureDeep(session)).isGreaterThan(singleTreeSize);
 
         // The session retains memory in the contained executor until the 
threads expire, so we wait for the threads
         // that ran the Tree -> SyncTask conversions to die and release the 
memory
-        int millisUntilFreed;
+        long millisUntilFreed;
         for (millisUntilFreed = 0; millisUntilFreed < TEST_TIMEOUT_S * 1000; 
millisUntilFreed += THREAD_TIMEOUT_MILLIS)
         {
             // The measured size of the syncingTasks, and result of the 
computation should be much smaller
@@ -241,18 +237,22 @@ public class RepairJobTest
                 break;
         }
 
-        assertTrue(millisUntilFreed < TEST_TIMEOUT_S * 1000);
+        assertThat(millisUntilFreed).isLessThan(TEST_TIMEOUT_S * 1000);
 
         List<SyncStat> results = syncResults.get(TEST_TIMEOUT_S, 
TimeUnit.SECONDS);
 
-        assertTrue(ObjectSizes.measureDeep(results) < 0.2 * singleTreeSize);
+        assertThat(ObjectSizes.measureDeep(results)).isLessThan(Math.round(0.2 
* singleTreeSize));
+        assertThat(session.getSyncingTasks()).isEmpty();
 
-        assertEquals(2, results.size());
-        assertEquals(0, session.getSyncingTasks().size());
-        assertTrue(results.stream().allMatch(s -> s.numberOfDifferences == 1));
+        assertThat(results)
+            .hasSize(2)
+            .extracting(s -> s.numberOfDifferences)
+            .containsOnly(1L);
 
-        assertEquals(2, messages.size());
-        assertTrue(messages.stream().allMatch(m -> m.verb() == Verb.SYNC_REQ));
+        assertThat(messages)
+            .hasSize(2)
+            .extracting(Message::verb)
+            .containsOnly(Verb.SYNC_REQ);
     }
 
     @Test
@@ -269,32 +269,31 @@ public class RepairJobTest
 
     public static void testCreateStandardSyncTasks(boolean pullRepair)
     {
-        List<TreeResponse> treeResponses = Arrays.asList(treeResponse(addr1, 
range1, "same",      range2, "same", range3, "same"),
-                                                         treeResponse(addr2, 
range1, "different", range2, "same", range3, "different"),
-                                                         treeResponse(addr3, 
range1, "same",      range2, "same", range3, "same"));
+        List<TreeResponse> treeResponses = Arrays.asList(treeResponse(addr1, 
RANGE_1, "same", RANGE_2, "same", RANGE_3, "same"),
+                                                         treeResponse(addr2, 
RANGE_1, "different", RANGE_2, "same", RANGE_3, "different"),
+                                                         treeResponse(addr3, 
RANGE_1, "same", RANGE_2, "same", RANGE_3, "same"));
 
-        Map<SyncNodePair, SyncTask> tasks = 
toMap(RepairJob.createStandardSyncTasks(desc,
+        Map<SyncNodePair, SyncTask> tasks = 
toMap(RepairJob.createStandardSyncTasks(JOB_DESC,
                                                                                
     treeResponses,
                                                                                
     addr1, // local
                                                                                
     noTransient(), // transient
                                                                                
     false,
                                                                                
     pullRepair,
                                                                                
     PreviewKind.ALL));
+        assertThat(tasks).hasSize(2);
 
-        Assert.assertEquals(2, tasks.size());
+        assertThat(tasks.get(pair(addr1, addr2)))
+                      .isLocal()
+                      .isRequestRanges()
+                      .hasTransferRanges(!pullRepair)
+                      .hasRanges(RANGE_1, RANGE_3);
 
-        SyncTask task = tasks.get(pair(addr1, addr2));
-        Assert.assertTrue(task.isLocal());
-        Assert.assertTrue(((LocalSyncTask) task).requestRanges);
-        Assert.assertEquals(!pullRepair, ((LocalSyncTask) 
task).transferRanges);
-        Assert.assertEquals(Arrays.asList(range1, range3), task.rangesToSync);
-
-        task = tasks.get(pair(addr2, addr3));
-        Assert.assertFalse(task.isLocal());
-        Assert.assertTrue(task instanceof SymmetricRemoteSyncTask);
-        Assert.assertEquals(Arrays.asList(range1, range3), task.rangesToSync);
+        assertThat(tasks.get(pair(addr2, addr3)))
+            .isInstanceOf(SymmetricRemoteSyncTask.class)
+            .isNotLocal()
+            .hasRanges(RANGE_1, RANGE_3);
 
-        Assert.assertNull(tasks.get(pair(addr1, addr3)));
+        assertThat(tasks.get(pair(addr1, addr3))).isNull();
     }
 
     @Test
@@ -307,10 +306,10 @@ public class RepairJobTest
 
     public void testStandardSyncTransient(boolean pullRepair)
     {
-        List<TreeResponse> treeResponses = Arrays.asList(treeResponse(addr1, 
range1, "same", range2, "same", range3, "same"),
-                                                         treeResponse(addr2, 
range1, "different", range2, "same", range3, "different"));
+        List<TreeResponse> treeResponses = Arrays.asList(treeResponse(addr1, 
RANGE_1, "same", RANGE_2, "same", RANGE_3, "same"),
+                                                         treeResponse(addr2, 
RANGE_1, "different", RANGE_2, "same", RANGE_3, "different"));
 
-        Map<SyncNodePair, SyncTask> tasks = 
toMap(RepairJob.createStandardSyncTasks(desc,
+        Map<SyncNodePair, SyncTask> tasks = 
toMap(RepairJob.createStandardSyncTasks(JOB_DESC,
                                                                                
     treeResponses,
                                                                                
     addr1, // local
                                                                                
     transientPredicate(addr2),
@@ -318,13 +317,13 @@ public class RepairJobTest
                                                                                
     pullRepair,
                                                                                
     PreviewKind.ALL));
 
-        Assert.assertEquals(1, tasks.size());
+        assertThat(tasks).hasSize(1);
 
-        SyncTask task = tasks.get(pair(addr1, addr2));
-        Assert.assertTrue(task.isLocal());
-        Assert.assertTrue(((LocalSyncTask) task).requestRanges);
-        Assert.assertFalse(((LocalSyncTask) task).transferRanges);
-        Assert.assertEquals(Arrays.asList(range1, range3), task.rangesToSync);
+        assertThat(tasks.get(pair(addr1, addr2)))
+            .isLocal()
+            .isRequestRanges()
+            .hasTransferRanges(false)
+            .hasRanges(RANGE_1, RANGE_3);
     }
 
     @Test
@@ -337,10 +336,10 @@ public class RepairJobTest
 
     public void testStandardSyncLocalTransient(boolean pullRepair)
     {
-        List<TreeResponse> treeResponses = Arrays.asList(treeResponse(addr1, 
range1, "same", range2, "same", range3, "same"),
-                                                         treeResponse(addr2, 
range1, "different", range2, "same", range3, "different"));
+        List<TreeResponse> treeResponses = Arrays.asList(treeResponse(addr1, 
RANGE_1, "same", RANGE_2, "same", RANGE_3, "same"),
+                                                         treeResponse(addr2, 
RANGE_1, "different", RANGE_2, "same", RANGE_3, "different"));
 
-        Map<SyncNodePair, SyncTask> tasks = 
toMap(RepairJob.createStandardSyncTasks(desc,
+        Map<SyncNodePair, SyncTask> tasks = 
toMap(RepairJob.createStandardSyncTasks(JOB_DESC,
                                                                                
     treeResponses,
                                                                                
     addr1, // local
                                                                                
     transientPredicate(addr1),
@@ -350,17 +349,17 @@ public class RepairJobTest
 
         if (pullRepair)
         {
-            Assert.assertTrue(tasks.isEmpty());
+            assertThat(tasks).isEmpty();
             return;
         }
 
-        Assert.assertEquals(1, tasks.size());
+        assertThat(tasks).hasSize(1);
+        assertThat(tasks.get(pair(addr1, addr2)))
+            .isLocal()
+            .isNotRequestRanges()
+            .hasTransferRanges(true)
+            .hasRanges(RANGE_1, RANGE_3);
 
-        SyncTask task = tasks.get(pair(addr1, addr2));
-        Assert.assertTrue(task.isLocal());
-        Assert.assertFalse(((LocalSyncTask) task).requestRanges);
-        Assert.assertTrue(((LocalSyncTask) task).transferRanges);
-        Assert.assertEquals(Arrays.asList(range1, range3), task.rangesToSync);
     }
 
     @Test
@@ -397,10 +396,10 @@ public class RepairJobTest
 
     public void testEmptyDifference(InetAddressAndPort local, 
Predicate<InetAddressAndPort> isTransient, boolean pullRepair)
     {
-        List<TreeResponse> treeResponses = Arrays.asList(treeResponse(addr1, 
range1, "same", range2, "same", range3, "same"),
-                                                         treeResponse(addr2, 
range1, "same", range2, "same", range3, "same"));
+        List<TreeResponse> treeResponses = Arrays.asList(treeResponse(addr1, 
RANGE_1, "same", RANGE_2, "same", RANGE_3, "same"),
+                                                         treeResponse(addr2, 
RANGE_1, "same", RANGE_2, "same", RANGE_3, "same"));
 
-        Map<SyncNodePair, SyncTask> tasks = 
toMap(RepairJob.createStandardSyncTasks(desc,
+        Map<SyncNodePair, SyncTask> tasks = 
toMap(RepairJob.createStandardSyncTasks(JOB_DESC,
                                                                                
     treeResponses,
                                                                                
     local, // local
                                                                                
     isTransient,
@@ -408,17 +407,17 @@ public class RepairJobTest
                                                                                
     pullRepair,
                                                                                
     PreviewKind.ALL));
 
-        Assert.assertTrue(tasks.isEmpty());
+        assertThat(tasks).isEmpty();
     }
 
     @Test
     public void testCreateStandardSyncTasksAllDifferent()
     {
-        List<TreeResponse> treeResponses = Arrays.asList(treeResponse(addr1, 
range1, "one",   range2, "one",   range3, "one"),
-                                                         treeResponse(addr2, 
range1, "two",   range2, "two",   range3, "two"),
-                                                         treeResponse(addr3, 
range1, "three", range2, "three", range3, "three"));
+        List<TreeResponse> treeResponses = Arrays.asList(treeResponse(addr1, 
RANGE_1, "one", RANGE_2, "one", RANGE_3, "one"),
+                                                         treeResponse(addr2, 
RANGE_1, "two", RANGE_2, "two", RANGE_3, "two"),
+                                                         treeResponse(addr3, 
RANGE_1, "three", RANGE_2, "three", RANGE_3, "three"));
 
-        Map<SyncNodePair, SyncTask> tasks = 
toMap(RepairJob.createStandardSyncTasks(desc,
+        Map<SyncNodePair, SyncTask> tasks = 
toMap(RepairJob.createStandardSyncTasks(JOB_DESC,
                                                                                
     treeResponses,
                                                                                
     addr1, // local
                                                                                
     ep -> ep.equals(addr3), // transient
@@ -426,31 +425,30 @@ public class RepairJobTest
                                                                                
     true,
                                                                                
     PreviewKind.ALL));
 
-        Assert.assertEquals(3, tasks.size());
-        SyncTask task = tasks.get(pair(addr1, addr2));
-        Assert.assertTrue(task.isLocal());
-        Assert.assertEquals(Arrays.asList(range1, range2, range3), 
task.rangesToSync);
-
-        task = tasks.get(pair(addr2, addr3));
-        Assert.assertFalse(task.isLocal());
-        Assert.assertEquals(Arrays.asList(range1, range2, range3), 
task.rangesToSync);
+        assertThat(tasks).hasSize(3);
 
-        task = tasks.get(pair(addr1, addr3));
-        Assert.assertTrue(task.isLocal());
-        Assert.assertEquals(Arrays.asList(range1, range2, range3), 
task.rangesToSync);
+        assertThat(tasks.get(pair(addr1, addr2)))
+            .isLocal()
+            .hasRanges(RANGE_1, RANGE_2, RANGE_3);
+        assertThat(tasks.get(pair(addr2, addr3)))
+            .isNotLocal()
+            .hasRanges(RANGE_1, RANGE_2, RANGE_3);
+        assertThat(tasks.get(pair(addr1, addr3)))
+            .isLocal()
+            .hasRanges(RANGE_1, RANGE_2, RANGE_3);
     }
 
     @Test
     public void testCreate5NodeStandardSyncTasksWithTransient()
     {
-        List<TreeResponse> treeResponses = Arrays.asList(treeResponse(addr1, 
range1, "one",   range2, "one",   range3, "one"),
-                                                         treeResponse(addr2, 
range1, "two",   range2, "two",   range3, "two"),
-                                                         treeResponse(addr3, 
range1, "three", range2, "three", range3, "three"),
-                                                         treeResponse(addr4, 
range1, "four",  range2, "four",  range3, "four"),
-                                                         treeResponse(addr5, 
range1, "five",  range2, "five",  range3, "five"));
+        List<TreeResponse> treeResponses = Arrays.asList(treeResponse(addr1, 
RANGE_1, "one", RANGE_2, "one", RANGE_3, "one"),
+                                                         treeResponse(addr2, 
RANGE_1, "two", RANGE_2, "two", RANGE_3, "two"),
+                                                         treeResponse(addr3, 
RANGE_1, "three", RANGE_2, "three", RANGE_3, "three"),
+                                                         treeResponse(addr4, 
RANGE_1, "four", RANGE_2, "four", RANGE_3, "four"),
+                                                         treeResponse(addr5, 
RANGE_1, "five", RANGE_2, "five", RANGE_3, "five"));
 
         Predicate<InetAddressAndPort> isTransient = ep -> ep.equals(addr4) || 
ep.equals(addr5);
-        Map<SyncNodePair, SyncTask> tasks = 
toMap(RepairJob.createStandardSyncTasks(desc,
+        Map<SyncNodePair, SyncTask> tasks = 
toMap(RepairJob.createStandardSyncTasks(JOB_DESC,
                                                                                
     treeResponses,
                                                                                
     addr1, // local
                                                                                
     isTransient, // transient
@@ -472,16 +470,17 @@ public class RepairJobTest
         {
             SyncTask task = tasks.get(pair);
             // Local only if addr1 is a coordinator
-            assertEquals(task.isLocal(), pair.coordinator.equals(addr1));
+            assertThat(task)
+                .hasLocal(pair.coordinator.equals(addr1))
+                // All ranges to be synchronised
+                .hasRanges(RANGE_1, RANGE_2, RANGE_3);
 
             boolean isRemote = !pair.coordinator.equals(addr1) && 
!pair.peer.equals(addr1);
             boolean involvesTransient = isTransient.test(pair.coordinator) || 
isTransient.test(pair.peer);
-            assertEquals(String.format("Coordinator: %s\n, Peer: 
%s\n",pair.coordinator, pair.peer),
-                         isRemote && involvesTransient,
-                         task instanceof AsymmetricRemoteSyncTask);
 
-            // All ranges to be synchronised
-            Assert.assertEquals(Arrays.asList(range1, range2, range3), 
task.rangesToSync);
+            assertThat(isRemote && involvesTransient)
+                .withFailMessage("Coordinator: %s\n, Peer: %s\n", 
pair.coordinator, pair.peer)
+                .isEqualTo(task instanceof AsymmetricRemoteSyncTask);
         }
     }
 
@@ -509,14 +508,14 @@ public class RepairJobTest
 
     public static void testLocalSyncWithTransient(InetAddressAndPort local, 
boolean pullRepair)
     {
-        List<TreeResponse> treeResponses = Arrays.asList(treeResponse(addr1, 
range1, "one",   range2, "one",   range3, "one"),
-                                                         treeResponse(addr2, 
range1, "two",   range2, "two",   range3, "two"),
-                                                         treeResponse(addr3, 
range1, "three", range2, "three", range3, "three"),
-                                                         treeResponse(addr4, 
range1, "four",  range2, "four",  range3, "four"),
-                                                         treeResponse(addr5, 
range1, "five",  range2, "five",  range3, "five"));
+        List<TreeResponse> treeResponses = Arrays.asList(treeResponse(addr1, 
RANGE_1, "one", RANGE_2, "one", RANGE_3, "one"),
+                                                         treeResponse(addr2, 
RANGE_1, "two", RANGE_2, "two", RANGE_3, "two"),
+                                                         treeResponse(addr3, 
RANGE_1, "three", RANGE_2, "three", RANGE_3, "three"),
+                                                         treeResponse(addr4, 
RANGE_1, "four", RANGE_2, "four", RANGE_3, "four"),
+                                                         treeResponse(addr5, 
RANGE_1, "five", RANGE_2, "five", RANGE_3, "five"));
 
         Predicate<InetAddressAndPort> isTransient = ep -> ep.equals(addr4) || 
ep.equals(addr5);
-        Map<SyncNodePair, SyncTask> tasks = 
toMap(RepairJob.createStandardSyncTasks(desc,
+        Map<SyncNodePair, SyncTask> tasks = 
toMap(RepairJob.createStandardSyncTasks(JOB_DESC,
                                                                                
     treeResponses,
                                                                                
     local, // local
                                                                                
     isTransient, // transient
@@ -524,24 +523,24 @@ public class RepairJobTest
                                                                                
     pullRepair,
                                                                                
     PreviewKind.ALL));
 
-        assertEquals(9, tasks.size());
+        assertThat(tasks).hasSize(9);
         for (InetAddressAndPort addr : new InetAddressAndPort[]{ addr1, addr2, 
addr3 })
         {
             if (local.equals(addr))
                 continue;
 
-            LocalSyncTask task = (LocalSyncTask) tasks.get(pair(local, addr));
-            assertTrue(task.requestRanges);
-            assertEquals(!pullRepair, task.transferRanges);
+            assertThat(tasks.get(pair(local, addr)))
+                .isRequestRanges()
+                .hasTransferRanges(!pullRepair);
         }
 
-        LocalSyncTask task = (LocalSyncTask) tasks.get(pair(local, addr4));
-        assertTrue(task.requestRanges);
-        assertFalse(task.transferRanges);
+        assertThat(tasks.get(pair(local, addr4)))
+            .isRequestRanges()
+            .hasTransferRanges(false);
 
-        task = (LocalSyncTask) tasks.get(pair(local, addr5));
-        assertTrue(task.requestRanges);
-        assertFalse(task.transferRanges);
+        assertThat(tasks.get(pair(local, addr5)))
+            .isRequestRanges()
+            .hasTransferRanges(false);
     }
 
     @Test
@@ -559,13 +558,13 @@ public class RepairJobTest
     private static void testLocalAndRemoteTransient(boolean pullRepair)
     {
         FBUtilities.setBroadcastInetAddress(addr4.address);
-        List<TreeResponse> treeResponses = Arrays.asList(treeResponse(addr1, 
range1, "one", range2, "one", range3, "one"),
-                                                         treeResponse(addr2, 
range1, "two", range2, "two", range3, "two"),
-                                                         treeResponse(addr3, 
range1, "three", range2, "three", range3, "three"),
-                                                         treeResponse(addr4, 
range1, "four", range2, "four", range3, "four"),
-                                                         treeResponse(addr5, 
range1, "five", range2, "five", range3, "five"));
+        List<TreeResponse> treeResponses = Arrays.asList(treeResponse(addr1, 
RANGE_1, "one", RANGE_2, "one", RANGE_3, "one"),
+                                                         treeResponse(addr2, 
RANGE_1, "two", RANGE_2, "two", RANGE_3, "two"),
+                                                         treeResponse(addr3, 
RANGE_1, "three", RANGE_2, "three", RANGE_3, "three"),
+                                                         treeResponse(addr4, 
RANGE_1, "four", RANGE_2, "four", RANGE_3, "four"),
+                                                         treeResponse(addr5, 
RANGE_1, "five", RANGE_2, "five", RANGE_3, "five"));
 
-        Map<SyncNodePair, SyncTask> tasks = 
toMap(RepairJob.createStandardSyncTasks(desc,
+        Map<SyncNodePair, SyncTask> tasks = 
toMap(RepairJob.createStandardSyncTasks(JOB_DESC,
                                                                                
     treeResponses,
                                                                                
     addr4, // local
                                                                                
     ep -> ep.equals(addr4) || ep.equals(addr5), // transient
@@ -573,17 +572,17 @@ public class RepairJobTest
                                                                                
     pullRepair,
                                                                                
     PreviewKind.ALL));
 
-        assertNull(tasks.get(pair(addr4, addr5)));
+        assertThat(tasks.get(pair(addr4, addr5))).isNull();
     }
 
     @Test
     public void testOptimizedCreateStandardSyncTasksAllDifferent()
     {
-        List<TreeResponse> treeResponses = Arrays.asList(treeResponse(addr1, 
range1, "one",   range2, "one",   range3, "one"),
-                                                         treeResponse(addr2, 
range1, "two",   range2, "two",   range3, "two"),
-                                                         treeResponse(addr3, 
range1, "three", range2, "three", range3, "three"));
+        List<TreeResponse> treeResponses = Arrays.asList(treeResponse(addr1, 
RANGE_1, "one", RANGE_2, "one", RANGE_3, "one"),
+                                                         treeResponse(addr2, 
RANGE_1, "two", RANGE_2, "two", RANGE_3, "two"),
+                                                         treeResponse(addr3, 
RANGE_1, "three", RANGE_2, "three", RANGE_3, "three"));
 
-        Map<SyncNodePair, SyncTask> tasks = 
toMap(RepairJob.createOptimisedSyncingSyncTasks(desc,
+        Map<SyncNodePair, SyncTask> tasks = 
toMap(RepairJob.createOptimisedSyncingSyncTasks(JOB_DESC,
                                                                                
             treeResponses,
                                                                                
             addr1, // local
                                                                                
             noTransient(),
@@ -598,18 +597,18 @@ public class RepairJobTest
                                                      pair(addr3, addr1),
                                                      pair(addr3, addr2) })
         {
-            assertEquals(Arrays.asList(range1, range2, range3), 
tasks.get(pair).rangesToSync);
+            assertThat(tasks.get(pair)).hasRanges(RANGE_1, RANGE_2, RANGE_3);
         }
     }
 
     @Test
     public void testOptimizedCreateStandardSyncTasks()
     {
-        List<TreeResponse> treeResponses = Arrays.asList(treeResponse(addr1, 
range1, "one",   range2, "one"),
-                                                         treeResponse(addr2, 
range1, "one",   range2, "two"),
-                                                         treeResponse(addr3, 
range1, "three", range2, "two"));
+        List<TreeResponse> treeResponses = Arrays.asList(treeResponse(addr1, 
RANGE_1, "one", RANGE_2, "one"),
+                                                         treeResponse(addr2, 
RANGE_1, "one", RANGE_2, "two"),
+                                                         treeResponse(addr3, 
RANGE_1, "three", RANGE_2, "two"));
 
-        Map<SyncNodePair, SyncTask> tasks = 
toMap(RepairJob.createOptimisedSyncingSyncTasks(desc,
+        Map<SyncNodePair, SyncTask> tasks = 
toMap(RepairJob.createOptimisedSyncingSyncTasks(JOB_DESC,
                                                                                
             treeResponses,
                                                                                
             addr4, // local
                                                                                
             noTransient(),
@@ -617,31 +616,29 @@ public class RepairJobTest
                                                                                
             false,
                                                                                
             PreviewKind.ALL));
 
-        for (SyncTask task : tasks.values())
-            assertTrue(task instanceof AsymmetricRemoteSyncTask);
+        
assertThat(tasks.values()).areAllInstanceOf(AsymmetricRemoteSyncTask.class);
 
-        assertEquals(Arrays.asList(range1), tasks.get(pair(addr1, 
addr3)).rangesToSync);
+        assertThat(tasks.get(pair(addr1, 
addr3)).rangesToSync).containsExactly(RANGE_1);
         // addr1 can get range2 from either addr2 or addr3 but not from both
-        assertStreamRangeFromEither(tasks, Arrays.asList(range2),
-                                    addr1, addr2, addr3);
+        assertStreamRangeFromEither(tasks, RANGE_2, addr1, addr2, addr3);
 
-        assertEquals(Arrays.asList(range1), tasks.get(pair(addr2, 
addr3)).rangesToSync);
-        assertEquals(Arrays.asList(range2), tasks.get(pair(addr2, 
addr1)).rangesToSync);
+        assertThat(tasks.get(pair(addr2, 
addr3)).rangesToSync).containsExactly(RANGE_1);
+        assertThat(tasks.get(pair(addr2, 
addr1)).rangesToSync).containsExactly(RANGE_2);
 
         // addr3 can get range1 from either addr1 or addr2 but not from both
-        assertStreamRangeFromEither(tasks, Arrays.asList(range1),
-                                    addr3, addr2, addr1);
-        assertEquals(Arrays.asList(range2), tasks.get(pair(addr3, 
addr1)).rangesToSync);
+        assertStreamRangeFromEither(tasks, RANGE_1, addr3, addr2, addr1);
+
+        assertThat(tasks.get(pair(addr3, 
addr1)).rangesToSync).containsExactly(RANGE_2);
     }
 
     @Test
     public void testOptimizedCreateStandardSyncTasksWithTransient()
     {
-        List<TreeResponse> treeResponses = Arrays.asList(treeResponse(addr1, 
range1, "same",      range2, "same", range3, "same"),
-                                                         treeResponse(addr2, 
range1, "different", range2, "same", range3, "different"),
-                                                         treeResponse(addr3, 
range1, "same",      range2, "same", range3, "same"));
+        List<TreeResponse> treeResponses = Arrays.asList(treeResponse(addr1, 
RANGE_1, "same", RANGE_2, "same", RANGE_3, "same"),
+                                                         treeResponse(addr2, 
RANGE_1, "different", RANGE_2, "same", RANGE_3, "different"),
+                                                         treeResponse(addr3, 
RANGE_1, "same", RANGE_2, "same", RANGE_3, "same"));
 
-        RepairJobDesc desc = new RepairJobDesc(UUID.randomUUID(), 
UUID.randomUUID(), "ks", "cf", Arrays.asList());
+        RepairJobDesc desc = new RepairJobDesc(UUID.randomUUID(), 
UUID.randomUUID(), "ks", "cf", Collections.emptyList());
         Map<SyncNodePair, SyncTask> tasks = 
toMap(RepairJob.createOptimisedSyncingSyncTasks(desc,
                                                                                
             treeResponses,
                                                                                
             addr1, // local
@@ -650,27 +647,26 @@ public class RepairJobTest
                                                                                
             false,
                                                                                
             PreviewKind.ALL));
 
-        assertEquals(3, tasks.size());
+        assertThat(tasks).hasSize(3);
         SyncTask task = tasks.get(pair(addr1, addr2));
-        assertTrue(task.isLocal());
-        assertElementEquals(Arrays.asList(range1, range3), task.rangesToSync);
-        assertTrue(((LocalSyncTask)task).requestRanges);
-        assertFalse(((LocalSyncTask)task).transferRanges);
 
-        assertStreamRangeFromEither(tasks, Arrays.asList(range3),
-                                    addr2, addr1, addr3);
+        assertThat(task)
+            .isLocal()
+            .hasRanges(RANGE_1, RANGE_3)
+            .isRequestRanges()
+            .hasTransferRanges(false);
 
-        assertStreamRangeFromEither(tasks, Arrays.asList(range1),
-                                    addr2, addr1, addr3);
+        assertStreamRangeFromEither(tasks, RANGE_3, addr2, addr1, addr3);
+        assertStreamRangeFromEither(tasks, RANGE_1, addr2, addr1, addr3);
     }
 
     // Asserts that ranges are streamed from one of the nodes but not from the 
both
-    public static void assertStreamRangeFromEither(Map<SyncNodePair, SyncTask> 
tasks, List<Range<Token>> ranges,
+    public static void assertStreamRangeFromEither(Map<SyncNodePair, SyncTask> 
tasks, Range<Token> range,
                                                    InetAddressAndPort target, 
InetAddressAndPort either, InetAddressAndPort or)
     {
         InetAddressAndPort streamsFrom;
         InetAddressAndPort doesntStreamFrom;
-        if (tasks.containsKey(pair(target, either)) && tasks.get(pair(target, 
either)).rangesToSync.equals(ranges))
+        if (tasks.containsKey(pair(target, either)) && tasks.get(pair(target, 
either)).rangesToSync.contains(range))
         {
             streamsFrom = either;
             doesntStreamFrom = or;
@@ -682,47 +678,30 @@ public class RepairJobTest
         }
 
         SyncTask task = tasks.get(pair(target, streamsFrom));
-        assertTrue(task instanceof AsymmetricRemoteSyncTask);
-        assertElementEquals(ranges, task.rangesToSync);
-        assertDoesntStreamRangeFrom(tasks, ranges, target, doesntStreamFrom);
+        assertThat(task).isInstanceOf(AsymmetricRemoteSyncTask.class);
+        assertThat(task.rangesToSync).containsOnly(range);
+        assertDoesntStreamRangeFrom(range, tasks.get(pair(target, 
doesntStreamFrom)));
     }
 
-    public static void assertDoesntStreamRangeFrom(Map<SyncNodePair, SyncTask> 
tasks, List<Range<Token>> ranges,
-                                                   InetAddressAndPort target, 
InetAddressAndPort source)
+    public static void assertDoesntStreamRangeFrom(Range<Token> range, 
SyncTask task)
     {
-        Set<Range<Token>> rangeSet = new HashSet<>(ranges);
-        SyncTask task = tasks.get(pair(target, source));
         if (task == null)
             return; // Doesn't stream anything
 
-        for (Range<Token> range : task.rangesToSync)
-        {
-            assertFalse(String.format("%s shouldn't stream %s from %s",
-                                      target, range, source),
-                        rangeSet.contains(range));
-        }
-    }
-
-    public static <T> void assertElementEquals(Collection<T> col1, 
Collection<T> col2)
-    {
-        Set<T> set1 = new HashSet<>(col1);
-        Set<T> set2 = new HashSet<>(col2);
-        Set<T> difference = Sets.difference(set1, set2);
-        assertTrue("Expected empty difference but got: " + 
difference.toString(),
-                   difference.isEmpty());
+        assertThat(task.rangesToSync).doesNotContain(range);
     }
 
-    public static Token tk(int i)
+    private static Token tk(int i)
     {
         return PARTITIONER.getToken(ByteBufferUtil.bytes(i));
     }
 
-    public static Range<Token> range(int from, int to)
+    private static Range<Token> range(int from, int to)
     {
         return new Range<>(tk(from), tk(to));
     }
 
-    public static TreeResponse treeResponse(InetAddressAndPort addr, Object... 
rangesAndHashes)
+    private static TreeResponse treeResponse(InetAddressAndPort addr, 
Object... rangesAndHashes)
     {
         MerkleTrees trees = new MerkleTrees(PARTITIONER);
         for (int i = 0; i < rangesAndHashes.length; i += 2)
@@ -736,24 +715,16 @@ public class RepairJobTest
         return new TreeResponse(addr, trees);
     }
 
-    public static SyncNodePair pair(InetAddressAndPort node1, 
InetAddressAndPort node2)
+    private static SyncNodePair pair(InetAddressAndPort node1, 
InetAddressAndPort node2)
     {
         return new SyncNodePair(node1, node2);
     }
 
     public static Map<SyncNodePair, SyncTask> toMap(List<SyncTask> tasks)
     {
-        Map<SyncNodePair, SyncTask> map = new HashMap();
-        for (SyncTask task : tasks)
-        {
-            SyncTask oldTask = map.put(task.nodePair, task);
-            Assert.assertNull(String.format("\nNode pair: %s\nOld task:  
%s\nNew task:  %s\n",
-                                            task.nodePair,
-                                            oldTask,
-                                            task),
-                              oldTask);
-        }
-        return map;
+        ImmutableMap.Builder<SyncNodePair, SyncTask> map = 
ImmutableMap.builder();
+        tasks.forEach(t -> map.put(t.nodePair, t));
+        return map.build();
     }
 
     public static Predicate<InetAddressAndPort> 
transientPredicate(InetAddressAndPort... transientNodes)
@@ -773,13 +744,13 @@ public class RepairJobTest
     private MerkleTrees createInitialTree(boolean invalidate)
     {
         MerkleTrees tree = new MerkleTrees(MURMUR3_PARTITIONER);
-        tree.addMerkleTrees((int) Math.pow(2, 15), fullRange);
+        tree.addMerkleTrees((int) Math.pow(2, 15), FULL_RANGE);
         tree.init();
 
         if (invalidate)
         {
             // change a range in one of the trees
-            Token token = MURMUR3_PARTITIONER.midpoint(fullRange.get(0).left, 
fullRange.get(0).right);
+            Token token = MURMUR3_PARTITIONER.midpoint(FULL_RANGE.get(0).left, 
FULL_RANGE.get(0).right);
             tree.invalidate(token);
             tree.get(token).hash("non-empty hash!".getBytes());
         }
@@ -796,7 +767,7 @@ public class RepairJobTest
                 return false;
 
             // So different Thread's messages don't overwrite each other.
-            synchronized (messageLock)
+            synchronized (MESSAGE_LOCK)
             {
                 messageCapture.add(message);
             }
diff --git 
a/test/unit/org/apache/cassandra/utils/asserts/SizeableObjectAssert.java 
b/test/unit/org/apache/cassandra/utils/asserts/SizeableObjectAssert.java
new file mode 100644
index 0000000..5e36625
--- /dev/null
+++ b/test/unit/org/apache/cassandra/utils/asserts/SizeableObjectAssert.java
@@ -0,0 +1,47 @@
+/*
+ * 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.utils.asserts;
+
+import org.apache.cassandra.utils.ObjectSizes;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+public interface SizeableObjectAssert<SELF extends SizeableObjectAssert<SELF>>
+{
+    Object actual();
+
+    default SELF hasSizeLessThan(double expectedSize)
+    {
+        double measured = (double) ObjectSizes.measureDeep(actual());
+        assertThat(measured)
+                  .withFailMessage("Size of measured object [%f] is not less 
than the expected size [%f]", measured, expectedSize)
+                  .isLessThan(expectedSize);
+        return ((SELF) this);
+
+    }
+
+    default SELF hasSizeGreaterThanOrEqual(double expectedSize)
+    {
+        double measured = (double) ObjectSizes.measureDeep(actual());
+        assertThat(measured)
+                  .withFailMessage("Size of measured object [%f] is not 
greater than or equal to the expected size [%f]", measured, expectedSize)
+                  .isGreaterThanOrEqualTo(expectedSize);
+        return ((SELF) this);
+    }
+}
diff --git a/test/unit/org/apache/cassandra/utils/asserts/SyncTaskAssert.java 
b/test/unit/org/apache/cassandra/utils/asserts/SyncTaskAssert.java
new file mode 100644
index 0000000..c480783
--- /dev/null
+++ b/test/unit/org/apache/cassandra/utils/asserts/SyncTaskAssert.java
@@ -0,0 +1,92 @@
+/*
+ * 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.utils.asserts;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.repair.LocalSyncTask;
+import org.apache.cassandra.repair.SyncTask;
+import org.assertj.core.api.AbstractObjectAssert;
+import org.assertj.core.api.Assertions;
+
+public class SyncTaskAssert extends AbstractObjectAssert<SyncTaskAssert, 
SyncTask> implements SizeableObjectAssert<SyncTaskAssert>
+{
+    private SyncTaskAssert(SyncTask syncTask)
+    {
+        super(syncTask, SyncTaskAssert.class);
+    }
+
+    public static SyncTaskAssert assertThat(SyncTask task)
+    {
+        return new SyncTaskAssert(task);
+    }
+
+    @Override
+    public Object actual()
+    {
+        return actual;
+    }
+
+    public SyncTaskAssert hasLocal(boolean expected)
+    {
+        Assertions.assertThat(actual.isLocal()).isEqualTo(expected);
+        return this;
+    }
+
+    public SyncTaskAssert isLocal()
+    {
+        Assertions.assertThat(actual.isLocal()).isTrue();
+        return this;
+    }
+
+    public SyncTaskAssert isNotLocal()
+    {
+        Assertions.assertThat(actual.isLocal()).isFalse();
+        return this;
+    }
+
+    public SyncTaskAssert isRequestRanges()
+    {
+        Preconditions.checkState(actual instanceof LocalSyncTask, "Tested 
value is not a LocalSyncTask");
+        Assertions.assertThat(((LocalSyncTask) actual).requestRanges).isTrue();
+        return this;
+    }
+
+    public SyncTaskAssert isNotRequestRanges()
+    {
+        Preconditions.checkState(actual instanceof LocalSyncTask, "Tested 
value is not a LocalSyncTask");
+        Assertions.assertThat(((LocalSyncTask) 
actual).requestRanges).isFalse();
+        return this;
+    }
+
+    public SyncTaskAssert hasTransferRanges(boolean expected)
+    {
+        Preconditions.checkState(actual instanceof LocalSyncTask, "Tested 
value is not a LocalSyncTask");
+        Assertions.assertThat(((LocalSyncTask) 
actual).transferRanges).isEqualTo(expected);
+        return this;
+    }
+
+    @SuppressWarnings("unchecked")
+    public SyncTaskAssert hasRanges(Range... ranges)
+    {
+        Assertions.assertThat(actual.rangesToSync).containsOnly(ranges);
+        return this;
+    }
+}
diff --git 
a/test/unit/org/apache/cassandra/utils/asserts/SyncTaskListAssert.java 
b/test/unit/org/apache/cassandra/utils/asserts/SyncTaskListAssert.java
new file mode 100644
index 0000000..668719b
--- /dev/null
+++ b/test/unit/org/apache/cassandra/utils/asserts/SyncTaskListAssert.java
@@ -0,0 +1,62 @@
+/*
+ * 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.utils.asserts;
+
+import java.util.List;
+
+import com.google.common.collect.ImmutableList;
+
+import org.apache.cassandra.repair.SyncTask;
+import org.assertj.core.api.AbstractListAssert;
+
+public class SyncTaskListAssert extends AbstractListAssert<SyncTaskListAssert, 
List<SyncTask>, SyncTask, SyncTaskAssert>
+implements SizeableObjectAssert<SyncTaskListAssert>
+{
+    public SyncTaskListAssert(List<SyncTask> syncTasks)
+    {
+        super(syncTasks, SyncTaskListAssert.class);
+    }
+
+    protected SyncTaskAssert toAssert(SyncTask value, String description)
+    {
+        return SyncTaskAssert.assertThat(value);
+    }
+
+    protected SyncTaskListAssert newAbstractIterableAssert(Iterable<? extends 
SyncTask> iterable)
+    {
+        return assertThat(iterable);
+    }
+
+    public static SyncTaskListAssert assertThat(Iterable<? extends SyncTask> 
iterable)
+    {
+        return new SyncTaskListAssert(ImmutableList.copyOf(iterable));
+    }
+
+    @Override
+    public Object actual()
+    {
+        return actual;
+    }
+
+    public SyncTaskListAssert areAllInstanceOf(Class<?> type)
+    {
+        actual.forEach(t -> toAssert(t, "").isInstanceOf(type));
+        return this;
+    }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to