You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ignite.apache.org by "Pavel Pereslegin (JIRA)" <ji...@apache.org> on 2018/03/26 16:37:00 UTC

[jira] [Created] (IGNITE-8051) Put operation may hang on LOCAL TRANSACTIONAL cache if it was stopped asynchronously.

Pavel Pereslegin created IGNITE-8051:
----------------------------------------

             Summary: Put operation may hang on LOCAL TRANSACTIONAL cache if it was stopped asynchronously.
                 Key: IGNITE-8051
                 URL: https://issues.apache.org/jira/browse/IGNITE-8051
             Project: Ignite
          Issue Type: Bug
          Components: cache
    Affects Versions: 2.4
            Reporter: Pavel Pereslegin


Put operation may hang if local cache was destroyed asynchronously. 
It seems that this happens because IgniteTxImplicitSingleStateImpl#topologyReadLock does not check cache status for local cache. 

Simple reproducer: 
{code:java}
public class DestroyLocalCacheTest extends GridCommonAbstractTest {
    /** */
    public void testDestroyAsync() throws Exception {
        try (Ignite node = startGrid()) {
            IgniteCache<Integer, Boolean> locCache = node.createCache(
                new CacheConfiguration<Integer, Boolean>(DEFAULT_CACHE_NAME).setCacheMode(CacheMode.LOCAL)
                    .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL));

            AtomicInteger cntr = new AtomicInteger();

            GridTestUtils.runMultiThreadedAsync(() -> {
                try {
                    int key;

                    while ((key = cntr.getAndIncrement()) < 10_000) {

                        if (key == 1000)
                            locCache.destroy();

                        locCache.put(key, true);
                    }
                }
                catch (Exception ignore) {
                    // No-op
                }

                return null;
            }, 5, "put-thread").get();
        }
    }
}
{code}

