maedhroz commented on code in PR #2660: URL: https://github.com/apache/cassandra/pull/2660#discussion_r1320061661
########## test/unit/org/apache/cassandra/repair/FuzzTest.java: ########## @@ -0,0 +1,1453 @@ +/* + * 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 + 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; Review Comment: Could just be... ``` 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]

