You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@ignite.apache.org by ihalilaltun <ib...@segmentify.com> on 2019/10/09 14:34:25 UTC

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

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/

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

Posted by ihalilaltun <ib...@segmentify.com>.
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/

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

Posted by userx <ga...@gmail.com>.
Hi Pavel,

The exchange finished taking its time, but during that time, new client was
not able to write to the cache.

So what happened was that

There were 4 Ignite servers out of a bunch of 19 (as you can see in the
consistentids) in my message above,  that their acknowledgement to
Coordinator node was pending because they possibly were finishing some
Atomic updates or transactions. This almost went for 2 hours. During those 2
hours, clients tried to activate 

if (ignite == null) {
        Ignition.setClientMode(true);
        String fileName = getRelevantFileName();
        ignite = Ignition.start(fileName);
      }
      ignite.cluster().active(true);

But the activation couldnt happen. For this task we have a timeout of 5
minutes. If this doesnt happen client gives up unless the next time it needs
to create a cache.

So when i talk about clients, say they are just individual java processes
running.





--
Sent from: http://apache-ignite-users.70518.x6.nabble.com/

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

Posted by Pavel Kovalenko <jo...@gmail.com>.
Hello,

I don't clearly understand from your message, but have the exchange finally
finished? Or you were getting this WARN message all the time?

пт, 1 мая 2020 г. в 12:32, Ilya Kasnacheev <il...@gmail.com>:

