Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

IGNITE-21830 Fixed logging of failed attempts to connect to the previous node in the ring. #11327

Merged
merged 24 commits into from
May 21, 2024
Original file line number Diff line number Diff line change
Expand Up @@ -6770,7 +6770,6 @@ else if (req.changeTopology()) {

if (previous != null && !previous.id().equals(nodeId) &&
(req.checkPreviousNodeId() == null || previous.id().equals(req.checkPreviousNodeId()))) {
Collection<InetSocketAddress> nodeAddrs = spi.getEffectiveNodeAddresses(previous);

// The connection recovery connection to one node is connCheckTick.
// We need to suppose network delays. So we use half of this time.
Expand All @@ -6781,13 +6780,7 @@ else if (req.changeTopology()) {
"previous [" + previous + "] with timeout " + backwardCheckTimeout);
}

liveAddr = checkConnection(new ArrayList<>(nodeAddrs), backwardCheckTimeout);

if (log.isInfoEnabled()) {
log.info("Connection check to previous node done: [liveAddr=" + liveAddr
+ ", previousNode=" + U.toShortString(previous) + ", addressesToCheck=" +
nodeAddrs + ", connectingNodeId=" + nodeId + ']');
}
liveAddr = checkConnection(previous, backwardCheckTimeout);
}

ok = liveAddr != null;
Expand Down Expand Up @@ -7254,9 +7247,11 @@ private void ringMessageReceived() {
}

/** @return Alive address if was able to connected to. {@code Null} otherwise. */
private InetSocketAddress checkConnection(List<InetSocketAddress> addrs, int timeout) {
private InetSocketAddress checkConnection(TcpDiscoveryNode node, int timeout) {
AtomicReference<InetSocketAddress> liveAddrHolder = new AtomicReference<>();

List<InetSocketAddress> addrs = new ArrayList<>(spi.getEffectiveNodeAddresses(node));

CountDownLatch latch = new CountDownLatch(addrs.size());

int addrLeft = addrs.size();
Expand Down Expand Up @@ -7289,8 +7284,9 @@ private InetSocketAddress checkConnection(List<InetSocketAddress> addrs, int tim
liveAddrHolder.compareAndSet(null, addr);
}
}
catch (Exception ignored) {
// No-op.
catch (Exception e) {
U.warn(log, "Failed to check connection to previous node [nodeId=" + node.id() + ", order="
+ node.order() + ", address=" + addr + ']', e);
}
finally {
latch.countDown();
Expand All @@ -7307,6 +7303,16 @@ private InetSocketAddress checkConnection(List<InetSocketAddress> addrs, int tim
// No-op.
}

if (liveAddrHolder.get() == null) {
U.warn(log, "Failed to check connection to previous node [connectingNodeId=" + nodeId
+ ", previousNode=" + U.toShortString(node) + ", previousNodeKnownAddresses=" + addrs + ']');
}
else if (log.isInfoEnabled()) {
log.info("Connection check to previous node done [connectingNodeId=" + nodeId + ", previousNode="
+ U.toShortString(node) + ", firstRespondedAddress=" + liveAddrHolder.get() +
", previousNodeKnownAddresses=" + addrs + ']');
}

return liveAddrHolder.get();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import java.net.Socket;
import java.net.SocketTimeoutException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
Expand Down Expand Up @@ -56,6 +57,8 @@
import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryHandshakeRequest;
import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryHandshakeResponse;
import org.apache.ignite.testframework.GridTestUtils;
import org.apache.ignite.testframework.ListeningTestLogger;
import org.apache.ignite.testframework.LogListener;
import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
import org.junit.Test;

Expand Down Expand Up @@ -383,6 +386,42 @@ private void simulateFailureOfTwoNodes(boolean sequentionally) throws Exception
}
}

/**
* This test uses node failure by stopping service threads, which makes the node unresponsive and results in
maksaska marked this conversation as resolved.
Show resolved Hide resolved
* failing connection to the server. Failures are simulated on the 1st node in the ring. In this case,
* the 2nd node in the ring will trigger 'Backward Connection Check', which should result in failing attempt of connection.
* This result is followed by the corresponding logs, indicating described failures. The test verifies the logs.
*
* @throws Exception If failed.
*/
@Test
public void testBackwardConnectionCheckFailedLogMessage() throws Exception {
ListeningTestLogger testLog = new ListeningTestLogger(log);

LogListener lsnr0 = LogListener.matches("Failed to check connection to previous node").times(2).build();

testLog.registerListener(lsnr0);

startGrid(0);

IgniteConfiguration cfg = getConfiguration(getTestIgniteInstanceName(1));
cfg.setGridLogger(testLog);

startGrid(cfg);

startGrid(2);

spi(grid(0)).simulateNodeFailure();

assertTrue(lsnr0.check(getTestTimeout()));

for (Ignite ig : Arrays.asList(grid(1), grid(2))) {
waitForCondition(() -> ig.cluster().nodes().size() == 2, getTestTimeout());

assertTrue(ig.cluster().nodes().stream().noneMatch(node -> node.order() == 1));
}
}

/**
* @param ig Ignite instance to get failedNodes collection from.
*/
Expand Down