Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

CP Subsystem/Raft: Instability when restarting members #24897

Open
lprimak opened this issue Jun 24, 2023 · 24 comments
Open

CP Subsystem/Raft: Instability when restarting members #24897

lprimak opened this issue Jun 24, 2023 · 24 comments
Labels
Source: Community PR or issue was opened by a community user Type: Defect

Comments

@lprimak
Copy link
Contributor

lprimak commented Jun 24, 2023

Describe the bug
When restarting a CP members, there are number of stability issues encountered when members go up/ down, restart, freeze, or crash.
The theme is the same, an infinite loop trying to select a new leader and failing due to looking for long-ago-dropped members.

Expected behavior
CP cluster keeps being stable

** Issues (expanded) **

  • CP subsystem is left in an unrecoverable state when only one node remains in the CP group (contractAndReExpandRaftGroup_leaveOneRunning() in Tests for CP Membership restart issues #24903 ). Looks like the last properly-shutdown member is not removed from the Raft Metadata group (bug?)
  • When a CP member is auto-removed (low auto-remove value), but is actually temporarily frozen, an infinite leader selection loop occurs when split-brain is healed (CPMemberSplitBrainTest in Tests for CP Membership restart issues #24903). I believe that in this scenario, the member should be re-instated, because otherwise the cluster remains in an unrecoverable state.
  • 100% CPU usage trying to elect a new leader when one group member is frozen (probably due to the same issue, but not 100% sure). I was not able to make a test-based reproducer for this. However, by running CacheTester https://github.com/flowlogix/hazelcast-issues/ and Ctrl-Z one of 3 instances, the problem always appears.
  • Member will not shut down (waits for 2+ minutes before forcefully terminating) when only one member of CP group remains. The result of this is that cluster can't be shut down cleanly. CP reset won't work since it requires 3 members. Looks like the (now-famous) Raft metadata leadership vote infinite loop is to blame. This happens when the master node is shut down first.

To Reproduce
PR #24903 has the additional tests

Exception
After the node rejoins, infinite loop trying to select a new CP leader is seen:

WARNING: [127.0.0.1]:5711 [dev] [5.4.0-SNAPSHOT] Retrying invocation: Invocation{op=com.hazelcast.cp.internal.operation.RaftQueryOp{serviceName='hz:core:raft', identityHash=1989326097, partitionId=0, replicaIndex=0, callId=464, invocationTime=1687454514104 (2023-06-22 13:21:54.104), waitTimeout=-1, callTimeout=60000, tenantControl=com.hazelcast.spi.impl.tenantcontrol.NoopTenantControl@0, op=com.hazelcast.cp.internal.raftop.metadata.GetActiveRaftGroupByNameOp{serviceName='hz:core:raft', groupName=default}, groupId=CPGroupId{name='METADATA', seed=0, groupId=0}, policy=LINEARIZABLE}, tryCount=250, tryPauseMillis=500, invokeCount=180, callTimeoutMillis=60000, firstInvocationTimeMs=1687454430092, firstInvocationTime='2023-06-22 13:20:30.092', lastHeartbeatMillis=0, lastHeartbeatTime='1969-12-31 19:00:00.000', targetAddress=null, targetMember=null, memberListVersion=0, pendingResponse={VOID}, backupsAcksExpected=-1, backupsAcksReceived=0, connection=null}, Reason: com.hazelcast.spi.exception.WrongTargetException: WrongTarget! local: Member [127.0.0.1]:5711 - 118e38ef-b2ae-4539-b8a9-93a49497fb58 this, expected-target: null, partitionId: 0, replicaIndex: 0, operation: com.hazelcast.cp.internal.operation.RaftQueryOp, service: hz:core:raft

Additional context
See slack conversations:
https://hazelcastcommunity.slack.com/archives/C015Q2TUBKL/p1687454583340749
https://hazelcastcommunity.slack.com/archives/C015Q2TUBKL/p1687530351955629

@lprimak lprimak changed the title CP Subsystem/Raft: member fail to rejoin groups CP Subsystem/Raft: members fail to rejoin groups Jun 26, 2023
@vbekiaris
Copy link
Contributor

CP membership, unlike Hazelcast cluster membership, is not dynamic. As discussed in CP subsystem management

Unlike the dynamic nature of Hazelcast clusters, CP Subsystem requires manual intervention while expanding/shrinking its size, or when a CP member crashes or becomes unreachable

So that's the expected behaviour of CP subsystem. In the context of your test, adding an explicit call to promote hz4 instance is required to get CP members count to 3.

    CompletionStage promotionFuture = hz4.getCPSubsystem().getCPSubsystemManagementService().promoteToCPMember();
    // normally you would handle possible failures as discussed in promoteToCPMember javadoc
    promotionFuture.toCompletableFuture().join();

@arodionov
Copy link
Contributor

Hi @lprimak
Please take a look at a similar test:

@Test
public void testExpandRaftGroup() throws ExecutionException, InterruptedException, TimeoutException {
HazelcastInstance[] instances = newInstances(3, 3, 1);
instances[0].shutdown();
instances[3].getCPSubsystem().getCPSubsystemManagementService().promoteToCPMember()
.toCompletableFuture().get(30, TimeUnit.SECONDS);
CPGroupId metadataGroupId = getMetadataGroupId(instances[1]);
CPGroup group = instances[1].getCPSubsystem().getCPSubsystemManagementService().getCPGroup(METADATA_CP_GROUP_NAME)
.toCompletableFuture().get();
assertEquals(3, group.members().size());
Collection<CPMember> members = group.members();
assertTrue(members.contains(instances[3].getCPSubsystem().getLocalCPMember()));
assertTrueEventually(() -> assertNotNull(getRaftNode(instances[3], metadataGroupId)));
}

@lprimak
Copy link
Contributor Author

lprimak commented Jun 26, 2023

Thank you!
This was quite buried in the docs :)
Is there room for enhancement here? I would think in a more 'realistic' dynamic clusters (think k8s) there is room for automation.
Perhaps an 'auto-upgrade' configuration parameter that 'fills' the missing slots in CP groups automatically?

@lprimak lprimak changed the title CP Subsystem/Raft: members fail to rejoin groups CP Subsystem/Raft: Instability when restarting members Jun 27, 2023
@lprimak
Copy link
Contributor Author

lprimak commented Jun 27, 2023

I have created (with great difficulty) code that work correctly to keep CP going while restarting nodes.
https://github.com/flowlogix/hazelcast-issues/blob/master/CacheTester/src/main/java/com/flowlogix/hazelcast/cachetester/CacheTester.java#L143
However, I've encountered more issues. I am in the process of recording them here by editing the description.
While the code above works well, I believe the underlying bugs still need to be fixed.

@lprimak
Copy link
Contributor Author

lprimak commented Jun 27, 2023

In general, the way I am testing this by running https://github.com/flowlogix/hazelcast-issues/ CacheTester, which is an interactive tool that exercises locking and maps (Hazelcast in general).
You can Ctrl-Z it and that can test many failure scenarios!
This is particularly good for uncovering all sorts of bugs :)