Log output:
{noformat}
[2018-03-26 19:10:11,350][INFO ][exchange-worker-#38%local.DestroyLocalCacheTest%][GridCacheProcessor] Started cache [name=default, id=1544803905, memoryPolicyName=default, mode=LOCAL, atomicity=TRANSACTIONAL, backups=0]
[2018-03-26 19:10:11,352][INFO ][exchange-worker-#38%local.DestroyLocalCacheTest%][GridDhtPartitionsExchangeFuture] Finished waiting for partition release future [topVer=AffinityTopologyVersion [topVer=1, minorTopVer=1], waitTime=0ms, futInfo=NA]
[2018-03-26 19:10:11,353][INFO ][exchange-worker-#38%local.DestroyLocalCacheTest%][GridDhtPartitionsExchangeFuture] finishExchangeOnCoordinator [topVer=AffinityTopologyVersion [topVer=1, minorTopVer=1], resVer=AffinityTopologyVersion [topVer=1, minorTopVer=1]]
[2018-03-26 19:10:11,354][INFO ][exchange-worker-#38%local.DestroyLocalCacheTest%][GridDhtPartitionsExchangeFuture] Finish exchange future [startVer=AffinityTopologyVersion [topVer=1, minorTopVer=1], resVer=AffinityTopologyVersion [topVer=1, minorTopVer=1], err=null]
[2018-03-26 19:10:11,354][INFO ][exchange-worker-#38%local.DestroyLocalCacheTest%][time] Finished exchange init [topVer=AffinityTopologyVersion [topVer=1, minorTopVer=1], crd=true]
[2018-03-26 19:10:11,355][INFO ][exchange-worker-#38%local.DestroyLocalCacheTest%][GridCachePartitionExchangeManager] Skipping rebalancing (nothing scheduled) [top=AffinityTopologyVersion [topVer=1, minorTopVer=1], evt=DISCOVERY_CUSTOM_EVT, node=fe867235-2fff-465a-b767-206abd291058]
[2018-03-26 19:10:11,590][INFO ][exchange-worker-#38%local.DestroyLocalCacheTest%][time] Started exchange init [topVer=AffinityTopologyVersion [topVer=1, minorTopVer=2], crd=true, evt=DISCOVERY_CUSTOM_EVT, evtNode=fe867235-2fff-465a-b767-206abd291058, customEvt=DynamicCacheChangeBatch [id=67313136261-d40acb94-363f-4c5e-b62b-1a95fbe73164, reqs=[DynamicCacheChangeRequest [cacheName=default, hasCfg=false, nodeId=fe867235-2fff-465a-b767-206abd291058, clientStartOnly=false, stop=true, destroy=false, disabledAfterStartfalse]], exchangeActions=ExchangeActions [startCaches=null, stopCaches=[default], startGrps=[], stopGrps=[default, destroy=true], resetParts=null, stateChangeRequest=null], startCaches=false], allowMerge=false]
[2018-03-26 19:10:11,590][INFO ][exchange-worker-#38%local.DestroyLocalCacheTest%][GridDhtPartitionsExchangeFuture] Finished waiting for partition release future [topVer=AffinityTopologyVersion [topVer=1, minorTopVer=2], waitTime=0ms, futInfo=NA]
[2018-03-26 19:10:11,591][INFO ][exchange-worker-#38%local.DestroyLocalCacheTest%][GridDhtPartitionsExchangeFuture] finishExchangeOnCoordinator [topVer=AffinityTopologyVersion [topVer=1, minorTopVer=2], resVer=AffinityTopologyVersion [topVer=1, minorTopVer=2]]
[2018-03-26 19:10:11,592][INFO ][exchange-worker-#38%local.DestroyLocalCacheTest%][GridDhtPartitionsExchangeFuture] Finish exchange future [startVer=AffinityTopologyVersion [topVer=1, minorTopVer=2], resVer=AffinityTopologyVersion [topVer=1, minorTopVer=2], err=null]
[2018-03-26 19:15:09,191][ERROR][main][root] Test has been timed out and will be interrupted (threads dump will be taken before interruption) [test=testDestroyAsync, timeout=300000]
[2018-03-26 19:15:09,191][WARN ][main][diagnostic] Dumping debug info for node [id=fe867235-2fff-465a-b767-206abd291058, name=local.DestroyLocalCacheTest, order=1, topVer=1, client=false]
[2018-03-26 19:15:09,191][WARN ][main][diagnostic] Ready affinity version: AffinityTopologyVersion [topVer=1, minorTopVer=1]
[2018-03-26 19:15:09,195][WARN ][main][diagnostic] Last exchange future: GridDhtPartitionsExchangeFuture [firstDiscoEvt=DiscoveryCustomEvent [customMsg=DynamicCacheChangeBatch [id=67313136261-d40acb94-363f-4c5e-b62b-1a95fbe73164, reqs=[DynamicCacheChangeRequest [cacheName=default, hasCfg=false, nodeId=fe867235-2fff-465a-b767-206abd291058, clientStartOnly=false, stop=true, destroy=false, disabledAfterStartfalse]], exchangeActions=ExchangeActions [startCaches=null, stopCaches=[default], startGrps=[], stopGrps=[default, destroy=true], resetParts=null, stateChangeRequest=null], startCaches=false], affTopVer=AffinityTopologyVersion [topVer=1, minorTopVer=2], super=DiscoveryEvent [evtNode=TcpDiscoveryNode [id=fe867235-2fff-465a-b767-206abd291058, addrs=[127.0.0.1], sockAddrs=[/127.0.0.1:47500], discPort=47500, order=1, intOrder=1, lastExchangeTime=1522080609915, loc=true, ver=2.5.0#19700101-sha1:00000000, isClient=false], topVer=1, nodeId8=fe867235, msg=null, type=DISCOVERY_CUSTOM_EVT, tstamp=1522080611585]], crd=TcpDiscoveryNode [id=fe867235-2fff-465a-b767-206abd291058, addrs=[127.0.0.1], sockAddrs=[/127.0.0.1:47500], discPort=47500, order=1, intOrder=1, lastExchangeTime=1522080609915, loc=true, ver=2.5.0#19700101-sha1:00000000, isClient=false], exchId=GridDhtPartitionExchangeId [topVer=AffinityTopologyVersion [topVer=1, minorTopVer=2], discoEvt=DiscoveryCustomEvent [customMsg=DynamicCacheChangeBatch [id=67313136261-d40acb94-363f-4c5e-b62b-1a95fbe73164, reqs=[DynamicCacheChangeRequest [cacheName=default, hasCfg=false, nodeId=fe867235-2fff-465a-b767-206abd291058, clientStartOnly=false, stop=true, destroy=false, disabledAfterStartfalse]], exchangeActions=ExchangeActions [startCaches=null, stopCaches=[default], startGrps=[], stopGrps=[default, destroy=true], resetParts=null, stateChangeRequest=null], startCaches=false], affTopVer=AffinityTopologyVersion [topVer=1, minorTopVer=2], super=DiscoveryEvent [evtNode=TcpDiscoveryNode [id=fe867235-2fff-465a-b767-206abd291058, addrs=[127.0.0.1], sockAddrs=[/127.0.0.1:47500], discPort=47500, order=1, intOrder=1, lastExchangeTime=1522080609915, loc=true, ver=2.5.0#19700101-sha1:00000000, isClient=false], topVer=1, nodeId8=fe867235, msg=null, type=DISCOVERY_CUSTOM_EVT, tstamp=1522080611585]], nodeId=fe867235, evt=DISCOVERY_CUSTOM_EVT], added=true, initFut=GridFutureAdapter [ignoreInterrupts=false, state=INIT, res=null, hash=13648335], init=false, lastVer=GridCacheVersion [topVer=133560612, order=1522080613658, nodeOrder=1], partReleaseFut=PartitionReleaseFuture [topVer=AffinityTopologyVersion [topVer=1, minorTopVer=2], futures=[ExplicitLockReleaseFuture [topVer=AffinityTopologyVersion [topVer=1, minorTopVer=2], futures=[]], TxReleaseFuture [topVer=AffinityTopologyVersion [topVer=1, minorTopVer=2], futures=[]], AtomicUpdateReleaseFuture [topVer=AffinityTopologyVersion [topVer=1, minorTopVer=2], futures=[]], DataStreamerReleaseFuture [topVer=AffinityTopologyVersion [topVer=1, minorTopVer=2], futures=[]]]], exchActions=ExchangeActions [startCaches=null, stopCaches=[default], startGrps=[], stopGrps=[default, destroy=true], resetParts=null, stateChangeRequest=null], affChangeMsg=null, initTs=1522080611585, centralizedAff=false, forceAffReassignment=false, changeGlobalStateE=null, done=true, state=DONE, evtLatch=0, remaining=[], super=GridFutureAdapter [ignoreInterrupts=false, state=INIT, res=null, hash=747464370]]
[2018-03-26 19:15:09,196][WARN ][main][GridCachePartitionExchangeManager] First 10 pending exchange futures [total=0]
[2018-03-26 19:15:09,197][WARN ][main][diagnostic] Last 10 exchange futures (total: 3):
[2018-03-26 19:15:09,197][WARN ][main][diagnostic] >>> GridDhtPartitionsExchangeFuture [topVer=AffinityTopologyVersion [topVer=1, minorTopVer=2], evt=DISCOVERY_CUSTOM_EVT, evtNode=TcpDiscoveryNode [id=fe867235-2fff-465a-b767-206abd291058, addrs=[127.0.0.1], sockAddrs=[/127.0.0.1:47500], discPort=47500, order=1, intOrder=1, lastExchangeTime=1522080609915, loc=true, ver=2.5.0#19700101-sha1:00000000, isClient=false], done=false]
[2018-03-26 19:15:09,198][WARN ][main][diagnostic] >>> GridDhtPartitionsExchangeFuture [topVer=AffinityTopologyVersion [topVer=1, minorTopVer=1], evt=DISCOVERY_CUSTOM_EVT, evtNode=TcpDiscoveryNode [id=fe867235-2fff-465a-b767-206abd291058, addrs=[127.0.0.1], sockAddrs=[/127.0.0.1:47500], discPort=47500, order=1, intOrder=1, lastExchangeTime=1522080609915, loc=true, ver=2.5.0#19700101-sha1:00000000, isClient=false], done=true]
[2018-03-26 19:15:09,198][WARN ][main][diagnostic] >>> GridDhtPartitionsExchangeFuture [topVer=AffinityTopologyVersion [topVer=1, minorTopVer=0], evt=NODE_JOINED, evtNode=TcpDiscoveryNode [id=fe867235-2fff-465a-b767-206abd291058, addrs=[127.0.0.1], sockAddrs=[/127.0.0.1:47500], discPort=47500, order=1, intOrder=1, lastExchangeTime=1522080609915, loc=true, ver=2.5.0#19700101-sha1:00000000, isClient=false], done=true]
[2018-03-26 19:15:09,198][WARN ][main][diagnostic] Pending transactions:
[2018-03-26 19:15:09,200][WARN ][main][diagnostic] >>> [txVer=null, tx=GridNearTxLocal [mappings=IgniteTxMappingsSingleImpl [mapping=null], nearLocallyMapped=false, colocatedLocallyMapped=false, needCheckBackup=null, hasRemoteLocks=false, trackTimeout=false, thread=put-thread-1, mappings=IgniteTxMappingsSingleImpl [mapping=null], super=GridDhtTxLocalAdapter [nearOnOriginatingNode=false, nearNodes=[], dhtNodes=[], explicitLock=false, super=IgniteTxLocalAdapter [completedBase=null, sndTransformedVals=false, depEnabled=false, txState=IgniteTxImplicitSingleStateImpl [init=true, recovery=false], super=IgniteTxAdapter [xidVer=GridCacheVersion [topVer=133560612, order=1522080613657, nodeOrder=1], writeVer=null, implicit=true, loc=true, threadId=62, startTime=1522080611585, nodeId=fe867235-2fff-465a-b767-206abd291058, startVer=GridCacheVersion [topVer=133560612, order=1522080613657, nodeOrder=1], endVer=null, isolation=READ_COMMITTED, concurrency=OPTIMISTIC, timeout=0, sysInvalidate=false, sys=false, plc=2, commitVer=null, finalizing=NONE, invalidParts=null, state=ACTIVE, timedOut=false, topVer=AffinityTopologyVersion [topVer=-1, minorTopVer=0], duration=297604ms, onePhaseCommit=false], size=1]]]]
[2018-03-26 19:15:09,200][WARN ][main][diagnostic] >>> [txVer=null, tx=GridNearTxLocal [mappings=IgniteTxMappingsSingleImpl [mapping=null], nearLocallyMapped=false, colocatedLocallyMapped=false, needCheckBackup=null, hasRemoteLocks=false, trackTimeout=false, thread=put-thread-2, mappings=IgniteTxMappingsSingleImpl [mapping=null], super=GridDhtTxLocalAdapter [nearOnOriginatingNode=false, nearNodes=[], dhtNodes=[], explicitLock=false, super=IgniteTxLocalAdapter [completedBase=null, sndTransformedVals=false, depEnabled=false, txState=IgniteTxImplicitSingleStateImpl [init=true, recovery=false], super=IgniteTxAdapter [xidVer=GridCacheVersion [topVer=133560612, order=1522080613653, nodeOrder=1], writeVer=null, implicit=true, loc=true, threadId=63, startTime=1522080611585, nodeId=fe867235-2fff-465a-b767-206abd291058, startVer=GridCacheVersion [topVer=133560612, order=1522080613653, nodeOrder=1], endVer=null, isolation=READ_COMMITTED, concurrency=OPTIMISTIC, timeout=0, sysInvalidate=false, sys=false, plc=2, commitVer=null, finalizing=NONE, invalidParts=null, state=ACTIVE, timedOut=false, topVer=AffinityTopologyVersion [topVer=-1, minorTopVer=0], duration=297614ms, onePhaseCommit=false], size=1]]]]
[2018-03-26 19:15:09,201][WARN ][main][diagnostic] >>> [txVer=null, tx=GridNearTxLocal [mappings=IgniteTxMappingsSingleImpl [mapping=null], nearLocallyMapped=false, colocatedLocallyMapped=false, needCheckBackup=null, hasRemoteLocks=false, trackTimeout=false, thread=put-thread-5, mappings=IgniteTxMappingsSingleImpl [mapping=null], super=GridDhtTxLocalAdapter [nearOnOriginatingNode=false, nearNodes=[], dhtNodes=[], explicitLock=false, super=IgniteTxLocalAdapter [completedBase=null, sndTransformedVals=false, depEnabled=false, txState=IgniteTxImplicitSingleStateImpl [init=true, recovery=false], super=IgniteTxAdapter [xidVer=GridCacheVersion [topVer=133560612, order=1522080613652, nodeOrder=1], writeVer=null, implicit=true, loc=true, threadId=66, startTime=1522080611585, nodeId=fe867235-2fff-465a-b767-206abd291058, startVer=GridCacheVersion [topVer=133560612, order=1522080613652, nodeOrder=1], endVer=null, isolation=READ_COMMITTED, concurrency=OPTIMISTIC, timeout=0, sysInvalidate=false, sys=false, plc=2, commitVer=null, finalizing=NONE, invalidParts=null, state=ACTIVE, timedOut=false, topVer=AffinityTopologyVersion [topVer=-1, minorTopVer=0], duration=297614ms, onePhaseCommit=false], size=1]]]]
[2018-03-26 19:15:09,201][WARN ][main][diagnostic] >>> [txVer=null, tx=GridNearTxLocal [mappings=IgniteTxMappingsSingleImpl [mapping=null], nearLocallyMapped=false, colocatedLocallyMapped=false, needCheckBackup=null, hasRemoteLocks=false, trackTimeout=false, thread=put-thread-4, mappings=IgniteTxMappingsSingleImpl [mapping=null], super=GridDhtTxLocalAdapter [nearOnOriginatingNode=false, nearNodes=[], dhtNodes=[], explicitLock=false, super=IgniteTxLocalAdapter [completedBase=null, sndTransformedVals=false, depEnabled=false, txState=IgniteTxImplicitSingleStateImpl [init=true, recovery=false], super=IgniteTxAdapter [xidVer=GridCacheVersion [topVer=133560612, order=1522080613645, nodeOrder=1], writeVer=null, implicit=true, loc=true, threadId=65, startTime=1522080611585, nodeId=fe867235-2fff-465a-b767-206abd291058, startVer=GridCacheVersion [topVer=133560612, order=1522080613645, nodeOrder=1], endVer=null, isolation=READ_COMMITTED, concurrency=OPTIMISTIC, timeout=0, sysInvalidate=false, sys=false, plc=2, commitVer=null, finalizing=NONE, invalidParts=null, state=ACTIVE, timedOut=false, topVer=AffinityTopologyVersion [topVer=-1, minorTopVer=0], duration=297614ms, onePhaseCommit=false], size=1]]]]
[2018-03-26 19:15:09,201][WARN ][main][diagnostic] Pending explicit locks:
[2018-03-26 19:15:09,201][WARN ][main][diagnostic] Pending cache futures:
[2018-03-26 19:15:09,203][WARN ][main][diagnostic] >>> GridNearTxFinishFuture [futId=8c313136261-d40acb94-363f-4c5e-b62b-1a95fbe73164, tx=GridNearTxLocal [mappings=IgniteTxMappingsSingleImpl [mapping=null], nearLocallyMapped=false, colocatedLocallyMapped=false, needCheckBackup=null, hasRemoteLocks=false, trackTimeout=false, thread=put-thread-5, mappings=IgniteTxMappingsSingleImpl [mapping=null], super=GridDhtTxLocalAdapter [nearOnOriginatingNode=false, nearNodes=[], dhtNodes=[], explicitLock=false, super=IgniteTxLocalAdapter [completedBase=null, sndTransformedVals=false, depEnabled=false, txState=IgniteTxImplicitSingleStateImpl [init=true, recovery=false], super=IgniteTxAdapter [xidVer=GridCacheVersion [topVer=133560612, order=1522080613652, nodeOrder=1], writeVer=null, implicit=true, loc=true, threadId=66, startTime=1522080611585, nodeId=fe867235-2fff-465a-b767-206abd291058, startVer=GridCacheVersion [topVer=133560612, order=1522080613652, nodeOrder=1], endVer=null, isolation=READ_COMMITTED, concurrency=OPTIMISTIC, timeout=0, sysInvalidate=false, sys=false, plc=2, commitVer=null, finalizing=NONE, invalidParts=null, state=ACTIVE, timedOut=false, topVer=AffinityTopologyVersion [topVer=-1, minorTopVer=0], duration=297614ms, onePhaseCommit=false], size=1]]], commit=true, mappings=IgniteTxMappingsSingleImpl [mapping=null], trackable=true, finishOnePhaseCalled=false, innerFuts=[], super=GridCompoundIdentityFuture [super=GridCompoundFuture [rdc=AlwaysTrueReducer [], initFlag=0, lsnrCalls=0, done=false, cancelled=false, err=null, futs=[]]]]
[2018-03-26 19:15:09,203][WARN ][main][diagnostic] >>> GridNearTxFinishFuture [futId=fc313136261-d40acb94-363f-4c5e-b62b-1a95fbe73164, tx=GridNearTxLocal [mappings=IgniteTxMappingsSingleImpl [mapping=null], nearLocallyMapped=false, colocatedLocallyMapped=false, needCheckBackup=null, hasRemoteLocks=false, trackTimeout=false, thread=put-thread-1, mappings=IgniteTxMappingsSingleImpl [mapping=null], super=GridDhtTxLocalAdapter [nearOnOriginatingNode=false, nearNodes=[], dhtNodes=[], explicitLock=false, super=IgniteTxLocalAdapter [completedBase=null, sndTransformedVals=false, depEnabled=false, txState=IgniteTxImplicitSingleStateImpl [init=true, recovery=false], super=IgniteTxAdapter [xidVer=GridCacheVersion [topVer=133560612, order=1522080613657, nodeOrder=1], writeVer=null, implicit=true, loc=true, threadId=62, startTime=1522080611585, nodeId=fe867235-2fff-465a-b767-206abd291058, startVer=GridCacheVersion [topVer=133560612, order=1522080613657, nodeOrder=1], endVer=null, isolation=READ_COMMITTED, concurrency=OPTIMISTIC, timeout=0, sysInvalidate=false, sys=false, plc=2, commitVer=null, finalizing=NONE, invalidParts=null, state=ACTIVE, timedOut=false, topVer=AffinityTopologyVersion [topVer=-1, minorTopVer=0], duration=297614ms, onePhaseCommit=false], size=1]]], commit=true, mappings=IgniteTxMappingsSingleImpl [mapping=null], trackable=true, finishOnePhaseCalled=false, innerFuts=[], super=GridCompoundIdentityFuture [super=GridCompoundFuture [rdc=AlwaysTrueReducer [], initFlag=0, lsnrCalls=0, done=false, cancelled=false, err=null, futs=[]]]]
[2018-03-26 19:15:09,204][WARN ][main][diagnostic] >>> GridNearTxFinishFuture [futId=cb313136261-d40acb94-363f-4c5e-b62b-1a95fbe73164, tx=GridNearTxLocal [mappings=IgniteTxMappingsSingleImpl [mapping=null], nearLocallyMapped=false, colocatedLocallyMapped=false, needCheckBackup=null, hasRemoteLocks=false, trackTimeout=false, thread=put-thread-4, mappings=IgniteTxMappingsSingleImpl [mapping=null], super=GridDhtTxLocalAdapter [nearOnOriginatingNode=false, nearNodes=[], dhtNodes=[], explicitLock=false, super=IgniteTxLocalAdapter [completedBase=null, sndTransformedVals=false, depEnabled=false, txState=IgniteTxImplicitSingleStateImpl [init=true, recovery=false], super=IgniteTxAdapter [xidVer=GridCacheVersion [topVer=133560612, order=1522080613645, nodeOrder=1], writeVer=null, implicit=true, loc=true, threadId=65, startTime=1522080611585, nodeId=fe867235-2fff-465a-b767-206abd291058, startVer=GridCacheVersion [topVer=133560612, order=1522080613645, nodeOrder=1], endVer=null, isolation=READ_COMMITTED, concurrency=OPTIMISTIC, timeout=0, sysInvalidate=false, sys=false, plc=2, commitVer=null, finalizing=NONE, invalidParts=null, state=ACTIVE, timedOut=false, topVer=AffinityTopologyVersion [topVer=-1, minorTopVer=0], duration=297614ms, onePhaseCommit=false], size=1]]], commit=true, mappings=IgniteTxMappingsSingleImpl [mapping=null], trackable=true, finishOnePhaseCalled=false, innerFuts=[], super=GridCompoundIdentityFuture [super=GridCompoundFuture [rdc=AlwaysTrueReducer [], initFlag=0, lsnrCalls=0, done=false, cancelled=false, err=null, futs=[]]]]
[2018-03-26 19:15:09,204][WARN ][main][diagnostic] >>> GridNearTxFinishFuture [futId=cc313136261-d40acb94-363f-4c5e-b62b-1a95fbe73164, tx=GridNearTxLocal [mappings=IgniteTxMappingsSingleImpl [mapping=null], nearLocallyMapped=false, colocatedLocallyMapped=false, needCheckBackup=null, hasRemoteLocks=false, trackTimeout=false, thread=put-thread-2, mappings=IgniteTxMappingsSingleImpl [mapping=null], super=GridDhtTxLocalAdapter [nearOnOriginatingNode=false, nearNodes=[], dhtNodes=[], explicitLock=false, super=IgniteTxLocalAdapter [completedBase=null, sndTransformedVals=false, depEnabled=false, txState=IgniteTxImplicitSingleStateImpl [init=true, recovery=false], super=IgniteTxAdapter [xidVer=GridCacheVersion [topVer=133560612, order=1522080613653, nodeOrder=1], writeVer=null, implicit=true, loc=true, threadId=63, startTime=1522080611585, nodeId=fe867235-2fff-465a-b767-206abd291058, startVer=GridCacheVersion [topVer=133560612, order=1522080613653, nodeOrder=1], endVer=null, isolation=READ_COMMITTED, concurrency=OPTIMISTIC, timeout=0, sysInvalidate=false, sys=false, plc=2, commitVer=null, finalizing=NONE, invalidParts=null, state=ACTIVE, timedOut=false, topVer=AffinityTopologyVersion [topVer=-1, minorTopVer=0], duration=297614ms, onePhaseCommit=false], size=1]]], commit=true, mappings=IgniteTxMappingsSingleImpl [mapping=null], trackable=true, finishOnePhaseCalled=false, innerFuts=[], super=GridCompoundIdentityFuture [super=GridCompoundFuture [rdc=AlwaysTrueReducer [], initFlag=0, lsnrCalls=0, done=false, cancelled=false, err=null, futs=[]]]]
[2018-03-26 19:15:09,204][WARN ][main][diagnostic] Pending atomic cache futures:
[2018-03-26 19:15:09,204][WARN ][main][diagnostic] Pending data streamer futures:
[2018-03-26 19:15:09,205][WARN ][main][diagnostic] Pending transaction deadlock detection futures:
[19:15:09] (wrn) No deadlocked threads detected.

[19:15:09] (wrn) Thread dump at 2018/03/26 19:15:09 MSK[2018-03-26 19:15:09,226][WARN ][main][root] No deadlocked threads detected.
Thread [name="sys-#69%local.DestroyLocalCacheTest%", id=95, state=TIMED_WAITING, blockCnt=0, waitCnt=1]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@e580929, 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-#68%local.DestroyLocalCacheTest%", id=94, state=TIMED_WAITING, blockCnt=0, waitCnt=1]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@e580929, 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-#67%local.DestroyLocalCacheTest%", id=93, state=TIMED_WAITING, blockCnt=0, waitCnt=1]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@e580929, 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-#66%local.DestroyLocalCacheTest%", id=92, state=TIMED_WAITING, blockCnt=0, waitCnt=1]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@e580929, 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-#65%local.DestroyLocalCacheTest%", id=91, state=TIMED_WAITING, blockCnt=0, waitCnt=1]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@e580929, 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-#64%local.DestroyLocalCacheTest%", id=90, state=TIMED_WAITING, blockCnt=0, waitCnt=1]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@e580929, 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="put-thread-5", id=66, state=WAITING, blockCnt=138, waitCnt=61]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.future.GridFutureAdapter.get0(GridFutureAdapter.java:177)
        at o.a.i.i.util.future.GridFutureAdapter.get(GridFutureAdapter.java:140)
        at o.a.i.i.processors.cache.GridCacheAdapter$22.op(GridCacheAdapter.java:2390)
        at o.a.i.i.processors.cache.GridCacheAdapter$22.op(GridCacheAdapter.java:2388)
        at o.a.i.i.processors.cache.GridCacheAdapter.syncOp(GridCacheAdapter.java:4088)
        at o.a.i.i.processors.cache.GridCacheAdapter.put0(GridCacheAdapter.java:2388)
        at o.a.i.i.processors.cache.GridCacheAdapter.put(GridCacheAdapter.java:2369)
        at o.a.i.i.processors.cache.GridCacheAdapter.put(GridCacheAdapter.java:2346)
        at o.a.i.i.processors.cache.IgniteCacheProxyImpl.put(IgniteCacheProxyImpl.java:1084)
        at o.a.i.i.processors.cache.GatewayProtectedCacheProxy.put(GatewayProtectedCacheProxy.java:886)
        at o.a.i.i.processors.cache.local.DestroyLocalCacheTest.lambda$testDestroyAsync$0(DestroyLocalCacheTest.java:32)
        at o.a.i.i.processors.cache.local.DestroyLocalCacheTest$$Lambda$7/759487471.call(Unknown Source)
        at o.a.i.testframework.GridTestThread.run(GridTestThread.java:86)

Thread [name="put-thread-4", id=65, state=WAITING, blockCnt=109, waitCnt=43]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.future.GridFutureAdapter.get0(GridFutureAdapter.java:177)
        at o.a.i.i.util.future.GridFutureAdapter.get(GridFutureAdapter.java:140)
        at o.a.i.i.processors.cache.GridCacheAdapter$22.op(GridCacheAdapter.java:2390)
        at o.a.i.i.processors.cache.GridCacheAdapter$22.op(GridCacheAdapter.java:2388)
        at o.a.i.i.processors.cache.GridCacheAdapter.syncOp(GridCacheAdapter.java:4088)
        at o.a.i.i.processors.cache.GridCacheAdapter.put0(GridCacheAdapter.java:2388)
        at o.a.i.i.processors.cache.GridCacheAdapter.put(GridCacheAdapter.java:2369)
        at o.a.i.i.processors.cache.GridCacheAdapter.put(GridCacheAdapter.java:2346)
        at o.a.i.i.processors.cache.IgniteCacheProxyImpl.put(IgniteCacheProxyImpl.java:1084)
        at o.a.i.i.processors.cache.GatewayProtectedCacheProxy.put(GatewayProtectedCacheProxy.java:886)
        at o.a.i.i.processors.cache.local.DestroyLocalCacheTest.lambda$testDestroyAsync$0(DestroyLocalCacheTest.java:32)
        at o.a.i.i.processors.cache.local.DestroyLocalCacheTest$$Lambda$7/759487471.call(Unknown Source)
        at o.a.i.testframework.GridTestThread.run(GridTestThread.java:86)

Thread [name="put-thread-3", id=64, state=WAITING, blockCnt=93, waitCnt=56]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.future.GridFutureAdapter.get0(GridFutureAdapter.java:177)
        at o.a.i.i.util.future.GridFutureAdapter.get(GridFutureAdapter.java:140)
        at o.a.i.i.util.future.IgniteFutureImpl.get(IgniteFutureImpl.java:134)
        at o.a.i.i.processors.cache.GatewayProtectedCacheProxy.destroy(GatewayProtectedCacheProxy.java:1503)
        at o.a.i.i.processors.cache.local.DestroyLocalCacheTest.lambda$testDestroyAsync$0(DestroyLocalCacheTest.java:30)
        at o.a.i.i.processors.cache.local.DestroyLocalCacheTest$$Lambda$7/759487471.call(Unknown Source)
        at o.a.i.testframework.GridTestThread.run(GridTestThread.java:86)

Thread [name="put-thread-2", id=63, state=WAITING, blockCnt=132, waitCnt=57]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.future.GridFutureAdapter.get0(GridFutureAdapter.java:177)
        at o.a.i.i.util.future.GridFutureAdapter.get(GridFutureAdapter.java:140)
        at o.a.i.i.processors.cache.GridCacheAdapter$22.op(GridCacheAdapter.java:2390)
        at o.a.i.i.processors.cache.GridCacheAdapter$22.op(GridCacheAdapter.java:2388)
        at o.a.i.i.processors.cache.GridCacheAdapter.syncOp(GridCacheAdapter.java:4088)
        at o.a.i.i.processors.cache.GridCacheAdapter.put0(GridCacheAdapter.java:2388)
        at o.a.i.i.processors.cache.GridCacheAdapter.put(GridCacheAdapter.java:2369)
        at o.a.i.i.processors.cache.GridCacheAdapter.put(GridCacheAdapter.java:2346)
        at o.a.i.i.processors.cache.IgniteCacheProxyImpl.put(IgniteCacheProxyImpl.java:1084)
        at o.a.i.i.processors.cache.GatewayProtectedCacheProxy.put(GatewayProtectedCacheProxy.java:886)
        at o.a.i.i.processors.cache.local.DestroyLocalCacheTest.lambda$testDestroyAsync$0(DestroyLocalCacheTest.java:32)
        at o.a.i.i.processors.cache.local.DestroyLocalCacheTest$$Lambda$7/759487471.call(Unknown Source)
        at o.a.i.testframework.GridTestThread.run(GridTestThread.java:86)

Thread [name="put-thread-1", id=62, state=WAITING, blockCnt=135, waitCnt=76]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.future.GridFutureAdapter.get0(GridFutureAdapter.java:177)
        at o.a.i.i.util.future.GridFutureAdapter.get(GridFutureAdapter.java:140)
        at o.a.i.i.processors.cache.GridCacheAdapter$22.op(GridCacheAdapter.java:2390)
        at o.a.i.i.processors.cache.GridCacheAdapter$22.op(GridCacheAdapter.java:2388)
        at o.a.i.i.processors.cache.GridCacheAdapter.syncOp(GridCacheAdapter.java:4088)
        at o.a.i.i.processors.cache.GridCacheAdapter.put0(GridCacheAdapter.java:2388)
        at o.a.i.i.processors.cache.GridCacheAdapter.put(GridCacheAdapter.java:2369)
        at o.a.i.i.processors.cache.GridCacheAdapter.put(GridCacheAdapter.java:2346)
        at o.a.i.i.processors.cache.IgniteCacheProxyImpl.put(IgniteCacheProxyImpl.java:1084)
        at o.a.i.i.processors.cache.GatewayProtectedCacheProxy.put(GatewayProtectedCacheProxy.java:886)
        at o.a.i.i.processors.cache.local.DestroyLocalCacheTest.lambda$testDestroyAsync$0(DestroyLocalCacheTest.java:32)
        at o.a.i.i.processors.cache.local.DestroyLocalCacheTest$$Lambda$7/759487471.call(Unknown Source)
        at o.a.i.testframework.GridTestThread.run(GridTestThread.java:86)

Thread [name="async-callable-runner-1", id=61, state=WAITING, blockCnt=0, waitCnt=1]
    Lock [object=o.a.i.testframework.GridTestSafeThreadFactory$1@1cd072a9, ownerName=null, ownerId=-1]
        at java.lang.Object.wait(Native Method)
        at java.lang.Thread.join(Thread.java:1252)
        at java.lang.Thread.join(Thread.java:1326)
        at o.a.i.testframework.GridTestUtils.runMultiThreaded(GridTestUtils.java:871)
        at o.a.i.testframework.GridTestUtils$4.call(GridTestUtils.java:811)
        at o.a.i.testframework.GridTestUtils$4.call(GridTestUtils.java:809)
        at o.a.i.testframework.GridTestUtils$7.run(GridTestUtils.java:979)
        at o.a.i.testframework.GridTestUtils$9.call(GridTestUtils.java:1275)
        at o.a.i.testframework.GridTestThread.run(GridTestThread.java:86)

Thread [name="ttl-cleanup-worker-#40%local.DestroyLocalCacheTest%", id=60, state=TIMED_WAITING, blockCnt=0, waitCnt=596]
        at java.lang.Thread.sleep(Native Method)
        at o.a.i.i.util.IgniteUtils.sleep(IgniteUtils.java:7598)
        at o.a.i.i.processors.cache.GridCacheSharedTtlCleanupManager$CleanupWorker.body(GridCacheSharedTtlCleanupManager.java:137)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:110)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="srvc-deploy-#39%local.DestroyLocalCacheTest%", id=58, state=WAITING, blockCnt=0, waitCnt=3]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@7c75222b, 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="exchange-worker-#38%local.DestroyLocalCacheTest%", id=57, state=TIMED_WAITING, blockCnt=0, waitCnt=1489]
    Lock [object=java.util.concurrent.locks.ReentrantReadWriteLock$NonfairSync@48140564, 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.doAcquireNanos(AbstractQueuedSynchronizer.java:934)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer.tryAcquireNanos(AbstractQueuedSynchronizer.java:1247)
        at java.util.concurrent.locks.ReentrantReadWriteLock$WriteLock.tryLock(ReentrantReadWriteLock.java:1115)
        at o.a.i.i.util.StripedCompositeReadWriteLock$WriteLock.tryLock(StripedCompositeReadWriteLock.java:201)
        at o.a.i.i.processors.cache.GridCacheGateway.onStopped(GridCacheGateway.java:305)
        at o.a.i.i.processors.cache.GridCacheProcessor.stopGateway(GridCacheProcessor.java:2060)
        at o.a.i.i.processors.cache.GridCacheProcessor.onExchangeDone(GridCacheProcessor.java:2229)
        at o.a.i.i.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture.onDone(GridDhtPartitionsExchangeFuture.java:1612)
        at o.a.i.i.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture.finishExchangeOnCoordinator(GridDhtPartitionsExchangeFuture.java:2610)
        at o.a.i.i.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture.onAllReceived(GridDhtPartitionsExchangeFuture.java:2408)
        at o.a.i.i.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture.distributedExchange(GridDhtPartitionsExchangeFuture.java:1135)
        at o.a.i.i.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture.init(GridDhtPartitionsExchangeFuture.java:704)
        at o.a.i.i.processors.cache.GridCachePartitionExchangeManager$ExchangeWorker.body(GridCachePartitionExchangeManager.java:2344)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:110)
        at java.lang.Thread.run(Thread.java:748)

    Locked synchronizers:
        java.util.concurrent.locks.ReentrantReadWriteLock$NonfairSync@4c203ea1
        java.util.concurrent.locks.ReentrantReadWriteLock$NonfairSync@27f674d
        java.util.concurrent.locks.ReentrantReadWriteLock$NonfairSync@1d251891
Thread [name="disco-event-worker-#37%local.DestroyLocalCacheTest%", id=56, state=WAITING, blockCnt=0, waitCnt=299]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@58ceff1, 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.managers.discovery.GridDiscoveryManager$DiscoveryWorker.body0(GridDiscoveryManager.java:2598)
        at o.a.i.i.managers.discovery.GridDiscoveryManager$DiscoveryWorker.body(GridDiscoveryManager.java:2580)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:110)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="tcp-disco-ip-finder-cleaner-#6%local.DestroyLocalCacheTest%", id=55, state=TIMED_WAITING, blockCnt=0, waitCnt=9]
        at java.lang.Thread.sleep(Native Method)
        at o.a.i.spi.discovery.tcp.ServerImpl$IpFinderCleaner.body(ServerImpl.java:1883)
        at o.a.i.spi.IgniteSpiThread.run(IgniteSpiThread.java:62)

Thread [name="tcp-disco-multicast-addr-sender-#5%local.DestroyLocalCacheTest%", id=54, state=RUNNABLE, blockCnt=0, waitCnt=0]
        at java.net.PlainDatagramSocketImpl.receive0(Native Method)
        - locked java.net.PlainDatagramSocketImpl@7c30a502
        at java.net.AbstractPlainDatagramSocketImpl.receive(AbstractPlainDatagramSocketImpl.java:143)
        - locked java.net.PlainDatagramSocketImpl@7c30a502
        at java.net.DatagramSocket.receive(DatagramSocket.java:812)
        - locked java.net.DatagramPacket@49e4cb85
        - locked java.net.MulticastSocket@2133c8f8
        at o.a.i.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder$AddressSender.body(TcpDiscoveryMulticastIpFinder.java:882)
        at o.a.i.spi.IgniteSpiThread.run(IgniteSpiThread.java:62)

Thread [name="tcp-disco-srvr-#4%local.DestroyLocalCacheTest%", id=53, state=RUNNABLE, blockCnt=0, waitCnt=0]
        at java.net.PlainSocketImpl.socketAccept(Native Method)
        at java.net.AbstractPlainSocketImpl.accept(AbstractPlainSocketImpl.java:409)
        at java.net.ServerSocket.implAccept(ServerSocket.java:545)
        at java.net.ServerSocket.accept(ServerSocket.java:513)
        at o.a.i.spi.discovery.tcp.ServerImpl$TcpServer.body(ServerImpl.java:5590)
        at o.a.i.spi.IgniteSpiThread.run(IgniteSpiThread.java:62)

Thread [name="tcp-disco-msg-worker-#3%local.DestroyLocalCacheTest%", id=52, state=TIMED_WAITING, blockCnt=4, waitCnt=29402]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@43a25848, 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.LinkedBlockingDeque.pollFirst(LinkedBlockingDeque.java:522)
        at java.util.concurrent.LinkedBlockingDeque.poll(LinkedBlockingDeque.java:684)
        at o.a.i.spi.discovery.tcp.ServerImpl$MessageWorkerAdapter.body(ServerImpl.java:6702)
        at o.a.i.spi.discovery.tcp.ServerImpl$RingMessageWorker.body(ServerImpl.java:2602)
        at o.a.i.spi.IgniteSpiThread.run(IgniteSpiThread.java:62)

Thread [name="grid-data-loader-flusher-#36%local.DestroyLocalCacheTest%", id=51, state=WAITING, blockCnt=0, waitCnt=1]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@3ac3fd8b, 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.DelayQueue.take(DelayQueue.java:211)
        at o.a.i.i.processors.datastreamer.DataStreamProcessor$2.body(DataStreamProcessor.java:108)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:110)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="grid-nio-worker-client-listener-3-#34%local.DestroyLocalCacheTest%", id=49, state=RUNNABLE, blockCnt=0, waitCnt=0]
        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
        at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
        at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:93)
        at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
        - locked o.a.i.i.util.nio.SelectedSelectionKeySet@5594a1b5
        - locked java.util.Collections$UnmodifiableSet@6a5fc7f7
        - locked sun.nio.ch.EPollSelectorImpl@3b6eb2ec
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
        at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.bodyInternal(GridNioServer.java:2075)
        at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.body(GridNioServer.java:1749)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:110)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="grid-nio-worker-client-listener-2-#33%local.DestroyLocalCacheTest%", id=48, state=RUNNABLE, blockCnt=0, waitCnt=0]
        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
        at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
        at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:93)
        at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
        - locked o.a.i.i.util.nio.SelectedSelectionKeySet@1e643faf
        - locked java.util.Collections$UnmodifiableSet@6e8dacdf
        - locked sun.nio.ch.EPollSelectorImpl@7a79be86
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
        at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.bodyInternal(GridNioServer.java:2075)
        at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.body(GridNioServer.java:1749)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:110)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="grid-nio-worker-client-listener-1-#32%local.DestroyLocalCacheTest%", id=47, state=RUNNABLE, blockCnt=0, waitCnt=0]
        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
        at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
        at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:93)
        at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
        - locked o.a.i.i.util.nio.SelectedSelectionKeySet@34ce8af7
        - locked java.util.Collections$UnmodifiableSet@b684286
        - locked sun.nio.ch.EPollSelectorImpl@880ec60
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
        at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.bodyInternal(GridNioServer.java:2075)
        at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.body(GridNioServer.java:1749)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:110)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="grid-nio-worker-client-listener-0-#31%local.DestroyLocalCacheTest%", id=46, state=RUNNABLE, blockCnt=0, waitCnt=0]
        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
        at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
        at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:93)
        at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
        - locked o.a.i.i.util.nio.SelectedSelectionKeySet@3f3afe78
        - locked java.util.Collections$UnmodifiableSet@7f63425a
        - locked sun.nio.ch.EPollSelectorImpl@36d64342
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
        at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.bodyInternal(GridNioServer.java:2075)
        at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.body(GridNioServer.java:1749)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:110)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="nio-acceptor-#30%local.DestroyLocalCacheTest%", id=45, state=RUNNABLE, blockCnt=0, waitCnt=0]
        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
        at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
        at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:93)
        at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
        - locked sun.nio.ch.Util$3@39ba5a14
        - locked java.util.Collections$UnmodifiableSet@511baa65
        - locked sun.nio.ch.EPollSelectorImpl@340f438e
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
        at o.a.i.i.util.nio.GridNioServer$GridNioAcceptWorker.accept(GridNioServer.java:2813)
        at o.a.i.i.util.nio.GridNioServer$GridNioAcceptWorker.body(GridNioServer.java:2785)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:110)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="grid-sharedfs-timeout-worker-#2%local.DestroyLocalCacheTest%", id=44, state=TIMED_WAITING, blockCnt=0, waitCnt=60]
    Lock [object=java.lang.Object@30c7da1e, ownerName=null, ownerId=-1]
        at java.lang.Object.wait(Native Method)
        at o.a.i.spi.checkpoint.sharedfs.SharedFsTimeoutTask.body(SharedFsTimeoutTask.java:99)
        at o.a.i.spi.IgniteSpiThread.run(IgniteSpiThread.java:62)

