You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@ignite.apache.org by "Semeria, Vincent" <Vi...@finastra.com> on 2021/10/16 12:01:22 UTC

RE: [EXT] Re: Crash of Ignite (B+Tree corrupted) on a large PutIfAbsent

Yes indeed, setting the WAL segment size to 150 megabytes accepts my PutIfAbsent. Thanks.

Will you add a test in Ignite to avoid the crash, give a clearer error message and mention in the documentation that the WAL segment size should be higher than any single cache entry ? At the moment the doc just says this
        //
        // Summary:
        //     Gets or sets the size of the WAL (Write Ahead Log) segment. For performance reasons,
        //     the whole WAL is split into files of fixed length called segments.

The limit should also be written in this page
Ignite Persistence | Ignite Documentation (apache.org)<https://ignite.apache.org/docs/latest/persistence/native-persistence>

From: Alex Plehanov <pl...@gmail.com>
Sent: Friday, October 15, 2021 16:12
To: user@ignite.apache.org
Subject: [EXT] Re: Crash of Ignite (B+Tree corrupted) on a large PutIfAbsent

Hello,

Perhaps you have too small WAL segment size (WAL segment should be large enough to fit the whole cache entry), try to change DataStorageConfiguration.WalSegmentSize property.

чт, 14 окт. 2021 г. в 00:43, Semeria, Vincent <Vi...@finastra.com>>:
Hello Ignite,

I currently use the C# API of Ignite 2.10 to store large objects of type V in an ICache<string, V>. Typically an object of V is around 100 megabytes. My data region is persisted on the hard drive. PutIfAbsent crashes Ignite with the complicated message below. As a workaround, I split type V into smaller types and used loops of smaller PutIfAbsent, which succeeded. Ultimately the data stored in the cache is the same, which shows that Ignite accepts my data (this is not a problem in the binary serializer).

Is there a configuration of the data region that would accept a single PutIfAbsent of 100 megabytes ?

Anyway Ignite should probably not crash when this limit is exceeded. Please send a clean error instead like “Insertion request exceeds limit XYZ” and keep Ignite alive in this case.

Regards,
Vincent Semeria


            error : PerformanceAttributionServer[PerformanceAttributionServer1]_Global@PARDH7JQHS2 (41660.6010) : [2021/10/12-10:21:48.215] : Apache.Ignite.NLog.IgniteNLogLogger::LoggerLog() : Critical system error detected. Will be handled accordingly to configured handler [hnd=StopNodeOrHaltFailureHandler [tryStop=false, timeout=0, super=AbstractFailureHandler [ignoredFailureTypes=UnmodifiableSet [SYSTEM_WORKER_BLOCKED, SYSTEM_CRITICAL_OPERATION_TIMEOUT]]], failureCtx=FailureContext [type=CRITICAL_ERROR, err=class o.a.i.i.processors.cache.persistence.tree.CorruptedTreeException: B+Tree is corrupted [pages(groupId, pageId)=[IgniteBiTuple [val1=241659666, val2=1127239936638982]], msg=Runtime failure on search row: SearchRow [key=KeyCacheObjectImpl [part=312, val=56ae72d3-a91a-4211-8279-0b0447881544, hasValBytes=true], hash=746501958, cacheId=0]]]]
            error : PerformanceAttributionServer[PerformanceAttributionServer1]_Global@PARDH7JQHS2 (41660.6010) : [2021/10/12-10:21:48.216] : org.apache.ignite.internal.processors.failure.FailureProcessor::LoggerLog() : A critical problem with persistence data structures was detected. Please make backup of persistence storage and WAL files for further analysis. Persistence storage path:  WAL path: db/wal WAL archive path: db/wal/archive
            error : PerformanceAttributionServer[PerformanceAttributionServer1]_Global@PARDH7JQHS2 (41660.6010) : [2021/10/12-10:21:48.219] : org.apache.ignite.internal.processors.failure.FailureProcessor::LoggerLog() : No deadlocked threads detected.
            error : PerformanceAttributionServer[PerformanceAttributionServer1]_Global@PARDH7JQHS2 (41660.6010) : [2021/10/12-10:21:48.276] : org.apache.ignite.internal.processors.failure.FailureProcessor::LoggerLog() : Thread dump at 2021/10/12 10:21:48 CEST
Thread [name="sys-#200", id=235, state=TIMED_WAITING, blockCnt=0, waitCnt=1]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@529e4706, ownerName=null, ownerId=-1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
        at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
        at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="sys-#199", id=234, state=TIMED_WAITING, blockCnt=0, waitCnt=1]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@529e4706, ownerName=null, ownerId=-1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
        at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
        at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="sys-#198", id=233, state=TIMED_WAITING, blockCnt=0, waitCnt=1]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@529e4706, ownerName=null, ownerId=-1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
        at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
        at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="sys-#197", id=232, state=TIMED_WAITING, blockCnt=0, waitCnt=1]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@529e4706, ownerName=null, ownerId=-1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
        at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
        at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="sys-#196", id=231, state=TIMED_WAITING, blockCnt=0, waitCnt=1]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@529e4706, ownerName=null, ownerId=-1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
        at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
        at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="sys-#195", id=230, state=TIMED_WAITING, blockCnt=0, waitCnt=1]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@529e4706, ownerName=null, ownerId=-1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
        at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
        at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="sys-#194", id=229, state=TIMED_WAITING, blockCnt=0, waitCnt=1]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@529e4706, ownerName=null, ownerId=-1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
        at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
        at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="Thread-18", id=204, state=RUNNABLE, blockCnt=0, waitCnt=0]

Thread [name="Thread-17", id=176, state=RUNNABLE, blockCnt=0, waitCnt=2]

Thread [name="Thread-16", id=168, state=RUNNABLE, blockCnt=0, waitCnt=0]

Thread [name="Thread-15", id=117, state=RUNNABLE, blockCnt=0, waitCnt=0]

Thread [name="Thread-14", id=116, state=RUNNABLE, blockCnt=0, waitCnt=0]

Thread [name="Thread-13", id=115, state=RUNNABLE, blockCnt=0, waitCnt=0]

Thread [name="Thread-12", id=111, state=RUNNABLE, blockCnt=0, waitCnt=82]
        at sun.management.ThreadImpl.dumpThreads0(Native Method)
        at sun.management.ThreadImpl.dumpAllThreads(ThreadImpl.java:454)
        at o.a.i.i.util.IgniteUtils.dumpThreads(IgniteUtils.java:1471)
        at o.a.i.i.processors.failure.FailureProcessor.process(FailureProcessor.java:192)
        - locked o.a.i.i.processors.failure.FailureProcessor@67349fda<ma...@67349fda>
        at o.a.i.i.processors.failure.FailureProcessor.process(FailureProcessor.java:155)
        at o.a.i.i.processors.cache.persistence.tree.BPlusTree.processFailure(BPlusTree.java:6152)
        at o.a.i.i.processors.cache.persistence.tree.BPlusTree.corruptedTreeException(BPlusTree.java:6139)
        at o.a.i.i.processors.cache.persistence.tree.BPlusTree.invoke(BPlusTree.java:1953)
        at o.a.i.i.processors.cache.IgniteCacheOffheapManagerImpl$CacheDataStoreImpl.invoke0(IgniteCacheOffheapManagerImpl.java:1758)
        at o.a.i.i.processors.cache.IgniteCacheOffheapManagerImpl$CacheDataStoreImpl.invoke(IgniteCacheOffheapManagerImpl.java:1741)
        at o.a.i.i.processors.cache.persistence.GridCacheOffheapManager$GridCacheDataStore.invoke(GridCacheOffheapManager.java:2766)
        at o.a.i.i.processors.cache.IgniteCacheOffheapManagerImpl.invoke(IgniteCacheOffheapManagerImpl.java:439)
        at o.a.i.i.processors.cache.GridCacheMapEntry.innerUpdate(GridCacheMapEntry.java:2338)
        at o.a.i.i.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.updateSingle(GridDhtAtomicCache.java:2654)
        at o.a.i.i.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.update(GridDhtAtomicCache.java:2114)
        at o.a.i.i.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.updateAllAsyncInternal0(GridDhtAtomicCache.java:1931)
        at o.a.i.i.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.updateAllAsyncInternal(GridDhtAtomicCache.java:1724)
        at o.a.i.i.processors.cache.distributed.dht.atomic.GridNearAtomicAbstractUpdateFuture.sendSingleRequest(GridNearAtomicAbstractUpdateFuture.java:306)
        at o.a.i.i.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateFuture.map(GridNearAtomicSingleUpdateFuture.java:486)
        at o.a.i.i.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateFuture.mapOnTopology(GridNearAtomicSingleUpdateFuture.java:446)
        at o.a.i.i.processors.cache.distributed.dht.atomic.GridNearAtomicAbstractUpdateFuture.map(GridNearAtomicAbstractUpdateFuture.java:255)
        at o.a.i.i.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.update0(GridDhtAtomicCache.java:1169)
        at o.a.i.i.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.put0(GridDhtAtomicCache.java:634)
        at o.a.i.i.processors.cache.GridCacheAdapter.put(GridCacheAdapter.java:2596)
        at o.a.i.i.processors.cache.GridCacheAdapter.putIfAbsent(GridCacheAdapter.java:3041)
        at o.a.i.i.processors.cache.IgniteCacheProxyImpl.putIfAbsent(IgniteCacheProxyImpl.java:1412)
        at o.a.i.i.processors.cache.GatewayProtectedCacheProxy.putIfAbsent(GatewayProtectedCacheProxy.java:928)
        at o.a.i.i.processors.platform.cache.PlatformCache.processInStreamOutLong(PlatformCache.java:516)
        at o.a.i.i.processors.platform.PlatformTargetProxyImpl.inStreamOutLong(PlatformTargetProxyImpl.java:67)

    Locked synchronizers:
        java.util.concurrent.locks.ReentrantLock$NonfairSync@3de6a6db<ma...@3de6a6db>