@lprimak
Copy link
Contributor Author

lprimak commented Jun 27, 2023

Apparently, I am not the only one who is having this issue. Hopefully the test reproducers can shed some more light on this.

@arodionov
Copy link
Contributor

arodionov commented Jun 27, 2023

This behaviour is by design. We shouldn’t allow the restarted member to join back the cluster automatically, because the cluster thinks that the old member is still present in a cluster, but temporarily became unavailable.
If you would like to have an automated way that restarted members joining back to the cluster you should use CP persistent feature (that preserves the member state).

@arodionov
Copy link
Contributor

arodionov commented Jun 27, 2023

Perhaps an 'auto-upgrade' configuration parameter that 'fills' the missing slots in CP groups automatically?

We have such an option - after 4 hours if the missed member is not joined back to the cluster, it will be auto-removed and will be replaced by an additional member.
Please take a look https://docs.hazelcast.com/hazelcast/5.3/cp-subsystem/configuration#remove

@arodionov
Copy link
Contributor

I would think in a more 'realistic' dynamic clusters (think k8s) there is room for automation.

Yes, we have a task in our backlog, to align k8s pods auto-restart and scaling up/down with CP members removal and promotion

@lprimak
Copy link
Contributor Author

lprimak commented Jun 27, 2023

Thanks for your feedback @arodionov

