Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
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 @@ -1819,11 +1819,10 @@ private void updateAllAsyncInternal0(
top.readLock();

try {
if (top.stopping()) {
if (ctx.shared().cache().isCacheRestarting(name()))
res.addFailedKeys(req.keys(), new IgniteCacheRestartingException(name()));
else
res.addFailedKeys(req.keys(), new CacheStoppedException(name()));
Exception validationErr = validateCacheOperation(top);

if (validationErr != null) {
res.addFailedKeys(req.keys(), validationErr);

completionCb.apply(req, res);

Expand Down Expand Up @@ -3693,6 +3692,21 @@ private void sendNearUpdateReply(UUID nodeId, GridNearAtomicUpdateResponse res)
}
}

/** */
@Nullable private Exception validateCacheOperation(GridDhtPartitionTopology top) {
if (ctx.kernalContext().isStopping())
return new NodeStoppingException("Failed to perform cache operation. Node is stopping [cacheName=" + name() + ']');

if (top.stopping()) {
if (ctx.shared().cache().isCacheRestarting(name()))
return new IgniteCacheRestartingException(name());
else
return new CacheStoppedException(name());
}

return null;
}

/** {@inheritDoc} */
@Override public String toString() {
return S.toString(GridDhtAtomicCache.class, this, super.toString());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -126,14 +126,21 @@ public void testFailover() throws Exception {

// Simple operation failover: put/get
assertOnUnstableCluster(cluster, () -> {
Integer key = rnd.nextInt();
String val = key.toString();
try {
Integer key = rnd.nextInt();
String val = key.toString();

cachePut(cache, key, val);
cachePut(cache, key, val);

String cachedVal = cache.get(key);
String cachedVal = cache.get(key);

assertEquals(val, cachedVal);
assertEquals(val, cachedVal);
}
catch (ClientException e) {
// A cache update may fail if the node is stopped while processing the request.
if (!e.getMessage().contains("Failed to update keys"))
throw e;
}
});

cache.clear();
Expand All @@ -143,12 +150,11 @@ public void testFailover() throws Exception {
.collect(Collectors.toMap(i -> i, i -> String.format("String %s", i)));

assertOnUnstableCluster(cluster, () -> {
cache.putAll(data);
try {
cache.putAll(data);

Query<Cache.Entry<Integer, String>> qry =
new ScanQuery<Integer, String>().setPageSize(data.size() / 10);
Query<Cache.Entry<Integer, String>> qry = new ScanQuery<Integer, String>().setPageSize(data.size() / 10);

try {
try (QueryCursor<Cache.Entry<Integer, String>> cur = cache.query(qry)) {
List<Cache.Entry<Integer, String>> res = cur.getAll();

Expand All @@ -164,6 +170,11 @@ public void testFailover() throws Exception {
// QueryCursor.getAll always executes on the same channel where the cursor is open,
// so failover is not possible, and the call will fail when connection drops.
}
catch (ClientException e) {
// putAll operation may fail if the participating node left the cluster during its processing.
if (!e.getMessage().contains("Failed to update keys"))
throw e;
}
});

// Client fails if all nodes go down
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,134 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.ignite.internal.processors.cache.distributed.dht.atomic;

import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.ignite.configuration.IgniteConfiguration;
import org.apache.ignite.failure.StopNodeOrHaltFailureHandler;
import org.apache.ignite.internal.IgniteEx;
import org.apache.ignite.internal.IgniteInternalFuture;
import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture;
import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.PartitionsExchangeAware;
import org.apache.ignite.testframework.GridTestUtils;
import org.apache.ignite.testframework.ListeningTestLogger;
import org.apache.ignite.testframework.LogListener;
import org.apache.ignite.testframework.junits.GridAbstractTest;
import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
import org.junit.Test;

import static java.util.concurrent.TimeUnit.MILLISECONDS;

/** */
public class AtomicCacheOperationRemappingOnNodeStopTest extends GridCommonAbstractTest {
/** */
private static final LogListener NEAR_UPDATE_RESP_LOG_LSNR = LogListener.matches("Sent near update response").build();

/** {@inheritDoc} */
@Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
ListeningTestLogger log = new ListeningTestLogger(GridAbstractTest.log);

log.registerListener(NEAR_UPDATE_RESP_LOG_LSNR);

return super.getConfiguration(igniteInstanceName)
.setGridLogger(log)
.setFailureHandler(new StopNodeOrHaltFailureHandler());
}

/** */
@Test
public void test() throws Exception {
startGrids(3);

grid(0).createCache(DEFAULT_CACHE_NAME);

IgniteEx firstNode = grid(1);

CountDownLatch zeroNodePmeFinishedLatch = new CountDownLatch(1);
CountDownLatch firstNodePmeStartedLatch = new CountDownLatch(1);
CountDownLatch firstNodePmeUnblockedLatch = new CountDownLatch(1);
CountDownLatch firstNodePmeFinishedLatch = new CountDownLatch(1);
CountDownLatch firstNodeStopUnblockedLatch = new CountDownLatch(1);

grid(0).context().cache().context().exchange().registerExchangeAwareComponent(new PartitionsExchangeAware() {
@Override public void onDoneAfterTopologyUnlock(GridDhtPartitionsExchangeFuture fut) {
zeroNodePmeFinishedLatch.countDown();
}
});

firstNode.context().cache().context().exchange().registerExchangeAwareComponent(new PartitionsExchangeAware() {
@Override public void onInitBeforeTopologyLock(GridDhtPartitionsExchangeFuture fut) {
try {
firstNodePmeStartedLatch.countDown();

firstNodePmeUnblockedLatch.await(getTestTimeout(), MILLISECONDS);
}
catch (InterruptedException e) {
throw new RuntimeException(e);
}
}
});

stopGrid(2);

firstNodePmeStartedLatch.await(getTestTimeout(), MILLISECONDS);

firstNode.context().cache().context().exchange().lastTopologyFuture().listen(() -> {
try {
firstNodePmeFinishedLatch.countDown();

firstNodeStopUnblockedLatch.await(getTestTimeout(), MILLISECONDS);
}
catch (InterruptedException e) {
throw new RuntimeException(e);
}
});

IgniteInternalFuture<Object> firstNodeStopFut = GridTestUtils.runAsync(() -> stopGrid(1));

firstNodePmeUnblockedLatch.countDown();

firstNodePmeFinishedLatch.await(getTestTimeout(), MILLISECONDS);

zeroNodePmeFinishedLatch.await(getTestTimeout(), MILLISECONDS);

int keyForFirstNode = keyForNode(grid(0).affinity(DEFAULT_CACHE_NAME), new AtomicInteger(), firstNode.localNode());

NEAR_UPDATE_RESP_LOG_LSNR.reset();

setLoggerDebugLevel();

IgniteInternalFuture<Object> putFut = GridTestUtils.runAsync(() ->
grid(0).cache(DEFAULT_CACHE_NAME).put(keyForFirstNode, "test-val"));

NEAR_UPDATE_RESP_LOG_LSNR.check(getTestTimeout());

firstNodeStopUnblockedLatch.countDown();

firstNodeStopFut.get(getTestTimeout(), MILLISECONDS);

putFut.get(getTestTimeout(), MILLISECONDS);

assertEquals("test-val", grid(0).cache(DEFAULT_CACHE_NAME).get(keyForFirstNode));
}

/** {@inheritDoc} */
@Override protected long getTestTimeout() {
return 30_000;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheDhtRemoveFailureTest;
import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheTxNodeFailureSelfTest;
import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteAtomicLongChangingTopologySelfTest;
import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.AtomicCacheOperationRemappingOnNodeStopTest;
import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.AtomicPutAllChangingTopologyTest;
import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridCacheAtomicClientInvalidPartitionHandlingSelfTest;
import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridCacheAtomicClientRemoveFailureTest;
Expand Down Expand Up @@ -75,6 +76,8 @@
GridCacheTxNodeFailureSelfTest.class,

AtomicPutAllChangingTopologyTest.class,

AtomicCacheOperationRemappingOnNodeStopTest.class,
})
public class IgniteCacheFailoverTestSuite {
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,12 +24,15 @@
import org.apache.ignite.IgniteException;
import org.apache.ignite.IgniteLogger;
import org.apache.ignite.cache.CacheAtomicityMode;
import org.apache.ignite.cache.CachePartialUpdateException;
import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
import org.apache.ignite.cluster.ClusterTopologyException;
import org.apache.ignite.configuration.CacheConfiguration;
import org.apache.ignite.internal.IgniteEx;
import org.apache.ignite.internal.IgniteInternalFuture;
import org.apache.ignite.internal.NodeStoppingException;
import org.apache.ignite.internal.util.lang.GridAbsPredicate;
import org.apache.ignite.internal.util.typedef.X;
import org.apache.ignite.lang.IgniteRunnable;
import org.apache.ignite.resources.IgniteInstanceResource;
import org.apache.ignite.resources.LoggerResource;
Expand Down Expand Up @@ -312,8 +315,14 @@ public NotReservedCacheOpAffinityRun(int orgId, int keyBegin, String cacheName)
log.info("Begin run " + keyBegin);
IgniteCache cache = ignite.cache(cacheName);

for (int i = 0; i < KEYS_CNT; ++i)
cache.put(i + keyBegin, i + keyBegin);
for (int i = 0; i < KEYS_CNT; ++i) {
try {
cache.put(i + keyBegin, i + keyBegin);
}
catch (CachePartialUpdateException e) {
assertTrue(X.hasCause(e, NodeStoppingException.class));
}
}

log.info("End run " + keyBegin);
}
Expand Down Expand Up @@ -356,7 +365,13 @@ public ReservedPartitionCacheOpAffinityRun(int orgId, int keyBegin) {

for (int i = 0; i < KEYS_CNT; ++i) {
Person p = new Person(i + keyBegin, orgId);
cache.put(p.createKey(), p);

try {
cache.put(p.createKey(), p);
}
catch (CachePartialUpdateException e) {
assertTrue(X.hasCause(e, NodeStoppingException.class));
}
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,11 +46,13 @@
import org.apache.ignite.events.Event;
import org.apache.ignite.events.EventType;
import org.apache.ignite.internal.IgniteInternalFuture;
import org.apache.ignite.internal.TestRecordingCommunicationSpi;
import org.apache.ignite.internal.processors.cache.index.AbstractIndexingCommonTest;
import org.apache.ignite.internal.processors.cache.query.SqlFieldsQueryEx;
import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
import org.apache.ignite.internal.processors.query.h2.twostep.GridMapQueryExecutor;
import org.apache.ignite.internal.processors.query.h2.twostep.GridReduceQueryExecutor;
import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2DmlResponse;
import org.apache.ignite.internal.processors.query.running.GridRunningQueryInfo;
import org.apache.ignite.internal.util.lang.GridAbsPredicate;
import org.apache.ignite.lang.IgnitePredicate;
Expand All @@ -59,6 +61,7 @@

import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static org.apache.ignite.events.EventType.EVT_CACHE_QUERY_EXECUTED;
import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;

/**
* Tests for distributed DML.
Expand Down Expand Up @@ -90,6 +93,8 @@ public class IgniteSqlSkipReducerOnUpdateDmlSelfTest extends AbstractIndexingCom
@Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
IgniteConfiguration c = super.getConfiguration(gridName);

c.setCommunicationSpi(new TestRecordingCommunicationSpi());

List<CacheConfiguration> ccfgs = new ArrayList<>();

ccfgs.add(buildCacheConfiguration(CACHE_ORG));
Expand Down Expand Up @@ -434,8 +439,12 @@ public void testNodeStopDuringUpdate() throws Exception {
}
}, 5000));

spi(grid(NODE_COUNT + 1)).blockMessages((n, m) -> m instanceof GridH2DmlResponse);

latch.countDown();

spi(grid(NODE_COUNT + 1)).waitForBlocked();

stopGrid(NODE_COUNT + 1);

GridTestUtils.assertThrows(log, new Callable<Object>() {
Expand Down
Loading