-
Notifications
You must be signed in to change notification settings - Fork 134
IGNITE-23019 Research before IGNITE-23010 #4477
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
Conversation
…n in case if a corresponding raft-node hasn't a leader and didn't apply any log actions yet
…mViewTest passing because tests are waiting for AVAILABLE state of partitions
modules/raft/src/main/java/org/apache/ignite/internal/raft/RaftGroupServiceImpl.java
Show resolved
Hide resolved
...raft/src/main/java/org/apache/ignite/internal/raft/storage/impl/RocksDbSharedLogStorage.java
Outdated
Show resolved
Hide resolved
...grationTest/java/org/apache/ignite/internal/replicator/ItPlacementDriverReplicaSideTest.java
Show resolved
Hide resolved
| assertNotNull(raftClient); | ||
|
|
||
| raftClient.refreshLeader().get(); | ||
| raftClient.refreshLeader().get(WAIT_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why do we need external timeout here? I mean that refreshLeader internally has time bounded limitations:
if (currentTimeMillis() >= stopTime) {
fut.completeExceptionally(
new TimeoutException(format("Send with retry timed out [retryCount = {}, groupId = {}].", retryCount, groupId)));
return;
}
?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Didn't notice internal limitation, just was triggered with get() without a timeout. Removed.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Please check other places, e.g. on line 237.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done
JIRA Ticket: IGNITE-23019
The goal
The goal of this PR is to fix tests that are failed due to
getLeaderflag removing.The reason
For now
org.apache.ignite.internal.raft.RaftGroupServiceImpl#startconsumegetLeaderflag that leads to:refreshLeaderinside.CompletableFutureas a return type.But this
refreshLeadercall is not nesseccary because in all raft-client's public methodsrefreshLeadertriggers inside. Besides, when we're starting a cluster a numerous of nodes and its replication groups are starting and awaiting raft leader election while replication groups are started. This awaition doesn't need for starting, but it is slowing down the starting process.Thus, there are reasons for such flag removal:
CompletableFutureresult;The solution
As a temporal solution before IGNITE-23010 is to set
getLeaderflag asfalseinsideorg.apache.ignite.internal.raft.RaftGroupServiceImpl#start. Then, several tests are failed. Below is a list of the failed tests that are fixed in this PR.ItLearnersTest#testAddLearnersReasons
We're waiting after raft-client's start that the leader will be the
followernode, but as a result we got anullvalue.Solution
Instead of getting the leader directly, let's do
refreshLeaderbefore:ItLearnersTest#testLearnersOnTheSameNodeAsPeersReasons
Two raft-clients are started and they're related to 2 nodes: one for the peer and another for the learner nodes respectively. The test checks that peer's client knows that it is related with the leader, but not learner. On the other side the learner's client checks that it has the peer as leader but not the learner. But the test fails because on each check we will get
nullvalue as a leader.Solution
As the previous test case we will do refresh leader before each check (peer & learner cases) begins. Follow check has already refreshed leader.
ItRaftGroupServiceTest#testTransferLeadershipReasons
We will get an NPE in the test's beginning while we're trying to get the first node's leader. But there is no leader yet and then we will write
nulltoleadervariable and will triggerNPEwhen will try to callconsistentIdfrom thenull-value leader.Solution
Let's refresh leader before it's usage:
ItLozaTest#testRaftServiceUsingSharedExecutorReasons
Raft-client's start isn't the primary scope for the test, but there is a mock calls counter check of
invokeinside ofsendWithRetry, but the method won't be called because there are no morerefreshLeadercalls that are triggeringsendWithRetry.Solution
Let's add
refreshLeaderafter the client's start with successful completion check then:ItDisasterRecoverySystemViewTestThere are 2 tests that are failed, but with a common reason:
testGlobalPartitionStatesSystemViewtestLocalPartitionStatesSystemViewReasons
After a leader acquiring an log replication some partitions have
LocalPartitionStateEnum#INITIALIZINGstate instead ofHEALTHY(seeLocalPartitionStateEnumWithLogIndex#of), that leads toUNAVAILABLEstate in a system table for the corresponding partition instead ofAVAILABLE(seeDisasterRecoveryManager:734). That leads for a check fail on status column.Solution
The observation actually is correct, the issue is in test's expectations there: test wants
AVAILABLEand it will observe it, but the test isn't waiting for it now, it rely on a soon refresh leader completion. The solution there is just to add refresh leader on each partition with successful completion waiting. Then, the test will obtain the desired state:ActiveActorTest#testChangeLeaderWhenActualLeftReasons
There are 2 issues that aren't related to
getLeaderdirectly:"localhost"check will fail (I got different192.168.*.*addresses instead that wasn't resolved as localhost), but there no actual need to check hosts. Moreover without fix we will step to the block just because the host issue, but the leader is already changed.leaderRef-- it isn't available there already and won't be updated.Solution
The first should be fixed with port check only. The second one is fixed with
leaderRefNoInitialNotifyinstead ofleaderRef.RaftGroupServiceTestReasons
There are unit tests that actively use
getLeaderflag explicitly for different tests invariant. It leads to invariants failure.Solutions
The =getLeader= flag for raft clients creation method is removed, but new method
startRaftGroupServiceWithRefreshLeaderis introduced. The method creates a raft client with mock on leader requests and then it callsrefreshLeader. So, in tests that are checking leader we will use the last one, and the originalstartRaftGroupServicewithout the flag otherwise.ItTableRaftSnapshotsTestThe most complicated test class, but in general there 2 major issues. They were happened in the follow tests list, but potentially every test in the class could be failed:
nodeCanInstallSnapshotsAfterSnapshotInstalledToIttestChangeLeaderOnInstallSnapshotInMiddleentriesKeepAppendedAfterSnapshotInstallationsnapshotInstallationRepeatsOnTimeoutAssertionError: Unexpected leadership change on group: 10_part_0
Reasons
This problem emerges in
doSnapshotOnprocessing with the follow scenario:It happens because in parallel there is primary replica acceptance is in process. The situation happens:
LeaseGrantMessagegoes to 1 with force flag (0 nor 1 doesn't know actual RAFT-leader yet, so we can't collocate lease and the leader, that's why node 1 will be forsed chosen).LeaseGrantMessagewithforce == truetriggers transfer leadership to the leaseholder -- node 1.doSnapshotOnfails because the leader was changed due to force lease acceptance on node 1.Solution
The described scenario is legal for Ignite, but for the tests scenario it interferes with desired invariant checks. Thus, the simple and correct solution is just to wait while lease will be accepted and then doing the transfer leadership with
waitForPrimaryReplicathat was introduced. It should be placed after cluster initialization for test work in each case.PrimaryReplicaMissExceptionwithcurrentEnlistmentConsistencyToken == 1Reasons
In failed tests (fail rate for
nodeCanInstallSnapshotsAfterSnapshotInstalledToItis ~25%) we can see the picture:The reason is that internally we have
leaseStartTimethat iscurrentEnlistmentConsistencyTokeninUpdateCommandResponse. This request-response pair is a raft command. When we're trying to put a value, it's going to node 1 as a primary replica, it creates the command and send to the leader (after p.7 it is node 2). But the leader see that internally it still has uninitializedleaseStartTime == 1that doesn't match with unxi-time-like token from the message from node 1. That's the reason why 0th sends the response with token mismatch flag turned on and itscurrentEnlistmentConsistencyToken == 1. The last question is why it is uninitialized: because some meta information (asleaseStartTime) that is stored in raft log wasn't restored from a snapshot installation on node 2 startup process. There is the corresponding issue: IGNITE-23379.Solution
Due to all tests are checking snapshots are installed (e.g. with
snapshotInstalledLatch) -- they're all may be affected before IGNITE-23379 will be fixed. So, this test class is disabled for a while.JVM crash on
ItPlacementDriverReplicaSideTestReasons
The main reason is an unhandled exception from
raftClient.refreshLeader().get()that is actuallyTimeoutExceptionon leader election process. We haveretryTimeout=2000, but leader election may took up a little more time. Then, uncaught exception is thrown and we doesn't stop replicas that leads to closed rocksDB entity access attempt, that crash JVM.Solution
There are 2 actions:
@BeforeEach's ends andstopReplicationGroup(GROUP_ID, grpNodes)to@AfterEach's beginning.