Uploaded image for project: 'Ignite'
  1. Ignite
  2. IGNITE-10794

MVCC: RemoveAll is broken on unstable topology

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Critical
    • Resolution: Fixed
    • None
    • 2.8
    • mvcc

    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

          Activity

            People

              amashenkov Andrey Mashenkov
              amashenkov Andrey Mashenkov
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved:

                Time Tracking

                  Estimated:
                  Original Estimate - Not Specified
                  Not Specified
                  Remaining:
                  Remaining Estimate - 0h
                  0h
                  Logged:
                  Time Spent - 10m
                  10m