Thread [name="tcp-comm-worker-#1%local.DestroyLocalCacheTest%", id=43, state=TIMED_WAITING, blockCnt=0, waitCnt=1]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@5b464ce8, 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 o.a.i.spi.communication.tcp.TcpCommunicationSpi$CommunicationWorker.body(TcpCommunicationSpi.java:4093)
        at o.a.i.spi.IgniteSpiThread.run(IgniteSpiThread.java:62)

Thread [name="grid-nio-worker-tcp-comm-3-#29%local.DestroyLocalCacheTest%", id=42, state=RUNNABLE, blockCnt=0, waitCnt=0]
        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
        at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
        at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:93)
        at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
        - locked o.a.i.i.util.nio.SelectedSelectionKeySet@57829d67
        - locked java.util.Collections$UnmodifiableSet@19dfb72a
        - locked sun.nio.ch.EPollSelectorImpl@17c68925
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
        at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.bodyInternal(GridNioServer.java:2075)
        at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.body(GridNioServer.java:1749)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:110)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="grid-nio-worker-tcp-comm-2-#28%local.DestroyLocalCacheTest%", id=41, state=RUNNABLE, blockCnt=0, waitCnt=0]
        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
        at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
        at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:93)
        at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
        - locked o.a.i.i.util.nio.SelectedSelectionKeySet@7e0ea639
        - locked java.util.Collections$UnmodifiableSet@3d24753a
        - locked sun.nio.ch.EPollSelectorImpl@59a6e353
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
        at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.bodyInternal(GridNioServer.java:2075)
        at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.body(GridNioServer.java:1749)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:110)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="grid-nio-worker-tcp-comm-1-#27%local.DestroyLocalCacheTest%", id=40, state=RUNNABLE, blockCnt=2, waitCnt=0]
        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
        at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
        at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:93)
        at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
        - locked o.a.i.i.util.nio.SelectedSelectionKeySet@7a0ac6e3
        - locked java.util.Collections$UnmodifiableSet@71be98f5
        - locked sun.nio.ch.EPollSelectorImpl@6fadae5d
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
        at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.bodyInternal(GridNioServer.java:2075)
        at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.body(GridNioServer.java:1749)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:110)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="grid-nio-worker-tcp-comm-0-#26%local.DestroyLocalCacheTest%", id=39, state=RUNNABLE, blockCnt=1, waitCnt=0]
        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
        at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
        at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:93)
        at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
        - locked o.a.i.i.util.nio.SelectedSelectionKeySet@17f6480
        - locked java.util.Collections$UnmodifiableSet@2d6e8792
        - locked sun.nio.ch.EPollSelectorImpl@2812cbfa
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
        at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.bodyInternal(GridNioServer.java:2075)
        at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.body(GridNioServer.java:1749)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:110)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="nio-acceptor-#25%local.DestroyLocalCacheTest%", id=38, state=RUNNABLE, blockCnt=1, waitCnt=0]
        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
        at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
        at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:93)
        at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
        - locked sun.nio.ch.Util$3@2acf57e3
        - locked java.util.Collections$UnmodifiableSet@506e6d5e
        - locked sun.nio.ch.EPollSelectorImpl@96532d6
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
        at o.a.i.i.util.nio.GridNioServer$GridNioAcceptWorker.accept(GridNioServer.java:2813)
        at o.a.i.i.util.nio.GridNioServer$GridNioAcceptWorker.body(GridNioServer.java:2785)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:110)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="grid-timeout-worker-#24%local.DestroyLocalCacheTest%", id=37, state=TIMED_WAITING, blockCnt=1, waitCnt=956]
    Lock [object=java.lang.Object@3796751b, ownerName=null, ownerId=-1]
        at java.lang.Object.wait(Native Method)
        at o.a.i.i.processors.timeout.GridTimeoutProcessor$TimeoutWorker.body(GridTimeoutProcessor.java:195)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:110)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="data-streamer-stripe-7-#17%local.DestroyLocalCacheTest%", id=30, state=WAITING, blockCnt=0, waitCnt=1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.StripedExecutor$StripeConcurrentQueue.take(StripedExecutor.java:651)
        at o.a.i.i.util.StripedExecutor$Stripe.run(StripedExecutor.java:499)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="data-streamer-stripe-6-#16%local.DestroyLocalCacheTest%", id=29, state=WAITING, blockCnt=0, waitCnt=1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.StripedExecutor$StripeConcurrentQueue.take(StripedExecutor.java:651)
        at o.a.i.i.util.StripedExecutor$Stripe.run(StripedExecutor.java:499)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="data-streamer-stripe-5-#15%local.DestroyLocalCacheTest%", id=28, state=WAITING, blockCnt=0, waitCnt=1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.StripedExecutor$StripeConcurrentQueue.take(StripedExecutor.java:651)
        at o.a.i.i.util.StripedExecutor$Stripe.run(StripedExecutor.java:499)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="data-streamer-stripe-4-#14%local.DestroyLocalCacheTest%", id=27, state=WAITING, blockCnt=0, waitCnt=1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.StripedExecutor$StripeConcurrentQueue.take(StripedExecutor.java:651)
        at o.a.i.i.util.StripedExecutor$Stripe.run(StripedExecutor.java:499)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="data-streamer-stripe-3-#13%local.DestroyLocalCacheTest%", id=26, state=WAITING, blockCnt=3, waitCnt=1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.StripedExecutor$StripeConcurrentQueue.take(StripedExecutor.java:651)
        at o.a.i.i.util.StripedExecutor$Stripe.run(StripedExecutor.java:499)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="data-streamer-stripe-2-#12%local.DestroyLocalCacheTest%", id=25, state=WAITING, blockCnt=3, waitCnt=1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.StripedExecutor$StripeConcurrentQueue.take(StripedExecutor.java:651)
        at o.a.i.i.util.StripedExecutor$Stripe.run(StripedExecutor.java:499)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="data-streamer-stripe-1-#11%local.DestroyLocalCacheTest%", id=24, state=WAITING, blockCnt=2, waitCnt=1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.StripedExecutor$StripeConcurrentQueue.take(StripedExecutor.java:651)
        at o.a.i.i.util.StripedExecutor$Stripe.run(StripedExecutor.java:499)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="data-streamer-stripe-0-#10%local.DestroyLocalCacheTest%", id=23, state=WAITING, blockCnt=1, waitCnt=1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.StripedExecutor$StripeConcurrentQueue.take(StripedExecutor.java:651)
        at o.a.i.i.util.StripedExecutor$Stripe.run(StripedExecutor.java:499)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="sys-stripe-7-#9%local.DestroyLocalCacheTest%", id=22, state=WAITING, blockCnt=0, waitCnt=1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.StripedExecutor$StripeConcurrentQueue.take(StripedExecutor.java:651)
        at o.a.i.i.util.StripedExecutor$Stripe.run(StripedExecutor.java:499)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="sys-stripe-6-#8%local.DestroyLocalCacheTest%", id=21, state=WAITING, blockCnt=0, waitCnt=1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.StripedExecutor$StripeConcurrentQueue.take(StripedExecutor.java:651)
        at o.a.i.i.util.StripedExecutor$Stripe.run(StripedExecutor.java:499)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="sys-stripe-5-#7%local.DestroyLocalCacheTest%", id=20, state=WAITING, blockCnt=0, waitCnt=1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.StripedExecutor$StripeConcurrentQueue.take(StripedExecutor.java:651)
        at o.a.i.i.util.StripedExecutor$Stripe.run(StripedExecutor.java:499)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="sys-stripe-4-#6%local.DestroyLocalCacheTest%", id=19, state=WAITING, blockCnt=0, waitCnt=1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.StripedExecutor$StripeConcurrentQueue.take(StripedExecutor.java:651)
        at o.a.i.i.util.StripedExecutor$Stripe.run(StripedExecutor.java:499)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="sys-stripe-3-#5%local.DestroyLocalCacheTest%", id=18, state=WAITING, blockCnt=0, waitCnt=1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.StripedExecutor$StripeConcurrentQueue.take(StripedExecutor.java:651)
        at o.a.i.i.util.StripedExecutor$Stripe.run(StripedExecutor.java:499)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="sys-stripe-2-#4%local.DestroyLocalCacheTest%", id=17, state=WAITING, blockCnt=0, waitCnt=1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.StripedExecutor$StripeConcurrentQueue.take(StripedExecutor.java:651)
        at o.a.i.i.util.StripedExecutor$Stripe.run(StripedExecutor.java:499)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="sys-stripe-1-#3%local.DestroyLocalCacheTest%", id=16, state=WAITING, blockCnt=0, waitCnt=1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.StripedExecutor$StripeConcurrentQueue.take(StripedExecutor.java:651)
        at o.a.i.i.util.StripedExecutor$Stripe.run(StripedExecutor.java:499)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="sys-stripe-0-#2%local.DestroyLocalCacheTest%", id=15, state=WAITING, blockCnt=0, waitCnt=1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.StripedExecutor$StripeConcurrentQueue.take(StripedExecutor.java:651)
        at o.a.i.i.util.StripedExecutor$Stripe.run(StripedExecutor.java:499)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="test-runner-#1%local.DestroyLocalCacheTest%", id=14, state=WAITING, blockCnt=4, waitCnt=5]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.future.GridFutureAdapter.get0(GridFutureAdapter.java:177)
        at o.a.i.i.util.future.GridFutureAdapter.get(GridFutureAdapter.java:140)
        at o.a.i.i.processors.cache.local.DestroyLocalCacheTest.testDestroyAsync(DestroyLocalCacheTest.java:40)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:498)
        at junit.framework.TestCase.runTest(TestCase.java:176)
        at o.a.i.testframework.junits.GridAbstractTest.runTestInternal(GridAbstractTest.java:2001)
        at o.a.i.testframework.junits.GridAbstractTest.access$000(GridAbstractTest.java:133)
        at o.a.i.testframework.junits.GridAbstractTest$5.run(GridAbstractTest.java:1916)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="jvm-pause-detector-worker", id=13, state=TIMED_WAITING, blockCnt=0, waitCnt=5984]
        at java.lang.Thread.sleep(Native Method)
        at o.a.i.i.LongJVMPauseDetector$1.run(LongJVMPauseDetector.java:90)

