Zen2: Add node id to log output of CoordinatorTests (#33929)
With recent changes to the logging framework, the node name can no longer be injected into the logging output using the node.name setting, which means that for the CoordinatorTests (which are simulating a cluster in a fully deterministic fashion using a single thread), as all the different nodes are running under the same test thread, we are not able to distinguish which log lines are coming from which node. This commit readds logging for node ids in the CoordinatorTests, making two very small changes to DeterministicTaskQueue and TestThreadInfoPatternConverter.
This commit is contained in:
parent
187f787f52
commit
a612dd1272
|
@ -557,10 +557,16 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery
|
|||
: "[" + currentPublication.get() + "] in progress, cannot start [" + publication + ']';
|
||||
currentPublication = Optional.of(publication);
|
||||
|
||||
transportService.getThreadPool().schedule(publishTimeout, Names.GENERIC, () -> {
|
||||
synchronized (mutex) {
|
||||
transportService.getThreadPool().schedule(publishTimeout, Names.GENERIC, new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
publication.onTimeout();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "scheduled timeout for " + publication;
|
||||
}
|
||||
});
|
||||
publication.start(Collections.emptySet()); // TODO start failure detector and put faultyNodes here
|
||||
}
|
||||
|
@ -625,15 +631,23 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery
|
|||
if (foundQuorum) {
|
||||
if (electionScheduler == null) {
|
||||
final TimeValue gracePeriod = TimeValue.ZERO; // TODO variable grace period
|
||||
electionScheduler = electionSchedulerFactory.startElectionScheduler(gracePeriod, () -> {
|
||||
synchronized (mutex) {
|
||||
if (mode == Mode.CANDIDATE) {
|
||||
if (prevotingRound != null) {
|
||||
prevotingRound.close();
|
||||
electionScheduler = electionSchedulerFactory.startElectionScheduler(gracePeriod, new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
synchronized (mutex) {
|
||||
if (mode == Mode.CANDIDATE) {
|
||||
if (prevotingRound != null) {
|
||||
prevotingRound.close();
|
||||
}
|
||||
prevotingRound = preVoteCollector.start(lastAcceptedState, getDiscoveredNodes());
|
||||
}
|
||||
prevotingRound = preVoteCollector.start(lastAcceptedState, getDiscoveredNodes());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "scheduling of new prevoting round";
|
||||
}
|
||||
});
|
||||
}
|
||||
} else {
|
||||
|
|
|
@ -82,14 +82,22 @@ public final class ListenableFuture<V> extends BaseFuture<V> implements ActionLi
|
|||
|
||||
private void notifyListener(ActionListener<V> listener, ExecutorService executorService) {
|
||||
try {
|
||||
executorService.submit(() -> {
|
||||
try {
|
||||
// call get in a non-blocking fashion as we could be on a network thread
|
||||
// or another thread like the scheduler, which we should never block!
|
||||
V value = FutureUtils.get(this, 0L, TimeUnit.NANOSECONDS);
|
||||
listener.onResponse(value);
|
||||
} catch (Exception e) {
|
||||
listener.onFailure(e);
|
||||
executorService.submit(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
// call get in a non-blocking fashion as we could be on a network thread
|
||||
// or another thread like the scheduler, which we should never block!
|
||||
V value = FutureUtils.get(ListenableFuture.this, 0L, TimeUnit.NANOSECONDS);
|
||||
listener.onResponse(value);
|
||||
} catch (Exception e) {
|
||||
listener.onFailure(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "ListenableFuture notification";
|
||||
}
|
||||
});
|
||||
} catch (Exception e) {
|
||||
|
|
|
@ -23,5 +23,10 @@ public abstract class TransportResponse extends TransportMessage {
|
|||
|
||||
public static class Empty extends TransportResponse {
|
||||
public static final Empty INSTANCE = new Empty();
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "Empty{}";
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,6 +18,7 @@
|
|||
*/
|
||||
package org.elasticsearch.cluster.coordination;
|
||||
|
||||
import org.apache.logging.log4j.CloseableThreadContext;
|
||||
import org.apache.logging.log4j.message.ParameterizedMessage;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
|
@ -38,13 +39,8 @@ import org.elasticsearch.common.transport.TransportAddress;
|
|||
import org.elasticsearch.discovery.zen.UnicastHostsProvider.HostsResolver;
|
||||
import org.elasticsearch.indices.cluster.FakeThreadPoolMasterService;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.test.disruption.DisruptableMockTransport;
|
||||
import org.elasticsearch.test.junit.annotations.TestLogging;
|
||||
import org.elasticsearch.test.transport.MockTransport;
|
||||
import org.elasticsearch.transport.RequestHandlerRegistry;
|
||||
import org.elasticsearch.transport.TransportChannel;
|
||||
import org.elasticsearch.transport.TransportRequest;
|
||||
import org.elasticsearch.transport.TransportResponse;
|
||||
import org.elasticsearch.transport.TransportResponseOptions;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.hamcrest.Matcher;
|
||||
|
||||
|
@ -55,7 +51,6 @@ import java.util.HashSet;
|
|||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.function.Consumer;
|
||||
import java.util.function.Predicate;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
|
@ -184,7 +179,7 @@ public class CoordinatorTests extends ESTestCase {
|
|||
private final PersistedState persistedState;
|
||||
private MasterService masterService;
|
||||
private TransportService transportService;
|
||||
private MockTransport mockTransport;
|
||||
private DisruptableMockTransport mockTransport;
|
||||
|
||||
ClusterNode(int nodeIndex) {
|
||||
super(Settings.builder().put(NODE_NAME_SETTING.getKey(), nodeIdFromIndex(nodeIndex)).build());
|
||||
|
@ -192,7 +187,7 @@ public class CoordinatorTests extends ESTestCase {
|
|||
localNode = createDiscoveryNode();
|
||||
persistedState = new InMemoryPersistedState(1L,
|
||||
clusterState(1L, 1L, localNode, initialConfiguration, initialConfiguration, 0L));
|
||||
setUp();
|
||||
onNode(localNode, this::setUp).run();
|
||||
}
|
||||
|
||||
private DiscoveryNode createDiscoveryNode() {
|
||||
|
@ -206,112 +201,44 @@ public class CoordinatorTests extends ESTestCase {
|
|||
}
|
||||
|
||||
private void setUp() {
|
||||
mockTransport = new MockTransport() {
|
||||
mockTransport = new DisruptableMockTransport(logger) {
|
||||
@Override
|
||||
protected void onSendRequest(long requestId, String action, TransportRequest request, DiscoveryNode destination) {
|
||||
assert destination.equals(localNode) == false : "non-local message from " + localNode + " to itself";
|
||||
super.onSendRequest(requestId, action, request, destination);
|
||||
protected DiscoveryNode getLocalNode() {
|
||||
return localNode;
|
||||
}
|
||||
|
||||
// connecting and handshaking with a new node happens synchronously, so we cannot enqueue these tasks for later
|
||||
final Consumer<Runnable> scheduler;
|
||||
@Override
|
||||
protected ConnectionStatus getConnectionStatus(DiscoveryNode sender, DiscoveryNode destination) {
|
||||
return ConnectionStatus.CONNECTED;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Optional<DisruptableMockTransport> getDisruptedCapturingTransport(DiscoveryNode node, String action) {
|
||||
final Predicate<ClusterNode> matchesDestination;
|
||||
if (action.equals(HANDSHAKE_ACTION_NAME)) {
|
||||
scheduler = Runnable::run;
|
||||
matchesDestination = n -> n.getLocalNode().getAddress().equals(destination.getAddress());
|
||||
matchesDestination = n -> n.getLocalNode().getAddress().equals(node.getAddress());
|
||||
} else {
|
||||
scheduler = deterministicTaskQueue::scheduleNow;
|
||||
matchesDestination = n -> n.getLocalNode().equals(destination);
|
||||
matchesDestination = n -> n.getLocalNode().equals(node);
|
||||
}
|
||||
return clusterNodes.stream().filter(matchesDestination).findAny().map(cn -> cn.mockTransport);
|
||||
}
|
||||
|
||||
scheduler.accept(new Runnable() {
|
||||
@Override
|
||||
public String toString() {
|
||||
return "delivery of [" + action + "][" + requestId + "]: " + request;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
clusterNodes.stream().filter(matchesDestination).findAny().ifPresent(
|
||||
destinationNode -> {
|
||||
|
||||
final RequestHandlerRegistry requestHandler
|
||||
= destinationNode.mockTransport.getRequestHandler(action);
|
||||
|
||||
final TransportChannel transportChannel = new TransportChannel() {
|
||||
@Override
|
||||
public String getProfileName() {
|
||||
return "default";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getChannelType() {
|
||||
return "coordinator-test-channel";
|
||||
}
|
||||
|
||||
@Override
|
||||
public void sendResponse(final TransportResponse response) {
|
||||
scheduler.accept(new Runnable() {
|
||||
@Override
|
||||
public String toString() {
|
||||
return "delivery of response " + response
|
||||
+ " to [" + action + "][" + requestId + "]: " + request;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
handleResponse(requestId, response);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public void sendResponse(TransportResponse response, TransportResponseOptions options) {
|
||||
sendResponse(response);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void sendResponse(Exception exception) {
|
||||
scheduler.accept(new Runnable() {
|
||||
@Override
|
||||
public String toString() {
|
||||
return "delivery of error response " + exception.getMessage()
|
||||
+ " to [" + action + "][" + requestId + "]: " + request;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
handleRemoteError(requestId, exception);
|
||||
}
|
||||
});
|
||||
}
|
||||
};
|
||||
|
||||
try {
|
||||
processMessageReceived(request, requestHandler, transportChannel);
|
||||
} catch (Exception e) {
|
||||
scheduler.accept(new Runnable() {
|
||||
@Override
|
||||
public String toString() {
|
||||
return "delivery of processing error response " + e.getMessage()
|
||||
+ " to [" + action + "][" + requestId + "]: " + request;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
handleRemoteError(requestId, e);
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
});
|
||||
@Override
|
||||
protected void handle(DiscoveryNode sender, DiscoveryNode destination, String action, Runnable doDelivery) {
|
||||
// handshake needs to run inline as the caller blockingly waits on the result
|
||||
if (action.equals(HANDSHAKE_ACTION_NAME)) {
|
||||
onNode(destination, doDelivery).run();
|
||||
} else {
|
||||
deterministicTaskQueue.scheduleNow(onNode(destination, doDelivery));
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
masterService = new FakeThreadPoolMasterService("test", deterministicTaskQueue::scheduleNow);
|
||||
masterService = new FakeThreadPoolMasterService("test",
|
||||
runnable -> deterministicTaskQueue.scheduleNow(onNode(localNode, runnable)));
|
||||
transportService = mockTransport.createTransportService(
|
||||
settings, deterministicTaskQueue.getThreadPool(), NOOP_TRANSPORT_INTERCEPTOR, a -> localNode, null, emptySet());
|
||||
settings, deterministicTaskQueue.getThreadPool(runnable -> onNode(localNode, runnable)), NOOP_TRANSPORT_INTERCEPTOR,
|
||||
a -> localNode, null, emptySet());
|
||||
coordinator = new Coordinator(settings, transportService, ESAllocationTestCase.createAllocationService(Settings.EMPTY),
|
||||
masterService, this::getPersistedState, Cluster.this::provideUnicastHosts, Randomness.get());
|
||||
masterService.setClusterStatePublisher(coordinator);
|
||||
|
@ -359,9 +286,20 @@ public class CoordinatorTests extends ESTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
private static void processMessageReceived(TransportRequest request, RequestHandlerRegistry requestHandler,
|
||||
TransportChannel transportChannel) throws Exception {
|
||||
requestHandler.processMessageReceived(request, transportChannel);
|
||||
private static Runnable onNode(DiscoveryNode node, Runnable runnable) {
|
||||
final String nodeId = "{" + node.getId() + "}{" + node.getEphemeralId() + "}";
|
||||
return new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
try (CloseableThreadContext.Instance ignored = CloseableThreadContext.put("nodeId", nodeId)) {
|
||||
runnable.run();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return nodeId + ": " + runnable.toString();
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -40,6 +40,7 @@ import java.util.concurrent.ScheduledExecutorService;
|
|||
import java.util.concurrent.ScheduledFuture;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.function.Function;
|
||||
|
||||
public class DeterministicTaskQueue extends AbstractComponent {
|
||||
|
||||
|
@ -182,6 +183,13 @@ public class DeterministicTaskQueue extends AbstractComponent {
|
|||
* @return A <code>ExecutorService</code> that uses this task queue.
|
||||
*/
|
||||
public ExecutorService getExecutorService() {
|
||||
return getExecutorService(Function.identity());
|
||||
}
|
||||
|
||||
/**
|
||||
* @return A <code>ExecutorService</code> that uses this task queue and wraps <code>Runnable</code>s in the given wrapper.
|
||||
*/
|
||||
public ExecutorService getExecutorService(Function<Runnable, Runnable> runnableWrapper) {
|
||||
return new ExecutorService() {
|
||||
|
||||
@Override
|
||||
|
@ -246,7 +254,7 @@ public class DeterministicTaskQueue extends AbstractComponent {
|
|||
|
||||
@Override
|
||||
public void execute(Runnable command) {
|
||||
scheduleNow(command);
|
||||
scheduleNow(runnableWrapper.apply(command));
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -255,6 +263,13 @@ public class DeterministicTaskQueue extends AbstractComponent {
|
|||
* @return A <code>ThreadPool</code> that uses this task queue.
|
||||
*/
|
||||
public ThreadPool getThreadPool() {
|
||||
return getThreadPool(Function.identity());
|
||||
}
|
||||
|
||||
/**
|
||||
* @return A <code>ThreadPool</code> that uses this task queue and wraps <code>Runnable</code>s in the given wrapper.
|
||||
*/
|
||||
public ThreadPool getThreadPool(Function<Runnable, Runnable> runnableWrapper) {
|
||||
return new ThreadPool(settings) {
|
||||
|
||||
{
|
||||
|
@ -303,12 +318,12 @@ public class DeterministicTaskQueue extends AbstractComponent {
|
|||
|
||||
@Override
|
||||
public ExecutorService generic() {
|
||||
return getExecutorService();
|
||||
return getExecutorService(runnableWrapper);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ExecutorService executor(String name) {
|
||||
return getExecutorService();
|
||||
return getExecutorService(runnableWrapper);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -318,7 +333,7 @@ public class DeterministicTaskQueue extends AbstractComponent {
|
|||
final int CANCELLED = 2;
|
||||
final AtomicInteger taskState = new AtomicInteger(NOT_STARTED);
|
||||
|
||||
scheduleAt(currentTimeMillis + delay.millis(), new Runnable() {
|
||||
scheduleAt(currentTimeMillis + delay.millis(), runnableWrapper.apply(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
if (taskState.compareAndSet(NOT_STARTED, STARTED)) {
|
||||
|
@ -330,7 +345,7 @@ public class DeterministicTaskQueue extends AbstractComponent {
|
|||
public String toString() {
|
||||
return command.toString();
|
||||
}
|
||||
});
|
||||
}));
|
||||
|
||||
return new ScheduledFuture<Object>() {
|
||||
@Override
|
||||
|
|
|
@ -56,6 +56,9 @@ public class TestThreadInfoPatternConverter extends LogEventPatternConverter {
|
|||
@Override
|
||||
public void format(LogEvent event, StringBuilder toAppendTo) {
|
||||
toAppendTo.append(threadInfo(event.getThreadName()));
|
||||
if (event.getContextData().isEmpty() == false) {
|
||||
toAppendTo.append(event.getContextData());
|
||||
}
|
||||
}
|
||||
|
||||
private static final Pattern ELASTICSEARCH_THREAD_NAME_PATTERN =
|
||||
|
@ -66,6 +69,7 @@ public class TestThreadInfoPatternConverter extends LogEventPatternConverter {
|
|||
Pattern.compile("SUITE-.+-worker");
|
||||
private static final Pattern NOT_YET_NAMED_NODE_THREAD_NAME_PATTERN =
|
||||
Pattern.compile("test_SUITE-CHILD_VM.+cluster\\[T#(.+)\\]");
|
||||
|
||||
static String threadInfo(String threadName) {
|
||||
Matcher m = ELASTICSEARCH_THREAD_NAME_PATTERN.matcher(threadName);
|
||||
if (m.matches()) {
|
||||
|
|
|
@ -42,15 +42,15 @@ public abstract class DisruptableMockTransport extends MockTransport {
|
|||
|
||||
protected abstract ConnectionStatus getConnectionStatus(DiscoveryNode sender, DiscoveryNode destination);
|
||||
|
||||
protected abstract Optional<DisruptableMockTransport> getDisruptedCapturingTransport(DiscoveryNode node);
|
||||
protected abstract Optional<DisruptableMockTransport> getDisruptedCapturingTransport(DiscoveryNode node, String action);
|
||||
|
||||
protected abstract void handle(DiscoveryNode sender, DiscoveryNode destination, Runnable doDelivery);
|
||||
protected abstract void handle(DiscoveryNode sender, DiscoveryNode destination, String action, Runnable doDelivery);
|
||||
|
||||
private void sendFromTo(DiscoveryNode sender, DiscoveryNode destination, Runnable doDelivery) {
|
||||
handle(sender, destination, new Runnable() {
|
||||
private void sendFromTo(DiscoveryNode sender, DiscoveryNode destination, String action, Runnable doDelivery) {
|
||||
handle(sender, destination, action, new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
if (getDisruptedCapturingTransport(destination).isPresent()) {
|
||||
if (getDisruptedCapturingTransport(destination, action).isPresent()) {
|
||||
doDelivery.run();
|
||||
} else {
|
||||
logger.trace("unknown destination in {}", this);
|
||||
|
@ -59,7 +59,7 @@ public abstract class DisruptableMockTransport extends MockTransport {
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
return doDelivery.toString() + " from " + sender + " to " + destination;
|
||||
return doDelivery.toString();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
@ -70,7 +70,7 @@ public abstract class DisruptableMockTransport extends MockTransport {
|
|||
assert destination.equals(getLocalNode()) == false : "non-local message from " + getLocalNode() + " to itself";
|
||||
super.onSendRequest(requestId, action, request, destination);
|
||||
|
||||
final String requestDescription = new ParameterizedMessage("{}[{}] from {} to {}",
|
||||
final String requestDescription = new ParameterizedMessage("[{}][{}] from {} to {}",
|
||||
action, requestId, getLocalNode(), destination).getFormattedMessage();
|
||||
|
||||
final Runnable returnConnectException = new Runnable() {
|
||||
|
@ -85,7 +85,7 @@ public abstract class DisruptableMockTransport extends MockTransport {
|
|||
}
|
||||
};
|
||||
|
||||
sendFromTo(getLocalNode(), destination, new Runnable() {
|
||||
sendFromTo(getLocalNode(), destination, action, new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
switch (getConnectionStatus(getLocalNode(), destination)) {
|
||||
|
@ -94,11 +94,11 @@ public abstract class DisruptableMockTransport extends MockTransport {
|
|||
break;
|
||||
|
||||
case DISCONNECTED:
|
||||
sendFromTo(destination, getLocalNode(), returnConnectException);
|
||||
sendFromTo(destination, getLocalNode(), action, returnConnectException);
|
||||
break;
|
||||
|
||||
case CONNECTED:
|
||||
Optional<DisruptableMockTransport> destinationTransport = getDisruptedCapturingTransport(destination);
|
||||
Optional<DisruptableMockTransport> destinationTransport = getDisruptedCapturingTransport(destination, action);
|
||||
assert destinationTransport.isPresent();
|
||||
|
||||
final RequestHandlerRegistry<TransportRequest> requestHandler =
|
||||
|
@ -117,7 +117,7 @@ public abstract class DisruptableMockTransport extends MockTransport {
|
|||
|
||||
@Override
|
||||
public void sendResponse(final TransportResponse response) {
|
||||
sendFromTo(destination, getLocalNode(), new Runnable() {
|
||||
sendFromTo(destination, getLocalNode(), action, new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
if (getConnectionStatus(destination, getLocalNode()) != ConnectionStatus.CONNECTED) {
|
||||
|
@ -143,7 +143,7 @@ public abstract class DisruptableMockTransport extends MockTransport {
|
|||
|
||||
@Override
|
||||
public void sendResponse(Exception exception) {
|
||||
sendFromTo(destination, getLocalNode(), new Runnable() {
|
||||
sendFromTo(destination, getLocalNode(), action, new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
if (getConnectionStatus(destination, getLocalNode()) != ConnectionStatus.CONNECTED) {
|
||||
|
|
|
@ -29,6 +29,7 @@ import java.util.List;
|
|||
import java.util.Random;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.ScheduledFuture;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import static org.elasticsearch.node.Node.NODE_NAME_SETTING;
|
||||
import static org.elasticsearch.threadpool.ThreadPool.Names.GENERIC;
|
||||
|
@ -264,6 +265,34 @@ public class DeterministicTaskQueueTests extends ESTestCase {
|
|||
assertThat(strings, containsInAnyOrder("foo", "bar"));
|
||||
}
|
||||
|
||||
public void testThreadPoolWrapsRunnable() {
|
||||
final DeterministicTaskQueue taskQueue = newTaskQueue();
|
||||
final AtomicBoolean called = new AtomicBoolean();
|
||||
final ThreadPool threadPool = taskQueue.getThreadPool(runnable -> () -> {
|
||||
assertFalse(called.get());
|
||||
called.set(true);
|
||||
runnable.run();
|
||||
});
|
||||
threadPool.generic().execute(() -> logger.info("runnable executed"));
|
||||
assertFalse(called.get());
|
||||
taskQueue.runAllRunnableTasks();
|
||||
assertTrue(called.get());
|
||||
}
|
||||
|
||||
public void testExecutorServiceWrapsRunnable() {
|
||||
final DeterministicTaskQueue taskQueue = newTaskQueue();
|
||||
final AtomicBoolean called = new AtomicBoolean();
|
||||
final ExecutorService executorService = taskQueue.getExecutorService(runnable -> () -> {
|
||||
assertFalse(called.get());
|
||||
called.set(true);
|
||||
runnable.run();
|
||||
});
|
||||
executorService.execute(() -> logger.info("runnable executed"));
|
||||
assertFalse(called.get());
|
||||
taskQueue.runAllRunnableTasks();
|
||||
assertTrue(called.get());
|
||||
}
|
||||
|
||||
public void testThreadPoolSchedulesFutureTasks() {
|
||||
final DeterministicTaskQueue taskQueue = newTaskQueue();
|
||||
advanceToRandomTime(taskQueue);
|
||||
|
|
|
@ -107,7 +107,7 @@ public class DisruptableMockTransportTests extends ESTestCase {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected Optional<DisruptableMockTransport> getDisruptedCapturingTransport(DiscoveryNode destination) {
|
||||
protected Optional<DisruptableMockTransport> getDisruptedCapturingTransport(DiscoveryNode destination, String action) {
|
||||
int index = discoNodes.indexOf(destination);
|
||||
if (index == -1) {
|
||||
return Optional.empty();
|
||||
|
@ -117,7 +117,7 @@ public class DisruptableMockTransportTests extends ESTestCase {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected void handle(DiscoveryNode sender, DiscoveryNode destination, Runnable doDelivery) {
|
||||
protected void handle(DiscoveryNode sender, DiscoveryNode destination, String action, Runnable doDelivery) {
|
||||
deterministicTaskQueue.scheduleNow(doDelivery);
|
||||
}
|
||||
};
|
||||
|
@ -134,7 +134,7 @@ public class DisruptableMockTransportTests extends ESTestCase {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected Optional<DisruptableMockTransport> getDisruptedCapturingTransport(DiscoveryNode destination) {
|
||||
protected Optional<DisruptableMockTransport> getDisruptedCapturingTransport(DiscoveryNode destination, String action) {
|
||||
int index = discoNodes.indexOf(destination);
|
||||
if (index == -1) {
|
||||
return Optional.empty();
|
||||
|
@ -144,7 +144,7 @@ public class DisruptableMockTransportTests extends ESTestCase {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected void handle(DiscoveryNode sender, DiscoveryNode destination, Runnable doDelivery) {
|
||||
protected void handle(DiscoveryNode sender, DiscoveryNode destination, String action, Runnable doDelivery) {
|
||||
deterministicTaskQueue.scheduleNow(doDelivery);
|
||||
}
|
||||
};
|
||||
|
|
Loading…
Reference in New Issue