krummas commented on code in PR #2660: URL: https://github.com/apache/cassandra/pull/2660#discussion_r1336703759
########## src/java/org/apache/cassandra/config/SharedContext.java: ########## @@ -0,0 +1,168 @@ +/* + * 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; Review Comment: nit; this should be in repair package I think? ########## src/java/org/apache/cassandra/repair/state/CoordinatorState.java: ########## @@ -114,14 +117,36 @@ public List<CommonRange> getFilteredCommonRanges() return neighborsAndRanges.filterCommonRanges(keyspace, getColumnFamilyNames()); } + @Override + public String status() + { + State currentState = getStatus(); + Result result = getResult(); + if (result != null) return result.kind.name(); Review Comment: nit; code style ########## test/unit/org/apache/cassandra/repair/FuzzTestBase.java: ########## @@ -0,0 +1,1301 @@ +/* + * 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.apache.cassandra.config.UnitConfigOverride; +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.compaction.ICompactionManager; +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.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 int MISMATCH_NUM_PARTITIONS = 1; + 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()); + // when running in CI an external actor will replace the test configs based off the test type (such as trie, cdc, etc.), this could then have failing tests + // that do not repo with the same seed! To fix that, go to UnitConfigOverride and update the config type to match the one that failed in CI, this should then + // use the same config, so the seed should not reproduce. + UnitConfigOverride.maybeOverrideConfig(); + + 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); + } + 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; + } + + 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, RepairCoordinator repair) + { + cluster.processAll(); + assertSuccess(example, shouldSync, repair); + } + + static void assertSuccess(int example, boolean shouldSync, RepairCoordinator 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(RepairCoordinator repair) + { + RepairOption options = repair.state.options; + if (options.isPreview()) + { + String postfix; Review Comment: nit, I think this should be `suffix` ########## src/java/org/apache/cassandra/repair/state/AbstractState.java: ########## @@ -23,20 +23,46 @@ public abstract class AbstractState<T extends Enum<T>, I> extends AbstractCompletable<I> implements State<T, I> { + protected enum UpdateType { Review Comment: nit; code style, brace on newline ########## src/java/org/apache/cassandra/repair/state/SessionState.java: ########## @@ -73,6 +75,17 @@ public Set<InetAddressAndPort> getParticipants() return commonRange.endpoints; } + @Override + public String status() + { + State state = getStatus(); + Result result = getResult(); + if (result != null) return result.kind.name(); Review Comment: nit; code style ########## src/java/org/apache/cassandra/repair/messages/RepairMessage.java: ########## @@ -42,57 +55,180 @@ */ public abstract class RepairMessage { - private static final CassandraVersion SUPPORTS_TIMEOUTS = new CassandraVersion("4.0.7-SNAPSHOT"); + private enum ErrorHandling { NONE, TIMEOUT, RETRY } + private static final CassandraVersion SUPPORTS_RETRY = new CassandraVersion("5.1.0-SNAPSHOT"); + private static final Map<Verb, CassandraVersion> VERB_TIMEOUT_VERSIONS; + + static + { + CassandraVersion timeoutVersion = new CassandraVersion("4.0.7-SNAPSHOT"); + EnumMap<Verb, CassandraVersion> map = new EnumMap<>(Verb.class); + map.put(Verb.VALIDATION_REQ, timeoutVersion); + map.put(Verb.SYNC_REQ, timeoutVersion); + map.put(Verb.VALIDATION_RSP, SUPPORTS_RETRY); + map.put(Verb.SYNC_RSP, SUPPORTS_RETRY); + VERB_TIMEOUT_VERSIONS = Collections.unmodifiableMap(map); + } + private static final Set<Verb> SUPPORTS_RETRY_WITHOUT_VERSION_CHECK = Collections.unmodifiableSet(EnumSet.of(Verb.CLEANUP_MSG)); + private static final Logger logger = LoggerFactory.getLogger(RepairMessage.class); + @Nullable public final RepairJobDesc desc; - protected RepairMessage(RepairJobDesc desc) + protected RepairMessage(@Nullable RepairJobDesc desc) { this.desc = desc; } + public TimeUUID parentRepairSession() + { + return desc.parentSessionId; + } + public interface RepairFailureCallback { void onFailure(Exception e); } - public static void sendMessageWithFailureCB(RepairMessage request, Verb verb, InetAddressAndPort endpoint, RepairFailureCallback failureCallback) + private static Backoff backoff(SharedContext ctx, Verb verb) + { + RepairRetrySpec retrySpec = DatabaseDescriptor.getRepairRetrySpec(); + RetrySpec spec = verb == Verb.VALIDATION_RSP ? retrySpec.getMerkleTreeResponseSpec() : retrySpec; + if (!spec.isEnabled()) + return Backoff.None.INSTANCE; + return new Backoff.ExponentialBackoff(spec.maxAttempts.value, spec.baseSleepTime.toMilliseconds(), spec.maxSleepTime.toMilliseconds(), ctx.random().get()::nextDouble); + } + + public static Supplier<Boolean> notDone(Future<?> f) { - RequestCallback<?> callback = new RequestCallback<Object>() + return () -> !f.isDone(); + } + + private static Supplier<Boolean> always() + { + return () -> true; + } + + public static <T> void sendMessageWithRetries(SharedContext ctx, Supplier<Boolean> allowRetry, RepairMessage request, Verb verb, InetAddressAndPort endpoint, RequestCallback<T> finalCallback) + { + sendMessageWithRetries(ctx, backoff(ctx, verb), allowRetry, request, verb, endpoint, finalCallback, 0); + } + + public static <T> void sendMessageWithRetries(SharedContext ctx, RepairMessage request, Verb verb, InetAddressAndPort endpoint, RequestCallback<T> finalCallback) + { + sendMessageWithRetries(ctx, backoff(ctx, verb), always(), request, verb, endpoint, finalCallback, 0); + } + + public static void sendMessageWithRetries(SharedContext ctx, RepairMessage request, Verb verb, InetAddressAndPort endpoint) + { + sendMessageWithRetries(ctx, backoff(ctx, verb), always(), request, verb, endpoint, new RequestCallback<>() { @Override public void onResponse(Message<Object> msg) { - logger.info("[#{}] {} received by {}", request.desc.parentSessionId, verb, endpoint); - // todo: at some point we should make repair messages follow the normal path, actually using this + } + + @Override + public void onFailure(InetAddressAndPort from, RequestFailureReason failureReason) + { + } + }, 0); + } + + private static <T> void sendMessageWithRetries(SharedContext ctx, Backoff backoff, Supplier<Boolean> allowRetry, RepairMessage request, Verb verb, InetAddressAndPort endpoint, RequestCallback<T> finalCallback, int attempt) + { + RequestCallback<T> callback = new RequestCallback<>() + { + @Override + public void onResponse(Message<T> msg) + { + finalCallback.onResponse(msg); + } + + @Override + public void onFailure(InetAddressAndPort from, RequestFailureReason failureReason) + { + ErrorHandling allowed = errorHandlingSupported(ctx, endpoint, verb, request.parentRepairSession()); + switch (allowed) + { + case NONE: + logger.error("[#{}] {} failed on {}: {}", request.parentRepairSession(), verb, from, failureReason); + return; + case TIMEOUT: + finalCallback.onFailure(from, failureReason); + return; + case RETRY: + int maxAttempts = backoff.maxAttempts(); + if (failureReason == RequestFailureReason.TIMEOUT && attempt < maxAttempts && allowRetry.get()) + { + ctx.optionalTasks().schedule(() -> sendMessageWithRetries(ctx, backoff, allowRetry, request, verb, endpoint, finalCallback, attempt + 1), + backoff.computeWaitTime(attempt), backoff.unit()); + return; + } + finalCallback.onFailure(from, failureReason); + return; + default: + throw new AssertionError("Unknown error handler: " + allowed); + } } @Override public boolean invokeOnFailure() { return true; } + }; + ctx.messaging().sendWithCallback(Message.outWithFlag(verb, request, CALL_BACK_ON_FAILURE), + endpoint, + callback); + } + public static void sendMessageWithFailureCB(SharedContext ctx, Supplier<Boolean> allowRetry, RepairMessage request, Verb verb, InetAddressAndPort endpoint, RepairFailureCallback failureCallback) + { + RequestCallback<?> callback = new RequestCallback<>() + { + @Override + public void onResponse(Message<Object> msg) + { + logger.info("[#{}] {} received by {}", request.parentRepairSession(), verb, endpoint); + // todo: at some point we should make repair messages follow the normal path, actually using this + } + + @Override public void onFailure(InetAddressAndPort from, RequestFailureReason failureReason) { - logger.error("[#{}] {} failed on {}: {}", request.desc.parentSessionId, verb, from, failureReason); + failureCallback.onFailure(RepairException.error(request.desc, PreviewKind.NONE, String.format("Got %s failure from %s: %s", verb, from, failureReason))); + } - if (supportsTimeouts(from, request.desc.parentSessionId)) - failureCallback.onFailure(RepairException.error(request.desc, PreviewKind.NONE, String.format("Got %s failure from %s: %s", verb, from, failureReason))); + @Override + public boolean invokeOnFailure() + { + return true; } }; - - MessagingService.instance().sendWithCallback(Message.outWithFlag(verb, request, CALL_BACK_ON_FAILURE), - endpoint, - callback); + sendMessageWithRetries(ctx, allowRetry, request, verb, endpoint, callback); } - private static boolean supportsTimeouts(InetAddressAndPort from, TimeUUID parentSessionId) + private static ErrorHandling errorHandlingSupported(SharedContext ctx, InetAddressAndPort from, Verb verb, TimeUUID parentSessionId) { - CassandraVersion remoteVersion = Gossiper.instance.getReleaseVersion(from); - if (remoteVersion != null && remoteVersion.compareTo(SUPPORTS_TIMEOUTS) >= 0) - return true; - logger.warn("[#{}] Not failing repair due to remote host {} not supporting repair message timeouts (version = {})", parentSessionId, from, remoteVersion); - return false; + if (SUPPORTS_RETRY_WITHOUT_VERSION_CHECK.contains(verb)) + return ErrorHandling.RETRY; + // Repair in mixed mode isn't fully supported, but also not activally blocked... so in the common case all participants + // will be on the same version as this instance, so can avoid the lookup from gossip + CassandraVersion remoteVersion = ctx.gossiper().getReleaseVersion(from); + if (remoteVersion == null) + { + if (VERB_TIMEOUT_VERSIONS.containsKey(verb)) + { + logger.warn("[#{}] Not failing repair due to remote host {} not supporting repair message timeouts (version is unknown)", parentSessionId, from); + return ErrorHandling.NONE; + } + return ErrorHandling.TIMEOUT; + } + if (remoteVersion.compareTo(SUPPORTS_RETRY) >= 0) return ErrorHandling.RETRY; Review Comment: nit; code style ########## src/java/org/apache/cassandra/repair/messages/RepairMessage.java: ########## @@ -42,57 +55,180 @@ */ public abstract class RepairMessage { - private static final CassandraVersion SUPPORTS_TIMEOUTS = new CassandraVersion("4.0.7-SNAPSHOT"); + private enum ErrorHandling { NONE, TIMEOUT, RETRY } + private static final CassandraVersion SUPPORTS_RETRY = new CassandraVersion("5.1.0-SNAPSHOT"); + private static final Map<Verb, CassandraVersion> VERB_TIMEOUT_VERSIONS; + + static + { + CassandraVersion timeoutVersion = new CassandraVersion("4.0.7-SNAPSHOT"); + EnumMap<Verb, CassandraVersion> map = new EnumMap<>(Verb.class); + map.put(Verb.VALIDATION_REQ, timeoutVersion); + map.put(Verb.SYNC_REQ, timeoutVersion); + map.put(Verb.VALIDATION_RSP, SUPPORTS_RETRY); + map.put(Verb.SYNC_RSP, SUPPORTS_RETRY); + VERB_TIMEOUT_VERSIONS = Collections.unmodifiableMap(map); + } + private static final Set<Verb> SUPPORTS_RETRY_WITHOUT_VERSION_CHECK = Collections.unmodifiableSet(EnumSet.of(Verb.CLEANUP_MSG)); + private static final Logger logger = LoggerFactory.getLogger(RepairMessage.class); + @Nullable public final RepairJobDesc desc; - protected RepairMessage(RepairJobDesc desc) + protected RepairMessage(@Nullable RepairJobDesc desc) { this.desc = desc; } + public TimeUUID parentRepairSession() + { + return desc.parentSessionId; + } + public interface RepairFailureCallback { void onFailure(Exception e); } - public static void sendMessageWithFailureCB(RepairMessage request, Verb verb, InetAddressAndPort endpoint, RepairFailureCallback failureCallback) + private static Backoff backoff(SharedContext ctx, Verb verb) + { + RepairRetrySpec retrySpec = DatabaseDescriptor.getRepairRetrySpec(); + RetrySpec spec = verb == Verb.VALIDATION_RSP ? retrySpec.getMerkleTreeResponseSpec() : retrySpec; + if (!spec.isEnabled()) + return Backoff.None.INSTANCE; + return new Backoff.ExponentialBackoff(spec.maxAttempts.value, spec.baseSleepTime.toMilliseconds(), spec.maxSleepTime.toMilliseconds(), ctx.random().get()::nextDouble); + } + + public static Supplier<Boolean> notDone(Future<?> f) { - RequestCallback<?> callback = new RequestCallback<Object>() + return () -> !f.isDone(); + } + + private static Supplier<Boolean> always() + { + return () -> true; + } + + public static <T> void sendMessageWithRetries(SharedContext ctx, Supplier<Boolean> allowRetry, RepairMessage request, Verb verb, InetAddressAndPort endpoint, RequestCallback<T> finalCallback) + { + sendMessageWithRetries(ctx, backoff(ctx, verb), allowRetry, request, verb, endpoint, finalCallback, 0); + } + + public static <T> void sendMessageWithRetries(SharedContext ctx, RepairMessage request, Verb verb, InetAddressAndPort endpoint, RequestCallback<T> finalCallback) + { + sendMessageWithRetries(ctx, backoff(ctx, verb), always(), request, verb, endpoint, finalCallback, 0); + } + + public static void sendMessageWithRetries(SharedContext ctx, RepairMessage request, Verb verb, InetAddressAndPort endpoint) + { + sendMessageWithRetries(ctx, backoff(ctx, verb), always(), request, verb, endpoint, new RequestCallback<>() { @Override public void onResponse(Message<Object> msg) { - logger.info("[#{}] {} received by {}", request.desc.parentSessionId, verb, endpoint); - // todo: at some point we should make repair messages follow the normal path, actually using this + } + + @Override + public void onFailure(InetAddressAndPort from, RequestFailureReason failureReason) + { + } + }, 0); + } + + private static <T> void sendMessageWithRetries(SharedContext ctx, Backoff backoff, Supplier<Boolean> allowRetry, RepairMessage request, Verb verb, InetAddressAndPort endpoint, RequestCallback<T> finalCallback, int attempt) + { + RequestCallback<T> callback = new RequestCallback<>() + { + @Override + public void onResponse(Message<T> msg) + { + finalCallback.onResponse(msg); + } + + @Override + public void onFailure(InetAddressAndPort from, RequestFailureReason failureReason) + { + ErrorHandling allowed = errorHandlingSupported(ctx, endpoint, verb, request.parentRepairSession()); + switch (allowed) + { + case NONE: + logger.error("[#{}] {} failed on {}: {}", request.parentRepairSession(), verb, from, failureReason); + return; + case TIMEOUT: + finalCallback.onFailure(from, failureReason); + return; + case RETRY: + int maxAttempts = backoff.maxAttempts(); + if (failureReason == RequestFailureReason.TIMEOUT && attempt < maxAttempts && allowRetry.get()) + { + ctx.optionalTasks().schedule(() -> sendMessageWithRetries(ctx, backoff, allowRetry, request, verb, endpoint, finalCallback, attempt + 1), + backoff.computeWaitTime(attempt), backoff.unit()); + return; + } + finalCallback.onFailure(from, failureReason); + return; + default: + throw new AssertionError("Unknown error handler: " + allowed); + } } @Override public boolean invokeOnFailure() { return true; } + }; + ctx.messaging().sendWithCallback(Message.outWithFlag(verb, request, CALL_BACK_ON_FAILURE), + endpoint, + callback); + } + public static void sendMessageWithFailureCB(SharedContext ctx, Supplier<Boolean> allowRetry, RepairMessage request, Verb verb, InetAddressAndPort endpoint, RepairFailureCallback failureCallback) + { + RequestCallback<?> callback = new RequestCallback<>() + { + @Override + public void onResponse(Message<Object> msg) + { + logger.info("[#{}] {} received by {}", request.parentRepairSession(), verb, endpoint); + // todo: at some point we should make repair messages follow the normal path, actually using this + } + + @Override public void onFailure(InetAddressAndPort from, RequestFailureReason failureReason) { - logger.error("[#{}] {} failed on {}: {}", request.desc.parentSessionId, verb, from, failureReason); + failureCallback.onFailure(RepairException.error(request.desc, PreviewKind.NONE, String.format("Got %s failure from %s: %s", verb, from, failureReason))); + } - if (supportsTimeouts(from, request.desc.parentSessionId)) - failureCallback.onFailure(RepairException.error(request.desc, PreviewKind.NONE, String.format("Got %s failure from %s: %s", verb, from, failureReason))); + @Override + public boolean invokeOnFailure() + { + return true; } }; - - MessagingService.instance().sendWithCallback(Message.outWithFlag(verb, request, CALL_BACK_ON_FAILURE), - endpoint, - callback); + sendMessageWithRetries(ctx, allowRetry, request, verb, endpoint, callback); } - private static boolean supportsTimeouts(InetAddressAndPort from, TimeUUID parentSessionId) + private static ErrorHandling errorHandlingSupported(SharedContext ctx, InetAddressAndPort from, Verb verb, TimeUUID parentSessionId) { - CassandraVersion remoteVersion = Gossiper.instance.getReleaseVersion(from); - if (remoteVersion != null && remoteVersion.compareTo(SUPPORTS_TIMEOUTS) >= 0) - return true; - logger.warn("[#{}] Not failing repair due to remote host {} not supporting repair message timeouts (version = {})", parentSessionId, from, remoteVersion); - return false; + if (SUPPORTS_RETRY_WITHOUT_VERSION_CHECK.contains(verb)) + return ErrorHandling.RETRY; + // Repair in mixed mode isn't fully supported, but also not activally blocked... so in the common case all participants + // will be on the same version as this instance, so can avoid the lookup from gossip + CassandraVersion remoteVersion = ctx.gossiper().getReleaseVersion(from); + if (remoteVersion == null) + { + if (VERB_TIMEOUT_VERSIONS.containsKey(verb)) + { + logger.warn("[#{}] Not failing repair due to remote host {} not supporting repair message timeouts (version is unknown)", parentSessionId, from); + return ErrorHandling.NONE; + } + return ErrorHandling.TIMEOUT; + } + if (remoteVersion.compareTo(SUPPORTS_RETRY) >= 0) return ErrorHandling.RETRY; + CassandraVersion timeoutVersion = VERB_TIMEOUT_VERSIONS.get(verb); + if (timeoutVersion == null || Review Comment: nit; code style ########## test/unit/org/apache/cassandra/repair/FuzzTestBase.java: ########## @@ -0,0 +1,1301 @@ +/* + * 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.apache.cassandra.config.UnitConfigOverride; +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.compaction.ICompactionManager; +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.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 int MISMATCH_NUM_PARTITIONS = 1; + 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()); + // when running in CI an external actor will replace the test configs based off the test type (such as trie, cdc, etc.), this could then have failing tests + // that do not repo with the same seed! To fix that, go to UnitConfigOverride and update the config type to match the one that failed in CI, this should then + // use the same config, so the seed should not reproduce. + UnitConfigOverride.maybeOverrideConfig(); + + 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; Review Comment: nit: `return caller.getClassName().startsWith("org.apache.cassandra.streaming.");` -- 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]

