Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
11 changes: 7 additions & 4 deletions ratis-common/src/main/java/org/apache/ratis/util/LifeCycle.java
Original file line number Diff line number Diff line change
Expand Up @@ -273,15 +273,18 @@ public String toString() {
/** Run the given start method and transition the current state accordingly. */
@SafeVarargs
public final <T extends Throwable> void startAndTransition(
CheckedRunnable<T> startImpl, Class<? extends Throwable>... exceptionClasses)
CheckedRunnable<T> startMethod, Class<? extends Throwable>... exceptionClasses)
throws T {
transition(State.STARTING);
try {
startImpl.run();
startMethod.run();
transition(State.RUNNING);
} catch (Throwable t) {
transition(ReflectionUtils.isInstance(t, exceptionClasses)?
State.NEW: State.EXCEPTION);
final State state = getCurrentState();
LOG.warn("{}: Failed to start (state={})", name, state, t);
if (!state.isClosingOrClosed()) {
transition(ReflectionUtils.isInstance(t, exceptionClasses) ? State.NEW : State.EXCEPTION);
}
throw t;
}
}
Expand Down
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
/**
/*
* 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
Expand All @@ -25,10 +25,13 @@
import java.util.EnumMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;

import static org.apache.ratis.util.LifeCycle.State.*;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertSame;
import static org.junit.jupiter.api.Assertions.assertTrue;


Expand Down Expand Up @@ -101,4 +104,68 @@ private static void testInvalidTransition(TriConsumer<LifeCycle.State, LifeCycle
}
}

@Test
public void testStartAndTransition() throws Exception {
final SimulatedServer simulatedServer = new SimulatedServer();
assertEquals(NEW, simulatedServer.getLifeCycleState());

final CompletableFuture<Throwable> f = CompletableFuture.supplyAsync(() -> {
try {
simulatedServer.start();
throw new AssertionError("start() should fail");
} catch (Exception e) {
return e.getCause();
}
});

Thread.sleep(100);
assertEquals(STARTING, simulatedServer.getLifeCycleState());
Comment on lines +121 to +122
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Would using JavaUtils.attemptUntilTrue be better than Thread.sleep?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Since the test is very simple, it is reliable to just use Thread.sleep.

An embarrassing reason is that TestRaftAsyncWithNetty often fails. It is not easy to pass all the tests. We probably should disable TestRaftAsyncWithNetty until it is fixed.


// call close() during STARTING, start() should throw the simulated exception
CompletableFuture.supplyAsync(simulatedServer::close);
assertSame(simulatedServer.getSimulatedException(), f.get());

assertEquals(CLOSING, simulatedServer.getLifeCycleState());
simulatedServer.getCloseFuture().complete(null);
Thread.sleep(100);
assertEquals(CLOSED, simulatedServer.getLifeCycleState());
}

private static final class SimulatedServer {
private final LifeCycle lifeCycle = new LifeCycle(getClass().getSimpleName());
private final Exception simulatedException = new Exception("Simulated exception");
private final CompletableFuture<Void> startFuture = new CompletableFuture<>();
private final CompletableFuture<Void> closeFuture = new CompletableFuture<>();

LifeCycle.State getLifeCycleState() {
return lifeCycle.getCurrentState();
}

Exception getSimulatedException() {
return simulatedException;
}

CompletableFuture<Void> getCloseFuture() {
return closeFuture;
}

void start() throws Exception {
lifeCycle.startAndTransition(this::startImpl);
}

void startImpl() throws Exception {
startFuture.get();
}

Void close() {
// simulate close and then cause start() to fail.
lifeCycle.checkStateAndClose(this::closeImpl);
return null;
}

void closeImpl() {
startFuture.completeExceptionally(simulatedException);
closeFuture.join();
}
}
}