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/12 15:55:27 UTC

[cassandra-harry] 02/02: Improvements: * Timestamp tie resolution * Test validating behavior during upgrades (mixed version clusters) * Improve recent partition validator * Add number of queries per partition * Make it easier run jobs from a local machine * Make string postfixes more random by avoiding reusing nibbles * Improve weight distributions * Allow clustering keys with less entropy than 8 bytes Notable bugfixes: * Fix a problem with generation of non-eq bounds following reverse columns * Fix a pr [...]

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

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

commit 5fa48c2bb678d7519b7a7b22615de88e5c068eec
Author: Alex Petrov <ol...@gmail.com>
AuthorDate: Thu Apr 22 12:09:28 2021 +0200

    Improvements:
     * Timestamp tie resolution
     * Test validating behavior during upgrades (mixed version clusters)
     * Improve recent partition validator
     * Add number of queries per partition
     * Make it easier run jobs from a local machine
     * Make string postfixes more random by avoiding reusing nibbles
     * Improve weight distributions
     * Allow clustering keys with less entropy than 8 bytes
    Notable bugfixes:
     * Fix a problem with generation of non-eq bounds following reverse columns
     * Fix a problem with empty static clusterings for single-column tables
    New features:
     * Add a trivial shrinker
     * Add a parallel recent partition validator
     * Add sampler
     * Quiescent Checker: Implement Statics
     * Add mixed-mode in-jvm SUT
     * Add validation log
    
    Numerous quality-of-life, debugging improvements, bugfixes, and cosmetic changes Several useful example config files. Exhaustive checker was removed in favour of quickly extending functionality.
    
    Patch by Alex Petrov for CASSANDRA-16262.
---
 harry-core/pom.xml                                 |   2 +-
 harry-core/src/harry/core/Configuration.java       | 137 +++--
 harry-core/src/harry/core/MetricReporter.java      |   2 +
 .../src/harry/corruptor/AddExtraRowCorruptor.java  |   2 +-
 .../src/harry/corruptor/ChangeValueCorruptor.java  |   6 +-
 .../src/harry/corruptor/HideValueCorruptor.java    |  31 +-
 .../harry/corruptor/QueryResponseCorruptor.java    |  26 +-
 harry-core/src/harry/corruptor/RowCorruptor.java   |   6 +
 .../src/harry/corruptor/ShowValueCorruptor.java    |   2 +-
 harry-core/src/harry/data/ResultSetRow.java        |  33 +-
 harry-core/src/harry/ddl/ColumnSpec.java           |  61 +-
 harry-core/src/harry/ddl/SchemaGenerators.java     | 123 ++--
 harry-core/src/harry/ddl/SchemaSpec.java           |  90 ++-
 harry-core/src/harry/generators/Bijections.java    |  14 +-
 .../src/harry/generators/DataGenerators.java       |  41 +-
 harry-core/src/harry/generators/RngUtils.java      |  46 ++
 .../src/harry/generators/StringBijection.java      |  26 +-
 harry-core/src/harry/generators/Surjections.java   |   8 +-
 harry-core/src/harry/model/DataTracker.java        | 123 ----
 .../src/harry/model/DescriptorSelectorBuilder.java |  22 +-
 harry-core/src/harry/model/ExhaustiveChecker.java  | 618 ---------------------
 harry-core/src/harry/model/Model.java              |  17 +-
 .../{DoNothingModel.java => NoOpChecker.java}      |  15 +-
 harry-core/src/harry/model/OpSelectors.java        | 274 +++++++--
 harry-core/src/harry/model/QuiescentChecker.java   | 134 ++++-
 harry-core/src/harry/model/SelectHelper.java       |  33 +-
 .../harry/model/StatelessVisibleRowsChecker.java   | 114 ----
 harry-core/src/harry/model/VisibleRowsChecker.java | 281 ----------
 .../src/harry/model/sut/SystemUnderTest.java       |  12 +-
 .../src/harry/operations/CompiledStatement.java    |  29 +-
 harry-core/src/harry/operations/DeleteHelper.java  |  80 ++-
 harry-core/src/harry/operations/Relation.java      |  20 +-
 harry-core/src/harry/operations/WriteHelper.java   |  16 +
 harry-core/src/harry/reconciler/Reconciler.java    | 380 ++++++++++---
 .../src/harry/runner/AbstractPartitionVisitor.java |  14 -
 .../src/harry/runner/AllPartitionsValidator.java   |  49 +-
 .../harry/runner/CorruptingPartitionVisitor.java   |  93 ++++
 harry-core/src/harry/runner/HarryRunner.java       |  64 +--
 .../src/harry/runner/LoggingPartitionVisitor.java  |  23 +-
 .../src/harry/runner/MutatingPartitionVisitor.java |  19 +-
 .../src/harry/runner/MutatingRowVisitor.java       |  30 +-
 .../runner/{RowVisitor.java => Operation.java}     |  18 +-
 .../runner/ParallelRecentPartitionValidator.java   | 170 ++++++
 harry-core/src/harry/runner/ParallelValidator.java | 112 ++++
 harry-core/src/harry/runner/Query.java             |   8 +-
 harry-core/src/harry/runner/QueryGenerator.java    |  87 ++-
 harry-core/src/harry/runner/QuerySelector.java     | 300 ----------
 .../src/harry/runner/RecentPartitionValidator.java |  74 ++-
 harry-core/src/harry/runner/Runner.java            |  57 +-
 harry-core/src/harry/runner/Sampler.java           | 110 ++++
 .../src/harry/runner/SinglePartitionValidator.java |   8 +-
 harry-core/src/harry/util/BitSet.java              |  56 +-
 .../test/harry/generators/RandomGeneratorTest.java |  27 +
 harry-core/test/harry/model/OpSelectorsTest.java   | 125 ++++-
 harry-core/test/harry/operations/RelationTest.java |   1 +
 .../src/harry/model/sut/ExternalClusterSut.java    |  28 +-
 .../src/harry/model/sut/InJvmSut.java              | 199 +------
 .../model/sut/{InJvmSut.java => InJvmSutBase.java} | 135 +++--
 .../src/harry/model/sut/MixedVersionInJvmSut.java  | 138 +++++
 .../runner/FaultInjectingPartitionVisitor.java     |   7 +-
 harry-integration/src/harry/runner/Reproduce.java  |  61 +-
 .../src/harry/runner/TrivialShrinker.java          | 232 ++++++++
 .../test/harry/ddl/SchemaGenTest.java              |   4 +-
 .../generators/DataGeneratorsIntegrationTest.java  |  63 +++
 .../model/ExhaustiveCheckerIntegrationTest.java    | 235 --------
 .../harry/model/ExhaustiveCheckerUnitTest.java     | 171 ------
 .../test/harry/model/IntegrationTestBase.java      |  34 +-
 harry-integration/test/harry/model/MockSchema.java |   9 +-
 harry-integration/test/harry/model/ModelTest.java  | 159 ------
 .../test/harry/model/ModelTestBase.java            |  67 ++-
 .../harry/model/QuerySelectorNegativeTest.java     |  19 +-
 .../test/harry/model/QuerySelectorTest.java        |  33 +-
 .../model/QuiescentCheckerIntegrationTest.java     |  55 +-
 .../test/harry/op/RowVisitorTest.java              |  28 +-
 pom.xml                                            |  23 +-
 75 files changed, 3058 insertions(+), 2911 deletions(-)

diff --git a/harry-core/pom.xml b/harry-core/pom.xml
index 6ca4ef0..fef070d 100755
--- a/harry-core/pom.xml
+++ b/harry-core/pom.xml
@@ -25,8 +25,8 @@
 
     <parent>
         <groupId>org.apache.cassandra</groupId>
-        <version>0.0.1-SNAPSHOT</version>
         <artifactId>harry-parent</artifactId>
+        <version>0.0.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>harry-core</artifactId>
diff --git a/harry-core/src/harry/core/Configuration.java b/harry-core/src/harry/core/Configuration.java
index 27ee176..bd011c2 100644
--- a/harry-core/src/harry/core/Configuration.java
+++ b/harry-core/src/harry/core/Configuration.java
@@ -26,7 +26,6 @@ import java.util.Map;
 import java.util.Objects;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Consumer;
-import java.util.function.Function;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
@@ -39,22 +38,24 @@ import harry.ddl.SchemaGenerators;
 import harry.ddl.SchemaSpec;
 import harry.generators.Surjections;
 import harry.generators.distribution.Distribution;
-import harry.model.ExhaustiveChecker;
 import harry.model.Model;
 import harry.model.OpSelectors;
 import harry.model.QuiescentChecker;
 import harry.model.clock.ApproximateMonotonicClock;
 import harry.model.sut.SystemUnderTest;
 import harry.runner.AllPartitionsValidator;
+import harry.runner.CorruptingPartitionVisitor;
 import harry.runner.DataTracker;
 import harry.runner.DefaultDataTracker;
 import harry.runner.LoggingPartitionVisitor;
 import harry.runner.MutatingPartitionVisitor;
 import harry.runner.MutatingRowVisitor;
+import harry.runner.Operation;
+import harry.runner.ParallelRecentPartitionValidator;
 import harry.runner.PartitionVisitor;
 import harry.runner.RecentPartitionValidator;
-import harry.runner.RowVisitor;
 import harry.runner.Runner;
+import harry.runner.Sampler;
 import harry.util.BitSet;
 
 public class Configuration
@@ -73,9 +74,10 @@ public class Configuration
         mapper.registerSubtypes(Configuration.ConcurrentRunnerConfig.class);
         mapper.registerSubtypes(Configuration.SequentialRunnerConfig.class);
         mapper.registerSubtypes(Configuration.DefaultDataTrackerConfiguration.class);
+        mapper.registerSubtypes(Configuration.NoOpDataTrackerConfiguration.class);
 
-        mapper.registerSubtypes(Configuration.ExhaustiveCheckerConfig.class);
         mapper.registerSubtypes(Configuration.QuiescentCheckerConfig.class);
+        mapper.registerSubtypes(NoOpCheckerConfig.class);
         mapper.registerSubtypes(Configuration.DefaultCDSelectorConfiguration.class);
         mapper.registerSubtypes(Configuration.DefaultPDSelectorConfiguration.class);
         mapper.registerSubtypes(Configuration.ConstantDistributionConfig.class);
@@ -85,6 +87,9 @@ public class Configuration
         mapper.registerSubtypes(MutatingPartitionVisitorConfiguation.class);
         mapper.registerSubtypes(LoggingPartitionVisitorConfiguration.class);
         mapper.registerSubtypes(AllPartitionsValidatorConfiguration.class);
+        mapper.registerSubtypes(ParallelRecentPartitionValidator.ParallelRecentPartitionValidatorConfig.class);
+        mapper.registerSubtypes(Sampler.SamplerConfiguration.class);
+        mapper.registerSubtypes(CorruptingPartitionVisitorConfiguration.class);
         mapper.registerSubtypes(RecentPartitionsValidatorConfiguration.class);
         mapper.registerSubtypes(FixedSchemaProviderConfiguration.class);
     }
@@ -226,7 +231,7 @@ public class Configuration
 
         OpSelectors.PdSelector pdSelector = snapshot.partition_descriptor_selector.make(rng);
         OpSelectors.DescriptorSelector descriptorSelector = snapshot.clustering_descriptor_selector.make(rng, schemaSpec);
-
+        // TODO: validate that operation kind is compactible with schema, due to statics etc
         SystemUnderTest sut = snapshot.system_under_test.make();
 
         return new Run(rng,
@@ -402,6 +407,46 @@ public class Configuration
 
     }
 
+    @JsonTypeName("no_op_tracker")
+    public static class NoOpDataTrackerConfiguration implements DataTrackerConfiguration
+    {
+        @JsonCreator
+        public NoOpDataTrackerConfiguration()
+        {
+        }
+
+        public DataTracker make()
+        {
+            return new DataTracker()
+            {
+                public void started(long lts)
+                {
+                }
+
+                public void finished(long lts)
+                {
+
+                }
+
+                public long maxStarted()
+                {
+                    return 0;
+                }
+
+                public long maxConsecutiveFinished()
+                {
+                    return 0;
+                }
+
+                public DataTrackerConfiguration toConfig()
+                {
+                    return null;
+                }
+            };
+        }
+    }
+
+
     @JsonTypeName("default")
     public static class DefaultDataTrackerConfiguration implements DataTrackerConfiguration
     {
@@ -554,32 +599,31 @@ public class Configuration
     {
     }
 
-    @JsonTypeName("exhaustive_checker")
-    public static class ExhaustiveCheckerConfig implements ModelConfiguration
+    @JsonTypeName("quiescent_checker")
+    public static class QuiescentCheckerConfig implements ModelConfiguration
     {
         @JsonCreator
-        public ExhaustiveCheckerConfig()
+        public QuiescentCheckerConfig()
         {
-
         }
 
         public Model make(Run run)
         {
-            return new ExhaustiveChecker(run);
+            return new QuiescentChecker(run);
         }
     }
 
-    @JsonTypeName("quiescent_checker")
-    public static class QuiescentCheckerConfig implements ModelConfiguration
+    @JsonTypeName("no_op_checker")
+    public static class NoOpCheckerConfig implements ModelConfiguration
     {
         @JsonCreator
-        public QuiescentCheckerConfig()
+        public NoOpCheckerConfig()
         {
         }
 
         public Model make(Run run)
         {
-            return new QuiescentChecker(run);
+            return new harry.model.NoOpChecker(run);
         }
     }
 
@@ -685,9 +729,10 @@ public class Configuration
             return this;
         }
 
-        public void setFractions(int[] fractions)
+        public CDSelectorConfigurationBuilder setFractions(int[] fractions)
         {
             this.fractions = fractions;
+            return this;
         }
 
         public DefaultCDSelectorConfiguration build()
@@ -735,12 +780,12 @@ public class Configuration
             this.column_mask_bitsets = column_mask_bitsets;
         }
 
-        protected Function<OpSelectors.OperationKind, Surjections.Surjection<BitSet>> columnSelector(SchemaSpec schemaSpec)
+        protected OpSelectors.ColumnSelector columnSelector(SchemaSpec schemaSpec)
         {
-            Function<OpSelectors.OperationKind, Surjections.Surjection<BitSet>> columnSelector;
+            OpSelectors.ColumnSelector columnSelector;
             if (column_mask_bitsets == null)
             {
-                columnSelector = OpSelectors.columnSelectorBuilder().forAll(schemaSpec.regularColumns.size()).build();
+                columnSelector = OpSelectors.columnSelectorBuilder().forAll(schemaSpec).build();
             }
             else
             {
@@ -749,11 +794,13 @@ public class Configuration
                 {
                     List<BitSet> bitSets = new ArrayList<>(entry.getValue().length);
                     for (long raw_bitset : entry.getValue())
-                        bitSets.add(BitSet.create(raw_bitset, schemaSpec.regularColumns.size()));
+                    {
+                        bitSets.add(BitSet.create(raw_bitset, schemaSpec.allColumns.size()));
+                    }
                     Surjections.Surjection<BitSet> selector = Surjections.pick(bitSets);
                     m.put(entry.getKey(), selector);
                 }
-                columnSelector = m::get;
+                columnSelector = (opKind, descr) -> m.get(opKind).inflate(descr);
             }
 
             return columnSelector;
@@ -763,7 +810,7 @@ public class Configuration
         {
             return new OpSelectors.DefaultDescriptorSelector(rng,
                                                              columnSelector(schemaSpec),
-                                                             Surjections.weighted(operation_kind_weights),
+                                                             OpSelectors.OperationSelector.weighted(operation_kind_weights),
                                                              modifications_per_lts.make(),
                                                              rows_per_modification.make(),
                                                              max_partition_size);
@@ -790,7 +837,7 @@ public class Configuration
             return new OpSelectors.HierarchicalDescriptorSelector(rng,
                                                                   fractions,
                                                                   columnSelector(schemaSpec),
-                                                                  Surjections.weighted(operation_kind_weights),
+                                                                  OpSelectors.OperationSelector.weighted(operation_kind_weights),
                                                                   modifications_per_lts.make(),
                                                                   rows_per_modification.make(),
                                                                   max_partition_size);
@@ -876,7 +923,7 @@ public class Configuration
     @JsonTypeName("mutating")
     public static class MutatingPartitionVisitorConfiguation implements PartitionVisitorConfiguration
     {
-        protected final RowVisitorConfiguration row_visitor;
+        public final RowVisitorConfiguration row_visitor;
 
         @JsonCreator
         public MutatingPartitionVisitorConfiguation(@JsonProperty("row_visitor") RowVisitorConfiguration row_visitor)
@@ -912,9 +959,9 @@ public class Configuration
     @JsonTypeName("validate_all_partitions")
     public static class AllPartitionsValidatorConfiguration implements Configuration.PartitionVisitorConfiguration
     {
-        private final int concurrency;
-        private final int trigger_after;
-        private final Configuration.ModelConfiguration modelConfiguration;
+        public final int concurrency;
+        public final int trigger_after;
+        public final Configuration.ModelConfiguration modelConfiguration;
 
         @JsonCreator
         public AllPartitionsValidatorConfiguration(@JsonProperty("concurrency") int concurrency,
@@ -932,19 +979,40 @@ public class Configuration
         }
     }
 
+    @JsonTypeName("corrupt")
+    public static class CorruptingPartitionVisitorConfiguration implements Configuration.PartitionVisitorConfiguration
+    {
+        public final int trigger_after;
+
+        @JsonCreator
+        public CorruptingPartitionVisitorConfiguration(@JsonProperty("trigger_after") int trigger_after)
+        {
+            this.trigger_after = trigger_after;
+        }
+
+        public PartitionVisitor make(Run run)
+        {
+            return new CorruptingPartitionVisitor(trigger_after, run);
+        }
+    }
+
     @JsonTypeName("validate_recent_partitions")
     public static class RecentPartitionsValidatorConfiguration implements Configuration.PartitionVisitorConfiguration
     {
-        private final int partition_count;
-        private final int trigger_after;
-        private final Configuration.ModelConfiguration modelConfiguration;
+        public final int partition_count;
+        public final int trigger_after;
+        public final int queries;
+        public final Configuration.ModelConfiguration modelConfiguration;
 
+        // TODO: make query selector configurable
         @JsonCreator
         public RecentPartitionsValidatorConfiguration(@JsonProperty("partition_count") int partition_count,
                                                       @JsonProperty("trigger_after") int trigger_after,
+                                                      @JsonProperty("queries_per_partition") int queries,
                                                       @JsonProperty("model") Configuration.ModelConfiguration model)
         {
             this.partition_count = partition_count;
+            this.queries = queries;
             this.trigger_after = trigger_after;
             this.modelConfiguration = model;
         }
@@ -952,12 +1020,12 @@ public class Configuration
         @Override
         public PartitionVisitor make(Run run)
         {
-            return new RecentPartitionValidator(partition_count, trigger_after, run, modelConfiguration);
+            return new RecentPartitionValidator(partition_count, queries, trigger_after, run, modelConfiguration);
         }
     }
 
     @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.WRAPPER_OBJECT)
-    public interface RowVisitorConfiguration extends RowVisitor.RowVisitorFactory
+    public interface RowVisitorConfiguration extends Operation.RowVisitorFactory
     {
     }
 
@@ -965,7 +1033,7 @@ public class Configuration
     public static class MutatingRowVisitorConfiguration implements RowVisitorConfiguration
     {
         @Override
-        public RowVisitor make(Run run)
+        public Operation make(Run run)
         {
             return new MutatingRowVisitor(run);
         }
@@ -996,10 +1064,11 @@ public class Configuration
                                                 @JsonProperty("table") String table,
                                                 @JsonProperty("partition_keys") Map<String, String> pks,
                                                 @JsonProperty("clustering_keys") Map<String, String> cks,
-                                                @JsonProperty("regular_columns") Map<String, String> regulars)
+                                                @JsonProperty("regular_columns") Map<String, String> regulars,
+                                                @JsonProperty("static_columns") Map<String, String> statics)
         {
             this.schemaSpec = SchemaGenerators.parse(keyspace, table,
-                                                     pks, cks, regulars);
+                                                     pks, cks, regulars, statics);
         }
 
         public SchemaSpec make(long seed)
diff --git a/harry-core/src/harry/core/MetricReporter.java b/harry-core/src/harry/core/MetricReporter.java
index b576ec3..fa19a49 100644
--- a/harry-core/src/harry/core/MetricReporter.java
+++ b/harry-core/src/harry/core/MetricReporter.java
@@ -22,6 +22,7 @@ public interface MetricReporter
 {
     void columnDelete();
     void rowDelete();
+    void partitionDelete();
     void insert();
     void rangeDelete();
 
@@ -41,6 +42,7 @@ public interface MetricReporter
 
         public void columnDelete(){}
         public void rowDelete(){}
+        public void partitionDelete(){}
         public void insert(){}
         public void rangeDelete(){}
         public void validatePartition(){}
diff --git a/harry-core/src/harry/corruptor/AddExtraRowCorruptor.java b/harry-core/src/harry/corruptor/AddExtraRowCorruptor.java
index f2a8b28..65f4a0d 100644
--- a/harry-core/src/harry/corruptor/AddExtraRowCorruptor.java
+++ b/harry-core/src/harry/corruptor/AddExtraRowCorruptor.java
@@ -84,7 +84,7 @@ public class AddExtraRowCorruptor implements QueryResponseCorruptor
         // written value and tombstone are resolved in favour of tombstone, so we're
         // just going to take the next lts.
         logger.info("Corrupting the resultset by writing a row with cd {}", cd);
-        sut.execute(WriteHelper.inflateInsert(schema, query.pd, cd, vds, clock.rts(maxLts) + 1), SystemUnderTest.ConsistencyLevel.ALL);
+        sut.execute(WriteHelper.inflateInsert(schema, query.pd, cd, vds, null, clock.rts(maxLts) + 1), SystemUnderTest.ConsistencyLevel.ALL);
         return true;
     }
 }
\ No newline at end of file
diff --git a/harry-core/src/harry/corruptor/ChangeValueCorruptor.java b/harry-core/src/harry/corruptor/ChangeValueCorruptor.java
index 032f962..5f23a06 100644
--- a/harry-core/src/harry/corruptor/ChangeValueCorruptor.java
+++ b/harry-core/src/harry/corruptor/ChangeValueCorruptor.java
@@ -75,11 +75,11 @@ public class ChangeValueCorruptor implements RowCorruptor
         final long oldV = row.vds[idx];
         do
         {
-            corruptedVds[idx] = +rng.next();
+            corruptedVds[idx] =+ rng.next();
         }
         // we need to find a value that sorts strictly greater than the current one
-        while (schema.regularColumns.get(idx).generator().compare(corruptedVds[idx], oldV) <= 0);
+        while (schema.regularColumns.get(idx).type.compareLexicographically(corruptedVds[idx], oldV) <= 0);
 
-        return WriteHelper.inflateInsert(schema, row.pd, row.cd, corruptedVds, clock.rts(row.lts[idx]));
+        return WriteHelper.inflateInsert(schema, row.pd, row.cd, corruptedVds, null, clock.rts(row.lts[idx]));
     }
 }
diff --git a/harry-core/src/harry/corruptor/HideValueCorruptor.java b/harry-core/src/harry/corruptor/HideValueCorruptor.java
index 871ac37..3cedcda 100644
--- a/harry-core/src/harry/corruptor/HideValueCorruptor.java
+++ b/harry-core/src/harry/corruptor/HideValueCorruptor.java
@@ -56,6 +56,31 @@ public class HideValueCorruptor implements RowCorruptor
 
     public CompiledStatement corrupt(ResultSetRow row)
     {
+        BitSet mask;
+        if (row.slts != null && rng.nextBoolean())
+        {
+            int cnt = 0;
+            int idx;
+            do
+            {
+                idx = rng.nextInt(row.slts.length - 1);
+                cnt++;
+            }
+            while (row.slts[idx] == Model.NO_TIMESTAMP && cnt < 10);
+
+            if (row.slts[idx] != Model.NO_TIMESTAMP)
+            {
+                mask = BitSet.allUnset(schema.allColumns.size());
+                mask.set(schema.staticColumnsOffset + idx);
+
+                return DeleteHelper.deleteColumn(schema,
+                                                 row.pd,
+                                                 mask,
+                                                 schema.regularAndStaticColumnsMask(),
+                                                 clock.rts(clock.maxLts()) + 1);
+            }
+        }
+
         int idx;
         do
         {
@@ -63,12 +88,14 @@ public class HideValueCorruptor implements RowCorruptor
         }
         while (row.lts[idx] == Model.NO_TIMESTAMP);
 
-        BitSet mask = BitSet.allUnset(schema.regularColumns.size());
-        mask.set(idx);
+        mask = BitSet.allUnset(schema.allColumns.size());
+        mask.set(schema.regularColumnsOffset + idx);
+
         return DeleteHelper.deleteColumn(schema,
                                          row.pd,
                                          row.cd,
                                          mask,
+                                         schema.regularAndStaticColumnsMask(),
                                          clock.rts(clock.maxLts()) + 1);
     }
 }
diff --git a/harry-core/src/harry/corruptor/QueryResponseCorruptor.java b/harry-core/src/harry/corruptor/QueryResponseCorruptor.java
index 301c50b..f4a3778 100644
--- a/harry-core/src/harry/corruptor/QueryResponseCorruptor.java
+++ b/harry-core/src/harry/corruptor/QueryResponseCorruptor.java
@@ -19,18 +19,25 @@
 package harry.corruptor;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import harry.data.ResultSetRow;
 import harry.ddl.SchemaSpec;
 import harry.model.OpSelectors;
 import harry.model.SelectHelper;
 import harry.model.sut.SystemUnderTest;
 import harry.operations.CompiledStatement;
+import harry.runner.HarryRunner;
 import harry.runner.Query;
 
 public interface QueryResponseCorruptor
 {
+    Logger logger = LoggerFactory.getLogger(QueryResponseCorruptor.class);
+
     boolean maybeCorrupt(Query query, SystemUnderTest sut);
 
     class SimpleQueryResponseCorruptor implements QueryResponseCorruptor
@@ -52,7 +59,8 @@ public interface QueryResponseCorruptor
         {
             List<ResultSetRow> result = new ArrayList<>();
             CompiledStatement statement = query.toSelectStatement();
-            for (Object[] obj : sut.execute(statement.cql(), SystemUnderTest.ConsistencyLevel.ALL, statement.bindings()))
+            Object[][] before = sut.execute(statement.cql(), SystemUnderTest.ConsistencyLevel.ALL, statement.bindings());
+            for (Object[] obj : before)
                 result.add(SelectHelper.resultSetToRow(schema, clock, obj));
 
             // TODO: technically, we can do this just depends on corruption strategy
@@ -63,7 +71,23 @@ public interface QueryResponseCorruptor
             for (ResultSetRow row : result)
             {
                 if (rowCorruptor.maybeCorrupt(row, sut))
+                {
+                    Object[][] after = sut.execute(statement.cql(), SystemUnderTest.ConsistencyLevel.ALL, statement.bindings());
+                    boolean mismatch = false;
+                    for (int i = 0; i < before.length && i < after.length; i++)
+                    {
+                        if (!Arrays.equals(before[i], after[i]))
+                        {
+                            logger.info("Corrupted: \nBefore: {}\n" +
+                                        "After:  {}\n",
+                                        Arrays.toString(before[i]),
+                                        Arrays.toString(after[i]));
+                            mismatch = true;
+                        }
+                    }
+                    assert mismatch || before.length != after.length;
                     return true;
+                }
             }
             return false;
         }
diff --git a/harry-core/src/harry/corruptor/RowCorruptor.java b/harry-core/src/harry/corruptor/RowCorruptor.java
index fbd5fa9..4c6b005 100644
--- a/harry-core/src/harry/corruptor/RowCorruptor.java
+++ b/harry-core/src/harry/corruptor/RowCorruptor.java
@@ -18,6 +18,9 @@
 
 package harry.corruptor;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import harry.data.ResultSetRow;
 import harry.ddl.SchemaSpec;
 import harry.model.OpSelectors;
@@ -26,6 +29,8 @@ import harry.operations.CompiledStatement;
 
 public interface RowCorruptor
 {
+    Logger logger = LoggerFactory.getLogger(QueryResponseCorruptor.class);
+
     boolean canCorrupt(ResultSetRow row);
 
     CompiledStatement corrupt(ResultSetRow row);
@@ -37,6 +42,7 @@ public interface RowCorruptor
         {
             CompiledStatement statement = corrupt(row);
             sut.execute(statement.cql(), SystemUnderTest.ConsistencyLevel.ALL, statement.bindings());
+            logger.info("Corrupting with: {} ({})", statement.cql(), CompiledStatement.bindingsToString(statement.bindings()));
             return true;
         }
         return false;
diff --git a/harry-core/src/harry/corruptor/ShowValueCorruptor.java b/harry-core/src/harry/corruptor/ShowValueCorruptor.java
index 450c687..01372ce 100644
--- a/harry-core/src/harry/corruptor/ShowValueCorruptor.java
+++ b/harry-core/src/harry/corruptor/ShowValueCorruptor.java
@@ -72,6 +72,6 @@ public class ShowValueCorruptor implements RowCorruptor
         // We do not know LTS of the deleted row. We could try inferring it, but that
         // still won't help since we can't use it anyways, since collisions between a
         // written value and tombstone are resolved in favour of tombstone.
-        return WriteHelper.inflateInsert(schema, row.pd, row.cd, corruptedVds, clock.rts(clock.maxLts()) + 1);
+        return WriteHelper.inflateInsert(schema, row.pd, row.cd, corruptedVds, null, clock.rts(clock.maxLts()) + 1);
     }
 }
diff --git a/harry-core/src/harry/data/ResultSetRow.java b/harry-core/src/harry/data/ResultSetRow.java
index e2bf02b..48cfc03 100644
--- a/harry-core/src/harry/data/ResultSetRow.java
+++ b/harry-core/src/harry/data/ResultSetRow.java
@@ -18,6 +18,10 @@
 
 package harry.data;
 
+import java.util.Arrays;
+
+import harry.ddl.SchemaSpec;
+
 public class ResultSetRow
 {
     public final long pd;
@@ -25,8 +29,13 @@ public class ResultSetRow
     public final long[] vds;
     public final long[] lts;
 
+    public final long[] sds;
+    public final long[] slts;
+
     public ResultSetRow(long pd,
                         long cd,
+                        long[] sds,
+                        long[] slts,
                         long[] vds,
                         long[] lts)
     {
@@ -34,6 +43,8 @@ public class ResultSetRow
         this.cd = cd;
         this.vds = vds;
         this.lts = lts;
+        this.sds = sds;
+        this.slts = slts;
     }
 
     public String toString()
@@ -41,8 +52,11 @@ public class ResultSetRow
         return "resultSetRow("
                + pd +
                "L, " + cd +
-               "L, values(" + toString(vds) + ")" +
-               ", lts(" + toString(lts) + "))";
+               (sds == null ? "" : "L, values(" + toString(sds) + ")") +
+               (slts == null ? "" : ", lts(" + toString(slts) + ")") +
+               ", values(" + toString(vds) + ")" +
+               ", lts(" + toString(lts) + ")" +
+               ")";
     }
 
     public String toString(long[] arr)
@@ -57,4 +71,19 @@ public class ResultSetRow
         }
         return s;
     }
+
+    public String toString(SchemaSpec schema)
+    {
+        return "resultSetRow("
+               + pd +
+               "L, " + cd +
+               (sds == null ? "" : "L, staticValues(" + toString(sds) + ")") +
+               (slts == null ? "" : ", slts(" + toString(slts) + ")") +
+               ", values(" + toString(vds) + ")" +
+               ", lts(" + toString(lts) + ")" +
+               ", clustering=" + Arrays.toString(schema.inflateClusteringKey(cd)) +
+               ", values=" + Arrays.toString(schema.inflateRegularColumns(vds)) +
+               (sds == null ? "" : ", statics=" + Arrays.toString(schema.inflateStaticColumns(sds))) +
+               ")";
+    }
 }
diff --git a/harry-core/src/harry/ddl/ColumnSpec.java b/harry-core/src/harry/ddl/ColumnSpec.java
index ee4a774..94e9881 100644
--- a/harry-core/src/harry/ddl/ColumnSpec.java
+++ b/harry-core/src/harry/ddl/ColumnSpec.java
@@ -30,6 +30,8 @@ import com.google.common.collect.ImmutableList;
 import harry.generators.Bijections;
 import harry.generators.StringBijection;
 
+import static harry.generators.StringBijection.getByte;
+
 public class ColumnSpec<T>
 {
     public final String name;
@@ -133,7 +135,7 @@ public class ColumnSpec<T>
 
     public static ColumnSpec<?> staticColumn(String name, DataType<?> type)
     {
-        return new ColumnSpec<>(name, type, Kind.CLUSTERING);
+        return new ColumnSpec<>(name, type, Kind.STATIC);
     }
 
     public enum Kind
@@ -155,6 +157,21 @@ public class ColumnSpec<T>
             return false;
         }
 
+        /**
+         * Cassandra uses lexicographical oder for resolving timestamp ties
+         */
+        public int compareLexicographically(long l, long r)
+        {
+            for (int i = Long.BYTES - 1; i >= 0; i--)
+            {
+                int cmp = Integer.compare((int) ((l >> (i * 8)) & 0xffL),
+                                          (int) ((r >> (i * 8)) & 0xffL));
+                if (cmp != 0)
+                    return cmp;
+            }
+            return 0;
+        }
+
         public abstract Bijections.Bijection<T> generator();
 
         public int maxSize()
@@ -166,6 +183,11 @@ public class ColumnSpec<T>
         {
             return cqlName;
         }
+
+        public String nameForParser()
+        {
+            return cqlName;
+        }
     }
 
     public static final DataType<Byte> int8Type = new DataType<Byte>("tinyint")
@@ -206,6 +228,11 @@ public class ColumnSpec<T>
         {
             return Bijections.BOOLEAN_GENERATOR;
         }
+
+        public int compareLexicographically(long l, long r)
+        {
+            throw new RuntimeException("Boolean does not support custom comparators");
+        }
     };
 
     public static final DataType<Float> floatType = new DataType<Float>("float")
@@ -232,11 +259,16 @@ public class ColumnSpec<T>
         {
             return gen;
         }
+
+        public int compareLexicographically(long l, long r)
+        {
+            return Long.compare(l, r);
+        }
     };
 
-    public static DataType<String> asciiType(int nibbleSize, int maxRandomNibbles)
+    public static DataType<String> asciiType(int nibbleSize, int maxRandomBytes)
     {
-        Bijections.Bijection<String> gen = new StringBijection(nibbleSize, maxRandomNibbles);
+        Bijections.Bijection<String> gen = new StringBijection(nibbleSize, maxRandomBytes);
 
         return new DataType<String>("ascii")
         {
@@ -244,6 +276,19 @@ public class ColumnSpec<T>
             {
                 return gen;
             }
+
+            public int compareLexicographically(long l, long r)
+            {
+                return Long.compare(l, r);
+            }
+
+            public String nameForParser()
+            {
+                return String.format("%s(%d,%d)",
+                                     super.nameForParser(),
+                                     nibbleSize,
+                                     maxRandomBytes);
+            }
         };
     }
 
@@ -253,6 +298,11 @@ public class ColumnSpec<T>
         {
             return Bijections.UUID_GENERATOR;
         }
+
+        public int compareLexicographically(long l, long r)
+        {
+            throw new RuntimeException("UUID does not support custom comparators");
+        }
     };
 
     public static final DataType<Date> timestampType = new DataType<Date>("timestamp")
@@ -261,6 +311,11 @@ public class ColumnSpec<T>
         {
             return Bijections.TIMESTAMP_GENERATOR;
         }
+
+        public int compareLexicographically(long l, long r)
+        {
+            throw new RuntimeException("Date does not support custom comparators");
+        }
     };
 
     public static final Collection<DataType<?>> DATA_TYPES = ImmutableList.of(
diff --git a/harry-core/src/harry/ddl/SchemaGenerators.java b/harry-core/src/harry/ddl/SchemaGenerators.java
index 53ab6f5..47936e1 100644
--- a/harry-core/src/harry/ddl/SchemaGenerators.java
+++ b/harry-core/src/harry/ddl/SchemaGenerators.java
@@ -55,10 +55,16 @@ public class SchemaGenerators
                     ColumnSpec.int16Type,
                     ColumnSpec.int32Type,
                     ColumnSpec.int64Type,
-// TODO re-enable boolean type; add it to ByteBufferUtil in Cassandra for that
-//                    ColumnSpec.booleanType,
-                    ColumnSpec.asciiType);
+                    ColumnSpec.asciiType,
+                    ColumnSpec.asciiType(4, 256),
+                    ColumnSpec.asciiType(4, 512));
+
         columnTypes = builder.build();
+        builder.add(ColumnSpec.int8Type,
+                    ColumnSpec.int16Type,
+                    ColumnSpec.int32Type,
+                    ColumnSpec.int64Type,
+                    ColumnSpec.asciiType);
         builder = ImmutableList.builder();
         builder.addAll(columnTypes);
 
@@ -69,8 +75,8 @@ public class SchemaGenerators
             ColumnSpec.DataType<?> reversedType = ColumnSpec.ReversedType.getInstance(columnType);
             builder.add(reversedType);
 
-            mapBuilder.put(columnType.toString(), columnType);
-            mapBuilder.put(String.format("desc(%s)", columnType.toString()), columnType);
+            mapBuilder.put(columnType.nameForParser(), columnType);
+            mapBuilder.put(String.format("desc(%s)", columnType.nameForParser()), columnType);
         }
 
         builder.add(ColumnSpec.floatType);
@@ -152,6 +158,7 @@ public class SchemaGenerators
         private Generator<ColumnSpec<?>> pkGenerator = columnSpecGenerator("pk", ColumnSpec.Kind.PARTITION_KEY);
         private Generator<ColumnSpec<?>> ckGenerator = clusteringColumnSpecGenerator("ck");
         private Generator<ColumnSpec<?>> regularGenerator = columnSpecGenerator("regular", ColumnSpec.Kind.REGULAR);
+        private Generator<ColumnSpec<?>> staticGenerator = columnSpecGenerator("regular", ColumnSpec.Kind.STATIC);
 
         private int minPks = 1;
         private int maxPks = 1;
@@ -159,6 +166,8 @@ public class SchemaGenerators
         private int maxCks = 0;
         private int minRegular = 0;
         private int maxRegular = 0;
+        private int minStatic = 0;
+        private int maxStatic = 0;
 
         public Builder(String keyspace)
         {
@@ -221,11 +230,6 @@ public class SchemaGenerators
             return this;
         }
 
-        public Builder regularColumnCount(int numCols)
-        {
-            return regularColumnCount(numCols, numCols);
-        }
-
         public Builder regularColumnCount(int minCols, int maxCols)
         {
             this.minRegular = minCols;
@@ -233,6 +237,11 @@ public class SchemaGenerators
             return this;
         }
 
+        public Builder regularColumnCount(int numCols)
+        {
+            return regularColumnCount(numCols, numCols);
+        }
+
         public Builder regularColumnSpec(int minCols, int maxCols, ColumnSpec.DataType<?>... columnTypes)
         {
             return this.regularColumnSpec(minCols, maxCols, Arrays.asList(columnTypes));
@@ -246,17 +255,44 @@ public class SchemaGenerators
             return this;
         }
 
+        public Builder staticColumnCount(int minCols, int maxCols)
+        {
+            this.minStatic = minCols;
+            this.maxStatic = maxCols;
+            return this;
+        }
+
+        public Builder staticColumnCount(int numCols)
+        {
+            return staticColumnCount(numCols, numCols);
+        }
+
+        public Builder staticColumnSpec(int minCols, int maxCols, ColumnSpec.DataType<?>... columnTypes)
+        {
+            return this.staticColumnSpec(minCols, maxCols, Arrays.asList(columnTypes));
+        }
+
+        public Builder staticColumnSpec(int minCols, int maxCols, Collection<ColumnSpec.DataType<?>> columnTypes)
+        {
+            this.minStatic = minCols;
+            this.maxStatic = maxCols;
+            this.staticGenerator = columnSpecGenerator(columnTypes, "static", ColumnSpec.Kind.STATIC);
+            return this;
+        }
+
         private static class ColumnCounts
         {
             private final int pks;
             private final int cks;
             private final int regulars;
+            private final int statics;
 
-            private ColumnCounts(int pks, int cks, int regulars)
+            private ColumnCounts(int pks, int cks, int regulars, int statics)
             {
                 this.pks = pks;
                 this.cks = cks;
                 this.regulars = regulars;
+                this.statics = statics;
             }
         }
 
@@ -266,8 +302,9 @@ public class SchemaGenerators
                 int pks = rand.nextInt(minPks, maxPks);
                 int cks = rand.nextInt(minCks, maxCks);
                 int regulars = rand.nextInt(minRegular, maxRegular);
+                int statics = rand.nextInt(minStatic, maxStatic);
 
-                return new ColumnCounts(pks, cks, regulars);
+                return new ColumnCounts(pks, cks, regulars, statics);
             };
         }
 
@@ -283,7 +320,8 @@ public class SchemaGenerators
                                           tableNameSupplier.get(),
                                           pk,
                                           ck,
-                                          regularGenerator.generate(rand, counts.regulars));
+                                          regularGenerator.generate(rand, counts.regulars),
+                                          staticGenerator.generate(rand, counts.statics));
                 };
             });
         }
@@ -297,30 +335,27 @@ public class SchemaGenerators
     public static Surjections.Surjection<SchemaSpec> defaultSchemaSpecGen(String ks, String table)
     {
         return new SchemaGenerators.Builder(ks, () -> table)
-               .partitionKeySpec(2, 4,
-//                                                                             ColumnSpec.int8Type,
-//                                                                             ColumnSpec.int16Type,
-                                 ColumnSpec.int32Type,
-                                 ColumnSpec.int64Type,
-//                                                                             ColumnSpec.floatType,
-//                                                                             ColumnSpec.doubleType,
-                                 ColumnSpec.asciiType(4, 10))
-               .clusteringKeySpec(2, 4,
-//                                                                              ColumnSpec.int8Type,
-//                                                                              ColumnSpec.int16Type,
-                                  ColumnSpec.int32Type,
-                                  ColumnSpec.int64Type,
-//                                                                              ColumnSpec.floatType,
-//                                                                              ColumnSpec.doubleType,
-                                  ColumnSpec.asciiType(4, 10))
-               .regularColumnSpec(1, 10,
-//                                                                              ColumnSpec.int8Type,
-//                                                                              ColumnSpec.int16Type,
+               .partitionKeySpec(1, 3,
+                                 columnTypes)
+               .clusteringKeySpec(1, 3,
+                                  clusteringKeyTypes)
+               .regularColumnSpec(3, 5,
+                                  ColumnSpec.int8Type,
+                                  ColumnSpec.int16Type,
                                   ColumnSpec.int32Type,
                                   ColumnSpec.int64Type,
-//                                                                              ColumnSpec.floatType,
-//                                                                              ColumnSpec.doubleType,
-                                  ColumnSpec.asciiType(5, 10))
+                                  ColumnSpec.floatType,
+                                  ColumnSpec.doubleType,
+                                  ColumnSpec.asciiType(5, 256))
+               .staticColumnSpec(3, 5,
+                                 ColumnSpec.int8Type,
+                                 ColumnSpec.int16Type,
+                                 ColumnSpec.int32Type,
+                                 ColumnSpec.int64Type,
+                                 ColumnSpec.floatType,
+                                 ColumnSpec.doubleType,
+                                 ColumnSpec.asciiType(4, 512),
+                                 ColumnSpec.asciiType(4, 2048))
                .surjection();
     }
 
@@ -330,10 +365,11 @@ public class SchemaGenerators
     private static final Supplier<String> tableNameSupplier = () -> DEFAULT_PREFIX + counter.getAndIncrement();
 
     // simplest schema gen, nothing can go wrong with it
-    public static final Surjections.Surjection<SchemaSpec> longOnlySpecBuilder = new SchemaGenerators.Builder(DEFAULT_KEYSPACE_NAME, tableNameSupplier)
+    public static final Surjections.Surjection<SchemaSpec> longOnlySpecBuilder = new Builder(DEFAULT_KEYSPACE_NAME, tableNameSupplier)
                                                                                  .partitionKeySpec(1, 1, ColumnSpec.int64Type)
                                                                                  .clusteringKeySpec(1, 1, ColumnSpec.int64Type)
                                                                                  .regularColumnSpec(1, 10, ColumnSpec.int64Type)
+                                                                                 .staticColumnSpec(1, 10, ColumnSpec.int64Type)
                                                                                  .surjection();
 
     private static final ColumnSpec.DataType<String> simpleStringType = ColumnSpec.asciiType(4, 10);
@@ -341,30 +377,35 @@ public class SchemaGenerators
                                                                                        .partitionKeySpec(2, 2, ColumnSpec.int64Type, simpleStringType)
                                                                                        .clusteringKeySpec(2, 2, ColumnSpec.int64Type, simpleStringType)
                                                                                        .regularColumnSpec(1, 10, ColumnSpec.int64Type, simpleStringType)
+                                                                                       .staticColumnSpec(1, 10, ColumnSpec.int64Type)
                                                                                        .surjection();
 
     public static final Surjections.Surjection<SchemaSpec> longOnlyWithReverseSpecBuilder = new SchemaGenerators.Builder(DEFAULT_KEYSPACE_NAME, tableNameSupplier)
                                                                                             .partitionKeySpec(1, 1, ColumnSpec.int64Type)
                                                                                             .clusteringKeySpec(1, 1, ColumnSpec.ReversedType.getInstance(ColumnSpec.int64Type))
                                                                                             .regularColumnSpec(1, 10, ColumnSpec.int64Type)
+                                                                                            .staticColumnSpec(1, 10, ColumnSpec.int64Type)
                                                                                             .surjection();
 
     public static final Surjections.Surjection<SchemaSpec> longAndStringSpecWithReversedLongBuilder = new SchemaGenerators.Builder(DEFAULT_KEYSPACE_NAME, tableNameSupplier)
                                                                                                       .partitionKeySpec(2, 2, ColumnSpec.int64Type, simpleStringType)
                                                                                                       .clusteringKeySpec(2, 2, ColumnSpec.ReversedType.getInstance(ColumnSpec.int64Type), simpleStringType)
                                                                                                       .regularColumnSpec(1, 10, ColumnSpec.int64Type, simpleStringType)
+                                                                                                      .staticColumnSpec(1, 10, ColumnSpec.int64Type)
                                                                                                       .surjection();
 
     public static final Surjections.Surjection<SchemaSpec> longAndStringSpecWithReversedStringBuilder = new SchemaGenerators.Builder(DEFAULT_KEYSPACE_NAME, tableNameSupplier)
                                                                                                         .partitionKeySpec(2, 2, ColumnSpec.int64Type, simpleStringType)
                                                                                                         .clusteringKeySpec(2, 2, ColumnSpec.int64Type, ColumnSpec.ReversedType.getInstance(simpleStringType))
                                                                                                         .regularColumnSpec(1, 10, ColumnSpec.int64Type, simpleStringType)
+                                                                                                        .staticColumnSpec(1, 10, ColumnSpec.int64Type)
                                                                                                         .surjection();
 
     public static final Surjections.Surjection<SchemaSpec> longAndStringSpecWithReversedBothBuilder = new SchemaGenerators.Builder(DEFAULT_KEYSPACE_NAME, tableNameSupplier)
                                                                                                       .partitionKeySpec(2, 2, ColumnSpec.int64Type, simpleStringType)
                                                                                                       .clusteringKeySpec(2, 2, ColumnSpec.ReversedType.getInstance(ColumnSpec.int64Type), ColumnSpec.ReversedType.getInstance(simpleStringType))
                                                                                                       .regularColumnSpec(1, 10, ColumnSpec.int64Type, simpleStringType)
+                                                                                                      .staticColumnSpec(1, 10, ColumnSpec.int64Type)
                                                                                                       .surjection();
 
     public static final Surjections.Surjection<SchemaSpec> withAllFeaturesEnabled = new SchemaGenerators.Builder(DEFAULT_KEYSPACE_NAME, tableNameSupplier)
@@ -399,7 +440,7 @@ public class SchemaGenerators
                 counter++;
                 SchemaSpec spec = generators[idx].get();
                 int tries = 100;
-                while ((spec.ckGenerator.byteSize() != Long.BYTES || spec.pkGenerator.byteSize() != Long.BYTES) && tries > 0)
+                while ((spec.pkGenerator.byteSize() != Long.BYTES) && tries > 0)
                 {
                     System.out.println("Skipping schema, since it doesn't have enough entropy bits available: " + spec.compile().cql());
                     spec = generators[idx].get();
@@ -411,8 +452,6 @@ public class SchemaGenerators
                 assert tries > 0 : String.format("Max number of tries exceeded on generator %d, can't generate a needed schema", idx);
                 return spec;
             }
-
-
         };
     }
 
@@ -435,12 +474,14 @@ public class SchemaGenerators
                                    String table,
                                    Map<String, String> pks,
                                    Map<String, String> cks,
-                                   Map<String, String> regulars)
+                                   Map<String, String> regulars,
+                                   Map<String, String> statics)
     {
         return new SchemaSpec(keyspace, table,
                               toColumns(pks, ColumnSpec.Kind.PARTITION_KEY, false),
                               toColumns(cks, ColumnSpec.Kind.CLUSTERING, false),
-                              toColumns(regulars, ColumnSpec.Kind.REGULAR, false));
+                              toColumns(regulars, ColumnSpec.Kind.REGULAR, false),
+                              toColumns(statics, ColumnSpec.Kind.STATIC, false));
     }
 
     public static int DEFAULT_SWITCH_AFTER = Integer.getInteger("harry.test.progression.switch-after", 5);
diff --git a/harry-core/src/harry/ddl/SchemaSpec.java b/harry-core/src/harry/ddl/SchemaSpec.java
index 11b7009..f07c106 100644
--- a/harry-core/src/harry/ddl/SchemaSpec.java
+++ b/harry-core/src/harry/ddl/SchemaSpec.java
@@ -25,6 +25,7 @@ import java.util.stream.Stream;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
+import com.google.common.collect.Streams;
 
 import harry.generators.DataGenerators;
 import harry.operations.CompiledStatement;
@@ -52,18 +53,25 @@ public class SchemaSpec
     public final List<ColumnSpec<?>> partitionKeys;
     public final List<ColumnSpec<?>> clusteringKeys;
     public final List<ColumnSpec<?>> regularColumns;
+    public final List<ColumnSpec<?>> staticColumns;
     public final List<ColumnSpec<?>> allColumns;
 
     public final BitSet ALL_COLUMNS_BITSET;
+    public final int regularColumnsOffset;
+    public final int staticColumnsOffset;
+    public final BitSet regularColumnsMask;
+    public final BitSet regularAndStaticColumnsMask;
+    public final BitSet staticColumnsMask;
 
     // TODO: forbid this constructor; add the one where column specs would be initialized through builder and have indexes
     public SchemaSpec(String keyspace,
                       String table,
                       List<ColumnSpec<?>> partitionKeys,
                       List<ColumnSpec<?>> clusteringKeys,
-                      List<ColumnSpec<?>> regularColumns)
+                      List<ColumnSpec<?>> regularColumns,
+                      List<ColumnSpec<?>> staticColumns)
     {
-        this(keyspace, table, partitionKeys, clusteringKeys, regularColumns, false);
+        this(keyspace, table, partitionKeys, clusteringKeys, regularColumns, staticColumns, false);
     }
 
     public SchemaSpec(String keyspace,
@@ -71,6 +79,7 @@ public class SchemaSpec
                       List<ColumnSpec<?>> partitionKeys,
                       List<ColumnSpec<?>> clusteringKeys,
                       List<ColumnSpec<?>> regularColumns,
+                      List<ColumnSpec<?>> staticColumns,
                       boolean isCompactStorage)
     {
         assert !isCompactStorage || clusteringKeys.size() == 0 || regularColumns.size() <= 1;
@@ -84,27 +93,83 @@ public class SchemaSpec
         this.clusteringKeys = ImmutableList.copyOf(clusteringKeys);
         for (int i = 0; i < clusteringKeys.size(); i++)
             clusteringKeys.get(i).setColumnIndex(i);
+        this.staticColumns = ImmutableList.copyOf(staticColumns);
+        for (int i = 0; i < staticColumns.size(); i++)
+            staticColumns.get(i).setColumnIndex(i);
         this.regularColumns = ImmutableList.copyOf(regularColumns);
         for (int i = 0; i < regularColumns.size(); i++)
             regularColumns.get(i).setColumnIndex(i);
         this.allColumns = ImmutableList.copyOf(Iterables.concat(partitionKeys,
                                                                 clusteringKeys,
+                                                                staticColumns,
                                                                 regularColumns));
         this.pkGenerator = DataGenerators.createKeyGenerator(partitionKeys);
         this.ckGenerator = DataGenerators.createKeyGenerator(clusteringKeys);
 
         this.ALL_COLUMNS_BITSET = BitSet.allSet(regularColumns.size());
+
+        this.staticColumnsOffset = partitionKeys.size() + clusteringKeys.size();
+        this.regularColumnsOffset = staticColumnsOffset + staticColumns.size();
+
+        this.regularColumnsMask = regularColumnsMask(this);
+        this.regularAndStaticColumnsMask = regularAndStaticColumnsMask(this);
+        this.staticColumnsMask = staticColumnsMask(this);
+    }
+
+    public static BitSet allColumnsMask(SchemaSpec schema)
+    {
+        return BitSet.allSet(schema.allColumns.size());
+    }
+
+    // todo: bitset views?
+
+    public BitSet regularColumnsMask()
+    {
+        return this.regularColumnsMask;
+    }
+
+    public BitSet regularAndStaticColumnsMask()
+    {
+        return this.regularAndStaticColumnsMask;
+    }
+
+    public BitSet staticColumnsMask()
+    {
+        return this.staticColumnsMask;
+    }
+
+    private static BitSet regularColumnsMask(SchemaSpec schema)
+    {
+        BitSet mask = BitSet.allUnset(schema.allColumns.size());
+        for (int i = 0; i < schema.regularColumns.size(); i++)
+            mask.set(schema.regularColumnsOffset + i);
+        return mask;
+    }
+
+    private static BitSet regularAndStaticColumnsMask(SchemaSpec schema)
+    {
+        BitSet mask = BitSet.allUnset(schema.allColumns.size());
+        for (int i = 0; i < schema.staticColumns.size() + schema.regularColumns.size(); i++)
+            mask.set(schema.staticColumnsOffset + i);
+        return mask;
+    }
+
+    private static BitSet staticColumnsMask(SchemaSpec schema)
+    {
+        BitSet mask = BitSet.allUnset(schema.allColumns.size());
+        for (int i = 0; i < schema.staticColumns.size(); i++)
+            mask.set(schema.staticColumnsOffset + i);
+        return mask;
     }
 
     public void validate()
     {
         assert pkGenerator.byteSize() == Long.BYTES : partitionKeys.toString();
-        assert ckGenerator.byteSize() == Long.BYTES : clusteringKeys.toString();
     }
 
-    public static interface AddRelationCallback
+    public interface AddRelationCallback
     {
-        public void accept(ColumnSpec spec, Relation.RelationKind kind, Object value);
+        public void accept(ColumnSpec<?> spec, Relation.RelationKind kind, Object value);
     }
 
     public void inflateRelations(long pd,
@@ -134,6 +199,11 @@ public class SchemaSpec
         return DataGenerators.inflateData(regularColumns, vds);
     }
 
+    public Object[] inflateStaticColumns(long[] sds)
+    {
+        return DataGenerators.inflateData(staticColumns, sds);
+    }
+
     // TODO: remove indirection; call directly
     public long adjustPdEntropy(long descriptor)
     {
@@ -155,6 +225,11 @@ public class SchemaSpec
         return ckGenerator.deflate(ck);
     }
 
+    public long[] deflateStaticColumns(Object[] statics)
+    {
+        return DataGenerators.deflateData(staticColumns, statics);
+    }
+
     public long[] deflateRegularColumns(Object[] regulars)
     {
         return DataGenerators.deflateData(regularColumns, regulars);
@@ -179,8 +254,9 @@ public class SchemaSpec
                 sb.append(" PRIMARY KEY");
         }
 
-        Stream.concat(clusteringKeys.stream(),
-                      regularColumns.stream())
+        Streams.concat(clusteringKeys.stream(),
+                       staticColumns.stream(),
+                       regularColumns.stream())
               .forEach((cd) -> {
                   commaAppender.accept(sb);
                   sb.append(cd.toCQL());
diff --git a/harry-core/src/harry/generators/Bijections.java b/harry-core/src/harry/generators/Bijections.java
index d0d6532..4cc06a7 100644
--- a/harry-core/src/harry/generators/Bijections.java
+++ b/harry-core/src/harry/generators/Bijections.java
@@ -52,6 +52,9 @@ public class Bijections
         // TODO: byteSize is great, but you know what's better? Bit size! For example, for `boolean`, we only need a single bit.
         int byteSize();
 
+        /**
+         * Compare as if we were comparing the values in question
+         */
         int compare(long l, long r);
 
         default long adjustEntropyDomain(long descriptor)
@@ -69,7 +72,7 @@ public class Bijections
             return maxForSize(byteSize());
         }
 
-        default boolean byteOrdered()
+        default boolean unsigned()
         {
             return false;
         }
@@ -280,7 +283,7 @@ public class Bijections
         }
 
         // In other words, there's no way we can extend entropy to a sign
-        public boolean byteOrdered()
+        public boolean unsigned()
         {
             return true;
         }
@@ -343,7 +346,12 @@ public class Bijections
             return SIZE;
         }
 
-        public boolean byteOrdered()
+        /**
+         * To avoid generating NaNs, we're using a smaller size for Double. But because of that, double became
+         * sign-less. In other words, even if we generate a double, it will always be positive, since its most
+         * significant bit isn't set. This means that
+         */
+        public boolean unsigned()
         {
             return true;
         }
diff --git a/harry-core/src/harry/generators/DataGenerators.java b/harry-core/src/harry/generators/DataGenerators.java
index 9d3593b..6878ae4 100644
--- a/harry-core/src/harry/generators/DataGenerators.java
+++ b/harry-core/src/harry/generators/DataGenerators.java
@@ -42,6 +42,8 @@ public class DataGenerators
             ColumnSpec columnSpec = columns.get(i);
             if (descriptors[i] == UNSET_DESCR)
                 data[i] = UNSET_VALUE;
+            else if (descriptors[i] == NIL_DESCR)
+                data[i] = null;
             else
                 data[i] = columnSpec.inflate(descriptors[i]);
         }
@@ -154,11 +156,20 @@ public class DataGenerators
         int fixedPart = Math.min(4, columns.size());
 
         long[] slices = new long[fixedPart];
+        boolean allNulls = true;
         for (int i = 0; i < fixedPart; i++)
         {
             ColumnSpec spec = columns.get(i);
-            slices[i] = spec.deflate(values[i]);
+            Object value = values[i];
+            if (value != null)
+                allNulls = false;
+
+            slices[i] = value == null ? NIL_DESCR : spec.deflate(value);
         }
+
+        if (allNulls)
+            return null;
+
         return slices;
     }
 
@@ -311,12 +322,15 @@ public class DataGenerators
 
         public boolean shouldInvertSign()
         {
-            return totalSize != Long.BYTES && !keyGen.byteOrdered();
+            return totalSize != Long.BYTES && !keyGen.unsigned();
         }
 
         public long deflate(Object[] value)
         {
-            long descriptor = keyGen.deflate(value[0]);
+            Object v = value[0];
+            if (v == null)
+                return NIL_DESCR;
+            long descriptor = keyGen.deflate(v);
             return stitch(new long[] { descriptor });
         }
 
@@ -352,7 +366,10 @@ public class DataGenerators
 
         public long deflate(Object[] values)
         {
-            return stitch(DataGenerators.deflateKey(columns, values));
+            long[] stiched = DataGenerators.deflateKey(columns, values);
+            if (stiched == null)
+                return NIL_DESCR;
+            return stitch(stiched);
         }
 
         public Object[] inflate(long descriptor)
@@ -368,29 +385,33 @@ public class DataGenerators
             int maxSliceSize = gen.byteSize();
             int actualSliceSize = sizes[idx];
 
+
             if (idx == 0)
             {
                 // We consume a sign of a descriptor (long, long), (int, int), etc.
                 if (totalSize == Long.BYTES)
                 {
-                    // if we use only 3 bytes for a 4-byte int, or 4 bytes for a 8-byte int,
+                    // If we use only 3 bytes for a 4-byte int, or 4 bytes for a 8-byte int,
                     // they're effectively unsigned/byte-ordered, so their order won't match
                     if (maxSliceSize > actualSliceSize)
                         return true;
-                    // Since descriptor is signed, invert sign of all byte-ordered types, since
-                    // their order won't match
+                    // Sign of the current descriptor should match the sign of the slice.
+                    // For example, (tinyint, double) or (double, tinyint). In the first case (tinyint first),
+                    // sign of the first component is going to match the sign of the descriptor.
+                    // In the second case (double first), double is 7-bit, but its most significant bit
+                    // does not hold a sign, so we have to invert it to match sign of the descriptor.
                     else
-                        return gen.byteOrdered();
+                        return gen.unsigned();
                 }
                 // We do not consume a sign of a descriptor (float, tinyint), (int, tinyint), etc,
                 // so we have to only invert signs of the values, since their order doesn't match.
                 else
                 {
                     assert maxSliceSize == actualSliceSize;
-                    return !gen.byteOrdered();
+                    return !gen.unsigned();
                 }
             }
-            else if (gen.byteOrdered())
+            else if (gen.unsigned())
                 return false;
             else
                 // We invert sign of all subsequent chunks if they have enough entropy to have a sign bit set
diff --git a/harry-core/src/harry/generators/RngUtils.java b/harry-core/src/harry/generators/RngUtils.java
index 92b97ee..749cf7f 100644
--- a/harry-core/src/harry/generators/RngUtils.java
+++ b/harry-core/src/harry/generators/RngUtils.java
@@ -18,6 +18,8 @@
 
 package harry.generators;
 
+import java.util.function.LongSupplier;
+
 public class RngUtils
 {
     public static long next(long input)
@@ -93,4 +95,48 @@ public class RngUtils
     {
         return Double.longBitsToDouble(current);
     }
+
+    static long bitmask(long n)
+    {
+        if (n == 64) return ~0L;
+        return (1L << n) - 1;
+    }
+
+    public static long randomBits(long bits, long length, long s)
+    {
+        return randomBits(bits, length, new LongSupplier()
+        {
+            private long seed = s;
+
+            @Override
+            public long getAsLong()
+            {
+                long next = PCGFastPure.advanceState(seed, 1, 1);
+                assert next != seed : seed;
+                seed = next;
+                return next;
+            }
+        });
+    }
+
+    public static long randomBits(long bits, long length, LongSupplier rng)
+    {
+        long mask = bitmask(length);
+        if (bits == length)
+            return mask;
+        long min = 0;
+        long max = ~0L;
+        int n = 0;
+        while (n != bits)
+        {
+            long x = rng.getAsLong() & mask;
+            x = min | (x & max);
+            n = Long.bitCount(x);
+            if (n > bits)
+                max = x;
+            else
+                min = x;
+        }
+        return min;
+    }
 }
diff --git a/harry-core/src/harry/generators/StringBijection.java b/harry-core/src/harry/generators/StringBijection.java
index 46b58e7..09c2c21 100644
--- a/harry-core/src/harry/generators/StringBijection.java
+++ b/harry-core/src/harry/generators/StringBijection.java
@@ -32,19 +32,19 @@ public class StringBijection implements Bijections.Bijection<String>
     private final String[] nibbles;
     private final Map<String, Integer> inverse;
     private final int nibbleSize;
-    private final int maxRandomNibbles;
+    private final int maxRandomBytes;
 
     public StringBijection()
     {
         this(alphabetNibbles(8), 8, 10);
     }
 
-    public StringBijection(int nibbleSize, int maxRandomNibbles)
+    public StringBijection(int nibbleSize, int maxRandomBytes)
     {
-        this(alphabetNibbles(nibbleSize), nibbleSize, maxRandomNibbles);
+        this(alphabetNibbles(nibbleSize), nibbleSize, maxRandomBytes);
     }
 
-    public StringBijection(String[] nibbles, int nibbleSize, int maxRandomNibbles)
+    public StringBijection(String[] nibbles, int nibbleSize, int maxRandomBytes)
     {
         assert nibbles.length == NIBBLES_SIZE;
         this.nibbles = nibbles;
@@ -57,7 +57,7 @@ public class StringBijection implements Bijections.Bijection<String>
             inverse.put(nibbles[i], i);
         }
 
-        this.maxRandomNibbles = maxRandomNibbles;
+        this.maxRandomBytes = maxRandomBytes;
     }
 
     public String inflate(long descriptor)
@@ -69,7 +69,7 @@ public class StringBijection implements Bijections.Bijection<String>
             builder.append(nibbles[idx]);
         }
 
-        appendRandomNibbles(builder, descriptor);
+        appendRandomBytes(builder, descriptor);
 
         // everything after this point can be just random, since strings are guaranteed
         // to have unique prefixes
@@ -87,15 +87,19 @@ public class StringBijection implements Bijections.Bijection<String>
 
 
     // TODO: shuld we switch to PCG here, too?
-    private void appendRandomNibbles(StringBuilder builder, long descriptor)
+    private void appendRandomBytes(StringBuilder builder, long descriptor)
     {
         long rnd = RngUtils.next(descriptor);
-        int count = RngUtils.asInt(rnd, 0, maxRandomNibbles);
+        int remaining = RngUtils.asInt(rnd, 0, maxRandomBytes);
 
-        for (int i = 0; i < count; i++)
+        while (remaining > 0)
         {
             rnd = RngUtils.next(rnd);
-            builder.append(nibbles[RngUtils.asByte(rnd) & 0xff]);
+            for (int i = 0; i < remaining && i < Long.BYTES; i++)
+            {
+                builder.append((char) (rnd >> (i * 8)) & 0xff);
+                remaining--;
+            }
         }
     }
 
@@ -136,7 +140,7 @@ public class StringBijection implements Bijections.Bijection<String>
     {
         return "ascii(" +
                "nibbleSize=" + nibbleSize +
-               ", maxRandomNibbles=" + maxRandomNibbles +
+               ", maxRandomBytes=" + maxRandomBytes +
                ')';
     }
 
diff --git a/harry-core/src/harry/generators/Surjections.java b/harry-core/src/harry/generators/Surjections.java
index 78a2b3b..e5a937c 100644
--- a/harry-core/src/harry/generators/Surjections.java
+++ b/harry-core/src/harry/generators/Surjections.java
@@ -60,15 +60,11 @@ public class Surjections
     public static long[] weights(int... weights)
     {
         long[] res = new long[weights.length];
-        int sum = 0;
         for (int i = 0; i < weights.length; i++)
         {
             long w = weights[i];
-            sum += w;
             res[i] = w << 32 | i;
         }
-        assert sum == 100;
-
         return res;
     }
 
@@ -109,9 +105,9 @@ public class Surjections
             weightMap.put(sum, entry.getKey());
         }
 
-        assert sum == 100;
+        int max = sum;
         return (i) -> {
-            int weight = RngUtils.asInt(i, 0, 100);
+            int weight = RngUtils.asInt(i, 0, max);
             return weightMap.ceilingEntry(weight).getValue();
         };
     }
diff --git a/harry-core/src/harry/model/DataTracker.java b/harry-core/src/harry/model/DataTracker.java
deleted file mode 100644
index dfd2503..0000000
--- a/harry-core/src/harry/model/DataTracker.java
+++ /dev/null
@@ -1,123 +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;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.PriorityBlockingQueue;
-import java.util.concurrent.atomic.AtomicLong;
-
-import com.google.common.annotations.VisibleForTesting;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class DataTracker
-{
-    private static final Logger logger = LoggerFactory.getLogger(DataTracker.class);
-
-    private final AtomicLong maxSeenLts;
-    // TODO: This is a trivial implementation that can be significantly improved upon
-    // for example, we could use a bitmap that records `1`s for all lts that are after
-    // the consective, and "collapse" the bitmap state into the long as soon as we see
-    // consecutive `1` on the left side.
-    private final AtomicLong maxCompleteLts;
-    private final PriorityBlockingQueue<Long> reorderBuffer;
-
-    DataTracker()
-    {
-        this.maxSeenLts = new AtomicLong(-1);
-        this.maxCompleteLts = new AtomicLong(-1);
-        this.reorderBuffer = new PriorityBlockingQueue<>(100);
-    }
-
-    // TODO: there's also some room for improvement in terms of concurrency
-    // TODO: remove pd?
-    public void recordEvent(long lts, boolean quorumAchieved)
-    {
-        // all seen LTS are allowed to be "in-flight"
-        maxSeenLts.getAndUpdate((old) -> Math.max(lts, old));
-
-        if (!quorumAchieved)
-            return;
-
-        long maxAchievedConsecutive = drainReorderQueue();
-
-        if (maxAchievedConsecutive + 1 == lts)
-            maxCompleteLts.compareAndSet(maxAchievedConsecutive, lts);
-        else
-            reorderBuffer.offer(lts);
-    }
-
-    public long drainReorderQueue()
-    {
-        long expected = maxCompleteLts.get();
-        long maxAchievedConsecutive = expected;
-        if (reorderBuffer.isEmpty())
-            return maxAchievedConsecutive;
-
-        boolean catchingUp = false;
-
-        Long smallest = reorderBuffer.poll();
-        while (smallest != null && smallest == maxAchievedConsecutive + 1)
-        {
-            maxAchievedConsecutive++;
-            catchingUp = true;
-            smallest = reorderBuffer.poll();
-        }
-
-        // put back
-        if (smallest != null)
-            reorderBuffer.offer(smallest);
-
-        if (catchingUp)
-            maxCompleteLts.compareAndSet(expected, maxAchievedConsecutive);
-
-        int bufferSize = reorderBuffer.size();
-        if (bufferSize > 100)
-            logger.warn("Reorder buffer size has grown up to " + reorderBuffer.size());
-        return maxAchievedConsecutive;
-    }
-
-    public long maxSeenLts()
-    {
-        return maxSeenLts.get();
-    }
-
-    public long maxCompleteLts()
-    {
-        return maxCompleteLts.get();
-    }
-
-    @VisibleForTesting
-    public void forceLts(long maxSeen, long maxComplete)
-    {
-        this.maxSeenLts.set(maxSeen);
-        this.maxCompleteLts.set(maxComplete);
-    }
-
-    public String toString()
-    {
-        List<Long> buf = new ArrayList<>(reorderBuffer);
-        return "DataTracker{" +
-               "maxSeenLts=" + maxSeenLts +
-               ", maxCompleteLts=" + maxCompleteLts +
-               ", reorderBuffer=" + buf +
-               '}';
-    }
-}
diff --git a/harry-core/src/harry/model/DescriptorSelectorBuilder.java b/harry-core/src/harry/model/DescriptorSelectorBuilder.java
index f34484f..ea2ffd8 100644
--- a/harry-core/src/harry/model/DescriptorSelectorBuilder.java
+++ b/harry-core/src/harry/model/DescriptorSelectorBuilder.java
@@ -27,12 +27,12 @@ import harry.generators.Surjections;
 import harry.generators.distribution.Distribution;
 import harry.util.BitSet;
 
-import static harry.model.OpSelectors.DefaultDescriptorSelector.DEFAULT_OP_TYPE_SELECTOR;
+import static harry.model.OpSelectors.DefaultDescriptorSelector.DEFAULT_OP_SELECTOR;
 
 public class DescriptorSelectorBuilder implements Configuration.CDSelectorConfiguration
 {
-    private Function<SchemaSpec, Function<OpSelectors.OperationKind, Surjections.Surjection<BitSet>>> columnMaskSelector;
-    private Surjections.Surjection<OpSelectors.OperationKind> operationTypeSelector = DEFAULT_OP_TYPE_SELECTOR;
+    private Function<SchemaSpec, OpSelectors.ColumnSelector> columnSelectorFactory;
+    private OpSelectors.OperationSelector operationSelector = DEFAULT_OP_SELECTOR;
     private Distribution numberOfRowsDistribution = new Distribution.ScaledDistribution(2, 30);
     private Distribution numberOfModificationsDistribution = new Distribution.ScaledDistribution(1, 3);
     private int maxPartitionSize = Integer.MAX_VALUE;
@@ -50,21 +50,21 @@ public class DescriptorSelectorBuilder implements Configuration.CDSelectorConfig
         return this;
     }
 
-    public DescriptorSelectorBuilder setColumnMaskSelector(Surjections.Surjection<BitSet> selector)
+    public DescriptorSelectorBuilder setColumnSelector(Surjections.Surjection<BitSet> selector)
     {
-        this.columnMaskSelector = (schemaSpec) -> new OpSelectors.ColumnSelectorBuilder().forAll(selector).build();
+        this.columnSelectorFactory = (schemaSpec) -> new OpSelectors.ColumnSelectorBuilder().forAll(schemaSpec, selector).build();
         return this;
     }
 
-    public DescriptorSelectorBuilder setColumnMaskSelector(Function<SchemaSpec, Function<OpSelectors.OperationKind, Surjections.Surjection<BitSet>>> columnMaskSelector)
+    public DescriptorSelectorBuilder setColumnSelectorFactory(Function<SchemaSpec, OpSelectors.ColumnSelector> columnMaskSelector)
     {
-        this.columnMaskSelector = Objects.requireNonNull(columnMaskSelector, "mask");
+        this.columnSelectorFactory = Objects.requireNonNull(columnMaskSelector, "mask");
         return this;
     }
 
-    public DescriptorSelectorBuilder setOperationTypeSelector(Surjections.Surjection<OpSelectors.OperationKind> operationTypeSelector)
+    public DescriptorSelectorBuilder setOperationSelector(OpSelectors.OperationSelector operationSelector)
     {
-        this.operationTypeSelector = Objects.requireNonNull(operationTypeSelector, "type");
+        this.operationSelector = Objects.requireNonNull(operationSelector, "type");
         return this;
     }
 
@@ -95,8 +95,8 @@ public class DescriptorSelectorBuilder implements Configuration.CDSelectorConfig
     public OpSelectors.DescriptorSelector make(OpSelectors.Rng rng, SchemaSpec schemaSpec)
     {
         return new OpSelectors.DefaultDescriptorSelector(rng,
-                                                         columnMaskSelector.apply(schemaSpec),
-                                                         operationTypeSelector,
+                                                         columnSelectorFactory.apply(schemaSpec),
+                                                         operationSelector,
                                                          numberOfModificationsDistribution,
                                                          numberOfRowsDistribution,
                                                          maxPartitionSize);
diff --git a/harry-core/src/harry/model/ExhaustiveChecker.java b/harry-core/src/harry/model/ExhaustiveChecker.java
deleted file mode 100644
index b98ef06..0000000
--- a/harry-core/src/harry/model/ExhaustiveChecker.java
+++ /dev/null
@@ -1,618 +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;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Comparator;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.NavigableMap;
-import java.util.Objects;
-import java.util.TreeMap;
-import java.util.function.Supplier;
-import java.util.stream.Collectors;
-
-import com.google.common.collect.Iterators;
-import com.google.common.collect.PeekingIterator;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import harry.core.Configuration;
-import harry.core.Run;
-import harry.data.ResultSetRow;
-import harry.ddl.SchemaSpec;
-import harry.model.sut.SystemUnderTest;
-import harry.runner.AbstractPartitionVisitor;
-import harry.runner.DataTracker;
-import harry.runner.PartitionVisitor;
-import harry.runner.Query;
-import harry.runner.QueryGenerator;
-import harry.util.BitSet;
-import harry.util.Ranges;
-
-import static harry.generators.DataGenerators.NIL_DESCR;
-import static harry.generators.DataGenerators.UNSET_DESCR;
-
-public class ExhaustiveChecker implements Model
-{
-    private static final Logger logger = LoggerFactory.getLogger(ExhaustiveChecker.class);
-
-    public static LongComparator FORWARD_COMPARATOR = Long::compare;
-    public static LongComparator REVERSE_COMPARATOR = (a, b) -> Long.compare(b, a);
-
-    protected final OpSelectors.DescriptorSelector descriptorSelector;
-    protected final OpSelectors.PdSelector pdSelector;
-    protected final OpSelectors.MonotonicClock clock;
-    protected final SystemUnderTest sut;
-    protected final QueryGenerator rangeSelector;
-
-    protected final DataTracker tracker;
-
-    private final SchemaSpec schema;
-
-    public ExhaustiveChecker(Run run)
-    {
-        this.descriptorSelector = run.descriptorSelector;
-        this.pdSelector = run.pdSelector;
-        this.tracker = run.tracker;
-        this.schema = run.schemaSpec;
-        this.sut = run.sut;
-        this.clock = run.clock;
-        this.rangeSelector = run.rangeSelector;
-    }
-
-    public void validate(Query query)
-    {
-        validatePartitionState(query,
-                               () -> {
-                                   while (!Thread.currentThread().isInterrupted())
-                                   {
-                                       try
-                                       {
-                                           return SelectHelper.execute(sut, clock, query);
-                                       }
-                                       catch (Throwable t)
-                                       {
-                                           logger.error(String.format("Caught error while trying execute query %s", query),
-                                                        t);
-                                       }
-                                   }
-                                   throw new RuntimeException("Interrupted");
-                               });
-    }
-
-    void validatePartitionState(Query query, Supplier<List<ResultSetRow>> rowsSupplier)
-    {
-        // Before we execute SELECT, we know what was the lts of operation that is guaranteed to be visible
-        long visibleLtsBound = tracker.maxConsecutiveFinished();
-
-        // TODO: when we implement a reorder buffer through a bitmap, we can just grab a bitmap _before_,
-        //       and know that a combination of `consecutive` + `bitmap` gives us _all possible guaranteed-to-be-seen_ values
-        List<ResultSetRow> rows = rowsSupplier.get();
-
-        // by the time SELECT done, we grab max "possible" lts
-        long maxSeenLts = tracker.maxStarted();
-        PartitionState partitionState = inflatePartitionState(maxSeenLts, query);
-        NavigableMap<Long, List<Operation>> operations = partitionState.operations;
-        LongComparator cmp = FORWARD_COMPARATOR;
-        if (query.reverse)
-        {
-            operations = partitionState.operations.descendingMap();
-            cmp = REVERSE_COMPARATOR;
-        }
-
-        if (!rows.isEmpty() && operations.isEmpty())
-        {
-            throw new ValidationException(String.format("Returned rows are not empty, but there were no records in the event log.\nRows: %s\nMax seen LTS: %s\nQuery: %s",
-                                                        rows, maxSeenLts, query));
-        }
-
-        PeekingIterator<ResultSetRow> rowIter = Iterators.peekingIterator(rows.iterator());
-
-        // TODO: these two are here only for debugging/logging purposes
-        List<ResultSetRow> validatedRows = new ArrayList<>();
-        List<Long> validatedNoRows = new ArrayList<>();
-        try
-        {
-            for (Map.Entry<Long, List<Operation>> entry : operations.entrySet())
-            {
-                long cd = entry.getKey();
-                Iterator<Operation> modificationIter = entry.getValue().iterator();
-
-                // Found a row that is present both in the model and in the resultset
-                if (rowIter.hasNext() && rowIter.peek().cd == cd)
-                {
-                    ResultSetRow row = rowIter.next();
-                    RowValidationState validationState = new RowValidationState(row, visibleLtsBound, maxSeenLts, partitionState.rangeTombstones);
-
-                    // TODO: We only need to go for as long as we explain every column. In fact, we make state _less_ precise by allowing
-                    // to continue moving back in time. So far this hasn't proven to be a source of any issues, but we should fix that.
-                    // One of the examples is a deletion followed by no writes, or a row write that completely overwrites all columns.
-                    while (modificationIter.hasNext())
-                        validationState.process(modificationIter.next());
-
-                    long minLts = Long.MAX_VALUE;
-                    for (int col = 0; col < validationState.causingOperations.length; col++)
-                    {
-                        long colLts = row.lts[col];
-                        if (colLts != NO_TIMESTAMP && colLts < minLts)
-                            minLts = colLts;
-
-                        long rowValueDescr = row.vds[col];
-                        switch (validationState.columnStates[col])
-                        {
-                            case REMOVED:
-                                if (colLts != NO_TIMESTAMP || rowValueDescr != NIL_DESCR)
-                                    throw new ValidationException("Inconsistency found: value of the column %d was supposed to be removed", col);
-                                break;
-                            case OBSERVED:
-                                if (colLts == NO_TIMESTAMP || rowValueDescr == NIL_DESCR)
-                                    throw new ValidationException("Inconsistency found: value of the column %d was supposed to be observed", col);
-                                break;
-                            case UNOBSERVED:
-                                if (colLts != NO_TIMESTAMP || rowValueDescr != NIL_DESCR)
-                                    throw new ValidationException("Inconsistency found: value of the column %d was never written. " +
-                                                                  "Row timestamp: %d. " +
-                                                                  "Row descriptor: %d",
-                                                                  col, colLts, rowValueDescr);
-                        }
-                    }
-
-                    // for any visible row, we have to make sure it is not shadowed by any range tombstones
-                    for (Ranges.Range rt : partitionState.rangeTombstones.shadowedBy(cd, minLts))
-                    {
-                        if (rt.timestamp <= visibleLtsBound)
-                        {
-                            throw new ValidationException("Row was supposed to be shadowed by the range tombstone." +
-                                                          "\nRow: %s" +
-                                                          "\nRange tombstone: %s" +
-                                                          "\nMin LTS: %d" +
-                                                          "\nVisible LTS Bound: %d",
-                                                          row,
-                                                          rt,
-                                                          minLts,
-                                                          visibleLtsBound);
-                        }
-                    }
-
-                    validatedRows.add(row);
-                }
-                // Modifications for this clustering are are not visible
-                else
-                {
-                    validateNoRow(cd, visibleLtsBound, modificationIter, partitionState.rangeTombstones);
-                    validatedNoRows.add(cd);
-
-                    // Row is not present in the resultset, and we currently look at modifications with a clustering past it
-                    if (rowIter.hasNext() && cmp.compare(rowIter.peek().cd, cd) < 0)
-                        throw new ValidationException("Couldn't find a corresponding explanation for the row %s in the model. %s",
-                                                      rowIter.next(),
-                                                      partitionState.operations.get(cd));
-                }
-            }
-
-            if (rowIter.hasNext())
-                throw new ValidationException(String.format("Observed unvalidated rows : %s", Iterators.toString(rowIter)));
-        }
-        catch (Throwable t)
-        {
-            throw new ValidationException(String.format("Caught exception while validating the resultset %s." +
-                                                        "\nRow Iter Peek: %s" +
-                                                        "\nValidated no rows:\n%s" +
-                                                        "\nValidated rows:\n%s" +
-                                                        "\nRows:\n%s",
-                                                        query,
-                                                        rowIter.hasNext() ? rowIter.peek() : null,
-                                                        validatedNoRows,
-                                                        validatedRows.stream().map(Object::toString).collect(Collectors.joining(",\n")),
-                                                        rows.stream().map(Object::toString).collect(Collectors.joining(",\n"))),
-                                          t);
-        }
-    }
-
-    // there seems to be some issue here, when validating too many in-flight rows
-    public void validateNoRow(long cd, long visibleLtsBound, Iterator<Operation> ops, Ranges rangeTombstones)
-    {
-        // Row was never written
-        if (!ops.hasNext())
-            return;
-
-        // There should have been at least one removal followed by no live updates
-        List<Operation> visibleWrites = new ArrayList<>();
-        while (ops.hasNext())
-        {
-            Operation op = ops.next();
-            boolean isVisible = op.lts <= visibleLtsBound;
-
-            switch (descriptorSelector.operationType(op.pd, op.lts, op.opId))
-            {
-                // we are going from the newest operation to the oldest one;
-                // eventually, we're getting to some write. if it should have propagated, we save it to overwrites
-                // if we find a row delete
-                case WRITE:
-                    if (isVisible)
-                        visibleWrites.add(op);
-
-                    break;
-                case DELETE_COLUMN:
-                    // TODO: continue only in case of non-compact storage. In case of compact storage, deletion of all cells results into row deletion
-                    if (!descriptorSelector.columnMask(op.pd, op.lts, op.opId).allUnset())
-                        continue;
-                    // otherwise, fall through, since we can't distinguish this from row delete
-                case DELETE_ROW:
-                    // row delete, followed by any number of non-propagated writes explains why descriptor is not visible
-                    if (!visibleWrites.isEmpty())
-                    {
-                        long newestVisibleLts = visibleWrites.get(0).lts;
-                        if (rangeTombstones.isShadowed(cd, newestVisibleLts))
-                            return;
-
-                        // if we have at least one write not shadowed by delete, we have an error
-                        throw new ValidationException("While validating %d, expected row not to be visible: a deletion, followed by no overwrites or all incomplete writes, but found %s." +
-                                                      "\nRange tombstones: %s",
-                                                      cd,
-                                                      visibleWrites,
-                                                      rangeTombstones.newerThan(newestVisibleLts));
-                    }
-                    return;
-            }
-        }
-
-        if (!visibleWrites.isEmpty())
-        {
-            long newestVisibleLts = visibleWrites.get(0).lts;
-            if (rangeTombstones.isShadowed(cd, newestVisibleLts))
-                return;
-
-            throw new ValidationException("While validating %d, expected row not to be visible: a deletion, followed by no overwrites or all incomplete writes, but found %s." +
-                                          "\nRange tombstones: %s",
-                                          cd,
-                                          visibleWrites,
-                                          rangeTombstones.newerThan(newestVisibleLts));
-        }
-    }
-
-    public static class PartitionState
-    {
-        public final NavigableMap<Long, List<Operation>> operations;
-        public final Ranges rangeTombstones;
-
-        public PartitionState(NavigableMap<Long, List<Operation>> operations, Ranges rangeTombstones)
-        {
-            this.operations = operations;
-            this.rangeTombstones = rangeTombstones;
-        }
-    }
-
-    public PartitionState inflatePartitionState(long maxLts, Query query)
-    {
-        long currentLts = pdSelector.maxLtsFor(query.pd);
-
-        NavigableMap<Long, List<Operation>> operations = new TreeMap<>();
-        List<Ranges.Range> ranges = new ArrayList<>();
-
-        PartitionVisitor partitionVisitor = new AbstractPartitionVisitor(pdSelector, descriptorSelector, schema)
-        {
-            public void operation(long lts, long pd, long cd, long m, long opId)
-            {
-                OpSelectors.OperationKind opType = descriptorSelector.operationType(pd, lts, opId);
-                if (opType == OpSelectors.OperationKind.DELETE_RANGE)
-                {
-                    ranges.add(maybeWrap(lts, opId, rangeSelector.inflate(lts, opId, Query.QueryKind.CLUSTERING_RANGE).toRange(lts)));
-                }
-                else if (opType == OpSelectors.OperationKind.DELETE_SLICE)
-                {
-                    ranges.add(maybeWrap(lts, opId, rangeSelector.inflate(lts, opId, Query.QueryKind.CLUSTERING_SLICE).toRange(lts)));
-                }
-                else if (query.match(cd)) // skip descriptors that are out of range
-                {
-                    operations.computeIfAbsent(cd, (cd_) -> new ArrayList<>());
-                    Operation operation = new Operation(pd, cd, lts, opId, opType);
-                    operations.get(cd).add(operation);
-                }
-            }
-        };
-
-        while (currentLts >= 0)
-        {
-            if (currentLts <= maxLts)
-                partitionVisitor.visitPartition(currentLts);
-
-            currentLts = pdSelector.prevLts(currentLts);
-        }
-        return new PartitionState(operations, new Ranges(ranges));
-    }
-
-    private static Ranges.Range maybeWrap(long lts, long opId, Ranges.Range range)
-    {
-        if (logger.isDebugEnabled())
-            return new DebugRange(lts, opId, range.minBound, range.maxBound, range.minInclusive, range.maxInclusive, range.timestamp);
-
-        return range;
-    }
-
-    private static class DebugRange extends Ranges.Range
-    {
-        private final long lts;
-        private final long opId;
-
-        public DebugRange(long lts, long opId,
-                          long minBound, long maxBound, boolean minInclusive, boolean maxInclusive, long timestamp)
-        {
-            super(minBound, maxBound, minInclusive, maxInclusive, timestamp);
-            this.lts = lts;
-            this.opId = opId;
-        }
-
-        public String toString()
-        {
-            return super.toString() +
-                   "(lts=" + lts +
-                   ", opId=" + opId +
-                   ')';
-        }
-    }
-
-    public String toString()
-    {
-        return "ExhaustiveChecker{" + tracker.toString() + '}';
-    }
-
-    public class RowValidationState
-    {
-        private final ColumnState[] columnStates;
-        private final Operation[] causingOperations;
-        private final PeekingIterator<Long> ltsIterator;
-        private final Ranges rangeTombstones;
-        private final ResultSetRow row;
-        private final long visibleLtsBound;
-        private final long inFlightLtsBound;
-
-        public RowValidationState(ResultSetRow row, long visibleLtsBound, long inFlightLtsBound, Ranges rangeTombstones)
-        {
-            this.row = row;
-            this.visibleLtsBound = visibleLtsBound;
-            this.inFlightLtsBound = inFlightLtsBound;
-
-            this.columnStates = new ColumnState[row.vds.length];
-            Arrays.fill(columnStates, ColumnState.UNOBSERVED);
-
-            this.causingOperations = new Operation[columnStates.length];
-            long[] ltsVector = new long[row.lts.length];
-            System.arraycopy(row.lts, 0, ltsVector, 0, ltsVector.length);
-            this.ltsIterator = Iterators.peekingIterator(ltsIterator(ltsVector).iterator());
-            this.rangeTombstones = rangeTombstones;
-        }
-
-        public void process(Operation op)
-        {
-            if (ltsIterator.hasNext() && op.lts > ltsIterator.peek())
-                ltsIterator.next();
-
-            assert row.pd == op.pd : String.format("Row and operation descriptors do not match: %d != %d", row.pd, op.pd);
-
-            switch (descriptorSelector.operationType(op.pd, op.lts, op.opId))
-            {
-                case WRITE:
-                    processInsert(op);
-                    break;
-                case DELETE_ROW:
-                    // In case of a visible row, deletion that was followed with a write can be considered equivalent
-                    // to a deletion of all column values.
-                    processDelete(op, schema.ALL_COLUMNS_BITSET);
-                    break;
-                case DELETE_COLUMN:
-                    BitSet mask = descriptorSelector.columnMask(row.pd, op.lts, op.opId);
-                    if (mask.allUnset())
-                        throw new IllegalArgumentException("Can't have a delete column query with no columns set. Column mask: " + mask);
-
-                    processDelete(op, mask);
-                    break;
-            }
-        }
-
-        private void transitionState(int idx, ColumnState newState, Operation modification)
-        {
-            ColumnState oldState = columnStates[idx];
-            switch (newState)
-            {
-                case UNOBSERVED:
-                    throw new IllegalArgumentException("Can not transition to UNOBSERVED state");
-                case REMOVED:
-                    if (!(oldState == ColumnState.UNOBSERVED || oldState == ColumnState.REMOVED))
-                        throw new ValidationException("Can not transition from %s to %s.", oldState, newState);
-                    break;
-                case OBSERVED:
-                    if (!(oldState == ColumnState.OBSERVED || oldState == ColumnState.UNOBSERVED))
-                        throw new ValidationException("Can not transition from %s to %s.", oldState, newState);
-                    break;
-            }
-            columnStates[idx] = newState;
-            causingOperations[idx] = modification;
-        }
-
-        private void processInsert(Operation op)
-        {
-            if (op.lts > inFlightLtsBound)
-                throw new IllegalStateException(String.format("Observed LTS not yet recorded by this model: %s. Max seen LTS: %s",
-                                                              op.lts, inFlightLtsBound));
-
-            boolean isVisible = op.lts <= visibleLtsBound;
-
-            long[] inflatedDescriptors = descriptorSelector.vds(op.pd, op.cd, op.lts, op.opId, schema);
-            for (int col = 0; col < row.lts.length; col++)
-            {
-                final long valueDescriptor = inflatedDescriptors[col];
-
-                // Write is visible
-                if (row.vds[col] == valueDescriptor && row.lts[col] == op.lts)
-                {
-                    transitionState(col, ColumnState.OBSERVED, op);
-                    continue;
-                }
-
-                if (!isVisible                            // write has never propagated
-                    || valueDescriptor == UNSET_DESCR     // this modification did not make this write
-                    || (columnStates[col] == ColumnState.REMOVED && causingOperations[col].lts >= op.lts)  // confirmed that this column was removed later
-                    || (columnStates[col] == ColumnState.OBSERVED && causingOperations[col].lts >= op.lts)  // we could confirm the overwrite earlier
-                    // TODO: that won't work. To reproduce this, take testDetectsRemovedColumn with range tombstones. Removed column will have a timestamp of min long,
-                    //       and _any_ range tombstone is going to be able to shadow it.
-                    || rangeTombstones.isShadowed(row.cd, row.lts[col])) // if this row's lts is shadowed, we can be certain that whole row is shadowed
-                    continue;
-
-                throw new ValidationException("Error caught while validating column %d. " +
-                                              "Expected value: %d. " +
-                                              "Modification should have been visible but was not." +
-                                              "\nOperation: %s" +
-                                              "\nRow: %s" +
-                                              "\nRow ID: %d" +
-                                              "\nColumn States: %s " +
-                                              "\nRange tombstones: %s",
-                                              col,
-                                              valueDescriptor,
-                                              op,
-                                              row,
-                                              descriptorSelector.rowId(row.pd, row.lts[col], row.cd),
-                                              Arrays.toString(columnStates),
-                                              rangeTombstones);
-            }
-        }
-
-        public void processDelete(Operation op, BitSet mask)
-        {
-            boolean isVisible = op.lts <= visibleLtsBound;
-
-            for (int col = 0; col < columnStates.length; col++)
-            {
-                // Deletion must have propagated
-                if (mask.isSet(col) && row.lts[col] == NO_TIMESTAMP)
-                {
-                    transitionState(col, ColumnState.REMOVED, op);
-                    continue;
-                }
-
-                if (!isVisible
-                    || mask.isSet(col)
-                    || columnStates[col] != ColumnState.OBSERVED
-                    || op.lts < causingOperations[col].lts)
-                    continue;
-
-                throw new ValidationException("Error caught wile validating column %d. " +
-                                              "Delete operation with lts %s should have been visible or shadowed by the later update, but was not. " +
-                                              "\nOperation: %s" +
-                                              "\nRow: %s" +
-                                              "\nColumn States: %s",
-                                              col, op.lts,
-                                              op, row, Arrays.toString(columnStates));
-            }
-        }
-
-        public String toString()
-        {
-            return String.format("Validated: %s." +
-                                 "\nObserved timestamps: %s",
-                                 Arrays.toString(columnStates),
-                                 row);
-        }
-    }
-
-    public static class Operation implements Comparable<Operation>
-    {
-        public final long pd;
-        public final long lts;
-        public final long cd;
-        public final long opId;
-        public final OpSelectors.OperationKind op;
-
-        public Operation(long pd, long cd, long lts, long opId, OpSelectors.OperationKind op)
-        {
-            this.pd = pd;
-            this.lts = lts;
-            this.cd = cd;
-            this.opId = opId;
-            this.op = op;
-        }
-
-        private static final Comparator<Operation> comparator = Comparator.comparingLong((Operation a) -> a.lts);
-
-        public int compareTo(Operation other)
-        {
-            // reverse order
-            return comparator.compare(other, this);
-        }
-
-        public boolean equals(Object o)
-        {
-            if (this == o) return true;
-            if (o == null || getClass() != o.getClass()) return false;
-            Operation operation = (Operation) o;
-            return pd == operation.pd &&
-                   lts == operation.lts &&
-                   cd == operation.cd &&
-                   opId == operation.opId;
-        }
-
-        public int hashCode()
-        {
-            return Objects.hash(pd, lts, cd, opId);
-        }
-
-        public String toString()
-        {
-            return "Operation{" +
-                   "pd=" + pd +
-                   ", cd=" + cd +
-                   ", lts=" + lts +
-                   ", opId=" + opId +
-                   ", op=" + op +
-                   '}';
-        }
-    }
-
-    public enum ColumnState
-    {
-        UNOBSERVED,
-        REMOVED,
-        OBSERVED
-    }
-
-    public static interface LongComparator
-    {
-        int compare(long o1, long o2);
-    }
-
-    public static List<Long> ltsIterator(long[] lts)
-    {
-        long[] sorted = Arrays.copyOf(lts, lts.length);
-        Arrays.sort(sorted);
-        List<Long> deduplicated = new ArrayList<>(lts.length);
-        for (int i = 0; i < sorted.length; i++)
-        {
-            if (sorted[i] <= 0)
-                continue;
-
-            if (deduplicated.size() == 0 || deduplicated.get(deduplicated.size() - 1) != sorted[i])
-                deduplicated.add(sorted[i]);
-        }
-
-        return deduplicated;
-    }
-}
diff --git a/harry-core/src/harry/model/Model.java b/harry-core/src/harry/model/Model.java
index 3168c8a..d6ab865 100644
--- a/harry-core/src/harry/model/Model.java
+++ b/harry-core/src/harry/model/Model.java
@@ -19,6 +19,7 @@
 package harry.model;
 
 import harry.core.Run;
+import harry.reconciler.Reconciler;
 import harry.runner.Query;
 
 public interface Model
@@ -34,19 +35,13 @@ public interface Model
 
     class ValidationException extends RuntimeException
     {
-        public ValidationException(String message)
+        public ValidationException(String partitionState, String observedState, String format, Object... objects)
         {
-            super(message);
+            super(String.format(format, objects) +
+                  "\nPartition state:\n" + partitionState +
+                  "\nObserved state:\n" + observedState);
         }
+    }
 
-        public ValidationException(String format, Object... objects)
-        {
-            super(String.format(format, objects));
-        }
 
-        public ValidationException(String message, Throwable cause)
-        {
-            super(message, cause);
-        }
-    }
 }
\ No newline at end of file
diff --git a/harry-core/src/harry/model/DoNothingModel.java b/harry-core/src/harry/model/NoOpChecker.java
similarity index 75%
rename from harry-core/src/harry/model/DoNothingModel.java
rename to harry-core/src/harry/model/NoOpChecker.java
index 35b5f86..4cf4606 100644
--- a/harry-core/src/harry/model/DoNothingModel.java
+++ b/harry-core/src/harry/model/NoOpChecker.java
@@ -18,17 +18,22 @@
 
 package harry.model;
 
-import harry.core.Configuration;
+import harry.core.Run;
+import harry.model.sut.SystemUnderTest;
 import harry.runner.Query;
 
-public class DoNothingModel implements Model
+public class NoOpChecker implements Model
 {
-    public void validate(Query query)
+    private final Run run;
+
+    public NoOpChecker(Run run)
     {
+        this.run = run;
     }
 
-    public Configuration.ModelConfiguration toConfig()
+    public void validate(Query query)
     {
-        throw new RuntimeException("not implemented");
+        run.sut.execute(query.toSelectStatement(),
+                        SystemUnderTest.ConsistencyLevel.ALL);
     }
 }
diff --git a/harry-core/src/harry/model/OpSelectors.java b/harry-core/src/harry/model/OpSelectors.java
index 7d7ad6d..3adbb95 100644
--- a/harry-core/src/harry/model/OpSelectors.java
+++ b/harry-core/src/harry/model/OpSelectors.java
@@ -19,8 +19,8 @@
 package harry.model;
 
 import java.util.EnumMap;
+import java.util.List;
 import java.util.Map;
-import java.util.function.Function;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -37,15 +37,15 @@ import harry.util.BitSet;
 import static harry.generators.DataGenerators.UNSET_DESCR;
 
 /**
- * Row (uninflated) data selectors. Not calling them generators, since their output is entirely
+ * Row (deflated) data selectors. Not calling them generators, since their output is entirely
  * deterministic, and for each input they are able to produce a single output.
  * <p>
  * This is more or less a direct translation of the formalization.
  * <p>
  * All functions implemented by this interface have to _always_ produce same outputs for same inputs.
- * Most of the functions, with the exception of real-time clock translations, shouold be pure.
+ * Most of the functions, with the exception of real-time clock translations, should be pure.
  * <p>
- * Functions that are reverse of their coutnerparts are prefixed with "un"
+ * Functions that are reverse of their counterparts are prefixed with "un"
  */
 public interface OpSelectors
 {
@@ -179,23 +179,34 @@ public interface OpSelectors
 
         public long[] vds(long pd, long cd, long lts, long opId, SchemaSpec schema)
         {
-            long[] vds = new long[schema.regularColumns.size()];
-            BitSet mask = columnMask(pd, cd, opId);
+            return descriptors(pd, cd, lts, opId, schema.regularColumns, schema.regularColumnsMask(), schema.regularColumnsOffset);
+        }
+
+        public long[] sds(long pd, long cd, long lts, long opId, SchemaSpec schema)
+        {
+            return descriptors(pd, cd, lts, opId, schema.staticColumns, schema.staticColumnsMask(), schema.staticColumnsOffset);
+        }
 
-            for (int col = 0; col < vds.length; col++)
+        public long[] descriptors(long pd, long cd, long lts, long opId, List<ColumnSpec<?>> columns, BitSet mask, int offset)
+        {
+            long[] descriptors = new long[columns.size()];
+            BitSet setColumns = columnMask(pd, cd, opId);
+
+            for (int i = 0; i < descriptors.length; i++)
             {
-                if (mask.isSet(col))
+                int col = offset + i;
+                if (setColumns.isSet(col, mask))
                 {
-                    ColumnSpec spec = schema.regularColumns.get(col);
-                    long vd = vd(pd, cd, lts, opId, col) & Bytes.signMaskFor(spec.type.maxSize());
-                    vds[col] = vd;
+                    ColumnSpec<?> spec = columns.get(i);
+                    long vd = vd(pd, cd, lts, opId, col) & Bytes.bytePatternFor(spec.type.maxSize());
+                    descriptors[i] = vd;
                 }
                 else
                 {
-                    vds[col] = UNSET_DESCR;
+                    descriptors[i] = UNSET_DESCR;
                 }
             }
-            return vds;
+            return descriptors;
         }
 
         public abstract OperationKind operationType(long pd, long lts, long opId);
@@ -352,6 +363,7 @@ public interface OpSelectors
 
     // TODO: add weights/probabilities to this
     // TODO: this looks like a hierarchical surjection
+
     public static class ColumnSelectorBuilder
     {
         private Map<OperationKind, Surjections.Surjection<BitSet>> m;
@@ -361,39 +373,60 @@ public interface OpSelectors
             this.m = new EnumMap<>(OperationKind.class);
         }
 
-        public ColumnSelectorBuilder forAll(int regularColumnsCount)
+        public ColumnSelectorBuilder forAll(SchemaSpec schema)
         {
-            return forAll(BitSet.surjection(regularColumnsCount));
+            return forAll(schema, BitSet.surjection(schema.allColumns.size()));
         }
 
-        public ColumnSelectorBuilder forAll(Surjections.Surjection<BitSet> orig)
+        // TODO: change bitsets to take into account _all_ columns not only regulars
+        public ColumnSelectorBuilder forAll(SchemaSpec schema, Surjections.Surjection<BitSet> orig)
         {
             for (OperationKind type : OperationKind.values())
             {
                 Surjections.Surjection<BitSet> gen = orig;
-                if (type == OperationKind.DELETE_COLUMN)
+
+                switch (type)
                 {
-                    gen = (descriptor) -> {
-                        while (true)
-                        {
-                            BitSet bitSet = orig.inflate(descriptor);
-                            if (!bitSet.allUnset())
-                                return bitSet;
-
-                            descriptor = RngUtils.next(descriptor);
-                        }
-                    };
+                    case DELETE_COLUMN_WITH_STATICS:
+                        gen = (descriptor) -> {
+                            long counter = 0;
+                            while (counter <= 100)
+                            {
+                                BitSet bitSet = orig.inflate(descriptor);
+
+                                if ((schema.regularColumns.isEmpty() || !bitSet.allUnset(schema.regularColumnsMask))
+                                    && (schema.staticColumns.isEmpty() || !bitSet.allUnset(schema.staticColumnsMask)))
+                                    return bitSet;
+
+                                descriptor = RngUtils.next(descriptor);
+                                counter++;
+                            }
+                            throw new RuntimeException(String.format("Could not generate a value after %d attempts.", counter));
+                        };
+                        break;
+                    case DELETE_COLUMN:
+                        gen = (descriptor) -> {
+                            long counter = 0;
+                            while (counter <= 100)
+                            {
+                                BitSet bitSet = orig.inflate(descriptor);
+                                BitSet mask = schema.regularColumnsMask;
+
+                                if (!bitSet.allUnset(mask))
+                                    return bitSet;
+
+                                descriptor = RngUtils.next(descriptor);
+                                counter++;
+                            }
+                            throw new RuntimeException(String.format("Could not generate a value after %d attempts.", counter));
+                        };
+                        break;
                 }
                 this.m.put(type, gen);
             }
             return this;
         }
 
-        public ColumnSelectorBuilder forAll(BitSet... pickFrom)
-        {
-            return forAll(Surjections.pick(pickFrom));
-        }
-
         public ColumnSelectorBuilder forWrite(Surjections.Surjection<BitSet> gen)
         {
             m.put(OperationKind.WRITE, gen);
@@ -427,12 +460,21 @@ public interface OpSelectors
             return forColumnDelete(Surjections.pick(pickFrom));
         }
 
-        public Function<OperationKind, Surjections.Surjection<BitSet>> build()
+        public ColumnSelector build()
         {
-            return m::get;
+            return (kind, descriptor) -> m.get(kind).inflate(descriptor);
         }
     }
 
+
+    /**
+     * ColumnSelector has to return BitSet specifying _all_ columns
+     */
+    public static interface ColumnSelector
+    {
+        public BitSet columnMask(OperationKind operationKind, long descriptor);
+    }
+
     // TODO: this can actually be further improved upon. Maybe not generation-wise, this part seems to be ok,
     //       but in the way it is hooked up with the rest of the system
     public static class HierarchicalDescriptorSelector extends DefaultDescriptorSelector
@@ -442,13 +484,18 @@ public interface OpSelectors
         public HierarchicalDescriptorSelector(Rng rng,
                                               // how many parts (at most) each subsequent "level" should contain
                                               int[] fractions,
-                                              Function<OperationKind, Surjections.Surjection<BitSet>> columnMaskSelector,
-                                              Surjections.Surjection<OperationKind> operationTypeSelector,
+                                              ColumnSelector columnSelector,
+                                              OperationSelector operationSelector,
                                               Distribution modificationsPerLtsDistribution,
                                               Distribution rowsPerModificationsDistribution,
                                               int maxPartitionSize)
         {
-            super(rng, columnMaskSelector, operationTypeSelector, modificationsPerLtsDistribution, rowsPerModificationsDistribution, maxPartitionSize);
+            super(rng,
+                  columnSelector,
+                  operationSelector,
+                  modificationsPerLtsDistribution,
+                  rowsPerModificationsDistribution,
+                  maxPartitionSize);
             this.fractions = fractions;
         }
 
@@ -499,26 +546,34 @@ public interface OpSelectors
         protected final static long ROWS_PER_OPERATION_STREAM = 0x5e03812e293L;
         protected final static long BITSET_IDX_STREAM = 0x92eb607bef1L;
 
-        public static Surjections.Surjection<OperationKind> DEFAULT_OP_TYPE_SELECTOR = Surjections.enumValues(OperationKind.class);
+        public static OperationSelector DEFAULT_OP_SELECTOR = OperationSelector.weighted(Surjections.weights(45, 45, 3, 2, 2, 1, 1, 1),
+                                                                                         OperationKind.WRITE,
+                                                                                         OperationKind.WRITE_WITH_STATICS,
+                                                                                         OperationKind.DELETE_ROW,
+                                                                                         OperationKind.DELETE_COLUMN,
+                                                                                         OperationKind.DELETE_COLUMN_WITH_STATICS,
+                                                                                         OperationKind.DELETE_PARTITION,
+                                                                                         OperationKind.DELETE_RANGE,
+                                                                                         OperationKind.DELETE_SLICE);
 
         protected final OpSelectors.Rng rng;
-        protected final Surjections.Surjection<OperationKind> operationTypeSelector;
-        protected final Function<OperationKind, Surjections.Surjection<BitSet>> columnMaskSelector;
+        protected final OperationSelector operationSelector;
+        protected final ColumnSelector columnSelector;
         protected final Distribution modificationsPerLtsDistribution;
         protected final Distribution rowsPerModificationsDistribution;
         protected final int maxPartitionSize;
 
         public DefaultDescriptorSelector(OpSelectors.Rng rng,
-                                         Function<OperationKind, Surjections.Surjection<BitSet>> columnMaskSelector,
-                                         Surjections.Surjection<OperationKind> operationTypeSelector,
+                                         ColumnSelector columnMaskSelector,
+                                         OperationSelector operationSelector,
                                          Distribution modificationsPerLtsDistribution,
                                          Distribution rowsPerModificationsDistribution,
                                          int maxPartitionSize)
         {
             this.rng = rng;
 
-            this.operationTypeSelector = operationTypeSelector;
-            this.columnMaskSelector = columnMaskSelector;
+            this.operationSelector = operationSelector;
+            this.columnSelector = columnMaskSelector;
 
             this.modificationsPerLtsDistribution = modificationsPerLtsDistribution;
             this.rowsPerModificationsDistribution = rowsPerModificationsDistribution;
@@ -561,7 +616,6 @@ public interface OpSelectors
 
         protected long cd(long pd, long lts, long opId)
         {
-            assert opId <= maxPartitionSize;
             int partitionSize = maxPartitionSize();
             int clusteringOffset = clusteringOffset(lts);
             if (clusteringOffset == 0)
@@ -571,7 +625,6 @@ public interface OpSelectors
             // So if we have 10 modifications per lts and 10 rows per modification,
             // we'll visit the same row twice per lts.
             int positionInPartition = (int) ((clusteringOffset + opId) % partitionSize);
-
             return rng.prev(positionInPartition, pd);
         }
 
@@ -596,15 +649,30 @@ public interface OpSelectors
 
         public OperationKind operationType(long pd, long lts, long opId)
         {
-            return operationTypeSelector.inflate(pd ^ lts ^ opId);
+            return operationType(pd, lts, opId, partitionLevelOperationsMask(pd, lts));
+        }
+
+        // TODO: create this bitset once per lts
+        public BitSet partitionLevelOperationsMask(long pd, long lts)
+        {
+            int totalOps = opsPerModification(lts) * numberOfModifications(lts);
+            long seed = rng.randomNumber(pd, lts);
+
+            int partitionLevelOps = (int) Math.ceil(operationSelector.partitionLevelThreshold * totalOps);
+            long partitionLevelOpsMask = RngUtils.randomBits(partitionLevelOps, totalOps, seed);
+
+            return BitSet.create(partitionLevelOpsMask, totalOps);
+        }
+
+        public OperationKind operationType(long pd, long lts, long opId, BitSet partitionLevelOperationsMask)
+        {
+            return operationSelector.inflate(pd ^ lts ^ opId, partitionLevelOperationsMask.isSet((int) opId));
         }
 
         public BitSet columnMask(long pd, long lts, long opId)
         {
-            Surjections.Surjection<BitSet> gen = columnMaskSelector.apply(operationType(pd, lts, opId));
-            if (gen == null)
-                throw new IllegalArgumentException("Can't find a selector for " + gen);
-            return gen.inflate(rng.randomNumber(pd ^ lts ^ opId, BITSET_IDX_STREAM));
+            long descriptor = rng.randomNumber(pd ^ lts ^ opId, BITSET_IDX_STREAM);
+            return columnSelector.columnMask(operationType(pd, lts, opId), descriptor);
         }
 
         public long vd(long pd, long cd, long lts, long opId, int col)
@@ -620,10 +688,104 @@ public interface OpSelectors
 
     public enum OperationKind
     {
-        WRITE,
-        DELETE_ROW,
-        DELETE_COLUMN,
-        DELETE_RANGE,
-        DELETE_SLICE
+        WRITE(false),
+        WRITE_WITH_STATICS(true),
+        DELETE_PARTITION(true),
+        DELETE_ROW(false),
+        DELETE_COLUMN(false),
+        DELETE_COLUMN_WITH_STATICS(true),
+        DELETE_RANGE(false),
+        DELETE_SLICE(false);
+
+        public final boolean partititonLevel;
+
+        OperationKind(boolean partitionLevel)
+        {
+            this.partititonLevel = partitionLevel;
+        }
+    }
+
+    public static class OperationSelector
+    {
+        public final Surjections.Surjection<OperationKind> partitionLevelOperationSelector;
+        public final Surjections.Surjection<OperationKind> rowLevelOperationSelector;
+        // TODO: start using partitionLevelThreshold
+        public final double partitionLevelThreshold;
+
+        public OperationSelector(Surjections.Surjection<OperationKind> partitionLevelOperationSelector,
+                                 Surjections.Surjection<OperationKind> rowLevelOperationSelector,
+                                 double partitionLevelThreshold)
+        {
+            this.partitionLevelOperationSelector = partitionLevelOperationSelector;
+            this.rowLevelOperationSelector = rowLevelOperationSelector;
+            this.partitionLevelThreshold = partitionLevelThreshold;
+        }
+
+        public OperationKind inflate(long descriptor, boolean partitionLevel)
+        {
+            OperationKind operationKind = partitionLevel ? partitionLevelOperationSelector.inflate(descriptor) : rowLevelOperationSelector.inflate(descriptor);
+            assert operationKind.partititonLevel == partitionLevel : "Generated operation with an incorrect partition level. Check your generators.";
+            return operationKind;
+        }
+
+        public static OperationSelector weighted(Map<OperationKind, Integer> weightsMap)
+        {
+            int[] weights = new int[weightsMap.size()];
+            OperationKind[] operationKinds = new OperationKind[weightsMap.size()];
+            int i = 0;
+            for (Map.Entry<OperationKind, Integer> entry : weightsMap.entrySet())
+            {
+                weights[i] = entry.getValue();
+                operationKinds[i] = entry.getKey();
+                i++;
+            }
+            return weighted(Surjections.weights(weights), operationKinds);
+        }
+
+        public static OperationSelector weighted(long[] weights, OperationKind... operationKinds)
+        {
+            assert weights.length == operationKinds.length;
+
+            Map<OperationKind, Integer> partitionLevel = new EnumMap<OperationKind, Integer>(OperationKind.class);
+            Map<OperationKind, Integer> rowLevel = new EnumMap<OperationKind, Integer>(OperationKind.class);
+
+            int partitionLevelSum = 0;
+            int rowLevelSum = 0;
+            for (int i = 0; i < weights.length; i++)
+            {
+                int v = (int) (weights[i] >> 32);
+                if (operationKinds[i].partititonLevel)
+                {
+                    partitionLevel.put(operationKinds[i], v);
+                    partitionLevelSum += v;
+                }
+                else
+                {
+                    rowLevel.put(operationKinds[i], v);
+                    rowLevelSum += v;
+                }
+            }
+            int sum = (partitionLevelSum + rowLevelSum);
+
+            return new OperationSelector(Surjections.weighted(normalize(partitionLevel)),
+                                         Surjections.weighted(normalize(rowLevel)),
+                                         (partitionLevelSum * 1.0) / sum);
+        }
+
+        public static Map<OperationKind, Integer> normalize(Map<OperationKind, Integer> weights)
+        {
+            Map<OperationKind, Integer> normalized = new EnumMap<OperationKind, Integer>(OperationKind.class);
+            int sum = 0;
+            for (Integer value : weights.values())
+                sum += value;
+
+            for (OperationKind kind : weights.keySet())
+            {
+                double dbl = (sum * ((double) weights.get(kind)) / sum);
+                normalized.put(kind, (int) Math.round(dbl));
+            }
+
+            return normalized;
+        }
     }
 }
diff --git a/harry-core/src/harry/model/QuiescentChecker.java b/harry-core/src/harry/model/QuiescentChecker.java
index ffd50b3..a48ccce 100644
--- a/harry-core/src/harry/model/QuiescentChecker.java
+++ b/harry-core/src/harry/model/QuiescentChecker.java
@@ -34,6 +34,8 @@ import harry.runner.DataTracker;
 import harry.runner.Query;
 import harry.runner.QueryGenerator;
 
+import static harry.generators.DataGenerators.NIL_DESCR;
+
 public class QuiescentChecker implements Model
 {
     protected final OpSelectors.MonotonicClock clock;
@@ -41,14 +43,16 @@ public class QuiescentChecker implements Model
     protected final DataTracker tracker;
     protected final SystemUnderTest sut;
     protected final Reconciler reconciler;
+    protected final SchemaSpec schemaSpec;
 
     public QuiescentChecker(Run run)
     {
         this.clock = run.clock;
         this.sut = run.sut;
 
-        this.reconciler = new Reconciler(run.schemaSpec, run.pdSelector, run.descriptorSelector, run.rangeSelector);
+        this.reconciler = new Reconciler(run);
         this.tracker = run.tracker;
+        this.schemaSpec = run.schemaSpec;
     }
 
     public void validate(Query query)
@@ -66,37 +70,141 @@ public class QuiescentChecker implements Model
 
         List<ResultSetRow> actualRows = rowsSupplier.get();
         Iterator<ResultSetRow> actual = actualRows.iterator();
-        Collection<Reconciler.RowState> expectedRows = reconciler.inflatePartitionState(query.pd, maxSeenLts, query).rows(query.reverse);
+        Reconciler.PartitionState partitionState = reconciler.inflatePartitionState(query.pd, maxSeenLts, query);
+        Collection<Reconciler.RowState> expectedRows = partitionState.rows(query.reverse);
+
         Iterator<Reconciler.RowState> expected = expectedRows.iterator();
 
+        // It is possible that we only get a single row in response, and it is equal to static row
+        if (partitionState.isEmpty() && partitionState.staticRow() != null && actual.hasNext())
+        {
+            ResultSetRow actualRowState = actual.next();
+            if (actualRowState.cd != partitionState.staticRow().cd)
+                throw new ValidationException(partitionState.toString(schemaSpec),
+                                              toString(actualRows, schemaSpec),
+                                              "Found a row while model predicts statics only:" +
+                                              "\nExpected: %s" +
+                                              "\nActual: %s" +
+                                              "\nQuery: %s",
+                                              partitionState.staticRow().cd,
+                                              actualRowState, query.toSelectStatement());
+
+            for (int i = 0; i < actualRowState.vds.length; i++)
+            {
+                if (actualRowState.vds[i] != NIL_DESCR || actualRowState.lts[i] != NO_TIMESTAMP)
+                    throw new ValidationException(partitionState.toString(schemaSpec),
+                                                  toString(actualRows, schemaSpec),
+                                                  "Found a row while model predicts statics only:" +
+                                                  "\nActual: %s" +
+                                                  "\nQuery: %s" +
+                                                  "\nQuery: %s",
+                                                  actualRowState, query.toSelectStatement());
+            }
+
+            assertStaticRow(partitionState, actualRows, partitionState.staticRow(), actualRowState, query, schemaSpec);
+        }
+
         while (actual.hasNext() && expected.hasNext())
         {
             ResultSetRow actualRowState = actual.next();
             Reconciler.RowState expectedRowState = expected.next();
             // TODO: this is not necessarily true. It can also be that ordering is incorrect.
             if (actualRowState.cd != expectedRowState.cd)
-                throw new ValidationException("Found a row in the model that is not present in the resultset:\nExpected: %s\nActual: %s",
-                                              expectedRowState, actualRowState);
+                throw new ValidationException(partitionState.toString(schemaSpec),
+                                              toString(actualRows, schemaSpec),
+                                              "Found a row in the model that is not present in the resultset:" +
+                                              "\nExpected: %s" +
+                                              "\nActual: %s" +
+                                              "\nQuery: %s",
+                                              expectedRowState.toString(schemaSpec),
+                                              actualRowState, query.toSelectStatement());
 
             if (!Arrays.equals(actualRowState.vds, expectedRowState.vds))
-                throw new ValidationException("Returned row state doesn't match the one predicted by the model:\nExpected: %s (%s)\nActual:   %s (%s).",
-                                              Arrays.toString(expectedRowState.vds), expectedRowState,
-                                              Arrays.toString(actualRowState.vds), actualRowState);
+                throw new ValidationException(partitionState.toString(schemaSpec),
+                                              toString(actualRows, schemaSpec),
+                                              "Returned row state doesn't match the one predicted by the model:" +
+                                              "\nExpected: %s (%s)" +
+                                              "\nActual:   %s (%s)." +
+                                              "\nQuery: %s",
+                                              Arrays.toString(expectedRowState.vds), expectedRowState.toString(schemaSpec),
+                                              Arrays.toString(actualRowState.vds), actualRowState,
+                                              query.toSelectStatement());
 
             if (!Arrays.equals(actualRowState.lts, expectedRowState.lts))
-                throw new ValidationException("Timestamps in the row state don't match ones predicted by the model:\nExpected: %s (%s)\nActual:   %s (%s).",
-                                              Arrays.toString(expectedRowState.lts), expectedRowState,
-                                              Arrays.toString(actualRowState.lts), actualRowState);
+                throw new ValidationException(partitionState.toString(schemaSpec),
+                                              toString(actualRows, schemaSpec),
+                                              "Timestamps in the row state don't match ones predicted by the model:" +
+                                              "\nExpected: %s (%s)" +
+                                              "\nActual:   %s (%s)." +
+                                              "\nQuery: %s",
+                                              Arrays.toString(expectedRowState.lts), expectedRowState.toString(schemaSpec),
+                                              Arrays.toString(actualRowState.lts), actualRowState,
+                                              query.toSelectStatement());
+
+            if (partitionState.staticRow() != null || actualRowState.sds != null || actualRowState.slts != null)
+                assertStaticRow(partitionState, actualRows, partitionState.staticRow(), actualRowState, query, schemaSpec);
         }
 
         if (actual.hasNext() || expected.hasNext())
         {
-            throw new ValidationException("Expected results to have the same number of results, but %s result iterator has more results." +
+            throw new ValidationException(partitionState.toString(schemaSpec),
+                                          toString(actualRows, schemaSpec),
+                                          "Expected results to have the same number of results, but %s result iterator has more results." +
                                           "\nExpected: %s" +
-                                          "\nActual:   %s",
+                                          "\nActual:   %s" +
+                                          "\nQuery: %s",
                                           actual.hasNext() ? "actual" : "expected",
                                           expectedRows,
-                                          actualRows);
+                                          actualRows,
+                                          query.toSelectStatement());
         }
     }
+
+    public static void assertStaticRow(Reconciler.PartitionState partitionState,
+                                       List<ResultSetRow> actualRows,
+                                       Reconciler.RowState staticRow,
+                                       ResultSetRow actualRowState,
+                                       Query query,
+                                       SchemaSpec schemaSpec)
+    {
+        if (!Arrays.equals(staticRow.vds, actualRowState.sds))
+            throw new ValidationException(partitionState.toString(schemaSpec),
+                                          toString(actualRows, schemaSpec),
+                                          "Returned static row state doesn't match the one predicted by the model:" +
+                                          "\nExpected: %s (%s)" +
+                                          "\nActual:   %s (%s)." +
+                                          "\nQuery: %s",
+                                          Arrays.toString(staticRow.vds), staticRow.toString(schemaSpec),
+                                          Arrays.toString(actualRowState.sds), actualRowState,
+                                          query.toSelectStatement());
+
+        if (!Arrays.equals(staticRow.lts, actualRowState.slts))
+            throw new ValidationException(partitionState.toString(schemaSpec),
+                                          toString(actualRows, schemaSpec),
+                                          "Timestamps in the static row state don't match ones predicted by the model:" +
+                                          "\nExpected: %s (%s)" +
+                                          "\nActual:   %s (%s)." +
+                                          "\nQuery: %s",
+                                          Arrays.toString(staticRow.lts), staticRow.toString(schemaSpec),
+                                          Arrays.toString(actualRowState.slts), actualRowState,
+                                          query.toSelectStatement());
+    }
+
+    public static String toString(Collection<Reconciler.RowState> collection, SchemaSpec schema)
+    {
+        StringBuilder builder = new StringBuilder();
+
+        for (Reconciler.RowState rowState : collection)
+            builder.append(rowState.toString(schema)).append("\n");
+        return builder.toString();
+    }
+
+    public static String toString(List<ResultSetRow> collection, SchemaSpec schema)
+    {
+        StringBuilder builder = new StringBuilder();
+
+        for (ResultSetRow rowState : collection)
+            builder.append(rowState.toString(schema)).append("\n");
+        return builder.toString();
+    }
 }
\ No newline at end of file
diff --git a/harry-core/src/harry/model/SelectHelper.java b/harry-core/src/harry/model/SelectHelper.java
index 2b50339..70d1eb2 100644
--- a/harry-core/src/harry/model/SelectHelper.java
+++ b/harry-core/src/harry/model/SelectHelper.java
@@ -19,6 +19,7 @@
 package harry.model;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 
@@ -53,6 +54,9 @@ public class SelectHelper
 
         if (includeWriteTime)
         {
+            for (ColumnSpec<?> column : schema.staticColumns)
+                select.writeTime(column.name);
+
             for (ColumnSpec<?> column : schema.regularColumns)
                 select.writeTime(column.name);
         }
@@ -67,6 +71,20 @@ public class SelectHelper
         return new CompiledStatement(where.toString(), bindingsArr);
     }
 
+    public static CompiledStatement count(SchemaSpec schema, long pd)
+    {
+        Select.Selection select = QueryBuilder.select();
+        select.countAll();
+
+        Select.Where where = select.from(schema.keyspace, schema.table).where();
+        List<Object> bindings = new ArrayList<>(schema.partitionKeys.size());
+
+        addRelations(schema, where, bindings, pd, Collections.emptyList());
+
+        Object[] bindingsArr = bindings.toArray(new Object[bindings.size()]);
+        return new CompiledStatement(where.toString(), bindingsArr);
+    }
+
     private static void addRelations(SchemaSpec schema, Select.Where where, List<Object> bindings, long pd, List<Relation> relations)
     {
         schema.inflateRelations(pd,
@@ -95,21 +113,32 @@ public class SelectHelper
     {
         Object[] partitionKey = new Object[schema.partitionKeys.size()];
         Object[] clusteringKey = new Object[schema.clusteringKeys.size()];
+        Object[] staticColumns = new Object[schema.staticColumns.size()];
         Object[] regularColumns = new Object[schema.regularColumns.size()];
 
         System.arraycopy(result, 0, partitionKey, 0, partitionKey.length);
         System.arraycopy(result, partitionKey.length, clusteringKey, 0, clusteringKey.length);
-        System.arraycopy(result, partitionKey.length + clusteringKey.length, regularColumns, 0, regularColumns.length);
+        System.arraycopy(result, partitionKey.length + clusteringKey.length, staticColumns, 0, staticColumns.length);
+        System.arraycopy(result, partitionKey.length + clusteringKey.length + staticColumns.length, regularColumns, 0, regularColumns.length);
+
+        long[] slts = new long[schema.staticColumns.size()];
+        for (int i = 0; i < slts.length; i++)
+        {
+            Object v = result[schema.allColumns.size() + i];
+            slts[i] = v == null ? Model.NO_TIMESTAMP : clock.lts((long) v);
+        }
 
         long[] lts = new long[schema.regularColumns.size()];
         for (int i = 0; i < lts.length; i++)
         {
-            Object v = result[schema.allColumns.size() + i];
+            Object v = result[schema.allColumns.size() + slts.length + i];
             lts[i] = v == null ? Model.NO_TIMESTAMP : clock.lts((long) v);
         }
 
         return new ResultSetRow(schema.deflatePartitionKey(partitionKey),
                                 schema.deflateClusteringKey(clusteringKey),
+                                schema.staticColumns.isEmpty() ? null : schema.deflateStaticColumns(staticColumns),
+                                schema.staticColumns.isEmpty() ? null : slts,
                                 schema.deflateRegularColumns(regularColumns),
                                 lts);
     }
diff --git a/harry-core/src/harry/model/StatelessVisibleRowsChecker.java b/harry-core/src/harry/model/StatelessVisibleRowsChecker.java
deleted file mode 100644
index 8573b3b..0000000
--- a/harry-core/src/harry/model/StatelessVisibleRowsChecker.java
+++ /dev/null
@@ -1,114 +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;
-
-import java.util.List;
-import java.util.function.Supplier;
-
-import harry.core.Configuration;
-import harry.core.Run;
-import harry.data.ResultSetRow;
-import harry.ddl.SchemaSpec;
-import harry.model.sut.SystemUnderTest;
-import harry.runner.Query;
-import harry.runner.QueryGenerator;
-
-import static harry.model.VisibleRowsChecker.descendingIterator;
-
-/**
- * A simple model to check whether or not the rows reported as visible by the database are reflected in
- * the model.
- */
-public class StatelessVisibleRowsChecker implements Model
-{
-    protected final OpSelectors.PdSelector pdSelector;
-    protected final OpSelectors.DescriptorSelector descriptorSelector;
-    protected final OpSelectors.MonotonicClock clock;
-    protected final SystemUnderTest sut;
-
-    protected final SchemaSpec schema;
-
-    public StatelessVisibleRowsChecker(Run run)
-    {
-        this.pdSelector = run.pdSelector;
-        this.descriptorSelector = run.descriptorSelector;
-        this.schema = run.schemaSpec;
-        this.clock = run.clock;
-        this.sut = run.sut;
-    }
-
-    public void validate(Query query)
-    {
-        validatePartitionState(query,
-                               () -> SelectHelper.execute(sut, clock, query));
-    }
-
-    public Configuration.ModelConfiguration toConfig()
-    {
-        throw new RuntimeException("not implemented");
-    }
-
-    void validatePartitionState(Query query, Supplier<List<ResultSetRow>> rowsSupplier)
-    {
-        // we ignore Query here, since our criteria for checking in this model is presence of the row in the resultset
-        long pd = query.pd;
-
-        List<ResultSetRow> rows = rowsSupplier.get();
-
-        for (ResultSetRow row : rows)
-        {
-            VisibleRowsChecker.LongIterator rowLtsIter = descendingIterator(row.lts);
-            VisibleRowsChecker.LongIterator modelLtsIter = descendingIterator(pdSelector, pd);
-
-            outer:
-            while (rowLtsIter.hasNext())
-            {
-                long rowLts = rowLtsIter.nextLong();
-
-                if (rowLts == NO_TIMESTAMP)
-                    continue;
-
-                if (!modelLtsIter.hasNext())
-                    throw new ValidationException(String.format("Model iterator is exhausted, could not verify %d lts for the row: \n%s %s",
-                                                                rowLts, row, query));
-
-                while (modelLtsIter.hasNext())
-                {
-                    long modelLts = modelLtsIter.nextLong();
-                    if (modelLts > rowLts)
-                        continue;
-                    if (modelLts < rowLts)
-                        throw new RuntimeException("Can't find a corresponding event id in the model for: " + rowLts + " " + modelLts);
-                    for (int col = 0; col < row.lts.length; col++)
-                    {
-                        if (row.lts[col] != rowLts)
-                            continue;
-
-                        long m = descriptorSelector.modificationId(pd, row.cd, rowLts, row.vds[col], col);
-                        long vd = descriptorSelector.vd(pd, row.cd, rowLts, m, col);
-
-                        if (vd != row.vds[col])
-                            throw new RuntimeException("Can't verify the row");
-                    }
-                    continue outer;
-                }
-            }
-        }
-    }
-}
\ No newline at end of file
diff --git a/harry-core/src/harry/model/VisibleRowsChecker.java b/harry-core/src/harry/model/VisibleRowsChecker.java
deleted file mode 100644
index 902bf07..0000000
--- a/harry-core/src/harry/model/VisibleRowsChecker.java
+++ /dev/null
@@ -1,281 +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;
-
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
-import java.util.function.Supplier;
-
-import com.google.common.collect.Iterators;
-import com.google.common.collect.PeekingIterator;
-
-import harry.core.Configuration;
-import harry.core.Run;
-import harry.data.ResultSetRow;
-import harry.ddl.SchemaSpec;
-import harry.model.sut.SystemUnderTest;
-import harry.runner.DefaultDataTracker;
-import harry.runner.Query;
-
-/**
- * A simple model to check whether or not the rows reported as visible by the database are reflected in
- * the model.
- */
-public class VisibleRowsChecker implements Model
-{
-    protected final OpSelectors.DescriptorSelector descriptorSelector;
-    protected final OpSelectors.PdSelector pdSelector;
-    protected final OpSelectors.MonotonicClock clock;
-    protected final LoggingDataTracker tracker;
-    protected final SystemUnderTest sut;
-    protected final SchemaSpec schema;
-
-    public VisibleRowsChecker(Run run)
-    {
-        assert run.tracker instanceof LoggingDataTracker : "Visible rows checker requires a logging data tracker to run";
-        this.tracker = (LoggingDataTracker) run.tracker;
-        this.tracker.pdSelector = run.pdSelector;
-        this.pdSelector = run.pdSelector;
-        this.descriptorSelector = run.descriptorSelector;
-        this.schema = run.schemaSpec;
-        this.clock = run.clock;
-        this.sut = run.sut;
-    }
-
-    public static class LoggingDataTracker extends DefaultDataTracker
-    {
-        protected final Map<Long, TreeMap<Long, Event>> eventLog = new HashMap<>();
-        private OpSelectors.PdSelector pdSelector;
-
-        public LoggingDataTracker()
-        {
-        }
-
-        public synchronized void started(long lts)
-        {
-            super.started(lts);
-            recordEvent(lts, false);
-        }
-
-        public synchronized void finished(long lts)
-        {
-            super.finished(lts);
-            recordEvent(lts, true);
-        }
-
-        public synchronized TreeMap<Long, Event> events(long pd)
-        {
-            TreeMap<Long, Event> log = eventLog.get(pd);
-            if (log == null)
-                return null;
-
-            return (TreeMap<Long, Event>) log.clone();
-        }
-
-        public long maxStarted()
-        {
-            return super.maxStarted();
-        }
-
-        public long maxConsecutiveFinished()
-        {
-            return super.maxConsecutiveFinished();
-        }
-
-        public void recordEvent(long lts, boolean finished)
-        {
-            long pd = pdSelector.pd(lts);
-
-            // TODO: This is definitely not optimal, but we probably use a better, potentially off-heap sorted structure for that anyways
-            TreeMap<Long, Event> events = eventLog.get(pd);
-            if (events == null)
-            {
-                events = new TreeMap<>();
-                eventLog.put(pd, events);
-            }
-
-            Event event = events.get(lts);
-            assert event == null || !event.quorumAchieved : "Operation should be partially visible before it is fully visible";
-            events.put(lts, new Event(lts, finished));
-        }
-    }
-
-    public void validate(Query query)
-    {
-        validatePartitionState(query,
-                               () -> SelectHelper.execute(sut, clock, query));
-    }
-
-    public Configuration.ModelConfiguration toConfig()
-    {
-        throw new RuntimeException("not implemented");
-    }
-
-    synchronized void validatePartitionState(Query query, Supplier<List<ResultSetRow>> rowsSupplier)
-    {
-        // TODO: Right now, we ignore Query here!
-        long pd = query.pd;
-        List<ResultSetRow> rows = rowsSupplier.get();
-        TreeMap<Long, Event> events = tracker.events(pd);
-
-        if (!rows.isEmpty() && (events == null || events.isEmpty()))
-        {
-            throw new ValidationException(String.format("Returned rows are not empty, but there were no records in the event log.\nRows: %s\nSeen pds: %s",
-                                                        rows, tracker.eventLog.keySet()));
-        }
-
-        for (ResultSetRow row : rows)
-        {
-            LongIterator rowLtsIter = descendingIterator(row.lts);
-            PeekingIterator<Event> modelLtsIter = Iterators.peekingIterator(events.subMap(0L, true, tracker.maxStarted(), true)
-                                                                                  .descendingMap()
-                                                                                  .values()
-                                                                                  .iterator());
-            outer:
-            while (rowLtsIter.hasNext())
-            {
-                long rowLts = rowLtsIter.nextLong();
-
-                if (rowLts == NO_TIMESTAMP)
-                    continue;
-
-                if (!modelLtsIter.hasNext())
-                    throw new ValidationException(String.format("Model iterator is exhausted, could not verify %d lts for the row: \n%s %s",
-                                                                rowLts, row, query));
-
-                while (modelLtsIter.hasNext())
-                {
-                    Event event = modelLtsIter.next();
-                    if (event.lts > rowLts)
-                        continue;
-                    if (event.lts < rowLts)
-                        throw new RuntimeException("Can't find a corresponding event id in the model for: " + rowLts + " " + event);
-                    for (int col = 0; col < row.lts.length; col++)
-                    {
-                        if (row.lts[col] != rowLts)
-                            continue;
-                        long m = descriptorSelector.modificationId(pd, row.cd, rowLts, row.vds[col], col);
-                        long vd = descriptorSelector.vd(pd, row.cd, rowLts, m, col);
-                        if (vd != row.vds[col])
-                            throw new RuntimeException("Can't verify the row");
-                    }
-                    continue outer;
-                }
-            }
-        }
-    }
-
-    public interface LongIterator extends Iterator<Long>
-    {
-        long nextLong();
-    }
-
-
-    public static LongIterator descendingIterator(OpSelectors.PdSelector pdSelector, long pd)
-    {
-        return new VisibleRowsChecker.LongIterator()
-        {
-            long next = pdSelector.maxLtsFor(pd);
-
-            public long nextLong()
-            {
-                long ret = next;
-                next = pdSelector.prevLts(next);
-                return ret;
-            }
-
-            public boolean hasNext()
-            {
-                return next >= 0;
-            }
-
-            public Long next()
-            {
-                return null;
-            }
-        };
-    }
-
-    public static LongIterator descendingIterator(long[] ltss)
-    {
-        long[] sorted = Arrays.copyOf(ltss, ltss.length);
-        Arrays.sort(sorted);
-
-        return new LongIterator()
-        {
-            private int lastUniqueIdx = -1;
-
-            public long nextLong()
-            {
-                if (lastUniqueIdx == -1)
-                    throw new RuntimeException("No elements left or hasNext hasn't been called");
-                return sorted[lastUniqueIdx];
-            }
-
-            public boolean hasNext()
-            {
-                if (lastUniqueIdx == -1 && sorted.length > 0)
-                {
-                    lastUniqueIdx = ltss.length - 1;
-                    return true;
-                }
-
-                long lastUnique = sorted[lastUniqueIdx];
-                while (lastUniqueIdx >= 0)
-                {
-                    if (sorted[lastUniqueIdx] != lastUnique)
-                        return true;
-                    lastUniqueIdx--;
-                }
-
-                lastUniqueIdx = -1;
-                return false;
-            }
-
-            public Long next()
-            {
-                return nextLong();
-            }
-        };
-    }
-
-    protected static class Event
-    {
-        final long lts;
-        volatile boolean quorumAchieved;
-
-        public Event(long lts, boolean quorumAchieved)
-        {
-            this.lts = lts;
-            this.quorumAchieved = quorumAchieved;
-        }
-
-        public String toString()
-        {
-            return "Event{" +
-                   "lts=" + lts +
-                   ", quorumAchieved=" + quorumAchieved +
-                   '}';
-        }
-    }
-}
\ No newline at end of file
diff --git a/harry-core/src/harry/model/sut/SystemUnderTest.java b/harry-core/src/harry/model/sut/SystemUnderTest.java
index ec0a48e..ea2e633 100644
--- a/harry-core/src/harry/model/sut/SystemUnderTest.java
+++ b/harry-core/src/harry/model/sut/SystemUnderTest.java
@@ -33,6 +33,10 @@ public interface SystemUnderTest
 
     public void shutdown();
 
+    default void afterSchemaInit()
+    {
+    }
+
     default void schemaChange(String statement)
     {
         execute(statement, ConsistencyLevel.ALL, new Object[]{});
@@ -53,7 +57,7 @@ public interface SystemUnderTest
     }
 
     enum ConsistencyLevel {
-        ALL, QUORUM, NODE_LOCAL
+        ALL, QUORUM, NODE_LOCAL, ONE
     }
 
     public static final SystemUnderTest NO_OP = new NoOpSut();
@@ -82,4 +86,10 @@ public interface SystemUnderTest
         }
     }
 
+    public static interface FaultInjectingSut extends SystemUnderTest
+    {
+        public Object[][] executeWithWriteFailure(String statement, ConsistencyLevel cl, Object... bindings);
+        public CompletableFuture<Object[][]> executeAsyncWithWriteFailure(String statement, ConsistencyLevel cl, Object... bindings);
+    }
+
 }
\ No newline at end of file
diff --git a/harry-core/src/harry/operations/CompiledStatement.java b/harry-core/src/harry/operations/CompiledStatement.java
index 21261f0..ead68cf 100644
--- a/harry-core/src/harry/operations/CompiledStatement.java
+++ b/harry-core/src/harry/operations/CompiledStatement.java
@@ -50,7 +50,34 @@ public class CompiledStatement
     {
         return "CompiledStatement{" +
                "cql='" + cql + '\'' +
-               ", bindings=" + Arrays.toString(bindings) +
+               ", bindings=" + bindingsToString(bindings) +
                '}';
     }
+
+    public static String bindingsToString(Object... bindings)
+    {
+        StringBuilder sb = new StringBuilder();
+        boolean isFirst = true;
+        for (Object binding : bindings)
+        {
+            if (isFirst)
+                isFirst = false;
+            else
+                sb.append(",");
+
+            if (binding instanceof String)
+                sb.append("\"").append(binding).append("\"");
+            else if (binding instanceof Short)
+                sb.append("(short)").append(binding);
+            else if (binding instanceof Byte)
+                sb.append("(byte)").append(binding);
+            else if (binding instanceof Float)
+                sb.append("(float)").append(binding);
+            else if (binding instanceof Long)
+                sb.append(binding).append("L");
+            else
+                sb.append(binding);
+        }
+        return sb.toString();
+    }
 }
diff --git a/harry-core/src/harry/operations/DeleteHelper.java b/harry-core/src/harry/operations/DeleteHelper.java
index 41a5d43..6f04bad 100644
--- a/harry-core/src/harry/operations/DeleteHelper.java
+++ b/harry-core/src/harry/operations/DeleteHelper.java
@@ -19,6 +19,7 @@
 package harry.operations;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 import java.util.function.IntConsumer;
 
@@ -26,6 +27,7 @@ import com.datastax.driver.core.querybuilder.Delete;
 import com.datastax.driver.core.querybuilder.QueryBuilder;
 import harry.ddl.ColumnSpec;
 import harry.ddl.SchemaSpec;
+import harry.runner.LoggingPartitionVisitor;
 import harry.util.BitSet;
 
 public class DeleteHelper
@@ -33,13 +35,26 @@ public class DeleteHelper
     public static CompiledStatement deleteColumn(SchemaSpec schema,
                                                  long pd,
                                                  long cd,
-                                                 BitSet columnsToDelete,
+                                                 BitSet columns,
+                                                 BitSet mask,
                                                  long rts)
     {
-        if (columnsToDelete == null || columnsToDelete.allUnset())
-            throw new IllegalArgumentException("Can't have a delete column query with no columns set. Column mask: " + columnsToDelete);
+        if (columns == null || columns.allUnset(mask))
+            throw new IllegalArgumentException("Can't have a delete column query with no columns set. Column mask: " + columns);
 
-        return delete(schema, pd, cd, columnsToDelete, rts);
+        return delete(schema, pd, cd, columns, mask, rts);
+    }
+
+    public static CompiledStatement deleteColumn(SchemaSpec schema,
+                                                 long pd,
+                                                 BitSet columns,
+                                                 BitSet mask,
+                                                 long rts)
+    {
+        if (columns == null || columns.allUnset(mask))
+            throw new IllegalArgumentException("Can't have a delete column query with no columns set. Column mask: " + columns);
+
+        return delete(schema, pd, columns, mask, rts);
     }
 
     public static CompiledStatement deleteRow(SchemaSpec schema,
@@ -47,25 +62,29 @@ public class DeleteHelper
                                               long cd,
                                               long rts)
     {
-        return delete(schema, pd, cd, null, rts);
+        return delete(schema, pd, cd, null, null, rts);
     }
 
     public static CompiledStatement delete(SchemaSpec schema,
                                            long pd,
                                            List<Relation> relations,
                                            BitSet columnsToDelete,
+                                           BitSet mask,
                                            long rts)
     {
+        assert (columnsToDelete == null && mask == null) || (columnsToDelete != null && mask != null);
         return compile(schema,
                        pd,
                        relations,
                        columnsToDelete,
+                       mask,
                        rts);
     }
 
     private static CompiledStatement delete(SchemaSpec schema,
                                             long pd,
                                             long cd,
+                                            BitSet columnsToDelete,
                                             BitSet mask,
                                             long rts)
     {
@@ -73,31 +92,68 @@ public class DeleteHelper
                        pd,
                        Relation.eqRelations(schema.ckGenerator.slice(cd),
                                             schema.clusteringKeys),
+                       columnsToDelete,
+                       mask,
+                       rts);
+    }
+
+    private static CompiledStatement delete(SchemaSpec schema,
+                                            long pd,
+                                            BitSet columnsToDelete,
+                                            BitSet mask,
+                                            long rts)
+    {
+        return compile(schema,
+                       pd,
+                       new ArrayList<>(),
+                       columnsToDelete,
                        mask,
                        rts);
     }
 
+    public static CompiledStatement delete(SchemaSpec schema,
+                                           long pd,
+                                           long rts)
+    {
+        return compile(schema,
+                       pd,
+                       Collections.emptyList(),
+                       null,
+                       null,
+                       rts);
+    }
+
     private static CompiledStatement compile(SchemaSpec schema,
                                              long pd,
                                              List<Relation> relations,
                                              BitSet columnsToDelete,
+                                             BitSet mask,
                                              long ts)
     {
         Delete delete;
         if (columnsToDelete == null)
             delete = QueryBuilder.delete().from(schema.keyspace, schema.table);
         else
-            delete = QueryBuilder.delete(columnNames(schema.regularColumns, columnsToDelete))
+        {
+            assert mask != null;
+            assert relations == null || relations.stream().allMatch((r) -> r.kind == Relation.RelationKind.EQ);
+            delete = QueryBuilder.delete(columnNames(schema.allColumns, columnsToDelete, mask))
                                  .from(schema.keyspace, schema.table);
+        }
 
         Delete.Where where = delete.where();
         List<Object> bindings = new ArrayList<>();
 
         addRelations(schema, where, bindings, pd, relations);
         delete.using(QueryBuilder.timestamp(ts));
-
+        delete.setForceNoValues(true);
         Object[] bindingsArr = bindings.toArray(new Object[bindings.size()]);
-        return new CompiledStatement(delete.toString(), bindingsArr);
+        String compiled = delete.getQueryString();
+        if (compiled.contains("built query (could not generate with default codec registry:"))
+            throw new IllegalArgumentException(String.format("Could not generate the query: %s. Bindings: (%s)",
+                                                             delete,
+                                                             CompiledStatement.bindingsToString(bindingsArr)));
+        return new CompiledStatement(compiled, bindingsArr);
     }
 
     private static void addRelations(SchemaSpec schema, Delete.Where where, List<Object> bindings, long pd, List<Relation> relations)
@@ -110,10 +166,10 @@ public class DeleteHelper
                                 });
     }
 
-    private static String[] columnNames(List<ColumnSpec<?>> columns, BitSet mask)
+    private static String[] columnNames(List<ColumnSpec<?>> columns, BitSet selectedColumns, BitSet mask)
     {
-        String[] columnNames = new String[mask.setCount()];
-        mask.eachSetBit(new IntConsumer()
+        String[] columnNames = new String[selectedColumns.setCount(mask)];
+        selectedColumns.eachSetBit(new IntConsumer()
         {
             int i = 0;
 
@@ -121,7 +177,7 @@ public class DeleteHelper
             {
                 columnNames[i++] = columns.get(idx).name;
             }
-        });
+        }, mask);
         return columnNames;
     }
 }
diff --git a/harry-core/src/harry/operations/Relation.java b/harry-core/src/harry/operations/Relation.java
index 1218837..19db0b4 100644
--- a/harry-core/src/harry/operations/Relation.java
+++ b/harry-core/src/harry/operations/Relation.java
@@ -23,7 +23,6 @@ import java.util.List;
 
 import com.datastax.driver.core.querybuilder.Clause;
 import harry.ddl.ColumnSpec;
-import harry.model.ExhaustiveChecker;
 
 import static com.datastax.driver.core.querybuilder.QueryBuilder.bindMarker;
 import static com.datastax.driver.core.querybuilder.QueryBuilder.eq;
@@ -150,7 +149,7 @@ public class Relation
                 return "<";
             }
 
-            public boolean match(ExhaustiveChecker.LongComparator comparator, long l, long r)
+            public boolean match(LongComparator comparator, long l, long r)
             {
                 return comparator.compare(l, r) < 0;
             }
@@ -188,7 +187,7 @@ public class Relation
                 return ">";
             }
 
-            public boolean match(ExhaustiveChecker.LongComparator comparator, long l, long r)
+            public boolean match(LongComparator comparator, long l, long r)
             {
                 return comparator.compare(l, r) > 0;
             }
@@ -231,7 +230,7 @@ public class Relation
                 return "<=";
             }
 
-            public boolean match(ExhaustiveChecker.LongComparator comparator, long l, long r)
+            public boolean match(LongComparator comparator, long l, long r)
             {
                 return comparator.compare(l, r) <= 0;
             }
@@ -274,7 +273,7 @@ public class Relation
                 return ">=";
             }
 
-            public boolean match(ExhaustiveChecker.LongComparator comparator, long l, long r)
+            public boolean match(LongComparator comparator, long l, long r)
             {
                 return comparator.compare(l, r) >= 0;
             }
@@ -322,13 +321,13 @@ public class Relation
                 return "=";
             }
 
-            public boolean match(ExhaustiveChecker.LongComparator comparator, long l, long r)
+            public boolean match(LongComparator comparator, long l, long r)
             {
                 return comparator.compare(l, r) == 0;
             }
         };
 
-        public abstract boolean match(ExhaustiveChecker.LongComparator comparator, long l, long r);
+        public abstract boolean match(LongComparator comparator, long l, long r);
 
         public abstract Clause getClause(String name, Object obj);
 
@@ -347,4 +346,11 @@ public class Relation
 
         public abstract long nextMatch(long n);
     }
+
+    public static interface LongComparator
+    {
+        public int compare(long l, long r);
+    }
+
+    public static LongComparator FORWARD_COMPARATOR = Long::compare;
 }
diff --git a/harry-core/src/harry/operations/WriteHelper.java b/harry-core/src/harry/operations/WriteHelper.java
index c1f9829..a0b7565 100644
--- a/harry-core/src/harry/operations/WriteHelper.java
+++ b/harry-core/src/harry/operations/WriteHelper.java
@@ -26,9 +26,11 @@ import harry.generators.DataGenerators;
 
 import static com.datastax.driver.core.querybuilder.QueryBuilder.bindMarker;
 import static com.datastax.driver.core.querybuilder.QueryBuilder.eq;
+import static com.datastax.driver.core.querybuilder.QueryBuilder.in;
 import static com.datastax.driver.core.querybuilder.QueryBuilder.insertInto;
 import static com.datastax.driver.core.querybuilder.QueryBuilder.set;
 import static com.datastax.driver.core.querybuilder.QueryBuilder.timestamp;
+import static com.datastax.driver.core.querybuilder.QueryBuilder.truncate;
 import static com.datastax.driver.core.querybuilder.QueryBuilder.update;
 
 public class WriteHelper
@@ -37,10 +39,12 @@ public class WriteHelper
                                                   long pd,
                                                   long cd,
                                                   long[] vds,
+                                                  long[] sds,
                                                   long timestamp)
     {
         Object[] partitionKey = schema.inflatePartitionKey(pd);
         Object[] clusteringKey = schema.inflateClusteringKey(cd);
+        Object[] staticColumns = sds == null ? null : schema.inflateStaticColumns(sds);
         Object[] regularColumns = schema.inflateRegularColumns(vds);
 
         Object[] bindings = new Object[schema.allColumns.size()];
@@ -50,6 +54,8 @@ public class WriteHelper
 
         bindingsCount += addValue(insert, bindings, schema.partitionKeys, partitionKey, bindingsCount);
         bindingsCount += addValue(insert, bindings, schema.clusteringKeys, clusteringKey, bindingsCount);
+        if (staticColumns != null)
+            bindingsCount += addValue(insert, bindings, schema.staticColumns, staticColumns, bindingsCount);
         bindingsCount += addValue(insert, bindings, schema.regularColumns, regularColumns, bindingsCount);
 
         insert.using(timestamp(timestamp));
@@ -61,9 +67,19 @@ public class WriteHelper
             System.arraycopy(bindings, 0, tmp, 0, bindingsCount);
             bindings = tmp;
         }
+
         return CompiledStatement.create(insert.toString(), bindings);
     }
 
+    public static boolean allUnset(long[] descriptors)
+    {
+        for (long descriptor : descriptors)
+        {
+            if (descriptor != DataGenerators.UNSET_DESCR)
+                return false;
+        }
+        return true;
+    }
     private static int addValue(com.datastax.driver.core.querybuilder.Insert insert,
                                 Object[] bindings,
                                 List<ColumnSpec<?>> columns,
diff --git a/harry-core/src/harry/reconciler/Reconciler.java b/harry-core/src/harry/reconciler/Reconciler.java
index cab8fb8..ca4772c 100644
--- a/harry-core/src/harry/reconciler/Reconciler.java
+++ b/harry-core/src/harry/reconciler/Reconciler.java
@@ -23,12 +23,16 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 import java.util.NavigableMap;
-import java.util.Set;
 import java.util.TreeMap;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import harry.core.Run;
+import harry.ddl.ColumnSpec;
 import harry.ddl.SchemaSpec;
-import harry.model.ExhaustiveChecker;
 import harry.model.OpSelectors;
 import harry.runner.AbstractPartitionVisitor;
 import harry.runner.PartitionVisitor;
@@ -46,99 +50,197 @@ import static harry.model.Model.NO_TIMESTAMP;
  * <p>
  * It is useful both as a testing/debugging tool (to avoid starting Cassandra
  * cluster to get a result set), and as a quiescent model checker.
+ *
+ * TODO: it might be useful to actually record deletions instead of just removing values as we do right now.
  */
 public class Reconciler
 {
+    private static final Logger logger = LoggerFactory.getLogger(Reconciler.class);
+
+    private static long STATIC_CLUSTERING = NIL_DESCR;
+
     private final OpSelectors.DescriptorSelector descriptorSelector;
     private final OpSelectors.PdSelector pdSelector;
     private final QueryGenerator rangeSelector;
     private final SchemaSpec schema;
 
-    public Reconciler(SchemaSpec schema,
-                      OpSelectors.PdSelector pdSelector,
-                      OpSelectors.DescriptorSelector descriptorSelector,
-                      QueryGenerator rangeSelector)
+    public Reconciler(Run run)
     {
-        this.descriptorSelector = descriptorSelector;
-        this.pdSelector = pdSelector;
-        this.schema = schema;
-        this.rangeSelector = rangeSelector;
+        this.descriptorSelector = run.descriptorSelector;
+        this.pdSelector = run.pdSelector;
+        this.schema = run.schemaSpec;
+        this.rangeSelector = run.rangeSelector;
     }
 
+    private final long debugCd = Long.getLong("harry.reconciler.debug_cd", -1L);
+
     public PartitionState inflatePartitionState(final long pd, long maxLts, Query query)
     {
-        List<Ranges.Range> ranges = new ArrayList<>();
+        PartitionState partitionState = new PartitionState();
 
-        // TODO: we should think of a single-pass algorithm that would allow us to inflate all deletes and range deletes for a partition
-        PartitionVisitor partitionVisitor = new AbstractPartitionVisitor(pdSelector, descriptorSelector, schema)
+        class Processor extends AbstractPartitionVisitor
         {
-            public void operation(long lts, long pd, long cd, long m, long opId)
+            public Processor(OpSelectors.PdSelector pdSelector, OpSelectors.DescriptorSelector descriptorSelector, SchemaSpec schema)
             {
-                OpSelectors.OperationKind opType = descriptorSelector.operationType(pd, lts, opId);
-                if (opType == OpSelectors.OperationKind.DELETE_RANGE)
-                {
-                    ranges.add(rangeSelector.inflate(lts, opId, Query.QueryKind.CLUSTERING_RANGE).toRange(lts));
-                }
-                else if (opType == OpSelectors.OperationKind.DELETE_SLICE)
-                {
-                    ranges.add(rangeSelector.inflate(lts, opId, Query.QueryKind.CLUSTERING_SLICE).toRange(lts));
-                }
-                else if (opType == OpSelectors.OperationKind.DELETE_ROW)
-                {
-                    ranges.add(new Ranges.Range(cd, cd, true, true, lts));
-                }
+                super(pdSelector, descriptorSelector, schema);
             }
-        };
 
-        long currentLts = pdSelector.minLtsFor(pd);
-
-        while (currentLts <= maxLts && currentLts >= 0)
-        {
-            partitionVisitor.visitPartition(currentLts);
-            currentLts = pdSelector.nextLts(currentLts);
-        }
+            // Whether or not a partition deletion was encountered on this LTS.
+            private boolean hadPartitionDeletion = false;
+            private final List<Ranges.Range> rangeDeletes = new ArrayList<>();
+            private final List<Long> writes = new ArrayList<>();
+            private final List<Long> columnDeletes = new ArrayList<>();
 
-        PartitionState partitionState = new PartitionState();
-        partitionVisitor = new AbstractPartitionVisitor(pdSelector, descriptorSelector, schema)
-        {
+            @Override
             public void operation(long lts, long pd, long cd, long m, long opId)
             {
-                if (!query.match(cd))
+                if (hadPartitionDeletion)
                     return;
 
                 OpSelectors.OperationKind opType = descriptorSelector.operationType(pd, lts, opId);
+                switch (opType)
+                {
+                    case DELETE_RANGE:
+                        Query query = rangeSelector.inflate(lts, opId, Query.QueryKind.CLUSTERING_RANGE);
+                        Ranges.Range range = query.toRange(lts);
+                        rangeDeletes.add(range);
+                        partitionState.delete(range, lts);
+                        break;
+                    case DELETE_SLICE:
+                        query = rangeSelector.inflate(lts, opId, Query.QueryKind.CLUSTERING_SLICE);
+                        range = query.toRange(lts);
+                        rangeDeletes.add(range);
+                        partitionState.delete(range, lts);
+                        break;
+                    case DELETE_ROW:
+                        range = new Ranges.Range(cd, cd, true, true, lts);
+                        rangeDeletes.add(range);
+                        partitionState.delete(cd, lts);
+                        break;
+                    case DELETE_PARTITION:
+                        partitionState.deletePartition(lts);
+                        rangeDeletes.clear();
+                        writes.clear();
+                        columnDeletes.clear();
+
+                        hadPartitionDeletion = true;
+                        break;
+                    case WRITE_WITH_STATICS:
+                    case WRITE:
+                        if (debugCd != -1 && cd == debugCd)
+                            logger.info("Writing {} ({}) at {}/{}", cd, opType, lts, opId);
+                        writes.add(opId);
+                        break;
+                    case DELETE_COLUMN_WITH_STATICS:
+                    case DELETE_COLUMN:
+                        columnDeletes.add(opId);
+                        break;
+                    default:
+                        throw new IllegalStateException();
+                }
+            }
 
-                if (opType == OpSelectors.OperationKind.DELETE_ROW
-                    || opType == OpSelectors.OperationKind.DELETE_RANGE
-                    || opType == OpSelectors.OperationKind.DELETE_SLICE)
+            @Override
+            protected void beforeLts(long lts, long pd)
+            {
+                rangeDeletes.clear();
+                writes.clear();
+                columnDeletes.clear();
+                hadPartitionDeletion = false;
+            }
+
+            @Override
+            protected void afterLts(long lts, long pd)
+            {
+                if (hadPartitionDeletion)
                     return;
 
-                // TODO: avoid linear scan
-                for (Ranges.Range range : ranges)
+                outer: for (Long opIdBoxed : writes)
                 {
-                    if (range.timestamp >= lts && range.contains(cd))
-                        return;
-                }
+                    long opId = opIdBoxed;
+                    long cd = descriptorSelector.cd(pd, lts, opId, schema);
 
-                if (opType == OpSelectors.OperationKind.WRITE)
-                {
-                    partitionState.add(cd,
-                                       descriptorSelector.vds(pd, cd, lts, opId, schema),
-                                       lts);
-                }
-                else if (opType == OpSelectors.OperationKind.DELETE_COLUMN)
-                {
-                    partitionState.deleteColumns(cd,
-                                                 descriptorSelector.columnMask(pd, lts, opId));
+                    OpSelectors.OperationKind opType = descriptorSelector.operationType(pd, lts, opId);
+
+                    switch (opType)
+                    {
+                        case WRITE_WITH_STATICS:
+                            // We could apply static columns during the first iteration, but it's more convenient
+                            // to reconcile static-level deletions.
+                            partitionState.writeStaticRow(descriptorSelector.sds(pd, cd, lts, opId, schema),
+                                                          lts);
+                        case WRITE:
+                            if (!query.match(cd))
+                            {
+                                if (debugCd != -1 && cd == debugCd)
+                                    logger.info("Hiding {} at {}/{} because there was no query match", debugCd, lts, opId);
+                                continue outer;
+                            }
+
+                            for (Ranges.Range range : rangeDeletes)
+                            {
+                                if (range.timestamp >= lts && range.contains(cd))
+                                {
+                                    if (debugCd != -1 && cd == debugCd)
+                                        logger.info("Hiding {} at {}/{} because of range tombstone {}", debugCd, lts, opId, range);
+                                    continue outer;
+                                }
+                            }
+
+                            partitionState.write(cd,
+                                                 descriptorSelector.vds(pd, cd, lts, opId, schema),
+                                                 lts);
+                            break;
+                        default:
+                            throw new IllegalStateException();
+                    }
                 }
-                else
+
+                outer: for (Long opIdBoxed : columnDeletes)
                 {
-                    throw new AssertionError();
+                    long opId = opIdBoxed;
+                    long cd = descriptorSelector.cd(pd, lts, opId, schema);
+
+                    OpSelectors.OperationKind opType = descriptorSelector.operationType(pd, lts, opId);
+
+                    switch (opType)
+                    {
+                        case DELETE_COLUMN_WITH_STATICS:
+                            partitionState.deleteStaticColumns(schema.staticColumnsOffset,
+                                                               descriptorSelector.columnMask(pd, lts, opId),
+                                                               schema.staticColumnsMask());
+                        case DELETE_COLUMN:
+                            if (!query.match(cd))
+                            {
+                                if (debugCd != -1 && cd == debugCd)
+                                    logger.info("Hiding {} at {}/{} because there was no query match", debugCd, lts, opId);
+                                continue outer;
+                            }
+
+                            for (Ranges.Range range : rangeDeletes)
+                            {
+                                if (range.timestamp >= lts && range.contains(cd))
+                                {
+                                    if (debugCd != -1 && cd == debugCd)
+                                        logger.info("Hiding {} at {}/{} because of range tombstone {}", debugCd, lts, opId, range);
+                                    continue outer;
+                                }
+                            }
+
+                            partitionState.deleteRegularColumns(cd,
+                                                                schema.regularColumnsOffset,
+                                                                descriptorSelector.columnMask(pd, lts, opId),
+                                                                schema.regularColumnsMask());
+                            break;
+                    }
                 }
             }
-        };
+        }
+
+        PartitionVisitor partitionVisitor = new Processor(pdSelector, descriptorSelector, schema);
+
+        long currentLts = pdSelector.minLtsFor(pd);
 
-        currentLts = pdSelector.minLtsFor(pd);
         while (currentLts <= maxLts && currentLts >= 0)
         {
             partitionVisitor.visitPartition(currentLts);
@@ -148,22 +250,70 @@ public class Reconciler
         return partitionState;
     }
 
-    public static class PartitionState implements Iterable<RowState>
+    public class PartitionState implements Iterable<RowState>
     {
-        private NavigableMap<Long, RowState> rows;
+        private final NavigableMap<Long, RowState> rows;
+        private RowState staticRow;
 
         private PartitionState()
         {
             rows = new TreeMap<>();
+            if (!schema.staticColumns.isEmpty())
+            {
+                staticRow = new RowState(STATIC_CLUSTERING,
+                                         arr(schema.staticColumns.size(), NIL_DESCR),
+                                         arr(schema.staticColumns.size(), NO_TIMESTAMP));
+            }
         }
 
-        private void add(long cd,
-                         long[] vds,
-                         long lts)
+        private void writeStaticRow(long[] staticVds,
+                                    long lts)
         {
-            RowState state = rows.get(cd);
+            if (staticRow != null)
+                staticRow = updateRowState(staticRow, schema.staticColumns, STATIC_CLUSTERING, staticVds, lts);
+        }
 
-            if (state == null)
+        private void write(long cd,
+                           long[] vds,
+                           long lts)
+        {
+            rows.compute(cd, (cd_, current) -> updateRowState(current, schema.regularColumns, cd, vds, lts));
+        }
+
+        private void delete(Ranges.Range range,
+                            long lts)
+        {
+            if (range.minBound > range.maxBound)
+                return;
+
+            Iterator<Map.Entry<Long, RowState>> iter = rows.subMap(range.minBound, range.minInclusive,
+                                                                   range.maxBound, range.maxInclusive)
+                                                           .entrySet()
+                                                           .iterator();
+            while (iter.hasNext())
+            {
+                Map.Entry<Long, RowState> e = iter.next();
+                if (debugCd != -1 && e.getKey() == debugCd)
+                    logger.info("Hiding {} at {} because of range tombstone {}", debugCd, lts, range);
+
+                // assert row state doesn't have fresher lts
+                iter.remove();
+            }
+        }
+
+        private void delete(long cd,
+                            long lts)
+        {
+            rows.remove(cd);
+        }
+        public boolean isEmpty()
+        {
+            return rows.isEmpty();
+        }
+
+        private RowState updateRowState(RowState currentState, List<ColumnSpec<?>> columns, long cd, long[] vds, long lts)
+        {
+            if (currentState == null)
             {
                 long[] ltss = new long[vds.length];
                 long[] vdsCopy = new long[vds.length];
@@ -181,32 +331,55 @@ public class Reconciler
                     }
                 }
 
-                state = new RowState(cd, vdsCopy, ltss);
-                rows.put(cd, state);
+                currentState = new RowState(cd, vdsCopy, ltss);
             }
             else
             {
+                assert currentState.vds.length == vds.length;
                 for (int i = 0; i < vds.length; i++)
                 {
-                    if (vds[i] != UNSET_DESCR)
+                    if (vds[i] == UNSET_DESCR)
+                        continue;
+
+                    assert lts >= currentState.lts[i] : String.format("Out-of-order LTS: %d. Max seen: %s", lts, currentState.lts[i]); // sanity check; we're iterating in lts order
+
+                    if (currentState.lts[i] == lts)
+                    {
+                        // Timestamp collision case
+                        ColumnSpec<?> column = columns.get(i);
+                        if (column.type.compareLexicographically(vds[i], currentState.vds[i]) > 0)
+                            currentState.vds[i] = vds[i];
+                    }
+                    else
                     {
-                        state.vds[i] = vds[i];
-                        assert lts > state.lts[i]; // sanity check; we're iterating in lts order
-                        state.lts[i] = lts;
+                        currentState.vds[i] = vds[i];
+                        assert lts > currentState.lts[i];
+                        currentState.lts[i] = lts;
                     }
                 }
             }
+
+            return currentState;
+        }
+
+        private void deleteRegularColumns(long cd, int columnOffset, BitSet columns, BitSet mask)
+        {
+            deleteColumns(rows.get(cd), columnOffset, columns, mask);
+        }
+
+        private void deleteStaticColumns(int columnOffset, BitSet columns, BitSet mask)
+        {
+            deleteColumns(staticRow, columnOffset, columns, mask);
         }
 
-        private void deleteColumns(long cd, BitSet mask)
+        private void deleteColumns(RowState state, int columnOffset, BitSet columns, BitSet mask)
         {
-            RowState state = rows.get(cd);
             if (state == null)
                 return;
 
-            for (int i = 0; i < mask.size(); i++)
+            for (int i = 0; i < state.vds.length; i++)
             {
-                if (mask.isSet(i))
+                if (columns.isSet(columnOffset + i, mask))
                 {
                     state.vds[i] = NIL_DESCR;
                     state.lts[i] = NO_TIMESTAMP;
@@ -214,6 +387,19 @@ public class Reconciler
             }
         }
 
+        private void deletePartition(long lts)
+        {
+            if (debugCd != -1)
+                logger.info("Hiding {} at {} because partition deletion", debugCd, lts);
+
+            rows.clear();
+            if (!schema.staticColumns.isEmpty())
+            {
+                Arrays.fill(staticRow.vds, NIL_DESCR);
+                Arrays.fill(staticRow.lts, NO_TIMESTAMP);
+            }
+        }
+
         public Iterator<RowState> iterator()
         {
             return iterator(false);
@@ -234,6 +420,31 @@ public class Reconciler
 
             return rows.values();
         }
+
+        public RowState staticRow()
+        {
+            return staticRow;
+        }
+
+        public String toString(SchemaSpec schema)
+        {
+            StringBuilder sb = new StringBuilder();
+
+            if (staticRow != null)
+                sb.append("Static row: " + staticRow.toString(schema)).append("\n");
+
+            for (RowState row : rows.values())
+                sb.append(row.toString(schema)).append("\n");
+
+            return sb.toString();
+        }
+    }
+
+    public static long[] arr(int length, long fill)
+    {
+        long[] arr = new long[length];
+        Arrays.fill(arr, fill);
+        return arr;
     }
 
     public static class RowState
@@ -254,9 +465,20 @@ public class Reconciler
         public String toString()
         {
             return "RowState{" +
-                   "cd=" + cd +
+                   "cd=" + (cd == STATIC_CLUSTERING ? "static" : cd) +
+                   ", vds=" + Arrays.toString(vds) +
+                   ", lts=" + Arrays.toString(lts) +
+                   '}';
+        }
+
+        public String toString(SchemaSpec schema)
+        {
+            return "RowState{" +
+                   "cd=" + (cd == STATIC_CLUSTERING ? "static" : cd) +
                    ", vds=" + Arrays.toString(vds) +
                    ", lts=" + Arrays.toString(lts) +
+                   ", clustering=" + (cd == STATIC_CLUSTERING ? "static" : Arrays.toString(schema.inflateClusteringKey(cd))) +
+                   ", values=" + Arrays.toString(cd == STATIC_CLUSTERING ? schema.inflateStaticColumns(vds) : schema.inflateRegularColumns(vds)) +
                    '}';
         }
     }
diff --git a/harry-core/src/harry/runner/AbstractPartitionVisitor.java b/harry-core/src/harry/runner/AbstractPartitionVisitor.java
index a3ebd33..0455cbe 100644
--- a/harry-core/src/harry/runner/AbstractPartitionVisitor.java
+++ b/harry-core/src/harry/runner/AbstractPartitionVisitor.java
@@ -18,11 +18,7 @@
 
 package harry.runner;
 
-import java.util.HashSet;
-import java.util.Set;
-
 import harry.ddl.SchemaSpec;
-import harry.model.Model;
 import harry.model.OpSelectors;
 
 public abstract class AbstractPartitionVisitor implements PartitionVisitor
@@ -51,24 +47,14 @@ public abstract class AbstractPartitionVisitor implements PartitionVisitor
 
         int modificationsCount = descriptorSelector.numberOfModifications(lts);
         int opsPerModification = descriptorSelector.opsPerModification(lts);
-        int maxPartitionSize = descriptorSelector.maxPartitionSize();
-        assert opsPerModification * modificationsCount <= maxPartitionSize : "Number of operations exceeds partition width";
 
         for (int m = 0; m < modificationsCount; m++)
         {
-            Set<Long> visitedCds = new HashSet<>(); // for debug purposes
             beforeBatch(lts, pd, m);
             for (int i = 0; i < opsPerModification; i++)
             {
                 long opId = m * opsPerModification + i;
                 long cd = descriptorSelector.cd(pd, lts, opId, schema);
-                if (!visitedCds.add(cd))
-                {
-                    throw new Model.ValidationException("Can't visit the same row twice in same LTS. Visited: %s. Current: %d. " +
-                                                        opId + " " + maxPartitionSize,
-                                                        visitedCds, cd);
-                }
-
                 operation(lts, pd, cd, m, opId);
             }
             afterBatch(lts, pd, m);
diff --git a/harry-core/src/harry/runner/AllPartitionsValidator.java b/harry-core/src/harry/runner/AllPartitionsValidator.java
index 674ab5e..f42ab65 100644
--- a/harry-core/src/harry/runner/AllPartitionsValidator.java
+++ b/harry-core/src/harry/runner/AllPartitionsValidator.java
@@ -22,6 +22,7 @@ import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.slf4j.Logger;
@@ -32,6 +33,7 @@ import harry.core.Run;
 import harry.ddl.SchemaSpec;
 import harry.model.Model;
 import harry.model.OpSelectors;
+import harry.model.sut.SystemUnderTest;
 
 // This might be something that potentially grows into the validator described in the design doc;
 // right now it's just a helper/container class
@@ -46,6 +48,7 @@ public class AllPartitionsValidator implements PartitionVisitor
     protected final OpSelectors.PdSelector pdSelector;
     protected final MetricReporter metricReporter;
     protected final ExecutorService executor;
+    protected final SystemUnderTest sut;
     protected final int concurrency;
     protected final int triggerAfter;
 
@@ -59,6 +62,7 @@ public class AllPartitionsValidator implements PartitionVisitor
         this.model = modelFactory.make(run);
         this.schema = run.schemaSpec;
         this.clock = run.clock;
+        this.sut = run.sut;
         this.pdSelector = run.pdSelector;
         this.concurrency = concurrency;
         this.executor = Executors.newFixedThreadPool(concurrency);
@@ -66,36 +70,63 @@ public class AllPartitionsValidator implements PartitionVisitor
 
     protected CompletableFuture<Void> validateAllPartitions(ExecutorService executor, int parallelism)
     {
-        long maxLts = clock.maxLts() - 1;
-        long maxPos = pdSelector.positionFor(maxLts);
+        final long maxPos = this.maxPos.get();
         AtomicLong counter = new AtomicLong();
         CompletableFuture[] futures = new CompletableFuture[parallelism];
+        AtomicBoolean isDone = new AtomicBoolean(false);
+
         for (int i = 0; i < parallelism; i++)
         {
             futures[i] = CompletableFuture.supplyAsync(() -> {
                 long pos;
-                while ((pos = counter.getAndIncrement()) < maxPos && !executor.isShutdown() && !Thread.interrupted())
+                while ((pos = counter.getAndIncrement()) < maxPos && !executor.isShutdown() && !Thread.interrupted() && !isDone.get())
                 {
-                    if (pos > 0 && pos % 1000 == 0)
-                        logger.debug(String.format("Validated %d out of %d partitions", pos, maxPos));
+                    if (pos > 0 && pos % 100 == 0)
+                        logger.info(String.format("Validated %d out of %d partitions", pos, maxPos));
                     long visitLts = pdSelector.minLtsAt(pos);
 
                     metricReporter.validatePartition();
+
                     for (boolean reverse : new boolean[]{ true, false })
                     {
-                        model.validate(Query.selectPartition(schema, pdSelector.pd(visitLts, schema), reverse));
+                        try
+                        {
+                            model.validate(Query.selectPartition(schema, pdSelector.pd(visitLts, schema), reverse));
+                        }
+                        catch (Throwable t)
+                        {
+                            isDone.set(true);
+                            logger.error("Caught an error while validating all partitions.", t);
+                            throw t;
+                        }
                     }
                 }
                 return null;
             }, executor);
         }
+
         return CompletableFuture.allOf(futures);
     }
 
+    private final AtomicLong maxPos = new AtomicLong(-1);
+
     public void visitPartition(long lts)
     {
-        if (lts % triggerAfter == 0)
-            validateAllPartitions(executor, concurrency);
+        maxPos.updateAndGet(current -> Math.max(pdSelector.positionFor(lts), current));
+
+        if (triggerAfter > 0 && lts % triggerAfter == 0)
+        {
+            logger.info("Starting validations of all {} partitions", maxPos.get());
+            try
+            {
+                validateAllPartitions(executor, concurrency).get();
+            }
+            catch (Throwable e)
+            {
+                throw new RuntimeException(e);
+            }
+            logger.info("Finished validations of all partitions");
+        }
     }
 
     public void shutdown() throws InterruptedException
@@ -103,4 +134,4 @@ public class AllPartitionsValidator implements PartitionVisitor
         executor.shutdown();
         executor.awaitTermination(60, TimeUnit.SECONDS);
     }
-}
+}
\ No newline at end of file
diff --git a/harry-core/src/harry/runner/CorruptingPartitionVisitor.java b/harry-core/src/harry/runner/CorruptingPartitionVisitor.java
new file mode 100644
index 0000000..ca5bb93
--- /dev/null
+++ b/harry-core/src/harry/runner/CorruptingPartitionVisitor.java
@@ -0,0 +1,93 @@
+/*
+ *  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 java.util.Random;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import harry.core.Run;
+import harry.corruptor.AddExtraRowCorruptor;
+import harry.corruptor.ChangeValueCorruptor;
+import harry.corruptor.HideRowCorruptor;
+import harry.corruptor.HideValueCorruptor;
+import harry.corruptor.QueryResponseCorruptor;
+
+public class CorruptingPartitionVisitor implements PartitionVisitor
+{
+    public static final Logger logger = LoggerFactory.getLogger(HarryRunner.class);
+
+    private final Run run;
+    private final QueryResponseCorruptor[] corruptors;
+    private final int triggerAfter;
+
+    public CorruptingPartitionVisitor(int triggerAfter,
+                                      Run run)
+    {
+        this.run = run;
+        this.triggerAfter = triggerAfter;
+
+        this.corruptors = new QueryResponseCorruptor[]{
+        new QueryResponseCorruptor.SimpleQueryResponseCorruptor(run.schemaSpec,
+                                                                run.clock,
+                                                                HideRowCorruptor::new),
+        new AddExtraRowCorruptor(run.schemaSpec,
+                                 run.clock,
+                                 run.descriptorSelector),
+        new QueryResponseCorruptor.SimpleQueryResponseCorruptor(run.schemaSpec,
+                                                                run.clock,
+                                                                HideValueCorruptor::new),
+        new QueryResponseCorruptor.SimpleQueryResponseCorruptor(run.schemaSpec,
+                                                                run.clock,
+                                                                ChangeValueCorruptor::new)
+        };
+    }
+
+    private final AtomicLong maxPos = new AtomicLong(-1);
+
+    public void visitPartition(long lts)
+    {
+        maxPos.updateAndGet(current -> Math.max(run.pdSelector.positionFor(lts), current));
+
+        if (lts == 0 || lts % triggerAfter != 0)
+            return;
+
+        Random random = new Random(1);
+
+        QueryResponseCorruptor corruptor = corruptors[random.nextInt(corruptors.length)];
+        long maxPos = this.maxPos.get();
+        long pd = run.pdSelector.pd(random.nextInt((int) maxPos), run.schemaSpec);
+        try
+        {
+            boolean success = corruptor.maybeCorrupt(Query.selectPartition(run.schemaSpec, pd, false),
+                                                     run.sut);
+            logger.info("{} tried to corrupt a partition with a pd {}@{} my means of {}", success ? "Successfully" : "Unsuccessfully", pd, lts, corruptor.getClass());
+        }
+        catch (Throwable t)
+        {
+            logger.error("Caught an exception while trying to corrupt a partition.", t);
+        }
+    }
+
+    public void shutdown() throws InterruptedException
+    {
+    }
+}
diff --git a/harry-core/src/harry/runner/HarryRunner.java b/harry-core/src/harry/runner/HarryRunner.java
index 77dcc8b..ba911c4 100644
--- a/harry-core/src/harry/runner/HarryRunner.java
+++ b/harry-core/src/harry/runner/HarryRunner.java
@@ -18,21 +18,19 @@
 
 package harry.runner;
 
-import harry.core.Configuration;
-import harry.core.Run;
-import harry.corruptor.*;
-import harry.util.ThrowingRunnable;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import java.io.File;
 import java.io.FileNotFoundException;
-import java.util.Random;
 import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import harry.core.Configuration;
+import harry.core.Run;
+import harry.util.ThrowingRunnable;
+
 public abstract class HarryRunner
 {
     public static final Logger logger = LoggerFactory.getLogger(HarryRunner.class);
@@ -68,9 +66,6 @@ public abstract class HarryRunner
         progress = runner.initAndStartAll();
         beforeRun(runner);
 
-        // Uncomment this if you want to have fun!
-        // scheduleCorruption(run, executor);
-
         Object result = null;
 
         try
@@ -79,8 +74,7 @@ public abstract class HarryRunner
                 if (b != null)
                     return b;
                 return a;
-            }).get(config.run_time_unit.toSeconds(config.run_time) + 30,
-                   TimeUnit.SECONDS);
+            }).get();
             if (result instanceof Throwable)
                 logger.error("Execution failed", result);
 
@@ -135,7 +129,7 @@ public abstract class HarryRunner
      * @return Configuration YAML file.
      * @throws Exception If file is not found or cannot be read.
      */
-    public File loadConfig(String[] args) throws Exception {
+    public static File loadConfig(String[] args) throws Exception {
         if (args == null || args.length == 0) {
             throw new Exception("Harry config YAML not provided.");
         }
@@ -144,49 +138,11 @@ public abstract class HarryRunner
         if (!configFile.exists()) {
             throw new FileNotFoundException(configFile.getAbsolutePath());
         }
+
         if (!configFile.canRead()) {
             throw new Exception("Cannot read config file, check your permissions on " + configFile.getAbsolutePath());
         }
 
         return configFile;
     }
-
-    /**
-     * If you want to see how Harry detects problems!
-     */
-    public static void scheduleCorruption(Run run, ScheduledExecutorService executor)
-    {
-        QueryResponseCorruptor[] corruptors = new QueryResponseCorruptor[]{
-        new QueryResponseCorruptor.SimpleQueryResponseCorruptor(run.schemaSpec,
-                                                                run.clock,
-                                                                HideRowCorruptor::new),
-        new AddExtraRowCorruptor(run.schemaSpec,
-                                 run.clock,
-                                 run.descriptorSelector),
-        new QueryResponseCorruptor.SimpleQueryResponseCorruptor(run.schemaSpec,
-                                                                run.clock,
-                                                                HideValueCorruptor::new),
-        new QueryResponseCorruptor.SimpleQueryResponseCorruptor(run.schemaSpec,
-                                                                run.clock,
-                                                                ChangeValueCorruptor::new)
-        };
-
-        Random random = new Random();
-        executor.scheduleWithFixedDelay(() -> {
-            try
-            {
-                QueryResponseCorruptor corruptor = corruptors[random.nextInt(corruptors.length)];
-                long lts = run.clock.maxLts();
-                long pd = run.pdSelector.pd(random.nextInt((int) lts), run.schemaSpec);
-                boolean success = corruptor.maybeCorrupt(Query.selectPartition(run.schemaSpec, pd, false),
-                                                         run.sut);
-                logger.info("{} tried to corrupt a partition with a pd {}@{}", success ? "Successfully" : "Unsuccessfully", pd, lts);
-            }
-            catch (Throwable t)
-            {
-                logger.error("Caught an exception while trying to corrupt a partition.", t);
-            }
-        }, 30, 1, TimeUnit.SECONDS);
-    }
-
 }
\ No newline at end of file
diff --git a/harry-core/src/harry/runner/LoggingPartitionVisitor.java b/harry-core/src/harry/runner/LoggingPartitionVisitor.java
index c3a77af..9cb827d 100644
--- a/harry-core/src/harry/runner/LoggingPartitionVisitor.java
+++ b/harry-core/src/harry/runner/LoggingPartitionVisitor.java
@@ -32,7 +32,7 @@ public class LoggingPartitionVisitor extends MutatingPartitionVisitor
 {
     private final BufferedWriter operationLog;
 
-    public LoggingPartitionVisitor(Run run, RowVisitor.RowVisitorFactory rowVisitorFactory)
+    public LoggingPartitionVisitor(Run run, Operation.RowVisitorFactory rowVisitorFactory)
     {
         super(run, rowVisitorFactory);
 
@@ -64,27 +64,6 @@ public class LoggingPartitionVisitor extends MutatingPartitionVisitor
         return statement;
     }
 
-    public static String bindingsToString(Object... bindings)
-    {
-        StringBuilder sb = new StringBuilder();
-        boolean isFirst = true;
-        for (Object binding : bindings)
-        {
-            if (isFirst)
-                isFirst = false;
-            else
-                sb.append(",");
-
-            if (binding instanceof String)
-                sb.append("\"").append(binding).append("\"");
-            else if (binding instanceof Long)
-                sb.append(binding).append("L");
-            else
-                sb.append(binding);
-        }
-        return sb.toString();
-    }
-
     private void log(String format, Object... objects)
     {
         try
diff --git a/harry-core/src/harry/runner/MutatingPartitionVisitor.java b/harry-core/src/harry/runner/MutatingPartitionVisitor.java
index a818091..02aa6a1 100644
--- a/harry-core/src/harry/runner/MutatingPartitionVisitor.java
+++ b/harry-core/src/harry/runner/MutatingPartitionVisitor.java
@@ -25,6 +25,9 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import harry.core.Run;
 import harry.model.Model;
 import harry.model.OpSelectors;
@@ -33,6 +36,8 @@ import harry.operations.CompiledStatement;
 
 public class MutatingPartitionVisitor extends AbstractPartitionVisitor
 {
+    private static final Logger logger = LoggerFactory.getLogger(MutatingPartitionVisitor.class);
+
     private final List<String> statements = new ArrayList<>();
     private final List<Object> bindings = new ArrayList<>();
 
@@ -41,9 +46,9 @@ public class MutatingPartitionVisitor extends AbstractPartitionVisitor
     protected final ScheduledExecutorService executor = Executors.newScheduledThreadPool(2);
     protected final DataTracker tracker;
     protected final SystemUnderTest sut;
-    protected final RowVisitor rowVisitor;
+    protected final Operation rowVisitor;
 
-    public MutatingPartitionVisitor(Run run, RowVisitor.RowVisitorFactory rowVisitorFactory)
+    public MutatingPartitionVisitor(Run run, Operation.RowVisitorFactory rowVisitorFactory)
     {
         super(run.pdSelector, run.descriptorSelector, run.schemaSpec);
         this.tracker = run.tracker;
@@ -66,7 +71,7 @@ public class MutatingPartitionVisitor extends AbstractPartitionVisitor
             }
             catch (Throwable t)
             {
-                throw new Model.ValidationException("Couldn't repeat operations within timeout bounds.", t);
+                throw new IllegalStateException("Couldn't repeat operations within timeout bounds.", t);
             }
         }
         futures.clear();
@@ -90,11 +95,17 @@ public class MutatingPartitionVisitor extends AbstractPartitionVisitor
     protected CompiledStatement operationInternal(long lts, long pd, long cd, long m, long opId)
     {
         OpSelectors.OperationKind op = descriptorSelector.operationType(pd, lts, opId);
-        return rowVisitor.visitRow(op, lts, pd, cd, opId);
+        return rowVisitor.perform(op, lts, pd, cd, opId);
     }
 
     public void afterBatch(long lts, long pd, long m)
     {
+        if (statements.isEmpty())
+        {
+            logger.warn("Encountered an empty batch on {}", lts);
+            return;
+        }
+
         String query = String.join(" ", statements);
 
         if (statements.size() > 1)
diff --git a/harry-core/src/harry/runner/MutatingRowVisitor.java b/harry-core/src/harry/runner/MutatingRowVisitor.java
index 01b5acd..b928df7 100644
--- a/harry-core/src/harry/runner/MutatingRowVisitor.java
+++ b/harry-core/src/harry/runner/MutatingRowVisitor.java
@@ -27,7 +27,7 @@ import harry.operations.DeleteHelper;
 import harry.operations.WriteHelper;
 import harry.util.BitSet;
 
-public class MutatingRowVisitor implements RowVisitor
+public class MutatingRowVisitor implements Operation
 {
     protected final SchemaSpec schema;
     protected final OpSelectors.MonotonicClock clock;
@@ -48,16 +48,32 @@ public class MutatingRowVisitor implements RowVisitor
     {
         metricReporter.insert();
         long[] vds = descriptorSelector.vds(pd, cd, lts, opId, schema);
-        return WriteHelper.inflateInsert(schema, pd, cd, vds, clock.rts(lts));
+        return WriteHelper.inflateInsert(schema, pd, cd, vds, null, clock.rts(lts));
+    }
+
+    public CompiledStatement writeWithStatics(long lts, long pd, long cd, long opId)
+    {
+        metricReporter.insert();
+        long[] vds = descriptorSelector.vds(pd, cd, lts, opId, schema);
+        long[] sds = descriptorSelector.sds(pd, cd, lts, opId, schema);
+        return WriteHelper.inflateInsert(schema, pd, cd, vds, sds, clock.rts(lts));
     }
 
     public CompiledStatement deleteColumn(long lts, long pd, long cd, long opId)
     {
         metricReporter.columnDelete();
-        BitSet mask = descriptorSelector.columnMask(pd, lts, opId);
-        return DeleteHelper.deleteColumn(schema, pd, cd, mask, clock.rts(lts));
+        BitSet columns = descriptorSelector.columnMask(pd, lts, opId);
+        BitSet mask = schema.regularColumnsMask();
+        return DeleteHelper.deleteColumn(schema, pd, cd, columns, mask, clock.rts(lts));
     }
 
+    public CompiledStatement deleteColumnWithStatics(long lts, long pd, long cd, long opId)
+    {
+        metricReporter.columnDelete();
+        BitSet columns = descriptorSelector.columnMask(pd, lts, opId);
+        BitSet mask = schema.regularAndStaticColumnsMask();
+        return DeleteHelper.deleteColumn(schema, pd, cd, columns, mask, clock.rts(lts));
+    }
 
     public CompiledStatement deleteRow(long lts, long pd, long cd, long opId)
     {
@@ -65,6 +81,12 @@ public class MutatingRowVisitor implements RowVisitor
         return DeleteHelper.deleteRow(schema, pd, cd, clock.rts(lts));
     }
 
+    public CompiledStatement deletePartition(long lts, long pd, long opId)
+    {
+        metricReporter.partitionDelete();
+        return DeleteHelper.delete(schema, pd, clock.rts(lts));
+    }
+
     public CompiledStatement deleteRange(long lts, long pd, long opId)
     {
         metricReporter.rangeDelete();
diff --git a/harry-core/src/harry/runner/RowVisitor.java b/harry-core/src/harry/runner/Operation.java
similarity index 73%
rename from harry-core/src/harry/runner/RowVisitor.java
rename to harry-core/src/harry/runner/Operation.java
index b24babb..e56be4c 100644
--- a/harry-core/src/harry/runner/RowVisitor.java
+++ b/harry-core/src/harry/runner/Operation.java
@@ -22,14 +22,14 @@ import harry.core.Run;
 import harry.model.OpSelectors;
 import harry.operations.CompiledStatement;
 
-public interface RowVisitor
+public interface Operation
 {
     interface RowVisitorFactory
     {
-        RowVisitor make(Run run);
+        Operation make(Run run);
     }
 
-    default CompiledStatement visitRow(OpSelectors.OperationKind op, long lts, long pd, long cd, long opId)
+    default CompiledStatement perform(OpSelectors.OperationKind op, long lts, long pd, long cd, long opId)
     {
         switch (op)
         {
@@ -39,8 +39,14 @@ public interface RowVisitor
                 return write(lts, pd, cd, opId);
             case DELETE_ROW:
                 return deleteRow(lts, pd, cd, opId);
+            case WRITE_WITH_STATICS:
+                return writeWithStatics(lts, pd, cd, opId);
+            case DELETE_PARTITION:
+                return deletePartition(lts, pd, opId);
             case DELETE_COLUMN:
                 return deleteColumn(lts, pd, cd, opId);
+            case DELETE_COLUMN_WITH_STATICS:
+                return deleteColumnWithStatics(lts, pd, cd, opId);
             case DELETE_RANGE:
                 return deleteRange(lts, pd, opId);
             case DELETE_SLICE:
@@ -54,8 +60,14 @@ public interface RowVisitor
 
     CompiledStatement deleteColumn(long lts, long pd, long cd, long opId);
 
+    CompiledStatement deleteColumnWithStatics(long lts, long pd, long cd, long opId);
+
     CompiledStatement deleteRow(long lts, long pd, long cd, long opId);
 
+    CompiledStatement deletePartition(long lts, long pd, long opId);
+
+    CompiledStatement writeWithStatics(long lts, long pd, long cd, long opId);
+
     CompiledStatement deleteRange(long lts, long pd, long opId);
 
     CompiledStatement deleteSlice(long lts, long pd, long opId);
diff --git a/harry-core/src/harry/runner/ParallelRecentPartitionValidator.java b/harry-core/src/harry/runner/ParallelRecentPartitionValidator.java
new file mode 100644
index 0000000..b363baf
--- /dev/null
+++ b/harry-core/src/harry/runner/ParallelRecentPartitionValidator.java
@@ -0,0 +1,170 @@
+/*
+ *  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 java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import harry.core.Configuration;
+import harry.core.Run;
+import harry.generators.Surjections;
+import harry.model.Model;
+
+public class ParallelRecentPartitionValidator extends ParallelValidator<ParallelRecentPartitionValidator.State>
+{
+    private static final Logger logger = LoggerFactory.getLogger(ParallelRecentPartitionValidator.class);
+
+    private final int partitionCount;
+    private final int queries;
+    private final QueryGenerator.TypedQueryGenerator querySelector;
+    private final Model model;
+    private final BufferedWriter validationLog;
+
+    public ParallelRecentPartitionValidator(int partitionCount, int concurrency, int triggerAfter,  int queries,
+                                            Run run,
+                                            Model.ModelFactory modelFactory)
+    {
+        super(concurrency, triggerAfter, run);
+        this.partitionCount = partitionCount;
+        this.queries = queries;
+        this.querySelector = new QueryGenerator.TypedQueryGenerator(run.rng,
+                                                                    // TODO: make query kind configurable
+                                                                    Surjections.enumValues(Query.QueryKind.class),
+                                                                    run.rangeSelector);
+        this.model = modelFactory.make(run);
+        File f = new File("validation.log");
+        try
+        {
+            validationLog = new BufferedWriter(new OutputStreamWriter(new FileOutputStream(f)));
+        }
+        catch (FileNotFoundException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    protected void doOne(State state)
+    {
+        long claim = state.claim();
+        if (claim < 0)
+            return;
+
+        long visitLts = run.pdSelector.minLtsAt(state.position - claim);
+        for (int i = 0; i < queries; i++)
+        {
+            run.metricReporter.validateRandomQuery();
+            Query query = querySelector.inflate(visitLts, i);
+            model.validate(query);
+            log(visitLts, i, query);
+        }
+    }
+
+    protected CompletableFuture<Void> startThreads(ExecutorService executor, int parallelism)
+    {
+        logger.info("Validating {} recent partitions", partitionCount);
+        return super.startThreads(executor, parallelism);
+    }
+
+    protected State initialState()
+    {
+        return new State(maxPos.get());
+    }
+
+    public class State extends ParallelValidator.State
+    {
+        private final long position;
+        private final AtomicLong counter;
+
+        public State(long maxPos)
+        {
+            this.position = maxPos;
+            this.counter = new AtomicLong(partitionCount);
+        }
+
+        public long claim()
+        {
+            long v = counter.getAndDecrement();
+            if (v <= 0)
+                signal();
+
+            return v;
+        }
+    }
+
+    private void log(long lts, int modifier, Query query)
+    {
+        try
+        {
+            validationLog.write("LTS: " + lts + ". Modifier: " + modifier + ". PD: " + query.pd);
+            validationLog.write("\t");
+            validationLog.write(query.toSelectStatement().toString());
+            validationLog.write("\n");
+            validationLog.flush();
+        }
+        catch (IOException e)
+        {
+            // ignore
+        }
+    }
+
+    @JsonTypeName("parallel_validate_recent_partitions")
+    public static class ParallelRecentPartitionValidatorConfig implements Configuration.PartitionVisitorConfiguration
+    {
+        public final int partition_count;
+        public final int trigger_after;
+        public final int queries;
+        public final int concurrency;
+        public final Configuration.ModelConfiguration modelConfiguration;
+
+        // TODO: make query selector configurable
+        @JsonCreator
+        public ParallelRecentPartitionValidatorConfig(@JsonProperty("partition_count") int partition_count,
+                                                      @JsonProperty("concurrency") int concurrency,
+                                                      @JsonProperty("trigger_after") int trigger_after,
+                                                      @JsonProperty("queries_per_partition") int queries,
+                                                      @JsonProperty("model") Configuration.ModelConfiguration model)
+        {
+            this.partition_count = partition_count;
+            this.concurrency = concurrency;
+            this.queries = queries;
+            this.trigger_after = trigger_after;
+            this.modelConfiguration = model;
+        }
+
+        @Override
+        public PartitionVisitor make(Run run)
+        {
+            return new ParallelRecentPartitionValidator(partition_count, concurrency, trigger_after, queries, run, modelConfiguration);
+        }
+    }
+
+}
diff --git a/harry-core/src/harry/runner/ParallelValidator.java b/harry-core/src/harry/runner/ParallelValidator.java
new file mode 100644
index 0000000..742a7cc
--- /dev/null
+++ b/harry-core/src/harry/runner/ParallelValidator.java
@@ -0,0 +1,112 @@
+/*
+ *  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 java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import harry.core.Run;
+
+public abstract class ParallelValidator<T extends ParallelValidator.State> implements PartitionVisitor
+{
+    private static final Logger logger = LoggerFactory.getLogger(AllPartitionsValidator.class);
+
+    protected final Run run;
+    protected final int parallelism;
+    protected final int triggerAfter;
+    protected final ExecutorService executor;
+    protected final AtomicLong maxPos = new AtomicLong(-1);
+
+    public ParallelValidator(int parallelism,
+                             int triggerAfter,
+                             Run run)
+    {
+        this.triggerAfter = triggerAfter;
+        this.run = run;
+        this.parallelism = parallelism;
+        this.executor = Executors.newFixedThreadPool(parallelism);
+    }
+
+    protected abstract void doOne(T state);
+    protected abstract T initialState();
+
+    protected CompletableFuture<Void> startThreads(ExecutorService executor, int parallelism)
+    {
+        CompletableFuture<?>[] futures = new CompletableFuture[parallelism];
+        T shared = initialState();
+
+        for (int i = 0; i < parallelism; i++)
+        {
+            futures[i] = CompletableFuture.supplyAsync(() -> {
+                while (!shared.signalled())
+                    doOne(shared);
+
+                return null;
+            }, executor);
+        }
+
+        return CompletableFuture.allOf(futures);
+    }
+
+    public abstract static class State
+    {
+        private final AtomicBoolean isDone = new AtomicBoolean(false);
+
+        public void signal()
+        {
+            isDone.set(true);
+        }
+
+        public boolean signalled()
+        {
+            return isDone.get();
+        }
+    }
+
+    public void visitPartition(long lts)
+    {
+        maxPos.updateAndGet(current -> Math.max(run.pdSelector.positionFor(lts), current));
+
+        if (triggerAfter > 0 && lts % triggerAfter == 0)
+        {
+            try
+            {
+                startThreads(executor, parallelism).get();
+            }
+            catch (Throwable e)
+            {
+                throw new RuntimeException(e);
+            }
+            logger.info("Finished validations");
+        }
+    }
+
+    public void shutdown() throws InterruptedException
+    {
+        executor.shutdown();
+        executor.awaitTermination(60, TimeUnit.SECONDS);
+    }
+}
\ No newline at end of file
diff --git a/harry-core/src/harry/runner/Query.java b/harry-core/src/harry/runner/Query.java
index d0742c0..1eef505 100644
--- a/harry-core/src/harry/runner/Query.java
+++ b/harry-core/src/harry/runner/Query.java
@@ -29,14 +29,13 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import harry.ddl.SchemaSpec;
-import harry.model.ExhaustiveChecker;
 import harry.model.SelectHelper;
 import harry.operations.CompiledStatement;
 import harry.operations.DeleteHelper;
 import harry.operations.Relation;
 import harry.util.Ranges;
 
-import static harry.model.ExhaustiveChecker.FORWARD_COMPARATOR;
+import static harry.operations.Relation.FORWARD_COMPARATOR;
 
 public abstract class Query
 {
@@ -205,7 +204,8 @@ public abstract class Query
 
         public boolean match(long cd)
         {
-            ExhaustiveChecker.LongComparator cmp = FORWARD_COMPARATOR;
+            // TODO: looks like we don't really need comparator here.
+            Relation.LongComparator cmp = FORWARD_COMPARATOR;
             boolean res = minRelation.match(cmp, cd, cdMin) && maxRelation.match(cmp, cd, cdMax);
             if (!logger.isDebugEnabled())
                 return res;
@@ -258,7 +258,7 @@ public abstract class Query
 
     public CompiledStatement toDeleteStatement(long rts)
     {
-        return DeleteHelper.delete(schemaSpec, pd, relations, null, rts);
+        return DeleteHelper.delete(schemaSpec, pd, relations, null, null, rts);
     }
 
     public abstract Ranges.Range toRange(long ts);
diff --git a/harry-core/src/harry/runner/QueryGenerator.java b/harry-core/src/harry/runner/QueryGenerator.java
index a50c450..65b21ac 100644
--- a/harry-core/src/harry/runner/QueryGenerator.java
+++ b/harry-core/src/harry/runner/QueryGenerator.java
@@ -19,6 +19,7 @@
 package harry.runner;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 import java.util.function.LongSupplier;
@@ -197,7 +198,18 @@ public class QueryGenerator
                             minBound[i] = minSupplier.getAsLong();
                             maxBound[i] = maxSupplier.getAsLong();
                         }
-                        else if (i > 0 && schema.clusteringKeys.get(i - 1).isReversed())
+                        // If we have a non-eq case, all subsequent bounds have to correspond to the maximum in normal case,
+                        // or minimum in case the last bound locked with a relation was reversed.
+                        //
+                        // For example, if we have (ck1, ck2, ck3) as (ASC, DESC, ASC), and query ck1 > X, we'll have:
+                        //  [xxxxx | max_value | max_value]
+                        //    ck1       ck2         ck3
+                        // which will exclude xxxx, but take every possible (ck1 > xxxxx) prefixed value.
+                        //
+                        // Similarly, if we have (ck1, ck2, ck3) as (ASC, DESC, ASC), and query ck1 <= X, we'll have:
+                        //  [xxxxx | max_value | max_value]
+                        // which will include every (ck1 < xxxxx), and any clustering prefixed with xxxxx.
+                        else if (schema.clusteringKeys.get(nonEqFrom).isReversed())
                             maxBound[i] = minBound[i] = isGt ? minSupplier.getAsLong() : maxSupplier.getAsLong();
                         else
                             maxBound[i] = minBound[i] = isGt ? maxSupplier.getAsLong() : minSupplier.getAsLong();
@@ -236,13 +248,12 @@ public class QueryGenerator
                 long cd1 = descriptorSelector.randomCd(pd, descriptor, schema);
                 boolean isMinEq = RngUtils.asBoolean(descriptor);
                 long cd2 = descriptorSelector.randomCd(pd, rng.next(descriptor, lts), schema);
-
                 boolean isMaxEq = RngUtils.asBoolean(rng.next(descriptor, lts));
 
                 long[] minBound = schema.ckGenerator.slice(cd1);
                 long[] maxBound = schema.ckGenerator.slice(cd2);
 
-                int lock = RngUtils.asInt(descriptor, 0, schema.clusteringKeys.size() - 1);
+                int nonEqFrom = RngUtils.asInt(descriptor, 0, schema.clusteringKeys.size() - 1);
 
                 // Logic here is similar to how clustering slices are implemented, except for both lower and upper bound
                 // get their values from sliced value in (1) and (2) cases:
@@ -263,75 +274,43 @@ public class QueryGenerator
                 for (int i = 0; i < schema.clusteringKeys.size(); i++)
                 {
                     ColumnSpec<?> col = schema.clusteringKeys.get(i);
-                    if (i < lock)
+                    if (i < nonEqFrom)
                     {
                         relations.add(Relation.eqRelation(col, minBound[i]));
                         maxBound[i] = minBound[i];
                     }
-                    else if (i == lock)
+                    else if (i == nonEqFrom)
                     {
-                        long minLocked = Math.min(minBound[lock], maxBound[lock]);
-                        long maxLocked = Math.max(minBound[lock], maxBound[lock]);
-
-                        relations.add(Relation.relation(relationKind(true, isMinEq), col, minLocked));
-                        minBound[i] = col.isReversed() ? maxLocked : minLocked;
-                        relations.add(Relation.relation(relationKind(false, isMaxEq), col, maxLocked));
-                        maxBound[i] = col.isReversed() ? minLocked : maxLocked;
+                        long minLocked = Math.min(minBound[nonEqFrom], maxBound[nonEqFrom]);
+                        long maxLocked = Math.max(minBound[nonEqFrom], maxBound[nonEqFrom]);
+                        relations.add(Relation.relation(relationKind(true, col.isReversed() ? isMaxEq : isMinEq), col,
+                                                        col.isReversed() ? maxLocked : minLocked));
+                        relations.add(Relation.relation(relationKind(false, col.isReversed() ? isMinEq : isMaxEq), col,
+                                                        col.isReversed() ? minLocked : maxLocked));
+                        minBound[i] = minLocked;
+                        maxBound[i] = maxLocked;
+
+                        // Impossible query
+                        if (i == 0 && minLocked == maxLocked)
+                        {
+                            return inflate(lts, modifier + 1, queryKind);
+                        }
                     }
                     else
                     {
-//                        if (i > 0 && schema.clusteringKeys.get(i - 1).isReversed())
-//                        {
-//                            minBound[i] = isMinEq ? schema.ckGenerator.maxValue(i) : schema.ckGenerator.minValue(i);
-//                            maxBound[i] = isMaxEq ? schema.ckGenerator.minValue(i) : schema.ckGenerator.maxValue(i);
-//                        }
-//                        else
-                        {
-                            minBound[i] = isMinEq ? schema.ckGenerator.minValue(i) : schema.ckGenerator.maxValue(i);
-                            maxBound[i] = isMaxEq ? schema.ckGenerator.maxValue(i) : schema.ckGenerator.minValue(i);
-                        }
+                        minBound[i] = isMinEq ? schema.ckGenerator.minValue(i) : schema.ckGenerator.maxValue(i);
+                        maxBound[i] = isMaxEq ? schema.ckGenerator.maxValue(i) : schema.ckGenerator.minValue(i);
                     }
                 }
 
                 long stitchedMin = schema.ckGenerator.stitch(minBound);
                 long stitchedMax = schema.ckGenerator.stitch(maxBound);
 
-//                if (stitchedMin > stitchedMax)
-//                {
-//                    long[] tmp = minBound;
-//                    minBound = maxBound;
-//                    maxBound = tmp;
-//                    stitchedMin = schema.ckGenerator.stitch(minBound);
-//                    stitchedMax = schema.ckGenerator.stitch(maxBound);
-//                }
-//
-//                for (int i = 0; i <= lock; i++)
-//                {
-//                    ColumnSpec<?> col = schema.clusteringKeys.get(i);
-//                    if (i < lock)
-//                    {
-//                        relations.add(Relation.eqRelation(col, minBound[i]));
-//                    }
-//                    else
-//                    {
-//                        relations.add(Relation.relation(relationKind(true, isMinEq), col, minBound[lock]));
-//                        relations.add(Relation.relation(relationKind(false, isMaxEq), col, maxBound[lock]));
-//                    }
-//                }
-
                 // if we're about to create an "impossible" query, just bump the modifier and re-generate
-                // TODO: so this isn't considered "normal" that we do it this way, but I'd rather fix it with
+                // TODO: this isn't considered "normal" that we do it this way, but I'd rather fix it with
                 //       a refactoring that's mentioned below
                 if (stitchedMin == stitchedMax)
                 {
-//                    if (modifier > 10)
-//                    {
-//                        logger.error(String.format("Unsuccessfully tried to generate query for %s%s;%s%s %s %d times. Schema: %s",
-//                                                   isMinEq ? "[" : "(", stitchedMin,
-//                                                   stitchedMax, isMaxEq ? "]" : ")",
-//                                                   queryKind, modifier, schema.compile().cql()),
-//                                     new RuntimeException());
-//                    }
                     return inflate(lts, modifier + 1, queryKind);
                 }
 
diff --git a/harry-core/src/harry/runner/QuerySelector.java b/harry-core/src/harry/runner/QuerySelector.java
deleted file mode 100644
index 00a567a..0000000
--- a/harry-core/src/harry/runner/QuerySelector.java
+++ /dev/null
@@ -1,300 +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.runner;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.function.LongSupplier;
-
-import harry.ddl.ColumnSpec;
-import harry.ddl.SchemaSpec;
-import harry.generators.DataGenerators;
-import harry.generators.RngUtils;
-import harry.generators.Surjections;
-import harry.model.OpSelectors;
-import harry.operations.Relation;
-
-// TODO: there's a lot of potential to reduce an amount of garbage here.
-// TODO: refactor. Currently, this class is a base for both SELECT and DELETE statements. In retrospect,
-//       a better way to do the same thing would've been to just inflate bounds, be able to inflate
-//       any type of query from the bounds, and leave things like "reverse" up to the last mile / implementation.
-public class QuerySelector
-{
-    private static final long GT_STREAM = 0b1;
-    private static final long E_STREAM = 0b10;
-
-    private final OpSelectors.Rng rng;
-    private final OpSelectors.PdSelector pdSelector;
-    private final OpSelectors.DescriptorSelector descriptorSelector;
-    private final Surjections.Surjection<Query.QueryKind> queryKindGen;
-    private final SchemaSpec schema;
-
-    public QuerySelector(SchemaSpec schema,
-                         OpSelectors.PdSelector pdSelector,
-                         OpSelectors.DescriptorSelector descriptorSelector,
-                         OpSelectors.Rng rng)
-    {
-        this(schema, pdSelector, descriptorSelector, Surjections.enumValues(Query.QueryKind.class), rng);
-    }
-
-    public QuerySelector(SchemaSpec schema,
-                         OpSelectors.PdSelector pdSelector,
-                         OpSelectors.DescriptorSelector descriptorSelector,
-                         Surjections.Surjection<Query.QueryKind> queryKindGen,
-                         OpSelectors.Rng rng)
-    {
-        this.pdSelector = pdSelector;
-        this.descriptorSelector = descriptorSelector;
-        this.schema = schema;
-        this.queryKindGen = queryKindGen;
-        this.rng = rng;
-    }
-
-    // Queries are inflated from LTS, which identifies the partition, and i, a modifier for the query to
-    // be able to generate different queries for the same lts.
-    public Query inflate(long lts, long modifier)
-    {
-        long pd = pdSelector.pd(lts, schema);
-        long descriptor = rng.next(modifier, lts);
-        Query.QueryKind queryKind = queryKindGen.inflate(descriptor);
-
-        boolean reverse = descriptor % 2 == 0;
-        switch (queryKind)
-        {
-            case SINGLE_PARTITION:
-                return new Query.SinglePartitionQuery(queryKind,
-                                                      pd,
-                                                      reverse,
-                                                      Collections.emptyList(),
-                                                      schema);
-            case SINGLE_CLUSTERING:
-            {
-                long cd = descriptorSelector.randomCd(pd, descriptor, schema);
-                return new Query.SingleClusteringQuery(queryKind,
-                                                       pd,
-                                                       cd,
-                                                       reverse,
-                                                       Relation.eqRelations(schema.ckGenerator.slice(cd), schema.clusteringKeys),
-                                                       schema);
-            }
-            case CLUSTERING_SLICE:
-            {
-                List<Relation> relations = new ArrayList<>();
-                long cd = descriptorSelector.randomCd(pd, descriptor, schema);
-                boolean isGt = RngUtils.asBoolean(rng.next(descriptor, GT_STREAM));
-                // TODO: make generation of EQ configurable; turn it off and on
-                boolean isEquals = RngUtils.asBoolean(rng.next(descriptor, E_STREAM));
-
-                long[] sliced = schema.ckGenerator.slice(cd);
-                long min;
-                long max;
-                int nonEqFrom = RngUtils.asInt(descriptor, 0, sliced.length - 1);
-
-                long[] minBound = new long[sliced.length];
-                long[] maxBound = new long[sliced.length];
-
-                // Algorithm that determines boundaries for a clustering slice.
-                //
-                // Basic principles are not hard but there are a few edge cases. I haven't figured out how to simplify
-                // those, so there might be some room for improvement. In short, what we want to achieve is:
-                //
-                // 1. Every part that is restricted with an EQ relation goes into the bound verbatim.
-                // 2. Every part that is restricted with a non-EQ relation (LT, GT, LTE, GTE) is taken into the bound
-                //    if it is required to satisfy the relationship. For example, in `ck1 = 0 AND ck2 < 5`, ck2 will go
-                //    to the _max_ boundary, and minimum value will go to the _min_ boundary, since we can select every
-                //    descriptor that is prefixed with ck1.
-                // 3. Every other part (e.g., ones that are not explicitly mentioned in the query) has to be restricted
-                //    according to equality. For example, in `ck1 = 0 AND ck2 < 5`, ck3 that is present in schema but not
-                //    mentioned in query, makes sure that any value between [0, min_value, min_value] and [0, 5, min_value]
-                //    is matched.
-                //
-                // One edge case is a query on the first clustering key: `ck1 < 5`. In this case, we have to fixup the lower
-                // value to the minimum possible value. We could really just do Long.MIN_VALUE, but in case we forget to
-                // adjust entropy elsewhere, it'll be caught correctly here.
-                for (int i = 0; i < sliced.length; i++)
-                {
-                    long v = sliced[i];
-                    DataGenerators.KeyGenerator gen = schema.ckGenerator;
-                    ColumnSpec column = schema.clusteringKeys.get(i);
-                    int idx = i;
-                    LongSupplier maxSupplier = () -> gen.maxValue(idx);
-                    LongSupplier minSupplier = () -> gen.minValue(idx);
-
-                    if (i < nonEqFrom)
-                    {
-                        relations.add(Relation.eqRelation(schema.clusteringKeys.get(i), v));
-                        minBound[i] = v;
-                        maxBound[i] = v;
-                    }
-                    else if (i == nonEqFrom)
-                    {
-                        relations.add(Relation.relation(relationKind(isGt, isEquals), schema.clusteringKeys.get(i), v));
-
-                        if (column.isReversed())
-                        {
-                            minBound[i] = isGt ? minSupplier.getAsLong() : v;
-                            maxBound[i] = isGt ? v : maxSupplier.getAsLong();
-                        }
-                        else
-                        {
-                            minBound[i] = isGt ? v : minSupplier.getAsLong();
-                            maxBound[i] = isGt ? maxSupplier.getAsLong() : v;
-                        }
-                    }
-                    else
-                    {
-                        if (isEquals)
-                        {
-                            minBound[i] = minSupplier.getAsLong();
-                            maxBound[i] = maxSupplier.getAsLong();
-                        }
-                        else if (i > 0 && schema.clusteringKeys.get(i - 1).isReversed())
-                            maxBound[i] = minBound[i] = isGt ? minSupplier.getAsLong() : maxSupplier.getAsLong();
-                        else
-                            maxBound[i] = minBound[i] = isGt ? maxSupplier.getAsLong() : minSupplier.getAsLong();
-                    }
-                }
-
-                if (schema.clusteringKeys.get(nonEqFrom).isReversed())
-                    isGt = !isGt;
-
-                min = schema.ckGenerator.stitch(minBound);
-                max = schema.ckGenerator.stitch(maxBound);
-
-                if (nonEqFrom == 0)
-                {
-                    min = isGt ? min : schema.ckGenerator.minValue();
-                    max = !isGt ? max : schema.ckGenerator.maxValue();
-                }
-
-                // if we're about to create an "impossible" query, just bump the modifier and re-generate
-                if (min == max && !isEquals)
-                    return inflate(lts, modifier + 1);
-
-                return new Query.ClusteringSliceQuery(Query.QueryKind.CLUSTERING_SLICE,
-                                                      pd,
-                                                      min,
-                                                      max,
-                                                      relationKind(true, isGt ? isEquals : true),
-                                                      relationKind(false, !isGt ? isEquals : true),
-                                                      reverse,
-                                                      relations,
-                                                      schema);
-            }
-            case CLUSTERING_RANGE:
-            {
-                List<Relation> relations = new ArrayList<>();
-                long cd1 = descriptorSelector.randomCd(pd, descriptor, schema);
-                boolean isMinEq = RngUtils.asBoolean(descriptor);
-                long cd2 = descriptorSelector.randomCd(pd, rng.next(descriptor, lts), schema);
-                boolean isMaxEq = RngUtils.asBoolean(rng.next(descriptor, lts));
-
-                long[] minBound = schema.ckGenerator.slice(cd1);
-                long[] maxBound = schema.ckGenerator.slice(cd2);
-
-                int lock = RngUtils.asInt(descriptor, 0, schema.clusteringKeys.size() - 1);
-
-                // Logic here is similar to how clustering slices are implemented, except for both lower and upper bound
-                // get their values from sliced value in (1) and (2) cases:
-                //
-                // 1. Every part that is restricted with an EQ relation, takes its value from the min bound.
-                //    TODO: this can actually be improved, since in case of hierarchical clustering generation we can
-                //          pick out of the keys that are already locked. That said, we'll exercise more cases the way
-                //          it is implemented right now.
-                // 2. Every part that is restricted with a non-EQ relation is taken into the bound, if it is used in
-                //    the query. For example in, `ck1 = 0 AND ck2 > 2 AND ck2 < 5`, ck2 values 2 and 5 will be placed,
-                //    correspondingly, to the min and max bound.
-                // 3. Every other part has to be restricted according to equality. Similar to clustering slice, we have
-                //    to decide whether we use a min or the max value for the bound. Foe example `ck1 = 0 AND ck2 > 2 AND ck2 <= 5`,
-                //    assuming we have ck3 that is present in schema but not mentioned in the query, we'll have bounds
-                //    created as follows: [0, 2, max_value] and [0, 5, max_value]. Idea here is that since ck2 = 2 is excluded,
-                //    we also disallow all ck3 values for [0, 2] prefix. Similarly, since ck2 = 5 is included, we allow every
-                //    ck3 value with a prefix of [0, 5].
-                for (int i = 0; i < schema.clusteringKeys.size(); i++)
-                {
-                    ColumnSpec<?> col = schema.clusteringKeys.get(i);
-                    if (i < lock)
-                    {
-                        relations.add(Relation.eqRelation(col, minBound[i]));
-                        maxBound[i] = minBound[i];
-                    }
-                    else if (i == lock)
-                    {
-                        long minLocked = Math.min(minBound[lock], maxBound[lock]);
-                        long maxLocked = Math.max(minBound[lock], maxBound[lock]);
-
-                        relations.add(Relation.relation(relationKind(true, isMinEq), col, minLocked));
-
-                        minBound[i] = col.isReversed() ? maxLocked : minLocked;
-                        relations.add(Relation.relation(relationKind(false, isMaxEq), col, maxLocked));
-                        maxBound[i] = col.isReversed() ? minLocked : maxLocked;
-                    }
-                    else
-                    {
-                        if (i > 0 && schema.clusteringKeys.get(i - 1).isReversed())
-                        {
-                            minBound[i] = isMinEq ? schema.ckGenerator.maxValue(i) : schema.ckGenerator.minValue(i);
-                            maxBound[i] = isMaxEq ? schema.ckGenerator.minValue(i) : schema.ckGenerator.maxValue(i);
-                        }
-                        else
-                        {
-                            minBound[i] = isMinEq ? schema.ckGenerator.minValue(i) : schema.ckGenerator.maxValue(i);
-                            maxBound[i] = isMaxEq ? schema.ckGenerator.maxValue(i) : schema.ckGenerator.minValue(i);
-                        }
-                    }
-                }
-
-                long stitchedMin = schema.ckGenerator.stitch(minBound);
-                long stitchedMax = schema.ckGenerator.stitch(maxBound);
-
-                // if we're about to create an "impossible" query, just bump the modifier and re-generate
-                if (((stitchedMin == stitchedMax) && (!isMinEq || !isMaxEq))
-                    // TODO: so this isn't considered "normal" that we do it this way, but I'd rather fix it with
-                    //       a refactoring that's mentioned below
-                    || stitchedMin > stitchedMax)
-                    return inflate(lts, modifier + 1);
-
-                // TODO: one of the ways to get rid of garbage here, and potentially even simplify the code is to
-                //       simply return bounds here. After bounds are created, we slice them and generate query right
-                //       from the bounds. In this case, we can even say that things like -inf/+inf are special values,
-                //       and use them as placeholdrs. Also, it'll be easier to manipulate relations.
-                return new Query.ClusteringRangeQuery(Query.QueryKind.CLUSTERING_RANGE,
-                                                      pd,
-                                                      stitchedMin,
-                                                      stitchedMax,
-                                                      relationKind(true, isMinEq),
-                                                      relationKind(false, isMaxEq),
-                                                      reverse,
-                                                      relations,
-                                                      schema);
-            }
-            default:
-                throw new IllegalArgumentException("Shouldn't happen");
-        }
-    }
-
-    public static Relation.RelationKind relationKind(boolean isGt, boolean isEquals)
-    {
-        if (isGt)
-            return isEquals ? Relation.RelationKind.GTE : Relation.RelationKind.GT;
-        else
-            return isEquals ? Relation.RelationKind.LTE : Relation.RelationKind.LT;
-    }
-}
diff --git a/harry-core/src/harry/runner/RecentPartitionValidator.java b/harry-core/src/harry/runner/RecentPartitionValidator.java
index 82d4bda..69388dd 100644
--- a/harry-core/src/harry/runner/RecentPartitionValidator.java
+++ b/harry-core/src/harry/runner/RecentPartitionValidator.java
@@ -18,32 +18,48 @@
 
 package harry.runner;
 
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.util.Arrays;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import harry.core.MetricReporter;
 import harry.core.Run;
 import harry.generators.Surjections;
 import harry.model.Model;
 import harry.model.OpSelectors;
+import harry.operations.CompiledStatement;
 
 public class RecentPartitionValidator implements PartitionVisitor
 {
+    private final BufferedWriter validationLog;
+    private static final Logger logger = LoggerFactory.getLogger(RecentPartitionValidator.class);
     private final Model model;
 
-    private final OpSelectors.MonotonicClock clock;
     private final OpSelectors.PdSelector pdSelector;
     private final QueryGenerator.TypedQueryGenerator querySelector;
     private final MetricReporter metricReporter;
     private final int partitionCount;
     private final int triggerAfter;
+    private final int queries;
 
     public RecentPartitionValidator(int partitionCount,
+                                    int queries,
                                     int triggerAfter,
                                     Run run,
                                     Model.ModelFactory modelFactory)
     {
         this.partitionCount = partitionCount;
         this.triggerAfter = triggerAfter;
+        this.queries = queries;
         this.metricReporter = run.metricReporter;
-        this.clock = run.clock;
         this.pdSelector = run.pdSelector;
 
         this.querySelector = new QueryGenerator.TypedQueryGenerator(run.rng,
@@ -51,21 +67,36 @@ public class RecentPartitionValidator implements PartitionVisitor
                                                                     Surjections.enumValues(Query.QueryKind.class),
                                                                     run.rangeSelector);
         this.model = modelFactory.make(run);
+        File f = new File("validation.log");
+        try
+        {
+            validationLog = new BufferedWriter(new OutputStreamWriter(new FileOutputStream(f)));
+        }
+        catch (FileNotFoundException e)
+        {
+            throw new RuntimeException(e);
+        }
     }
 
+    private final AtomicLong maxPos = new AtomicLong(-1);
+
     // TODO: expose metric, how many times validated recent partitions
-    private void validateRecentPartitions(int partitionCount)
+    private void validateRecentPartitions(long lts)
     {
-        long maxLts = clock.maxLts();
-        long pos = pdSelector.positionFor(maxLts);
+        long pos = maxPos.get();
 
         int maxPartitions = partitionCount;
-        while (pos > 0 && maxPartitions > 0 && !Thread.currentThread().isInterrupted())
+        while (pos >= 0 && maxPartitions > 0 && !Thread.currentThread().isInterrupted())
         {
             long visitLts = pdSelector.minLtsAt(pos);
-
-            metricReporter.validateRandomQuery();
-            model.validate(querySelector.inflate(visitLts, 0));
+            for (int i = 0; i < queries; i++)
+            {
+                metricReporter.validateRandomQuery();
+                Query query = querySelector.inflate(visitLts, i);
+                // TODO: add pd skipping from shrinker here, too
+                log(lts, i, query);
+                model.validate(query);
+            }
 
             pos--;
             maxPartitions--;
@@ -74,11 +105,32 @@ public class RecentPartitionValidator implements PartitionVisitor
 
     public void visitPartition(long lts)
     {
-        if (lts % triggerAfter == 0)
-            validateRecentPartitions(partitionCount);
+        maxPos.updateAndGet(current -> Math.max(pdSelector.positionFor(lts), current));
+
+        if (triggerAfter > 0 && lts % triggerAfter == 0)
+        {
+            logger.info("Validating {} recent partitions", partitionCount);
+            validateRecentPartitions(lts);
+        }
     }
 
     public void shutdown() throws InterruptedException
     {
     }
+
+    private void log(long lts, int modifier, Query query)
+    {
+        try
+        {
+            validationLog.write("LTS: " + lts + ". Modifier: " + modifier + ". PD: " + query.pd);
+            validationLog.write("\t");
+            validationLog.write(query.toSelectStatement().toString());
+            validationLog.write("\n");
+            validationLog.flush();
+        }
+        catch (IOException e)
+        {
+            // ignore
+        }
+    }
 }
\ No newline at end of file
diff --git a/harry-core/src/harry/runner/Runner.java b/harry-core/src/harry/runner/Runner.java
index a96dd69..06bc481 100644
--- a/harry-core/src/harry/runner/Runner.java
+++ b/harry-core/src/harry/runner/Runner.java
@@ -28,8 +28,10 @@ import java.util.List;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.function.BooleanSupplier;
 
 import org.slf4j.Logger;
@@ -46,6 +48,7 @@ public abstract class Runner
 
     protected final Run run;
     protected final Configuration config;
+
     // If there's an error, there's a good chance we're going to hit it more than once
     //  since we have multiple concurrent checkers running
     protected final CopyOnWriteArrayList<Throwable> errors;
@@ -83,6 +86,8 @@ public abstract class Runner
                                                run.schemaSpec.keyspace,
                                                run.schemaSpec.table));
         }
+
+        run.sut.afterSchemaInit();
     }
 
     public void teardown()
@@ -98,7 +103,7 @@ public abstract class Runner
             }
 
             logger.info("Dropping table: " + run.schemaSpec.table);
-            run.sut.schemaChange(String.format("DROP TABLE %s.%s;",
+            run.sut.schemaChange(String.format("DROP TABLE IF EXISTS %s.%s;",
                                                run.schemaSpec.keyspace,
                                                run.schemaSpec.table));
         }
@@ -110,11 +115,11 @@ public abstract class Runner
             dumpStateToFile(run, config, errors);
     }
 
-    public abstract CompletableFuture initAndStartAll() throws InterruptedException;
+    public abstract CompletableFuture<?> initAndStartAll() throws InterruptedException;
 
     public abstract void shutdown() throws InterruptedException;
 
-    protected Runnable reportThrowable(Runnable runnable, CompletableFuture future)
+    protected Runnable reportThrowable(Runnable runnable, CompletableFuture<?> future)
     {
         return () -> {
             try
@@ -152,23 +157,24 @@ public abstract class Runner
                 partitionVisitors.add(factory.make(run));
         }
 
-        public CompletableFuture initAndStartAll()
+        public CompletableFuture<?> initAndStartAll()
         {
             init();
-            CompletableFuture future = new CompletableFuture();
+            CompletableFuture<?> future = new CompletableFuture<>();
             future.whenComplete((a, b) -> maybeReportErrors());
 
+            AtomicBoolean completed = new AtomicBoolean(false);
             shutdownExceutor.schedule(() -> {
                 logger.info("Completed");
                 // TODO: wait for the last full validation?
-                future.complete(null);
+                completed.set(true);
             }, config.run_time, config.run_time_unit);
 
             executor.submit(reportThrowable(() -> {
                                                 try
                                                 {
-                                                    run(partitionVisitors, run.clock,
-                                                        () -> Thread.currentThread().isInterrupted() || future.isDone());
+                                                    SequentialRunner.run(partitionVisitors, run.clock, future,
+                                                                         () -> Thread.currentThread().isInterrupted() || future.isDone() || completed.get());
                                                 }
                                                 catch (Throwable t)
                                                 {
@@ -180,16 +186,33 @@ public abstract class Runner
             return future;
         }
 
-        void run(List<PartitionVisitor> visitors,
-                 OpSelectors.MonotonicClock clock,
-                 BooleanSupplier exitCondition)
+        static void run(List<PartitionVisitor> visitors,
+                        OpSelectors.MonotonicClock clock,
+                        CompletableFuture<?> future,
+                        BooleanSupplier exitCondition)
         {
             while (!exitCondition.getAsBoolean())
             {
                 long lts = clock.nextLts();
-                for (PartitionVisitor partitionVisitor : visitors)
-                    partitionVisitor.visitPartition(lts);
+
+                if (lts > 0 && lts % 10_000 == 0)
+                    logger.info("Visited {} logical timestamps", lts);
+
+                for (int i = 0; i < visitors.size() && !exitCondition.getAsBoolean(); i++)
+                {
+                    try
+                    {
+                        PartitionVisitor partitionVisitor = visitors.get(i);
+                        partitionVisitor.visitPartition(lts);
+                    }
+                    catch (Throwable t)
+                    {
+                        future.completeExceptionally(t);
+                        throw t;
+                    }
+                }
             }
+            future.complete(null);
         }
 
         public void shutdown() throws InterruptedException
@@ -238,10 +261,10 @@ public abstract class Runner
             this.allVisitors = new CopyOnWriteArrayList<>();
         }
 
-        public CompletableFuture initAndStartAll()
+        public CompletableFuture<?> initAndStartAll()
         {
             init();
-            CompletableFuture future = new CompletableFuture();
+            CompletableFuture<?> future = new CompletableFuture<>();
             future.whenComplete((a, b) -> maybeReportErrors());
 
             shutdownExecutor.schedule(() -> {
@@ -255,11 +278,9 @@ public abstract class Runner
             {
                 List<PartitionVisitor> partitionVisitors = new ArrayList<>();
                 executor.submit(reportThrowable(() -> {
-
                                                     for (PartitionVisitor.PartitionVisitorFactory factory : partitionVisitorFactories)
-                                                    {
                                                         partitionVisitors.add(factory.make(run));
-                                                    }
+
                                                     allVisitors.addAll(partitionVisitors);
                                                     run(partitionVisitors, run.clock, exitCondition);
                                                 },
diff --git a/harry-core/src/harry/runner/Sampler.java b/harry-core/src/harry/runner/Sampler.java
new file mode 100644
index 0000000..c362bd9
--- /dev/null
+++ b/harry-core/src/harry/runner/Sampler.java
@@ -0,0 +1,110 @@
+/*
+ *  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 java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.commons.math3.stat.descriptive.DescriptiveStatistics;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import harry.core.Configuration;
+import harry.core.Run;
+import harry.ddl.SchemaSpec;
+import harry.generators.RngUtils;
+import harry.model.OpSelectors;
+import harry.model.SelectHelper;
+import harry.model.sut.SystemUnderTest;
+
+public class Sampler implements PartitionVisitor
+{
+    private static final Logger logger = LoggerFactory.getLogger(AllPartitionsValidator.class);
+
+    private final SystemUnderTest sut;
+    // TODO: move maxPos to data tracker since we seem to use quite a lot?
+    private final AtomicLong maxPos = new AtomicLong(-1);
+    private final OpSelectors.PdSelector pdSelector;
+    private final SchemaSpec schema;
+    private final int triggerAfter;
+    private final int samplePartitions;
+    public Sampler(Run run,
+                   int triggerAfter,
+                   int samplePartitions)
+    {
+        this.sut = run.sut;
+        this.pdSelector = run.pdSelector;
+        this.schema = run.schemaSpec;
+        this.triggerAfter = triggerAfter;
+        this.samplePartitions = samplePartitions;
+    }
+
+    public void visitPartition(long lts)
+    {
+        maxPos.updateAndGet(current -> Math.max(pdSelector.positionFor(lts), current));
+
+        if (triggerAfter > 0 && lts % triggerAfter == 0)
+        {
+            long max = maxPos.get();
+            DescriptiveStatistics ds = new DescriptiveStatistics();
+            int empty = 0;
+
+            long n = RngUtils.next(lts);
+            for (long i = 0; i < this.samplePartitions; i++)
+            {
+                long posLts = pdSelector.minLtsAt(RngUtils.asInt(n, (int) max));
+                n = RngUtils.next(n);
+                // TODO: why not just pd at pos?
+                long pd = pdSelector.pd(posLts, schema);
+                long count = (long) sut.execute(SelectHelper.count(schema, pd), SystemUnderTest.ConsistencyLevel.ONE)[0][0];
+                if (count == 0)
+                    empty++;
+                ds.addValue(count);
+            }
+            logger.info("Visited {} partitions (sampled {} empty out of {}), with mean size of {}. Median: {}. Min: {}. Max: {}",
+                        max, empty, samplePartitions, ds.getMean(), ds.getPercentile(0.5), ds.getMin(), ds.getMax());
+        }
+    }
+
+    public void shutdown() throws InterruptedException
+    {
+    }
+
+    @JsonTypeName("sampler")
+    public static class SamplerConfiguration implements Configuration.PartitionVisitorConfiguration
+    {
+        public final int trigger_after;
+        public final int sample_partitions;
+
+        @JsonCreator
+        public SamplerConfiguration(@JsonProperty(value = "trigger_after", defaultValue = "1000") int trigger_after,
+                                    @JsonProperty(value = "sample_partitions", defaultValue = "10") int sample_partitions)
+        {
+            this.trigger_after = trigger_after;
+            this.sample_partitions = sample_partitions;
+        }
+
+        public PartitionVisitor make(Run run)
+        {
+            return new Sampler(run, trigger_after, sample_partitions);
+        }
+    }
+}
diff --git a/harry-core/src/harry/runner/SinglePartitionValidator.java b/harry-core/src/harry/runner/SinglePartitionValidator.java
index e9d4f27..febfc6a 100644
--- a/harry-core/src/harry/runner/SinglePartitionValidator.java
+++ b/harry-core/src/harry/runner/SinglePartitionValidator.java
@@ -26,7 +26,7 @@ public class SinglePartitionValidator implements PartitionVisitor
     protected final int iterations;
     protected final Model model;
     protected final QueryGenerator queryGenerator;
-
+    protected final Run run;
     public SinglePartitionValidator(int iterations,
                                     Run run,
                                     Model.ModelFactory modelFactory)
@@ -34,6 +34,7 @@ public class SinglePartitionValidator implements PartitionVisitor
         this.iterations = iterations;
         this.model = modelFactory.make(run);
         this.queryGenerator = new QueryGenerator(run);
+        this.run = run;
     }
 
     public void shutdown() throws InterruptedException
@@ -45,6 +46,11 @@ public class SinglePartitionValidator implements PartitionVisitor
     {
         model.validate(queryGenerator.inflate(lts, 0, Query.QueryKind.SINGLE_PARTITION));
 
+        for (boolean reverse : new boolean[]{ true, false })
+        {
+            model.validate(Query.selectPartition(run.schemaSpec, run.pdSelector.pd(lts, run.schemaSpec), reverse));
+        }
+
         for (Query.QueryKind queryKind : new Query.QueryKind[]{ Query.QueryKind.CLUSTERING_RANGE, Query.QueryKind.CLUSTERING_SLICE, Query.QueryKind.SINGLE_CLUSTERING })
         {
             for (int i = 0; i < iterations; i++)
diff --git a/harry-core/src/harry/util/BitSet.java b/harry-core/src/harry/util/BitSet.java
index 82baf9b..3c70052 100644
--- a/harry-core/src/harry/util/BitSet.java
+++ b/harry-core/src/harry/util/BitSet.java
@@ -55,11 +55,14 @@ public interface BitSet
     public void eachBit(BitConsumer iter);
 
     public void eachSetBit(IntConsumer iter);
+    public void eachSetBit(IntConsumer iter, BitSet mask);
 
     public void eachUnsetBit(IntConsumer iter);
 
     public boolean allUnset();
 
+    public boolean allUnset(BitSet mask);
+
     public boolean allSet();
 
     public void set(int idx);
@@ -67,13 +70,26 @@ public interface BitSet
     public void unset(int idx);
 
     public boolean isSet(int idx);
+    public boolean isSet(int idx, BitSet mask);
 
     public static boolean isSet(long bits, int idx)
     {
         return (bits & (1L << idx)) != 0;
     }
 
+    public static int setCount(long bits, int size)
+    {
+        int count = 0;
+        for (int i = 0; i < size; i++)
+        {
+            if (BitSet.isSet(bits, i))
+                count++;
+        }
+        return count;
+    }
+
     public int setCount();
+    public int setCount(BitSet mask);
 
     public int size();
 
@@ -101,6 +117,12 @@ public interface BitSet
             return bits == 0;
         }
 
+        public boolean allUnset(BitSet mask)
+        {
+            assert mask instanceof BitSet64Bit;
+            return (((BitSet64Bit) mask).bits & bits) == 0;
+        }
+
         public boolean allSet()
         {
             return bits == bitMask(count);
@@ -122,6 +144,18 @@ public interface BitSet
             }
         }
 
+        public void eachSetBit(IntConsumer iter, BitSet mask)
+        {
+            assert mask instanceof BitSet64Bit;
+            long bits = (((BitSet64Bit) mask).bits & this.bits);
+            for (int i = 0; i < count; i++)
+            {
+                boolean isSet = BitSet.isSet(bits, i);
+                if (isSet)
+                    iter.accept(i);
+            }
+        }
+
         public void eachUnsetBit(IntConsumer iter)
         {
             for (int i = 0; i < count; i++)
@@ -144,18 +178,26 @@ public interface BitSet
 
         public boolean isSet(int idx)
         {
+            assert idx < size();
             return BitSet.isSet(bits, idx);
         }
 
+        public boolean isSet(int idx, BitSet mask)
+        {
+            assert idx < size();
+            assert mask instanceof BitSet64Bit;
+            return BitSet.isSet(bits & ((BitSet64Bit) mask).bits, idx);
+        }
+
         public int setCount()
         {
-            int count = 0;
-            for (int i = 0; i < size(); i++)
-            {
-                if (isSet(i))
-                    count++;
-            }
-            return count;
+            return BitSet.setCount(bits, size());
+        }
+
+        public int setCount(BitSet mask)
+        {
+            assert mask instanceof BitSet64Bit;
+            return BitSet.setCount(bits & ((BitSet64Bit) mask).bits, size());
         }
 
         public int size()
diff --git a/harry-core/test/harry/generators/RandomGeneratorTest.java b/harry-core/test/harry/generators/RandomGeneratorTest.java
index b34553d..d735670 100644
--- a/harry-core/test/harry/generators/RandomGeneratorTest.java
+++ b/harry-core/test/harry/generators/RandomGeneratorTest.java
@@ -23,6 +23,7 @@ import java.util.Random;
 import org.junit.Assert;
 import org.junit.Test;
 
+import harry.generators.distribution.Distribution;
 import harry.model.OpSelectors;
 
 import static junit.framework.TestCase.assertEquals;
@@ -34,6 +35,32 @@ public class RandomGeneratorTest
     private static int RUNS = 100000;
 
     @Test
+    public void testScale()
+    {
+        Random rand = new Random();
+        for (int cycle = 0; cycle < RUNS; cycle++)
+        {
+            int a = rand.nextInt(100);
+            int b = rand.nextInt(100);
+            while (a == b)
+                b = rand.nextInt(100);
+
+            int min = Math.min(a, b);
+            int max = Math.max(a, b);
+            long[] cardinality = new long[max - min];
+            for (int i = 0; i < 100000; i++)
+            {
+                long rnd = rand.nextLong();
+                long scaled = Distribution.ScaledDistribution.scale(rnd, min, max);
+                cardinality[(int) scaled - min]++;
+            }
+
+            for (long c : cardinality)
+                Assert.assertTrue(c > 0);
+        }
+    }
+
+    @Test
     public void testShuffleUnshuffle()
     {
         Random rnd = new Random();
diff --git a/harry-core/test/harry/model/OpSelectorsTest.java b/harry-core/test/harry/model/OpSelectorsTest.java
index d90fa5f..b291cbc 100644
--- a/harry-core/test/harry/model/OpSelectorsTest.java
+++ b/harry-core/test/harry/model/OpSelectorsTest.java
@@ -21,9 +21,11 @@ package harry.model;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.EnumMap;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
+import java.util.Random;
 import java.util.Set;
 import java.util.TreeSet;
 import java.util.function.BiConsumer;
@@ -37,6 +39,7 @@ import harry.core.Run;
 import harry.ddl.ColumnSpec;
 import harry.ddl.SchemaGenerators;
 import harry.ddl.SchemaSpec;
+import harry.generators.RngUtils;
 import harry.generators.Surjections;
 import harry.generators.distribution.Distribution;
 import harry.model.clock.OffsetClock;
@@ -45,11 +48,9 @@ import harry.operations.CompiledStatement;
 import harry.runner.DataTracker;
 import harry.runner.MutatingPartitionVisitor;
 import harry.runner.PartitionVisitor;
-import harry.runner.RowVisitor;
+import harry.runner.Operation;
 import harry.util.BitSet;
 
-import static harry.model.OpSelectors.DefaultDescriptorSelector.DEFAULT_OP_TYPE_SELECTOR;
-
 public class OpSelectorsTest
 {
     private static int RUNS = 10000;
@@ -58,13 +59,19 @@ public class OpSelectorsTest
     public void testRowDataDescriptorSupplier()
     {
         OpSelectors.Rng rng = new OpSelectors.PCGFast(1);
+        SchemaSpec schema = new SchemaSpec("ks", "tbl1",
+                                           Arrays.asList(ColumnSpec.pk("pk1", ColumnSpec.asciiType),
+                                                         ColumnSpec.pk("pk2", ColumnSpec.int64Type)),
+                                           Arrays.asList(ColumnSpec.ck("ck1", ColumnSpec.asciiType, false),
+                                                         ColumnSpec.ck("ck2", ColumnSpec.int64Type, false)),
+                                           Arrays.asList(ColumnSpec.regularColumn("v1", ColumnSpec.int32Type),
+                                                         ColumnSpec.regularColumn("v2", ColumnSpec.int64Type)),
+                                           Arrays.asList(ColumnSpec.staticColumn("static1", ColumnSpec.asciiType),
+                                                         ColumnSpec.staticColumn("static2", ColumnSpec.int64Type)));
         OpSelectors.DefaultDescriptorSelector descriptorSelector = new OpSelectors.DefaultDescriptorSelector(rng,
-                                                                                                             new OpSelectors.ColumnSelectorBuilder()
-                                                                                                             .forAll(BitSet.create(0b001, 3),
-                                                                                                                     BitSet.create(0b011, 3),
-                                                                                                                     BitSet.create(0b111, 3))
-                                                                                                             .build(),
-                                                                                                             DEFAULT_OP_TYPE_SELECTOR,
+                                                                                                             new OpSelectors.ColumnSelectorBuilder().forAll(schema)
+                                                                                                                                                    .build(),
+                                                                                                             OpSelectors.DefaultDescriptorSelector.DEFAULT_OP_SELECTOR,
                                                                                                              new Distribution.ScaledDistribution(1, 3),
                                                                                                              new Distribution.ScaledDistribution(2, 10),
                                                                                                              50);
@@ -178,11 +185,11 @@ public class OpSelectorsTest
         OpSelectors.Rng rng = new OpSelectors.PCGFast(1);
         OpSelectors.PdSelector pdSelector = new OpSelectors.DefaultPdSelector(rng, 10, 10);
         OpSelectors.DescriptorSelector ckSelector = new OpSelectors.DefaultDescriptorSelector(rng,
-                                                                                              new OpSelectors.ColumnSelectorBuilder().forAll(BitSet.allUnset(0)).build(),
-                                                                                              Surjections.weighted(Surjections.weights(10, 10, 80),
-                                                                                                                   OpSelectors.OperationKind.DELETE_ROW,
-                                                                                                                   OpSelectors.OperationKind.DELETE_COLUMN,
-                                                                                                                   OpSelectors.OperationKind.WRITE),
+                                                                                              new OpSelectors.ColumnSelectorBuilder().forAll(schema, Surjections.pick(BitSet.allUnset(0))).build(),
+                                                                                              OpSelectors.OperationSelector.weighted(Surjections.weights(10, 10, 80),
+                                                                                                                                     OpSelectors.OperationKind.DELETE_ROW,
+                                                                                                                                     OpSelectors.OperationKind.DELETE_COLUMN,
+                                                                                                                                     OpSelectors.OperationKind.WRITE),
                                                                                               new Distribution.ConstantDistribution(2),
                                                                                               new Distribution.ConstantDistribution(5),
                                                                                               10);
@@ -208,7 +215,7 @@ public class OpSelectorsTest
                 MetricReporter.NO_OP);
 
         PartitionVisitor partitionVisitor = new MutatingPartitionVisitor(run,
-                                                                         (r) -> new RowVisitor()
+                                                                         (r) -> new Operation()
                                                                          {
                                                                              public CompiledStatement write(long lts, long pd, long cd, long m)
                                                                              {
@@ -222,12 +229,30 @@ public class OpSelectorsTest
                                                                                  return compiledStatement;
                                                                              }
 
+                                                                             public CompiledStatement deleteColumnWithStatics(long lts, long pd, long cd, long opId)
+                                                                             {
+                                                                                 consumer.accept(pd, cd);
+                                                                                 return compiledStatement;
+                                                                             }
+
                                                                              public CompiledStatement deleteRow(long lts, long pd, long cd, long m)
                                                                              {
                                                                                  consumer.accept(pd, cd);
                                                                                  return compiledStatement;
                                                                              }
 
+                                                                             public CompiledStatement deletePartition(long lts, long pd, long opId)
+                                                                             {
+                                                                                 // ignore
+                                                                                 return compiledStatement;
+                                                                             }
+
+                                                                             public CompiledStatement writeWithStatics(long lts, long pd, long cd, long opId)
+                                                                             {
+                                                                                 consumer.accept(pd, cd);
+                                                                                 return compiledStatement;
+                                                                             }
+
                                                                              public CompiledStatement deleteRange(long lts, long pd, long opId)
                                                                              {
                                                                                  // ignore
@@ -258,16 +283,17 @@ public class OpSelectorsTest
                                            Arrays.asList(ColumnSpec.ck("ck1", ColumnSpec.asciiType),
                                                          ColumnSpec.ck("ck2", ColumnSpec.asciiType),
                                                          ColumnSpec.ck("ck3", ColumnSpec.asciiType)),
-                                           Collections.singletonList(ColumnSpec.regularColumn("v1", ColumnSpec.asciiType)));
+                                           Collections.singletonList(ColumnSpec.regularColumn("v1", ColumnSpec.asciiType)),
+                                           Collections.emptyList());
 
         OpSelectors.Rng rng = new OpSelectors.PCGFast(1);
         OpSelectors.DescriptorSelector ckSelector = new OpSelectors.HierarchicalDescriptorSelector(rng,
                                                                                                    new int[] {10, 20},
-                                                                                                   OpSelectors.columnSelectorBuilder().forAll(BitSet.allUnset(0)).build(),
-                                                                                                   Surjections.weighted(Surjections.weights(10, 10, 80),
-                                                                                                                        OpSelectors.OperationKind.DELETE_ROW,
-                                                                                                                        OpSelectors.OperationKind.DELETE_COLUMN,
-                                                                                                                        OpSelectors.OperationKind.WRITE),
+                                                                                                   OpSelectors.columnSelectorBuilder().forAll(schema, Surjections.pick(BitSet.allUnset(0))).build(),
+                                                                                                   OpSelectors.OperationSelector.weighted(Surjections.weights(10, 10, 80),
+                                                                                                                                          OpSelectors.OperationKind.DELETE_ROW,
+                                                                                                                                          OpSelectors.OperationKind.DELETE_COLUMN,
+                                                                                                                                          OpSelectors.OperationKind.WRITE),
                                                                                                    new Distribution.ConstantDistribution(2),
                                                                                                    new Distribution.ConstantDistribution(5),
                                                                                                    100);
@@ -286,4 +312,61 @@ public class OpSelectorsTest
         Assert.assertEquals(20, ck2.size());
         Assert.assertEquals(100, ck3.size());
     }
+
+    @Test
+    public void testWeights()
+    {
+        Map<OpSelectors.OperationKind, Integer> config = new EnumMap<>(OpSelectors.OperationKind.class);
+        config.put(OpSelectors.OperationKind.DELETE_RANGE, 1);
+        config.put(OpSelectors.OperationKind.DELETE_SLICE, 1);
+        config.put(OpSelectors.OperationKind.DELETE_ROW, 1);
+        config.put(OpSelectors.OperationKind.DELETE_COLUMN, 1);
+        config.put(OpSelectors.OperationKind.DELETE_PARTITION, 1);
+        config.put(OpSelectors.OperationKind.DELETE_COLUMN_WITH_STATICS, 1);
+        config.put(OpSelectors.OperationKind.WRITE_WITH_STATICS, 1000);
+        config.put(OpSelectors.OperationKind.WRITE, 1000);
+
+        int[] weights = new int[config.size()];
+        for (int i = 0; i < config.values().size(); i++)
+            weights[i] = config.get(OpSelectors.OperationKind.values()[i]);
+        OpSelectors.OperationSelector selector = OpSelectors.OperationSelector.weighted(Surjections.weights(weights),
+        OpSelectors.OperationKind.values());
+
+        OpSelectors.Rng rng = new OpSelectors.PCGFast(1);
+        OpSelectors.PdSelector pdSelector = new OpSelectors.DefaultPdSelector(rng, 10, 10);
+        OpSelectors.DescriptorSelector descriptorSelector = new OpSelectors.DefaultDescriptorSelector(rng,
+                                                                                                      null,
+                                                                                                      selector,
+                                                                                                      new Distribution.ConstantDistribution(2),
+                                                                                                      new Distribution.ConstantDistribution(2),
+                                                                                                      100);
+
+        EnumMap<OpSelectors.OperationKind, Integer> m = new EnumMap<OpSelectors.OperationKind, Integer>(OpSelectors.OperationKind.class);
+        for (int lts = 0; lts < 1000000; lts++)
+        {
+            int total = descriptorSelector.numberOfModifications(lts) * descriptorSelector.numberOfModifications(lts);
+            long pd = pdSelector.pd(lts);
+            for (int opId = 0; opId < total; opId++)
+            {
+                m.compute(descriptorSelector.operationType(pd, lts, opId),
+                          (OpSelectors.OperationKind k, Integer old) -> {
+                              if (old == null) return 1;
+                              else return old + 1;
+                          });
+            }
+        }
+
+        for (OpSelectors.OperationKind l : OpSelectors.OperationKind.values())
+        {
+            for (OpSelectors.OperationKind r : OpSelectors.OperationKind.values())
+            {
+                if (l != r)
+                {
+                    Assert.assertEquals(m.get(l) * 1.0 / m.get(r),
+                                        config.get(l) * 1.0 / config.get(r),
+                                        (config.get(l) * 1.0 / config.get(r)) * 0.10);
+                }
+            }
+        }
+    }
 }
\ No newline at end of file
diff --git a/harry-core/test/harry/operations/RelationTest.java b/harry-core/test/harry/operations/RelationTest.java
index 70ccd69..122b224 100644
--- a/harry-core/test/harry/operations/RelationTest.java
+++ b/harry-core/test/harry/operations/RelationTest.java
@@ -66,6 +66,7 @@ public class RelationTest
                                                        Collections.singletonList(ColumnSpec.pk("pk", ColumnSpec.int64Type)),
                                                        spec,
                                                        Collections.emptyList(),
+                                                       Collections.emptyList(),
                                                        false);
 
                 long[] cds = new long[100];
diff --git a/harry-integration/src/harry/model/sut/ExternalClusterSut.java b/harry-integration/src/harry/model/sut/ExternalClusterSut.java
index aefe508..74085e4 100644
--- a/harry-integration/src/harry/model/sut/ExternalClusterSut.java
+++ b/harry-integration/src/harry/model/sut/ExternalClusterSut.java
@@ -35,12 +35,21 @@ 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);
@@ -54,7 +63,6 @@ public class ExternalClusterSut implements SystemUnderTest
 
     public static ExternalClusterSut create()
     {
-        // TODO: close Cluster and Session!
         return new ExternalClusterSut(Cluster.builder()
                                              .withQueryOptions(new QueryOptions().setConsistencyLevel(toDriverCl(ConsistencyLevel.QUORUM)))
                                              .addContactPoints("127.0.0.1")
@@ -158,4 +166,22 @@ public class ExternalClusterSut implements SystemUnderTest
         }
         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/InJvmSut.java b/harry-integration/src/harry/model/sut/InJvmSut.java
index 1786f60..930c4d4 100644
--- a/harry-integration/src/harry/model/sut/InJvmSut.java
+++ b/harry-integration/src/harry/model/sut/InJvmSut.java
@@ -20,12 +20,7 @@ package harry.model.sut;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -35,13 +30,10 @@ import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
 import harry.core.Configuration;
 import org.apache.cassandra.distributed.Cluster;
-import org.apache.cassandra.distributed.api.Feature;
-import org.apache.cassandra.distributed.api.ICluster;
-import org.apache.cassandra.distributed.api.IMessage;
-import org.apache.cassandra.distributed.api.IMessageFilters;
-import org.apache.cassandra.net.Verb;
+import org.apache.cassandra.distributed.api.IInstanceConfig;
+import org.apache.cassandra.distributed.api.IInvokableInstance;
 
-public class InJvmSut implements SystemUnderTest
+public class InJvmSut extends InJvmSutBase<IInvokableInstance, Cluster>
 {
     public static void init()
     {
@@ -50,204 +42,45 @@ public class InJvmSut implements SystemUnderTest
 
     private static final Logger logger = LoggerFactory.getLogger(InJvmSut.class);
 
-    // TODO: shut down properly
-    private final ExecutorService executor;
-    public final Cluster cluster;
-    private final AtomicLong cnt = new AtomicLong();
-    private final AtomicBoolean isShutdown = new AtomicBoolean(false);
-
     public InJvmSut(Cluster cluster)
     {
-        this(cluster, 10);
+        super(cluster, 10);
     }
 
     public InJvmSut(Cluster cluster, int threads)
     {
-        this.cluster = cluster;
-        this.executor = Executors.newFixedThreadPool(threads);
-    }
-
-    public Cluster cluster()
-    {
-        return cluster;
-    }
-
-    public boolean isShutdown()
-    {
-        return isShutdown.get();
-    }
-
-    public void shutdown()
-    {
-        assert isShutdown.compareAndSet(false, true);
-
-        cluster.close();
-        executor.shutdown();
-
-        try
-        {
-            executor.awaitTermination(30, TimeUnit.SECONDS);
-        }
-        catch (InterruptedException e)
-        {
-            throw new RuntimeException(e);
-        }
-    }
-
-    public void schemaChange(String statement)
-    {
-        cluster.schemaChange(statement);
-    }
-
-    public Object[][] execute(String statement, ConsistencyLevel cl, Object... bindings)
-    {
-        return execute(statement, cl, (int) (cnt.getAndIncrement() % cluster.size() + 1), bindings);
-    }
-
-    public Object[][] execute(String statement, ConsistencyLevel cl, int coordinator, Object... bindings)
-    {
-        if (isShutdown.get())
-            throw new RuntimeException("Instance is shut down");
-
-        try
-        {
-            if (cl == ConsistencyLevel.NODE_LOCAL)
-            {
-                return cluster.get(coordinator)
-                              .executeInternal(statement, bindings);
-            }
-            else
-            {
-                return cluster
-                       // round-robin
-                       .coordinator(coordinator)
-                       .execute(statement, toApiCl(cl), bindings);
-            }
-        }
-        catch (Throwable t)
-        {
-            logger.error(String.format("Caught error while trying execute statement %s: %s", statement, t.getMessage()),
-                         t);
-            throw t;
-        }
-    }
-
-    // TODO: Ideally, we need to be able to induce a failure of a single specific message
-    public Object[][] executeWithWriteFailure(String statement, ConsistencyLevel cl, Object... bindings)
-    {
-        if (isShutdown.get())
-            throw new RuntimeException("Instance is shut down");
-
-        try
-        {
-            int coordinator = (int) (cnt.getAndIncrement() % cluster.size() + 1);
-            IMessageFilters filters = cluster.filters();
-
-            // Drop exactly one coordinated message
-            filters.verbs(Verb.MUTATION_REQ.id).from(coordinator).messagesMatching(new IMessageFilters.Matcher()
-            {
-                private final AtomicBoolean issued = new AtomicBoolean();
-                public boolean matches(int from, int to, IMessage message)
-                {
-                    if (from != coordinator || message.verb() != Verb.MUTATION_REQ.id)
-                        return false;
-
-                    return !issued.getAndSet(true);
-                }
-            }).drop().on();
-            Object[][] res = cluster
-                             .coordinator(coordinator)
-                             .execute(statement, toApiCl(cl), bindings);
-            filters.reset();
-            return res;
-        }
-        catch (Throwable t)
-        {
-            logger.error(String.format("Caught error while trying execute statement %s", statement),
-                         t);
-            throw t;
-        }
-    }
-
-    public CompletableFuture<Object[][]> executeAsync(String statement, ConsistencyLevel cl, Object... bindings)
-    {
-        return CompletableFuture.supplyAsync(() -> execute(statement, cl, bindings), executor);
-    }
-
-    public CompletableFuture<Object[][]> executeAsyncWithWriteFailure(String statement, ConsistencyLevel cl, Object... bindings)
-    {
-        return CompletableFuture.supplyAsync(() -> executeWithWriteFailure(statement, cl, bindings), executor);
+        super(cluster, threads);
     }
 
     @JsonTypeName("in_jvm")
-    public static class InJvmSutConfiguration implements Configuration.SutConfiguration
+    public static class InJvmSutConfiguration extends InJvmSutBaseConfiguration<IInvokableInstance, Cluster>
     {
-        public final int nodes;
-        public final int worker_threads;
-        public final String root;
-
         @JsonCreator
         public InJvmSutConfiguration(@JsonProperty(value = "nodes", defaultValue = "3") int nodes,
                                      @JsonProperty(value = "worker_threads", defaultValue = "10") int worker_threads,
                                      @JsonProperty("root") String root)
         {
-            this.nodes = nodes;
-            this.worker_threads = worker_threads;
-            this.root = root;
+            super(nodes, worker_threads, root);
         }
 
-        public SystemUnderTest make()
+        protected Cluster cluster(Consumer<IInstanceConfig> cfg, int nodes, File root)
         {
             try
             {
-                ICluster.setup();
-            }
-            catch (Throwable throwable)
-            {
-                throwable.printStackTrace();
-            }
-
-            Cluster cluster;
-            try
-            {
-                cluster = Cluster.build().withConfig((cfg) -> {
-                    // TODO: make this configurable
-                    cfg.with(Feature.NETWORK, Feature.GOSSIP)
-                       .set("row_cache_size_in_mb", 10L)
-                       .set("index_summary_capacity_in_mb", 10L)
-                       .set("counter_cache_size_in_mb", 10L)
-                       .set("key_cache_size_in_mb", 10L)
-                       .set("file_cache_size_in_mb", 10)
-                       .set("memtable_heap_space_in_mb", 128)
-                       .set("memtable_offheap_space_in_mb", 128)
-                       .set("memtable_flush_writers", 1)
-                       .set("concurrent_compactors", 1)
-                       .set("concurrent_reads", 5)
-                       .set("concurrent_writes", 5)
-                       .set("compaction_throughput_mb_per_sec", 10)
-                       .set("hinted_handoff_enabled", false);
-                })
-                                 .withNodes(nodes)
-                                 .withRoot(new File(root)).createWithoutStarting();
+                return Cluster.build().withConfig(cfg)
+                               .withNodes(nodes)
+                               .withRoot(root)
+                              .createWithoutStarting();
             }
             catch (IOException e)
             {
-                throw new RuntimeException(e);
+                throw new IllegalStateException(e);
             }
-
-            cluster.startup();
-            return new InJvmSut(cluster);
         }
-    }
 
-    public static org.apache.cassandra.distributed.api.ConsistencyLevel toApiCl(ConsistencyLevel cl)
-    {
-        switch (cl)
+        protected InJvmSutBase<IInvokableInstance, Cluster> sut(Cluster cluster)
         {
-            case ALL:    return org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
-            case QUORUM: return org.apache.cassandra.distributed.api.ConsistencyLevel.QUORUM;
-            case NODE_LOCAL: return org.apache.cassandra.distributed.api.ConsistencyLevel.NODE_LOCAL;
+            return new InJvmSut(cluster);
         }
-        throw new IllegalArgumentException("Don't know a CL: " + cl);
     }
 }
\ No newline at end of file
diff --git a/harry-integration/src/harry/model/sut/InJvmSut.java b/harry-integration/src/harry/model/sut/InJvmSutBase.java
similarity index 60%
copy from harry-integration/src/harry/model/sut/InJvmSut.java
copy to harry-integration/src/harry/model/sut/InJvmSutBase.java
index 1786f60..e29481f 100644
--- a/harry-integration/src/harry/model/sut/InJvmSut.java
+++ b/harry-integration/src/harry/model/sut/InJvmSutBase.java
@@ -20,54 +20,58 @@ package harry.model.sut;
 
 import java.io.File;
 import java.io.IOException;
+import java.nio.file.Files;
+import java.util.Arrays;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 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 com.fasterxml.jackson.annotation.JsonTypeName;
 import harry.core.Configuration;
-import org.apache.cassandra.distributed.Cluster;
+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 org.apache.cassandra.net.Verb;
 
-public class InJvmSut implements SystemUnderTest
+public class InJvmSutBase<NODE extends IInstance, CLUSTER extends ICluster<NODE>> implements SystemUnderTest.FaultInjectingSut
 {
     public static void init()
     {
-        Configuration.registerSubtypes(InJvmSutConfiguration.class);
+        Configuration.registerSubtypes(InJvmSutBaseConfiguration.class);
     }
 
-    private static final Logger logger = LoggerFactory.getLogger(InJvmSut.class);
+    private static final Logger logger = LoggerFactory.getLogger(InJvmSutBase.class);
 
     // TODO: shut down properly
     private final ExecutorService executor;
-    public final Cluster cluster;
+    public final CLUSTER cluster;
     private final AtomicLong cnt = new AtomicLong();
     private final AtomicBoolean isShutdown = new AtomicBoolean(false);
 
-    public InJvmSut(Cluster cluster)
+    public InJvmSutBase(CLUSTER cluster)
     {
         this(cluster, 10);
     }
 
-    public InJvmSut(Cluster cluster, int threads)
+    public InJvmSutBase(CLUSTER cluster, int threads)
     {
         this.cluster = cluster;
         this.executor = Executors.newFixedThreadPool(threads);
     }
 
-    public Cluster cluster()
+    public CLUSTER cluster()
     {
         return cluster;
     }
@@ -81,14 +85,13 @@ public class InJvmSut implements SystemUnderTest
     {
         assert isShutdown.compareAndSet(false, true);
 
-        cluster.close();
-        executor.shutdown();
-
         try
         {
+            cluster.close();
+            executor.shutdown();
             executor.awaitTermination(30, TimeUnit.SECONDS);
         }
-        catch (InterruptedException e)
+        catch (Throwable e)
         {
             throw new RuntimeException(e);
         }
@@ -116,6 +119,14 @@ public class InJvmSut implements SystemUnderTest
                 return cluster.get(coordinator)
                               .executeInternal(statement, bindings);
             }
+            else if (statement.contains("SELECT"))
+            {
+                return Iterators.toArray(cluster
+                                         // round-robin
+                                         .coordinator(coordinator)
+                                         .executeWithPaging(statement, toApiCl(cl), 1, bindings),
+                                         Object[].class);
+            }
             else
             {
                 return cluster
@@ -126,7 +137,12 @@ public class InJvmSut implements SystemUnderTest
         }
         catch (Throwable t)
         {
-            logger.error(String.format("Caught error while trying execute statement %s: %s", statement, t.getMessage()),
+            // TODO: find a better way to work around timeouts
+            if (t.getMessage().contains("timed out"))
+                return execute(statement, cl, coordinator, bindings);
+
+            logger.error(String.format("Caught error while trying execute statement %s (%s): %s",
+                                       statement, Arrays.toString(bindings), t.getMessage()),
                          t);
             throw t;
         }
@@ -144,12 +160,14 @@ public class InJvmSut implements SystemUnderTest
             IMessageFilters filters = cluster.filters();
 
             // Drop exactly one coordinated message
-            filters.verbs(Verb.MUTATION_REQ.id).from(coordinator).messagesMatching(new IMessageFilters.Matcher()
+            int MUTATION_REQ = 0;
+            // TODO: make dropping deterministic
+            filters.verbs(MUTATION_REQ).from(coordinator).messagesMatching(new IMessageFilters.Matcher()
             {
                 private final AtomicBoolean issued = new AtomicBoolean();
                 public boolean matches(int from, int to, IMessage message)
                 {
-                    if (from != coordinator || message.verb() != Verb.MUTATION_REQ.id)
+                    if (from != coordinator || message.verb() != MUTATION_REQ)
                         return false;
 
                     return !issued.getAndSet(true);
@@ -179,23 +197,39 @@ public class InJvmSut implements SystemUnderTest
         return CompletableFuture.supplyAsync(() -> executeWithWriteFailure(statement, cl, bindings), executor);
     }
 
-    @JsonTypeName("in_jvm")
-    public static class InJvmSutConfiguration implements Configuration.SutConfiguration
+    public static abstract class InJvmSutBaseConfiguration<NODE extends IInstance, CLUSTER extends ICluster<NODE>> implements Configuration.SutConfiguration
     {
         public final int nodes;
         public final int worker_threads;
         public final String root;
 
         @JsonCreator
-        public InJvmSutConfiguration(@JsonProperty(value = "nodes", defaultValue = "3") int nodes,
-                                     @JsonProperty(value = "worker_threads", defaultValue = "10") int worker_threads,
-                                     @JsonProperty("root") String root)
+        public InJvmSutBaseConfiguration(@JsonProperty(value = "nodes", defaultValue = "3") int nodes,
+                                         @JsonProperty(value = "worker_threads", defaultValue = "10") int worker_threads,
+                                         @JsonProperty("root") String root)
         {
             this.nodes = nodes;
             this.worker_threads = worker_threads;
-            this.root = root;
+            if (root == null)
+            {
+                try
+                {
+                    this.root = Files.createTempDirectory("cluster_" + nodes + "_nodes").toString();
+                }
+                catch (IOException e)
+                {
+                    throw new IllegalArgumentException(e);
+                }
+            }
+            else
+            {
+                this.root = root;
+            }
         }
 
+        protected abstract CLUSTER cluster(Consumer<IInstanceConfig> cfg, int nodes, File root);
+        protected abstract InJvmSutBase<NODE, CLUSTER> sut(CLUSTER cluster);
+
         public SystemUnderTest make()
         {
             try
@@ -204,39 +238,33 @@ public class InJvmSut implements SystemUnderTest
             }
             catch (Throwable throwable)
             {
-                throwable.printStackTrace();
+                throw new RuntimeException(throwable);
             }
 
-            Cluster cluster;
-            try
-            {
-                cluster = Cluster.build().withConfig((cfg) -> {
-                    // TODO: make this configurable
-                    cfg.with(Feature.NETWORK, Feature.GOSSIP)
-                       .set("row_cache_size_in_mb", 10L)
-                       .set("index_summary_capacity_in_mb", 10L)
-                       .set("counter_cache_size_in_mb", 10L)
-                       .set("key_cache_size_in_mb", 10L)
-                       .set("file_cache_size_in_mb", 10)
-                       .set("memtable_heap_space_in_mb", 128)
-                       .set("memtable_offheap_space_in_mb", 128)
-                       .set("memtable_flush_writers", 1)
-                       .set("concurrent_compactors", 1)
-                       .set("concurrent_reads", 5)
-                       .set("concurrent_writes", 5)
-                       .set("compaction_throughput_mb_per_sec", 10)
-                       .set("hinted_handoff_enabled", false);
-                })
-                                 .withNodes(nodes)
-                                 .withRoot(new File(root)).createWithoutStarting();
-            }
-            catch (IOException e)
-            {
-                throw new RuntimeException(e);
-            }
+            CLUSTER cluster;
+
+            cluster = cluster((cfg) -> {
+                                  // TODO: make this configurable
+                                  cfg.with(Feature.NETWORK, Feature.GOSSIP, Feature.NATIVE_PROTOCOL)
+                                     .set("row_cache_size_in_mb", 10L)
+                                     .set("index_summary_capacity_in_mb", 10L)
+                                     .set("counter_cache_size_in_mb", 10L)
+                                     .set("key_cache_size_in_mb", 10L)
+                                     .set("file_cache_size_in_mb", 10)
+                                     .set("memtable_heap_space_in_mb", 128)
+                                     .set("memtable_offheap_space_in_mb", 128)
+                                     .set("memtable_flush_writers", 1)
+                                     .set("concurrent_compactors", 1)
+                                     .set("concurrent_reads", 5)
+                                     .set("concurrent_writes", 5)
+                                     .set("compaction_throughput_mb_per_sec", 10)
+                                     .set("hinted_handoff_enabled", false);
+                              },
+                              nodes,
+                              new File(root));
 
             cluster.startup();
-            return new InJvmSut(cluster);
+            return sut(cluster);
         }
     }
 
@@ -247,7 +275,8 @@ public class InJvmSut implements SystemUnderTest
             case ALL:    return org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
             case QUORUM: return org.apache.cassandra.distributed.api.ConsistencyLevel.QUORUM;
             case NODE_LOCAL: return org.apache.cassandra.distributed.api.ConsistencyLevel.NODE_LOCAL;
+            case ONE: return org.apache.cassandra.distributed.api.ConsistencyLevel.ONE;
         }
         throw new IllegalArgumentException("Don't know a CL: " + cl);
     }
-}
\ No newline at end of file
+}
diff --git a/harry-integration/src/harry/model/sut/MixedVersionInJvmSut.java b/harry-integration/src/harry/model/sut/MixedVersionInJvmSut.java
new file mode 100644
index 0000000..5ac03a7
--- /dev/null
+++ b/harry-integration/src/harry/model/sut/MixedVersionInJvmSut.java
@@ -0,0 +1,138 @@
+/*
+ *  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.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Consumer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import harry.core.Configuration;
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.IInstanceConfig;
+import org.apache.cassandra.distributed.api.IUpgradeableInstance;
+import org.apache.cassandra.distributed.shared.Versions;
+
+/**
+ * Make sure to set -Dcassandra.test.dtest_jar_path when using this class
+ */
+public class MixedVersionInJvmSut extends InJvmSutBase<IUpgradeableInstance, UpgradeableCluster>
+{
+    public static void init()
+    {
+        Configuration.registerSubtypes(MixedInJvmSutConfiguration.class);
+    }
+
+    private static final Logger logger = LoggerFactory.getLogger(MixedVersionInJvmSut.class);
+    private final Versions.Version initialVersion;
+    private final List<Versions.Version> versions;
+
+    public MixedVersionInJvmSut(UpgradeableCluster cluster, Versions.Version initialVersion, List<Versions.Version> versions)
+    {
+        super(cluster, 10);
+        this.initialVersion = initialVersion;
+        this.versions = versions;
+    }
+
+    @JsonTypeName("mixed_in_jvm")
+    public static class MixedInJvmSutConfiguration extends InJvmSutBaseConfiguration<IUpgradeableInstance, UpgradeableCluster>
+    {
+        public final String initial_version;
+        public final List<String> versions;
+
+        private final Versions.Version initialVersion;
+        private final List<Versions.Version> upgradeVersions;
+
+        @JsonCreator
+        public MixedInJvmSutConfiguration(@JsonProperty(value = "nodes", defaultValue = "3") int nodes,
+                                          @JsonProperty(value = "worker_threads", defaultValue = "10") int worker_threads,
+                                          @JsonProperty(value = "initial_version") String initial_version,
+                                          @JsonProperty(value = "versions") List<String> versions,
+                                          @JsonProperty("root") String root)
+        {
+            super(nodes, worker_threads, root);
+
+            this.initial_version = initial_version;
+            this.versions = versions;
+            Versions allVersions = Versions.find();
+
+            this.initialVersion = allVersions.get(initial_version);
+            this.upgradeVersions = new ArrayList<>();
+            for (String version : versions)
+                upgradeVersions.add(allVersions.get(version));
+        }
+
+        protected UpgradeableCluster cluster(Consumer<IInstanceConfig> cfg, int nodes, File root)
+        {
+            try
+            {
+                return UpgradeableCluster.build()
+                                         .withConfig(cfg)
+                                         .withNodes(nodes)
+                                         .withRoot(root)
+                                         .withVersion(initialVersion)
+                                         .createWithoutStarting();
+            }
+            catch (IOException e)
+            {
+                throw new IllegalStateException(e);
+            }
+        }
+
+        protected InJvmSutBase<IUpgradeableInstance, UpgradeableCluster> sut(UpgradeableCluster cluster)
+        {
+            return new MixedVersionInJvmSut(cluster, initialVersion, upgradeVersions);
+        }
+    }
+
+    @Override
+    public void afterSchemaInit()
+    {
+        for (int i = 1; i <= cluster.size(); i++)
+        {
+            Versions.Version v = versions.get(i - 1);
+            if (!v.equals(initialVersion))
+            {
+                logger.info("Upgrading {} node from {} to {}", i, initialVersion, v);
+                IUpgradeableInstance instance = cluster.get(i);
+                try
+                {
+                    instance.shutdown().get();
+                }
+                catch (Throwable e)
+                {
+                    throw new RuntimeException(e);
+                }
+                instance.setVersion(v);
+                instance.startup();
+            }
+            else
+            {
+                logger.info("Skipping {} node upgrade, since it is already at the required version ({})", i, initialVersion);
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/harry-integration/src/harry/runner/FaultInjectingPartitionVisitor.java b/harry-integration/src/harry/runner/FaultInjectingPartitionVisitor.java
index 84e1a73..a6483bc 100644
--- a/harry-integration/src/harry/runner/FaultInjectingPartitionVisitor.java
+++ b/harry-integration/src/harry/runner/FaultInjectingPartitionVisitor.java
@@ -28,6 +28,7 @@ import com.fasterxml.jackson.annotation.JsonTypeName;
 import harry.core.Configuration;
 import harry.core.Run;
 import harry.model.sut.InJvmSut;
+import harry.model.sut.MixedVersionInJvmSut;
 import harry.model.sut.SystemUnderTest;
 import harry.operations.CompiledStatement;
 
@@ -56,12 +57,12 @@ public class FaultInjectingPartitionVisitor extends LoggingPartitionVisitor
 
     private final AtomicInteger cnt = new AtomicInteger();
 
-    private final InJvmSut sut;
+    private final SystemUnderTest.FaultInjectingSut sut;
 
-    public FaultInjectingPartitionVisitor(Run run, RowVisitor.RowVisitorFactory rowVisitorFactory)
+    public FaultInjectingPartitionVisitor(Run run, Operation.RowVisitorFactory rowVisitorFactory)
     {
         super(run, rowVisitorFactory);
-        this.sut = (InJvmSut) run.sut;
+        this.sut = (SystemUnderTest.FaultInjectingSut) run.sut;
     }
 
     void executeAsyncWithRetries(CompletableFuture<Object[][]> originator, CompiledStatement statement)
diff --git a/harry-integration/src/harry/runner/Reproduce.java b/harry-integration/src/harry/runner/Reproduce.java
index 78a96f6..f3ae9cb 100644
--- a/harry-integration/src/harry/runner/Reproduce.java
+++ b/harry-integration/src/harry/runner/Reproduce.java
@@ -18,51 +18,34 @@
 
 package harry.runner;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import java.io.File;
 
 import harry.core.Configuration;
 import harry.core.Run;
-import harry.model.sut.InJvmSut;
-import org.apache.cassandra.distributed.test.TestBaseImpl;
+import harry.model.sut.PrintlnSut;
+import harry.reconciler.Reconciler;
 
-public class Reproduce extends TestBaseImpl
+public class Reproduce
 {
 
-    private static final Logger logger = LoggerFactory.getLogger(HarryRunner.class);
-
-    public void runWithInJvmDtest() throws Throwable
+    public static void main(String[] args) throws Throwable
     {
-        InJvmSut.init();
-
-        System.setProperty("cassandra.disable_tcactive_openssl", "true");
-        System.setProperty("relocated.shaded.io.netty.transport.noNative", "true");
-        System.setProperty("org.apache.cassandra.disable_mbean_registration", "true");
-
-        Configuration configuration = Configuration.fromFile("shared/run.yaml");
-
-        Runner runner = configuration.createRunner();
-        Run run = runner.getRun();
-
-        try
-        {
-//            run.validator.validatePartition(0L);
-        }
-        catch(Throwable t)
-        {
-            logger.error(t.getMessage(), t);
-        }
-    }
-
-    public static void main(String[] args) throws Throwable {
-        try
-        {
-            new Reproduce().runWithInJvmDtest();
-        }
-        catch (Throwable t)
-        {
-            logger.error("Error: ", t);
-        }
+        File configFile = HarryRunner.loadConfig(args);
+        Configuration configuration = Configuration.fromFile(configFile);
+        System.out.println(Configuration.toYamlString(configuration));
+        configuration = configuration.unbuild().setSUT(PrintlnSut::new).build();
+
+        Run run = configuration.createRun();
+
+        Reconciler reconciler = new Reconciler(run);
+        long pd = 8135884698435133227L;
+        System.out.println(reconciler.inflatePartitionState(pd,
+                                                            5908L,
+                                                            Query.selectPartition(run.schemaSpec,
+                                                                                  pd,
+                                                                                  false))
+                                     .toString(run.schemaSpec));
+
+        // Try out everything you might want to try with a given run
     }
 }
-
diff --git a/harry-integration/src/harry/runner/TrivialShrinker.java b/harry-integration/src/harry/runner/TrivialShrinker.java
new file mode 100644
index 0000000..82e0572
--- /dev/null
+++ b/harry-integration/src/harry/runner/TrivialShrinker.java
@@ -0,0 +1,232 @@
+/*
+ *  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 java.io.File;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Predicate;
+
+import harry.core.Configuration;
+import harry.core.Run;
+
+/**
+ * A most trivial imaginable shrinker: attempts to skip partitions and/or logical timestamps to see if the
+ * issue is still reproducible.
+ */
+public class TrivialShrinker
+{
+    public static void main(String[] args) throws Throwable
+    {
+        try
+        {
+            File configFile = HarryRunner.loadConfig(args);
+            Configuration configuration = Configuration.fromFile(configFile);
+            System.out.println(Configuration.toYamlString(configuration));
+
+            Set<Long> pdsToSkip = new HashSet<>(Arrays.asList(
+            // put pds you want to skip here, or Harry will find them for you
+            ));
+
+            Set<Long> ltsToSkip = new HashSet<>(Arrays.asList(
+            // put lts you want to skip here, or Harry will find them for you
+            ));
+
+            // Which LTS failure has occurred on
+            final long maxLts = 7000L;
+
+            // Check if we've found exactly the exception that is causing the failure
+            Predicate<Exception> check = (e) -> true;
+
+            Run run = configuration.createRun();
+            Configuration.SequentialRunnerConfig config = (Configuration.SequentialRunnerConfig) configuration.runner;
+            List<PartitionVisitor> visitors = new ArrayList<>();
+            for (Configuration.PartitionVisitorConfiguration factory : config.partition_visitor_factories)
+            {
+                PartitionVisitor visitor = factory.make(run);
+                if (visitor instanceof AbstractPartitionVisitor)
+                {
+                    visitors.add(new SkippingPartitionVisitor((AbstractPartitionVisitor) visitor,
+                                                              ltsToSkip,
+                                                              pdsToSkip));
+                }
+                else
+                {
+                    visitors.add(visitor);
+                }
+            }
+
+            Set<Long> partitions = new HashSet<>();
+            for (long i = 0; i < maxLts; i++)
+                partitions.add(run.pdSelector.pd(i, run.schemaSpec));
+
+            // Step one: figure out which partitions we can skip while still keeping it reproducible
+            for (Long pdToCheck : partitions)
+            {
+                if (pdsToSkip.contains(pdToCheck))
+                    continue;
+                pdsToSkip.add(pdToCheck);
+                config.make(run, configuration).init();
+
+                try
+                {
+                    runOnce(run, visitors, maxLts);
+                    System.out.println("Can not skip " + pdToCheck + "\nCan only skip these: " + toString(pdsToSkip));
+                    pdsToSkip.remove(pdToCheck);
+                }
+                catch (RuntimeException t)
+                {
+                    if (check.test(t))
+                    {
+                        System.out.printf("Safe to skip: %d because without it we're still hitting an exception %s.\n%s\n",
+                                          pdToCheck,
+                                          t.getMessage(),
+                                          toString(pdsToSkip));
+                    }
+                    else
+                    {
+                        System.out.println("Can not skip " + pdToCheck + "\n, since we seem to repro a different issue. Can only skip these: " + toString(pdsToSkip));
+                        pdsToSkip.remove(pdToCheck);
+                    }
+                }
+                run.sut.schemaChange("DROP KEYSPACE " + run.schemaSpec.keyspace);
+            }
+
+            // Step two: figure out which lts can be skipped within the remaining partitions
+            for (long lts = 0; lts < maxLts; lts++)
+            {
+                long ltsToCheck = lts;
+                if (ltsToSkip.contains(ltsToCheck) || pdsToSkip.contains(run.pdSelector.pd(lts, run.schemaSpec)))
+                    continue;
+                ltsToSkip.add(ltsToCheck);
+                config.make(run, configuration).init();
+
+                try
+                {
+                    runOnce(run, visitors, maxLts);
+                    System.out.println("Can not skip " + ltsToCheck + "\nCan only skip these: " + toString(ltsToSkip));
+                    ltsToSkip.remove(ltsToCheck);
+                }
+                catch (RuntimeException t)
+                {
+                    if (check.test(t))
+                    {
+                        System.out.printf("Safe to skip: %d because without it we're still hitting an exception %s.\n%s\n",
+                                          ltsToCheck,
+                                          t.getMessage(),
+                                          toString(ltsToSkip));
+                    }
+                    else
+                    {
+                        System.out.println("Can not skip " + lts + "\n, since we seem to repro a different issue. Can only skip these: " + toString(ltsToSkip));
+                        ltsToSkip.remove(ltsToCheck);
+                    }
+
+                }
+                run.sut.schemaChange("DROP KEYSPACE " + run.schemaSpec.keyspace);
+            }
+        }
+        catch (Throwable t)
+        {
+            System.out.println(t.getMessage());
+            t.printStackTrace();
+        }
+        finally
+        {
+            System.exit(1);
+        }
+    }
+
+    public static void runOnce(Run run, List<PartitionVisitor> visitors, long maxLts)
+    {
+        for (long lts = 0; lts <= maxLts; lts++)
+        {
+            for (PartitionVisitor visitor : visitors)
+            {
+                visitor.visitPartition(lts);
+            }
+        }
+    }
+
+    public static class SkippingPartitionVisitor extends AbstractPartitionVisitor
+    {
+        private final AbstractPartitionVisitor delegate;
+        private final Set<Long> ltsToSkip;
+        private final Set<Long> pdsToSkip;
+
+        public SkippingPartitionVisitor(AbstractPartitionVisitor delegate,
+                                        Set<Long> ltsToSkip,
+                                        Set<Long> pdsToSkip)
+        {
+            super(delegate.pdSelector, delegate.descriptorSelector, delegate.schema);
+            this.delegate = delegate;
+            this.ltsToSkip = ltsToSkip;
+            this.pdsToSkip = pdsToSkip;
+        }
+
+        protected void beforeLts(long lts, long pd)
+        {
+            delegate.beforeLts(lts, pd);
+        }
+
+        protected void afterLts(long lts, long pd)
+        {
+            delegate.afterLts(lts, pd);
+        }
+
+        protected void beforeBatch(long lts, long pd, long m)
+        {
+            delegate.beforeBatch(lts, pd, m);
+        }
+
+        protected void operation(long lts, long pd, long cd, long m, long opId)
+        {
+            if (pdsToSkip.contains(pd) || ltsToSkip.contains(lts))
+                return;
+
+            delegate.operation(lts, pd, cd, m, opId);
+        }
+
+        protected void afterBatch(long lts, long pd, long m)
+        {
+            delegate.afterBatch(lts, pd, m);
+        }
+
+        public void shutdown() throws InterruptedException
+        {
+            delegate.shutdown();
+        }
+    }
+
+    public static String toString(Set<Long> longs)
+    {
+        if (longs.isEmpty())
+            return "";
+
+        String s = "";
+        for (Long aLong : longs)
+        {
+            s += aLong + "L,";
+        }
+        return s.substring(0, s.length() - 1);
+    }
+}
diff --git a/harry-integration/test/harry/ddl/SchemaGenTest.java b/harry-integration/test/harry/ddl/SchemaGenTest.java
index 318887a..74a9c53 100644
--- a/harry-integration/test/harry/ddl/SchemaGenTest.java
+++ b/harry-integration/test/harry/ddl/SchemaGenTest.java
@@ -107,7 +107,9 @@ public class SchemaGenTest extends CQLTester
                                          Arrays.asList(ColumnSpec.regularColumn("v1", ColumnSpec.asciiType),
                                                        ColumnSpec.regularColumn("v2", ColumnSpec.asciiType),
                                                        ColumnSpec.regularColumn("v3", ColumnSpec.int64Type),
-                                                       ColumnSpec.regularColumn("v4", ColumnSpec.int64Type)));
+                                                       ColumnSpec.regularColumn("v4", ColumnSpec.int64Type)),
+                                         Arrays.asList(ColumnSpec.staticColumn("regular1", ColumnSpec.asciiType),
+                                                       ColumnSpec.staticColumn("regular2", ColumnSpec.int64Type)));
 
 
         String tableDef = spec.compile().cql();
diff --git a/harry-integration/test/harry/generators/DataGeneratorsIntegrationTest.java b/harry-integration/test/harry/generators/DataGeneratorsIntegrationTest.java
new file mode 100644
index 0000000..affd954
--- /dev/null
+++ b/harry-integration/test/harry/generators/DataGeneratorsIntegrationTest.java
@@ -0,0 +1,63 @@
+/*
+ *  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.generators;
+
+import java.util.Random;
+
+import org.junit.Test;
+
+import harry.ddl.ColumnSpec;
+import org.apache.cassandra.cql3.CQLTester;
+
+public class DataGeneratorsIntegrationTest extends CQLTester
+{
+    @Test
+    public void testTimestampTieResolution() throws Throwable
+    {
+        Random rng = new Random(1);
+        for (ColumnSpec.DataType<?> dataType : new ColumnSpec.DataType[]{ ColumnSpec.int8Type,
+                                                                       ColumnSpec.int16Type,
+                                                                       ColumnSpec.int32Type,
+                                                                       ColumnSpec.int64Type,
+                                                                       ColumnSpec.asciiType,
+                                                                       ColumnSpec.floatType,
+                                                                       ColumnSpec.doubleType })
+        {
+            createTable(String.format("CREATE TABLE %%s (pk int PRIMARY KEY, v %s)",
+                                      dataType.toString()));
+            for (int i = 0; i < 10_000; i++)
+            {
+                long d1 = dataType.generator().adjustEntropyDomain(rng.nextLong());
+                long d2 = dataType.generator().adjustEntropyDomain(rng.nextLong());
+                for (long d : new long[]{ d1, d2 })
+                {
+                    execute("INSERT INTO %s (pk, v) VALUES (?,?) USING TIMESTAMP 1",
+                            i, dataType.generator().inflate(d));
+                }
+
+                if (dataType.compareLexicographically(d1, d2) > 0)
+                    assertRows(execute("SELECT v FROM %s WHERE pk=?", i),
+                               row(dataType.generator().inflate(d1)));
+                else
+                    assertRows(execute("SELECT v FROM %s WHERE pk=?", i),
+                               row(dataType.generator().inflate(d2)));
+            }
+        }
+    }
+}
diff --git a/harry-integration/test/harry/model/ExhaustiveCheckerIntegrationTest.java b/harry-integration/test/harry/model/ExhaustiveCheckerIntegrationTest.java
deleted file mode 100644
index ecbf499..0000000
--- a/harry-integration/test/harry/model/ExhaustiveCheckerIntegrationTest.java
+++ /dev/null
@@ -1,235 +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;
-
-import java.util.concurrent.CompletableFuture;
-import java.util.function.Supplier;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import harry.core.Configuration;
-import harry.core.Run;
-import harry.corruptor.AddExtraRowCorruptor;
-import harry.corruptor.ChangeValueCorruptor;
-import harry.corruptor.HideRowCorruptor;
-import harry.corruptor.HideValueCorruptor;
-import harry.corruptor.QueryResponseCorruptor;
-import harry.corruptor.QueryResponseCorruptor.SimpleQueryResponseCorruptor;
-import harry.ddl.SchemaGenerators;
-import harry.model.sut.InJvmSut;
-import harry.model.sut.SystemUnderTest;
-import harry.runner.MutatingPartitionVisitor;
-import harry.runner.MutatingRowVisitor;
-import harry.runner.PartitionVisitor;
-import harry.runner.Query;
-import harry.runner.SinglePartitionValidator;
-
-public class ExhaustiveCheckerIntegrationTest extends ModelTestBase
-{
-    @Test
-    public void testVerifyPartitionState()
-    {
-        Supplier<Configuration.ConfigurationBuilder> gen = sharedConfiguration();
-
-        for (int cnt = 0; cnt < SchemaGenerators.DEFAULT_RUNS; cnt++)
-        {
-            Configuration config = gen.get().build();
-            Run run = config.createRun();
-            run.sut.schemaChange(run.schemaSpec.compile().cql());
-            OpSelectors.MonotonicClock clock = run.clock;
-
-            SinglePartitionValidator validator = new SinglePartitionValidator(100, run, modelConfiguration());
-            PartitionVisitor partitionVisitor = new MutatingPartitionVisitor(run, MutatingRowVisitor::new);
-
-            for (int i = 0; i < 2000; i++)
-            {
-                long lts = clock.nextLts();
-                partitionVisitor.visitPartition(lts);
-            }
-
-            validator.visitPartition(0);
-        }
-    }
-
-    @Test
-    public void testDetectsMissingRow()
-    {
-        negativeTest((run) -> {
-                         SimpleQueryResponseCorruptor corruptor = new SimpleQueryResponseCorruptor(run.schemaSpec,
-                                                                                                   run.clock,
-                                                                                                   HideRowCorruptor::new);
-
-                         return corruptor.maybeCorrupt(Query.selectPartition(run.schemaSpec,
-                                                                             run.pdSelector.pd(0, run.schemaSpec),
-                                                                             false),
-                                                       run.sut);
-                     },
-                     (t, run) -> Assert.assertTrue(String.format("Throwable: %s\nCause: %s", t, t.getCause()),
-                                                   t.getCause() != null && t.getCause().toString().contains(OpSelectors.OperationKind.WRITE.toString())));
-    }
-
-    @Test
-    public void testDetectsExtraRow()
-    {
-        negativeTest((run) -> {
-                         QueryResponseCorruptor corruptor = new AddExtraRowCorruptor(run.schemaSpec,
-                                                                                     run.clock,
-                                                                                     run.descriptorSelector);
-
-                         return corruptor.maybeCorrupt(Query.selectPartition(run.schemaSpec,
-                                                                             run.pdSelector.pd(0, run.schemaSpec),
-                                                                             false),
-                                                       run.sut);
-                     },
-                     (t, run) -> {
-                         Assert.assertTrue(String.format("Throwable: %s\nCause: %s", t, t.getCause()),
-                                           // TODO: this is not entirely correct. Right now, after registering a deletion followed by no writes,
-                                           // we would continue going back in time and checking other operations, even though we don't have to do this.
-                                           t.getCause().getMessage().contains("Modification should have been visible but was not") ||
-                                           // TODO: this is not entirely correct, either. This row is, in fact, present in both dataset _and_
-                                           // in the model, it's just there might be _another_ row right in front of it.
-                                           t.getCause().getMessage().contains("expected row not to be visible") ||
-                                           t.getCause().getMessage().contains("Observed unvalidated rows") ||
-                                           t.getCause() != null && t.getCause().getMessage().contains("was never written"));
-                     });
-    }
-
-
-    @Test
-    public void testDetectsRemovedColumn()
-    {
-        negativeTest((run) -> {
-                         SimpleQueryResponseCorruptor corruptor = new SimpleQueryResponseCorruptor(run.schemaSpec,
-                                                                                                   run.clock,
-                                                                                                   HideValueCorruptor::new);
-
-                         return corruptor.maybeCorrupt(Query.selectPartition(run.schemaSpec,
-                                                                             run.pdSelector.pd(0, run.schemaSpec),
-                                                                             false),
-                                                       run.sut);
-                     },
-                     (t, run) -> Assert.assertTrue(String.format("Throwable: %s\nCause: %s", t, t.getCause()),
-                                                   t.getCause() != null && t.getCause().getMessage().contains("Modification should have been visible but was not")));
-    }
-
-
-    @Test
-    public void testDetectsOverwrittenRow()
-    {
-        negativeTest((run) -> {
-                         SimpleQueryResponseCorruptor corruptor = new SimpleQueryResponseCorruptor(run.schemaSpec,
-                                                                                                   run.clock,
-                                                                                                   ChangeValueCorruptor::new);
-
-                         return corruptor.maybeCorrupt(Query.selectPartition(run.schemaSpec,
-                                                                             run.pdSelector.pd(0, run.schemaSpec),
-                                                                             false),
-                                                       run.sut);
-                     },
-                     (t, run) -> Assert.assertTrue(String.format("Throwable: %s\nCause: %s", t, t.getCause()),
-                                                   t.getCause() != null && t.getCause().getMessage().contains("Modification should have been visible but was not.")));
-    }
-
-    @Test
-    public void testLocalOnlyExecution()
-    {
-        LocalOnlySut localOnlySut = new LocalOnlySut();
-
-        Supplier<Configuration.ConfigurationBuilder> gen = sharedConfiguration();
-
-        for (int cnt = 0; cnt < SchemaGenerators.DEFAULT_RUNS; cnt++)
-        {
-            Configuration config = gen.get()
-                                   .setClusteringDescriptorSelector((builder) -> {
-                                       builder.setOperationKindWeights(new Configuration.OperationKindSelectorBuilder()
-                                                                       .addWeight(OpSelectors.OperationKind.DELETE_ROW, 80)
-                                                                       .addWeight(OpSelectors.OperationKind.DELETE_COLUMN, 10)
-                                                                       .addWeight(OpSelectors.OperationKind.WRITE, 10)
-                                                                       .build());
-                                   })
-                                   .setSUT(() -> localOnlySut)
-                                   .build();
-
-            Run run = config.createRun();
-            run.sut.schemaChange(run.schemaSpec.compile().cql());
-
-            OpSelectors.MonotonicClock clock = run.clock;
-
-            PartitionVisitor partitionVisitor = new MutatingPartitionVisitor(run, MutatingRowVisitor::new);
-
-            localOnlySut.localOnly(() -> {
-                for (int i = 0; i < 5; i++)
-                {
-                    long lts = clock.nextLts();
-                    partitionVisitor.visitPartition(lts);
-                }
-            });
-
-            SinglePartitionValidator validator = new SinglePartitionValidator(100, run, ExhaustiveChecker::new);
-            validator.visitPartition(0);
-        }
-    }
-
-    Configuration.ModelConfiguration modelConfiguration()
-    {
-        return new Configuration.ExhaustiveCheckerConfig();
-    }
-
-    public static class LocalOnlySut implements SystemUnderTest
-    {
-        private boolean localOnly = false;
-        private int counter = 0;
-
-        public boolean isShutdown()
-        {
-            return cluster.size() == 0;
-        }
-
-        public void shutdown()
-        {
-            cluster.close();
-        }
-
-        public void schemaChange(String statement)
-        {
-            cluster.schemaChange(statement);
-        }
-
-        public Object[][] execute(String statement, ConsistencyLevel cl, Object... bindings)
-        {
-            if (localOnly)
-                return cluster.get((counter++) % cluster.size() + 1).executeInternal(statement, bindings);
-            else
-                return cluster.coordinator((counter++) % cluster.size() + 1).execute(statement, InJvmSut.toApiCl(ConsistencyLevel.ALL), bindings);
-        }
-
-        public CompletableFuture<Object[][]> executeAsync(String statement, ConsistencyLevel cl, Object... bindings)
-        {
-            return CompletableFuture.supplyAsync(() -> execute(statement, cl, bindings));
-        }
-
-        public void localOnly(Runnable r)
-        {
-            localOnly = true;
-            r.run();
-            localOnly = false;
-        }
-    }
-}
\ No newline at end of file
diff --git a/harry-integration/test/harry/model/ExhaustiveCheckerUnitTest.java b/harry-integration/test/harry/model/ExhaustiveCheckerUnitTest.java
deleted file mode 100644
index 7c1ca04..0000000
--- a/harry-integration/test/harry/model/ExhaustiveCheckerUnitTest.java
+++ /dev/null
@@ -1,171 +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;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-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.model.sut.SystemUnderTest;
-import harry.operations.CompiledStatement;
-import harry.runner.PartitionVisitor;
-import harry.runner.Query;
-import harry.runner.RowVisitor;
-
-public class ExhaustiveCheckerUnitTest
-{
-    @Test
-    public void testOperationConsistency()
-    {
-        Supplier<Configuration.ConfigurationBuilder> gen = IntegrationTestBase.sharedConfiguration();
-
-        for (int cnt = 0; cnt < SchemaGenerators.DEFAULT_RUNS; cnt++)
-        {
-            LoggingRowVisitor rowVisitor = new LoggingRowVisitor();
-            Configuration config = gen.get()
-                                      .setPartitionDescriptorSelector(new Configuration.DefaultPDSelectorConfiguration(10, 10))
-                                      .setClusteringDescriptorSelector((builder) -> {
-                                          builder.setOperationKindWeights(new Configuration.OperationKindSelectorBuilder()
-                                                                          .addWeight(OpSelectors.OperationKind.DELETE_ROW, 33)
-                                                                          .addWeight(OpSelectors.OperationKind.DELETE_COLUMN, 33)
-                                                                          .addWeight(OpSelectors.OperationKind.WRITE, 34)
-                                                                          .build());
-                                      })
-                                      .setSUT(() -> SystemUnderTest.NO_OP)
-                                      .setCreateSchema(true)
-                                      .build();
-
-            Run run = config.createRun();
-
-            ExhaustiveChecker checker = new ExhaustiveChecker(run);
-
-            PartitionVisitor visitor = new Configuration.MutatingPartitionVisitorConfiguation((r) -> rowVisitor).make(run);
-            int iterations = 10;
-
-            for (int i = 0; i < iterations; i++)
-                visitor.visitPartition(i);
-
-            for (List<ExhaustiveChecker.Operation> value : rowVisitor.executed.values())
-                Collections.sort(value);
-
-            for (int lts = 0; lts < iterations; lts++)
-            {
-                // TODO: turn query into interface to make it easier to deal with here
-                for (Collection<ExhaustiveChecker.Operation> modelOps : checker.inflatePartitionState(iterations - 1,
-                                                                                                      Query.selectPartition(run.schemaSpec,
-                                                                                                                            run.pdSelector.pd(lts),
-                                                                                                                            false))
-                                                                        .operations.values())
-                {
-                    ExhaustiveChecker.Operation op = modelOps.iterator().next();
-                    List<ExhaustiveChecker.Operation> executedOps = rowVisitor.executed.get(new Pair(op.pd, op.cd));
-                    Iterator<ExhaustiveChecker.Operation> modelIterator = modelOps.iterator();
-                    Iterator<ExhaustiveChecker.Operation> executedIterator = executedOps.iterator();
-                    while (modelIterator.hasNext() && executedIterator.hasNext())
-                    {
-                        Assert.assertEquals(String.format("\n%s\n%s", modelOps, executedOps),
-                                            executedIterator.next(), modelIterator.next());
-                    }
-                    Assert.assertEquals(String.format("\n%s\n%s", modelOps, executedOps),
-                                        modelIterator.hasNext(), executedIterator.hasNext());
-                }
-            }
-        }
-    }
-
-    private static class LoggingRowVisitor implements RowVisitor
-    {
-        private final Map<Pair, List<ExhaustiveChecker.Operation>> executed = new HashMap<>();
-
-        public CompiledStatement visitRow(OpSelectors.OperationKind op, long lts, long pd, long cd, long opId)
-        {
-            executed.compute(new Pair(pd, cd), (lts_, list) -> {
-                if (list == null)
-                    list = new ArrayList<>();
-                list.add(new ExhaustiveChecker.Operation(pd, cd, lts, opId, op));
-                return list;
-            });
-
-            return CompiledStatement.create("");
-        }
-
-        public CompiledStatement write(long lts, long pd, long cd, long opId)
-        {
-            return null;
-        }
-
-        public CompiledStatement deleteColumn(long lts, long pd, long cd, long opId)
-        {
-            return null;
-        }
-
-        public CompiledStatement deleteRow(long lts, long pd, long cd, long opId)
-        {
-            return null;
-        }
-
-        public CompiledStatement deleteRange(long lts, long pd, long opId)
-        {
-            return null;
-        }
-
-        public CompiledStatement deleteSlice(long lts, long pd, long opId)
-        {
-            return null;
-        }
-    }
-
-    private static class Pair
-    {
-        final long pd;
-        final long cd;
-
-        Pair(long pd, long cd)
-        {
-            this.pd = pd;
-            this.cd = cd;
-        }
-
-        public boolean equals(Object o)
-        {
-            if (this == o) return true;
-            if (o == null || getClass() != o.getClass()) return false;
-            Pair pair = (Pair) o;
-            return pd == pair.pd &&
-                   cd == pair.cd;
-        }
-
-        public int hashCode()
-        {
-            return Objects.hash(pd, cd);
-        }
-    }
-}
\ No newline at end of file
diff --git a/harry-integration/test/harry/model/IntegrationTestBase.java b/harry-integration/test/harry/model/IntegrationTestBase.java
index a3c2dfa..551124f 100644
--- a/harry-integration/test/harry/model/IntegrationTestBase.java
+++ b/harry-integration/test/harry/model/IntegrationTestBase.java
@@ -40,7 +40,7 @@ public class IntegrationTestBase extends TestBaseImpl
     public static void before() throws Throwable
     {
         cluster = init(Cluster.build()
-                              .withNodes(3)
+                              .withNodes(1)
                               .start());
         sut = new InJvmSut(cluster, 1);
     }
@@ -55,7 +55,7 @@ public class IntegrationTestBase extends TestBaseImpl
     public void beforeEach()
     {
         cluster.schemaChange("DROP KEYSPACE IF EXISTS harry");
-        cluster.schemaChange("CREATE KEYSPACE harry WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3};");
+        cluster.schemaChange("CREATE KEYSPACE harry WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1};");
     }
 
     private static long seed = 0;
@@ -68,6 +68,24 @@ public class IntegrationTestBase extends TestBaseImpl
         };
     }
 
+    public static Configuration.CDSelectorConfigurationBuilder sharedCDSelectorConfiguration()
+    {
+        return new Configuration.CDSelectorConfigurationBuilder()
+               .setNumberOfModificationsDistribution(new Configuration.ConstantDistributionConfig(2))
+               .setRowsPerModificationDistribution(new Configuration.ConstantDistributionConfig(2))
+               .setMaxPartitionSize(100)
+               .setOperationKindWeights(new Configuration.OperationKindSelectorBuilder()
+                                        .addWeight(OpSelectors.OperationKind.DELETE_ROW, 1)
+                                        .addWeight(OpSelectors.OperationKind.DELETE_COLUMN, 1)
+                                        .addWeight(OpSelectors.OperationKind.DELETE_RANGE, 1)
+                                        .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)
+                                        .build());
+    }
+
     public static Configuration.ConfigurationBuilder sharedConfiguration(long seed, SchemaSpec schema)
     {
         return new Configuration.ConfigurationBuilder().setSeed(seed)
@@ -76,17 +94,7 @@ public class IntegrationTestBase extends TestBaseImpl
                                                        .setTruncateTable(false)
                                                        .setDropSchema(true)
                                                        .setSchemaProvider(seed1 -> schema)
-                                                       .setClusteringDescriptorSelector((builder) -> {
-                                                           builder
-                                                           .setNumberOfModificationsDistribution(new Configuration.ConstantDistributionConfig(1))
-                                                           .setRowsPerModificationDistribution(new Configuration.ConstantDistributionConfig(1))
-                                                           .setOperationKindWeights(new Configuration.OperationKindSelectorBuilder()
-                                                                                    .addWeight(OpSelectors.OperationKind.DELETE_ROW, 10)
-                                                                                    .addWeight(OpSelectors.OperationKind.DELETE_COLUMN, 10)
-                                                                                    .addWeight(OpSelectors.OperationKind.WRITE, 80)
-                                                                                    .build())
-                                                           .setMaxPartitionSize(100);
-                                                       })
+                                                       .setClusteringDescriptorSelector(sharedCDSelectorConfiguration().build())
                                                        .setPartitionDescriptorSelector(new Configuration.DefaultPDSelectorConfiguration(1, 200))
                                                        .setSUT(() -> sut);
     }
diff --git a/harry-integration/test/harry/model/MockSchema.java b/harry-integration/test/harry/model/MockSchema.java
index b7a11ef..d97c12b 100644
--- a/harry-integration/test/harry/model/MockSchema.java
+++ b/harry-integration/test/harry/model/MockSchema.java
@@ -64,14 +64,18 @@ public class MockSchema
                                             ColumnSpec.regularColumn("v4", ColumnSpec.asciiType),
                                             ColumnSpec.regularColumn("v5", ColumnSpec.int64Type),
                                             ColumnSpec.regularColumn("v6", ColumnSpec.floatType),
-                                            ColumnSpec.regularColumn("v7", ColumnSpec.doubleType)));
+                                            ColumnSpec.regularColumn("v7", ColumnSpec.doubleType)),
+                              Arrays.asList(ColumnSpec.staticColumn("static1", ColumnSpec.asciiType),
+                                            ColumnSpec.staticColumn("static2", ColumnSpec.int64Type)));
 
         tbl2 = new SchemaSpec(KEYSPACE, "tbl2",
                               Arrays.asList(ColumnSpec.pk("pk1", ColumnSpec.asciiType),
                                             ColumnSpec.pk("pk2", ColumnSpec.int64Type)),
                               Arrays.asList(ColumnSpec.ck("ck1", ColumnSpec.asciiType, true)),
                               Arrays.asList(ColumnSpec.regularColumn("v1", ColumnSpec.int32Type),
-                                            ColumnSpec.regularColumn("v2", ColumnSpec.asciiType)));
+                                            ColumnSpec.regularColumn("v2", ColumnSpec.asciiType)),
+                              Arrays.asList(ColumnSpec.regularColumn("static1", ColumnSpec.int32Type),
+                                            ColumnSpec.regularColumn("static2", ColumnSpec.asciiType)));
 
         compact_schema = new SchemaSpec(KEYSPACE, "tbl3",
                                         Arrays.asList(ColumnSpec.pk("pk1", ColumnSpec.asciiType),
@@ -79,6 +83,7 @@ public class MockSchema
                                         Arrays.asList(ColumnSpec.ck("ck1", ColumnSpec.asciiType, true),
                                                       ColumnSpec.ck("ck2", ColumnSpec.int64Type, false)),
                                         Arrays.asList(ColumnSpec.regularColumn("v1", ColumnSpec.asciiType)),
+                                        Arrays.asList(ColumnSpec.staticColumn("static1", ColumnSpec.asciiType)),
                                         true);
     }
 
diff --git a/harry-integration/test/harry/model/ModelTest.java b/harry-integration/test/harry/model/ModelTest.java
deleted file mode 100644
index 7a7cfe9..0000000
--- a/harry-integration/test/harry/model/ModelTest.java
+++ /dev/null
@@ -1,159 +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;
-
-import java.util.Arrays;
-import java.util.Random;
-import java.util.concurrent.TimeUnit;
-import java.util.function.Consumer;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import harry.core.Configuration;
-import harry.ddl.SchemaSpec;
-import harry.generators.Surjections;
-import harry.generators.distribution.Distribution;
-import harry.model.sut.InJvmSut;
-import harry.runner.LoggingPartitionVisitor;
-import harry.runner.Runner;
-import harry.util.BitSet;
-import org.apache.cassandra.distributed.Cluster;
-
-// TODO: split tests into concurrent and sequential
-public class ModelTest extends TestBaseImpl
-{
-    static
-    {
-        KEYSPACE = MockSchema.KEYSPACE;
-    }
-
-    private final static Surjections.Surjection<BitSet> columnMaskSelector = MockSchema.columnMaskSelector1;
-    private final static SchemaSpec schema = MockSchema.tbl1;
-    private final static int cycles = 1000;
-
-    @Test
-    public void testScale()
-    {
-        Random rand = new Random();
-        for (int cycle = 0; cycle < cycles; cycle++)
-        {
-            int a = rand.nextInt(100);
-            int b = rand.nextInt(100);
-            while (a == b)
-                b = rand.nextInt(100);
-
-            int min = Math.min(a, b);
-            int max = Math.max(a, b);
-            long[] cardinality = new long[max - min];
-            for (int i = 0; i < 100000; i++)
-            {
-                long rnd = rand.nextLong();
-                long scaled = Distribution.ScaledDistribution.scale(rnd, min, max);
-                cardinality[(int) scaled - min]++;
-            }
-
-            for (long c : cardinality)
-                Assert.assertTrue(c > 0);
-        }
-    }
-
-    @Test
-    public void statelessVisibleRowsCheckerTest() throws Throwable
-    {
-        visibleRowsCheckerTest(StatelessVisibleRowsChecker::new);
-    }
-
-    @Test
-    public void statefulVisibleRowsCheckerTest() throws Throwable
-    {
-        visibleRowsCheckerTest(VisibleRowsChecker::new,
-                               (cfg) -> {
-                                   cfg.setDataTracker(VisibleRowsChecker.LoggingDataTracker::new);
-                               });
-    }
-
-    @Test
-    public void exhaustiveCheckerTest() throws Throwable
-    {
-        visibleRowsCheckerTest(ExhaustiveChecker::new);
-    }
-
-    public void visibleRowsCheckerTest(Model.ModelFactory factory) throws Throwable
-    {
-        visibleRowsCheckerTest(factory, (a) -> {});
-    }
-    public void visibleRowsCheckerTest(Model.ModelFactory factory, Consumer<Configuration.ConfigurationBuilder> configurator) throws Throwable
-    {
-        try (Cluster cluster = Cluster.create(3))
-        {
-            Configuration.ConfigurationBuilder builder = new Configuration.ConfigurationBuilder();
-            builder.setClock(new Configuration.ApproximateMonotonicClockConfiguration((int) TimeUnit.MINUTES.toMillis(2),
-                                                                                            1, TimeUnit.SECONDS))
-                         .setRunTime(1, TimeUnit.MINUTES)
-                         .setRunner(new Configuration.ConcurrentRunnerConfig(2,
-                                                                             Arrays.asList(new Configuration.LoggingPartitionVisitorConfiguration(new Configuration.MutatingRowVisitorConfiguration()),
-                                                                                           new Configuration.RecentPartitionsValidatorConfiguration(10, 10, factory::make),
-                                                                                           new Configuration.AllPartitionsValidatorConfiguration(10, 10, factory::make))))
-                         .setSchemaProvider((seed) -> schema)
-                         .setClusteringDescriptorSelector((OpSelectors.Rng rng, SchemaSpec schemaSpec) -> {
-                             return new DescriptorSelectorBuilder()
-                             .setColumnMaskSelector(columnMaskSelector)
-                             .setOperationTypeSelector(Surjections.constant(OpSelectors.OperationKind.WRITE))
-                             .make(rng, schemaSpec);
-                         })
-                         .setPartitionDescriptorSelector(new Configuration.DefaultPDSelectorConfiguration(10, 10))
-                         .setCreateSchema(true)
-                         .setTruncateTable(false)
-                         .setDropSchema(false)
-                         .setSUT(() -> new InJvmSut(cluster));
-            configurator.accept(builder);
-            Runner runner = builder.build().createRunner();
-            try
-            {
-                runner.initAndStartAll().get(2, TimeUnit.MINUTES);
-            }
-            catch (Throwable t)
-            {
-                throw t;
-            }
-            finally
-            {
-                runner.shutdown();
-            }
-
-        }
-    }
-
-    @Test
-    public void descendingIteratorTest()
-    {
-        VisibleRowsChecker.LongIterator iter = VisibleRowsChecker.descendingIterator(new long[] { 4,2,1,3,2,1,3,4});;
-        Assert.assertTrue(iter.hasNext());
-        Assert.assertEquals(4, iter.nextLong());
-        Assert.assertTrue(iter.hasNext());
-        Assert.assertEquals(3, iter.nextLong());
-        Assert.assertTrue(iter.hasNext());
-        Assert.assertEquals(2, iter.nextLong());
-        Assert.assertTrue(iter.hasNext());
-        Assert.assertEquals(1, iter.nextLong());
-    }
-}
-
-// TODO: test things gradually. First with simple schemas, then with more complex, then with completely random.
diff --git a/harry-integration/test/harry/model/ModelTestBase.java b/harry-integration/test/harry/model/ModelTestBase.java
index 6421355..5d1659d 100644
--- a/harry-integration/test/harry/model/ModelTestBase.java
+++ b/harry-integration/test/harry/model/ModelTestBase.java
@@ -18,8 +18,9 @@
 
 package harry.model;
 
+import java.util.Arrays;
+import java.util.concurrent.TimeUnit;
 import java.util.function.BiConsumer;
-import java.util.function.Consumer;
 import java.util.function.Function;
 import java.util.function.Supplier;
 
@@ -30,6 +31,7 @@ import harry.ddl.SchemaSpec;
 import harry.runner.LoggingPartitionVisitor;
 import harry.runner.MutatingRowVisitor;
 import harry.runner.PartitionVisitor;
+import harry.runner.Runner;
 import harry.runner.SinglePartitionValidator;
 
 public abstract class ModelTestBase extends IntegrationTestBase
@@ -44,39 +46,67 @@ public abstract class ModelTestBase extends IntegrationTestBase
         }
     }
 
+    void negativeIntegrationTest(Model.ModelFactory factory) throws Throwable
+    {
+        Supplier<SchemaSpec> supplier = SchemaGenerators.progression(1);
+        for (int i = 0; i < SchemaGenerators.DEFAULT_RUNS; i++)
+        {
+            SchemaSpec schema = supplier.get();
+            Configuration.ConfigurationBuilder builder = configuration(i, schema);
+            builder.setClock(new Configuration.ApproximateMonotonicClockConfiguration((int) TimeUnit.MINUTES.toMillis(10),
+                                                                                      1, TimeUnit.SECONDS))
+                   .setRunTime(1, TimeUnit.MINUTES)
+                   .setCreateSchema(false)
+                   .setDropSchema(false)
+                   .setRunner(new Configuration.SequentialRunnerConfig(Arrays.asList(new Configuration.LoggingPartitionVisitorConfiguration(new Configuration.MutatingRowVisitorConfiguration()),
+                                                                                     new Configuration.RecentPartitionsValidatorConfiguration(10, 10, 1, factory::make),
+                                                                                     new Configuration.AllPartitionsValidatorConfiguration(10, 10, factory::make))));
+            Runner runner = builder.build().createRunner();
+            try
+            {
+                Run run = runner.getRun();
+                beforeEach();
+                run.sut.schemaChange(run.schemaSpec.compile().cql());
+
+                runner.initAndStartAll().get(2, TimeUnit.MINUTES);
+            }
+            catch (Throwable t)
+            {
+                throw t;
+            }
+            finally
+            {
+                runner.shutdown();
+            }
+        }
+    }
+
     abstract Configuration.ModelConfiguration modelConfiguration();
 
     protected PartitionVisitor validator(Run run)
     {
-        return new SinglePartitionValidator(100, run, modelConfiguration());
+        return new SinglePartitionValidator(100, run , modelConfiguration());
+    }
+
+    public Configuration.ConfigurationBuilder configuration(long seed, SchemaSpec schema)
+    {
+        return sharedConfiguration(seed, schema);
     }
 
     void negativeTest(Function<Run, Boolean> corrupt, BiConsumer<Throwable, Run> validate, int counter, SchemaSpec schemaSpec)
     {
-        Configuration config = sharedConfiguration(counter, schemaSpec)
-                               .setClusteringDescriptorSelector((builder) -> {
-                                   builder.setNumberOfModificationsDistribution(new Configuration.ConstantDistributionConfig(10))
-                                          .setRowsPerModificationDistribution(new Configuration.ConstantDistributionConfig(10))
-                                          .setMaxPartitionSize(100)
-                                          .setOperationKindWeights(new Configuration.OperationKindSelectorBuilder()
-                                                                   .addWeight(OpSelectors.OperationKind.DELETE_ROW, 1)
-                                                                   .addWeight(OpSelectors.OperationKind.DELETE_COLUMN, 1)
-                                                                   .addWeight(OpSelectors.OperationKind.DELETE_RANGE, 1)
-                                                                   .addWeight(OpSelectors.OperationKind.DELETE_SLICE, 1)
-                                                                   .addWeight(OpSelectors.OperationKind.WRITE, 96)
-                                                                   .build());
-                               })
-                               .build();
+        Configuration config = configuration(counter, schemaSpec).build();
 
         Run run = config.createRun();
         beforeEach();
         run.sut.schemaChange(run.schemaSpec.compile().cql());
+        System.out.println(run.schemaSpec.compile().cql());
         OpSelectors.MonotonicClock clock = run.clock;
 
         PartitionVisitor validator = validator(run);
         PartitionVisitor partitionVisitor = new LoggingPartitionVisitor(run, MutatingRowVisitor::new);
 
-        for (int i = 0; i < 200; i++)
+        for (int i = 0; i < 20000; i++)
         {
             long lts = clock.nextLts();
             partitionVisitor.visitPartition(lts);
@@ -99,4 +129,5 @@ public abstract class ModelTestBase extends IntegrationTestBase
             validate.accept(t, run);
         }
     }
-}
\ No newline at end of file
+}
+
diff --git a/harry-integration/test/harry/model/QuerySelectorNegativeTest.java b/harry-integration/test/harry/model/QuerySelectorNegativeTest.java
index 8085a06..8c635c8 100644
--- a/harry-integration/test/harry/model/QuerySelectorNegativeTest.java
+++ b/harry-integration/test/harry/model/QuerySelectorNegativeTest.java
@@ -50,7 +50,7 @@ import static harry.corruptor.QueryResponseCorruptor.SimpleQueryResponseCorrupto
 @RunWith(Parameterized.class)
 public class QuerySelectorNegativeTest extends IntegrationTestBase
 {
-    private final int ltss = 1000;
+    private final int CYCLES = 1000;
 
     private final Random rnd = new Random();
 
@@ -78,7 +78,8 @@ public class QuerySelectorNegativeTest extends IntegrationTestBase
                                                                        HideRowCorruptor::new),
                              (run) -> new AddExtraRowCorruptor(run.schemaSpec,
                                                                run.clock,
-                                                               run.descriptorSelector));
+                                                               run.descriptorSelector)
+        );
     }
 
     interface QueryResponseCorruptorFactory
@@ -99,21 +100,23 @@ public class QuerySelectorNegativeTest extends IntegrationTestBase
         {
             beforeEach();
             Configuration config = gen.get()
-                                      .setClusteringDescriptorSelector((builder) -> {
-                                          builder.setMaxPartitionSize(2000);
-                                      })
+                                      .setClusteringDescriptorSelector(sharedCDSelectorConfiguration()
+                                                                       .setNumberOfModificationsDistribution(new Configuration.ConstantDistributionConfig(2))
+                                                                       .setRowsPerModificationDistribution(new Configuration.ConstantDistributionConfig(2))
+                                                                       .setMaxPartitionSize(2000)
+                                                                       .build())
                                       .build();
             Run run = config.createRun();
             run.sut.schemaChange(run.schemaSpec.compile().cql());
-
+            System.out.println(run.schemaSpec.compile().cql());
             OpSelectors.MonotonicClock clock = run.clock;
 
             PartitionVisitor partitionVisitor = new MutatingPartitionVisitor(run, MutatingRowVisitor::new);
-            Model model = new ExhaustiveChecker(run);
+            Model model = new QuiescentChecker(run);
 
             QueryResponseCorruptor corruptor = this.corruptorFactory.create(run);
 
-            for (int i = 0; i < ltss; i++)
+            for (int i = 0; i < CYCLES; i++)
             {
                 long lts = clock.nextLts();
                 partitionVisitor.visitPartition(lts);
diff --git a/harry-integration/test/harry/model/QuerySelectorTest.java b/harry-integration/test/harry/model/QuerySelectorTest.java
index 3e7f3f0..484d14a 100644
--- a/harry-integration/test/harry/model/QuerySelectorTest.java
+++ b/harry-integration/test/harry/model/QuerySelectorTest.java
@@ -37,6 +37,8 @@ import harry.runner.PartitionVisitor;
 import harry.runner.Query;
 import harry.runner.QueryGenerator;
 
+import static harry.generators.DataGenerators.NIL_DESCR;
+
 public class QuerySelectorTest extends IntegrationTestBase
 {
     private static int CYCLES = 300;
@@ -60,11 +62,12 @@ public class QuerySelectorTest extends IntegrationTestBase
             }
 
             Configuration config = sharedConfiguration(cnt, schemaSpec)
-                                   .setClusteringDescriptorSelector((builder) -> {
-                                       builder.setMaxPartitionSize(partitionSize)
-                                              .setFractions(fractions);
-                                   })
+                                   .setClusteringDescriptorSelector(sharedCDSelectorConfiguration()
+                                                                    .setMaxPartitionSize(partitionSize)
+                                                                    .setFractions(fractions)
+                                                                    .build())
                                    .build();
+
             Run run = config.createRun();
             run.sut.schemaChange(run.schemaSpec.compile().cql());
             OpSelectors.MonotonicClock clock = run.clock;
@@ -103,8 +106,16 @@ public class QuerySelectorTest extends IntegrationTestBase
                                                           run.clock,
                                                           row).cd;
 
-                    Assert.assertEquals(matchingClusterings.contains(cd),
-                                        query.match(cd));
+                    // Skip static clustering
+                    if (cd == NIL_DESCR)
+                        continue;
+
+                    boolean expected = matchingClusterings.contains(cd);
+                    boolean actual = query.match(cd);
+                    Assert.assertEquals(String.format("Mismatch for clustering: %d. Expected: %s. Actual: %s.\nQuery: %s",
+                                                      cd, expected, actual, query.toSelectStatement()),
+                                        expected,
+                                        actual);
                 }
             }
         }
@@ -127,10 +138,10 @@ public class QuerySelectorTest extends IntegrationTestBase
             }
 
             Configuration config = sharedConfiguration(cnt, schemaSpec)
-                                   .setClusteringDescriptorSelector((builder) -> {
-                                       builder.setMaxPartitionSize(partitionSize)
-                                              .setFractions(fractions);
-                                   })
+                                   .setClusteringDescriptorSelector(sharedCDSelectorConfiguration()
+                                                                    .setMaxPartitionSize(partitionSize)
+                                                                    .setFractions(fractions)
+                                                                    .build())
                                    .build();
             Run run = config.createRun();
             run.sut.schemaChange(run.schemaSpec.compile().cql());
@@ -144,7 +155,7 @@ public class QuerySelectorTest extends IntegrationTestBase
             }
 
             QueryGenerator.TypedQueryGenerator querySelector = new QueryGenerator.TypedQueryGenerator(run);
-            Model model = new ExhaustiveChecker(run);
+            Model model = new QuiescentChecker(run);
 
             long verificationLts = 10;
             for (int i = 0; i < CYCLES; i++)
diff --git a/harry-integration/test/harry/model/QuiescentCheckerIntegrationTest.java b/harry-integration/test/harry/model/QuiescentCheckerIntegrationTest.java
index 22d7171..4c83276 100644
--- a/harry-integration/test/harry/model/QuiescentCheckerIntegrationTest.java
+++ b/harry-integration/test/harry/model/QuiescentCheckerIntegrationTest.java
@@ -18,7 +18,6 @@
 
 package harry.model;
 
-import org.junit.Assert;
 import org.junit.Test;
 
 import harry.core.Configuration;
@@ -29,6 +28,7 @@ import harry.corruptor.HideRowCorruptor;
 import harry.corruptor.HideValueCorruptor;
 import harry.corruptor.QueryResponseCorruptor;
 import harry.corruptor.QueryResponseCorruptor.SimpleQueryResponseCorruptor;
+import harry.ddl.SchemaSpec;
 import harry.runner.PartitionVisitor;
 import harry.runner.Query;
 import harry.runner.SinglePartitionValidator;
@@ -44,7 +44,7 @@ public class QuiescentCheckerIntegrationTest extends ModelTestBase
     @Test
     public void testNormalCondition()
     {
-        negativeTest((run) -> { return  true; },
+        negativeTest((run) -> true,
                      (t, run) -> {
                          if (t != null)
                              throw new AssertionError(String.format("Throwable was supposed to be null. Schema: %s",
@@ -54,6 +54,12 @@ public class QuiescentCheckerIntegrationTest extends ModelTestBase
     }
 
     @Test
+    public void normalConditionIntegrationTest() throws Throwable
+    {
+        negativeIntegrationTest(modelConfiguration()::make);
+    }
+
+    @Test
     public void testDetectsMissingRow()
     {
         negativeTest((run) -> {
@@ -71,8 +77,12 @@ public class QuiescentCheckerIntegrationTest extends ModelTestBase
                          // TODO: We can actually pinpoint the difference
                          String expected = "Expected results to have the same number of results, but expected result iterator has more results";
                          String expected2 = "Found a row in the model that is not present in the resultset";
-                         Assert.assertTrue(String.format("Exception string mismatch.\nExpected error: %s.\nActual error: %s", expected, t.getMessage()),
-                                           t.getMessage().contains(expected) || t.getMessage().contains(expected2));
+
+                         if (t.getMessage().contains(expected) || t.getMessage().contains(expected2))
+                             return;
+
+                         throw new AssertionError(String.format("Exception string mismatch.\nExpected error: %s.\nActual error: %s", expected, t.getMessage()),
+                                                  t);
                      });
     }
 
@@ -93,8 +103,11 @@ public class QuiescentCheckerIntegrationTest extends ModelTestBase
                          String expected = "Found a row in the model that is not present in the resultset";
                          String expected2 = "Expected results to have the same number of results, but actual result iterator has more results";
 
-                         Assert.assertTrue(String.format("Exception string mismatch.\nExpected error: %s.\nActual error: %s", expected, t.getMessage()),
-                                           t.getMessage().contains(expected) || t.getMessage().contains(expected2));
+                         if (t.getMessage().contains(expected) || t.getMessage().contains(expected2))
+                             return;
+
+                         throw new AssertionError(String.format("Exception string mismatch.\nExpected error: %s.\nActual error: %s", expected, t.getMessage()),
+                                                  t);
                      });
     }
 
@@ -113,9 +126,13 @@ public class QuiescentCheckerIntegrationTest extends ModelTestBase
                                                        run.sut);
                      },
                      (t, run) -> {
-                         String expected = "Returned row state doesn't match the one predicted by the model";
-                         Assert.assertTrue(String.format("Exception string mismatch.\nExpected error: %s.\nActual error: %s", expected, t.getMessage()),
-                                           t.getMessage().contains(expected));
+                         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))
+                             return;
+
+                         throw new AssertionError(String.format("Exception string mismatch.\nExpected error: %s.\nActual error: %s", expected, t.getMessage()),
+                                                  t);
                      });
     }
 
@@ -135,13 +152,29 @@ public class QuiescentCheckerIntegrationTest extends ModelTestBase
                      },
                      (t, run) -> {
                          String expected = "Returned row state doesn't match the one predicted by the model";
-                         Assert.assertTrue(String.format("Exception string mismatch.\nExpected error: %s.\nActual error: %s", expected, t.getMessage()),
-                                           t.getMessage().contains(expected));
+                         String expected2 = "Timestamps in the row state don't match ones predicted by the model";
+
+                         if (t.getMessage().contains(expected) || t.getMessage().contains(expected2))
+                             return;
+
+                         throw new AssertionError(String.format("Exception string mismatch.\nExpected error: %s.\nActual error: %s", expected, t.getMessage()),
+                                                  t);
                      });
     }
 
+    @Override
     Configuration.ModelConfiguration modelConfiguration()
     {
         return new Configuration.QuiescentCheckerConfig();
     }
+
+    public Configuration.ConfigurationBuilder configuration(long seed, SchemaSpec schema)
+    {
+        return super.configuration(seed, schema)
+                    .setClusteringDescriptorSelector(sharedCDSelectorConfiguration()
+                                                     .setNumberOfModificationsDistribution(new Configuration.ConstantDistributionConfig(2))
+                                                     .setRowsPerModificationDistribution(new Configuration.ConstantDistributionConfig(2))
+                                                     .setMaxPartitionSize(100)
+                                                     .build());
+    }
 }
\ No newline at end of file
diff --git a/harry-integration/test/harry/op/RowVisitorTest.java b/harry-integration/test/harry/op/RowVisitorTest.java
index c8ad182..d7061f9 100644
--- a/harry-integration/test/harry/op/RowVisitorTest.java
+++ b/harry-integration/test/harry/op/RowVisitorTest.java
@@ -30,18 +30,15 @@ import harry.ddl.SchemaGenerators;
 import harry.ddl.SchemaSpec;
 import harry.generators.RandomGenerator;
 import harry.generators.distribution.Distribution;
+import harry.model.OpSelectors;
+import harry.model.clock.OffsetClock;
 import harry.model.sut.SystemUnderTest;
+import harry.operations.CompiledStatement;
 import harry.runner.DataTracker;
 import harry.runner.MutatingRowVisitor;
-import harry.model.clock.OffsetClock;
-import harry.model.OpSelectors;
-import harry.operations.CompiledStatement;
-import harry.runner.QueryGenerator;
-import harry.util.BitSet;
 import org.apache.cassandra.cql3.CQLTester;
 
-import static harry.util.TestRunner.test;
-import static harry.model.OpSelectors.DefaultDescriptorSelector.DEFAULT_OP_TYPE_SELECTOR;
+import static harry.model.OpSelectors.DefaultDescriptorSelector.DEFAULT_OP_SELECTOR;
 
 public class RowVisitorTest extends CQLTester
 {
@@ -60,21 +57,20 @@ public class RowVisitorTest extends CQLTester
         OpSelectors.Rng rng = new OpSelectors.PCGFast(1);
 
         OpSelectors.PdSelector pdSelector = new OpSelectors.DefaultPdSelector(rng, 10, 10);
-        OpSelectors.DescriptorSelector descriptorSelector = new OpSelectors.DefaultDescriptorSelector(rng,
-                                                                                                      OpSelectors.columnSelectorBuilder().forAll(BitSet.create(0b001, 3),
-                                                                                                                                                 BitSet.create(0b011, 3),
-                                                                                                                                                 BitSet.create(0b111, 3))
-                                                                                                                 .build(),
-                                                                                                      DEFAULT_OP_TYPE_SELECTOR,
-                                                                                                      new Distribution.ScaledDistribution(1, 3),
-                                                                                                      new Distribution.ScaledDistribution(2, 30),
-                                                                                                      100);
 
         for (int i = 0; i < SchemaGenerators.DEFAULT_RUNS; i++)
         {
             SchemaSpec schema = specGenerator.get();
             createTable(schema.compile().cql());
 
+            OpSelectors.DescriptorSelector descriptorSelector = new OpSelectors.DefaultDescriptorSelector(rng,
+                                                                                                          new OpSelectors.ColumnSelectorBuilder().forAll(schema)
+                                                                                                                                                 .build(),
+                                                                                                          DEFAULT_OP_SELECTOR,
+                                                                                                          new Distribution.ScaledDistribution(1, 3),
+                                                                                                          new Distribution.ScaledDistribution(2, 30),
+                                                                                                          100);
+
             Run run = new Run(rng,
                               new OffsetClock(10000),
                               pdSelector,
diff --git a/pom.xml b/pom.xml
index 8a172eb..7e23efd 100755
--- a/pom.xml
+++ b/pom.xml
@@ -20,12 +20,26 @@
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
          xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
 
+    <parent>
+        <groupId>org.apache</groupId>
+        <artifactId>apache</artifactId>
+        <version>23</version>
+    </parent>
+
+    <licenses>
+        <license>
+            <name>Apache License, Version 2.0</name>
+            <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+            <distribution>repo</distribution>
+        </license>
+    </licenses>
+
     <modelVersion>4.0.0</modelVersion>
     <packaging>pom</packaging>
 
     <groupId>org.apache.cassandra</groupId>
-    <version>0.0.1-SNAPSHOT</version>
     <artifactId>harry-parent</artifactId>
+    <version>0.0.1-SNAPSHOT</version>
 
     <name>Harry</name>
 
@@ -249,4 +263,11 @@
             </plugin>
         </plugins>
     </build>
+
+    <scm>
+        <connection>scm:git:https://gitbox.apache.org/repos/asf/cassandra-harry.git</connection>
+        <developerConnection>scm:git:https://gitbox.apache.org/repos/asf/cassandra-harry.git</developerConnection>
+        <url>https://gitbox.apache.org/repos/asf/cassandra-harry.git</url>
+        <tag>0.0.1</tag>
+    </scm>
 </project>

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