You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@helix.apache.org by GitBox <gi...@apache.org> on 2020/03/24 05:03:28 UTC

[GitHub] [helix] narendly opened a new pull request #908: Make Helix REST realm-aware

narendly opened a new pull request #908: Make Helix REST realm-aware
URL: https://github.com/apache/helix/pull/908
 
 
   ### Issues
   
   - [x] My PR addresses the following Helix issues and references them in the PR description:
   
   Fixes #907 
   
   ### Description
   
   - [x] Here are some details about my PR, including screenshots of any UI changes:
   
   Helix REST needs to start using a realm-aware ZkClient on multi-zk mode. Also it needs to become a listener on routing data because we don't want to restart the HelixRestServer every time we update the routing data.
   
   Changelist:
   1. Make ServerContext listen on routing data paths if run on multi-zk mode
   2. Make HelixRestServer use RealmAwareZkClient (FederatedZkClient) on multi-zk mode
   
   ### Tests
   
   - [x] The following tests are written for this issue:
   
   TestRoutingDataUpdate
   
   - [x] The following is the result of the "mvn test" command on the appropriate module:
   
   on helix-rest module:
   
   > [INFO] Results:
   > [INFO] 
   > [ERROR] Failures: 
   > [ERROR]   TestResourceAccessor.testResourceHealth:289 expected:<HEALTHY> but was:<UNHEALTHY>
   > [INFO] 
   > [ERROR] Tests run: 144, Failures: 1, Errors: 0, Skipped: 7
   > [INFO] 
   > [INFO] ------------------------------------------------------------------------
   > [INFO] BUILD FAILURE
   > [INFO] ------------------------------------------------------------------------
   > [INFO] Total time:  37.847 s
   > [INFO] Finished at: 2020-03-23T22:02:47-07:00
   > [INFO] ------------------------------------------------------------------------
   
   TestResourceAccessor is an unrelated failure that will be fixed in future iterations.
   
   ### Commits
   
   - [x] 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
   
   - [x] 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] dasahcc commented on a change in pull request #908: Make Helix REST realm-aware