This behaviour is by design. We shouldn’t allow the restarted member to join back the cluster automatically

My current CP use case is only for CP FencedLock. Therefore, the persistence feature does not bring any value.
I wrote the code to auto-promote to CP, however, it's not trivial and IMHO should be a Hazelcast feature. (see #24904)

because the cluster thinks that the old member is still present in a cluster, but temporarily became unavailable.

Above is incorrect, because nodes that shut down normally do not re-join the CP cluster upon restart either.
In fact, when, in a 3-node CP cluster, one shuts down, I can see there are only 2 nodes present in the logs.
When the 3rd restarts, it will not rejoin by default.

We have such an option - after 4 hours if the missed member is not joined back to the cluster, it will be auto-removed

There is an issue with this. See the description (#24897 (comment)) - bullet point 2

Yes, we have a task in our backlog, to align k8s pods auto-restart and scaling up/down with CP members removal and promotion

Can you link to the issue and / or PR so I can test it out?

The main critical issue is still (see description, bullet point 1) that cluster becomes unusable and needs to be restarted if CP goes down to one member (reproducer tests in #24903)

@arodionov
Copy link
Contributor

arodionov commented Jun 27, 2023

In fact, when, in a 3-node CP cluster, one shuts down, I can see there are only 2 nodes present in the logs.

Yes, only in the case of a so-called graceful shutdown the CP member will be automatically removed from the CP subsystem, and in logs, there should be 2 CP members:

CP Group Members {groupId: default(6894), size:2, term:1, logIndex:4} [
	CPMember{uuid=791e078d-ef0e-4f56-9139-e03f607c9e4f, address=[192.168.1.13]:5703} - LEADER
	CPMember{uuid=da402e4e-a353-47a7-934c-0189c735f35c, address=[192.168.1.13]:5701} - FOLLOWER this
]

But if a CP member has been terminated or stopped responding, we can't distinguish if it's temporarily unavailable (due to network problems) or if it has crashed. In that case, in logs, there will be the only information about cluster status, but the CP subsystem will consider that this CP member is temporarily unavailable:

Members {size:2, ver:4} [
	Member [192.168.1.13]:5701 - da402e4e-a353-47a7-934c-0189c735f35c this
	Member [192.168.1.13]:5703 - 791e078d-ef0e-4f56-9139-e03f607c9e4f
] 

Therefore, the persistence feature does not bring any value.

Yes, it will. The CP Persistence only persists the CP member's state in the form of a Raft log (there will be Meta-information about CP members, FencedLock owner, Raft sessions, etc)

@arodionov
Copy link
Contributor

The main critical issue is still (see description, bullet point 1) that cluster becomes unusable and needs to be restarted if CP goes down to one member (reproducer tests in #24903)

This is also by design - if a cluster lost the majority of its members it will be blocked and should be recovered manually https://docs.hazelcast.com/hazelcast/5.3/cp-subsystem/management#handling-a-lost-majority

@lprimak
Copy link
Contributor Author

lprimak commented Jun 27, 2023

This is also by design - if a cluster lost the majority of its members it will be blocked and should be recovered manually

Just tested this scenario, and there is no "majority lost" or "availability decreased" events delivered in case of normal shutdown of two other members, just "member removed" message. I think you are referring to scenario if two other members "crash" The scenario I am talking about is normal shutdown of members, not crashes. This should not be any different from "initial state" of CP and should be able to accept new members.

Member removed by proper shutdown (one node left):

INFO: [127.0.0.1]:5710 [dev] [5.3.1] Shutdown request of Member [127.0.0.1]:5711 - e7713c4e-4a72-4efd-b3ee-080ea2b36bb2 is handled
**** memberRemoved: CPMembershipEvent{type=REMOVED, member=CPMember{uuid=e7713c4e-4a72-4efd-b3ee-080ea2b36bb2, address=[127.0.0.1]:5711}}

However, CP subsystem is unusable and still tries to find the now-shutdown member:

INFO: [127.0.0.1]:5710 [dev] [5.3.1] Partition balance is ok, no need to repartition.
Jun 28, 2023 11:56:45 AM com.hazelcast.cp.internal.raft.impl.RaftNodeImpl$LeaderFailureDetectionTask(METADATA)
WARNING: [127.0.0.1]:5710 [dev] [5.3.1] Current leader RaftEndpoint{uuid='e7713c4e-4a72-4efd-b3ee-080ea2b36bb2'} is not reachable. Will start new election round...
Jun 28, 2023 11:56:45 AM com.hazelcast.cp.internal.raft.impl.RaftNode(METADATA)
INFO: [127.0.0.1]:5710 [dev] [5.3.1] 

CP Group Members {groupId: METADATA(0), size:2, term:1, logIndex:6} [
	CPMember{uuid=e7713c4e-4a72-4efd-b3ee-080ea2b36bb2, address=[127.0.0.1]:5711}
	CPMember{uuid=6d34b6d8-3a75-477a-9acb-8ae179a22243, address=[127.0.0.1]:5710} - FOLLOWER this
]

Jun 28, 2023 11:56:45 AM com.hazelcast.cp.internal.raft.impl.task.PreVoteTask(METADATA)
INFO: [127.0.0.1]:5710 [dev] [5.3.1] Pre-vote started for next term: 2, last log index: 9, last log term: 1
Jun 28, 2023 11:56:45 AM com.hazelcast.cp.internal.raft.impl.RaftNode(METADATA)
INFO: [127.0.0.1]:5710 [dev] [5.3.1] 

CP Group Members {groupId: METADATA(0), size:2, term:1, logIndex:6} [
	CPMember{uuid=e7713c4e-4a72-4efd-b3ee-080ea2b36bb2, address=[127.0.0.1]:5711}
	CPMember{uuid=6d34b6d8-3a75-477a-9acb-8ae179a22243, address=[127.0.0.1]:5710} - FOLLOWER this
]

and the loop continues indefinitely

Maybe has something to do with this?

Jun 28, 2023 11:56:44 AM com.hazelcast.cp.internal.MetadataRaftGroupManager
WARNING: [127.0.0.1]:5710 [dev] [5.3.1] CPMember{uuid=e7713c4e-4a72-4efd-b3ee-080ea2b36bb2, address=[127.0.0.1]:5711} is directly removed as there are only 2 CP members.

Yes, it will. The CP Persistence only persists the CP member's state in the form of a Raft log

a matter of opinion, perhaps in some applications, but not in mine.
Also, persistence has other issues, such as filesystem management, making the node not ephemeral, etc, etc.
Persistence is not the solution to this problem

Maybe an issue is that the "normal shutdown" events don't result in a clear state of things in the logs. For example, if the CP system becomes "unusable" by design, there should be some ERROR or WARNING message in the logs, but there is none.
Perhaps it's not "good design" that a system can have "unusable" state anyway. Whole point of software that it should self-correct and never become "unusable"

@lprimak
Copy link
Contributor Author

lprimak commented Jun 27, 2023

Another issue... I have tried to auto-reset the subsystem via code when CP goes down to one member, but I get the below exception...

Currently, there is no way to run a CP system without some manual intervention for simple tasks...
There no way to know when to intervene because there is no event from Hazelcast that is emitted...
And manual intervention is error prone and requires too much knowledge about the RAFT algorithm and system logic,
far too much for the ops people to need to know :)
I believe the design (it his is by design indeed) needs to be improved.

**** Gone Down to One Member, resetting CP...
Jun 27, 2023 5:37:08 PM com.hazelcast.spi.impl.eventservice.EventService
WARNING: [127.0.0.1]:5710 [dev] [5.3.1] Error while logging processing event
java.util.concurrent.CompletionException: java.lang.IllegalStateException: Not enough cluster members to reset CP Subsystem! Required: 3, available: 2
	at java.base/java.util.concurrent.CompletableFuture.reportJoin(CompletableFuture.java:413)
	at java.base/java.util.concurrent.CompletableFuture.join(CompletableFuture.java:2118)
	at com.hazelcast.spi.impl.DeserializingCompletableFuture.join(DeserializingCompletableFuture.java:99)
	at com.flowlogix.hazelcast.cachetester.CacheTester$2.memberRemoved(CacheTester.java:193)
	at com.hazelcast.cp.internal.RaftService.dispatchEvent(RaftService.java:1436)
	at com.hazelcast.cp.internal.RaftService.dispatchEvent(RaftService.java:160)
	at com.hazelcast.spi.impl.eventservice.impl.EventProcessor.process(EventProcessor.java:63)
	at com.hazelcast.spi.impl.eventservice.impl.RemoteEventProcessor.run(RemoteEventProcessor.java:48)
	at com.hazelcast.internal.util.executor.StripedExecutor$Worker.process(StripedExecutor.java:245)
	at com.hazelcast.internal.util.executor.StripedExecutor$Worker.run(StripedExecutor.java:228)
Caused by: java.lang.IllegalStateException: Not enough cluster members to reset CP Subsystem! Required: 3, available: 2
	at com.hazelcast.cp.internal.RaftService.resetCPSubsystem(RaftService.java:303)
	at com.hazelcast.cp.internal.CPSubsystemImpl$CPSubsystemManagementServiceImpl.reset(CPSubsystemImpl.java:214)
	at com.hazelcast.cp.internal.CPSubsystemImpl$CPSubsystemManagementServiceImpl.reset(CPSubsystemImpl.java:170)
	... 7 more

@lprimak
Copy link
Contributor Author

lprimak commented Jun 28, 2023

Another one: shutdown is not possible in the "unusable" state. What that means in practice is that a CP cluster can't be cleanly and quickly shut down. It gets into the leader election loop again and won't exit.
And no, I am not exiting members concurrently, following documentation.
This doesn't happen every time, but most times.

I think this happens when the application is has used CP FencedLock and any CP sessions are open.

Jun 27, 2023 11:03:54 PM com.hazelcast.cp.internal.raft.impl.task.MembershipChangeTask(METADATA)
SEVERE: [127.0.0.1]:5711 [dev] [5.3.1] Cannot REMOVE RaftEndpoint{uuid='33378d7a-bd3c-4778-85f4-33215e404fa8'} because expected members commit index: 0 is different than group members commit index: 7
Jun 27, 2023 11:03:54 PM com.hazelcast.cp.internal.MetadataRaftGroupManager
INFO: [127.0.0.1]:5711 [dev] [5.3.1] CPMember{uuid=33378d7a-bd3c-4778-85f4-33215e404fa8, address=[127.0.0.1]:5712} is removed from CP Subsystem.
**** memberRemoved: CPMembershipEvent{type=REMOVED, member=CPMember{uuid=33378d7a-bd3c-4778-85f4-33215e404fa8, address=[127.0.0.1]:5712}}
Jun 27, 2023 11:03:54 PM com.hazelcast.cp.internal.MetadataRaftGroupManager
WARNING: [127.0.0.1]:5711 [dev] [5.3.1] Cannot apply CP membership changes: {CPGroupId{name='default', seed=0, groupId=340}=BiTuple{element1=0, element2=16}, CPGroupId{name='METADATA', seed=0, groupId=0}=BiTuple{element1=0, element2=7}} since there is no membership change context!
Jun 27, 2023 11:03:54 PM com.hazelcast.cp.internal.raftop.metadata.CompleteRaftGroupMembershipChangesOp
WARNING: [127.0.0.1]:5711 [dev] [5.3.1] Cannot apply CP membership changes: {CPGroupId{name='default', seed=0, groupId=340}=BiTuple{element1=0, element2=16}, CPGroupId{name='METADATA', seed=0, groupId=0}=BiTuple{element1=0, element2=7}} since there is no membership change context!
java.lang.IllegalStateException: Cannot apply CP membership changes: {CPGroupId{name='default', seed=0, groupId=340}=BiTuple{element1=0, element2=16}, CPGroupId{name='METADATA', seed=0, groupId=0}=BiTuple{element1=0, element2=7}} since there is no membership change context!
	at com.hazelcast.cp.internal.MetadataRaftGroupManager.completeRaftGroupMembershipChanges(MetadataRaftGroupManager.java:785)
	at com.hazelcast.cp.internal.raftop.metadata.CompleteRaftGroupMembershipChangesOp.run(CompleteRaftGroupMembershipChangesOp.java:52)
	at com.hazelcast.cp.internal.raftop.metadata.MetadataRaftGroupOp.run(MetadataRaftGroupOp.java:38)
	at com.hazelcast.cp.internal.NodeEngineRaftIntegration.runOperation(NodeEngineRaftIntegration.java:219)
	at com.hazelcast.cp.internal.raft.impl.RaftNodeImpl.applyLogEntry(RaftNodeImpl.java:849)
	at com.hazelcast.cp.internal.raft.impl.RaftNodeImpl.applyLogEntries(RaftNodeImpl.java:792)
	at com.hazelcast.cp.internal.raft.impl.RaftNodeImpl.commitEntries(RaftNodeImpl.java:1299)
	at com.hazelcast.cp.internal.raft.impl.RaftNodeImpl.tryAdvanceCommitIndex(RaftNodeImpl.java:1283)
	at com.hazelcast.cp.internal.raft.impl.handler.AppendSuccessResponseHandlerTask.handleResponse(AppendSuccessResponseHandlerTask.java:71)
	at com.hazelcast.cp.internal.raft.impl.handler.AbstractResponseHandlerTask.innerRun(AbstractResponseHandlerTask.java:43)
	at com.hazelcast.cp.internal.raft.impl.task.RaftNodeStatusAwareTask.run(RaftNodeStatusAwareTask.java:52)
	at com.hazelcast.cp.internal.NodeEngineRaftIntegration.execute(NodeEngineRaftIntegration.java:114)
	at com.hazelcast.cp.internal.raft.impl.RaftNodeImpl.execute(RaftNodeImpl.java:883)
	at com.hazelcast.cp.internal.raft.impl.RaftNodeImpl.handleAppendResponse(RaftNodeImpl.java:403)
	at com.hazelcast.cp.internal.RaftService.handleAppendResponse(RaftService.java:722)
	at com.hazelcast.cp.internal.operation.integration.AppendSuccessResponseOp.run(AppendSuccessResponseOp.java:48)
	at com.hazelcast.spi.impl.operationservice.Operation.call(Operation.java:193)
	at com.hazelcast.spi.impl.operationservice.impl.OperationRunnerImpl.call(OperationRunnerImpl.java:299)
	at com.hazelcast.spi.impl.operationservice.impl.OperationRunnerImpl.run(OperationRunnerImpl.java:270)
	at com.hazelcast.spi.impl.operationservice.impl.OperationRunnerImpl.run(OperationRunnerImpl.java:502)
	at com.hazelcast.spi.impl.operationexecutor.impl.OperationThread.process(OperationThread.java:202)
	at com.hazelcast.spi.impl.operationexecutor.impl.OperationThread.process(OperationThread.java:142)
	at com.hazelcast.spi.impl.operationexecutor.impl.OperationThread.loop(OperationThread.java:134)
	at com.hazelcast.spi.impl.operationexecutor.impl.OperationThread.executeRun(OperationThread.java:115)
	at com.hazelcast.internal.util.executor.HazelcastManagedThread.run(HazelcastManagedThread.java:111)

Another one:

INFO: [127.0.0.1]:5710 [dev] [5.3.1] Shutting down node engine...
Jun 27, 2023 11:06:34 PM com.hazelcast.cp.internal.RaftGroupMembershipManager
SEVERE: [127.0.0.1]:5710 [dev] [5.3.1] Cannot commit CP group membership changes: {CPGroupId{name='METADATA', seed=0, groupId=0}=BiTuple{element1=0, element2=7}, CPGroupId{name='default', seed=0, groupId=5690}=BiTuple{element1=0, element2=10}}
java.util.concurrent.ExecutionException: com.hazelcast.core.HazelcastInstanceNotActiveException: Hazelcast instance is not active!
	at com.hazelcast.spi.impl.operationservice.impl.InvocationFuture.returnOrThrowWithGetConventions(InvocationFuture.java:121)
	at com.hazelcast.spi.impl.operationservice.impl.InvocationFuture.resolveAndThrowIfException(InvocationFuture.java:100)
	at com.hazelcast.spi.impl.AbstractInvocationFuture.get(AbstractInvocationFuture.java:618)
	at com.hazelcast.cp.internal.RaftGroupMembershipManager$RaftGroupMembershipChangeHandlerTask.completeMembershipChanges(RaftGroupMembershipManager.java:447)
	at com.hazelcast.cp.internal.RaftGroupMembershipManager$RaftGroupMembershipChangeHandlerTask.run(RaftGroupMembershipManager.java:273)
	at com.hazelcast.spi.impl.executionservice.impl.DelegateAndSkipOnConcurrentExecutionDecorator$DelegateDecorator.run(DelegateAndSkipOnConcurrentExecutionDecorator.java:77)
	at com.hazelcast.internal.util.executor.CachedExecutorServiceDelegate$Worker.run(CachedExecutorServiceDelegate.java:217)
	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1144)
	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:642)
	at java.base/java.lang.Thread.run(Thread.java:1623)
	at com.hazelcast.internal.util.executor.HazelcastManagedThread.executeRun(HazelcastManagedThread.java:76)
	at com.hazelcast.internal.util.executor.HazelcastManagedThread.run(HazelcastManagedThread.java:111)
