You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@zookeeper.apache.org by eolivelli <gi...@git.apache.org> on 2018/08/17 13:17:57 UTC

[GitHub] zookeeper pull request #601: WIP - ZOOKEEPER-3123 MetricsProvider Lifecycle ...

GitHub user eolivelli opened a pull request:

    https://github.com/apache/zookeeper/pull/601

    WIP - ZOOKEEPER-3123 MetricsProvider Lifecycle in ZooKeeper Server

    Manage the lifecycle of a MetricsProvider inside a ZooKeeper server.
    - handle configuration
    - start and configure the MetricsProvider
    - notify shutdown to the MetricsProvider
    
    This is an early preview, because there are some points to discuss:
    - We have to throw an IOException in case of failure (in order not to change the current signature of main methods used to start the server)
    - The patch only provides the lifecycle, it introduces some dead fields (root metrics context), this is expected as the real instrumentation will be done in a further step, is it okay ?
    - Test cases cover only standalone mode, do we need to add a new suite for testing configuration and boot errors on QuorumPeer mode ? (the answer should be YES)
    - MetricsProvider configuration is not subject to dynamic 'reconfig'
    
    Configuration to the MetricsProvider is not yet handled, the idea is to let the user configure properties like
    metricsProvider.className=o.a.z.metrics.prometheus.PrometheusMetricsProvider
    metricsProvider.customParam1=value1
    metricsProvider.customParam2=value2
    
    in this case the MetricsProvider will receive {customParam1=value1, customParam2=value2} as parameter in configure()
    
    is it okay ?

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/eolivelli/zookeeper fix/boot-provider

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/zookeeper/pull/601.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #601
    
----
commit 11528f62175930fd02f7734ec65454ef0f86d817
Author: Enrico Olivelli <eo...@...>
Date:   2018-08-17T13:08:59Z

    ZOOKEEPER-3123 MetricsProvider Lifecycle in ZooKeeper Server

----


---

[GitHub] zookeeper pull request #601: WIP - ZOOKEEPER-3123 MetricsProvider Lifecycle ...