> Hello!
>
> This description sounds like a typical hanging Partition Map Exchange, but
> you should be able to see that in logs.
> If you don't, you can collect thread dumps from all nodes with jstack and
> check it for any stalling operations (or share with us).
>
> Regards,
> --
> Ilya Kasnacheev
>
>
> пт, 1 мая 2020 г. в 11:53, userx <ga...@gmail.com>:
>
>> Hi Pavel,
>>
>> I am using 2.8 and still getting the same issue. Here is the ecosystem
>>
>> 19 Ignite servers (S1 to S19) running at 16GB of max JVM and in persistent
>> mode.
>>
>> 96 Clients (C1 to C96)
>>
>> There are 19 machines, 1 Ignite server is started on 1 machine. The
>> clients
>> are evenly distributed across machines.
>>
>> C19 tries to create a cache, it gets a timeout exception as i have 5 mins
>> of
>> timeout. When I looked into the coordinator logs, between a span of 5
>> minutes, it gets the messages
>>
>>
>> 2020-04-24 15:37:09,434 WARN [exchange-worker-#45%S1%] {}
>>
>> org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture
>> - Unable to await partitions release latch within timeout. Some nodes have
>> not sent acknowledgement for latch completion. It's possible due to
>> unfinishined atomic updates, transactions or not released explicit locks
>> on
>> that nodes. Please check logs for errors on nodes with ids reported in
>> latch
>> `pendingAcks` collection [latch=ServerLatch [permits=4,
>> pendingAcks=HashSet
>> [84b8416c-fa06-4544-9ce0-e3dfba41038a,
>> 19bd7744-0ced-4123-a35f-ddf0cf9f55c4,
>> 533af8f9-c0f6-44b6-92d4-658f86ffaca0,
>> 1b31cb25-abbc-4864-88a3-5a4df37a0cf4],
>> super=CompletableLatch [id=CompletableLatchUid [id=exchange,
>> topVer=AffinityTopologyVersion [topVer=174, minorTopVer=1]]]]]
>>
>> And the 4 nodes which have not been able to acknowledge latch completion
>> are
>> S14, S7, S18, S4
>>
>> I went to see the logs of S4, it just records the addition of C19 into
>> topology and then C19 leaving it after 5 minutes. The only thing is that
>> in
>> GC I see this consistently "Total time for which application threads were
>> stopped: 0.0006225 seconds, Stopping threads took: 0.0000887 seconds"
>>
>> I understand that until the time all the atomic updates and transactions
>> are
>> finished Clients are not able to create caches by communicating with
>> Coordinator but is there a way around ?
>>
>> So the question is that is it still prevalent on 2.8 ?
>>
>>
>>
>>
>>
>>
>>
>>
>>
>> --
>> Sent from: http://apache-ignite-users.70518.x6.nabble.com/
>>
>

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

Posted by Ilya Kasnacheev <il...@gmail.com>.
Hello!

This description sounds like a typical hanging Partition Map Exchange, but
you should be able to see that in logs.
If you don't, you can collect thread dumps from all nodes with jstack and
check it for any stalling operations (or share with us).

Regards,
-- 
Ilya Kasnacheev


пт, 1 мая 2020 г. в 11:53, userx <ga...@gmail.com>:

> Hi Pavel,
>
> I am using 2.8 and still getting the same issue. Here is the ecosystem
>
> 19 Ignite servers (S1 to S19) running at 16GB of max JVM and in persistent
> mode.
>
> 96 Clients (C1 to C96)
>
> There are 19 machines, 1 Ignite server is started on 1 machine. The clients
> are evenly distributed across machines.
>
> C19 tries to create a cache, it gets a timeout exception as i have 5 mins
> of
> timeout. When I looked into the coordinator logs, between a span of 5
> minutes, it gets the messages
>
>
> 2020-04-24 15:37:09,434 WARN [exchange-worker-#45%S1%] {}
>
> org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture
> - Unable to await partitions release latch within timeout. Some nodes have
> not sent acknowledgement for latch completion. It's possible due to
> unfinishined atomic updates, transactions or not released explicit locks on
> that nodes. Please check logs for errors on nodes with ids reported in
> latch
> `pendingAcks` collection [latch=ServerLatch [permits=4, pendingAcks=HashSet
> [84b8416c-fa06-4544-9ce0-e3dfba41038a,
> 19bd7744-0ced-4123-a35f-ddf0cf9f55c4,
> 533af8f9-c0f6-44b6-92d4-658f86ffaca0,
> 1b31cb25-abbc-4864-88a3-5a4df37a0cf4],
> super=CompletableLatch [id=CompletableLatchUid [id=exchange,
> topVer=AffinityTopologyVersion [topVer=174, minorTopVer=1]]]]]
>
> And the 4 nodes which have not been able to acknowledge latch completion
> are
> S14, S7, S18, S4
>
> I went to see the logs of S4, it just records the addition of C19 into
> topology and then C19 leaving it after 5 minutes. The only thing is that in
> GC I see this consistently "Total time for which application threads were
> stopped: 0.0006225 seconds, Stopping threads took: 0.0000887 seconds"
>
> I understand that until the time all the atomic updates and transactions
> are
> finished Clients are not able to create caches by communicating with
> Coordinator but is there a way around ?
>
> So the question is that is it still prevalent on 2.8 ?
>
>
>
>
>
>
>
>
>
> --
> Sent from: http://apache-ignite-users.70518.x6.nabble.com/
>

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

Posted by userx <ga...@gmail.com>.
Hi Pavel,

I am using 2.8 and still getting the same issue. Here is the ecosystem 

19 Ignite servers (S1 to S19) running at 16GB of max JVM and in persistent
mode.

96 Clients (C1 to C96)

There are 19 machines, 1 Ignite server is started on 1 machine. The clients
are evenly distributed across machines.

C19 tries to create a cache, it gets a timeout exception as i have 5 mins of
timeout. When I looked into the coordinator logs, between a span of 5
minutes, it gets the messages 


2020-04-24 15:37:09,434 WARN [exchange-worker-#45%S1%] {}
org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture
- Unable to await partitions release latch within timeout. Some nodes have
not sent acknowledgement for latch completion. It's possible due to
unfinishined atomic updates, transactions or not released explicit locks on
that nodes. Please check logs for errors on nodes with ids reported in latch
`pendingAcks` collection [latch=ServerLatch [permits=4, pendingAcks=HashSet
[84b8416c-fa06-4544-9ce0-e3dfba41038a, 19bd7744-0ced-4123-a35f-ddf0cf9f55c4,
533af8f9-c0f6-44b6-92d4-658f86ffaca0, 1b31cb25-abbc-4864-88a3-5a4df37a0cf4],
super=CompletableLatch [id=CompletableLatchUid [id=exchange,
topVer=AffinityTopologyVersion [topVer=174, minorTopVer=1]]]]]

And the 4 nodes which have not been able to acknowledge latch completion are
S14, S7, S18, S4

I went to see the logs of S4, it just records the addition of C19 into
topology and then C19 leaving it after 5 minutes. The only thing is that in
GC I see this consistently "Total time for which application threads were
stopped: 0.0006225 seconds, Stopping threads took: 0.0000887 seconds"

I understand that until the time all the atomic updates and transactions are
finished Clients are not able to create caches by communicating with
Coordinator but is there a way around ?

So the question is that is it still prevalent on 2.8 ?









--
Sent from: http://apache-ignite-users.70518.x6.nabble.com/

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

Posted by Pavel Kovalenko <jo...@gmail.com>.
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 <ib...@segmentify.com>:

> 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/
>

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

Posted by ihalilaltun <ib...@segmentify.com>.
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/

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

Posted by Pavel Kovalenko <jo...@gmail.com>.
Ibrahim,

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

чт, 10 окт. 2019 г. в 19:09, Pavel Kovalenko <jo...@gmail.com>:

> 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 <ib...@segmentify.com>:
>
>> 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/
>>
>

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

Posted by Pavel Kovalenko <jo...@gmail.com>.
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 <ib...@segmentify.com>:

> 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/
>