Skip to content

Commit

Permalink
Make PeerFinder log messages happier (#83222)
Browse files Browse the repository at this point in the history
Since #73128 a sufficiently old `PeerFinder` will report all exceptions
encountered during discovery to help diagnose cluster formation
problems. We throw exceptions on genuine connection failures, but we
also throw exceptions if the discovered node is the local node or is
master-ineligible because these nodes are no use in discovery. We report
all such exceptions as failures:

    [instance-0000000001]
        address [10.0.0.1:12345], node [null], requesting [false]
        connection failed:
            [instance-0000000002][10.0.0.1:12345]
            non-master-eligible node found

Experience shows that users often have master-ineligible nodes in their
discovery config so will see these messages frequently if the cluster
cannot form, and may interpret the `connection failed` as the source of
the problems even though they're benign.

This commit adjusts the language in these messages to be more balanced,
replacing `connection failed` with `discovery result`, including the
phrase `successfully discovered` in the exception messsage, and giving
advice on how to suppress the message.
  • Loading branch information
DaveCTurner authored Jan 28, 2022
1 parent 61e1b08 commit f380ada
Show file tree
Hide file tree
Showing 5 changed files with 67 additions and 19 deletions.
5 changes: 5 additions & 0 deletions docs/changelog/83222.yaml
Original file line number Diff line number Diff line change
@@ -0,0 +1,5 @@
pr: 83222
summary: Make `PeerFinder` log messages happier
area: Cluster Coordination
type: enhancement
issues: []
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,8 @@
import org.elasticsearch.transport.TransportRequestOptions.Type;
import org.elasticsearch.transport.TransportService;

import java.util.Locale;

import static java.util.Collections.emptyMap;
import static java.util.Collections.emptySet;

Expand Down Expand Up @@ -110,9 +112,33 @@ protected void innerOnResponse(DiscoveryNode remoteNode) {
IOUtils.closeWhileHandlingException(connection);

if (remoteNode.equals(transportService.getLocalNode())) {
listener.onFailure(new ConnectTransportException(remoteNode, "local node found"));
listener.onFailure(
new ConnectTransportException(
remoteNode,
String.format(
Locale.ROOT,
"successfully discovered local node %s at [%s]",
remoteNode.descriptionWithoutAttributes(),
transportAddress
)
)
);
} else if (remoteNode.isMasterNode() == false) {
listener.onFailure(new ConnectTransportException(remoteNode, "non-master-eligible node found"));
listener.onFailure(
new ConnectTransportException(
remoteNode,
String.format(
Locale.ROOT,
"""
successfully discovered master-ineligible node %s at [%s]; to suppress this \
message, remove address [%s] from your discovery configuration or ensure that \
traffic to this address is routed only to master-eligible nodes""",
remoteNode.descriptionWithoutAttributes(),
transportAddress,
transportAddress
)
)
);
} else {
transportService.connectToNode(remoteNode, new ActionListener<>() {
@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -413,7 +413,7 @@ public void onFailure(Exception e) {
if (verboseFailureLogging) {
if (logger.isDebugEnabled()) {
// log message at level WARN, but since DEBUG logging is enabled we include the full stack trace
logger.warn(new ParameterizedMessage("{} connection failed", Peer.this), e);
logger.warn(new ParameterizedMessage("{} discovery result", Peer.this), e);
} else {
final StringBuilder messageBuilder = new StringBuilder();
Throwable cause = e;
Expand All @@ -424,10 +424,10 @@ public void onFailure(Exception e) {
final String message = messageBuilder.length() < 1024
? messageBuilder.toString()
: (messageBuilder.substring(0, 1023) + "...");
logger.warn("{} connection failed{}", Peer.this, message);
logger.warn("{} discovery result{}", Peer.this, message);
}
} else {
logger.debug(new ParameterizedMessage("{} connection failed", Peer.this), e);
logger.debug(new ParameterizedMessage("{} discovery result", Peer.this), e);
}
synchronized (mutex) {
assert probeConnectionResult.get() == null
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,8 @@
import static org.elasticsearch.discovery.HandshakingTransportAddressConnector.PROBE_CONNECT_TIMEOUT_SETTING;
import static org.elasticsearch.discovery.HandshakingTransportAddressConnector.PROBE_HANDSHAKE_TIMEOUT_SETTING;
import static org.elasticsearch.node.Node.NODE_NAME_SETTING;
import static org.hamcrest.Matchers.allOf;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.notNullValue;
import static org.hamcrest.Matchers.oneOf;
Expand Down Expand Up @@ -172,7 +174,7 @@ public void testLogsFullConnectionFailureAfterSuccessfulHandshake() throws Excep

try {
handshakingTransportAddressConnector.connectToRemoteMasterNode(discoveryAddress, failureListener);
failureListener.assertFailure();
assertThat(failureListener.getFailureMessage(), containsString("simulated"));
mockAppender.assertAllExpectationsMatched();
} finally {
Loggers.removeAppender(targetLogger, mockAppender);
Expand All @@ -187,7 +189,16 @@ public void testDoesNotConnectToNonMasterNode() throws InterruptedException {

FailureListener failureListener = new FailureListener();
handshakingTransportAddressConnector.connectToRemoteMasterNode(discoveryAddress, failureListener);
failureListener.assertFailure();
assertThat(
failureListener.getFailureMessage(),
allOf(
containsString("successfully discovered master-ineligible node"),
containsString(remoteNode.descriptionWithoutAttributes()),
containsString("to suppress this message"),
containsString("remove address [" + discoveryAddress + "] from your discovery configuration"),
containsString("ensure that traffic to this address is routed only to master-eligible nodes")
)
);
}

public void testDoesNotConnectToLocalNode() throws Exception {
Expand All @@ -197,7 +208,7 @@ public void testDoesNotConnectToLocalNode() throws Exception {

FailureListener failureListener = new FailureListener();
handshakingTransportAddressConnector.connectToRemoteMasterNode(discoveryAddress, failureListener);
failureListener.assertFailure();
assertThat(failureListener.getFailureMessage(), containsString("successfully discovered local node"));
}

public void testDoesNotConnectToDifferentCluster() throws InterruptedException {
Expand All @@ -207,7 +218,10 @@ public void testDoesNotConnectToDifferentCluster() throws InterruptedException {

FailureListener failureListener = new FailureListener();
handshakingTransportAddressConnector.connectToRemoteMasterNode(discoveryAddress, failureListener);
failureListener.assertFailure();
assertThat(
failureListener.getFailureMessage(),
containsString("remote cluster name [another-cluster] does not match local cluster name [local-cluster]")
);
}

public void testTimeoutDefaults() {
Expand All @@ -223,7 +237,7 @@ public void testHandshakeTimesOut() throws InterruptedException {

FailureListener failureListener = new FailureListener();
handshakingTransportAddressConnector.connectToRemoteMasterNode(discoveryAddress, failureListener);
failureListener.assertFailure();
assertThat(failureListener.getFailureMessage(), containsString("timed out"));
}

private TransportAddress getDiscoveryAddress() {
Expand All @@ -232,6 +246,7 @@ private TransportAddress getDiscoveryAddress() {

private static class FailureListener implements ActionListener<ProbeConnectionResult> {
final CountDownLatch completionLatch = new CountDownLatch(1);
String message;

@Override
public void onResponse(ProbeConnectionResult connectResult) {
Expand All @@ -240,11 +255,13 @@ public void onResponse(ProbeConnectionResult connectResult) {

@Override
public void onFailure(Exception e) {
message = e.getMessage();
completionLatch.countDown();
}

void assertFailure() throws InterruptedException {
assertTrue(completionLatch.await(15, TimeUnit.SECONDS));
String getFailureMessage() throws InterruptedException {
assertTrue("timed out waiting for listener to complete", completionLatch.await(15, TimeUnit.SECONDS));
return message;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -792,10 +792,10 @@ public void testLogsWarningsIfActiveForLongEnough() throws IllegalAccessExceptio

appender.addExpectation(
new MockLogAppender.SeenEventExpectation(
"connection failed",
"discovery result",
"org.elasticsearch.discovery.PeerFinder",
Level.WARN,
"address [" + otherNode.getAddress() + "]* connection failed: cannot connect to*"
"address [" + otherNode.getAddress() + "]* discovery result: cannot connect to*"
) {
@Override
public boolean innerMatch(LogEvent event) {
Expand All @@ -816,7 +816,7 @@ public boolean innerMatch(LogEvent event) {
}

@TestLogging(reason = "testing logging at DEBUG level", value = "org.elasticsearch.discovery:DEBUG")
public void testLogsStackTraceInConnectionFailedMessages() throws IllegalAccessException {
public void testLogsStackTraceInDiscoveryResultMessages() throws IllegalAccessException {
final DiscoveryNode otherNode = newDiscoveryNode("node-from-hosts-list");

providedAddresses.add(otherNode.getAddress());
Expand All @@ -832,10 +832,10 @@ public void testLogsStackTraceInConnectionFailedMessages() throws IllegalAccessE
Loggers.addAppender(LogManager.getLogger("org.elasticsearch.discovery.PeerFinder"), appender);
appender.addExpectation(
new MockLogAppender.SeenEventExpectation(
"connection failed",
"discovery result",
"org.elasticsearch.discovery.PeerFinder",
Level.DEBUG,
"address [" + otherNode.getAddress() + "]* connection failed*"
"address [" + otherNode.getAddress() + "]* discovery result*"
) {
@Override
public boolean innerMatch(LogEvent event) {
Expand All @@ -850,10 +850,10 @@ public boolean innerMatch(LogEvent event) {

appender.addExpectation(
new MockLogAppender.SeenEventExpectation(
"connection failed",
"discovery result",
"org.elasticsearch.discovery.PeerFinder",
Level.WARN,
"address [" + otherNode.getAddress() + "]* connection failed*"
"address [" + otherNode.getAddress() + "]* discovery result*"
) {
@Override
public boolean innerMatch(LogEvent event) {
Expand Down

0 comments on commit f380ada

Please sign in to comment.