Caused by: com.hazelcast.core.HazelcastInstanceNotActiveException: Hazelcast instance is not active!
	at com.hazelcast.spi.impl.operationservice.impl.InvocationRegistry.shutdown(InvocationRegistry.java:229)
	at com.hazelcast.spi.impl.operationservice.impl.OperationServiceImpl.shutdownInvocations(OperationServiceImpl.java:562)
	at com.hazelcast.spi.impl.NodeEngineImpl.shutdown(NodeEngineImpl.java:587)
	at com.hazelcast.instance.impl.Node.shutdownServices(Node.java:630)
	at com.hazelcast.instance.impl.Node.shutdown(Node.java:566)
	at com.hazelcast.instance.impl.LifecycleServiceImpl.shutdown(LifecycleServiceImpl.java:101)
	at com.hazelcast.instance.impl.LifecycleServiceImpl.shutdown(LifecycleServiceImpl.java:84)
	at com.hazelcast.instance.impl.HazelcastInstanceFactory.shutdownAll(HazelcastInstanceFactory.java:306)
	at com.hazelcast.instance.impl.HazelcastInstanceFactory.shutdownAll(HazelcastInstanceFactory.java:280)
	at com.hazelcast.core.Hazelcast.shutdownAll(Hazelcast.java:44)
	at com.flowlogix.hazelcast.cachetester.CacheTester.teardown(CacheTester.java:254)
	at com.flowlogix.hazelcast.cachetester.CacheTester.main(CacheTester.java:75)
	at ------ submitted from ------.()
	at com.hazelcast.internal.util.ExceptionUtil.cloneExceptionWithFixedAsyncStackTrace(ExceptionUtil.java:348)
	at com.hazelcast.spi.impl.operationservice.impl.InvocationFuture.returnOrThrowWithGetConventions(InvocationFuture.java:112)
	... 11 more

