Details
-
Bug
-
Status: Resolved
-
Critical
-
Resolution: Fixed
-
None
Description
Enlist batch holds key and values in arrays structures. This implies that keys and vals arrays sizes should be equals.
Also, we have an optimization and do not save 'null' vals for 'remove' operation.
This invariant can become broken on removeAll operation for 2 entries belonging to partitions in different states (moving and owning). For the first one, it's 'mvcc history' will be added to 'vals' array, but nothing will be added for the second one.
Reproducer IgniteCacheEntryProcessorNodeJoinTest.testEntryProcessorNodeLeave
See stacktrace:
java.lang.AssertionError: at org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxAbstractEnlistFuture$Batch.add(GridDhtTxAbstractEnlistFuture.java:1156) at org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxAbstractEnlistFuture.addToBatch(GridDhtTxAbstractEnlistFuture.java:705) at org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxAbstractEnlistFuture.processEntry(GridDhtTxAbstractEnlistFuture.java:650) at org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxAbstractEnlistFuture.continueLoop(GridDhtTxAbstractEnlistFuture.java:533) at org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxAbstractEnlistFuture.init(GridDhtTxAbstractEnlistFuture.java:362) at org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxEnlistFuture.enlistLocal(GridNearTxEnlistFuture.java:531) at org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxEnlistFuture.sendBatch(GridNearTxEnlistFuture.java:426) at org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxEnlistFuture.sendNextBatches(GridNearTxEnlistFuture.java:173) at org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxEnlistFuture.map(GridNearTxEnlistFuture.java:149) at org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxAbstractEnlistFuture.mapOnTopology(GridNearTxAbstractEnlistFuture.java:342) at org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxAbstractEnlistFuture.init(GridNearTxAbstractEnlistFuture.java:257) at org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal.updateAsync(GridNearTxLocal.java:2074) at org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal.mvccRemoveAllAsync0(GridNearTxLocal.java:1951) at org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal.removeAllAsync0(GridNearTxLocal.java:1670) at org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal.removeAllAsync(GridNearTxLocal.java:550)
Attachments
Issue Links
- incorporates
-
IGNITE-10254 MVCC: invokeAll may hangs on unstable topology.
- Closed
-
IGNITE-10391 MVCC: Invoke request fails on backup while rebalance is in progress.
- Closed
- is related to
-
IGNITE-10604 MVCC: mvcc history can be missed during remove operation.
- Resolved
- links to