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

[improve][meta] Stop election operations when LeaderElectionImpl gets closed #23995

Draft
wants to merge 2 commits into
base: master
Choose a base branch
from
Draft
Show file tree
Hide file tree
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 @@ -35,9 +35,7 @@
import static org.apache.pulsar.metadata.api.extended.SessionEvent.Reconnected;
import static org.apache.pulsar.metadata.api.extended.SessionEvent.SessionLost;
import static org.apache.pulsar.metadata.api.extended.SessionEvent.SessionReestablished;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
import static org.hamcrest.Matchers.lessThanOrEqualTo;
import static org.assertj.core.api.Assertions.assertThat;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyLong;
import static org.mockito.ArgumentMatchers.eq;
Expand Down Expand Up @@ -258,10 +256,18 @@ public void channelOwnerTest() throws Exception {
assertEquals(channelOwner1, channelOwner2);
LeaderElectionService leaderElectionService1 = (LeaderElectionService) FieldUtils.readDeclaredField(
channel1, "leaderElectionService", true);
leaderElectionService1.close();
waitUntilNewChannelOwner(channel2, channelOwner1);
leaderElectionService1.start();
waitUntilNewChannelOwner(channel1, channelOwner1);

// delete the leader node to trigger a new leader election
pulsar1.getLocalMetadataStore().delete("/loadbalance/leader", Optional.of(-1L)).get();

// wait for leader to lose leadership
Thread.sleep(500);

// wait for leader election to happen
Awaitility.await().untilAsserted(() -> {
assertThat(channel1.getChannelOwnerAsync().get()).isPresent();
assertThat(channel2.getChannelOwnerAsync().get()).isPresent();
});

var newChannelOwner1 = channel1.getChannelOwnerAsync().get(2, TimeUnit.SECONDS);
var newChannelOwner2 = channel2.getChannelOwnerAsync().get(2, TimeUnit.SECONDS);
Expand Down Expand Up @@ -698,35 +704,31 @@ public void handleMetadataSessionEventTest() throws IllegalAccessException {
var lastMetadataSessionEventTimestamp = getLastMetadataSessionEventTimestamp(channel1);

assertEquals(SessionReestablished, lastMetadataSessionEvent);
assertThat(lastMetadataSessionEventTimestamp,
greaterThanOrEqualTo(ts));
assertThat(lastMetadataSessionEventTimestamp).isGreaterThanOrEqualTo(ts);

ts = System.currentTimeMillis();
channel1.handleMetadataSessionEvent(SessionLost);
lastMetadataSessionEvent = getLastMetadataSessionEvent(channel1);
lastMetadataSessionEventTimestamp = getLastMetadataSessionEventTimestamp(channel1);

assertEquals(SessionLost, lastMetadataSessionEvent);
assertThat(lastMetadataSessionEventTimestamp,
greaterThanOrEqualTo(ts));
assertThat(lastMetadataSessionEventTimestamp).isGreaterThanOrEqualTo(ts);

ts = System.currentTimeMillis();
channel1.handleMetadataSessionEvent(ConnectionLost);
lastMetadataSessionEvent = getLastMetadataSessionEvent(channel1);
lastMetadataSessionEventTimestamp = getLastMetadataSessionEventTimestamp(channel1);

assertEquals(SessionLost, lastMetadataSessionEvent);
assertThat(lastMetadataSessionEventTimestamp,
lessThanOrEqualTo(ts));
assertThat(lastMetadataSessionEventTimestamp).isLessThanOrEqualTo(ts);

ts = System.currentTimeMillis();
channel1.handleMetadataSessionEvent(Reconnected);
lastMetadataSessionEvent = getLastMetadataSessionEvent(channel1);
lastMetadataSessionEventTimestamp = getLastMetadataSessionEventTimestamp(channel1);

assertEquals(SessionLost, lastMetadataSessionEvent);
assertThat(lastMetadataSessionEventTimestamp,
lessThanOrEqualTo(ts));
assertThat(lastMetadataSessionEventTimestamp).isLessThanOrEqualTo(ts);

}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -104,6 +104,9 @@ public synchronized CompletableFuture<LeaderElectionState> elect(T proposedValue
}

private synchronized CompletableFuture<LeaderElectionState> elect() {
if (internalState == InternalState.Closed) {
return FutureUtils.exception(new AlreadyClosedException("The leader election was already closed"));
}
// First check if there's already a leader elected
internalState = InternalState.ElectionInProgress;
return store.get(path).thenCompose(optLock -> {
Expand All @@ -113,6 +116,11 @@ private synchronized CompletableFuture<LeaderElectionState> elect() {
return tryToBecomeLeader();
}
}).thenCompose(leaderElectionState -> {
synchronized (this) {
if (internalState == InternalState.Closed) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Need to add volatile on internalState?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Already covered with synchronized (this). this points to LeaderElectionImpl instance here.

return FutureUtils.exception(new AlreadyClosedException("The leader election was already closed"));
}
}
// make sure that the cache contains the current leader
// so that getLeaderValueIfPresent works on all brokers
cache.refresh(path);
Expand All @@ -122,6 +130,9 @@ private synchronized CompletableFuture<LeaderElectionState> elect() {
}

private synchronized CompletableFuture<LeaderElectionState> handleExistingLeaderValue(GetResult res) {
if (internalState == InternalState.Closed) {
return FutureUtils.exception(new AlreadyClosedException("The leader election was already closed"));
}
T existingValue;
try {
existingValue = serde.deserialize(path, res.getValue(), res.getStat());
Expand Down Expand Up @@ -163,6 +174,9 @@ private synchronized CompletableFuture<LeaderElectionState> handleExistingLeader
}

private synchronized void changeState(LeaderElectionState les) {
if (internalState == InternalState.Closed) {
return;
}
internalState = InternalState.LeaderIsPresent;
if (this.leaderElectionState != les) {
this.leaderElectionState = les;
Expand All @@ -175,6 +189,9 @@ private synchronized void changeState(LeaderElectionState les) {
}

private synchronized CompletableFuture<LeaderElectionState> tryToBecomeLeader() {
if (internalState == InternalState.Closed) {
return FutureUtils.exception(new AlreadyClosedException("The leader election was already closed"));
}
T value = proposedValue.get();
byte[] payload;
try {
Expand All @@ -192,6 +209,11 @@ private synchronized CompletableFuture<LeaderElectionState> tryToBecomeLeader()
cache.get(path)
.thenRun(() -> {
synchronized (LeaderElectionImpl.this) {
if (internalState == InternalState.Closed) {
result.completeExceptionally(new AlreadyClosedException(
"The leader election was already closed"));
return;
}
log.info("Acquired leadership on {} with {}", path, value);
internalState = InternalState.LeaderIsPresent;
if (leaderElectionState != LeaderElectionState.Leading) {
Expand Down Expand Up @@ -254,7 +276,6 @@ private synchronized CompletableFuture<LeaderElectionState> tryToBecomeLeader()

@Override
public void close() throws Exception {
updateCachedValueFuture.cancel(true);
try {
asyncClose().join();
} catch (CompletionException e) {
Expand All @@ -268,6 +289,7 @@ public synchronized CompletableFuture<Void> asyncClose() {
return CompletableFuture.completedFuture(null);
}

updateCachedValueFuture.cancel(true);
internalState = InternalState.Closed;

if (leaderElectionState != LeaderElectionState.Leading) {
Expand Down
Loading