Jun 27, 2023 11:06:34 PM com.hazelcast.cache.impl.CacheService
INFO: [127.0.0.1]:5710 [dev] [5.3.1] Removed cache config: CacheConfig{name='myCache', managerPrefix='/hz/', inMemoryFormat=BINARY, backupCount=1, hotRestart=HotRestartConfig{enabled=false, fsync=false}, dataPersistenceConfig=DataPersistenceConfig{enabled=false, fsync=false}, wanReplicationRef=null, merkleTreeConfig=MerkleTreeConfig{enabled=null, depth=10}}
Jun 27, 2023 11:06:34 PM com.hazelcast.instance.impl.NodeExtension
INFO: [127.0.0.1]:5710 [dev] [5.3.1] Destroying node NodeExtension.
Jun 27, 2023 11:06:34 PM com.hazelcast.instance.impl.Node
INFO: [127.0.0.1]:5710 [dev] [5.3.1] Hazelcast Shutdown is completed in 6648 ms.
Jun 27, 2023 11:06:34 PM com.hazelcast.core.LifecycleService
INFO: [127.0.0.1]:5710 [dev] [5.3.1] [127.0.0.1]:5710 is SHUTDOWN

@Anonymous-Coward
Copy link

This bug has plagued me for more than a year already, and I'm happy to see it finally documented. Never had time to investigate it in more depth myself, glad someone else did it.