Thread [name="ignite-clock-for-tests", id=11, state=TIMED_WAITING, blockCnt=0, waitCnt=29548]
        at java.lang.Thread.sleep(Native Method)
        at o.a.i.i.util.GridTestClockTimer.run(GridTestClockTimer.java:41)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="Monitor Ctrl-Break", id=5, state=RUNNABLE, blockCnt=0, waitCnt=0]
        at java.net.SocketInputStream.socketRead0(Native Method)
        at java.net.SocketInputStream.socketRead(SocketInputStream.java:116)
        at java.net.SocketInputStream.read(SocketInputStream.java:171)
        at java.net.SocketInputStream.read(SocketInputStream.java:141)
        at sun.nio.cs.StreamDecoder.readBytes(StreamDecoder.java:284)
        at sun.nio.cs.StreamDecoder.implRead(StreamDecoder.java:326)
        at sun.nio.cs.StreamDecoder.read(StreamDecoder.java:178)
        - locked java.io.InputStreamReader@67b64c45
        at java.io.InputStreamReader.read(InputStreamReader.java:184)
        at java.io.BufferedReader.fill(BufferedReader.java:161)
        at java.io.BufferedReader.readLine(BufferedReader.java:324)
        - locked java.io.InputStreamReader@67b64c45
        at java.io.BufferedReader.readLine(BufferedReader.java:389)
        at com.intellij.rt.execution.application.AppMainV2$1.run(AppMainV2.java:64)

