Skip to content

Commit

Permalink
RATIS-2088. Fix name and imports related check in test-directory (apa…
Browse files Browse the repository at this point in the history
  • Loading branch information
symious authored and szetszwo committed Jun 16, 2024
1 parent 7b85a65 commit e9a9b16
Show file tree
Hide file tree
Showing 30 changed files with 114 additions and 123 deletions.
4 changes: 2 additions & 2 deletions ratis-common/src/test/java/org/apache/ratis/BaseTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -124,7 +124,7 @@ public int getGlobalTimeoutSeconds() {
return 100;
}

private static final Supplier<File> rootTestDir = JavaUtils.memoize(
private static final Supplier<File> ROOT_TEST_DIR = JavaUtils.memoize(
() -> JavaUtils.callAsUnchecked(() -> {
final File dir = new File(System.getProperty("test.build.data", "target/test/data"),
Integer.toHexString(ThreadLocalRandom.current().nextInt()));
Expand All @@ -138,7 +138,7 @@ public int getGlobalTimeoutSeconds() {


public static File getRootTestDir() {
return rootTestDir.get();
return ROOT_TEST_DIR.get();
}

public File getClassTestDir() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,6 @@
import java.util.Set;

import org.apache.ratis.thirdparty.com.google.common.collect.Lists;
import org.apache.ratis.util.RefCountingMap;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,11 +48,11 @@ public static Collection<Object[]> data() {
}

/** For {@link ParameterizedTest} so that a cluster can be shared by multiple {@link Test} */
private static final AtomicReference<MiniRaftCluster> currentCluster = new AtomicReference<>();
private static final AtomicReference<MiniRaftCluster> CURRENT_CLUSTER = new AtomicReference<>();

/** Set {@link #currentCluster} to the given cluster and start it if {@link #currentCluster} is changed. */
/** Set {@link #CURRENT_CLUSTER} to the given cluster and start it if {@link #CURRENT_CLUSTER} is changed. */
public static void setAndStart(MiniRaftCluster cluster) throws InterruptedException, IOException {
final MiniRaftCluster previous = currentCluster.getAndSet(cluster);
final MiniRaftCluster previous = CURRENT_CLUSTER.getAndSet(cluster);
if (previous != cluster) {
if (previous != null) {
previous.shutdown();
Expand All @@ -65,7 +65,7 @@ public static void setAndStart(MiniRaftCluster cluster) throws InterruptedExcept

@AfterAll
public static void shutdownCurrentCluster() {
final MiniRaftCluster cluster = currentCluster.getAndSet(null);
final MiniRaftCluster cluster = CURRENT_CLUSTER.getAndSet(null);
if (cluster != null) {
cluster.shutdown();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,17 +47,17 @@ public class TestArithmeticLogDump extends BaseTest {

public static final int NUM_SERVERS = 1;

protected static final RaftProperties properties = new RaftProperties();
protected static final RaftProperties PROPERTIES = new RaftProperties();

private final MiniRaftClusterWithSimulatedRpc cluster = MiniRaftClusterWithSimulatedRpc
.FACTORY.newCluster(NUM_SERVERS, getProperties());

public RaftProperties getProperties() {
RaftServerConfigKeys.Rpc
.setSlownessTimeout(properties, TimeDuration.valueOf(1, TimeUnit.SECONDS));
properties.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY,
.setSlownessTimeout(PROPERTIES, TimeDuration.valueOf(1, TimeUnit.SECONDS));
PROPERTIES.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY,
SimpleStateMachine4Testing.class, StateMachine.class);
return properties;
return PROPERTIES;
}

@BeforeEach
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,12 +21,9 @@

import java.util.Collection;
import java.util.Collections;
import org.junit.Test;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.MethodSource;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;

public class TestSubCommand {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ default Factory<MiniRaftClusterWithGrpc> getFactory() {
}
}

public static final DelayLocalExecutionInjection sendServerRequestInjection =
public static final DelayLocalExecutionInjection SEND_SERVER_REQUEST_INJECTION =
new DelayLocalExecutionInjection(GrpcService.GRPC_SEND_SERVER_REQUEST);

protected MiniRaftClusterWithGrpc(String[] ids, String[] listenerIds, RaftProperties properties, Parameters parameters) {
Expand All @@ -72,7 +72,7 @@ protected Parameters setPropertiesAndInitParameters(RaftPeerId id, RaftGroup gro
@Override
protected void blockQueueAndSetDelay(String leaderId, int delayMs)
throws InterruptedException {
RaftTestUtil.blockQueueAndSetDelay(getServers(), sendServerRequestInjection,
RaftTestUtil.blockQueueAndSetDelay(getServers(), SEND_SERVER_REQUEST_INJECTION,
leaderId, delayMs, getTimeoutMax());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@ default Factory<MiniRaftClusterWithNetty> getFactory() {
}
}

public static final DelayLocalExecutionInjection sendServerRequest
public static final DelayLocalExecutionInjection SEND_SERVER_REQUEST
= new DelayLocalExecutionInjection(NettyRpcService.SEND_SERVER_REQUEST);

protected MiniRaftClusterWithNetty(String[] ids, String[] listenerIds, RaftProperties properties) {
Expand All @@ -64,7 +64,7 @@ protected Parameters setPropertiesAndInitParameters(RaftPeerId id, RaftGroup gro
@Override
protected void blockQueueAndSetDelay(String leaderId, int delayMs)
throws InterruptedException {
RaftTestUtil.blockQueueAndSetDelay(getServers(), sendServerRequest,
RaftTestUtil.blockQueueAndSetDelay(getServers(), SEND_SERVER_REQUEST,
leaderId, delayMs, getTimeoutMax());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -83,7 +83,7 @@ public abstract class InstallSnapshotNotificationTests<CLUSTER extends MiniRaftC

private static final int SNAPSHOT_TRIGGER_THRESHOLD = 64;
private static final int PURGE_GAP = 8;
private static final AtomicReference<SnapshotInfo> leaderSnapshotInfoRef = new AtomicReference<>();
private static final AtomicReference<SnapshotInfo> LEADER_SNAPSHOT_INFO_REF = new AtomicReference<>();

private static final AtomicInteger numSnapshotRequests = new AtomicInteger();
private static final AtomicInteger numNotifyInstallSnapshotFinished = new AtomicInteger();
Expand All @@ -102,7 +102,7 @@ public CompletableFuture<TermIndex> notifyInstallSnapshotFromLeader(
}
numSnapshotRequests.incrementAndGet();

final SingleFileSnapshotInfo leaderSnapshotInfo = (SingleFileSnapshotInfo) leaderSnapshotInfoRef.get();
final SingleFileSnapshotInfo leaderSnapshotInfo = (SingleFileSnapshotInfo) LEADER_SNAPSHOT_INFO_REF.get();
LOG.info("{}: leaderSnapshotInfo = {}", getId(), leaderSnapshotInfo);
if (leaderSnapshotInfo == null) {
return super.notifyInstallSnapshotFromLeader(roleInfoProto, termIndex);
Expand Down Expand Up @@ -137,7 +137,7 @@ public void notifySnapshotInstalled(RaftProtos.InstallSnapshotResult result, lon
return;
}
numNotifyInstallSnapshotFinished.incrementAndGet();
final SingleFileSnapshotInfo leaderSnapshotInfo = (SingleFileSnapshotInfo) leaderSnapshotInfoRef.get();
final SingleFileSnapshotInfo leaderSnapshotInfo = (SingleFileSnapshotInfo) LEADER_SNAPSHOT_INFO_REF.get();
File leaderSnapshotFile = leaderSnapshotInfo.getFile().getPath().toFile();
synchronized (this) {
try {
Expand Down Expand Up @@ -184,7 +184,7 @@ public void testAddNewFollowersNoSnapshot() throws Exception {

private void testAddNewFollowers(CLUSTER cluster, int numRequests) throws Exception {
final boolean shouldInstallSnapshot = numRequests >= SNAPSHOT_TRIGGER_THRESHOLD;
leaderSnapshotInfoRef.set(null);
LEADER_SNAPSHOT_INFO_REF.set(null);
final List<LogSegmentPath> logs;
int i = 0;
try {
Expand Down Expand Up @@ -236,7 +236,7 @@ private void testAddNewFollowers(CLUSTER cluster, int numRequests) throws Except

final SnapshotInfo leaderSnapshotInfo = cluster.getLeader().getStateMachine().getLatestSnapshot();
LOG.info("LeaderSnapshotInfo: {}", leaderSnapshotInfo.getTermIndex());
final boolean set = leaderSnapshotInfoRef.compareAndSet(null, leaderSnapshotInfo);
final boolean set = LEADER_SNAPSHOT_INFO_REF.compareAndSet(null, leaderSnapshotInfo);
Assert.assertTrue(set);

// add two more peers
Expand Down Expand Up @@ -271,7 +271,7 @@ public void testRestartFollower() throws Exception {
}

private void testRestartFollower(CLUSTER cluster) throws Exception {
leaderSnapshotInfoRef.set(null);
LEADER_SNAPSHOT_INFO_REF.set(null);
int i = 0;
final RaftServer.Division leader = RaftTestUtil.waitForLeader(cluster);
final RaftPeerId leaderId = leader.getId();
Expand Down Expand Up @@ -321,7 +321,7 @@ public void testInstallSnapshotNotificationCount() throws Exception {


private void testInstallSnapshotNotificationCount(CLUSTER cluster) throws Exception {
leaderSnapshotInfoRef.set(null);
LEADER_SNAPSHOT_INFO_REF.set(null);
numSnapshotRequests.set(0);

int i = 0;
Expand Down Expand Up @@ -352,7 +352,7 @@ private void testInstallSnapshotNotificationCount(CLUSTER cluster) throws Except
Assert.assertEquals(20, snapshotIndex);
final SnapshotInfo leaderSnapshotInfo = cluster.getLeader().getStateMachine().getLatestSnapshot();
Assert.assertEquals(20, leaderSnapshotInfo.getIndex());
final boolean set = leaderSnapshotInfoRef.compareAndSet(null, leaderSnapshotInfo);
final boolean set = LEADER_SNAPSHOT_INFO_REF.compareAndSet(null, leaderSnapshotInfo);
Assert.assertTrue(set);

// Wait for the snapshot to be done.
Expand Down Expand Up @@ -425,7 +425,7 @@ public void testInstallSnapshotInstalledEvent() throws Exception{
}

private void testInstallSnapshotInstalledEvent(CLUSTER cluster) throws Exception{
leaderSnapshotInfoRef.set(null);
LEADER_SNAPSHOT_INFO_REF.set(null);
numNotifyInstallSnapshotFinished.set(0);
final List<LogSegmentPath> logs;
int i = 0;
Expand Down Expand Up @@ -475,7 +475,7 @@ private void testInstallSnapshotInstalledEvent(CLUSTER cluster) throws Exception

final SnapshotInfo leaderSnapshotInfo = cluster.getLeader().getStateMachine().getLatestSnapshot();
LOG.info("LeaderSnapshotInfo: {}", leaderSnapshotInfo.getTermIndex());
final boolean set = leaderSnapshotInfoRef.compareAndSet(null, leaderSnapshotInfo);
final boolean set = LEADER_SNAPSHOT_INFO_REF.compareAndSet(null, leaderSnapshotInfo);
Assert.assertTrue(set);

// add one new peer
Expand Down Expand Up @@ -524,7 +524,7 @@ public void testInstallSnapshotDuringBootstrap() throws Exception {
}

private void testInstallSnapshotDuringBootstrap(CLUSTER cluster) throws Exception {
leaderSnapshotInfoRef.set(null);
LEADER_SNAPSHOT_INFO_REF.set(null);
numSnapshotRequests.set(0);
int i = 0;
try {
Expand Down Expand Up @@ -553,7 +553,7 @@ private void testInstallSnapshotDuringBootstrap(CLUSTER cluster) throws Exceptio
RaftSnapshotBaseTest.assertLeaderContent(cluster);

final SnapshotInfo leaderSnapshotInfo = cluster.getLeader().getStateMachine().getLatestSnapshot();
final boolean set = leaderSnapshotInfoRef.compareAndSet(null, leaderSnapshotInfo);
final boolean set = LEADER_SNAPSHOT_INFO_REF.compareAndSet(null, leaderSnapshotInfo);
Assert.assertTrue(set);

// add two more peers
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,13 +22,11 @@
import org.apache.ratis.proto.RaftProtos.LogEntryProto.LogEntryBodyCase;
import org.apache.ratis.server.RaftServer;
import org.apache.ratis.server.impl.MiniRaftCluster;
import org.apache.ratis.server.protocol.TermIndex;
import org.apache.ratis.server.raftlog.LogEntryHeader;
import org.apache.ratis.server.raftlog.RaftLog;
import org.apache.ratis.util.SizeInBytes;
import org.apache.ratis.util.StringUtils;
import org.junit.Assert;
import org.junit.Ignore;
import org.junit.Test;

import java.io.IOException;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -104,7 +104,7 @@ private void runTestTimeoutException(CLUSTER cluster) throws Exception {
final RaftClientReply reply = client.io().send(new SimpleMessage("m0"));
Assert.assertTrue(reply.isSuccess());

RaftClientConfigKeys.Rpc.setRequestTimeout(properties.get(), ONE_SECOND);
RaftClientConfigKeys.Rpc.setRequestTimeout(PROPERTIES.get(), ONE_SECOND);
// Block StartTransaction
StreamSupport.stream(cluster.getServers().spliterator(), false)
.map(cluster::getDivision)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,7 @@ public abstract class RaftAsyncTests<CLUSTER extends MiniRaftCluster> extends Ba

public static final int NUM_SERVERS = 3;

private static final DelayLocalExecutionInjection logSyncDelay = RaftServerTestUtil.getLogSyncDelay();
private static final DelayLocalExecutionInjection LOG_SYNC_DELAY = RaftServerTestUtil.getLogSyncDelay();

{
getProperties().setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY,
Expand Down Expand Up @@ -430,7 +430,7 @@ void runTestCheckLeadershipFailure(CLUSTER cluster) throws Exception {
cluster.getServerAliveStream()
.filter(impl -> !impl.getInfo().isLeader())
.map(SimpleStateMachine4Testing::get)
.forEach(peer -> logSyncDelay.setDelayMs(peer.getId().toString(), 1000));
.forEach(peer -> LOG_SYNC_DELAY.setDelayMs(peer.getId().toString(), 1000));

// trigger append entries request
client.async().send(new SimpleMessage("abc"));
Expand All @@ -446,7 +446,7 @@ void runTestCheckLeadershipFailure(CLUSTER cluster) throws Exception {

} finally {
// unblock append entries request
logSyncDelay.clear();
LOG_SYNC_DELAY.clear();
}

waitForLeader(cluster);
Expand Down
6 changes: 3 additions & 3 deletions ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java
Original file line number Diff line number Diff line change
Expand Up @@ -405,8 +405,8 @@ public ByteString getContent() {
}

class SimpleOperation {
private static final ClientId clientId = ClientId.randomId();
private static final AtomicLong callId = new AtomicLong();
private static final ClientId CLIENT_ID = ClientId.randomId();
private static final AtomicLong CALL_ID = new AtomicLong();

private final String op;
private final StateMachineLogEntryProto smLogEntryProto;
Expand All @@ -416,7 +416,7 @@ public SimpleOperation(String op) {
}

public SimpleOperation(String op, boolean hasStateMachineData) {
this(clientId, callId.incrementAndGet(), op, hasStateMachineData);
this(CLIENT_ID, CALL_ID.incrementAndGet(), op, hasStateMachineData);
}

private SimpleOperation(ClientId clientId, long callId, String op, boolean hasStateMachineData) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,14 +49,12 @@
import org.slf4j.event.Level;

import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import java.util.stream.Stream;

import static org.junit.Assert.fail;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -63,20 +63,20 @@ public abstract class GroupManagementBaseTest extends BaseTest {
Slf4jUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
}

static final RaftProperties prop = new RaftProperties();
static final RaftProperties PROP = new RaftProperties();

static {
// avoid flaky behaviour in CI environment
RaftServerConfigKeys.Rpc.setTimeoutMin(prop, TimeDuration.valueOf(1500, TimeUnit.MILLISECONDS));
RaftServerConfigKeys.Rpc.setTimeoutMax(prop, TimeDuration.valueOf(2000, TimeUnit.MILLISECONDS));
RaftServerConfigKeys.Rpc.setTimeoutMin(PROP, TimeDuration.valueOf(1500, TimeUnit.MILLISECONDS));
RaftServerConfigKeys.Rpc.setTimeoutMax(PROP, TimeDuration.valueOf(2000, TimeUnit.MILLISECONDS));
// it takes 5s+ to finish the blocking group add call
RaftClientConfigKeys.Rpc.setRequestTimeout(prop, TimeDuration.valueOf(12, TimeUnit.SECONDS));
RaftClientConfigKeys.Rpc.setRequestTimeout(PROP, TimeDuration.valueOf(12, TimeUnit.SECONDS));
}

public abstract MiniRaftCluster.Factory<? extends MiniRaftCluster> getClusterFactory();

public MiniRaftCluster getCluster(int peerNum) {
return getClusterFactory().newCluster(peerNum, prop);
return getClusterFactory().newCluster(peerNum, PROP);
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -92,14 +92,14 @@ public abstract class MiniRaftCluster implements Closeable {
TimeDuration.valueOf(100, TimeUnit.MILLISECONDS);
static final AtomicInteger THREAD_COUNT = new AtomicInteger(0);

public static abstract class Factory<CLUSTER extends MiniRaftCluster> {
public abstract static class Factory<CLUSTER extends MiniRaftCluster> {
public interface Get<CLUSTER extends MiniRaftCluster> {
Supplier<RaftProperties> properties = JavaUtils.memoize(RaftProperties::new);
Supplier<RaftProperties> PROPERTIES = JavaUtils.memoize(RaftProperties::new);

Factory<CLUSTER> getFactory();

default RaftProperties getProperties() {
return properties.get();
return PROPERTIES.get();
}

default RaftProperties setStateMachine(Class<? extends StateMachine> stateMachineClass) {
Expand Down Expand Up @@ -198,7 +198,7 @@ public CLUSTER newCluster(int numServer, int numListeners, RaftProperties prop)
}
}

public static abstract class RpcBase extends MiniRaftCluster {
public abstract static class RpcBase extends MiniRaftCluster {
public RpcBase(String[] ids, String[] listenerIds, RaftProperties properties, Parameters parameters) {
super(ids, listenerIds, properties, parameters);
}
Expand Down
Loading

0 comments on commit e9a9b16

Please sign in to comment.