Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 7E171200CD8 for ; Wed, 19 Jul 2017 01:34:50 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 7AE51167C94; Tue, 18 Jul 2017 23:34:50 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id C1734167C92 for ; Wed, 19 Jul 2017 01:34:49 +0200 (CEST) Received: (qmail 83237 invoked by uid 500); 18 Jul 2017 23:34:48 -0000 Mailing-List: contact dev-help@zookeeper.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@zookeeper.apache.org Delivered-To: mailing list dev@zookeeper.apache.org Received: (qmail 83223 invoked by uid 99); 18 Jul 2017 23:34:48 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 18 Jul 2017 23:34:48 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 6727EE382F; Tue, 18 Jul 2017 23:34:48 +0000 (UTC) From: karanmehta93 To: dev@zookeeper.apache.org Reply-To: dev@zookeeper.apache.org References: In-Reply-To: Subject: [GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log Content-Type: text/plain Message-Id: <20170718233448.6727EE382F@git1-us-west.apache.org> Date: Tue, 18 Jul 2017 23:34:48 +0000 (UTC) archived-at: Tue, 18 Jul 2017 23:34:50 -0000 Github user karanmehta93 commented on a diff in the pull request: https://github.com/apache/zookeeper/pull/307#discussion_r128122188 --- Diff: src/java/test/org/apache/zookeeper/server/ZooKeeperServerMainTest.java --- @@ -138,14 +145,56 @@ void delete(File f) throws IOException { ServerCnxnFactory getCnxnFactory() { return main.getCnxnFactory(); } + } - public static class TestZKSMain extends ZooKeeperServerMain { + public static class TestZKSMain extends ZooKeeperServerMain { + + private ServerStats serverStats; + + @Override + public ZooKeeperServer getZooKeeperServer(FileTxnSnapLog txnLog, ServerConfig config, ZKDatabase zkDb) { + ZooKeeperServer zooKeeperServer = super.getZooKeeperServer(txnLog, config, zkDb); + serverStats = zooKeeperServer.serverStats(); + return zooKeeperServer; + } + + @Override public void shutdown() { super.shutdown(); } } + // Test for ZOOKEEPER-2770 ZooKeeper slow operation log + @Test + public void testRequestWarningThreshold() throws IOException, KeeperException, InterruptedException { + ClientBase.setupTestEnv(); + + final int CLIENT_PORT = PortAssignment.unique(); + + MainThread main = new MainThread(CLIENT_PORT, true, null, 0); + main.start(); + + Assert.assertTrue("waiting for server being up", + ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT, + CONNECTION_TIMEOUT)); + // Get the stats object from the ZooKeeperServer to keep track of high latency requests. + ServerStats stats = main.main.serverStats; + + ZooKeeper zk = new ZooKeeper("127.0.0.1:" + CLIENT_PORT, + ClientBase.CONNECTION_TIMEOUT, this); + + zk.create("/foo1", "foobar".getBytes(), Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT); + + Assert.assertEquals(new String(zk.getData("/foo1", null, null)), "foobar"); + // It takes a while for the counter to get updated sometimes, this is added to reduce flakyness + Thread.sleep(1000); --- End diff -- @tdunning > OK. The point of the timeout, however, is that an update has not been applied. The sync() is designed to wait for exactly this. Where does the update have to applied? To ZKDb or the counter? This is what my understanding is, correct me where I am getting it wrong. The call to `zk.getData()` is a blocking call. The `FinalRequestProcessor` completes all the relevant task related to this request and in the end sends out a reply packet to the client. It puts the packet on a queue which is handled by the `ServerCnxn` on a separate thread. In the meanwhile, the former thread goes and increments the `numRequestsAboveThresholdTime` counter. In a case when client gets acknowledged about the data being persisted at the node, the test proceeds to check the counter value. If the later thread got suspended due to any reason and didn't update the counter, this might cause the test the fail. > Code that depends on timeouts is not deterministic in my book. It may just have a low probability of failure. Agreed, using the term `completely deterministic` on my part was incorrect. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastructure@apache.org or file a JIRA ticket with INFRA. ---