Skip to content

Commit

Permalink
GG-23781 Prevent pessimistic tx lock request processing over incomple…
Browse files Browse the repository at this point in the history
…te topology.

(cherry picked from commit 27c44f7)
  • Loading branch information
ascherbakoff committed Sep 12, 2019
1 parent a8c2ca2 commit fcee6f7
Show file tree
Hide file tree
Showing 5 changed files with 106 additions and 59 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -111,6 +111,6 @@ public void cacheId(int cacheId) {

/** {@inheritDoc} */
@Override public String toString() {
return S.toString(GridCacheIdMessage.class, this);
return S.toString(GridCacheIdMessage.class, this, "super", super.toString());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -1704,10 +1704,6 @@ void onResult(GridNearLockResponse res) {

if (res.compatibleRemapVersion()) {
if (tx != null) {
// Versions are compatible.
cctx.shared().exchange().
lastAffinityChangedTopologyVersion(res.clientRemapVersion(), tx.topologyVersionSnapshot());

tx.onRemap(res.clientRemapVersion(), false);

// Use remapped version for all subsequent mappings.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1672,10 +1672,6 @@ void onResult(GridNearLockResponse res) {

if (res.compatibleRemapVersion()) {
if (tx != null) {
// Versions are compatible.
cctx.shared().exchange().
lastAffinityChangedTopologyVersion(res.clientRemapVersion(), tx.topologyVersionSnapshot());

tx.onRemap(res.clientRemapVersion(), false);

// Use remapped version for all subsequent mappings.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,12 +16,10 @@

package org.apache.ignite.internal.processors.cache.transactions;

import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.ignite.Ignite;
import org.apache.ignite.IgniteDataStreamer;
import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
import org.apache.ignite.cluster.ClusterNode;
Expand All @@ -30,16 +28,19 @@
import org.apache.ignite.configuration.DataStorageConfiguration;
import org.apache.ignite.configuration.IgniteConfiguration;
import org.apache.ignite.internal.IgniteEx;
import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException;
import org.apache.ignite.internal.IgniteInternalFuture;
import org.apache.ignite.internal.TestRecordingCommunicationSpi;
import org.apache.ignite.internal.processors.affinity.AffinityAssignment;
import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
import org.apache.ignite.internal.processors.cache.GridCacheUtils;
import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionExchangeId;
import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessage;
import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleMessage;
import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsFullMessage;
import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopology;
import org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockRequest;
import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareRequest;
import org.apache.ignite.internal.util.GridConcurrentSkipListSet;
import org.apache.ignite.internal.util.typedef.internal.CU;
import org.apache.ignite.internal.util.typedef.internal.U;
import org.apache.ignite.lang.IgniteBiPredicate;
Expand Down Expand Up @@ -129,7 +130,19 @@ public void testCrossCacheTxMapOnInvalidTopologyOptimisticSerializable() throws
}

/**
* Test scenario: cross-cache tx is started when node is left in the middle of rebalance, first cache is rebalanced
* and second is partially rebalanced.
*
* First cache map request will trigger client compatible remap for pessimistic txs,
* second cache map request should use new topology version.
*
* For optimistic tx remap is enforced if more than one mapping in transaction or all enlisted caches have compatible
* assignments.
*
* Success: tx is finished on ideal topology version over all mapped nodes.
*
* @param concurrency Concurrency.
* @param isolation Isolation.
*/
private void doTestCrossCacheTxMapOnInvalidTopology(TransactionConcurrency concurrency, TransactionIsolation isolation) throws Exception {
try {
Expand All @@ -138,7 +151,7 @@ private void doTestCrossCacheTxMapOnInvalidTopology(TransactionConcurrency concu

awaitPartitionMapExchange();

Ignite client = startGrid("client");
IgniteEx client = startGrid("client");
assertNotNull(client.cache(CACHE1));
assertNotNull(client.cache(CACHE2));

Expand All @@ -158,9 +171,12 @@ private void doTestCrossCacheTxMapOnInvalidTopology(TransactionConcurrency concu

final AffinityTopologyVersion joinVer = new AffinityTopologyVersion(4, 0);
AffinityTopologyVersion leftVer = new AffinityTopologyVersion(5, 0);
AffinityTopologyVersion idealVer = new AffinityTopologyVersion(5, 1);

AtomicReference<Set<Integer>> full = new AtomicReference<>();

GridConcurrentSkipListSet<Integer> leftVerParts = new GridConcurrentSkipListSet<>();

crdSpi.blockMessages((node, m) -> {
if (m instanceof GridDhtPartitionSupplyMessage) {
GridDhtPartitionSupplyMessage msg = (GridDhtPartitionSupplyMessage)m;
Expand All @@ -182,7 +198,20 @@ private void doTestCrossCacheTxMapOnInvalidTopology(TransactionConcurrency concu
return true;
}

if (msg.topologyVersion().equals(leftVer))
if (msg.topologyVersion().equals(leftVer)) {
Map<Integer, Long> last = U.field(msg, "last");

leftVerParts.addAll(last.keySet());

return true;
}
} else if (m instanceof GridDhtPartitionsFullMessage) {
GridDhtPartitionsFullMessage msg = (GridDhtPartitionsFullMessage)m;

// Delay full message for ideal topology switch.
GridDhtPartitionExchangeId exchId = msg.exchangeId();

if (exchId != null && exchId.topologyVersion().equals(idealVer))
return true;
}

Expand Down Expand Up @@ -236,76 +265,104 @@ private void doTestCrossCacheTxMapOnInvalidTopology(TransactionConcurrency concu
grid(0).cachex(CACHE2).context().affinity().affinityReadyFuture(leftVer).get();
grid(2).cachex(CACHE2).context().affinity().affinityReadyFuture(leftVer).get();

AffinityAssignment assignment0 = grid(0).cachex(CACHE1).context().affinity().assignment(leftVer);
AffinityAssignment assignment = grid(0).cachex(CACHE2).context().affinity().assignment(leftVer);

// Search for a partition with incompatible assignment.
int movingPart = -1;
int stablePart = -1; // Partition for cache1 which is mapped for both late and ideal topologies to the same primary.
int movingPart = -1; // Partition for cache2 which is mapped for both late and ideal topologies on different primaries.

for (int p = 0; p < assignment.assignment().size(); p++) {
List<ClusterNode> nodes = assignment.assignment().get(p);
List<ClusterNode> nodes2 = assignment.idealAssignment().get(p);
for (int p = 0; p < assignment0.assignment().size(); p++) {
List<ClusterNode> curr = assignment.assignment().get(p);
List<ClusterNode> ideal = assignment.idealAssignment().get(p);

if (!nodes.equals(nodes2) && nodes.get(0).order() == 1)
movingPart = p;
if (curr.equals(ideal) && curr.get(0).order() == 1) {
stablePart = p;

break;
}
}

assertFalse(movingPart == -1);
assertFalse(stablePart == -1);

// Delay rebalance for next top ver.
TestRecordingCommunicationSpi.spi(grid(2)).blockMessages((node, message) -> {
if (message instanceof GridDhtPartitionsSingleMessage) {
GridDhtPartitionsSingleMessage sm = (GridDhtPartitionsSingleMessage)message;
for (int p = 0; p < assignment.assignment().size(); p++) {
List<ClusterNode> curr = assignment.assignment().get(p);
List<ClusterNode> ideal = assignment.idealAssignment().get(p);

return sm.exchangeId() != null;
if (!curr.equals(ideal) && curr.get(0).order() == 1) {
movingPart = p;

break;
}
}

return false;
});
assertFalse(movingPart == -1);

TestRecordingCommunicationSpi.spi(client).blockMessages(new IgniteBiPredicate<ClusterNode, Message>() {
@Override public boolean apply(ClusterNode node, Message message) {
@Override public boolean apply(ClusterNode node, Message msg) {
if (concurrency == PESSIMISTIC)
return message instanceof GridNearLockRequest;
return msg instanceof GridNearLockRequest;
else
return message instanceof GridNearTxPrepareRequest;
return msg instanceof GridNearTxPrepareRequest;
}
});

final int finalStablePart = stablePart;
final int finalMovingPart = movingPart;
IgniteInternalFuture<?> txFut = multithreadedAsync(new Runnable() {
@Override public void run() {
try (Transaction tx = client.transactions().txStart(concurrency, isolation)) {
Map<Integer, Integer> map = new LinkedHashMap<>();

for (int p = 0; p < PARTS_CNT; p++)
map.put(p, p);
IgniteInternalFuture<?> txFut = multithreadedAsync(() -> {
try (Transaction tx = client.transactions().txStart(concurrency, isolation)) {
client.cache(CACHE1).put(finalStablePart, 0); // Will map on crd(order=1).

client.cache(CACHE1).putAll(map); // Will successfully lock topology.
client.cache(CACHE2).put(finalMovingPart, 0); // Should remap but will go through without fix.
// Next request will remap to ideal topology, but it's not ready on other node except crd.
client.cache(CACHE2).put(finalMovingPart, 0);

tx.commit();
}
tx.commit();
}
}, 1, "tx-thread");

// Wait until all missing supply messages are blocked.
assertTrue(GridTestUtils.waitForCondition(() -> leftVerParts.size() == PARTS_CNT - full.get().size(), 5_000));

// Delay first lock request on late topology.
TestRecordingCommunicationSpi.spi(client).waitForBlocked();

crdSpi.stopBlock(); // Continue rebalance and trigger next topology switch.
// At this point only supply messages should be blocked.
// Unblock to continue rebalance and trigger ideal topology switch.
crdSpi.stopBlock(true, null, false, true);

// Wait until ideal topology is ready on crd.
crd.context().cache().context().exchange().affinityReadyFuture(idealVer).get(10_000);

TestRecordingCommunicationSpi.spi(grid(2)).waitForBlocked();
// Other node must wait for full message.
assertFalse(GridTestUtils.waitForCondition(() ->
grid(2).context().cache().context().exchange().affinityReadyFuture(idealVer).isDone(), 1_000));

// Map on unstable topology (PME is in progress on other node).
TestRecordingCommunicationSpi.spi(client).stopBlock();

doSleep(5000); // Make sure request will listen for current topology future completion.
// Capture local transaction.
IgniteInternalTx tx0 = client.context().cache().context().tm().activeTransactions().iterator().next();

TestRecordingCommunicationSpi.spi(grid(2)).stopBlock();
// Expected behavior: tx must hang (both pessimistic and optimistic) because topology is not ready.
try {
txFut.get(3_000);

crdSpi.stopBlock();
fail("TX must not complete");
}
catch (IgniteFutureTimeoutCheckedException e) {
// Expected.
}

awaitPartitionMapExchange();
crdSpi.stopBlock();

txFut.get();

// Check transaction map version. Should be mapped on ideal topology.
assertEquals(tx0.topologyVersionSnapshot(), idealVer);

awaitPartitionMapExchange();

checkFutures();
}
finally {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -161,7 +161,7 @@ public void testPartitionConsistencyWithPrimaryRestart() throws Exception {

List<Integer> primaryKeys = primaryKeys(prim.cache(DEFAULT_CACHE_NAME), 10_000);

long stop = U.currentTimeMillis() + GridTestUtils.SF.applyLB(60_000, 30_000);
long stop = U.currentTimeMillis() + GridTestUtils.SF.applyLB(2 * 60_000, 30_000);

Random r = new Random();

Expand Down Expand Up @@ -213,7 +213,7 @@ public void testPartitionConsistencyWithBackupsRestart() throws Exception {

assertFalse(backups.contains(prim));

long stop = U.currentTimeMillis() + GridTestUtils.SF.applyLB( 3 * 60_000, 30_000);
long stop = U.currentTimeMillis() + GridTestUtils.SF.applyLB(2 * 60_000, 30_000);

long seed = System.nanoTime();

Expand Down Expand Up @@ -626,9 +626,9 @@ public void testPartitionConsistencyDuringRebalanceAndConcurrentUpdates_TxDuring
Integer key0 = primaryKey(grid(1).cache(DEFAULT_CACHE_NAME));
Integer key = primaryKey(grid(0).cache(DEFAULT_CACHE_NAME));

TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(crd);
TestRecordingCommunicationSpi crdSpi = TestRecordingCommunicationSpi.spi(crd);

spi.blockMessages((node, message) -> {
crdSpi.blockMessages((node, message) -> {
if (message instanceof GridDhtPartitionsFullMessage) {
GridDhtPartitionsFullMessage tmp = (GridDhtPartitionsFullMessage)message;

Expand All @@ -641,11 +641,11 @@ public void testPartitionConsistencyDuringRebalanceAndConcurrentUpdates_TxDuring
// Locks mapped wait.
CountDownLatch l = new CountDownLatch(1);

IgniteInternalFuture fut = GridTestUtils.runAsync(() -> {
IgniteInternalFuture startNodeFut = GridTestUtils.runAsync(() -> {
U.awaitQuiet(l);

try {
startGrid(SERVER_NODES);
startGrid(SERVER_NODES); // Start node out of BLT.
}
catch (Exception e) {
fail(X.getFullStackTrace(e));
Expand Down Expand Up @@ -677,7 +677,7 @@ public void testPartitionConsistencyDuringRebalanceAndConcurrentUpdates_TxDuring

l.countDown();

spi.waitForBlocked(); // Block PME after finish on crd and wait on others.
crdSpi.waitForBlocked(); // Block PME after finish on crd and wait on others.

cliSpi.stopBlock(); // Start remote lock mapping.
}
Expand All @@ -686,12 +686,10 @@ public void testPartitionConsistencyDuringRebalanceAndConcurrentUpdates_TxDuring
}
});

txFut.get(); // Transaction should not be blocked by concurrent PME.
lockFut.get();

spi.stopBlock();

fut.get();
crdSpi.stopBlock();
txFut.get();
startNodeFut.get();

awaitPartitionMapExchange();

Expand Down

0 comments on commit fcee6f7

Please sign in to comment.