Skip to content

Commit 50c6e7d

Browse files
authored
RATIS-2344. Remove the unused mock from StateMachineShutdownTests. (#1299)
1 parent 047444b commit 50c6e7d

File tree

1 file changed

+1
-19
lines changed

1 file changed

+1
-19
lines changed

ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java

Lines changed: 1 addition & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -28,12 +28,8 @@
2828
import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing;
2929
import org.apache.ratis.statemachine.StateMachine;
3030
import org.apache.ratis.statemachine.TransactionContext;
31-
import org.junit.jupiter.api.AfterEach;
3231
import org.junit.jupiter.api.Assertions;
33-
import org.junit.jupiter.api.BeforeEach;
3432
import org.junit.jupiter.api.Test;
35-
import org.mockito.MockedStatic;
36-
import org.mockito.Mockito;
3733
import org.slf4j.Logger;
3834
import org.slf4j.LoggerFactory;
3935

@@ -51,7 +47,6 @@ public abstract class StateMachineShutdownTests<CLUSTER extends MiniRaftCluster>
5147
extends BaseTest
5248
implements MiniRaftCluster.Factory.Get<CLUSTER> {
5349
public static Logger LOG = LoggerFactory.getLogger(StateMachineUpdater.class);
54-
private static MockedStatic<CompletableFuture> mocked;
5550
protected static class StateMachineWithConditionalWait extends
5651
SimpleStateMachine4Testing {
5752
boolean unblockAllTxns = false;
@@ -124,19 +119,6 @@ public void unblockAllTxns() {
124119
}
125120
}
126121

127-
@BeforeEach
128-
public void setup() {
129-
mocked = Mockito.mockStatic(CompletableFuture.class, Mockito.CALLS_REAL_METHODS);
130-
}
131-
132-
@AfterEach
133-
public void tearDownClass() {
134-
if (mocked != null) {
135-
mocked.close();
136-
}
137-
138-
}
139-
140122
@Test
141123
public void testStateMachineShutdownWaitsForApplyTxn() throws Exception {
142124
final RaftProperties prop = getProperties();
@@ -195,7 +177,7 @@ public void testStateMachineShutdownWaitsForApplyTxn() throws Exception {
195177

196178
// Now wait for the thread
197179
t.join(5000);
198-
Assertions.assertEquals(logIndex, secondFollower.getInfo().getLastAppliedIndex());
180+
Assertions.assertTrue(logIndex <= secondFollower.getInfo().getLastAppliedIndex());
199181
Assertions.assertEquals(3, StateMachineWithConditionalWait.numTxns.values().stream()
200182
.filter(val -> val.get() == 3).count());
201183

0 commit comments

Comments
 (0)