Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import org.apache.commons.cli.ParseException;
import org.apache.commons.cli.Parser;
import org.apache.commons.cli.PosixParser;
import org.apache.zookeeper.KeeperException;

/**
* sync command for cli
Expand Down Expand Up @@ -68,6 +69,8 @@ public boolean exec() throws CliException {
int resultCode = cf.get(SYNC_TIMEOUT, TimeUnit.MILLISECONDS);
if (resultCode == 0) {
out.println("Sync is OK");
} else if (resultCode == KeeperException.Code.NONODE.intValue()) {
throw new KeeperException.NoNodeException(path);
} else {
out.println("Sync has failed. rc=" + resultCode);
}
Expand All @@ -76,7 +79,7 @@ public boolean exec() throws CliException {
} catch (InterruptedException ie) {
Thread.currentThread().interrupt();
throw new CliWrapperException(ie);
} catch (TimeoutException | ExecutionException ex) {
} catch (TimeoutException | ExecutionException | KeeperException ex) {
throw new CliWrapperException(ex);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -336,7 +336,12 @@ public void processRequest(Request request) {
lastOp = "SYNC";
SyncRequest syncRequest = new SyncRequest();
ByteBufferInputStream.byteBuffer2Record(request.request, syncRequest);
rsp = new SyncResponse(syncRequest.getPath());
path = syncRequest.getPath();
DataNode n = zks.getZKDatabase().getNode(path);
if (n == null) {
throw new KeeperException.NoNodeException();
}
rsp = new SyncResponse(path);
requestPathMetricsCollector.registerRequest(request.type, syncRequest.getPath());
break;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -691,4 +691,18 @@ public void testSyncCommand() throws Exception {
runCommandExpect(cmd, expected);
}

@Test
public void testSyncCommandFailure() throws Exception {
final ZooKeeper zk = createClient();
final SyncCommand cmd = new SyncCommand();
cmd.setZk(zk);
cmd.parse("sync /dddd".split(" "));
try {
runCommandExpect(cmd, new ArrayList<String>());
fail("Command did not fail, even the path does not exist.");
} catch (CliWrapperException e) {
assertEquals(KeeperException.Code.NONODE, ((KeeperException) e.getCause()).code());
assertEquals("Node does not exist: /dddd", e.getMessage());
}
}
}