maedhroz commented on code in PR #2660: URL: https://github.com/apache/cassandra/pull/2660#discussion_r1323320544
########## src/java/org/apache/cassandra/config/RetrySpec.java: ########## @@ -0,0 +1,182 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.config; + +import java.util.Objects; + +import javax.annotation.Nullable; + +import org.apache.cassandra.config.DurationSpec.LongNanosecondsBound; + +public class RetrySpec +{ + public enum Type { Expoential } + + public static class MaxAttempt + { + public static final MaxAttempt DISABLED = new MaxAttempt(); + + public int value; + + public MaxAttempt(int value) + { + if (value < 1) + throw new IllegalArgumentException("max attempt must be positive; but given " + value); + this.value = value; + } + + private MaxAttempt() + { + value = 0; + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null) return false; + if (o instanceof Integer) return this.value == ((Integer) o).intValue(); + if (getClass() != o.getClass()) return false; + MaxAttempt that = (MaxAttempt) o; + return value == that.value; + } + + @Override + public int hashCode() + { + return Objects.hash(value); + } + + @Override + public String toString() + { + return Integer.toString(value); + } + } + + public static class Partial extends RetrySpec + { + public Partial() + { + this.type = null; + this.maxAttempts = null; + this.baseSleepTime = null; + this.maxSleepTime = null; + } + + public RetrySpec withDefaults(RetrySpec defaultValues) + { + Type type = nonNull(this.type, defaultValues.getType(), DEFAULT_TYPE); + MaxAttempt maxAttempts = nonNull(this.maxAttempts, defaultValues.getMaxAttempts(), DEFAULT_MAX_ATTEMPTS); + LongNanosecondsBound baseSleepTime = nonNull(this.baseSleepTime, defaultValues.getBaseSleepTime(), DEFAULT_BASE_SLEEP); + LongNanosecondsBound maxSleepTime = nonNull(this.maxSleepTime, defaultValues.getMaxSleepTime(), DEFAULT_MAX_SLEEP); + return new RetrySpec(type, maxAttempts, baseSleepTime, maxSleepTime); + } + + private static <T> T nonNull(@Nullable T left, @Nullable T right, T defaultValue) + { + if (left != null) + return left; + if (right != null) + return right; + return defaultValue; + } + } + + private static final Type DEFAULT_TYPE = Type.Expoential; + private static final MaxAttempt DEFAULT_MAX_ATTEMPTS = MaxAttempt.DISABLED; + private static final LongNanosecondsBound DEFAULT_BASE_SLEEP = new LongNanosecondsBound("200ms"); + private static final LongNanosecondsBound DEFAULT_MAX_SLEEP = new LongNanosecondsBound("1s"); + + public Type type = DEFAULT_TYPE; + /** + * Represents how many retry attempts are allowed. If the value is 2, this will cause 2 retries + 1 original request, for a total of 3 requests! + * <p/> + * To disable, set to 0. + * <p/> + * To make it unbound, used -1 + */ + public MaxAttempt maxAttempts = DEFAULT_MAX_ATTEMPTS; // 2 retries, 1 original request; so 3 total + // TODO (api): These are really only useful if type=Expoential, but we don't have dynamic deserialization so can not really split into a type specific parser atm... + public LongNanosecondsBound baseSleepTime = DEFAULT_BASE_SLEEP; + public LongNanosecondsBound maxSleepTime = DEFAULT_MAX_SLEEP; + + public RetrySpec() + { + } + + public RetrySpec(Type type, MaxAttempt maxAttempts, LongNanosecondsBound baseSleepTime, LongNanosecondsBound maxSleepTime) + { + this.type = type; + this.maxAttempts = maxAttempts; + this.baseSleepTime = baseSleepTime; + this.maxSleepTime = maxSleepTime; + } + + public boolean isEnabled() + { + return maxAttempts != MaxAttempt.DISABLED; + } + + public void setEnabled(boolean enabled) + { + if (!enabled) + { + maxAttempts = MaxAttempt.DISABLED; + } + else if (maxAttempts == MaxAttempt.DISABLED) + { + maxAttempts = new MaxAttempt(2); Review Comment: I think I'm just wary of being able to set this enabled at all without explicitly setting `max_attempts`. If we're going to do it though, I guess 3 total attempts is reasonable. ########## src/java/org/apache/cassandra/utils/Backoff.java: ########## @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.utils; + +import java.util.concurrent.TimeUnit; +import java.util.function.DoubleSupplier; + +public interface Backoff +{ + /** + * @return max attempts allowed, {@code < 0} implies unbounded retries + */ + int maxAttempts(); + long computeWaitTime(int retryCount); + TimeUnit unit(); + + enum None implements Backoff + { + INSTANCE; + + @Override + public int maxAttempts() + { + return 0; + } + + @Override + public long computeWaitTime(int retryCount) + { + throw new UnsupportedOperationException(); + } + + @Override + public TimeUnit unit() + { + throw new UnsupportedOperationException(); + } + } + + class ExpoentialBackoff implements Backoff + { + private final int maxAttempts; + private final long baseSleepTimeMillis; + private final long maxSleepMillis; + private final DoubleSupplier randomSource; + + public ExpoentialBackoff(int maxAttempts, long baseSleepTimeMillis, long maxSleepMillis, DoubleSupplier randomSource) + { + this.maxAttempts = maxAttempts; + this.baseSleepTimeMillis = baseSleepTimeMillis; + this.maxSleepMillis = maxSleepMillis; + this.randomSource = randomSource; + } + + @Override + public int maxAttempts() + { + return maxAttempts; + } + + @Override + public long computeWaitTime(int retryCount) + { + long baseTimeMillis = baseSleepTimeMillis * (1L << retryCount); + // its possible that this overflows, so fall back to max; + if (baseTimeMillis <= 0) + baseTimeMillis = maxSleepMillis; + // now make sure this is capped to target max + baseTimeMillis = Math.min(baseTimeMillis, maxSleepMillis); + + return (long) (baseTimeMillis * (randomSource.getAsDouble() + 0.5)); Review Comment: I'm fine w/ it as is. ########## test/unit/org/apache/cassandra/repair/FuzzTestBase.java: ########## @@ -0,0 +1,1267 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.repair; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.sql.Timestamp; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Random; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiFunction; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.function.LongSupplier; +import java.util.function.Supplier; +import javax.annotation.Nullable; + +import com.google.common.collect.Iterables; +import com.google.common.collect.Maps; +import org.junit.Before; +import org.junit.BeforeClass; + +import accord.utils.DefaultRandom; +import accord.utils.Gen; +import accord.utils.Gens; +import accord.utils.RandomSource; +import org.agrona.collections.Long2ObjectHashMap; +import org.agrona.collections.LongHashSet; +import org.apache.cassandra.concurrent.ExecutorBuilder; +import org.apache.cassandra.concurrent.ExecutorBuilderFactory; +import org.apache.cassandra.concurrent.ExecutorFactory; +import org.apache.cassandra.concurrent.ExecutorPlus; +import org.apache.cassandra.concurrent.InfiniteLoopExecutor; +import org.apache.cassandra.concurrent.Interruptible; +import org.apache.cassandra.concurrent.ScheduledExecutorPlus; +import org.apache.cassandra.concurrent.SequentialExecutorPlus; +import org.apache.cassandra.concurrent.SimulatedExecutorFactory; +import org.apache.cassandra.concurrent.Stage; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.SharedContext; +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.Digest; +import org.apache.cassandra.db.marshal.EmptyType; +import org.apache.cassandra.db.repair.CassandraTableRepairManager; +import org.apache.cassandra.db.repair.PendingAntiCompaction; +import org.apache.cassandra.dht.Murmur3Partitioner; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.exceptions.RequestFailureReason; +import org.apache.cassandra.gms.ApplicationState; +import org.apache.cassandra.gms.EndpointState; +import org.apache.cassandra.gms.HeartBeatState; +import org.apache.cassandra.gms.IEndpointStateChangeSubscriber; +import org.apache.cassandra.gms.IFailureDetector; +import org.apache.cassandra.gms.IGossiper; +import org.apache.cassandra.gms.VersionedValue; +import org.apache.cassandra.io.util.DataInputBuffer; +import org.apache.cassandra.io.util.DataOutputBuffer; +import org.apache.cassandra.locator.IEndpointSnitch; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.locator.LocalStrategy; +import org.apache.cassandra.locator.RangesAtEndpoint; +import org.apache.cassandra.locator.TokenMetadata; +import org.apache.cassandra.net.ConnectionType; +import org.apache.cassandra.net.IVerbHandler; +import org.apache.cassandra.net.Message; +import org.apache.cassandra.net.MessageDelivery; +import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.net.RequestCallback; +import org.apache.cassandra.repair.messages.RepairOption; +import org.apache.cassandra.repair.messages.ValidationResponse; +import org.apache.cassandra.repair.state.AbstractCompletable; +import org.apache.cassandra.repair.state.AbstractState; +import org.apache.cassandra.repair.state.Completable; +import org.apache.cassandra.repair.state.CoordinatorState; +import org.apache.cassandra.repair.state.JobState; +import org.apache.cassandra.repair.state.SessionState; +import org.apache.cassandra.repair.state.ValidationState; +import org.apache.cassandra.schema.KeyspaceMetadata; +import org.apache.cassandra.schema.KeyspaceParams; +import org.apache.cassandra.schema.SchemaConstants; +import org.apache.cassandra.schema.SystemDistributedKeyspace; +import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.schema.Tables; +import org.apache.cassandra.service.ActiveRepairService; +import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.streaming.StreamEventHandler; +import org.apache.cassandra.streaming.StreamReceiveException; +import org.apache.cassandra.streaming.StreamSession; +import org.apache.cassandra.streaming.StreamState; +import org.apache.cassandra.streaming.StreamingChannel; +import org.apache.cassandra.streaming.StreamingDataInputPlus; +import org.apache.cassandra.tools.nodetool.Repair; +import org.apache.cassandra.utils.AbstractTypeGenerators; +import org.apache.cassandra.utils.CassandraGenerators; +import org.apache.cassandra.utils.Clock; +import org.apache.cassandra.utils.Closeable; +import org.apache.cassandra.utils.FailingBiConsumer; +import org.apache.cassandra.utils.Generators; +import org.apache.cassandra.utils.MBeanWrapper; +import org.apache.cassandra.utils.MerkleTree; +import org.apache.cassandra.utils.MerkleTrees; +import org.apache.cassandra.utils.concurrent.AsyncPromise; +import org.apache.cassandra.utils.concurrent.Future; +import org.apache.cassandra.utils.concurrent.ImmediateFuture; +import org.apache.cassandra.utils.progress.ProgressEventType; +import org.assertj.core.api.Assertions; +import org.mockito.Mockito; +import org.quicktheories.impl.JavaRandom; + +import static org.apache.cassandra.config.CassandraRelevantProperties.CLOCK_GLOBAL; +import static org.apache.cassandra.config.CassandraRelevantProperties.ORG_APACHE_CASSANDRA_DISABLE_MBEAN_REGISTRATION; + +public abstract class FuzzTestBase extends CQLTester.InMemory +{ + private static final Gen<String> IDENTIFIER_GEN = fromQT(Generators.IDENTIFIER_GEN); + private static final Gen<String> KEYSPACE_NAME_GEN = fromQT(CassandraGenerators.KEYSPACE_NAME_GEN); + private static final Gen<TableId> TABLE_ID_GEN = fromQT(CassandraGenerators.TABLE_ID_GEN); + private static final Gen<InetAddressAndPort> ADDRESS_W_PORT = fromQT(CassandraGenerators.INET_ADDRESS_AND_PORT_GEN); + + private static boolean SETUP_SCHEMA = false; + static String KEYSPACE; + static List<String> TABLES; + + @BeforeClass + public static void setUpClass() + { + ORG_APACHE_CASSANDRA_DISABLE_MBEAN_REGISTRATION.setBoolean(true); + CLOCK_GLOBAL.setString(ClockAccess.class.getName()); + DatabaseDescriptor.daemonInitialization(); + DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance); // TOOD (coverage): random select + DatabaseDescriptor.setLocalDataCenter("test"); + IFailureDetector fd = Mockito.mock(IFailureDetector.class); + Mockito.when(fd.isAlive(Mockito.any())).thenReturn(true); + DatabaseDescriptor.setFailureDetector(() -> fd); + StreamingChannel.Factory.Global.unsafeSet(new StreamingChannel.Factory() + { + private final AtomicInteger counter = new AtomicInteger(); + + @Override + public StreamingChannel create(InetSocketAddress to, int messagingVersion, StreamingChannel.Kind kind) throws IOException + { + StreamingChannel mock = Mockito.mock(StreamingChannel.class); + int id = counter.incrementAndGet(); + StreamSession session = Mockito.mock(StreamSession.class); + StreamReceiveException access = new StreamReceiveException(session, "mock access rejected"); + StreamingDataInputPlus input = Mockito.mock(StreamingDataInputPlus.class, invocationOnMock -> { + throw access; + }); + Mockito.doNothing().when(input).close(); + Mockito.when(mock.in()).thenReturn(input); + Mockito.when(mock.id()).thenReturn(id); + Mockito.when(mock.peer()).thenReturn(to); + Mockito.when(mock.connectedTo()).thenReturn(to); + Mockito.when(mock.send(Mockito.any())).thenReturn(ImmediateFuture.success(null)); + Mockito.when(mock.close()).thenReturn(ImmediateFuture.success(null)); + return mock; + } + }); + ExecutorFactory delegate = ExecutorFactory.Global.executorFactory(); + ExecutorFactory.Global.unsafeSet(new ExecutorFactory() + { + @Override + public LocalAwareSubFactory localAware() + { + return delegate.localAware(); + } + + @Override + public ScheduledExecutorPlus scheduled(boolean executeOnShutdown, String name, int priority, SimulatorSemantics simulatorSemantics) + { + return delegate.scheduled(executeOnShutdown, name, priority, simulatorSemantics); + } + + private boolean shouldMock() + { + return StackWalker.getInstance().walk(frame -> { + StackWalker.StackFrame caller = frame.skip(3).findFirst().get(); + if (caller.getClassName().startsWith("org.apache.cassandra.streaming.")) return true; + return false; + }); + } + + @Override + public Thread startThread(String name, Runnable runnable, InfiniteLoopExecutor.Daemon daemon) + { + if (shouldMock()) return new Thread(); + return delegate.startThread(name, runnable, daemon); + } + + @Override + public Interruptible infiniteLoop(String name, Interruptible.Task task, InfiniteLoopExecutor.SimulatorSafe simulatorSafe, InfiniteLoopExecutor.Daemon daemon, InfiniteLoopExecutor.Interrupts interrupts) + { + return delegate.infiniteLoop(name, task, simulatorSafe, daemon, interrupts); + } + + @Override + public ThreadGroup newThreadGroup(String name) + { + return delegate.newThreadGroup(name); + } + + @Override + public ExecutorBuilderFactory<ExecutorPlus, SequentialExecutorPlus> withJmx(String jmxPath) + { + return delegate.withJmx(jmxPath); + } + + @Override + public ExecutorBuilder<? extends SequentialExecutorPlus> configureSequential(String name) + { + return delegate.configureSequential(name); + } + + @Override + public ExecutorBuilder<? extends ExecutorPlus> configurePooled(String name, int threads) + { + return delegate.configurePooled(name, threads); + } + }); + + // will both make sure this is loaded and used + if (!(Clock.Global.clock() instanceof ClockAccess)) throw new IllegalStateException("Unable to override clock"); + + // set the repair rcp timeout high so we don't hit it... this class is mostly testing repair reaching success + // so don't want to deal with unlucky histories... + DatabaseDescriptor.setRepairRpcTimeout(TimeUnit.DAYS.toMillis(1)); + + + InMemory.setUpClass(); + } + + @Before + public void setupSchema() + { + if (SETUP_SCHEMA) return; + SETUP_SCHEMA = true; + // StorageService can not be mocked out, nor can ColumnFamilyStores, so make sure that the keyspace is a "local" keyspace to avoid replication as the peers don't actually exist for replication + schemaChange(String.format("CREATE KEYSPACE %s WITH REPLICATION = {'class': '%s'}", SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, HackStrat.class.getName())); + for (TableMetadata table : SystemDistributedKeyspace.metadata().tables) + schemaChange(table.toCqlString(false, false)); + + createSchema(); + } + + private void createSchema() + { + // The main reason to use random here with a fixed seed is just to have a set of tables that are not hard coded. + // The tables will have diversity to them that most likely doesn't matter to repair (hence why the tables are shared), but + // is useful just in case some assumptions change. + RandomSource rs = new DefaultRandom(42); + String ks = KEYSPACE_NAME_GEN.next(rs); + List<String> tableNames = Gens.lists(IDENTIFIER_GEN).unique().ofSizeBetween(10, 100).next(rs); + JavaRandom qt = new JavaRandom(rs.asJdkRandom()); + Tables.Builder tableBuilder = Tables.builder(); + List<TableId> ids = Gens.lists(TABLE_ID_GEN).unique().ofSize(tableNames.size()).next(rs); + for (int i = 0; i < tableNames.size(); i++) + { + String name = tableNames.get(i); + TableId id = ids.get(i); + TableMetadata tableMetadata = new CassandraGenerators.TableMetadataBuilder().withKeyspaceName(ks).withTableName(name).withTableId(id).withTableKinds(TableMetadata.Kind.REGULAR) + // shouldn't matter, just wanted to avoid UDT as that needs more setup + .withDefaultTypeGen(AbstractTypeGenerators.builder().withTypeKinds(AbstractTypeGenerators.TypeKind.PRIMITIVE).withoutPrimitive(EmptyType.instance).build()).build().generate(qt); + tableBuilder.add(tableMetadata); + } + // TODO (now): add random selection + KeyspaceParams params = KeyspaceParams.simple(3); + KeyspaceMetadata metadata = KeyspaceMetadata.create(ks, params, tableBuilder.build()); + + // create + schemaChange(metadata.toCqlString(false, false)); + KEYSPACE = ks; + for (TableMetadata table : metadata.tables) + schemaChange(table.toCqlString(false, false)); + TABLES = tableNames; + } + + // TODO (now): truncate repair tables + + static void enableMessageFaults(Cluster cluster) + { + cluster.allowedMessageFaults(new BiFunction<>() + { + private final LongHashSet noFaults = new LongHashSet(); + private final LongHashSet allowDrop = new LongHashSet(); + + @Override + public Set<Faults> apply(Cluster.Node node, Message<?> message) + { + switch (message.verb()) + { + case PREPARE_MSG: + case VALIDATION_REQ: + case VALIDATION_RSP: + case SYNC_REQ: + case SYNC_RSP: + case SNAPSHOT_MSG: + case CLEANUP_MSG: + allowDrop.add(message.id()); + return Faults.DROPPED; + // these messages are not resilent to ephemeral issues + case PREPARE_CONSISTENT_REQ: + case PREPARE_CONSISTENT_RSP: + case FINALIZE_PROPOSE_MSG: + case FINALIZE_PROMISE_MSG: + case FINALIZE_COMMIT_MSG: + case FAILED_SESSION_MSG: + + noFaults.add(message.id()); + return Faults.NONE; + default: + if (noFaults.contains(message.id())) return Faults.NONE; + if (allowDrop.contains(message.id())) return Faults.DROPPED; + // was a new message added and the test not updated? + IllegalStateException e = new IllegalStateException("Verb: " + message.verb()); + cluster.failures.add(e); + throw e; + } + } + }); + } + + static void runAndAssertSuccess(Cluster cluster, int example, boolean shouldSync, RepairRunnable repair) + { + cluster.processAll(); + assertSuccess(example, shouldSync, repair); + } + + static void assertSuccess(int example, boolean shouldSync, RepairRunnable repair) + { + Assertions.assertThat(repair.state.getResult()).describedAs("Unexpected state: %s -> %s; example %d", repair.state, repair.state.getResult(), example).isEqualTo(Completable.Result.success(repairSuccessMessage(repair))); + Assertions.assertThat(repair.state.getStateTimesMillis().keySet()).isEqualTo(EnumSet.allOf(CoordinatorState.State.class)); + Assertions.assertThat(repair.state.getSessions()).isNotEmpty(); + boolean shouldSnapshot = repair.state.options.getParallelism() != RepairParallelism.PARALLEL + && (!repair.state.options.isIncremental() || repair.state.options.isPreview()); + for (SessionState session : repair.state.getSessions()) + { + Assertions.assertThat(session.getStateTimesMillis().keySet()).isEqualTo(EnumSet.allOf(SessionState.State.class)); + Assertions.assertThat(session.getJobs()).isNotEmpty(); + for (JobState job : session.getJobs()) + { + EnumSet<JobState.State> expected = EnumSet.allOf(JobState.State.class); + if (!shouldSnapshot) + { + expected.remove(JobState.State.SNAPSHOT_START); + expected.remove(JobState.State.SNAPSHOT_COMPLETE); + } + if (!shouldSync) + { + expected.remove(JobState.State.STREAM_START); + } + Set<JobState.State> actual = job.getStateTimesMillis().keySet(); + Assertions.assertThat(actual).isEqualTo(expected); + } + } + } + + static String repairSuccessMessage(RepairRunnable repair) + { + RepairOption options = repair.state.options; + if (options.isPreview()) + { + String postfix; + switch (options.getPreviewKind()) + { + case UNREPAIRED: + case ALL: + postfix = "Previewed data was in sync"; + break; + case REPAIRED: + postfix = "Repaired data is in sync"; + break; + default: + throw new IllegalArgumentException("Unexpected preview repair kind: " + options.getPreviewKind()); + } + return "Repair preview completed successfully; " + postfix; + } + return "Repair completed successfully"; + } + + InetAddressAndPort pickParticipant(RandomSource rs, Cluster.Node coordinator, RepairRunnable repair) + { + if (repair.state.isComplete()) + throw new IllegalStateException("Repair is completed! " + repair.state.getResult()); + List<InetAddressAndPort> participaents = new ArrayList<>(repair.state.getNeighborsAndRanges().participants.size() + 1); + if (rs.nextBoolean()) participaents.add(coordinator.broadcastAddressAndPort()); + participaents.addAll(repair.state.getNeighborsAndRanges().participants); + participaents.sort(Comparator.naturalOrder()); + + InetAddressAndPort selected = rs.pick(participaents); + return selected; + } + + static void addMismatch(RandomSource rs, ColumnFamilyStore cfs, Validator validator) + { + ValidationState state = validator.state; + int numPartitions = 1; + int maxDepth = DatabaseDescriptor.getRepairSessionMaxTreeDepth(); + state.phase.start(numPartitions, 1024); + + MerkleTrees trees = new MerkleTrees(cfs.getPartitioner()); + for (Range<Token> range : validator.desc.ranges) + { + int depth = numPartitions > 0 ? (int) Math.min(Math.ceil(Math.log(numPartitions) / Math.log(2)), maxDepth) : 0; + trees.addMerkleTree((int) Math.pow(2, depth), range); + } + Set<Token> allTokens = new HashSet<>(); + for (Range<Token> range : validator.desc.ranges) + { + Gen<Token> gen = fromQT(CassandraGenerators.tokensInRange(range)); + Set<Token> tokens = new LinkedHashSet<>(); + for (int i = 0, size = rs.nextInt(1, 10); i < size; i++) + { + for (int attempt = 0; !tokens.add(gen.next(rs)) && attempt < 5; attempt++) + { + } + } + // tokens may or may not be of the expected size; this depends on how wide the range is + for (Token token : tokens) + trees.split(token); + allTokens.addAll(tokens); + } + for (Token token : allTokens) + { + findCorrectRange(trees, token, range -> { + Digest digest = Digest.forValidator(); + digest.update(ByteBuffer.wrap(token.toString().getBytes(StandardCharsets.UTF_8))); + range.addHash(new MerkleTree.RowHash(token, digest.digest(), 1)); + }); + } + state.partitionsProcessed++; + state.bytesRead = 1024; + state.phase.sendingTrees(); + Stage.ANTI_ENTROPY.execute(() -> { + state.phase.success(); + validator.respond(new ValidationResponse(validator.desc, trees)); + }); + } + + private static void findCorrectRange(MerkleTrees trees, Token token, Consumer<MerkleTree.TreeRange> fn) + { + MerkleTrees.TreeRangeIterator it = trees.rangeIterator(); + while (it.hasNext()) + { + MerkleTree.TreeRange next = it.next(); + if (next.contains(token)) + { + fn.accept(next); + return; + } + } + } + + private enum RepairType + {FULL, IR} // TODO (coverage): --paxos-only; + + private enum PreviewType + {NONE, REPAIRED, UNREPAIRED} + + static RepairOption repairOption(RandomSource rs, Cluster.Node coordinator, String ks, List<String> tableNames) + { + return repairOption(rs, coordinator, ks, Gens.lists(Gens.pick(tableNames)).ofSizeBetween(1, tableNames.size()), Gens.enums().all(RepairType.class), Gens.enums().all(PreviewType.class), Gens.enums().all(RepairParallelism.class)); + } + + static RepairOption irOption(RandomSource rs, Cluster.Node coordinator, String ks, Gen<List<String>> tablesGen) + { + return repairOption(rs, coordinator, ks, tablesGen, Gens.constant(RepairType.IR), Gens.constant(PreviewType.NONE), Gens.enums().all(RepairParallelism.class)); + } + + static RepairOption previewOption(RandomSource rs, Cluster.Node coordinator, String ks, Gen<List<String>> tablesGen) + { + return repairOption(rs, coordinator, ks, tablesGen, Gens.constant(RepairType.FULL), Gens.constant(PreviewType.REPAIRED), Gens.enums().all(RepairParallelism.class)); + } + + private static RepairOption repairOption(RandomSource rs, Cluster.Node coordinator, String ks, Gen<List<String>> tablesGen, Gen<RepairType> repairTypeGen, Gen<PreviewType> previewTypeGen, Gen<RepairParallelism> repairParallelismGen) + { + List<String> args = new ArrayList<>(); + args.add(ks); + args.addAll(tablesGen.next(rs)); + args.add("-pr"); + RepairType type = repairTypeGen.next(rs); + switch (type) + { + case IR: + // default + break; + case FULL: + args.add("--full"); + break; + default: + throw new AssertionError("Unsupported repair type: " + type); + } + PreviewType previewType = previewTypeGen.next(rs); + switch (previewType) + { + case NONE: + break; + case REPAIRED: + args.add("--validate"); + break; + case UNREPAIRED: + args.add("--preview"); + break; + default: + throw new AssertionError("Unsupported preview type: " + previewType); + } + RepairParallelism parallelism = repairParallelismGen.next(rs); + switch (parallelism) + { + case SEQUENTIAL: + args.add("--sequential"); + break; + case PARALLEL: + // default + break; + case DATACENTER_AWARE: + args.add("--dc-parallel"); + break; + default: + throw new AssertionError("Unknown parallelism: " + parallelism); + } + if (rs.nextBoolean()) args.add("--optimise-streams"); + RepairOption options = RepairOption.parse(Repair.parseOptionMap(args), DatabaseDescriptor.getPartitioner()); + if (options.getRanges().isEmpty()) + { + if (options.isPrimaryRange()) + { + // when repairing only primary range, neither dataCenters nor hosts can be set + if (options.getDataCenters().isEmpty() && options.getHosts().isEmpty()) + options.getRanges().addAll(coordinator.getPrimaryRanges(ks)); + // except dataCenters only contain local DC (i.e. -local) + else if (options.isInLocalDCOnly()) + options.getRanges().addAll(coordinator.getPrimaryRangesWithinDC(ks)); + else + throw new IllegalArgumentException("You need to run primary range repair on all nodes in the cluster."); + } + else + { + Iterables.addAll(options.getRanges(), coordinator.getLocalReplicas(ks).onlyFull().ranges()); + } + } + return options; + } + + enum Faults + { + DELAY, DROP; + + public static final Set<Faults> NONE = Collections.emptySet(); + public static final Set<Faults> DELAYED = EnumSet.of(DELAY); + public static final Set<Faults> DROPPED = EnumSet.of(DELAY, DROP); + } + + private static class Connection + { + final InetAddressAndPort from, to; + + private Connection(InetAddressAndPort from, InetAddressAndPort to) + { + this.from = from; + this.to = to; + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Connection that = (Connection) o; + return from.equals(that.from) && to.equals(that.to); + } + + @Override + public int hashCode() + { + return Objects.hash(from, to); + } + + @Override + public String toString() + { + return "Connection{" + "from=" + from + ", to=" + to + '}'; + } + } + + static class Cluster + { + private static final FailingBiConsumer<ColumnFamilyStore, Validator> DEFAULT_VALIDATION = ValidationManager::doValidation; + + final Map<InetAddressAndPort, Node> nodes; + private final IFailureDetector failureDetector = Mockito.mock(IFailureDetector.class); + private final IEndpointSnitch snitch = Mockito.mock(IEndpointSnitch.class); + private final SimulatedExecutorFactory globalExecutor; + final ScheduledExecutorPlus unorderedScheduled; + final ExecutorPlus orderedExecutor; + private final Gossip gossiper = new Gossip(); + private final MBeanWrapper mbean = Mockito.mock(MBeanWrapper.class); + private final List<Throwable> failures = new ArrayList<>(); + private final RandomSource rs; + private BiFunction<Node, Message<?>, Set<Faults>> allowedMessageFaults = (a, b) -> Collections.emptySet(); + + private final Map<Connection, LongSupplier> networkLatencies = new HashMap<>(); + private final Map<Connection, Supplier<Boolean>> networkDrops = new HashMap<>(); + + Cluster(RandomSource rs) + { + this.rs = rs; + globalExecutor = new SimulatedExecutorFactory(rs, fromQT(Generators.TIMESTAMP_GEN.map(Timestamp::getTime)).next(rs)); + orderedExecutor = globalExecutor.configureSequential("ignore").build(); + unorderedScheduled = globalExecutor.scheduled("ignored"); + + // We run tests in an isolated JVM per class, so not cleaing up is safe... but if that assumption ever changes, will need to cleanup + Stage.ANTI_ENTROPY.unsafeSetExecutor(orderedExecutor); + Stage.INTERNAL_RESPONSE.unsafeSetExecutor(unorderedScheduled); + Mockito.when(failureDetector.isAlive(Mockito.any())).thenReturn(true); + int numNodes = rs.nextInt(3, 10); + List<String> dcs = Gens.lists(IDENTIFIER_GEN).unique().ofSizeBetween(1, Math.min(10, numNodes)).next(rs); + Map<InetAddressAndPort, Node> nodes = Maps.newHashMapWithExpectedSize(numNodes); + Gen<Token> tokenGen = fromQT(CassandraGenerators.token(DatabaseDescriptor.getPartitioner())); + Gen<UUID> hostIdGen = fromQT(Generators.UUID_RANDOM_GEN); + Set<Token> tokens = new HashSet<>(); + Set<UUID> hostIds = new HashSet<>(); + for (int i = 0; i < numNodes; i++) + { + InetAddressAndPort addressAndPort = ADDRESS_W_PORT.next(rs); + while (nodes.containsKey(addressAndPort)) addressAndPort = ADDRESS_W_PORT.next(rs); + // TODO (coverage): vnodes + // TODO (coverage): use provided partitioner and not the hard coded murmur + Token token; + while (!tokens.add(token = tokenGen.next(rs))) + { + } + UUID hostId; + while (!hostIds.add(hostId = hostIdGen.next(rs))) + { + } + + String dc = rs.pick(dcs); + String rack = "rack"; + Mockito.when(snitch.getDatacenter(Mockito.eq(addressAndPort))).thenReturn(dc); + Mockito.when(snitch.getRack(Mockito.eq(addressAndPort))).thenReturn(rack); + + VersionedValue.VersionedValueFactory valueFactory = new VersionedValue.VersionedValueFactory(DatabaseDescriptor.getPartitioner()); + EndpointState state = new EndpointState(new HeartBeatState(42, 42)); + // TODO (correctness): what is really required? ATM its only RELEASE_VERSION... + state.addApplicationState(ApplicationState.STATUS, valueFactory.normal(Collections.singleton(token))); Review Comment: nit: STATUS has been deprecated for a while, so I'm guessing we definitely don't need that. ########## test/unit/org/apache/cassandra/repair/FuzzTest.java: ########## @@ -0,0 +1,1454 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.repair; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.sql.Timestamp; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Random; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiFunction; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.function.LongSupplier; +import java.util.function.Supplier; +import javax.annotation.Nullable; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import com.google.common.collect.Maps; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import accord.utils.DefaultRandom; +import accord.utils.Gen; +import accord.utils.Gens; +import accord.utils.RandomSource; +import org.agrona.collections.Long2ObjectHashMap; +import org.agrona.collections.LongHashSet; +import org.apache.cassandra.concurrent.ExecutorBuilder; +import org.apache.cassandra.concurrent.ExecutorBuilderFactory; +import org.apache.cassandra.concurrent.ExecutorFactory; +import org.apache.cassandra.concurrent.ExecutorPlus; +import org.apache.cassandra.concurrent.InfiniteLoopExecutor; +import org.apache.cassandra.concurrent.Interruptible; +import org.apache.cassandra.concurrent.ScheduledExecutorPlus; +import org.apache.cassandra.concurrent.SequentialExecutorPlus; +import org.apache.cassandra.concurrent.SimulatedExecutorFactory; +import org.apache.cassandra.concurrent.Stage; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.RetrySpec; +import org.apache.cassandra.config.SharedContext; +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.Digest; +import org.apache.cassandra.db.marshal.EmptyType; +import org.apache.cassandra.db.repair.CassandraTableRepairManager; +import org.apache.cassandra.db.repair.PendingAntiCompaction; +import org.apache.cassandra.dht.Murmur3Partitioner; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.exceptions.RequestFailureReason; +import org.apache.cassandra.gms.ApplicationState; +import org.apache.cassandra.gms.EndpointState; +import org.apache.cassandra.gms.HeartBeatState; +import org.apache.cassandra.gms.IEndpointStateChangeSubscriber; +import org.apache.cassandra.gms.IFailureDetector; +import org.apache.cassandra.gms.IGossiper; +import org.apache.cassandra.gms.VersionedValue; +import org.apache.cassandra.io.util.DataInputBuffer; +import org.apache.cassandra.io.util.DataOutputBuffer; +import org.apache.cassandra.locator.IEndpointSnitch; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.locator.LocalStrategy; +import org.apache.cassandra.locator.RangesAtEndpoint; +import org.apache.cassandra.locator.TokenMetadata; +import org.apache.cassandra.net.ConnectionType; +import org.apache.cassandra.net.IVerbHandler; +import org.apache.cassandra.net.Message; +import org.apache.cassandra.net.MessageDelivery; +import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.net.RequestCallback; +import org.apache.cassandra.repair.consistent.LocalSessions; +import org.apache.cassandra.repair.messages.RepairOption; +import org.apache.cassandra.repair.messages.ValidationResponse; +import org.apache.cassandra.repair.state.AbstractCompletable; +import org.apache.cassandra.repair.state.AbstractState; +import org.apache.cassandra.repair.state.Completable; +import org.apache.cassandra.repair.state.ValidationState; +import org.apache.cassandra.schema.KeyspaceMetadata; +import org.apache.cassandra.schema.KeyspaceParams; +import org.apache.cassandra.schema.SchemaConstants; +import org.apache.cassandra.schema.SystemDistributedKeyspace; +import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.schema.Tables; +import org.apache.cassandra.service.ActiveRepairService; +import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.streaming.StreamEventHandler; +import org.apache.cassandra.streaming.StreamReceiveException; +import org.apache.cassandra.streaming.StreamSession; +import org.apache.cassandra.streaming.StreamState; +import org.apache.cassandra.streaming.StreamingChannel; +import org.apache.cassandra.streaming.StreamingDataInputPlus; +import org.apache.cassandra.tools.nodetool.Repair; +import org.apache.cassandra.utils.AbstractTypeGenerators; +import org.apache.cassandra.utils.CassandraGenerators; +import org.apache.cassandra.utils.Clock; +import org.apache.cassandra.utils.Closeable; +import org.apache.cassandra.utils.FailingBiConsumer; +import org.apache.cassandra.utils.Generators; +import org.apache.cassandra.utils.MBeanWrapper; +import org.apache.cassandra.utils.MerkleTree; +import org.apache.cassandra.utils.MerkleTrees; +import org.apache.cassandra.utils.concurrent.AsyncPromise; +import org.apache.cassandra.utils.concurrent.Future; +import org.apache.cassandra.utils.concurrent.ImmediateFuture; +import org.apache.cassandra.utils.progress.ProgressEventType; +import org.assertj.core.api.AbstractStringAssert; +import org.assertj.core.api.Assertions; +import org.mockito.Mockito; +import org.quicktheories.impl.JavaRandom; + +import static accord.utils.Property.qt; +import static org.apache.cassandra.config.CassandraRelevantProperties.CLOCK_GLOBAL; +import static org.apache.cassandra.config.CassandraRelevantProperties.ORG_APACHE_CASSANDRA_DISABLE_MBEAN_REGISTRATION; + +// TODO (now, coverage): randomly fail validation +public class FuzzTest extends CQLTester.InMemory +{ + private enum RepairJobStage { VALIDATION, SYNC } + + private static final Gen<String> IDENTIFIER_GEN = fromQT(Generators.IDENTIFIER_GEN); + private static final Gen<String> KEYSPACE_NAME_GEN = fromQT(CassandraGenerators.KEYSPACE_NAME_GEN); + private static final Gen<TableId> TABLE_ID_GEN = fromQT(CassandraGenerators.TABLE_ID_GEN); + private static final Gen<InetAddressAndPort> ADDRESS_W_PORT = fromQT(CassandraGenerators.INET_ADDRESS_AND_PORT_GEN); + + private static boolean SETUP_SCHEMA = false; + private static String KEYSPACE; + private static List<String> TABLES; + + @BeforeClass + public static void setUpClass() + { + ORG_APACHE_CASSANDRA_DISABLE_MBEAN_REGISTRATION.setBoolean(true); + CLOCK_GLOBAL.setString(ClockAccess.class.getName()); + DatabaseDescriptor.daemonInitialization(); + DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance); // TOOD (coverage): random select + DatabaseDescriptor.setLocalDataCenter("test"); + IFailureDetector fd = Mockito.mock(IFailureDetector.class); + Mockito.when(fd.isAlive(Mockito.any())).thenReturn(true); + DatabaseDescriptor.setFailureDetector(() -> fd); + StreamingChannel.Factory.Global.unsafeSet(new StreamingChannel.Factory() + { + private final AtomicInteger counter = new AtomicInteger(); + + @Override + public StreamingChannel create(InetSocketAddress to, int messagingVersion, StreamingChannel.Kind kind) throws IOException + { + StreamingChannel mock = Mockito.mock(StreamingChannel.class); + int id = counter.incrementAndGet(); + StreamSession session = Mockito.mock(StreamSession.class); + StreamReceiveException access = new StreamReceiveException(session, "mock access rejected"); + StreamingDataInputPlus input = Mockito.mock(StreamingDataInputPlus.class, invocationOnMock -> { + throw access; + }); + Mockito.doNothing().when(input).close(); + Mockito.when(mock.in()).thenReturn(input); + Mockito.when(mock.id()).thenReturn(id); + Mockito.when(mock.peer()).thenReturn(to); + Mockito.when(mock.connectedTo()).thenReturn(to); + Mockito.when(mock.send(Mockito.any())).thenReturn(ImmediateFuture.success(null)); + Mockito.when(mock.close()).thenReturn(ImmediateFuture.success(null)); + return mock; + } + }); + ExecutorFactory delegate = ExecutorFactory.Global.executorFactory(); + ExecutorFactory.Global.unsafeSet(new ExecutorFactory() + { + @Override + public LocalAwareSubFactory localAware() + { + return delegate.localAware(); + } + + @Override + public ScheduledExecutorPlus scheduled(boolean executeOnShutdown, String name, int priority, SimulatorSemantics simulatorSemantics) + { + return delegate.scheduled(executeOnShutdown, name, priority, simulatorSemantics); + } + + private boolean shouldMock() + { + return StackWalker.getInstance().walk(frame -> { + StackWalker.StackFrame caller = frame.skip(3).findFirst().get(); + if (caller.getClassName().startsWith("org.apache.cassandra.streaming.")) return true; + return false; + }); + } + + @Override + public Thread startThread(String name, Runnable runnable, InfiniteLoopExecutor.Daemon daemon) + { + if (shouldMock()) return new Thread(); + return delegate.startThread(name, runnable, daemon); + } + + @Override + public Interruptible infiniteLoop(String name, Interruptible.Task task, InfiniteLoopExecutor.SimulatorSafe simulatorSafe, InfiniteLoopExecutor.Daemon daemon, InfiniteLoopExecutor.Interrupts interrupts) + { + return delegate.infiniteLoop(name, task, simulatorSafe, daemon, interrupts); + } + + @Override + public ThreadGroup newThreadGroup(String name) + { + return delegate.newThreadGroup(name); + } + + @Override + public ExecutorBuilderFactory<ExecutorPlus, SequentialExecutorPlus> withJmx(String jmxPath) + { + return delegate.withJmx(jmxPath); + } + + @Override + public ExecutorBuilder<? extends SequentialExecutorPlus> configureSequential(String name) + { + return delegate.configureSequential(name); + } + + @Override + public ExecutorBuilder<? extends ExecutorPlus> configurePooled(String name, int threads) + { + return delegate.configurePooled(name, threads); + } + }); + + // will both make sure this is loaded and used + if (!(Clock.Global.clock() instanceof ClockAccess)) throw new IllegalStateException("Unable to override clock"); + + // set the repair rcp timeout high so we don't hit it... this class is mostly testing repair reaching success + // so don't want to deal with unlucky histories... + DatabaseDescriptor.setRepairRpcTimeout(TimeUnit.DAYS.toMillis(1)); + + + InMemory.setUpClass(); + } + + @Before + public void setupSchema() + { + if (SETUP_SCHEMA) return; + SETUP_SCHEMA = true; + // StorageService can not be mocked out, nor can ColumnFamilyStores, so make sure that the keyspace is a "local" keyspace to avoid replication as the peers don't actually exist for replication + schemaChange(String.format("CREATE KEYSPACE %s WITH REPLICATION = {'class': '%s'}", SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, HackStrat.class.getName())); + for (TableMetadata table : SystemDistributedKeyspace.metadata().tables) + schemaChange(table.toCqlString(false, false)); + + createSchema(); + } + + private void createSchema() + { + // The main reason to use random here with a fixed seed is just to have a set of tables that are not hard coded. + // The tables will have diversity to them that most likely doesn't matter to repair (hence why the tables are shared), but + // is useful just in case some assumptions change. + RandomSource rs = new DefaultRandom(42); + String ks = KEYSPACE_NAME_GEN.next(rs); + List<String> tableNames = Gens.lists(IDENTIFIER_GEN).unique().ofSizeBetween(10, 100).next(rs); + JavaRandom qt = new JavaRandom(rs.asJdkRandom()); + Tables.Builder tableBuilder = Tables.builder(); + List<TableId> ids = Gens.lists(TABLE_ID_GEN).unique().ofSize(tableNames.size()).next(rs); + for (int i = 0; i < tableNames.size(); i++) + { + String name = tableNames.get(i); + TableId id = ids.get(i); + TableMetadata tableMetadata = new CassandraGenerators.TableMetadataBuilder().withKeyspaceName(ks).withTableName(name).withTableId(id).withTableKinds(TableMetadata.Kind.REGULAR) + // shouldn't matter, just wanted to avoid UDT as that needs more setup + .withDefaultTypeGen(AbstractTypeGenerators.builder().withTypeKinds(AbstractTypeGenerators.TypeKind.PRIMITIVE).withoutPrimitive(EmptyType.instance).build()).build().generate(qt); + tableBuilder.add(tableMetadata); + } + // TODO (now): add random selection + KeyspaceParams params = KeyspaceParams.simple(3); + KeyspaceMetadata metadata = KeyspaceMetadata.create(ks, params, tableBuilder.build()); + + // create + schemaChange(metadata.toCqlString(false, false)); + KEYSPACE = ks; + for (TableMetadata table : metadata.tables) + schemaChange(table.toCqlString(false, false)); + TABLES = tableNames; + } + + // TODO (now): truncate repair tables + + @Test + public void happyPath() + { + // disable all retries, no delays/drops are possible + DatabaseDescriptor.getRepairRetrys().maxAttempts = RetrySpec.MaxAttempt.DISABLED; + qt().withPure(false).withExamples(10).check(rs -> { + Cluster cluster = new Cluster(rs); + Gen<Cluster.Node> coordinatorGen = Gens.pick(cluster.nodes.keySet()).map(cluster.nodes::get); + + List<Closeable> closeables = new ArrayList<>(); + for (int example = 0; example < 100; example++) + { + Cluster.Node coordinator = coordinatorGen.next(rs); + + RepairRunnable repair = coordinator.repair(KEYSPACE, repairOption(rs, coordinator, KEYSPACE, TABLES)); + repair.run(); + if (rs.nextBoolean()) + closeables.add(cluster.nodes.get(pickParticipant(rs, coordinator, repair)).doValidation((cfs, validator) -> addMismatch(rs, cfs, validator))); + + runAndAssertSuccess(cluster, example, repair); Review Comment: So to confirm my understanding, at this point, we've done the following: 1.) Run the preparation phase of the `RepairRunnable`, but not actually started repair itself. 2.) Enqueued a validation function that will introduce a mismatch. 3.) `RepairRunnable` has added the actual repair task to our simulated executor. At this point, we release the executor to process 2 and 3, then make sure the repair completed successfully? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]