Thread [name="Signal Dispatcher", id=4, state=RUNNABLE, blockCnt=0, waitCnt=0]

Thread [name="Finalizer", id=3, state=WAITING, blockCnt=3, waitCnt=4]
    Lock [object=java.lang.ref.ReferenceQueue$Lock@4411d970, ownerName=null, ownerId=-1]
        at java.lang.Object.wait(Native Method)
        at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:143)
        at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:164)
        at java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:209)

Thread [name="Reference Handler", id=2, state=WAITING, blockCnt=3, waitCnt=3]
    Lock [object=java.lang.ref.Reference$Lock@6442b0a6, ownerName=null, ownerId=-1]
        at java.lang.Object.wait(Native Method)
        at java.lang.Object.wait(Object.java:502)
        at java.lang.ref.Reference.tryHandlePending(Reference.java:191)
        at java.lang.ref.Reference$ReferenceHandler.run(Reference.java:153)

Thread [name="main", id=1, state=RUNNABLE, blockCnt=0, waitCnt=1]
        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:1324)
        at o.a.i.testframework.junits.GridAbstractTest.runTest(GridAbstractTest.java:1942)
        at junit.framework.TestCase.runBare(TestCase.java:141)
        at junit.framework.TestResult$1.protect(TestResult.java:122)
        at junit.framework.TestResult.runProtected(TestResult.java:142)
        at junit.framework.TestResult.run(TestResult.java:125)
        at junit.framework.TestCase.run(TestCase.java:129)
        at junit.framework.TestSuite.runTest(TestSuite.java:255)
        at junit.framework.TestSuite.run(TestSuite.java:250)
        at org.junit.internal.runners.JUnit38ClassRunner.run(JUnit38ClassRunner.java:84)
        at org.junit.runner.JUnitCore.run(JUnitCore.java:160)
        at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:68)
        at com.intellij.rt.execution.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:47)
        at com.intellij.rt.execution.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:242)
        at com.intellij.rt.execution.junit.JUnitStarter.main(JUnitStarter.java:70)



