xintongsong commented on a change in pull request #13644:
URL: https://github.com/apache/flink/pull/13644#discussion_r512440752



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionService.java
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.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.Objects;
+import java.util.UUID;
+
+/**
+ * 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. */
+       @GuardedBy("lock")
+       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 = leaderElectionDriverFactory;

Review comment:
       minor: null check

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderInformation.java
##########
@@ -0,0 +1,86 @@
+/*
+ * 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 javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.Objects;
+import java.util.UUID;
+
+/**
+ * Information about leader including the confirmed leader session id and 
leader address.
+ */
+public class LeaderInformation implements Serializable {
+
+       private static final long serialVersionUID = 1L;
+
+       private final UUID leaderSessionID;
+
+       private final String leaderAddress;
+
+       private static final LeaderInformation EMPTY = new 
LeaderInformation(null, null);
+
+       public LeaderInformation(@Nullable UUID leaderSessionID, @Nullable 
String leaderAddress) {
+               this.leaderSessionID = leaderSessionID;
+               this.leaderAddress = leaderAddress;
+       }
+
+       @Nullable
+       public UUID getLeaderSessionID() {
+               return leaderSessionID;
+       }
+
+       @Nullable
+       public String getLeaderAddress() {
+               return leaderAddress;
+       }
+
+       @Override
+       public boolean equals(Object obj) {
+               if (obj == this) {
+                       return true;
+               } else if (obj != null && obj.getClass() == 
LeaderInformation.class) {
+                       final LeaderInformation that = (LeaderInformation) obj;
+                       return Objects.equals(this.leaderSessionID, 
that.leaderSessionID) &&
+                               Objects.equals(this.leaderAddress, 
that.leaderAddress);
+               } else {
+                       return false;
+               }
+       }
+
+       @Override
+       public int hashCode() {
+               int result = leaderSessionID.hashCode();

Review comment:
       This may throw NPE if `leaderSessionID` is `null`.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderInformation.java
##########
@@ -0,0 +1,86 @@
+/*
+ * 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 javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.Objects;
+import java.util.UUID;
+
+/**
+ * Information about leader including the confirmed leader session id and 
leader address.
+ */
+public class LeaderInformation implements Serializable {
+
+       private static final long serialVersionUID = 1L;
+
+       private final UUID leaderSessionID;
+
+       private final String leaderAddress;

Review comment:
       These 2 fields should also be annotated with `@Nullable`.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/DefaultLeaderRetrievalService.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.leaderretrieval;
+
+import org.apache.flink.runtime.leaderelection.LeaderInformation;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Objects;
+import java.util.UUID;
+
+/**
+ * The counterpart to the {@link 
org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService}.
+ * Composed with different {@link LeaderRetrievalDriver}, we could retrieve 
the leader information from
+ * different storage. The leader address as well as the current leader session 
ID will be retrieved from
+ * {@link LeaderRetrievalDriver}.
+ */
+public class DefaultLeaderRetrievalService implements LeaderRetrievalService, 
LeaderRetrievalEventHandler {
+       private static final Logger LOG = 
LoggerFactory.getLogger(DefaultLeaderRetrievalService.class);
+
+       private final Object lock = new Object();
+
+       private final LeaderRetrievalDriverFactory leaderRetrievalDriverFactory;
+
+       @GuardedBy("lock")
+       private String lastLeaderAddress;
+
+       @GuardedBy("lock")
+       private UUID lastLeaderSessionID;
+
+       @GuardedBy("lock")
+       private volatile boolean running;
+
+       /** Listener which will be notified about leader changes. */
+       @GuardedBy("lock")
+       private volatile LeaderRetrievalListener leaderListener;
+
+       @GuardedBy("lock")
+       private LeaderRetrievalDriver leaderRetrievalDriver;
+
+       /**
+        * Creates a default leader retrieval service with specified {@link 
LeaderRetrievalDriverFactory}.
+        *
+        * @param leaderRetrievalDriverFactory {@link 
LeaderRetrievalDriverFactory} used for creating
+        * {@link LeaderRetrievalDriver}.
+        */
+       public DefaultLeaderRetrievalService(LeaderRetrievalDriverFactory 
leaderRetrievalDriverFactory) {
+               this.leaderRetrievalDriverFactory = 
leaderRetrievalDriverFactory;

Review comment:
       null check




----------------------------------------------------------------
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:
us...@infra.apache.org


Reply via email to