Skip to content

Commit

Permalink
ZOOKEEPER-2694: sync CLI command does not wait for result from server
Browse files Browse the repository at this point in the history
- 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 <maoling199210191@sina.com>

Reviewers: eolivelli@apache.org, andor@apache.org

Closes #823 from maoling/ZOOKEEPER-2694 and squashes the following commits:

41723cd [maoling] print the resultCode when sync has failed
95123c0 [maoling] make the Sync more user-friendly
a3ce170 [maoling] SYNC_TIMEOUT & use the TimeUnit.SECONDS.toMillis & a responsible way of handling the InterruptedException
df0bf73 [maoling] CliWrapperException
52c3ad0 [maoling] change CountDownLatch to CompletableFuture
0cc1edd [maoling] add the wait_timeout: 30s
cf01294 [maoling] ZOOKEEPER-2694:sync CLI command does not wait for result from server

(cherry picked from commit e2bb6e8)
Signed-off-by: Andor Molnar <andor@apache.org>
  • Loading branch information
maoling authored and anmolnar committed May 23, 2019
1 parent 1cf2b84 commit 2a1b69a
Show file tree
Hide file tree
Showing 2 changed files with 32 additions and 3 deletions.
Expand Up @@ -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;
Expand All @@ -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");
Expand All @@ -55,18 +61,30 @@ public CliCommand parse(String[] cmdArgs) throws CliParseException {
@Override
public boolean exec() throws CliException {
String path = args[1];
CompletableFuture<Integer> 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;
}
}
Expand Up @@ -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<String> expected = new ArrayList<String>();
expected.add("Sync is OK");

runCommandExpect(cmd, expected);
}
}

0 comments on commit 2a1b69a

Please sign in to comment.