[ https://issues.apache.org/jira/browse/IGNITE-7412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16326229#comment-16326229 ]
Andrey Gura commented on IGNITE-7412: ------------------------------------- {\{written}} field can be updated with wrong value from \{{addRecord}} method when \{{mmap}} is enabled. For example thread T1 reserves buffer segment with end position P1 for record R1 with size X1, and then thread T2 reserves buffer segment with end position P2 > P1 for record R2 with size X2 < X1. In this case record R2 can be serialized into buffer earlier than record R1 and T2 will update \{{written}} field first, and then T1 will update \{{written}} field with smaller value. Solution: Use "setIfGreater" operation because written value must grow only. > WAL: Written bytes amount can be updated by wrong value and fail with > assertion error > ------------------------------------------------------------------------------------- > > Key: IGNITE-7412 > URL: https://issues.apache.org/jira/browse/IGNITE-7412 > Project: Ignite > Issue Type: Bug > Reporter: Andrey Gura > Assignee: Andrey Gura > Priority: Major > Fix For: 2.4 > > > PDS tests with WAL (w.g. > \{{IgniteWalRecoveryWithCompactionTest#testWalRolloverMultithreadedDefault}}) > can fail with assertion error like this: > > {noformat} > java.lang.AssertionError: null > at > org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager$FileWriteHandle.fsync(FileWriteAheadLogManager.java:2445) > at > org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager$FileWriteHandle.access$2200(FileWriteAheadLogManager.java:2206) > at > org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager.fsync(FileWriteAheadLogManager.java:661) > at > org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.updateAllAsyncInternal0(GridDhtAtomicCache.java:1771) > at > org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.updateAllAsyncInternal(GridDhtAtomicCache.java:1628) > at > org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicAbstractUpdateFuture.sendSingleRequest(GridNearAtomicAbstractUpdateFuture.java:299) > at > org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateFuture.map(GridNearAtomicSingleUpdateFuture.java:483) > at > org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateFuture.mapOnTopology(GridNearAtomicSingleUpdateFuture.java:443) > at > org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicAbstractUpdateFuture.map(GridNearAtomicAbstractUpdateFuture.java:248) > at > org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.update0(GridDhtAtomicCache.java:1117) > at > org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.put0(GridDhtAtomicCache.java:606) > at > org.apache.ignite.internal.processors.cache.GridCacheAdapter.put(GridCacheAdapter.java:2354) > at > org.apache.ignite.internal.processors.cache.GridCacheAdapter.put(GridCacheAdapter.java:2331) > at > org.apache.ignite.internal.processors.cache.IgniteCacheProxyImpl.put(IgniteCacheProxyImpl.java:1005) > at > org.apache.ignite.internal.processors.cache.GatewayProtectedCacheProxy.put(GatewayProtectedCacheProxy.java:872) > at > org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalRecoveryTest$2.call(IgniteWalRecoveryTest.java:550) > at > org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalRecoveryTest$2.call(IgniteWalRecoveryTest.java:545) > at > org.apache.ignite.testframework.GridTestThread.run(GridTestThread.java:86) > {noformat} -- This message was sent by Atlassian JIRA (v7.6.3#76005)