Thread [name="Thread-11", id=102, state=RUNNABLE, blockCnt=0, waitCnt=0]

Thread [name="ttl-cleanup-worker-#72", id=99, state=TIMED_WAITING, blockCnt=0, waitCnt=2297]
        at java.lang.Thread.sleep(Native Method)
        at o.a.i.i.util.IgniteUtils.sleep(IgniteUtils.java:7984)
        at o.a.i.i.processors.cache.GridCacheSharedTtlCleanupManager$CleanupWorker.body(GridCacheSharedTtlCleanupManager.java:210)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="checkpoint-runner-IO-#71", id=98, state=WAITING, blockCnt=1, waitCnt=722]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@565aa90c, ownerName=null, ownerId=-1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
        at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="checkpoint-runner-IO-#70", id=97, state=WAITING, blockCnt=2, waitCnt=585]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@565aa90c, ownerName=null, ownerId=-1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
        at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="checkpoint-runner-IO-#69", id=96, state=WAITING, blockCnt=4, waitCnt=660]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@565aa90c, ownerName=null, ownerId=-1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
        at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="checkpoint-runner-IO-#68", id=95, state=WAITING, blockCnt=2, waitCnt=867]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@565aa90c, ownerName=null, ownerId=-1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
        at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="checkpoint-runner-cpu-#65", id=92, state=WAITING, blockCnt=54, waitCnt=478]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@76dc62b6, ownerName=null, ownerId=-1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
        at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="checkpoint-runner-cpu-#64", id=91, state=WAITING, blockCnt=38, waitCnt=494]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@76dc62b6, ownerName=null, ownerId=-1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
        at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="checkpoint-runner-cpu-#63", id=90, state=WAITING, blockCnt=75, waitCnt=505]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@76dc62b6, ownerName=null, ownerId=-1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
        at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="checkpoint-runner-cpu-#62", id=89, state=WAITING, blockCnt=17, waitCnt=545]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@76dc62b6, ownerName=null, ownerId=-1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
        at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="db-checkpoint-thread-#61", id=88, state=TIMED_WAITING, blockCnt=0, waitCnt=45]
    Lock [object=o.a.i.i.processors.cache.persistence.checkpoint.Checkpointer@35d5f9f8, ownerName=null, ownerId=-1]
        at java.lang.Object.wait(Native Method)
        at o.a.i.i.processors.cache.persistence.checkpoint.Checkpointer.waitCheckpointEvent(Checkpointer.java:755)
        at o.a.i.i.processors.cache.persistence.checkpoint.Checkpointer.body(Checkpointer.java:246)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="dms-writer-thread-#55", id=82, state=WAITING, blockCnt=1, waitCnt=1]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@3d8d51e7, ownerName=null, ownerId=-1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
        at o.a.i.i.processors.metastorage.persistence.DmsDataWriterWorker.body(DmsDataWriterWorker.java:146)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="wal-segment-syncer-#54", id=81, state=TIMED_WAITING, blockCnt=0, waitCnt=4577]
        at java.lang.Thread.sleep(Native Method)
        at o.a.i.i.util.IgniteUtils.sleep(IgniteUtils.java:7984)
        at o.a.i.i.processors.cache.persistence.wal.filehandle.FileHandleManagerImpl$WalSegmentSyncer.body(FileHandleManagerImpl.java:622)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="wal-file-cleaner%null-#53", id=80, state=WAITING, blockCnt=0, waitCnt=1]
    Lock [object=o.a.i.i.processors.cache.persistence.wal.aware.SegmentArchiveSizeStorage@3e9c9d93, ownerName=null, ownerId=-1]
        at java.lang.Object.wait(Native Method)
        at java.lang.Object.wait(Object.java:502)
        at o.a.i.i.processors.cache.persistence.wal.aware.SegmentArchiveSizeStorage.awaitExceedMaxSize(SegmentArchiveSizeStorage.java:79)
        at o.a.i.i.processors.cache.persistence.wal.aware.SegmentAware.awaitExceedMaxArchiveSize(SegmentAware.java:354)
        at o.a.i.i.processors.cache.persistence.wal.FileWriteAheadLogManager$FileCleaner.body(FileWriteAheadLogManager.java:3171)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
        at java.lang.Thread.run(          warning : PerformanceAttributionServer[PerformanceAttributionServer1]_Global@PARDH7JQHS2 (41660.6010) : [2021/10/12-10:21:48.296] : org.apache.ignite.internal.processors.cache.CacheDiagnosticManager::LoggerLog() : Page locks dump:

Thread=[name=Thread-12, id=111], state=RUNNABLE
Locked pages = []
Locked pages log: name=Thread-12 time=(1634026908291, 2021-10-12 10:21:48.291)


Thread=[name=Thread-13, id=115], state=RUNNABLE
Locked pages = []
Locked pages log: name=Thread-13 time=(1634026908291, 2021-10-12 10:21:48.291)


Thread=[name=Thread-14, id=116], state=RUNNABLE
Locked pages = []
Locked pages log: name=Thread-14 time=(1634026908291, 2021-10-12 10:21:48.291)


Thread=[name=Thread-15, id=117], state=RUNNABLE
Locked pages = []
Locked pages log: name=Thread-15 time=(1634026908291, 2021-10-12 10:21:48.291)


Thread=[name=Thread-16, id=168], state=RUNNABLE
Locked pages = []
Locked pages log: name=Thread-16 time=(1634026908291, 2021-10-12 10:21:48.291)


Thread=[name=Thread-17, id=176], state=RUNNABLE
Locked pages = []
Locked pages log: name=Thread-17 time=(1634026908291, 2021-10-12 10:21:48.291)


Thread=[name=Thread-18, id=204], state=RUNNABLE
Locked pages = []
Locked pages log: name=Thread-18 time=(1634026908291, 2021-10-12 10:21:48.291)


Thread=[name=checkpoint-runner-cpu-#62, id=89], state=WAITING
Locked pages = []
Locked pages log: name=checkpoint-runner-cpu-#62 time=(1634026908291, 2021-10-12 10:21:48.291)


Thread=[name=checkpoint-runner-cpu-#63, id=90], state=WAITING
Locked pages = []
Locked pages log: name=checkpoint-runner-cpu-#63 time=(1634026908291, 2021-10-12 10:21:48.291)


Thread=[name=db-checkpoint-thread-#61, id=88], state=TIMED_WAITING
Locked pages = []
Locked pages log: name=db-checkpoint-thread-#61 time=(1634026908291, 2021-10-12 10:21:48.291)


Thread=[name=dms-writer-thread-#55, id=82], state=WAITING
Locked pages = []
Locked pages log: name=dms-writer-thread-#55 time=(1634026908291, 2021-10-12 10:21:48.291)


Thread=[name=exchange-worker-#50, id=76], state=TIMED_WAITING
Locked pages = []
Locked pages log: name=exchange-worker-#50 time=(1634026908291, 2021-10-12 10:21:48.291)


Thread=[name=main, id=1], state=RUNNABLE
Locked pages = []
Locked pages log: name=main time=(1634026908291, 2021-10-12 10:21:48.291)



Reached logical end of the segment for file D:\sophis_beta\tests\installs\Debug\servers\sophis\PerformanceAttribution\PerformanceAttributionServer\work\db\wal\node00-d410e934-f1a8-474b-b5c2-a5dc783b186c\0000000000000001.wal
Reached logical end of the segment for file D:\sophis_beta\tests\installs\Debug\servers\sophis\PerformanceAttribution\PerformanceAttributionServer\work\db\wal\node00-d410e934-f1a8-474b-b5c2-a5dc783b186c\0000000000000002.wal
Reached logical end of the segment for file D:\sophis_beta\tests\installs\Debug\servers\sophis\PerformanceAttribution\PerformanceAttributionServer\work\db\wal\node00-d410e934-f1a8-474b-b5c2-a5dc783b186c\0000000000000003.wal
Reached logical end of the segment for file D:\sophis_beta\tests\installs\Debug\servers\sophis\PerformanceAttribution\PerformanceAttributionServer\work\db\wal\node00-d410e934-f1a8-474b-b5c2-a5dc783b186c\0000000000000004.wal
Reached logical end of the segment for file D:\sophis_beta\tests\installs\Debug\servers\sophis\PerformanceAttribution\PerformanceAttributionServer\work\db\wal\node00-d410e934-f1a8-474b-b5c2-a5dc783b186c\0000000000000005.wal
Reached logical end of the segment for file D:\sophis_beta\tests\installs\Debug\servers\sophis\PerformanceAttribution\PerformanceAttributionServer\work\db\wal\node00-d410e934-f1a8-474b-b5c2-a5dc783b186c\0000000000000006.wal
Reached logical end of the segment for file D:\sophis_beta\tests\installs\Debug\servers\sophis\PerformanceAttribution\PerformanceAttributionServer\work\db\wal\node00-d410e934-f1a8-474b-b5c2-a5dc783b186c\0000000000000007.wal
Reached logical end of the segment for file D:\sophis_beta\tests\installs\Debug\servers\sophis\PerformanceAttribution\PerformanceAttributionServer\work\db\wal\node00-d410e934-f1a8-474b-b5c2-a5dc783b186c\0000000000000008.wal
Reached logical end of the segment for file D:\sophis_beta\tests\installs\Debug\servers\sophis\PerformanceAttribution\PerformanceAttributionServer\work\db\wal\node00-d410e934-f1a8-474b-b5c2-a5dc783b186c\0000000000000009.wal
            error : PerformanceAttributionServer[PerformanceAttributionServer1]_Global@PARDH7JQHS2 (41660.6010) : [2021/10/12-10:21:48.574] : Apache.Ignite.NLog.IgniteNLogLogger::LoggerLog() : JVM will be halted immediately due to the failure: [failureCtx=FailureContext [type=CRITICAL_ERROR, err=class o.a.i.i.processors.cache.persistence.tree.CorruptedTreeException: B+Tree is corrupted [pages(groupId, pageId)=[IgniteBiTuple [val1=241659666, val2=1127239936638982]], msg=Runtime failure on search row: SearchRow [key=KeyCacheObjectImpl [part=312, val=56ae72d3-a91a-4211-8279-0b0447881544, hasValBytes=true], hash=746501958, cacheId=0]]]]
Application is shutting down...

"FINASTRA" is the trade name of the FINASTRA group of companies. This email and any attachments have been scanned for known viruses using multiple scanners. This email message is intended for the named recipient only. It may be privileged and/or confidential. If you are not the named recipient of this email please notify us immediately and do not copy it or use it for any purpose, nor disclose its contents to any other person. This email does not constitute the commencement of legal relations between you and FINASTRA. Please refer to the executed contract between you and the relevant member of the FINASTRA group for the identity of the contracting party with which you are dealing.
"FINASTRA" is the trade name of the FINASTRA group of companies. This email and any attachments have been scanned for known viruses using multiple scanners. This email message is intended for the named recipient only. It may be privileged and/or confidential. If you are not the named recipient of this email please notify us immediately and do not copy it or use it for any purpose, nor disclose its contents to any other person. This email does not constitute the commencement of legal relations between you and FINASTRA. Please refer to the executed contract between you and the relevant member of the FINASTRA group for the identity of the contracting party with which you are dealing.

RE: [EXT] Re: Crash of Ignite (B+Tree corrupted) on a large PutIfAbsent

Posted by "Semeria, Vincent" <Vi...@finastra.com>.
> some changes can be already made in the DB prior to the exception
Ok. Then fixing the .NET error and the documentation would be nice.

From: Alex Plehanov <pl...@gmail.com>
Sent: lundi 18 octobre 2021 14:55
To: user@ignite.apache.org
Subject: Re: [EXT] Re: Crash of Ignite (B+Tree corrupted) on a large PutIfAbsent

> Will you add a test in Ignite to avoid the crash
I'm not quite sure, but perhaps a crash is the best we can do in this case. Throwing exception to the user might be not enough, since some changes can be already made in the DB prior to the exception and this can lead to data inconsistency.

> give a clearer error message
Nested exception already contains a root cause of the problem (For example: "Record is too long [capacity=67108864, size=92984514]", see ticket with the same problem reproduced with java [1]), but perhaps this nested exception is not correctly displayed by Ignite .NET

[1]: https://issues.apache.org/jira/browse/IGNITE-13965<https://eur02.safelinks.protection.outlook.com/?url=https%3A%2F%2Fissues.apache.org%2Fjira%2Fbrowse%2FIGNITE-13965&data=04%7C01%7CVincent.Semeria%40finastra.com%7Cd54873a188604413936908d9923686c0%7C0b9b90da3fe1457ab340f1b67e1024fb%7C0%7C0%7C637701585167858681%7CUnknown%7CTWFpbGZsb3d8eyJWIjoiMC4wLjAwMDAiLCJQIjoiV2luMzIiLCJBTiI6Ik1haWwiLCJXVCI6Mn0%3D%7C1000&sdata=ZXPgqf1vuz%2FzPJ95DbFxDxaKdjjHtrdcjFqUZ%2BeTqWE%3D&reserved=0>

сб, 16 окт. 2021 г. в 15:02, Semeria, Vincent <Vi...@finastra.com>>:
Yes indeed, setting the WAL segment size to 150 megabytes accepts my PutIfAbsent. Thanks.

Will you add a test in Ignite to avoid the crash, give a clearer error message and mention in the documentation that the WAL segment size should be higher than any single cache entry ? At the moment the doc just says this
        //
        // Summary:
        //     Gets or sets the size of the WAL (Write Ahead Log) segment. For performance reasons,
        //     the whole WAL is split into files of fixed length called segments.

The limit should also be written in this page
Ignite Persistence | Ignite Documentation (apache.org)<https://eur02.safelinks.protection.outlook.com/?url=https%3A%2F%2Fignite.apache.org%2Fdocs%2Flatest%2Fpersistence%2Fnative-persistence&data=04%7C01%7CVincent.Semeria%40finastra.com%7Cd54873a188604413936908d9923686c0%7C0b9b90da3fe1457ab340f1b67e1024fb%7C0%7C0%7C637701585167868630%7CUnknown%7CTWFpbGZsb3d8eyJWIjoiMC4wLjAwMDAiLCJQIjoiV2luMzIiLCJBTiI6Ik1haWwiLCJXVCI6Mn0%3D%7C1000&sdata=esygpLK%2BBs9ZkH%2FO95%2F28J4E6ZgU2qvBw7dd7%2FjBF3o%3D&reserved=0>

From: Alex Plehanov <pl...@gmail.com>>
Sent: Friday, October 15, 2021 16:12
To: user@ignite.apache.org<ma...@ignite.apache.org>
Subject: [EXT] Re: Crash of Ignite (B+Tree corrupted) on a large PutIfAbsent

Hello,

Perhaps you have too small WAL segment size (WAL segment should be large enough to fit the whole cache entry), try to change DataStorageConfiguration.WalSegmentSize property.

чт, 14 окт. 2021 г. в 00:43, Semeria, Vincent <Vi...@finastra.com>>:
Hello Ignite,

I currently use the C# API of Ignite 2.10 to store large objects of type V in an ICache<string, V>. Typically an object of V is around 100 megabytes. My data region is persisted on the hard drive. PutIfAbsent crashes Ignite with the complicated message below. As a workaround, I split type V into smaller types and used loops of smaller PutIfAbsent, which succeeded. Ultimately the data stored in the cache is the same, which shows that Ignite accepts my data (this is not a problem in the binary serializer).

Is there a configuration of the data region that would accept a single PutIfAbsent of 100 megabytes ?

Anyway Ignite should probably not crash when this limit is exceeded. Please send a clean error instead like "Insertion request exceeds limit XYZ" and keep Ignite alive in this case.

Regards,
Vincent Semeria


            error : PerformanceAttributionServer[PerformanceAttributionServer1]_Global@PARDH7JQHS2 (41660.6010) : [2021/10/12-10:21:48.215] : Apache.Ignite.NLog.IgniteNLogLogger::LoggerLog() : Critical system error detected. Will be handled accordingly to configured handler [hnd=StopNodeOrHaltFailureHandler [tryStop=false, timeout=0, super=AbstractFailureHandler [ignoredFailureTypes=UnmodifiableSet [SYSTEM_WORKER_BLOCKED, SYSTEM_CRITICAL_OPERATION_TIMEOUT]]], failureCtx=FailureContext [type=CRITICAL_ERROR, err=class o.a.i.i.processors.cache.persistence.tree.CorruptedTreeException: B+Tree is corrupted [pages(groupId, pageId)=[IgniteBiTuple [val1=241659666, val2=1127239936638982]], msg=Runtime failure on search row: SearchRow [key=KeyCacheObjectImpl [part=312, val=56ae72d3-a91a-4211-8279-0b0447881544, hasValBytes=true], hash=746501958, cacheId=0]]]]
            error : PerformanceAttributionServer[PerformanceAttributionServer1]_Global@PARDH7JQHS2 (41660.6010) : [2021/10/12-10:21:48.216] : org.apache.ignite.internal.processors.failure.FailureProcessor::LoggerLog() : A critical problem with persistence data structures was detected. Please make backup of persistence storage and WAL files for further analysis. Persistence storage path:  WAL path: db/wal WAL archive path: db/wal/archive
            error : PerformanceAttributionServer[PerformanceAttributionServer1]_Global@PARDH7JQHS2 (41660.6010) : [2021/10/12-10:21:48.219] : org.apache.ignite.internal.processors.failure.FailureProcessor::LoggerLog() : No deadlocked threads detected.
            error : PerformanceAttributionServer[PerformanceAttributionServer1]_Global@PARDH7JQHS2 (41660.6010) : [2021/10/12-10:21:48.276] : org.apache.ignite.internal.processors.failure.FailureProcessor::LoggerLog() : Thread dump at 2021/10/12 10:21:48 CEST
Thread [name="sys-#200", id=235, state=TIMED_WAITING, blockCnt=0, waitCnt=1]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@529e4706, ownerName=null, ownerId=-1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
        at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
        at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="sys-#199", id=234, state=TIMED_WAITING, blockCnt=0, waitCnt=1]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@529e4706, ownerName=null, ownerId=-1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
        at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
        at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="sys-#198", id=233, state=TIMED_WAITING, blockCnt=0, waitCnt=1]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@529e4706, ownerName=null, ownerId=-1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
        at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
        at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="sys-#197", id=232, state=TIMED_WAITING, blockCnt=0, waitCnt=1]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@529e4706, ownerName=null, ownerId=-1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
        at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
        at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="sys-#196", id=231, state=TIMED_WAITING, blockCnt=0, waitCnt=1]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@529e4706, ownerName=null, ownerId=-1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
        at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
        at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="sys-#195", id=230, state=TIMED_WAITING, blockCnt=0, waitCnt=1]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@529e4706, ownerName=null, ownerId=-1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
        at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
        at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="sys-#194", id=229, state=TIMED_WAITING, blockCnt=0, waitCnt=1]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@529e4706, ownerName=null, ownerId=-1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
        at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
        at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="Thread-18", id=204, state=RUNNABLE, blockCnt=0, waitCnt=0]

