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 2023/01/18 22:04:35 UTC

[GitHub] [helix] Marcosrico opened a new pull request, #2343: ZkMetaclient - implementation of TransactionOp

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

   ### Issues
   
   - [ ] My PR addresses the following Helix issues and references them in the PR description:
   
   #2237 
   
   ### Description
   
   - [ ] Here are some details about my PR, including screenshots of any UI changes:
   
   Implementing transactional support from CRUD operations for ZkMetaClient.
   
   ### Tests
   
   - [ ] The following tests are written for this issue:
   
   `testMultiOps` and `testMultiFail` in `TestZkMetaClient`.
   
   - The following is the result of the "mvn test" command on the appropriate module:
   
   (If CI test fails due to known issue, please specify the issue and test PR locally. Then copy & paste the result of "mvn test" to here.)
   
   ### Code Quality
   
   - My diff has been formatted using helix-style.xml 
   (helix-style-intellij.xml if IntelliJ IDE is used)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] Marcosrico commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

Posted by GitBox <gi...@apache.org>.
Marcosrico commented on code in PR #2343:
URL: https://github.com/apache/helix/pull/2343#discussion_r1081355091


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/ZkMetaClient.java:
##########
@@ -414,4 +414,130 @@ private static EntryMode convertZkEntryMode(long ephemeralOwner) {
         throw new IllegalArgumentException(zkEphemeralType + " is not supported.");
     }
   }
+
+
+  @Override
+  public List<OpResult> transactionOP(Iterable<Op> iterable) throws KeeperException {
+    // Convert list of MetaClient Ops to Zk Ops
+    List<org.apache.zookeeper.Op> zkOps = metaClientOpToZk(iterable);
+    // Execute Zk transactional support
+    List<org.apache.zookeeper.OpResult> zkResult = _zkClient.multi(zkOps);
+    // Convert list of Zk OpResults to MetaClient OpResults
+    return zkOpResultToMetaClient(zkResult);
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public List<org.apache.zookeeper.Op> metaClientOpToZk(Iterable<Op> ops) throws KeeperException {

Review Comment:
   Nope not required, thanks for the 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.

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

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


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


[GitHub] [helix] Marcosrico commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

Posted by GitBox <gi...@apache.org>.
Marcosrico commented on code in PR #2343:
URL: https://github.com/apache/helix/pull/2343#discussion_r1081394971


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/ZkMetaClient.java:
##########
@@ -414,4 +414,130 @@ private static EntryMode convertZkEntryMode(long ephemeralOwner) {
         throw new IllegalArgumentException(zkEphemeralType + " is not supported.");
     }
   }
+
+
+  @Override
+  public List<OpResult> transactionOP(Iterable<Op> iterable) throws KeeperException {
+    // Convert list of MetaClient Ops to Zk Ops
+    List<org.apache.zookeeper.Op> zkOps = metaClientOpToZk(iterable);
+    // Execute Zk transactional support
+    List<org.apache.zookeeper.OpResult> zkResult = _zkClient.multi(zkOps);
+    // Convert list of Zk OpResults to MetaClient OpResults
+    return zkOpResultToMetaClient(zkResult);
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public List<org.apache.zookeeper.Op> metaClientOpToZk(Iterable<Op> ops) throws KeeperException {
+    List<org.apache.zookeeper.Op> zkOps = new ArrayList<>();
+    org.apache.zookeeper.Op temp;
+    for (Op op : ops) {
+      switch (op.getType()) {
+        case CREATE:
+          int zkFlag = zkFlagFromEntryMode(((Op.Create) op).getEntryMode());
+          temp = org.apache.zookeeper.Op.create(
+              op.getPath(), ((Op.Create) op).getData(), DEFAULT_ACL, CreateMode.fromFlag(zkFlag));
+          break;
+        case DELETE:
+          temp = org.apache.zookeeper.Op.delete(
+              op.getPath(), ((Op.Delete) op).getVersion());
+          break;
+        case SET:
+          temp = org.apache.zookeeper.Op.setData(
+              op.getPath(), ((Op.Set) op).getData(), ((Op.Set) op).getVersion());
+          break;
+        case CHECK:
+          temp = org.apache.zookeeper.Op.check(
+              op.getPath(), ((Op.Check) op).getVersion());
+          break;
+        default:
+          throw new IllegalArgumentException(op.getType() + " is not supported.");
+      }
+      zkOps.add(temp);
+    }
+    return zkOps;
+  }
+
+  /**
+   * Helper function for transactionOP. Converts the result from calling zk transactional support into
+   * metaclient OpResults.
+   * @param zkResult
+   * @return
+   */
+  public List<OpResult> zkOpResultToMetaClient(List<org.apache.zookeeper.OpResult> zkResult) throws KeeperException.BadArgumentsException {

Review Comment:
   Sounds good, will create a specific MetaClientKeeperException for these cases (where KeeperExceptions are thrown)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] qqu0127 commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/util/ZkMetaClientUtil.java:
##########
@@ -0,0 +1,213 @@
+package org.apache.helix.metaclient.impl.zk.util;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.api.OpResult;
+import org.apache.helix.metaclient.constants.*;
+import org.apache.helix.zookeeper.zkclient.exception.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.EphemeralType;
+
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.function.Function;
+
+public class ZkMetaClientUtil {
+  //TODO Implement MetaClient ACL
+  //Default ACL value until metaClient Op has ACL of its own.
+  private static final List<ACL> DEFAULT_ACL = Collections.unmodifiableList(ZooDefs.Ids.OPEN_ACL_UNSAFE);
+
+  private ZkMetaClientUtil(){
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public static List<Op> metaClientOpsToZkOps(Iterable<org.apache.helix.metaclient.api.Op> ops) {
+    if (getOpMap().isEmpty()) {
+      initializeOpMap();
+    }
+
+    List<Op> zkOps = new ArrayList<>();
+    for (org.apache.helix.metaclient.api.Op op : ops) {
+      Function<org.apache.helix.metaclient.api.Op, Op> function = getOpMap().get(op.getType());
+      if (function != null) {
+        zkOps.add(function.apply(op));
+      } else {
+        throw new IllegalArgumentException("Op type " + op.getType().name() + " is not supported.");
+      }
+    }
+    return zkOps;
+  }
+
+  private static final class OpMapHolder {
+    static final Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> OPMAP =
+        new EnumMap<>(org.apache.helix.metaclient.api.Op.Type.class);
+  }
+
+  private static Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> getOpMap() {
+    return OpMapHolder.OPMAP;
+  }
+
+  private static void initializeOpMap() {
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.CREATE, op -> {
+      try {
+        CreateMode mode = convertMetaClientMode(((org.apache.helix.metaclient.api.Op.Create) op).getEntryMode());
+        return Op.create(op.getPath(), ((org.apache.helix.metaclient.api.Op.Create) op).getData(), DEFAULT_ACL, mode);
+      } catch (KeeperException e) {
+        throw translateZkExceptionToMetaclientException(ZkException.create(e));
+      }
+    });
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.DELETE,
+        op -> Op.delete(op.getPath(), ((org.apache.helix.metaclient.api.Op.Delete) op).getVersion()));
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.SET,
+        op -> Op.setData(op.getPath(),
+            ((org.apache.helix.metaclient.api.Op.Set) op).getData(), ((org.apache.helix.metaclient.api.Op.Set) op).getVersion()));
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.CHECK,
+        op -> Op.check(op.getPath(), ((org.apache.helix.metaclient.api.Op.Check) op).getVersion()));
+  }
+
+  private static CreateMode convertMetaClientMode(MetaClientInterface.EntryMode entryMode) throws KeeperException {
+    switch (entryMode) {
+      case PERSISTENT:
+        return CreateMode.PERSISTENT;
+      case EPHEMERAL:
+        return CreateMode.EPHEMERAL;
+      case CONTAINER:
+        return CreateMode.CONTAINER;
+      default:
+        throw new IllegalArgumentException(entryMode.name() + " is not a supported EntryMode.");
+    }
+  }
+
+  /**
+   * Helper function for transactionOP. Converts the result from calling zk transactional support into
+   * metaclient OpResults.
+   * @param zkResult
+   * @return
+   */
+  public static List<OpResult> zkOpResultToMetaClientOpResults(List<org.apache.zookeeper.OpResult> zkResult) {
+    if (getOpResultMap().isEmpty()) {
+      initializeOpResultMap();
+    }
+    List<OpResult> metaClientOpResult = new ArrayList<>();
+    for (org.apache.zookeeper.OpResult opResult : zkResult) {
+      Function<org.apache.zookeeper.OpResult, OpResult> function = getOpResultMap().get(opResult.getClass());
+      if (function != null) {
+        metaClientOpResult.add(function.apply(opResult));
+      } else {
+        throw new IllegalArgumentException("OpResult type " + opResult.getType() + "is not supported.");
+      }
+    }
+
+    return metaClientOpResult;
+  }
+
+  private static final class OpResultMapHolder {
+    static final Map<Class<? extends org.apache.zookeeper.OpResult>, Function<org.apache.zookeeper.OpResult, OpResult>> OPRESULTMAP =
+        new HashMap<>();
+  }

Review Comment:
   Thanks for the work Marcos. 
   I see the current lazy loading functionally works, but it's kind of redundant with this holder. You might miss the point of using a on-demand holder. I believe we can move the init() method inline. Please refer to JVM classloading and on-demand holder idiom. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] xyuanlu commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

Posted by GitBox <gi...@apache.org>.
xyuanlu commented on code in PR #2343:
URL: https://github.com/apache/helix/pull/2343#discussion_r1080686322


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/ZkMetaClient.java:
##########
@@ -414,4 +414,130 @@ private static EntryMode convertZkEntryMode(long ephemeralOwner) {
         throw new IllegalArgumentException(zkEphemeralType + " is not supported.");
     }
   }
