You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by if...@apache.org on 2021/07/15 07:27:46 UTC

[cassandra-harry] 02/04: Integration improvements

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

ifesdjeen pushed a commit to branch CASSANDRA-16262-2
in repository https://gitbox.apache.org/repos/asf/cassandra-harry.git

commit 57053decc6b810d1148e1b7f680e54feece57d99
Author: Alex Petrov <ol...@gmail.com>
AuthorDate: Mon Jul 12 17:54:42 2021 +0200

    Integration improvements
---
 .../test/resources/single_partition_test.yml       |  55 ++++++
 harry-integration-external/pom.xml                 |   5 +
 .../model/sut/external/ExternalClusterSut.java     |  14 +-
 harry-integration/pom.xml                          |  14 +-
 .../src/harry/model/sut/ExternalClusterSut.java    | 187 ---------------------
 .../src/harry/model/sut/InJvmSutBase.java          |   3 +-
 .../src/harry/runner/HarryRunnerJvm.java           |  43 +++++
 harry-integration/test/conf/cassandra.yaml         |  44 +++++
 .../test/harry/ddl/SchemaGenTest.java              |  15 +-
 .../generators/DataGeneratorsIntegrationTest.java  | 104 ++++++++++++
 .../test/harry/model/IntegrationTestBase.java      |   6 +-
 .../model/QuiescentCheckerIntegrationTest.java     |   3 +-
 .../test/harry/model/TestEveryClustering.java      |  89 ++++++++++
 .../test/harry/op/RowVisitorTest.java              |   8 +-
 .../test/resources/single_partition_test.yml       |  55 ++++++
 15 files changed, 435 insertions(+), 210 deletions(-)

diff --git a/harry-integration-backup/test/resources/single_partition_test.yml b/harry-integration-backup/test/resources/single_partition_test.yml
new file mode 100644
index 0000000..0ebe2aa
--- /dev/null
+++ b/harry-integration-backup/test/resources/single_partition_test.yml
@@ -0,0 +1,55 @@
+seed: 1
+
+# Default schema provider generates random schema
+schema_provider:
+  default: {}
+
+drop_schema: false
+create_schema: true
+truncate_table: false
+
+clock:
+  offset:
+    offset: 1000
+
+run_time: 10
+run_time_unit: "MINUTES"
+
+system_under_test:
+  println: {}
+
+partition_descriptor_selector:
+  always_same:
+    pd: 12345
+
+clustering_descriptor_selector:
+  default:
+    modifications_per_lts:
+      type: "constant"
+      constant: 2
+    rows_per_modification:
+      type: "constant"
+      constant: 2
+    operation_kind_weights:
+      DELETE_RANGE: 1
+      DELETE_SLICE: 1
+      DELETE_ROW: 1
+      DELETE_COLUMN: 1
+      DELETE_PARTITION: 1
+      DELETE_COLUMN_WITH_STATICS: 1
+      INSERT_WITH_STATICS: 24
+      INSERT: 24
+      UPDATE_WITH_STATICS: 23
+      UPDATE: 23
+    column_mask_bitsets: null
+    max_partition_size: 100
+
+data_tracker:
+  no_op: {}
+
+runner:
+  sequential:
+    partition_visitors: []
+
+metric_reporter:
+  no_op: {}
\ No newline at end of file
diff --git a/harry-integration-external/pom.xml b/harry-integration-external/pom.xml
index 8c57e0e..0b49f6d 100755
--- a/harry-integration-external/pom.xml
+++ b/harry-integration-external/pom.xml
@@ -34,6 +34,11 @@
 
     <dependencies>
         <dependency>
+            <groupId>com.datastax.cassandra</groupId>
+            <artifactId>cassandra-driver-core</artifactId>
+        </dependency>
+
+        <dependency>
             <groupId>org.apache.cassandra</groupId>
             <artifactId>harry-core</artifactId>
 	        <version>${project.parent.version}</version>
