Cluster went down after "Unable to await partitions release latch within timeout" WARN

classic Classic list List threaded Threaded
6 messages Options
ihalilaltun ihalilaltun
Reply | Threaded
Open this post in threaded view
|

Cluster went down after "Unable to await partitions release latch within timeout" WARN

This post was updated on .
Hi There Igniters,

We had a very strange cluster behivour while creating new caches on the fly.
Just after caches are created we start get following warnings from all
cluster nodes, including coordinator node;

[2019-09-27T15:00:17,727][WARN
][exchange-worker-#219][GridDhtPartitionsExchangeFuture] Unable to await
partitions release latch within timeout: ServerLatch [permits=1,
pendingAcks=[3561ac09-6752-4e2e-8279-d975c268d045], super=CompletableLatch
[id=exchange, topVer=AffinityTopologyVersion [topVer=92, minorTopVer=2]]]

After a while all client nodes are seemed to disconnected from cluster with
no logs on clients' side.

Coordinator node has many logs like;
2019-09-27T15:00:03,124][WARN
][sys-#337823][GridDhtPartitionsExchangeFuture] Partition states validation
has failed for group: acc_1306acd07be78000_userPriceDrop. Partitions cache
sizes are inconsistent for Part 129:
[9497f1c4-13bd-4f90-bbf7-be7371cea22f=757
1486cd47-7d40-400c-8e36-b66947865602=2427 ] Part 138:
[1486cd47-7d40-400c-8e36-b66947865602=2463
f9cf594b-24f2-4a91-8d84-298c97eb0f98=736 ] Part 156:
[b7782803-10da-45d8-b042-b5b4a880eb07=672
9f0c2155-50a4-4147-b444-5cc002cf6f5d=2414 ] Part 284:
[b7782803-10da-45d8-b042-b5b4a880eb07=690
1486cd47-7d40-400c-8e36-b66947865602=1539 ] Part 308:
[1486cd47-7d40-400c-8e36-b66947865602=2401
7750e2f1-7102-4da2-9a9d-ea202f73905a=706 ] Part 362:
[1486cd47-7d40-400c-8e36-b66947865602=2387
7750e2f1-7102-4da2-9a9d-ea202f73905a=697 ] Part 434:
[53c253e1-ccbe-4af1-a3d6-178523023c8b=681
1486cd47-7d40-400c-8e36-b66947865602=1541 ] Part 499:
[1486cd47-7d40-400c-8e36-b66947865602=2505
7750e2f1-7102-4da2-9a9d-ea202f73905a=699 ] Part 622:
[1486cd47-7d40-400c-8e36-b66947865602=2436
e97a0f3f-3175-49f7-a476-54eddd59d493=662 ] Part 662:
[b7782803-10da-45d8-b042-b5b4a880eb07=686
1486cd47-7d40-400c-8e36-b66947865602=2445 ] Part 699:
[1486cd47-7d40-400c-8e36-b66947865602=2427
f9cf594b-24f2-4a91-8d84-298c97eb0f98=646 ] Part 827:
[62a05754-3f3a-4dc8-b0fa-53c0a0a0da63=703
1486cd47-7d40-400c-8e36-b66947865602=1549 ] Part 923:
[1486cd47-7d40-400c-8e36-b66947865602=2434
a9e9eaba-d227-4687-8c6c-7ed522e6c342=706 ] Part 967:
[62a05754-3f3a-4dc8-b0fa-53c0a0a0da63=673
1486cd47-7d40-400c-8e36-b66947865602=1595 ] Part 976:
[33301384-3293-417f-b94a-ed36ebc82583=666
1486cd47-7d40-400c-8e36-b66947865602=2384 ]

Coordinator's log and one of the cluster node's log is attached.
coordinator_log.gz
<http://apache-ignite-users.70518.x6.nabble.com/file/t2515/coordinator_log.gz
cluster_node_log.gz
<http://apache-ignite-users.70518.x6.nabble.com/file/t2515/cluster_node_log.gz

Any help/comment is appriciated.

Thanks.





-----
İbrahim Halil Altun
Senior Software Engineer @ Segmentify
--
Sent from: http://apache-ignite-users.70518.x6.nabble.com/
İbrahim Halil Altun
Senior Software Engineer @ Segmentify
Pavel Kovalenko Pavel Kovalenko
Reply | Threaded
Open this post in threaded view
|

Re: Cluster went down after "Unable to await partitions release latch within timeout" WARN

Hi Ibrahim,

I see that one node didn't send acknowledgment during cache creation:
[2019-09-27T15:00:17,727][WARN ][exchange-worker-#219][GridDhtPartitionsExchangeFuture] Unable to await partitions release latch within timeout: ServerLatch [permits=1, pendingAcks=[3561ac09-6752-4e2e-8279-d975c268d045], super=CompletableLatch [id=exchange, topVer=AffinityTopologyVersion [topVer=92, minorTopVer=2]]]

Do you have any logs from a node with id = "3561ac09-6752-4e2e-8279-d975c268d045".
You can find this node by grepping the following "locNodeId=3561ac09-6752-4e2e-8279-d975c268d045" like in line:
[2019-09-27T15:24:03,532][INFO ][main][TcpDiscoverySpi] Successfully bound to TCP port [port=47500, localHost=0.0.0.0/0.0.0.0, locNodeId=70b49e00-5b9f-4459-9055-a05ce358be10]
 

ср, 9 окт. 2019 г. в 17:34, ihalilaltun <[hidden email]>:
Hi There Igniters,

We had a very strange cluster behivour while creating new caches on the fly.
Just after caches are created we start get following warnings from all
cluster nodes, including coordinator node;

[2019-09-27T15:00:17,727][WARN
][exchange-worker-#219][GridDhtPartitionsExchangeFuture] Unable to await
partitions release latch within timeout: ServerLatch [permits=1,
pendingAcks=[3561ac09-6752-4e2e-8279-d975c268d045], super=CompletableLatch
[id=exchange, topVer=AffinityTopologyVersion [topVer=92, minorTopVer=2]]]

After a while all client nodes are seemed to disconnected from cluster with
no logs on clients' side.

Coordinator node has many logs like;
2019-09-27T15:00:03,124][WARN
][sys-#337823][GridDhtPartitionsExchangeFuture] Partition states validation
has failed for group: acc_1306acd07be78000_userPriceDrop. Partitions cache
sizes are inconsistent for Part 129:
[9497f1c4-13bd-4f90-bbf7-be7371cea22f=757
1486cd47-7d40-400c-8e36-b66947865602=2427 ] Part 138:
[1486cd47-7d40-400c-8e36-b66947865602=2463
f9cf594b-24f2-4a91-8d84-298c97eb0f98=736 ] Part 156:
[b7782803-10da-45d8-b042-b5b4a880eb07=672
9f0c2155-50a4-4147-b444-5cc002cf6f5d=2414 ] Part 284:
[b7782803-10da-45d8-b042-b5b4a880eb07=690
1486cd47-7d40-400c-8e36-b66947865602=1539 ] Part 308:
[1486cd47-7d40-400c-8e36-b66947865602=2401
7750e2f1-7102-4da2-9a9d-ea202f73905a=706 ] Part 362:
[1486cd47-7d40-400c-8e36-b66947865602=2387
7750e2f1-7102-4da2-9a9d-ea202f73905a=697 ] Part 434:
[53c253e1-ccbe-4af1-a3d6-178523023c8b=681
1486cd47-7d40-400c-8e36-b66947865602=1541 ] Part 499:
[1486cd47-7d40-400c-8e36-b66947865602=2505
7750e2f1-7102-4da2-9a9d-ea202f73905a=699 ] Part 622:
[1486cd47-7d40-400c-8e36-b66947865602=2436
e97a0f3f-3175-49f7-a476-54eddd59d493=662 ] Part 662:
[b7782803-10da-45d8-b042-b5b4a880eb07=686
1486cd47-7d40-400c-8e36-b66947865602=2445 ] Part 699:
[1486cd47-7d40-400c-8e36-b66947865602=2427
f9cf594b-24f2-4a91-8d84-298c97eb0f98=646 ] Part 827:
[62a05754-3f3a-4dc8-b0fa-53c0a0a0da63=703
1486cd47-7d40-400c-8e36-b66947865602=1549 ] Part 923:
[1486cd47-7d40-400c-8e36-b66947865602=2434
a9e9eaba-d227-4687-8c6c-7ed522e6c342=706 ] Part 967:
[62a05754-3f3a-4dc8-b0fa-53c0a0a0da63=673
1486cd47-7d40-400c-8e36-b66947865602=1595 ] Part 976:
[33301384-3293-417f-b94a-ed36ebc82583=666
1486cd47-7d40-400c-8e36-b66947865602=2384 ]

Coordinator's log and one of the cluster node's log is attached.
coordinator_log.gz
<http://apache-ignite-users.70518.x6.nabble.com/file/t2515/coordinator_log.gz
cluster_node_log.gz
<http://apache-ignite-users.70518.x6.nabble.com/file/t2515/cluster_node_log.gz

Any help/comment is appriciated.

Thanks.





-----
İbrahim Halil Altun
Senior Software Engineer @ Segmentify
--
Sent from: http://apache-ignite-users.70518.x6.nabble.com/
Pavel Kovalenko Pavel Kovalenko
Reply | Threaded
Open this post in threaded view
|

Re: Cluster went down after "Unable to await partitions release latch within timeout" WARN

Ibrahim,

Could you please also share the cache configuration that is used for dynamic creation?

чт, 10 окт. 2019 г. в 19:09, Pavel Kovalenko <[hidden email]>:
Hi Ibrahim,

I see that one node didn't send acknowledgment during cache creation:
[2019-09-27T15:00:17,727][WARN ][exchange-worker-#219][GridDhtPartitionsExchangeFuture] Unable to await partitions release latch within timeout: ServerLatch [permits=1, pendingAcks=[3561ac09-6752-4e2e-8279-d975c268d045], super=CompletableLatch [id=exchange, topVer=AffinityTopologyVersion [topVer=92, minorTopVer=2]]]

Do you have any logs from a node with id = "3561ac09-6752-4e2e-8279-d975c268d045".
You can find this node by grepping the following "locNodeId=3561ac09-6752-4e2e-8279-d975c268d045" like in line:
[2019-09-27T15:24:03,532][INFO ][main][TcpDiscoverySpi] Successfully bound to TCP port [port=47500, localHost=0.0.0.0/0.0.0.0, locNodeId=70b49e00-5b9f-4459-9055-a05ce358be10]
 

ср, 9 окт. 2019 г. в 17:34, ihalilaltun <[hidden email]>:
Hi There Igniters,

We had a very strange cluster behivour while creating new caches on the fly.
Just after caches are created we start get following warnings from all
cluster nodes, including coordinator node;

[2019-09-27T15:00:17,727][WARN
][exchange-worker-#219][GridDhtPartitionsExchangeFuture] Unable to await
partitions release latch within timeout: ServerLatch [permits=1,
pendingAcks=[3561ac09-6752-4e2e-8279-d975c268d045], super=CompletableLatch
[id=exchange, topVer=AffinityTopologyVersion [topVer=92, minorTopVer=2]]]

After a while all client nodes are seemed to disconnected from cluster with
no logs on clients' side.

Coordinator node has many logs like;
2019-09-27T15:00:03,124][WARN
][sys-#337823][GridDhtPartitionsExchangeFuture] Partition states validation
has failed for group: acc_1306acd07be78000_userPriceDrop. Partitions cache
sizes are inconsistent for Part 129:
[9497f1c4-13bd-4f90-bbf7-be7371cea22f=757
1486cd47-7d40-400c-8e36-b66947865602=2427 ] Part 138:
[1486cd47-7d40-400c-8e36-b66947865602=2463
f9cf594b-24f2-4a91-8d84-298c97eb0f98=736 ] Part 156:
[b7782803-10da-45d8-b042-b5b4a880eb07=672
9f0c2155-50a4-4147-b444-5cc002cf6f5d=2414 ] Part 284:
[b7782803-10da-45d8-b042-b5b4a880eb07=690
1486cd47-7d40-400c-8e36-b66947865602=1539 ] Part 308:
[1486cd47-7d40-400c-8e36-b66947865602=2401
7750e2f1-7102-4da2-9a9d-ea202f73905a=706 ] Part 362:
[1486cd47-7d40-400c-8e36-b66947865602=2387
7750e2f1-7102-4da2-9a9d-ea202f73905a=697 ] Part 434:
[53c253e1-ccbe-4af1-a3d6-178523023c8b=681
1486cd47-7d40-400c-8e36-b66947865602=1541 ] Part 499:
[1486cd47-7d40-400c-8e36-b66947865602=2505
7750e2f1-7102-4da2-9a9d-ea202f73905a=699 ] Part 622:
[1486cd47-7d40-400c-8e36-b66947865602=2436
e97a0f3f-3175-49f7-a476-54eddd59d493=662 ] Part 662:
[b7782803-10da-45d8-b042-b5b4a880eb07=686
1486cd47-7d40-400c-8e36-b66947865602=2445 ] Part 699:
[1486cd47-7d40-400c-8e36-b66947865602=2427
f9cf594b-24f2-4a91-8d84-298c97eb0f98=646 ] Part 827:
[62a05754-3f3a-4dc8-b0fa-53c0a0a0da63=703
1486cd47-7d40-400c-8e36-b66947865602=1549 ] Part 923:
[1486cd47-7d40-400c-8e36-b66947865602=2434
a9e9eaba-d227-4687-8c6c-7ed522e6c342=706 ] Part 967:
[62a05754-3f3a-4dc8-b0fa-53c0a0a0da63=673
1486cd47-7d40-400c-8e36-b66947865602=1595 ] Part 976:
[33301384-3293-417f-b94a-ed36ebc82583=666
1486cd47-7d40-400c-8e36-b66947865602=2384 ]

Coordinator's log and one of the cluster node's log is attached.
coordinator_log.gz
<http://apache-ignite-users.70518.x6.nabble.com/file/t2515/coordinator_log.gz
cluster_node_log.gz
<http://apache-ignite-users.70518.x6.nabble.com/file/t2515/cluster_node_log.gz

Any help/comment is appriciated.

Thanks.





-----
İbrahim Halil Altun
Senior Software Engineer @ Segmentify
--
Sent from: http://apache-ignite-users.70518.x6.nabble.com/
ihalilaltun ihalilaltun
Reply | Threaded
Open this post in threaded view
|

Re: Cluster went down after "Unable to await partitions release latch within timeout" WARN

Hi Pavel,

Here is the logs from node with localId:3561ac09-6752-4e2e-8279-d975c268d045
ignite-2019-10-06.gz
<http://apache-ignite-users.70518.x6.nabble.com/file/t2515/ignite-2019-10-06.gz>  

cache creation is done with java code on our side, we use getOrCreateCache
method, here is the piece of code how we configure and create caches;

...
ignite.getOrCreateCache(getCommonCacheConfigurationForAccount(accountId,
initCacheType));

private <K, V> CacheConfiguration<K, V>
getCommonCacheConfigurationForAccount(String accountId, IgniteCacheType
cacheType) {
        CacheConfiguration<K, V> cacheConfiguration = new
CacheConfiguration<>();
       
cacheConfiguration.setName(accountId.concat(cacheType.getCacheNameSuffix()));
        if (cacheType.isSqlTable()) {
            cacheConfiguration.setIndexedTypes(cacheType.getKeyClass(),
cacheType.getValueClass());
            cacheConfiguration.setSqlSchema(accountId);
            cacheConfiguration.setSqlEscapeAll(true);
        }
        cacheConfiguration.setEventsDisabled(true);
        cacheConfiguration.setStoreKeepBinary(true);
        cacheConfiguration.setAtomicityMode(CacheAtomicityMode.ATOMIC);
        cacheConfiguration.setBackups(1);
        if (!cacheType.getCacheGroupName().isEmpty()) {
            cacheConfiguration.setGroupName(cacheType.getCacheGroupName());
        }
        if (cacheType.getExpiryDurationInDays().getDurationAmount() > 0) {
           
cacheConfiguration.setExpiryPolicyFactory(TouchedExpiryPolicy.factoryOf(cacheType.getExpiryDurationInDays()));
        }
        return cacheConfiguration;
    }



-----
İbrahim Halil Altun
Senior Software Engineer @ Segmentify
--
Sent from: http://apache-ignite-users.70518.x6.nabble.com/
İbrahim Halil Altun
Senior Software Engineer @ Segmentify
Pavel Kovalenko Pavel Kovalenko
Reply | Threaded
Open this post in threaded view
|

Re: Cluster went down after "Unable to await partitions release latch within timeout" WARN

Ibrahim,

I've checked logs and found the following issue:
[2019-09-27T15:00:06,164][ERROR][sys-stripe-32-#33][atomic] Received message without registered handler (will ignore)
[msg=GridDhtAtomicDeferredUpdateResponse [futIds=GridLongList [idx=1, arr=[6389728]]], node=e39bd72e-acee-48a7-ad45-2019dfff9df4, locTopVer=AffinityTopologyVersion [topVer=92, minorTopVer=1], ...

This response was needed to complete (finish) AtomicUpdateFuture:
[2019-09-27T15:00:36,287][WARN ][exchange-worker-#219][diagnostic] >>> GridDhtAtomicSingleUpdateFuture [allUpdated=true, super=GridDhtAtomicAbstractUpdateFuture [futId=6389728, resCnt=0, addedReader=false, dhtRes={e39bd72e-acee-48a7-ad45-2019dfff9df4=[res=false, size=1, nearSize=0]}]]

During exchange, all nodes wait for atomic updates and transaction completion and then send an acknowledgment to the coordinator to continue processing exchange.
Because atomic update on that node was not finished, the node didn't send the acknowledgement to the coordinator and that's why you have seen messages like:
[2019-09-27T15:00:17,727][WARN ][exchange-worker-#219][GridDhtPartitionsExchangeFuture] Unable to await partitions release latch within timeout: ServerLatch [permits=1, pendingAcks=[3561ac09-6752-4e2e-8279-d975c268d045], super=CompletableLatch [id=exchange, topVer=AffinityTopologyVersion [topVer=92, minorTopVer=2]]]

The handler to complete AtomicUpdateFuture was not found due to the concurrency issue in 2.7.6 codebase. There is a map that contains handlers for cache messages:
org/apache/ignite/internal/processors/cache/GridCacheIoManager.java:1575
In 2.7.6 it's just HashMap with volatile read/write publishing. However, because of improper synchronization with adding and getting a handler in rare cases, it can lead to false-positive missing a handler for a message that you may see in logs.
This issue was fixed at https://issues.apache.org/jira/browse/IGNITE-8006 which will be in 2.8 release.
However, if it's critical, you can make a hotfix by yourself:
Checkout ignite-2.7.6 branch from https://github.com/apache/ignite
Change HashMap declaration to ConcurrentHashMap here:
org/apache/ignite/internal/processors/cache/GridCacheIoManager.java:1575
Rebuild ignite-core module and deploy new ignite-core-jar on your server nodes.
This hotfix will work for your case.

Another option is you can use the last version of GridGain Community Edition instead of Apache Ignite which is fully compatible with Ignite.

Regarding message:
[sys-#337823][GridDhtPartitionsExchangeFuture] Partition states validation
has failed for group: acc_1306acd07be78000_userPriceDrop. Partitions cachesizes are inconsistent for Part 129

I see that you create caches with ExpiryPolicy. If you use expiry policies you can have different partition sizes on primary-backup nodes, because expiring is not synchronized and performed independently on different nodes.
So it's OK to see such warnings. They are false-positive. Such warning messages will not be printed if a cache has an expiry policy set. That was fixed in https://issues.apache.org/jira/browse/IGNITE-12206


пт, 11 окт. 2019 г. в 14:40, ihalilaltun <[hidden email]>:
Hi Pavel,

Here is the logs from node with localId:3561ac09-6752-4e2e-8279-d975c268d045
ignite-2019-10-06.gz
<http://apache-ignite-users.70518.x6.nabble.com/file/t2515/ignite-2019-10-06.gz

cache creation is done with java code on our side, we use getOrCreateCache
method, here is the piece of code how we configure and create caches;

...
ignite.getOrCreateCache(getCommonCacheConfigurationForAccount(accountId,
initCacheType));

private <K, V> CacheConfiguration<K, V>
getCommonCacheConfigurationForAccount(String accountId, IgniteCacheType
cacheType) {
        CacheConfiguration<K, V> cacheConfiguration = new
CacheConfiguration<>();

cacheConfiguration.setName(accountId.concat(cacheType.getCacheNameSuffix()));
        if (cacheType.isSqlTable()) {
            cacheConfiguration.setIndexedTypes(cacheType.getKeyClass(),
cacheType.getValueClass());
            cacheConfiguration.setSqlSchema(accountId);
            cacheConfiguration.setSqlEscapeAll(true);
        }
        cacheConfiguration.setEventsDisabled(true);
        cacheConfiguration.setStoreKeepBinary(true);
        cacheConfiguration.setAtomicityMode(CacheAtomicityMode.ATOMIC);
        cacheConfiguration.setBackups(1);
        if (!cacheType.getCacheGroupName().isEmpty()) {
            cacheConfiguration.setGroupName(cacheType.getCacheGroupName());
        }
        if (cacheType.getExpiryDurationInDays().getDurationAmount() > 0) {

cacheConfiguration.setExpiryPolicyFactory(TouchedExpiryPolicy.factoryOf(cacheType.getExpiryDurationInDays()));
        }
        return cacheConfiguration;
    }



-----
İbrahim Halil Altun
Senior Software Engineer @ Segmentify
--
Sent from: http://apache-ignite-users.70518.x6.nabble.com/
ihalilaltun ihalilaltun
Reply | Threaded
Open this post in threaded view
|

Re: Cluster went down after "Unable to await partitions release latch within timeout" WARN

Hi Pavel,

Thank you for detailed explanation. We are discussing hotfix with
management, but i think decision will be negative :(

I think we'll have to wait 2.8 release, which seems to be released on
January 17, 2020. I hope we'll have this issue by then.

Regards.



-----
İbrahim Halil Altun
Senior Software Engineer @ Segmentify
--
Sent from: http://apache-ignite-users.70518.x6.nabble.com/
İbrahim Halil Altun
Senior Software Engineer @ Segmentify