Posted by lvfangmin <gi...@git.apache.org>.
Github user lvfangmin commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/601#discussion_r214131563
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ZooKeeperServerMainTest.java ---
    @@ -346,6 +353,211 @@ public void testWithMinSessionTimeoutGreaterThanMaxSessionTimeout()
             }
         }
     
    +    /**
    +     * Test verifies that the server shouldn't boot with an invalid metrics provider
    +     */
    +    @Test
    +    public void testInvalidMetricsProvider()
    +            throws Exception {
    +        ClientBase.setupTestEnv();
    +
    +        final int CLIENT_PORT = PortAssignment.unique();
    +        final String configs = "metricsProvider.className=BadClass\n";
    +        MainThread main = new MainThread(CLIENT_PORT, true, configs);
    +        String args[] = new String[1];
    +        args[0] = main.confFile.toString();
    +        try {
    +            main.main.initializeAndRun(args);
    +            Assert.fail("Must throw exception as metrics provider is not "
    +                    + "well configured");
    +        } catch (ConfigException iae) {
    +            // expected
    +        }
    +    }
    +
    +    /**
    +     * Test verifies that the server shouldn't boot with a faulty metrics provider
    +     */
    +    @Test
    +    public void testFaultyMetricsProviderOnStart()
    +            throws Exception {
    +        ClientBase.setupTestEnv();
    +
    +        final int CLIENT_PORT = PortAssignment.unique();
    +        final String configs = "metricsProvider.className="+MetricsProviderWithErrorInStart.class.getName()+"\n";
    +        MainThread main = new MainThread(CLIENT_PORT, true, configs);
    +        String args[] = new String[1];
    +        args[0] = main.confFile.toString();
    +        try {
    +            main.main.initializeAndRun(args);
    +            Assert.fail("Must throw exception as metrics provider is cannot boot");
    +        } catch (IOException iae) {
    +            // expected
    +        }
    +    }
    +
    +    /**
    +     * Test verifies that the server shouldn't boot with a faulty metrics provider
    +     */
    +    @Test
    +    public void testFaultyMetricsProviderOnConfigure()
    +            throws Exception {
    +        ClientBase.setupTestEnv();
    +
    +        final int CLIENT_PORT = PortAssignment.unique();
    +        final String configs = "metricsProvider.className="+MetricsProviderWithErrorInConfigure.class.getName()+"\n";
    +        MainThread main = new MainThread(CLIENT_PORT, true, configs);
    +        String args[] = new String[1];
    +        args[0] = main.confFile.toString();
    +        try {
    +            main.main.initializeAndRun(args);
    +            Assert.fail("Must throw exception as metrics provider is cannot boot");
    +        } catch (IOException iae) {
    +            // expected
    +        }
    +    }
    +
    +
    +    /**
    +     * Test verifies that the server shouldn't be affected but runtime errors on stop()
    +     */
    +    @Test
    +    public void testFaultyMetricsProviderOnStop()
    +            throws Exception {
    +        ClientBase.setupTestEnv();
    +
    +        final int CLIENT_PORT = PortAssignment.unique();
    +        MetricsProviderWithErrorInStop.stopCalled.set(false);
    +        final String configs = "metricsProvider.className="+MetricsProviderWithErrorInStop.class.getName()+"\n";
    +        MainThread main = new MainThread(CLIENT_PORT, true, configs);
    +        main.start();
    +
    +        Assert.assertTrue("waiting for server being up",
    +                ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT,
    +                        CONNECTION_TIMEOUT));
    +
    +        clientConnected = new CountDownLatch(1);
    +        ZooKeeper zk = new ZooKeeper("127.0.0.1:" + CLIENT_PORT,
    +                ClientBase.CONNECTION_TIMEOUT, this);
    +        Assert.assertTrue("Failed to establish zkclient connection!",
    +                clientConnected.await(CONNECTION_TIMEOUT, TimeUnit.MILLISECONDS));
    +
    +        zk.create("/foo", "foobar".getBytes(), Ids.OPEN_ACL_UNSAFE,
    +                CreateMode.PERSISTENT);
    +        Assert.assertEquals(new String(zk.getData("/foo", null, null)), "foobar");
    +        zk.close();
    +
    +        main.shutdown();
    +        main.join();
    +        main.deleteDirs();
    +
    +        Assert.assertTrue("waiting for server down",
    +                ClientBase.waitForServerDown("127.0.0.1:" + CLIENT_PORT,
    +                        ClientBase.CONNECTION_TIMEOUT));
    +        Assert.assertTrue(MetricsProviderWithErrorInStop.stopCalled.get());
    +    }
    +
    +    /**
    +     * Test verifies that configuration is passed to the MetricsProvider.
    +     */
    +    @Test
    +    public void testMetricsProviderConfiguration()
    +            throws Exception {
    +        ClientBase.setupTestEnv();
    +
    +        final int CLIENT_PORT = PortAssignment.unique();
    +        MetricsProviderWithConfiguration.httpPort.set(0);
    +        final String configs = "metricsProvider.className="+MetricsProviderWithConfiguration.class.getName()+"\n"+
    +                               "metricsProvider.httpPort=1234\n";
    +        MainThread main = new MainThread(CLIENT_PORT, true, configs);
    +        main.start();
    +
    +        Assert.assertTrue("waiting for server being up",
    --- End diff --
    
    Agreed, we should to it separately in a different patch.


---

[GitHub] zookeeper issue #601: ZOOKEEPER-3123 MetricsProvider Lifecycle in ZooKeeper ...

Posted by eolivelli <gi...@git.apache.org>.
Github user eolivelli commented on the issue:

    https://github.com/apache/zookeeper/pull/601
  
    Seems there is an error on CI related to the patch.


---

[GitHub] zookeeper issue #601: ZOOKEEPER-3123 MetricsProvider Lifecycle in ZooKeeper ...

Posted by eolivelli <gi...@git.apache.org>.
Github user eolivelli commented on the issue:

    https://github.com/apache/zookeeper/pull/601
  
    @lvfangmin 
    CI  failed on  org.apache.zookeeper.server.quorum.ReconfigRecoveryTest.testCurrentServersAreObserversInNextConfig
    and it seems unrelated to the change
    
    I will re-run tests again
    



---

[GitHub] zookeeper issue #601: WIP - ZOOKEEPER-3123 MetricsProvider Lifecycle in ZooK...

Posted by lvfangmin <gi...@git.apache.org>.
Github user lvfangmin commented on the issue:

    https://github.com/apache/zookeeper/pull/601
  
    @eolivelli I'll take a look and reply by EOD.


---

[GitHub] zookeeper pull request #601: WIP - ZOOKEEPER-3123 MetricsProvider Lifecycle ...

Posted by eolivelli <gi...@git.apache.org>.
Github user eolivelli commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/601#discussion_r212587862
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java ---
    @@ -325,6 +327,8 @@ public void parseProperties(Properties zkProp)
                     quorumServicePrincipal = value;
                 } else if (key.equals("quorum.cnxn.threads.size")) {
                     quorumCnxnThreadsSize = Integer.parseInt(value);
    +            } else if (key.equals("metricsProvider.className")) {
    --- End diff --
    
    In the case of the server side there is no problem.
    For the client, you can have multiple zk clients in the same jvm.
    It would be better to have a single way of configuring metrics


---

[GitHub] zookeeper pull request #601: WIP - ZOOKEEPER-3123 MetricsProvider Lifecycle ...

Posted by lvfangmin <gi...@git.apache.org>.
Github user lvfangmin commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/601#discussion_r212528161
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java ---
    @@ -141,6 +145,16 @@ public void runFromConfig(QuorumPeerConfig config)
     
           LOG.info("Starting quorum peer");
           try {
    +          MetricsProvider metricsProvider;
    +          try {
    +              metricsProvider = MetricsProviderBootstrap
    +                      .startMetricsProvider(config.metricsProviderClassName, new Properties());
    +          } catch (MetricsProviderLifeCycleException error) {
    +              LOG.error("Cannot boot MetricsProvider {}", config.metricsProviderClassName, error);
    +              throw new IOException("Cannot boot MetricsProvider " + config.metricsProviderClassName,
    --- End diff --
    
    What kind of failure will cause this IOException to throw, will the external dependency failure cause the ZK not to start up? If so I don't think that's a good idea, services like ZK should have no critical external dependency, if the external service failed we should still able to start ZK service.


---

[GitHub] zookeeper pull request #601: WIP - ZOOKEEPER-3123 MetricsProvider Lifecycle ...

Posted by eolivelli <gi...@git.apache.org>.
Github user eolivelli commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/601#discussion_r212985974
  
    --- Diff: src/java/main/org/apache/zookeeper/server/ZooKeeperServerMain.java ---
    @@ -117,13 +121,23 @@ public void runFromConfig(ServerConfig config)
             LOG.info("Starting server");
             FileTxnSnapLog txnLog = null;
             try {
    +            try {
    +                metricsProvider = MetricsProviderBootstrap
    +                        .startMetricsProvider(config.metricsProviderClassName, new Properties());
    +            } catch (MetricsProviderLifeCycleException error) {
    +                LOG.error("Cannot boot MetricsProvider {}", config.metricsProviderClassName, error);
    --- End diff --
    
    removed


---

[GitHub] zookeeper pull request #601: ZOOKEEPER-3123 MetricsProvider Lifecycle in Zoo...

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/601#discussion_r216653165
  
    --- Diff: src/java/test/org/apache/zookeeper/metrics/BaseTestMetricsProvider.java ---
    @@ -0,0 +1,137 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.zookeeper.metrics;
    +
    +import java.util.Properties;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import org.apache.zookeeper.metrics.impl.NullMetricsProvider;
    +
    +/**
    + * Simple MetricsProvider for tests.
    + */
    +public abstract class BaseTestMetricsProvider implements MetricsProvider {
    +
    +    @Override
    +    public void configure(Properties prprts) throws MetricsProviderLifeCycleException {
    +    }
    +
    +    @Override
    +    public void start() throws MetricsProviderLifeCycleException {
    +    }
    +
    +    @Override
    +    public MetricsContext getRootContext() {
    +        return NullMetricsProvider.NullMetricsContext.INSTANCE;
    +    }
    +
    +    @Override
    +    public void stop() {
    +    }
    +
    +    public static final class MetricsProviderCapturingLifecycle extends BaseTestMetricsProvider {
    +
    +        public static final AtomicBoolean configureCalled = new AtomicBoolean();
    --- End diff --
    
    This kind of stuff (tracking calls) is exactly the thing that can be nicely achieved with Mockito mocks and/or spies in unit tests.
    
    https://www.baeldung.com/mockito-verify
    
    Would you please consider replace these test classes with mocks?



---

[GitHub] zookeeper pull request #601: WIP - ZOOKEEPER-3123 MetricsProvider Lifecycle ...

Posted by lvfangmin <gi...@git.apache.org>.
Github user lvfangmin commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/601#discussion_r212527464
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java ---
    @@ -325,6 +327,8 @@ public void parseProperties(Properties zkProp)
                     quorumServicePrincipal = value;
                 } else if (key.equals("quorum.cnxn.threads.size")) {
                     quorumCnxnThreadsSize = Integer.parseInt(value);
    +            } else if (key.equals("metricsProvider.className")) {
    --- End diff --
    
    What's the preference between JVM property and the property in config file? It looks like have a provider name JVM option in MetricsProviderBootstrap is simpler?


---

[GitHub] zookeeper pull request #601: WIP - ZOOKEEPER-3123 MetricsProvider Lifecycle ...

Posted by lvfangmin <gi...@git.apache.org>.
Github user lvfangmin commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/601#discussion_r212526114
  
    --- Diff: src/java/main/org/apache/zookeeper/server/ZooKeeperServerMain.java ---
    @@ -117,13 +121,23 @@ public void runFromConfig(ServerConfig config)
             LOG.info("Starting server");
             FileTxnSnapLog txnLog = null;
             try {
    +            try {
    +                metricsProvider = MetricsProviderBootstrap
    +                        .startMetricsProvider(config.metricsProviderClassName, new Properties());
    +            } catch (MetricsProviderLifeCycleException error) {
    +                LOG.error("Cannot boot MetricsProvider {}", config.metricsProviderClassName, error);
    --- End diff --
    
    This is already logged inside MetricsProviderBootstrap. startMetricsProvider


---

[GitHub] zookeeper pull request #601: WIP - ZOOKEEPER-3123 MetricsProvider Lifecycle ...

Posted by eolivelli <gi...@git.apache.org>.
Github user eolivelli commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/601#discussion_r212544888
  
    --- Diff: src/java/main/org/apache/zookeeper/metrics/impl/NullMetricsProvider.java ---
    @@ -0,0 +1,100 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.zookeeper.metrics.impl;
    +
    +import java.util.Properties;
    +import org.apache.zookeeper.metrics.Counter;
    +import org.apache.zookeeper.metrics.Gauge;
    +import org.apache.zookeeper.metrics.MetricsContext;
    +import org.apache.zookeeper.metrics.MetricsProvider;
    +import org.apache.zookeeper.metrics.MetricsProviderLifeCycleException;
    +import org.apache.zookeeper.metrics.Summary;
    +
    +/**
    + * This is a dummy MetricsProvider which does nothing.
    + */
    +public class NullMetricsProvider implements MetricsProvider {
    --- End diff --
    
    We will have a lot of null checks without a dummy provider.
    Such dummy provider should not have a significant impact


---

[GitHub] zookeeper pull request #601: WIP - ZOOKEEPER-3123 MetricsProvider Lifecycle ...

Posted by eolivelli <gi...@git.apache.org>.
Github user eolivelli commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/601#discussion_r212985288
  
    --- Diff: src/java/main/org/apache/zookeeper/server/ServerConfig.java ---
    @@ -115,4 +118,6 @@ public InetSocketAddress getSecureClientPortAddress() {
         public int getMinSessionTimeout() { return minSessionTimeout; }
         /** maximum session timeout in milliseconds, -1 if unset */
         public int getMaxSessionTimeout() { return maxSessionTimeout; }
    +    public String getMetricsProviderClassName() { return metricsProviderClassName;}
    --- End diff --
    
    done


---

[GitHub] zookeeper issue #601: ZOOKEEPER-3123 MetricsProvider Lifecycle in ZooKeeper ...

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on the issue:

    https://github.com/apache/zookeeper/pull/601
  
    Latest build is green. I'll commit this one.



---

[GitHub] zookeeper issue #601: WIP - ZOOKEEPER-3123 MetricsProvider Lifecycle in ZooK...

Posted by eolivelli <gi...@git.apache.org>.
Github user eolivelli commented on the issue:

    https://github.com/apache/zookeeper/pull/601
  
    @anmolnar @lvfangmin please take a look to my questions , so that we can move forward.



---

[GitHub] zookeeper pull request #601: WIP - ZOOKEEPER-3123 MetricsProvider Lifecycle ...

Posted by lvfangmin <gi...@git.apache.org>.
Github user lvfangmin commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/601#discussion_r214131395
  
    --- Diff: src/java/main/org/apache/zookeeper/metrics/impl/NullMetricsProvider.java ---
    @@ -0,0 +1,100 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.zookeeper.metrics.impl;
    +
    +import java.util.Properties;
    +import org.apache.zookeeper.metrics.Counter;
    +import org.apache.zookeeper.metrics.Gauge;
    +import org.apache.zookeeper.metrics.MetricsContext;
    +import org.apache.zookeeper.metrics.MetricsProvider;
    +import org.apache.zookeeper.metrics.MetricsProviderLifeCycleException;
    +import org.apache.zookeeper.metrics.Summary;
    +
    +/**
    + * This is a dummy MetricsProvider which does nothing.
    + */
    +public class NullMetricsProvider implements MetricsProvider {
    --- End diff --
    
    Okay, that seems reasonable to me to keep the code cleaner.


---

[GitHub] zookeeper pull request #601: WIP - ZOOKEEPER-3123 MetricsProvider Lifecycle ...

Posted by lvfangmin <gi...@git.apache.org>.
Github user lvfangmin commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/601#discussion_r212522424
  
    --- Diff: src/java/main/org/apache/zookeeper/server/ServerConfig.java ---
    @@ -115,4 +118,6 @@ public InetSocketAddress getSecureClientPortAddress() {
         public int getMinSessionTimeout() { return minSessionTimeout; }
         /** maximum session timeout in milliseconds, -1 if unset */
         public int getMaxSessionTimeout() { return maxSessionTimeout; }
    +    public String getMetricsProviderClassName() { return metricsProviderClassName;}
    --- End diff --
    
    nit: space after ;


---

[GitHub] zookeeper issue #601: ZOOKEEPER-3123 MetricsProvider Lifecycle in ZooKeeper ...

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on the issue:

    https://github.com/apache/zookeeper/pull/601
  
    Manually triggered Jenkins.


---

[GitHub] zookeeper pull request #601: WIP - ZOOKEEPER-3123 MetricsProvider Lifecycle ...

Posted by lvfangmin <gi...@git.apache.org>.
Github user lvfangmin commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/601#discussion_r212526201
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java ---
    @@ -141,6 +145,16 @@ public void runFromConfig(QuorumPeerConfig config)
     
           LOG.info("Starting quorum peer");
           try {
    +          MetricsProvider metricsProvider;
    +          try {
    +              metricsProvider = MetricsProviderBootstrap
    +                      .startMetricsProvider(config.metricsProviderClassName, new Properties());
    +          } catch (MetricsProviderLifeCycleException error) {
    +              LOG.error("Cannot boot MetricsProvider {}", config.metricsProviderClassName, error);
    --- End diff --
    
    ditto.


---

[GitHub] zookeeper issue #601: WIP - ZOOKEEPER-3123 MetricsProvider Lifecycle in ZooK...

Posted by eolivelli <gi...@git.apache.org>.
Github user eolivelli commented on the issue:

    https://github.com/apache/zookeeper/pull/601
  
    @lvfangmin addressed most of your comments (or provided answers).
    I will add tests about QuorumPeerConfig (there is no similar test case, I will have to start from scratch) then the patch will be ready to go for me


---

[GitHub] zookeeper pull request #601: WIP - ZOOKEEPER-3123 MetricsProvider Lifecycle ...

Posted by eolivelli <gi...@git.apache.org>.
Github user eolivelli commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/601#discussion_r212987631
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java ---
    @@ -325,6 +327,8 @@ public void parseProperties(Properties zkProp)
                     quorumServicePrincipal = value;
                 } else if (key.equals("quorum.cnxn.threads.size")) {
                     quorumCnxnThreadsSize = Integer.parseInt(value);
    +            } else if (key.equals("metricsProvider.className")) {
    --- End diff --
    
    For instance Prometheus may expose a configuration parameter for the URI to bind to (it will default to '/metrics').
    Other systems may need authentication parameters.
    



---

[GitHub] zookeeper issue #601: WIP - ZOOKEEPER-3123 MetricsProvider Lifecycle in ZooK...

Posted by eolivelli <gi...@git.apache.org>.
Github user eolivelli commented on the issue:

    https://github.com/apache/zookeeper/pull/601
  
    tagging @anmolnar @lvfangmin 


---

[GitHub] zookeeper pull request #601: ZOOKEEPER-3123 MetricsProvider Lifecycle in Zoo...

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/601#discussion_r216654230
  
    --- Diff: src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java ---
    @@ -1486,6 +1490,377 @@ public TestQPMain getTestQPMain() {
             }
         }
     
    +    /**
    +     * Verify boot works configuring a MetricsProvider
    +     */
    +    @Test
    +    public void testMetricsProviderLifecycle() throws Exception {
    --- End diff --
    
    `QuorumPeerMainTest` takes about 193 seconds to run on current master branch on my machine. These new tests add another ~30 seconds to it. I'm also seeing these test cases more or less covered by new ZooKeeperServer tests. What do you think of skip adding new tests here?


---

[GitHub] zookeeper pull request #601: WIP - ZOOKEEPER-3123 MetricsProvider Lifecycle ...

Posted by lvfangmin <gi...@git.apache.org>.
Github user lvfangmin commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/601#discussion_r214131287
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java ---
    @@ -325,6 +327,8 @@ public void parseProperties(Properties zkProp)
                     quorumServicePrincipal = value;
                 } else if (key.equals("quorum.cnxn.threads.size")) {
                     quorumCnxnThreadsSize = Integer.parseInt(value);
    +            } else if (key.equals("metricsProvider.className")) {
    --- End diff --
    
    @eolivelli are you saying we want to keep the same config name in static file because we may want to reuse the same config on client side as well?


---

[GitHub] zookeeper pull request #601: WIP - ZOOKEEPER-3123 MetricsProvider Lifecycle ...

Posted by eolivelli <gi...@git.apache.org>.
Github user eolivelli commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/601#discussion_r213002472
  
    --- Diff: src/java/main/org/apache/zookeeper/server/ZooKeeperServerMain.java ---
    @@ -117,13 +121,23 @@ public void runFromConfig(ServerConfig config)
             LOG.info("Starting server");
             FileTxnSnapLog txnLog = null;
             try {
    +            try {
    +                metricsProvider = MetricsProviderBootstrap
    +                        .startMetricsProvider(config.metricsProviderClassName, new Properties());
    +            } catch (MetricsProviderLifeCycleException error) {
    +                LOG.error("Cannot boot MetricsProvider {}", config.metricsProviderClassName, error);
    +                throw new IOException("Cannot boot MetricsProvider "+config.metricsProviderClassName,
    --- End diff --
    
    @lvfangmin I can change it if you feel strong.


---

[GitHub] zookeeper pull request #601: ZOOKEEPER-3123 MetricsProvider Lifecycle in Zoo...

Posted by eolivelli <gi...@git.apache.org>.
GitHub user eolivelli reopened a pull request:

    https://github.com/apache/zookeeper/pull/601

    ZOOKEEPER-3123 MetricsProvider Lifecycle in ZooKeeper Server

    Manage the lifecycle of a MetricsProvider inside a ZooKeeper server.
    - handle configuration
    - start and configure the MetricsProvider
    - notify shutdown to the MetricsProvider
    
    This is an early preview, because there are some points to discuss:
    - We have to throw an IOException in case of failure (in order not to change the current signature of main methods used to start the server)
    - The patch only provides the lifecycle, it introduces some dead fields (root metrics context), this is expected as the real instrumentation will be done in a further step, is it okay ?
    - Test cases cover only standalone mode, do we need to add a new suite for testing configuration and boot errors on QuorumPeer mode ? (the answer should be YES)
    - MetricsProvider configuration is not subject to dynamic 'reconfig'
    
    Configuration to the MetricsProvider is not yet handled, the idea is to let the user configure properties like
    metricsProvider.className=o.a.z.metrics.prometheus.PrometheusMetricsProvider
    metricsProvider.customParam1=value1
    metricsProvider.customParam2=value2
    
    in this case the MetricsProvider will receive {customParam1=value1, customParam2=value2} as parameter in configure()
    
    is it okay ?

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/eolivelli/zookeeper fix/boot-provider

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/zookeeper/pull/601.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #601
    
----
commit f4f66ecb8b9371299d965569e64b2db37ab2d5bd
Author: Enrico Olivelli <eo...@...>
Date:   2018-08-17T13:08:59Z

    ZOOKEEPER-3123 MetricsProvider Lifecycle in ZooKeeper Server

commit c92450e4ee39c1e38ef75f11af4cedc6e28dba04
Author: Enrico Olivelli <eo...@...>
Date:   2018-08-27T14:44:39Z

    implement MetricsProvider configuration, fix some review comments

commit 22f79eb84f6ceed53bbc0f5c76566e662e5772a3
Author: Enrico Olivelli <eo...@...>
Date:   2018-08-27T14:52:40Z

    clean up

commit 7ad552dbdab36c4ce02b72339782b50847fa6f03
Author: Enrico Olivelli <eo...@...>
Date:   2018-08-31T22:38:05Z

    Add testcases around QuorumPeerMain

commit 93749f6a5076d7498c2abdbd570871092d533684
Author: Enrico Olivelli <eo...@...>
Date:   2018-08-31T22:46:57Z

    fix imports

commit 8964ed17bf44ade0d6af0255ef4e241bed70125b
Author: Enrico Olivelli <eo...@...>
Date:   2018-09-01T10:37:08Z

    Fix tests, use getters in order to support Mock QuorumPeerConfig

----


---

[GitHub] zookeeper issue #601: ZOOKEEPER-3123 MetricsProvider Lifecycle in ZooKeeper ...

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on the issue:

    https://github.com/apache/zookeeper/pull/601
  
    Merged. Thanks @eolivelli !


---

[GitHub] zookeeper issue #601: ZOOKEEPER-3123 MetricsProvider Lifecycle in ZooKeeper ...

Posted by eolivelli <gi...@git.apache.org>.
Github user eolivelli commented on the issue:

    https://github.com/apache/zookeeper/pull/601
  
    @lvfangmin  on my laptop ReconfigRecoveryTest always passes.... it should be a flaky test


---

[GitHub] zookeeper pull request #601: ZOOKEEPER-3123 MetricsProvider Lifecycle in Zoo...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/zookeeper/pull/601


---

[GitHub] zookeeper issue #601: ZOOKEEPER-3123 MetricsProvider Lifecycle in ZooKeeper ...

Posted by lvfangmin <gi...@git.apache.org>.
Github user lvfangmin commented on the issue:

    https://github.com/apache/zookeeper/pull/601
  
    Thanks for adding the test, please trigger Jenkins again to see if those failed tests are due to flaky or not.


---

[GitHub] zookeeper pull request #601: ZOOKEEPER-3123 MetricsProvider Lifecycle in Zoo...

Posted by eolivelli <gi...@git.apache.org>.
Github user eolivelli closed the pull request at:

    https://github.com/apache/zookeeper/pull/601


---

[GitHub] zookeeper issue #601: ZOOKEEPER-3123 MetricsProvider Lifecycle in ZooKeeper ...

Posted by eolivelli <gi...@git.apache.org>.
Github user eolivelli commented on the issue:

    https://github.com/apache/zookeeper/pull/601
  
    @lvfangmin I have pushed a bunch of tests for the QuorumPeerMain case.
    
    As already said for the ZooKeeperServerMain there is much "copy and paste" for test cases, but this is current "style" of QuorumPeerMainTest.
    In the future we should clean up.
    
    The last commit is only about new testcases, I did not touch the core change you had already reviewed.


---

[GitHub] zookeeper pull request #601: WIP - ZOOKEEPER-3123 MetricsProvider Lifecycle ...

Posted by lvfangmin <gi...@git.apache.org>.
Github user lvfangmin commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/601#discussion_r212526157
  
    --- Diff: src/java/main/org/apache/zookeeper/server/ZooKeeperServerMain.java ---
    @@ -117,13 +121,23 @@ public void runFromConfig(ServerConfig config)
             LOG.info("Starting server");
             FileTxnSnapLog txnLog = null;
             try {
    +            try {
    +                metricsProvider = MetricsProviderBootstrap
    +                        .startMetricsProvider(config.metricsProviderClassName, new Properties());
    +            } catch (MetricsProviderLifeCycleException error) {
    +                LOG.error("Cannot boot MetricsProvider {}", config.metricsProviderClassName, error);
    +                throw new IOException("Cannot boot MetricsProvider "+config.metricsProviderClassName,
    --- End diff --
    
    Why not just throw IOException in MetricsProviderBootstrap. startMetricsProvider?


---

[GitHub] zookeeper pull request #601: ZOOKEEPER-3123 MetricsProvider Lifecycle in Zoo...

Posted by eolivelli <gi...@git.apache.org>.
GitHub user eolivelli reopened a pull request:

    https://github.com/apache/zookeeper/pull/601

    ZOOKEEPER-3123 MetricsProvider Lifecycle in ZooKeeper Server

    Manage the lifecycle of a MetricsProvider inside a ZooKeeper server.
    - handle configuration
    - start and configure the MetricsProvider
    - notify shutdown to the MetricsProvider
    
    This is an early preview, because there are some points to discuss:
    - We have to throw an IOException in case of failure (in order not to change the current signature of main methods used to start the server)
    - The patch only provides the lifecycle, it introduces some dead fields (root metrics context), this is expected as the real instrumentation will be done in a further step, is it okay ?
    - Test cases cover only standalone mode, do we need to add a new suite for testing configuration and boot errors on QuorumPeer mode ? (the answer should be YES)
    - MetricsProvider configuration is not subject to dynamic 'reconfig'
    
    Configuration to the MetricsProvider is not yet handled, the idea is to let the user configure properties like
    metricsProvider.className=o.a.z.metrics.prometheus.PrometheusMetricsProvider
    metricsProvider.customParam1=value1
    metricsProvider.customParam2=value2
    
    in this case the MetricsProvider will receive {customParam1=value1, customParam2=value2} as parameter in configure()
    
    is it okay ?

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/eolivelli/zookeeper fix/boot-provider

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/zookeeper/pull/601.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #601
    
----
commit f4f66ecb8b9371299d965569e64b2db37ab2d5bd
Author: Enrico Olivelli <eo...@...>
Date:   2018-08-17T13:08:59Z

    ZOOKEEPER-3123 MetricsProvider Lifecycle in ZooKeeper Server

commit c92450e4ee39c1e38ef75f11af4cedc6e28dba04
Author: Enrico Olivelli <eo...@...>
Date:   2018-08-27T14:44:39Z

    implement MetricsProvider configuration, fix some review comments

commit 22f79eb84f6ceed53bbc0f5c76566e662e5772a3
Author: Enrico Olivelli <eo...@...>
Date:   2018-08-27T14:52:40Z

    clean up

commit 7ad552dbdab36c4ce02b72339782b50847fa6f03
Author: Enrico Olivelli <eo...@...>
Date:   2018-08-31T22:38:05Z

    Add testcases around QuorumPeerMain

commit 93749f6a5076d7498c2abdbd570871092d533684
Author: Enrico Olivelli <eo...@...>
Date:   2018-08-31T22:46:57Z

    fix imports

commit 8964ed17bf44ade0d6af0255ef4e241bed70125b
Author: Enrico Olivelli <eo...@...>
Date:   2018-09-01T10:37:08Z

    Fix tests, use getters in order to support Mock QuorumPeerConfig

----


---

[GitHub] zookeeper pull request #601: WIP - ZOOKEEPER-3123 MetricsProvider Lifecycle ...

Posted by eolivelli <gi...@git.apache.org>.
Github user eolivelli commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/601#discussion_r213001935
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java ---
    @@ -159,6 +173,7 @@ public void runFromConfig(QuorumPeerConfig config)
               }
     
               quorumPeer = getQuorumPeer();
    +          quorumPeer.setRootMetricsContext(metricsProvider.getRootContext());
    --- End diff --
    
    No, MetricsProvider will be stopped in the finally block, it is better that QuorumPeerMain creates, starts and stops it


---

[GitHub] zookeeper pull request #601: ZOOKEEPER-3123 MetricsProvider Lifecycle in Zoo...

Posted by eolivelli <gi...@git.apache.org>.
Github user eolivelli commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/601#discussion_r216658766
  
    --- Diff: src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java ---
    @@ -1486,6 +1490,377 @@ public TestQPMain getTestQPMain() {
             }
         }
     
    +    /**
    +     * Verify boot works configuring a MetricsProvider
    +     */
    +    @Test
    +    public void testMetricsProviderLifecycle() throws Exception {
    --- End diff --
    
    The whole file need a good refactor as already commented with @lvfangmin 
    I did not change the overall style of the class in order not to pollute the patch.
    
    I will be happy to follow up with a refactor of this class.
    
    As told with @lvfangmin it is important to test the lifecycle of QuorumPeerMain which is actually used in production, instead of ZooKeeperServerMain which is never used in production sites.
    
    Alternatively I can move these new tests to a new class.
    It is not simple to mock QuorumPeerMain without creating an useless test.
    I could use PowerMock to remove all real code (TxLog, Database....) and speed up
    
    As soon as we will have Maven+surefire it will be easier to run single methods and not be forced to run the whole file (or but a lot of @Ignore) 



---

[GitHub] zookeeper pull request #601: WIP - ZOOKEEPER-3123 MetricsProvider Lifecycle ...

Posted by eolivelli <gi...@git.apache.org>.
Github user eolivelli commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/601#discussion_r213003505
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ZooKeeperServerMainTest.java ---
    @@ -346,6 +353,211 @@ public void testWithMinSessionTimeoutGreaterThanMaxSessionTimeout()
             }
         }
     
    +    /**
    +     * Test verifies that the server shouldn't boot with an invalid metrics provider
    +     */
    +    @Test
    +    public void testInvalidMetricsProvider()
    +            throws Exception {
    +        ClientBase.setupTestEnv();
    +
    +        final int CLIENT_PORT = PortAssignment.unique();
    +        final String configs = "metricsProvider.className=BadClass\n";
    +        MainThread main = new MainThread(CLIENT_PORT, true, configs);
    +        String args[] = new String[1];
    +        args[0] = main.confFile.toString();
    +        try {
    +            main.main.initializeAndRun(args);
    +            Assert.fail("Must throw exception as metrics provider is not "
    +                    + "well configured");
    +        } catch (ConfigException iae) {
    +            // expected
    +        }
    +    }
    +
    +    /**
    +     * Test verifies that the server shouldn't boot with a faulty metrics provider
    +     */
    +    @Test
    +    public void testFaultyMetricsProviderOnStart()
    +            throws Exception {
    +        ClientBase.setupTestEnv();
    +
    +        final int CLIENT_PORT = PortAssignment.unique();
    +        final String configs = "metricsProvider.className="+MetricsProviderWithErrorInStart.class.getName()+"\n";
    +        MainThread main = new MainThread(CLIENT_PORT, true, configs);
    +        String args[] = new String[1];
    +        args[0] = main.confFile.toString();
    +        try {
    +            main.main.initializeAndRun(args);
    +            Assert.fail("Must throw exception as metrics provider is cannot boot");
    +        } catch (IOException iae) {
    +            // expected
    +        }
    +    }
    +
    +    /**
    +     * Test verifies that the server shouldn't boot with a faulty metrics provider
    +     */
    +    @Test
    +    public void testFaultyMetricsProviderOnConfigure()
    +            throws Exception {
    +        ClientBase.setupTestEnv();
    +
    +        final int CLIENT_PORT = PortAssignment.unique();
    +        final String configs = "metricsProvider.className="+MetricsProviderWithErrorInConfigure.class.getName()+"\n";
    +        MainThread main = new MainThread(CLIENT_PORT, true, configs);
    +        String args[] = new String[1];
    +        args[0] = main.confFile.toString();
    +        try {
    +            main.main.initializeAndRun(args);
    +            Assert.fail("Must throw exception as metrics provider is cannot boot");
    +        } catch (IOException iae) {
    +            // expected
    +        }
    +    }
    +
    +
    +    /**
    +     * Test verifies that the server shouldn't be affected but runtime errors on stop()
    +     */
    +    @Test
    +    public void testFaultyMetricsProviderOnStop()
    +            throws Exception {
    +        ClientBase.setupTestEnv();
    +
    +        final int CLIENT_PORT = PortAssignment.unique();
    +        MetricsProviderWithErrorInStop.stopCalled.set(false);
    +        final String configs = "metricsProvider.className="+MetricsProviderWithErrorInStop.class.getName()+"\n";
    +        MainThread main = new MainThread(CLIENT_PORT, true, configs);
    +        main.start();
    +
    +        Assert.assertTrue("waiting for server being up",
    +                ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT,
    +                        CONNECTION_TIMEOUT));
    +
    +        clientConnected = new CountDownLatch(1);
    +        ZooKeeper zk = new ZooKeeper("127.0.0.1:" + CLIENT_PORT,
    +                ClientBase.CONNECTION_TIMEOUT, this);
    +        Assert.assertTrue("Failed to establish zkclient connection!",
    +                clientConnected.await(CONNECTION_TIMEOUT, TimeUnit.MILLISECONDS));
    +
    +        zk.create("/foo", "foobar".getBytes(), Ids.OPEN_ACL_UNSAFE,
    +                CreateMode.PERSISTENT);
    +        Assert.assertEquals(new String(zk.getData("/foo", null, null)), "foobar");
    +        zk.close();
    +
    +        main.shutdown();
    +        main.join();
    +        main.deleteDirs();
    +
    +        Assert.assertTrue("waiting for server down",
    +                ClientBase.waitForServerDown("127.0.0.1:" + CLIENT_PORT,
    +                        ClientBase.CONNECTION_TIMEOUT));
    +        Assert.assertTrue(MetricsProviderWithErrorInStop.stopCalled.get());
    +    }
    +
    +    /**
    +     * Test verifies that configuration is passed to the MetricsProvider.
    +     */
    +    @Test
    +    public void testMetricsProviderConfiguration()
    +            throws Exception {
    +        ClientBase.setupTestEnv();
    +
    +        final int CLIENT_PORT = PortAssignment.unique();
    +        MetricsProviderWithConfiguration.httpPort.set(0);
    +        final String configs = "metricsProvider.className="+MetricsProviderWithConfiguration.class.getName()+"\n"+
    +                               "metricsProvider.httpPort=1234\n";
    +        MainThread main = new MainThread(CLIENT_PORT, true, configs);
    +        main.start();
    +
    +        Assert.assertTrue("waiting for server being up",
    --- End diff --
    
    there is a lot of stuff in this class we can refactor to a common method, I did not do it in order not to pollute the patch with unrelated changes


---

[GitHub] zookeeper issue #601: WIP - ZOOKEEPER-3123 MetricsProvider Lifecycle in ZooK...

Posted by eolivelli <gi...@git.apache.org>.
Github user eolivelli commented on the issue:

    https://github.com/apache/zookeeper/pull/601
  
    Thank you @lvfangmin .
    I will add tests for QuorumPeer. I hope I will have some cycle on Friday


---

[GitHub] zookeeper issue #601: WIP - ZOOKEEPER-3123 MetricsProvider Lifecycle in ZooK...

Posted by lvfangmin <gi...@git.apache.org>.
Github user lvfangmin commented on the issue:

    https://github.com/apache/zookeeper/pull/601
  
    Thanks @eolivelli, the updated diff LGTM, let's keep this open for a few more days to see if there is any other comments, I'll accept it if there is none.


---

[GitHub] zookeeper pull request #601: WIP - ZOOKEEPER-3123 MetricsProvider Lifecycle ...

Posted by eolivelli <gi...@git.apache.org>.
Github user eolivelli commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/601#discussion_r212986954
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java ---
    @@ -141,6 +145,16 @@ public void runFromConfig(QuorumPeerConfig config)
     
           LOG.info("Starting quorum peer");
           try {
    +          MetricsProvider metricsProvider;
    +          try {
    +              metricsProvider = MetricsProviderBootstrap
    +                      .startMetricsProvider(config.metricsProviderClassName, new Properties());
    +          } catch (MetricsProviderLifeCycleException error) {
    +              LOG.error("Cannot boot MetricsProvider {}", config.metricsProviderClassName, error);
    +              throw new IOException("Cannot boot MetricsProvider " + config.metricsProviderClassName,
    --- End diff --
    
    Services like Prometheus Client will open an http endpoint (in our specific case I think we will leverage existing admin server, but anyway the registration in the servlet container may fail), or some other service will try to bind to JMX, so it is possible that something goes wrong during start.
    You cannot start a server without metrics, otherwise monitoring system won't have control of what is going on.



---

[GitHub] zookeeper pull request #601: ZOOKEEPER-3123 MetricsProvider Lifecycle in Zoo...

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/601#discussion_r216663176
  
    --- Diff: src/java/test/org/apache/zookeeper/metrics/BaseTestMetricsProvider.java ---
    @@ -0,0 +1,137 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.zookeeper.metrics;
    +
    +import java.util.Properties;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import org.apache.zookeeper.metrics.impl.NullMetricsProvider;
    +
    +/**
    + * Simple MetricsProvider for tests.
    + */
    +public abstract class BaseTestMetricsProvider implements MetricsProvider {
    +
    +    @Override
    +    public void configure(Properties prprts) throws MetricsProviderLifeCycleException {
    +    }
    +
    +    @Override
    +    public void start() throws MetricsProviderLifeCycleException {
    +    }
    +
    +    @Override
    +    public MetricsContext getRootContext() {
    +        return NullMetricsProvider.NullMetricsContext.INSTANCE;
    +    }
    +
    +    @Override
    +    public void stop() {
    +    }
    +
    +    public static final class MetricsProviderCapturingLifecycle extends BaseTestMetricsProvider {
    +
    +        public static final AtomicBoolean configureCalled = new AtomicBoolean();
    --- End diff --
    
    I see, that makes sense.


---

[GitHub] zookeeper pull request #601: ZOOKEEPER-3123 MetricsProvider Lifecycle in Zoo...

Posted by eolivelli <gi...@git.apache.org>.
Github user eolivelli commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/601#discussion_r216657154
  
    --- Diff: src/java/test/org/apache/zookeeper/metrics/BaseTestMetricsProvider.java ---
    @@ -0,0 +1,137 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.zookeeper.metrics;
    +
    +import java.util.Properties;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import org.apache.zookeeper.metrics.impl.NullMetricsProvider;
    +
    +/**
    + * Simple MetricsProvider for tests.
    + */
    +public abstract class BaseTestMetricsProvider implements MetricsProvider {
    +
    +    @Override
    +    public void configure(Properties prprts) throws MetricsProviderLifeCycleException {
    +    }
    +
    +    @Override
    +    public void start() throws MetricsProviderLifeCycleException {
    +    }
    +
    +    @Override
    +    public MetricsContext getRootContext() {
    +        return NullMetricsProvider.NullMetricsContext.INSTANCE;
    +    }
    +
    +    @Override
    +    public void stop() {
    +    }
    +
    +    public static final class MetricsProviderCapturingLifecycle extends BaseTestMetricsProvider {
    +
    +        public static final AtomicBoolean configureCalled = new AtomicBoolean();
    --- End diff --
    
    @anmolnar 
    
    The problem here is that the object is instantiated using the classname, so I cannot inject a mock object. And injecting objects into the runtime in this case will make the test mostly useless, as I want to test exactly the lifecycle of the object.
    
    I could 'PowerMock#replace' many methods of NullMetricsProvider, but IMHO it is adding some extra complexity which is not worth.


---

[GitHub] zookeeper pull request #601: WIP - ZOOKEEPER-3123 MetricsProvider Lifecycle ...

Posted by lvfangmin <gi...@git.apache.org>.
Github user lvfangmin commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/601#discussion_r212526680
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java ---
    @@ -159,6 +173,7 @@ public void runFromConfig(QuorumPeerConfig config)
               }
     
               quorumPeer = getQuorumPeer();
    +          quorumPeer.setRootMetricsContext(metricsProvider.getRootContext());
    --- End diff --
    
    I think we need to pass in metricsProvider to QuorumPeer, right? Because we need to stop the metricsProvider in QuorumPeer.shutdown.


---

[GitHub] zookeeper pull request #601: WIP - ZOOKEEPER-3123 MetricsProvider Lifecycle ...

Posted by lvfangmin <gi...@git.apache.org>.
Github user lvfangmin commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/601#discussion_r212527486
  
    --- Diff: src/java/main/org/apache/zookeeper/metrics/impl/NullMetricsProvider.java ---
    @@ -0,0 +1,100 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.zookeeper.metrics.impl;
    +
    +import java.util.Properties;
    +import org.apache.zookeeper.metrics.Counter;
    +import org.apache.zookeeper.metrics.Gauge;
    +import org.apache.zookeeper.metrics.MetricsContext;
    +import org.apache.zookeeper.metrics.MetricsProvider;
    +import org.apache.zookeeper.metrics.MetricsProviderLifeCycleException;
    +import org.apache.zookeeper.metrics.Summary;
    +
    +/**
    + * This is a dummy MetricsProvider which does nothing.
    + */
    +public class NullMetricsProvider implements MetricsProvider {
    --- End diff --
    
    Are we introducing the dummy metric provider to make the code cleaner? Why not check whether the provider is configured or not, if not we do nothing? 


---

[GitHub] zookeeper issue #601: ZOOKEEPER-3123 MetricsProvider Lifecycle in ZooKeeper ...

Posted by eolivelli <gi...@git.apache.org>.
Github user eolivelli commented on the issue:

    https://github.com/apache/zookeeper/pull/601
  
    @anmolnar @lvfangmin I think this is good to go


---

[GitHub] zookeeper pull request #601: WIP - ZOOKEEPER-3123 MetricsProvider Lifecycle ...

Posted by eolivelli <gi...@git.apache.org>.
Github user eolivelli commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/601#discussion_r212545767
  
    --- Diff: src/java/main/org/apache/zookeeper/server/ZooKeeperServerMain.java ---
    @@ -117,13 +121,23 @@ public void runFromConfig(ServerConfig config)
             LOG.info("Starting server");
             FileTxnSnapLog txnLog = null;
             try {
    +            try {
    +                metricsProvider = MetricsProviderBootstrap
    +                        .startMetricsProvider(config.metricsProviderClassName, new Properties());
    +            } catch (MetricsProviderLifeCycleException error) {
    +                LOG.error("Cannot boot MetricsProvider {}", config.metricsProviderClassName, error);
    +                throw new IOException("Cannot boot MetricsProvider "+config.metricsProviderClassName,
    --- End diff --
    
    Personally I don't like IOException, here in ZK  it is like a RuntimeException, with no explicitly meaning. I would prefer not to throw such generic exceptions from nee code


---

[GitHub] zookeeper pull request #601: ZOOKEEPER-3123 MetricsProvider Lifecycle in Zoo...

Posted by eolivelli <gi...@git.apache.org>.
Github user eolivelli closed the pull request at:

    https://github.com/apache/zookeeper/pull/601


---