Thread [name="Thread-17", id=176, state=RUNNABLE, blockCnt=0, waitCnt=2]

Thread [name="Thread-16", id=168, state=RUNNABLE, blockCnt=0, waitCnt=0]

Thread [name="Thread-15", id=117, state=RUNNABLE, blockCnt=0, waitCnt=0]

Thread [name="Thread-14", id=116, state=RUNNABLE, blockCnt=0, waitCnt=0]

Thread [name="Thread-13", id=115, state=RUNNABLE, blockCnt=0, waitCnt=0]

Thread [name="Thread-12", id=111, state=RUNNABLE, blockCnt=0, waitCnt=82]
        at sun.management.ThreadImpl.dumpThreads0(Native Method)
        at sun.management.ThreadImpl.dumpAllThreads(ThreadImpl.java:454)
        at o.a.i.i.util.IgniteUtils.dumpThreads(IgniteUtils.java:1471)
        at o.a.i.i.processors.failure.FailureProcessor.process(FailureProcessor.java:192)
        - locked o.a.i.i.processors.failure.FailureProcessor@67349fda<ma...@67349fda>
        at o.a.i.i.processors.failure.FailureProcessor.process(FailureProcessor.java:155)
        at o.a.i.i.processors.cache.persistence.tree.BPlusTree.processFailure(BPlusTree.java:6152)
        at o.a.i.i.processors.cache.persistence.tree.BPlusTree.corruptedTreeException(BPlusTree.java:6139)
        at o.a.i.i.processors.cache.persistence.tree.BPlusTree.invoke(BPlusTree.java:1953)
        at o.a.i.i.processors.cache.IgniteCacheOffheapManagerImpl$CacheDataStoreImpl.invoke0(IgniteCacheOffheapManagerImpl.java:1758)
        at o.a.i.i.processors.cache.IgniteCacheOffheapManagerImpl$CacheDataStoreImpl.invoke(IgniteCacheOffheapManagerImpl.java:1741)
        at o.a.i.i.processors.cache.persistence.GridCacheOffheapManager$GridCacheDataStore.invoke(GridCacheOffheapManager.java:2766)
        at o.a.i.i.processors.cache.IgniteCacheOffheapManagerImpl.invoke(IgniteCacheOffheapManagerImpl.java:439)
        at o.a.i.i.processors.cache.GridCacheMapEntry.innerUpdate(GridCacheMapEntry.java:2338)
        at o.a.i.i.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.updateSingle(GridDhtAtomicCache.java:2654)
        at o.a.i.i.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.update(GridDhtAtomicCache.java:2114)
        at o.a.i.i.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.updateAllAsyncInternal0(GridDhtAtomicCache.java:1931)
        at o.a.i.i.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.updateAllAsyncInternal(GridDhtAtomicCache.java:1724)
        at o.a.i.i.processors.cache.distributed.dht.atomic.GridNearAtomicAbstractUpdateFuture.sendSingleRequest(GridNearAtomicAbstractUpdateFuture.java:306)
        at o.a.i.i.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateFuture.map(GridNearAtomicSingleUpdateFuture.java:486)
        at o.a.i.i.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateFuture.mapOnTopology(GridNearAtomicSingleUpdateFuture.java:446)
        at o.a.i.i.processors.cache.distributed.dht.atomic.GridNearAtomicAbstractUpdateFuture.map(GridNearAtomicAbstractUpdateFuture.java:255)
        at o.a.i.i.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.update0(GridDhtAtomicCache.java:1169)
        at o.a.i.i.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.put0(GridDhtAtomicCache.java:634)
        at o.a.i.i.processors.cache.GridCacheAdapter.put(GridCacheAdapter.java:2596)
        at o.a.i.i.processors.cache.GridCacheAdapter.putIfAbsent(GridCacheAdapter.java:3041)
        at o.a.i.i.processors.cache.IgniteCacheProxyImpl.putIfAbsent(IgniteCacheProxyImpl.java:1412)
        at o.a.i.i.processors.cache.GatewayProtectedCacheProxy.putIfAbsent(GatewayProtectedCacheProxy.java:928)
        at o.a.i.i.processors.platform.cache.PlatformCache.processInStreamOutLong(PlatformCache.java:516)
        at o.a.i.i.processors.platform.PlatformTargetProxyImpl.inStreamOutLong(PlatformTargetProxyImpl.java:67)

    Locked synchronizers:
        java.util.concurrent.locks.ReentrantLock$NonfairSync@3de6a6db<ma...@3de6a6db>
