Skip to content
Open
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,9 @@
import com.google.common.util.concurrent.AsyncFunction;
import com.google.common.util.concurrent.FutureCallback;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import org.apache.cassandra.concurrent.ExecutorFactory;
import org.apache.cassandra.config.ParameterizedClass;
import org.apache.cassandra.distributed.Cluster;
Expand Down Expand Up @@ -112,6 +115,8 @@
@SuppressWarnings("RedundantCast")
public class ClusterSimulation<S extends Simulation> implements AutoCloseable
{
private static final Logger logger = LoggerFactory.getLogger(ClusterSimulation.class);

public static final Class<?>[] SHARE = new Class[]
{
AsyncFunction.class,
Expand Down Expand Up @@ -188,6 +193,7 @@ public static abstract class Builder<S extends Simulation>
protected HeapPool.Logged.Listener memoryListener;
protected SimulatedTime.Listener timeListener = (i1, i2) -> {};
protected LongConsumer onThreadLocalRandomCheck;
protected Boolean useTrieMemtable = null; // null = random

public Debug debug()
{
Expand Down Expand Up @@ -516,6 +522,12 @@ public Builder<S> onThreadLocalRandomCheck(LongConsumer runnable)
return this;
}

public Builder<S> useTrieMemtable(boolean useTrie)
{
this.useTrieMemtable = useTrie;
return this;
}

public abstract ClusterSimulation<S> create(long seed) throws IOException;
}

Expand Down Expand Up @@ -654,6 +666,12 @@ public ClusterSimulation(RandomSource random, long seed, int uniqueNum,

execution = new SimulatedExecution();

boolean useTrieMemtable = builder.useTrieMemtable != null
? builder.useTrieMemtable
: random.uniform(0, 2) == 0; // 50/50 chance
logger.info("Seed 0x{} using memtable: {}", Long.toHexString(seed),
useTrieMemtable ? "TrieMemtable" : "SkipListMemtable");

KindOfSequence kindOfDriftSequence = Choices.uniform(KindOfSequence.values()).choose(random);
KindOfSequence kindOfDiscontinuitySequence = Choices.uniform(KindOfSequence.values()).choose(random);
time = new SimulatedTime(numOfNodes, random, 1577836800000L /*Jan 1st UTC*/, builder.clockDriftNanos, kindOfDriftSequence,
Expand Down Expand Up @@ -700,6 +718,14 @@ public ClusterSimulation(RandomSource random, long seed, int uniqueNum,

if (commitlogCompressed)
config.set("commitlog_compression", new ParameterizedClass(LZ4Compressor.class.getName(), emptyMap()));

if (useTrieMemtable)
{
config.set("memtable", Map.of(
"configurations", Map.of(
"default", Map.of("class_name", "TrieMemtable"))));
}

configUpdater.accept(threadAllocator.update(config));
})
.withInstanceInitializer(new IInstanceInitializer()
Expand Down