You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@helix.apache.org by "Marcosrico (via GitHub)" <gi...@apache.org> on 2023/04/20 21:10:13 UTC

[GitHub] [helix] Marcosrico opened a new pull request, #2457: [WIP] Lattice LockClient Implementation

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

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


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

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

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


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


[GitHub] [helix] qqu0127 commented on a diff in pull request #2457: Lattice LockClient Implementation

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/lock/LockClient.java:
##########
@@ -0,0 +1,107 @@
+package org.apache.helix.metaclient.recipes.lock;
+
+/*
+ * 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.Op;
+import org.apache.helix.metaclient.datamodel.DataRecord;
+import org.apache.helix.metaclient.exception.MetaClientException;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+import org.apache.helix.metaclient.factories.MetaClientFactory;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientConfig;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientFactory;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class LockClient implements LockClientInterface, AutoCloseable {
+  private final MetaClientInterface<LockInfo> _metaClient;
+
+  public LockClient(MetaClientConfig config) {
+    if (!config.getStoreType().equals(MetaClientConfig.StoreType.ZOOKEEPER)) {
+      throw new MetaClientException("Unsupported store type: " + config.getStoreType());
+    }
+    ZkMetaClientConfig zkMetaClientConfig = new ZkMetaClientConfig.ZkMetaClientConfigBuilder().
+        setConnectionAddress(config.getConnectionAddress())
+        // Currently only support ZNRecordSerializer. TODO: make this configurable
+        .setZkSerializer((new ZNRecordSerializer()))
+        .build();
+    _metaClient = new ZkMetaClientFactory().getMetaClient(zkMetaClientConfig);
+    _metaClient.connect();
+  }
+
+  public LockClient(MetaClientInterface<LockInfo> client) {
+    _metaClient = client;
+    _metaClient.connect();
+  }
+
+  @Override
+  public void acquireLock(String key, LockInfo lockInfo, MetaClientInterface.EntryMode mode) {
+    _metaClient.create(key, lockInfo, mode);
+  }
+
+  @Override
+  public void acquireLockWithTTL(String key, LockInfo lockInfo, long ttl) {
+    _metaClient.createWithTTL(key, lockInfo, ttl);
+  }
+
+  @Override
+  public void renewTTLLock(String key) {
+    _metaClient.renewTTLNode(key);
+  }
+
+  @Override
+  public void releaseLock(String key) {
+    MetaClientInterface.Stat stat = _metaClient.exists(key);
+    if (stat != null) {
+      int version = stat.getVersion();
+      List<Op> ops = Arrays.asList(
+          Op.check(key, version),
+          Op.delete(key, version));
+      _metaClient.transactionOP(ops);
+      if (_metaClient.exists(key) != null) {
+        throw new MetaClientException("Failed to release lock for key: " + key);
+      }
+    }
+  }
+
+  @Override
+  public LockInfo retrieveLock(String key) {
+    MetaClientInterface.Stat stat = _metaClient.exists(key);
+    if (stat == null) {
+      return null;
+    }
+    //Create a new DataRecord from underlying record
+    DataRecord dataRecord = new DataRecord(_metaClient.get(key));
+    //Create a new LockInfo from DataRecord
+    LockInfo lockInfo = new LockInfo(dataRecord);
+    //Synchronize the lockInfo with the stat
+    lockInfo.setGrantedAt(stat.getCreationTime());
+    lockInfo.setLastRenewedAt(stat.getModifiedTime());

Review Comment:
   I was thinking of creating a new constructor for LockInfo, with `stat` as a param. That way we don't need to have these two setters. And if we have similar logic elsewhere, it reduces duplication.



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

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

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


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


[GitHub] [helix] junkaixue commented on a diff in pull request #2457: Lattice LockClient Implementation

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/lock/LockClient.java:
##########
@@ -0,0 +1,112 @@
+package org.apache.helix.metaclient.recipes.lock;
+
+/*
+ * 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.Op;
+import org.apache.helix.metaclient.datamodel.DataRecord;
+import org.apache.helix.metaclient.exception.MetaClientException;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+import org.apache.helix.metaclient.factories.MetaClientFactory;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientConfig;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientFactory;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class LockClient implements LockClientInterface, AutoCloseable {
+  private final MetaClientInterface<LockInfo> _metaClient;
+
+  public LockClient(MetaClientConfig config) {
+    if (config.getStoreType() == MetaClientConfig.StoreType.ZOOKEEPER) {

Review Comment:
   Yeah. Good deep analysis. In usual pattern Enum equals is calling the "==", but for some overrides enums, especially it has internal property ones, equals can be override. When we do coding, it would be good not making assumption of it is native Enum class.
   
   The comment is more leaning to let Marcos learn some Java coding conventions. 



-- 
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 #2457: Lattice LockClient Implementation

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/lock/LockClient.java:
##########
@@ -0,0 +1,107 @@
+package org.apache.helix.metaclient.recipes.lock;
+
+/*
+ * 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.Op;
+import org.apache.helix.metaclient.datamodel.DataRecord;
+import org.apache.helix.metaclient.exception.MetaClientException;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+import org.apache.helix.metaclient.factories.MetaClientFactory;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientConfig;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientFactory;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class LockClient implements LockClientInterface, AutoCloseable {
+  private final MetaClientInterface<LockInfo> _metaClient;
+
+  public LockClient(MetaClientConfig config) {
+    if (!config.getStoreType().equals(MetaClientConfig.StoreType.ZOOKEEPER)) {
+      throw new MetaClientException("Unsupported store type: " + config.getStoreType());
+    }
+    ZkMetaClientConfig zkMetaClientConfig = new ZkMetaClientConfig.ZkMetaClientConfigBuilder().
+        setConnectionAddress(config.getConnectionAddress())
+        // Currently only support ZNRecordSerializer. TODO: make this configurable
+        .setZkSerializer((new ZNRecordSerializer()))
+        .build();
+    _metaClient = new ZkMetaClientFactory().getMetaClient(zkMetaClientConfig);
+    _metaClient.connect();
+  }
+
+  public LockClient(MetaClientInterface<LockInfo> client) {
+    _metaClient = client;
+    _metaClient.connect();
+  }
+
+  @Override
+  public void acquireLock(String key, LockInfo lockInfo, MetaClientInterface.EntryMode mode) {
+    _metaClient.create(key, lockInfo, mode);
+  }
+
+  @Override
+  public void acquireLockWithTTL(String key, LockInfo lockInfo, long ttl) {
+    _metaClient.createWithTTL(key, lockInfo, ttl);
+  }
+
+  @Override
+  public void renewTTLLock(String key) {
+    _metaClient.renewTTLNode(key);
+  }
+
+  @Override
+  public void releaseLock(String key) {
+    MetaClientInterface.Stat stat = _metaClient.exists(key);
+    if (stat != null) {
+      int version = stat.getVersion();
+      List<Op> ops = Arrays.asList(
+          Op.check(key, version),
+          Op.delete(key, version));
+      _metaClient.transactionOP(ops);
+      if (_metaClient.exists(key) != null) {
+        throw new MetaClientException("Failed to release lock for key: " + key);
+      }
+    }
+  }
+
+  @Override
+  public LockInfo retrieveLock(String key) {
+    MetaClientInterface.Stat stat = _metaClient.exists(key);
+    if (stat == null) {
+      return null;
+    }
+    //Create a new DataRecord from underlying record
+    DataRecord dataRecord = new DataRecord(_metaClient.get(key));
+    //Create a new LockInfo from DataRecord
+    LockInfo lockInfo = new LockInfo(dataRecord);
+    //Synchronize the lockInfo with the stat
+    lockInfo.setGrantedAt(stat.getCreationTime());
+    lockInfo.setLastRenewedAt(stat.getModifiedTime());

Review Comment:
   That's a good idea, thank you!



-- 
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 #2457: Lattice LockClient Implementation

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/lock/LockClient.java:
##########
@@ -0,0 +1,107 @@
+package org.apache.helix.metaclient.recipes.lock;
+
+/*
+ * 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.Op;
+import org.apache.helix.metaclient.datamodel.DataRecord;
+import org.apache.helix.metaclient.exception.MetaClientException;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+import org.apache.helix.metaclient.factories.MetaClientFactory;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientConfig;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientFactory;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class LockClient implements LockClientInterface, AutoCloseable {
+  private final MetaClientInterface<LockInfo> _metaClient;
+
+  public LockClient(MetaClientConfig config) {
+    if (!config.getStoreType().equals(MetaClientConfig.StoreType.ZOOKEEPER)) {
+      throw new MetaClientException("Unsupported store type: " + config.getStoreType());
+    }
+    ZkMetaClientConfig zkMetaClientConfig = new ZkMetaClientConfig.ZkMetaClientConfigBuilder().
+        setConnectionAddress(config.getConnectionAddress())
+        // Currently only support ZNRecordSerializer. TODO: make this configurable
+        .setZkSerializer((new ZNRecordSerializer()))
+        .build();
+    _metaClient = new ZkMetaClientFactory().getMetaClient(zkMetaClientConfig);
+    _metaClient.connect();
+  }
+
+  public LockClient(MetaClientInterface<LockInfo> client) {
+    _metaClient = client;
+    _metaClient.connect();
+  }
+
+  @Override
+  public void acquireLock(String key, LockInfo lockInfo, MetaClientInterface.EntryMode mode) {
+    _metaClient.create(key, lockInfo, mode);
+  }
+
+  @Override
+  public void acquireLockWithTTL(String key, LockInfo lockInfo, long ttl) {
+    _metaClient.createWithTTL(key, lockInfo, ttl);
+  }
+
+  @Override
+  public void renewTTLLock(String key) {
+    _metaClient.renewTTLNode(key);
+  }
+
+  @Override
+  public void releaseLock(String key) {
+    MetaClientInterface.Stat stat = _metaClient.exists(key);
+    if (stat != null) {
+      int version = stat.getVersion();
+      List<Op> ops = Arrays.asList(
+          Op.check(key, version),
+          Op.delete(key, version));
+      _metaClient.transactionOP(ops);
+      if (_metaClient.exists(key) != null) {
+        throw new MetaClientException("Failed to release lock for key: " + key);
+      }
+    }
+  }
+
+  @Override
+  public LockInfo retrieveLock(String key) {
+    MetaClientInterface.Stat stat = _metaClient.exists(key);
+    if (stat == null) {
+      return null;
+    }
+    //Create a new DataRecord from underlying record
+    DataRecord dataRecord = new DataRecord(_metaClient.get(key));
+    //Create a new LockInfo from DataRecord
+    LockInfo lockInfo = new LockInfo(dataRecord);
+    //Synchronize the lockInfo with the stat
+    lockInfo.setGrantedAt(stat.getCreationTime());
+    lockInfo.setLastRenewedAt(stat.getModifiedTime());

Review Comment:
   If I understood this correctly, do you mean abstracting updating the lockInfo data in a separate method? Or when creating the LockInfo object?



-- 
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 #2457: Lattice LockClient Implementation

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/lock/LockClient.java:
##########
@@ -0,0 +1,112 @@
+package org.apache.helix.metaclient.recipes.lock;
+
+/*
+ * 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.Op;
+import org.apache.helix.metaclient.datamodel.DataRecord;
+import org.apache.helix.metaclient.exception.MetaClientException;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+import org.apache.helix.metaclient.factories.MetaClientFactory;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientConfig;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientFactory;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class LockClient implements LockClientInterface, AutoCloseable {
+  private final MetaClientInterface<LockInfo> _metaClient;
+
+  public LockClient(MetaClientConfig config) {
+    if (config.getStoreType() == MetaClientConfig.StoreType.ZOOKEEPER) {

Review Comment:
   @junkaixue I have the opposite opinion on Enum compare.
   Using `==` is completely fine, actually `Enum::equals()` directly calls to `==`, it avoid NPE 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] Marcosrico commented on a diff in pull request #2457: Lattice LockClient Implementation

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/lock/LockClient.java:
##########
@@ -0,0 +1,107 @@
+package org.apache.helix.metaclient.recipes.lock;
+
+/*
+ * 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.Op;
+import org.apache.helix.metaclient.datamodel.DataRecord;
+import org.apache.helix.metaclient.exception.MetaClientException;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+import org.apache.helix.metaclient.factories.MetaClientFactory;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientConfig;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientFactory;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class LockClient implements LockClientInterface, AutoCloseable {
+  private final MetaClientInterface<LockInfo> _metaClient;
+
+  public LockClient(MetaClientConfig config) {
+    if (!config.getStoreType().equals(MetaClientConfig.StoreType.ZOOKEEPER)) {
+      throw new MetaClientException("Unsupported store type: " + config.getStoreType());
+    }
+    ZkMetaClientConfig zkMetaClientConfig = new ZkMetaClientConfig.ZkMetaClientConfigBuilder().
+        setConnectionAddress(config.getConnectionAddress())
+        // Currently only support ZNRecordSerializer. TODO: make this configurable
+        .setZkSerializer((new ZNRecordSerializer()))
+        .build();
+    _metaClient = new ZkMetaClientFactory().getMetaClient(zkMetaClientConfig);
+    _metaClient.connect();
+  }
+
+  public LockClient(MetaClientInterface<LockInfo> client) {
+    _metaClient = client;
+    _metaClient.connect();
+  }
+
+  @Override
+  public void acquireLock(String key, LockInfo lockInfo, MetaClientInterface.EntryMode mode) {
+    _metaClient.create(key, lockInfo, mode);
+  }
+
+  @Override
+  public void acquireLockWithTTL(String key, LockInfo lockInfo, long ttl) {
+    _metaClient.createWithTTL(key, lockInfo, ttl);
+  }
+
+  @Override
+  public void renewTTLLock(String key) {
+    _metaClient.renewTTLNode(key);
+  }
+
+  @Override
+  public void releaseLock(String key) {
+    MetaClientInterface.Stat stat = _metaClient.exists(key);
+    if (stat != null) {
+      int version = stat.getVersion();
+      List<Op> ops = Arrays.asList(
+          Op.check(key, version),
+          Op.delete(key, version));
+      _metaClient.transactionOP(ops);
+      if (_metaClient.exists(key) != null) {
+        throw new MetaClientException("Failed to release lock for key: " + key);
+      }
+    }
+  }
+
+  @Override
+  public LockInfo retrieveLock(String key) {
+    MetaClientInterface.Stat stat = _metaClient.exists(key);
+    if (stat == null) {
+      return null;
+    }
+    //Create a new DataRecord from underlying record
+    DataRecord dataRecord = new DataRecord(_metaClient.get(key));
+    //Create a new LockInfo from DataRecord
+    LockInfo lockInfo = new LockInfo(dataRecord);
+    //Synchronize the lockInfo with the stat
+    lockInfo.setGrantedAt(stat.getCreationTime());
+    lockInfo.setLastRenewedAt(stat.getModifiedTime());

Review Comment:
   If I understood this correctly, do you mean abstracting updating the lockInfo data in a separate method? I can do that



-- 
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 #2457: Lattice LockClient Implementation

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/lock/LockClient.java:
##########
@@ -0,0 +1,108 @@
+package org.apache.helix.metaclient.recipes.lock;
+
+/*
+ * 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.Op;
+import org.apache.helix.metaclient.datamodel.DataRecord;
+import org.apache.helix.metaclient.exception.MetaClientException;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+import org.apache.helix.metaclient.factories.MetaClientFactory;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientConfig;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientFactory;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class LockClient implements LockClientInterface, AutoCloseable {
+  private final MetaClientInterface<LockInfo> _metaClient;
+
+  public LockClient(MetaClientConfig config) {
+    if (config.getStoreType().equals(MetaClientConfig.StoreType.ZOOKEEPER)) {
+      ZkMetaClientConfig zkMetaClientConfig = new ZkMetaClientConfig.ZkMetaClientConfigBuilder().
+          setConnectionAddress(config.getConnectionAddress())
+          // Currently only support ZNRecordSerializer. TODO: make this configurable
+          .setZkSerializer((new ZNRecordSerializer()))
+          .build();
+      _metaClient = new ZkMetaClientFactory().getMetaClient(zkMetaClientConfig);
+    } else {
+      throw new MetaClientException("Unsupported store type: " + config.getStoreType());
+    }
+    _metaClient.connect();
+  }
+
+  public LockClient(MetaClientInterface<LockInfo> client) {
+    _metaClient = client;
+    _metaClient.connect();
+  }
+
+  @Override
+  public void acquireLock(String key, LockInfo lockInfo, MetaClientInterface.EntryMode mode) {
+    _metaClient.create(key, lockInfo, mode);
+  }
+
+  @Override
+  public void acquireLockWithTTL(String key, LockInfo lockInfo, long ttl) {
+    _metaClient.createWithTTL(key, lockInfo, ttl);
+  }
+
+  @Override
+  public void renewTTLLock(String key) {
+    _metaClient.renewTTLNode(key);
+  }
+
+  @Override
+  public void releaseLock(String key) {
+    MetaClientInterface.Stat stat = _metaClient.exists(key);
+    if (stat != null) {
+      int version = stat.getVersion();
+      List<Op> ops = Arrays.asList(
+          Op.check(key, version),
+          Op.delete(key, version));
+      _metaClient.transactionOP(ops);
+      if (_metaClient.exists(key) != null) {
+        throw new MetaClientException("Failed to release lock for key: " + key);
+      }
+    }
+  }
+
+  @Override
+  public LockInfo retrieveLock(String key) {
+    MetaClientInterface.Stat stat = _metaClient.exists(key);
+    if (stat == null) {
+      return null;
+    }
+    //Create a new DataRecord from underlying record
+    DataRecord dataRecord = new DataRecord(_metaClient.get(key));
+    //Create a new LockInfo from DataRecord
+    LockInfo lockInfo = new LockInfo(dataRecord);
+    //Synchronize the lockInfo with the stat
+    lockInfo.setGrantedAt(stat.getCreationTime());
+    lockInfo.setLastRenewedAt(stat.getModifiedTime());
+    return lockInfo;
+  }
+
+  @Override
+  public void close() {
+    _metaClient.disconnect();

Review Comment:
   If user uses the constructor `public LockClient(MetaClientInterface<LockInfo> client)`, the metaclient shouldn't be closed, it will have external unexpected effect.



##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/lock/LockClient.java:
##########
@@ -0,0 +1,107 @@
+package org.apache.helix.metaclient.recipes.lock;
+
+/*
+ * 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.Op;
+import org.apache.helix.metaclient.datamodel.DataRecord;
+import org.apache.helix.metaclient.exception.MetaClientException;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+import org.apache.helix.metaclient.factories.MetaClientFactory;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientConfig;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientFactory;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class LockClient implements LockClientInterface, AutoCloseable {
+  private final MetaClientInterface<LockInfo> _metaClient;
+
+  public LockClient(MetaClientConfig config) {
+    if (!config.getStoreType().equals(MetaClientConfig.StoreType.ZOOKEEPER)) {
+      throw new MetaClientException("Unsupported store type: " + config.getStoreType());
+    }
+    ZkMetaClientConfig zkMetaClientConfig = new ZkMetaClientConfig.ZkMetaClientConfigBuilder().
+        setConnectionAddress(config.getConnectionAddress())
+        // Currently only support ZNRecordSerializer. TODO: make this configurable
+        .setZkSerializer((new ZNRecordSerializer()))
+        .build();
+    _metaClient = new ZkMetaClientFactory().getMetaClient(zkMetaClientConfig);
+    _metaClient.connect();
+  }
+
+  public LockClient(MetaClientInterface<LockInfo> client) {
+    _metaClient = client;
+    _metaClient.connect();
+  }
+
+  @Override
+  public void acquireLock(String key, LockInfo lockInfo, MetaClientInterface.EntryMode mode) {
+    _metaClient.create(key, lockInfo, mode);
+  }
+
+  @Override
+  public void acquireLockWithTTL(String key, LockInfo lockInfo, long ttl) {
+    _metaClient.createWithTTL(key, lockInfo, ttl);
+  }
+
+  @Override
+  public void renewTTLLock(String key) {
+    _metaClient.renewTTLNode(key);
+  }
+
+  @Override
+  public void releaseLock(String key) {
+    MetaClientInterface.Stat stat = _metaClient.exists(key);
+    if (stat != null) {
+      int version = stat.getVersion();
+      List<Op> ops = Arrays.asList(
+          Op.check(key, version),
+          Op.delete(key, version));
+      _metaClient.transactionOP(ops);
+      if (_metaClient.exists(key) != null) {
+        throw new MetaClientException("Failed to release lock for key: " + key);
+      }
+    }
+  }
+
+  @Override
+  public LockInfo retrieveLock(String key) {
+    MetaClientInterface.Stat stat = _metaClient.exists(key);
+    if (stat == null) {
+      return null;
+    }
+    //Create a new DataRecord from underlying record
+    DataRecord dataRecord = new DataRecord(_metaClient.get(key));
+    //Create a new LockInfo from DataRecord
+    LockInfo lockInfo = new LockInfo(dataRecord);
+    //Synchronize the lockInfo with the stat
+    lockInfo.setGrantedAt(stat.getCreationTime());
+    lockInfo.setLastRenewedAt(stat.getModifiedTime());

Review Comment:
   nit: can this be wrapped into a constructor?



##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/lock/LockClient.java:
##########
@@ -0,0 +1,107 @@
+package org.apache.helix.metaclient.recipes.lock;
+
+/*
+ * 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.Op;
+import org.apache.helix.metaclient.datamodel.DataRecord;
+import org.apache.helix.metaclient.exception.MetaClientException;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+import org.apache.helix.metaclient.factories.MetaClientFactory;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientConfig;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientFactory;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class LockClient implements LockClientInterface, AutoCloseable {
+  private final MetaClientInterface<LockInfo> _metaClient;
+
+  public LockClient(MetaClientConfig config) {
+    if (!config.getStoreType().equals(MetaClientConfig.StoreType.ZOOKEEPER)) {

Review Comment:
   let's follow Junkai's comment to avoid NPE



-- 
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 #2457: Lattice LockClient Implementation

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/lock/LockClient.java:
##########
@@ -0,0 +1,112 @@
+package org.apache.helix.metaclient.recipes.lock;
+
+/*
+ * 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.Op;
+import org.apache.helix.metaclient.datamodel.DataRecord;
+import org.apache.helix.metaclient.exception.MetaClientException;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+import org.apache.helix.metaclient.factories.MetaClientFactory;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientConfig;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientFactory;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class LockClient implements LockClientInterface, AutoCloseable {
+  private final MetaClientInterface<LockInfo> _metaClient;
+
+  public LockClient(MetaClientConfig config) {
+    if (config.getStoreType() == MetaClientConfig.StoreType.ZOOKEEPER) {

Review Comment:
   Thanks @junkaixue and @qqu0127 for the input! Even though it is essentially the same, I'll change it to enum compare for good practices.



-- 
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 #2457: Lattice LockClient Implementation

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


-- 
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 #2457: Lattice LockClient Implementation

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/lock/LockClient.java:
##########
@@ -0,0 +1,108 @@
+package org.apache.helix.metaclient.recipes.lock;
+
+/*
+ * 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.Op;
+import org.apache.helix.metaclient.datamodel.DataRecord;
+import org.apache.helix.metaclient.exception.MetaClientException;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+import org.apache.helix.metaclient.factories.MetaClientFactory;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientConfig;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientFactory;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+
+public class LockClient implements LockClientInterface, AutoCloseable {
+  private final MetaClientInterface<LockInfo> _metaClient;
+
+  public LockClient(MetaClientConfig config) {
+    if (!MetaClientConfig.StoreType.ZOOKEEPER.equals(config.getStoreType())) {
+      throw new MetaClientException("Unsupported store type: " + config.getStoreType());
+    }
+    ZkMetaClientConfig zkMetaClientConfig = new ZkMetaClientConfig.ZkMetaClientConfigBuilder().
+        setConnectionAddress(config.getConnectionAddress())
+        // Currently only support ZNRecordSerializer. TODO: make this configurable
+        .setZkSerializer((new ZNRecordSerializer()))
+        .build();
+    _metaClient = new ZkMetaClientFactory().getMetaClient(zkMetaClientConfig);

Review Comment:
   I changed it based off of @junkaixue previous comment on reducing the number of if /else clauses. But this makes sense, will revert it back, thank you!



-- 
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 #2457: Lattice LockClient Implementation

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/lock/LockClient.java:
##########
@@ -0,0 +1,108 @@
+package org.apache.helix.metaclient.recipes.lock;
+
+/*
+ * 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.Op;
+import org.apache.helix.metaclient.datamodel.DataRecord;
+import org.apache.helix.metaclient.exception.MetaClientException;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+import org.apache.helix.metaclient.factories.MetaClientFactory;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientConfig;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientFactory;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class LockClient implements LockClientInterface, AutoCloseable {
+  private final MetaClientInterface<LockInfo> _metaClient;
+
+  public LockClient(MetaClientConfig config) {
+    if (config.getStoreType().equals(MetaClientConfig.StoreType.ZOOKEEPER)) {
+      ZkMetaClientConfig zkMetaClientConfig = new ZkMetaClientConfig.ZkMetaClientConfigBuilder().
+          setConnectionAddress(config.getConnectionAddress())
+          // Currently only support ZNRecordSerializer. TODO: make this configurable
+          .setZkSerializer((new ZNRecordSerializer()))
+          .build();
+      _metaClient = new ZkMetaClientFactory().getMetaClient(zkMetaClientConfig);
+    } else {
+      throw new MetaClientException("Unsupported store type: " + config.getStoreType());
+    }
+    _metaClient.connect();
+  }
+
+  public LockClient(MetaClientInterface<LockInfo> client) {
+    _metaClient = client;
+    _metaClient.connect();
+  }
+
+  @Override
+  public void acquireLock(String key, LockInfo lockInfo, MetaClientInterface.EntryMode mode) {
+    _metaClient.create(key, lockInfo, mode);
+  }
+
+  @Override
+  public void acquireLockWithTTL(String key, LockInfo lockInfo, long ttl) {
+    _metaClient.createWithTTL(key, lockInfo, ttl);
+  }
+
+  @Override
+  public void renewTTLLock(String key) {
+    _metaClient.renewTTLNode(key);
+  }
+
+  @Override
+  public void releaseLock(String key) {
+    MetaClientInterface.Stat stat = _metaClient.exists(key);
+    if (stat != null) {
+      int version = stat.getVersion();
+      List<Op> ops = Arrays.asList(
+          Op.check(key, version),
+          Op.delete(key, version));
+      _metaClient.transactionOP(ops);
+      if (_metaClient.exists(key) != null) {
+        throw new MetaClientException("Failed to release lock for key: " + key);
+      }
+    }
+  }
+
+  @Override
+  public LockInfo retrieveLock(String key) {
+    MetaClientInterface.Stat stat = _metaClient.exists(key);
+    if (stat == null) {
+      return null;
+    }
+    //Create a new DataRecord from underlying record
+    DataRecord dataRecord = new DataRecord(_metaClient.get(key));
+    //Create a new LockInfo from DataRecord
+    LockInfo lockInfo = new LockInfo(dataRecord);
+    //Synchronize the lockInfo with the stat
+    lockInfo.setGrantedAt(stat.getCreationTime());
+    lockInfo.setLastRenewedAt(stat.getModifiedTime());
+    return lockInfo;
+  }
+
+  @Override
+  public void close() {
+    _metaClient.disconnect();

Review Comment:
   I'm not quite sure I understand. The metaclient will only be closed when close is called, which is the desired effect



-- 
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 #2457: Lattice LockClient Implementation

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/lock/LockClient.java:
##########
@@ -0,0 +1,108 @@
+package org.apache.helix.metaclient.recipes.lock;
+
+/*
+ * 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.Op;
+import org.apache.helix.metaclient.datamodel.DataRecord;
+import org.apache.helix.metaclient.exception.MetaClientException;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+import org.apache.helix.metaclient.factories.MetaClientFactory;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientConfig;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientFactory;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class LockClient implements LockClientInterface {
+  private final MetaClientInterface<LockInfo> _metaClient;
+
+  public LockClient(MetaClientConfig config) {
+    if (config.getStoreType() == MetaClientConfig.StoreType.ZOOKEEPER) {
+      ZkMetaClientConfig zkMetaClientConfig = new ZkMetaClientConfig.ZkMetaClientConfigBuilder().
+          setConnectionAddress(config.getConnectionAddress())
+          // Currently only support ZNRecordSerializer. TODO: make this configurable
+          .setZkSerializer((new ZNRecordSerializer()))
+          .build();
+      _metaClient = new ZkMetaClientFactory().getMetaClient(zkMetaClientConfig);
+    } else {
+      throw new MetaClientException("Unsupported store type: " + config.getStoreType());
+    }
+    _metaClient.connect();
+  }
+
+  public LockClient(MetaClientInterface<LockInfo> client) {
+    _metaClient = client;
+    _metaClient.connect();
+  }
+
+  @Override
+  public boolean acquireLock(String key, LockInfo lockInfo, MetaClientInterface.EntryMode mode) {
+    _metaClient.create(key, lockInfo, mode);
+    return true;

Review Comment:
   Addressed in previous comment



-- 
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 #2457: Lattice LockClient Implementation

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/lock/LockClient.java:
##########
@@ -0,0 +1,108 @@
+package org.apache.helix.metaclient.recipes.lock;
+
+/*
+ * 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.Op;
+import org.apache.helix.metaclient.datamodel.DataRecord;
+import org.apache.helix.metaclient.exception.MetaClientException;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+import org.apache.helix.metaclient.factories.MetaClientFactory;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientConfig;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientFactory;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class LockClient implements LockClientInterface {
+  private final MetaClientInterface<LockInfo> _metaClient;
+
+  public LockClient(MetaClientConfig config) {
+    if (config.getStoreType() == MetaClientConfig.StoreType.ZOOKEEPER) {
+      ZkMetaClientConfig zkMetaClientConfig = new ZkMetaClientConfig.ZkMetaClientConfigBuilder().
+          setConnectionAddress(config.getConnectionAddress())
+          // Currently only support ZNRecordSerializer. TODO: make this configurable
+          .setZkSerializer((new ZNRecordSerializer()))
+          .build();
+      _metaClient = new ZkMetaClientFactory().getMetaClient(zkMetaClientConfig);
+    } else {
+      throw new MetaClientException("Unsupported store type: " + config.getStoreType());
+    }
+    _metaClient.connect();
+  }
+
+  public LockClient(MetaClientInterface<LockInfo> client) {
+    _metaClient = client;
+    _metaClient.connect();
+  }
+
+  @Override
+  public boolean acquireLock(String key, LockInfo lockInfo, MetaClientInterface.EntryMode mode) {
+    _metaClient.create(key, lockInfo, mode);
+    return true;

Review Comment:
   when do we return false?
   



-- 
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 #2457: Lattice LockClient Implementation

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/lock/LockClient.java:
##########
@@ -0,0 +1,107 @@
+package org.apache.helix.metaclient.recipes.lock;
+
+/*
+ * 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.Op;
+import org.apache.helix.metaclient.datamodel.DataRecord;
+import org.apache.helix.metaclient.exception.MetaClientException;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+import org.apache.helix.metaclient.factories.MetaClientFactory;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientConfig;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientFactory;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class LockClient implements LockClientInterface, AutoCloseable {
+  private final MetaClientInterface<LockInfo> _metaClient;
+
+  public LockClient(MetaClientConfig config) {
+    if (!config.getStoreType().equals(MetaClientConfig.StoreType.ZOOKEEPER)) {

Review Comment:
   Oh okay I wasn't aware the order mattered. Will fix 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 #2457: Lattice LockClient Implementation

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/factories/MetaClientFactory.java:
##########
@@ -32,6 +35,12 @@ public class MetaClientFactory {
   private static final Logger LOG = LoggerFactory.getLogger(MetaClientFactory.class);
 
   public MetaClientInterface getMetaClient(MetaClientConfig config) {
+    if (config.getStoreType() == MetaClientConfig.StoreType.ZOOKEEPER) {
+      ZkMetaClientConfig zkMetaClientConfig = new ZkMetaClientConfig.ZkMetaClientConfigBuilder().

Review Comment:
   Yes good catch! I'll set all configurable fields from metaclientconfig when creating a zkmetaclientconfig



-- 
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 #2457: Lattice LockClient Implementation

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/lock/LockClient.java:
##########
@@ -0,0 +1,108 @@
+package org.apache.helix.metaclient.recipes.lock;
+
+/*
+ * 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.Op;
+import org.apache.helix.metaclient.datamodel.DataRecord;
+import org.apache.helix.metaclient.exception.MetaClientException;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+import org.apache.helix.metaclient.factories.MetaClientFactory;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientConfig;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientFactory;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class LockClient implements LockClientInterface {
+  private final MetaClientInterface<LockInfo> _metaClient;
+
+  public LockClient(MetaClientConfig config) {
+    if (config.getStoreType() == MetaClientConfig.StoreType.ZOOKEEPER) {
+      ZkMetaClientConfig zkMetaClientConfig = new ZkMetaClientConfig.ZkMetaClientConfigBuilder().
+          setConnectionAddress(config.getConnectionAddress())
+          // Currently only support ZNRecordSerializer. TODO: make this configurable
+          .setZkSerializer((new ZNRecordSerializer()))
+          .build();
+      _metaClient = new ZkMetaClientFactory().getMetaClient(zkMetaClientConfig);
+    } else {
+      throw new MetaClientException("Unsupported store type: " + config.getStoreType());
+    }
+    _metaClient.connect();
+  }
+
+  public LockClient(MetaClientInterface<LockInfo> client) {
+    _metaClient = client;
+    _metaClient.connect();
+  }
+
+  @Override
+  public boolean acquireLock(String key, LockInfo lockInfo, MetaClientInterface.EntryMode mode) {
+    _metaClient.create(key, lockInfo, mode);
+    return true;
+  }
+
+  @Override
+  public boolean acquireLockWithTTL(String key, LockInfo lockInfo, long ttl) {
+    _metaClient.createWithTTL(key, lockInfo, ttl);
+    return true;
+  }
+
+  @Override
+  public boolean renewTTLLock(String key) {
+    _metaClient.renewTTLNode(key);
+    return true;
+  }
+
+  @Override
+  public boolean releaseLock(String key) {
+    MetaClientInterface.Stat stat = _metaClient.exists(key);
+    if (stat != null) {
+      int version = stat.getVersion();
+      List<Op> ops = Arrays.asList(
+          Op.check(key, version),
+          Op.delete(key, version));
+      _metaClient.transactionOP(ops);
+      if (_metaClient.exists(key) != null) {
+        throw new MetaClientException("Failed to release lock for key: " + key);
+      }
+    }
+    return true;
+  }
+
+  @Override
+  public LockInfo retrieveLock(String key) {
+    MetaClientInterface.Stat stat = _metaClient.exists(key);
+    if (stat != null) {
+      //Create a new DataRecord from underlying record
+      DataRecord dataRecord = new DataRecord(_metaClient.get(key));
+      //Create a new LockInfo from DataRecord
+      LockInfo lockInfo = new LockInfo(dataRecord);
+      //Synchronize the lockInfo with the stat
+      lockInfo.setGrantedAt(stat.getCreationTime());
+      lockInfo.setLastRenewedAt(stat.getModifiedTime());
+      return lockInfo;
+    } else {
+      return null;
+    }

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] qqu0127 commented on a diff in pull request #2457: Lattice LockClient Implementation

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/lock/LockClient.java:
##########
@@ -0,0 +1,108 @@
+package org.apache.helix.metaclient.recipes.lock;
+
+/*
+ * 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.Op;
+import org.apache.helix.metaclient.datamodel.DataRecord;
+import org.apache.helix.metaclient.exception.MetaClientException;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+import org.apache.helix.metaclient.factories.MetaClientFactory;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientConfig;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientFactory;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class LockClient implements LockClientInterface {
+  private final MetaClientInterface<LockInfo> _metaClient;
+
+  public LockClient(MetaClientConfig config) {
+    if (config.getStoreType() == MetaClientConfig.StoreType.ZOOKEEPER) {
+      ZkMetaClientConfig zkMetaClientConfig = new ZkMetaClientConfig.ZkMetaClientConfigBuilder().
+          setConnectionAddress(config.getConnectionAddress())
+          // Currently only support ZNRecordSerializer. TODO: make this configurable
+          .setZkSerializer((new ZNRecordSerializer()))
+          .build();
+      _metaClient = new ZkMetaClientFactory().getMetaClient(zkMetaClientConfig);
+    } else {
+      throw new MetaClientException("Unsupported store type: " + config.getStoreType());
+    }

Review Comment:
   Can this section be moved to MetaClientFactory or somewhere? Basically it's how we instantiate a metaclient from config.



##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/lock/LockClient.java:
##########
@@ -0,0 +1,108 @@
+package org.apache.helix.metaclient.recipes.lock;
+
+/*
+ * 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.Op;
+import org.apache.helix.metaclient.datamodel.DataRecord;
+import org.apache.helix.metaclient.exception.MetaClientException;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+import org.apache.helix.metaclient.factories.MetaClientFactory;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientConfig;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientFactory;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class LockClient implements LockClientInterface {
+  private final MetaClientInterface<LockInfo> _metaClient;
+
+  public LockClient(MetaClientConfig config) {
+    if (config.getStoreType() == MetaClientConfig.StoreType.ZOOKEEPER) {
+      ZkMetaClientConfig zkMetaClientConfig = new ZkMetaClientConfig.ZkMetaClientConfigBuilder().
+          setConnectionAddress(config.getConnectionAddress())
+          // Currently only support ZNRecordSerializer. TODO: make this configurable
+          .setZkSerializer((new ZNRecordSerializer()))
+          .build();
+      _metaClient = new ZkMetaClientFactory().getMetaClient(zkMetaClientConfig);
+    } else {
+      throw new MetaClientException("Unsupported store type: " + config.getStoreType());
+    }
+    _metaClient.connect();
+  }
+
+  public LockClient(MetaClientInterface<LockInfo> client) {
+    _metaClient = client;
+    _metaClient.connect();
+  }
+
+  @Override
+  public boolean acquireLock(String key, LockInfo lockInfo, MetaClientInterface.EntryMode mode) {
+    _metaClient.create(key, lockInfo, mode);
+    return true;
+  }
+
+  @Override
+  public boolean acquireLockWithTTL(String key, LockInfo lockInfo, long ttl) {
+    _metaClient.createWithTTL(key, lockInfo, ttl);
+    return true;
+  }
+
+  @Override
+  public boolean renewTTLLock(String key) {
+    _metaClient.renewTTLNode(key);
+    return true;
+  }

Review Comment:
   Open question, do these and below methods ever return false? Should they?



##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/lock/LockClient.java:
##########
@@ -0,0 +1,108 @@
+package org.apache.helix.metaclient.recipes.lock;
+
+/*
+ * 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.Op;
+import org.apache.helix.metaclient.datamodel.DataRecord;
+import org.apache.helix.metaclient.exception.MetaClientException;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+import org.apache.helix.metaclient.factories.MetaClientFactory;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientConfig;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientFactory;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class LockClient implements LockClientInterface {
+  private final MetaClientInterface<LockInfo> _metaClient;
+
+  public LockClient(MetaClientConfig config) {
+    if (config.getStoreType() == MetaClientConfig.StoreType.ZOOKEEPER) {
+      ZkMetaClientConfig zkMetaClientConfig = new ZkMetaClientConfig.ZkMetaClientConfigBuilder().
+          setConnectionAddress(config.getConnectionAddress())
+          // Currently only support ZNRecordSerializer. TODO: make this configurable
+          .setZkSerializer((new ZNRecordSerializer()))
+          .build();
+      _metaClient = new ZkMetaClientFactory().getMetaClient(zkMetaClientConfig);
+    } else {
+      throw new MetaClientException("Unsupported store type: " + config.getStoreType());
+    }
+    _metaClient.connect();
+  }
+
+  public LockClient(MetaClientInterface<LockInfo> client) {
+    _metaClient = client;
+    _metaClient.connect();

Review Comment:
   Not an issue, but a reminder related to the AutoCloseable, we should differentiate the 2 cases whether to close metaclient.



##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/lock/LockClient.java:
##########
@@ -0,0 +1,108 @@
+package org.apache.helix.metaclient.recipes.lock;
+
+/*
+ * 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.Op;
+import org.apache.helix.metaclient.datamodel.DataRecord;
+import org.apache.helix.metaclient.exception.MetaClientException;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+import org.apache.helix.metaclient.factories.MetaClientFactory;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientConfig;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientFactory;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class LockClient implements LockClientInterface {
+  private final MetaClientInterface<LockInfo> _metaClient;

Review Comment:
   Let's make this class AutoCloseable, and write cleanup logic there.



##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/lock/LockClient.java:
##########
@@ -0,0 +1,108 @@
+package org.apache.helix.metaclient.recipes.lock;
+
+/*
+ * 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.Op;
+import org.apache.helix.metaclient.datamodel.DataRecord;
+import org.apache.helix.metaclient.exception.MetaClientException;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+import org.apache.helix.metaclient.factories.MetaClientFactory;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientConfig;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientFactory;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class LockClient implements LockClientInterface {
+  private final MetaClientInterface<LockInfo> _metaClient;
+
+  public LockClient(MetaClientConfig config) {
+    if (config.getStoreType() == MetaClientConfig.StoreType.ZOOKEEPER) {
+      ZkMetaClientConfig zkMetaClientConfig = new ZkMetaClientConfig.ZkMetaClientConfigBuilder().
+          setConnectionAddress(config.getConnectionAddress())
+          // Currently only support ZNRecordSerializer. TODO: make this configurable
+          .setZkSerializer((new ZNRecordSerializer()))
+          .build();
+      _metaClient = new ZkMetaClientFactory().getMetaClient(zkMetaClientConfig);
+    } else {
+      throw new MetaClientException("Unsupported store type: " + config.getStoreType());
+    }
+    _metaClient.connect();
+  }
+
+  public LockClient(MetaClientInterface<LockInfo> client) {
+    _metaClient = client;
+    _metaClient.connect();
+  }
+
+  @Override
+  public boolean acquireLock(String key, LockInfo lockInfo, MetaClientInterface.EntryMode mode) {
+    _metaClient.create(key, lockInfo, mode);
+    return true;
+  }
+
+  @Override
+  public boolean acquireLockWithTTL(String key, LockInfo lockInfo, long ttl) {
+    _metaClient.createWithTTL(key, lockInfo, ttl);
+    return true;
+  }
+
+  @Override
+  public boolean renewTTLLock(String key) {
+    _metaClient.renewTTLNode(key);
+    return true;
+  }
+
+  @Override
+  public boolean releaseLock(String key) {
+    MetaClientInterface.Stat stat = _metaClient.exists(key);
+    if (stat != null) {
+      int version = stat.getVersion();
+      List<Op> ops = Arrays.asList(
+          Op.check(key, version),
+          Op.delete(key, version));
+      _metaClient.transactionOP(ops);
+      if (_metaClient.exists(key) != null) {
+        throw new MetaClientException("Failed to release lock for key: " + key);
+      }
+    }
+    return true;
+  }
+
+  @Override
+  public LockInfo retrieveLock(String key) {
+    MetaClientInterface.Stat stat = _metaClient.exists(key);
+    if (stat != null) {
+      //Create a new DataRecord from underlying record
+      DataRecord dataRecord = new DataRecord(_metaClient.get(key));
+      //Create a new LockInfo from DataRecord
+      LockInfo lockInfo = new LockInfo(dataRecord);
+      //Synchronize the lockInfo with the stat
+      lockInfo.setGrantedAt(stat.getCreationTime());
+      lockInfo.setLastRenewedAt(stat.getModifiedTime());
+      return lockInfo;
+    } else {
+      return null;
+    }

Review Comment:
   nit: you can check and return this first, so no need for the else branch.



-- 
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 #2457: Lattice LockClient Implementation

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/factories/MetaClientFactory.java:
##########
@@ -32,6 +35,12 @@ public class MetaClientFactory {
   private static final Logger LOG = LoggerFactory.getLogger(MetaClientFactory.class);
 
   public MetaClientInterface getMetaClient(MetaClientConfig config) {
+    if (config.getStoreType() == MetaClientConfig.StoreType.ZOOKEEPER) {
+      ZkMetaClientConfig zkMetaClientConfig = new ZkMetaClientConfig.ZkMetaClientConfigBuilder().

Review Comment:
   If we create a new zkMetaClientConfig and only set the `getConnectionAddress`, will other settings lost?



-- 
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 #2457: Lattice LockClient Implementation

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/lock/LockClient.java:
##########
@@ -0,0 +1,108 @@
+package org.apache.helix.metaclient.recipes.lock;
+
+/*
+ * 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.Op;
+import org.apache.helix.metaclient.datamodel.DataRecord;
+import org.apache.helix.metaclient.exception.MetaClientException;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+import org.apache.helix.metaclient.factories.MetaClientFactory;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientConfig;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientFactory;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class LockClient implements LockClientInterface {
+  private final MetaClientInterface<LockInfo> _metaClient;

Review Comment:
   Sounds good! Thanks for the comment



-- 
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 #2457: Lattice LockClient Implementation

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/lock/LockClient.java:
##########
@@ -0,0 +1,108 @@
+package org.apache.helix.metaclient.recipes.lock;
+
+/*
+ * 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.Op;
+import org.apache.helix.metaclient.datamodel.DataRecord;
+import org.apache.helix.metaclient.exception.MetaClientException;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+import org.apache.helix.metaclient.factories.MetaClientFactory;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientConfig;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientFactory;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+
+public class LockClient implements LockClientInterface, AutoCloseable {
+  private final MetaClientInterface<LockInfo> _metaClient;
+
+  public LockClient(MetaClientConfig config) {
+    if (!MetaClientConfig.StoreType.ZOOKEEPER.equals(config.getStoreType())) {
+      throw new MetaClientException("Unsupported store type: " + config.getStoreType());
+    }
+    ZkMetaClientConfig zkMetaClientConfig = new ZkMetaClientConfig.ZkMetaClientConfigBuilder().
+        setConnectionAddress(config.getConnectionAddress())
+        // Currently only support ZNRecordSerializer. TODO: make this configurable
+        .setZkSerializer((new ZNRecordSerializer()))
+        .build();
+    _metaClient = new ZkMetaClientFactory().getMetaClient(zkMetaClientConfig);

Review Comment:
   If this class is an ZK specific impl, probably we want to move it under 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 #2457: Lattice LockClient Implementation

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/lock/LockClient.java:
##########
@@ -0,0 +1,108 @@
+package org.apache.helix.metaclient.recipes.lock;
+
+/*
+ * 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.Op;
+import org.apache.helix.metaclient.datamodel.DataRecord;
+import org.apache.helix.metaclient.exception.MetaClientException;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+import org.apache.helix.metaclient.factories.MetaClientFactory;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientConfig;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientFactory;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class LockClient implements LockClientInterface {
+  private final MetaClientInterface<LockInfo> _metaClient;
+
+  public LockClient(MetaClientConfig config) {
+    if (config.getStoreType() == MetaClientConfig.StoreType.ZOOKEEPER) {
+      ZkMetaClientConfig zkMetaClientConfig = new ZkMetaClientConfig.ZkMetaClientConfigBuilder().
+          setConnectionAddress(config.getConnectionAddress())
+          // Currently only support ZNRecordSerializer. TODO: make this configurable
+          .setZkSerializer((new ZNRecordSerializer()))
+          .build();
+      _metaClient = new ZkMetaClientFactory().getMetaClient(zkMetaClientConfig);
+    } else {
+      throw new MetaClientException("Unsupported store type: " + config.getStoreType());
+    }
+    _metaClient.connect();
+  }
+
+  public LockClient(MetaClientInterface<LockInfo> client) {
+    _metaClient = client;
+    _metaClient.connect();
+  }
+
+  @Override
+  public boolean acquireLock(String key, LockInfo lockInfo, MetaClientInterface.EntryMode mode) {
+    _metaClient.create(key, lockInfo, mode);
+    return true;
+  }
+
+  @Override
+  public boolean acquireLockWithTTL(String key, LockInfo lockInfo, long ttl) {
+    _metaClient.createWithTTL(key, lockInfo, ttl);
+    return true;
+  }
+
+  @Override
+  public boolean renewTTLLock(String key) {
+    _metaClient.renewTTLNode(key);
+    return true;
+  }

Review Comment:
   If we never return false, maybe consider change the API to return void



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

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

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


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


[GitHub] [helix] junkaixue commented on a diff in pull request #2457: Lattice LockClient Implementation

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/lock/LockClient.java:
##########
@@ -0,0 +1,112 @@
+package org.apache.helix.metaclient.recipes.lock;
+
+/*
+ * 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.Op;
+import org.apache.helix.metaclient.datamodel.DataRecord;
+import org.apache.helix.metaclient.exception.MetaClientException;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+import org.apache.helix.metaclient.factories.MetaClientFactory;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientConfig;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientFactory;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class LockClient implements LockClientInterface, AutoCloseable {
+  private final MetaClientInterface<LockInfo> _metaClient;
+
+  public LockClient(MetaClientConfig config) {
+    if (config.getStoreType() == MetaClientConfig.StoreType.ZOOKEEPER) {

Review Comment:
   Enum suggest to use equals. Also better impl like:
   
   MetaClientConfig.StoreType.ZOOKEEPER.equals(config.getStoreType()) 
   
   to avoid NPE
   
   
   Also for entire piece of logic, it would be better:
   
   if (!MetaClientConfig.StoreType.ZOOKEEPER.equals(config.getStoreType())) {
      throw new MetaClientException("Unsupported store type: " + config.getStoreType());
   }
   
   xxxx // do your logic.
   
   
   This does not require many if else clauses.



-- 
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 #2457: Lattice LockClient Implementation

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/lock/LockClient.java:
##########
@@ -0,0 +1,108 @@
+package org.apache.helix.metaclient.recipes.lock;
+
+/*
+ * 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.Op;
+import org.apache.helix.metaclient.datamodel.DataRecord;
+import org.apache.helix.metaclient.exception.MetaClientException;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+import org.apache.helix.metaclient.factories.MetaClientFactory;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientConfig;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientFactory;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class LockClient implements LockClientInterface {
+  private final MetaClientInterface<LockInfo> _metaClient;
+
+  public LockClient(MetaClientConfig config) {
+    if (config.getStoreType() == MetaClientConfig.StoreType.ZOOKEEPER) {
+      ZkMetaClientConfig zkMetaClientConfig = new ZkMetaClientConfig.ZkMetaClientConfigBuilder().
+          setConnectionAddress(config.getConnectionAddress())
+          // Currently only support ZNRecordSerializer. TODO: make this configurable
+          .setZkSerializer((new ZNRecordSerializer()))
+          .build();
+      _metaClient = new ZkMetaClientFactory().getMetaClient(zkMetaClientConfig);
+    } else {
+      throw new MetaClientException("Unsupported store type: " + config.getStoreType());
+    }

Review Comment:
   Ideally yes. However, the reasoning behind putting it in the LockClient construction method is because currently LockClient must be configured with the specific ZNRecord serializer. MetaClientFactory should be generic. Let me know if this makes sense, 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 #2457: Lattice LockClient Implementation

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/lock/LockClient.java:
##########
@@ -0,0 +1,108 @@
+package org.apache.helix.metaclient.recipes.lock;
+
+/*
+ * 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.Op;
+import org.apache.helix.metaclient.datamodel.DataRecord;
+import org.apache.helix.metaclient.exception.MetaClientException;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+import org.apache.helix.metaclient.factories.MetaClientFactory;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientConfig;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientFactory;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class LockClient implements LockClientInterface {
+  private final MetaClientInterface<LockInfo> _metaClient;
+
+  public LockClient(MetaClientConfig config) {
+    if (config.getStoreType() == MetaClientConfig.StoreType.ZOOKEEPER) {
+      ZkMetaClientConfig zkMetaClientConfig = new ZkMetaClientConfig.ZkMetaClientConfigBuilder().
+          setConnectionAddress(config.getConnectionAddress())
+          // Currently only support ZNRecordSerializer. TODO: make this configurable
+          .setZkSerializer((new ZNRecordSerializer()))
+          .build();
+      _metaClient = new ZkMetaClientFactory().getMetaClient(zkMetaClientConfig);
+    } else {
+      throw new MetaClientException("Unsupported store type: " + config.getStoreType());
+    }
+    _metaClient.connect();
+  }
+
+  public LockClient(MetaClientInterface<LockInfo> client) {
+    _metaClient = client;
+    _metaClient.connect();
+  }
+
+  @Override
+  public boolean acquireLock(String key, LockInfo lockInfo, MetaClientInterface.EntryMode mode) {
+    _metaClient.create(key, lockInfo, mode);
+    return true;
+  }
+
+  @Override
+  public boolean acquireLockWithTTL(String key, LockInfo lockInfo, long ttl) {
+    _metaClient.createWithTTL(key, lockInfo, ttl);
+    return true;
+  }
+
+  @Override
+  public boolean renewTTLLock(String key) {
+    _metaClient.renewTTLNode(key);
+    return true;
+  }

Review Comment:
   These methods never return false. If the method fails to execute what it's designed to, it will throw an error that is surfaced from an underlying layer.



-- 
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 #2457: Lattice LockClient Implementation

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

   PR approved by @xyuanlu 
   Commit message:
   Implementation of LockClient for Lattice.


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

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

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


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


[GitHub] [helix] xyuanlu commented on pull request #2457: Lattice LockClient Implementation

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

   Generally LGTM. Please add log as our offline discussion. 


-- 
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 #2457: Lattice LockClient Implementation

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/lock/LockClient.java:
##########
@@ -0,0 +1,108 @@
+package org.apache.helix.metaclient.recipes.lock;
+
+/*
+ * 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.Op;
+import org.apache.helix.metaclient.datamodel.DataRecord;
+import org.apache.helix.metaclient.exception.MetaClientException;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+import org.apache.helix.metaclient.factories.MetaClientFactory;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientConfig;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientFactory;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+
+public class LockClient implements LockClientInterface, AutoCloseable {
+  private final MetaClientInterface<LockInfo> _metaClient;
+
+  public LockClient(MetaClientConfig config) {
+    if (!MetaClientConfig.StoreType.ZOOKEEPER.equals(config.getStoreType())) {
+      throw new MetaClientException("Unsupported store type: " + config.getStoreType());
+    }
+    ZkMetaClientConfig zkMetaClientConfig = new ZkMetaClientConfig.ZkMetaClientConfigBuilder().
+        setConnectionAddress(config.getConnectionAddress())
+        // Currently only support ZNRecordSerializer. TODO: make this configurable
+        .setZkSerializer((new ZNRecordSerializer()))
+        .build();
+    _metaClient = new ZkMetaClientFactory().getMetaClient(zkMetaClientConfig);

Review Comment:
   We would like to make this class as generic as possible. Probably do 
   if (type ==ZK) {
      //cast and create
   } else {
     throw error.
   }



-- 
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 #2457: Lattice LockClient Implementation

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/lock/LockClient.java:
##########
@@ -0,0 +1,108 @@
+package org.apache.helix.metaclient.recipes.lock;
+
+/*
+ * 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.Op;
+import org.apache.helix.metaclient.datamodel.DataRecord;
+import org.apache.helix.metaclient.exception.MetaClientException;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+import org.apache.helix.metaclient.factories.MetaClientFactory;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientConfig;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientFactory;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class LockClient implements LockClientInterface {
+  private final MetaClientInterface<LockInfo> _metaClient;
+
+  public LockClient(MetaClientConfig config) {
+    if (config.getStoreType() == MetaClientConfig.StoreType.ZOOKEEPER) {
+      ZkMetaClientConfig zkMetaClientConfig = new ZkMetaClientConfig.ZkMetaClientConfigBuilder().
+          setConnectionAddress(config.getConnectionAddress())
+          // Currently only support ZNRecordSerializer. TODO: make this configurable
+          .setZkSerializer((new ZNRecordSerializer()))
+          .build();
+      _metaClient = new ZkMetaClientFactory().getMetaClient(zkMetaClientConfig);
+    } else {
+      throw new MetaClientException("Unsupported store type: " + config.getStoreType());
+    }
+    _metaClient.connect();
+  }
+
+  public LockClient(MetaClientInterface<LockInfo> client) {
+    _metaClient = client;
+    _metaClient.connect();
+  }
+
+  @Override
+  public boolean acquireLock(String key, LockInfo lockInfo, MetaClientInterface.EntryMode mode) {
+    _metaClient.create(key, lockInfo, mode);
+    return true;
+  }
+
+  @Override
+  public boolean acquireLockWithTTL(String key, LockInfo lockInfo, long ttl) {
+    _metaClient.createWithTTL(key, lockInfo, ttl);
+    return true;
+  }
+
+  @Override
+  public boolean renewTTLLock(String key) {
+    _metaClient.renewTTLNode(key);
+    return true;
+  }

Review Comment:
   Sounds good, changing it to void makes 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 #2457: Lattice LockClient Implementation

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


##########
meta-client/src/main/java/org/apache/helix/metaclient/recipes/lock/LockClient.java:
##########
@@ -0,0 +1,108 @@
+package org.apache.helix.metaclient.recipes.lock;
+
+/*
+ * 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.Op;
+import org.apache.helix.metaclient.datamodel.DataRecord;
+import org.apache.helix.metaclient.exception.MetaClientException;
+import org.apache.helix.metaclient.factories.MetaClientConfig;
+import org.apache.helix.metaclient.factories.MetaClientFactory;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientConfig;
+import org.apache.helix.metaclient.impl.zk.factory.ZkMetaClientFactory;
+import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+
+public class LockClient implements LockClientInterface, AutoCloseable {
+  private final MetaClientInterface<LockInfo> _metaClient;
+
+  public LockClient(MetaClientConfig config) {
+    if (!MetaClientConfig.StoreType.ZOOKEEPER.equals(config.getStoreType())) {
+      throw new MetaClientException("Unsupported store type: " + config.getStoreType());
+    }
+    ZkMetaClientConfig zkMetaClientConfig = new ZkMetaClientConfig.ZkMetaClientConfigBuilder().
+        setConnectionAddress(config.getConnectionAddress())
+        // Currently only support ZNRecordSerializer. TODO: make this configurable
+        .setZkSerializer((new ZNRecordSerializer()))
+        .build();
+    _metaClient = new ZkMetaClientFactory().getMetaClient(zkMetaClientConfig);

Review Comment:
   Sounds good, will fix it. Thank you!



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