You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by dj...@apache.org on 2019/08/23 23:08:44 UTC

[cassandra] branch trunk updated: Enhance & reenable RepairTest with compression=off and compression=on

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

djoshi 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 3a4e006  Enhance & reenable RepairTest with compression=off and compression=on
3a4e006 is described below

commit 3a4e00615b14275e5dac535a304add9e1cf4e4eb
Author: Dinesh A. Joshi <di...@apple.com>
AuthorDate: Thu Aug 8 17:08:02 2019 -0700

    Enhance & reenable RepairTest with compression=off and compression=on
    
    patch by Dinesh Joshi; reviewed by Jon Meredith for CASSANDRA-15272
---
 CHANGES.txt                                        |  1 +
 .../cassandra/distributed/test/RepairTest.java     | 71 ++++++++++++++--------
 2 files changed, 45 insertions(+), 27 deletions(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index 8ac7fbf..25f52c0 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.0
+ * Enhance & reenable RepairTest with compression=off and compression=on (CASSANDRA-15272)
  * Improve readability of Table metrics Virtual tables units (CASSANDRA-15194)
  * Fix error with non-existent table for nodetool tablehistograms (CASSANDRA-14410)
  * Avoid result truncation in decimal operations (CASSANDRA-15232)
diff --git a/test/distributed/org/apache/cassandra/distributed/test/RepairTest.java b/test/distributed/org/apache/cassandra/distributed/test/RepairTest.java
index 143b5cd..1c488aa 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/RepairTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/RepairTest.java
@@ -25,6 +25,7 @@ import java.util.function.Consumer;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
+import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
 
@@ -38,11 +39,13 @@ import static java.util.concurrent.TimeUnit.MINUTES;
 import static org.apache.cassandra.distributed.impl.ExecUtil.rethrow;
 import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
 import static org.apache.cassandra.distributed.api.Feature.NETWORK;
-@Ignore
+
 public class RepairTest extends DistributedTestBase
 {
     private static final String insert = withKeyspace("INSERT INTO %s.test (k, c1, c2) VALUES (?, 'value1', 'value2');");
     private static final String query = withKeyspace("SELECT k, c1, c2 FROM %s.test WHERE k = ?;");
+    private static Cluster cluster;
+
     private static void insert(Cluster cluster, int start, int end, int ... nodes)
     {
         for (int i = start ; i < end ; ++i)
@@ -71,19 +74,38 @@ public class RepairTest extends DistributedTestBase
             cluster.get(node).runOnInstance(rethrow(() -> StorageService.instance.forceKeyspaceFlush(KEYSPACE)));
     }
 
-    private Cluster create(Consumer<InstanceConfig> configModifier) throws IOException
+    private static Cluster create(Consumer<InstanceConfig> configModifier) throws IOException
     {
         configModifier = configModifier.andThen(
-            config -> config.set("hinted_handoff_enabled", false)
-                            .set("commitlog_sync_batch_window_in_ms", 5)
-                            .with(NETWORK)
-                            .with(GOSSIP)
+        config -> config.set("hinted_handoff_enabled", false)
+                        .set("commitlog_sync_batch_window_in_ms", 5)
+                        .with(NETWORK)
+                        .with(GOSSIP)
         );
 
         Cluster cluster = init(Cluster.build(3).withConfig(configModifier).start());
+        return cluster;
+    }
+
+    private void repair(Cluster cluster, Map<String, String> options)
+    {
+        cluster.get(1).runOnInstance(rethrow(() -> {
+            SimpleCondition await = new SimpleCondition();
+            StorageService.instance.repair(KEYSPACE, options, ImmutableList.of((tag, event) -> {
+                if (event.getType() == ProgressEventType.COMPLETE)
+                    await.signalAll();
+            })).right.get();
+            await.await(1L, MINUTES);
+        }));
+    }
+
+    void populate(Cluster cluster, boolean compression)
+    {
         try
         {
-            cluster.schemaChange(withKeyspace("CREATE TABLE %s.test (k text, c1 text, c2 text, PRIMARY KEY (k));"));
+            cluster.schemaChange(withKeyspace("DROP TABLE IF EXISTS %s.test;"));
+            cluster.schemaChange(withKeyspace("CREATE TABLE %s.test (k text, c1 text, c2 text, PRIMARY KEY (k))") +
+                                 (compression == false ? " WITH compression = {'enabled' : false};" : ";"));
 
             insert(cluster,    0, 1000, 1, 2, 3);
             flush(cluster, 1);
@@ -94,42 +116,37 @@ public class RepairTest extends DistributedTestBase
             verify(cluster,    0, 1000, 1, 2, 3);
             verify(cluster, 1000, 1001, 1, 2);
             verify(cluster, 1001, 2001, 1, 2, 3);
-            return cluster;
         }
         catch (Throwable t)
         {
             cluster.close();
             throw t;
         }
-    }
 
-    private void repair(Cluster cluster, Map<String, String> options)
-    {
-        cluster.get(1).runOnInstance(rethrow(() -> {
-            SimpleCondition await = new SimpleCondition();
-            StorageService.instance.repair(KEYSPACE, options, ImmutableList.of((tag, event) -> {
-                if (event.getType() == ProgressEventType.COMPLETE)
-                    await.signalAll();
-            })).right.get();
-            await.await(1L, MINUTES);
-        }));
     }
 
-    void simpleRepair(boolean orderPreservingPartitioner, boolean sequential) throws IOException
+    void simpleRepair(Cluster cluster, boolean sequential, boolean compression) throws IOException
     {
-        Cluster cluster = create(config -> {
-            if (orderPreservingPartitioner)
-                config.set("partitioner", "org.apache.cassandra.dht.ByteOrderedPartitioner");
-        });
+        populate(cluster, compression);
         repair(cluster, ImmutableMap.of("parallelism", sequential ? "sequential" : "parallel"));
         verify(cluster, 0, 2001, 1, 2, 3);
     }
 
-    @Test
-    public void testSimpleSequentialRepair() throws IOException
+    @BeforeClass
+    public static void setupCluster() throws IOException
     {
-        simpleRepair(false, true);
+        cluster = create(config -> {});
     }
 
+    @Ignore("Test requires CASSANDRA-13938 to be merged")
+    public void testSimpleSequentialRepairDefaultCompression() throws IOException
+    {
+        simpleRepair(cluster, true, true);
+    }
 
+    @Test
+    public void testSimpleSequentialRepairCompressionOff() throws IOException
+    {
+        simpleRepair(cluster, true, false);
+    }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org