[
https://issues.apache.org/jira/browse/DRILL-4726?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15524280#comment-15524280
]
ASF GitHub Bot commented on DRILL-4726:
---------------------------------------
Github user paul-rogers commented on a diff in the pull request:
https://github.com/apache/drill/pull/574#discussion_r80534600
--- Diff:
exec/java-exec/src/main/java/org/apache/drill/exec/coord/zk/ZookeeperClient.java
---
@@ -257,14 +263,47 @@ public void put(final String path, final byte[] data,
DataChangeVersion version)
}
if (hasNode) {
if (version != null) {
-
curator.setData().withVersion(version.getVersion()).forPath(target, data);
+ try {
+
curator.setData().withVersion(version.getVersion()).forPath(target, data);
+ } catch (final KeeperException.BadVersionException e) {
+ throw new VersionMismatchException("Unable to put data.
Version mismatch is detected.", version.getVersion(), e);
+ }
} else {
curator.setData().forPath(target, data);
}
}
getCache().rebuildNode(target);
- } catch (final KeeperException.BadVersionException e) {
- throw new VersionMismatchException(e);
+ } catch (final VersionMismatchException e) {
+ throw e;
+ } catch (final Exception e) {
+ throw new DrillRuntimeException("unable to put ", e);
+ }
+ }
+
+ /**
+ * Puts the given byte sequence into the given path if path is does not
exist.
+ *
+ * @param path target path
+ * @param data data to store
+ * @return null if path was created, else data stored for the given path
+ */
+ public byte[] putIfAbsent(final String path, final byte[] data) {
+ Preconditions.checkNotNull(path, "path is required");
+ Preconditions.checkNotNull(data, "data is required");
+
+ final String target = PathUtils.join(root, path);
+ try {
+ boolean hasNode = hasPath(path, true);
--- End diff --
Isn't this a race condition? What if some other client creates the node
between the check for hasPath and the create call below? The ZK documentation
is sparse here, but my tests show that create( ) will fail if the node exists;
you'll get a NodeExistsException, so no need to do the hasPath check.
> Dynamic UDFs support
> --------------------
>
> Key: DRILL-4726
> URL: https://issues.apache.org/jira/browse/DRILL-4726
> Project: Apache Drill
> Issue Type: New Feature
> Affects Versions: 1.6.0
> Reporter: Arina Ielchiieva
> Assignee: Arina Ielchiieva
> Fix For: Future
>
>
> Allow register UDFs without restart of Drillbits.
> Design is described in document below:
> https://docs.google.com/document/d/1FfyJtWae5TLuyheHCfldYUpCdeIezR2RlNsrOTYyAB4/edit?usp=sharing
>
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)