[ 
https://issues.apache.org/jira/browse/IGNITE-1135?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14847318#comment-14847318
 ] 

Andrey Gura commented on IGNITE-1135:
-------------------------------------

Alexey,

I found the following places in code that potentially can lead to 
{{AssertionError}} with {{skipVals == true}}:

# {{GridPartitionedGetFuture#createResultMap:590}} (it's our case)
# {{GridNearGetFuture#loadEntries:706}}
# {{IgniteTxLocalAdapter#getAllAsync:1978}}

But I can't find any invokes of {{getAllAsync}} with {{skipVals == true}} 
except {{containsKey}} and {{containsKeys}}. {{getAllAsync0}} method alwyas 
invokes with {{skipVals}} parameter that passed from {{getAllAsync}} method.



> GridPartitionedGetFuture assertion error
> ----------------------------------------
>
>                 Key: IGNITE-1135
>                 URL: https://issues.apache.org/jira/browse/IGNITE-1135
>             Project: Ignite
>          Issue Type: Bug
>            Reporter: Andrey Gura
>            Assignee: Andrey Gura
>             Fix For: ignite-1.4
>
>
> During working on IGNITE-1079 ticket the following problem was found:
> {noformat}
> Failed processing message [senderId=7a07cc81-11e5-42f2-8ffc-184c6c377fc5, 
> msg=GridNearGetResponse 
> [futId=575ef41be41-1a1482a7-e1b9-48c8-9cb7-382fe83792df, 
> miniId=675ef41be41-1a1482a7-e1b9-48c8-9cb7-382fe83792df, ver=GridCacheVersion 
> [topVer=48973890, nodeOrderDrId=3, globalTime=1437493884602, 
> order=1437493880034], entries=[GridCacheEntryInfo [key=KeyCacheObjectImpl 
> [val=7a07cc81-11e5-42f2-8ffc-184c6c377fc5, hasValBytes=true], 
> cacheId=414885281, val=null, ttl=0, expireTime=0, ver=GridCacheVersion 
> [topVer=48973890, nodeOrderDrId=7, globalTime=1437493884824, 
> order=1437493880064], isNew=false, deleted=false]], invalidParts=[], 
> topVer=AffinityTopologyVersion [topVer=7, minorTopVer=0], err=null]] 
> java.lang.AssertionError
>       at 
> org.apache.ignite.internal.processors.cache.GridCacheContext.addResult(GridCacheContext.java:1811)
>       at 
> org.apache.ignite.internal.processors.cache.distributed.dht.GridPartitionedGetFuture.createResultMap(GridPartitionedGetFuture.java:535)
>       at 
> org.apache.ignite.internal.processors.cache.distributed.dht.GridPartitionedGetFuture.access$200(GridPartitionedGetFuture.java:45)
>       at 
> org.apache.ignite.internal.processors.cache.distributed.dht.GridPartitionedGetFuture$MiniFuture.onResult(GridPartitionedGetFuture.java:715)
>       at 
> org.apache.ignite.internal.processors.cache.distributed.dht.GridPartitionedGetFuture.onResult(GridPartitionedGetFuture.java:234)
>       at 
> org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.processNearGetResponse(GridDhtAtomicCache.java:2345)
>       at 
> org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.access$1000(GridDhtAtomicCache.java:64)
>       at 
> org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$9.apply(GridDhtAtomicCache.java:218)
>       at 
> org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$9.apply(GridDhtAtomicCache.java:216)
>       at 
> org.apache.ignite.internal.processors.cache.GridCacheIoManager.processMessage(GridCacheIoManager.java:534)
>       at 
> org.apache.ignite.internal.processors.cache.GridCacheIoManager.onMessage0(GridCacheIoManager.java:240)
>       at 
> org.apache.ignite.internal.processors.cache.GridCacheIoManager.handleMessage(GridCacheIoManager.java:158)
>       at 
> org.apache.ignite.internal.processors.cache.GridCacheIoManager.access$000(GridCacheIoManager.java:48)
>       at 
> org.apache.ignite.internal.processors.cache.GridCacheIoManager$1.onMessage(GridCacheIoManager.java:127)
>       at 
> org.apache.ignite.internal.managers.communication.GridIoManager.processRegularMessage0(GridIoManager.java:761)
>       at 
> org.apache.ignite.internal.managers.communication.GridIoManager.access$1500(GridIoManager.java:59)
>       at 
> org.apache.ignite.internal.managers.communication.GridIoManager$5.run(GridIoManager.java:724)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>       at java.lang.Thread.run(Thread.java:745)
> {noformat}
> In order to reproduce this need to start IgniteComplexHATest (see 
> https://github.com/agura/vertx-ignite) and run tests several times. Test 
> hangs. Thread dump contains thread that hangs on waiting for future 
> completion:
> {noformat}
> "vert.x-worker-thread-3" #4236 prio=5 os_prio=0 tid=0x00007f47f43c3800 
> nid=0xc2f waiting on condition [0x00007f475ebe6000]
>    java.lang.Thread.State: WAITING (parking)
>       at sun.misc.Unsafe.park(Native Method)
>       - parking to wait for  <0x00000000cee89bc8> (a 
> org.apache.ignite.internal.util.future.GridFutureAdapter$ChainFuture)
>       at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
>       at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)
>       at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:997)
>       at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304)
>       at 
> org.apache.ignite.internal.util.future.GridFutureAdapter.get(GridFutureAdapter.java:115)
>       at 
> org.apache.ignite.internal.processors.cache.GridCacheAdapter.containsKey(GridCacheAdapter.java:510)
>       at 
> org.apache.ignite.internal.processors.cache.IgniteCacheProxy.containsKey(IgniteCacheProxy.java:869)
>       at io.vertx.spi.cluster.ignite.impl.MapImpl.containsKey(MapImpl.java:63)
>       at io.vertx.core.impl.HAManager.checkQuorumWhenAdded(HAManager.java:287)
>       - locked <0x00000000c3f295d0> (a io.vertx.core.impl.HAManager)
>       at io.vertx.core.impl.HAManager.nodeAdded(HAManager.java:253)
>       - locked <0x00000000c3f295d0> (a io.vertx.core.impl.HAManager)
>       at io.vertx.core.impl.HAManager.access$000(HAManager.java:96)
>       at io.vertx.core.impl.HAManager$1.nodeAdded(HAManager.java:139)
>       at 
> io.vertx.spi.cluster.ignite.IgniteClusterManager.lambda$null$4(IgniteClusterManager.java:221)
>       at 
> io.vertx.spi.cluster.ignite.IgniteClusterManager$$Lambda$54/05230435.handle(Unknown
>  Source)
>       at 
> io.vertx.core.impl.ContextImpl.lambda$executeBlocking$18(ContextImpl.java:279)
>       at io.vertx.core.impl.ContextImpl$$Lambda$20/445157774.run(Unknown 
> Source)
>       at 
> io.vertx.core.impl.OrderedExecutorFactory$OrderedExecutor.lambda$new$101(OrderedExecutorFactory.java:91)
>       at 
> io.vertx.core.impl.OrderedExecutorFactory$OrderedExecutor$$Lambda$16/061804750.run(Unknown
>  Source)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>       at java.lang.Thread.run(Thread.java:745)
> {noformat}
> This thread also hold lock and blocks other threads waiting on this lock. 
> Similar hangs are possible for other cache opeartions (e.g. get, getAndPut)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to