+
+
+  @Override
+  public List<OpResult> transactionOP(Iterable<Op> iterable) throws KeeperException {
+    // Convert list of MetaClient Ops to Zk Ops
+    List<org.apache.zookeeper.Op> zkOps = metaClientOpToZk(iterable);
+    // Execute Zk transactional support
+    List<org.apache.zookeeper.OpResult> zkResult = _zkClient.multi(zkOps);
+    // Convert list of Zk OpResults to MetaClient OpResults
+    return zkOpResultToMetaClient(zkResult);
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public List<org.apache.zookeeper.Op> metaClientOpToZk(Iterable<Op> ops) throws KeeperException {
+    List<org.apache.zookeeper.Op> zkOps = new ArrayList<>();
+    org.apache.zookeeper.Op temp;
+    for (Op op : ops) {
+      switch (op.getType()) {
+        case CREATE:
+          int zkFlag = zkFlagFromEntryMode(((Op.Create) op).getEntryMode());
+          temp = org.apache.zookeeper.Op.create(
+              op.getPath(), ((Op.Create) op).getData(), DEFAULT_ACL, CreateMode.fromFlag(zkFlag));
+          break;
+        case DELETE:
+          temp = org.apache.zookeeper.Op.delete(
+              op.getPath(), ((Op.Delete) op).getVersion());
+          break;
+        case SET:
+          temp = org.apache.zookeeper.Op.setData(
+              op.getPath(), ((Op.Set) op).getData(), ((Op.Set) op).getVersion());
+          break;
+        case CHECK:
+          temp = org.apache.zookeeper.Op.check(
+              op.getPath(), ((Op.Check) op).getVersion());
+          break;
+        default:
+          throw new IllegalArgumentException(op.getType() + " is not supported.");
+      }
+      zkOps.add(temp);
+    }
+    return zkOps;
+  }
+
+  /**
+   * Helper function for transactionOP. Converts the result from calling zk transactional support into
+   * metaclient OpResults.
+   * @param zkResult
+   * @return
+   */
+  public List<OpResult> zkOpResultToMetaClient(List<org.apache.zookeeper.OpResult> zkResult) throws KeeperException.BadArgumentsException {

Review Comment:
   Please define metaclient exception under org.apache.helix.metaclient.constants. 
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] Marcosrico commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

Posted by GitBox <gi...@apache.org>.
Marcosrico commented on code in PR #2343:
URL: https://github.com/apache/helix/pull/2343#discussion_r1081842259


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/ZkMetaClient.java:
##########
@@ -50,11 +51,15 @@
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.server.EphemeralType;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
 
 
 public class ZkMetaClient<T> implements MetaClientInterface<T>, AutoCloseable {
 
   private final ZkClient _zkClient;
+  //Default ACL value until metaClient Op has ACL of its own.
+  private final List<ACL> DEFAULT_ACL = ZooDefs.Ids.OPEN_ACL_UNSAFE;

Review Comment:
   Realizing that it would still be modifiable in this case. Calling Collections.unmodifiableList(DEFAULT_ACL) is the best course of action



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] Marcosrico commented on pull request #2343: ZkMetaclient - implementation of TransactionOp

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

   Fixing merge issues still


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] mgao0 commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/util/ZkMetaClientUtil.java:
##########
@@ -0,0 +1,208 @@
+package org.apache.helix.metaclient.impl.zk.util;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.api.OpResult;
+import org.apache.helix.metaclient.constants.*;
+import org.apache.helix.zookeeper.zkclient.exception.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.EphemeralType;
+
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.function.Function;
+
+public class ZkMetaClientUtil {
+  //TODO Implement MetaClient ACL
+  //Default ACL value until metaClient Op has ACL of its own.
+  private static final List<ACL> DEFAULT_ACL = Collections.unmodifiableList(ZooDefs.Ids.OPEN_ACL_UNSAFE);
+
+  private ZkMetaClientUtil(){
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public static List<Op> metaClientOpsToZkOps(Iterable<org.apache.helix.metaclient.api.Op> ops) {
+    initializeOpMap();
+    List<Op> zkOps = new ArrayList<>();
+    for (org.apache.helix.metaclient.api.Op op : ops) {
+      Function<org.apache.helix.metaclient.api.Op, Op> function = getOpMap().get(op.getType());
+      if (function != null) {
+        zkOps.add(function.apply(op));
+      } else {
+        throw new IllegalArgumentException("Op type " + op.getType().name() + " is not supported.");
+      }
+    }
+    return zkOps;
+  }
+
+  private static final class OpMapHolder {
+    static final Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> OPMAP =
+        new EnumMap<>(org.apache.helix.metaclient.api.Op.Type.class);
+  }
+
+  private static Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> getOpMap() {

Review Comment:
   Curious, why do we need this method? Can we directly refer to OPMAP?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] Marcosrico commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/util/ZkMetaClientUtil.java:
##########
@@ -0,0 +1,201 @@
+package org.apache.helix.metaclient.impl.zk.util;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.api.OpResult;
+import org.apache.helix.metaclient.constants.*;
+import org.apache.helix.zookeeper.zkclient.exception.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.EphemeralType;
+
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.function.Function;
+
+public class ZkMetaClientUtil {
+  //Default ACL value until metaClient Op has ACL of its own.
+  private static final List<ACL> DEFAULT_ACL = Collections.unmodifiableList(ZooDefs.Ids.OPEN_ACL_UNSAFE);
+
+  private ZkMetaClientUtil(){
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public static List<Op> metaClientOpToZk(Iterable<org.apache.helix.metaclient.api.Op> ops) {
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.CREATE, op -> {

Review Comment:
   Unfortunately because both package names (metaclient and zookeeper) have the same name (op and opResult) they must be specified with their full name.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] Marcosrico commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

Posted by GitBox <gi...@apache.org>.
Marcosrico commented on code in PR #2343:
URL: https://github.com/apache/helix/pull/2343#discussion_r1081380987


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/ZkMetaClient.java:
##########
@@ -414,4 +414,130 @@ private static EntryMode convertZkEntryMode(long ephemeralOwner) {
         throw new IllegalArgumentException(zkEphemeralType + " is not supported.");
     }
   }
+
+
+  @Override
+  public List<OpResult> transactionOP(Iterable<Op> iterable) throws KeeperException {
+    // Convert list of MetaClient Ops to Zk Ops
+    List<org.apache.zookeeper.Op> zkOps = metaClientOpToZk(iterable);
+    // Execute Zk transactional support
+    List<org.apache.zookeeper.OpResult> zkResult = _zkClient.multi(zkOps);
+    // Convert list of Zk OpResults to MetaClient OpResults
+    return zkOpResultToMetaClient(zkResult);
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public List<org.apache.zookeeper.Op> metaClientOpToZk(Iterable<Op> ops) throws KeeperException {

Review Comment:
   The `CreateMode.fromFlag()`  method throws KeeperException. Referring to a comment below, I'll create a specific MetaClientKeeperException to be thrown for these cases.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] Marcosrico commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

Posted by GitBox <gi...@apache.org>.
Marcosrico commented on code in PR #2343:
URL: https://github.com/apache/helix/pull/2343#discussion_r1081974549


##########
meta-client/src/main/java/org/apache/helix/metaclient/constants/MetaClientKeeperException.java:
##########
@@ -0,0 +1,39 @@
+package org.apache.helix.metaclient.constants;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+public final class MetaClientKeeperException extends MetaClientException {

Review Comment:
   In this case, it wouldn't be necessary to create a separate MetaClientKeeperException then? A ZkException then throws a generic MetaClient exception



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] Marcosrico commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/util/ZkMetaClientUtil.java:
##########
@@ -0,0 +1,213 @@
+package org.apache.helix.metaclient.impl.zk.util;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.api.OpResult;
+import org.apache.helix.metaclient.constants.*;
+import org.apache.helix.zookeeper.zkclient.exception.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.EphemeralType;
+
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.function.Function;
+
+public class ZkMetaClientUtil {
+  //TODO Implement MetaClient ACL
+  //Default ACL value until metaClient Op has ACL of its own.
+  private static final List<ACL> DEFAULT_ACL = Collections.unmodifiableList(ZooDefs.Ids.OPEN_ACL_UNSAFE);
+
+  private ZkMetaClientUtil(){
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public static List<Op> metaClientOpsToZkOps(Iterable<org.apache.helix.metaclient.api.Op> ops) {
+    if (getOpMap().isEmpty()) {
+      initializeOpMap();
+    }
+
+    List<Op> zkOps = new ArrayList<>();
+    for (org.apache.helix.metaclient.api.Op op : ops) {
+      Function<org.apache.helix.metaclient.api.Op, Op> function = getOpMap().get(op.getType());
+      if (function != null) {
+        zkOps.add(function.apply(op));
+      } else {
+        throw new IllegalArgumentException("Op type " + op.getType().name() + " is not supported.");
+      }
+    }
+    return zkOps;
+  }
+
+  private static final class OpMapHolder {
+    static final Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> OPMAP =
+        new EnumMap<>(org.apache.helix.metaclient.api.Op.Type.class);
+  }
+
+  private static Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> getOpMap() {
+    return OpMapHolder.OPMAP;
+  }
+
+  private static void initializeOpMap() {
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.CREATE, op -> {
+      try {
+        CreateMode mode = convertMetaClientMode(((org.apache.helix.metaclient.api.Op.Create) op).getEntryMode());
+        return Op.create(op.getPath(), ((org.apache.helix.metaclient.api.Op.Create) op).getData(), DEFAULT_ACL, mode);
+      } catch (KeeperException e) {
+        throw translateZkExceptionToMetaclientException(ZkException.create(e));
+      }
+    });
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.DELETE,
+        op -> Op.delete(op.getPath(), ((org.apache.helix.metaclient.api.Op.Delete) op).getVersion()));
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.SET,
+        op -> Op.setData(op.getPath(),
+            ((org.apache.helix.metaclient.api.Op.Set) op).getData(), ((org.apache.helix.metaclient.api.Op.Set) op).getVersion()));
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.CHECK,
+        op -> Op.check(op.getPath(), ((org.apache.helix.metaclient.api.Op.Check) op).getVersion()));
+  }
+
+  private static CreateMode convertMetaClientMode(MetaClientInterface.EntryMode entryMode) throws KeeperException {
+    switch (entryMode) {
+      case PERSISTENT:
+        return CreateMode.PERSISTENT;
+      case EPHEMERAL:
+        return CreateMode.EPHEMERAL;
+      case CONTAINER:
+        return CreateMode.CONTAINER;
+      default:
+        throw new IllegalArgumentException(entryMode.name() + " is not a supported EntryMode.");
+    }
+  }
+
+  /**
+   * Helper function for transactionOP. Converts the result from calling zk transactional support into
+   * metaclient OpResults.
+   * @param zkResult
+   * @return
+   */
+  public static List<OpResult> zkOpResultToMetaClientOpResults(List<org.apache.zookeeper.OpResult> zkResult) {
+    if (getOpResultMap().isEmpty()) {
+      initializeOpResultMap();
+    }
+    List<OpResult> metaClientOpResult = new ArrayList<>();
+    for (org.apache.zookeeper.OpResult opResult : zkResult) {
+      Function<org.apache.zookeeper.OpResult, OpResult> function = getOpResultMap().get(opResult.getClass());
+      if (function != null) {
+        metaClientOpResult.add(function.apply(opResult));
+      } else {
+        throw new IllegalArgumentException("OpResult type " + opResult.getType() + "is not supported.");
+      }
+    }
+
+    return metaClientOpResult;
+  }
+
+  private static final class OpResultMapHolder {
+    static final Map<Class<? extends org.apache.zookeeper.OpResult>, Function<org.apache.zookeeper.OpResult, OpResult>> OPRESULTMAP =
+        new HashMap<>();
+  }

Review Comment:
   Correct me if I am wrong, but I believe what you're saying is to have the map be populated when the static variable `OPRESULTMAP` is declared? I have attempted several methods and none seemed feasible:
   1. Create a constructor that invokes `initializeOpResultMap` method when `OpResultMapHolder` object is created. However, given the nature of it being a util class, making it an object is counterproductive.
   2. Populating the map directly when declaring the `OPRESULTMAP` static variable instead of having a separate method. This can only be done with java JDK 9+ (current library is at 8) so decided against it.
   
   With the current implementation, `initializeOpResultMap` is called just once when needed in the `zkOpResultToMetaClientOpResults` method. Inside, `getOpResultMap` is called first which initializes the `OPRESULTMAP` for the first time, effectively achieving lazy loading. Though ideally when initializing `OPRESULTMAP` for the first time, we would like it to automatically be populated with its values, as mentioned above, I couldn't find a reasonable way to do so. Please suggest a solution that I may have missed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] Marcosrico commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

Posted by GitBox <gi...@apache.org>.
Marcosrico commented on code in PR #2343:
URL: https://github.com/apache/helix/pull/2343#discussion_r1081994436


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/util/ZkMetaClientUtil.java:
##########
@@ -0,0 +1,156 @@
+package org.apache.helix.metaclient.impl.zk.util;
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.api.OpResult;
+import org.apache.helix.metaclient.constants.*;
+import org.apache.helix.zookeeper.zkclient.exception.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.EphemeralType;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.function.Function;
+
+public class ZkMetaClientUtil {
+  //Default ACL value until metaClient Op has ACL of its own.
+  private static final List<ACL> DEFAULT_ACL = Collections.unmodifiableList(ZooDefs.Ids.OPEN_ACL_UNSAFE);
+
+  private ZkMetaClientUtil(){
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public static List<Op> metaClientOpToZk(Iterable<org.apache.helix.metaclient.api.Op> ops) {
+    Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> opMap = new HashMap<>();
+
+    opMap.put(org.apache.helix.metaclient.api.Op.Type.CREATE, op -> {
+      try {
+        CreateMode mode = convertMetaClientMode(((org.apache.helix.metaclient.api.Op.Create) op).getEntryMode());
+        return Op.create(op.getPath(), ((org.apache.helix.metaclient.api.Op.Create) op).getData(), DEFAULT_ACL, mode);
+      } catch (KeeperException e) {
+        throw translateZkExceptionToMetaclientException(ZkException.create(e));
+      }
+    });
+
+    opMap.put(org.apache.helix.metaclient.api.Op.Type.DELETE,
+        op -> Op.delete(op.getPath(), ((org.apache.helix.metaclient.api.Op.Delete) op).getVersion()));
+
+    opMap.put(org.apache.helix.metaclient.api.Op.Type.SET,
+        op -> Op.setData(op.getPath(),
+            ((org.apache.helix.metaclient.api.Op.Set) op).getData(), ((org.apache.helix.metaclient.api.Op.Set) op).getVersion()));
+
+    opMap.put(org.apache.helix.metaclient.api.Op.Type.CHECK,
+        op -> Op.check(op.getPath(), ((org.apache.helix.metaclient.api.Op.Check) op).getVersion()));
+
+    List<Op> zkOps = new ArrayList<>();
+    for (org.apache.helix.metaclient.api.Op op : ops) {
+      Op temp = opMap.get(op.getType()).apply(op);
+      zkOps.add(temp);
+    }
+    return zkOps;
+  }
+
+  private static CreateMode convertMetaClientMode(MetaClientInterface.EntryMode entryMode) throws KeeperException {
+    String mode = entryMode.name();
+    if (mode.equals(MetaClientInterface.EntryMode.PERSISTENT.name())) {
+      return CreateMode.fromFlag(0);
+    }
+    if (mode.equals(MetaClientInterface.EntryMode.EPHEMERAL.name())) {
+      return CreateMode.fromFlag(1);
+    }
+    return CreateMode.fromFlag(-1);
+  }
+
+  /**
+   * Helper function for transactionOP. Converts the result from calling zk transactional support into
+   * metaclient OpResults.
+   * @param zkResult
+   * @return
+   */
+  public static List<OpResult> zkOpResultToMetaClient(List<org.apache.zookeeper.OpResult> zkResult) {
+    Map<Class<? extends org.apache.zookeeper.OpResult>, Function<org.apache.zookeeper.OpResult, OpResult>> map = new HashMap<>();
+
+    map.put(org.apache.zookeeper.OpResult.CreateResult.class, opResult -> {
+      org.apache.zookeeper.OpResult.CreateResult zkOpCreateResult = (org.apache.zookeeper.OpResult.CreateResult) opResult;
+      if (opResult.getType() == 1) {
+        return new OpResult.CreateResult(zkOpCreateResult.getPath());
+      } else {
+        MetaClientInterface.Stat metaClientStat = new MetaClientInterface.Stat(convertZkEntryMode(zkOpCreateResult.getStat().getEphemeralOwner()),
+            zkOpCreateResult.getStat().getVersion());
+        return new OpResult.CreateResult(zkOpCreateResult.getPath(), metaClientStat);
+      }});
+
+    map.put(org.apache.zookeeper.OpResult.DeleteResult.class, opResult -> new OpResult.DeleteResult());
+
+    map.put(org.apache.zookeeper.OpResult.GetDataResult.class, opResult -> {
+      org.apache.zookeeper.OpResult.GetDataResult zkOpGetDataResult = (org.apache.zookeeper.OpResult.GetDataResult) opResult;
+      MetaClientInterface.Stat metaClientStat = new MetaClientInterface.Stat(convertZkEntryMode(zkOpGetDataResult.getStat().getEphemeralOwner()),
+          zkOpGetDataResult.getStat().getVersion());
+      return new OpResult.GetDataResult(zkOpGetDataResult.getData(), metaClientStat);
+    });
+
+    map.put(org.apache.zookeeper.OpResult.SetDataResult.class, opResult -> {
+      org.apache.zookeeper.OpResult.SetDataResult zkOpSetDataResult = (org.apache.zookeeper.OpResult.SetDataResult) opResult;
+      MetaClientInterface.Stat metaClientStat = new MetaClientInterface.Stat(convertZkEntryMode(zkOpSetDataResult.getStat().getEphemeralOwner()),
+          zkOpSetDataResult.getStat().getVersion());
+      return new OpResult.SetDataResult(metaClientStat);
+    });
+
+    map.put(org.apache.zookeeper.OpResult.GetChildrenResult.class, opResult -> new OpResult.GetChildrenResult(
+        ((org.apache.zookeeper.OpResult.GetChildrenResult) opResult).getChildren()));
+
+    map.put(org.apache.zookeeper.OpResult.CheckResult.class, opResult -> new OpResult.CheckResult());
+
+    map.put(org.apache.zookeeper.OpResult.ErrorResult.class, opResult -> new OpResult.ErrorResult(
+        ((org.apache.zookeeper.OpResult.ErrorResult) opResult).getErr()));
+
+    List<OpResult> metaClientOpResult = new ArrayList<>();
+    for (org.apache.zookeeper.OpResult opResult : zkResult) {
+      metaClientOpResult.add(map.get(opResult.getClass()).apply(opResult));
+    }
+
+    return metaClientOpResult;
+  }
+
+  public static MetaClientInterface.EntryMode convertZkEntryMode(long ephemeralOwner) {

Review Comment:
   This method is refactored from ZkMetaClient



##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/util/ZkMetaClientUtil.java:
##########
@@ -0,0 +1,156 @@
+package org.apache.helix.metaclient.impl.zk.util;
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.api.OpResult;
+import org.apache.helix.metaclient.constants.*;
+import org.apache.helix.zookeeper.zkclient.exception.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.EphemeralType;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.function.Function;
+
+public class ZkMetaClientUtil {
+  //Default ACL value until metaClient Op has ACL of its own.
+  private static final List<ACL> DEFAULT_ACL = Collections.unmodifiableList(ZooDefs.Ids.OPEN_ACL_UNSAFE);
+
+  private ZkMetaClientUtil(){
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public static List<Op> metaClientOpToZk(Iterable<org.apache.helix.metaclient.api.Op> ops) {
+    Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> opMap = new HashMap<>();
+
+    opMap.put(org.apache.helix.metaclient.api.Op.Type.CREATE, op -> {
+      try {
+        CreateMode mode = convertMetaClientMode(((org.apache.helix.metaclient.api.Op.Create) op).getEntryMode());
+        return Op.create(op.getPath(), ((org.apache.helix.metaclient.api.Op.Create) op).getData(), DEFAULT_ACL, mode);
+      } catch (KeeperException e) {
+        throw translateZkExceptionToMetaclientException(ZkException.create(e));
+      }
+    });
+
+    opMap.put(org.apache.helix.metaclient.api.Op.Type.DELETE,
+        op -> Op.delete(op.getPath(), ((org.apache.helix.metaclient.api.Op.Delete) op).getVersion()));
+
+    opMap.put(org.apache.helix.metaclient.api.Op.Type.SET,
+        op -> Op.setData(op.getPath(),
+            ((org.apache.helix.metaclient.api.Op.Set) op).getData(), ((org.apache.helix.metaclient.api.Op.Set) op).getVersion()));
+
+    opMap.put(org.apache.helix.metaclient.api.Op.Type.CHECK,
+        op -> Op.check(op.getPath(), ((org.apache.helix.metaclient.api.Op.Check) op).getVersion()));
+
+    List<Op> zkOps = new ArrayList<>();
+    for (org.apache.helix.metaclient.api.Op op : ops) {
+      Op temp = opMap.get(op.getType()).apply(op);
+      zkOps.add(temp);
+    }
+    return zkOps;
+  }
+
+  private static CreateMode convertMetaClientMode(MetaClientInterface.EntryMode entryMode) throws KeeperException {
+    String mode = entryMode.name();
+    if (mode.equals(MetaClientInterface.EntryMode.PERSISTENT.name())) {
+      return CreateMode.fromFlag(0);
+    }
+    if (mode.equals(MetaClientInterface.EntryMode.EPHEMERAL.name())) {
+      return CreateMode.fromFlag(1);
+    }
+    return CreateMode.fromFlag(-1);
+  }
+
+  /**
+   * Helper function for transactionOP. Converts the result from calling zk transactional support into
+   * metaclient OpResults.
+   * @param zkResult
+   * @return
+   */
+  public static List<OpResult> zkOpResultToMetaClient(List<org.apache.zookeeper.OpResult> zkResult) {
+    Map<Class<? extends org.apache.zookeeper.OpResult>, Function<org.apache.zookeeper.OpResult, OpResult>> map = new HashMap<>();
+
+    map.put(org.apache.zookeeper.OpResult.CreateResult.class, opResult -> {
+      org.apache.zookeeper.OpResult.CreateResult zkOpCreateResult = (org.apache.zookeeper.OpResult.CreateResult) opResult;
+      if (opResult.getType() == 1) {
+        return new OpResult.CreateResult(zkOpCreateResult.getPath());
+      } else {
+        MetaClientInterface.Stat metaClientStat = new MetaClientInterface.Stat(convertZkEntryMode(zkOpCreateResult.getStat().getEphemeralOwner()),
+            zkOpCreateResult.getStat().getVersion());
+        return new OpResult.CreateResult(zkOpCreateResult.getPath(), metaClientStat);
+      }});
+
+    map.put(org.apache.zookeeper.OpResult.DeleteResult.class, opResult -> new OpResult.DeleteResult());
+
+    map.put(org.apache.zookeeper.OpResult.GetDataResult.class, opResult -> {
+      org.apache.zookeeper.OpResult.GetDataResult zkOpGetDataResult = (org.apache.zookeeper.OpResult.GetDataResult) opResult;
+      MetaClientInterface.Stat metaClientStat = new MetaClientInterface.Stat(convertZkEntryMode(zkOpGetDataResult.getStat().getEphemeralOwner()),
+          zkOpGetDataResult.getStat().getVersion());
+      return new OpResult.GetDataResult(zkOpGetDataResult.getData(), metaClientStat);
+    });
+
+    map.put(org.apache.zookeeper.OpResult.SetDataResult.class, opResult -> {
+      org.apache.zookeeper.OpResult.SetDataResult zkOpSetDataResult = (org.apache.zookeeper.OpResult.SetDataResult) opResult;
+      MetaClientInterface.Stat metaClientStat = new MetaClientInterface.Stat(convertZkEntryMode(zkOpSetDataResult.getStat().getEphemeralOwner()),
+          zkOpSetDataResult.getStat().getVersion());
+      return new OpResult.SetDataResult(metaClientStat);
+    });
+
+    map.put(org.apache.zookeeper.OpResult.GetChildrenResult.class, opResult -> new OpResult.GetChildrenResult(
+        ((org.apache.zookeeper.OpResult.GetChildrenResult) opResult).getChildren()));
+
+    map.put(org.apache.zookeeper.OpResult.CheckResult.class, opResult -> new OpResult.CheckResult());
+
+    map.put(org.apache.zookeeper.OpResult.ErrorResult.class, opResult -> new OpResult.ErrorResult(
+        ((org.apache.zookeeper.OpResult.ErrorResult) opResult).getErr()));
+
+    List<OpResult> metaClientOpResult = new ArrayList<>();
+    for (org.apache.zookeeper.OpResult opResult : zkResult) {
+      metaClientOpResult.add(map.get(opResult.getClass()).apply(opResult));
+    }
+
+    return metaClientOpResult;
+  }
+
+  public static MetaClientInterface.EntryMode convertZkEntryMode(long ephemeralOwner) {
+    EphemeralType zkEphemeralType = EphemeralType.get(ephemeralOwner);
+    switch (zkEphemeralType) {
+      case VOID:
+        return MetaClientInterface.EntryMode.PERSISTENT;
+      case CONTAINER:
+        return MetaClientInterface.EntryMode.CONTAINER;
+      case NORMAL:
+        return MetaClientInterface.EntryMode.EPHEMERAL;
+      // TODO: TTL is not supported now.
+      //case TTL:
+      //  return EntryMode.TTL;
+      default:
+        throw new IllegalArgumentException(zkEphemeralType + " is not supported.");
+    }
+  }
+
+  public static MetaClientException translateZkExceptionToMetaclientException(ZkException e) {

Review Comment:
   Same as above



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] Marcosrico commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

Posted by GitBox <gi...@apache.org>.
Marcosrico commented on code in PR #2343:
URL: https://github.com/apache/helix/pull/2343#discussion_r1081995237


##########
meta-client/src/main/java/org/apache/helix/metaclient/constants/MetaClientKeeperException.java:
##########
@@ -0,0 +1,39 @@
+package org.apache.helix.metaclient.constants;

Review Comment:
   As discussed asynchronously, I will create a separate PR with these changes after these are merged



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] qqu0127 commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

Posted by GitBox <gi...@apache.org>.
qqu0127 commented on code in PR #2343:
URL: https://github.com/apache/helix/pull/2343#discussion_r1081649362


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/ZkMetaClient.java:
##########
@@ -414,4 +415,128 @@ private static EntryMode convertZkEntryMode(long ephemeralOwner) {
         throw new IllegalArgumentException(zkEphemeralType + " is not supported.");
     }
   }
+
+
+  @Override
+  public List<OpResult> transactionOP(Iterable<Op> iterable) {
+    // Convert list of MetaClient Ops to Zk Ops
+    List<org.apache.zookeeper.Op> zkOps = metaClientOpToZk(iterable);
+    // Execute Zk transactional support
+    List<org.apache.zookeeper.OpResult> zkResult = _zkClient.multi(zkOps);
+    // Convert list of Zk OpResults to MetaClient OpResults
+    return zkOpResultToMetaClient(zkResult);
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public List<org.apache.zookeeper.Op> metaClientOpToZk(Iterable<Op> ops) {
+    List<org.apache.zookeeper.Op> zkOps = new ArrayList<>();
+    org.apache.zookeeper.Op temp;
+    for (Op op : ops) {
+      switch (op.getType()) {
+        case CREATE:
+          int zkFlag = zkFlagFromEntryMode(((Op.Create) op).getEntryMode());
+          try {
+            temp = org.apache.zookeeper.Op.create(
+                op.getPath(), ((Op.Create) op).getData(), DEFAULT_ACL, CreateMode.fromFlag(zkFlag));
+          } catch (KeeperException e) {
+            throw new MetaClientKeeperException(e);
+          }
+          break;
+        case DELETE:
+          temp = org.apache.zookeeper.Op.delete(
+              op.getPath(), ((Op.Delete) op).getVersion());
+          break;
+        case SET:
+          temp = org.apache.zookeeper.Op.setData(
+              op.getPath(), ((Op.Set) op).getData(), ((Op.Set) op).getVersion());
+          break;
+        case CHECK:
+          temp = org.apache.zookeeper.Op.check(
+              op.getPath(), ((Op.Check) op).getVersion());
+          break;
+        default:
+          throw new IllegalArgumentException(op.getType() + " is not supported.");
+      }
+      zkOps.add(temp);
+    }
+    return zkOps;
+  }
+
+  /**
+   * Helper function for transactionOP. Converts the result from calling zk transactional support into
+   * metaclient OpResults.
+   * @param zkResult
+   * @return
+   */
+  public List<OpResult> zkOpResultToMetaClient(List<org.apache.zookeeper.OpResult> zkResult) {

Review Comment:
   Yes, with a util class, you can also create a util package under impl/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.

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

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


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


[GitHub] [helix] xyuanlu commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

Posted by GitBox <gi...@apache.org>.
xyuanlu commented on code in PR #2343:
URL: https://github.com/apache/helix/pull/2343#discussion_r1081668733


##########
meta-client/src/main/java/org/apache/helix/metaclient/constants/MetaClientKeeperException.java:
##########
@@ -0,0 +1,39 @@
+package org.apache.helix.metaclient.constants;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+public final class MetaClientKeeperException extends MetaClientException {

Review Comment:
   I think we probably want more specific exception than just keeper exception. 
   Helix defined  ZkException translate KeeperException to ZkException, and metaclient translates ZkException to metaclient defined exception. 
   you can do 
   ```
      try {
        //........ 
       } catch (KeeperException e) {
         throw translateZkExceptionToMetaclientException(ZkException.create(e));
       }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] desaikomal commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

Posted by GitBox <gi...@apache.org>.
desaikomal commented on code in PR #2343:
URL: https://github.com/apache/helix/pull/2343#discussion_r1081522935


##########
meta-client/src/main/java/org/apache/helix/metaclient/constants/MetaClientKeeperException.java:
##########
@@ -0,0 +1,39 @@
+package org.apache.helix.metaclient.constants;

Review Comment:
   you should have a separate directory for exception. 
   
   eg:
   ls <HELIX-home>/metadata-store-directory-common/src/main/java/org/apache/helix/msdcommon/exception
   InvalidRoutingDataException.java
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] Marcosrico commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

Posted by GitBox <gi...@apache.org>.
Marcosrico commented on code in PR #2343:
URL: https://github.com/apache/helix/pull/2343#discussion_r1081380987


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/ZkMetaClient.java:
##########
@@ -414,4 +414,130 @@ private static EntryMode convertZkEntryMode(long ephemeralOwner) {
         throw new IllegalArgumentException(zkEphemeralType + " is not supported.");
     }
   }
+
+
+  @Override
+  public List<OpResult> transactionOP(Iterable<Op> iterable) throws KeeperException {
+    // Convert list of MetaClient Ops to Zk Ops
+    List<org.apache.zookeeper.Op> zkOps = metaClientOpToZk(iterable);
+    // Execute Zk transactional support
+    List<org.apache.zookeeper.OpResult> zkResult = _zkClient.multi(zkOps);
+    // Convert list of Zk OpResults to MetaClient OpResults
+    return zkOpResultToMetaClient(zkResult);
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public List<org.apache.zookeeper.Op> metaClientOpToZk(Iterable<Op> ops) throws KeeperException {

Review Comment:
   The `CreateMode.fromFlag()`  method throws KeeperException. I can catch it in the method, however, I don't believe it would fall under IllegalArgument exception and would need to throw a different one. This would show in the method signature. Referring to a comment below, I'll create a specific MetaClientKeeperException to be thrown for these cases.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] Marcosrico commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/util/ZkMetaClientUtil.java:
##########
@@ -0,0 +1,208 @@
+package org.apache.helix.metaclient.impl.zk.util;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.api.OpResult;
+import org.apache.helix.metaclient.constants.*;
+import org.apache.helix.zookeeper.zkclient.exception.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.EphemeralType;
+
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.function.Function;
+
+public class ZkMetaClientUtil {
+  //TODO Implement MetaClient ACL
+  //Default ACL value until metaClient Op has ACL of its own.
+  private static final List<ACL> DEFAULT_ACL = Collections.unmodifiableList(ZooDefs.Ids.OPEN_ACL_UNSAFE);
+
+  private ZkMetaClientUtil(){
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public static List<Op> metaClientOpsToZkOps(Iterable<org.apache.helix.metaclient.api.Op> ops) {
+    initializeOpMap();
+    List<Op> zkOps = new ArrayList<>();
+    for (org.apache.helix.metaclient.api.Op op : ops) {
+      Function<org.apache.helix.metaclient.api.Op, Op> function = getOpMap().get(op.getType());
+      if (function != null) {
+        zkOps.add(function.apply(op));
+      } else {
+        throw new IllegalArgumentException("Op type " + op.getType().name() + " is not supported.");
+      }
+    }
+    return zkOps;
+  }
+
+  private static final class OpMapHolder {
+    static final Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> OPMAP =
+        new EnumMap<>(org.apache.helix.metaclient.api.Op.Type.class);
+  }
+
+  private static Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> getOpMap() {

Review Comment:
   Very true, calling OPMAP directly would still work for lazy instantiation. However, the get method provides a layer of abstraction in case the implementation of OPMAP changes, which in my opinion outweighs the minor overhead cost of calling the get method.
   
   And you bring up a good point, I was outweighing the costs of implementing lazy instantiation to begin with and having the map be eagerly instantiated in the beginning of ZkMetaClientUtil class. Considering that this class is a util class for anything zkmetaclient related, it can be used in the future for other actions not related to transactional support. While there isn't much else going on, by providing lazy instantiation we can minimize overhead in the long run if ZkMetaClientUtil has increased use cases and/or each map gets larger. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] Marcosrico commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/util/ZkMetaClientUtil.java:
##########
@@ -0,0 +1,201 @@
+package org.apache.helix.metaclient.impl.zk.util;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.api.OpResult;
+import org.apache.helix.metaclient.constants.*;
+import org.apache.helix.zookeeper.zkclient.exception.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.EphemeralType;
+
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.function.Function;
+
+public class ZkMetaClientUtil {
+  //Default ACL value until metaClient Op has ACL of its own.
+  private static final List<ACL> DEFAULT_ACL = Collections.unmodifiableList(ZooDefs.Ids.OPEN_ACL_UNSAFE);
+
+  private ZkMetaClientUtil(){
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public static List<Op> metaClientOpToZk(Iterable<org.apache.helix.metaclient.api.Op> ops) {
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.CREATE, op -> {
+      try {
+        CreateMode mode = convertMetaClientMode(((org.apache.helix.metaclient.api.Op.Create) op).getEntryMode());
+        return Op.create(op.getPath(), ((org.apache.helix.metaclient.api.Op.Create) op).getData(), DEFAULT_ACL, mode);
+      } catch (KeeperException e) {
+        throw translateZkExceptionToMetaclientException(ZkException.create(e));
+      }
+    });
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.DELETE,
+        op -> Op.delete(op.getPath(), ((org.apache.helix.metaclient.api.Op.Delete) op).getVersion()));
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.SET,
+        op -> Op.setData(op.getPath(),
+            ((org.apache.helix.metaclient.api.Op.Set) op).getData(), ((org.apache.helix.metaclient.api.Op.Set) op).getVersion()));
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.CHECK,
+        op -> Op.check(op.getPath(), ((org.apache.helix.metaclient.api.Op.Check) op).getVersion()));
+
+    List<Op> zkOps = new ArrayList<>();
+    for (org.apache.helix.metaclient.api.Op op : ops) {
+      Function<org.apache.helix.metaclient.api.Op, Op> function = getOpMap().get(op.getType());
+      if (function != null) {
+        zkOps.add(function.apply(op));
+      } else {
+        throw new IllegalArgumentException("Op type " + op.getType().name() + "is not supported.");
+      }
+    }
+    return zkOps;
+  }
+
+  private static final class OpmapHolder {
+    static final Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> OPMAP =
+        new EnumMap<>(org.apache.helix.metaclient.api.Op.Type.class);
+  }
+
+  private static Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> getOpMap() {
+    return OpmapHolder.OPMAP;
+  }
+
+  private static CreateMode convertMetaClientMode(MetaClientInterface.EntryMode entryMode) throws KeeperException {
+    if (entryMode == MetaClientInterface.EntryMode.PERSISTENT) {
+      return CreateMode.fromFlag(0);

Review Comment:
   Yes that makes more sense as the flag parameter provides no value. Thanks for the suggestions!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] qqu0127 commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/util/ZkMetaClientUtil.java:
##########
@@ -0,0 +1,201 @@
+package org.apache.helix.metaclient.impl.zk.util;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.api.OpResult;
+import org.apache.helix.metaclient.constants.*;
+import org.apache.helix.zookeeper.zkclient.exception.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.EphemeralType;
+
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.function.Function;
+
+public class ZkMetaClientUtil {
+  //Default ACL value until metaClient Op has ACL of its own.
+  private static final List<ACL> DEFAULT_ACL = Collections.unmodifiableList(ZooDefs.Ids.OPEN_ACL_UNSAFE);
+
+  private ZkMetaClientUtil(){
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public static List<Op> metaClientOpToZk(Iterable<org.apache.helix.metaclient.api.Op> ops) {
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.CREATE, op -> {
+      try {
+        CreateMode mode = convertMetaClientMode(((org.apache.helix.metaclient.api.Op.Create) op).getEntryMode());
+        return Op.create(op.getPath(), ((org.apache.helix.metaclient.api.Op.Create) op).getData(), DEFAULT_ACL, mode);
+      } catch (KeeperException e) {
+        throw translateZkExceptionToMetaclientException(ZkException.create(e));
+      }
+    });
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.DELETE,
+        op -> Op.delete(op.getPath(), ((org.apache.helix.metaclient.api.Op.Delete) op).getVersion()));
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.SET,
+        op -> Op.setData(op.getPath(),
+            ((org.apache.helix.metaclient.api.Op.Set) op).getData(), ((org.apache.helix.metaclient.api.Op.Set) op).getVersion()));
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.CHECK,
+        op -> Op.check(op.getPath(), ((org.apache.helix.metaclient.api.Op.Check) op).getVersion()));
+
+    List<Op> zkOps = new ArrayList<>();
+    for (org.apache.helix.metaclient.api.Op op : ops) {
+      Function<org.apache.helix.metaclient.api.Op, Op> function = getOpMap().get(op.getType());
+      if (function != null) {
+        zkOps.add(function.apply(op));
+      } else {
+        throw new IllegalArgumentException("Op type " + op.getType().name() + "is not supported.");
+      }
+    }
+    return zkOps;
+  }
+
+  private static final class OpmapHolder {
+    static final Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> OPMAP =
+        new EnumMap<>(org.apache.helix.metaclient.api.Op.Type.class);
+  }
+
+  private static Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> getOpMap() {
+    return OpmapHolder.OPMAP;
+  }
+
+  private static CreateMode convertMetaClientMode(MetaClientInterface.EntryMode entryMode) throws KeeperException {
+    if (entryMode == MetaClientInterface.EntryMode.PERSISTENT) {
+      return CreateMode.fromFlag(0);
+    }
+    if (entryMode == MetaClientInterface.EntryMode.EPHEMERAL) {
+      return CreateMode.fromFlag(1);
+    }
+    if (entryMode == MetaClientInterface.EntryMode.CONTAINER) {
+      return CreateMode.fromFlag(4);
+    }
+    throw new IllegalArgumentException(entryMode.name() + " is not a supported EntryMode.");
+  }
+
+  /**
+   * Helper function for transactionOP. Converts the result from calling zk transactional support into
+   * metaclient OpResults.
+   * @param zkResult
+   * @return
+   */
+  public static List<OpResult> zkOpResultToMetaClient(List<org.apache.zookeeper.OpResult> zkResult) {
+    getOpResultMap().put(org.apache.zookeeper.OpResult.CreateResult.class, opResult -> {

Review Comment:
   This doesn't look like lazy loading. Each time the method invoked, we still go through all map.put() process.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] Marcosrico commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

Posted by GitBox <gi...@apache.org>.
Marcosrico commented on code in PR #2343:
URL: https://github.com/apache/helix/pull/2343#discussion_r1081380987


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/ZkMetaClient.java:
##########
@@ -414,4 +414,130 @@ private static EntryMode convertZkEntryMode(long ephemeralOwner) {
         throw new IllegalArgumentException(zkEphemeralType + " is not supported.");
     }
   }
+
+
+  @Override
+  public List<OpResult> transactionOP(Iterable<Op> iterable) throws KeeperException {
+    // Convert list of MetaClient Ops to Zk Ops
+    List<org.apache.zookeeper.Op> zkOps = metaClientOpToZk(iterable);
+    // Execute Zk transactional support
+    List<org.apache.zookeeper.OpResult> zkResult = _zkClient.multi(zkOps);
+    // Convert list of Zk OpResults to MetaClient OpResults
+    return zkOpResultToMetaClient(zkResult);
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public List<org.apache.zookeeper.Op> metaClientOpToZk(Iterable<Op> ops) throws KeeperException {

Review Comment:
   The `CreateMode.fromFlag()`  method throws KeeperException. I can catch it in the method, however, I don't believe it would fall under IllegalArgument exception and would need to throw a different one. This would show in the method signature so I left it at this. Please let me know what the best practices are for this, thanks!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] Marcosrico commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

Posted by GitBox <gi...@apache.org>.
Marcosrico commented on code in PR #2343:
URL: https://github.com/apache/helix/pull/2343#discussion_r1082753521


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/util/ZkMetaClientUtil.java:
##########
@@ -0,0 +1,156 @@
+package org.apache.helix.metaclient.impl.zk.util;
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.api.OpResult;
+import org.apache.helix.metaclient.constants.*;
+import org.apache.helix.zookeeper.zkclient.exception.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.EphemeralType;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.function.Function;
+
+public class ZkMetaClientUtil {
+  //Default ACL value until metaClient Op has ACL of its own.
+  private static final List<ACL> DEFAULT_ACL = Collections.unmodifiableList(ZooDefs.Ids.OPEN_ACL_UNSAFE);
+
+  private ZkMetaClientUtil(){
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public static List<Op> metaClientOpToZk(Iterable<org.apache.helix.metaclient.api.Op> ops) {
+    Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> opMap = new HashMap<>();
+
+    opMap.put(org.apache.helix.metaclient.api.Op.Type.CREATE, op -> {
+      try {
+        CreateMode mode = convertMetaClientMode(((org.apache.helix.metaclient.api.Op.Create) op).getEntryMode());
+        return Op.create(op.getPath(), ((org.apache.helix.metaclient.api.Op.Create) op).getData(), DEFAULT_ACL, mode);
+      } catch (KeeperException e) {
+        throw translateZkExceptionToMetaclientException(ZkException.create(e));
+      }
+    });
+
+    opMap.put(org.apache.helix.metaclient.api.Op.Type.DELETE,
+        op -> Op.delete(op.getPath(), ((org.apache.helix.metaclient.api.Op.Delete) op).getVersion()));
+
+    opMap.put(org.apache.helix.metaclient.api.Op.Type.SET,
+        op -> Op.setData(op.getPath(),
+            ((org.apache.helix.metaclient.api.Op.Set) op).getData(), ((org.apache.helix.metaclient.api.Op.Set) op).getVersion()));
+
+    opMap.put(org.apache.helix.metaclient.api.Op.Type.CHECK,
+        op -> Op.check(op.getPath(), ((org.apache.helix.metaclient.api.Op.Check) op).getVersion()));
+
+    List<Op> zkOps = new ArrayList<>();
+    for (org.apache.helix.metaclient.api.Op op : ops) {
+      Op temp = opMap.get(op.getType()).apply(op);
+      zkOps.add(temp);
+    }
+    return zkOps;
+  }
+
+  private static CreateMode convertMetaClientMode(MetaClientInterface.EntryMode entryMode) throws KeeperException {
+    String mode = entryMode.name();
+    if (mode.equals(MetaClientInterface.EntryMode.PERSISTENT.name())) {
+      return CreateMode.fromFlag(0);
+    }
+    if (mode.equals(MetaClientInterface.EntryMode.EPHEMERAL.name())) {
+      return CreateMode.fromFlag(1);
+    }

Review Comment:
   Thanks for the review, will fix the comparison.
   
   CreateMode.fromFlag() throws KeeperException which is why this method throws the exception. I catch it in metaClientOpToZk that calls this method.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] Marcosrico commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/util/ZkMetaClientUtil.java:
##########
@@ -0,0 +1,201 @@
+package org.apache.helix.metaclient.impl.zk.util;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.api.OpResult;
+import org.apache.helix.metaclient.constants.*;
+import org.apache.helix.zookeeper.zkclient.exception.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.EphemeralType;
+
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.function.Function;
+
+public class ZkMetaClientUtil {
+  //Default ACL value until metaClient Op has ACL of its own.
+  private static final List<ACL> DEFAULT_ACL = Collections.unmodifiableList(ZooDefs.Ids.OPEN_ACL_UNSAFE);
+
+  private ZkMetaClientUtil(){
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public static List<Op> metaClientOpToZk(Iterable<org.apache.helix.metaclient.api.Op> ops) {

Review Comment:
   Correct, that would make more sense!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] Marcosrico commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/util/ZkMetaClientUtil.java:
##########
@@ -0,0 +1,201 @@
+package org.apache.helix.metaclient.impl.zk.util;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.api.OpResult;
+import org.apache.helix.metaclient.constants.*;
+import org.apache.helix.zookeeper.zkclient.exception.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.EphemeralType;
+
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.function.Function;
+
+public class ZkMetaClientUtil {
+  //Default ACL value until metaClient Op has ACL of its own.
+  private static final List<ACL> DEFAULT_ACL = Collections.unmodifiableList(ZooDefs.Ids.OPEN_ACL_UNSAFE);
+
+  private ZkMetaClientUtil(){
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public static List<Op> metaClientOpToZk(Iterable<org.apache.helix.metaclient.api.Op> ops) {
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.CREATE, op -> {
+      try {
+        CreateMode mode = convertMetaClientMode(((org.apache.helix.metaclient.api.Op.Create) op).getEntryMode());
+        return Op.create(op.getPath(), ((org.apache.helix.metaclient.api.Op.Create) op).getData(), DEFAULT_ACL, mode);
+      } catch (KeeperException e) {
+        throw translateZkExceptionToMetaclientException(ZkException.create(e));
+      }
+    });
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.DELETE,
+        op -> Op.delete(op.getPath(), ((org.apache.helix.metaclient.api.Op.Delete) op).getVersion()));
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.SET,
+        op -> Op.setData(op.getPath(),
+            ((org.apache.helix.metaclient.api.Op.Set) op).getData(), ((org.apache.helix.metaclient.api.Op.Set) op).getVersion()));
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.CHECK,
+        op -> Op.check(op.getPath(), ((org.apache.helix.metaclient.api.Op.Check) op).getVersion()));
+
+    List<Op> zkOps = new ArrayList<>();
+    for (org.apache.helix.metaclient.api.Op op : ops) {
+      Function<org.apache.helix.metaclient.api.Op, Op> function = getOpMap().get(op.getType());
+      if (function != null) {
+        zkOps.add(function.apply(op));
+      } else {
+        throw new IllegalArgumentException("Op type " + op.getType().name() + "is not supported.");
+      }
+    }
+    return zkOps;
+  }
+
+  private static final class OpmapHolder {
+    static final Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> OPMAP =
+        new EnumMap<>(org.apache.helix.metaclient.api.Op.Type.class);
+  }
+
+  private static Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> getOpMap() {
+    return OpmapHolder.OPMAP;
+  }
+
+  private static CreateMode convertMetaClientMode(MetaClientInterface.EntryMode entryMode) throws KeeperException {
+    if (entryMode == MetaClientInterface.EntryMode.PERSISTENT) {

Review Comment:
   Yup it definitely would, thanks for the suggestion!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] qqu0127 commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/util/ZkMetaClientUtil.java:
##########
@@ -0,0 +1,213 @@
+package org.apache.helix.metaclient.impl.zk.util;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.api.OpResult;
+import org.apache.helix.metaclient.constants.*;
+import org.apache.helix.zookeeper.zkclient.exception.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.EphemeralType;
+
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.function.Function;
+
+public class ZkMetaClientUtil {
+  //TODO Implement MetaClient ACL
+  //Default ACL value until metaClient Op has ACL of its own.
+  private static final List<ACL> DEFAULT_ACL = Collections.unmodifiableList(ZooDefs.Ids.OPEN_ACL_UNSAFE);
+
+  private ZkMetaClientUtil(){
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public static List<Op> metaClientOpsToZkOps(Iterable<org.apache.helix.metaclient.api.Op> ops) {
+    if (getOpMap().isEmpty()) {
+      initializeOpMap();
+    }
+
+    List<Op> zkOps = new ArrayList<>();
+    for (org.apache.helix.metaclient.api.Op op : ops) {
+      Function<org.apache.helix.metaclient.api.Op, Op> function = getOpMap().get(op.getType());
+      if (function != null) {
+        zkOps.add(function.apply(op));
+      } else {
+        throw new IllegalArgumentException("Op type " + op.getType().name() + " is not supported.");
+      }
+    }
+    return zkOps;
+  }
+
+  private static final class OpMapHolder {
+    static final Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> OPMAP =
+        new EnumMap<>(org.apache.helix.metaclient.api.Op.Type.class);
+  }
+
+  private static Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> getOpMap() {
+    return OpMapHolder.OPMAP;
+  }
+
+  private static void initializeOpMap() {
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.CREATE, op -> {
+      try {
+        CreateMode mode = convertMetaClientMode(((org.apache.helix.metaclient.api.Op.Create) op).getEntryMode());
+        return Op.create(op.getPath(), ((org.apache.helix.metaclient.api.Op.Create) op).getData(), DEFAULT_ACL, mode);
+      } catch (KeeperException e) {
+        throw translateZkExceptionToMetaclientException(ZkException.create(e));
+      }
+    });
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.DELETE,
+        op -> Op.delete(op.getPath(), ((org.apache.helix.metaclient.api.Op.Delete) op).getVersion()));
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.SET,
+        op -> Op.setData(op.getPath(),
+            ((org.apache.helix.metaclient.api.Op.Set) op).getData(), ((org.apache.helix.metaclient.api.Op.Set) op).getVersion()));
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.CHECK,
+        op -> Op.check(op.getPath(), ((org.apache.helix.metaclient.api.Op.Check) op).getVersion()));
+  }
+
+  private static CreateMode convertMetaClientMode(MetaClientInterface.EntryMode entryMode) throws KeeperException {
+    switch (entryMode) {
+      case PERSISTENT:
+        return CreateMode.PERSISTENT;
+      case EPHEMERAL:
+        return CreateMode.EPHEMERAL;
+      case CONTAINER:
+        return CreateMode.CONTAINER;
+      default:
+        throw new IllegalArgumentException(entryMode.name() + " is not a supported EntryMode.");
+    }
+  }
+
+  /**
+   * Helper function for transactionOP. Converts the result from calling zk transactional support into
+   * metaclient OpResults.
+   * @param zkResult
+   * @return
+   */
+  public static List<OpResult> zkOpResultToMetaClientOpResults(List<org.apache.zookeeper.OpResult> zkResult) {
+    if (getOpResultMap().isEmpty()) {
+      initializeOpResultMap();
+    }
+    List<OpResult> metaClientOpResult = new ArrayList<>();
+    for (org.apache.zookeeper.OpResult opResult : zkResult) {
+      Function<org.apache.zookeeper.OpResult, OpResult> function = getOpResultMap().get(opResult.getClass());
+      if (function != null) {
+        metaClientOpResult.add(function.apply(opResult));
+      } else {
+        throw new IllegalArgumentException("OpResult type " + opResult.getType() + "is not supported.");
+      }
+    }
+
+    return metaClientOpResult;
+  }
+
+  private static final class OpResultMapHolder {
+    static final Map<Class<? extends org.apache.zookeeper.OpResult>, Function<org.apache.zookeeper.OpResult, OpResult>> OPRESULTMAP =
+        new HashMap<>();
+  }

Review Comment:
   This is incorrect. In your impl, `initializeOpResultMap` is called every time `zkOpResultToMetaClientOpResults` is called, which trace back to `ZkMetaClient.transactionOP`. 
   I don't get why point 2 is not applicable. To achieve lazy loading, you can combine the "initialize" and "value population" in your static method `initializeOpResultMap` by changing the signature and content to build and return the map, so you can call it directly at declaration. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] qqu0127 commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

Posted by GitBox <gi...@apache.org>.
qqu0127 commented on code in PR #2343:
URL: https://github.com/apache/helix/pull/2343#discussion_r1081444524


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/ZkMetaClient.java:
##########
@@ -50,11 +51,15 @@
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.server.EphemeralType;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
 
 
 public class ZkMetaClient<T> implements MetaClientInterface<T>, AutoCloseable {
 
   private final ZkClient _zkClient;
+  //Default ACL value until metaClient Op has ACL of its own.
+  private final List<ACL> DEFAULT_ACL = ZooDefs.Ids.OPEN_ACL_UNSAFE;

Review Comment:
   This can be static? Also we can use ImmutableList as a wrapper around this.



##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/ZkMetaClient.java:
##########
@@ -414,4 +414,130 @@ private static EntryMode convertZkEntryMode(long ephemeralOwner) {
         throw new IllegalArgumentException(zkEphemeralType + " is not supported.");
     }
   }
+
+
+  @Override
+  public List<OpResult> transactionOP(Iterable<Op> iterable) throws KeeperException {
+    // Convert list of MetaClient Ops to Zk Ops
+    List<org.apache.zookeeper.Op> zkOps = metaClientOpToZk(iterable);
+    // Execute Zk transactional support
+    List<org.apache.zookeeper.OpResult> zkResult = _zkClient.multi(zkOps);
+    // Convert list of Zk OpResults to MetaClient OpResults
+    return zkOpResultToMetaClient(zkResult);
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public List<org.apache.zookeeper.Op> metaClientOpToZk(Iterable<Op> ops) throws KeeperException {
+    List<org.apache.zookeeper.Op> zkOps = new ArrayList<>();
+    org.apache.zookeeper.Op temp;
+    for (Op op : ops) {
+      switch (op.getType()) {
+        case CREATE:
+          int zkFlag = zkFlagFromEntryMode(((Op.Create) op).getEntryMode());
+          temp = org.apache.zookeeper.Op.create(
+              op.getPath(), ((Op.Create) op).getData(), DEFAULT_ACL, CreateMode.fromFlag(zkFlag));
+          break;
+        case DELETE:
+          temp = org.apache.zookeeper.Op.delete(
+              op.getPath(), ((Op.Delete) op).getVersion());
+          break;
+        case SET:
+          temp = org.apache.zookeeper.Op.setData(
+              op.getPath(), ((Op.Set) op).getData(), ((Op.Set) op).getVersion());
+          break;
+        case CHECK:
+          temp = org.apache.zookeeper.Op.check(
+              op.getPath(), ((Op.Check) op).getVersion());
+          break;
+        default:
+          throw new IllegalArgumentException(op.getType() + " is not supported.");
+      }
+      zkOps.add(temp);
+    }
+    return zkOps;
+  }
+
+  /**
+   * Helper function for transactionOP. Converts the result from calling zk transactional support into
+   * metaclient OpResults.
+   * @param zkResult
+   * @return
+   */
+  public List<OpResult> zkOpResultToMetaClient(List<org.apache.zookeeper.OpResult> zkResult) throws KeeperException.BadArgumentsException {
+    List<OpResult> metaClientOpResult = new ArrayList<>();
+    OpResult temp;
+    EntryMode zkMode;
+    for (org.apache.zookeeper.OpResult opResult : zkResult) {
+      Stat metaClientStat;
+      switch (opResult.getType()) {
+        // CreateResult
+        case 1:
+          temp = new OpResult.CreateResult(
+                  ((org.apache.zookeeper.OpResult.CreateResult) opResult).getPath());
+          break;
+        // DeleteResult
+        case 2:
+          temp = new OpResult.DeleteResult();
+          break;
+        // GetDataResult
+        case 4:
+          org.apache.zookeeper.OpResult.GetDataResult zkOpGetDataResult =
+                  (org.apache.zookeeper.OpResult.GetDataResult) opResult;
+          metaClientStat = new Stat(convertZkEntryMode(zkOpGetDataResult.getStat().getEphemeralOwner()),
+              zkOpGetDataResult.getStat().getVersion());
+          temp = new OpResult.GetDataResult(zkOpGetDataResult.getData(), metaClientStat);
+          break;
+        //SetDataResult
+        case 5:
+          org.apache.zookeeper.OpResult.SetDataResult zkOpSetDataResult =
+                  (org.apache.zookeeper.OpResult.SetDataResult) opResult;
+          metaClientStat = new Stat(convertZkEntryMode(zkOpSetDataResult.getStat().getEphemeralOwner()),
+              zkOpSetDataResult.getStat().getVersion());
+          temp = new OpResult.SetDataResult(metaClientStat);
+          break;
+        //GetChildrenResult
+        case 8:
+          temp = new OpResult.GetChildrenResult(
+                  ((org.apache.zookeeper.OpResult.GetChildrenResult) opResult).getChildren());
+          break;
+        //CheckResult
+        case 13:
+          temp = new OpResult.CheckResult();
+          break;
+        //CreateResult with stat
+        case 15:
+          org.apache.zookeeper.OpResult.CreateResult zkOpCreateResult =
+                  (org.apache.zookeeper.OpResult.CreateResult) opResult;
+          metaClientStat = new Stat(convertZkEntryMode(zkOpCreateResult.getStat().getEphemeralOwner()),
+              zkOpCreateResult.getStat().getVersion());
+          temp = new OpResult.CreateResult(zkOpCreateResult.getPath(), metaClientStat);
+          break;
+        //ErrorResult
+        case -1:
+          temp = new OpResult.ErrorResult(
+                  ((org.apache.zookeeper.OpResult.ErrorResult) opResult).getErr());
+          break;
+        default:
+          throw new IllegalArgumentException(opResult.getType() + " is not supported.");
+      }
+      metaClientOpResult.add(temp);
+    }
+    return metaClientOpResult;
+  }
+
+  private int zkFlagFromEntryMode (EntryMode entryMode) {
+    String mode = entryMode.name();
+    if (mode.equals(EntryMode.PERSISTENT.name())) {
+      return 0;
+    }
+    if (mode.equals(EntryMode.EPHEMERAL.name())) {
+      return 1;
+    }
+    return -1;
+  }

Review Comment:
   If I understand correctly, in the end, what you are trying to achieve is translate entryMode to ZK.CreateMode. Both of them are enums, let's try to bridge them in one pass, without the "flag" as intermediate param.



##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/ZkMetaClient.java:
##########
@@ -414,4 +415,128 @@ private static EntryMode convertZkEntryMode(long ephemeralOwner) {
         throw new IllegalArgumentException(zkEphemeralType + " is not supported.");
     }
   }
+
+
+  @Override
+  public List<OpResult> transactionOP(Iterable<Op> iterable) {
+    // Convert list of MetaClient Ops to Zk Ops
+    List<org.apache.zookeeper.Op> zkOps = metaClientOpToZk(iterable);
+    // Execute Zk transactional support
+    List<org.apache.zookeeper.OpResult> zkResult = _zkClient.multi(zkOps);
+    // Convert list of Zk OpResults to MetaClient OpResults
+    return zkOpResultToMetaClient(zkResult);
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public List<org.apache.zookeeper.Op> metaClientOpToZk(Iterable<Op> ops) {
+    List<org.apache.zookeeper.Op> zkOps = new ArrayList<>();
+    org.apache.zookeeper.Op temp;
+    for (Op op : ops) {
+      switch (op.getType()) {
+        case CREATE:
+          int zkFlag = zkFlagFromEntryMode(((Op.Create) op).getEntryMode());
+          try {
+            temp = org.apache.zookeeper.Op.create(
+                op.getPath(), ((Op.Create) op).getData(), DEFAULT_ACL, CreateMode.fromFlag(zkFlag));
+          } catch (KeeperException e) {
+            throw new MetaClientKeeperException(e);
+          }
+          break;
+        case DELETE:
+          temp = org.apache.zookeeper.Op.delete(
+              op.getPath(), ((Op.Delete) op).getVersion());
+          break;
+        case SET:
+          temp = org.apache.zookeeper.Op.setData(
+              op.getPath(), ((Op.Set) op).getData(), ((Op.Set) op).getVersion());
+          break;
+        case CHECK:
+          temp = org.apache.zookeeper.Op.check(
+              op.getPath(), ((Op.Check) op).getVersion());
+          break;
+        default:
+          throw new IllegalArgumentException(op.getType() + " is not supported.");
+      }
+      zkOps.add(temp);
+    }
+    return zkOps;
+  }
+
+  /**
+   * Helper function for transactionOP. Converts the result from calling zk transactional support into
+   * metaclient OpResults.
+   * @param zkResult
+   * @return
+   */
+  public List<OpResult> zkOpResultToMetaClient(List<org.apache.zookeeper.OpResult> zkResult) {

Review Comment:
   Overall suggestion, for all helper functions like this, let's move to a standalone class



##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/ZkMetaClient.java:
##########
@@ -414,4 +415,128 @@ private static EntryMode convertZkEntryMode(long ephemeralOwner) {
         throw new IllegalArgumentException(zkEphemeralType + " is not supported.");
     }
   }
+
+
+  @Override
+  public List<OpResult> transactionOP(Iterable<Op> iterable) {
+    // Convert list of MetaClient Ops to Zk Ops
+    List<org.apache.zookeeper.Op> zkOps = metaClientOpToZk(iterable);
+    // Execute Zk transactional support
+    List<org.apache.zookeeper.OpResult> zkResult = _zkClient.multi(zkOps);
+    // Convert list of Zk OpResults to MetaClient OpResults
+    return zkOpResultToMetaClient(zkResult);
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public List<org.apache.zookeeper.Op> metaClientOpToZk(Iterable<Op> ops) {
+    List<org.apache.zookeeper.Op> zkOps = new ArrayList<>();
+    org.apache.zookeeper.Op temp;
+    for (Op op : ops) {
+      switch (op.getType()) {
+        case CREATE:
+          int zkFlag = zkFlagFromEntryMode(((Op.Create) op).getEntryMode());
+          try {
+            temp = org.apache.zookeeper.Op.create(
+                op.getPath(), ((Op.Create) op).getData(), DEFAULT_ACL, CreateMode.fromFlag(zkFlag));
+          } catch (KeeperException e) {
+            throw new MetaClientKeeperException(e);
+          }
+          break;
+        case DELETE:
+          temp = org.apache.zookeeper.Op.delete(
+              op.getPath(), ((Op.Delete) op).getVersion());
+          break;
+        case SET:
+          temp = org.apache.zookeeper.Op.setData(
+              op.getPath(), ((Op.Set) op).getData(), ((Op.Set) op).getVersion());
+          break;
+        case CHECK:
+          temp = org.apache.zookeeper.Op.check(
+              op.getPath(), ((Op.Check) op).getVersion());
+          break;
+        default:
+          throw new IllegalArgumentException(op.getType() + " is not supported.");
+      }
+      zkOps.add(temp);
+    }
+    return zkOps;
+  }
+
+  /**
+   * Helper function for transactionOP. Converts the result from calling zk transactional support into
+   * metaclient OpResults.
+   * @param zkResult
+   * @return
+   */
+  public List<OpResult> zkOpResultToMetaClient(List<org.apache.zookeeper.OpResult> zkResult) {
+    List<OpResult> metaClientOpResult = new ArrayList<>();
+    OpResult temp;
+    String resultClass;
+    for (org.apache.zookeeper.OpResult opResult : zkResult) {
+      Stat metaClientStat;
+      resultClass = opResult.getClass().getSimpleName();
+      switch (resultClass) {
+        case "CreateResult":

Review Comment:
   Comparing on classname is too risky and low on performance (it's still magic number to some extent). If you must do it this way, please check `instance of`.



##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/ZkMetaClient.java:
##########
@@ -414,4 +414,130 @@ private static EntryMode convertZkEntryMode(long ephemeralOwner) {
         throw new IllegalArgumentException(zkEphemeralType + " is not supported.");
     }
   }
+
+
+  @Override
+  public List<OpResult> transactionOP(Iterable<Op> iterable) throws KeeperException {
+    // Convert list of MetaClient Ops to Zk Ops
+    List<org.apache.zookeeper.Op> zkOps = metaClientOpToZk(iterable);
+    // Execute Zk transactional support
+    List<org.apache.zookeeper.OpResult> zkResult = _zkClient.multi(zkOps);
+    // Convert list of Zk OpResults to MetaClient OpResults
+    return zkOpResultToMetaClient(zkResult);
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public List<org.apache.zookeeper.Op> metaClientOpToZk(Iterable<Op> ops) throws KeeperException {

Review Comment:
   Does this have to be public?



##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/ZkMetaClient.java:
##########
@@ -414,4 +415,128 @@ private static EntryMode convertZkEntryMode(long ephemeralOwner) {
         throw new IllegalArgumentException(zkEphemeralType + " is not supported.");
     }
   }
+
+
+  @Override
+  public List<OpResult> transactionOP(Iterable<Op> iterable) {
+    // Convert list of MetaClient Ops to Zk Ops
+    List<org.apache.zookeeper.Op> zkOps = metaClientOpToZk(iterable);
+    // Execute Zk transactional support
+    List<org.apache.zookeeper.OpResult> zkResult = _zkClient.multi(zkOps);
+    // Convert list of Zk OpResults to MetaClient OpResults
+    return zkOpResultToMetaClient(zkResult);
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public List<org.apache.zookeeper.Op> metaClientOpToZk(Iterable<Op> ops) {
+    List<org.apache.zookeeper.Op> zkOps = new ArrayList<>();
+    org.apache.zookeeper.Op temp;
+    for (Op op : ops) {
+      switch (op.getType()) {
+        case CREATE:
+          int zkFlag = zkFlagFromEntryMode(((Op.Create) op).getEntryMode());
+          try {
+            temp = org.apache.zookeeper.Op.create(
+                op.getPath(), ((Op.Create) op).getData(), DEFAULT_ACL, CreateMode.fromFlag(zkFlag));
+          } catch (KeeperException e) {
+            throw new MetaClientKeeperException(e);
+          }
+          break;
+        case DELETE:
+          temp = org.apache.zookeeper.Op.delete(
+              op.getPath(), ((Op.Delete) op).getVersion());
+          break;
+        case SET:
+          temp = org.apache.zookeeper.Op.setData(
+              op.getPath(), ((Op.Set) op).getData(), ((Op.Set) op).getVersion());
+          break;
+        case CHECK:
+          temp = org.apache.zookeeper.Op.check(
+              op.getPath(), ((Op.Check) op).getVersion());
+          break;
+        default:
+          throw new IllegalArgumentException(op.getType() + " is not supported.");
+      }
+      zkOps.add(temp);
+    }
+    return zkOps;
+  }
+
+  /**
+   * Helper function for transactionOP. Converts the result from calling zk transactional support into
+   * metaclient OpResults.
+   * @param zkResult
+   * @return
+   */
+  public List<OpResult> zkOpResultToMetaClient(List<org.apache.zookeeper.OpResult> zkResult) {
+    List<OpResult> metaClientOpResult = new ArrayList<>();
+    OpResult temp;
+    String resultClass;
+    for (org.apache.zookeeper.OpResult opResult : zkResult) {
+      Stat metaClientStat;
+      resultClass = opResult.getClass().getSimpleName();
+      switch (resultClass) {
+        case "CreateResult":

Review Comment:
   I see the difficulties here in binding two classes without knowing the type in compile time. One potential way to improve similar code, is to try using a static map to store the binding and utilize lambda functional.
   e.g.
   ```
   // mapping from class object => construction function of OpResult
   Map<Class<? extends org.apache.zookeeper.OpResult>, Function<org.apache.zookeeper.OpResult, OpResult>> map = new HashMap<>();
   map.put(org.apache.zookeeper.OpResult.CheckResult.class, op -> new OpResult.CheckResult()); 
   for (org.apache.zookeeper.OpResult opResult : zkResult) {
     map.get(opResult.getClass()).apply(opResult);
   }
   
   ``` 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] Marcosrico commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

Posted by GitBox <gi...@apache.org>.
Marcosrico commented on code in PR #2343:
URL: https://github.com/apache/helix/pull/2343#discussion_r1081455964


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/ZkMetaClient.java:
##########
@@ -414,4 +414,130 @@ private static EntryMode convertZkEntryMode(long ephemeralOwner) {
         throw new IllegalArgumentException(zkEphemeralType + " is not supported.");
     }
   }
+
+
+  @Override
+  public List<OpResult> transactionOP(Iterable<Op> iterable) throws KeeperException {
+    // Convert list of MetaClient Ops to Zk Ops
+    List<org.apache.zookeeper.Op> zkOps = metaClientOpToZk(iterable);
+    // Execute Zk transactional support
+    List<org.apache.zookeeper.OpResult> zkResult = _zkClient.multi(zkOps);
+    // Convert list of Zk OpResults to MetaClient OpResults
+    return zkOpResultToMetaClient(zkResult);
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public List<org.apache.zookeeper.Op> metaClientOpToZk(Iterable<Op> ops) throws KeeperException {
+    List<org.apache.zookeeper.Op> zkOps = new ArrayList<>();
+    org.apache.zookeeper.Op temp;
+    for (Op op : ops) {
+      switch (op.getType()) {
+        case CREATE:
+          int zkFlag = zkFlagFromEntryMode(((Op.Create) op).getEntryMode());
+          temp = org.apache.zookeeper.Op.create(
+              op.getPath(), ((Op.Create) op).getData(), DEFAULT_ACL, CreateMode.fromFlag(zkFlag));
+          break;
+        case DELETE:
+          temp = org.apache.zookeeper.Op.delete(
+              op.getPath(), ((Op.Delete) op).getVersion());
+          break;
+        case SET:
+          temp = org.apache.zookeeper.Op.setData(
+              op.getPath(), ((Op.Set) op).getData(), ((Op.Set) op).getVersion());
+          break;
+        case CHECK:
+          temp = org.apache.zookeeper.Op.check(
+              op.getPath(), ((Op.Check) op).getVersion());
+          break;
+        default:
+          throw new IllegalArgumentException(op.getType() + " is not supported.");
+      }
+      zkOps.add(temp);
+    }
+    return zkOps;
+  }
+
+  /**
+   * Helper function for transactionOP. Converts the result from calling zk transactional support into
+   * metaclient OpResults.
+   * @param zkResult
+   * @return
+   */
+  public List<OpResult> zkOpResultToMetaClient(List<org.apache.zookeeper.OpResult> zkResult) throws KeeperException.BadArgumentsException {
+    List<OpResult> metaClientOpResult = new ArrayList<>();
+    OpResult temp;
+    EntryMode zkMode;
+    for (org.apache.zookeeper.OpResult opResult : zkResult) {
+      Stat metaClientStat;
+      switch (opResult.getType()) {
+        // CreateResult
+        case 1:

Review Comment:
   Sounds good, I'll use the class name instead of opResult type.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] Marcosrico commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/util/ZkMetaClientUtil.java:
##########
@@ -0,0 +1,201 @@
+package org.apache.helix.metaclient.impl.zk.util;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.api.OpResult;
+import org.apache.helix.metaclient.constants.*;
+import org.apache.helix.zookeeper.zkclient.exception.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.EphemeralType;
+
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.function.Function;
+
+public class ZkMetaClientUtil {
+  //Default ACL value until metaClient Op has ACL of its own.

Review Comment:
   Sounds 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.

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

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


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


[GitHub] [helix] Marcosrico commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

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


##########
meta-client/src/test/java/org/apache/helix/metaclient/impl/zk/TestZkMetaClient.java:
##########
@@ -39,8 +39,11 @@
 import org.apache.commons.lang3.NotImplementedException;
 import org.apache.helix.metaclient.api.DataChangeListener;
 import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientConfig;
+import org.apache.helix.metaclient.api.Op;
+import org.apache.helix.metaclient.api.OpResult;
 import org.apache.helix.zookeeper.zkclient.IDefaultNameSpace;
 import org.apache.helix.zookeeper.zkclient.ZkServer;
+import org.apache.zookeeper.*;

Review Comment:
   Absolutely not! Must have auto corrected after many imports, nice 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.

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

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


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


[GitHub] [helix] Marcosrico commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

Posted by GitBox <gi...@apache.org>.
Marcosrico commented on code in PR #2343:
URL: https://github.com/apache/helix/pull/2343#discussion_r1081709850


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/ZkMetaClient.java:
##########
@@ -414,4 +414,130 @@ private static EntryMode convertZkEntryMode(long ephemeralOwner) {
         throw new IllegalArgumentException(zkEphemeralType + " is not supported.");
     }
   }
+
+
+  @Override
+  public List<OpResult> transactionOP(Iterable<Op> iterable) throws KeeperException {
+    // Convert list of MetaClient Ops to Zk Ops
+    List<org.apache.zookeeper.Op> zkOps = metaClientOpToZk(iterable);
+    // Execute Zk transactional support
+    List<org.apache.zookeeper.OpResult> zkResult = _zkClient.multi(zkOps);
+    // Convert list of Zk OpResults to MetaClient OpResults
+    return zkOpResultToMetaClient(zkResult);
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public List<org.apache.zookeeper.Op> metaClientOpToZk(Iterable<Op> ops) throws KeeperException {
+    List<org.apache.zookeeper.Op> zkOps = new ArrayList<>();
+    org.apache.zookeeper.Op temp;
+    for (Op op : ops) {
+      switch (op.getType()) {
+        case CREATE:
+          int zkFlag = zkFlagFromEntryMode(((Op.Create) op).getEntryMode());
+          temp = org.apache.zookeeper.Op.create(
+              op.getPath(), ((Op.Create) op).getData(), DEFAULT_ACL, CreateMode.fromFlag(zkFlag));
+          break;
+        case DELETE:
+          temp = org.apache.zookeeper.Op.delete(
+              op.getPath(), ((Op.Delete) op).getVersion());
+          break;
+        case SET:
+          temp = org.apache.zookeeper.Op.setData(
+              op.getPath(), ((Op.Set) op).getData(), ((Op.Set) op).getVersion());
+          break;
+        case CHECK:
+          temp = org.apache.zookeeper.Op.check(
+              op.getPath(), ((Op.Check) op).getVersion());
+          break;
+        default:
+          throw new IllegalArgumentException(op.getType() + " is not supported.");
+      }
+      zkOps.add(temp);
+    }
+    return zkOps;
+  }
+
+  /**
+   * Helper function for transactionOP. Converts the result from calling zk transactional support into
+   * metaclient OpResults.
+   * @param zkResult
+   * @return
+   */
+  public List<OpResult> zkOpResultToMetaClient(List<org.apache.zookeeper.OpResult> zkResult) throws KeeperException.BadArgumentsException {
+    List<OpResult> metaClientOpResult = new ArrayList<>();
+    OpResult temp;
+    EntryMode zkMode;
+    for (org.apache.zookeeper.OpResult opResult : zkResult) {
+      Stat metaClientStat;
+      switch (opResult.getType()) {
+        // CreateResult
+        case 1:
+          temp = new OpResult.CreateResult(
+                  ((org.apache.zookeeper.OpResult.CreateResult) opResult).getPath());
+          break;
+        // DeleteResult
+        case 2:
+          temp = new OpResult.DeleteResult();
+          break;
+        // GetDataResult
+        case 4:
+          org.apache.zookeeper.OpResult.GetDataResult zkOpGetDataResult =
+                  (org.apache.zookeeper.OpResult.GetDataResult) opResult;
+          metaClientStat = new Stat(convertZkEntryMode(zkOpGetDataResult.getStat().getEphemeralOwner()),
+              zkOpGetDataResult.getStat().getVersion());
+          temp = new OpResult.GetDataResult(zkOpGetDataResult.getData(), metaClientStat);
+          break;
+        //SetDataResult
+        case 5:
+          org.apache.zookeeper.OpResult.SetDataResult zkOpSetDataResult =
+                  (org.apache.zookeeper.OpResult.SetDataResult) opResult;
+          metaClientStat = new Stat(convertZkEntryMode(zkOpSetDataResult.getStat().getEphemeralOwner()),
+              zkOpSetDataResult.getStat().getVersion());
+          temp = new OpResult.SetDataResult(metaClientStat);
+          break;
+        //GetChildrenResult
+        case 8:
+          temp = new OpResult.GetChildrenResult(
+                  ((org.apache.zookeeper.OpResult.GetChildrenResult) opResult).getChildren());
+          break;
+        //CheckResult
+        case 13:
+          temp = new OpResult.CheckResult();
+          break;
+        //CreateResult with stat
+        case 15:
+          org.apache.zookeeper.OpResult.CreateResult zkOpCreateResult =
+                  (org.apache.zookeeper.OpResult.CreateResult) opResult;
+          metaClientStat = new Stat(convertZkEntryMode(zkOpCreateResult.getStat().getEphemeralOwner()),
+              zkOpCreateResult.getStat().getVersion());
+          temp = new OpResult.CreateResult(zkOpCreateResult.getPath(), metaClientStat);
+          break;
+        //ErrorResult
+        case -1:
+          temp = new OpResult.ErrorResult(
+                  ((org.apache.zookeeper.OpResult.ErrorResult) opResult).getErr());
+          break;
+        default:
+          throw new IllegalArgumentException(opResult.getType() + " is not supported.");
+      }
+      metaClientOpResult.add(temp);
+    }
+    return metaClientOpResult;
+  }
+
+  private int zkFlagFromEntryMode (EntryMode entryMode) {
+    String mode = entryMode.name();
+    if (mode.equals(EntryMode.PERSISTENT.name())) {
+      return 0;
+    }
+    if (mode.equals(EntryMode.EPHEMERAL.name())) {
+      return 1;
+    }
+    return -1;
+  }

Review Comment:
   Okay I'll modify the helper to translate entryMode directly to CreateMode. Thanks for the feedback!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] Marcosrico commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

Posted by GitBox <gi...@apache.org>.
Marcosrico commented on code in PR #2343:
URL: https://github.com/apache/helix/pull/2343#discussion_r1082869761


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/util/ZkMetaClientUtil.java:
##########
@@ -0,0 +1,156 @@
+package org.apache.helix.metaclient.impl.zk.util;
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.api.OpResult;
+import org.apache.helix.metaclient.constants.*;
+import org.apache.helix.zookeeper.zkclient.exception.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.EphemeralType;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.function.Function;
+
+public class ZkMetaClientUtil {
+  //Default ACL value until metaClient Op has ACL of its own.
+  private static final List<ACL> DEFAULT_ACL = Collections.unmodifiableList(ZooDefs.Ids.OPEN_ACL_UNSAFE);
+
+  private ZkMetaClientUtil(){
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public static List<Op> metaClientOpToZk(Iterable<org.apache.helix.metaclient.api.Op> ops) {
+    Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> opMap = new HashMap<>();

Review Comment:
   Sounds good thank you for the review. I updated the methods with a lazy and thread safe implementation of the maps, please let me know if it looks good, thanks.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] Marcosrico commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/util/ZkMetaClientUtil.java:
##########
@@ -0,0 +1,213 @@
+package org.apache.helix.metaclient.impl.zk.util;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.api.OpResult;
+import org.apache.helix.metaclient.constants.*;
+import org.apache.helix.zookeeper.zkclient.exception.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.EphemeralType;
+
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.function.Function;
+
+public class ZkMetaClientUtil {
+  //TODO Implement MetaClient ACL
+  //Default ACL value until metaClient Op has ACL of its own.
+  private static final List<ACL> DEFAULT_ACL = Collections.unmodifiableList(ZooDefs.Ids.OPEN_ACL_UNSAFE);
+
+  private ZkMetaClientUtil(){
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public static List<Op> metaClientOpsToZkOps(Iterable<org.apache.helix.metaclient.api.Op> ops) {
+    if (getOpMap().isEmpty()) {
+      initializeOpMap();
+    }
+
+    List<Op> zkOps = new ArrayList<>();
+    for (org.apache.helix.metaclient.api.Op op : ops) {
+      Function<org.apache.helix.metaclient.api.Op, Op> function = getOpMap().get(op.getType());
+      if (function != null) {
+        zkOps.add(function.apply(op));
+      } else {
+        throw new IllegalArgumentException("Op type " + op.getType().name() + " is not supported.");
+      }
+    }
+    return zkOps;
+  }
+
+  private static final class OpMapHolder {
+    static final Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> OPMAP =
+        new EnumMap<>(org.apache.helix.metaclient.api.Op.Type.class);
+  }
+
+  private static Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> getOpMap() {
+    return OpMapHolder.OPMAP;
+  }
+
+  private static void initializeOpMap() {
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.CREATE, op -> {
+      try {
+        CreateMode mode = convertMetaClientMode(((org.apache.helix.metaclient.api.Op.Create) op).getEntryMode());
+        return Op.create(op.getPath(), ((org.apache.helix.metaclient.api.Op.Create) op).getData(), DEFAULT_ACL, mode);
+      } catch (KeeperException e) {
+        throw translateZkExceptionToMetaclientException(ZkException.create(e));
+      }
+    });
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.DELETE,
+        op -> Op.delete(op.getPath(), ((org.apache.helix.metaclient.api.Op.Delete) op).getVersion()));
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.SET,
+        op -> Op.setData(op.getPath(),
+            ((org.apache.helix.metaclient.api.Op.Set) op).getData(), ((org.apache.helix.metaclient.api.Op.Set) op).getVersion()));
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.CHECK,
+        op -> Op.check(op.getPath(), ((org.apache.helix.metaclient.api.Op.Check) op).getVersion()));
+  }
+
+  private static CreateMode convertMetaClientMode(MetaClientInterface.EntryMode entryMode) throws KeeperException {
+    switch (entryMode) {
+      case PERSISTENT:
+        return CreateMode.PERSISTENT;
+      case EPHEMERAL:
+        return CreateMode.EPHEMERAL;
+      case CONTAINER:
+        return CreateMode.CONTAINER;
+      default:
+        throw new IllegalArgumentException(entryMode.name() + " is not a supported EntryMode.");
+    }
+  }
+
+  /**
+   * Helper function for transactionOP. Converts the result from calling zk transactional support into
+   * metaclient OpResults.
+   * @param zkResult
+   * @return
+   */
+  public static List<OpResult> zkOpResultToMetaClientOpResults(List<org.apache.zookeeper.OpResult> zkResult) {
+    if (getOpResultMap().isEmpty()) {
+      initializeOpResultMap();
+    }
+    List<OpResult> metaClientOpResult = new ArrayList<>();
+    for (org.apache.zookeeper.OpResult opResult : zkResult) {
+      Function<org.apache.zookeeper.OpResult, OpResult> function = getOpResultMap().get(opResult.getClass());
+      if (function != null) {
+        metaClientOpResult.add(function.apply(opResult));
+      } else {
+        throw new IllegalArgumentException("OpResult type " + opResult.getType() + "is not supported.");
+      }
+    }
+
+    return metaClientOpResult;
+  }
+
+  private static final class OpResultMapHolder {
+    static final Map<Class<? extends org.apache.zookeeper.OpResult>, Function<org.apache.zookeeper.OpResult, OpResult>> OPRESULTMAP =
+        new HashMap<>();
+  }

Review Comment:
   Thank you for your correction, I understand what you mean. I was implementing option 2 wrong which was why the error occurred. Thanks for the feedback and I have implemented those changes, let me know if it's 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.

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

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


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


[GitHub] [helix] Marcosrico commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

Posted by GitBox <gi...@apache.org>.
Marcosrico commented on code in PR #2343:
URL: https://github.com/apache/helix/pull/2343#discussion_r1081355091


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/ZkMetaClient.java:
##########
@@ -414,4 +414,130 @@ private static EntryMode convertZkEntryMode(long ephemeralOwner) {
         throw new IllegalArgumentException(zkEphemeralType + " is not supported.");
     }
   }
+
+
+  @Override
+  public List<OpResult> transactionOP(Iterable<Op> iterable) throws KeeperException {
+    // Convert list of MetaClient Ops to Zk Ops
+    List<org.apache.zookeeper.Op> zkOps = metaClientOpToZk(iterable);
+    // Execute Zk transactional support
+    List<org.apache.zookeeper.OpResult> zkResult = _zkClient.multi(zkOps);
+    // Convert list of Zk OpResults to MetaClient OpResults
+    return zkOpResultToMetaClient(zkResult);
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public List<org.apache.zookeeper.Op> metaClientOpToZk(Iterable<Op> ops) throws KeeperException {

Review Comment:
   Nope not required, thanks for the 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.

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

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


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


[GitHub] [helix] Marcosrico commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

Posted by GitBox <gi...@apache.org>.
Marcosrico commented on code in PR #2343:
URL: https://github.com/apache/helix/pull/2343#discussion_r1081640082


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/ZkMetaClient.java:
##########
@@ -414,4 +415,128 @@ private static EntryMode convertZkEntryMode(long ephemeralOwner) {
         throw new IllegalArgumentException(zkEphemeralType + " is not supported.");
     }
   }
+
+
+  @Override
+  public List<OpResult> transactionOP(Iterable<Op> iterable) {
+    // Convert list of MetaClient Ops to Zk Ops
+    List<org.apache.zookeeper.Op> zkOps = metaClientOpToZk(iterable);
+    // Execute Zk transactional support
+    List<org.apache.zookeeper.OpResult> zkResult = _zkClient.multi(zkOps);
+    // Convert list of Zk OpResults to MetaClient OpResults
+    return zkOpResultToMetaClient(zkResult);
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public List<org.apache.zookeeper.Op> metaClientOpToZk(Iterable<Op> ops) {
+    List<org.apache.zookeeper.Op> zkOps = new ArrayList<>();
+    org.apache.zookeeper.Op temp;
+    for (Op op : ops) {
+      switch (op.getType()) {
+        case CREATE:
+          int zkFlag = zkFlagFromEntryMode(((Op.Create) op).getEntryMode());
+          try {
+            temp = org.apache.zookeeper.Op.create(
+                op.getPath(), ((Op.Create) op).getData(), DEFAULT_ACL, CreateMode.fromFlag(zkFlag));
+          } catch (KeeperException e) {
+            throw new MetaClientKeeperException(e);
+          }
+          break;
+        case DELETE:
+          temp = org.apache.zookeeper.Op.delete(
+              op.getPath(), ((Op.Delete) op).getVersion());
+          break;
+        case SET:
+          temp = org.apache.zookeeper.Op.setData(
+              op.getPath(), ((Op.Set) op).getData(), ((Op.Set) op).getVersion());
+          break;
+        case CHECK:
+          temp = org.apache.zookeeper.Op.check(
+              op.getPath(), ((Op.Check) op).getVersion());
+          break;
+        default:
+          throw new IllegalArgumentException(op.getType() + " is not supported.");
+      }
+      zkOps.add(temp);
+    }
+    return zkOps;
+  }
+
+  /**
+   * Helper function for transactionOP. Converts the result from calling zk transactional support into
+   * metaclient OpResults.
+   * @param zkResult
+   * @return
+   */
+  public List<OpResult> zkOpResultToMetaClient(List<org.apache.zookeeper.OpResult> zkResult) {

Review Comment:
   Are you saying a separate class in the same directory with these two helper 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.

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

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


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


[GitHub] [helix] qqu0127 commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

Posted by GitBox <gi...@apache.org>.
qqu0127 commented on code in PR #2343:
URL: https://github.com/apache/helix/pull/2343#discussion_r1082719864


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/util/ZkMetaClientUtil.java:
##########
@@ -0,0 +1,156 @@
+package org.apache.helix.metaclient.impl.zk.util;
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.api.OpResult;
+import org.apache.helix.metaclient.constants.*;
+import org.apache.helix.zookeeper.zkclient.exception.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.EphemeralType;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.function.Function;
+

Review Comment:
   Add license



##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/util/ZkMetaClientUtil.java:
##########
@@ -0,0 +1,156 @@
+package org.apache.helix.metaclient.impl.zk.util;
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.api.OpResult;
+import org.apache.helix.metaclient.constants.*;
+import org.apache.helix.zookeeper.zkclient.exception.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.EphemeralType;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.function.Function;
+
+public class ZkMetaClientUtil {
+  //Default ACL value until metaClient Op has ACL of its own.
+  private static final List<ACL> DEFAULT_ACL = Collections.unmodifiableList(ZooDefs.Ids.OPEN_ACL_UNSAFE);
+
+  private ZkMetaClientUtil(){
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public static List<Op> metaClientOpToZk(Iterable<org.apache.helix.metaclient.api.Op> ops) {
+    Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> opMap = new HashMap<>();
+
+    opMap.put(org.apache.helix.metaclient.api.Op.Type.CREATE, op -> {
+      try {
+        CreateMode mode = convertMetaClientMode(((org.apache.helix.metaclient.api.Op.Create) op).getEntryMode());
+        return Op.create(op.getPath(), ((org.apache.helix.metaclient.api.Op.Create) op).getData(), DEFAULT_ACL, mode);
+      } catch (KeeperException e) {
+        throw translateZkExceptionToMetaclientException(ZkException.create(e));
+      }
+    });
+
+    opMap.put(org.apache.helix.metaclient.api.Op.Type.DELETE,
+        op -> Op.delete(op.getPath(), ((org.apache.helix.metaclient.api.Op.Delete) op).getVersion()));
+
+    opMap.put(org.apache.helix.metaclient.api.Op.Type.SET,
+        op -> Op.setData(op.getPath(),
+            ((org.apache.helix.metaclient.api.Op.Set) op).getData(), ((org.apache.helix.metaclient.api.Op.Set) op).getVersion()));
+
+    opMap.put(org.apache.helix.metaclient.api.Op.Type.CHECK,
+        op -> Op.check(op.getPath(), ((org.apache.helix.metaclient.api.Op.Check) op).getVersion()));
+
+    List<Op> zkOps = new ArrayList<>();
+    for (org.apache.helix.metaclient.api.Op op : ops) {
+      Op temp = opMap.get(op.getType()).apply(op);
+      zkOps.add(temp);
+    }
+    return zkOps;
+  }
+
+  private static CreateMode convertMetaClientMode(MetaClientInterface.EntryMode entryMode) throws KeeperException {
+    String mode = entryMode.name();
+    if (mode.equals(MetaClientInterface.EntryMode.PERSISTENT.name())) {
+      return CreateMode.fromFlag(0);
+    }
+    if (mode.equals(MetaClientInterface.EntryMode.EPHEMERAL.name())) {
+      return CreateMode.fromFlag(1);
+    }

Review Comment:
   We can directly compare e.g. `entryMode == EntryMode.PERSISTENT`, there is no need for string compare or `CreateMode.fromFlag`. 
   This method doesn't need to throw `KeeperException` either.



##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/util/ZkMetaClientUtil.java:
##########
@@ -0,0 +1,156 @@
+package org.apache.helix.metaclient.impl.zk.util;
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.api.OpResult;
+import org.apache.helix.metaclient.constants.*;
+import org.apache.helix.zookeeper.zkclient.exception.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.EphemeralType;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.function.Function;
+
+public class ZkMetaClientUtil {
+  //Default ACL value until metaClient Op has ACL of its own.
+  private static final List<ACL> DEFAULT_ACL = Collections.unmodifiableList(ZooDefs.Ids.OPEN_ACL_UNSAFE);
+
+  private ZkMetaClientUtil(){
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public static List<Op> metaClientOpToZk(Iterable<org.apache.helix.metaclient.api.Op> ops) {
+    Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> opMap = new HashMap<>();

Review Comment:
   The map is a constant and can be reused, let's store that as a static variable. Please also consider lazy initialization. 



##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/util/ZkMetaClientUtil.java:
##########
@@ -0,0 +1,156 @@
+package org.apache.helix.metaclient.impl.zk.util;
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.api.OpResult;
+import org.apache.helix.metaclient.constants.*;
+import org.apache.helix.zookeeper.zkclient.exception.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.EphemeralType;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.function.Function;
+
+public class ZkMetaClientUtil {
+  //Default ACL value until metaClient Op has ACL of its own.
+  private static final List<ACL> DEFAULT_ACL = Collections.unmodifiableList(ZooDefs.Ids.OPEN_ACL_UNSAFE);
+
+  private ZkMetaClientUtil(){
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public static List<Op> metaClientOpToZk(Iterable<org.apache.helix.metaclient.api.Op> ops) {
+    Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> opMap = new HashMap<>();
+
+    opMap.put(org.apache.helix.metaclient.api.Op.Type.CREATE, op -> {
+      try {
+        CreateMode mode = convertMetaClientMode(((org.apache.helix.metaclient.api.Op.Create) op).getEntryMode());
+        return Op.create(op.getPath(), ((org.apache.helix.metaclient.api.Op.Create) op).getData(), DEFAULT_ACL, mode);
+      } catch (KeeperException e) {
+        throw translateZkExceptionToMetaclientException(ZkException.create(e));
+      }
+    });
+
+    opMap.put(org.apache.helix.metaclient.api.Op.Type.DELETE,
+        op -> Op.delete(op.getPath(), ((org.apache.helix.metaclient.api.Op.Delete) op).getVersion()));
+
+    opMap.put(org.apache.helix.metaclient.api.Op.Type.SET,
+        op -> Op.setData(op.getPath(),
+            ((org.apache.helix.metaclient.api.Op.Set) op).getData(), ((org.apache.helix.metaclient.api.Op.Set) op).getVersion()));
+
+    opMap.put(org.apache.helix.metaclient.api.Op.Type.CHECK,
+        op -> Op.check(op.getPath(), ((org.apache.helix.metaclient.api.Op.Check) op).getVersion()));
+
+    List<Op> zkOps = new ArrayList<>();
+    for (org.apache.helix.metaclient.api.Op op : ops) {
+      Op temp = opMap.get(op.getType()).apply(op);
+      zkOps.add(temp);
+    }
+    return zkOps;
+  }
+
+  private static CreateMode convertMetaClientMode(MetaClientInterface.EntryMode entryMode) throws KeeperException {
+    String mode = entryMode.name();
+    if (mode.equals(MetaClientInterface.EntryMode.PERSISTENT.name())) {
+      return CreateMode.fromFlag(0);
+    }
+    if (mode.equals(MetaClientInterface.EntryMode.EPHEMERAL.name())) {
+      return CreateMode.fromFlag(1);
+    }
+    return CreateMode.fromFlag(-1);
+  }
+
+  /**
+   * Helper function for transactionOP. Converts the result from calling zk transactional support into
+   * metaclient OpResults.
+   * @param zkResult
+   * @return
+   */
+  public static List<OpResult> zkOpResultToMetaClient(List<org.apache.zookeeper.OpResult> zkResult) {
+    Map<Class<? extends org.apache.zookeeper.OpResult>, Function<org.apache.zookeeper.OpResult, OpResult>> map = new HashMap<>();

Review Comment:
   same



##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/util/ZkMetaClientUtil.java:
##########
@@ -0,0 +1,156 @@
+package org.apache.helix.metaclient.impl.zk.util;
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.api.OpResult;
+import org.apache.helix.metaclient.constants.*;
+import org.apache.helix.zookeeper.zkclient.exception.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.EphemeralType;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.function.Function;
+
+public class ZkMetaClientUtil {
+  //Default ACL value until metaClient Op has ACL of its own.
+  private static final List<ACL> DEFAULT_ACL = Collections.unmodifiableList(ZooDefs.Ids.OPEN_ACL_UNSAFE);
+
+  private ZkMetaClientUtil(){
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public static List<Op> metaClientOpToZk(Iterable<org.apache.helix.metaclient.api.Op> ops) {
+    Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> opMap = new HashMap<>();

Review Comment:
   For this particular case, since the key is enum, please use EnumMap for better performance.



##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/util/ZkMetaClientUtil.java:
##########
@@ -0,0 +1,156 @@
+package org.apache.helix.metaclient.impl.zk.util;
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.api.OpResult;
+import org.apache.helix.metaclient.constants.*;
+import org.apache.helix.zookeeper.zkclient.exception.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.EphemeralType;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.function.Function;
+
+public class ZkMetaClientUtil {
+  //Default ACL value until metaClient Op has ACL of its own.
+  private static final List<ACL> DEFAULT_ACL = Collections.unmodifiableList(ZooDefs.Ids.OPEN_ACL_UNSAFE);
+
+  private ZkMetaClientUtil(){
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public static List<Op> metaClientOpToZk(Iterable<org.apache.helix.metaclient.api.Op> ops) {
+    Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> opMap = new HashMap<>();
+
+    opMap.put(org.apache.helix.metaclient.api.Op.Type.CREATE, op -> {
+      try {
+        CreateMode mode = convertMetaClientMode(((org.apache.helix.metaclient.api.Op.Create) op).getEntryMode());
+        return Op.create(op.getPath(), ((org.apache.helix.metaclient.api.Op.Create) op).getData(), DEFAULT_ACL, mode);
+      } catch (KeeperException e) {
+        throw translateZkExceptionToMetaclientException(ZkException.create(e));
+      }
+    });
+
+    opMap.put(org.apache.helix.metaclient.api.Op.Type.DELETE,
+        op -> Op.delete(op.getPath(), ((org.apache.helix.metaclient.api.Op.Delete) op).getVersion()));
+
+    opMap.put(org.apache.helix.metaclient.api.Op.Type.SET,
+        op -> Op.setData(op.getPath(),
+            ((org.apache.helix.metaclient.api.Op.Set) op).getData(), ((org.apache.helix.metaclient.api.Op.Set) op).getVersion()));
+
+    opMap.put(org.apache.helix.metaclient.api.Op.Type.CHECK,
+        op -> Op.check(op.getPath(), ((org.apache.helix.metaclient.api.Op.Check) op).getVersion()));
+
+    List<Op> zkOps = new ArrayList<>();
+    for (org.apache.helix.metaclient.api.Op op : ops) {
+      Op temp = opMap.get(op.getType()).apply(op);
+      zkOps.add(temp);
+    }
+    return zkOps;
+  }
+
+  private static CreateMode convertMetaClientMode(MetaClientInterface.EntryMode entryMode) throws KeeperException {
+    String mode = entryMode.name();
+    if (mode.equals(MetaClientInterface.EntryMode.PERSISTENT.name())) {
+      return CreateMode.fromFlag(0);
+    }
+    if (mode.equals(MetaClientInterface.EntryMode.EPHEMERAL.name())) {
+      return CreateMode.fromFlag(1);
+    }
+    return CreateMode.fromFlag(-1);
+  }
+
+  /**
+   * Helper function for transactionOP. Converts the result from calling zk transactional support into
+   * metaclient OpResults.
+   * @param zkResult
+   * @return
+   */
+  public static List<OpResult> zkOpResultToMetaClient(List<org.apache.zookeeper.OpResult> zkResult) {
+    Map<Class<? extends org.apache.zookeeper.OpResult>, Function<org.apache.zookeeper.OpResult, OpResult>> map = new HashMap<>();
+
+    map.put(org.apache.zookeeper.OpResult.CreateResult.class, opResult -> {
+      org.apache.zookeeper.OpResult.CreateResult zkOpCreateResult = (org.apache.zookeeper.OpResult.CreateResult) opResult;
+      if (opResult.getType() == 1) {
+        return new OpResult.CreateResult(zkOpCreateResult.getPath());
+      } else {
+        MetaClientInterface.Stat metaClientStat = new MetaClientInterface.Stat(convertZkEntryMode(zkOpCreateResult.getStat().getEphemeralOwner()),
+            zkOpCreateResult.getStat().getVersion());
+        return new OpResult.CreateResult(zkOpCreateResult.getPath(), metaClientStat);
+      }});
+
+    map.put(org.apache.zookeeper.OpResult.DeleteResult.class, opResult -> new OpResult.DeleteResult());
+
+    map.put(org.apache.zookeeper.OpResult.GetDataResult.class, opResult -> {
+      org.apache.zookeeper.OpResult.GetDataResult zkOpGetDataResult = (org.apache.zookeeper.OpResult.GetDataResult) opResult;
+      MetaClientInterface.Stat metaClientStat = new MetaClientInterface.Stat(convertZkEntryMode(zkOpGetDataResult.getStat().getEphemeralOwner()),
+          zkOpGetDataResult.getStat().getVersion());
+      return new OpResult.GetDataResult(zkOpGetDataResult.getData(), metaClientStat);
+    });
+
+    map.put(org.apache.zookeeper.OpResult.SetDataResult.class, opResult -> {
+      org.apache.zookeeper.OpResult.SetDataResult zkOpSetDataResult = (org.apache.zookeeper.OpResult.SetDataResult) opResult;
+      MetaClientInterface.Stat metaClientStat = new MetaClientInterface.Stat(convertZkEntryMode(zkOpSetDataResult.getStat().getEphemeralOwner()),
+          zkOpSetDataResult.getStat().getVersion());
+      return new OpResult.SetDataResult(metaClientStat);
+    });
+
+    map.put(org.apache.zookeeper.OpResult.GetChildrenResult.class, opResult -> new OpResult.GetChildrenResult(
+        ((org.apache.zookeeper.OpResult.GetChildrenResult) opResult).getChildren()));
+
+    map.put(org.apache.zookeeper.OpResult.CheckResult.class, opResult -> new OpResult.CheckResult());
+
+    map.put(org.apache.zookeeper.OpResult.ErrorResult.class, opResult -> new OpResult.ErrorResult(
+        ((org.apache.zookeeper.OpResult.ErrorResult) opResult).getErr()));
+
+    List<OpResult> metaClientOpResult = new ArrayList<>();
+    for (org.apache.zookeeper.OpResult opResult : zkResult) {
+      metaClientOpResult.add(map.get(opResult.getClass()).apply(opResult));

Review Comment:
   let's do a null check after map.get() to be safer



##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/util/ZkMetaClientUtil.java:
##########
@@ -0,0 +1,156 @@
+package org.apache.helix.metaclient.impl.zk.util;
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.api.OpResult;
+import org.apache.helix.metaclient.constants.*;
+import org.apache.helix.zookeeper.zkclient.exception.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.EphemeralType;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.function.Function;
+
+public class ZkMetaClientUtil {
+  //Default ACL value until metaClient Op has ACL of its own.
+  private static final List<ACL> DEFAULT_ACL = Collections.unmodifiableList(ZooDefs.Ids.OPEN_ACL_UNSAFE);
+
+  private ZkMetaClientUtil(){
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public static List<Op> metaClientOpToZk(Iterable<org.apache.helix.metaclient.api.Op> ops) {
+    Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> opMap = new HashMap<>();
+
+    opMap.put(org.apache.helix.metaclient.api.Op.Type.CREATE, op -> {
+      try {
+        CreateMode mode = convertMetaClientMode(((org.apache.helix.metaclient.api.Op.Create) op).getEntryMode());
+        return Op.create(op.getPath(), ((org.apache.helix.metaclient.api.Op.Create) op).getData(), DEFAULT_ACL, mode);
+      } catch (KeeperException e) {
+        throw translateZkExceptionToMetaclientException(ZkException.create(e));
+      }
+    });
+
+    opMap.put(org.apache.helix.metaclient.api.Op.Type.DELETE,
+        op -> Op.delete(op.getPath(), ((org.apache.helix.metaclient.api.Op.Delete) op).getVersion()));
+
+    opMap.put(org.apache.helix.metaclient.api.Op.Type.SET,
+        op -> Op.setData(op.getPath(),
+            ((org.apache.helix.metaclient.api.Op.Set) op).getData(), ((org.apache.helix.metaclient.api.Op.Set) op).getVersion()));
+
+    opMap.put(org.apache.helix.metaclient.api.Op.Type.CHECK,
+        op -> Op.check(op.getPath(), ((org.apache.helix.metaclient.api.Op.Check) op).getVersion()));
+
+    List<Op> zkOps = new ArrayList<>();
+    for (org.apache.helix.metaclient.api.Op op : ops) {
+      Op temp = opMap.get(op.getType()).apply(op);
+      zkOps.add(temp);
+    }
+    return zkOps;
+  }
+
+  private static CreateMode convertMetaClientMode(MetaClientInterface.EntryMode entryMode) throws KeeperException {
+    String mode = entryMode.name();
+    if (mode.equals(MetaClientInterface.EntryMode.PERSISTENT.name())) {
+      return CreateMode.fromFlag(0);
+    }
+    if (mode.equals(MetaClientInterface.EntryMode.EPHEMERAL.name())) {
+      return CreateMode.fromFlag(1);
+    }
+    return CreateMode.fromFlag(-1);

Review Comment:
   Does -1 mean illegal argument?
   Let's explicitly throw `IllegalArgumentException` to be more clear



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] rahulrane50 commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/ZkMetaClient.java:
##########
@@ -391,34 +380,13 @@ public int hashCode() {
     }
   }
 
-  private static MetaClientException translateZkExceptionToMetaclientException(ZkException e) {
-    if (e instanceof ZkNodeExistsException) {
-      return new MetaClientNoNodeException(e);
-    } else if (e instanceof ZkBadVersionException) {
-      return new MetaClientBadVersionException(e);
-    } else if (e instanceof ZkTimeoutException) {
-      return new MetaClientTimeoutException(e);
-    } else if (e instanceof ZkInterruptedException) {
-      return new MetaClientInterruptException(e);
-    } else {
-      return new MetaClientException(e);
-    }
-  }
-
-  private static EntryMode convertZkEntryMode(long ephemeralOwner) {
-    EphemeralType zkEphemeralType = EphemeralType.get(ephemeralOwner);
-    switch (zkEphemeralType) {
-      case VOID:
-        return EntryMode.PERSISTENT;
-      case CONTAINER:
-        return EntryMode.CONTAINER;
-      case NORMAL:
-        return EntryMode.EPHEMERAL;
-      // TODO: TTL is not supported now.
-      //case TTL:
-      //  return EntryMode.TTL;
-      default:
-        throw new IllegalArgumentException(zkEphemeralType + " is not supported.");
-    }
+  @Override
+  public List<OpResult> transactionOP(Iterable<Op> iterable) {

Review Comment:
   neat: iterable looks bad variable name. Do you want to change it something like iterableOps or just ops or anything which tells that these are ops list.



##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/util/ZkMetaClientUtil.java:
##########
@@ -0,0 +1,201 @@
+package org.apache.helix.metaclient.impl.zk.util;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.api.OpResult;
+import org.apache.helix.metaclient.constants.*;
+import org.apache.helix.zookeeper.zkclient.exception.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.EphemeralType;
+
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.function.Function;
+
+public class ZkMetaClientUtil {
+  //Default ACL value until metaClient Op has ACL of its own.
+  private static final List<ACL> DEFAULT_ACL = Collections.unmodifiableList(ZooDefs.Ids.OPEN_ACL_UNSAFE);
+
+  private ZkMetaClientUtil(){
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public static List<Op> metaClientOpToZk(Iterable<org.apache.helix.metaclient.api.Op> ops) {
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.CREATE, op -> {
+      try {
+        CreateMode mode = convertMetaClientMode(((org.apache.helix.metaclient.api.Op.Create) op).getEntryMode());
+        return Op.create(op.getPath(), ((org.apache.helix.metaclient.api.Op.Create) op).getData(), DEFAULT_ACL, mode);
+      } catch (KeeperException e) {
+        throw translateZkExceptionToMetaclientException(ZkException.create(e));
+      }
+    });
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.DELETE,
+        op -> Op.delete(op.getPath(), ((org.apache.helix.metaclient.api.Op.Delete) op).getVersion()));
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.SET,
+        op -> Op.setData(op.getPath(),
+            ((org.apache.helix.metaclient.api.Op.Set) op).getData(), ((org.apache.helix.metaclient.api.Op.Set) op).getVersion()));
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.CHECK,
+        op -> Op.check(op.getPath(), ((org.apache.helix.metaclient.api.Op.Check) op).getVersion()));
+
+    List<Op> zkOps = new ArrayList<>();
+    for (org.apache.helix.metaclient.api.Op op : ops) {
+      Function<org.apache.helix.metaclient.api.Op, Op> function = getOpMap().get(op.getType());
+      if (function != null) {
+        zkOps.add(function.apply(op));
+      } else {
+        throw new IllegalArgumentException("Op type " + op.getType().name() + "is not supported.");
+      }
+    }
+    return zkOps;
+  }
+
+  private static final class OpmapHolder {
+    static final Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> OPMAP =
+        new EnumMap<>(org.apache.helix.metaclient.api.Op.Type.class);
+  }
+
+  private static Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> getOpMap() {
+    return OpmapHolder.OPMAP;
+  }
+
+  private static CreateMode convertMetaClientMode(MetaClientInterface.EntryMode entryMode) throws KeeperException {
+    if (entryMode == MetaClientInterface.EntryMode.PERSISTENT) {
+      return CreateMode.fromFlag(0);
+    }
+    if (entryMode == MetaClientInterface.EntryMode.EPHEMERAL) {
+      return CreateMode.fromFlag(1);
+    }
+    if (entryMode == MetaClientInterface.EntryMode.CONTAINER) {
+      return CreateMode.fromFlag(4);
+    }
+    throw new IllegalArgumentException(entryMode.name() + " is not a supported EntryMode.");
+  }
+
+  /**
+   * Helper function for transactionOP. Converts the result from calling zk transactional support into
+   * metaclient OpResults.
+   * @param zkResult
+   * @return
+   */
+  public static List<OpResult> zkOpResultToMetaClient(List<org.apache.zookeeper.OpResult> zkResult) {

Review Comment:
   neat: similar comment about plural.



##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/util/ZkMetaClientUtil.java:
##########
@@ -0,0 +1,201 @@
+package org.apache.helix.metaclient.impl.zk.util;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.api.OpResult;
+import org.apache.helix.metaclient.constants.*;
+import org.apache.helix.zookeeper.zkclient.exception.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.EphemeralType;
+
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.function.Function;
+
+public class ZkMetaClientUtil {
+  //Default ACL value until metaClient Op has ACL of its own.
+  private static final List<ACL> DEFAULT_ACL = Collections.unmodifiableList(ZooDefs.Ids.OPEN_ACL_UNSAFE);
+
+  private ZkMetaClientUtil(){
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public static List<Op> metaClientOpToZk(Iterable<org.apache.helix.metaclient.api.Op> ops) {
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.CREATE, op -> {
+      try {
+        CreateMode mode = convertMetaClientMode(((org.apache.helix.metaclient.api.Op.Create) op).getEntryMode());
+        return Op.create(op.getPath(), ((org.apache.helix.metaclient.api.Op.Create) op).getData(), DEFAULT_ACL, mode);
+      } catch (KeeperException e) {
+        throw translateZkExceptionToMetaclientException(ZkException.create(e));
+      }
+    });
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.DELETE,
+        op -> Op.delete(op.getPath(), ((org.apache.helix.metaclient.api.Op.Delete) op).getVersion()));
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.SET,
+        op -> Op.setData(op.getPath(),
+            ((org.apache.helix.metaclient.api.Op.Set) op).getData(), ((org.apache.helix.metaclient.api.Op.Set) op).getVersion()));
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.CHECK,
+        op -> Op.check(op.getPath(), ((org.apache.helix.metaclient.api.Op.Check) op).getVersion()));
+
+    List<Op> zkOps = new ArrayList<>();
+    for (org.apache.helix.metaclient.api.Op op : ops) {
+      Function<org.apache.helix.metaclient.api.Op, Op> function = getOpMap().get(op.getType());
+      if (function != null) {
+        zkOps.add(function.apply(op));
+      } else {
+        throw new IllegalArgumentException("Op type " + op.getType().name() + "is not supported.");
+      }
+    }
+    return zkOps;
+  }
+
+  private static final class OpmapHolder {

Review Comment:
   neat: OpMapHolder.



##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/util/ZkMetaClientUtil.java:
##########
@@ -0,0 +1,201 @@
+package org.apache.helix.metaclient.impl.zk.util;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.api.OpResult;
+import org.apache.helix.metaclient.constants.*;
+import org.apache.helix.zookeeper.zkclient.exception.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.EphemeralType;
+
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.function.Function;
+
+public class ZkMetaClientUtil {
+  //Default ACL value until metaClient Op has ACL of its own.

Review Comment:
   neat: let's add a TODO here as well. We might need to think about ACLs being set by helix. 



##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/util/ZkMetaClientUtil.java:
##########
@@ -0,0 +1,156 @@
+package org.apache.helix.metaclient.impl.zk.util;
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.api.OpResult;
+import org.apache.helix.metaclient.constants.*;
+import org.apache.helix.zookeeper.zkclient.exception.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.EphemeralType;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.function.Function;
+

Review Comment:
   Okey i am seeing this pattern a lot now let me file a issue to automate this. 



##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/util/ZkMetaClientUtil.java:
##########
@@ -0,0 +1,201 @@
+package org.apache.helix.metaclient.impl.zk.util;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.api.OpResult;
+import org.apache.helix.metaclient.constants.*;
+import org.apache.helix.zookeeper.zkclient.exception.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.EphemeralType;
+
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.function.Function;
+
+public class ZkMetaClientUtil {
+  //Default ACL value until metaClient Op has ACL of its own.
+  private static final List<ACL> DEFAULT_ACL = Collections.unmodifiableList(ZooDefs.Ids.OPEN_ACL_UNSAFE);
+
+  private ZkMetaClientUtil(){
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public static List<Op> metaClientOpToZk(Iterable<org.apache.helix.metaclient.api.Op> ops) {

Review Comment:
   neat: If it's gonna return list of ops then should the name of the method be metaClientOpsToZkOps? or anything which suggest plural.



##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/util/ZkMetaClientUtil.java:
##########
@@ -0,0 +1,201 @@
+package org.apache.helix.metaclient.impl.zk.util;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.api.OpResult;
+import org.apache.helix.metaclient.constants.*;
+import org.apache.helix.zookeeper.zkclient.exception.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.EphemeralType;
+
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.function.Function;
+
+public class ZkMetaClientUtil {
+  //Default ACL value until metaClient Op has ACL of its own.
+  private static final List<ACL> DEFAULT_ACL = Collections.unmodifiableList(ZooDefs.Ids.OPEN_ACL_UNSAFE);
+
+  private ZkMetaClientUtil(){
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public static List<Op> metaClientOpToZk(Iterable<org.apache.helix.metaclient.api.Op> ops) {
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.CREATE, op -> {

Review Comment:
   Can we avoid having full name of org.apache.helix.metaclient.api.Op.Type here? i see that at multiple places down below.



##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/util/ZkMetaClientUtil.java:
##########
@@ -0,0 +1,201 @@
+package org.apache.helix.metaclient.impl.zk.util;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.api.OpResult;
+import org.apache.helix.metaclient.constants.*;
+import org.apache.helix.zookeeper.zkclient.exception.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.EphemeralType;
+
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.function.Function;
+
+public class ZkMetaClientUtil {
+  //Default ACL value until metaClient Op has ACL of its own.
+  private static final List<ACL> DEFAULT_ACL = Collections.unmodifiableList(ZooDefs.Ids.OPEN_ACL_UNSAFE);
+
+  private ZkMetaClientUtil(){
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public static List<Op> metaClientOpToZk(Iterable<org.apache.helix.metaclient.api.Op> ops) {
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.CREATE, op -> {
+      try {
+        CreateMode mode = convertMetaClientMode(((org.apache.helix.metaclient.api.Op.Create) op).getEntryMode());
+        return Op.create(op.getPath(), ((org.apache.helix.metaclient.api.Op.Create) op).getData(), DEFAULT_ACL, mode);
+      } catch (KeeperException e) {
+        throw translateZkExceptionToMetaclientException(ZkException.create(e));
+      }
+    });
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.DELETE,
+        op -> Op.delete(op.getPath(), ((org.apache.helix.metaclient.api.Op.Delete) op).getVersion()));
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.SET,
+        op -> Op.setData(op.getPath(),
+            ((org.apache.helix.metaclient.api.Op.Set) op).getData(), ((org.apache.helix.metaclient.api.Op.Set) op).getVersion()));
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.CHECK,
+        op -> Op.check(op.getPath(), ((org.apache.helix.metaclient.api.Op.Check) op).getVersion()));
+
+    List<Op> zkOps = new ArrayList<>();
+    for (org.apache.helix.metaclient.api.Op op : ops) {
+      Function<org.apache.helix.metaclient.api.Op, Op> function = getOpMap().get(op.getType());
+      if (function != null) {
+        zkOps.add(function.apply(op));
+      } else {
+        throw new IllegalArgumentException("Op type " + op.getType().name() + "is not supported.");
+      }
+    }
+    return zkOps;
+  }
+
+  private static final class OpmapHolder {
+    static final Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> OPMAP =
+        new EnumMap<>(org.apache.helix.metaclient.api.Op.Type.class);
+  }
+
+  private static Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> getOpMap() {
+    return OpmapHolder.OPMAP;
+  }
+
+  private static CreateMode convertMetaClientMode(MetaClientInterface.EntryMode entryMode) throws KeeperException {
+    if (entryMode == MetaClientInterface.EntryMode.PERSISTENT) {

Review Comment:
   just wondering if making it to switch case would make it cleaner?



##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/util/ZkMetaClientUtil.java:
##########
@@ -0,0 +1,201 @@
+package org.apache.helix.metaclient.impl.zk.util;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.api.OpResult;
+import org.apache.helix.metaclient.constants.*;
+import org.apache.helix.zookeeper.zkclient.exception.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.EphemeralType;
+
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.function.Function;
+
+public class ZkMetaClientUtil {
+  //Default ACL value until metaClient Op has ACL of its own.
+  private static final List<ACL> DEFAULT_ACL = Collections.unmodifiableList(ZooDefs.Ids.OPEN_ACL_UNSAFE);
+
+  private ZkMetaClientUtil(){
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public static List<Op> metaClientOpToZk(Iterable<org.apache.helix.metaclient.api.Op> ops) {
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.CREATE, op -> {
+      try {
+        CreateMode mode = convertMetaClientMode(((org.apache.helix.metaclient.api.Op.Create) op).getEntryMode());
+        return Op.create(op.getPath(), ((org.apache.helix.metaclient.api.Op.Create) op).getData(), DEFAULT_ACL, mode);
+      } catch (KeeperException e) {
+        throw translateZkExceptionToMetaclientException(ZkException.create(e));
+      }
+    });
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.DELETE,
+        op -> Op.delete(op.getPath(), ((org.apache.helix.metaclient.api.Op.Delete) op).getVersion()));
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.SET,
+        op -> Op.setData(op.getPath(),
+            ((org.apache.helix.metaclient.api.Op.Set) op).getData(), ((org.apache.helix.metaclient.api.Op.Set) op).getVersion()));
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.CHECK,
+        op -> Op.check(op.getPath(), ((org.apache.helix.metaclient.api.Op.Check) op).getVersion()));
+
+    List<Op> zkOps = new ArrayList<>();
+    for (org.apache.helix.metaclient.api.Op op : ops) {
+      Function<org.apache.helix.metaclient.api.Op, Op> function = getOpMap().get(op.getType());
+      if (function != null) {
+        zkOps.add(function.apply(op));
+      } else {
+        throw new IllegalArgumentException("Op type " + op.getType().name() + "is not supported.");
+      }
+    }
+    return zkOps;
+  }
+
+  private static final class OpmapHolder {
+    static final Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> OPMAP =
+        new EnumMap<>(org.apache.helix.metaclient.api.Op.Type.class);
+  }
+
+  private static Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> getOpMap() {
+    return OpmapHolder.OPMAP;
+  }
+
+  private static CreateMode convertMetaClientMode(MetaClientInterface.EntryMode entryMode) throws KeeperException {
+    if (entryMode == MetaClientInterface.EntryMode.PERSISTENT) {
+      return CreateMode.fromFlag(0);

Review Comment:
   Should we use something verbose or informative than using hard coded values like 0, 1 or 4?



##########
meta-client/src/test/java/org/apache/helix/metaclient/impl/zk/TestZkMetaClient.java:
##########
@@ -39,8 +39,11 @@
 import org.apache.commons.lang3.NotImplementedException;
 import org.apache.helix.metaclient.api.DataChangeListener;
 import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientConfig;
+import org.apache.helix.metaclient.api.Op;
+import org.apache.helix.metaclient.api.OpResult;
 import org.apache.helix.zookeeper.zkclient.IDefaultNameSpace;
 import org.apache.helix.zookeeper.zkclient.ZkServer;
+import org.apache.zookeeper.*;

Review Comment:
   this would be huge import! Do we want everything from this package?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] Marcosrico commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/util/ZkMetaClientUtil.java:
##########
@@ -0,0 +1,208 @@
+package org.apache.helix.metaclient.impl.zk.util;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.api.OpResult;
+import org.apache.helix.metaclient.constants.*;
+import org.apache.helix.zookeeper.zkclient.exception.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.EphemeralType;
+
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.function.Function;
+
+public class ZkMetaClientUtil {
+  //TODO Implement MetaClient ACL
+  //Default ACL value until metaClient Op has ACL of its own.
+  private static final List<ACL> DEFAULT_ACL = Collections.unmodifiableList(ZooDefs.Ids.OPEN_ACL_UNSAFE);
+
+  private ZkMetaClientUtil(){
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public static List<Op> metaClientOpsToZkOps(Iterable<org.apache.helix.metaclient.api.Op> ops) {
+    initializeOpMap();
+    List<Op> zkOps = new ArrayList<>();
+    for (org.apache.helix.metaclient.api.Op op : ops) {
+      Function<org.apache.helix.metaclient.api.Op, Op> function = getOpMap().get(op.getType());
+      if (function != null) {
+        zkOps.add(function.apply(op));
+      } else {
+        throw new IllegalArgumentException("Op type " + op.getType().name() + " is not supported.");
+      }
+    }
+    return zkOps;
+  }
+
+  private static final class OpMapHolder {
+    static final Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> OPMAP =
+        new EnumMap<>(org.apache.helix.metaclient.api.Op.Type.class);
+  }
+
+  private static Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> getOpMap() {

Review Comment:
   Good question, the method is to lazily initialize the map. See 3.2 of https://www.baeldung.com/java-singleton-double-checked-locking for more detail.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] mgao0 commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/util/ZkMetaClientUtil.java:
##########
@@ -0,0 +1,208 @@
+package org.apache.helix.metaclient.impl.zk.util;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.api.OpResult;
+import org.apache.helix.metaclient.constants.*;
+import org.apache.helix.zookeeper.zkclient.exception.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.EphemeralType;
+
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.function.Function;
+
+public class ZkMetaClientUtil {
+  //TODO Implement MetaClient ACL
+  //Default ACL value until metaClient Op has ACL of its own.
+  private static final List<ACL> DEFAULT_ACL = Collections.unmodifiableList(ZooDefs.Ids.OPEN_ACL_UNSAFE);
+
+  private ZkMetaClientUtil(){
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public static List<Op> metaClientOpsToZkOps(Iterable<org.apache.helix.metaclient.api.Op> ops) {
+    initializeOpMap();
+    List<Op> zkOps = new ArrayList<>();
+    for (org.apache.helix.metaclient.api.Op op : ops) {
+      Function<org.apache.helix.metaclient.api.Op, Op> function = getOpMap().get(op.getType());
+      if (function != null) {
+        zkOps.add(function.apply(op));
+      } else {
+        throw new IllegalArgumentException("Op type " + op.getType().name() + " is not supported.");
+      }
+    }
+    return zkOps;
+  }
+
+  private static final class OpMapHolder {
+    static final Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> OPMAP =
+        new EnumMap<>(org.apache.helix.metaclient.api.Op.Type.class);
+  }
+
+  private static Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> getOpMap() {

Review Comment:
   Thanks for the reply. Now I understand that you are trying to do lazy initialization here. But let's say you use `OpMapHolder.OPMAP` instead of `getOpMap()`, it should work for the lazy instantiation as well? I'm also not sure if it's necessary to create a nested class in order to do lazy instantiation if the map itself is not something that's too costly.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] xyuanlu merged pull request #2343: ZkMetaclient - implementation of TransactionOp

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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] xyuanlu commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

Posted by GitBox <gi...@apache.org>.
xyuanlu commented on code in PR #2343:
URL: https://github.com/apache/helix/pull/2343#discussion_r1080686941


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/ZkMetaClient.java:
##########
@@ -414,4 +414,130 @@ private static EntryMode convertZkEntryMode(long ephemeralOwner) {
         throw new IllegalArgumentException(zkEphemeralType + " is not supported.");
     }
   }
+
+
+  @Override
+  public List<OpResult> transactionOP(Iterable<Op> iterable) throws KeeperException {
+    // Convert list of MetaClient Ops to Zk Ops
+    List<org.apache.zookeeper.Op> zkOps = metaClientOpToZk(iterable);
+    // Execute Zk transactional support
+    List<org.apache.zookeeper.OpResult> zkResult = _zkClient.multi(zkOps);
+    // Convert list of Zk OpResults to MetaClient OpResults
+    return zkOpResultToMetaClient(zkResult);
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public List<org.apache.zookeeper.Op> metaClientOpToZk(Iterable<Op> ops) throws KeeperException {
+    List<org.apache.zookeeper.Op> zkOps = new ArrayList<>();
+    org.apache.zookeeper.Op temp;
+    for (Op op : ops) {
+      switch (op.getType()) {
+        case CREATE:
+          int zkFlag = zkFlagFromEntryMode(((Op.Create) op).getEntryMode());
+          temp = org.apache.zookeeper.Op.create(
+              op.getPath(), ((Op.Create) op).getData(), DEFAULT_ACL, CreateMode.fromFlag(zkFlag));
+          break;
+        case DELETE:
+          temp = org.apache.zookeeper.Op.delete(
+              op.getPath(), ((Op.Delete) op).getVersion());
+          break;
+        case SET:
+          temp = org.apache.zookeeper.Op.setData(
+              op.getPath(), ((Op.Set) op).getData(), ((Op.Set) op).getVersion());
+          break;
+        case CHECK:
+          temp = org.apache.zookeeper.Op.check(
+              op.getPath(), ((Op.Check) op).getVersion());
+          break;
+        default:
+          throw new IllegalArgumentException(op.getType() + " is not supported.");
+      }
+      zkOps.add(temp);
+    }
+    return zkOps;
+  }
+
+  /**
+   * Helper function for transactionOP. Converts the result from calling zk transactional support into
+   * metaclient OpResults.
+   * @param zkResult
+   * @return
+   */
+  public List<OpResult> zkOpResultToMetaClient(List<org.apache.zookeeper.OpResult> zkResult) throws KeeperException.BadArgumentsException {
+    List<OpResult> metaClientOpResult = new ArrayList<>();
+    OpResult temp;
+    EntryMode zkMode;
+    for (org.apache.zookeeper.OpResult opResult : zkResult) {
+      Stat metaClientStat;
+      switch (opResult.getType()) {
+        // CreateResult
+        case 1:

Review Comment:
   Please try to avoid magic number. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] Marcosrico commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/util/ZkMetaClientUtil.java:
##########
@@ -0,0 +1,213 @@
+package org.apache.helix.metaclient.impl.zk.util;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.api.OpResult;
+import org.apache.helix.metaclient.constants.*;
+import org.apache.helix.zookeeper.zkclient.exception.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.EphemeralType;
+
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.function.Function;
+
+public class ZkMetaClientUtil {
+  //TODO Implement MetaClient ACL
+  //Default ACL value until metaClient Op has ACL of its own.
+  private static final List<ACL> DEFAULT_ACL = Collections.unmodifiableList(ZooDefs.Ids.OPEN_ACL_UNSAFE);
+
+  private ZkMetaClientUtil(){
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public static List<Op> metaClientOpsToZkOps(Iterable<org.apache.helix.metaclient.api.Op> ops) {
+    if (getOpMap().isEmpty()) {
+      initializeOpMap();
+    }
+
+    List<Op> zkOps = new ArrayList<>();
+    for (org.apache.helix.metaclient.api.Op op : ops) {
+      Function<org.apache.helix.metaclient.api.Op, Op> function = getOpMap().get(op.getType());
+      if (function != null) {
+        zkOps.add(function.apply(op));
+      } else {
+        throw new IllegalArgumentException("Op type " + op.getType().name() + " is not supported.");
+      }
+    }
+    return zkOps;
+  }
+
+  private static final class OpMapHolder {
+    static final Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> OPMAP =
+        new EnumMap<>(org.apache.helix.metaclient.api.Op.Type.class);
+  }
+
+  private static Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> getOpMap() {
+    return OpMapHolder.OPMAP;
+  }
+
+  private static void initializeOpMap() {
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.CREATE, op -> {
+      try {
+        CreateMode mode = convertMetaClientMode(((org.apache.helix.metaclient.api.Op.Create) op).getEntryMode());
+        return Op.create(op.getPath(), ((org.apache.helix.metaclient.api.Op.Create) op).getData(), DEFAULT_ACL, mode);
+      } catch (KeeperException e) {
+        throw translateZkExceptionToMetaclientException(ZkException.create(e));
+      }
+    });
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.DELETE,
+        op -> Op.delete(op.getPath(), ((org.apache.helix.metaclient.api.Op.Delete) op).getVersion()));
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.SET,
+        op -> Op.setData(op.getPath(),
+            ((org.apache.helix.metaclient.api.Op.Set) op).getData(), ((org.apache.helix.metaclient.api.Op.Set) op).getVersion()));
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.CHECK,
+        op -> Op.check(op.getPath(), ((org.apache.helix.metaclient.api.Op.Check) op).getVersion()));
+  }
+
+  private static CreateMode convertMetaClientMode(MetaClientInterface.EntryMode entryMode) throws KeeperException {
+    switch (entryMode) {
+      case PERSISTENT:
+        return CreateMode.PERSISTENT;
+      case EPHEMERAL:
+        return CreateMode.EPHEMERAL;
+      case CONTAINER:
+        return CreateMode.CONTAINER;
+      default:
+        throw new IllegalArgumentException(entryMode.name() + " is not a supported EntryMode.");
+    }
+  }
+
+  /**
+   * Helper function for transactionOP. Converts the result from calling zk transactional support into
+   * metaclient OpResults.
+   * @param zkResult
+   * @return
+   */
+  public static List<OpResult> zkOpResultToMetaClientOpResults(List<org.apache.zookeeper.OpResult> zkResult) {
+    if (getOpResultMap().isEmpty()) {
+      initializeOpResultMap();
+    }
+    List<OpResult> metaClientOpResult = new ArrayList<>();
+    for (org.apache.zookeeper.OpResult opResult : zkResult) {
+      Function<org.apache.zookeeper.OpResult, OpResult> function = getOpResultMap().get(opResult.getClass());
+      if (function != null) {
+        metaClientOpResult.add(function.apply(opResult));
+      } else {
+        throw new IllegalArgumentException("OpResult type " + opResult.getType() + "is not supported.");
+      }
+    }
+
+    return metaClientOpResult;
+  }
+
+  private static final class OpResultMapHolder {
+    static final Map<Class<? extends org.apache.zookeeper.OpResult>, Function<org.apache.zookeeper.OpResult, OpResult>> OPRESULTMAP =
+        new HashMap<>();
+  }

Review Comment:
   I removed the unnecessary null check before each "op translate" as initializing the map guarantees non null value of opMap and opResultMap. Let me know if that addresses the comment, thanks!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] qqu0127 commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/util/ZkMetaClientUtil.java:
##########
@@ -0,0 +1,213 @@
+package org.apache.helix.metaclient.impl.zk.util;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.api.OpResult;
+import org.apache.helix.metaclient.constants.*;
+import org.apache.helix.zookeeper.zkclient.exception.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.EphemeralType;
+
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.function.Function;
+
+public class ZkMetaClientUtil {
+  //TODO Implement MetaClient ACL
+  //Default ACL value until metaClient Op has ACL of its own.
+  private static final List<ACL> DEFAULT_ACL = Collections.unmodifiableList(ZooDefs.Ids.OPEN_ACL_UNSAFE);
+
+  private ZkMetaClientUtil(){
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public static List<Op> metaClientOpsToZkOps(Iterable<org.apache.helix.metaclient.api.Op> ops) {
+    if (getOpMap().isEmpty()) {
+      initializeOpMap();
+    }
+
+    List<Op> zkOps = new ArrayList<>();
+    for (org.apache.helix.metaclient.api.Op op : ops) {
+      Function<org.apache.helix.metaclient.api.Op, Op> function = getOpMap().get(op.getType());
+      if (function != null) {
+        zkOps.add(function.apply(op));
+      } else {
+        throw new IllegalArgumentException("Op type " + op.getType().name() + " is not supported.");
+      }
+    }
+    return zkOps;
+  }
+
+  private static final class OpMapHolder {
+    static final Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> OPMAP =
+        new EnumMap<>(org.apache.helix.metaclient.api.Op.Type.class);
+  }
+
+  private static Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> getOpMap() {
+    return OpMapHolder.OPMAP;
+  }
+
+  private static void initializeOpMap() {
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.CREATE, op -> {
+      try {
+        CreateMode mode = convertMetaClientMode(((org.apache.helix.metaclient.api.Op.Create) op).getEntryMode());
+        return Op.create(op.getPath(), ((org.apache.helix.metaclient.api.Op.Create) op).getData(), DEFAULT_ACL, mode);
+      } catch (KeeperException e) {
+        throw translateZkExceptionToMetaclientException(ZkException.create(e));
+      }
+    });
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.DELETE,
+        op -> Op.delete(op.getPath(), ((org.apache.helix.metaclient.api.Op.Delete) op).getVersion()));
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.SET,
+        op -> Op.setData(op.getPath(),
+            ((org.apache.helix.metaclient.api.Op.Set) op).getData(), ((org.apache.helix.metaclient.api.Op.Set) op).getVersion()));
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.CHECK,
+        op -> Op.check(op.getPath(), ((org.apache.helix.metaclient.api.Op.Check) op).getVersion()));
+  }
+
+  private static CreateMode convertMetaClientMode(MetaClientInterface.EntryMode entryMode) throws KeeperException {
+    switch (entryMode) {
+      case PERSISTENT:
+        return CreateMode.PERSISTENT;
+      case EPHEMERAL:
+        return CreateMode.EPHEMERAL;
+      case CONTAINER:
+        return CreateMode.CONTAINER;
+      default:
+        throw new IllegalArgumentException(entryMode.name() + " is not a supported EntryMode.");
+    }
+  }
+
+  /**
+   * Helper function for transactionOP. Converts the result from calling zk transactional support into
+   * metaclient OpResults.
+   * @param zkResult
+   * @return
+   */
+  public static List<OpResult> zkOpResultToMetaClientOpResults(List<org.apache.zookeeper.OpResult> zkResult) {
+    if (getOpResultMap().isEmpty()) {
+      initializeOpResultMap();
+    }
+    List<OpResult> metaClientOpResult = new ArrayList<>();
+    for (org.apache.zookeeper.OpResult opResult : zkResult) {
+      Function<org.apache.zookeeper.OpResult, OpResult> function = getOpResultMap().get(opResult.getClass());
+      if (function != null) {
+        metaClientOpResult.add(function.apply(opResult));
+      } else {
+        throw new IllegalArgumentException("OpResult type " + opResult.getType() + "is not supported.");
+      }
+    }
+
+    return metaClientOpResult;
+  }
+
+  private static final class OpResultMapHolder {
+    static final Map<Class<? extends org.apache.zookeeper.OpResult>, Function<org.apache.zookeeper.OpResult, OpResult>> OPRESULTMAP =
+        new HashMap<>();
+  }

Review Comment:
   Has this been addressed? Why initializeOpResultMap is not invoked for OPRESULTMAP?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] Marcosrico commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

Posted by GitBox <gi...@apache.org>.
Marcosrico commented on code in PR #2343:
URL: https://github.com/apache/helix/pull/2343#discussion_r1081630325


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/ZkMetaClient.java:
##########
@@ -50,11 +51,15 @@
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.server.EphemeralType;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
 
 
 public class ZkMetaClient<T> implements MetaClientInterface<T>, AutoCloseable {
 
   private final ZkClient _zkClient;
+  //Default ACL value until metaClient Op has ACL of its own.
+  private final List<ACL> DEFAULT_ACL = ZooDefs.Ids.OPEN_ACL_UNSAFE;

Review Comment:
   Correct, I'll change it. Thanks!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] desaikomal commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

Posted by GitBox <gi...@apache.org>.
desaikomal commented on code in PR #2343:
URL: https://github.com/apache/helix/pull/2343#discussion_r1080665623


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/ZkMetaClient.java:
##########
@@ -414,4 +414,130 @@ private static EntryMode convertZkEntryMode(long ephemeralOwner) {
         throw new IllegalArgumentException(zkEphemeralType + " is not supported.");
     }
   }
+
+
+  @Override
+  public List<OpResult> transactionOP(Iterable<Op> iterable) throws KeeperException {
+    // Convert list of MetaClient Ops to Zk Ops
+    List<org.apache.zookeeper.Op> zkOps = metaClientOpToZk(iterable);
+    // Execute Zk transactional support
+    List<org.apache.zookeeper.OpResult> zkResult = _zkClient.multi(zkOps);
+    // Convert list of Zk OpResults to MetaClient OpResults
+    return zkOpResultToMetaClient(zkResult);
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public List<org.apache.zookeeper.Op> metaClientOpToZk(Iterable<Op> ops) throws KeeperException {
+    List<org.apache.zookeeper.Op> zkOps = new ArrayList<>();
+    org.apache.zookeeper.Op temp;
+    for (Op op : ops) {
+      switch (op.getType()) {
+        case CREATE:
+          int zkFlag = zkFlagFromEntryMode(((Op.Create) op).getEntryMode());

Review Comment:
   this seems like double time evaluation. You want to pass CreateMode type. Can you not have a simple enum which you can pass directly? You are anyway, hardcoding to Op.Create, so why not pass that enum directly. We don't want to create code complexity if passing something directly saves CPU cycles. 



##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/ZkMetaClient.java:
##########
@@ -414,4 +414,130 @@ private static EntryMode convertZkEntryMode(long ephemeralOwner) {
         throw new IllegalArgumentException(zkEphemeralType + " is not supported.");
     }
   }
+
+
+  @Override
+  public List<OpResult> transactionOP(Iterable<Op> iterable) throws KeeperException {
+    // Convert list of MetaClient Ops to Zk Ops
+    List<org.apache.zookeeper.Op> zkOps = metaClientOpToZk(iterable);
+    // Execute Zk transactional support
+    List<org.apache.zookeeper.OpResult> zkResult = _zkClient.multi(zkOps);
+    // Convert list of Zk OpResults to MetaClient OpResults
+    return zkOpResultToMetaClient(zkResult);
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public List<org.apache.zookeeper.Op> metaClientOpToZk(Iterable<Op> ops) throws KeeperException {
+    List<org.apache.zookeeper.Op> zkOps = new ArrayList<>();
+    org.apache.zookeeper.Op temp;
+    for (Op op : ops) {
+      switch (op.getType()) {
+        case CREATE:
+          int zkFlag = zkFlagFromEntryMode(((Op.Create) op).getEntryMode());
+          temp = org.apache.zookeeper.Op.create(
+              op.getPath(), ((Op.Create) op).getData(), DEFAULT_ACL, CreateMode.fromFlag(zkFlag));
+          break;
+        case DELETE:
+          temp = org.apache.zookeeper.Op.delete(
+              op.getPath(), ((Op.Delete) op).getVersion());
+          break;
+        case SET:
+          temp = org.apache.zookeeper.Op.setData(
+              op.getPath(), ((Op.Set) op).getData(), ((Op.Set) op).getVersion());
+          break;
+        case CHECK:
+          temp = org.apache.zookeeper.Op.check(
+              op.getPath(), ((Op.Check) op).getVersion());
+          break;
+        default:
+          throw new IllegalArgumentException(op.getType() + " is not supported.");
+      }
+      zkOps.add(temp);
+    }
+    return zkOps;
+  }
+
+  /**
+   * Helper function for transactionOP. Converts the result from calling zk transactional support into
+   * metaclient OpResults.
+   * @param zkResult
+   * @return
+   */
+  public List<OpResult> zkOpResultToMetaClient(List<org.apache.zookeeper.OpResult> zkResult) throws KeeperException.BadArgumentsException {

Review Comment:
   ditto.



##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/ZkMetaClient.java:
##########
@@ -414,4 +414,130 @@ private static EntryMode convertZkEntryMode(long ephemeralOwner) {
         throw new IllegalArgumentException(zkEphemeralType + " is not supported.");
     }
   }
+
+
+  @Override
+  public List<OpResult> transactionOP(Iterable<Op> iterable) throws KeeperException {
+    // Convert list of MetaClient Ops to Zk Ops
+    List<org.apache.zookeeper.Op> zkOps = metaClientOpToZk(iterable);
+    // Execute Zk transactional support
+    List<org.apache.zookeeper.OpResult> zkResult = _zkClient.multi(zkOps);
+    // Convert list of Zk OpResults to MetaClient OpResults
+    return zkOpResultToMetaClient(zkResult);
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public List<org.apache.zookeeper.Op> metaClientOpToZk(Iterable<Op> ops) throws KeeperException {

Review Comment:
   you changed exception from KeeperException to IllegalArgument.
   
   so is this still required?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] Marcosrico commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

Posted by GitBox <gi...@apache.org>.
Marcosrico commented on code in PR #2343:
URL: https://github.com/apache/helix/pull/2343#discussion_r1081675851


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/ZkMetaClient.java:
##########
@@ -50,11 +51,15 @@
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.server.EphemeralType;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
 
 
 public class ZkMetaClient<T> implements MetaClientInterface<T>, AutoCloseable {
 
   private final ZkClient _zkClient;
+  //Default ACL value until metaClient Op has ACL of its own.
+  private final List<ACL> DEFAULT_ACL = ZooDefs.Ids.OPEN_ACL_UNSAFE;

Review Comment:
   `ZooDefs.Ids.OPEN_ACL_UNSAFE` is an arraylist. To make it immutable I can call `Collections.unmodifiableList()` though that seems a bit convoluted. Would making it static and final not be sufficient?



##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/ZkMetaClient.java:
##########
@@ -50,11 +51,15 @@
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.server.EphemeralType;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
 
 
 public class ZkMetaClient<T> implements MetaClientInterface<T>, AutoCloseable {
 
   private final ZkClient _zkClient;
+  //Default ACL value until metaClient Op has ACL of its own.
+  private final List<ACL> DEFAULT_ACL = ZooDefs.Ids.OPEN_ACL_UNSAFE;

Review Comment:
   Correct, I'll change it. Thanks!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] Marcosrico commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

Posted by GitBox <gi...@apache.org>.
Marcosrico commented on code in PR #2343:
URL: https://github.com/apache/helix/pull/2343#discussion_r1081684126


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/ZkMetaClient.java:
##########
@@ -414,4 +414,130 @@ private static EntryMode convertZkEntryMode(long ephemeralOwner) {
         throw new IllegalArgumentException(zkEphemeralType + " is not supported.");
     }
   }
+
+
+  @Override
+  public List<OpResult> transactionOP(Iterable<Op> iterable) throws KeeperException {
+    // Convert list of MetaClient Ops to Zk Ops
+    List<org.apache.zookeeper.Op> zkOps = metaClientOpToZk(iterable);
+    // Execute Zk transactional support
+    List<org.apache.zookeeper.OpResult> zkResult = _zkClient.multi(zkOps);
+    // Convert list of Zk OpResults to MetaClient OpResults
+    return zkOpResultToMetaClient(zkResult);
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public List<org.apache.zookeeper.Op> metaClientOpToZk(Iterable<Op> ops) throws KeeperException {

Review Comment:
   Now that I am moving them to a separate util class, I'm assuming they must be. Please let me know if I'm wrong



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] Marcosrico commented on pull request #2343: ZkMetaclient - implementation of TransactionOp

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

   This PR is ready to merge, approved by @qqu0127 
   Commit message:
   ZkMetaClient - Implementation of TransactionOp


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] Marcosrico commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

Posted by GitBox <gi...@apache.org>.
Marcosrico commented on code in PR #2343:
URL: https://github.com/apache/helix/pull/2343#discussion_r1082808508


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/util/ZkMetaClientUtil.java:
##########
@@ -0,0 +1,156 @@
+package org.apache.helix.metaclient.impl.zk.util;
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.api.OpResult;
+import org.apache.helix.metaclient.constants.*;
+import org.apache.helix.zookeeper.zkclient.exception.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.EphemeralType;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.function.Function;
+
+public class ZkMetaClientUtil {
+  //Default ACL value until metaClient Op has ACL of its own.
+  private static final List<ACL> DEFAULT_ACL = Collections.unmodifiableList(ZooDefs.Ids.OPEN_ACL_UNSAFE);
+
+  private ZkMetaClientUtil(){
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public static List<Op> metaClientOpToZk(Iterable<org.apache.helix.metaclient.api.Op> ops) {
+    Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> opMap = new HashMap<>();
+
+    opMap.put(org.apache.helix.metaclient.api.Op.Type.CREATE, op -> {
+      try {
+        CreateMode mode = convertMetaClientMode(((org.apache.helix.metaclient.api.Op.Create) op).getEntryMode());
+        return Op.create(op.getPath(), ((org.apache.helix.metaclient.api.Op.Create) op).getData(), DEFAULT_ACL, mode);
+      } catch (KeeperException e) {
+        throw translateZkExceptionToMetaclientException(ZkException.create(e));
+      }
+    });
+
+    opMap.put(org.apache.helix.metaclient.api.Op.Type.DELETE,
+        op -> Op.delete(op.getPath(), ((org.apache.helix.metaclient.api.Op.Delete) op).getVersion()));
+
+    opMap.put(org.apache.helix.metaclient.api.Op.Type.SET,
+        op -> Op.setData(op.getPath(),
+            ((org.apache.helix.metaclient.api.Op.Set) op).getData(), ((org.apache.helix.metaclient.api.Op.Set) op).getVersion()));
+
+    opMap.put(org.apache.helix.metaclient.api.Op.Type.CHECK,
+        op -> Op.check(op.getPath(), ((org.apache.helix.metaclient.api.Op.Check) op).getVersion()));
+
+    List<Op> zkOps = new ArrayList<>();
+    for (org.apache.helix.metaclient.api.Op op : ops) {
+      Op temp = opMap.get(op.getType()).apply(op);
+      zkOps.add(temp);
+    }
+    return zkOps;
+  }
+
+  private static CreateMode convertMetaClientMode(MetaClientInterface.EntryMode entryMode) throws KeeperException {
+    String mode = entryMode.name();
+    if (mode.equals(MetaClientInterface.EntryMode.PERSISTENT.name())) {
+      return CreateMode.fromFlag(0);
+    }
+    if (mode.equals(MetaClientInterface.EntryMode.EPHEMERAL.name())) {
+      return CreateMode.fromFlag(1);
+    }
+    return CreateMode.fromFlag(-1);

Review Comment:
   Yes that's correct and sounds 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.

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

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


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


[GitHub] [helix] Marcosrico commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/ZkMetaClient.java:
##########
@@ -391,34 +380,13 @@ public int hashCode() {
     }
   }
 
-  private static MetaClientException translateZkExceptionToMetaclientException(ZkException e) {
-    if (e instanceof ZkNodeExistsException) {
-      return new MetaClientNoNodeException(e);
-    } else if (e instanceof ZkBadVersionException) {
-      return new MetaClientBadVersionException(e);
-    } else if (e instanceof ZkTimeoutException) {
-      return new MetaClientTimeoutException(e);
-    } else if (e instanceof ZkInterruptedException) {
-      return new MetaClientInterruptException(e);
-    } else {
-      return new MetaClientException(e);
-    }
-  }
-
-  private static EntryMode convertZkEntryMode(long ephemeralOwner) {
-    EphemeralType zkEphemeralType = EphemeralType.get(ephemeralOwner);
-    switch (zkEphemeralType) {
-      case VOID:
-        return EntryMode.PERSISTENT;
-      case CONTAINER:
-        return EntryMode.CONTAINER;
-      case NORMAL:
-        return EntryMode.EPHEMERAL;
-      // TODO: TTL is not supported now.
-      //case TTL:
-      //  return EntryMode.TTL;
-      default:
-        throw new IllegalArgumentException(zkEphemeralType + " is not supported.");
-    }
+  @Override
+  public List<OpResult> transactionOP(Iterable<Op> iterable) {

Review Comment:
   Good catch. The metaclient interface has it as ops as well so it only makes sense to be consistent with the naming too.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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


[GitHub] [helix] qqu0127 commented on a diff in pull request #2343: ZkMetaclient - implementation of TransactionOp

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/util/ZkMetaClientUtil.java:
##########
@@ -0,0 +1,201 @@
+package org.apache.helix.metaclient.impl.zk.util;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import org.apache.helix.metaclient.api.MetaClientInterface;
+import org.apache.helix.metaclient.api.OpResult;
+import org.apache.helix.metaclient.constants.*;
+import org.apache.helix.zookeeper.zkclient.exception.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.EphemeralType;
+
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.function.Function;
+
+public class ZkMetaClientUtil {
+  //Default ACL value until metaClient Op has ACL of its own.
+  private static final List<ACL> DEFAULT_ACL = Collections.unmodifiableList(ZooDefs.Ids.OPEN_ACL_UNSAFE);
+
+  private ZkMetaClientUtil(){
+  }
+
+  /**
+   * Helper function for transactionOp. Converts MetaClient Op's into Zk Ops to execute
+   * zk transactional support.
+   * @param ops
+   * @return
+   */
+  public static List<Op> metaClientOpToZk(Iterable<org.apache.helix.metaclient.api.Op> ops) {
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.CREATE, op -> {
+      try {
+        CreateMode mode = convertMetaClientMode(((org.apache.helix.metaclient.api.Op.Create) op).getEntryMode());
+        return Op.create(op.getPath(), ((org.apache.helix.metaclient.api.Op.Create) op).getData(), DEFAULT_ACL, mode);
+      } catch (KeeperException e) {
+        throw translateZkExceptionToMetaclientException(ZkException.create(e));
+      }
+    });
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.DELETE,
+        op -> Op.delete(op.getPath(), ((org.apache.helix.metaclient.api.Op.Delete) op).getVersion()));
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.SET,
+        op -> Op.setData(op.getPath(),
+            ((org.apache.helix.metaclient.api.Op.Set) op).getData(), ((org.apache.helix.metaclient.api.Op.Set) op).getVersion()));
+
+    getOpMap().put(org.apache.helix.metaclient.api.Op.Type.CHECK,
+        op -> Op.check(op.getPath(), ((org.apache.helix.metaclient.api.Op.Check) op).getVersion()));
+
+    List<Op> zkOps = new ArrayList<>();
+    for (org.apache.helix.metaclient.api.Op op : ops) {
+      Function<org.apache.helix.metaclient.api.Op, Op> function = getOpMap().get(op.getType());
+      if (function != null) {
+        zkOps.add(function.apply(op));
+      } else {
+        throw new IllegalArgumentException("Op type " + op.getType().name() + "is not supported.");
+      }
+    }
+    return zkOps;
+  }
+
+  private static final class OpmapHolder {
+    static final Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> OPMAP =
+        new EnumMap<>(org.apache.helix.metaclient.api.Op.Type.class);
+  }
+
+  private static Map<org.apache.helix.metaclient.api.Op.Type, Function<org.apache.helix.metaclient.api.Op, Op>> getOpMap() {
+    return OpmapHolder.OPMAP;
+  }
+
+  private static CreateMode convertMetaClientMode(MetaClientInterface.EntryMode entryMode) throws KeeperException {
+    if (entryMode == MetaClientInterface.EntryMode.PERSISTENT) {
+      return CreateMode.fromFlag(0);

Review Comment:
   +1 to Rahul's comment, I think we can just clearly type to return CreateMode.PERSISTENT or CreateMode.EPHEMERAL etc.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


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