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 f37e6420b11..d904b298151 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 @@ -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 @@ -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); } @@ -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); } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/FinalRequestProcessor.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/FinalRequestProcessor.java index 9ffde55c10c..643791b98f7 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/FinalRequestProcessor.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/FinalRequestProcessor.java @@ -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; } 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 d19f627e069..83d84c51c58 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/ZooKeeperTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/ZooKeeperTest.java @@ -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()); + 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()); + } + } }