Posted by GitBox <gi...@apache.org>.
dasahcc commented on a change in pull request #908: Make Helix REST realm-aware
URL: https://github.com/apache/helix/pull/908#discussion_r397492633
 
 

 ##########
 File path: helix-rest/src/main/java/org/apache/helix/rest/metadatastore/accessor/ZkRoutingDataReader.java
 ##########
 @@ -138,40 +122,51 @@ public synchronized void handleChildChange(String s, List<String> list) {
 
   @Override
   public synchronized void handleStateChanged(Watcher.Event.KeeperState state) {
-    if (_zkClient.isClosed()) {
+    if (_zkClient == null || _zkClient.isClosed()) {
       return;
     }
     _routingDataListener.refreshRoutingData(_namespace);
   }
 
   @Override
   public synchronized void handleNewSession(String sessionId) {
-    if (_zkClient.isClosed()) {
+    if (_zkClient == null || _zkClient.isClosed()) {
       return;
     }
     _routingDataListener.refreshRoutingData(_namespace);
   }
 
   @Override
   public synchronized void handleSessionEstablishmentError(Throwable error) {
-    if (_zkClient.isClosed()) {
+    if (_zkClient == null || _zkClient.isClosed()) {
       return;
     }
     _routingDataListener.refreshRoutingData(_namespace);
   }
 
+  /**
+   * Subscribes to the routing data paths using the provided ZkClient.
+   * @param zkClient
+   * @param childListener
+   * @param dataListener
+   */
+  public static void subscribeRoutingDataChanges(RealmAwareZkClient zkClient,
+      IZkChildListener childListener, IZkDataListener dataListener) {
+    zkClient.subscribeChildChanges(MetadataStoreRoutingConstants.ROUTING_DATA_PATH, childListener);
+    for (String child : zkClient.getChildren(MetadataStoreRoutingConstants.ROUTING_DATA_PATH)) {
+      zkClient.subscribeDataChanges(MetadataStoreRoutingConstants.ROUTING_DATA_PATH + "/" + child,
+          dataListener);
+    }
+  }
+
   private void handleResubscription() {
-    if (_zkClient.isClosed()) {
+    if (_zkClient == null || _zkClient.isClosed()) {
       return;
     }
 
     // Renew subscription
     _zkClient.unsubscribeAll();
-    _zkClient.subscribeChildChanges(MetadataStoreRoutingConstants.ROUTING_DATA_PATH, this);
-    for (String child : _zkClient.getChildren(MetadataStoreRoutingConstants.ROUTING_DATA_PATH)) {
-      _zkClient.subscribeDataChanges(MetadataStoreRoutingConstants.ROUTING_DATA_PATH + "/" + child,
-          this);
-    }
+    ZkRoutingDataReader.subscribeRoutingDataChanges(_zkClient, this, this);
 
 Review comment:
   Let's make a util class for this purpose.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] NealSun96 commented on a change in pull request #908: Make Helix REST realm-aware

Posted by GitBox <gi...@apache.org>.
NealSun96 commented on a change in pull request #908: Make Helix REST realm-aware
URL: https://github.com/apache/helix/pull/908#discussion_r397333305
 
 

 ##########
 File path: helix-rest/src/main/java/org/apache/helix/rest/metadatastore/accessor/ZkRoutingDataWriter.java
 ##########
 @@ -82,20 +82,14 @@ public ZkRoutingDataWriter(String namespace, String zkAddress) {
         .buildZkClient(new HelixZkClient.ZkConnectionConfig(zkAddress),
             new HelixZkClient.ZkClientConfig().setZkSerializer(new ZNRecordSerializer()));
 
-    // Ensure that ROUTING_DATA_PATH exists in ZK. If not, create
-    // create() semantic will fail if it already exists
-    try {
-      _zkClient.createPersistent(MetadataStoreRoutingConstants.ROUTING_DATA_PATH, true);
-    } catch (ZkNodeExistsException e) {
-      // This is okay
-    }
 
 Review comment:
   Likewise, without this section, some ZNode creation might fail if they didn't create the parent (ROUTING_DATA_PATH) and the parent doesn't exist. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] dasahcc commented on a change in pull request #908: Make Helix REST realm-aware

Posted by GitBox <gi...@apache.org>.
dasahcc commented on a change in pull request #908: Make Helix REST realm-aware
URL: https://github.com/apache/helix/pull/908#discussion_r397493791
 
 

 ##########
 File path: helix-rest/src/main/java/org/apache/helix/rest/server/ServerContext.java
 ##########
 @@ -20,42 +20,67 @@
  * under the License.
  */
 
-import java.util.HashMap;
+import java.io.IOException;
+import java.util.List;
 import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.helix.ConfigAccessor;
 import org.apache.helix.HelixAdmin;
 import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.HelixException;
 import org.apache.helix.InstanceType;
-import org.apache.helix.rest.metadatastore.ZkMetadataStoreDirectory;
-import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.SystemPropertyKeys;
 import org.apache.helix.manager.zk.ZKHelixAdmin;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
-import org.apache.helix.manager.zk.ZNRecordSerializer;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.zookeeper.impl.client.ZkClient;
-import org.apache.helix.zookeeper.api.client.HelixZkClient;
-import org.apache.helix.zookeeper.impl.factory.SharedZkClientFactory;
+import org.apache.helix.msdcommon.exception.InvalidRoutingDataException;
+import org.apache.helix.rest.metadatastore.ZkMetadataStoreDirectory;
+import org.apache.helix.rest.metadatastore.accessor.ZkRoutingDataReader;
 import org.apache.helix.task.TaskDriver;
 import org.apache.helix.tools.ClusterSetup;
+import org.apache.helix.zookeeper.api.client.HelixZkClient;
+import org.apache.helix.zookeeper.api.client.RealmAwareZkClient;
+import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+import org.apache.helix.zookeeper.impl.client.FederatedZkClient;
+import org.apache.helix.zookeeper.impl.client.ZkClient;
+import org.apache.helix.zookeeper.impl.factory.DedicatedZkClientFactory;
+import org.apache.helix.zookeeper.impl.factory.SharedZkClientFactory;
+import org.apache.helix.zookeeper.util.HttpRoutingDataReader;
+import org.apache.helix.zookeeper.zkclient.IZkChildListener;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.helix.zookeeper.zkclient.IZkStateListener;
 import org.apache.helix.zookeeper.zkclient.exception.ZkMarshallingError;
 import org.apache.helix.zookeeper.zkclient.serialize.ZkSerializer;
+import org.apache.zookeeper.Watcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
-public class ServerContext {
+public class ServerContext implements IZkDataListener, IZkChildListener, IZkStateListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ServerContext.class);
+
   private final String _zkAddr;
-  private HelixZkClient _zkClient;
-  private ZKHelixAdmin _zkHelixAdmin;
-  private ClusterSetup _clusterSetup;
-  private ConfigAccessor _configAccessor;
+  private volatile RealmAwareZkClient _zkClient;
 
 Review comment:
   The volatile keyword is trying to make variable sharing between threads. Do we have many threads for using these variables, which are not able to get the instance?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] narendly commented on issue #908: Make Helix REST realm-aware

Posted by GitBox <gi...@apache.org>.
narendly commented on issue #908: Make Helix REST realm-aware
URL: https://github.com/apache/helix/pull/908#issuecomment-604234992
 
 
   This PR is ready to be merged, approved by @dasahcc 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] narendly commented on a change in pull request #908: Make Helix REST realm-aware

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #908: Make Helix REST realm-aware
URL: https://github.com/apache/helix/pull/908#discussion_r397523411
 
 

 ##########
 File path: helix-rest/src/main/java/org/apache/helix/rest/metadatastore/accessor/ZkRoutingDataReader.java
 ##########
 @@ -138,40 +122,51 @@ public synchronized void handleChildChange(String s, List<String> list) {
 
   @Override
   public synchronized void handleStateChanged(Watcher.Event.KeeperState state) {
-    if (_zkClient.isClosed()) {
+    if (_zkClient == null || _zkClient.isClosed()) {
       return;
     }
     _routingDataListener.refreshRoutingData(_namespace);
   }
 
   @Override
   public synchronized void handleNewSession(String sessionId) {
-    if (_zkClient.isClosed()) {
+    if (_zkClient == null || _zkClient.isClosed()) {
       return;
     }
     _routingDataListener.refreshRoutingData(_namespace);
   }
 
   @Override
   public synchronized void handleSessionEstablishmentError(Throwable error) {
-    if (_zkClient.isClosed()) {
+    if (_zkClient == null || _zkClient.isClosed()) {
       return;
     }
     _routingDataListener.refreshRoutingData(_namespace);
   }
 
+  /**
+   * Subscribes to the routing data paths using the provided ZkClient.
+   * @param zkClient
+   * @param childListener
+   * @param dataListener
+   */
+  public static void subscribeRoutingDataChanges(RealmAwareZkClient zkClient,
+      IZkChildListener childListener, IZkDataListener dataListener) {
+    zkClient.subscribeChildChanges(MetadataStoreRoutingConstants.ROUTING_DATA_PATH, childListener);
+    for (String child : zkClient.getChildren(MetadataStoreRoutingConstants.ROUTING_DATA_PATH)) {
+      zkClient.subscribeDataChanges(MetadataStoreRoutingConstants.ROUTING_DATA_PATH + "/" + child,
+          dataListener);
+    }
+  }
+
   private void handleResubscription() {
-    if (_zkClient.isClosed()) {
+    if (_zkClient == null || _zkClient.isClosed()) {
       return;
     }
 
     // Renew subscription
     _zkClient.unsubscribeAll();
-    _zkClient.subscribeChildChanges(MetadataStoreRoutingConstants.ROUTING_DATA_PATH, this);
-    for (String child : _zkClient.getChildren(MetadataStoreRoutingConstants.ROUTING_DATA_PATH)) {
-      _zkClient.subscribeDataChanges(MetadataStoreRoutingConstants.ROUTING_DATA_PATH + "/" + child,
-          this);
-    }
+    ZkRoutingDataReader.subscribeRoutingDataChanges(_zkClient, this, this);
 
 Review comment:
   Moved to RealmAwareZkClient interface.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] narendly commented on a change in pull request #908: Make Helix REST realm-aware

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #908: Make Helix REST realm-aware
URL: https://github.com/apache/helix/pull/908#discussion_r397519930
 
 

 ##########
 File path: helix-rest/src/main/java/org/apache/helix/rest/metadatastore/ZkMetadataStoreDirectory.java
 ##########
 @@ -339,4 +348,22 @@ public synchronized void close() {
     _routingDataMap.clear();
     _zkMetadataStoreDirectoryInstance = null;
   }
+
+  /**
+   * Make sure the root routing data path exists. Also, register the routing ZK address.
+   * @param zkClient
+   */
+  private void createRoutingDataPath(HelixZkClient zkClient, String zkAddress) {
+    try {
+      zkClient.createPersistent(MetadataStoreRoutingConstants.ROUTING_DATA_PATH, true);
+    } catch (ZkNodeExistsException e) {
+      // The node already exists and it's okay
+    }
+    // Make sure ROUTING_DATA_PATH is mapped to the routing ZK so that FederatedZkClient used
+    // in Helix REST can subscribe to the routing data path
+    ZNRecord znRecord = new ZNRecord(MetadataStoreRoutingConstants.ROUTING_DATA_PATH);
 
 Review comment:
   That's okay. This is just an ID - it's for display purposes only.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] narendly merged pull request #908: Make Helix REST realm-aware

Posted by GitBox <gi...@apache.org>.
narendly merged pull request #908: Make Helix REST realm-aware
URL: https://github.com/apache/helix/pull/908
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] narendly commented on a change in pull request #908: Make Helix REST realm-aware

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #908: Make Helix REST realm-aware
URL: https://github.com/apache/helix/pull/908#discussion_r397523926
 
 

 ##########
 File path: helix-rest/src/main/java/org/apache/helix/rest/server/ServerContext.java
 ##########
 @@ -20,42 +20,67 @@
  * under the License.
  */
 
-import java.util.HashMap;
+import java.io.IOException;
+import java.util.List;
 import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.helix.ConfigAccessor;
 import org.apache.helix.HelixAdmin;
 import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.HelixException;
 import org.apache.helix.InstanceType;
-import org.apache.helix.rest.metadatastore.ZkMetadataStoreDirectory;
-import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.SystemPropertyKeys;
 import org.apache.helix.manager.zk.ZKHelixAdmin;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
-import org.apache.helix.manager.zk.ZNRecordSerializer;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.zookeeper.impl.client.ZkClient;
-import org.apache.helix.zookeeper.api.client.HelixZkClient;
-import org.apache.helix.zookeeper.impl.factory.SharedZkClientFactory;
+import org.apache.helix.msdcommon.exception.InvalidRoutingDataException;
+import org.apache.helix.rest.metadatastore.ZkMetadataStoreDirectory;
+import org.apache.helix.rest.metadatastore.accessor.ZkRoutingDataReader;
 import org.apache.helix.task.TaskDriver;
 import org.apache.helix.tools.ClusterSetup;
+import org.apache.helix.zookeeper.api.client.HelixZkClient;
+import org.apache.helix.zookeeper.api.client.RealmAwareZkClient;
+import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+import org.apache.helix.zookeeper.impl.client.FederatedZkClient;
+import org.apache.helix.zookeeper.impl.client.ZkClient;
+import org.apache.helix.zookeeper.impl.factory.DedicatedZkClientFactory;
+import org.apache.helix.zookeeper.impl.factory.SharedZkClientFactory;
+import org.apache.helix.zookeeper.util.HttpRoutingDataReader;
+import org.apache.helix.zookeeper.zkclient.IZkChildListener;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.helix.zookeeper.zkclient.IZkStateListener;
 import org.apache.helix.zookeeper.zkclient.exception.ZkMarshallingError;
 import org.apache.helix.zookeeper.zkclient.serialize.ZkSerializer;
+import org.apache.zookeeper.Watcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
-public class ServerContext {
+public class ServerContext implements IZkDataListener, IZkChildListener, IZkStateListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ServerContext.class);
+
   private final String _zkAddr;
-  private HelixZkClient _zkClient;
-  private ZKHelixAdmin _zkHelixAdmin;
-  private ClusterSetup _clusterSetup;
-  private ConfigAccessor _configAccessor;
+  private volatile RealmAwareZkClient _zkClient;
 
 Review comment:
   Yes. Jersey spins up a thread to serve each incoming request.
   
   Also volatile is used to as a double-checked locking idiom. So it is needed here for lazy initialization.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] narendly commented on a change in pull request #908: Make Helix REST realm-aware

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #908: Make Helix REST realm-aware
URL: https://github.com/apache/helix/pull/908#discussion_r397398015
 
 

 ##########
 File path: helix-rest/src/main/java/org/apache/helix/rest/metadatastore/accessor/ZkRoutingDataReader.java
 ##########
 @@ -58,25 +58,9 @@ public ZkRoutingDataReader(String namespace, String zkAddress,
     _zkClient = DedicatedZkClientFactory.getInstance()
         .buildZkClient(new HelixZkClient.ZkConnectionConfig(zkAddress),
             new HelixZkClient.ZkClientConfig().setZkSerializer(new ZNRecordSerializer()));
-
-    // Ensure that ROUTING_DATA_PATH exists in ZK. If not, create
-    // create() semantic will fail if it already exists
-    try {
-      _zkClient.createPersistent(MetadataStoreRoutingConstants.ROUTING_DATA_PATH, true);
-    } catch (ZkNodeExistsException e) {
-      // This is okay
-    }
 
 Review comment:
   Adding it back.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] dasahcc commented on a change in pull request #908: Make Helix REST realm-aware

Posted by GitBox <gi...@apache.org>.
dasahcc commented on a change in pull request #908: Make Helix REST realm-aware
URL: https://github.com/apache/helix/pull/908#discussion_r397534613
 
 

 ##########
 File path: helix-rest/src/main/java/org/apache/helix/rest/server/ServerContext.java
 ##########
 @@ -20,42 +20,67 @@
  * under the License.
  */
 
-import java.util.HashMap;
+import java.io.IOException;
+import java.util.List;
 import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.helix.ConfigAccessor;
 import org.apache.helix.HelixAdmin;
 import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.HelixException;
 import org.apache.helix.InstanceType;
-import org.apache.helix.rest.metadatastore.ZkMetadataStoreDirectory;
-import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.SystemPropertyKeys;
 import org.apache.helix.manager.zk.ZKHelixAdmin;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
-import org.apache.helix.manager.zk.ZNRecordSerializer;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.zookeeper.impl.client.ZkClient;
-import org.apache.helix.zookeeper.api.client.HelixZkClient;
-import org.apache.helix.zookeeper.impl.factory.SharedZkClientFactory;
+import org.apache.helix.msdcommon.exception.InvalidRoutingDataException;
+import org.apache.helix.rest.metadatastore.ZkMetadataStoreDirectory;
+import org.apache.helix.rest.metadatastore.accessor.ZkRoutingDataReader;
 import org.apache.helix.task.TaskDriver;
 import org.apache.helix.tools.ClusterSetup;
+import org.apache.helix.zookeeper.api.client.HelixZkClient;
+import org.apache.helix.zookeeper.api.client.RealmAwareZkClient;
+import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+import org.apache.helix.zookeeper.impl.client.FederatedZkClient;
+import org.apache.helix.zookeeper.impl.client.ZkClient;
+import org.apache.helix.zookeeper.impl.factory.DedicatedZkClientFactory;
+import org.apache.helix.zookeeper.impl.factory.SharedZkClientFactory;
+import org.apache.helix.zookeeper.util.HttpRoutingDataReader;
+import org.apache.helix.zookeeper.zkclient.IZkChildListener;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.helix.zookeeper.zkclient.IZkStateListener;
 import org.apache.helix.zookeeper.zkclient.exception.ZkMarshallingError;
 import org.apache.helix.zookeeper.zkclient.serialize.ZkSerializer;
+import org.apache.zookeeper.Watcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
-public class ServerContext {
+public class ServerContext implements IZkDataListener, IZkChildListener, IZkStateListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ServerContext.class);
+
   private final String _zkAddr;
-  private HelixZkClient _zkClient;
-  private ZKHelixAdmin _zkHelixAdmin;
-  private ClusterSetup _clusterSetup;
-  private ConfigAccessor _configAccessor;
+  private volatile RealmAwareZkClient _zkClient;
+
+  private volatile ZKHelixAdmin _zkHelixAdmin;
+  private volatile ClusterSetup _clusterSetup;
+  private volatile ConfigAccessor _configAccessor;
   // A lazily-initialized base data accessor that reads/writes byte array to ZK
   // TODO: Only read (deserialize) is supported at this time. This baseDataAccessor should support write (serialize) as needs arise
   private volatile ZkBaseDataAccessor<byte[]> _byteArrayZkBaseDataAccessor;
   // 1 Cluster name will correspond to 1 helix data accessor
   private final Map<String, HelixDataAccessor> _helixDataAccessorPool;
   // 1 Cluster name will correspond to 1 task driver
   private final Map<String, TaskDriver> _taskDriverPool;
 
 Review comment:
   Can you elaborate why it is not backward compatible? I think FederatedZKClient can support single ZK access, right? We can make it accept single ZK while combining with multi ZK access. Otherwise, we already started multiple Servlet for different ZK works fine and I did not get the idea of making REST realm-aware.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] dasahcc commented on a change in pull request #908: Make Helix REST realm-aware

Posted by GitBox <gi...@apache.org>.
dasahcc commented on a change in pull request #908: Make Helix REST realm-aware
URL: https://github.com/apache/helix/pull/908#discussion_r397491416
 
 

 ##########
 File path: helix-rest/src/main/java/org/apache/helix/rest/metadatastore/ZkMetadataStoreDirectory.java
 ##########
 @@ -339,4 +348,22 @@ public synchronized void close() {
     _routingDataMap.clear();
     _zkMetadataStoreDirectoryInstance = null;
   }
+
+  /**
+   * Make sure the root routing data path exists. Also, register the routing ZK address.
+   * @param zkClient
+   */
+  private void createRoutingDataPath(HelixZkClient zkClient, String zkAddress) {
+    try {
+      zkClient.createPersistent(MetadataStoreRoutingConstants.ROUTING_DATA_PATH, true);
+    } catch (ZkNodeExistsException e) {
+      // The node already exists and it's okay
+    }
+    // Make sure ROUTING_DATA_PATH is mapped to the routing ZK so that FederatedZkClient used
+    // in Helix REST can subscribe to the routing data path
+    ZNRecord znRecord = new ZNRecord(MetadataStoreRoutingConstants.ROUTING_DATA_PATH);
 
 Review comment:
   The path contains"/" symbol. Would that be OK?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] NealSun96 commented on a change in pull request #908: Make Helix REST realm-aware

Posted by GitBox <gi...@apache.org>.
NealSun96 commented on a change in pull request #908: Make Helix REST realm-aware
URL: https://github.com/apache/helix/pull/908#discussion_r397330048
 
 

 ##########
 File path: helix-rest/src/main/java/org/apache/helix/rest/metadatastore/accessor/ZkRoutingDataReader.java
 ##########
 @@ -58,25 +58,9 @@ public ZkRoutingDataReader(String namespace, String zkAddress,
     _zkClient = DedicatedZkClientFactory.getInstance()
         .buildZkClient(new HelixZkClient.ZkConnectionConfig(zkAddress),
             new HelixZkClient.ZkClientConfig().setZkSerializer(new ZNRecordSerializer()));
-
-    // Ensure that ROUTING_DATA_PATH exists in ZK. If not, create
-    // create() semantic will fail if it already exists
-    try {
-      _zkClient.createPersistent(MetadataStoreRoutingConstants.ROUTING_DATA_PATH, true);
-    } catch (ZkNodeExistsException e) {
-      // This is okay
-    }
 
 Review comment:
   Why is this section removed? In a scenario where `ZkRoutingDataReader` is instantiated by itself, `subscribeRoutingDataChanges` might encounter `NoNodeException` if we don't ensure that ROUTING_DATA_PATH exists in ZK.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] narendly commented on a change in pull request #908: Make Helix REST realm-aware

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #908: Make Helix REST realm-aware
URL: https://github.com/apache/helix/pull/908#discussion_r397523278
 
 

 ##########
 File path: helix-rest/src/main/java/org/apache/helix/rest/metadatastore/accessor/ZkRoutingDataReader.java
 ##########
 @@ -138,40 +122,51 @@ public synchronized void handleChildChange(String s, List<String> list) {
 
   @Override
   public synchronized void handleStateChanged(Watcher.Event.KeeperState state) {
-    if (_zkClient.isClosed()) {
+    if (_zkClient == null || _zkClient.isClosed()) {
       return;
     }
     _routingDataListener.refreshRoutingData(_namespace);
   }
 
   @Override
   public synchronized void handleNewSession(String sessionId) {
-    if (_zkClient.isClosed()) {
+    if (_zkClient == null || _zkClient.isClosed()) {
       return;
     }
     _routingDataListener.refreshRoutingData(_namespace);
   }
 
   @Override
   public synchronized void handleSessionEstablishmentError(Throwable error) {
-    if (_zkClient.isClosed()) {
+    if (_zkClient == null || _zkClient.isClosed()) {
       return;
     }
     _routingDataListener.refreshRoutingData(_namespace);
   }
 
+  /**
+   * Subscribes to the routing data paths using the provided ZkClient.
+   * @param zkClient
+   * @param childListener
+   * @param dataListener
+   */
+  public static void subscribeRoutingDataChanges(RealmAwareZkClient zkClient,
 
 Review comment:
   As suggested, I moved it to RealmAwareZkClient.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] narendly commented on a change in pull request #908: Make Helix REST realm-aware

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #908: Make Helix REST realm-aware
URL: https://github.com/apache/helix/pull/908#discussion_r397397806
 
 

 ##########
 File path: helix-rest/src/main/java/org/apache/helix/rest/server/ServerContext.java
 ##########
 @@ -165,5 +238,93 @@ public void close() {
     if (_zkMetadataStoreDirectory != null) {
       _zkMetadataStoreDirectory.close();
     }
+    if (_zkClientForListener != null) {
+      _zkClientForListener.close();
+    }
+  }
+
+  @Override
+  public void handleChildChange(String parentPath, List<String> currentChilds) {
+    if (_zkClientForListener == null || _zkClientForListener.isClosed()) {
+      return;
+    }
+    // Resubscribe
+    _zkClientForListener.unsubscribeAll();
+    ZkRoutingDataReader.subscribeRoutingDataChanges(_zkClientForListener, this, this);
+    resetZkResources();
+  }
+
+  @Override
+  public void handleDataChange(String dataPath, Object data) {
+    if (_zkClientForListener == null || _zkClientForListener.isClosed()) {
+      return;
+    }
+    resetZkResources();
+  }
+
+  @Override
+  public void handleDataDeleted(String dataPath) {
+    if (_zkClientForListener == null || _zkClientForListener.isClosed()) {
+      return;
+    }
+    // Resubscribe
+    _zkClientForListener.unsubscribeAll();
+    ZkRoutingDataReader.subscribeRoutingDataChanges(_zkClientForListener, this, this);
+    resetZkResources();
 
 Review comment:
   Updated. It's hard to reduce duplication further because we need to sneak in unsubscribe/resubscribe for some callback functions.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] dasahcc commented on a change in pull request #908: Make Helix REST realm-aware

Posted by GitBox <gi...@apache.org>.
dasahcc commented on a change in pull request #908: Make Helix REST realm-aware
URL: https://github.com/apache/helix/pull/908#discussion_r397533584
 
 

 ##########
 File path: helix-rest/src/main/java/org/apache/helix/rest/metadatastore/ZkMetadataStoreDirectory.java
 ##########
 @@ -339,4 +348,22 @@ public synchronized void close() {
     _routingDataMap.clear();
     _zkMetadataStoreDirectoryInstance = null;
   }
+
+  /**
+   * Make sure the root routing data path exists. Also, register the routing ZK address.
+   * @param zkClient
+   */
+  private void createRoutingDataPath(HelixZkClient zkClient, String zkAddress) {
+    try {
+      zkClient.createPersistent(MetadataStoreRoutingConstants.ROUTING_DATA_PATH, true);
+    } catch (ZkNodeExistsException e) {
+      // The node already exists and it's okay
+    }
+    // Make sure ROUTING_DATA_PATH is mapped to the routing ZK so that FederatedZkClient used
+    // in Helix REST can subscribe to the routing data path
+    ZNRecord znRecord = new ZNRecord(MetadataStoreRoutingConstants.ROUTING_DATA_PATH);
 
 Review comment:
   That was confusing user. In our convention, the path to a certain will be "/ID of ZNode1/ID of ZNode2".  Usually we dont have folder name is one thing and id of ZNRecord is another thing. Suggest to make it consisnt.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] NealSun96 commented on a change in pull request #908: Make Helix REST realm-aware

Posted by GitBox <gi...@apache.org>.
NealSun96 commented on a change in pull request #908: Make Helix REST realm-aware
URL: https://github.com/apache/helix/pull/908#discussion_r397343442
 
 

 ##########
 File path: helix-rest/src/main/java/org/apache/helix/rest/server/ServerContext.java
 ##########
 @@ -165,5 +238,93 @@ public void close() {
     if (_zkMetadataStoreDirectory != null) {
       _zkMetadataStoreDirectory.close();
     }
+    if (_zkClientForListener != null) {
+      _zkClientForListener.close();
+    }
+  }
+
+  @Override
+  public void handleChildChange(String parentPath, List<String> currentChilds) {
+    if (_zkClientForListener == null || _zkClientForListener.isClosed()) {
+      return;
+    }
+    // Resubscribe
+    _zkClientForListener.unsubscribeAll();
+    ZkRoutingDataReader.subscribeRoutingDataChanges(_zkClientForListener, this, this);
+    resetZkResources();
+  }
+
+  @Override
+  public void handleDataChange(String dataPath, Object data) {
+    if (_zkClientForListener == null || _zkClientForListener.isClosed()) {
+      return;
+    }
+    resetZkResources();
+  }
+
+  @Override
+  public void handleDataDeleted(String dataPath) {
+    if (_zkClientForListener == null || _zkClientForListener.isClosed()) {
+      return;
+    }
+    // Resubscribe
+    _zkClientForListener.unsubscribeAll();
+    ZkRoutingDataReader.subscribeRoutingDataChanges(_zkClientForListener, this, this);
+    resetZkResources();
 
 Review comment:
   Is it possible to reduce duplication among the handle functions? 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] narendly commented on a change in pull request #908: Make Helix REST realm-aware

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #908: Make Helix REST realm-aware
URL: https://github.com/apache/helix/pull/908#discussion_r397397897
 
 

 ##########
 File path: helix-rest/src/main/java/org/apache/helix/rest/metadatastore/accessor/ZkRoutingDataWriter.java
 ##########
 @@ -82,20 +82,14 @@ public ZkRoutingDataWriter(String namespace, String zkAddress) {
         .buildZkClient(new HelixZkClient.ZkConnectionConfig(zkAddress),
             new HelixZkClient.ZkClientConfig().setZkSerializer(new ZNRecordSerializer()));
 
-    // Ensure that ROUTING_DATA_PATH exists in ZK. If not, create
-    // create() semantic will fail if it already exists
-    try {
-      _zkClient.createPersistent(MetadataStoreRoutingConstants.ROUTING_DATA_PATH, true);
-    } catch (ZkNodeExistsException e) {
-      // This is okay
-    }
 
 Review comment:
   I'll re-add.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] dasahcc commented on a change in pull request #908: Make Helix REST realm-aware

Posted by GitBox <gi...@apache.org>.
dasahcc commented on a change in pull request #908: Make Helix REST realm-aware
URL: https://github.com/apache/helix/pull/908#discussion_r398055555
 
 

 ##########
 File path: helix-rest/src/main/java/org/apache/helix/rest/server/ServerContext.java
 ##########
 @@ -20,42 +20,67 @@
  * under the License.
  */
 
-import java.util.HashMap;
+import java.io.IOException;
+import java.util.List;
 import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.helix.ConfigAccessor;
 import org.apache.helix.HelixAdmin;
 import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.HelixException;
 import org.apache.helix.InstanceType;
-import org.apache.helix.rest.metadatastore.ZkMetadataStoreDirectory;
-import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.SystemPropertyKeys;
 import org.apache.helix.manager.zk.ZKHelixAdmin;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
-import org.apache.helix.manager.zk.ZNRecordSerializer;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.zookeeper.impl.client.ZkClient;
-import org.apache.helix.zookeeper.api.client.HelixZkClient;
-import org.apache.helix.zookeeper.impl.factory.SharedZkClientFactory;
+import org.apache.helix.msdcommon.exception.InvalidRoutingDataException;
+import org.apache.helix.rest.metadatastore.ZkMetadataStoreDirectory;
+import org.apache.helix.rest.metadatastore.accessor.ZkRoutingDataReader;
 import org.apache.helix.task.TaskDriver;
 import org.apache.helix.tools.ClusterSetup;
+import org.apache.helix.zookeeper.api.client.HelixZkClient;
+import org.apache.helix.zookeeper.api.client.RealmAwareZkClient;
+import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+import org.apache.helix.zookeeper.impl.client.FederatedZkClient;
+import org.apache.helix.zookeeper.impl.client.ZkClient;
+import org.apache.helix.zookeeper.impl.factory.DedicatedZkClientFactory;
+import org.apache.helix.zookeeper.impl.factory.SharedZkClientFactory;
+import org.apache.helix.zookeeper.util.HttpRoutingDataReader;
+import org.apache.helix.zookeeper.zkclient.IZkChildListener;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.helix.zookeeper.zkclient.IZkStateListener;
 import org.apache.helix.zookeeper.zkclient.exception.ZkMarshallingError;
 import org.apache.helix.zookeeper.zkclient.serialize.ZkSerializer;
+import org.apache.zookeeper.Watcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
-public class ServerContext {
+public class ServerContext implements IZkDataListener, IZkChildListener, IZkStateListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ServerContext.class);
+
   private final String _zkAddr;
-  private HelixZkClient _zkClient;
-  private ZKHelixAdmin _zkHelixAdmin;
-  private ClusterSetup _clusterSetup;
-  private ConfigAccessor _configAccessor;
+  private volatile RealmAwareZkClient _zkClient;
+
+  private volatile ZKHelixAdmin _zkHelixAdmin;
+  private volatile ClusterSetup _clusterSetup;
+  private volatile ConfigAccessor _configAccessor;
   // A lazily-initialized base data accessor that reads/writes byte array to ZK
   // TODO: Only read (deserialize) is supported at this time. This baseDataAccessor should support write (serialize) as needs arise
   private volatile ZkBaseDataAccessor<byte[]> _byteArrayZkBaseDataAccessor;
   // 1 Cluster name will correspond to 1 helix data accessor
   private final Map<String, HelixDataAccessor> _helixDataAccessorPool;
   // 1 Cluster name will correspond to 1 task driver
   private final Map<String, TaskDriver> _taskDriverPool;
 
 Review comment:
   I agree with that. But the routing logic should not be at service layer. Otherwise, each service using TaskDriver or ConfigAccessor will have their own routing logic. 
   
   TaskDriver / Config Accessor is instantiated from RealmAwareZKClient. Then can we make some smart things underneath for sharding key routing?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] narendly commented on a change in pull request #908: Make Helix REST realm-aware

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #908: Make Helix REST realm-aware
URL: https://github.com/apache/helix/pull/908#discussion_r398066168
 
 

 ##########
 File path: helix-rest/src/main/java/org/apache/helix/rest/server/ServerContext.java
 ##########
 @@ -20,42 +20,67 @@
  * under the License.
  */
 
-import java.util.HashMap;
+import java.io.IOException;
+import java.util.List;
 import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.helix.ConfigAccessor;
 import org.apache.helix.HelixAdmin;
 import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.HelixException;
 import org.apache.helix.InstanceType;
-import org.apache.helix.rest.metadatastore.ZkMetadataStoreDirectory;
-import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.SystemPropertyKeys;
 import org.apache.helix.manager.zk.ZKHelixAdmin;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
-import org.apache.helix.manager.zk.ZNRecordSerializer;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.zookeeper.impl.client.ZkClient;
-import org.apache.helix.zookeeper.api.client.HelixZkClient;
-import org.apache.helix.zookeeper.impl.factory.SharedZkClientFactory;
+import org.apache.helix.msdcommon.exception.InvalidRoutingDataException;
+import org.apache.helix.rest.metadatastore.ZkMetadataStoreDirectory;
+import org.apache.helix.rest.metadatastore.accessor.ZkRoutingDataReader;
 import org.apache.helix.task.TaskDriver;
 import org.apache.helix.tools.ClusterSetup;
+import org.apache.helix.zookeeper.api.client.HelixZkClient;
+import org.apache.helix.zookeeper.api.client.RealmAwareZkClient;
+import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+import org.apache.helix.zookeeper.impl.client.FederatedZkClient;
+import org.apache.helix.zookeeper.impl.client.ZkClient;
+import org.apache.helix.zookeeper.impl.factory.DedicatedZkClientFactory;
+import org.apache.helix.zookeeper.impl.factory.SharedZkClientFactory;
+import org.apache.helix.zookeeper.util.HttpRoutingDataReader;
+import org.apache.helix.zookeeper.zkclient.IZkChildListener;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.helix.zookeeper.zkclient.IZkStateListener;
 import org.apache.helix.zookeeper.zkclient.exception.ZkMarshallingError;
 import org.apache.helix.zookeeper.zkclient.serialize.ZkSerializer;
+import org.apache.zookeeper.Watcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
-public class ServerContext {
+public class ServerContext implements IZkDataListener, IZkChildListener, IZkStateListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ServerContext.class);
+
   private final String _zkAddr;
-  private HelixZkClient _zkClient;
-  private ZKHelixAdmin _zkHelixAdmin;
-  private ClusterSetup _clusterSetup;
-  private ConfigAccessor _configAccessor;
+  private volatile RealmAwareZkClient _zkClient;
+
+  private volatile ZKHelixAdmin _zkHelixAdmin;
+  private volatile ClusterSetup _clusterSetup;
+  private volatile ConfigAccessor _configAccessor;
   // A lazily-initialized base data accessor that reads/writes byte array to ZK
   // TODO: Only read (deserialize) is supported at this time. This baseDataAccessor should support write (serialize) as needs arise
   private volatile ZkBaseDataAccessor<byte[]> _byteArrayZkBaseDataAccessor;
   // 1 Cluster name will correspond to 1 helix data accessor
   private final Map<String, HelixDataAccessor> _helixDataAccessorPool;
   // 1 Cluster name will correspond to 1 task driver
   private final Map<String, TaskDriver> _taskDriverPool;
 
 Review comment:
   I understand your concern and where you're coming from.  But it's still difficult. TaskDriver and ConfigAccessor are different APIs by design.
   - TaskDriver/HelixManager/HelixDataAccessor -> only serve 1 single Helix cluster
   - ConfigAccessor/ClusterSetup/etc.. -> can serve many Helix clusters
   So we need to keep a pool regardless.
   To help you understand:
   
   ```
     public TaskDriver(HelixManager manager) {
     }
     @Deprecated
     public TaskDriver(RealmAwareZkClient client, String clusterName) {
     }
     @Deprecated
     public TaskDriver(RealmAwareZkClient client, ZkBaseDataAccessor<ZNRecord> baseAccessor,
         String clusterName) {}
     @Deprecated
     public TaskDriver(HelixAdmin admin, HelixDataAccessor accessor, ConfigAccessor cfgAccessor,
         HelixPropertyStore<ZNRecord> propertyStore, String clusterName) {
       this(admin, accessor, propertyStore, clusterName);
     }
   ```
   You see that every constructor of TaskDriver takes in a cluster name.
   
   Although TaskDriver does not support any ephemeral operations, by design it's meant to be a single cluster API.
   
   I think what you wanted to see was to pass in some FederatedZkClient into TaskDriver and use one TaskDriver for all clusters, but 1) that does not work, and 2) if we wanted to make that work, that's going to require a lot of code change in TaskDriver.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] dasahcc commented on a change in pull request #908: Make Helix REST realm-aware

Posted by GitBox <gi...@apache.org>.
dasahcc commented on a change in pull request #908: Make Helix REST realm-aware
URL: https://github.com/apache/helix/pull/908#discussion_r397492348
 
 

 ##########
 File path: helix-rest/src/main/java/org/apache/helix/rest/metadatastore/accessor/ZkRoutingDataReader.java
 ##########
 @@ -138,40 +122,51 @@ public synchronized void handleChildChange(String s, List<String> list) {
 
   @Override
   public synchronized void handleStateChanged(Watcher.Event.KeeperState state) {
-    if (_zkClient.isClosed()) {
+    if (_zkClient == null || _zkClient.isClosed()) {
       return;
     }
     _routingDataListener.refreshRoutingData(_namespace);
   }
 
   @Override
   public synchronized void handleNewSession(String sessionId) {
-    if (_zkClient.isClosed()) {
+    if (_zkClient == null || _zkClient.isClosed()) {
       return;
     }
     _routingDataListener.refreshRoutingData(_namespace);
   }
 
   @Override
   public synchronized void handleSessionEstablishmentError(Throwable error) {
-    if (_zkClient.isClosed()) {
+    if (_zkClient == null || _zkClient.isClosed()) {
       return;
     }
     _routingDataListener.refreshRoutingData(_namespace);
   }
 
+  /**
+   * Subscribes to the routing data paths using the provided ZkClient.
+   * @param zkClient
+   * @param childListener
+   * @param dataListener
+   */
+  public static void subscribeRoutingDataChanges(RealmAwareZkClient zkClient,
 
 Review comment:
   Would you like to make it as public util function? If yes, then move it to some util class, otherwise, I think we have internal member of _zkClient, does not need to make the zkclient is passing in.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] narendly commented on issue #908: Make Helix REST realm-aware

Posted by GitBox <gi...@apache.org>.
narendly commented on issue #908: Make Helix REST realm-aware
URL: https://github.com/apache/helix/pull/908#issuecomment-604235186
 
 
   Helix REST needs to start using a realm-aware ZkClient on multi-zk mode. Also it needs to become a listener on routing data because we don't want to restart the HelixRestServer every time we update the routing data.
   
   Changelist:
   
   Make ServerContext listen on routing data paths if run on multi-zk mode
   Make HelixRestServer use RealmAwareZkClient (FederatedZkClient) on multi-zk mode

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] dasahcc commented on a change in pull request #908: Make Helix REST realm-aware

Posted by GitBox <gi...@apache.org>.
dasahcc commented on a change in pull request #908: Make Helix REST realm-aware
URL: https://github.com/apache/helix/pull/908#discussion_r397495180
 
 

 ##########
 File path: helix-rest/src/main/java/org/apache/helix/rest/server/ServerContext.java
 ##########
 @@ -20,42 +20,67 @@
  * under the License.
  */
 
-import java.util.HashMap;
+import java.io.IOException;
+import java.util.List;
 import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.helix.ConfigAccessor;
 import org.apache.helix.HelixAdmin;
 import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.HelixException;
 import org.apache.helix.InstanceType;
-import org.apache.helix.rest.metadatastore.ZkMetadataStoreDirectory;
-import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.SystemPropertyKeys;
 import org.apache.helix.manager.zk.ZKHelixAdmin;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
-import org.apache.helix.manager.zk.ZNRecordSerializer;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.zookeeper.impl.client.ZkClient;
-import org.apache.helix.zookeeper.api.client.HelixZkClient;
-import org.apache.helix.zookeeper.impl.factory.SharedZkClientFactory;
+import org.apache.helix.msdcommon.exception.InvalidRoutingDataException;
+import org.apache.helix.rest.metadatastore.ZkMetadataStoreDirectory;
+import org.apache.helix.rest.metadatastore.accessor.ZkRoutingDataReader;
 import org.apache.helix.task.TaskDriver;
 import org.apache.helix.tools.ClusterSetup;
+import org.apache.helix.zookeeper.api.client.HelixZkClient;
+import org.apache.helix.zookeeper.api.client.RealmAwareZkClient;
+import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+import org.apache.helix.zookeeper.impl.client.FederatedZkClient;
+import org.apache.helix.zookeeper.impl.client.ZkClient;
+import org.apache.helix.zookeeper.impl.factory.DedicatedZkClientFactory;
+import org.apache.helix.zookeeper.impl.factory.SharedZkClientFactory;
+import org.apache.helix.zookeeper.util.HttpRoutingDataReader;
+import org.apache.helix.zookeeper.zkclient.IZkChildListener;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.helix.zookeeper.zkclient.IZkStateListener;
 import org.apache.helix.zookeeper.zkclient.exception.ZkMarshallingError;
 import org.apache.helix.zookeeper.zkclient.serialize.ZkSerializer;
+import org.apache.zookeeper.Watcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
-public class ServerContext {
+public class ServerContext implements IZkDataListener, IZkChildListener, IZkStateListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ServerContext.class);
+
   private final String _zkAddr;
-  private HelixZkClient _zkClient;
-  private ZKHelixAdmin _zkHelixAdmin;
-  private ClusterSetup _clusterSetup;
-  private ConfigAccessor _configAccessor;
+  private volatile RealmAwareZkClient _zkClient;
+
+  private volatile ZKHelixAdmin _zkHelixAdmin;
+  private volatile ClusterSetup _clusterSetup;
+  private volatile ConfigAccessor _configAccessor;
   // A lazily-initialized base data accessor that reads/writes byte array to ZK
   // TODO: Only read (deserialize) is supported at this time. This baseDataAccessor should support write (serialize) as needs arise
   private volatile ZkBaseDataAccessor<byte[]> _byteArrayZkBaseDataAccessor;
   // 1 Cluster name will correspond to 1 helix data accessor
   private final Map<String, HelixDataAccessor> _helixDataAccessorPool;
   // 1 Cluster name will correspond to 1 task driver
   private final Map<String, TaskDriver> _taskDriverPool;
 
 Review comment:
   Why we need a pool? Since you already make the task driver could accept the RealmAzreZkClient, we can pass in the FederatedZkClient as input, right? And TaskDriver does not have any session aware operations. So we can use FederatedZkClient for multiple cluster accessing purpose.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] narendly commented on a change in pull request #908: Make Helix REST realm-aware

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #908: Make Helix REST realm-aware
URL: https://github.com/apache/helix/pull/908#discussion_r397584201
 
 

 ##########
 File path: helix-rest/src/main/java/org/apache/helix/rest/server/ServerContext.java
 ##########
 @@ -20,42 +20,67 @@
  * under the License.
  */
 
-import java.util.HashMap;
+import java.io.IOException;
+import java.util.List;
 import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.helix.ConfigAccessor;
 import org.apache.helix.HelixAdmin;
 import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.HelixException;
 import org.apache.helix.InstanceType;
-import org.apache.helix.rest.metadatastore.ZkMetadataStoreDirectory;
-import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.SystemPropertyKeys;
 import org.apache.helix.manager.zk.ZKHelixAdmin;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
-import org.apache.helix.manager.zk.ZNRecordSerializer;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.zookeeper.impl.client.ZkClient;
-import org.apache.helix.zookeeper.api.client.HelixZkClient;
-import org.apache.helix.zookeeper.impl.factory.SharedZkClientFactory;
+import org.apache.helix.msdcommon.exception.InvalidRoutingDataException;
+import org.apache.helix.rest.metadatastore.ZkMetadataStoreDirectory;
+import org.apache.helix.rest.metadatastore.accessor.ZkRoutingDataReader;
 import org.apache.helix.task.TaskDriver;
 import org.apache.helix.tools.ClusterSetup;
+import org.apache.helix.zookeeper.api.client.HelixZkClient;
+import org.apache.helix.zookeeper.api.client.RealmAwareZkClient;
+import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+import org.apache.helix.zookeeper.impl.client.FederatedZkClient;
+import org.apache.helix.zookeeper.impl.client.ZkClient;
+import org.apache.helix.zookeeper.impl.factory.DedicatedZkClientFactory;
+import org.apache.helix.zookeeper.impl.factory.SharedZkClientFactory;
+import org.apache.helix.zookeeper.util.HttpRoutingDataReader;
+import org.apache.helix.zookeeper.zkclient.IZkChildListener;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.helix.zookeeper.zkclient.IZkStateListener;
 import org.apache.helix.zookeeper.zkclient.exception.ZkMarshallingError;
 import org.apache.helix.zookeeper.zkclient.serialize.ZkSerializer;
+import org.apache.zookeeper.Watcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
-public class ServerContext {
+public class ServerContext implements IZkDataListener, IZkChildListener, IZkStateListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ServerContext.class);
+
   private final String _zkAddr;
-  private HelixZkClient _zkClient;
-  private ZKHelixAdmin _zkHelixAdmin;
-  private ClusterSetup _clusterSetup;
-  private ConfigAccessor _configAccessor;
+  private volatile RealmAwareZkClient _zkClient;
+
+  private volatile ZKHelixAdmin _zkHelixAdmin;
+  private volatile ClusterSetup _clusterSetup;
+  private volatile ConfigAccessor _configAccessor;
   // A lazily-initialized base data accessor that reads/writes byte array to ZK
   // TODO: Only read (deserialize) is supported at this time. This baseDataAccessor should support write (serialize) as needs arise
   private volatile ZkBaseDataAccessor<byte[]> _byteArrayZkBaseDataAccessor;
   // 1 Cluster name will correspond to 1 helix data accessor
   private final Map<String, HelixDataAccessor> _helixDataAccessorPool;
   // 1 Cluster name will correspond to 1 task driver
   private final Map<String, TaskDriver> _taskDriverPool;
 
 Review comment:
   If multiZk system config is not set and MSDS routing data is not available (which is the case for a single-zk realm user), you cannot create a FederatedZkClient. Does that make sense? So we need to preserve the behavior if the user is not using the multi-zk feature.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] narendly commented on a change in pull request #908: Make Helix REST realm-aware

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #908: Make Helix REST realm-aware
URL: https://github.com/apache/helix/pull/908#discussion_r397524486
 
 

 ##########
 File path: helix-rest/src/main/java/org/apache/helix/rest/server/ServerContext.java
 ##########
 @@ -20,42 +20,67 @@
  * under the License.
  */
 
-import java.util.HashMap;
+import java.io.IOException;
+import java.util.List;
 import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.helix.ConfigAccessor;
 import org.apache.helix.HelixAdmin;
 import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.HelixException;
 import org.apache.helix.InstanceType;
-import org.apache.helix.rest.metadatastore.ZkMetadataStoreDirectory;
-import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.SystemPropertyKeys;
 import org.apache.helix.manager.zk.ZKHelixAdmin;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
-import org.apache.helix.manager.zk.ZNRecordSerializer;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.zookeeper.impl.client.ZkClient;
-import org.apache.helix.zookeeper.api.client.HelixZkClient;
-import org.apache.helix.zookeeper.impl.factory.SharedZkClientFactory;
+import org.apache.helix.msdcommon.exception.InvalidRoutingDataException;
+import org.apache.helix.rest.metadatastore.ZkMetadataStoreDirectory;
+import org.apache.helix.rest.metadatastore.accessor.ZkRoutingDataReader;
 import org.apache.helix.task.TaskDriver;
 import org.apache.helix.tools.ClusterSetup;
+import org.apache.helix.zookeeper.api.client.HelixZkClient;
+import org.apache.helix.zookeeper.api.client.RealmAwareZkClient;
+import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+import org.apache.helix.zookeeper.impl.client.FederatedZkClient;
+import org.apache.helix.zookeeper.impl.client.ZkClient;
+import org.apache.helix.zookeeper.impl.factory.DedicatedZkClientFactory;
+import org.apache.helix.zookeeper.impl.factory.SharedZkClientFactory;
+import org.apache.helix.zookeeper.util.HttpRoutingDataReader;
+import org.apache.helix.zookeeper.zkclient.IZkChildListener;
+import org.apache.helix.zookeeper.zkclient.IZkDataListener;
+import org.apache.helix.zookeeper.zkclient.IZkStateListener;
 import org.apache.helix.zookeeper.zkclient.exception.ZkMarshallingError;
 import org.apache.helix.zookeeper.zkclient.serialize.ZkSerializer;
+import org.apache.zookeeper.Watcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
-public class ServerContext {
+public class ServerContext implements IZkDataListener, IZkChildListener, IZkStateListener {
+  private static final Logger LOG = LoggerFactory.getLogger(ServerContext.class);
+
   private final String _zkAddr;
-  private HelixZkClient _zkClient;
-  private ZKHelixAdmin _zkHelixAdmin;
-  private ClusterSetup _clusterSetup;
-  private ConfigAccessor _configAccessor;
+  private volatile RealmAwareZkClient _zkClient;
+
+  private volatile ZKHelixAdmin _zkHelixAdmin;
+  private volatile ClusterSetup _clusterSetup;
+  private volatile ConfigAccessor _configAccessor;
   // A lazily-initialized base data accessor that reads/writes byte array to ZK
   // TODO: Only read (deserialize) is supported at this time. This baseDataAccessor should support write (serialize) as needs arise
   private volatile ZkBaseDataAccessor<byte[]> _byteArrayZkBaseDataAccessor;
   // 1 Cluster name will correspond to 1 helix data accessor
   private final Map<String, HelixDataAccessor> _helixDataAccessorPool;
   // 1 Cluster name will correspond to 1 task driver
   private final Map<String, TaskDriver> _taskDriverPool;
 
 Review comment:
   Your line of thinking is correct, but if we made that change here, Helix REST will no longer be backward-compatible. We leave it here so that for users that use Helix REST in a single ZK mode will be able to use it the way they were able to use before (1 TaskDriver per cluster)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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