tillrohrmann commented on a change in pull request #13644: URL: https://github.com/apache/flink/pull/13644#discussion_r517218385
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionService.java ########## @@ -0,0 +1,296 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.leaderelection; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import javax.annotation.concurrent.GuardedBy; + +import java.util.UUID; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Default implementation for leader election service. Composed with different {@link LeaderElectionDriver}, we could + * perform a leader election for the contender, and then persist the leader information to various storage. + */ +public class DefaultLeaderElectionService implements LeaderElectionService, LeaderElectionEventHandler { + + private static final Logger LOG = LoggerFactory.getLogger(DefaultLeaderElectionService.class); + + private final Object lock = new Object(); + + private final LeaderElectionDriverFactory leaderElectionDriverFactory; + + /** The leader contender which applies for leadership. */ + private volatile LeaderContender leaderContender; + + @GuardedBy("lock") + private volatile UUID issuedLeaderSessionID; + + @GuardedBy("lock") + private volatile UUID confirmedLeaderSessionID; + + @GuardedBy("lock") + private volatile String confirmedLeaderAddress; + + @GuardedBy("lock") + private volatile boolean running; + + @GuardedBy("lock") + private LeaderElectionDriver leaderElectionDriver; + + public DefaultLeaderElectionService(LeaderElectionDriverFactory leaderElectionDriverFactory) { + this.leaderElectionDriverFactory = checkNotNull(leaderElectionDriverFactory); + + leaderContender = null; + + issuedLeaderSessionID = null; + confirmedLeaderSessionID = null; + confirmedLeaderAddress = null; + + running = false; + } + + @Override + public final void start(LeaderContender contender) throws Exception { + checkNotNull(contender, "Contender must not be null."); + Preconditions.checkState(leaderContender == null, "Contender was already set."); + + synchronized (lock) { + leaderContender = contender; + leaderElectionDriver = leaderElectionDriverFactory.createLeaderElectionDriver( + this, new LeaderElectionFatalErrorHandler(), leaderContender.getDescription()); + LOG.info("Starting DefaultLeaderElectionService with {}.", leaderElectionDriver); + + running = true; + } + } + + @Override + public final void stop() throws Exception { + LOG.info("Stopping DefaultLeaderElectionService."); + + synchronized (lock) { + if (!running) { + return; + } + running = false; + clearConfirmedLeaderInformation(); + leaderElectionDriver.close(); + } + } + + @Override + public void confirmLeadership(UUID leaderSessionID, String leaderAddress) { + if (LOG.isDebugEnabled()) { + LOG.debug( + "Confirm leader session ID {} for leader {}.", + leaderSessionID, + leaderAddress); + } + + checkNotNull(leaderSessionID); + + synchronized (lock) { + if (hasLeadership(leaderSessionID)) { + if (running) { + confirmLeaderInformation(leaderSessionID, leaderAddress); + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("Ignoring the leader session Id {} confirmation, since the " + + "LeaderElectionService has already been stopped.", leaderSessionID); + } + } + } else { + // Received an old confirmation call + if (!leaderSessionID.equals(this.issuedLeaderSessionID)) { + if (LOG.isDebugEnabled()) { + LOG.debug("Receive an old confirmation call of leader session ID {}, " + + "current issued session ID is {}", leaderSessionID, issuedLeaderSessionID); + } + } else { + LOG.warn("The leader session ID {} was confirmed even though the " + + "corresponding JobManager was not elected as the leader.", leaderSessionID); + } + } + } + } + + @Override + public boolean hasLeadership(@Nonnull UUID leaderSessionId) { + synchronized (lock) { + if (running) { + return leaderElectionDriver.hasLeadership() && leaderSessionId.equals(issuedLeaderSessionID); + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("hasLeadership is called after the service is stopped, returning false."); + } + return false; + } + } + } + + /** + * Returns the current leader session ID or null, if the contender is not the leader. + * + * @return The last leader session ID or null, if the contender is not the leader + */ + @VisibleForTesting + @Nullable + public UUID getLeaderSessionID() { + return confirmedLeaderSessionID; + } + + @GuardedBy("lock") + private void confirmLeaderInformation(UUID leaderSessionID, String leaderAddress) { + confirmedLeaderSessionID = leaderSessionID; + confirmedLeaderAddress = leaderAddress; + leaderElectionDriver.writeLeaderInformation( + LeaderInformation.known(confirmedLeaderSessionID, confirmedLeaderAddress)); + } + + @GuardedBy("lock") + private void clearConfirmedLeaderInformation() { + confirmedLeaderSessionID = null; + confirmedLeaderAddress = null; + } + + @Override + @GuardedBy("lock") + public void onGrantLeadership() { + synchronized (lock) { + if (running) { + issuedLeaderSessionID = UUID.randomUUID(); + clearConfirmedLeaderInformation(); + + if (LOG.isDebugEnabled()) { + LOG.debug( + "Grant leadership to contender {} with session ID {}.", + leaderContender.getDescription(), + issuedLeaderSessionID); + } + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("Ignoring the grant leadership notification since the {} has " + + "already been closed.", leaderElectionDriver); + } + return; + } + } + // The contender callback should be executed out of lock to avoid potential deadlock. + leaderContender.grantLeadership(issuedLeaderSessionID); + } + + @Override + @GuardedBy("lock") + public void onRevokeLeadership() { + synchronized (lock) { + if (running) { + if (LOG.isDebugEnabled()) { + LOG.debug( + "Revoke leadership of {} ({}@{}).", + leaderContender.getDescription(), + confirmedLeaderSessionID, + confirmedLeaderAddress); + } + + issuedLeaderSessionID = null; + clearConfirmedLeaderInformation(); + + // Clear the old leader information on the external storage + leaderElectionDriver.writeLeaderInformation(LeaderInformation.empty()); + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("Ignoring the revoke leadership notification since the {} " + + "has already been closed.", leaderElectionDriver); + } + return; + } + } + // The contender callback should be executed out of lock to avoid potential deadlock. + leaderContender.revokeLeadership(); Review comment: I am wondering whether it is potentially a problem that we only let the `leaderContender` know about the lost leadership after performing a potentially blocking operation in `leaderElectionDriver.writeLeaderInformation`. Assume that this operation takes a bit of time, then the `leaderContender` can continue working as the leader. Maybe it would be better to only try to clear the old leader information after having notified the `leaderContender` about it in particular since the clearing is not strictly required. ########## File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientITCase.java ########## @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.kubernetes.kubeclient; + +import org.apache.flink.kubernetes.KubernetesResource; +import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap; +import org.apache.flink.runtime.concurrent.FutureUtils; + +import io.fabric8.kubernetes.api.model.ConfigMapBuilder; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; + +import static org.hamcrest.Matchers.everyItem; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThat; + +/** + * IT Tests for {@link org.apache.flink.kubernetes.kubeclient.Fabric8FlinkKubeClient} with real K8s server and client. + */ +public class Fabric8FlinkKubeClientITCase { + + @ClassRule + public static KubernetesResource kubernetesResource = new KubernetesResource(); + + private static final String TEST_CONFIG_MAP_NAME = "test-config-map"; + + private static final long TIMEOUT = 120L * 1000L; + + private static final Map<String, String> data = new HashMap<String, String>() { + { + put("key1", "0"); + put("key2", "0"); + put("key3", "0"); + } + }; + + private FlinkKubeClient flinkKubeClient; + + @Before + public void setup() throws Exception { + flinkKubeClient = kubernetesResource.getFlinkKubeClient(); + flinkKubeClient.createConfigMap(new KubernetesConfigMap( + new ConfigMapBuilder() + .withNewMetadata() + .withName(TEST_CONFIG_MAP_NAME) + .endMetadata() + .withData(data) + .build())).get(); + } + + @After + public void teardown() throws Exception { + flinkKubeClient.deleteConfigMap(TEST_CONFIG_MAP_NAME).get(); + } + + /** + * {@link org.apache.flink.kubernetes.kubeclient.FlinkKubeClient#checkAndUpdateConfigMap} is a transactional + * operation, we should definitely guarantee that the concurrent modification could work. + */ + @Test + public void testCheckAndUpdateConfigMapConcurrently() throws Exception { + // Start multiple instances to update ConfigMap concurrently + final List<CompletableFuture<Void>> futures = new ArrayList<>(); + final int target = 10; + final int updateIntervalMs = 100; + for (String key : data.keySet()) { + futures.add(FutureUtils.runAfterwardsAsync(FutureUtils.completedVoidFuture(), () -> { Review comment: I guess we could still fix this here. Very minor though. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected]