[2018-03-26 19:15:09,244][WARN ][main][root] Thread dump at 2018/03/26 19:15:09 MSK
Thread [name="sys-#69%local.DestroyLocalCacheTest%", id=95, state=TIMED_WAITING, blockCnt=0, waitCnt=1]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@e580929, 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-#68%local.DestroyLocalCacheTest%", id=94, state=TIMED_WAITING, blockCnt=0, waitCnt=1]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@e580929, 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-#67%local.DestroyLocalCacheTest%", id=93, state=TIMED_WAITING, blockCnt=0, waitCnt=1]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@e580929, 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-#66%local.DestroyLocalCacheTest%", id=92, state=TIMED_WAITING, blockCnt=0, waitCnt=1]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@e580929, 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-#65%local.DestroyLocalCacheTest%", id=91, state=TIMED_WAITING, blockCnt=0, waitCnt=1]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@e580929, 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-#64%local.DestroyLocalCacheTest%", id=90, state=TIMED_WAITING, blockCnt=0, waitCnt=1]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@e580929, 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="put-thread-5", id=66, state=WAITING, blockCnt=138, waitCnt=61]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.future.GridFutureAdapter.get0(GridFutureAdapter.java:177)
        at o.a.i.i.util.future.GridFutureAdapter.get(GridFutureAdapter.java:140)
        at o.a.i.i.processors.cache.GridCacheAdapter$22.op(GridCacheAdapter.java:2390)
        at o.a.i.i.processors.cache.GridCacheAdapter$22.op(GridCacheAdapter.java:2388)
        at o.a.i.i.processors.cache.GridCacheAdapter.syncOp(GridCacheAdapter.java:4088)
        at o.a.i.i.processors.cache.GridCacheAdapter.put0(GridCacheAdapter.java:2388)
        at o.a.i.i.processors.cache.GridCacheAdapter.put(GridCacheAdapter.java:2369)
        at o.a.i.i.processors.cache.GridCacheAdapter.put(GridCacheAdapter.java:2346)
        at o.a.i.i.processors.cache.IgniteCacheProxyImpl.put(IgniteCacheProxyImpl.java:1084)
        at o.a.i.i.processors.cache.GatewayProtectedCacheProxy.put(GatewayProtectedCacheProxy.java:886)
        at o.a.i.i.processors.cache.local.DestroyLocalCacheTest.lambda$testDestroyAsync$0(DestroyLocalCacheTest.java:32)
        at o.a.i.i.processors.cache.local.DestroyLocalCacheTest$$Lambda$7/759487471.call(Unknown Source)
        at o.a.i.testframework.GridTestThread.run(GridTestThread.java:86)

Thread [name="put-thread-4", id=65, state=WAITING, blockCnt=109, waitCnt=43]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.future.GridFutureAdapter.get0(GridFutureAdapter.java:177)
        at o.a.i.i.util.future.GridFutureAdapter.get(GridFutureAdapter.java:140)
        at o.a.i.i.processors.cache.GridCacheAdapter$22.op(GridCacheAdapter.java:2390)
        at o.a.i.i.processors.cache.GridCacheAdapter$22.op(GridCacheAdapter.java:2388)
        at o.a.i.i.processors.cache.GridCacheAdapter.syncOp(GridCacheAdapter.java:4088)
        at o.a.i.i.processors.cache.GridCacheAdapter.put0(GridCacheAdapter.java:2388)
        at o.a.i.i.processors.cache.GridCacheAdapter.put(GridCacheAdapter.java:2369)
        at o.a.i.i.processors.cache.GridCacheAdapter.put(GridCacheAdapter.java:2346)
        at o.a.i.i.processors.cache.IgniteCacheProxyImpl.put(IgniteCacheProxyImpl.java:1084)
        at o.a.i.i.processors.cache.GatewayProtectedCacheProxy.put(GatewayProtectedCacheProxy.java:886)
        at o.a.i.i.processors.cache.local.DestroyLocalCacheTest.lambda$testDestroyAsync$0(DestroyLocalCacheTest.java:32)
        at o.a.i.i.processors.cache.local.DestroyLocalCacheTest$$Lambda$7/759487471.call(Unknown Source)
        at o.a.i.testframework.GridTestThread.run(GridTestThread.java:86)

Thread [name="put-thread-3", id=64, state=WAITING, blockCnt=93, waitCnt=56]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.future.GridFutureAdapter.get0(GridFutureAdapter.java:177)
        at o.a.i.i.util.future.GridFutureAdapter.get(GridFutureAdapter.java:140)
        at o.a.i.i.util.future.IgniteFutureImpl.get(IgniteFutureImpl.java:134)
        at o.a.i.i.processors.cache.GatewayProtectedCacheProxy.destroy(GatewayProtectedCacheProxy.java:1503)
        at o.a.i.i.processors.cache.local.DestroyLocalCacheTest.lambda$testDestroyAsync$0(DestroyLocalCacheTest.java:30)
        at o.a.i.i.processors.cache.local.DestroyLocalCacheTest$$Lambda$7/759487471.call(Unknown Source)
        at o.a.i.testframework.GridTestThread.run(GridTestThread.java:86)

Thread [name="put-thread-2", id=63, state=WAITING, blockCnt=132, waitCnt=57]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.future.GridFutureAdapter.get0(GridFutureAdapter.java:177)
        at o.a.i.i.util.future.GridFutureAdapter.get(GridFutureAdapter.java:140)
        at o.a.i.i.processors.cache.GridCacheAdapter$22.op(GridCacheAdapter.java:2390)
        at o.a.i.i.processors.cache.GridCacheAdapter$22.op(GridCacheAdapter.java:2388)
        at o.a.i.i.processors.cache.GridCacheAdapter.syncOp(GridCacheAdapter.java:4088)
        at o.a.i.i.processors.cache.GridCacheAdapter.put0(GridCacheAdapter.java:2388)
        at o.a.i.i.processors.cache.GridCacheAdapter.put(GridCacheAdapter.java:2369)
        at o.a.i.i.processors.cache.GridCacheAdapter.put(GridCacheAdapter.java:2346)
        at o.a.i.i.processors.cache.IgniteCacheProxyImpl.put(IgniteCacheProxyImpl.java:1084)
        at o.a.i.i.processors.cache.GatewayProtectedCacheProxy.put(GatewayProtectedCacheProxy.java:886)
        at o.a.i.i.processors.cache.local.DestroyLocalCacheTest.lambda$testDestroyAsync$0(DestroyLocalCacheTest.java:32)
        at o.a.i.i.processors.cache.local.DestroyLocalCacheTest$$Lambda$7/759487471.call(Unknown Source)
        at o.a.i.testframework.GridTestThread.run(GridTestThread.java:86)

Thread [name="put-thread-1", id=62, state=WAITING, blockCnt=135, waitCnt=76]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.future.GridFutureAdapter.get0(GridFutureAdapter.java:177)
        at o.a.i.i.util.future.GridFutureAdapter.get(GridFutureAdapter.java:140)
        at o.a.i.i.processors.cache.GridCacheAdapter$22.op(GridCacheAdapter.java:2390)
        at o.a.i.i.processors.cache.GridCacheAdapter$22.op(GridCacheAdapter.java:2388)
        at o.a.i.i.processors.cache.GridCacheAdapter.syncOp(GridCacheAdapter.java:4088)
        at o.a.i.i.processors.cache.GridCacheAdapter.put0(GridCacheAdapter.java:2388)
        at o.a.i.i.processors.cache.GridCacheAdapter.put(GridCacheAdapter.java:2369)
        at o.a.i.i.processors.cache.GridCacheAdapter.put(GridCacheAdapter.java:2346)
        at o.a.i.i.processors.cache.IgniteCacheProxyImpl.put(IgniteCacheProxyImpl.java:1084)
        at o.a.i.i.processors.cache.GatewayProtectedCacheProxy.put(GatewayProtectedCacheProxy.java:886)
        at o.a.i.i.processors.cache.local.DestroyLocalCacheTest.lambda$testDestroyAsync$0(DestroyLocalCacheTest.java:32)
        at o.a.i.i.processors.cache.local.DestroyLocalCacheTest$$Lambda$7/759487471.call(Unknown Source)
        at o.a.i.testframework.GridTestThread.run(GridTestThread.java:86)

Thread [name="async-callable-runner-1", id=61, state=WAITING, blockCnt=0, waitCnt=1]
    Lock [object=o.a.i.testframework.GridTestSafeThreadFactory$1@1cd072a9, ownerName=null, ownerId=-1]
        at java.lang.Object.wait(Native Method)
        at java.lang.Thread.join(Thread.java:1252)
        at java.lang.Thread.join(Thread.java:1326)
        at o.a.i.testframework.GridTestUtils.runMultiThreaded(GridTestUtils.java:871)
        at o.a.i.testframework.GridTestUtils$4.call(GridTestUtils.java:811)
        at o.a.i.testframework.GridTestUtils$4.call(GridTestUtils.java:809)
        at o.a.i.testframework.GridTestUtils$7.run(GridTestUtils.java:979)
        at o.a.i.testframework.GridTestUtils$9.call(GridTestUtils.java:1275)
        at o.a.i.testframework.GridTestThread.run(GridTestThread.java:86)

