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/07/30 07:27:16 UTC

[GitHub] [helix] jiajunwang opened a new pull request #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

jiajunwang opened a new pull request #1190:
URL: https://github.com/apache/helix/pull/1190


   ### Issues
   
   - [X] My PR addresses the following Helix issues and references them in the PR description:
   
   #1189
   
   ### Description
   
   - [X] Here are some details about my PR, including screenshots of any UI changes:
   
   Add a new Helix rest API in the ZookeeperAccessor for deleting an ephemeral ZNode.
   
   Note that before we have ACL/audit support in the Helix rest, allowing raw ZK write operation is dangerous.
   This API is introduced prematurely for resolving the issue of "zombie" participant (the instance has an active zk connection, but refuse to do any work). Currently, the existence of such a node may block the normal state transitions and then impact the cluster's availability. This PR restricts that only an ephemeral node can be deleted to minimize the risk.
   
   ### Tests
   
   - [X] The following tests are written for this issue:
   
   TestZooKeeperAccessor.testDelete()
   
   - [ ] The following is the result of the "mvn test" command on the appropriate module:
   
   helix-rest
   
   [INFO] Tests run: 164, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 43.852 s - in TestSuite
   [INFO] 
   [INFO] Results:
   [INFO] 
   [INFO] Tests run: 164, Failures: 0, Errors: 0, Skipped: 0
   [INFO] 
   [INFO] ------------------------------------------------------------------------
   [INFO] BUILD SUCCESS
   [INFO] ------------------------------------------------------------------------
   [INFO] Total time: 52.162 s
   [INFO] Finished at: 2020-07-29T18:06:03-07:00
   [INFO] ------------------------------------------------------------------------
   
   ### 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"
   
   ### Documentation (Optional)
   
   - [ ] In case of new functionality, my PR adds documentation in the following wiki page:
   
   (Link the GitHub wiki you added)
   
   ### 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



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


[GitHub] [helix] pkuwm commented on pull request #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
pkuwm commented on pull request #1190:
URL: https://github.com/apache/helix/pull/1190#issuecomment-666811379


   @jiajunwang By "zombie" participant, you meant the ephemeral node doesn't have any active zk connection/session, but it is not deleted by 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



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


[GitHub] [helix] jiajunwang commented on a change in pull request #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1190:
URL: https://github.com/apache/helix/pull/1190#discussion_r463750665



##########
File path: helix-rest/src/main/java/org/apache/helix/rest/server/resources/zookeeper/ZooKeeperAccessor.java
##########
@@ -52,12 +51,15 @@
   private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperAccessor.class.getName());
   private BaseDataAccessor<byte[]> _zkBaseDataAccessor;
 
+  private static final String PATH_STR = "path";
+
   public enum ZooKeeperCommand {
     exists,
     getBinaryData,
     getStringData,
     getChildren,
-    getStat
+    getStat,
+    delete

Review comment:
       This information is carried in the response entity as a string for now. I don't think we need to make it too structural (complicated) given it is a temporary restriction. And eventually, we do not have a clear standard for the response format now. So I would prefer holding on any more complex idea.




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



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


[GitHub] [helix] jiajunwang commented on a change in pull request #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1190:
URL: https://github.com/apache/helix/pull/1190#discussion_r463339030



##########
File path: helix-rest/src/main/java/org/apache/helix/rest/server/resources/zookeeper/ZooKeeperAccessor.java
##########
@@ -192,7 +196,29 @@ private Response getStat(BaseDataAccessor<byte[]> zkBaseDataAccessor, String pat
           .entity(String.format("The ZNode at path %s does not exist!", path)).build());
     }
     Map<String, String> result = ZKUtil.fromStatToMap(stat);
