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

[FLINK-34897][test] Enables JobMasterServiceLeadershipRunnerTest#testJobMasterServiceLeadershipRunnerCloseWhenElectionServiceGrantLeaderShip again. #24546

Merged
merged 1 commit into from
Mar 27, 2024
Merged
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -45,18 +45,19 @@
import org.apache.flink.runtime.messages.Acknowledge;
import org.apache.flink.runtime.scheduler.ExecutionGraphInfo;
import org.apache.flink.runtime.testtasks.NoOpInvokable;
import org.apache.flink.runtime.testutils.CommonTestUtils;
import org.apache.flink.runtime.testutils.TestingJobResultStore;
import org.apache.flink.runtime.util.TestingFatalErrorHandler;
import org.apache.flink.util.ExceptionUtils;
import org.apache.flink.util.FlinkException;
import org.apache.flink.util.FlinkRuntimeException;
import org.apache.flink.util.Preconditions;
import org.apache.flink.util.function.SupplierWithException;
import org.apache.flink.util.function.ThrowingRunnable;

import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Disabled;
import org.junit.jupiter.api.Test;

import javax.annotation.Nonnull;
Expand Down Expand Up @@ -676,26 +677,25 @@ void testJobAlreadyDone() throws Exception {
}
}

@Disabled
@Test
void testJobMasterServiceLeadershipRunnerCloseWhenElectionServiceGrantLeaderShip()
throws Exception {
final AtomicReference<LeaderInformationRegister> storedLeaderInformation =
new AtomicReference<>(LeaderInformationRegister.empty());
final AtomicBoolean haBackendLeadershipFlag = new AtomicBoolean();

final TestingLeaderElectionDriver.Factory driverFactory =
new TestingLeaderElectionDriver.Factory(
TestingLeaderElectionDriver.newBuilder(
new AtomicBoolean(), storedLeaderInformation, new AtomicBoolean()));
haBackendLeadershipFlag,
storedLeaderInformation,
new AtomicBoolean()));

// we need to use DefaultLeaderElectionService here because JobMasterServiceLeadershipRunner
// in connection with the DefaultLeaderElectionService generates the nested locking
final DefaultLeaderElectionService defaultLeaderElectionService =
new DefaultLeaderElectionService(driverFactory, fatalErrorHandler);

final TestingLeaderElectionDriver currentLeaderDriver =
driverFactory.assertAndGetOnlyCreatedDriver();

// latch to detect when we reached the first synchronized section having a lock on the
// JobMasterServiceProcess#stop side
final OneShotLatch closeAsyncCalledTrigger = new OneShotLatch();
Expand Down Expand Up @@ -733,6 +733,7 @@ void testJobMasterServiceLeadershipRunnerCloseWhenElectionServiceGrantLeaderShip
// before calling stop on the
// DefaultLeaderElectionService
triggerClassLoaderLeaseRelease.await();

// In order to reproduce the deadlock, we
// need to ensure that
// leaderContender#grantLeadership can be
Expand Down Expand Up @@ -770,13 +771,19 @@ void testJobMasterServiceLeadershipRunnerCloseWhenElectionServiceGrantLeaderShip
jobManagerRunner.start();

// grant leadership to create jobMasterServiceProcess
haBackendLeadershipFlag.set(true);
final UUID leaderSessionID = UUID.randomUUID();
defaultLeaderElectionService.onGrantLeadership(leaderSessionID);

while (!currentLeaderDriver.hasLeadership()
|| !leaderElection.hasLeadership(leaderSessionID)) {
Thread.sleep(100);
}
final SupplierWithException<Boolean, Exception> confirmationForSessionIdReceived =
() ->
storedLeaderInformation
.get()
.forComponentId(componentId)
.map(LeaderInformation::getLeaderSessionID)
.map(sessionId -> sessionId.equals(leaderSessionID))
.orElse(false);
CommonTestUtils.waitUntilCondition(confirmationForSessionIdReceived);

final CheckedThread contenderCloseThread = createCheckedThread(jobManagerRunner::close);
contenderCloseThread.start();
Expand Down