You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ignite.apache.org by "Valentin Kulichenko (JIRA)" <ji...@apache.org> on 2016/08/10 21:58:21 UTC

[jira] [Closed] (IGNITE-2783) Frequent deadlock during regular operations

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

Valentin Kulichenko closed IGNITE-2783.
---------------------------------------

> Frequent deadlock during regular operations
> -------------------------------------------
>
>                 Key: IGNITE-2783
>                 URL: https://issues.apache.org/jira/browse/IGNITE-2783
>             Project: Ignite
>          Issue Type: Bug
>          Components: cache, community, important
>    Affects Versions: 1.5.0.final
>            Reporter: Noam Liran
>            Assignee: Valentin Kulichenko
>            Priority: Critical
>             Fix For: 1.8
>
>         Attachments: w6-m4-threads.txt
>
>
> We've ran into this severe deadlock several times in the past few weeks. This time we've managed to collect quite a bit of data which might help us discover the cause.
> In general, we believe the deadlock involves two operations:
> * Cache creation (and thus subsequent partition exchange)
> * Service creation (call to IgniteServices.deployMultiple() )
> h3. General information about the environment
> * 77 ignite nodes; out of them:
> ** 3 seeds
> ** 74 regular nodes
> * ~15 caches, some replicated and some distributed
> * ~100 computing jobs running concurrently
> * ~10 new computing jobs per second
> h3. Relevant logs
> h4. w2:m5 (uuid={{1e3897db-2fb2-4bf2-8a99-10b40f8bc72b}}) starts a cache
> * Relevant logs:
> {noformat}
> 2016-03-07 12:38:04.989 WARN  o.a.i.i.p.c.GridCacheEvictionManager exchange-worker-#97%null% ctx:             actor:             - <adallom.adalib.discovery.InetResolvingCache__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150> Evictions are not synchronized with other nodes in topology which provides 2x-3x better performance but may cause data inconsistency if cache store is not configured (consider changing 'evictSynchronized' configuration property).
> 2016-03-07 12:38:04.992 INFO  o.a.i.i.p.cache.GridCacheProcessor   exchange-worker-#97%null% ctx:             actor:             - Started cache [name=adallom.adalib.discovery.InetResolvingCache__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150, mode=PARTITIONED]
> {noformat}
> * Unfortunately we don't have the stacktraces of this node but we have reason to believe we're still blocking in the call to getOrCreateCache().
> * Cache configuration:
> {code}
>         CACHE_CONFIGURATION = new CacheConfiguration<String, Integer>()
>                 .setName(CodeVersion.getVersionedName("adallom.adalib.discovery.InetResolvingCache"))
>                 .setCacheMode(CacheMode.PARTITIONED)
>                 .setAtomicityMode(CacheAtomicityMode.ATOMIC)
>                 .setBackups(0) // No need for backup, just resolve again if needed
>                 .setAffinity(new RendezvousAffinityFunction(true, 256))
>                 .setEvictionPolicy(new IgniteUtils2.MemoryFractionLruEvictionPolicy(10_000_000, 0.02))
>         ;
>         CACHE_CONFIGURATION
>                 .setExpiryPolicyFactory(CreatedExpiryPolicy.factoryOf(CACHE_EXPIRE_PERIOD));
> {code}
> h4. s2:m2 (uuid={{79060e43-df7b-460d-836e-508beda65cc4}}) deploys a service
> * Around the same time, another node (s2:m2) deploys a service (thus starting a transaction); we know this from the stack trace:
> {noformat}
> "ignite-#183%null%" #271 prio=5 os_prio=0 tid=0x00007fde3c450000 nid=0x2f2e waiting on condition [0x00007fdd6ee76000]
>    java.lang.Thread.State: WAITING (parking)
> 	at sun.misc.Unsafe.park(Native Method)
> 	- parking to wait for  <0x000000060a625040> (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.doAcquireShared(AbstractQueuedSynchronizer.java:967)
> 	at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireShared(AbstractQueuedSynchronizer.java:1283)
> 	at org.apache.ignite.internal.util.future.GridFutureAdapter.get0(GridFutureAdapter.java:155)
> 	at org.apache.ignite.internal.util.future.GridFutureAdapter.get(GridFutureAdapter.java:115)
> 	at org.apache.ignite.internal.processors.cache.GridCacheAdapter$32.op(GridCacheAdapter.java:2375)
> 	at org.apache.ignite.internal.processors.cache.GridCacheAdapter.syncOp(GridCacheAdapter.java:4073)
> 	at org.apache.ignite.internal.processors.cache.GridCacheAdapter.getAndPutIfAbsent(GridCacheAdapter.java:2373)
> 	at org.apache.ignite.internal.processors.service.GridServiceProcessor.deploy(GridServiceProcessor.java:411)
> 	at org.apache.ignite.internal.processors.service.GridServiceProcessor.deployMultiple(GridServiceProcessor.java:347)
> 	at org.apache.ignite.internal.IgniteServicesImpl.deployMultiple(IgniteServicesImpl.java:119)
> 	at com.adallom.minion.services.ServicePluginsDeployer.deploySingleService(ServicePluginsDeployer.java:109)
> 	at com.adallom.minion.services.ServicePluginsDeployer.deployAllServices(ServicePluginsDeployer.java:137)
> 	at com.adallom.minion.services.ServicePluginsDeployer.innerExecute(ServicePluginsDeployer.java:172)
> 	at com.adallom.minion.utils.ServiceWrapper.execute(ServiceWrapper.java:18)
> 	at org.apache.ignite.internal.processors.service.GridServiceProcessor$1.run(GridServiceProcessor.java:908)
> 	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> 	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> 	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}
> * The relevant logs in this node:
> {noformat}
> 2016-03-07 12:38:25.255 WARN  o.a.i.i.p.c.GridCacheEvictionManager exchange-worker-#145%null% ctx:             actor:             - <adallom.adalib.discovery.InetResolvingCache__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150> Evictions are not synchronized with other nodes in topology which provides 2x-3x better performance but may cause data inconsistency if cache store is not configured (consider changing 'evictSynchronized' configuration property).
> 2016-03-07 12:38:25.406 INFO  o.a.i.i.p.cache.GridCacheProcessor   exchange-worker-#145%null% ctx:             actor:             - Started cache [name=adallom.adalib.discovery.InetResolvingCache__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150, mode=PARTITIONED]
> 2016-03-07 12:38:36.877 WARN  o.a.i.i.p.c.d.d.p.GridDhtPartitionsExchangeFuture exchange-worker-#145%null% ctx:             actor:             - Failed to wait for partition release future [topVer=AffinityTopologyVersion [topVer=79, minorTopVer=3], node=79060e43-df7b-460d-836e-508beda65cc4]. Dumping pending objects that might be the cause:
> 2016-03-07 12:38:36.878 WARN  o.a.i.i.p.c.GridCachePartitionExchangeManager exchange-worker-#145%null% ctx:             actor:             - Pending transactions:
> 2016-03-07 12:38:36.892 WARN  o.a.i.i.p.c.GridCachePartitionExchangeManager exchange-worker-#145%null% ctx:             actor:             - >>> GridNearTxLocal [mappings=IgniteTxMappingsSingleImpl [mapping=GridDistributedTxMapping [entries=[IgniteTxEntry [key=KeyCacheObjectImpl [val=GridServiceDeploymentKey [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150], hasValBytes=true], cacheId=-2100569601, txKey=null, val=[op=CREATE, val=UserCacheObjectImpl [val=GridServiceDeployment [nodeId=79060e43-df7b-460d-836e-508beda65cc4, cfg=ServiceConfiguration [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150, totalCnt=1, maxPerNodeCnt=1, cacheName=null, svcCls=$Proxy59, nodeFilterCls=]], hasValBytes=true]], prevVal=[op=NOOP, val=null], entryProcessorsCol=null, ttl=-1, conflictExpireTime=-1, conflictVer=null, explicitVer=null, dhtVer=null, filters=[o.a.i.i.processors.cache.CacheEntrySerializablePredicate@3e091a15], filtersPassed=false, filtersSet=true, entry=GridDhtDetachedCacheEntry [super=GridDistributedCacheEntry [super=GridCacheMapEntry [key=KeyCacheObjectImpl [val=GridServiceDeploymentKey [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150], hasValBytes=true], val=null, startVer=1456927417382, ver=GridCacheVersion [topVer=68405809, nodeOrderDrId=77, globalTime=1457232145504, order=1456927417382], hash=1253220104, extras=null, flags=0]]], prepared=false, locked=false, nodeId=7001dd4b-9a06-4036-a6d7-b1e52e43a178, locMapped=false, expiryPlc=null, transferExpiryPlc=false, flags=0, partUpdateCntr=0, serReadVer=null, xidVer=GridCacheVersion [topVer=68405809, nodeOrderDrId=77, globalTime=1457232145504, order=1456927417380]]], explicitLock=false, dhtVer=null, last=false, near=false, clientFirst=false, node=7001dd4b-9a06-4036-a6d7-b1e52e43a178]], nearLocallyMapped=false, colocatedLocallyMapped=false, needCheckBackup=null, hasRemoteLocks=false, mappings=IgniteTxMappingsSingleImpl [mapping=GridDistributedTxMapping [entries=[IgniteTxEntry [key=KeyCacheObjectImpl [val=GridServiceDeploymentKey [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150], hasValBytes=true], cacheId=-2100569601, txKey=null, val=[op=CREATE, val=UserCacheObjectImpl [val=GridServiceDeployment [nodeId=79060e43-df7b-460d-836e-508beda65cc4, cfg=ServiceConfiguration [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150, totalCnt=1, maxPerNodeCnt=1, cacheName=null, svcCls=$Proxy59, nodeFilterCls=]], hasValBytes=true]], prevVal=[op=NOOP, val=null], entryProcessorsCol=null, ttl=-1, conflictExpireTime=-1, conflictVer=null, explicitVer=null, dhtVer=null, filters=[o.a.i.i.processors.cache.CacheEntrySerializablePredicate@3e091a15], filtersPassed=false, filtersSet=true, entry=GridDhtDetachedCacheEntry [super=GridDistributedCacheEntry [super=GridCacheMapEntry [key=KeyCacheObjectImpl [val=GridServiceDeploymentKey [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150], hasValBytes=true], val=null, startVer=1456927417382, ver=GridCacheVersion [topVer=68405809, nodeOrderDrId=77, globalTime=1457232145504, order=1456927417382], hash=1253220104, extras=null, flags=0]]], prepared=false, locked=false, nodeId=7001dd4b-9a06-4036-a6d7-b1e52e43a178, locMapped=false, expiryPlc=null, transferExpiryPlc=false, flags=0, partUpdateCntr=0, serReadVer=null, xidVer=GridCacheVersion [topVer=68405809, nodeOrderDrId=77, globalTime=1457232145504, order=1456927417380]]], explicitLock=false, dhtVer=null, last=false, near=false, clientFirst=false, node=7001dd4b-9a06-4036-a6d7-b1e52e43a178]], super=GridDhtTxLocalAdapter [nearOnOriginatingNode=false, nearNodes=[], dhtNodes=[], explicitLock=false, super=IgniteTxLocalAdapter [completedBase=null, sndTransformedVals=false, depEnabled=false, txState=IgniteTxImplicitSingleStateImpl [entry=IgniteTxEntry [key=KeyCacheObjectImpl [val=GridServiceDeploymentKey [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150], hasValBytes=true], cacheId=-2100569601, txKey=null, val=[op=CREATE, val=UserCacheObjectImpl [val=GridServiceDeployment [nodeId=79060e43-df7b-460d-836e-508beda65cc4, cfg=ServiceConfiguration [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150, totalCnt=1, maxPerNodeCnt=1, cacheName=null, svcCls=$Proxy59, nodeFilterCls=]], hasValBytes=true]], prevVal=[op=NOOP, val=null], entryProcessorsCol=null, ttl=-1, conflictExpireTime=-1, conflictVer=null, explicitVer=null, dhtVer=null, filters=[o.a.i.i.processors.cache.CacheEntrySerializablePredicate@3e091a15], filtersPassed=false, filtersSet=true, entry=GridDhtDetachedCacheEntry [super=GridDistributedCacheEntry [super=GridCacheMapEntry [key=KeyCacheObjectImpl [val=GridServiceDeploymentKey [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150], hasValBytes=true], val=null, startVer=1456927417382, ver=GridCacheVersion [topVer=68405809, nodeOrderDrId=77, globalTime=1457232145504, order=1456927417382], hash=1253220104, extras=null, flags=0]]], prepared=false, locked=false, nodeId=7001dd4b-9a06-4036-a6d7-b1e52e43a178, locMapped=false, expiryPlc=null, transferExpiryPlc=false, flags=0, partUpdateCntr=0, serReadVer=null, xidVer=GridCacheVersion [topVer=68405809, nodeOrderDrId=77, globalTime=1457232145504, order=1456927417380]], init=true], super=IgniteTxAdapter [xidVer=GridCacheVersion [topVer=68405809, nodeOrderDrId=77, globalTime=1457232145504, order=1456927417380], writeVer=null, implicit=true, loc=true, threadId=271, startTime=1457232145496, nodeId=79060e43-df7b-460d-836e-508beda65cc4, startVer=GridCacheVersion [topVer=68405809, nodeOrderDrId=77, globalTime=1457232145504, order=1456927417380], endVer=null, isolation=READ_COMMITTED, concurrency=OPTIMISTIC, timeout=0, sysInvalidate=false, sys=true, plc=5, commitVer=GridCacheVersion [topVer=68405809, nodeOrderDrId=77, globalTime=1457232145504, order=1456927417380], finalizing=NONE, preparing=false, invalidParts=null, state=PREPARING, timedOut=false, topVer=AffinityTopologyVersion [topVer=79, minorTopVer=2], duration=122171380ms, onePhaseCommit=false], size=1]]]
> 2016-03-07 12:38:36.895 WARN  o.a.i.i.p.c.GridCachePartitionExchangeManager exchange-worker-#145%null% ctx:             actor:             - >>> GridDhtTxRemote [nearNodeId=79060e43-df7b-460d-836e-508beda65cc4, rmtFutId=0e077983351-4e68462b-6b17-4b35-bcb3-c200e3020006, nearXidVer=GridCacheVersion [topVer=68405809, nodeOrderDrId=77, globalTime=1457232145504, order=1456927417380], super=GridDistributedTxRemoteAdapter [explicitVers=null, started=true, commitAllowed=false, txState=IgniteTxRemoteSingleStateImpl [entry=IgniteTxEntry [key=KeyCacheObjectImpl [val=GridServiceDeploymentKey [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150], hasValBytes=true], cacheId=-2100569601, txKey=IgniteTxKey [key=KeyCacheObjectImpl [val=GridServiceDeploymentKey [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150], hasValBytes=true], cacheId=-2100569601], val=[op=NOOP, val=CacheObjectImpl [val=null, hasValBytes=true]], prevVal=[op=NOOP, val=null], entryProcessorsCol=null, ttl=-1, conflictExpireTime=-1, conflictVer=null, explicitVer=null, dhtVer=null, filters=[o.a.i.i.processors.cache.CacheEntrySerializablePredicate@5814ecf5], filtersPassed=false, filtersSet=false, entry=GridDhtColocatedCacheEntry [super=GridDhtCacheEntry [rdrs=[], locPart=GridDhtLocalPartition [id=4, mapPubSize=0, rmvQueue=GridCircularBuffer [sizeMask=127, idxGen=0], cntr=4, state=OWNING, reservations=0, empty=false, createTime=03/02/2016 13:41:52, mapPubSize=0], super=GridDistributedCacheEntry [super=GridCacheMapEntry [key=KeyCacheObjectImpl [val=GridServiceDeploymentKey [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150], hasValBytes=true], val=CacheObjectImpl [val=null, hasValBytes=true], startVer=1456926107557, ver=GridCacheVersion [topVer=68405807, nodeOrderDrId=24, globalTime=1456926106701, order=1456926107557], hash=36575208, extras=GridCacheMvccEntryExtras [mvcc=GridCacheMvcc [locs=null, rmts=[GridCacheMvccCandidate [nodeId=7001dd4b-9a06-4036-a6d7-b1e52e43a178, ver=GridCacheVersion [topVer=68405809, nodeOrderDrId=24, globalTime=1457232128528, order=1456927417381], timeout=0, ts=1457232145506, threadId=256, id=45808, topVer=AffinityTopologyVersion [topVer=-1, minorTopVer=0], reentry=null, otherNodeId=79060e43-df7b-460d-836e-508beda65cc4, otherVer=null, mappedDhtNodes=null, mappedNearNodes=null, ownerVer=null, serOrder=null, key=KeyCacheObjectImpl [val=GridServiceDeploymentKey [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150], hasValBytes=true], masks=local=0|owner=0|ready=0|reentry=0|used=0|tx=1|single_implicit=0|dht_local=0|near_local=0|removed=0, prevVer=null, nextVer=null]]]], flags=0]]]], prepared=true, locked=false, nodeId=null, locMapped=false, expiryPlc=null, transferExpiryPlc=false, flags=0, partUpdateCntr=0, serReadVer=null, xidVer=null]], super=IgniteTxAdapter [xidVer=GridCacheVersion [topVer=68405809, nodeOrderDrId=24, globalTime=1457232128528, order=1456927417381], writeVer=GridCacheVersion [topVer=68405809, nodeOrderDrId=24, globalTime=1457232128529, order=1456927417382], implicit=false, loc=false, threadId=256, startTime=1457232145506, nodeId=7001dd4b-9a06-4036-a6d7-b1e52e43a178, startVer=GridCacheVersion [topVer=68405809, nodeOrderDrId=77, globalTime=1457232145504, order=1456927417382], endVer=null, isolation=READ_COMMITTED, concurrency=OPTIMISTIC, timeout=0, sysInvalidate=false, sys=true, plc=5, commitVer=null, finalizing=NONE, preparing=false, invalidParts=null, state=PREPARED, timedOut=false, topVer=AffinityTopologyVersion [topVer=79, minorTopVer=2], duration=122171381ms, onePhaseCommit=false]]]
> 2016-03-07 12:38:36.895 WARN  o.a.i.i.p.c.GridCachePartitionExchangeManager exchange-worker-#145%null% ctx:             actor:             - Pending explicit locks:
> 2016-03-07 12:38:36.895 WARN  o.a.i.i.p.c.GridCachePartitionExchangeManager exchange-worker-#145%null% ctx:             actor:             - Pending cache futures:
> 2016-03-07 12:38:36.900 WARN  o.a.i.i.p.c.GridCachePartitionExchangeManager exchange-worker-#145%null% ctx:             actor:             - >>> GridNearOptimisticTxPrepareFuture [innerFuts=[[node=7001dd4b-9a06-4036-a6d7-b1e52e43a178, loc=false, done=false]], keyLockFut=null, tx=GridNearTxLocal [mappings=IgniteTxMappingsSingleImpl [mapping=GridDistributedTxMapping [entries=[IgniteTxEntry [key=KeyCacheObjectImpl [val=GridServiceDeploymentKey [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150], hasValBytes=true], cacheId=-2100569601, txKey=null, val=[op=CREATE, val=UserCacheObjectImpl [val=GridServiceDeployment [nodeId=79060e43-df7b-460d-836e-508beda65cc4, cfg=ServiceConfiguration [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150, totalCnt=1, maxPerNodeCnt=1, cacheName=null, svcCls=$Proxy59, nodeFilterCls=]], hasValBytes=true]], prevVal=[op=NOOP, val=null], entryProcessorsCol=null, ttl=-1, conflictExpireTime=-1, conflictVer=null, explicitVer=null, dhtVer=null, filters=[o.a.i.i.processors.cache.CacheEntrySerializablePredicate@3e091a15], filtersPassed=false, filtersSet=true, entry=GridDhtDetachedCacheEntry [super=GridDistributedCacheEntry [super=GridCacheMapEntry [key=KeyCacheObjectImpl [val=GridServiceDeploymentKey [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150], hasValBytes=true], val=null, startVer=1456927417382, ver=GridCacheVersion [topVer=68405809, nodeOrderDrId=77, globalTime=1457232145504, order=1456927417382], hash=1253220104, extras=null, flags=0]]], prepared=false, locked=false, nodeId=7001dd4b-9a06-4036-a6d7-b1e52e43a178, locMapped=false, expiryPlc=null, transferExpiryPlc=false, flags=0, partUpdateCntr=0, serReadVer=null, xidVer=GridCacheVersion [topVer=68405809, nodeOrderDrId=77, globalTime=1457232145504, order=1456927417380]]], explicitLock=false, dhtVer=null, last=false, near=false, clientFirst=false, node=7001dd4b-9a06-4036-a6d7-b1e52e43a178]], nearLocallyMapped=false, colocatedLocallyMapped=false, needCheckBackup=null, hasRemoteLocks=false, mappings=IgniteTxMappingsSingleImpl [mapping=GridDistributedTxMapping [entries=[IgniteTxEntry [key=KeyCacheObjectImpl [val=GridServiceDeploymentKey [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150], hasValBytes=true], cacheId=-2100569601, txKey=null, val=[op=CREATE, val=UserCacheObjectImpl [val=GridServiceDeployment [nodeId=79060e43-df7b-460d-836e-508beda65cc4, cfg=ServiceConfiguration [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150, totalCnt=1, maxPerNodeCnt=1, cacheName=null, svcCls=$Proxy59, nodeFilterCls=]], hasValBytes=true]], prevVal=[op=NOOP, val=null], entryProcessorsCol=null, ttl=-1, conflictExpireTime=-1, conflictVer=null, explicitVer=null, dhtVer=null, filters=[o.a.i.i.processors.cache.CacheEntrySerializablePredicate@3e091a15], filtersPassed=false, filtersSet=true, entry=GridDhtDetachedCacheEntry [super=GridDistributedCacheEntry [super=GridCacheMapEntry [key=KeyCacheObjectImpl [val=GridServiceDeploymentKey [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150], hasValBytes=true], val=null, startVer=1456927417382, ver=GridCacheVersion [topVer=68405809, nodeOrderDrId=77, globalTime=1457232145504, order=1456927417382], hash=1253220104, extras=null, flags=0]]], prepared=false, locked=false, nodeId=7001dd4b-9a06-4036-a6d7-b1e52e43a178, locMapped=false, expiryPlc=null, transferExpiryPlc=false, flags=0, partUpdateCntr=0, serReadVer=null, xidVer=GridCacheVersion [topVer=68405809, nodeOrderDrId=77, globalTime=1457232145504, order=1456927417380]]], explicitLock=false, dhtVer=null, last=false, near=false, clientFirst=false, node=7001dd4b-9a06-4036-a6d7-b1e52e43a178]], super=GridDhtTxLocalAdapter [nearOnOriginatingNode=false, nearNodes=[], dhtNodes=[], explicitLock=false, super=IgniteTxLocalAdapter [completedBase=null, sndTransformedVals=false, depEnabled=false, txState=IgniteTxImplicitSingleStateImpl [entry=IgniteTxEntry [key=KeyCacheObjectImpl [val=GridServiceDeploymentKey [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150], hasValBytes=true], cacheId=-2100569601, txKey=null, val=[op=CREATE, val=UserCacheObjectImpl [val=GridServiceDeployment [nodeId=79060e43-df7b-460d-836e-508beda65cc4, cfg=ServiceConfiguration [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150, totalCnt=1, maxPerNodeCnt=1, cacheName=null, svcCls=$Proxy59, nodeFilterCls=]], hasValBytes=true]], prevVal=[op=NOOP, val=null], entryProcessorsCol=null, ttl=-1, conflictExpireTime=-1, conflictVer=null, explicitVer=null, dhtVer=null, filters=[o.a.i.i.processors.cache.CacheEntrySerializablePredicate@3e091a15], filtersPassed=false, filtersSet=true, entry=GridDhtDetachedCacheEntry [super=GridDistributedCacheEntry [super=GridCacheMapEntry [key=KeyCacheObjectImpl [val=GridServiceDeploymentKey [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150], hasValBytes=true], val=null, startVer=1456927417382, ver=GridCacheVersion [topVer=68405809, nodeOrderDrId=77, globalTime=1457232145504, order=1456927417382], hash=1253220104, extras=null, flags=0]]], prepared=false, locked=false, nodeId=7001dd4b-9a06-4036-a6d7-b1e52e43a178, locMapped=false, expiryPlc=null, transferExpiryPlc=false, flags=0, partUpdateCntr=0, serReadVer=null, xidVer=GridCacheVersion [topVer=68405809, nodeOrderDrId=77, globalTime=1457232145504, order=1456927417380]], init=true], super=IgniteTxAdapter [xidVer=GridCacheVersion [topVer=68405809, nodeOrderDrId=77, globalTime=1457232145504, order=1456927417380], writeVer=null, implicit=true, loc=true, threadId=271, startTime=1457232145496, nodeId=79060e43-df7b-460d-836e-508beda65cc4, startVer=GridCacheVersion [topVer=68405809, nodeOrderDrId=77, globalTime=1457232145504, order=1456927417380], endVer=null, isolation=READ_COMMITTED, concurrency=OPTIMISTIC, timeout=0, sysInvalidate=false, sys=true, plc=5, commitVer=GridCacheVersion [topVer=68405809, nodeOrderDrId=77, globalTime=1457232145504, order=1456927417380], finalizing=NONE, preparing=false, invalidParts=null, state=PREPARING, timedOut=false, topVer=AffinityTopologyVersion [topVer=79, minorTopVer=2], duration=122171401ms, onePhaseCommit=false], size=1]]], super=GridCompoundFuture [rdc=o.a.i.i.processors.cache.distributed.near.GridNearTxPrepareFutureAdapter$1@1e98759c, flags=1, lsnrCalls=0, done=false, cancelled=false, err=null, futs=[false]]]
> 2016-03-07 12:38:36.903 WARN  o.a.i.i.p.c.GridCachePartitionExchangeManager exchange-worker-#145%null% ctx:             actor:             - >>> GridNearTxFinishFuture [futId=bfcec983351-ce3a6aa2-f457-45ca-84b9-3a8de060e116, tx=GridNearTxLocal [mappings=IgniteTxMappingsSingleImpl [mapping=GridDistributedTxMapping [entries=[IgniteTxEntry [key=KeyCacheObjectImpl [val=GridServiceDeploymentKey [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150], hasValBytes=true], cacheId=-2100569601, txKey=null, val=[op=CREATE, val=UserCacheObjectImpl [val=GridServiceDeployment [nodeId=79060e43-df7b-460d-836e-508beda65cc4, cfg=ServiceConfiguration [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150, totalCnt=1, maxPerNodeCnt=1, cacheName=null, svcCls=$Proxy59, nodeFilterCls=]], hasValBytes=true]], prevVal=[op=NOOP, val=null], entryProcessorsCol=null, ttl=-1, conflictExpireTime=-1, conflictVer=null, explicitVer=null, dhtVer=null, filters=[o.a.i.i.processors.cache.CacheEntrySerializablePredicate@3e091a15], filtersPassed=false, filtersSet=true, entry=GridDhtDetachedCacheEntry [super=GridDistributedCacheEntry [super=GridCacheMapEntry [key=KeyCacheObjectImpl [val=GridServiceDeploymentKey [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150], hasValBytes=true], val=null, startVer=1456927417382, ver=GridCacheVersion [topVer=68405809, nodeOrderDrId=77, globalTime=1457232145504, order=1456927417382], hash=1253220104, extras=null, flags=0]]], prepared=false, locked=false, nodeId=7001dd4b-9a06-4036-a6d7-b1e52e43a178, locMapped=false, expiryPlc=null, transferExpiryPlc=false, flags=0, partUpdateCntr=0, serReadVer=null, xidVer=GridCacheVersion [topVer=68405809, nodeOrderDrId=77, globalTime=1457232145504, order=1456927417380]]], explicitLock=false, dhtVer=null, last=false, near=false, clientFirst=false, node=7001dd4b-9a06-4036-a6d7-b1e52e43a178]], nearLocallyMapped=false, colocatedLocallyMapped=false, needCheckBackup=null, hasRemoteLocks=false, mappings=IgniteTxMappingsSingleImpl [mapping=GridDistributedTxMapping [entries=[IgniteTxEntry [key=KeyCacheObjectImpl [val=GridServiceDeploymentKey [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150], hasValBytes=true], cacheId=-2100569601, txKey=null, val=[op=CREATE, val=UserCacheObjectImpl [val=GridServiceDeployment [nodeId=79060e43-df7b-460d-836e-508beda65cc4, cfg=ServiceConfiguration [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150, totalCnt=1, maxPerNodeCnt=1, cacheName=null, svcCls=$Proxy59, nodeFilterCls=]], hasValBytes=true]], prevVal=[op=NOOP, val=null], entryProcessorsCol=null, ttl=-1, conflictExpireTime=-1, conflictVer=null, explicitVer=null, dhtVer=null, filters=[o.a.i.i.processors.cache.CacheEntrySerializablePredicate@3e091a15], filtersPassed=false, filtersSet=true, entry=GridDhtDetachedCacheEntry [super=GridDistributedCacheEntry [super=GridCacheMapEntry [key=KeyCacheObjectImpl [val=GridServiceDeploymentKey [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150], hasValBytes=true], val=null, startVer=1456927417382, ver=GridCacheVersion [topVer=68405809, nodeOrderDrId=77, globalTime=1457232145504, order=1456927417382], hash=1253220104, extras=null, flags=0]]], prepared=false, locked=false, nodeId=7001dd4b-9a06-4036-a6d7-b1e52e43a178, locMapped=false, expiryPlc=null, transferExpiryPlc=false, flags=0, partUpdateCntr=0, serReadVer=null, xidVer=GridCacheVersion [topVer=68405809, nodeOrderDrId=77, globalTime=1457232145504, order=1456927417380]]], explicitLock=false, dhtVer=null, last=false, near=false, clientFirst=false, node=7001dd4b-9a06-4036-a6d7-b1e52e43a178]], super=GridDhtTxLocalAdapter [nearOnOriginatingNode=false, nearNodes=[], dhtNodes=[], explicitLock=false, super=IgniteTxLocalAdapter [completedBase=null, sndTransformedVals=false, depEnabled=false, txState=IgniteTxImplicitSingleStateImpl [entry=IgniteTxEntry [key=KeyCacheObjectImpl [val=GridServiceDeploymentKey [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150], hasValBytes=true], cacheId=-2100569601, txKey=null, val=[op=CREATE, val=UserCacheObjectImpl [val=GridServiceDeployment [nodeId=79060e43-df7b-460d-836e-508beda65cc4, cfg=ServiceConfiguration [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150, totalCnt=1, maxPerNodeCnt=1, cacheName=null, svcCls=$Proxy59, nodeFilterCls=]], hasValBytes=true]], prevVal=[op=NOOP, val=null], entryProcessorsCol=null, ttl=-1, conflictExpireTime=-1, conflictVer=null, explicitVer=null, dhtVer=null, filters=[o.a.i.i.processors.cache.CacheEntrySerializablePredicate@3e091a15], filtersPassed=false, filtersSet=true, entry=GridDhtDetachedCacheEntry [super=GridDistributedCacheEntry [super=GridCacheMapEntry [key=KeyCacheObjectImpl [val=GridServiceDeploymentKey [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150], hasValBytes=true], val=null, startVer=1456927417382, ver=GridCacheVersion [topVer=68405809, nodeOrderDrId=77, globalTime=1457232145504, order=1456927417382], hash=1253220104, extras=null, flags=0]]], prepared=false, locked=false, nodeId=7001dd4b-9a06-4036-a6d7-b1e52e43a178, locMapped=false, expiryPlc=null, transferExpiryPlc=false, flags=0, partUpdateCntr=0, serReadVer=null, xidVer=GridCacheVersion [topVer=68405809, nodeOrderDrId=77, globalTime=1457232145504, order=1456927417380]], init=true], super=IgniteTxAdapter [xidVer=GridCacheVersion [topVer=68405809, nodeOrderDrId=77, globalTime=1457232145504, order=1456927417380], writeVer=null, implicit=true, loc=true, threadId=271, startTime=1457232145496, nodeId=79060e43-df7b-460d-836e-508beda65cc4, startVer=GridCacheVersion [topVer=68405809, nodeOrderDrId=77, globalTime=1457232145504, order=1456927417380], endVer=null, isolation=READ_COMMITTED, concurrency=OPTIMISTIC, timeout=0, sysInvalidate=false, sys=true, plc=5, commitVer=GridCacheVersion [topVer=68405809, nodeOrderDrId=77, globalTime=1457232145504, order=1456927417380], finalizing=NONE, preparing=false, invalidParts=null, state=PREPARING, timedOut=false, topVer=AffinityTopologyVersion [topVer=79, minorTopVer=2], duration=122171401ms, onePhaseCommit=false], size=1]]], commit=true, mappings=IgniteTxMappingsSingleImpl [mapping=GridDistributedTxMapping [entries=[IgniteTxEntry [key=KeyCacheObjectImpl [val=GridServiceDeploymentKey [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150], hasValBytes=true], cacheId=-2100569601, txKey=null, val=[op=CREATE, val=UserCacheObjectImpl [val=GridServiceDeployment [nodeId=79060e43-df7b-460d-836e-508beda65cc4, cfg=ServiceConfiguration [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150, totalCnt=1, maxPerNodeCnt=1, cacheName=null, svcCls=$Proxy59, nodeFilterCls=]], hasValBytes=true]], prevVal=[op=NOOP, val=null], entryProcessorsCol=null, ttl=-1, conflictExpireTime=-1, conflictVer=null, explicitVer=null, dhtVer=null, filters=[o.a.i.i.processors.cache.CacheEntrySerializablePredicate@3e091a15], filtersPassed=false, filtersSet=true, entry=GridDhtDetachedCacheEntry [super=GridDistributedCacheEntry [super=GridCacheMapEntry [key=KeyCacheObjectImpl [val=GridServiceDeploymentKey [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150], hasValBytes=true], val=null, startVer=1456927417382, ver=GridCacheVersion[topVer=68405809, nodeOrderDrId=77, globalTime=1457232145504, order=1456927417382], hash=1253220104, extras=null, flags=0]]], prepared=false, locked=false, nodeId=7001dd4b-9a06-4036-a6d7-b1e52e43a178, locMapped=false, expiryPlc=null, transferExpiryPlc=false, flags=0, partUpdateCntr=0, serReadVer=null, xidVer=GridCacheVersion [topVer=68405809, nodeOrderDrId=77, globalTime=1457232145504, order=1456927417380]]], explicitLock=false, dhtVer=null, last=false, near=false, clientFirst=false, node=7001dd4b-9a06-4036-a6d7-b1e52e43a178]], trackable=true, finishOnePhaseCalled=false, innerFuts=[], super=GridCompoundIdentityFuture [super=GridCompoundFuture [rdc=o.a.i.i.util.lang.GridFunc$27@7bca97a5, flags=0, lsnrCalls=0, done=false, cancelled=false, err=null, futs=[]]]]
> 2016-03-07 12:38:36.904 WARN  o.a.i.i.p.c.GridCachePartitionExchangeManager exchange-worker-#145%null% ctx:             actor:             - Pending atomic cache futures:
> {noformat}
> * The service deployment uses an OPTIMISTIC READ_COMMITED (2-phase commit) transaction. The primary node of the key {{CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150}} is w4:m1 (uuid={{7001dd4b-9a06-4036-a6d7-b1e52e43a178}}).
> h4. w4:m1 (uuid={{7001dd4b-9a06-4036-a6d7-b1e52e43a178}}) sends a GridDhtTxPrepareRequest request to all the cluster
> * Relevant logs from this node:
> {noformat}
> 2016-03-07 12:38:05.195 WARN  o.a.i.i.p.c.GridCacheEvictionManager exchange-worker-#97%null% ctx:             actor:             - <adallom.adalib.discovery.InetResolvingCache__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150> Evictions are not synchronized with other nodes in topology which provides 2x-3x better performance but may cause data inconsistency if cache store is not configured (consider changing 'evictSynchronized' configuration property).
> 2016-03-07 12:38:05.196 INFO  o.a.i.i.p.cache.GridCacheProcessor   exchange-worker-#97%null% ctx:             actor:             - Started cache [name=adallom.adalib.discovery.InetResolvingCache__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150, mode=PARTITIONED]
> 2016-03-07 12:38:16.519 WARN  o.a.i.i.p.c.d.d.p.GridDhtPartitionsExchangeFuture exchange-worker-#97%null% ctx:             actor:             - Failed to wait for partition release future [topVer=AffinityTopologyVersion [topVer=79, minorTopVer=3], node=7001dd4b-9a06-4036-a6d7-b1e52e43a178]. Dumping pending objects that might be the cause:
> 2016-03-07 12:38:16.520 WARN  o.a.i.i.p.c.GridCachePartitionExchangeManager exchange-worker-#97%null% ctx:             actor:             - Pending transactions:
> 2016-03-07 12:38:16.527 WARN  o.a.i.i.p.c.GridCachePartitionExchangeManager exchange-worker-#97%null% ctx:             actor:             - >>> GridDhtTxLocal [nearNodeId=79060e43-df7b-460d-836e-508beda65cc4, nearFutId=9fcec983351-ce3a6aa2-f457-45ca-84b9-3a8de060e116, nearMiniId=afcec983351-ce3a6aa2-f457-45ca-84b9-3a8de060e116, nearFinFutId=null, nearFinMiniId=null, nearXidVer=GridCacheVersion [topVer=68405809, nodeOrderDrId=77, globalTime=1457232145504, order=1456927417380], super=GridDhtTxLocalAdapter [nearOnOriginatingNode=false, nearNodes=[], dhtNodes=[9b9aaff8-4fb1-4abf-8e75-d42c0091bead, 4091212e-522f-4d85-8bd6-ca5d9a2e5d79, c69c94dc-bf00-4872-9fa2-6abe7d08dbc7, 11b12205-befa-4718-8e2b-56f470b8c7ec, 1b08c371-abb0-457a-a8ca-c8556ca5520a, 79060e43-df7b-460d-836e-508beda65cc4, bc7faf9a-33a5-455a-b37a-ec4308b237f2, f00308e6-7722-4cdc-a9ea-1295db61e991, ca68972c-b202-47d4-bfe7-4e8529940538, 4b015572-e3ee-47dc-8c0c-d28db88feb02, 2ee9a413-12f6-4ce3-9711-7fe3a6a9c467, 793286f2-f838-4162-b3c1-49bde16ecc8e, 847dcaaf-9a43-4054-80e1-ceb4ef660cab, 9f86de50-996d-47ad-b316-828054fa381e, 41234aa4-d103-4c5d-9196-e49d0ad026de, 680ad185-d233-4a07-9d7f-8530abf1d738, 4139e967-02cb-481c-9f8e-66e9b8d5d5ed, cd9b47a4-d85d-4a9f-921c-9fe5919b7264, a162ecd5-542f-489d-96b0-a0e30c9b8b95, 8cda7a9d-1f9a-4f97-8a22-adc1999f6247, b09accf0-0d48-49bb-bdaf-5e79998cc8ee, 2b543a72-a081-4aac-9c9f-f7a0aadec5ad, 16dfc445-d614-4aae-b81d-74d9c5ea64ce, 94777ec7-2e96-45b9-affb-f5be511df4cb, d7b5c049-a409-4880-9851-af698999ba7f, 2a061509-839e-4a70-b8b1-a4b67799c6d7, 0f0bd17f-1031-45d0-a859-17df6722ba54, 0e2545b4-d7e1-4952-9022-f5944f90230a, d4c6f8d8-92c3-4124-91f0-edbeff9aa95d, c0a82a7e-7517-4b04-8042-c46658ab0290, 990bbdf7-cce2-48e0-8801-3898e8084021, 81cee701-07bb-4935-a9af-e87a4f66cfa4, e3777049-9577-425d-9501-3df6bc8c71df, 56802eb6-ae1e-4dc5-b3d9-dd326e24b752, 2113399c-0291-414f-9695-65c1de363787, 420b592f-987b-411b-b863-9663bac05ef4, 2f472427-b63c-4ba2-a08c-0bf8f13cf10c, c7f8cee3-4542-4d26-9c27-485549b6ac5c, 6e7883b1-cd41-4ed9-af61-f073e38fb5f7, 48d3a013-aa20-4814-b52f-463646612715, 58815231-319b-479d-aadf-c4603c5c7812, 4e6d188d-2cd7-4faf-84cc-2d88b04c446c, 31ddaece-5e04-4330-88bb-a03f5d3983c6, 8a20ed61-2e67-452c-aa45-0462e0dd7b99, 82ef0718-4d67-42a5-aa34-d34f32abd8f5, 2e6dbdde-327a-438e-adc9-50527066cc3f, b126c5e0-ecba-4679-b0a7-60957ffb310e, 071375af-0299-4980-a86a-df6d6e51e731, 01a6d0ff-69d4-4653-a5c3-6b493a351aef, 28cdf6c2-2727-4c24-a71e-31049882f908, 4a701be7-b730-4aca-a75c-0f9e97debb21, 8bfeb9a6-287d-4425-982a-638a8559f189, 96c77e8f-ea05-403a-bc8a-5c216e3a0556, a6bb5558-1f52-4e03-892d-fe2309d5b24f, 82e1cc53-4d7e-4cdf-8347-2f49b4eea092, 87c9477d-3b7d-4555-a730-e4a2e1cd82e6, d928a07a-81d3-4eb3-bc18-419a3e8912f0, 0c9330e4-fac1-4fc4-a043-ce3eb0abd0c4, 69b98a2b-6fac-47aa-a0a6-47382015f099, 5c5e3885-9d3f-48b1-9c38-3511dbaebee8, b5cc0e86-be5a-4b76-a82a-3271d8c79314, 5938589f-5c43-4e02-bb72-38488db01b40, 733cae8a-2b2c-4d12-b73d-d5656b00b1be, 02853b59-e878-4947-bd80-f970c020f08e, cf1819b9-cd8d-41cd-a839-722e433d2829, f91acdbb-09a7-4e3e-b85c-34b465c2527b, 0c37fa67-5d27-4d6b-a364-4c50aee2f018, 9dabafe2-d5ca-4aa8-8b7f-e6d061ff6d99, f9edc55f-d071-4604-b4be-ff76b79ac576, d2d28a22-2a28-453e-8d99-fab8ceea759d, 96daff83-35db-46a7-81ca-8725e0b36ef1, 97a66c9c-0eb7-4612-845c-ae3431051acf, d17dac4e-a89d-447e-b81e-6a068ae0cb52, 0e4c4028-5a9b-4b17-83b0-9cec501d7e72, d268794d-8790-49ed-8cae-c97eedae0c7c, 87cb8715-ad56-4da0-9655-a02e12d29340], explicitLock=false, super=IgniteTxLocalAdapter [completedBase=null, sndTransformedVals=false, depEnabled=false, txState=IgniteTxImplicitSingleStateImpl [entry=IgniteTxEntry [key=KeyCacheObjectImpl [val=GridServiceDeploymentKey [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150], hasValBytes=true], cacheId=-2100569601, txKey=IgniteTxKey [key=KeyCacheObjectImpl [val=GridServiceDeploymentKey [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150], hasValBytes=true], cacheId=-2100569601], val=[op=NOOP, val=CacheObjectImpl [val=null, hasValBytes=true]], prevVal=[op=NOOP, val=null], entryProcessorsCol=null, ttl=-1, conflictExpireTime=-1, conflictVer=null, explicitVer=null, dhtVer=null, filters=[o.a.i.i.processors.cache.CacheEntrySerializablePredicate@1c988081], filtersPassed=false, filtersSet=false, entry=GridDhtColocatedCacheEntry [super=GridDhtCacheEntry [rdrs=[], locPart=GridDhtLocalPartition [id=4, mapPubSize=0, rmvQueue=GridCircularBuffer [sizeMask=127, idxGen=0], cntr=4, state=OWNING, reservations=0, empty=false, createTime=03/02/2016 13:38:38, mapPubSize=0], super=GridDistributedCacheEntry [super=GridCacheMapEntry [key=KeyCacheObjectImpl [val=GridServiceDeploymentKey [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150], hasValBytes=true], val=CacheObjectImpl [val=null, hasValBytes=true], startVer=1456926107556, ver=GridCacheVersion [topVer=68405807, nodeOrderDrId=24, globalTime=1456926106701, order=1456926107557], hash=36575208, extras=GridCacheMvccEntryExtras [mvcc=GridCacheMvcc [locs=[GridCacheMvccCandidate [nodeId=7001dd4b-9a06-4036-a6d7-b1e52e43a178, ver=GridCacheVersion [topVer=68405809, nodeOrderDrId=24, globalTime=1457232128528, order=1456927417381], timeout=0, ts=1457232128527, threadId=271, id=45808, topVer=AffinityTopologyVersion [topVer=79, minorTopVer=2], reentry=null, otherNodeId=79060e43-df7b-460d-836e-508beda65cc4, otherVer=GridCacheVersion [topVer=68405809, nodeOrderDrId=77, globalTime=1457232145504, order=1456927417380], mappedDhtNodes=null, mappedNearNodes=null, ownerVer=null, serOrder=null, key=KeyCacheObjectImpl [val=GridServiceDeploymentKey [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150], hasValBytes=true], masks=local=1|owner=1|ready=1|reentry=0|used=0|tx=1|single_implicit=1|dht_local=1|near_local=0|removed=0, prevVer=null, nextVer=null]], rmts=null]], flags=0]]]], prepared=true, locked=false, nodeId=null, locMapped=false, expiryPlc=null, transferExpiryPlc=false, flags=0, partUpdateCntr=0, serReadVer=null, xidVer=null], init=true], super=IgniteTxAdapter [xidVer=GridCacheVersion [topVer=68405809, nodeOrderDrId=24, globalTime=1457232128528, order=1456927417381], writeVer=GridCacheVersion [topVer=68405809, nodeOrderDrId=24, globalTime=1457232128529, order=1456927417382], implicit=true, loc=true, threadId=271, startTime=1457232128527, nodeId=7001dd4b-9a06-4036-a6d7-b1e52e43a178, startVer=GridCacheVersion [topVer=68405809, nodeOrderDrId=24, globalTime=1457232128528, order=1456927417381], endVer=null, isolation=READ_COMMITTED, concurrency=OPTIMISTIC, timeout=0, sysInvalidate=false, sys=true, plc=5, commitVer=null, finalizing=NONE, preparing=false, invalidParts=null, state=PREPARING, timedOut=false, topVer=AffinityTopologyVersion [topVer=79, minorTopVer=2], duration=122167992ms, onePhaseCommit=false], size=1]]]
> 2016-03-07 12:38:16.527 WARN  o.a.i.i.p.c.GridCachePartitionExchangeManager exchange-worker-#97%null% ctx:             actor:             - Pending explicit locks:
> 2016-03-07 12:38:16.527 WARN  o.a.i.i.p.c.GridCachePartitionExchangeManager exchange-worker-#97%null% ctx:             actor:             - Pending cache futures:
> 2016-03-07 12:38:16.531 WARN  o.a.i.i.p.c.GridCachePartitionExchangeManager exchange-worker-#97%null% ctx:             actor:             - >>> GridDhtTxPrepareFuture [futId=0e077983351-4e68462b-6b17-4b35-bcb3-c200e3020006, err=null, replied=false, mapped=true, reads=null, writes=[IgniteTxEntry [key=KeyCacheObjectImpl [val=GridServiceDeploymentKey [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150], hasValBytes=true], cacheId=-2100569601, txKey=IgniteTxKey [key=KeyCacheObjectImpl [val=GridServiceDeploymentKey [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150], hasValBytes=true], cacheId=-2100569601], val=[op=NOOP, val=CacheObjectImpl [val=null, hasValBytes=true]], prevVal=[op=NOOP, val=null], entryProcessorsCol=null, ttl=-1, conflictExpireTime=-1, conflictVer=null, explicitVer=null, dhtVer=null, filters=[o.a.i.i.processors.cache.CacheEntrySerializablePredicate@1c988081], filtersPassed=false, filtersSet=false, entry=GridDhtColocatedCacheEntry [super=GridDhtCacheEntry [rdrs=[], locPart=GridDhtLocalPartition [id=4, mapPubSize=0, rmvQueue=GridCircularBuffer [sizeMask=127, idxGen=0], cntr=4, state=OWNING, reservations=0, empty=false, createTime=03/02/2016 13:38:38, mapPubSize=0], super=GridDistributedCacheEntry [super=GridCacheMapEntry [key=KeyCacheObjectImpl [val=GridServiceDeploymentKey [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150], hasValBytes=true], val=CacheObjectImpl [val=null, hasValBytes=true], startVer=1456926107556, ver=GridCacheVersion [topVer=68405807, nodeOrderDrId=24, globalTime=1456926106701, order=1456926107557], hash=36575208, extras=GridCacheMvccEntryExtras [mvcc=GridCacheMvcc [locs=[GridCacheMvccCandidate [nodeId=7001dd4b-9a06-4036-a6d7-b1e52e43a178, ver=GridCacheVersion [topVer=68405809, nodeOrderDrId=24, globalTime=1457232128528, order=1456927417381], timeout=0, ts=1457232128527, threadId=271, id=45808, topVer=AffinityTopologyVersion [topVer=79, minorTopVer=2], reentry=null, otherNodeId=79060e43-df7b-460d-836e-508beda65cc4, otherVer=GridCacheVersion [topVer=68405809, nodeOrderDrId=77, globalTime=1457232145504, order=1456927417380], mappedDhtNodes=null, mappedNearNodes=null, ownerVer=null, serOrder=null, key=KeyCacheObjectImpl [val=GridServiceDeploymentKey [name=CakeService_Default__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150], hasValBytes=true], masks=local=1|owner=1|ready=1|reentry=0|used=0|tx=1|single_implicit=1|dht_local=1|near_local=0|removed=0, prevVer=null, nextVer=null]], rmts=null]], flags=0]]]], prepared=true, locked=false, nodeId=null, locMapped=false, expiryPlc=null, transferExpiryPlc=false, flags=0, partUpdateCntr=0, serReadVer=null, xidVer=null]], trackable=true, nearMiniId=afcec983351-ce3a6aa2-f457-45ca-84b9-3a8de060e116, last=true, retVal=true, ret=GridCacheReturn [v=null, cacheObj=CacheObjectImpl [val=null, hasValBytes=true], success=false, invokeRes=false, loc=false, cacheId=-2100569601], lockKeys=[], forceKeysFut=GridDhtForceKeysFuture [topCntr=1, topVer=AffinityTopologyVersion [topVer=79, minorTopVer=2], futId=1e077983351-4e68462b-6b17-4b35-bcb3-c200e3020006, preloader=o.a.i.i.processors.cache.distributed.dht.preloader.GridDhtPreloader@111cb058, trackable=false, innerFuts=[], super=GridCompoundFuture [rdc=null, flags=1, lsnrCalls=0, done=true, cancelled=false, err=null, futs=[]]], locksReady=true, invoke=false, xid=GridCacheVersion [topVer=68405809, nodeOrderDrId=24, globalTime=1457232128528, order=1456927417381], innerFuts=[[node=9b9aaff8-4fb1-4abf-8e75-d42c0091bead, loc=false, done=true], [node=4091212e-522f-4d85-8bd6-ca5d9a2e5d79, loc=false, done=true], [node=c69c94dc-bf00-4872-9fa2-6abe7d08dbc7, loc=false, done=true], [node=11b12205-befa-4718-8e2b-56f470b8c7ec, loc=false, done=true], [node=1b08c371-abb0-457a-a8ca-c8556ca5520a, loc=false, done=true], [node=79060e43-df7b-460d-836e-508beda65cc4, loc=false, done=true], [node=bc7faf9a-33a5-455a-b37a-ec4308b237f2, loc=false, done=true], [node=f00308e6-7722-4cdc-a9ea-1295db61e991, loc=false, done=true], [node=ca68972c-b202-47d4-bfe7-4e8529940538, loc=false, done=true], [node=4b015572-e3ee-47dc-8c0c-d28db88feb02, loc=false, done=true], [node=2ee9a413-12f6-4ce3-9711-7fe3a6a9c467, loc=false, done=true], [node=793286f2-f838-4162-b3c1-49bde16ecc8e, loc=false, done=true], [node=847dcaaf-9a43-4054-80e1-ceb4ef660cab, loc=false, done=true], [node=9f86de50-996d-47ad-b316-828054fa381e, loc=false, done=true], [node=41234aa4-d103-4c5d-9196-e49d0ad026de, loc=false, done=true], [node=680ad185-d233-4a07-9d7f-8530abf1d738, loc=false, done=true], [node=4139e967-02cb-481c-9f8e-66e9b8d5d5ed, loc=false, done=true], [node=cd9b47a4-d85d-4a9f-921c-9fe5919b7264, loc=false, done=true], [node=a162ecd5-542f-489d-96b0-a0e30c9b8b95, loc=false, done=true], [node=8cda7a9d-1f9a-4f97-8a22-adc1999f6247, loc=false, done=true], [node=b09accf0-0d48-49bb-bdaf-5e79998cc8ee, loc=false, done=true], [node=2b543a72-a081-4aac-9c9f-f7a0aadec5ad, loc=false, done=true], [node=16dfc445-d614-4aae-b81d-74d9c5ea64ce, loc=false, done=true], [node=94777ec7-2e96-45b9-affb-f5be511df4cb, loc=false, done=true], [node=d7b5c049-a409-4880-9851-af698999ba7f, loc=false, done=true], [node=2a061509-839e-4a70-b8b1-a4b67799c6d7, loc=false, done=true], [node=0f0bd17f-1031-45d0-a859-17df6722ba54, loc=false, done=true], [node=0e2545b4-d7e1-4952-9022-f5944f90230a, loc=false, done=true], [node=d4c6f8d8-92c3-4124-91f0-edbeff9aa95d, loc=false, done=true], [node=c0a82a7e-7517-4b04-8042-c46658ab0290, loc=false, done=true], [node=990bbdf7-cce2-48e0-8801-3898e8084021, loc=false, done=true], [node=81cee701-07bb-4935-a9af-e87a4f66cfa4, loc=false, done=true], [node=e3777049-9577-425d-9501-3df6bc8c71df, loc=false, done=true], [node=56802eb6-ae1e-4dc5-b3d9-dd326e24b752, loc=false, done=true], [node=2113399c-0291-414f-9695-65c1de363787, loc=false, done=true], [node=420b592f-987b-411b-b863-9663bac05ef4, loc=false, done=true], [node=2f472427-b63c-4ba2-a08c-0bf8f13cf10c, loc=false, done=true], [node=c7f8cee3-4542-4d26-9c27-485549b6ac5c, loc=false, done=true], [node=6e7883b1-cd41-4ed9-af61-f073e38fb5f7, loc=false, done=true], [node=48d3a013-aa20-4814-b52f-463646612715, loc=false, done=true], [node=58815231-319b-479d-aadf-c4603c5c7812, loc=false, done=true], [node=4e6d188d-2cd7-4faf-84cc-2d88b04c446c, loc=false, done=true], [node=31ddaece-5e04-4330-88bb-a03f5d3983c6, loc=false, done=false], [node=8a20ed61-2e67-452c-aa45-0462e0dd7b99, loc=false, done=true], [node=82ef0718-4d67-42a5-aa34-d34f32abd8f5, loc=false, done=true], [node=2e6dbdde-327a-438e-adc9-50527066cc3f, loc=false, done=true], [node=b126c5e0-ecba-4679-b0a7-60957ffb310e, loc=false, done=true], [node=071375af-0299-4980-a86a-df6d6e51e731, loc=false, done=true], [node=01a6d0ff-69d4-4653-a5c3-6b493a351aef, loc=false, done=true], [node=28cdf6c2-2727-4c24-a71e-31049882f908, loc=false, done=true], [node=4a701be7-b730-4aca-a75c-0f9e97debb21, loc=false, done=true], [node=8bfeb9a6-287d-4425-982a-638a8559f189, loc=false, done=true], [node=96c77e8f-ea05-403a-bc8a-5c216e3a0556, loc=false, done=true], [node=a6bb5558-1f52-4e03-892d-fe2309d5b24f, loc=false, done=true], [node=82e1cc53-4d7e-4cdf-8347-2f49b4eea092, loc=false, done=true], [node=87c9477d-3b7d-4555-a730-e4a2e1cd82e6, loc=false, done=true], [node=d928a07a-81d3-4eb3-bc18-419a3e8912f0, loc=false, done=true], [node=0c9330e4-fac1-4fc4-a043-ce3eb0abd0c4, loc=false, done=true], [node=69b98a2b-6fac-47aa-a0a6-47382015f099, loc=false, done=true], [node=5c5e3885-9d3f-48b1-9c38-3511dbaebee8, loc=false, done=true], [node=b5cc0e86-be5a-4b76-a82a-3271d8c79314, loc=false, done=true], [node=5938589f-5c43-4e02-bb72-38488db01b40, loc=false, done=true], [node=733cae8a-2b2c-4d12-b73d-d5656b00b1be, loc=false, done=true], [node=02853b59-e878-4947-bd80-f970c020f08e, loc=false, done=true], [node=cf1819b9-cd8d-41cd-a839-722e433d2829, loc=false, done=true], [node=f91acdbb-09a7-4e3e-b85c-34b465c2527b, loc=false, done=true], [node=0c37fa67-5d27-4d6b-a364-4c50aee2f018, loc=false, done=true], [node=9dabafe2-d5ca-4aa8-8b7f-e6d061ff6d99, loc=false, done=true], [node=f9edc55f-d071-4604-b4be-ff76b79ac576, loc=false, done=true], [node=d2d28a22-2a28-453e-8d99-fab8ceea759d, loc=false, done=true], [node=96daff83-35db-46a7-81ca-8725e0b36ef1, loc=false, done=true], [node=97a66c9c-0eb7-4612-845c-ae3431051acf, loc=false, done=true], [node=d17dac4e-a89d-447e-b81e-6a068ae0cb52, loc=false, done=true], [node=0e4c4028-5a9b-4b17-83b0-9cec501d7e72, loc=false, done=true], [node=d268794d-8790-49ed-8cae-c97eedae0c7c, loc=false, done=true], [node=87cb8715-ad56-4da0-9655-a02e12d29340, loc=false, done=true]], super=GridCompoundFuture [rdc=o.a.i.i.processors.cache.distributed.dht.GridDhtTxPrepareFuture$1@728d8065, flags=1, lsnrCalls=75, done=false, cancelled=false, err=null, futs=[true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, false, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true]]]
> 2016-03-07 12:38:16.531 WARN  o.a.i.i.p.c.GridCachePartitionExchangeManager exchange-worker-#97%null% ctx:             actor:             - Pending atomic cache futures:
> {noformat}
> * The node with the unfinished future according to GridDhtTxPrepareRequest's list of futures (done=false) is w6:m4 (uuid={{31ddaece-5e04-4330-88bb-a03f5d3983c6}})
> h4. w6:m4 (uuid={{31ddaece-5e04-4330-88bb-a03f5d3983c6}}) blocks the transaction
> * This node is in a "Retry preload partition exchange" loop according to the logs:
> {noformat}
> 2016-03-07 12:38:10.500 WARN  o.a.i.i.p.c.GridCacheEvictionManager exchange-worker-#97%null% ctx:             actor:             - <adallom.adalib.discovery.InetResolvingCache__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150> Evictions are not synchronized with other nodes in topology which provides 2x-3x better performance but may cause data inconsistency if cache store is not configured (consider changing 'evictSynchronized' configuration property).
> 2016-03-07 12:38:10.502 INFO  o.a.i.i.p.cache.GridCacheProcessor   exchange-worker-#97%null% ctx:             actor:             - Started cache [name=adallom.adalib.discovery.InetResolvingCache__adalib-0.66.147_adalibpy-0.66.110_ae-0.66.136_cb-0.66.171_ep-0.66.147_lg-0.66.150_mn-0.66.181_qc-0.66.115_rp-0.66.168_sg-0.66.150, mode=PARTITIONED]
> 2016-03-07 12:38:26.993 WARN  o.a.i.i.p.c.d.d.p.GridDhtPartitionsExchangeFuture sys-#58%null%        ctx:             actor:             - Retrying preload partition exchange due to timeout [done=false, dummy=false, exchId=GridDhtPartitionExchangeId [topVer=AffinityTopologyVersion [topVer=79, minorTopVer=3], nodeId=793286f2, evt=DISCOVERY_CUSTOM_EVT], rcvdIds=[], rmtIds=[1b08c371, 2e6dbdde, 69b98a2b, 793286f2, 96daff83, 9b9aaff8, 94777ec7, 0e2545b4, 01a6d0ff, 82ef0718, f00308e6, 48d3a013, 28cdf6c2, 87c9477d, a6bb5558, 2a061509, d7b5c049, 5938589f, b09accf0, e3777049, 96c77e8f, d2d28a22, 2113399c, 79060e43, 5c5e3885, 2f472427, 6e7883b1, 680ad185, 97a66c9c, 02853b59, 16dfc445, 87cb8715, b126c5e0, 071375af, 58815231, 2b543a72, 4091212e, d928a07a, 4b015572, 990bbdf7, 4a701be7, 0c37fa67, 2ee9a413, 81cee701, 4139e967, a162ecd5, d268794d, c0a82a7e, 0e4c4028, 41234aa4, ca68972c, 11b12205, 847dcaaf, 0c9330e4, 82e1cc53, c69c94dc, 9dabafe2, cf1819b9, f9edc55f, bc7faf9a, 9f86de50, 0f0bd17f, c7f8cee3, f91acdbb, 8a20ed61, 733cae8a, b5cc0e86, d4c6f8d8, 8bfeb9a6, 56802eb6, 420b592f, 8cda7a9d, 7001dd4b, d17dac4e, cd9b47a4, 4e6d188d], remaining=[1b08c371, 2e6dbdde, 69b98a2b, 793286f2, 96daff83, 9b9aaff8, 94777ec7, 0e2545b4, 01a6d0ff, 82ef0718, f00308e6, 48d3a013, 28cdf6c2, 87c9477d, a6bb5558, 2a061509, d7b5c049, 5938589f, b09accf0, e3777049, 96c77e8f, d2d28a22, 2113399c, 79060e43, 5c5e3885, 2f472427, 6e7883b1, 680ad185, 97a66c9c, 02853b59, 16dfc445, 87cb8715, b126c5e0, 071375af, 58815231, 2b543a72, 4091212e, d928a07a, 4b015572, 990bbdf7, 4a701be7, 0c37fa67, 2ee9a413, 81cee701, 4139e967, a162ecd5, d268794d, c0a82a7e, 0e4c4028, 41234aa4, ca68972c, 11b12205, 847dcaaf, 0c9330e4, 82e1cc53, c69c94dc, 9dabafe2, cf1819b9, f9edc55f, bc7faf9a, 9f86de50, 0f0bd17f, c7f8cee3, f91acdbb, 8a20ed61, 733cae8a, b5cc0e86, d4c6f8d8, 8bfeb9a6, 56802eb6, 420b592f, 8cda7a9d, 7001dd4b, d17dac4e, cd9b47a4, 4e6d188d], init=true, initFut=true, ready=true, replied=false, added=true, oldest=4a701be7, oldestOrder=1, evtLatch=0, locNodeOrder=29, locNodeId=31ddaece-5e04-4330-88bb-a03f5d3983c6]
> {noformat}
> * The last logs repeats itself every few seconds.
> * I've set org.apache.ignite logging to DEBUG and recorded it but I'm not sure it's relevant. Let me know if you need it.
> * In any case, the GridDhtPartitionsSingleMessage message is sent to seed1 (uuid={{4a701be7-b730-4aca-a75c-0f9e97debb21}}) of the cluster.
> h4. seed1 does not process the partition exchange (probably because a transaction is pending)
> * Relevant stack trace:
> {noformat}
> "exchange-worker-#81%null%" #145 prio=5 os_prio=0 tid=0x00007fc343d95000 nid=0x4fa4 waiting on condition [0x00007fc2923e4000]
>    java.lang.Thread.State: TIMED_WAITING (parking)
> 	at sun.misc.Unsafe.park(Native Method)
> 	- parking to wait for  <0x00000007245e9fa0> (a org.apache.ignite.internal.util.future.GridCompoundFuture)
> 	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
> 	at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedNanos(AbstractQueuedSynchronizer.java:1037)
> 	at java.util.concurrent.locks.AbstractQueuedSynchronizer.tryAcquireSharedNanos(AbstractQueuedSynchronizer.java:1328)
> 	at org.apache.ignite.internal.util.future.GridFutureAdapter.get0(GridFutureAdapter.java:185)
> 	at org.apache.ignite.internal.util.future.GridFutureAdapter.get(GridFutureAdapter.java:135)
> 	at org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture.init(GridDhtPartitionsExchangeFuture.java:779)
> 	at org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$ExchangeWorker.body(GridCachePartitionExchangeManager.java:1297)
> 	at org.apache.ignite.internal.util.worker.GridWorker.run(GridWorker.java:110)
> 	at java.lang.Thread.run(Thread.java:745)
> {noformat}
> * The .get() is for:
> {code}
>         GridCompoundFuture f = new GridCompoundFuture();
>         f.add(mvcc().finishExplicitLocks(topVer));
>         f.add(tm().finishTxs(topVer));
>         f.add(mvcc().finishAtomicUpdates(topVer));
>         f.markInitialized();
> {code}
> * which probably blocks on the transaction that is pending.
> * The big issue is - why didn't w6:m4 process the transaction prepare request?
> * *A file with w6:m4's stack traces is attached*
> We will be happy to provide any information required;
> Regards,
> Noam and Avihai



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