Deploying apps on k8s is pretty widespread, these days, and k8s does move pods around quite frequently, for large and active clusters (like we operate - thousands of pods), in order to optimize load on nodes. Under those circumstances, 4h as a default for kicking pods/hazelcast nodes that no longer exist and not somehow using info from the k8s API to speed up the cleanup/recovery process after a node from a CP group vanishes are both less than ideal decisions re. hazelcast's implementation, IMO.

@lprimak
Copy link
Contributor Author

lprimak commented Jun 28, 2023

Thank you for listening @arodionov
I understand that this is a difficult problem, but I am trying to make already fantastic product better.

@lprimak
Copy link
Contributor Author

lprimak commented Sep 14, 2023

@arodionov I have added a new test, whenMemberExitsNormally_thenReceiveEvents to hazelcast/src/test/java/com/hazelcast/cp/CPGroupAvailabilityListenerTest.java in #24903

This test is failing and clearly demonstrates incorrect behavior described here: #24897 (comment)

@arodionov
Copy link
Contributor

@lprimak have a PR for your test whenMemberExitsNormally_thenReceiveEvents #25490

@lprimak
Copy link
Contributor Author

lprimak commented Sep 22, 2023

@arodionov Thanks! Looks great!

@lprimak
Copy link
Contributor Author

lprimak commented Oct 16, 2023

After some back-and-fourth, it's become clear that whenMemberExitsNormally_thenReceiveEvents is no longer necessary after the PR and is working correctly.

@ioandy
Copy link

ioandy commented Oct 26, 2023

I have encountered the same issue, using version 5.2.4. Has this issue been ultimately resolved?

@lprimak
Copy link
Contributor Author

lprimak commented Oct 26, 2023

No. First step to fix it will be in Hazelcast 5.4 where the majorityLost event would be reliable. but the rest is not fixed yet. The workaround in https://github.com/flowlogix/hazelcast-issues/ does work correctly though

@AyberkSorgun AyberkSorgun added the Source: Community PR or issue was opened by a community user label Jan 29, 2024
@Anonymous-Coward
Copy link

After trying for months to get the CP subsystem behaving properly in the face of pod restarts in kubernetes, I finally gave up, and switched to a solution which doesn't use it at all. As far as I can tell, Hazelcast's CP subsystem is not at all a good fit for using in an environment where starting and stopping nodes isn't uncommon.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Source: Community PR or issue was opened by a community user Type: Defect
Projects
None yet
Development

Successfully merging a pull request may close this issue.

6 participants