You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@helix.apache.org by "xyuanlu (via GitHub)" <gi...@apache.org> on 2023/05/16 01:07:09 UTC

[GitHub] [helix] xyuanlu opened a new pull request, #2488: add leader election API and dummy class

xyuanlu opened a new pull request, #2488:
URL: https://github.com/apache/helix/pull/2488

   ### Issues
   
   - [ ] My PR addresses the following Helix issues and references them in the PR description:
   
   (#200 - Link your issue number here: You can write "Fixes #XXX". Please use the proper keyword so that the issue gets closed automatically. See https://docs.github.com/en/github/managing-your-work-on-github/linking-a-pull-request-to-an-issue
   Any of the following keywords can be used: close, closes, closed, fix, fixes, fixed, resolve, resolves, resolved)
   
   ### Description
   
   - [ ] Here are some details about my PR, including screenshots of any UI changes:
   
   (Write a concise description including what, why, how)
   
   ### Tests
   
   - [ ] The following tests are written for this issue:
   
   (List the names of added unit/integration tests)
   
   - The following is the result of the "mvn test" command on the appropriate module:
   
   (If CI test fails due to known issue, please specify the issue and test PR locally. Then copy & paste the result of "mvn test" to here.)
   
   ### Changes that Break Backward Compatibility (Optional)
   
   - My PR contains changes that break backward compatibility or previous assumptions for certain methods or API. They include:
   
   (Consider including all behavior changes for public methods or API. Also include these changes in merge description so that other developers are aware of these changes. This allows them to make relevant code changes in feature branches accounting for the new method/API behavior.)
   
   ### Documentation (Optional)
   
   - In case of new functionality, my PR adds documentation in the following wiki page:
   
   (Link the GitHub wiki you added)
   
   ### Commits
   
   - My commits all reference appropriate Apache Helix GitHub issues in their subject lines. In addition, my commits follow the guidelines from "[How to write a good git commit message](http://chris.beams.io/posts/git-commit/)":
     1. Subject is separated from body by a blank line
     1. Subject is limited to 50 characters (not including Jira issue reference)
     1. Subject does not end with a period
     1. Subject uses the imperative mood ("add", not "adding")
     1. Body wraps at 72 characters
     1. Body explains "what" and "why", not "how"
   
   ### Code Quality
   
   - My diff has been formatted using helix-style.xml 
   (helix-style-intellij.xml if IntelliJ IDE is used)
   


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] xyuanlu commented on a diff in pull request #2488: add leader election API and dummy class

