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

Anton Vinogradov commented on IGNITE-8783:
------------------------------------------

[~ilantukh],
 4 problems related to ExchangeLatch hang found:

1) pendingAcks was ignored at client latch recreation on coordinator change. 
+Fixed+.
{noformat}
// There is final ack for created latch.
if (pendingAcks.containsKey(latchId)) {
{noformat}
was replaced with
{noformat}
Set<UUID> nodeIds = pendingAcks.get(latchId);

// There is final ack for created latch.
if (nodeIds != null && nodeIds.contains(coordinator)) {
{noformat}
2) Topology change could cause coordinator change even in case coordinator node 
not failed. +Fixed+.
 added sorting by order to {{getLatchCoordinator}}
{noformat}
.sorted(Comparator.comparing(ClusterNode::order))
{noformat}
Now coordinator is alwais oldest node.

3) Sometimes Latch fails on message send in case connection was not established 
yet.
{noformat}
[2018-07-13 
19:06:43,910][ERROR][exchange-worker-#233015%partitioned.GridCachePartitionedDataStructuresFailoverSelfTest4%][TcpCommunicationSpi]
 Failed to send message to remote node [node=TcpDiscoveryNode 
[id=3838f6ed-1b4d-484d-9773-df4493700000, addrs=[127.0.0.1], 
sockAddrs=[/127.0.0.1:47500], discPort=47500, order=1, intOrder=1, 
lastExchangeTime=1531498003891, loc=false, ver=2.6.0#19700101-sha1:00000000, 
isClient=false], msg=GridIoMessage [plc=2, topic=TOPIC_EXCHANGE, topicOrd=31, 
ordered=false, timeout=0, skipOnTimeout=false, 
msg=org.apache.ignite.internal.processors.cache.distributed.dht.preloader.latch.LatchAckMessage@77e7c81f]]
class org.apache.ignite.IgniteCheckedException: Failed to connect to node (is 
node still alive?). Make sure that each ComputeTask and cache Transaction has a 
timeout set in order to prevent parties from waiting forever in case of network 
issues [nodeId=3838f6ed-1b4d-484d-9773-df4493700000, addrs=[/127.0.0.1:45010]]
        at 
org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi.createTcpClient(TcpCommunicationSpi.java:3449)
        at 
org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi.createNioClient(TcpCommunicationSpi.java:2977)
        at 
org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi.reserveClient(TcpCommunicationSpi.java:2860)
        at 
org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi.sendMessage0(TcpCommunicationSpi.java:2703)
        at 
org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi.sendMessage(TcpCommunicationSpi.java:2662)
        at 
org.apache.ignite.internal.managers.communication.GridIoManager.send(GridIoManager.java:1643)
        at 
org.apache.ignite.internal.managers.communication.GridIoManager.sendToGridTopic(GridIoManager.java:1715)
        at 
org.apache.ignite.internal.processors.cache.distributed.dht.preloader.latch.ExchangeLatchManager$ClientLatch.sendAck(ExchangeLatchManager.java:624)
        at 
org.apache.ignite.internal.processors.cache.distributed.dht.preloader.latch.ExchangeLatchManager$ClientLatch.countDown(ExchangeLatchManager.java:642)
        at 
org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture.waitPartitionRelease(GridDhtPartitionsExchangeFuture.java:1406)
        at 
org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture.distributedExchange(GridDhtPartitionsExchangeFuture.java:1177)
        at 
org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture.init(GridDhtPartitionsExchangeFuture.java:732)
        at 
org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$ExchangeWorker.body0(GridCachePartitionExchangeManager.java:2477)
        at 
org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$ExchangeWorker.body(GridCachePartitionExchangeManager.java:2357)
        at 
org.apache.ignite.internal.util.worker.GridWorker.run(GridWorker.java:110)
        at java.lang.Thread.run(Thread.java:748)
        Suppressed: class org.apache.ignite.IgniteCheckedException: Failed to 
connect to address [addr=/127.0.0.1:45010, err=Address already in use: no 
further information]
                at 
org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi.createTcpClient(TcpCommunicationSpi.java:3452)
                ... 15 more
        Caused by: java.net.BindException: Address already in use: no further 
information
                at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
                at 
sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
                at sun.nio.ch.SocketAdaptor.connect(SocketAdaptor.java:111)
                at 
org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi.createTcpClient(TcpCommunicationSpi.java:3289)
                ... 15 more
[2018-07-13 19:06:43,911][INFO 
][exchange-worker-#233015%partitioned.GridCachePartitionedDataStructuresFailoverSelfTest4%][GridDhtPartitionsExchangeFuture]
 Finished waiting for partition release future [topVer=AffinityTopologyVersion 
[topVer=4, minorTopVer=0], waitTime=0ms, futInfo=NA]
[2018-07-13 
19:06:43,911][ERROR][exchange-worker-#233015%partitioned.GridCachePartitionedDataStructuresFailoverSelfTest4%][ExchangeLatchManager]
 Failed to send ack [latch=exchange-AffinityTopologyVersion [topVer=4, 
minorTopVer=0], to=3838f6ed-1b4d-484d-9773-df4493700000]: Failed to send 
message (node may have left the grid or TCP connection cannot be established 
due to firewall issues) [node=TcpDiscoveryNode 
[id=3838f6ed-1b4d-484d-9773-df4493700000, addrs=[127.0.0.1], 
sockAddrs=[/127.0.0.1:47500], discPort=47500, order=1, intOrder=1, 
lastExchangeTime=1531498003891, loc=false, ver=2.6.0#19700101-sha1:00000000, 
isClient=false], topic=TOPIC_EXCHANGE, 
msg=org.apache.ignite.internal.processors.cache.distributed.dht.preloader.latch.LatchAckMessage@77e7c81f,
 policy=2]
[2018-07-13 19:06:43,913][INFO 
][grid-nio-worker-tcp-comm-2-#232820%partitioned.GridCachePartitionedDataStructuresFailoverSelfTest0%][TcpCommunicationSpi]
 Accepted incoming communication connection [locAddr=/127.0.0.1:45010, 
rmtAddr=/127.0.0.1:62577]
[2018-07-13 19:06:43,913][INFO 
][grid-nio-worker-tcp-comm-0-#232990%partitioned.GridCachePartitionedDataStructuresFailoverSelfTest4%][TcpCommunicationSpi]
 Established outgoing communication connection [locAddr=/127.0.0.1:62577, 
rmtAddr=/127.0.0.1:45010]
{noformat}
Seems, we have some race/bug at TcpCommunicationSpi?

4) 
{{GridCachePartitionedDataStructuresFailoverSelfTest#testReentrantLockConstantTopologyChangeNonFailoverSafe}}
 can hang in case of broken tx
{noformat}
 Pending transactions:
[2018-07-15 14:13:41,210][WARN 
][exchange-worker-#1596354%partitioned.GridCachePartitionedDataStructuresFailoverSelfTest1%][diagnostic]
 >>> [txVer=AffinityTopologyVersion [topVer=7, minorTopVer=0], exchWait=true, 
tx=GridDhtTxLocal [nearNodeId=1392b1bd-c807-4479-9bfe-fc9f70500000, 
nearFutId=14ffca0a461-999e75d0-a333-4bd6-a2a2-7f143d0af773, nearMiniId=1, 
nearFinFutId=null, nearFinMiniId=0, nearXidVer=GridCacheVersion 
[topVer=143133203, order=1531653200153, nodeOrder=1], 
super=GridDhtTxLocalAdapter [nearOnOriginatingNode=false, nearNodes=[], 
dhtNodes=[], explicitLock=false, super=IgniteTxLocalAdapter 
[completedBase=null, sndTransformedVals=false, depEnabled=false, 
txState=IgniteTxStateImpl [activeCacheIds=[1968300681], recovery=false, 
txMap=[IgniteTxEntry [key=KeyCacheObjectImpl [part=494, 
val=GridCacheInternalKeyImpl [name=structure, 
grpName=default-volatile-ds-group], hasValBytes=true], cacheId=1968300681, 
txKey=IgniteTxKey [key=KeyCacheObjectImpl [part=494, 
val=GridCacheInternalKeyImpl [name=structure, 
grpName=default-volatile-ds-group], hasValBytes=true], cacheId=1968300681], 
val=[op=NOOP, val=null], prevVal=[op=NOOP, val=null], oldVal=[op=NOOP, 
val=null], entryProcessorsCol=null, ttl=-1, conflictExpireTime=-1, 
conflictVer=null, explicitVer=null, dhtVer=null, filters=[], 
filtersPassed=false, filtersSet=false, entry=GridDhtCacheEntry [rdrs=[], 
part=494, super=GridDistributedCacheEntry [super=GridCacheMapEntry 
[key=KeyCacheObjectImpl [part=494, val=GridCacheInternalKeyImpl 
[name=structure, grpName=default-volatile-ds-group], hasValBytes=true], 
val=CacheObjectImpl [val=null, hasValBytes=true], ver=GridCacheVersion 
[topVer=143133201, order=1531653200154, nodeOrder=2], hash=2095426867, 
extras=GridCacheMvccEntryExtras [mvcc=GridCacheMvcc 
[locs=[GridCacheMvccCandidate [nodeId=1bf28b00-feed-412b-a20b-ca9fc1100001, 
ver=GridCacheVersion [topVer=143133203, order=1531653200157, nodeOrder=2], 
threadId=1947290, id=31143709, topVer=AffinityTopologyVersion [topVer=7, 
minorTopVer=0], reentry=null, otherNodeId=1392b1bd-c807-4479-9bfe-fc9f70500000, 
otherVer=GridCacheVersion [topVer=143133203, order=1531653200153, nodeOrder=1], 
mappedDhtNodes=null, mappedNearNodes=null, ownerVer=null, serOrder=null, 
key=KeyCacheObjectImpl [part=494, val=GridCacheInternalKeyImpl [name=structure, 
grpName=default-volatile-ds-group], hasValBytes=true], 
masks=local=1|owner=1|ready=1|reentry=0|used=0|tx=1|single_implicit=0|dht_local=1|near_local=0|removed=0|read=0,
 prevVer=null, nextVer=null]], rmts=null]], flags=2]]], prepared=0, 
locked=false, nodeId=null, locMapped=false, expiryPlc=null, 
transferExpiryPlc=false, flags=0, partUpdateCntr=0, serReadVer=null, 
xidVer=GridCacheVersion [topVer=143133203, order=1531653200157, 
nodeOrder=2]]]], super=IgniteTxAdapter [xidVer=GridCacheVersion 
[topVer=143133203, order=1531653200157, nodeOrder=2], writeVer=null, 
implicit=false, loc=true, threadId=1947290, startTime=1531653200578, 
nodeId=1bf28b00-feed-412b-a20b-ca9fc1100001, startVer=GridCacheVersion 
[topVer=143133203, order=1531653200157, nodeOrder=2], endVer=null, 
isolation=REPEATABLE_READ, concurrency=PESSIMISTIC, timeout=0, 
sysInvalidate=false, sys=true, plc=2, commitVer=null, finalizing=NONE, 
invalidParts=null, state=ACTIVE, timedOut=false, topVer=AffinityTopologyVersion 
[topVer=7, minorTopVer=0], duration=20632ms, onePhaseCommit=false], size=1]]]]
{noformat}
__
 So, I propose to merge #1 & #2 (this solves 95% hangs), and create issues for 
#3 & #4.

TC checked:
 - 
[https://ci.ignite.apache.org/viewType.html?buildTypeId=IgniteTests24Java8_DataStructures&tab=buildTypeStatusDiv&branch_IgniteTests24Java8=ignite-8783]
 - 
[https://ci.ignite.apache.org/project.html?projectId=IgniteTests24Java8&branch_IgniteTests24Java8=ignite-8783]

> Failover tests periodically cause hanging of the whole Data Structures suite 
> on TC
> ----------------------------------------------------------------------------------
>
>                 Key: IGNITE-8783
>                 URL: https://issues.apache.org/jira/browse/IGNITE-8783
>             Project: Ignite
>          Issue Type: Bug
>          Components: data structures
>            Reporter: Ivan Rakov
>            Assignee: Anton Vinogradov
>            Priority: Major
>              Labels: MakeTeamcityGreenAgain
>
> History of suite runs: 
> https://ci.ignite.apache.org/viewType.html?buildTypeId=IgniteTests24Java8_DataStructures&tab=buildTypeHistoryList&branch_IgniteTests24Java8=%3Cdefault%3E
> Chance of suite hang is 18% in master (based on previous 50 runs).
> Hang is always caused by one of the following failover tests:
> {noformat}
> GridCacheReplicatedDataStructuresFailoverSelfTest#testAtomicSequenceConstantTopologyChange
> GridCachePartitionedDataStructuresFailoverSelfTest#testFairReentrantLockConstantTopologyChangeNonFailoverSafe
> {noformat}



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

Reply via email to