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:
Yannick Welsch 2018-09-21 18:40:12 +02:00 committed by GitHub
parent 187f787f52
commit a612dd1272
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
9 changed files with 158 additions and 145 deletions

View File

@ -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 {

View File

@ -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) {

View File

@ -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{}";
}
}
}

View File

@ -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();
}
};
}
}

View File

@ -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

View File

@ -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()) {

View File

@ -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) {

View File

@ -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);

View File

@ -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);
}
};