Thread [name="ttl-cleanup-worker-#40%local.DestroyLocalCacheTest%", id=60, state=TIMED_WAITING, blockCnt=0, waitCnt=596]
        at java.lang.Thread.sleep(Native Method)
        at o.a.i.i.util.IgniteUtils.sleep(IgniteUtils.java:7598)
        at o.a.i.i.processors.cache.GridCacheSharedTtlCleanupManager$CleanupWorker.body(GridCacheSharedTtlCleanupManager.java:137)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:110)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="srvc-deploy-#39%local.DestroyLocalCacheTest%", id=58, state=WAITING, blockCnt=0, waitCnt=3]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@7c75222b, 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="exchange-worker-#38%local.DestroyLocalCacheTest%", id=57, state=TIMED_WAITING, blockCnt=0, waitCnt=1489]
    Lock [object=java.util.concurrent.locks.ReentrantReadWriteLock$NonfairSync@48140564, 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.doAcquireNanos(AbstractQueuedSynchronizer.java:934)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer.tryAcquireNanos(AbstractQueuedSynchronizer.java:1247)
        at java.util.concurrent.locks.ReentrantReadWriteLock$WriteLock.tryLock(ReentrantReadWriteLock.java:1115)
        at o.a.i.i.util.StripedCompositeReadWriteLock$WriteLock.tryLock(StripedCompositeReadWriteLock.java:201)
        at o.a.i.i.processors.cache.GridCacheGateway.onStopped(GridCacheGateway.java:305)
        at o.a.i.i.processors.cache.GridCacheProcessor.stopGateway(GridCacheProcessor.java:2060)
        at o.a.i.i.processors.cache.GridCacheProcessor.onExchangeDone(GridCacheProcessor.java:2229)
        at o.a.i.i.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture.onDone(GridDhtPartitionsExchangeFuture.java:1612)
        at o.a.i.i.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture.finishExchangeOnCoordinator(GridDhtPartitionsExchangeFuture.java:2610)
        at o.a.i.i.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture.onAllReceived(GridDhtPartitionsExchangeFuture.java:2408)
        at o.a.i.i.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture.distributedExchange(GridDhtPartitionsExchangeFuture.java:1135)
        at o.a.i.i.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture.init(GridDhtPartitionsExchangeFuture.java:704)
        at o.a.i.i.processors.cache.GridCachePartitionExchangeManager$ExchangeWorker.body(GridCachePartitionExchangeManager.java:2344)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:110)
        at java.lang.Thread.run(Thread.java:748)

    Locked synchronizers:
        java.util.concurrent.locks.ReentrantReadWriteLock$NonfairSync@4c203ea1
        java.util.concurrent.locks.ReentrantReadWriteLock$NonfairSync@27f674d
        java.util.concurrent.locks.ReentrantReadWriteLock$NonfairSync@1d251891
Thread [name="disco-event-worker-#37%local.DestroyLocalCacheTest%", id=56, state=WAITING, blockCnt=0, waitCnt=299]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@58ceff1, 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.managers.discovery.GridDiscoveryManager$DiscoveryWorker.body0(GridDiscoveryManager.java:2598)
        at o.a.i.i.managers.discovery.GridDiscoveryManager$DiscoveryWorker.body(GridDiscoveryManager.java:2580)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:110)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="tcp-disco-ip-finder-cleaner-#6%local.DestroyLocalCacheTest%", id=55, state=TIMED_WAITING, blockCnt=0, waitCnt=9]
        at java.lang.Thread.sleep(Native Method)
        at o.a.i.spi.discovery.tcp.ServerImpl$IpFinderCleaner.body(ServerImpl.java:1883)
        at o.a.i.spi.IgniteSpiThread.run(IgniteSpiThread.java:62)

Thread [name="tcp-disco-multicast-addr-sender-#5%local.DestroyLocalCacheTest%", id=54, state=RUNNABLE, blockCnt=0, waitCnt=0]
        at java.net.PlainDatagramSocketImpl.receive0(Native Method)
        - locked java.net.PlainDatagramSocketImpl@7c30a502
        at java.net.AbstractPlainDatagramSocketImpl.receive(AbstractPlainDatagramSocketImpl.java:143)
        - locked java.net.PlainDatagramSocketImpl@7c30a502
        at java.net.DatagramSocket.receive(DatagramSocket.java:812)
        - locked java.net.DatagramPacket@49e4cb85
        - locked java.net.MulticastSocket@2133c8f8
        at o.a.i.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder$AddressSender.body(TcpDiscoveryMulticastIpFinder.java:882)
        at o.a.i.spi.IgniteSpiThread.run(IgniteSpiThread.java:62)

Thread [name="tcp-disco-srvr-#4%local.DestroyLocalCacheTest%", id=53, state=RUNNABLE, blockCnt=0, waitCnt=0]
        at java.net.PlainSocketImpl.socketAccept(Native Method)
        at java.net.AbstractPlainSocketImpl.accept(AbstractPlainSocketImpl.java:409)
        at java.net.ServerSocket.implAccept(ServerSocket.java:545)
        at java.net.ServerSocket.accept(ServerSocket.java:513)
        at o.a.i.spi.discovery.tcp.ServerImpl$TcpServer.body(ServerImpl.java:5590)
        at o.a.i.spi.IgniteSpiThread.run(IgniteSpiThread.java:62)

Thread [name="tcp-disco-msg-worker-#3%local.DestroyLocalCacheTest%", id=52, state=TIMED_WAITING, blockCnt=4, waitCnt=29404]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@43a25848, 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.LinkedBlockingDeque.pollFirst(LinkedBlockingDeque.java:522)
        at java.util.concurrent.LinkedBlockingDeque.poll(LinkedBlockingDeque.java:684)
        at o.a.i.spi.discovery.tcp.ServerImpl$MessageWorkerAdapter.body(ServerImpl.java:6702)
        at o.a.i.spi.discovery.tcp.ServerImpl$RingMessageWorker.body(ServerImpl.java:2602)
        at o.a.i.spi.IgniteSpiThread.run(IgniteSpiThread.java:62)

Thread [name="grid-data-loader-flusher-#36%local.DestroyLocalCacheTest%", id=51, state=WAITING, blockCnt=0, waitCnt=1]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@3ac3fd8b, 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.DelayQueue.take(DelayQueue.java:211)
        at o.a.i.i.processors.datastreamer.DataStreamProcessor$2.body(DataStreamProcessor.java:108)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:110)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="grid-nio-worker-client-listener-3-#34%local.DestroyLocalCacheTest%", id=49, state=RUNNABLE, blockCnt=0, waitCnt=0]
        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
        at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
        at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:93)
        at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
        - locked o.a.i.i.util.nio.SelectedSelectionKeySet@5594a1b5
        - locked java.util.Collections$UnmodifiableSet@6a5fc7f7
        - locked sun.nio.ch.EPollSelectorImpl@3b6eb2ec
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
        at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.bodyInternal(GridNioServer.java:2075)
        at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.body(GridNioServer.java:1749)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:110)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="grid-nio-worker-client-listener-2-#33%local.DestroyLocalCacheTest%", id=48, state=RUNNABLE, blockCnt=0, waitCnt=0]
        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
        at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
        at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:93)
        at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
        - locked o.a.i.i.util.nio.SelectedSelectionKeySet@1e643faf
        - locked java.util.Collections$UnmodifiableSet@6e8dacdf
        - locked sun.nio.ch.EPollSelectorImpl@7a79be86
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
        at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.bodyInternal(GridNioServer.java:2075)
        at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.body(GridNioServer.java:1749)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:110)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="grid-nio-worker-client-listener-1-#32%local.DestroyLocalCacheTest%", id=47, state=RUNNABLE, blockCnt=0, waitCnt=0]
        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
        at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
        at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:93)
        at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
        - locked o.a.i.i.util.nio.SelectedSelectionKeySet@34ce8af7
        - locked java.util.Collections$UnmodifiableSet@b684286
        - locked sun.nio.ch.EPollSelectorImpl@880ec60
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
        at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.bodyInternal(GridNioServer.java:2075)
        at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.body(GridNioServer.java:1749)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:110)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="grid-nio-worker-client-listener-0-#31%local.DestroyLocalCacheTest%", id=46, state=RUNNABLE, blockCnt=0, waitCnt=0]
        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
        at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
        at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:93)
        at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
        - locked o.a.i.i.util.nio.SelectedSelectionKeySet@3f3afe78
        - locked java.util.Collections$UnmodifiableSet@7f63425a
        - locked sun.nio.ch.EPollSelectorImpl@36d64342
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
        at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.bodyInternal(GridNioServer.java:2075)
        at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.body(GridNioServer.java:1749)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:110)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="nio-acceptor-#30%local.DestroyLocalCacheTest%", id=45, state=RUNNABLE, blockCnt=0, waitCnt=0]
        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
        at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
        at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:93)
        at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
        - locked sun.nio.ch.Util$3@39ba5a14
        - locked java.util.Collections$UnmodifiableSet@511baa65
        - locked sun.nio.ch.EPollSelectorImpl@340f438e
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
        at o.a.i.i.util.nio.GridNioServer$GridNioAcceptWorker.accept(GridNioServer.java:2813)
        at o.a.i.i.util.nio.GridNioServer$GridNioAcceptWorker.body(GridNioServer.java:2785)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:110)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="grid-sharedfs-timeout-worker-#2%local.DestroyLocalCacheTest%", id=44, state=TIMED_WAITING, blockCnt=0, waitCnt=60]
    Lock [object=java.lang.Object@30c7da1e, ownerName=null, ownerId=-1]
        at java.lang.Object.wait(Native Method)
        at o.a.i.spi.checkpoint.sharedfs.SharedFsTimeoutTask.body(SharedFsTimeoutTask.java:99)
        at o.a.i.spi.IgniteSpiThread.run(IgniteSpiThread.java:62)

Thread [name="tcp-comm-worker-#1%local.DestroyLocalCacheTest%", id=43, state=TIMED_WAITING, blockCnt=0, waitCnt=1]
    Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@5b464ce8, 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 o.a.i.spi.communication.tcp.TcpCommunicationSpi$CommunicationWorker.body(TcpCommunicationSpi.java:4093)
        at o.a.i.spi.IgniteSpiThread.run(IgniteSpiThread.java:62)

