[
https://issues.apache.org/jira/browse/IGNITE-16374?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Anton Vinogradov updated IGNITE-16374:
--------------------------------------
Parent: IGNITE-17845
Issue Type: Sub-task (was: Bug)
> Entry Processor may cause transactional cache inconsistency
> ------------------------------------------------------------
>
> Key: IGNITE-16374
> URL: https://issues.apache.org/jira/browse/IGNITE-16374
> Project: Ignite
> Issue Type: Sub-task
> Reporter: Anton Vinogradov
> Priority: Critical
> Labels: iep-31, ise
>
> At two-phase commit, primary invokes EP on prepare phase.
> In case EP will cause an exception, the operation will be retried or failed
> (when retry also caused an exception).
> Typical stacktrace is:
> {code}
> at
> org.apache.ignite.internal.processors.cache.EntryProcessorResourceInjectorProxy.process(EntryProcessorResourceInjectorProxy.java:68)
> at
> org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareFuture.onEntriesLocked(GridDhtTxPrepareFuture.java:460)
> at
> org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareFuture.prepare0(GridDhtTxPrepareFuture.java:1325)
> at
> org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareFuture.mapIfLocked(GridDhtTxPrepareFuture.java:732)
> at
> org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareFuture.prepare(GridDhtTxPrepareFuture.java:1138)
> at
> org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal.prepareAsyncLocal(GridNearTxLocal.java:4458)
> at
> org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler.prepareColocatedTx(IgniteTxHandler.java:302)
> at
> org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFuture.proceedPrepare(GridNearOptimisticTxPrepareFuture.java:565)
> at
> org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFuture.prepareSingle(GridNearOptimisticTxPrepareFuture.java:392)
> at
> org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFuture.prepare0(GridNearOptimisticTxPrepareFuture.java:335)
> at
> org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFutureAdapter.prepareOnTopology(GridNearOptimisticTxPrepareFutureAdapter.java:205)
> at
> org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFutureAdapter.prepare(GridNearOptimisticTxPrepareFutureAdapter.java:129)
> at
> org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal.prepareNearTxLocal(GridNearTxLocal.java:4122)
> at
> org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal.commitNearTxLocalAsync(GridNearTxLocal.java:4170)
> at
> org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal.optimisticPutFuture(GridNearTxLocal.java:3066)
> at
> org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal.putAsync0(GridNearTxLocal.java:731)
> at
> org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal.invokeAsync(GridNearTxLocal.java:508)
> at
> org.apache.ignite.internal.processors.cache.GridCacheAdapter$25.op(GridCacheAdapter.java:2695)
> at
> org.apache.ignite.internal.processors.cache.GridCacheAdapter$25.op(GridCacheAdapter.java:2682)
> at
> org.apache.ignite.internal.processors.cache.GridCacheAdapter.syncOp(GridCacheAdapter.java:4381)
> at
> org.apache.ignite.internal.processors.cache.GridCacheAdapter.invoke0(GridCacheAdapter.java:2682)
> at
> org.apache.ignite.internal.processors.cache.GridCacheAdapter.invoke(GridCacheAdapter.java:2663)
> at
> org.apache.ignite.internal.processors.cache.IgniteCacheProxyImpl.invoke(IgniteCacheProxyImpl.java:1735)
> at
> org.apache.ignite.internal.processors.cache.GatewayProtectedCacheProxy.invoke(GatewayProtectedCacheProxy.java:1254)
> {code}
> At the same time, EP processing on backup will happen in the finishing phase.
> At this time, primary already committed EP execution result, while EP
> processing on backup is not guaranteed to be successful.
> So, Exceptions during EP execution on backup will cause inconsistency.
> Typical stacktrace is:
> {code}
> at
> org.apache.ignite.internal.processors.cache.EntryProcessorResourceInjectorProxy.process(EntryProcessorResourceInjectorProxy.java:68)
> at
> org.apache.ignite.internal.processors.cache.transactions.IgniteTxAdapter.applyTransformClosures(IgniteTxAdapter.java:1700)
> at
> org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxRemoteAdapter.commitIfLocked(GridDistributedTxRemoteAdapter.java:574)
> at
> org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxRemoteAdapter.commitRemoteTx(GridDistributedTxRemoteAdapter.java:886)
> at
> org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler.finish(IgniteTxHandler.java:1471)
> at
> org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler.processDhtTxFinishRequest(IgniteTxHandler.java:1393)
> at
> org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler.access$600(IgniteTxHandler.java:134)
> at
> org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$7.apply(IgniteTxHandler.java:258)
> at
> org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$7.apply(IgniteTxHandler.java:256)
> at
> org.apache.ignite.internal.processors.cache.GridCacheIoManager.processMessage(GridCacheIoManager.java:1151)
> at
> org.apache.ignite.internal.processors.cache.GridCacheIoManager.onMessage0(GridCacheIoManager.java:592)
> at
> org.apache.ignite.internal.processors.cache.GridCacheIoManager.handleMessage(GridCacheIoManager.java:393)
> at
> org.apache.ignite.internal.processors.cache.GridCacheIoManager.handleMessage(GridCacheIoManager.java:319)
> at
> org.apache.ignite.internal.processors.cache.GridCacheIoManager.access$100(GridCacheIoManager.java:110)
> at
> org.apache.ignite.internal.processors.cache.GridCacheIoManager$1.onMessage(GridCacheIoManager.java:309)
> at
> org.apache.ignite.internal.managers.communication.GridIoManager.invokeListener(GridIoManager.java:1907)
> at
> org.apache.ignite.internal.managers.communication.GridIoManager.processRegularMessage0(GridIoManager.java:1528)
> at
> org.apache.ignite.internal.managers.communication.GridIoManager.access$5300(GridIoManager.java:242)
> at
> org.apache.ignite.internal.managers.communication.GridIoManager$9.execute(GridIoManager.java:1421)
> at
> org.apache.ignite.internal.managers.communication.TraceRunnable.run(TraceRunnable.java:55)
> at
> org.apache.ignite.internal.util.StripedExecutor$Stripe.body(StripedExecutor.java:569)
> at
> org.apache.ignite.internal.util.worker.GridWorker.run(GridWorker.java:125)
> {code}
> Possible solutions are
> 1) Retry on execution fail on backup without locks release until committed.
> This should guarantee consistency when successful EP execution is possible
> after some retries.
> But, this may cause a neverending unsuccessful retry story and even cause
> cluster failure.
> 2) EP execution on backups also should happen in the preparation phase.
> In this case, we will have all the data prepared on the finishing phase and
> have no chance to fail because of unsuccessful EP execution.
> 3) Another possible solution is to execute EP only on the primary node and
> update backups with the execution result, how atomics do.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)