[ 
https://issues.apache.org/jira/browse/IGNITE-8559?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Dmitriy Pavlov updated IGNITE-8559:
-----------------------------------
    Fix Version/s:     (was: 2.6)
                   2.7

> WAL rollOver can be blocked by WAL iterator reservation
> -------------------------------------------------------
>
>                 Key: IGNITE-8559
>                 URL: https://issues.apache.org/jira/browse/IGNITE-8559
>             Project: Ignite
>          Issue Type: Improvement
>            Reporter: Alexey Goncharuk
>            Assignee: Sergey Chugunov
>            Priority: Critical
>             Fix For: 2.7
>
>
> I've got the following thread dump from one of the Ignite nodes (only 
> meaningful threads are kept for simplicity)
> WAL archiver is waiting for locked segment release
> TX commit is waiting for WAL rollover
> WAL rollover is blocked by the archiver
> Exchange is blocked by TX commit
> {code}
> "sys-stripe-55-#56%GRID%GridNodeName%" #246 daemon prio=5 os_prio=0 
> tid=0x00007fdd1eeff000 nid=0x164252 waiting on condition [0x00007fdb36eec000]
>    java.lang.Thread.State: WAITING (parking)
>       at sun.misc.Unsafe.park(Native Method)
>       - parking to wait for  <0x00007fe0a5e96278> (a 
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>       at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
>       at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitUninterruptibly(AbstractQueuedSynchronizer.java:1976)
>       at 
> org.apache.ignite.internal.util.IgniteUtils.awaitQuiet(IgniteUtils.java:7400)
>       at 
> org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager$FileWriteHandle.awaitNext(FileWriteAheadLogManager.java:2819)
>       at 
> org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager$FileWriteHandle.access$2900(FileWriteAheadLogManager.java:2390)
>       at 
> org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager.rollOver(FileWriteAheadLogManager.java:1065)
>       at 
> org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager.log(FileWriteAheadLogManager.java:715)
>       at 
> org.gridgain.grid.internal.processors.cache.database.snapshot.GridCacheSnapshotManager.onChangeTrackerPage(GridCacheSnapshotManager.java:2436)
>       at 
> org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager$9.applyx(GridCacheDatabaseSharedManager.java:942)
>       at 
> org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager$9.applyx(GridCacheDatabaseSharedManager.java:935)
>       at 
> org.apache.ignite.internal.util.lang.GridInClosure3X.apply(GridInClosure3X.java:34)
>       at 
> org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryImpl.writeUnlockPage(PageMemoryImpl.java:1341)
>       at 
> org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryImpl.writeUnlock(PageMemoryImpl.java:415)
>       at 
> org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryImpl.writeUnlock(PageMemoryImpl.java:409)
>       at 
> org.apache.ignite.internal.processors.cache.persistence.tree.util.PageHandler.writeUnlock(PageHandler.java:377)
>       at 
> org.apache.ignite.internal.processors.cache.persistence.tree.util.PageHandler.writePage(PageHandler.java:287)
>       at 
> org.apache.ignite.internal.processors.cache.persistence.DataStructure.write(DataStructure.java:282)
>       at 
> org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList.insertDataRow(AbstractFreeList.java:509)
>       at 
> org.apache.ignite.internal.processors.cache.persistence.RowStore.addRow(RowStore.java:102)
>       at 
> org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManagerImpl$CacheDataStoreImpl.createRow(IgniteCacheOffheapManagerImpl.java:1252)
>       at 
> org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager$GridCacheDataStore.createRow(GridCacheOffheapManager.java:1370)
>       at 
> org.apache.ignite.internal.processors.cache.GridCacheMapEntry$UpdateClosure.call(GridCacheMapEntry.java:4429)
>       at 
> org.apache.ignite.internal.processors.cache.GridCacheMapEntry$UpdateClosure.call(GridCacheMapEntry.java:4374)
>       at 
> org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree$Invoke.invokeClosure(BPlusTree.java:3083)
>       at 
> org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree$Invoke.access$6200(BPlusTree.java:2977)
>       at 
> org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree.invokeDown(BPlusTree.java:1726)
>       at 
> org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree.invokeDown(BPlusTree.java:1703)
>       at 
> org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree.invokeDown(BPlusTree.java:1703)
>       at 
> org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree.invoke(BPlusTree.java:1610)
>       at 
> org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManagerImpl$CacheDataStoreImpl.invoke(IgniteCacheOffheapManagerImpl.java:1199)
>       at 
> org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager$GridCacheDataStore.invoke(GridCacheOffheapManager.java:1378)
>       at 
> org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManagerImpl.invoke(IgniteCacheOffheapManagerImpl.java:345)
>       at 
> org.apache.ignite.internal.processors.cache.GridCacheMapEntry.storeValue(GridCacheMapEntry.java:3605)
>       at 
> org.apache.ignite.internal.processors.cache.GridCacheMapEntry.storeValue(GridCacheMapEntry.java:3581)
>       at 
> org.apache.ignite.internal.processors.cache.GridCacheMapEntry.innerSet(GridCacheMapEntry.java:1040)
>       at 
> org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxRemoteAdapter.commitIfLocked(GridDistributedTxRemoteAdapter.java:609)
>       at 
> org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxRemoteAdapter.commitRemoteTx(GridDistributedTxRemoteAdapter.java:813)
>       at 
> org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler.finish(IgniteTxHandler.java:1319)
>       at 
> org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler.processDhtTxFinishRequest(IgniteTxHandler.java:1231)
>       at 
> org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler.access$600(IgniteTxHandler.java:97)
>       at 
> org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$7.apply(IgniteTxHandler.java:213)
>       at 
> org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$7.apply(IgniteTxHandler.java:211)
>       at 
> org.apache.ignite.internal.processors.cache.GridCacheIoManager.processMessage(GridCacheIoManager.java:1060)
>       at 
> org.apache.ignite.internal.processors.cache.GridCacheIoManager.onMessage0(GridCacheIoManager.java:579)
>       at 
> org.apache.ignite.internal.processors.cache.GridCacheIoManager.handleMessage(GridCacheIoManager.java:378)
>       at 
> org.apache.ignite.internal.processors.cache.GridCacheIoManager.handleMessage(GridCacheIoManager.java:304)
>       at 
> org.apache.ignite.internal.processors.cache.GridCacheIoManager.access$100(GridCacheIoManager.java:99)
>       at 
> org.apache.ignite.internal.processors.cache.GridCacheIoManager$1.onMessage(GridCacheIoManager.java:293)
>       at 
> org.apache.ignite.internal.managers.communication.GridIoManager.invokeListener(GridIoManager.java:1555)
>       at 
> org.apache.ignite.internal.managers.communication.GridIoManager.processRegularMessage0(GridIoManager.java:1183)
>       at 
> org.apache.ignite.internal.managers.communication.GridIoManager.access$4200(GridIoManager.java:126)
>       at 
> org.apache.ignite.internal.managers.communication.GridIoManager$9.run(GridIoManager.java:1090)
>       at 
> org.apache.ignite.internal.util.StripedExecutor$Stripe.run(StripedExecutor.java:499)
>       at java.lang.Thread.run(Thread.java:748)
> "exchange-worker-#152%GRID%GridNodeName%" #431 daemon prio=5 os_prio=0 
> tid=0x00007fdd1efd3000 nid=0x167b23 waiting on condition [0x00007fda375f4000]
>    java.lang.Thread.State: WAITING (parking)
>       at sun.misc.Unsafe.park(Native Method)
>       - parking to wait for  <0x00007fe1222b2838> (a 
> java.util.concurrent.locks.ReentrantLock$NonfairSync)
>       at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
>       at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)
>       at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:870)
>       at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1199)
>       at 
> java.util.concurrent.locks.ReentrantLock$NonfairSync.lock(ReentrantLock.java:209)
>       at java.util.concurrent.locks.ReentrantLock.lock(ReentrantLock.java:285)
>       at 
> org.apache.ignite.internal.processors.cache.GridCacheMapEntry.lockEntry(GridCacheMapEntry.java:4275)
>       at 
> org.apache.ignite.internal.processors.cache.distributed.GridDistributedCacheEntry.removeExplicitNodeLocks(GridDistributedCacheEntry.java:266)
>       at 
> org.apache.ignite.internal.processors.cache.GridCacheMvccManager.removeExplicitNodeLocks(GridCacheMvccManager.java:322)
>       at 
> org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture.onLeft(GridDhtPartitionsExchangeFuture.java:1241)
>       at 
> org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture.onClientNodeEvent(GridDhtPartitionsExchangeFuture.java:988)
>       at 
> org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture.init(GridDhtPartitionsExchangeFuture.java:657)
>       at 
> org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$ExchangeWorker.body(GridCachePartitionExchangeManager.java:2329)
>       at 
> org.apache.ignite.internal.util.worker.GridWorker.run(GridWorker.java:110)
>       at java.lang.Thread.run(Thread.java:748)
> "wal-file-archiver%GRID%GridNodeName" #628 daemon prio=5 os_prio=0 
> tid=0x00007fd9fc003800 nid=0x1708d4 in Object.wait() [0x00007fda2d9de000]
>    java.lang.Thread.State: WAITING (on object monitor)
>       at java.lang.Object.wait(Native Method)
>       at java.lang.Object.wait(Object.java:502)
>       at 
> org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager$FileArchiver.run(FileWriteAheadLogManager.java:1540)
>       - locked <0x00007fdf5a000620> (a 
> org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager$FileArchiver)
>    Locked ownable synchronizers:
>       - None
> {code}
> On the first glance, the most robust fix would be to fix iterator not to lock 
> segments, but prefetch the necessary amount of data from files.



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

Reply via email to