-    result.put("path", path);
+    result.put(PATH_STR, path);
+    return JSONRepresentation(result);
+  }
+
+  /**
+   * Delete the ZNode at the given path if exists.
+   * @param zkBaseDataAccessor
+   * @param path
+   * @return The delete result and the operated path.
+   */
+  private Response delete(BaseDataAccessor zkBaseDataAccessor, String path) {
+    // TODO: Remove this restriction once we have audit and ACL for the API calls.
+    // TODO: This method is added pre-maturely to support removing the live instance of a zombie
+    // TODO: instance. It is risky to allow all deleting requests before audit and ACL are done.
+    Stat stat = zkBaseDataAccessor.getStat(path, AccessOption.PERSISTENT);
+    if (stat != null && stat.getEphemeralOwner() <= 0) {
+      throw new WebApplicationException(Response.status(Response.Status.FORBIDDEN)
+          .entity(String.format("Deleting a non ephemeral node is not allowed", path)).build());
+    }
+
+    Boolean ret = zkBaseDataAccessor.remove(path, AccessOption.PERSISTENT);

Review comment:
       Let's don't touch the option for this PR. Let me create an issue for the unnecessary AccessOption.




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



---------------------------------------------------------------------
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 #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #1190:
URL: https://github.com/apache/helix/pull/1190#discussion_r463922492



##########
File path: helix-rest/src/main/java/org/apache/helix/rest/server/resources/zookeeper/ZooKeeperAccessor.java
##########
@@ -196,6 +209,32 @@ private Response getStat(BaseDataAccessor<byte[]> zkBaseDataAccessor, String pat
     return JSONRepresentation(result);
   }
 
+  /**
+   * Delete the ZNode at the given path if exists.
+   * @param zkBaseDataAccessor
+   * @param path
+   * @return The delete result and the operated path.
+   */
+  private Response delete(BaseDataAccessor zkBaseDataAccessor, String path) {

Review comment:
       I'm not seeing how this is any different from using delete. This is no better than using `delete` for two different types of delete's - delete and deleteEphemeral.
   
   Perhaps you could add a commandStr here to differentiate two different types of `delete`s, and when you want to add an endpoint for regular delete backed by ACL checks, then just implement that _if_ that becomes necessary? I don't think this adds any more work/difficulty for the purposes of this PR? (If any, it saves you the work of adding a TODO)
   
   My point was not about what kind of REST verb we should use - it's pretty clear we should use DELETE in this case. But it's more about following a good API design which, again, is something that is hard to misuse by not embedding hidden assumptions or TODOs that may cause a behavior change down the road. Also, seen from another angle, supporting it as `deleteEphemeral` gives the user a clear meaning to the command string as opposed to just calling it a HTTP verb DELETE, which might leave the user confused and question the meaning of the API when it fails to delete regular ZNodes. 
   
   You could add two commands, delete and deleteEphemeral, and make the default commandStr delete, and throw a not authorized or not supported, and only let deleteEphemeral go through. This way, there's no confusion when we do decide to support delete operation with ACL, there's no confusion or change in behavior.




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



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


[GitHub] [helix] jiajunwang commented on a change in pull request #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1190:
URL: https://github.com/apache/helix/pull/1190#discussion_r464725607



##########
File path: helix-rest/src/main/java/org/apache/helix/rest/server/resources/zookeeper/ZooKeeperAccessor.java
##########
@@ -196,6 +209,32 @@ private Response getStat(BaseDataAccessor<byte[]> zkBaseDataAccessor, String pat
     return JSONRepresentation(result);
   }
 
+  /**
+   * Delete the ZNode at the given path if exists.
+   * @param zkBaseDataAccessor
+   * @param path
+   * @return The delete result and the operated path.
+   */
+  private Response delete(BaseDataAccessor zkBaseDataAccessor, String path) {

Review comment:
       Discussed with Junkai in slack, his point is that we don't need the additional cmd layer for now.
   




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



---------------------------------------------------------------------
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 #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #1190:
URL: https://github.com/apache/helix/pull/1190#discussion_r463131093



##########
File path: helix-rest/src/main/java/org/apache/helix/rest/server/resources/zookeeper/ZooKeeperAccessor.java
##########
@@ -192,7 +196,29 @@ private Response getStat(BaseDataAccessor<byte[]> zkBaseDataAccessor, String pat
           .entity(String.format("The ZNode at path %s does not exist!", path)).build());
     }
     Map<String, String> result = ZKUtil.fromStatToMap(stat);
-    result.put("path", path);
+    result.put(PATH_STR, path);
+    return JSONRepresentation(result);
+  }
+
+  /**
+   * Delete the ZNode at the given path if exists.
+   * @param zkBaseDataAccessor
+   * @param path
+   * @return The delete result and the operated path.
+   */
+  private Response delete(BaseDataAccessor zkBaseDataAccessor, String path) {
+    // TODO: Remove this restriction once we have audit and ACL for the API calls.
+    // TODO: This method is added pre-maturely to support removing the live instance of a zombie
+    // TODO: instance. It is risky to allow all deleting requests before audit and ACL are done.
+    Stat stat = zkBaseDataAccessor.getStat(path, AccessOption.PERSISTENT);
+    if (stat != null && stat.getEphemeralOwner() <= 0) {
+      throw new WebApplicationException(Response.status(Response.Status.FORBIDDEN)
+          .entity(String.format("Deleting a non ephemeral node is not allowed", path)).build());
+    }
+
+    Boolean ret = zkBaseDataAccessor.remove(path, AccessOption.PERSISTENT);

Review comment:
       Would `zkBaseDataAccessor.remove(path, AccessOption.EPHEMERAL)` be an option?




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



---------------------------------------------------------------------
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 #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #1190:
URL: https://github.com/apache/helix/pull/1190#discussion_r463922492



##########
File path: helix-rest/src/main/java/org/apache/helix/rest/server/resources/zookeeper/ZooKeeperAccessor.java
##########
@@ -196,6 +209,32 @@ private Response getStat(BaseDataAccessor<byte[]> zkBaseDataAccessor, String pat
     return JSONRepresentation(result);
   }
 
+  /**
+   * Delete the ZNode at the given path if exists.
+   * @param zkBaseDataAccessor
+   * @param path
+   * @return The delete result and the operated path.
+   */
+  private Response delete(BaseDataAccessor zkBaseDataAccessor, String path) {

Review comment:
       I'm not seeing how this is any different from using delete. This is no better than using `delete` for two different types of delete's - delete and deleteEphemeral.
   
   Perhaps you could add a commandStr here to differentiate two different types of `delete`s, and when you want to add an endpoint for regular delete backed by ACL checks, then just implement that _if_ that becomes necessary? I don't think this adds any more work/difficulty for the purposes of this PR? (If any, it saves you the work of adding a TODO)
   
   My point was not about what kind of REST verb we should use - it's pretty clear we should use DELETE in this case. But it's more about following a good API design which, again, is something that is hard to misuse by not embedding hidden assumptions or TODOs that may cause a behavior change down the road.




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



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


[GitHub] [helix] jiajunwang commented on a change in pull request #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1190:
URL: https://github.com/apache/helix/pull/1190#discussion_r463228709



##########
File path: helix-rest/src/main/java/org/apache/helix/rest/server/resources/zookeeper/ZooKeeperAccessor.java
##########
@@ -192,7 +196,29 @@ private Response getStat(BaseDataAccessor<byte[]> zkBaseDataAccessor, String pat
           .entity(String.format("The ZNode at path %s does not exist!", path)).build());
     }
     Map<String, String> result = ZKUtil.fromStatToMap(stat);
-    result.put("path", path);
+    result.put(PATH_STR, path);
+    return JSONRepresentation(result);
+  }
+
+  /**
+   * Delete the ZNode at the given path if exists.
+   * @param zkBaseDataAccessor
+   * @param path
+   * @return The delete result and the operated path.
+   */
+  private Response delete(BaseDataAccessor zkBaseDataAccessor, String path) {
+    // TODO: Remove this restriction once we have audit and ACL for the API calls.
+    // TODO: This method is added pre-maturely to support removing the live instance of a zombie
+    // TODO: instance. It is risky to allow all deleting requests before audit and ACL are done.
+    Stat stat = zkBaseDataAccessor.getStat(path, AccessOption.PERSISTENT);
+    if (stat != null && stat.getEphemeralOwner() <= 0) {
+      throw new WebApplicationException(Response.status(Response.Status.FORBIDDEN)
+          .entity(String.format("Deleting a non ephemeral node is not allowed", path)).build());
+    }
+
+    Boolean ret = zkBaseDataAccessor.remove(path, AccessOption.PERSISTENT);

Review comment:
       The latter parameter is not used. Put EPHEMERAL or PERSISTENT is equally confusing. So let me just put the first enum item for now. I actually intend to avoid using EPHEMERAL since it indicates this parameter has some usage inside (but it is not true).
   
   I think we need to discard this interesting API for good.




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



---------------------------------------------------------------------
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 #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #1190:
URL: https://github.com/apache/helix/pull/1190#discussion_r463130598



##########
File path: helix-rest/src/main/java/org/apache/helix/rest/server/resources/zookeeper/ZooKeeperAccessor.java
##########
@@ -52,12 +51,15 @@
   private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperAccessor.class.getName());
   private BaseDataAccessor<byte[]> _zkBaseDataAccessor;
 
+  private static final String PATH_STR = "path";
+
   public enum ZooKeeperCommand {
     exists,
     getBinaryData,
     getStringData,
     getChildren,
-    getStat
+    getStat,
+    delete

Review comment:
       I think it would be smarter to use deleteEphemeral and rename your methods accordingly because it seems that it's  not the general delete you're trying to support.




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



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


[GitHub] [helix] pkuwm commented on pull request #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
pkuwm commented on pull request #1190:
URL: https://github.com/apache/helix/pull/1190#issuecomment-666862179


   > > @jiajunwang By "zombie" participant, you meant the ephemeral node doesn't have any active zk connection/session, but it is not deleted by ZK?
   > 
   > Please read the description of PR, "the instance has an active zk connection, but refuse to do any work"
   
   Oh I read "**an active** zkconnection" as **inactive** zkconnection...


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



---------------------------------------------------------------------
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 change in pull request #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
xyuanlu commented on a change in pull request #1190:
URL: https://github.com/apache/helix/pull/1190#discussion_r463236804



##########
File path: helix-rest/src/test/java/org/apache/helix/rest/server/TestZooKeeperAccessor.java
##########
@@ -192,4 +191,33 @@ public void testGetStat() throws IOException {
     // Clean up
     _testBaseDataAccessor.remove(path, AccessOption.PERSISTENT);
   }
+
+  @Test
+  public void testDelete() throws IOException {
+    String path = "/path";
+    String deletePath = path + "/delete";
+
+    try {
+      // 1. Create a persistent node. Delete shall fail.
+      _testBaseDataAccessor.create(deletePath, null, AccessOption.PERSISTENT);
+      // Verify with the REST endpoint
+      new JerseyUriRequestBuilder("zookeeper{}?command=delete").format(deletePath)
+          .expectedReturnStatusCode(Response.Status.FORBIDDEN.getStatusCode());
+      Assert.assertTrue(_testBaseDataAccessor.exists(deletePath, AccessOption.PERSISTENT));
+
+      // 2. Create a ephemeral node. Delete shall be done successfully.
+      _testBaseDataAccessor.remove(deletePath, AccessOption.PERSISTENT);
+      _testBaseDataAccessor.create(deletePath, null, AccessOption.EPHEMERAL);
+      // Verify with the REST endpoint
+      String data = new JerseyUriRequestBuilder("zookeeper{}?command=delete").format(deletePath)
+          .isBodyReturnExpected(true).get(this);
+      Map<String, String> result = OBJECT_MAPPER.readValue(data, HashMap.class);
+      Assert.assertEquals(result.get("path"), deletePath);
+      Assert.assertEquals(result.get("delete"), new Boolean(true).toString());
+      Assert.assertFalse(_testBaseDataAccessor.exists(deletePath, AccessOption.PERSISTENT));

Review comment:
       Yea. That confuses me as well. It seems many functions has a not-used 'int option' as input. May I suggest using a value consist with the node type as input? For example, we want to check if the newly created EPHEMERAL node is deleted, let use EPHEMERAL here to avoid further confusion. 




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



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


[GitHub] [helix] jiajunwang commented on a change in pull request #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1190:
URL: https://github.com/apache/helix/pull/1190#discussion_r463751027



##########
File path: helix-rest/src/main/java/org/apache/helix/rest/server/resources/zookeeper/ZooKeeperAccessor.java
##########
@@ -101,6 +100,26 @@ public Response get(@PathParam("path") String path, @QueryParam("command") Strin
     }
   }
 
+  @DELETE
+  @Path("{path: .+}")
+  public Response delete(@PathParam("path") String path) {
+    // Lazily initialize ZkBaseDataAccessor
+    ServerContext _serverContext =
+        (ServerContext) _application.getProperties().get(ContextPropertyKeys.SERVER_CONTEXT.name());
+    _zkBaseDataAccessor = _serverContext.getByteArrayZkBaseDataAccessor();
+
+    path = prependPath(path);

Review comment:
       Neat, changed the code accordingly. There is more code like that. But I would not change them in this PR.




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



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


[GitHub] [helix] jiajunwang commented on a change in pull request #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1190:
URL: https://github.com/apache/helix/pull/1190#discussion_r463225814



##########
File path: helix-rest/src/main/java/org/apache/helix/rest/server/resources/zookeeper/ZooKeeperAccessor.java
##########
@@ -94,6 +96,8 @@ public Response get(@PathParam("path") String path, @QueryParam("command") Strin
         return getChildren(_zkBaseDataAccessor, path);
       case getStat:
         return getStat(_zkBaseDataAccessor, path);
+      case delete:

Review comment:
       That makes more sense, let me have a try.




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



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


[GitHub] [helix] jiajunwang commented on pull request #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on pull request #1190:
URL: https://github.com/apache/helix/pull/1190#issuecomment-666838643


   > @jiajunwang By "zombie" participant, you meant the ephemeral node doesn't have any active zk connection/session, but it is not deleted by ZK?
   
   Please read the description of PR, "the instance has an active zk connection, but refuse to do any work"


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



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


[GitHub] [helix] pkuwm commented on a change in pull request #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #1190:
URL: https://github.com/apache/helix/pull/1190#discussion_r463477490



##########
File path: helix-rest/src/main/java/org/apache/helix/rest/server/resources/zookeeper/ZooKeeperAccessor.java
##########
@@ -52,12 +51,15 @@
   private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperAccessor.class.getName());
   private BaseDataAccessor<byte[]> _zkBaseDataAccessor;
 
+  private static final String PATH_STR = "path";
+
   public enum ZooKeeperCommand {
     exists,
     getBinaryData,
     getStringData,
     getChildren,
-    getStat
+    getStat,
+    delete

Review comment:
       My opinion is, 
   1. use `DELETE` verb that is designed for the REST delete operation.
   2. if we only want to support deleting ephemeral, document it well and return a clear response like:
   ```
       HTTP/1.1  404
       Content-Type: application/json
    
       {
         "message": "Deleting a non-ephemeral node is not supported/allowed",
         "path": "/a/b/c"
       }
   ```
   And it is extensible if we want to support deleting persistent node in the future.




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



---------------------------------------------------------------------
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 #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #1190:
URL: https://github.com/apache/helix/pull/1190#discussion_r463230652



##########
File path: helix-rest/src/main/java/org/apache/helix/rest/server/resources/zookeeper/ZooKeeperAccessor.java
##########
@@ -52,12 +51,15 @@
   private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperAccessor.class.getName());
   private BaseDataAccessor<byte[]> _zkBaseDataAccessor;
 
+  private static final String PATH_STR = "path";
+
   public enum ZooKeeperCommand {
     exists,
     getBinaryData,
     getStringData,
     getChildren,
-    getStat
+    getStat,
+    delete

Review comment:
       I still don't agree - it seems that you are suggesting using `delete`, expecting its behavior to change in the future. That will bring about backward-compatibility issues and make the meaning of "delete" murky. 
   
   It's fine if you need to provide another endpoint/API, but a good API design involves something that is 1) easy to do and 2) doing exactly what it's advertising to do. Do you see why it might be less desirable to add hidden assumptions to `delete`?




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



---------------------------------------------------------------------
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 #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #1190:
URL: https://github.com/apache/helix/pull/1190#discussion_r463922492



##########
File path: helix-rest/src/main/java/org/apache/helix/rest/server/resources/zookeeper/ZooKeeperAccessor.java
##########
@@ -196,6 +209,32 @@ private Response getStat(BaseDataAccessor<byte[]> zkBaseDataAccessor, String pat
     return JSONRepresentation(result);
   }
 
+  /**
+   * Delete the ZNode at the given path if exists.
+   * @param zkBaseDataAccessor
+   * @param path
+   * @return The delete result and the operated path.
+   */
+  private Response delete(BaseDataAccessor zkBaseDataAccessor, String path) {

Review comment:
       I'm not seeing how this is any different from using delete. This is no better than using `delete` for two different types of delete's - delete and deleteEphemeral.
   
   Perhaps you could add a commandStr here to differentiate two different types of `delete`s, and when you want to add an endpoint for regular delete backed by ACL checks, then just implement that _if_ that becomes necessary? I don't think this adds any more work/difficulty for the purposes of this PR? (If any, it saves you the work of adding a TODO)
   
   My point was not about what kind of REST verb we should use - it's pretty clear we should use DELETE in this case. But it's more about following a good API design which, again, is something that is hard to misuse by not embedding hidden assumptions or TODOs that may cause a behavior change down the road. Also, seen from another angle, supporting it as `deleteEphemeral` gives the user a clear meaning to the command string as opposed to just calling it a HTTP verb DELETE, which might leave the user confused and question the meaning of the API when it fails to delete regular ZNodes. 




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



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


[GitHub] [helix] jiajunwang commented on a change in pull request #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1190:
URL: https://github.com/apache/helix/pull/1190#discussion_r463229646



##########
File path: helix-rest/src/test/java/org/apache/helix/rest/server/TestZooKeeperAccessor.java
##########
@@ -192,4 +191,33 @@ public void testGetStat() throws IOException {
     // Clean up
     _testBaseDataAccessor.remove(path, AccessOption.PERSISTENT);
   }
+
+  @Test
+  public void testDelete() throws IOException {
+    String path = "/path";
+    String deletePath = path + "/delete";
+
+    try {
+      // 1. Create a persistent node. Delete shall fail.
+      _testBaseDataAccessor.create(deletePath, null, AccessOption.PERSISTENT);
+      // Verify with the REST endpoint
+      new JerseyUriRequestBuilder("zookeeper{}?command=delete").format(deletePath)
+          .expectedReturnStatusCode(Response.Status.FORBIDDEN.getStatusCode());
+      Assert.assertTrue(_testBaseDataAccessor.exists(deletePath, AccessOption.PERSISTENT));
+
+      // 2. Create a ephemeral node. Delete shall be done successfully.
+      _testBaseDataAccessor.remove(deletePath, AccessOption.PERSISTENT);
+      _testBaseDataAccessor.create(deletePath, null, AccessOption.EPHEMERAL);
+      // Verify with the REST endpoint
+      String data = new JerseyUriRequestBuilder("zookeeper{}?command=delete").format(deletePath)
+          .isBodyReturnExpected(true).get(this);
+      Map<String, String> result = OBJECT_MAPPER.readValue(data, HashMap.class);
+      Assert.assertEquals(result.get("path"), deletePath);
+      Assert.assertEquals(result.get("delete"), new Boolean(true).toString());
+      Assert.assertFalse(_testBaseDataAccessor.exists(deletePath, AccessOption.PERSISTENT));

Review comment:
       No use at all. There seems to be no such an exist or getStat method which takes the option in ZK lib, either. I think we shall clean them up in a separate PR.




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



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


[GitHub] [helix] jiajunwang commented on a change in pull request #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1190:
URL: https://github.com/apache/helix/pull/1190#discussion_r463206540



##########
File path: helix-rest/src/main/java/org/apache/helix/rest/server/resources/zookeeper/ZooKeeperAccessor.java
##########
@@ -192,7 +196,29 @@ private Response getStat(BaseDataAccessor<byte[]> zkBaseDataAccessor, String pat
           .entity(String.format("The ZNode at path %s does not exist!", path)).build());
     }
     Map<String, String> result = ZKUtil.fromStatToMap(stat);
-    result.put("path", path);
+    result.put(PATH_STR, path);
+    return JSONRepresentation(result);
+  }
+
+  /**
+   * Delete the ZNode at the given path if exists.
+   * @param zkBaseDataAccessor
+   * @param path
+   * @return The delete result and the operated path.
+   */
+  private Response delete(BaseDataAccessor zkBaseDataAccessor, String path) {
+    // TODO: Remove this restriction once we have audit and ACL for the API calls.
+    // TODO: This method is added pre-maturely to support removing the live instance of a zombie
+    // TODO: instance. It is risky to allow all deleting requests before audit and ACL are done.
+    Stat stat = zkBaseDataAccessor.getStat(path, AccessOption.PERSISTENT);
+    if (stat != null && stat.getEphemeralOwner() <= 0) {
+      throw new WebApplicationException(Response.status(Response.Status.FORBIDDEN)
+          .entity(String.format("Deleting a non ephemeral node is not allowed", path)).build());

Review comment:
       good catch!




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



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


[GitHub] [helix] pkuwm commented on a change in pull request #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #1190:
URL: https://github.com/apache/helix/pull/1190#discussion_r462829238



##########
File path: helix-rest/src/main/java/org/apache/helix/rest/server/resources/zookeeper/ZooKeeperAccessor.java
##########
@@ -94,6 +96,8 @@ public Response get(@PathParam("path") String path, @QueryParam("command") Strin
         return getChildren(_zkBaseDataAccessor, path);
       case getStat:
         return getStat(_zkBaseDataAccessor, path);
+      case delete:

Review comment:
       Would this go to a "DELETE" HTTP verb?

##########
File path: helix-rest/src/main/java/org/apache/helix/rest/server/resources/zookeeper/ZooKeeperAccessor.java
##########
@@ -192,7 +196,29 @@ private Response getStat(BaseDataAccessor<byte[]> zkBaseDataAccessor, String pat
           .entity(String.format("The ZNode at path %s does not exist!", path)).build());
     }
     Map<String, String> result = ZKUtil.fromStatToMap(stat);
-    result.put("path", path);
+    result.put(PATH_STR, path);
+    return JSONRepresentation(result);
+  }
+
+  /**
+   * Delete the ZNode at the given path if exists.
+   * @param zkBaseDataAccessor
+   * @param path
+   * @return The delete result and the operated path.
+   */
+  private Response delete(BaseDataAccessor zkBaseDataAccessor, String path) {
+    // TODO: Remove this restriction once we have audit and ACL for the API calls.
+    // TODO: This method is added pre-maturely to support removing the live instance of a zombie
+    // TODO: instance. It is risky to allow all deleting requests before audit and ACL are done.
+    Stat stat = zkBaseDataAccessor.getStat(path, AccessOption.PERSISTENT);
+    if (stat != null && stat.getEphemeralOwner() <= 0) {
+      throw new WebApplicationException(Response.status(Response.Status.FORBIDDEN)
+          .entity(String.format("Deleting a non ephemeral node is not allowed", path)).build());

Review comment:
       Missing `%s`?




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



---------------------------------------------------------------------
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 #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #1190:
URL: https://github.com/apache/helix/pull/1190#discussion_r463230652



##########
File path: helix-rest/src/main/java/org/apache/helix/rest/server/resources/zookeeper/ZooKeeperAccessor.java
##########
@@ -52,12 +51,15 @@
   private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperAccessor.class.getName());
   private BaseDataAccessor<byte[]> _zkBaseDataAccessor;
 
+  private static final String PATH_STR = "path";
+
   public enum ZooKeeperCommand {
     exists,
     getBinaryData,
     getStringData,
     getChildren,
-    getStat
+    getStat,
+    delete

Review comment:
       I still don't agree - it seems that you are suggesting using `delete`, expecting its behavior to change in the future. That will bring about backward-compatibility issues and make the meaning of "delete" murky. 
   
   It's fine if you need to provide another endpoint/API, but a good API design involves something that is 1) easy to do and 2) doing exactly what it's advertising to do. Do you see why I think it might be less desirable to add hidden assumptions to `delete`?




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



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


[GitHub] [helix] jiajunwang merged pull request #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
jiajunwang merged pull request #1190:
URL: https://github.com/apache/helix/pull/1190


   


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



---------------------------------------------------------------------
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 #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #1190:
URL: https://github.com/apache/helix/pull/1190#discussion_r463230652



##########
File path: helix-rest/src/main/java/org/apache/helix/rest/server/resources/zookeeper/ZooKeeperAccessor.java
##########
@@ -52,12 +51,15 @@
   private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperAccessor.class.getName());
   private BaseDataAccessor<byte[]> _zkBaseDataAccessor;
 
+  private static final String PATH_STR = "path";
+
   public enum ZooKeeperCommand {
     exists,
     getBinaryData,
     getStringData,
     getChildren,
-    getStat
+    getStat,
+    delete

Review comment:
       It seems that you are suggesting adding and using `delete`, expecting its behavior to change in the future. Adding an endpoint and changing its behavior will bring about backward-compatibility issues and make the meaning of "delete" murky. Moreover, there's no harm in having `deleteEphemeral` - it does what it does, and if the user no longer wishes to use it, then there's no harm in having it.
   
   A good API design I believe is something that is 1) easy to use and 2) doing exactly what it's advertising to do. Do you see why I think it might be less desirable to add hidden assumptions to `delete`?




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



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


[GitHub] [helix] pkuwm commented on a change in pull request #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #1190:
URL: https://github.com/apache/helix/pull/1190#discussion_r463901648



##########
File path: helix-rest/src/main/java/org/apache/helix/rest/server/resources/zookeeper/ZooKeeperAccessor.java
##########
@@ -196,6 +209,32 @@ private Response getStat(BaseDataAccessor<byte[]> zkBaseDataAccessor, String pat
     return JSONRepresentation(result);
   }
 
+  /**
+   * Delete the ZNode at the given path if exists.
+   * @param zkBaseDataAccessor
+   * @param path
+   * @return The delete result and the operated path.
+   */
+  private Response delete(BaseDataAccessor zkBaseDataAccessor, String path) {
+    Stat stat = zkBaseDataAccessor.getStat(path, AccessOption.PERSISTENT);
+    if (stat == null) {
+      return notFound();

Review comment:
       Could we add a msg to this as well: ("Path %s does not exist", path)? I think it gives a user a better idea. Otherwise the msg returned is unfriendly if we use `curl endpoint` in terminal.
   ```
   <html>
   <head>
   <meta http-equiv="Content-Type" content="text/html;charset=ISO-8859-1"/>
   <title>Error 404 </title>
   </head>
   <body>
   <h2>HTTP ERROR: 404</h2>
   <p>Problem accessing /admin/v2/zookeeper/aa. Reason:
   <pre>    Not Found</pre></p>
   <hr /><a href="http://eclipse.org/jetty">Powered by Jetty:// 9.4.12.v20180830</a><hr/>
   </body>
   </html>
   ```
   
   VS
   ```
   {
     "message" : "Path /aa does not exist",
     "status": 404
   }
   ```
   

##########
File path: helix-rest/src/main/java/org/apache/helix/rest/server/resources/zookeeper/ZooKeeperAccessor.java
##########
@@ -196,6 +209,32 @@ private Response getStat(BaseDataAccessor<byte[]> zkBaseDataAccessor, String pat
     return JSONRepresentation(result);
   }
 
+  /**
+   * Delete the ZNode at the given path if exists.
+   * @param zkBaseDataAccessor
+   * @param path
+   * @return The delete result and the operated path.
+   */
+  private Response delete(BaseDataAccessor zkBaseDataAccessor, String path) {
+    Stat stat = zkBaseDataAccessor.getStat(path, AccessOption.PERSISTENT);
+    if (stat == null) {
+      return notFound();
+    } else if (stat.getEphemeralOwner() <= 0) {
+      // TODO: Remove this restriction once we have audit and ACL for the API calls.
+      // TODO: This method is added pre-maturely to support removing the live instance of a zombie
+      // TODO: instance. It is risky to allow all deleting requests before audit and ACL are done.
+      throw new WebApplicationException(Response.status(Response.Status.FORBIDDEN)
+          .entity(String.format("Deleting a non-ephemeral node is not allowed.")).build());
+    }
+
+    if (zkBaseDataAccessor.remove(path, AccessOption.PERSISTENT)) {
+      return OK();

Review comment:
       At least add a message `OK("Success")`?




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



---------------------------------------------------------------------
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 #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #1190:
URL: https://github.com/apache/helix/pull/1190#discussion_r463230652



##########
File path: helix-rest/src/main/java/org/apache/helix/rest/server/resources/zookeeper/ZooKeeperAccessor.java
##########
@@ -52,12 +51,15 @@
   private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperAccessor.class.getName());
   private BaseDataAccessor<byte[]> _zkBaseDataAccessor;
 
+  private static final String PATH_STR = "path";
+
   public enum ZooKeeperCommand {
     exists,
     getBinaryData,
     getStringData,
     getChildren,
-    getStat
+    getStat,
+    delete

Review comment:
       I still don't agree - it seems that you are suggesting using `delete`, expecting its behavior to change in the future. That will bring about backward-compatibility issues and make the meaning of "delete" murky. Moreover, there's no harm in having `deleteEphemeral`.
   
   It's fine if you need to provide another endpoint/API, but a good API design involves something that is 1) easy to do and 2) doing exactly what it's advertising to do. Do you see why I think it might be less desirable to add hidden assumptions to `delete`?




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



---------------------------------------------------------------------
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 #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #1190:
URL: https://github.com/apache/helix/pull/1190#discussion_r463922492



##########
File path: helix-rest/src/main/java/org/apache/helix/rest/server/resources/zookeeper/ZooKeeperAccessor.java
##########
@@ -196,6 +209,32 @@ private Response getStat(BaseDataAccessor<byte[]> zkBaseDataAccessor, String pat
     return JSONRepresentation(result);
   }
 
+  /**
+   * Delete the ZNode at the given path if exists.
+   * @param zkBaseDataAccessor
+   * @param path
+   * @return The delete result and the operated path.
+   */
+  private Response delete(BaseDataAccessor zkBaseDataAccessor, String path) {

Review comment:
       I'm not seeing how this is any different from using delete. This is no better than using `delete` for two different types of delete's - delete and deleteEphemeral.
   
   Perhaps you could add a commandStr here to differentiate two different types of `delete`s, and when you want to add an endpoint for regular delete backed by ACL checks, then just implement that _if_ that becomes necessary? I don't think this adds any more work/difficulty for the purposes of this PR? (If any, it saves you the work of adding a TODO)
   
   My point was not about what kind of REST verb we should use - it's pretty clear we should use DELETE in this case. But it's more about following a good API design which, again, is something that is hard to misuse by not embedding hidden assumptions or TODOs that may cause a behavior change down the road. Also, seen from another angle, supporting it as `deleteEphemeral` gives the user a clear meaning to the command string as opposed to just calling it @DELETE, which might leave the user confused and question the meaning of the API when it fails to delete regular ZNodes. 




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



---------------------------------------------------------------------
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 change in pull request #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
xyuanlu commented on a change in pull request #1190:
URL: https://github.com/apache/helix/pull/1190#discussion_r463236804



##########
File path: helix-rest/src/test/java/org/apache/helix/rest/server/TestZooKeeperAccessor.java
##########
@@ -192,4 +191,33 @@ public void testGetStat() throws IOException {
     // Clean up
     _testBaseDataAccessor.remove(path, AccessOption.PERSISTENT);
   }
+
+  @Test
+  public void testDelete() throws IOException {
+    String path = "/path";
+    String deletePath = path + "/delete";
+
+    try {
+      // 1. Create a persistent node. Delete shall fail.
+      _testBaseDataAccessor.create(deletePath, null, AccessOption.PERSISTENT);
+      // Verify with the REST endpoint
+      new JerseyUriRequestBuilder("zookeeper{}?command=delete").format(deletePath)
+          .expectedReturnStatusCode(Response.Status.FORBIDDEN.getStatusCode());
+      Assert.assertTrue(_testBaseDataAccessor.exists(deletePath, AccessOption.PERSISTENT));
+
+      // 2. Create a ephemeral node. Delete shall be done successfully.
+      _testBaseDataAccessor.remove(deletePath, AccessOption.PERSISTENT);
+      _testBaseDataAccessor.create(deletePath, null, AccessOption.EPHEMERAL);
+      // Verify with the REST endpoint
+      String data = new JerseyUriRequestBuilder("zookeeper{}?command=delete").format(deletePath)
+          .isBodyReturnExpected(true).get(this);
+      Map<String, String> result = OBJECT_MAPPER.readValue(data, HashMap.class);
+      Assert.assertEquals(result.get("path"), deletePath);
+      Assert.assertEquals(result.get("delete"), new Boolean(true).toString());
+      Assert.assertFalse(_testBaseDataAccessor.exists(deletePath, AccessOption.PERSISTENT));

Review comment:
       Right. That confuses me as well. It seems many functions has a not-used 'int option' as input. May I suggest using a value consist with the node type as input? For example, we want to check if the newly created EPHEMERAL node is deleted, let use EPHEMERAL here to avoid further confusion. 




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



---------------------------------------------------------------------
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 #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #1190:
URL: https://github.com/apache/helix/pull/1190#discussion_r463922492



##########
File path: helix-rest/src/main/java/org/apache/helix/rest/server/resources/zookeeper/ZooKeeperAccessor.java
##########
@@ -196,6 +209,32 @@ private Response getStat(BaseDataAccessor<byte[]> zkBaseDataAccessor, String pat
     return JSONRepresentation(result);
   }
 
+  /**
+   * Delete the ZNode at the given path if exists.
+   * @param zkBaseDataAccessor
+   * @param path
+   * @return The delete result and the operated path.
+   */
+  private Response delete(BaseDataAccessor zkBaseDataAccessor, String path) {

Review comment:
       I'm not seeing how this is any different from using delete. This is no better than using `delete` for two different types of delete's - delete and deleteEphemeral.
   
   Perhaps you could add a commandStr here to differentiate two different types of `delete`s, and when you want to add an endpoint for regular delete backed by ACL checks, then just implement that _if_ that becomes necessary? I don't think this adds any more work/difficulty for the purposes of this PR? (If any, it saves you the work of adding a TODO)
   
   My point was not about what kind of REST verb we should use - it's pretty clear we should use DELETE in this case. But it's more about following a good API design which, again, is something that is hard to misuse by not embedding hidden assumptions or TODOs that may cause a behavior change down the road. Also, seen from another angle, supporting it as `deleteEphemeral` gives the user a clear meaning to the command string as opposed to just calling it a HTTP verb DELETE, which might leave the user confused and question the meaning of the API when it fails to delete regular ZNodes. 
   
   You could add two commands, delete and deleteEphemeral, and make the default commandStr delete, and throw a not authorized or not supported, and only let deleteEphemeral go through. This way, when we do decide to support delete operation with ACL, there's no confusion or change in behavior.




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



---------------------------------------------------------------------
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 #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #1190:
URL: https://github.com/apache/helix/pull/1190#discussion_r463922492



##########
File path: helix-rest/src/main/java/org/apache/helix/rest/server/resources/zookeeper/ZooKeeperAccessor.java
##########
@@ -196,6 +209,32 @@ private Response getStat(BaseDataAccessor<byte[]> zkBaseDataAccessor, String pat
     return JSONRepresentation(result);
   }
 
+  /**
+   * Delete the ZNode at the given path if exists.
+   * @param zkBaseDataAccessor
+   * @param path
+   * @return The delete result and the operated path.
+   */
+  private Response delete(BaseDataAccessor zkBaseDataAccessor, String path) {

Review comment:
       I'm not seeing how this is any different from using delete. This is no better than using `delete` for two different types of delete's - delete and deleteEphemeral.
   
   Perhaps you could add a commandStr here to differentiate two different types of `delete`s, and when you want to add an endpoint for regular delete backed by ACL checks, then just implement that _if_ that becomes necessary? I don't think this adds any more work/difficulty for the purposes of this PR? (If any, it saves you the work of adding a TODO)
   
   My point was not about what kind of REST verb we should use - it's pretty clear we should use DELETE in this case. But it's more about following a good API design which, again, is something that is hard to misuse by not embedding hidden assumptions or TODOs that may cause a behavior change down the road. Also, seen from another angle, supporting it as `deleteEphemeral` gives the user a clear meaning to the command string as opposed to just calling it a HTTP verb DELETE, which might leave the user confused and question the meaning of the API when it fails to delete regular ZNodes. 
   
   You could add two commands, delete and deleteEphemeral, and make the default commandStr delete, and throw a not authorized or not supported, and only let deleteEphemeral go through.




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



---------------------------------------------------------------------
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 #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #1190:
URL: https://github.com/apache/helix/pull/1190#discussion_r463922492



##########
File path: helix-rest/src/main/java/org/apache/helix/rest/server/resources/zookeeper/ZooKeeperAccessor.java
##########
@@ -196,6 +209,32 @@ private Response getStat(BaseDataAccessor<byte[]> zkBaseDataAccessor, String pat
     return JSONRepresentation(result);
   }
 
+  /**
+   * Delete the ZNode at the given path if exists.
+   * @param zkBaseDataAccessor
+   * @param path
+   * @return The delete result and the operated path.
+   */
+  private Response delete(BaseDataAccessor zkBaseDataAccessor, String path) {

Review comment:
       I'm not seeing how this is any different from using delete. This is no better than using `delete` for two different types of delete's - delete and deleteEphemeral.
   
   Perhaps you could add a commandStr here to differentiate two different types of `delete`s, and when you want to add an endpoint for regular delete backed by ACL checks, then just implement that _if_ that becomes necessary?
   
   My point was not about what kind of REST verb we should use - it's pretty clear we should use DELETE in this case. But it's more about following a good API design which, again, is something that is hard to misuse by not embedding hidden assumptions or TODOs that may cause a behavior change down the road.




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



---------------------------------------------------------------------
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 change in pull request #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
xyuanlu commented on a change in pull request #1190:
URL: https://github.com/apache/helix/pull/1190#discussion_r463236804



##########
File path: helix-rest/src/test/java/org/apache/helix/rest/server/TestZooKeeperAccessor.java
##########
@@ -192,4 +191,33 @@ public void testGetStat() throws IOException {
     // Clean up
     _testBaseDataAccessor.remove(path, AccessOption.PERSISTENT);
   }
+
+  @Test
+  public void testDelete() throws IOException {
+    String path = "/path";
+    String deletePath = path + "/delete";
+
+    try {
+      // 1. Create a persistent node. Delete shall fail.
+      _testBaseDataAccessor.create(deletePath, null, AccessOption.PERSISTENT);
+      // Verify with the REST endpoint
+      new JerseyUriRequestBuilder("zookeeper{}?command=delete").format(deletePath)
+          .expectedReturnStatusCode(Response.Status.FORBIDDEN.getStatusCode());
+      Assert.assertTrue(_testBaseDataAccessor.exists(deletePath, AccessOption.PERSISTENT));
+
+      // 2. Create a ephemeral node. Delete shall be done successfully.
+      _testBaseDataAccessor.remove(deletePath, AccessOption.PERSISTENT);
+      _testBaseDataAccessor.create(deletePath, null, AccessOption.EPHEMERAL);
+      // Verify with the REST endpoint
+      String data = new JerseyUriRequestBuilder("zookeeper{}?command=delete").format(deletePath)
+          .isBodyReturnExpected(true).get(this);
+      Map<String, String> result = OBJECT_MAPPER.readValue(data, HashMap.class);
+      Assert.assertEquals(result.get("path"), deletePath);
+      Assert.assertEquals(result.get("delete"), new Boolean(true).toString());
+      Assert.assertFalse(_testBaseDataAccessor.exists(deletePath, AccessOption.PERSISTENT));

Review comment:
       Yea. That confuses me as well. It seems many functions has a not-used 'int option' as input in BaseDataAccessor. May I suggest using a value consist with the node type as input? For example, we want to check if the newly created EPHEMERAL node is deleted, let use EPHEMERAL here to avoid further confusion. 




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



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


[GitHub] [helix] pkuwm commented on a change in pull request #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #1190:
URL: https://github.com/apache/helix/pull/1190#discussion_r463471112



##########
File path: helix-rest/src/main/java/org/apache/helix/rest/server/resources/zookeeper/ZooKeeperAccessor.java
##########
@@ -101,6 +100,26 @@ public Response get(@PathParam("path") String path, @QueryParam("command") Strin
     }
   }
 
+  @DELETE
+  @Path("{path: .+}")
+  public Response delete(@PathParam("path") String path) {
+    // Lazily initialize ZkBaseDataAccessor
+    ServerContext _serverContext =
+        (ServerContext) _application.getProperties().get(ContextPropertyKeys.SERVER_CONTEXT.name());
+    _zkBaseDataAccessor = _serverContext.getByteArrayZkBaseDataAccessor();
+
+    path = prependPath(path);

Review comment:
       Actually if we set the root path as below, we don't need to prepend the root slash, which is not that clean.
   ```
   @Path("/zookeeper{path: /.+}")
   public class ZooKeeperAccessor extends AbstractResource {
   ```




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



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


[GitHub] [helix] jiajunwang commented on a change in pull request #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1190:
URL: https://github.com/apache/helix/pull/1190#discussion_r463227545



##########
File path: helix-rest/src/main/java/org/apache/helix/rest/server/resources/zookeeper/ZooKeeperAccessor.java
##########
@@ -52,12 +51,15 @@
   private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperAccessor.class.getName());
   private BaseDataAccessor<byte[]> _zkBaseDataAccessor;
 
+  private static final String PATH_STR = "path";
+
   public enum ZooKeeperCommand {
     exists,
     getBinaryData,
     getStringData,
     getChildren,
-    getStat
+    getStat,
+    delete

Review comment:
       I don't think we want "deleteEphemeral" eventually. As mentioned in the description, this is a premature feature that we add now for unblocking our users.
   
   Alternatively, I tried to allow deleting live instance only. But that will pollute the ZookeeperAccessor API with Helix logics. So I discarded that idea.




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



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


[GitHub] [helix] jiajunwang commented on a change in pull request #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1190:
URL: https://github.com/apache/helix/pull/1190#discussion_r463229646



##########
File path: helix-rest/src/test/java/org/apache/helix/rest/server/TestZooKeeperAccessor.java
##########
@@ -192,4 +191,33 @@ public void testGetStat() throws IOException {
     // Clean up
     _testBaseDataAccessor.remove(path, AccessOption.PERSISTENT);
   }
+
+  @Test
+  public void testDelete() throws IOException {
+    String path = "/path";
+    String deletePath = path + "/delete";
+
+    try {
+      // 1. Create a persistent node. Delete shall fail.
+      _testBaseDataAccessor.create(deletePath, null, AccessOption.PERSISTENT);
+      // Verify with the REST endpoint
+      new JerseyUriRequestBuilder("zookeeper{}?command=delete").format(deletePath)
+          .expectedReturnStatusCode(Response.Status.FORBIDDEN.getStatusCode());
+      Assert.assertTrue(_testBaseDataAccessor.exists(deletePath, AccessOption.PERSISTENT));
+
+      // 2. Create a ephemeral node. Delete shall be done successfully.
+      _testBaseDataAccessor.remove(deletePath, AccessOption.PERSISTENT);
+      _testBaseDataAccessor.create(deletePath, null, AccessOption.EPHEMERAL);
+      // Verify with the REST endpoint
+      String data = new JerseyUriRequestBuilder("zookeeper{}?command=delete").format(deletePath)
+          .isBodyReturnExpected(true).get(this);
+      Map<String, String> result = OBJECT_MAPPER.readValue(data, HashMap.class);
+      Assert.assertEquals(result.get("path"), deletePath);
+      Assert.assertEquals(result.get("delete"), new Boolean(true).toString());
+      Assert.assertFalse(_testBaseDataAccessor.exists(deletePath, AccessOption.PERSISTENT));

Review comment:
       No use at all. There is no such thing in ZK lib. I think we shall clean them up in a separate PR.




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



---------------------------------------------------------------------
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 #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #1190:
URL: https://github.com/apache/helix/pull/1190#discussion_r463922492



##########
File path: helix-rest/src/main/java/org/apache/helix/rest/server/resources/zookeeper/ZooKeeperAccessor.java
##########
@@ -196,6 +209,32 @@ private Response getStat(BaseDataAccessor<byte[]> zkBaseDataAccessor, String pat
     return JSONRepresentation(result);
   }
 
+  /**
+   * Delete the ZNode at the given path if exists.
+   * @param zkBaseDataAccessor
+   * @param path
+   * @return The delete result and the operated path.
+   */
+  private Response delete(BaseDataAccessor zkBaseDataAccessor, String path) {

Review comment:
       I'm not seeing how this is any different from using delete. This is no better than using `delete` for two different types of delete's - delete and deleteEphemeral.
   
   Perhaps you could add a commandStr here to differentiate two different types of `delete`s, and when you want to add an endpoint for regular delete backed by ACL checks, then just implement that _if_ that becomes necessary? Is this difficult? I don't think this adds any more work for the purposes of this PR?
   
   My point was not about what kind of REST verb we should use - it's pretty clear we should use DELETE in this case. But it's more about following a good API design which, again, is something that is hard to misuse by not embedding hidden assumptions or TODOs that may cause a behavior change down the road.




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



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


[GitHub] [helix] jiajunwang commented on a change in pull request #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1190:
URL: https://github.com/apache/helix/pull/1190#discussion_r463338776



##########
File path: helix-rest/src/main/java/org/apache/helix/rest/server/resources/zookeeper/ZooKeeperAccessor.java
##########
@@ -52,12 +51,15 @@
   private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperAccessor.class.getName());
   private BaseDataAccessor<byte[]> _zkBaseDataAccessor;
 
+  private static final String PATH_STR = "path";
+
   public enum ZooKeeperCommand {
     exists,
     getBinaryData,
     getStringData,
     getChildren,
-    getStat
+    getStat,
+    delete

Review comment:
       I can see where you are coming from. Could you check the latest change that I have modified the method to use DELETE verb according to what Huizhi suggested? I think it is cleaner. However, in this case, we need some more parameters to separate the cases. And I think it might be overcomplicated.
   
   @dasahcc and @pkuwm please also share your opinion since you also contributed to the Helix rest.




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



---------------------------------------------------------------------
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 change in pull request #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
xyuanlu commented on a change in pull request #1190:
URL: https://github.com/apache/helix/pull/1190#discussion_r463223224



##########
File path: helix-rest/src/test/java/org/apache/helix/rest/server/TestZooKeeperAccessor.java
##########
@@ -192,4 +191,33 @@ public void testGetStat() throws IOException {
     // Clean up
     _testBaseDataAccessor.remove(path, AccessOption.PERSISTENT);
   }
+
+  @Test
+  public void testDelete() throws IOException {
+    String path = "/path";
+    String deletePath = path + "/delete";
+
+    try {
+      // 1. Create a persistent node. Delete shall fail.
+      _testBaseDataAccessor.create(deletePath, null, AccessOption.PERSISTENT);
+      // Verify with the REST endpoint
+      new JerseyUriRequestBuilder("zookeeper{}?command=delete").format(deletePath)
+          .expectedReturnStatusCode(Response.Status.FORBIDDEN.getStatusCode());
+      Assert.assertTrue(_testBaseDataAccessor.exists(deletePath, AccessOption.PERSISTENT));
+
+      // 2. Create a ephemeral node. Delete shall be done successfully.
+      _testBaseDataAccessor.remove(deletePath, AccessOption.PERSISTENT);
+      _testBaseDataAccessor.create(deletePath, null, AccessOption.EPHEMERAL);
+      // Verify with the REST endpoint
+      String data = new JerseyUriRequestBuilder("zookeeper{}?command=delete").format(deletePath)
+          .isBodyReturnExpected(true).get(this);
+      Map<String, String> result = OBJECT_MAPPER.readValue(data, HashMap.class);
+      Assert.assertEquals(result.get("path"), deletePath);
+      Assert.assertEquals(result.get("delete"), new Boolean(true).toString());
+      Assert.assertFalse(_testBaseDataAccessor.exists(deletePath, AccessOption.PERSISTENT));

Review comment:
       Question here (may not related to your change).
   Is the second param here in `exists(String path, int options)` useful? I did not find it being considered in any implementation of BaseDataAccessor.  




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



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


[GitHub] [helix] jiajunwang commented on pull request #1190: Add Helix rest Zookeeper delete API to allow removing ephemeral ZNode

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on pull request #1190:
URL: https://github.com/apache/helix/pull/1190#issuecomment-668300447


   This PR is ready to be merged, approved by @pkuwm 


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



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