Posted by "xyuanlu (via GitHub)" <gi...@apache.org>.
xyuanlu commented on code in PR #2488:
URL: https://github.com/apache/helix/pull/2488#discussion_r1197311650


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/leaderelection/LeaderElectionClient.java:
##########
@@ -0,0 +1,125 @@
+package org.apache.helix.metaclient.recipes.leaderelection;
+
+/*
+ * 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.
+ */
+
+import java.util.List;
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+
+
+/**
+ * LeaderElectionClient does distributed leader election using CRUD and change notification APIs
+ * provided by underlying metadata client. Leader election config can provide many
+ * configs like base path for all participating nodes, sync/async mode, TTL etc.
+ *
+ * Participants join a leader election group by calling the following API.
+ * The Leader Election client maintains and elect an active leader from participant pool
+ * All participants wanted to be elected as leader joins a pool
+ * LeaderElection client maintains an active leader, by monitoring liveness of current leader and
+ * re-elect if needed and user no need to call elect or re-elect explicitly.
+ * This LeaderElection client will notify registered listeners for any leadership change.
+ */
+public class LeaderElectionClient {
+
+  /**
+   * Construct a LeaderElectionClient using a user passed in leaderElectionConfig. It creates a MetaClient
+   * instance underneath.
+   * When MetaClient is auto closed be cause of being disconnected and auto retry connection timed out, A new
+   * MetaClient instance will be created and keeps retry connection.
+   */
+  public LeaderElectionClient(MetaClientConfig metaClientConfig) {
+
+  }
+
+  /**
+   * Construct a LeaderElectionClient using a user passed in MetaClient object
+   * When MetaClient is auto closed be cause of being disconnected and auto retry connection timed out, user
+   * will need to create a new MetaClient and a new LeaderElectionClient instance.
+   */
+  public LeaderElectionClient(MetaClientInterface MetaClient) {
+
+  }
+
+  /**
+   * Returns true if current participant is able to acquire leadership.
+   */
+  public boolean isLeader(String leaderPath, String participant) {

Review Comment:
   TFTR. Updated.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] xyuanlu commented on a diff in pull request #2488: add leader election API and dummy class

Posted by "xyuanlu (via GitHub)" <gi...@apache.org>.
xyuanlu commented on code in PR #2488:
URL: https://github.com/apache/helix/pull/2488#discussion_r1197020365


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/leaderelection/LeaderElectionClient.java:
##########
@@ -0,0 +1,180 @@
+package org.apache.helix.metaclient.recipes.leaderelection;
+
+/*
+ * 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.
+ */
+
+import java.util.List;
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+
+
+/**
+ * LeaderElectionClient does distributed leader election using CRUD and change notification APIs
+ * provided by underlying metadata client. Leader election config can provide many
+ * configs like base path for all participating nodes, sync/async mode, TTL etc.
+ *
+ * Participants join a leader election group by calling the following API.
+ * The Leader Election client maintains and elect an active leader from participant pool.
+ * All participants wanted to be elected as leader joins a pool.
+ * LeaderElection client maintains an active leader, by monitoring liveness of current leader and
+ * re-elect if needed and user no need to call elect or re-elect explicitly.
+ * This LeaderElection client will notify registered listeners for any leadership change.
+ */
+public class LeaderElectionClient {
+
+  /**
+   * Construct a LeaderElectionClient using a user passed in leaderElectionConfig. It creates a MetaClient
+   * instance underneath.
+   * When MetaClient is auto closed be cause of being disconnected and auto retry connection timed out, A new
+   * MetaClient instance will be created and keeps retry connection.
+   *
+   * @param metaClientConfig The config used to create an metaclient.
+   */
+  public LeaderElectionClient(MetaClientConfig metaClientConfig) {
+
+  }
+
+  /**
+   * Construct a LeaderElectionClient using a user passed in MetaClient object
+   * When MetaClient is auto closed be cause of being disconnected and auto retry connection timed out, user
+   * will need to create a new MetaClient and a new LeaderElectionClient instance.
+   *
+   * @param metaClient metaClient object to be used.
+   */
+  public LeaderElectionClient(MetaClientInterface metaClient) {
+
+  }
+
+  /**
+   * Returns true if current participant is the current leadership.
+   */
+  public boolean isLeader(String leaderPath, String participant) {
+    return false;
+  }
+
+  /**
+   * Participants join a leader election group by calling the following API.
+   * The Leader Election client maintains and elect an active leader from the participant pool.
+   *
+   * @param leaderPath The path for leader election.
+   * @param participant The participant name to join pool.
+   * @return boolean indicating if the operation is succeeded.
+   */
+  public boolean joinLeaderElectionParticipantPool(String leaderPath, String participant) {
+    return false;
+  }
+
+  /**
+   * Participants join a leader election group by calling the following API.
+   * The Leader Election client maintains and elect an active leader from the participant pool.
+   *
+   * @param leaderPath The path for leader election.
+   * @param participant The participant name to join pool.
+   * @param userInfo Any additional information to associate with this participant.
+   * @return boolean indicating if the operation is succeeded.
+   */
+  public boolean joinLeaderElectionParticipantPool(String leaderPath, String participant,
+      Object userInfo) {
+    return false;
+  }
+
+  /**
+   * Any participant may exit the exitLeaderElectionParticipantPool by calling the API.
+   * If the participant is not the current leader, it leaves the pool and won't participant future
+   * leader election process.
+   * If the participant is the current leader, it leaves the pool and a new leader will be elected
+   * if there are other participants in the pool.
+   * Throws exception if the participant is not in the pool.
+   *
+   * @param leaderPath The path for leader election.
+   * @param participant The participant name to exit pool
+   * @return boolean indicating if the operation is succeeded.
+   *
+   * @throws RuntimeException If the participant did not join participant pool via this client. // TODO: define exp type
+   */
+  public boolean exitLeaderElectionParticipantPool(String leaderPath, String participant) {
+    return false;
+  }
+
+  /**
+   * Releases leadership for participant.
+   *
+   * @param leaderPath The path for leader election.
+   * @param participant The participant to release leadership.
+   *
+   * @throws RuntimeException if the leadership is not owned by this participant, or if the
+   *                          participant did not join participant pool via this client. // TODO: define exp type
+   */
+  public void relinquishLeader(String leaderPath, String participant) {

Review Comment:
   Thanks for the review. Not essentially. Only leader acquired through this client can be released through the client. 
   
   ```
   * @throws RuntimeException if the leadership is not owned by this participant, or if the
   *                          participant did not join participant pool via this client. // TODO: define exp type
   ```



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] junkaixue commented on a diff in pull request #2488: add leader election API and dummy class

Posted by "junkaixue (via GitHub)" <gi...@apache.org>.
junkaixue commented on code in PR #2488:
URL: https://github.com/apache/helix/pull/2488#discussion_r1199201664


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/leaderelection/LeaderElectionListenerInterface.java:
##########
@@ -0,0 +1,32 @@
+package org.apache.helix.metaclient.recipes.leaderelection;
+
+/*
+ * 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.
+ */
+
+/**
+ * It provides APIs for listener listening on events like a new leader is created or current
+ * leader node is deleted.
+ */
+public interface LeaderElectionListenerInterface {
+  // When new leader is elected:
+  //                             noLeader (null)                  ->    has leader (new leader name)
+  // When existing leader not leader anymore:

Review Comment:
   Question: does this mean the leader history will be persisted? Because if the leader is gone, usually we dont know which was prev leader.



##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/leaderelection/LeaderElectionClient.java:
##########
@@ -0,0 +1,180 @@
+package org.apache.helix.metaclient.recipes.leaderelection;

Review Comment:
   Since this is a descriptive class for APIs. It can be either interface or abstract class. To my understand, abstract class should be better. wdyt?



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] desaikomal commented on a diff in pull request #2488: add leader election API and dummy class

Posted by "desaikomal (via GitHub)" <gi...@apache.org>.
desaikomal commented on code in PR #2488:
URL: https://github.com/apache/helix/pull/2488#discussion_r1195506806


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/leaderelection/LeaderElectionClient.java:
##########
@@ -0,0 +1,125 @@
+package org.apache.helix.metaclient.recipes.leaderelection;
+
+/*
+ * 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.
+ */
+
+import java.util.List;
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+
+
+/**
+ * LeaderElectionClient does distributed leader election using CRUD and change notification APIs
+ * provided by underlying metadata client. Leader election config can provide many
+ * configs like base path for all participating nodes, sync/async mode, TTL etc.
+ *
+ * Participants join a leader election group by calling the following API.

Review Comment:
   nit: all sentences can end with '.' at the end. just a nit.



##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/leaderelection/LeaderElectionClient.java:
##########
@@ -0,0 +1,125 @@
+package org.apache.helix.metaclient.recipes.leaderelection;
+
+/*
+ * 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.
+ */
+
+import java.util.List;
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+
+
+/**
+ * LeaderElectionClient does distributed leader election using CRUD and change notification APIs
+ * provided by underlying metadata client. Leader election config can provide many
+ * configs like base path for all participating nodes, sync/async mode, TTL etc.
+ *
+ * Participants join a leader election group by calling the following API.
+ * The Leader Election client maintains and elect an active leader from participant pool
+ * All participants wanted to be elected as leader joins a pool
+ * LeaderElection client maintains an active leader, by monitoring liveness of current leader and
+ * re-elect if needed and user no need to call elect or re-elect explicitly.
+ * This LeaderElection client will notify registered listeners for any leadership change.
+ */
+public class LeaderElectionClient {
+
+  /**
+   * Construct a LeaderElectionClient using a user passed in leaderElectionConfig. It creates a MetaClient
+   * instance underneath.
+   * When MetaClient is auto closed be cause of being disconnected and auto retry connection timed out, A new
+   * MetaClient instance will be created and keeps retry connection.
+   */
+  public LeaderElectionClient(MetaClientConfig metaClientConfig) {
+
+  }
+
+  /**
+   * Construct a LeaderElectionClient using a user passed in MetaClient object
+   * When MetaClient is auto closed be cause of being disconnected and auto retry connection timed out, user
+   * will need to create a new MetaClient and a new LeaderElectionClient instance.
+   */
+  public LeaderElectionClient(MetaClientInterface MetaClient) {
+
+  }
+
+  /**
+   * Returns true if current participant is able to acquire leadership.
+   */
+  public boolean isLeader(String leaderPath, String participant) {
+    return false;
+  }
+
+  /**
+   * Participants join a leader election group by calling the following API.
+   * The Leader Election client maintains and elect an active leader from participant pool
+   */
+  public boolean joinLeaderElectionParticipantPool(String leaderPath, String participant,
+      Object userInfo) {
+    return false;
+  }
+
+  /**
+   * Any participant may exit the exitLeaderElectionParticipantPool by calling the API.
+   * If the participant is not the current leader, it leaves the pool and won't be elected as next leader.
+   * If the participant is the current leader, it leaves the pool and a new leader will be elected if there are other participants in the pool.
+   * Throws exception if the participant is not in the pool.
+   */
+  public boolean exitLeaderElectionParticipantPool(String leaderPath, String participant) {
+    return false;
+  }
+
+  /**
+   * Releases leadership for participant.
+   * Throws exception if the leadership is not owned by this participant.
+   */
+  public void relinquishLeader(String leaderPath, String participant) {
+  }
+
+  /**
+   * Get current leader nodes.
+   * Returns null if no Leader at a given point.
+   */
+  public String getLeader(String leaderPath) {
+    return null;
+  }
+
+  /**
+   * Return a list of hosts in participant pool
+   */
+  public List<String> getFollowers(String leaderPath) {

Review Comment:
   getParticipants sounds appropriate. this is not master/follower correct?



##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/leaderelection/LeaderElectionClient.java:
##########
@@ -0,0 +1,125 @@
+package org.apache.helix.metaclient.recipes.leaderelection;
+
+/*
+ * 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.
+ */
+
+import java.util.List;
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+
+
+/**
+ * LeaderElectionClient does distributed leader election using CRUD and change notification APIs
+ * provided by underlying metadata client. Leader election config can provide many
+ * configs like base path for all participating nodes, sync/async mode, TTL etc.
+ *
+ * Participants join a leader election group by calling the following API.
+ * The Leader Election client maintains and elect an active leader from participant pool
+ * All participants wanted to be elected as leader joins a pool
+ * LeaderElection client maintains an active leader, by monitoring liveness of current leader and
+ * re-elect if needed and user no need to call elect or re-elect explicitly.
+ * This LeaderElection client will notify registered listeners for any leadership change.
+ */
+public class LeaderElectionClient {
+
+  /**
+   * Construct a LeaderElectionClient using a user passed in leaderElectionConfig. It creates a MetaClient
+   * instance underneath.
+   * When MetaClient is auto closed be cause of being disconnected and auto retry connection timed out, A new
+   * MetaClient instance will be created and keeps retry connection.
+   */
+  public LeaderElectionClient(MetaClientConfig metaClientConfig) {
+
+  }
+
+  /**
+   * Construct a LeaderElectionClient using a user passed in MetaClient object
+   * When MetaClient is auto closed be cause of being disconnected and auto retry connection timed out, user
+   * will need to create a new MetaClient and a new LeaderElectionClient instance.
+   */
+  public LeaderElectionClient(MetaClientInterface MetaClient) {
+
+  }
+
+  /**
+   * Returns true if current participant is able to acquire leadership.
+   */
+  public boolean isLeader(String leaderPath, String participant) {
+    return false;
+  }
+
+  /**
+   * Participants join a leader election group by calling the following API.
+   * The Leader Election client maintains and elect an active leader from participant pool
+   */
+  public boolean joinLeaderElectionParticipantPool(String leaderPath, String participant,
+      Object userInfo) {
+    return false;
+  }
+
+  /**
+   * Any participant may exit the exitLeaderElectionParticipantPool by calling the API.
+   * If the participant is not the current leader, it leaves the pool and won't be elected as next leader.
+   * If the participant is the current leader, it leaves the pool and a new leader will be elected if there are other participants in the pool.
+   * Throws exception if the participant is not in the pool.
+   */
+  public boolean exitLeaderElectionParticipantPool(String leaderPath, String participant) {
+    return false;
+  }
+
+  /**
+   * Releases leadership for participant.
+   * Throws exception if the leadership is not owned by this participant.
+   */
+  public void relinquishLeader(String leaderPath, String participant) {
+  }
+
+  /**
+   * Get current leader nodes.
+   * Returns null if no Leader at a given point.
+   */
+  public String getLeader(String leaderPath) {
+    return null;
+  }
+
+  /**
+   * Return a list of hosts in participant pool
+   */
+  public List<String> getFollowers(String leaderPath) {
+    return null;
+  }
+
+  /**
+   * APIs to register/unregister listener to leader path. All the participants can listen to any
+   * leaderPath, Including leader going down or a new leader comes up.
+   * Whenever current leader for that leaderPath goes down (considering it's ephemeral entity which
+   * get's auto-deleted after TTL or session timeout) or a new leader comes up, it notifies all
+   * participants who have been listening on entryChange event.
+   */
+  public boolean subscribeLeadershipChanges(String leaderPath,

Review Comment:
   what is the contract - when leaderPath doesn't exist? will we create and wait for others to participate?



##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/leaderelection/LeaderInfo.java:
##########
@@ -0,0 +1,42 @@
+package org.apache.helix.metaclient.recipes.leaderelection;
+
+/*
+ * 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.
+ */
+
+import org.apache.helix.metaclient.datamodel.DataRecord;
+
+
+public class LeaderInfo {
+
+  private String  _leaderElectionGroupName;
+  private final DataRecord _record;
+
+
+  public LeaderInfo( String leaderElectionGroupName) {
+    _leaderElectionGroupName = leaderElectionGroupName;
+    _record = new DataRecord(_leaderElectionGroupName);
+  }
+
+  public enum LeaderAttribute {
+    LEADER_NAME,
+    PARTICIPANTS_LIST

Review Comment:
   Are these meta data which will be persisted?



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] xyuanlu commented on pull request #2488: add leader election API and dummy class

Posted by "xyuanlu (via GitHub)" <gi...@apache.org>.
xyuanlu commented on PR #2488:
URL: https://github.com/apache/helix/pull/2488#issuecomment-1555418163

   This PR is ready to be merged. Approved by @junkaixue and @desaikomal 
   Commit message:
   Add leader election API and dummy class


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] parakhnr commented on a diff in pull request #2488: add leader election API and dummy class

Posted by "parakhnr (via GitHub)" <gi...@apache.org>.
parakhnr commented on code in PR #2488:
URL: https://github.com/apache/helix/pull/2488#discussion_r1195465430


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/leaderelection/LeaderElectionClient.java:
##########
@@ -0,0 +1,125 @@
+package org.apache.helix.metaclient.recipes.leaderelection;
+
+/*
+ * 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.
+ */
+
+import java.util.List;
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+
+
+/**
+ * LeaderElectionClient does distributed leader election using CRUD and change notification APIs
+ * provided by underlying metadata client. Leader election config can provide many
+ * configs like base path for all participating nodes, sync/async mode, TTL etc.
+ *
+ * Participants join a leader election group by calling the following API.
+ * The Leader Election client maintains and elect an active leader from participant pool
+ * All participants wanted to be elected as leader joins a pool
+ * LeaderElection client maintains an active leader, by monitoring liveness of current leader and
+ * re-elect if needed and user no need to call elect or re-elect explicitly.
+ * This LeaderElection client will notify registered listeners for any leadership change.
+ */
+public class LeaderElectionClient {
+
+  /**
+   * Construct a LeaderElectionClient using a user passed in leaderElectionConfig. It creates a MetaClient
+   * instance underneath.
+   * When MetaClient is auto closed be cause of being disconnected and auto retry connection timed out, A new
+   * MetaClient instance will be created and keeps retry connection.
+   */
+  public LeaderElectionClient(MetaClientConfig metaClientConfig) {
+
+  }
+
+  /**
+   * Construct a LeaderElectionClient using a user passed in MetaClient object
+   * When MetaClient is auto closed be cause of being disconnected and auto retry connection timed out, user
+   * will need to create a new MetaClient and a new LeaderElectionClient instance.
+   */
+  public LeaderElectionClient(MetaClientInterface MetaClient) {
+
+  }
+
+  /**
+   * Returns true if current participant is able to acquire leadership.
+   */
+  public boolean isLeader(String leaderPath, String participant) {

Review Comment:
   Is it like if the participant is able to acquire leadership return true else false, if so can we rename this to `acquireLeadership` 



##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/leaderelection/LeaderElectionClient.java:
##########
@@ -0,0 +1,125 @@
+package org.apache.helix.metaclient.recipes.leaderelection;
+
+/*
+ * 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.
+ */
+
+import java.util.List;
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+
+
+/**
+ * LeaderElectionClient does distributed leader election using CRUD and change notification APIs
+ * provided by underlying metadata client. Leader election config can provide many
+ * configs like base path for all participating nodes, sync/async mode, TTL etc.
+ *
+ * Participants join a leader election group by calling the following API.
+ * The Leader Election client maintains and elect an active leader from participant pool
+ * All participants wanted to be elected as leader joins a pool
+ * LeaderElection client maintains an active leader, by monitoring liveness of current leader and
+ * re-elect if needed and user no need to call elect or re-elect explicitly.
+ * This LeaderElection client will notify registered listeners for any leadership change.
+ */
+public class LeaderElectionClient {
+
+  /**
+   * Construct a LeaderElectionClient using a user passed in leaderElectionConfig. It creates a MetaClient
+   * instance underneath.
+   * When MetaClient is auto closed be cause of being disconnected and auto retry connection timed out, A new
+   * MetaClient instance will be created and keeps retry connection.
+   */
+  public LeaderElectionClient(MetaClientConfig metaClientConfig) {
+
+  }
+
+  /**
+   * Construct a LeaderElectionClient using a user passed in MetaClient object
+   * When MetaClient is auto closed be cause of being disconnected and auto retry connection timed out, user
+   * will need to create a new MetaClient and a new LeaderElectionClient instance.
+   */
+  public LeaderElectionClient(MetaClientInterface MetaClient) {
+
+  }
+
+  /**
+   * Returns true if current participant is able to acquire leadership.
+   */
+  public boolean isLeader(String leaderPath, String participant) {
+    return false;
+  }
+
+  /**
+   * Participants join a leader election group by calling the following API.
+   * The Leader Election client maintains and elect an active leader from participant pool
+   */

Review Comment:
   nit: Can we add a comment `returns true if participant is able to join the pool else false` and also the descriptions for the parameters of the function. (Sorry but I couldn't get what is the purpose of `userInfo`).



##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/leaderelection/LeaderElectionClient.java:
##########
@@ -0,0 +1,125 @@
+package org.apache.helix.metaclient.recipes.leaderelection;
+
+/*
+ * 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.
+ */
+
+import java.util.List;
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+
+
+/**
+ * LeaderElectionClient does distributed leader election using CRUD and change notification APIs
+ * provided by underlying metadata client. Leader election config can provide many
+ * configs like base path for all participating nodes, sync/async mode, TTL etc.
+ *
+ * Participants join a leader election group by calling the following API.
+ * The Leader Election client maintains and elect an active leader from participant pool
+ * All participants wanted to be elected as leader joins a pool
+ * LeaderElection client maintains an active leader, by monitoring liveness of current leader and
+ * re-elect if needed and user no need to call elect or re-elect explicitly.
+ * This LeaderElection client will notify registered listeners for any leadership change.
+ */
+public class LeaderElectionClient {

Review Comment:
   One question for my understanding, do we want the `LeaderElectionClient` to be for a specific `path` or is it like this is one single stop for all the leadership related stuff for multiple paths. I feel that we should make it specific to a path, and sort of have a manager that manages multiple paths if wanted. In this way we keep the `LeaderElectionClient` clean. 
   
   Can you also add that to the description if possible.



##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/leaderelection/LeaderElectionClient.java:
##########
@@ -0,0 +1,125 @@
+package org.apache.helix.metaclient.recipes.leaderelection;
+
+/*
+ * 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.
+ */
+
+import java.util.List;
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+
+
+/**
+ * LeaderElectionClient does distributed leader election using CRUD and change notification APIs
+ * provided by underlying metadata client. Leader election config can provide many
+ * configs like base path for all participating nodes, sync/async mode, TTL etc.
+ *
+ * Participants join a leader election group by calling the following API.
+ * The Leader Election client maintains and elect an active leader from participant pool
+ * All participants wanted to be elected as leader joins a pool
+ * LeaderElection client maintains an active leader, by monitoring liveness of current leader and
+ * re-elect if needed and user no need to call elect or re-elect explicitly.
+ * This LeaderElection client will notify registered listeners for any leadership change.
+ */
+public class LeaderElectionClient {
+
+  /**
+   * Construct a LeaderElectionClient using a user passed in leaderElectionConfig. It creates a MetaClient
+   * instance underneath.
+   * When MetaClient is auto closed be cause of being disconnected and auto retry connection timed out, A new
+   * MetaClient instance will be created and keeps retry connection.
+   */
+  public LeaderElectionClient(MetaClientConfig metaClientConfig) {
+
+  }
+
+  /**
+   * Construct a LeaderElectionClient using a user passed in MetaClient object
+   * When MetaClient is auto closed be cause of being disconnected and auto retry connection timed out, user
+   * will need to create a new MetaClient and a new LeaderElectionClient instance.
+   */
+  public LeaderElectionClient(MetaClientInterface MetaClient) {
+
+  }
+
+  /**
+   * Returns true if current participant is able to acquire leadership.
+   */
+  public boolean isLeader(String leaderPath, String participant) {
+    return false;
+  }
+
+  /**
+   * Participants join a leader election group by calling the following API.
+   * The Leader Election client maintains and elect an active leader from participant pool
+   */
+  public boolean joinLeaderElectionParticipantPool(String leaderPath, String participant,
+      Object userInfo) {
+    return false;
+  }
+
+  /**
+   * Any participant may exit the exitLeaderElectionParticipantPool by calling the API.
+   * If the participant is not the current leader, it leaves the pool and won't be elected as next leader.
+   * If the participant is the current leader, it leaves the pool and a new leader will be elected if there are other participants in the pool.
+   * Throws exception if the participant is not in the pool.
+   */
+  public boolean exitLeaderElectionParticipantPool(String leaderPath, String participant) {
+    return false;
+  }
+
+  /**
+   * Releases leadership for participant.
+   * Throws exception if the leadership is not owned by this participant.
+   */
+  public void relinquishLeader(String leaderPath, String participant) {
+  }
+
+  /**
+   * Get current leader nodes.

Review Comment:
   nit: `node` and not `nodes` 😅 



##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/leaderelection/LeaderInfo.java:
##########
@@ -0,0 +1,42 @@
+package org.apache.helix.metaclient.recipes.leaderelection;
+
+/*
+ * 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.
+ */
+
+import org.apache.helix.metaclient.datamodel.DataRecord;
+
+
+public class LeaderInfo {

Review Comment:
   Can we add some description related to this class. Also should we return `LeaderInfo` when you do `LeaderElectionClient.getLeader()`?



##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/leaderelection/LeaderElectionClient.java:
##########
@@ -0,0 +1,125 @@
+package org.apache.helix.metaclient.recipes.leaderelection;
+
+/*
+ * 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.
+ */
+
+import java.util.List;
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+
+
+/**
+ * LeaderElectionClient does distributed leader election using CRUD and change notification APIs
+ * provided by underlying metadata client. Leader election config can provide many
+ * configs like base path for all participating nodes, sync/async mode, TTL etc.
+ *
+ * Participants join a leader election group by calling the following API.
+ * The Leader Election client maintains and elect an active leader from participant pool
+ * All participants wanted to be elected as leader joins a pool
+ * LeaderElection client maintains an active leader, by monitoring liveness of current leader and
+ * re-elect if needed and user no need to call elect or re-elect explicitly.
+ * This LeaderElection client will notify registered listeners for any leadership change.
+ */
+public class LeaderElectionClient {
+
+  /**
+   * Construct a LeaderElectionClient using a user passed in leaderElectionConfig. It creates a MetaClient
+   * instance underneath.
+   * When MetaClient is auto closed be cause of being disconnected and auto retry connection timed out, A new
+   * MetaClient instance will be created and keeps retry connection.
+   */
+  public LeaderElectionClient(MetaClientConfig metaClientConfig) {
+
+  }
+
+  /**
+   * Construct a LeaderElectionClient using a user passed in MetaClient object
+   * When MetaClient is auto closed be cause of being disconnected and auto retry connection timed out, user
+   * will need to create a new MetaClient and a new LeaderElectionClient instance.
+   */
+  public LeaderElectionClient(MetaClientInterface MetaClient) {
+
+  }
+
+  /**
+   * Returns true if current participant is able to acquire leadership.
+   */
+  public boolean isLeader(String leaderPath, String participant) {
+    return false;
+  }
+
+  /**
+   * Participants join a leader election group by calling the following API.
+   * The Leader Election client maintains and elect an active leader from participant pool
+   */
+  public boolean joinLeaderElectionParticipantPool(String leaderPath, String participant,
+      Object userInfo) {
+    return false;
+  }
+
+  /**
+   * Any participant may exit the exitLeaderElectionParticipantPool by calling the API.
+   * If the participant is not the current leader, it leaves the pool and won't be elected as next leader.

Review Comment:
   nit:  May be `If the participant is not the current leader, it leaves the pool and thus won't be able to participate in the leader election process moving forward.` cause `won't be elected as next leader sounds like this participant was going to be the next leader` 😅 



##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/leaderelection/LeaderElectionClient.java:
##########
@@ -0,0 +1,125 @@
+package org.apache.helix.metaclient.recipes.leaderelection;
+
+/*
+ * 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.
+ */
+
+import java.util.List;
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+
+
+/**
+ * LeaderElectionClient does distributed leader election using CRUD and change notification APIs
+ * provided by underlying metadata client. Leader election config can provide many
+ * configs like base path for all participating nodes, sync/async mode, TTL etc.
+ *
+ * Participants join a leader election group by calling the following API.
+ * The Leader Election client maintains and elect an active leader from participant pool
+ * All participants wanted to be elected as leader joins a pool
+ * LeaderElection client maintains an active leader, by monitoring liveness of current leader and
+ * re-elect if needed and user no need to call elect or re-elect explicitly.
+ * This LeaderElection client will notify registered listeners for any leadership change.
+ */
+public class LeaderElectionClient {
+
+  /**
+   * Construct a LeaderElectionClient using a user passed in leaderElectionConfig. It creates a MetaClient
+   * instance underneath.
+   * When MetaClient is auto closed be cause of being disconnected and auto retry connection timed out, A new
+   * MetaClient instance will be created and keeps retry connection.
+   */
+  public LeaderElectionClient(MetaClientConfig metaClientConfig) {
+
+  }
+
+  /**
+   * Construct a LeaderElectionClient using a user passed in MetaClient object
+   * When MetaClient is auto closed be cause of being disconnected and auto retry connection timed out, user
+   * will need to create a new MetaClient and a new LeaderElectionClient instance.
+   */
+  public LeaderElectionClient(MetaClientInterface MetaClient) {
+
+  }
+
+  /**
+   * Returns true if current participant is able to acquire leadership.
+   */
+  public boolean isLeader(String leaderPath, String participant) {
+    return false;
+  }
+
+  /**
+   * Participants join a leader election group by calling the following API.
+   * The Leader Election client maintains and elect an active leader from participant pool
+   */
+  public boolean joinLeaderElectionParticipantPool(String leaderPath, String participant,
+      Object userInfo) {
+    return false;
+  }
+
+  /**
+   * Any participant may exit the exitLeaderElectionParticipantPool by calling the API.
+   * If the participant is not the current leader, it leaves the pool and won't be elected as next leader.
+   * If the participant is the current leader, it leaves the pool and a new leader will be elected if there are other participants in the pool.
+   * Throws exception if the participant is not in the pool.
+   */
+  public boolean exitLeaderElectionParticipantPool(String leaderPath, String participant) {

Review Comment:
   nit: If you want you can rename this to `leave` instead of `exit` cause when you join something you leave 😂 



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] parakhnr commented on a diff in pull request #2488: add leader election API and dummy class

Posted by "parakhnr (via GitHub)" <gi...@apache.org>.
parakhnr commented on code in PR #2488:
URL: https://github.com/apache/helix/pull/2488#discussion_r1195459997


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/leaderelection/LeaderElectionListenerInterface.java:
##########
@@ -0,0 +1,32 @@
+package org.apache.helix.metaclient.recipes.leaderelection;
+
+/*
+ * 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.
+ */
+
+/**
+ * It provides APIs for listener listening on events like a new leader is created or current
+ * leader node is deleted.
+ */
+public interface LeaderElectionListenerInterface {
+  // When new leader is elected:
+  //                             noLeader (null)                  ->    has leader (new leader name)
+  // When existing leader not leader anymore:
+  //                             has Leader (prevleader name)     ->    no leader (null)
+  public void onLeadershipChange(String leaderPath, String prevLeader, String curLeader);

Review Comment:
   Going by the comment since either curLeader/prevLeader will be null on acquire/relinquish, I feel it's better to modify the signature of the function to use `onLeadershipChange(String leaderPath, String leader, Enum EVENT( RELINQUISH/ACQUIRE) event`. 
   
   In case I misunderstood, can you rename `noLeader/ hasLeader` to match the context of `prevLeader/cureLeader`



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] xyuanlu commented on a diff in pull request #2488: add leader election API and dummy class

Posted by "xyuanlu (via GitHub)" <gi...@apache.org>.
xyuanlu commented on code in PR #2488:
URL: https://github.com/apache/helix/pull/2488#discussion_r1195807605


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/leaderelection/LeaderElectionClient.java:
##########
@@ -0,0 +1,125 @@
+package org.apache.helix.metaclient.recipes.leaderelection;
+
+/*
+ * 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.
+ */
+
+import java.util.List;
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+
+
+/**
+ * LeaderElectionClient does distributed leader election using CRUD and change notification APIs
+ * provided by underlying metadata client. Leader election config can provide many
+ * configs like base path for all participating nodes, sync/async mode, TTL etc.
+ *
+ * Participants join a leader election group by calling the following API.
+ * The Leader Election client maintains and elect an active leader from participant pool
+ * All participants wanted to be elected as leader joins a pool
+ * LeaderElection client maintains an active leader, by monitoring liveness of current leader and
+ * re-elect if needed and user no need to call elect or re-elect explicitly.
+ * This LeaderElection client will notify registered listeners for any leadership change.
+ */
+public class LeaderElectionClient {

Review Comment:
   `LeaderElectionManager ` sounds like a centralized service. It is still a client.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] xyuanlu merged pull request #2488: add leader election API and dummy class

Posted by "xyuanlu (via GitHub)" <gi...@apache.org>.
xyuanlu merged PR #2488:
URL: https://github.com/apache/helix/pull/2488


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] xyuanlu commented on a diff in pull request #2488: add leader election API and dummy class

Posted by "xyuanlu (via GitHub)" <gi...@apache.org>.
xyuanlu commented on code in PR #2488:
URL: https://github.com/apache/helix/pull/2488#discussion_r1198570543


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/leaderelection/LeaderElectionClient.java:
##########
@@ -0,0 +1,180 @@
+package org.apache.helix.metaclient.recipes.leaderelection;
+
+/*
+ * 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.
+ */
+
+import java.util.List;
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+
+
+/**
+ * LeaderElectionClient does distributed leader election using CRUD and change notification APIs
+ * provided by underlying metadata client. Leader election config can provide many
+ * configs like base path for all participating nodes, sync/async mode, TTL etc.
+ *
+ * Participants join a leader election group by calling the following API.
+ * The Leader Election client maintains and elect an active leader from participant pool.
+ * All participants wanted to be elected as leader joins a pool.
+ * LeaderElection client maintains an active leader, by monitoring liveness of current leader and
+ * re-elect if needed and user no need to call elect or re-elect explicitly.
+ * This LeaderElection client will notify registered listeners for any leadership change.
+ */
+public class LeaderElectionClient {
+
+  /**
+   * Construct a LeaderElectionClient using a user passed in leaderElectionConfig. It creates a MetaClient
+   * instance underneath.
+   * When MetaClient is auto closed be cause of being disconnected and auto retry connection timed out, A new
+   * MetaClient instance will be created and keeps retry connection.
+   *
+   * @param metaClientConfig The config used to create an metaclient.
+   */
+  public LeaderElectionClient(MetaClientConfig metaClientConfig) {
+
+  }
+
+  /**
+   * Construct a LeaderElectionClient using a user passed in MetaClient object
+   * When MetaClient is auto closed be cause of being disconnected and auto retry connection timed out, user
+   * will need to create a new MetaClient and a new LeaderElectionClient instance.
+   *
+   * @param metaClient metaClient object to be used.
+   */
+  public LeaderElectionClient(MetaClientInterface metaClient) {
+
+  }
+
+  /**
+   * Returns true if current participant is the current leadership.
+   */
+  public boolean isLeader(String leaderPath, String participant) {
+    return false;
+  }
+
+  /**
+   * Participants join a leader election group by calling the following API.
+   * The Leader Election client maintains and elect an active leader from the participant pool.
+   *
+   * @param leaderPath The path for leader election.
+   * @param participant The participant name to join pool.
+   * @return boolean indicating if the operation is succeeded.
+   */
+  public boolean joinLeaderElectionParticipantPool(String leaderPath, String participant) {
+    return false;
+  }
+
+  /**
+   * Participants join a leader election group by calling the following API.
+   * The Leader Election client maintains and elect an active leader from the participant pool.
+   *
+   * @param leaderPath The path for leader election.
+   * @param participant The participant name to join pool.
+   * @param userInfo Any additional information to associate with this participant.
+   * @return boolean indicating if the operation is succeeded.
+   */
+  public boolean joinLeaderElectionParticipantPool(String leaderPath, String participant,
+      Object userInfo) {
+    return false;
+  }
+
+  /**
+   * Any participant may exit the exitLeaderElectionParticipantPool by calling the API.
+   * If the participant is not the current leader, it leaves the pool and won't participant future
+   * leader election process.
+   * If the participant is the current leader, it leaves the pool and a new leader will be elected
+   * if there are other participants in the pool.
+   * Throws exception if the participant is not in the pool.
+   *
+   * @param leaderPath The path for leader election.
+   * @param participant The participant name to exit pool
+   * @return boolean indicating if the operation is succeeded.
+   *
+   * @throws RuntimeException If the participant did not join participant pool via this client. // TODO: define exp type
+   */
+  public boolean exitLeaderElectionParticipantPool(String leaderPath, String participant) {
+    return false;
+  }
+
+  /**
+   * Releases leadership for participant.
+   *
+   * @param leaderPath The path for leader election.
+   * @param participant The participant to release leadership.
+   *
+   * @throws RuntimeException if the leadership is not owned by this participant, or if the
+   *                          participant did not join participant pool via this client. // TODO: define exp type
+   */
+  public void relinquishLeader(String leaderPath, String participant) {

Review Comment:
   Had an offline discussion. 
   One leadership election client is mapped to one host or participant. A participant may join multiple leader election group.  Updated the client API. 



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] parakhnr commented on a diff in pull request #2488: add leader election API and dummy class

Posted by "parakhnr (via GitHub)" <gi...@apache.org>.
parakhnr commented on code in PR #2488:
URL: https://github.com/apache/helix/pull/2488#discussion_r1196801107


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/leaderelection/LeaderElectionClient.java:
##########
@@ -0,0 +1,125 @@
+package org.apache.helix.metaclient.recipes.leaderelection;
+
+/*
+ * 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.
+ */
+
+import java.util.List;
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+
+
+/**
+ * LeaderElectionClient does distributed leader election using CRUD and change notification APIs
+ * provided by underlying metadata client. Leader election config can provide many
+ * configs like base path for all participating nodes, sync/async mode, TTL etc.
+ *
+ * Participants join a leader election group by calling the following API.
+ * The Leader Election client maintains and elect an active leader from participant pool
+ * All participants wanted to be elected as leader joins a pool
+ * LeaderElection client maintains an active leader, by monitoring liveness of current leader and
+ * re-elect if needed and user no need to call elect or re-elect explicitly.
+ * This LeaderElection client will notify registered listeners for any leadership change.
+ */
+public class LeaderElectionClient {

Review Comment:
   Ok. Thanks!



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] xyuanlu commented on a diff in pull request #2488: add leader election API and dummy class

Posted by "xyuanlu (via GitHub)" <gi...@apache.org>.
xyuanlu commented on code in PR #2488:
URL: https://github.com/apache/helix/pull/2488#discussion_r1195688526


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/leaderelection/LeaderElectionListenerInterface.java:
##########
@@ -0,0 +1,32 @@
+package org.apache.helix.metaclient.recipes.leaderelection;
+
+/*
+ * 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.
+ */
+
+/**
+ * It provides APIs for listener listening on events like a new leader is created or current
+ * leader node is deleted.
+ */
+public interface LeaderElectionListenerInterface {
+  // When new leader is elected:
+  //                             noLeader (null)                  ->    has leader (new leader name)
+  // When existing leader not leader anymore:
+  //                             has Leader (prevleader name)     ->    no leader (null)
+  public void onLeadershipChange(String leaderPath, String prevLeader, String curLeader);

Review Comment:
   As discussed offline. This API is more aligned with current use case. 



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] xyuanlu commented on a diff in pull request #2488: add leader election API and dummy class

Posted by "xyuanlu (via GitHub)" <gi...@apache.org>.
xyuanlu commented on code in PR #2488:
URL: https://github.com/apache/helix/pull/2488#discussion_r1195673650


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/leaderelection/LeaderElectionClient.java:
##########
@@ -0,0 +1,125 @@
+package org.apache.helix.metaclient.recipes.leaderelection;
+
+/*
+ * 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.
+ */
+
+import java.util.List;
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+
+
+/**
+ * LeaderElectionClient does distributed leader election using CRUD and change notification APIs
+ * provided by underlying metadata client. Leader election config can provide many
+ * configs like base path for all participating nodes, sync/async mode, TTL etc.
+ *
+ * Participants join a leader election group by calling the following API.
+ * The Leader Election client maintains and elect an active leader from participant pool
+ * All participants wanted to be elected as leader joins a pool
+ * LeaderElection client maintains an active leader, by monitoring liveness of current leader and
+ * re-elect if needed and user no need to call elect or re-elect explicitly.
+ * This LeaderElection client will notify registered listeners for any leadership change.
+ */
+public class LeaderElectionClient {
+
+  /**
+   * Construct a LeaderElectionClient using a user passed in leaderElectionConfig. It creates a MetaClient
+   * instance underneath.
+   * When MetaClient is auto closed be cause of being disconnected and auto retry connection timed out, A new
+   * MetaClient instance will be created and keeps retry connection.
+   */
+  public LeaderElectionClient(MetaClientConfig metaClientConfig) {
+
+  }
+
+  /**
+   * Construct a LeaderElectionClient using a user passed in MetaClient object
+   * When MetaClient is auto closed be cause of being disconnected and auto retry connection timed out, user
+   * will need to create a new MetaClient and a new LeaderElectionClient instance.
+   */
+  public LeaderElectionClient(MetaClientInterface MetaClient) {
+
+  }
+
+  /**
+   * Returns true if current participant is able to acquire leadership.
+   */
+  public boolean isLeader(String leaderPath, String participant) {

Review Comment:
   This client maintains an active leader from pool of participants. There is no explicit elect leader call.
   (Please refer to the comment part. )
   
   I will update the comment as this is ambiguous. 



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] xyuanlu commented on a diff in pull request #2488: add leader election API and dummy class

Posted by "xyuanlu (via GitHub)" <gi...@apache.org>.
xyuanlu commented on code in PR #2488:
URL: https://github.com/apache/helix/pull/2488#discussion_r1195663772


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/leaderelection/LeaderElectionClient.java:
##########
@@ -0,0 +1,125 @@
+package org.apache.helix.metaclient.recipes.leaderelection;
+
+/*
+ * 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.
+ */
+
+import java.util.List;
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+
+
+/**
+ * LeaderElectionClient does distributed leader election using CRUD and change notification APIs
+ * provided by underlying metadata client. Leader election config can provide many
+ * configs like base path for all participating nodes, sync/async mode, TTL etc.
+ *
+ * Participants join a leader election group by calling the following API.
+ * The Leader Election client maintains and elect an active leader from participant pool
+ * All participants wanted to be elected as leader joins a pool
+ * LeaderElection client maintains an active leader, by monitoring liveness of current leader and
+ * re-elect if needed and user no need to call elect or re-elect explicitly.
+ * This LeaderElection client will notify registered listeners for any leadership change.
+ */
+public class LeaderElectionClient {

Review Comment:
   Thanks for raising the question. 
   This client is designed to be used for 
   1. End user that need to leader ship election or coordination. There may be one leader election group or more than one leader election group on each app or JVM.
   2. Potentially an leader election service that hosts distributed leader election service.
   
   In either case, more than one path may co-exist. Having one client is more efficient. 



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] parakhnr commented on a diff in pull request #2488: add leader election API and dummy class

Posted by "parakhnr (via GitHub)" <gi...@apache.org>.
parakhnr commented on code in PR #2488:
URL: https://github.com/apache/helix/pull/2488#discussion_r1195751673


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/leaderelection/LeaderElectionClient.java:
##########
@@ -0,0 +1,125 @@
+package org.apache.helix.metaclient.recipes.leaderelection;
+
+/*
+ * 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.
+ */
+
+import java.util.List;
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+
+
+/**
+ * LeaderElectionClient does distributed leader election using CRUD and change notification APIs
+ * provided by underlying metadata client. Leader election config can provide many
+ * configs like base path for all participating nodes, sync/async mode, TTL etc.
+ *
+ * Participants join a leader election group by calling the following API.
+ * The Leader Election client maintains and elect an active leader from participant pool
+ * All participants wanted to be elected as leader joins a pool
+ * LeaderElection client maintains an active leader, by monitoring liveness of current leader and
+ * re-elect if needed and user no need to call elect or re-elect explicitly.
+ * This LeaderElection client will notify registered listeners for any leadership change.
+ */
+public class LeaderElectionClient {
+
+  /**
+   * Construct a LeaderElectionClient using a user passed in leaderElectionConfig. It creates a MetaClient
+   * instance underneath.
+   * When MetaClient is auto closed be cause of being disconnected and auto retry connection timed out, A new
+   * MetaClient instance will be created and keeps retry connection.
+   */
+  public LeaderElectionClient(MetaClientConfig metaClientConfig) {
+
+  }
+
+  /**
+   * Construct a LeaderElectionClient using a user passed in MetaClient object
+   * When MetaClient is auto closed be cause of being disconnected and auto retry connection timed out, user
+   * will need to create a new MetaClient and a new LeaderElectionClient instance.
+   */
+  public LeaderElectionClient(MetaClientInterface MetaClient) {
+
+  }
+
+  /**
+   * Returns true if current participant is able to acquire leadership.
+   */
+  public boolean isLeader(String leaderPath, String participant) {

Review Comment:
   Yeah, the comment made me think like it's acquiring leadership, so an update would be helpful. 
   
   When you say there is no explicit elect leader call, do you mean `joinLeaderElectionParticipantPool()` will trigger a leader election for that path if there isn't a leader elected? If so can you update the description of the method.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] xyuanlu commented on a diff in pull request #2488: add leader election API and dummy class

Posted by "xyuanlu (via GitHub)" <gi...@apache.org>.
xyuanlu commented on code in PR #2488:
URL: https://github.com/apache/helix/pull/2488#discussion_r1195810825


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/leaderelection/LeaderElectionClient.java:
##########
@@ -0,0 +1,125 @@
+package org.apache.helix.metaclient.recipes.leaderelection;
+
+/*
+ * 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.
+ */
+
+import java.util.List;
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+
+
+/**
+ * LeaderElectionClient does distributed leader election using CRUD and change notification APIs
+ * provided by underlying metadata client. Leader election config can provide many
+ * configs like base path for all participating nodes, sync/async mode, TTL etc.
+ *
+ * Participants join a leader election group by calling the following API.
+ * The Leader Election client maintains and elect an active leader from participant pool
+ * All participants wanted to be elected as leader joins a pool
+ * LeaderElection client maintains an active leader, by monitoring liveness of current leader and
+ * re-elect if needed and user no need to call elect or re-elect explicitly.
+ * This LeaderElection client will notify registered listeners for any leadership change.
+ */
+public class LeaderElectionClient {
+
+  /**
+   * Construct a LeaderElectionClient using a user passed in leaderElectionConfig. It creates a MetaClient
+   * instance underneath.
+   * When MetaClient is auto closed be cause of being disconnected and auto retry connection timed out, A new
+   * MetaClient instance will be created and keeps retry connection.
+   */
+  public LeaderElectionClient(MetaClientConfig metaClientConfig) {
+
+  }
+
+  /**
+   * Construct a LeaderElectionClient using a user passed in MetaClient object
+   * When MetaClient is auto closed be cause of being disconnected and auto retry connection timed out, user
+   * will need to create a new MetaClient and a new LeaderElectionClient instance.
+   */
+  public LeaderElectionClient(MetaClientInterface MetaClient) {
+
+  }
+
+  /**
+   * Returns true if current participant is able to acquire leadership.
+   */
+  public boolean isLeader(String leaderPath, String participant) {
+    return false;
+  }
+
+  /**
+   * Participants join a leader election group by calling the following API.
+   * The Leader Election client maintains and elect an active leader from participant pool
+   */
+  public boolean joinLeaderElectionParticipantPool(String leaderPath, String participant,
+      Object userInfo) {
+    return false;
+  }
+
+  /**
+   * Any participant may exit the exitLeaderElectionParticipantPool by calling the API.
+   * If the participant is not the current leader, it leaves the pool and won't be elected as next leader.

Review Comment:
   TFTR. Will update comments. 



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] junkaixue commented on a diff in pull request #2488: add leader election API and dummy class

Posted by "junkaixue (via GitHub)" <gi...@apache.org>.
junkaixue commented on code in PR #2488:
URL: https://github.com/apache/helix/pull/2488#discussion_r1197011508


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/leaderelection/LeaderElectionClient.java:
##########
@@ -0,0 +1,180 @@
+package org.apache.helix.metaclient.recipes.leaderelection;
+
+/*
+ * 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.
+ */
+
+import java.util.List;
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+
+
+/**
+ * LeaderElectionClient does distributed leader election using CRUD and change notification APIs
+ * provided by underlying metadata client. Leader election config can provide many
+ * configs like base path for all participating nodes, sync/async mode, TTL etc.
+ *
+ * Participants join a leader election group by calling the following API.
+ * The Leader Election client maintains and elect an active leader from participant pool.
+ * All participants wanted to be elected as leader joins a pool.
+ * LeaderElection client maintains an active leader, by monitoring liveness of current leader and
+ * re-elect if needed and user no need to call elect or re-elect explicitly.
+ * This LeaderElection client will notify registered listeners for any leadership change.
+ */
+public class LeaderElectionClient {
+
+  /**
+   * Construct a LeaderElectionClient using a user passed in leaderElectionConfig. It creates a MetaClient
+   * instance underneath.
+   * When MetaClient is auto closed be cause of being disconnected and auto retry connection timed out, A new
+   * MetaClient instance will be created and keeps retry connection.
+   *
+   * @param metaClientConfig The config used to create an metaclient.
+   */
+  public LeaderElectionClient(MetaClientConfig metaClientConfig) {
+
+  }
+
+  /**
+   * Construct a LeaderElectionClient using a user passed in MetaClient object
+   * When MetaClient is auto closed be cause of being disconnected and auto retry connection timed out, user
+   * will need to create a new MetaClient and a new LeaderElectionClient instance.
+   *
+   * @param metaClient metaClient object to be used.
+   */
+  public LeaderElectionClient(MetaClientInterface metaClient) {
+
+  }
+
+  /**
+   * Returns true if current participant is the current leadership.
+   */
+  public boolean isLeader(String leaderPath, String participant) {
+    return false;
+  }
+
+  /**
+   * Participants join a leader election group by calling the following API.
+   * The Leader Election client maintains and elect an active leader from the participant pool.
+   *
+   * @param leaderPath The path for leader election.
+   * @param participant The participant name to join pool.
+   * @return boolean indicating if the operation is succeeded.
+   */
+  public boolean joinLeaderElectionParticipantPool(String leaderPath, String participant) {
+    return false;
+  }
+
+  /**
+   * Participants join a leader election group by calling the following API.
+   * The Leader Election client maintains and elect an active leader from the participant pool.
+   *
+   * @param leaderPath The path for leader election.
+   * @param participant The participant name to join pool.
+   * @param userInfo Any additional information to associate with this participant.
+   * @return boolean indicating if the operation is succeeded.
+   */
+  public boolean joinLeaderElectionParticipantPool(String leaderPath, String participant,
+      Object userInfo) {
+    return false;
+  }
+
+  /**
+   * Any participant may exit the exitLeaderElectionParticipantPool by calling the API.
+   * If the participant is not the current leader, it leaves the pool and won't participant future
+   * leader election process.
+   * If the participant is the current leader, it leaves the pool and a new leader will be elected
+   * if there are other participants in the pool.
+   * Throws exception if the participant is not in the pool.
+   *
+   * @param leaderPath The path for leader election.
+   * @param participant The participant name to exit pool
+   * @return boolean indicating if the operation is succeeded.
+   *
+   * @throws RuntimeException If the participant did not join participant pool via this client. // TODO: define exp type
+   */
+  public boolean exitLeaderElectionParticipantPool(String leaderPath, String participant) {
+    return false;
+  }
+
+  /**
+   * Releases leadership for participant.
+   *
+   * @param leaderPath The path for leader election.
+   * @param participant The participant to release leadership.
+   *
+   * @throws RuntimeException if the leadership is not owned by this participant, or if the
+   *                          participant did not join participant pool via this client. // TODO: define exp type
+   */
+  public void relinquishLeader(String leaderPath, String participant) {

Review Comment:
   does this mean anyone can call this and release leader



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] xyuanlu commented on a diff in pull request #2488: add leader election API and dummy class

Posted by "xyuanlu (via GitHub)" <gi...@apache.org>.
xyuanlu commented on code in PR #2488:
URL: https://github.com/apache/helix/pull/2488#discussion_r1195692256


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/leaderelection/LeaderInfo.java:
##########
@@ -0,0 +1,42 @@
+package org.apache.helix.metaclient.recipes.leaderelection;
+
+/*
+ * 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.
+ */
+
+import org.apache.helix.metaclient.datamodel.DataRecord;
+
+
+public class LeaderInfo {
+
+  private String  _leaderElectionGroupName;
+  private final DataRecord _record;
+
+
+  public LeaderInfo( String leaderElectionGroupName) {
+    _leaderElectionGroupName = leaderElectionGroupName;
+    _record = new DataRecord(_leaderElectionGroupName);
+  }
+
+  public enum LeaderAttribute {
+    LEADER_NAME,
+    PARTICIPANTS_LIST

Review Comment:
   Yes. These will be written into the leader ZNode. 



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] parakhnr commented on a diff in pull request #2488: add leader election API and dummy class

Posted by "parakhnr (via GitHub)" <gi...@apache.org>.
parakhnr commented on code in PR #2488:
URL: https://github.com/apache/helix/pull/2488#discussion_r1195751421


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/leaderelection/LeaderElectionClient.java:
##########
@@ -0,0 +1,125 @@
+package org.apache.helix.metaclient.recipes.leaderelection;
+
+/*
+ * 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.
+ */
+
+import java.util.List;
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+
+
+/**
+ * LeaderElectionClient does distributed leader election using CRUD and change notification APIs
+ * provided by underlying metadata client. Leader election config can provide many
+ * configs like base path for all participating nodes, sync/async mode, TTL etc.
+ *
+ * Participants join a leader election group by calling the following API.
+ * The Leader Election client maintains and elect an active leader from participant pool
+ * All participants wanted to be elected as leader joins a pool
+ * LeaderElection client maintains an active leader, by monitoring liveness of current leader and
+ * re-elect if needed and user no need to call elect or re-elect explicitly.
+ * This LeaderElection client will notify registered listeners for any leadership change.
+ */
+public class LeaderElectionClient {

Review Comment:
   Gotcha! I agree that we should have one client. It's more from a naming perspective where I feel there should a `LeaderElectionManager` that manages `LeaderElectionClient which only deals with one single path` and consumers interact with the manager and not the client.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] xyuanlu commented on a diff in pull request #2488: add leader election API and dummy class

Posted by "xyuanlu (via GitHub)" <gi...@apache.org>.
xyuanlu commented on code in PR #2488:
URL: https://github.com/apache/helix/pull/2488#discussion_r1195812707


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/leaderelection/LeaderElectionClient.java:
##########
@@ -0,0 +1,125 @@
+package org.apache.helix.metaclient.recipes.leaderelection;
+
+/*
+ * 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.
+ */
+
+import java.util.List;
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+
+
+/**
+ * LeaderElectionClient does distributed leader election using CRUD and change notification APIs
+ * provided by underlying metadata client. Leader election config can provide many
+ * configs like base path for all participating nodes, sync/async mode, TTL etc.
+ *
+ * Participants join a leader election group by calling the following API.
+ * The Leader Election client maintains and elect an active leader from participant pool
+ * All participants wanted to be elected as leader joins a pool
+ * LeaderElection client maintains an active leader, by monitoring liveness of current leader and
+ * re-elect if needed and user no need to call elect or re-elect explicitly.
+ * This LeaderElection client will notify registered listeners for any leadership change.
+ */
+public class LeaderElectionClient {
+
+  /**
+   * Construct a LeaderElectionClient using a user passed in leaderElectionConfig. It creates a MetaClient
+   * instance underneath.
+   * When MetaClient is auto closed be cause of being disconnected and auto retry connection timed out, A new
+   * MetaClient instance will be created and keeps retry connection.
+   */
+  public LeaderElectionClient(MetaClientConfig metaClientConfig) {
+
+  }
+
+  /**
+   * Construct a LeaderElectionClient using a user passed in MetaClient object
+   * When MetaClient is auto closed be cause of being disconnected and auto retry connection timed out, user
+   * will need to create a new MetaClient and a new LeaderElectionClient instance.
+   */
+  public LeaderElectionClient(MetaClientInterface MetaClient) {
+
+  }
+
+  /**
+   * Returns true if current participant is able to acquire leadership.
+   */
+  public boolean isLeader(String leaderPath, String participant) {
+    return false;
+  }
+
+  /**
+   * Participants join a leader election group by calling the following API.
+   * The Leader Election client maintains and elect an active leader from participant pool
+   */
+  public boolean joinLeaderElectionParticipantPool(String leaderPath, String participant,
+      Object userInfo) {
+    return false;
+  }
+
+  /**
+   * Any participant may exit the exitLeaderElectionParticipantPool by calling the API.
+   * If the participant is not the current leader, it leaves the pool and won't be elected as next leader.
+   * If the participant is the current leader, it leaves the pool and a new leader will be elected if there are other participants in the pool.
+   * Throws exception if the participant is not in the pool.
+   */
+  public boolean exitLeaderElectionParticipantPool(String leaderPath, String participant) {
+    return false;
+  }
+
+  /**
+   * Releases leadership for participant.
+   * Throws exception if the leadership is not owned by this participant.
+   */
+  public void relinquishLeader(String leaderPath, String participant) {
+  }
+
+  /**
+   * Get current leader nodes.
+   * Returns null if no Leader at a given point.
+   */
+  public String getLeader(String leaderPath) {
+    return null;
+  }
+
+  /**
+   * Return a list of hosts in participant pool
+   */
+  public List<String> getFollowers(String leaderPath) {
+    return null;
+  }
+
+  /**
+   * APIs to register/unregister listener to leader path. All the participants can listen to any
+   * leaderPath, Including leader going down or a new leader comes up.
+   * Whenever current leader for that leaderPath goes down (considering it's ephemeral entity which
+   * get's auto-deleted after TTL or session timeout) or a new leader comes up, it notifies all
+   * participants who have been listening on entryChange event.
+   */
+  public boolean subscribeLeadershipChanges(String leaderPath,

Review Comment:
   subscribe for change is not a blocking call. If there is no leaderPath we still allow user to register listener and will notify if any change.  



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] xyuanlu commented on a diff in pull request #2488: add leader election API and dummy class

Posted by "xyuanlu (via GitHub)" <gi...@apache.org>.
xyuanlu commented on code in PR #2488:
URL: https://github.com/apache/helix/pull/2488#discussion_r1199519286


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/leaderelection/LeaderElectionClient.java:
##########
@@ -0,0 +1,180 @@
+package org.apache.helix.metaclient.recipes.leaderelection;

Review Comment:
   TFTR. We had an offline discussion. This class will be a solid class to use directly. Will add impl in following change. 



##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/leaderelection/LeaderElectionListenerInterface.java:
##########
@@ -0,0 +1,32 @@
+package org.apache.helix.metaclient.recipes.leaderelection;
+
+/*
+ * 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.
+ */
+
+/**
+ * It provides APIs for listener listening on events like a new leader is created or current
+ * leader node is deleted.
+ */
+public interface LeaderElectionListenerInterface {
+  // When new leader is elected:
+  //                             noLeader (null)                  ->    has leader (new leader name)
+  // When existing leader not leader anymore:

Review Comment:
   Yes. We will present leader info in memory. 



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] xyuanlu commented on a diff in pull request #2488: add leader election API and dummy class

Posted by "xyuanlu (via GitHub)" <gi...@apache.org>.
xyuanlu commented on code in PR #2488:
URL: https://github.com/apache/helix/pull/2488#discussion_r1197020365


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/leaderelection/LeaderElectionClient.java:
##########
@@ -0,0 +1,180 @@
+package org.apache.helix.metaclient.recipes.leaderelection;
+
+/*
+ * 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.
+ */
+
+import java.util.List;
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+
+
+/**
+ * LeaderElectionClient does distributed leader election using CRUD and change notification APIs
+ * provided by underlying metadata client. Leader election config can provide many
+ * configs like base path for all participating nodes, sync/async mode, TTL etc.
+ *
+ * Participants join a leader election group by calling the following API.
+ * The Leader Election client maintains and elect an active leader from participant pool.
+ * All participants wanted to be elected as leader joins a pool.
+ * LeaderElection client maintains an active leader, by monitoring liveness of current leader and
+ * re-elect if needed and user no need to call elect or re-elect explicitly.
+ * This LeaderElection client will notify registered listeners for any leadership change.
+ */
+public class LeaderElectionClient {
+
+  /**
+   * Construct a LeaderElectionClient using a user passed in leaderElectionConfig. It creates a MetaClient
+   * instance underneath.
+   * When MetaClient is auto closed be cause of being disconnected and auto retry connection timed out, A new
+   * MetaClient instance will be created and keeps retry connection.
+   *
+   * @param metaClientConfig The config used to create an metaclient.
+   */
+  public LeaderElectionClient(MetaClientConfig metaClientConfig) {
+
+  }
+
+  /**
+   * Construct a LeaderElectionClient using a user passed in MetaClient object
+   * When MetaClient is auto closed be cause of being disconnected and auto retry connection timed out, user
+   * will need to create a new MetaClient and a new LeaderElectionClient instance.
+   *
+   * @param metaClient metaClient object to be used.
+   */
+  public LeaderElectionClient(MetaClientInterface metaClient) {
+
+  }
+
+  /**
+   * Returns true if current participant is the current leadership.
+   */
+  public boolean isLeader(String leaderPath, String participant) {
+    return false;
+  }
+
+  /**
+   * Participants join a leader election group by calling the following API.
+   * The Leader Election client maintains and elect an active leader from the participant pool.
+   *
+   * @param leaderPath The path for leader election.
+   * @param participant The participant name to join pool.
+   * @return boolean indicating if the operation is succeeded.
+   */
+  public boolean joinLeaderElectionParticipantPool(String leaderPath, String participant) {
+    return false;
+  }
+
+  /**
+   * Participants join a leader election group by calling the following API.
+   * The Leader Election client maintains and elect an active leader from the participant pool.
+   *
+   * @param leaderPath The path for leader election.
+   * @param participant The participant name to join pool.
+   * @param userInfo Any additional information to associate with this participant.
+   * @return boolean indicating if the operation is succeeded.
+   */
+  public boolean joinLeaderElectionParticipantPool(String leaderPath, String participant,
+      Object userInfo) {
+    return false;
+  }
+
+  /**
+   * Any participant may exit the exitLeaderElectionParticipantPool by calling the API.
+   * If the participant is not the current leader, it leaves the pool and won't participant future
+   * leader election process.
+   * If the participant is the current leader, it leaves the pool and a new leader will be elected
+   * if there are other participants in the pool.
+   * Throws exception if the participant is not in the pool.
+   *
+   * @param leaderPath The path for leader election.
+   * @param participant The participant name to exit pool
+   * @return boolean indicating if the operation is succeeded.
+   *
+   * @throws RuntimeException If the participant did not join participant pool via this client. // TODO: define exp type
+   */
+  public boolean exitLeaderElectionParticipantPool(String leaderPath, String participant) {
+    return false;
+  }
+
+  /**
+   * Releases leadership for participant.
+   *
+   * @param leaderPath The path for leader election.
+   * @param participant The participant to release leadership.
+   *
+   * @throws RuntimeException if the leadership is not owned by this participant, or if the
+   *                          participant did not join participant pool via this client. // TODO: define exp type
+   */
+  public void relinquishLeader(String leaderPath, String participant) {

Review Comment:
   ```
   * @throws RuntimeException if the leadership is not owned by this participant, or if the
   *                          participant did not join participant pool via this client. // TODO: define exp type
   ```



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] xyuanlu commented on a diff in pull request #2488: add leader election API and dummy class

Posted by "xyuanlu (via GitHub)" <gi...@apache.org>.
xyuanlu commented on code in PR #2488:
URL: https://github.com/apache/helix/pull/2488#discussion_r1199209273


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/leaderelection/LeaderElectionListenerInterface.java:
##########
@@ -0,0 +1,32 @@
+package org.apache.helix.metaclient.recipes.leaderelection;
+
+/*
+ * 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.
+ */
+
+/**
+ * It provides APIs for listener listening on events like a new leader is created or current
+ * leader node is deleted.
+ */
+public interface LeaderElectionListenerInterface {
+  // When new leader is elected:
+  //                             noLeader (null)                  ->    has leader (new leader name)
+  // When existing leader not leader anymore:

Review Comment:
   If we register data listener on a particular entry, we will get an event for 'node is deleted' event when leader (node) is gone. We will know prev leader -> no leader change.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org