Thread [name="Thread-11", id=102, state=RUNNABLE, blockCnt=0, waitCnt=0]

Thread [name="ttl-cleanup-worker-#72", id=99, state=TIMED_WAITING, blockCnt=0, waitCnt=2297]
        at java.lang.Thread.sleep(Native Method)
        at o.a.i.i.util.IgniteUtils.sleep(IgniteUtils.java:7984)
        at o.a.i.i.processors.cache.GridCacheSharedTtlCleanupManager$CleanupWorker.body(GridCacheSharedTtlCleanupManager.java:210)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="checkpoint-runner-IO-#71", id=98, state=WAITING, blockCnt=1, waitCnt=722]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@565aa90c, ownerName=null, ownerId=-1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
        at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="checkpoint-runner-IO-#70", id=97, state=WAITING, blockCnt=2, waitCnt=585]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@565aa90c, ownerName=null, ownerId=-1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
        at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="checkpoint-runner-IO-#69", id=96, state=WAITING, blockCnt=4, waitCnt=660]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@565aa90c, ownerName=null, ownerId=-1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
        at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="checkpoint-runner-IO-#68", id=95, state=WAITING, blockCnt=2, waitCnt=867]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@565aa90c, ownerName=null, ownerId=-1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
        at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="checkpoint-runner-cpu-#65", id=92, state=WAITING, blockCnt=54, waitCnt=478]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@76dc62b6, ownerName=null, ownerId=-1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
        at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="checkpoint-runner-cpu-#64", id=91, state=WAITING, blockCnt=38, waitCnt=494]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@76dc62b6, ownerName=null, ownerId=-1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
        at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="checkpoint-runner-cpu-#63", id=90, state=WAITING, blockCnt=75, waitCnt=505]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@76dc62b6, ownerName=null, ownerId=-1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
        at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="checkpoint-runner-cpu-#62", id=89, state=WAITING, blockCnt=17, waitCnt=545]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@76dc62b6, ownerName=null, ownerId=-1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
        at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="db-checkpoint-thread-#61", id=88, state=TIMED_WAITING, blockCnt=0, waitCnt=45]
    Lock [object=o.a.i.i.processors.cache.persistence.checkpoint.Checkpointer@35d5f9f8, ownerName=null, ownerId=-1]
        at java.lang.Object.wait(Native Method)
        at o.a.i.i.processors.cache.persistence.checkpoint.Checkpointer.waitCheckpointEvent(Checkpointer.java:755)
        at o.a.i.i.processors.cache.persistence.checkpoint.Checkpointer.body(Checkpointer.java:246)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="dms-writer-thread-#55", id=82, state=WAITING, blockCnt=1, waitCnt=1]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@3d8d51e7, ownerName=null, ownerId=-1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
        at o.a.i.i.processors.metastorage.persistence.DmsDataWriterWorker.body(DmsDataWriterWorker.java:146)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="wal-segment-syncer-#54", id=81, state=TIMED_WAITING, blockCnt=0, waitCnt=4577]
        at java.lang.Thread.sleep(Native Method)
        at o.a.i.i.util.IgniteUtils.sleep(IgniteUtils.java:7984)
        at o.a.i.i.processors.cache.persistence.wal.filehandle.FileHandleManagerImpl$WalSegmentSyncer.body(FileHandleManagerImpl.java:622)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="wal-file-cleaner%null-#53", id=80, state=WAITING, blockCnt=0, waitCnt=1]
    Lock [object=o.a.i.i.processors.cache.persistence.wal.aware.SegmentArchiveSizeStorage@3e9c9d93, ownerName=null, ownerId=-1]
        at java.lang.Object.wait(Native Method)
        at java.lang.Object.wait(Object.java:502)
        at o.a.i.i.processors.cache.persistence.wal.aware.SegmentArchiveSizeStorage.awaitExceedMaxSize(SegmentArchiveSizeStorage.java:79)
        at o.a.i.i.processors.cache.persistence.wal.aware.SegmentAware.awaitExceedMaxArchiveSize(SegmentAware.java:354)
        at o.a.i.i.processors.cache.persistence.wal.FileWriteAheadLogManager$FileCleaner.body(FileWriteAheadLogManager.java:3171)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
        at java.lang.Thread.run(          warning : PerformanceAttributionServer[PerformanceAttributionServer1]_Global@PARDH7JQHS2 (41660.6010) : [2021/10/12-10:21:48.296] : org.apache.ignite.internal.processors.cache.CacheDiagnosticManager::LoggerLog() : Page locks dump:

Thread=[name=Thread-12, id=111], state=RUNNABLE
Locked pages = []
Locked pages log: name=Thread-12 time=(1634026908291, 2021-10-12 10:21:48.291)


Thread=[name=Thread-13, id=115], state=RUNNABLE
Locked pages = []
Locked pages log: name=Thread-13 time=(1634026908291, 2021-10-12 10:21:48.291)


Thread=[name=Thread-14, id=116], state=RUNNABLE
Locked pages = []
Locked pages log: name=Thread-14 time=(1634026908291, 2021-10-12 10:21:48.291)


Thread=[name=Thread-15, id=117], state=RUNNABLE
Locked pages = []
Locked pages log: name=Thread-15 time=(1634026908291, 2021-10-12 10:21:48.291)


Thread=[name=Thread-16, id=168], state=RUNNABLE
Locked pages = []
Locked pages log: name=Thread-16 time=(1634026908291, 2021-10-12 10:21:48.291)


Thread=[name=Thread-17, id=176], state=RUNNABLE
Locked pages = []
Locked pages log: name=Thread-17 time=(1634026908291, 2021-10-12 10:21:48.291)


Thread=[name=Thread-18, id=204], state=RUNNABLE
Locked pages = []
Locked pages log: name=Thread-18 time=(1634026908291, 2021-10-12 10:21:48.291)


Thread=[name=checkpoint-runner-cpu-#62, id=89], state=WAITING
Locked pages = []
Locked pages log: name=checkpoint-runner-cpu-#62 time=(1634026908291, 2021-10-12 10:21:48.291)


Thread=[name=checkpoint-runner-cpu-#63, id=90], state=WAITING
Locked pages = []
Locked pages log: name=checkpoint-runner-cpu-#63 time=(1634026908291, 2021-10-12 10:21:48.291)


Thread=[name=db-checkpoint-thread-#61, id=88], state=TIMED_WAITING
Locked pages = []
Locked pages log: name=db-checkpoint-thread-#61 time=(1634026908291, 2021-10-12 10:21:48.291)


Thread=[name=dms-writer-thread-#55, id=82], state=WAITING
Locked pages = []
Locked pages log: name=dms-writer-thread-#55 time=(1634026908291, 2021-10-12 10:21:48.291)


Thread=[name=exchange-worker-#50, id=76], state=TIMED_WAITING
Locked pages = []
Locked pages log: name=exchange-worker-#50 time=(1634026908291, 2021-10-12 10:21:48.291)


Thread=[name=main, id=1], state=RUNNABLE
Locked pages = []
Locked pages log: name=main time=(1634026908291, 2021-10-12 10:21:48.291)



Reached logical end of the segment for file D:\sophis_beta\tests\installs\Debug\servers\sophis\PerformanceAttribution\PerformanceAttributionServer\work\db\wal\node00-d410e934-f1a8-474b-b5c2-a5dc783b186c\0000000000000001.wal
Reached logical end of the segment for file D:\sophis_beta\tests\installs\Debug\servers\sophis\PerformanceAttribution\PerformanceAttributionServer\work\db\wal\node00-d410e934-f1a8-474b-b5c2-a5dc783b186c\0000000000000002.wal
Reached logical end of the segment for file D:\sophis_beta\tests\installs\Debug\servers\sophis\PerformanceAttribution\PerformanceAttributionServer\work\db\wal\node00-d410e934-f1a8-474b-b5c2-a5dc783b186c\0000000000000003.wal
Reached logical end of the segment for file D:\sophis_beta\tests\installs\Debug\servers\sophis\PerformanceAttribution\PerformanceAttributionServer\work\db\wal\node00-d410e934-f1a8-474b-b5c2-a5dc783b186c\0000000000000004.wal
Reached logical end of the segment for file D:\sophis_beta\tests\installs\Debug\servers\sophis\PerformanceAttribution\PerformanceAttributionServer\work\db\wal\node00-d410e934-f1a8-474b-b5c2-a5dc783b186c\0000000000000005.wal
Reached logical end of the segment for file D:\sophis_beta\tests\installs\Debug\servers\sophis\PerformanceAttribution\PerformanceAttributionServer\work\db\wal\node00-d410e934-f1a8-474b-b5c2-a5dc783b186c\0000000000000006.wal
Reached logical end of the segment for file D:\sophis_beta\tests\installs\Debug\servers\sophis\PerformanceAttribution\PerformanceAttributionServer\work\db\wal\node00-d410e934-f1a8-474b-b5c2-a5dc783b186c\0000000000000007.wal
Reached logical end of the segment for file D:\sophis_beta\tests\installs\Debug\servers\sophis\PerformanceAttribution\PerformanceAttributionServer\work\db\wal\node00-d410e934-f1a8-474b-b5c2-a5dc783b186c\0000000000000008.wal
Reached logical end of the segment for file D:\sophis_beta\tests\installs\Debug\servers\sophis\PerformanceAttribution\PerformanceAttributionServer\work\db\wal\node00-d410e934-f1a8-474b-b5c2-a5dc783b186c\0000000000000009.wal
            error : PerformanceAttributionServer[PerformanceAttributionServer1]_Global@PARDH7JQHS2 (41660.6010) : [2021/10/12-10:21:48.574] : Apache.Ignite.NLog.IgniteNLogLogger::LoggerLog() : JVM will be halted immediately due to the failure: [failureCtx=FailureContext [type=CRITICAL_ERROR, err=class o.a.i.i.processors.cache.persistence.tree.CorruptedTreeException: B+Tree is corrupted [pages(groupId, pageId)=[IgniteBiTuple [val1=241659666, val2=1127239936638982]], msg=Runtime failure on search row: SearchRow [key=KeyCacheObjectImpl [part=312, val=56ae72d3-a91a-4211-8279-0b0447881544, hasValBytes=true], hash=746501958, cacheId=0]]]]
Application is shutting down...

"FINASTRA" is the trade name of the FINASTRA group of companies. This email and any attachments have been scanned for known viruses using multiple scanners. This email message is intended for the named recipient only. It may be privileged and/or confidential. If you are not the named recipient of this email please notify us immediately and do not copy it or use it for any purpose, nor disclose its contents to any other person. This email does not constitute the commencement of legal relations between you and FINASTRA. Please refer to the executed contract between you and the relevant member of the FINASTRA group for the identity of the contracting party with which you are dealing.
"FINASTRA" is the trade name of the FINASTRA group of companies. This email and any attachments have been scanned for known viruses using multiple scanners. This email message is intended for the named recipient only. It may be privileged and/or confidential. If you are not the named recipient of this email please notify us immediately and do not copy it or use it for any purpose, nor disclose its contents to any other person. This email does not constitute the commencement of legal relations between you and FINASTRA. Please refer to the executed contract between you and the relevant member of the FINASTRA group for the identity of the contracting party with which you are dealing.
"FINASTRA" is the trade name of the FINASTRA group of companies. This email and any attachments have been scanned for known viruses using multiple scanners. This email message is intended for the named recipient only. It may be privileged and/or confidential. If you are not the named recipient of this email please notify us immediately and do not copy it or use it for any purpose, nor disclose its contents to any other person. This email does not constitute the commencement of legal relations between you and FINASTRA. Please refer to the executed contract between you and the relevant member of the FINASTRA group for the identity of the contracting party with which you are dealing.

Re: [EXT] Re: Crash of Ignite (B+Tree corrupted) on a large PutIfAbsent

Posted by Alex Plehanov <pl...@gmail.com>.
> Will you add a test in Ignite to avoid the crash
I'm not quite sure, but perhaps a crash is the best we can do in this case.
Throwing exception to the user might be not enough, since some changes can
be already made in the DB prior to the exception and this can lead to data
inconsistency.

> give a clearer error message
Nested exception already contains a root cause of the problem (For example:
"Record is too long [capacity=67108864, size=92984514]", see ticket with
the same problem reproduced with java [1]), but perhaps this nested
exception is not correctly displayed by Ignite .NET

[1]: https://issues.apache.org/jira/browse/IGNITE-13965

сб, 16 окт. 2021 г. в 15:02, Semeria, Vincent <Vincent.Semeria@finastra.com
>:

> Yes indeed, setting the WAL segment size to 150 megabytes accepts my
> PutIfAbsent. Thanks.
>
>
>
> Will you add a test in Ignite to avoid the crash, give a clearer error
> message and mention in the documentation that the WAL segment size should
> be higher than any single cache entry ? At the moment the doc just says this
>
>         //
>
>         // Summary:
>
>         //     Gets or sets the size of the WAL (Write Ahead Log) segment.
> For performance reasons,
>
>         //     the whole WAL is split into files of fixed length called
> segments.
>
>
>
> The limit should also be written in this page
>
> Ignite Persistence | Ignite Documentation (apache.org)
> <https://ignite.apache.org/docs/latest/persistence/native-persistence>
>
>
>
> *From:* Alex Plehanov <pl...@gmail.com>
> *Sent:* Friday, October 15, 2021 16:12
> *To:* user@ignite.apache.org
> *Subject:* [EXT] Re: Crash of Ignite (B+Tree corrupted) on a large
> PutIfAbsent
>
>
>
> Hello,
>
>
>
> Perhaps you have too small WAL segment size (WAL segment should be large
> enough to fit the whole cache entry), try to
> change DataStorageConfiguration.WalSegmentSize property.
>
>
>
> чт, 14 окт. 2021 г. в 00:43, Semeria, Vincent <
> Vincent.Semeria@finastra.com>:
>
> Hello Ignite,
>
>
>
> I currently use the C# API of Ignite 2.10 to store large objects of type V
> in an ICache<string, V>. Typically an object of V is around 100 megabytes.
> My data region is persisted on the hard drive. PutIfAbsent crashes Ignite
> with the complicated message below. As a workaround, I split type V into
> smaller types and used loops of smaller PutIfAbsent, which succeeded.
> Ultimately the data stored in the cache is the same, which shows that
> Ignite accepts my data (this is not a problem in the binary serializer).
>
>
>
> Is there a configuration of the data region that would accept a single
> PutIfAbsent of 100 megabytes ?
>
>
>
> Anyway Ignite should probably not crash when this limit is exceeded.
> Please send a clean error instead like “Insertion request exceeds limit
> XYZ” and keep Ignite alive in this case.
>
>
>
> Regards,
>
> Vincent Semeria
>
>
>
>
>
>             error :
> PerformanceAttributionServer[PerformanceAttributionServer1]_Global@PARDH7JQHS2
> (41660.6010) : [2021/10/12-10:21:48.215] :
> Apache.Ignite.NLog.IgniteNLogLogger::LoggerLog() : Critical system error
> detected. Will be handled accordingly to configured handler
> [hnd=StopNodeOrHaltFailureHandler [tryStop=false, timeout=0,
> super=AbstractFailureHandler [ignoredFailureTypes=UnmodifiableSet
> [SYSTEM_WORKER_BLOCKED, SYSTEM_CRITICAL_OPERATION_TIMEOUT]]],
> failureCtx=FailureContext [type=CRITICAL_ERROR, err=class
> o.a.i.i.processors.cache.persistence.tree.CorruptedTreeException: B+Tree is
> corrupted [pages(groupId, pageId)=[IgniteBiTuple [val1=241659666,
> val2=1127239936638982]], msg=Runtime failure on search row: SearchRow
> [key=KeyCacheObjectImpl [part=312,
> val=56ae72d3-a91a-4211-8279-0b0447881544, hasValBytes=true],
> hash=746501958, cacheId=0]]]]
>
>             error :
> PerformanceAttributionServer[PerformanceAttributionServer1]_Global@PARDH7JQHS2
> (41660.6010) : [2021/10/12-10:21:48.216] :
> org.apache.ignite.internal.processors.failure.FailureProcessor::LoggerLog()
> : A critical problem with persistence data structures was detected. Please
> make backup of persistence storage and WAL files for further analysis.
> Persistence storage path:  WAL path: db/wal WAL archive path: db/wal/archive
>
>             error :
> PerformanceAttributionServer[PerformanceAttributionServer1]_Global@PARDH7JQHS2
> (41660.6010) : [2021/10/12-10:21:48.219] :
> org.apache.ignite.internal.processors.failure.FailureProcessor::LoggerLog()
> : No deadlocked threads detected.
>
>             error :
> PerformanceAttributionServer[PerformanceAttributionServer1]_Global@PARDH7JQHS2
> (41660.6010) : [2021/10/12-10:21:48.276] :
> org.apache.ignite.internal.processors.failure.FailureProcessor::LoggerLog()
> : Thread dump at 2021/10/12 10:21:48 CEST
>
> Thread [name="sys-#200", id=235, state=TIMED_WAITING, blockCnt=0,
> waitCnt=1]
>
>     Lock
> [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@529e4706,
> ownerName=null, ownerId=-1]
>
>         at sun.misc.Unsafe.park(Native Method)
>
>         at
> java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
>
>         at
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
>
>         at
> java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
>
>         at
> java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
>
>         at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
>
>         at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>
>         at java.lang.Thread.run(Thread.java:748)
>
>
>
> Thread [name="sys-#199", id=234, state=TIMED_WAITING, blockCnt=0,
> waitCnt=1]
>
>     Lock
> [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@529e4706,
> ownerName=null, ownerId=-1]
>
>         at sun.misc.Unsafe.park(Native Method)
>
>         at
> java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
>
>         at
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
>
>         at
> java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
>
>         at
> java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
>
>         at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
>
>         at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>
>         at java.lang.Thread.run(Thread.java:748)
>
>
>
> Thread [name="sys-#198", id=233, state=TIMED_WAITING, blockCnt=0,
> waitCnt=1]
>
>     Lock
> [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@529e4706,
> ownerName=null, ownerId=-1]
>
>         at sun.misc.Unsafe.park(Native Method)
>
>         at
> java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
>
>         at
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
>
>         at
> java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
>
>         at
> java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
>
>         at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
>
>         at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>
>         at java.lang.Thread.run(Thread.java:748)
>
>
>
> Thread [name="sys-#197", id=232, state=TIMED_WAITING, blockCnt=0,
> waitCnt=1]
>
>     Lock
> [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@529e4706,
> ownerName=null, ownerId=-1]
>
>         at sun.misc.Unsafe.park(Native Method)
>
>         at
> java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
>
>         at
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
>
>         at
> java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
>
>         at
> java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
>
>         at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
>
>         at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>
>         at java.lang.Thread.run(Thread.java:748)
>
>
>
> Thread [name="sys-#196", id=231, state=TIMED_WAITING, blockCnt=0,
> waitCnt=1]
>
>     Lock
> [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@529e4706,
> ownerName=null, ownerId=-1]
>
>         at sun.misc.Unsafe.park(Native Method)
>
>         at
> java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
>
>         at
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
>
>         at
> java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
>
>         at
> java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
>
>         at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
>
>         at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>
>         at java.lang.Thread.run(Thread.java:748)
>
>
>
> Thread [name="sys-#195", id=230, state=TIMED_WAITING, blockCnt=0,
> waitCnt=1]
>
>     Lock
> [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@529e4706,
> ownerName=null, ownerId=-1]
>
>         at sun.misc.Unsafe.park(Native Method)
>
>         at
> java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
>
>         at
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
>
>         at
> java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
>
>         at
> java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
>
>         at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
>
>         at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>
>         at java.lang.Thread.run(Thread.java:748)
>
>
>
> Thread [name="sys-#194", id=229, state=TIMED_WAITING, blockCnt=0,
> waitCnt=1]
>
>     Lock
> [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@529e4706,
> ownerName=null, ownerId=-1]
>
>         at sun.misc.Unsafe.park(Native Method)
>
>         at
> java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
>
>         at
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
>
>         at
> java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
>
>         at
> java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
>
>         at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
>
>         at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>
>         at java.lang.Thread.run(Thread.java:748)
>
>
>
> Thread [name="Thread-18", id=204, state=RUNNABLE, blockCnt=0, waitCnt=0]
>
>
>
> Thread [name="Thread-17", id=176, state=RUNNABLE, blockCnt=0, waitCnt=2]
>
>
>
> Thread [name="Thread-16", id=168, state=RUNNABLE, blockCnt=0, waitCnt=0]
>
>
>
> Thread [name="Thread-15", id=117, state=RUNNABLE, blockCnt=0, waitCnt=0]
>
>
>
> Thread [name="Thread-14", id=116, state=RUNNABLE, blockCnt=0, waitCnt=0]
>
>
>
> Thread [name="Thread-13", id=115, state=RUNNABLE, blockCnt=0, waitCnt=0]
>
>
>
> Thread [name="Thread-12", id=111, state=RUNNABLE, blockCnt=0, waitCnt=82]
>
>         at sun.management.ThreadImpl.dumpThreads0(Native Method)
>
>         at sun.management.ThreadImpl.dumpAllThreads(ThreadImpl.java:454)
>
>         at o.a.i.i.util.IgniteUtils.dumpThreads(IgniteUtils.java:1471)
>
>         at
> o.a.i.i.processors.failure.FailureProcessor.process(FailureProcessor.java:192)
>
>         - locked o.a.i.i.processors.failure.FailureProcessor@67349fda
>
>         at
> o.a.i.i.processors.failure.FailureProcessor.process(FailureProcessor.java:155)
>
>         at
> o.a.i.i.processors.cache.persistence.tree.BPlusTree.processFailure(BPlusTree.java:6152)
>
>         at
> o.a.i.i.processors.cache.persistence.tree.BPlusTree.corruptedTreeException(BPlusTree.java:6139)
>
>         at
> o.a.i.i.processors.cache.persistence.tree.BPlusTree.invoke(BPlusTree.java:1953)
>
>         at
> o.a.i.i.processors.cache.IgniteCacheOffheapManagerImpl$CacheDataStoreImpl.invoke0(IgniteCacheOffheapManagerImpl.java:1758)
>
>         at
> o.a.i.i.processors.cache.IgniteCacheOffheapManagerImpl$CacheDataStoreImpl.invoke(IgniteCacheOffheapManagerImpl.java:1741)
>
>         at
> o.a.i.i.processors.cache.persistence.GridCacheOffheapManager$GridCacheDataStore.invoke(GridCacheOffheapManager.java:2766)
>
>         at
> o.a.i.i.processors.cache.IgniteCacheOffheapManagerImpl.invoke(IgniteCacheOffheapManagerImpl.java:439)
>
>         at
> o.a.i.i.processors.cache.GridCacheMapEntry.innerUpdate(GridCacheMapEntry.java:2338)
>
>         at
> o.a.i.i.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.updateSingle(GridDhtAtomicCache.java:2654)
>
>         at
> o.a.i.i.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.update(GridDhtAtomicCache.java:2114)
>
>         at
> o.a.i.i.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.updateAllAsyncInternal0(GridDhtAtomicCache.java:1931)
>
>         at
> o.a.i.i.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.updateAllAsyncInternal(GridDhtAtomicCache.java:1724)
>
>         at
> o.a.i.i.processors.cache.distributed.dht.atomic.GridNearAtomicAbstractUpdateFuture.sendSingleRequest(GridNearAtomicAbstractUpdateFuture.java:306)
>
>         at
> o.a.i.i.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateFuture.map(GridNearAtomicSingleUpdateFuture.java:486)
>
>         at
> o.a.i.i.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateFuture.mapOnTopology(GridNearAtomicSingleUpdateFuture.java:446)
>
>         at
> o.a.i.i.processors.cache.distributed.dht.atomic.GridNearAtomicAbstractUpdateFuture.map(GridNearAtomicAbstractUpdateFuture.java:255)
>
>         at
> o.a.i.i.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.update0(GridDhtAtomicCache.java:1169)
>
>         at
> o.a.i.i.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.put0(GridDhtAtomicCache.java:634)
>
>         at
> o.a.i.i.processors.cache.GridCacheAdapter.put(GridCacheAdapter.java:2596)
>
>         at
> o.a.i.i.processors.cache.GridCacheAdapter.putIfAbsent(GridCacheAdapter.java:3041)
>
>         at
> o.a.i.i.processors.cache.IgniteCacheProxyImpl.putIfAbsent(IgniteCacheProxyImpl.java:1412)
>
>         at
> o.a.i.i.processors.cache.GatewayProtectedCacheProxy.putIfAbsent(GatewayProtectedCacheProxy.java:928)
>
>         at
> o.a.i.i.processors.platform.cache.PlatformCache.processInStreamOutLong(PlatformCache.java:516)
>
>         at
> o.a.i.i.processors.platform.PlatformTargetProxyImpl.inStreamOutLong(PlatformTargetProxyImpl.java:67)
>
>
>
>     Locked synchronizers:
>
>         java.util.concurrent.locks.ReentrantLock$NonfairSync@3de6a6db
>
> Thread [name="Thread-11", id=102, state=RUNNABLE, blockCnt=0, waitCnt=0]
>
>
>
> Thread [name="ttl-cleanup-worker-#72", id=99, state=TIMED_WAITING,
> blockCnt=0, waitCnt=2297]
>
>         at java.lang.Thread.sleep(Native Method)
>
>         at o.a.i.i.util.IgniteUtils.sleep(IgniteUtils.java:7984)
>
>         at
> o.a.i.i.processors.cache.GridCacheSharedTtlCleanupManager$CleanupWorker.body(GridCacheSharedTtlCleanupManager.java:210)
>
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>
>         at java.lang.Thread.run(Thread.java:748)
>
>
>
> Thread [name="checkpoint-runner-IO-#71", id=98, state=WAITING, blockCnt=1,
> waitCnt=722]
>
>     Lock
> [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@565aa90c,
> ownerName=null, ownerId=-1]
>
>         at sun.misc.Unsafe.park(Native Method)
>
>         at
> java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
>
>         at
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
>
>         at
> java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
>
>         at
> java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
>
>         at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
>
>         at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>
>         at java.lang.Thread.run(Thread.java:748)
>
>
>
> Thread [name="checkpoint-runner-IO-#70", id=97, state=WAITING, blockCnt=2,
> waitCnt=585]
>
>     Lock
> [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@565aa90c,
> ownerName=null, ownerId=-1]
>
>         at sun.misc.Unsafe.park(Native Method)
>
>         at
> java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
>
>         at
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
>
>         at
> java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
>
>         at
> java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
>
>         at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
>
>         at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>
>         at java.lang.Thread.run(Thread.java:748)
>
>
>
> Thread [name="checkpoint-runner-IO-#69", id=96, state=WAITING, blockCnt=4,
> waitCnt=660]
>
>     Lock
> [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@565aa90c,
> ownerName=null, ownerId=-1]
>
>         at sun.misc.Unsafe.park(Native Method)
>
>         at
> java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
>
>         at
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
>
>         at
> java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
>
>         at
> java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
>
>         at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
>
>         at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>
>         at java.lang.Thread.run(Thread.java:748)
>
>
>
> Thread [name="checkpoint-runner-IO-#68", id=95, state=WAITING, blockCnt=2,
> waitCnt=867]
>
>     Lock
> [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@565aa90c,
> ownerName=null, ownerId=-1]
>
>         at sun.misc.Unsafe.park(Native Method)
>
>         at
> java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
>
>         at
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
>
>         at
> java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
>
>         at
> java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
>
>         at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
>
>         at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>
>         at java.lang.Thread.run(Thread.java:748)
>
>
>
> Thread [name="checkpoint-runner-cpu-#65", id=92, state=WAITING,
> blockCnt=54, waitCnt=478]
>
>     Lock
> [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@76dc62b6,
> ownerName=null, ownerId=-1]
>
>         at sun.misc.Unsafe.park(Native Method)
>
>         at
> java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
>
>         at
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
>
>         at
> java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
>
>         at
> java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
>
>         at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
>
>         at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>
>         at java.lang.Thread.run(Thread.java:748)
>
>
>
> Thread [name="checkpoint-runner-cpu-#64", id=91, state=WAITING,
> blockCnt=38, waitCnt=494]
>
>     Lock
> [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@76dc62b6,
> ownerName=null, ownerId=-1]
>
>         at sun.misc.Unsafe.park(Native Method)
>
>         at
> java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
>
>         at
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
>
>         at
> java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
>
>         at
> java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
>
>         at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
>
>         at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>
>         at java.lang.Thread.run(Thread.java:748)
>
>
>
> Thread [name="checkpoint-runner-cpu-#63", id=90, state=WAITING,
> blockCnt=75, waitCnt=505]
>
>     Lock
> [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@76dc62b6,
> ownerName=null, ownerId=-1]
>
>         at sun.misc.Unsafe.park(Native Method)
>
>         at
> java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
>
>         at
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
>
>         at
> java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
>
>         at
> java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
>
>         at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
>
>         at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>
>         at java.lang.Thread.run(Thread.java:748)
>
>
>
> Thread [name="checkpoint-runner-cpu-#62", id=89, state=WAITING,
> blockCnt=17, waitCnt=545]
>
>     Lock
> [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@76dc62b6,
> ownerName=null, ownerId=-1]
>
>         at sun.misc.Unsafe.park(Native Method)
>
>         at
> java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
>
>         at
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
>
>         at
> java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
>
>         at
> java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
>
>         at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
>
>         at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>
>         at java.lang.Thread.run(Thread.java:748)
>
>
>
> Thread [name="db-checkpoint-thread-#61", id=88, state=TIMED_WAITING,
> blockCnt=0, waitCnt=45]
>
>     Lock
> [object=o.a.i.i.processors.cache.persistence.checkpoint.Checkpointer@35d5f9f8,
> ownerName=null, ownerId=-1]
>
>         at java.lang.Object.wait(Native Method)
>
>         at
> o.a.i.i.processors.cache.persistence.checkpoint.Checkpointer.waitCheckpointEvent(Checkpointer.java:755)
>
>         at
> o.a.i.i.processors.cache.persistence.checkpoint.Checkpointer.body(Checkpointer.java:246)
>
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>
>         at java.lang.Thread.run(Thread.java:748)
>
>
>
> Thread [name="dms-writer-thread-#55", id=82, state=WAITING, blockCnt=1,
> waitCnt=1]
>
>     Lock
> [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@3d8d51e7,
> ownerName=null, ownerId=-1]
>
>         at sun.misc.Unsafe.park(Native Method)
>
>         at
> java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
>
>         at
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
>
>         at
> java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
>
>         at
> o.a.i.i.processors.metastorage.persistence.DmsDataWriterWorker.body(DmsDataWriterWorker.java:146)
>
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>
>         at java.lang.Thread.run(Thread.java:748)
>
>
>
> Thread [name="wal-segment-syncer-#54", id=81, state=TIMED_WAITING,
> blockCnt=0, waitCnt=4577]
>
>         at java.lang.Thread.sleep(Native Method)
>
>         at o.a.i.i.util.IgniteUtils.sleep(IgniteUtils.java:7984)
>
>         at
> o.a.i.i.processors.cache.persistence.wal.filehandle.FileHandleManagerImpl$WalSegmentSyncer.body(FileHandleManagerImpl.java:622)
>
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>
>         at java.lang.Thread.run(Thread.java:748)
>
>
>
> Thread [name="wal-file-cleaner%null-#53", id=80, state=WAITING,
> blockCnt=0, waitCnt=1]
>
>     Lock
> [object=o.a.i.i.processors.cache.persistence.wal.aware.SegmentArchiveSizeStorage@3e9c9d93,
> ownerName=null, ownerId=-1]
>
>         at java.lang.Object.wait(Native Method)
>
>         at java.lang.Object.wait(Object.java:502)
>
>         at
> o.a.i.i.processors.cache.persistence.wal.aware.SegmentArchiveSizeStorage.awaitExceedMaxSize(SegmentArchiveSizeStorage.java:79)
>
>         at
> o.a.i.i.processors.cache.persistence.wal.aware.SegmentAware.awaitExceedMaxArchiveSize(SegmentAware.java:354)
>
>         at
> o.a.i.i.processors.cache.persistence.wal.FileWriteAheadLogManager$FileCleaner.body(FileWriteAheadLogManager.java:3171)
>
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>
>         at java.lang.Thread.run(          warning :
> PerformanceAttributionServer[PerformanceAttributionServer1]_Global@PARDH7JQHS2
> (41660.6010) : [2021/10/12-10:21:48.296] :
> org.apache.ignite.internal.processors.cache.CacheDiagnosticManager::LoggerLog()
> : Page locks dump:
>
>
>
> Thread=[name=Thread-12, id=111], state=RUNNABLE
>
> Locked pages = []
>
> Locked pages log: name=Thread-12 time=(1634026908291, 2021-10-12
> 10:21:48.291)
>
>
>
>
>
> Thread=[name=Thread-13, id=115], state=RUNNABLE
>
> Locked pages = []
>
> Locked pages log: name=Thread-13 time=(1634026908291, 2021-10-12
> 10:21:48.291)
>
>
>
>
>
> Thread=[name=Thread-14, id=116], state=RUNNABLE
>
> Locked pages = []
>
> Locked pages log: name=Thread-14 time=(1634026908291, 2021-10-12
> 10:21:48.291)
>
>
>
>
>
> Thread=[name=Thread-15, id=117], state=RUNNABLE
>
> Locked pages = []
>
> Locked pages log: name=Thread-15 time=(1634026908291, 2021-10-12
> 10:21:48.291)
>
>
>
>
>
> Thread=[name=Thread-16, id=168], state=RUNNABLE
>
> Locked pages = []
>
> Locked pages log: name=Thread-16 time=(1634026908291, 2021-10-12
> 10:21:48.291)
>
>
>
>
>
> Thread=[name=Thread-17, id=176], state=RUNNABLE
>
> Locked pages = []
>
> Locked pages log: name=Thread-17 time=(1634026908291, 2021-10-12
> 10:21:48.291)
>
>
>
>
>
> Thread=[name=Thread-18, id=204], state=RUNNABLE
>
> Locked pages = []
>
> Locked pages log: name=Thread-18 time=(1634026908291, 2021-10-12
> 10:21:48.291)
>
>
>
>
>
> Thread=[name=checkpoint-runner-cpu-#62, id=89], state=WAITING
>
> Locked pages = []
>
> Locked pages log: name=checkpoint-runner-cpu-#62 time=(1634026908291,
> 2021-10-12 10:21:48.291)
>
>
>
>
>
> Thread=[name=checkpoint-runner-cpu-#63, id=90], state=WAITING
>
> Locked pages = []
>
> Locked pages log: name=checkpoint-runner-cpu-#63 time=(1634026908291,
> 2021-10-12 10:21:48.291)
>
>
>
>
>
> Thread=[name=db-checkpoint-thread-#61, id=88], state=TIMED_WAITING
>
> Locked pages = []
>
> Locked pages log: name=db-checkpoint-thread-#61 time=(1634026908291,
> 2021-10-12 10:21:48.291)
>
>
>
>
>
> Thread=[name=dms-writer-thread-#55, id=82], state=WAITING
>
> Locked pages = []
>
> Locked pages log: name=dms-writer-thread-#55 time=(1634026908291,
> 2021-10-12 10:21:48.291)
>
>
>
>
>
> Thread=[name=exchange-worker-#50, id=76], state=TIMED_WAITING
>
> Locked pages = []
>
> Locked pages log: name=exchange-worker-#50 time=(1634026908291, 2021-10-12
> 10:21:48.291)
>
>
>
>
>
> Thread=[name=main, id=1], state=RUNNABLE
>
> Locked pages = []
>
> Locked pages log: name=main time=(1634026908291, 2021-10-12 10:21:48.291)
>
>
>
>
>
>
>
> Reached logical end of the segment for file
> D:\sophis_beta\tests\installs\Debug\servers\sophis\PerformanceAttribution\PerformanceAttributionServer\work\db\wal\node00-d410e934-f1a8-474b-b5c2-a5dc783b186c\0000000000000001.wal
>
> Reached logical end of the segment for file
> D:\sophis_beta\tests\installs\Debug\servers\sophis\PerformanceAttribution\PerformanceAttributionServer\work\db\wal\node00-d410e934-f1a8-474b-b5c2-a5dc783b186c\0000000000000002.wal
>
> Reached logical end of the segment for file
> D:\sophis_beta\tests\installs\Debug\servers\sophis\PerformanceAttribution\PerformanceAttributionServer\work\db\wal\node00-d410e934-f1a8-474b-b5c2-a5dc783b186c\0000000000000003.wal
>
> Reached logical end of the segment for file
> D:\sophis_beta\tests\installs\Debug\servers\sophis\PerformanceAttribution\PerformanceAttributionServer\work\db\wal\node00-d410e934-f1a8-474b-b5c2-a5dc783b186c\0000000000000004.wal
>
> Reached logical end of the segment for file
> D:\sophis_beta\tests\installs\Debug\servers\sophis\PerformanceAttribution\PerformanceAttributionServer\work\db\wal\node00-d410e934-f1a8-474b-b5c2-a5dc783b186c\0000000000000005.wal
>
> Reached logical end of the segment for file
> D:\sophis_beta\tests\installs\Debug\servers\sophis\PerformanceAttribution\PerformanceAttributionServer\work\db\wal\node00-d410e934-f1a8-474b-b5c2-a5dc783b186c\0000000000000006.wal
>
> Reached logical end of the segment for file
> D:\sophis_beta\tests\installs\Debug\servers\sophis\PerformanceAttribution\PerformanceAttributionServer\work\db\wal\node00-d410e934-f1a8-474b-b5c2-a5dc783b186c\0000000000000007.wal
>
> Reached logical end of the segment for file
> D:\sophis_beta\tests\installs\Debug\servers\sophis\PerformanceAttribution\PerformanceAttributionServer\work\db\wal\node00-d410e934-f1a8-474b-b5c2-a5dc783b186c\0000000000000008.wal
>
> Reached logical end of the segment for file
> D:\sophis_beta\tests\installs\Debug\servers\sophis\PerformanceAttribution\PerformanceAttributionServer\work\db\wal\node00-d410e934-f1a8-474b-b5c2-a5dc783b186c\0000000000000009.wal
>
>             error :
> PerformanceAttributionServer[PerformanceAttributionServer1]_Global@PARDH7JQHS2
> (41660.6010) : [2021/10/12-10:21:48.574] :
> Apache.Ignite.NLog.IgniteNLogLogger::LoggerLog() : JVM will be halted
> immediately due to the failure: [failureCtx=FailureContext
> [type=CRITICAL_ERROR, err=class
> o.a.i.i.processors.cache.persistence.tree.CorruptedTreeException: B+Tree is
> corrupted [pages(groupId, pageId)=[IgniteBiTuple [val1=241659666,
> val2=1127239936638982]], msg=Runtime failure on search row: SearchRow
> [key=KeyCacheObjectImpl [part=312,
> val=56ae72d3-a91a-4211-8279-0b0447881544, hasValBytes=true],
> hash=746501958, cacheId=0]]]]
>
> Application is shutting down...
>
>
>
> "FINASTRA" is the trade name of the FINASTRA group of companies. This
> email and any attachments have been scanned for known viruses using
> multiple scanners. This email message is intended for the named recipient
> only. It may be privileged and/or confidential. If you are not the named
> recipient of this email please notify us immediately and do not copy it or
> use it for any purpose, nor disclose its contents to any other person. This
> email does not constitute the commencement of legal relations between you
> and FINASTRA. Please refer to the executed contract between you and the
> relevant member of the FINASTRA group for the identity of the contracting
> party with which you are dealing.
>
> "FINASTRA" is the trade name of the FINASTRA group of companies. This
> email and any attachments have been scanned for known viruses using
> multiple scanners. This email message is intended for the named recipient
> only. It may be privileged and/or confidential. If you are not the named
> recipient of this email please notify us immediately and do not copy it or
> use it for any purpose, nor disclose its contents to any other person. This
> email does not constitute the commencement of legal relations between you
> and FINASTRA. Please refer to the executed contract between you and the
> relevant member of the FINASTRA group for the identity of the contracting
> party with which you are dealing.
>