Thread [name="grid-nio-worker-tcp-comm-3-#29%local.DestroyLocalCacheTest%", id=42, state=RUNNABLE, blockCnt=0, waitCnt=0]
        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
        at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
        at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:93)
        at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
        - locked o.a.i.i.util.nio.SelectedSelectionKeySet@57829d67
        - locked java.util.Collections$UnmodifiableSet@19dfb72a
        - locked sun.nio.ch.EPollSelectorImpl@17c68925
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
        at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.bodyInternal(GridNioServer.java:2075)
        at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.body(GridNioServer.java:1749)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:110)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="grid-nio-worker-tcp-comm-2-#28%local.DestroyLocalCacheTest%", id=41, state=RUNNABLE, blockCnt=0, waitCnt=0]
        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
        at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
        at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:93)
        at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
        - locked o.a.i.i.util.nio.SelectedSelectionKeySet@7e0ea639
        - locked java.util.Collections$UnmodifiableSet@3d24753a
        - locked sun.nio.ch.EPollSelectorImpl@59a6e353
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
        at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.bodyInternal(GridNioServer.java:2075)
        at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.body(GridNioServer.java:1749)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:110)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="grid-nio-worker-tcp-comm-1-#27%local.DestroyLocalCacheTest%", id=40, state=RUNNABLE, blockCnt=2, waitCnt=0]
        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
        at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
        at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:93)
        at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
        - locked o.a.i.i.util.nio.SelectedSelectionKeySet@7a0ac6e3
        - locked java.util.Collections$UnmodifiableSet@71be98f5
        - locked sun.nio.ch.EPollSelectorImpl@6fadae5d
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
        at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.bodyInternal(GridNioServer.java:2075)
        at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.body(GridNioServer.java:1749)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:110)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="grid-nio-worker-tcp-comm-0-#26%local.DestroyLocalCacheTest%", id=39, state=RUNNABLE, blockCnt=1, waitCnt=0]
        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
        at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
        at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:93)
        at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
        - locked o.a.i.i.util.nio.SelectedSelectionKeySet@17f6480
        - locked java.util.Collections$UnmodifiableSet@2d6e8792
        - locked sun.nio.ch.EPollSelectorImpl@2812cbfa
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
        at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.bodyInternal(GridNioServer.java:2075)
        at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.body(GridNioServer.java:1749)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:110)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="nio-acceptor-#25%local.DestroyLocalCacheTest%", id=38, state=RUNNABLE, blockCnt=1, waitCnt=0]
        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
        at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
        at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:93)
        at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
        - locked sun.nio.ch.Util$3@2acf57e3
        - locked java.util.Collections$UnmodifiableSet@506e6d5e
        - locked sun.nio.ch.EPollSelectorImpl@96532d6
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
        at o.a.i.i.util.nio.GridNioServer$GridNioAcceptWorker.accept(GridNioServer.java:2813)
        at o.a.i.i.util.nio.GridNioServer$GridNioAcceptWorker.body(GridNioServer.java:2785)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:110)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="grid-timeout-worker-#24%local.DestroyLocalCacheTest%", id=37, state=TIMED_WAITING, blockCnt=1, waitCnt=956]
    Lock [object=java.lang.Object@3796751b, ownerName=null, ownerId=-1]
        at java.lang.Object.wait(Native Method)
        at o.a.i.i.processors.timeout.GridTimeoutProcessor$TimeoutWorker.body(GridTimeoutProcessor.java:195)
        at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:110)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="data-streamer-stripe-7-#17%local.DestroyLocalCacheTest%", id=30, state=WAITING, blockCnt=0, waitCnt=1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.StripedExecutor$StripeConcurrentQueue.take(StripedExecutor.java:651)
        at o.a.i.i.util.StripedExecutor$Stripe.run(StripedExecutor.java:499)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="data-streamer-stripe-6-#16%local.DestroyLocalCacheTest%", id=29, state=WAITING, blockCnt=0, waitCnt=1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.StripedExecutor$StripeConcurrentQueue.take(StripedExecutor.java:651)
        at o.a.i.i.util.StripedExecutor$Stripe.run(StripedExecutor.java:499)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="data-streamer-stripe-5-#15%local.DestroyLocalCacheTest%", id=28, state=WAITING, blockCnt=0, waitCnt=1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.StripedExecutor$StripeConcurrentQueue.take(StripedExecutor.java:651)
        at o.a.i.i.util.StripedExecutor$Stripe.run(StripedExecutor.java:499)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="data-streamer-stripe-4-#14%local.DestroyLocalCacheTest%", id=27, state=WAITING, blockCnt=0, waitCnt=1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.StripedExecutor$StripeConcurrentQueue.take(StripedExecutor.java:651)
        at o.a.i.i.util.StripedExecutor$Stripe.run(StripedExecutor.java:499)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="data-streamer-stripe-3-#13%local.DestroyLocalCacheTest%", id=26, state=WAITING, blockCnt=3, waitCnt=1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.StripedExecutor$StripeConcurrentQueue.take(StripedExecutor.java:651)
        at o.a.i.i.util.StripedExecutor$Stripe.run(StripedExecutor.java:499)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="data-streamer-stripe-2-#12%local.DestroyLocalCacheTest%", id=25, state=WAITING, blockCnt=3, waitCnt=1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.StripedExecutor$StripeConcurrentQueue.take(StripedExecutor.java:651)
        at o.a.i.i.util.StripedExecutor$Stripe.run(StripedExecutor.java:499)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="data-streamer-stripe-1-#11%local.DestroyLocalCacheTest%", id=24, state=WAITING, blockCnt=2, waitCnt=1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.StripedExecutor$StripeConcurrentQueue.take(StripedExecutor.java:651)
        at o.a.i.i.util.StripedExecutor$Stripe.run(StripedExecutor.java:499)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="data-streamer-stripe-0-#10%local.DestroyLocalCacheTest%", id=23, state=WAITING, blockCnt=1, waitCnt=1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.StripedExecutor$StripeConcurrentQueue.take(StripedExecutor.java:651)
        at o.a.i.i.util.StripedExecutor$Stripe.run(StripedExecutor.java:499)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="sys-stripe-7-#9%local.DestroyLocalCacheTest%", id=22, state=WAITING, blockCnt=0, waitCnt=1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.StripedExecutor$StripeConcurrentQueue.take(StripedExecutor.java:651)
        at o.a.i.i.util.StripedExecutor$Stripe.run(StripedExecutor.java:499)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="sys-stripe-6-#8%local.DestroyLocalCacheTest%", id=21, state=WAITING, blockCnt=0, waitCnt=1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.StripedExecutor$StripeConcurrentQueue.take(StripedExecutor.java:651)
        at o.a.i.i.util.StripedExecutor$Stripe.run(StripedExecutor.java:499)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="sys-stripe-5-#7%local.DestroyLocalCacheTest%", id=20, state=WAITING, blockCnt=0, waitCnt=1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.StripedExecutor$StripeConcurrentQueue.take(StripedExecutor.java:651)
        at o.a.i.i.util.StripedExecutor$Stripe.run(StripedExecutor.java:499)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="sys-stripe-4-#6%local.DestroyLocalCacheTest%", id=19, state=WAITING, blockCnt=0, waitCnt=1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.StripedExecutor$StripeConcurrentQueue.take(StripedExecutor.java:651)
        at o.a.i.i.util.StripedExecutor$Stripe.run(StripedExecutor.java:499)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="sys-stripe-3-#5%local.DestroyLocalCacheTest%", id=18, state=WAITING, blockCnt=0, waitCnt=1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.StripedExecutor$StripeConcurrentQueue.take(StripedExecutor.java:651)
        at o.a.i.i.util.StripedExecutor$Stripe.run(StripedExecutor.java:499)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="sys-stripe-2-#4%local.DestroyLocalCacheTest%", id=17, state=WAITING, blockCnt=0, waitCnt=1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.StripedExecutor$StripeConcurrentQueue.take(StripedExecutor.java:651)
        at o.a.i.i.util.StripedExecutor$Stripe.run(StripedExecutor.java:499)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="sys-stripe-1-#3%local.DestroyLocalCacheTest%", id=16, state=WAITING, blockCnt=0, waitCnt=1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.StripedExecutor$StripeConcurrentQueue.take(StripedExecutor.java:651)
        at o.a.i.i.util.StripedExecutor$Stripe.run(StripedExecutor.java:499)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="sys-stripe-0-#2%local.DestroyLocalCacheTest%", id=15, state=WAITING, blockCnt=0, waitCnt=1]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.StripedExecutor$StripeConcurrentQueue.take(StripedExecutor.java:651)
        at o.a.i.i.util.StripedExecutor$Stripe.run(StripedExecutor.java:499)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="test-runner-#1%local.DestroyLocalCacheTest%", id=14, state=WAITING, blockCnt=4, waitCnt=5]
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at o.a.i.i.util.future.GridFutureAdapter.get0(GridFutureAdapter.java:177)
        at o.a.i.i.util.future.GridFutureAdapter.get(GridFutureAdapter.java:140)
        at o.a.i.i.processors.cache.local.DestroyLocalCacheTest.testDestroyAsync(DestroyLocalCacheTest.java:40)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:498)
        at junit.framework.TestCase.runTest(TestCase.java:176)
        at o.a.i.testframework.junits.GridAbstractTest.runTestInternal(GridAbstractTest.java:2001)
        at o.a.i.testframework.junits.GridAbstractTest.access$000(GridAbstractTest.java:133)
        at o.a.i.testframework.junits.GridAbstractTest$5.run(GridAbstractTest.java:1916)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="jvm-pause-detector-worker", id=13, state=TIMED_WAITING, blockCnt=0, waitCnt=5984]
        at java.lang.Thread.sleep(Native Method)
        at o.a.i.i.LongJVMPauseDetector$1.run(LongJVMPauseDetector.java:90)

Thread [name="ignite-clock-for-tests", id=11, state=TIMED_WAITING, blockCnt=0, waitCnt=29550]
        at java.lang.Thread.sleep(Native Method)
        at o.a.i.i.util.GridTestClockTimer.run(GridTestClockTimer.java:41)
        at java.lang.Thread.run(Thread.java:748)

Thread [name="Monitor Ctrl-Break", id=5, state=RUNNABLE, blockCnt=0, waitCnt=0]
        at java.net.SocketInputStream.socketRead0(Native Method)
        at java.net.SocketInputStream.socketRead(SocketInputStream.java:116)
        at java.net.SocketInputStream.read(SocketInputStream.java:171)
        at java.net.SocketInputStream.read(SocketInputStream.java:141)
        at sun.nio.cs.StreamDecoder.readBytes(StreamDecoder.java:284)
        at sun.nio.cs.StreamDecoder.implRead(StreamDecoder.java:326)
        at sun.nio.cs.StreamDecoder.read(StreamDecoder.java:178)
        - locked java.io.InputStreamReader@67b64c45
        at java.io.InputStreamReader.read(InputStreamReader.java:184)
        at java.io.BufferedReader.fill(BufferedReader.java:161)
        at java.io.BufferedReader.readLine(BufferedReader.java:324)
        - locked java.io.InputStreamReader@67b64c45
        at java.io.BufferedReader.readLine(BufferedReader.java:389)
        at com.intellij.rt.execution.application.AppMainV2$1.run(AppMainV2.java:64)

Thread [name="Signal Dispatcher", id=4, state=RUNNABLE, blockCnt=0, waitCnt=0]

Thread [name="Finalizer", id=3, state=WAITING, blockCnt=3, waitCnt=4]
    Lock [object=java.lang.ref.ReferenceQueue$Lock@4411d970, ownerName=null, ownerId=-1]
        at java.lang.Object.wait(Native Method)
        at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:143)
        at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:164)
        at java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:209)

Thread [name="Reference Handler", id=2, state=WAITING, blockCnt=3, waitCnt=3]
    Lock [object=java.lang.ref.Reference$Lock@6442b0a6, ownerName=null, ownerId=-1]
        at java.lang.Object.wait(Native Method)
        at java.lang.Object.wait(Object.java:502)
        at java.lang.ref.Reference.tryHandlePending(Reference.java:191)
        at java.lang.ref.Reference$ReferenceHandler.run(Reference.java:153)

Thread [name="main", id=1, state=RUNNABLE, blockCnt=0, waitCnt=1]
        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:1324)
        at o.a.i.testframework.junits.GridAbstractTest.runTest(GridAbstractTest.java:1944)
        at junit.framework.TestCase.runBare(TestCase.java:141)
        at junit.framework.TestResult$1.protect(TestResult.java:122)
        at junit.framework.TestResult.runProtected(TestResult.java:142)
        at junit.framework.TestResult.run(TestResult.java:125)
        at junit.framework.TestCase.run(TestCase.java:129)
        at junit.framework.TestSuite.runTest(TestSuite.java:255)
        at junit.framework.TestSuite.run(TestSuite.java:250)
        at org.junit.internal.runners.JUnit38ClassRunner.run(JUnit38ClassRunner.java:84)
        at org.junit.runner.JUnitCore.run(JUnitCore.java:160)
        at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:68)
        at com.intellij.rt.execution.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:47)
        at com.intellij.rt.execution.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:242)
        at com.intellij.rt.execution.junit.JUnitStarter.main(JUnitStarter.java:70)
{noformat}




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