From 2a1b69a901f0727b7871af2d762048fb623e6702 Mon Sep 17 00:00:00 2001 From: maoling Date: Thu, 23 May 2019 18:42:39 +0200 Subject: [PATCH] ZOOKEEPER-2694: sync CLI command does not wait for result from server - Thanks the original work from [arshad.mohammad ](https://issues.apache.org/jira/secure/ViewProfile.jspa?name=arshad.mohammad) - more details in [ZOOKEEPER-2694](https://issues.apache.org/jira/browse/ZOOKEEPER-2694) Author: maoling Reviewers: eolivelli@apache.org, andor@apache.org Closes #823 from maoling/ZOOKEEPER-2694 and squashes the following commits: 41723cdca [maoling] print the resultCode when sync has failed 95123c003 [maoling] make the Sync more user-friendly a3ce1704d [maoling] SYNC_TIMEOUT & use the TimeUnit.SECONDS.toMillis & a responsible way of handling the InterruptedException df0bf7371 [maoling] CliWrapperException 52c3ad0ac [maoling] change CountDownLatch to CompletableFuture 0cc1edd98 [maoling] add the wait_timeout: 30s cf01294f4 [maoling] ZOOKEEPER-2694:sync CLI command does not wait for result from server (cherry picked from commit e2bb6e80f598415bba975edcff166ff1bd750340) Signed-off-by: Andor Molnar --- .../org/apache/zookeeper/cli/SyncCommand.java | 24 ++++++++++++++++--- .../org/apache/zookeeper/ZooKeeperTest.java | 11 +++++++++ 2 files changed, 32 insertions(+), 3 deletions(-) diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/cli/SyncCommand.java b/zookeeper-server/src/main/java/org/apache/zookeeper/cli/SyncCommand.java index c0be18d7379..f82ff8434c3 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/cli/SyncCommand.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/cli/SyncCommand.java @@ -16,6 +16,11 @@ */ package org.apache.zookeeper.cli; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Options; import org.apache.commons.cli.ParseException; @@ -30,6 +35,7 @@ public class SyncCommand extends CliCommand { private static Options options = new Options(); private String[] args; + public static final long SYNC_TIMEOUT = TimeUnit.SECONDS.toMillis(30L); public SyncCommand() { super("sync", "path"); @@ -55,18 +61,30 @@ public CliCommand parse(String[] cmdArgs) throws CliParseException { @Override public boolean exec() throws CliException { String path = args[1]; + CompletableFuture cf = new CompletableFuture<>(); + try { zk.sync(path, new AsyncCallback.VoidCallback() { - public void processResult(int rc, String path, Object ctx) { - out.println("Sync returned " + rc); + cf.complete(rc); } }, null); + + int resultCode = cf.get(SYNC_TIMEOUT, TimeUnit.MILLISECONDS); + if(resultCode == 0) { + out.println("Sync is OK"); + } else { + out.println("Sync has failed. rc=" + resultCode); + } } catch (IllegalArgumentException ex) { throw new MalformedPathException(ex.getMessage()); + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + throw new CliWrapperException(ie); + } catch (TimeoutException | ExecutionException ex) { + throw new CliWrapperException(ex); } - return false; } } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/ZooKeeperTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/ZooKeeperTest.java index 9c67374de93..00c4125e722 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/ZooKeeperTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/ZooKeeperTest.java @@ -619,4 +619,15 @@ public void testClientReconnectWithZKClientConfig() throws Exception { } } + @Test + public void testSyncCommand() throws Exception { + final ZooKeeper zk = createClient(); + SyncCommand cmd = new SyncCommand(); + cmd.setZk(zk); + cmd.parse("sync /".split(" ")); + List expected = new ArrayList(); + expected.add("Sync is OK"); + + runCommandExpect(cmd, expected); + } }