diff --git a/harry-integration-external/src/harry/model/sut/external/ExternalClusterSut.java b/harry-integration-external/src/harry/model/sut/external/ExternalClusterSut.java
index e471ede..76a4e3c 100644
--- a/harry-integration-external/src/harry/model/sut/external/ExternalClusterSut.java
+++ b/harry-integration-external/src/harry/model/sut/external/ExternalClusterSut.java
@@ -93,9 +93,7 @@ public class ExternalClusterSut implements SystemUnderTest
         {
             try
             {
-                Statement st = new SimpleStatement(statement, bindings);
-                st.setConsistencyLevel(toDriverCl(cl));
-                return resultSetToObjectArray(session.execute(st));
+                return resultSetToObjectArray(session.execute(statement, bindings));
             }
             catch (Throwable t)
             {
@@ -134,9 +132,7 @@ public class ExternalClusterSut implements SystemUnderTest
     public CompletableFuture<Object[][]> executeAsync(String statement, ConsistencyLevel cl, Object... bindings)
     {
         CompletableFuture<Object[][]> future = new CompletableFuture<>();
-        Statement st = new SimpleStatement(statement, bindings);
-        st.setConsistencyLevel(toDriverCl(cl));
-        Futures.addCallback(session.executeAsync(st),
+        Futures.addCallback(session.executeAsync(statement, bindings),
                             new FutureCallback<ResultSet>()
                             {
                                 public void onSuccess(ResultSet rows)
@@ -185,8 +181,10 @@ public class ExternalClusterSut implements SystemUnderTest
     {
         switch (cl)
         {
-            case ALL:    return com.datastax.driver.core.ConsistencyLevel.ALL;
-            case QUORUM: return com.datastax.driver.core.ConsistencyLevel.QUORUM;
+            case ALL:
+                return com.datastax.driver.core.ConsistencyLevel.ALL;
+            case QUORUM:
+                return com.datastax.driver.core.ConsistencyLevel.QUORUM;
         }
         throw new IllegalArgumentException("Don't know a CL: " + cl);
     }
diff --git a/harry-integration/pom.xml b/harry-integration/pom.xml
index 4695d38..0b2d131 100755
--- a/harry-integration/pom.xml
+++ b/harry-integration/pom.xml
@@ -34,9 +34,14 @@
 
     <dependencies>
         <dependency>
+            <groupId>org.reflections</groupId>
+            <artifactId>reflections</artifactId>
+        </dependency>
+
+        <dependency>
             <groupId>org.apache.cassandra</groupId>
             <artifactId>harry-core</artifactId>
-	        <version>${project.parent.version}</version>
+            <version>${project.parent.version}</version>
         </dependency>
 
         <dependency>
@@ -62,5 +67,12 @@
             <scope>test</scope>
         </dependency>
     </dependencies>
+    <build>
+        <testResources>
+            <testResource>
+                <directory>test/resources</directory>
+            </testResource>
+        </testResources>
+    </build>
 </project>
 
diff --git a/harry-integration/src/harry/model/sut/ExternalClusterSut.java b/harry-integration/src/harry/model/sut/ExternalClusterSut.java
deleted file mode 100644
index 74085e4..0000000
--- a/harry-integration/src/harry/model/sut/ExternalClusterSut.java
+++ /dev/null
@@ -1,187 +0,0 @@
-/*
- *  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 harry.model.sut;
-
-import java.util.List;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-
-import com.google.common.util.concurrent.FutureCallback;
-import com.google.common.util.concurrent.Futures;
-
-import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.ColumnDefinitions;
-import com.datastax.driver.core.QueryOptions;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.Row;
-import com.datastax.driver.core.Session;
-import com.datastax.driver.core.SimpleStatement;
-import com.datastax.driver.core.Statement;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-import harry.core.Configuration;
-
-public class ExternalClusterSut implements SystemUnderTest
-{
-    public static void init()
-    {
-        Configuration.registerSubtypes(ExternalClusterSutConfiguration.class);
-    }
-
-    private final Session session;
-    private final ExecutorService executor;
-
-    // TODO: pass cluster, not session
-    public ExternalClusterSut(Session session)
-    {
-        this(session, 10);
-    }
-
-    public ExternalClusterSut(Session session, int threads)
-    {
-        this.session = session;
-        this.executor = Executors.newFixedThreadPool(threads);
-    }
-
-    public static ExternalClusterSut create()
-    {
-        return new ExternalClusterSut(Cluster.builder()
-                                             .withQueryOptions(new QueryOptions().setConsistencyLevel(toDriverCl(ConsistencyLevel.QUORUM)))
-                                             .addContactPoints("127.0.0.1")
-                                             .build()
-                                             .connect());
-    }
-
-    public boolean isShutdown()
-    {
-        return session.isClosed();
-    }
-
-    public void shutdown()
-    {
-        session.close();
-        executor.shutdown();
-        try
-        {
-            executor.awaitTermination(60, TimeUnit.SECONDS);
-        }
-        catch (InterruptedException e)
-        {
-            throw new RuntimeException(e);
-        }
-    }
-
-    // TODO: this is rather simplistic
-    public Object[][] execute(String statement, ConsistencyLevel cl, Object... bindings)
-    {
-        int repeat = 10;
-        while (true)
-        {
-            try
-            {
-                Statement st = new SimpleStatement(statement, bindings);
-                st.setConsistencyLevel(toDriverCl(cl));
-                return resultSetToObjectArray(session.execute(st));
-            }
-            catch (Throwable t)
-            {
-                if (repeat < 0)
-                    throw t;
-
-                t.printStackTrace();
-                repeat--;
-                // retry unconditionally
-            }
-        }
-    }
-
-    public static Object[][] resultSetToObjectArray(ResultSet rs)
-    {
-        List<Row> rows = rs.all();
-        if (rows.size() == 0)
-            return new Object[0][];
-        Object[][] results = new Object[rows.size()][];
-        for (int i = 0; i < results.length; i++)
-        {
-            Row row = rows.get(i);
-            ColumnDefinitions cds = row.getColumnDefinitions();
-            Object[] result = new Object[cds.size()];
-            for (int j = 0; j < cds.size(); j++)
-            {
-                if (!row.isNull(j))
-                    result[j] = row.getObject(j);
-            }
-            results[i] = result;
-        }
-        return results;
-    }
-
-    public CompletableFuture<Object[][]> executeAsync(String statement, ConsistencyLevel cl, Object... bindings)
-    {
-        CompletableFuture<Object[][]> future = new CompletableFuture<>();
-        Statement st = new SimpleStatement(statement, bindings);
-        st.setConsistencyLevel(toDriverCl(cl));
-        Futures.addCallback(session.executeAsync(st),
-                            new FutureCallback<ResultSet>()
-                            {
-                                public void onSuccess(ResultSet rows)
-                                {
-                                    future.complete(resultSetToObjectArray(rows));
-                                }
-
-                                public void onFailure(Throwable throwable)
-                                {
-                                    future.completeExceptionally(throwable);
-                                }
-                            },
-                            executor);
-
-        return future;
-    }
-
-    public static com.datastax.driver.core.ConsistencyLevel toDriverCl(SystemUnderTest.ConsistencyLevel cl)
-    {
-        switch (cl)
-        {
-            case ALL:    return com.datastax.driver.core.ConsistencyLevel.ALL;
-            case QUORUM: return com.datastax.driver.core.ConsistencyLevel.QUORUM;
-        }
-        throw new IllegalArgumentException("Don't know a CL: " + cl);
-    }
-
-    @JsonTypeName("external")
-    public static class ExternalClusterSutConfiguration implements Configuration.SutConfiguration
-    {
-        public final String[] hosts;
-
-        public ExternalClusterSutConfiguration(@JsonProperty(value = "hosts") String[] hosts)
-        {
-            this.hosts = hosts;
-        }
-
-        public SystemUnderTest make()
-        {
-            Cluster cluster = Cluster.builder().addContactPoints(hosts).build();
-            Session session = cluster.newSession().init();
-            return new ExternalClusterSut(session);
-        }
-    }
-}
\ No newline at end of file
diff --git a/harry-integration/src/harry/model/sut/InJvmSutBase.java b/harry-integration/src/harry/model/sut/InJvmSutBase.java
index e29481f..aa2608f 100644
--- a/harry-integration/src/harry/model/sut/InJvmSutBase.java
+++ b/harry-integration/src/harry/model/sut/InJvmSutBase.java
@@ -30,20 +30,19 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.function.Consumer;
 
-import com.google.common.collect.Iterators;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import harry.core.Configuration;
-import org.apache.cassandra.distributed.api.ConsistencyLevel;
 import org.apache.cassandra.distributed.api.Feature;
 import org.apache.cassandra.distributed.api.ICluster;
 import org.apache.cassandra.distributed.api.IInstance;
 import org.apache.cassandra.distributed.api.IInstanceConfig;
 import org.apache.cassandra.distributed.api.IMessage;
 import org.apache.cassandra.distributed.api.IMessageFilters;
+import relocated.shaded.com.google.common.collect.Iterators;
 
 public class InJvmSutBase<NODE extends IInstance, CLUSTER extends ICluster<NODE>> implements SystemUnderTest.FaultInjectingSut
 {
diff --git a/harry-integration/src/harry/runner/HarryRunnerJvm.java b/harry-integration/src/harry/runner/HarryRunnerJvm.java
new file mode 100644
index 0000000..fbb30b9
--- /dev/null
+++ b/harry-integration/src/harry/runner/HarryRunnerJvm.java
@@ -0,0 +1,43 @@
+/*
+ *  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 harry.runner;
+
+import harry.core.Configuration;
+import harry.model.sut.InJvmSut;
+
+import java.io.File;
+
+public class HarryRunnerJvm extends HarryRunner {
+
+    public static void main(String[] args) throws Throwable {
+        InJvmSut.init();
+
+        HarryRunnerJvm runner = new HarryRunnerJvm();
+        File configFile = runner.loadConfig(args);
+
+        Configuration configuration = Configuration.fromFile(configFile);
+        runner.run(configuration);
+    }
+
+
+    @Override
+    public void beforeRun(Runner runner) {
+
+    }
+}
diff --git a/harry-integration/test/conf/cassandra.yaml b/harry-integration/test/conf/cassandra.yaml
new file mode 100644
index 0000000..2536aa8
--- /dev/null
+++ b/harry-integration/test/conf/cassandra.yaml
@@ -0,0 +1,44 @@
+#
+# Warning!
+# Consider the effects on 'o.a.c.i.s.LegacySSTableTest' before changing schemas in this file.
+#
+cluster_name: Test Cluster
+memtable_allocation_type: heap_buffers
+commitlog_sync: batch
+commitlog_sync_batch_window_in_ms: 1.0
+commitlog_segment_size_in_mb: 5
+commitlog_directory: build/test/cassandra/commitlog
+hints_directory: build/test/cassandra/hints
+partitioner: org.apache.cassandra.dht.ByteOrderedPartitioner
+listen_address: 127.0.0.1
+storage_port: 7010
+rpc_port: 9170
+start_native_transport: true
+native_transport_port: 9042
+column_index_size_in_kb: 4
+saved_caches_directory: build/test/cassandra/saved_caches
+data_file_directories:
+    - build/test/cassandra/data
+disk_access_mode: mmap
+seed_provider:
+    - class_name: org.apache.cassandra.locator.SimpleSeedProvider
+      parameters:
+          - seeds: "127.0.0.1"
+endpoint_snitch: org.apache.cassandra.locator.SimpleSnitch
+dynamic_snitch: true
+request_scheduler: org.apache.cassandra.scheduler.RoundRobinScheduler
+request_scheduler_id: keyspace
+server_encryption_options:
+    internode_encryption: none
+    keystore: conf/.keystore
+    keystore_password: cassandra
+    truststore: conf/.truststore
+    truststore_password: cassandra
+incremental_backups: true
+concurrent_compactors: 4
+compaction_throughput_mb_per_sec: 0
+row_cache_class_name: org.apache.cassandra.cache.OHCProvider
+row_cache_size_in_mb: 16
+enable_user_defined_functions: true
+enable_scripted_user_defined_functions: true
+enable_drop_compact_storage: true
diff --git a/harry-integration/test/harry/ddl/SchemaGenTest.java b/harry-integration/test/harry/ddl/SchemaGenTest.java
index 74a9c53..2f7e898 100644
--- a/harry-integration/test/harry/ddl/SchemaGenTest.java
+++ b/harry-integration/test/harry/ddl/SchemaGenTest.java
@@ -50,11 +50,12 @@ public class SchemaGenTest extends CQLTester
 
     // TODO: compact storage tests
     @Test
-    public void testSelectForwardAndReverseIteration()
+    public void testSelectForwardAndReverseIteration() throws Throwable
     {
         Generator<SchemaSpec> gen = new SchemaGenerators.Builder(KEYSPACE).partitionKeyColumnCount(1, 4)
                                                                           .clusteringColumnCount(1, 10)
                                                                           .regularColumnCount(0, 10)
+                                                                          .staticColumnCount(0, 10)
                                                                           .generator();
 
 
@@ -78,11 +79,12 @@ public class SchemaGenTest extends CQLTester
     }
 
     @Test
-    public void createTableRoundTrip()
+    public void createTableRoundTrip() throws Throwable
     {
         Generator<SchemaSpec> gen = new SchemaGenerators.Builder(KEYSPACE).partitionKeyColumnCount(1, 10)
-                                                                          .clusteringColumnCount(0, 10)
+                                                                          .clusteringColumnCount(1, 10)
                                                                           .regularColumnCount(0, 10)
+                                                                          .staticColumnCount(0, 10)
                                                                           .generator();
 
         TestRunner.test(gen,
@@ -93,6 +95,7 @@ public class SchemaGenTest extends CQLTester
                             compareColumns(schemaDefinition.partitionKeys, tableMetadata.partitionKeyColumns());
                             compareColumns(schemaDefinition.clusteringKeys, tableMetadata.clusteringColumns());
                             compareColumns(schemaDefinition.regularColumns, tableMetadata.regularColumns());
+                            compareColumns(schemaDefinition.staticColumns, tableMetadata.staticColumns());
                         });
     }
 
@@ -108,8 +111,8 @@ public class SchemaGenTest extends CQLTester
                                                        ColumnSpec.regularColumn("v2", ColumnSpec.asciiType),
                                                        ColumnSpec.regularColumn("v3", ColumnSpec.int64Type),
                                                        ColumnSpec.regularColumn("v4", ColumnSpec.int64Type)),
-                                         Arrays.asList(ColumnSpec.staticColumn("regular1", ColumnSpec.asciiType),
-                                                       ColumnSpec.staticColumn("regular2", ColumnSpec.int64Type)));
+                                         Arrays.asList(ColumnSpec.staticColumn("static1", ColumnSpec.asciiType),
+                                                       ColumnSpec.staticColumn("static2", ColumnSpec.int64Type)));
 
 
         String tableDef = spec.compile().cql();
@@ -118,6 +121,7 @@ public class SchemaGenTest extends CQLTester
         compareColumns(spec.partitionKeys, tableMetadata.partitionKeyColumns());
         compareColumns(spec.clusteringKeys, tableMetadata.clusteringColumns());
         compareColumns(spec.regularColumns, tableMetadata.regularColumns());
+        compareColumns(spec.staticColumns, tableMetadata.staticColumns());
     }
 
 
@@ -126,6 +130,7 @@ public class SchemaGenTest extends CQLTester
     {
         Gen<Pair<Integer, Integer>> ckCounts = integers().between(0, 4).zip(integers().between(0, 6), Pair::create);
         Gen<Pair<Integer, Integer>> regCounts = integers().between(0, 4).zip(integers().between(0, 6), Pair::create);
+//        Gen<Pair<Integer, Integer>> staticCounts = integers().between(0, 4).zip(integers().between(0, 6), Pair::create);
         Gen<Pair<Integer, Integer>> pkCounts = integers().between(1, 4).zip(integers().between(0, 6), Pair::create);
 
         Gen<SchemaGenerationInputs> inputs = pkCounts.zip(ckCounts, regCounts,
diff --git a/harry-integration/test/harry/generators/DataGeneratorsIntegrationTest.java b/harry-integration/test/harry/generators/DataGeneratorsIntegrationTest.java
index affd954..cadee0c 100644
--- a/harry-integration/test/harry/generators/DataGeneratorsIntegrationTest.java
+++ b/harry-integration/test/harry/generators/DataGeneratorsIntegrationTest.java
@@ -19,11 +19,28 @@
 package harry.generators;
 
 import java.util.Random;
+import java.util.concurrent.CompletableFuture;
 
 import org.junit.Test;
 
+import harry.core.Configuration;
+import harry.core.Run;
 import harry.ddl.ColumnSpec;
+import harry.ddl.SchemaGenerators;
+import harry.ddl.SchemaSpec;
+import harry.generators.distribution.Distribution;
+import harry.model.NoOpChecker;
+import harry.model.OpSelectors;
+import harry.model.sut.SystemUnderTest;
+import harry.runner.MutatingPartitionVisitor;
+import harry.runner.MutatingRowVisitor;
+import harry.runner.PartitionVisitor;
+import harry.runner.SinglePartitionValidator;
+import harry.util.TestRunner;
 import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.distributed.impl.RowUtil;
+import relocated.shaded.com.google.common.collect.Iterators;
 
 public class DataGeneratorsIntegrationTest extends CQLTester
 {
@@ -60,4 +77,91 @@ public class DataGeneratorsIntegrationTest extends CQLTester
             }
         }
     }
+
+    @Test
+    public void queryParseabilityTest() throws Throwable
+    {
+        Generator<SchemaSpec> gen = new SchemaGenerators.Builder(KEYSPACE).partitionKeyColumnCount(2, 4)
+                                                                          .clusteringColumnCount(1, 4)
+                                                                          .regularColumnCount(1, 4)
+                                                                          .staticColumnCount(1, 4)
+                                                                          .generator();
+
+        TestRunner.test(gen,
+                        (schema) -> {
+                            createTable(schema.compile().cql());
+
+                            Configuration.ConfigurationBuilder builder = Configuration.fromFile(getClass().getClassLoader().getResource("single_partition_test.yml").getFile())
+                                         .unbuild()
+                                         .setSchemaProvider(new Configuration.FixedSchemaProviderConfiguration(schema, null, null, null, null))
+                                         .setSUT(CqlTesterSut::new);
+
+                            for (OpSelectors.OperationKind opKind : OpSelectors.OperationKind.values())
+                            {
+                                Run run = builder
+                                          .setClusteringDescriptorSelector((rng, schema_) -> {
+                                              return new OpSelectors.DefaultDescriptorSelector(rng,
+                                                                                               OpSelectors.columnSelectorBuilder().forAll(schema_).build(),
+                                                                                               OpSelectors.OperationSelector.weighted(Surjections.weights(100), opKind),
+                                                                                               new Distribution.ConstantDistribution(2),
+                                                                                               new Distribution.ConstantDistribution(2),
+                                                                                               100);
+                                          })
+                                          .build()
+                                          .createRun();
+
+                                PartitionVisitor visitor = new MutatingPartitionVisitor(run, MutatingRowVisitor::new);
+                                for (int lts = 0; lts < 100; lts++)
+                                    visitor.visitPartition(lts);
+                            }
+
+                            Run run = builder.build()
+                                             .createRun();
+                            PartitionVisitor visitor = new SinglePartitionValidator(100, run, NoOpChecker::new);
+                            for (int lts = 0; lts < 100; lts++)
+                                visitor.visitPartition(lts);
+
+                        });
+
+    }
+
+    public class CqlTesterSut implements SystemUnderTest
+    {
+        public boolean isShutdown()
+        {
+            return false;
+        }
+
+        public void shutdown()
+        {
+            cleanup();
+        }
+
+        public void schemaChange(String statement)
+        {
+            createTable(statement);
+        }
+
+        public Object[][] execute(String statement, ConsistencyLevel cl, Object... bindings)
+        {
+            try
+            {
+                UntypedResultSet res = DataGeneratorsIntegrationTest.this.execute(statement, bindings);
+                if (res == null)
+                    return new Object[][] {};
+
+                return Iterators.toArray(RowUtil.toIter(res), Object[].class);
+            }
+            catch (Throwable throwable)
+            {
+                throw new RuntimeException(throwable);
+            }
+        }
+
+        public CompletableFuture<Object[][]> executeAsync(String statement, ConsistencyLevel cl, Object... bindings)
+        {
+            return CompletableFuture.completedFuture(execute(statement, cl, bindings));
+        }
+    }
 }
+
diff --git a/harry-integration/test/harry/model/IntegrationTestBase.java b/harry-integration/test/harry/model/IntegrationTestBase.java
index 551124f..a6a9697 100644
--- a/harry-integration/test/harry/model/IntegrationTestBase.java
+++ b/harry-integration/test/harry/model/IntegrationTestBase.java
@@ -81,8 +81,10 @@ public class IntegrationTestBase extends TestBaseImpl
                                         .addWeight(OpSelectors.OperationKind.DELETE_SLICE, 1)
                                         .addWeight(OpSelectors.OperationKind.DELETE_PARTITION, 1)
                                         .addWeight(OpSelectors.OperationKind.DELETE_COLUMN_WITH_STATICS, 5)
-                                        .addWeight(OpSelectors.OperationKind.WRITE_WITH_STATICS, 45)
-                                        .addWeight(OpSelectors.OperationKind.WRITE, 45)
+                                        .addWeight(OpSelectors.OperationKind.INSERT_WITH_STATICS, 20)
+                                        .addWeight(OpSelectors.OperationKind.INSERT, 20)
+                                        .addWeight(OpSelectors.OperationKind.UPDATE_WITH_STATICS, 25)
+                                        .addWeight(OpSelectors.OperationKind.UPDATE, 25)
                                         .build());
     }
 
diff --git a/harry-integration/test/harry/model/QuiescentCheckerIntegrationTest.java b/harry-integration/test/harry/model/QuiescentCheckerIntegrationTest.java
index 4c83276..7d5956f 100644
--- a/harry-integration/test/harry/model/QuiescentCheckerIntegrationTest.java
+++ b/harry-integration/test/harry/model/QuiescentCheckerIntegrationTest.java
@@ -128,7 +128,8 @@ public class QuiescentCheckerIntegrationTest extends ModelTestBase
                      (t, run) -> {
                          String expected = "doesn't match the one predicted by the model";
                          String expected2 = "don't match ones predicted by the model";
-                         if (t.getMessage().contains(expected) || t.getMessage().contains(expected2))
+                         String expected3 = "Found a row in the model that is not present in the resultset";
+                         if (t.getMessage().contains(expected) || t.getMessage().contains(expected2) || t.getMessage().contains(expected3))
                              return;
 
                          throw new AssertionError(String.format("Exception string mismatch.\nExpected error: %s.\nActual error: %s", expected, t.getMessage()),
diff --git a/harry-integration/test/harry/model/TestEveryClustering.java b/harry-integration/test/harry/model/TestEveryClustering.java
new file mode 100644
index 0000000..f844b41
--- /dev/null
+++ b/harry-integration/test/harry/model/TestEveryClustering.java
@@ -0,0 +1,89 @@
+package harry.model;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.function.Supplier;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import harry.core.Configuration;
+import harry.core.Run;
+import harry.ddl.SchemaGenerators;
+import harry.ddl.SchemaSpec;
+import harry.generators.distribution.Distribution;
+import harry.model.sut.SystemUnderTest;
+import harry.operations.CompiledStatement;
+import harry.operations.Relation;
+import harry.runner.FaultInjectingPartitionVisitor;
+import harry.runner.LoggingPartitionVisitor;
+import harry.runner.MutatingPartitionVisitor;
+import harry.runner.MutatingRowVisitor;
+import harry.runner.PartitionVisitor;
+import harry.runner.Query;
+import harry.runner.QueryGenerator;
+import org.apache.cassandra.distributed.api.IInvokableInstance;
+
+public class TestEveryClustering extends IntegrationTestBase
+{
+    int CYCLES = 1000;
+
+    @Test
+    public void basicQuerySelectorTest()
+    {
+        Supplier<SchemaSpec> schemaGen = SchemaGenerators.progression(SchemaGenerators.DEFAULT_SWITCH_AFTER);
+        for (int cnt = 0; cnt < Integer.MAX_VALUE; cnt++)
+        {
+            beforeEach();
+            SchemaSpec schemaSpec = schemaGen.get();
+
+            System.out.println(schemaSpec.compile().cql());
+            int partitionSize = 1000;
+
+            Configuration config = sharedConfiguration(cnt, schemaSpec)
+                                   .setPartitionDescriptorSelector(new Configuration.DefaultPDSelectorConfiguration(1, partitionSize))
+                                   .setClusteringDescriptorSelector(sharedCDSelectorConfiguration()
+                                                                    .setNumberOfModificationsDistribution(() -> new Distribution.ConstantDistribution(1L))
+                                                                    .setRowsPerModificationDistribution(() -> new Distribution.ConstantDistribution(1L))
+                                                                    .setMaxPartitionSize(250)
+                                                                    .build())
+                                   .build();
+
+            Run run = config.createRun();
+            run.sut.schemaChange(run.schemaSpec.compile().cql());
+            OpSelectors.MonotonicClock clock = run.clock;
+
+            Set<Long> visitedCds = new HashSet<>();
+            PartitionVisitor partitionVisitor = new LoggingPartitionVisitor(run, (r) -> {
+                return new MutatingRowVisitor(r) {
+                    public CompiledStatement perform(OpSelectors.OperationKind op, long lts, long pd, long cd, long opId)
+                    {
+                        visitedCds.add(cd);
+                        return super.perform(op, lts, pd, cd, opId);
+                    }
+                };
+            });
+            sut.cluster().stream().forEach((IInvokableInstance node) -> node.nodetool("disableautocompaction"));
+            for (int i = 0; i < CYCLES; i++)
+            {
+                long lts = clock.nextLts();
+                partitionVisitor.visitPartition(lts);
+
+                if (i > 0 && i % 250 == 0)
+                    sut.cluster().stream().forEach((IInvokableInstance node) -> node.nodetool("flush", schemaSpec.keyspace, schemaSpec.table));
+            }
+
+            for (Long cd : visitedCds)
+            {
+                Query query = new Query.SingleClusteringQuery(Query.QueryKind.SINGLE_CLUSTERING,
+                                                              run.pdSelector.pd(0),
+                                                              cd,
+                                                              false,
+                                                              Relation.eqRelations(run.schemaSpec.ckGenerator.slice(cd), run.schemaSpec.clusteringKeys),
+                                                              run.schemaSpec);
+                Model model = new QuiescentChecker(run);
+                model.validate(query);
+            }
+        }
+    }
+}
diff --git a/harry-integration/test/harry/op/RowVisitorTest.java b/harry-integration/test/harry/op/RowVisitorTest.java
index d7061f9..ae51115 100644
--- a/harry-integration/test/harry/op/RowVisitorTest.java
+++ b/harry-integration/test/harry/op/RowVisitorTest.java
@@ -83,10 +83,10 @@ public class RowVisitorTest extends CQLTester
             MutatingRowVisitor visitor = new MutatingRowVisitor(run);
             long[] descriptors = rand.next(4);
 
-            execute(visitor.write(Math.abs(descriptors[0]),
-                                  descriptors[1],
-                                  descriptors[2],
-                                  descriptors[3]));
+            execute(visitor.insert(Math.abs(descriptors[0]),
+                                   descriptors[1],
+                                   descriptors[2],
+                                   descriptors[3]));
         }
     }
 
diff --git a/harry-integration/test/resources/single_partition_test.yml b/harry-integration/test/resources/single_partition_test.yml
new file mode 100644
index 0000000..0ebe2aa
--- /dev/null
+++ b/harry-integration/test/resources/single_partition_test.yml
@@ -0,0 +1,55 @@
+seed: 1
+
+# Default schema provider generates random schema
+schema_provider:
+  default: {}
+
+drop_schema: false
+create_schema: true
+truncate_table: false
+
+clock:
+  offset:
+    offset: 1000
+
+run_time: 10
+run_time_unit: "MINUTES"
+
+system_under_test:
+  println: {}
+
+partition_descriptor_selector:
+  always_same:
+    pd: 12345
+
+clustering_descriptor_selector:
+  default:
+    modifications_per_lts:
+      type: "constant"
+      constant: 2
+    rows_per_modification:
+      type: "constant"
+      constant: 2
+    operation_kind_weights:
+      DELETE_RANGE: 1
+      DELETE_SLICE: 1
+      DELETE_ROW: 1
+      DELETE_COLUMN: 1
+      DELETE_PARTITION: 1
+      DELETE_COLUMN_WITH_STATICS: 1
+      INSERT_WITH_STATICS: 24
+      INSERT: 24
+      UPDATE_WITH_STATICS: 23
+      UPDATE: 23
+    column_mask_bitsets: null
+    max_partition_size: 100
+
+data_tracker:
+  no_op: {}
+
+runner:
+  sequential:
+    partition_visitors: []
+
+metric_reporter:
+  no_op: {}
\ No newline at end of file

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