You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@ignite.apache.org by "Sergey Kosarev (JIRA)" <ji...@apache.org> on 2018/11/30 19:37:00 UTC

[jira] [Created] (IGNITE-10499) Stopping ignite node when a transaction on this node in process of commit(rollback) may cause hang up

Sergey Kosarev created IGNITE-10499:
---------------------------------------

             Summary: Stopping ignite node when a transaction on this node in process of commit(rollback) may cause hang up
                 Key: IGNITE-10499
                 URL: https://issues.apache.org/jira/browse/IGNITE-10499
             Project: Ignite
          Issue Type: Bug
            Reporter: Sergey Kosarev



1 start a node
2 start workload (transactions ) on this node
3 stop node

problem that checkpointReadLock throws IgniteException - unchecked - so it does not catched anywhere
and brokes commit process GridNearTxLocal#finishFutfinishFutrure stays unfinished and
*morover it don't lsten prepareFut.*
 
stacktrace to follow
{code}
018-11-30 20:37:56,508][ERROR][tx-load-thread-4][IgniteTxAdapter] Prepare error
class org.apache.ignite.IgniteException: Failed to perform cache update: node is stopping.
        at org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.checkpointReadLock(GridCacheDatabaseSharedManager.java:1501)
        at org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareFuture.onEntriesLocked(GridDhtTxPrepareFuture.java:371)
        at org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareFuture.prepare0(GridDhtTxPrepareFuture.java:1295)
        at org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareFuture.mapIfLocked(GridDhtTxPrepareFuture.java:710)
        at org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareFuture.prepare(GridDhtTxPrepareFuture.java:1100)
        at org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal.prepareAsyncLocal(GridNearTxLocal.java:4064)
        at org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler.prepareColocatedTx(IgniteTxHandler.java:292)
        at org.apache.ignite.internal.processors.cache.distributed.near.GridNearPessimisticTxPrepareFuture.prepareLocal(GridNearPessimisticTxPrepareFuture.java:258)
        at org.apache.ignite.internal.processors.cache.distributed.near.GridNearPessimisticTxPrepareFuture.preparePessimistic(GridNearPessimisticTxPrepareFuture.java:384)
        at org.apache.ignite.internal.processors.cache.distributed.near.GridNearPessimisticTxPrepareFuture.prepare(GridNearPessimisticTxPrepareFuture.java:188)
        at org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal.prepareNearTxLocal(GridNearTxLocal.java:3709)
        at org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal.commitNearTxLocalAsync(GridNearTxLocal.java:3761)
        at org.apache.ignite.internal.processors.cache.GridCacheAdapter.commitTxAsync(GridCacheAdapter.java:4214)
        at org.apache.ignite.internal.processors.cache.GridCacheSharedContext.commitTxAsync(GridCacheSharedContext.java:1034)
        at org.apache.ignite.internal.processors.cache.transactions.TransactionProxyImpl.commit(TransactionProxyImpl.java:297)
        at org.gridgain.grid.internal.processors.cache.database.txdr.AbstractReplicationTest.lambda$startTxLoad$2(AbstractReplicationTest.java:694)
        at org.apache.ignite.testframework.GridTestUtils$7.call(GridTestUtils.java:1300)
        at org.apache.ignite.testframework.GridTestThread.run(GridTestThread.java:84)
Caused by: class org.apache.ignite.internal.NodeStoppingException: Failed to perform cache update: node is stopping.
{code}

so after commit caused Exception 
we are going to close as transaction Autoclosable
and wait forever on
{code}
org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal#close(boolean)
in rollbackNearTxLocalAsync(clearThreadMap, false).get();
{code}
and we are locked GridKernalGateway#readLock 

and so ignite.stop thread hangs on trying to acquire GridKernalGateway#writeLock




--
This message was sent by Atlassian JIRA
(v7.6.3#76005)