You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ignite.apache.org by "Sergey Kozlov (JIRA)" <ji...@apache.org> on 2018/10/29 20:19:01 UTC

[jira] [Updated] (IGNITE-10056) Attempt to create MVCC cache with TTL causes full cluster halt

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

Sergey Kozlov updated IGNITE-10056:
-----------------------------------
    Summary: Attempt to create MVCC cache with TTL causes full cluster halt  (was: Attemp to create MVCC cache with TTL causes full cluster halt)

> Attempt to create MVCC cache with TTL causes full cluster halt
> --------------------------------------------------------------
>
>                 Key: IGNITE-10056
>                 URL: https://issues.apache.org/jira/browse/IGNITE-10056
>             Project: Ignite
>          Issue Type: Bug
>          Components: mvcc
>    Affects Versions: 2.7
>         Environment: 
>            Reporter: Sergey Kozlov
>            Priority: Critical
>             Fix For: 2.7
>
>
> 1. Start cluster with cache template and activate:
> {code:xml}
>             <bean class="org.apache.ignite.configuration.CacheConfiguration">
>                 <property name="name" value="tmpl*"/>
>                 <property name="atomicityMode" value="TRANSACTIONAL_SNAPSHOT"/>
>                 <property name="backups" value="1"/>
>                 <property name="cacheMode" value="PARTITIONED"/>
>                 <property name="writeSynchronizationMode" value="FULL_SYNC"/>
> 		<property name="eagerTtl" value="true"/>
> 		<property name="expiryPolicyFactory">
> 		    <bean class="javax.cache.expiry.CreatedExpiryPolicy" factory-method="factoryOf">
>             		<constructor-arg>
>                 	    <bean class="javax.cache.expiry.Duration">
>                     		<constructor-arg value="SECONDS"/>
>                     		<constructor-arg value="30"/>
>                 	   </bean>
>             		</constructor-arg>
>         	   </bean>
>                </property>
>             </bean>
> {code}
> 2. Try to create cache via {{sqlline}}
> {noformat}
> 0: jdbc:ignite:thin://127.0.0.1/> create table t1(a int, b varchar, primary key(a)) with "template=tmpl";
> Error: class org.apache.ignite.IgniteCheckedException: Grid configuration parameter invalid: expiry policy cannot be used with TRANSACTIONAL_SNAPSHOT atomicity mode (state=50000,code=1)
> java.sql.SQLException: class org.apache.ignite.IgniteCheckedException: Grid configuration parameter invalid: expiry policy cannot be used with TRANSACTIONAL_SNAPSHOT atomicity mode
>         at org.apache.ignite.internal.jdbc.thin.JdbcThinConnection.sendRequest(JdbcThinConnection.java:750)
>         at org.apache.ignite.internal.jdbc.thin.JdbcThinStatement.execute0(JdbcThinStatement.java:212)
>         at org.apache.ignite.internal.jdbc.thin.JdbcThinStatement.execute(JdbcThinStatement.java:475)
>         at sqlline.Commands.execute(Commands.java:823)
>         at sqlline.Commands.sql(Commands.java:733)
>         at sqlline.SqlLine.dispatch(SqlLine.java:795)
>         at sqlline.SqlLine.begin(SqlLine.java:668)
>         at sqlline.SqlLine.start(SqlLine.java:373)
>         at sqlline.SqlLine.main(SqlLine.java:265)
> {noformat}
> That's fine but all nodes stopped by failure handler:
> {noformat}
> [21:40:03,657][SEVERE][exchange-worker-#43][CacheAffinitySharedManager] Failed to initialize cache. Will try to rollback cache start routine. [cacheName=SQL_PUBLIC_T1]
> class org.apache.ignite.IgniteCheckedException: Grid configuration parameter invalid: expiry policy cannot be used with TRANSACTIONAL_SNAPSHOT atomicity mode
> 	at org.apache.ignite.internal.processors.GridProcessorAdapter.assertParameter(GridProcessorAdapter.java:140)
> 	at org.apache.ignite.internal.processors.cache.GridCacheProcessor.validate(GridCacheProcessor.java:521)
> 	at org.apache.ignite.internal.processors.cache.GridCacheProcessor.createCache(GridCacheProcessor.java:1560)
> 	at org.apache.ignite.internal.processors.cache.GridCacheProcessor.prepareCacheStart(GridCacheProcessor.java:2146)
> 	at org.apache.ignite.internal.processors.cache.CacheAffinitySharedManager.processCacheStartRequests(CacheAffinitySharedManager.java:898)
> 	at org.apache.ignite.internal.processors.cache.CacheAffinitySharedManager.onCacheChangeRequest(CacheAffinitySharedManager.java:798)
> 	at org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture.onCacheChangeRequest(GridDhtPartitionsExchangeFuture.java:1231)
> 	at org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture.init(GridDhtPartitionsExchangeFuture.java:738)
> 	at org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$ExchangeWorker.body0(GridCachePartitionExchangeManager.java:2667)
> 	at org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$ExchangeWorker.body(GridCachePartitionExchangeManager.java:2539)
> 	at org.apache.ignite.internal.util.worker.GridWorker.run(GridWorker.java:120)
> 	at java.lang.Thread.run(Thread.java:748)
> [21:40:03,660][INFO][exchange-worker-#43][GridCacheProcessor] Can not finish proxy initialization because proxy does not exist, cacheName=SQL_PUBLIC_T1, localNodeId=0c6a653d-b151-46b5-bd0c-7fea4b94ca26
> [21:40:03,662][INFO][exchange-worker-#43][CacheAffinitySharedManager] Caches starting performed in 66 ms.
> [21:40:03,667][INFO][exchange-worker-#43][CacheAffinitySharedManager] Affinity initialization for started caches performed in 5 ms.
> [21:40:03,673][INFO][exchange-worker-#43][GridDhtPartitionsExchangeFuture] Finished waiting for partition release future [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], waitTime=0ms, futInfo=NA, mode=DISTRIBUTED]
> [21:40:03,695][INFO][exchange-worker-#43][GridDhtPartitionsExchangeFuture] Finished waiting for partitions release latch: ClientLatch [coordinator=TcpDiscoveryNode [id=8a320d9a-c057-45e5-958a-6fff470c8485, addrs=ArrayList [127.0.0.1], sockAddrs=HashSet [/127.0.0.1:47500], discPort=47500, order=1, intOrder=1, lastExchangeTime=1540838375187, loc=false, ver=2.7.1#20181029-sha1:83c27088, isClient=false], ackSent=true, super=CompletableLatch [id=exchange, topVer=AffinityTopologyVersion [topVer=2, minorTopVer=2]]]
> [21:40:03,696][INFO][exchange-worker-#43][GridDhtPartitionsExchangeFuture] Finished waiting for partition release future [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], waitTime=0ms, futInfo=NA, mode=LOCAL]
> [21:40:03,721][INFO][exchange-worker-#43][GridCacheDatabaseSharedManager] Logical recovery performed in 23 ms.
> [21:40:03,727][SEVERE][exchange-worker-#43][GridDhtPartitionsExchangeFuture] Failed to reinitialize local partitions (rebalancing will be stopped): GridDhtPartitionExchangeId [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], discoEvt=DiscoveryEvent [evtNode=TcpDiscoveryNode [id=8a320d9a-c057-45e5-958a-6fff470c8485, addrs=ArrayList [127.0.0.1], sockAddrs=HashSet [/127.0.0.1:47500], discPort=47500, order=1, intOrder=1, lastExchangeTime=1540838375187, loc=false, ver=2.7.1#20181029-sha1:83c27088, isClient=false], topVer=2, nodeId8=0c6a653d, msg=null, type=DISCOVERY_CUSTOM_EVT, tstamp=1540838403586]DiscoveryCustomEvent [customMsg=DynamicCacheChangeBatch [id=1876f11c661-41abad2f-0c12-4bcb-ad8c-f37aea8f7c42, reqs=ArrayList [DynamicCacheChangeRequest [cacheName=SQL_PUBLIC_T1, hasCfg=true, nodeId=8a320d9a-c057-45e5-958a-6fff470c8485, clientStartOnly=false, stop=false, destroy=false, disabledAfterStartfalse]], exchangeActions=ExchangeActions [startCaches=[SQL_PUBLIC_T1], stopCaches=null, startGrps=[SQL_PUBLIC_T1], stopGrps=[], resetParts=null, stateChangeRequest=null], startCaches=false], affTopVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], super=], nodeId=8a320d9a, evt=DISCOVERY_CUSTOM_EVT]
> java.lang.IndexOutOfBoundsException: Index: 0, Size: 0
> 	at java.util.ArrayList.rangeCheck(ArrayList.java:657)
> 	at java.util.ArrayList.get(ArrayList.java:433)
> 	at org.apache.ignite.internal.processors.cache.CacheGroupContext.singleCacheContext(CacheGroupContext.java:387)
> 	at org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition.<init>(GridDhtLocalPartition.java:200)
> 	at org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopologyImpl.getOrCreatePartition(GridDhtPartitionTopologyImpl.java:853)
> 	at org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopologyImpl.initPartitions(GridDhtPartitionTopologyImpl.java:406)
> 	at org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopologyImpl.beforeExchange(GridDhtPartitionTopologyImpl.java:585)
> 	at org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture.distributedExchange(GridDhtPartitionsExchangeFuture.java:1470)
> 	at org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture.init(GridDhtPartitionsExchangeFuture.java:806)
> 	at org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$ExchangeWorker.body0(GridCachePartitionExchangeManager.java:2667)
> 	at org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$ExchangeWorker.body(GridCachePartitionExchangeManager.java:2539)
> 	at org.apache.ignite.internal.util.worker.GridWorker.run(GridWorker.java:120)
> 	at java.lang.Thread.run(Thread.java:748)
> [21:40:03,729][INFO][exchange-worker-#43][GridDhtPartitionsExchangeFuture] Finish exchange future [startVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], resVer=null, err=java.lang.IndexOutOfBoundsException: Index: 0, Size: 0]
> [21:40:03,734][INFO][exchange-worker-#43][GridDhtPartitionsExchangeFuture] Completed partition exchange [localNode=0c6a653d-b151-46b5-bd0c-7fea4b94ca26, exchange=GridDhtPartitionsExchangeFuture [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], evt=DISCOVERY_CUSTOM_EVT, evtNode=TcpDiscoveryNode [id=8a320d9a-c057-45e5-958a-6fff470c8485, addrs=ArrayList [127.0.0.1], sockAddrs=HashSet [/127.0.0.1:47500], discPort=47500, order=1, intOrder=1, lastExchangeTime=1540838375187, loc=false, ver=2.7.1#20181029-sha1:83c27088, isClient=false], done=true], topVer=null, durationFromInit=140]
> [21:40:03,739][SEVERE][exchange-worker-#43][GridCachePartitionExchangeManager] Failed to wait for completion of partition map exchange (preloading will not start): GridDhtPartitionsExchangeFuture [firstDiscoEvt=DiscoveryEvent [evtNode=TcpDiscoveryNode [id=8a320d9a-c057-45e5-958a-6fff470c8485, addrs=ArrayList [127.0.0.1], sockAddrs=HashSet [/127.0.0.1:47500], discPort=47500, order=1, intOrder=1, lastExchangeTime=1540838375187, loc=false, ver=2.7.1#20181029-sha1:83c27088, isClient=false], topVer=2, nodeId8=0c6a653d, msg=null, type=DISCOVERY_CUSTOM_EVT, tstamp=1540838403586]DiscoveryCustomEvent [customMsg=null, affTopVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], super=], crd=TcpDiscoveryNode [id=8a320d9a-c057-45e5-958a-6fff470c8485, addrs=ArrayList [127.0.0.1], sockAddrs=HashSet [/127.0.0.1:47500], discPort=47500, order=1, intOrder=1, lastExchangeTime=1540838375187, loc=false, ver=2.7.1#20181029-sha1:83c27088, isClient=false], exchId=GridDhtPartitionExchangeId [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], discoEvt=DiscoveryEvent [evtNode=TcpDiscoveryNode [id=8a320d9a-c057-45e5-958a-6fff470c8485, addrs=ArrayList [127.0.0.1], sockAddrs=HashSet [/127.0.0.1:47500], discPort=47500, order=1, intOrder=1, lastExchangeTime=1540838375187, loc=false, ver=2.7.1#20181029-sha1:83c27088, isClient=false], topVer=2, nodeId8=0c6a653d, msg=null, type=DISCOVERY_CUSTOM_EVT, tstamp=1540838403586]DiscoveryCustomEvent [customMsg=null, affTopVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], super=], nodeId=8a320d9a, evt=DISCOVERY_CUSTOM_EVT], added=true, initFut=GridFutureAdapter [ignoreInterrupts=false, state=DONE, res=false, hash=644854445], init=false, lastVer=null, partReleaseFut=AffinityTopologyVersion [topVer=2, minorTopVer=2]AffinityTopologyVersion [topVer=2, minorTopVer=2]AffinityTopologyVersion [topVer=2, minorTopVer=2]AffinityTopologyVersion [topVer=2, minorTopVer=2]AffinityTopologyVersion [topVer=2, minorTopVer=2]AffinityTopologyVersion [topVer=2, minorTopVer=2]AffinityTopologyVersion [topVer=2, minorTopVer=2]PartitionReleaseFuture [topVer=, futures=[ExplicitLockReleaseFuture [topVer=, futures=[]], AtomicUpdateReleaseFuture [topVer=, futures=[]], DataStreamerReleaseFuture [topVer=, futures=[]], LocalTxReleaseFuture [topVer=, futures=[]], AllTxReleaseFuture [topVer=, futures=[RemoteTxReleaseFuture [topVer=, futures=[]]]]]], exchActions=null, affChangeMsg=null, initTs=1540838403586, centralizedAff=false, forceAffReassignment=false, exchangeLocE=null, cacheChangeFailureMsgSent=false, done=true, state=SRV, registerCachesFuture=GridFutureAdapter [ignoreInterrupts=false, state=DONE, res=null, hash=2025498967], partitionsSent=false, partitionsReceived=false, delayedLatestMsg=null, afterLsnrCompleteFut=GridFutureAdapter [ignoreInterrupts=false, state=DONE, res=null, hash=1710701621], evtLatch=0, remaining=HashSet [8a320d9a-c057-45e5-958a-6fff470c8485], super=GridFutureAdapter [ignoreInterrupts=false, state=DONE, res=java.lang.IndexOutOfBoundsException: Index: 0, Size: 0, hash=2101396492]]
> class org.apache.ignite.IgniteCheckedException: Index: 0, Size: 0
> 	at org.apache.ignite.internal.util.IgniteUtils.cast(IgniteUtils.java:7428)
> 	at org.apache.ignite.internal.util.future.GridFutureAdapter.resolve(GridFutureAdapter.java:261)
> 	at org.apache.ignite.internal.util.future.GridFutureAdapter.get0(GridFutureAdapter.java:209)
> 	at org.apache.ignite.internal.util.future.GridFutureAdapter.get(GridFutureAdapter.java:160)
> 	at org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$ExchangeWorker.body0(GridCachePartitionExchangeManager.java:2695)
> 	at org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$ExchangeWorker.body(GridCachePartitionExchangeManager.java:2539)
> 	at org.apache.ignite.internal.util.worker.GridWorker.run(GridWorker.java:120)
> 	at java.lang.Thread.run(Thread.java:748)
> Caused by: java.lang.IndexOutOfBoundsException: Index: 0, Size: 0
> 	at java.util.ArrayList.rangeCheck(ArrayList.java:657)
> 	at java.util.ArrayList.get(ArrayList.java:433)
> 	at org.apache.ignite.internal.processors.cache.CacheGroupContext.singleCacheContext(CacheGroupContext.java:387)
> 	at org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition.<init>(GridDhtLocalPartition.java:200)
> 	at org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopologyImpl.getOrCreatePartition(GridDhtPartitionTopologyImpl.java:853)
> 	at org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopologyImpl.initPartitions(GridDhtPartitionTopologyImpl.java:406)
> 	at org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopologyImpl.beforeExchange(GridDhtPartitionTopologyImpl.java:585)
> 	at org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture.distributedExchange(GridDhtPartitionsExchangeFuture.java:1470)
> 	at org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture.init(GridDhtPartitionsExchangeFuture.java:806)
> 	at org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$ExchangeWorker.body0(GridCachePartitionExchangeManager.java:2667)
> 	... 3 more
> [21:40:03,742][SEVERE][exchange-worker-#43][] Critical system error detected. Will be handled accordingly to configured handler [hnd=StopNodeOrHaltFailureHandler [tryStop=false, timeout=0, super=AbstractFailureHandler [ignoredFailureTypes=UnmodifiableSet [SYSTEM_WORKER_BLOCKED]]], failureCtx=FailureContext [type=SYSTEM_WORKER_TERMINATION, err=class o.a.i.IgniteCheckedException: Index: 0, Size: 0]]
> class org.apache.ignite.IgniteCheckedException: Index: 0, Size: 0
> 	at org.apache.ignite.internal.util.IgniteUtils.cast(IgniteUtils.java:7428)
> 	at org.apache.ignite.internal.util.future.GridFutureAdapter.resolve(GridFutureAdapter.java:261)
> 	at org.apache.ignite.internal.util.future.GridFutureAdapter.get0(GridFutureAdapter.java:209)
> 	at org.apache.ignite.internal.util.future.GridFutureAdapter.get(GridFutureAdapter.java:160)
> 	at org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$ExchangeWorker.body0(GridCachePartitionExchangeManager.java:2695)
> 	at org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$ExchangeWorker.body(GridCachePartitionExchangeManager.java:2539)
> 	at org.apache.ignite.internal.util.worker.GridWorker.run(GridWorker.java:120)
> 	at java.lang.Thread.run(Thread.java:748)
> Caused by: java.lang.IndexOutOfBoundsException: Index: 0, Size: 0
> 	at java.util.ArrayList.rangeCheck(ArrayList.java:657)
> 	at java.util.ArrayList.get(ArrayList.java:433)
> 	at org.apache.ignite.internal.processors.cache.CacheGroupContext.singleCacheContext(CacheGroupContext.java:387)
> 	at org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition.<init>(GridDhtLocalPartition.java:200)
> 	at org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopologyImpl.getOrCreatePartition(GridDhtPartitionTopologyImpl.java:853)
> 	at org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopologyImpl.initPartitions(GridDhtPartitionTopologyImpl.java:406)
> 	at org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopologyImpl.beforeExchange(GridDhtPartitionTopologyImpl.java:585)
> 	at org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture.distributedExchange(GridDhtPartitionsExchangeFuture.java:1470)
> 	at org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture.init(GridDhtPartitionsExchangeFuture.java:806)
> 	at org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$ExchangeWorker.body0(GridCachePartitionExchangeManager.java:2667)
> 	... 3 more
> [21:40:03,744][WARNING][exchange-worker-#43][FailureProcessor] No deadlocked threads detected.
> [21:40:03,762][WARNING][exchange-worker-#43][FailureProcessor] Thread dump at 2018/10/29 21:40:03 MSK
> Thread [name="sys-#59", id=89, state=TIMED_WAITING, blockCnt=0, waitCnt=1]
>     Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@54fa4ca8, 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-#58", id=88, state=TIMED_WAITING, blockCnt=0, waitCnt=1]
>     Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@54fa4ca8, 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-#57", id=87, state=TIMED_WAITING, blockCnt=0, waitCnt=1]
>     Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@54fa4ca8, 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="Keep-Alive-Timer", id=86, state=TIMED_WAITING, blockCnt=0, waitCnt=5]
>         at java.lang.Thread.sleep(Native Method)
>         at sun.net.www.http.KeepAliveCache.run(KeepAliveCache.java:172)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="sys-#56", id=84, state=TIMED_WAITING, blockCnt=0, waitCnt=1]
>     Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@54fa4ca8, 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="srvc-deploy-#55", id=83, state=WAITING, blockCnt=0, waitCnt=2]
>     Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@67b82f40, 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="sys-#54", id=82, state=TIMED_WAITING, blockCnt=0, waitCnt=1]
>     Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@54fa4ca8, 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-#53", id=81, state=TIMED_WAITING, blockCnt=8, waitCnt=1]
>     Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@54fa4ca8, 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-#52", id=80, state=TIMED_WAITING, blockCnt=1, waitCnt=1]
>     Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@54fa4ca8, 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="checkpoint-runner-#51", id=79, state=WAITING, blockCnt=0, waitCnt=224]
>     Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@40b0b661, ownerName=null, ownerId=-1]
>         at sun.misc.Unsafe.park(Native Method)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
>         at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="checkpoint-runner-#50", id=78, state=WAITING, blockCnt=1, waitCnt=210]
>     Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@40b0b661, ownerName=null, ownerId=-1]
>         at sun.misc.Unsafe.park(Native Method)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
>         at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="checkpoint-runner-#49", id=77, state=WAITING, blockCnt=0, waitCnt=231]
>     Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@40b0b661, ownerName=null, ownerId=-1]
>         at sun.misc.Unsafe.park(Native Method)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
>         at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="checkpoint-runner-#48", id=76, state=WAITING, blockCnt=1, waitCnt=217]
>     Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@40b0b661, ownerName=null, ownerId=-1]
>         at sun.misc.Unsafe.park(Native Method)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
>         at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="db-checkpoint-thread-#47", id=75, state=TIMED_WAITING, blockCnt=0, waitCnt=93]
>     Lock [object=o.a.i.i.processors.cache.persistence.GridCacheDatabaseSharedManager$Checkpointer@5605c775, ownerName=null, ownerId=-1]
>         at java.lang.Object.wait(Native Method)
>         at o.a.i.i.processors.cache.persistence.GridCacheDatabaseSharedManager$Checkpointer.waitCheckpointEvent(GridCacheDatabaseSharedManager.java:3537)
>         at o.a.i.i.processors.cache.persistence.GridCacheDatabaseSharedManager$Checkpointer.body(GridCacheDatabaseSharedManager.java:3085)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="sys-#46", id=74, state=TIMED_WAITING, blockCnt=0, waitCnt=2]
>     Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@54fa4ca8, 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="wal-segment-syncer-#45", id=73, state=TIMED_WAITING, blockCnt=0, waitCnt=51]
>         at java.lang.Thread.sleep(Native Method)
>         at o.a.i.i.util.IgniteUtils.sleep(IgniteUtils.java:7774)
>         at o.a.i.i.processors.cache.persistence.wal.FileWriteAheadLogManager$WalSegmentSyncer.body(FileWriteAheadLogManager.java:3514)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="wal-file-archiver%null-#44", id=72, state=WAITING, blockCnt=0, waitCnt=1]
>     Lock [object=o.a.i.i.processors.cache.persistence.wal.aware.SegmentCurrentStateStorage@509c4e11, ownerName=null, ownerId=-1]
>         at java.lang.Object.wait(Native Method)
>         at java.lang.Object.wait(Object.java:502)
>         at o.a.i.i.processors.cache.persistence.wal.aware.SegmentCurrentStateStorage.awaitSegment(SegmentCurrentStateStorage.java:73)
>         at o.a.i.i.processors.cache.persistence.wal.aware.SegmentCurrentStateStorage.waitNextSegmentForArchivation(SegmentCurrentStateStorage.java:90)
>         at o.a.i.i.processors.cache.persistence.wal.aware.SegmentAware.waitNextSegmentForArchivation(SegmentAware.java:79)
>         at o.a.i.i.processors.cache.persistence.wal.FileWriteAheadLogManager$FileArchiver.body(FileWriteAheadLogManager.java:1719)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="session-timeout-worker-#34", id=50, state=TIMED_WAITING, blockCnt=0, waitCnt=29]
>         at java.lang.Thread.sleep(Native Method)
>         at o.a.i.i.processors.rest.GridRestProcessor$4.body(GridRestProcessor.java:495)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="exchange-worker-#43", id=70, state=RUNNABLE, blockCnt=0, waitCnt=20]
>         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:1364)
>         at o.a.i.i.processors.failure.FailureProcessor.process(FailureProcessor.java:132)
>         - locked o.a.i.i.processors.failure.FailureProcessor@76f8b96d
>         at o.a.i.i.processors.failure.FailureProcessor.process(FailureProcessor.java:108)
>         at o.a.i.i.processors.cache.GridCachePartitionExchangeManager$ExchangeWorker.body(GridCachePartitionExchangeManager.java:2555)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="ignite-update-notifier-timer", id=69, state=TIMED_WAITING, blockCnt=1, waitCnt=3]
>     Lock [object=java.util.TaskQueue@333e0a6d, ownerName=null, ownerId=-1]
>         at java.lang.Object.wait(Native Method)
>         at java.util.TimerThread.mainLoop(Timer.java:552)
>         at java.util.TimerThread.run(Timer.java:505)
> Thread [name="upd-ver-checker", id=68, state=TIMED_WAITING, blockCnt=0, waitCnt=6]
>         at java.lang.Thread.sleep(Native Method)
>         at o.a.i.i.processors.cluster.GridUpdateNotifier$1.run(GridUpdateNotifier.java:115)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="disco-event-worker-#42", id=67, state=WAITING, blockCnt=0, waitCnt=34]
>     Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@4bf84d10, 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:2856)
>         at o.a.i.i.managers.discovery.GridDiscoveryManager$DiscoveryWorker.body(GridDiscoveryManager.java:2825)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="tcp-disco-sock-reader-#4", id=66, 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 java.io.BufferedInputStream.fill(BufferedInputStream.java:246)
>         at java.io.BufferedInputStream.read1(BufferedInputStream.java:286)
>         at java.io.BufferedInputStream.read(BufferedInputStream.java:345)
>         - locked java.io.BufferedInputStream@79266eb8
>         at o.a.i.marshaller.jdk.JdkMarshallerInputStreamWrapper.read(JdkMarshallerInputStreamWrapper.java:53)
>         at java.io.ObjectInputStream$PeekInputStream.read(ObjectInputStream.java:2653)
>         at java.io.ObjectInputStream$PeekInputStream.readFully(ObjectInputStream.java:2669)
>         at java.io.ObjectInputStream$BlockDataInputStream.readShort(ObjectInputStream.java:3146)
>         at java.io.ObjectInputStream.readStreamHeader(ObjectInputStream.java:858)
>         at java.io.ObjectInputStream.<init>(ObjectInputStream.java:354)
>         at o.a.i.marshaller.jdk.JdkMarshallerObjectInputStream.<init>(JdkMarshallerObjectInputStream.java:43)
>         at o.a.i.marshaller.jdk.JdkMarshaller.unmarshal0(JdkMarshaller.java:137)
>         at o.a.i.marshaller.AbstractNodeNameAwareMarshaller.unmarshal(AbstractNodeNameAwareMarshaller.java:94)
>         at o.a.i.i.util.IgniteUtils.unmarshal(IgniteUtils.java:10085)
>         at o.a.i.spi.discovery.tcp.ServerImpl$SocketReader.body(ServerImpl.java:6246)
>         at o.a.i.spi.IgniteSpiThread.run(IgniteSpiThread.java:62)
> Thread [name="tcp-disco-srvr-#3", id=65, state=RUNNABLE, blockCnt=0, waitCnt=0]
>         at java.net.DualStackPlainSocketImpl.accept0(Native Method)
>         at java.net.DualStackPlainSocketImpl.socketAccept(DualStackPlainSocketImpl.java:131)
>         at java.net.AbstractPlainSocketImpl.accept(AbstractPlainSocketImpl.java:409)
>         at java.net.PlainSocketImpl.accept(PlainSocketImpl.java:199)
>         - locked java.net.SocksSocketImpl@4aa68320
>         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:5845)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at o.a.i.spi.discovery.tcp.ServerImpl$TcpServerThread.body(ServerImpl.java:5763)
>         at o.a.i.spi.IgniteSpiThread.run(IgniteSpiThread.java:62)
> Thread [name="tcp-disco-msg-worker-#2", id=64, state=TIMED_WAITING, blockCnt=0, waitCnt=4120]
>     Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@988957b, 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$MessageWorker.body(ServerImpl.java:7183)
>         at o.a.i.spi.discovery.tcp.ServerImpl$RingMessageWorker.body(ServerImpl.java:2700)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at o.a.i.spi.discovery.tcp.ServerImpl$MessageWorkerThread.body(ServerImpl.java:7119)
>         at o.a.i.spi.IgniteSpiThread.run(IgniteSpiThread.java:62)
> Thread [name="disco-notifier-worker-#41", id=63, state=WAITING, blockCnt=0, waitCnt=35]
>     Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@91d997, 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$DiscoveryMessageNotifierWorker.body0(GridDiscoveryManager.java:2660)
>         at o.a.i.i.managers.discovery.GridDiscoveryManager$DiscoveryMessageNotifierWorker.body(GridDiscoveryManager.java:2705)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="Thread-6", id=62, state=TIMED_WAITING, blockCnt=0, waitCnt=20]
>     Lock [object=java.util.concurrent.SynchronousQueue$TransferStack@4a882493, 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.SynchronousQueue$TransferStack.awaitFulfill(SynchronousQueue.java:460)
>         at java.util.concurrent.SynchronousQueue$TransferStack.transfer(SynchronousQueue.java:362)
>         at java.util.concurrent.SynchronousQueue.poll(SynchronousQueue.java:941)
>         at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1073)
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="Thread-5", id=61, state=RUNNABLE, blockCnt=1, waitCnt=0]
>         at sun.nio.ch.Iocp.getQueuedCompletionStatus(Native Method)
>         at sun.nio.ch.Iocp.access$300(Iocp.java:46)
>         at sun.nio.ch.Iocp$EventHandlerTask.run(Iocp.java:333)
>         at sun.nio.ch.AsynchronousChannelGroupImpl$1.run(AsynchronousChannelGroupImpl.java:112)
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>         at java.lang.Thread.run(Thread.java:748)
>     Locked synchronizers:
>         java.util.concurrent.ThreadPoolExecutor$Worker@361c294e
> Thread [name="Thread-4", id=60, state=RUNNABLE, blockCnt=0, waitCnt=0]
>         at sun.nio.ch.Iocp.getQueuedCompletionStatus(Native Method)
>         at sun.nio.ch.Iocp.access$300(Iocp.java:46)
>         at sun.nio.ch.Iocp$EventHandlerTask.run(Iocp.java:333)
>         at sun.nio.ch.AsynchronousChannelGroupImpl$1.run(AsynchronousChannelGroupImpl.java:112)
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>         at java.lang.Thread.run(Thread.java:748)
>     Locked synchronizers:
>         java.util.concurrent.ThreadPoolExecutor$Worker@4a8a60bc
> Thread [name="Thread-3", id=59, state=RUNNABLE, blockCnt=3, waitCnt=0]
>         at sun.nio.ch.Iocp.getQueuedCompletionStatus(Native Method)
>         at sun.nio.ch.Iocp.access$300(Iocp.java:46)
>         at sun.nio.ch.Iocp$EventHandlerTask.run(Iocp.java:333)
>         at sun.nio.ch.AsynchronousChannelGroupImpl$1.run(AsynchronousChannelGroupImpl.java:112)
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>         at java.lang.Thread.run(Thread.java:748)
>     Locked synchronizers:
>         java.util.concurrent.ThreadPoolExecutor$Worker@3a1d593e
> Thread [name="Thread-2", id=58, state=RUNNABLE, blockCnt=4, waitCnt=0]
>         at sun.nio.ch.Iocp.getQueuedCompletionStatus(Native Method)
>         at sun.nio.ch.Iocp.access$300(Iocp.java:46)
>         at sun.nio.ch.Iocp$EventHandlerTask.run(Iocp.java:333)
>         at sun.nio.ch.AsynchronousChannelGroupImpl$1.run(AsynchronousChannelGroupImpl.java:112)
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>         at java.lang.Thread.run(Thread.java:748)
>     Locked synchronizers:
>         java.util.concurrent.ThreadPoolExecutor$Worker@669513d8
> Thread [name="Thread-1", id=57, state=RUNNABLE, blockCnt=1, waitCnt=0]
>         at sun.nio.ch.Iocp.getQueuedCompletionStatus(Native Method)
>         at sun.nio.ch.Iocp.access$300(Iocp.java:46)
>         at sun.nio.ch.Iocp$EventHandlerTask.run(Iocp.java:333)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="grid-data-loader-flusher-#40", id=56, state=WAITING, blockCnt=0, waitCnt=1]
>     Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@34c91161, 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:109)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="grid-nio-worker-tcp-rest-3-#38", id=54, state=RUNNABLE, blockCnt=0, waitCnt=0]
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.poll0(Native Method)
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.poll(WindowsSelectorImpl.java:296)
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.access$400(WindowsSelectorImpl.java:278)
>         at sun.nio.ch.WindowsSelectorImpl.doSelect(WindowsSelectorImpl.java:159)
>         at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>         - locked o.a.i.i.util.nio.SelectedSelectionKeySet@792769ce
>         - locked java.util.Collections$UnmodifiableSet@77f45a1c
>         - locked sun.nio.ch.WindowsSelectorImpl@7c5354cd
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>         at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.bodyInternal(GridNioServer.java:2151)
>         at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.body(GridNioServer.java:1797)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="grid-nio-worker-tcp-rest-2-#37", id=53, state=RUNNABLE, blockCnt=0, waitCnt=0]
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.poll0(Native Method)
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.poll(WindowsSelectorImpl.java:296)
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.access$400(WindowsSelectorImpl.java:278)
>         at sun.nio.ch.WindowsSelectorImpl.doSelect(WindowsSelectorImpl.java:159)
>         at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>         - locked o.a.i.i.util.nio.SelectedSelectionKeySet@ed103f
>         - locked java.util.Collections$UnmodifiableSet@42c660f5
>         - locked sun.nio.ch.WindowsSelectorImpl@17b7f60b
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>         at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.bodyInternal(GridNioServer.java:2151)
>         at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.body(GridNioServer.java:1797)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="grid-nio-worker-tcp-rest-1-#36", id=52, state=RUNNABLE, blockCnt=0, waitCnt=0]
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.poll0(Native Method)
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.poll(WindowsSelectorImpl.java:296)
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.access$400(WindowsSelectorImpl.java:278)
>         at sun.nio.ch.WindowsSelectorImpl.doSelect(WindowsSelectorImpl.java:159)
>         at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>         - locked o.a.i.i.util.nio.SelectedSelectionKeySet@71243d0c
>         - locked java.util.Collections$UnmodifiableSet@1920b7ff
>         - locked sun.nio.ch.WindowsSelectorImpl@6890bde6
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>         at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.bodyInternal(GridNioServer.java:2151)
>         at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.body(GridNioServer.java:1797)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="grid-nio-worker-tcp-rest-0-#35", id=51, state=RUNNABLE, blockCnt=0, waitCnt=0]
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.poll0(Native Method)
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.poll(WindowsSelectorImpl.java:296)
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.access$400(WindowsSelectorImpl.java:278)
>         at sun.nio.ch.WindowsSelectorImpl.doSelect(WindowsSelectorImpl.java:159)
>         at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>         - locked o.a.i.i.util.nio.SelectedSelectionKeySet@403f1951
>         - locked java.util.Collections$UnmodifiableSet@10a39007
>         - locked sun.nio.ch.WindowsSelectorImpl@c2052a3
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>         at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.bodyInternal(GridNioServer.java:2151)
>         at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.body(GridNioServer.java:1797)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="nio-acceptor-tcp-rest-#39", id=55, state=RUNNABLE, blockCnt=0, waitCnt=0]
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.poll0(Native Method)
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.poll(WindowsSelectorImpl.java:296)
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.access$400(WindowsSelectorImpl.java:278)
>         at sun.nio.ch.WindowsSelectorImpl.doSelect(WindowsSelectorImpl.java:159)
>         at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>         - locked sun.nio.ch.Util$3@61b2b14c
>         - locked java.util.Collections$UnmodifiableSet@597fd9ba
>         - locked sun.nio.ch.WindowsSelectorImpl@55d0593a
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>         at o.a.i.i.util.nio.GridNioServer$GridNioAcceptWorker.accept(GridNioServer.java:2929)
>         at o.a.i.i.util.nio.GridNioServer$GridNioAcceptWorker.body(GridNioServer.java:2877)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="grid-nio-worker-client-listener-3-#32", id=48, state=RUNNABLE, blockCnt=0, waitCnt=0]
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.poll0(Native Method)
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.poll(WindowsSelectorImpl.java:296)
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.access$400(WindowsSelectorImpl.java:278)
>         at sun.nio.ch.WindowsSelectorImpl.doSelect(WindowsSelectorImpl.java:159)
>         at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>         - locked o.a.i.i.util.nio.SelectedSelectionKeySet@5bebbc69
>         - locked java.util.Collections$UnmodifiableSet@2f275e89
>         - locked sun.nio.ch.WindowsSelectorImpl@832f0de
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>         at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.bodyInternal(GridNioServer.java:2151)
>         at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.body(GridNioServer.java:1797)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="grid-nio-worker-client-listener-2-#31", id=47, state=RUNNABLE, blockCnt=0, waitCnt=0]
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.poll0(Native Method)
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.poll(WindowsSelectorImpl.java:296)
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.access$400(WindowsSelectorImpl.java:278)
>         at sun.nio.ch.WindowsSelectorImpl.doSelect(WindowsSelectorImpl.java:159)
>         at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>         - locked o.a.i.i.util.nio.SelectedSelectionKeySet@5f7c616b
>         - locked java.util.Collections$UnmodifiableSet@59729619
>         - locked sun.nio.ch.WindowsSelectorImpl@4c3458a8
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>         at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.bodyInternal(GridNioServer.java:2151)
>         at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.body(GridNioServer.java:1797)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="grid-nio-worker-client-listener-1-#30", id=46, state=RUNNABLE, blockCnt=0, waitCnt=0]
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.poll0(Native Method)
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.poll(WindowsSelectorImpl.java:296)
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.access$400(WindowsSelectorImpl.java:278)
>         at sun.nio.ch.WindowsSelectorImpl.doSelect(WindowsSelectorImpl.java:159)
>         at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>         - locked o.a.i.i.util.nio.SelectedSelectionKeySet@531ff5f0
>         - locked java.util.Collections$UnmodifiableSet@6fd4b1c1
>         - locked sun.nio.ch.WindowsSelectorImpl@22db307c
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>         at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.bodyInternal(GridNioServer.java:2151)
>         at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.body(GridNioServer.java:1797)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="grid-nio-worker-client-listener-0-#29", id=45, state=RUNNABLE, blockCnt=0, waitCnt=0]
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.poll0(Native Method)
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.poll(WindowsSelectorImpl.java:296)
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.access$400(WindowsSelectorImpl.java:278)
>         at sun.nio.ch.WindowsSelectorImpl.doSelect(WindowsSelectorImpl.java:159)
>         at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>         - locked o.a.i.i.util.nio.SelectedSelectionKeySet@4c44dd97
>         - locked java.util.Collections$UnmodifiableSet@6058119a
>         - locked sun.nio.ch.WindowsSelectorImpl@2a48eee9
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>         at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.bodyInternal(GridNioServer.java:2151)
>         at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.body(GridNioServer.java:1797)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="nio-acceptor-client-listener-#33", id=49, state=RUNNABLE, blockCnt=0, waitCnt=0]
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.poll0(Native Method)
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.poll(WindowsSelectorImpl.java:296)
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.access$400(WindowsSelectorImpl.java:278)
>         at sun.nio.ch.WindowsSelectorImpl.doSelect(WindowsSelectorImpl.java:159)
>         at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>         - locked sun.nio.ch.Util$3@51eb730c
>         - locked java.util.Collections$UnmodifiableSet@3b29b4c3
>         - locked sun.nio.ch.WindowsSelectorImpl@1bdda6fd
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>         at o.a.i.i.util.nio.GridNioServer$GridNioAcceptWorker.accept(GridNioServer.java:2929)
>         at o.a.i.i.util.nio.GridNioServer$GridNioAcceptWorker.body(GridNioServer.java:2877)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="tcp-comm-worker-#1", id=44, state=TIMED_WAITING, blockCnt=0, waitCnt=1]
>     Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@760f3cc9, 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:4287)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at o.a.i.spi.communication.tcp.TcpCommunicationSpi$5.body(TcpCommunicationSpi.java:2237)
>         at o.a.i.spi.IgniteSpiThread.run(IgniteSpiThread.java:62)
> Thread [name="grid-nio-worker-tcp-comm-3-#27", id=42, state=RUNNABLE, blockCnt=0, waitCnt=0]
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.poll0(Native Method)
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.poll(WindowsSelectorImpl.java:296)
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.access$400(WindowsSelectorImpl.java:278)
>         at sun.nio.ch.WindowsSelectorImpl.doSelect(WindowsSelectorImpl.java:159)
>         at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>         - locked o.a.i.i.util.nio.SelectedSelectionKeySet@7c764217
>         - locked java.util.Collections$UnmodifiableSet@a8f7ef6
>         - locked sun.nio.ch.WindowsSelectorImpl@7c93cb14
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>         at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.bodyInternal(GridNioServer.java:2151)
>         at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.body(GridNioServer.java:1797)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="grid-nio-worker-tcp-comm-2-#26", id=41, state=RUNNABLE, blockCnt=0, waitCnt=0]
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.poll0(Native Method)
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.poll(WindowsSelectorImpl.java:296)
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.access$400(WindowsSelectorImpl.java:278)
>         at sun.nio.ch.WindowsSelectorImpl.doSelect(WindowsSelectorImpl.java:159)
>         at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>         - locked o.a.i.i.util.nio.SelectedSelectionKeySet@73f5593b
>         - locked java.util.Collections$UnmodifiableSet@3ddddba8
>         - locked sun.nio.ch.WindowsSelectorImpl@4c547a2b
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>         at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.bodyInternal(GridNioServer.java:2151)
>         at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.body(GridNioServer.java:1797)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="grid-nio-worker-tcp-comm-1-#25", id=40, state=RUNNABLE, blockCnt=0, waitCnt=0]
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.poll0(Native Method)
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.poll(WindowsSelectorImpl.java:296)
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.access$400(WindowsSelectorImpl.java:278)
>         at sun.nio.ch.WindowsSelectorImpl.doSelect(WindowsSelectorImpl.java:159)
>         at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>         - locked o.a.i.i.util.nio.SelectedSelectionKeySet@2e7dd3de
>         - locked java.util.Collections$UnmodifiableSet@77187bab
>         - locked sun.nio.ch.WindowsSelectorImpl@244bfe7b
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>         at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.bodyInternal(GridNioServer.java:2151)
>         at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.body(GridNioServer.java:1797)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="grid-nio-worker-tcp-comm-0-#24", id=39, state=RUNNABLE, blockCnt=1, waitCnt=0]
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.poll0(Native Method)
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.poll(WindowsSelectorImpl.java:296)
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.access$400(WindowsSelectorImpl.java:278)
>         at sun.nio.ch.WindowsSelectorImpl.doSelect(WindowsSelectorImpl.java:159)
>         at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>         - locked o.a.i.i.util.nio.SelectedSelectionKeySet@4b214dfb
>         - locked java.util.Collections$UnmodifiableSet@b828462
>         - locked sun.nio.ch.WindowsSelectorImpl@3f73639a
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>         at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.bodyInternal(GridNioServer.java:2151)
>         at o.a.i.i.util.nio.GridNioServer$AbstractNioClientWorker.body(GridNioServer.java:1797)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="nio-acceptor-tcp-comm-#28", id=43, state=RUNNABLE, blockCnt=0, waitCnt=0]
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.poll0(Native Method)
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.poll(WindowsSelectorImpl.java:296)
>         at sun.nio.ch.WindowsSelectorImpl$SubSelector.access$400(WindowsSelectorImpl.java:278)
>         at sun.nio.ch.WindowsSelectorImpl.doSelect(WindowsSelectorImpl.java:159)
>         at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>         - locked sun.nio.ch.Util$3@4430fa29
>         - locked java.util.Collections$UnmodifiableSet@53f39c1
>         - locked sun.nio.ch.WindowsSelectorImpl@1a811c90
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>         at o.a.i.i.util.nio.GridNioServer$GridNioAcceptWorker.accept(GridNioServer.java:2929)
>         at o.a.i.i.util.nio.GridNioServer$GridNioAcceptWorker.body(GridNioServer.java:2877)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="grid-timeout-worker-#23", id=38, state=TIMED_WAITING, blockCnt=2, waitCnt=110]
>     Lock [object=java.lang.Object@3e4ad3e9, ownerName=null, ownerId=-1]
>         at java.lang.Object.wait(Native Method)
>         at o.a.i.i.processors.timeout.GridTimeoutProcessor$TimeoutWorker.body(GridTimeoutProcessor.java:269)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="ignite-clock", id=37, state=TIMED_WAITING, blockCnt=0, waitCnt=3594]
>         at java.lang.Thread.sleep(Native Method)
>         at o.a.i.i.util.IgniteUtils$22.run(IgniteUtils.java:3379)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="pub-#22", id=36, state=TIMED_WAITING, blockCnt=0, waitCnt=1]
>     Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@7de887bb, 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="pub-#21", id=35, state=TIMED_WAITING, blockCnt=0, waitCnt=1]
>     Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@7de887bb, 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="pub-#20", id=34, state=TIMED_WAITING, blockCnt=1, waitCnt=1]
>     Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@7de887bb, 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="pub-#19", id=33, state=TIMED_WAITING, blockCnt=1, waitCnt=1]
>     Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@7de887bb, 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="pub-#18", id=32, state=TIMED_WAITING, blockCnt=1, waitCnt=1]
>     Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@7de887bb, 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="pub-#17", id=31, state=TIMED_WAITING, blockCnt=1, waitCnt=1]
>     Lock [object=java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@7de887bb, 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="jvm-pause-detector-worker", id=30, state=TIMED_WAITING, blockCnt=0, waitCnt=738]
>         at java.lang.Thread.sleep(Native Method)
>         at o.a.i.i.LongJVMPauseDetector$1.run(LongJVMPauseDetector.java:100)
> Thread [name="data-streamer-stripe-7-#16", 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:669)
>         at o.a.i.i.util.StripedExecutor$Stripe.body(StripedExecutor.java:493)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="data-streamer-stripe-6-#15", 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:669)
>         at o.a.i.i.util.StripedExecutor$Stripe.body(StripedExecutor.java:493)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="data-streamer-stripe-5-#14", 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:669)
>         at o.a.i.i.util.StripedExecutor$Stripe.body(StripedExecutor.java:493)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="data-streamer-stripe-4-#13", id=26, 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:669)
>         at o.a.i.i.util.StripedExecutor$Stripe.body(StripedExecutor.java:493)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="data-streamer-stripe-3-#12", id=25, 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:669)
>         at o.a.i.i.util.StripedExecutor$Stripe.body(StripedExecutor.java:493)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="data-streamer-stripe-2-#11", id=24, 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:669)
>         at o.a.i.i.util.StripedExecutor$Stripe.body(StripedExecutor.java:493)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="data-streamer-stripe-1-#10", id=23, 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:669)
>         at o.a.i.i.util.StripedExecutor$Stripe.body(StripedExecutor.java:493)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="data-streamer-stripe-0-#9", 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:669)
>         at o.a.i.i.util.StripedExecutor$Stripe.body(StripedExecutor.java:493)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="sys-stripe-7-#8", 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:669)
>         at o.a.i.i.util.StripedExecutor$Stripe.body(StripedExecutor.java:493)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="sys-stripe-6-#7", 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:669)
>         at o.a.i.i.util.StripedExecutor$Stripe.body(StripedExecutor.java:493)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="sys-stripe-5-#6", 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:669)
>         at o.a.i.i.util.StripedExecutor$Stripe.body(StripedExecutor.java:493)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="sys-stripe-4-#5", 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:669)
>         at o.a.i.i.util.StripedExecutor$Stripe.body(StripedExecutor.java:493)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="sys-stripe-3-#4", 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:669)
>         at o.a.i.i.util.StripedExecutor$Stripe.body(StripedExecutor.java:493)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="sys-stripe-2-#3", 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:669)
>         at o.a.i.i.util.StripedExecutor$Stripe.body(StripedExecutor.java:493)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="sys-stripe-1-#2", id=15, 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:669)
>         at o.a.i.i.util.StripedExecutor$Stripe.body(StripedExecutor.java:493)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="sys-stripe-0-#1", id=14, 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:669)
>         at o.a.i.i.util.StripedExecutor$Stripe.body(StripedExecutor.java:493)
>         at o.a.i.i.util.worker.GridWorker.run(GridWorker.java:120)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="RMI TCP Accept-0", id=13, state=RUNNABLE, blockCnt=0, waitCnt=0]
>         at java.net.DualStackPlainSocketImpl.accept0(Native Method)
>         at java.net.DualStackPlainSocketImpl.socketAccept(DualStackPlainSocketImpl.java:131)
>         at java.net.AbstractPlainSocketImpl.accept(AbstractPlainSocketImpl.java:409)
>         at java.net.PlainSocketImpl.accept(PlainSocketImpl.java:199)
>         - locked java.net.SocksSocketImpl@16548f6
>         at java.net.ServerSocket.implAccept(ServerSocket.java:545)
>         at java.net.ServerSocket.accept(ServerSocket.java:513)
>         at sun.management.jmxremote.LocalRMIServerSocketFactory$1.accept(LocalRMIServerSocketFactory.java:52)
>         at sun.rmi.transport.tcp.TCPTransport$AcceptLoop.executeAcceptLoop(TCPTransport.java:400)
>         at sun.rmi.transport.tcp.TCPTransport$AcceptLoop.run(TCPTransport.java:372)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="RMI TCP Accept-49137", id=12, state=RUNNABLE, blockCnt=0, waitCnt=0]
>         at java.net.DualStackPlainSocketImpl.accept0(Native Method)
>         at java.net.DualStackPlainSocketImpl.socketAccept(DualStackPlainSocketImpl.java:131)
>         at java.net.AbstractPlainSocketImpl.accept(AbstractPlainSocketImpl.java:409)
>         at java.net.PlainSocketImpl.accept(PlainSocketImpl.java:199)
>         - locked java.net.SocksSocketImpl@1392b1d6
>         at java.net.ServerSocket.implAccept(ServerSocket.java:545)
>         at java.net.ServerSocket.accept(ServerSocket.java:513)
>         at sun.rmi.transport.tcp.TCPTransport$AcceptLoop.executeAcceptLoop(TCPTransport.java:400)
>         at sun.rmi.transport.tcp.TCPTransport$AcceptLoop.run(TCPTransport.java:372)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="RMI TCP Accept-0", id=11, state=RUNNABLE, blockCnt=0, waitCnt=0]
>         at java.net.DualStackPlainSocketImpl.accept0(Native Method)
>         at java.net.DualStackPlainSocketImpl.socketAccept(DualStackPlainSocketImpl.java:131)
>         at java.net.AbstractPlainSocketImpl.accept(AbstractPlainSocketImpl.java:409)
>         at java.net.PlainSocketImpl.accept(PlainSocketImpl.java:199)
>         - locked java.net.SocksSocketImpl@7b2fedd6
>         at java.net.ServerSocket.implAccept(ServerSocket.java:545)
>         at java.net.ServerSocket.accept(ServerSocket.java:513)
>         at sun.rmi.transport.tcp.TCPTransport$AcceptLoop.executeAcceptLoop(TCPTransport.java:400)
>         at sun.rmi.transport.tcp.TCPTransport$AcceptLoop.run(TCPTransport.java:372)
>         at java.lang.Thread.run(Thread.java:748)
> Thread [name="Attach Listener", id=5, state=RUNNABLE, blockCnt=0, waitCnt=0]
> Thread [name="Signal Dispatcher", id=4, state=RUNNABLE, blockCnt=0, waitCnt=0]
> Thread [name="Finalizer", id=3, state=WAITING, blockCnt=123, waitCnt=5]
>     Lock [object=java.lang.ref.ReferenceQueue$Lock@42c37857, 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=6, waitCnt=4]
>     Lock [object=java.lang.ref.Reference$Lock@69caba01, 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=WAITING, blockCnt=2, waitCnt=5]
>     Lock [object=java.util.concurrent.CountDownLatch$Sync@6fd0baef, 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.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:997)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304)
>         at java.util.concurrent.CountDownLatch.await(CountDownLatch.java:231)
>         at o.a.i.startup.cmdline.CommandLineStartup.main(CommandLineStartup.java:334)
> [21:40:03,783][SEVERE][exchange-worker-#43][] JVM will be halted immediately due to the failure: [failureCtx=FailureContext [type=SYSTEM_WORKER_TERMINATION, err=class o.a.i.IgniteCheckedException: Index: 0, Size: 0]]
> {noformat}



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