Skip to content
Merged
Show file tree
Hide file tree
Changes from 22 commits
Commits
Show all changes
29 commits
Select commit Hold shift + click to select a range
9b12c27
IGNITE-25793 Add missing logs for single message errors
w3ll1ngt Jun 30, 2025
8f32c5a
Merge branch 'refs/heads/master' into ignite-25793
w3ll1ngt Jul 15, 2025
18b5315
IGNITE-25793 Add missing logs for single message errors
w3ll1ngt Jul 15, 2025
32a5636
IGNITE-25793 Add missing logs for single message errors
w3ll1ngt Jul 15, 2025
2f79b67
Merge branch 'refs/heads/master' into ignite-25793
w3ll1ngt Jul 18, 2025
8207b24
IGNITE-25793 Add missing logs for single message errors
w3ll1ngt Jul 18, 2025
b0d196c
Merge branch 'master' into ignite-25793
w3ll1ngt Jul 19, 2025
208370f
IGNITE-25793 Add missing logs for single message errors
w3ll1ngt Jul 19, 2025
9cbd553
Merge branch 'master' into ignite-25793
w3ll1ngt Jul 20, 2025
774cdff
Merge branch 'master' into ignite-25793
w3ll1ngt Jul 21, 2025
7c69db6
IGNITE-25793 Add tests
w3ll1ngt Jul 22, 2025
5ddd08f
Merge branch 'master' into ignite-25793
w3ll1ngt Jul 25, 2025
14f7268
IGNITE-25293 Add test
w3ll1ngt Jul 29, 2025
4ff477d
Merge branch 'master' into ignite-25793
w3ll1ngt Aug 8, 2025
63ec148
IGNITE-25793 Revert useless logs
w3ll1ngt Aug 11, 2025
9ccf1bf
Merge branch 'master' into ignite-25793
w3ll1ngt Aug 16, 2025
a9f9c13
IGNITE-25793 Fix flaky test
w3ll1ngt Aug 16, 2025
f2752e2
IGNITE-25793 Minor fix revert useless changes
w3ll1ngt Aug 18, 2025
45c6099
Merge branch 'master' into ignite-25793
w3ll1ngt Aug 18, 2025
fdb6eba
Merge branch 'master' into ignite-25793
Mar 3, 2026
10ae71e
fix reviewed
Mar 3, 2026
d823efe
fix minor
w3ll1ngt Mar 4, 2026
0566781
Update GridCachePartitionExchangeManager.java
w3ll1ngt Mar 4, 2026
9732f76
Update log substring
w3ll1ngt Mar 4, 2026
970a290
Update log substring
w3ll1ngt Mar 4, 2026
8308b00
Merge remote-tracking branch 'origin/ignite-25793' into ignite-25793
w3ll1ngt Mar 4, 2026
680cc58
Remove excessive finally logic in test
w3ll1ngt Mar 4, 2026
d500f7a
Apply suggestions from code review
w3ll1ngt Mar 13, 2026
75a6173
Fix tests with timeout removal
w3ll1ngt Mar 13, 2026
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
Original file line number Diff line number Diff line change
Expand Up @@ -1499,10 +1499,8 @@ private void sendLocalPartitions(
try {
cctx.io().sendNoRetry(node, m, SYSTEM_POOL);
}
catch (ClusterTopologyCheckedException ignore) {
if (log.isDebugEnabled())
log.debug("Failed to send partition update to node because it left grid (will ignore) [node=" +
node.id() + ", msg=" + m + ']');
catch (ClusterTopologyCheckedException e) {
log.warning("Failed to send local partitions [nodeId=" + node.id() + ", exchId=" + id + ']', e);
Comment thread
w3ll1ngt marked this conversation as resolved.
Outdated
}
catch (IgniteCheckedException e) {
U.error(log, "Failed to send local partition map to node [node=" + node + ", exchId=" + id + ']', e);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2117,15 +2117,16 @@ else if (localJoinExchange())
try {
cctx.io().send(node, msg, SYSTEM_POOL);
}
catch (ClusterTopologyCheckedException ignored) {
if (log.isDebugEnabled()) {
log.debug(
"Failed to send local partitions on exchange [nodeId=" + node.id() + ", exchId=" + exchId + ']'
);
}
catch (ClusterTopologyCheckedException e) {
long retryDelay = cctx.gridConfig().getNetworkSendRetryDelay();

log.warning(
"Failed to send local partitions on exchange (node left) [nodeId=" + node.id() +
", exchId=" + exchId + ", retryDelay=" + retryDelay + "ms]", e
);

if (cctx.discovery().alive(node.id())) {
U.sleep(cctx.gridConfig().getNetworkSendRetryDelay());
U.sleep(retryDelay);

continue;
}
Expand Down Expand Up @@ -2255,10 +2256,9 @@ private void sendPartitions(ClusterNode oldestNode) {
try {
sendLocalPartitions(oldestNode);
}
catch (ClusterTopologyCheckedException ignore) {
if (log.isDebugEnabled())
log.debug("Coordinator left during partition exchange [nodeId=" + oldestNode.id() +
", exchId=" + exchId + ']');
catch (ClusterTopologyCheckedException e) {
log.warning("Coordinator left during partition exchange [nodeId=" + oldestNode.id() +
", exchId=" + exchId + ']', e);
}
catch (IgniteCheckedException e) {
if (reconnectOnError(e))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,12 +19,14 @@

import java.io.ByteArrayOutputStream;
import java.io.PrintStream;
import java.util.UUID;
import java.util.concurrent.Callable;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Supplier;
import org.apache.ignite.Ignite;
import org.apache.ignite.IgniteCache;
import org.apache.ignite.IgniteCheckedException;
Expand All @@ -36,6 +38,7 @@
import org.apache.ignite.configuration.IgniteConfiguration;
import org.apache.ignite.internal.managers.communication.GridIoMessage;
import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicDeferredUpdateResponse;
import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleMessage;
import org.apache.ignite.internal.processors.cache.persistence.DatabaseLifecycleListener;
import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager;
import org.apache.ignite.lang.IgniteInClosure;
Expand Down Expand Up @@ -81,6 +84,9 @@ public class GridCachePartitionExchangeManagerWarningsTest extends GridCommonAbs
/** */
private ListeningTestLogger testLog;

/** */
private Supplier<TcpCommunicationSpi> spiSupp = CustomTcpCommunicationSpi::new;

/** {@inheritDoc} */
@Override protected void beforeTestsStarted() throws Exception {
super.beforeTestsStarted();
Expand All @@ -101,8 +107,8 @@ public class GridCachePartitionExchangeManagerWarningsTest extends GridCommonAbs
if (testLog != null)
testLog.clearListeners();

spiSupp = CustomTcpCommunicationSpi::new;
testLog = null;

lifecycleBean = null;

stopAllGrids();
Expand All @@ -112,7 +118,7 @@ public class GridCachePartitionExchangeManagerWarningsTest extends GridCommonAbs
@Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);

cfg.setCommunicationSpi(new CustomTcpCommunicationSpi());
cfg.setCommunicationSpi(spiSupp.get());

if (testLog != null)
cfg.setGridLogger(testLog);
Expand All @@ -131,6 +137,77 @@ public class GridCachePartitionExchangeManagerWarningsTest extends GridCommonAbs
return cfg;
}

/**
*
*/
@Test
public void testSingleMessageErrorWarnings() throws Exception {
long waitingTimeout = 5_000;
Comment thread
shishkovilja marked this conversation as resolved.
Outdated
String logSubstr = "Failed to send local partitions [nodeId=";

LogListener logListener = LogListener.matches(logSubstr).atLeast(1).build();
testLog = new ListeningTestLogger(log, logListener);

CountDownLatch beforeSendLatch = new CountDownLatch(1);
CountDownLatch proceedLatch = new CountDownLatch(1);

AtomicReference<UUID> crdIdRef = new AtomicReference<>();

spiSupp = () -> new TcpCommunicationSpi() {
/** {@inheritDoc} */
@Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure<IgniteException> ackC)
throws IgniteSpiException {
boolean isSingleMsg = ((GridIoMessage)msg).message() instanceof GridDhtPartitionsSingleMessage;
UUID crdId = crdIdRef.get();

if (isSingleMsg && node != null && crdId != null && crdId.equals(node.id())) {
beforeSendLatch.countDown();

try {
if (!proceedLatch.await(waitingTimeout, TimeUnit.MILLISECONDS))
throw new IgniteSpiException("Test timeout waiting to proceed");
Comment thread
w3ll1ngt marked this conversation as resolved.
Outdated
}
catch (InterruptedException e) {
Thread.currentThread().interrupt();
Comment thread
w3ll1ngt marked this conversation as resolved.

throw new IgniteSpiException("Interrupted while waiting to proceed", e);
}
}

super.sendMessage(node, msg, ackC);
}
};

IgniteEx crd = startGrid(0);
IgniteEx node1 = startGrid(1);
Comment thread
w3ll1ngt marked this conversation as resolved.

awaitPartitionMapExchange();

crdIdRef.set(crd.localNode().id());

IgniteInternalFuture<?> fut = GridTestUtils.runAsync(() ->
node1.context().cache().context().exchange().refreshPartitions());

boolean entered = false;
Comment thread
w3ll1ngt marked this conversation as resolved.
Outdated
try {
assertTrue("Did not enter sendMessage() in time", entered =
beforeSendLatch.await(waitingTimeout, TimeUnit.MILLISECONDS));

stopGrid(0);

proceedLatch.countDown();

fut.get(waitingTimeout);

assertTrue("Expected log not found", GridTestUtils.waitForCondition(logListener::check, waitingTimeout));
Comment thread
w3ll1ngt marked this conversation as resolved.
Outdated
}
finally {
proceedLatch.countDown();

if (!entered) beforeSendLatch.countDown();
}
}

/**
* @throws Exception If failed.
*/
Expand Down