You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@zookeeper.apache.org by anmolnar <gi...@git.apache.org> on 2018/01/02 15:37:22 UTC

[GitHub] zookeeper pull request #440: ZOOKEEPER-2939 Deal with maxbuffer as it relate...

GitHub user anmolnar opened a pull request:

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

    ZOOKEEPER-2939 Deal with maxbuffer as it relates to proposals - Use dropwizard stats library

    This PR is intended to be the successor of https://github.com/apache/zookeeper/pull/415
    Using dropwizard library's Histogram component we're able to provide more sophisticated statistics on Proposal sizes. 
    
    From the docs:
    "A histogram measures the statistical distribution of values in a stream of data. In addition to minimum, maximum, mean, etc., it also measures median, 75th, 90th, 95th, 98th, 99th, and 99.9th percentiles."
    
    http://metrics.dropwizard.io/3.1.0/manual/core/#histograms


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

    $ git pull https://github.com/anmolnar/zookeeper ZOOKEEPER-2939-dropwizard

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

    https://github.com/apache/zookeeper/pull/440.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 #440
    
----
commit 6738e2d36b07f4643a97afd7e251a9d6c1c115f2
Author: Andor Molnár <an...@...>
Date:   2017-11-06T16:31:44Z

    ZOOKEEPER-2939: Added last/min/max proposal size JMX beans
    
    - Refactor proposal serialization logic to a common place,
    - Add JMX metric to monitor jute.maxbuffer setting
    - Add JMX metric to monitor min/max/last size of generated proposals
    - This patch deals with server side maxbuffer monitoring only.

commit 68328d9027bf2a26d3425b9c83b4a9d5a526fc9b
Author: Andor Molnár <an...@...>
Date:   2017-12-01T11:59:48Z

    ZOOKEEPER-2939. Fixed findbugs issue: newline in format string

commit ece8f09cb09c456ccb77903c9a2f6977c34c1543
Author: Andor Molnar <an...@...>
Date:   2017-12-15T18:11:17Z

    ZOOKEEPER-2933. Added proposal size statistics to 'mntr' command

commit 89376898995ecb60e6f26252853fce1d6d85ba2b
Author: Andor Molnar <an...@...>
Date:   2017-12-18T16:34:24Z

    ZOOKEEPER-2939. Added dropwizard library to calculate histogram of proposal sizes

commit 70faba6c8f7478d44edf5fd22557df533bbbc6ce
Author: Andor Molnar <an...@...>
Date:   2018-01-02T15:24:12Z

    ZOOKEEPER-2939. Fixed/added unit tests for Stat and Monitor commands

----


---

[GitHub] zookeeper issue #440: ZOOKEEPER-2939 Deal with maxbuffer as it relates to pr...

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

    https://github.com/apache/zookeeper/pull/440
  
    I apologize for the somewhat substance-light addition to the conversation, but having grafted some super crude histogram upgrades on counters in our patchset I'd be very much in favor of pulling in a library that does it right, which this seems to do.
    
    That said, splitting concerns between the new dependency and new metric also sounds correct to me.


---

[GitHub] zookeeper issue #440: ZOOKEEPER-2939 Deal with maxbuffer as it relates to pr...

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

    https://github.com/apache/zookeeper/pull/440
  
    No apologies necessary @aberghage - appreciate the feedback.


---

[GitHub] zookeeper pull request #440: ZOOKEEPER-2939 Deal with maxbuffer as it relate...

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

    https://github.com/apache/zookeeper/pull/440#discussion_r163391127
  
    --- Diff: src/java/main/org/apache/zookeeper/server/ZKDatabase.java ---
    @@ -264,19 +262,8 @@ public void addCommittedProposal(Request request) {
                     maxCommittedLog = request.zxid;
                 }
     
    -            ByteArrayOutputStream baos = new ByteArrayOutputStream();
    -            BinaryOutputArchive boa = BinaryOutputArchive.getArchive(baos);
    -            try {
    -                request.getHdr().serialize(boa, "hdr");
    -                if (request.getTxn() != null) {
    -                    request.getTxn().serialize(boa, "txn");
    -                }
    -                baos.close();
    -            } catch (IOException e) {
    -                LOG.error("This really should be impossible", e);
    -            }
    -            QuorumPacket pp = new QuorumPacket(Leader.PROPOSAL, request.zxid,
    -                    baos.toByteArray(), null);
    +            byte[] data = SerializeUtils.serializeRequest(request);
    --- End diff --
    
    Is this really apropos to the stated reason for the PR? If not separating out to another PR means 1) easier to review this patch, and 2) might be easier to get this committed separately rather than tying it to another issue.


---

[GitHub] zookeeper pull request #440: ZOOKEEPER-2939 Deal with maxbuffer as it relate...

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

    https://github.com/apache/zookeeper/pull/440#discussion_r160236436
  
    --- Diff: src/java/test/org/apache/zookeeper/server/quorum/LeaderBeanTest.java ---
    @@ -0,0 +1,119 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.server.quorum;
    +
    +import org.apache.jute.OutputArchive;
    +import org.apache.jute.Record;
    +import org.apache.zookeeper.server.Request;
    +import org.apache.zookeeper.server.ZKDatabase;
    +import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
    +import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier;
    +import org.apache.zookeeper.server.util.SerializeUtils;
    +import org.apache.zookeeper.test.ClientBase;
    +import org.apache.zookeeper.txn.TxnHeader;
    +import org.junit.After;
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.mockito.invocation.InvocationOnMock;
    +import org.mockito.stubbing.Answer;
    +
    +import java.io.File;
    +import java.io.IOException;
    +
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertNotEquals;
    +import static org.mockito.Matchers.any;
    +import static org.mockito.Matchers.anyString;
    +import static org.mockito.Mockito.doAnswer;
    +import static org.mockito.Mockito.mock;
    +
    +public class LeaderBeanTest {
    +    private Leader leader;
    +    private LeaderBean leaderBean;
    +    private FileTxnSnapLog fileTxnSnapLog;
    +    private LeaderZooKeeperServer zks;
    +    private QuorumPeer qp;
    +
    +    @Before
    +    public void setUp() throws IOException {
    +        qp = new QuorumPeer();
    +        QuorumVerifier quorumVerifierMock = mock(QuorumVerifier.class);
    +        qp.setQuorumVerifier(quorumVerifierMock, false);
    +        File tmpDir = ClientBase.createEmptyTestDir();
    +        fileTxnSnapLog = new FileTxnSnapLog(new File(tmpDir, "data"),
    +                new File(tmpDir, "data_txnlog"));
    +        ZKDatabase zkDb = new ZKDatabase(fileTxnSnapLog);
    +
    +        zks = new LeaderZooKeeperServer(fileTxnSnapLog, qp, zkDb);
    +        leader = new Leader(qp, zks);
    +        leaderBean = new LeaderBean(leader, zks);
    +    }
    +
    +    @After
    +    public void tearDown() throws IOException {
    +        fileTxnSnapLog.close();
    +    }
    +
    +    @Test
    +    public void testGetName() {
    +        assertEquals("Leader", leaderBean.getName());
    +    }
    +
    +    @Test
    +    public void testGetCurrentZxid() {
    +        // Arrange
    +        zks.setZxid(1);
    +
    +        // Assert
    +        assertEquals("0x1", leaderBean.getCurrentZxid());
    +    }
    +
    +    @Test
    +    public void testGetElectionTimeTaken() {
    +        // Arrange
    +        qp.setElectionTimeTaken(1);
    +
    +        // Assert
    +        assertEquals(1, leaderBean.getElectionTimeTaken());
    +    }
    +
    +    private Request createMockRequest() throws IOException {
    --- End diff --
    
    is this ever used?


---

[GitHub] zookeeper pull request #440: ZOOKEEPER-2939 Deal with maxbuffer as it relate...

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

    https://github.com/apache/zookeeper/pull/440#discussion_r160422461
  
    --- Diff: src/java/test/org/apache/zookeeper/server/quorum/LeaderBeanTest.java ---
    @@ -0,0 +1,119 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.server.quorum;
    +
    +import org.apache.jute.OutputArchive;
    +import org.apache.jute.Record;
    +import org.apache.zookeeper.server.Request;
    +import org.apache.zookeeper.server.ZKDatabase;
    +import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
    +import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier;
    +import org.apache.zookeeper.server.util.SerializeUtils;
    +import org.apache.zookeeper.test.ClientBase;
    +import org.apache.zookeeper.txn.TxnHeader;
    +import org.junit.After;
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.mockito.invocation.InvocationOnMock;
    +import org.mockito.stubbing.Answer;
    +
    +import java.io.File;
    +import java.io.IOException;
    +
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertNotEquals;
    +import static org.mockito.Matchers.any;
    +import static org.mockito.Matchers.anyString;
    +import static org.mockito.Mockito.doAnswer;
    +import static org.mockito.Mockito.mock;
    +
    +public class LeaderBeanTest {
    +    private Leader leader;
    +    private LeaderBean leaderBean;
    +    private FileTxnSnapLog fileTxnSnapLog;
    +    private LeaderZooKeeperServer zks;
    +    private QuorumPeer qp;
    +
    +    @Before
    +    public void setUp() throws IOException {
    +        qp = new QuorumPeer();
    +        QuorumVerifier quorumVerifierMock = mock(QuorumVerifier.class);
    +        qp.setQuorumVerifier(quorumVerifierMock, false);
    +        File tmpDir = ClientBase.createEmptyTestDir();
    +        fileTxnSnapLog = new FileTxnSnapLog(new File(tmpDir, "data"),
    +                new File(tmpDir, "data_txnlog"));
    +        ZKDatabase zkDb = new ZKDatabase(fileTxnSnapLog);
    +
    +        zks = new LeaderZooKeeperServer(fileTxnSnapLog, qp, zkDb);
    +        leader = new Leader(qp, zks);
    +        leaderBean = new LeaderBean(leader, zks);
    +    }
    +
    +    @After
    +    public void tearDown() throws IOException {
    +        fileTxnSnapLog.close();
    +    }
    +
    +    @Test
    +    public void testGetName() {
    +        assertEquals("Leader", leaderBean.getName());
    +    }
    +
    +    @Test
    +    public void testGetCurrentZxid() {
    +        // Arrange
    +        zks.setZxid(1);
    +
    +        // Assert
    +        assertEquals("0x1", leaderBean.getCurrentZxid());
    +    }
    +
    +    @Test
    +    public void testGetElectionTimeTaken() {
    +        // Arrange
    +        qp.setElectionTimeTaken(1);
    +
    +        // Assert
    +        assertEquals(1, leaderBean.getElectionTimeTaken());
    +    }
    +
    +    private Request createMockRequest() throws IOException {
    --- End diff --
    
    No, removed.


---

[GitHub] zookeeper pull request #440: ZOOKEEPER-2939 Deal with maxbuffer as it relate...

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

    https://github.com/apache/zookeeper/pull/440#discussion_r160689248
  
    --- Diff: build.xml ---
    @@ -198,7 +198,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant">
         <!-- ====================================================== -->
         <!-- Dependency versions                                    -->
         <!-- ====================================================== -->
    -    <property name="slf4j.version" value="1.7.25"/>
    +    <property name="slf4j.version" value="1.7.22"/>
    --- End diff --
    
    Here's the issue:
    {code}
    BUILD FAILED
    /Users/Molnar/git/my-zookeeper/build.xml:420: impossible to resolve dependencies:
    	org.slf4j#slf4j-api;1.7.22 (needed by [io.dropwizard.metrics#metrics-core;3.2.5]) conflicts with org.slf4j#slf4j-api;1.7.25 (needed by [org.apache.zookeeper#zookeeper;3.6.0-SNAPSHOT])
    {code}


---

[GitHub] zookeeper pull request #440: ZOOKEEPER-2939 Deal with maxbuffer as it relate...

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

    https://github.com/apache/zookeeper/pull/440#discussion_r160422407
  
    --- Diff: src/java/test/org/apache/zookeeper/server/quorum/LeaderBeanTest.java ---
    @@ -0,0 +1,119 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.server.quorum;
    +
    +import org.apache.jute.OutputArchive;
    --- End diff --
    
    thanks, done.


---

[GitHub] zookeeper issue #440: ZOOKEEPER-2939 Deal with maxbuffer as it relates to pr...

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

    https://github.com/apache/zookeeper/pull/440
  
    @phunt Do you have a chance to take a look at this approach?


---

[GitHub] zookeeper issue #440: ZOOKEEPER-2939 Deal with maxbuffer as it relates to pr...

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

    https://github.com/apache/zookeeper/pull/440
  
    I merged https://github.com/apache/zookeeper/pull/415. Please go ahead and create a new JIRA to correspond to this PR and I'll mark ZOOKEEPER-2939 as resolved.


---

[GitHub] zookeeper pull request #440: ZOOKEEPER-2939 Deal with maxbuffer as it relate...

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

    https://github.com/apache/zookeeper/pull/440#discussion_r163458563
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/ProposalStats.java ---
    @@ -0,0 +1,70 @@
    +/**
    + * 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.server.quorum;
    +
    +import com.codahale.metrics.ExponentiallyDecayingReservoir;
    +import com.codahale.metrics.Histogram;
    +import com.codahale.metrics.JmxReporter;
    +import com.codahale.metrics.MetricRegistry;
    +import com.codahale.metrics.Reservoir;
    +import com.codahale.metrics.Snapshot;
    +import org.apache.zookeeper.jmx.CommonNames;
    +import org.apache.zookeeper.jmx.MBeanRegistry;
    +
    +import static com.codahale.metrics.MetricRegistry.name;
    +
    +/**
    + * Provides real-time metrics on Leader's proposal size.
    + * The class uses a histogram included in Dropwizard metrics library with ExponentiallyDecayingReservoir.
    + * It provides stats of proposal sizes from the last 5 minutes with acceptable cpu/memory footprint optimized for streaming data.
    + */
    +public class ProposalStats {
    +    private final Histogram proposalSizes;
    +
    +    ProposalStats() {
    +        final MetricRegistry metrics = new MetricRegistry();
    +        Reservoir reservoir = new ExponentiallyDecayingReservoir();
    --- End diff --
    
    http://metrics.dropwizard.io/3.1.0/apidocs/com/codahale/metrics/ExponentiallyDecayingReservoir.html
    Default constructor:
    > Creates a new ExponentiallyDecayingReservoir of 1028 elements, which offers a 99.9% confidence level with a 5% margin of error assuming a normal distribution, and an alpha factor of 0.015, which heavily biases the reservoir to the past 5 minutes of measurements.
    There're 2 other constructors which the reservoir can be configured with by size, alpha and clock. So, the window can be extended, though it's not as obvious as with `SlidingWindow`.
    
    Let's go with that as you suggested.


---

[GitHub] zookeeper pull request #440: ZOOKEEPER-2939 Deal with maxbuffer as it relate...

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

    https://github.com/apache/zookeeper/pull/440#discussion_r159987502
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/ProposalStats.java ---
    @@ -0,0 +1,70 @@
    +/**
    + * 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.server.quorum;
    +
    +import com.codahale.metrics.ExponentiallyDecayingReservoir;
    +import com.codahale.metrics.Histogram;
    +import com.codahale.metrics.JmxReporter;
    +import com.codahale.metrics.MetricRegistry;
    +import com.codahale.metrics.Reservoir;
    +import com.codahale.metrics.Snapshot;
    +import org.apache.zookeeper.jmx.CommonNames;
    +import org.apache.zookeeper.jmx.MBeanRegistry;
    +
    +import static com.codahale.metrics.MetricRegistry.name;
    +
    +/**
    + * Provides real-time metrics on Leader's proposal size.
    + * The class uses a histogram included in Dropwizard metrics library with ExponentiallyDecayingReservoir.
    + * It provides stats of proposal sizes from the last 5 minutes with acceptable cpu/memory footprint optimized for streaming data.
    + */
    +public class ProposalStats {
    +    private final Histogram proposalSizes;
    +
    +    ProposalStats() {
    +        final MetricRegistry metrics = new MetricRegistry();
    +        Reservoir reservoir = new ExponentiallyDecayingReservoir();
    --- End diff --
    
    I won't pretend to know much about exponentially decaying reservoirs. I'm curious what the behavior is with minimum and maximum values. Are these guaranteed to always be exact and for what time period?


---

[GitHub] zookeeper pull request #440: ZOOKEEPER-2939 Deal with maxbuffer as it relate...

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

    https://github.com/apache/zookeeper/pull/440#discussion_r160524365
  
    --- Diff: build.xml ---
    @@ -198,7 +198,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant">
         <!-- ====================================================== -->
         <!-- Dependency versions                                    -->
         <!-- ====================================================== -->
    -    <property name="slf4j.version" value="1.7.25"/>
    +    <property name="slf4j.version" value="1.7.22"/>
    --- End diff --
    
    yes, to be consistent with slf4j


---

[GitHub] zookeeper issue #440: ZOOKEEPER-2979 Use dropwizard library histogram for pr...

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

    https://github.com/apache/zookeeper/pull/440
  
    Rebased and resolved conflicts.


---

[GitHub] zookeeper pull request #440: ZOOKEEPER-2939 Deal with maxbuffer as it relate...

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

    https://github.com/apache/zookeeper/pull/440#discussion_r160891125
  
    --- Diff: build.xml ---
    @@ -198,7 +198,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant">
         <!-- ====================================================== -->
         <!-- Dependency versions                                    -->
         <!-- ====================================================== -->
    -    <property name="slf4j.version" value="1.7.25"/>
    +    <property name="slf4j.version" value="1.7.22"/>
    --- End diff --
    
    Great. Works.


---

[GitHub] zookeeper issue #440: ZOOKEEPER-2979 Use dropwizard library histogram for pr...

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

    https://github.com/apache/zookeeper/pull/440
  
    3.5 is still in beta, so I'm not super worried wrt it being an addition. I also think it's super valuable to improve reporting. I'm erring on the side of inclusion - any objections?


---

[GitHub] zookeeper pull request #440: ZOOKEEPER-2939 Deal with maxbuffer as it relate...

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

    https://github.com/apache/zookeeper/pull/440#discussion_r160401512
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/ProposalStats.java ---
    @@ -0,0 +1,70 @@
    +/**
    + * 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.server.quorum;
    +
    +import com.codahale.metrics.ExponentiallyDecayingReservoir;
    +import com.codahale.metrics.Histogram;
    +import com.codahale.metrics.JmxReporter;
    +import com.codahale.metrics.MetricRegistry;
    +import com.codahale.metrics.Reservoir;
    +import com.codahale.metrics.Snapshot;
    +import org.apache.zookeeper.jmx.CommonNames;
    +import org.apache.zookeeper.jmx.MBeanRegistry;
    +
    +import static com.codahale.metrics.MetricRegistry.name;
    +
    +/**
    + * Provides real-time metrics on Leader's proposal size.
    + * The class uses a histogram included in Dropwizard metrics library with ExponentiallyDecayingReservoir.
    + * It provides stats of proposal sizes from the last 5 minutes with acceptable cpu/memory footprint optimized for streaming data.
    + */
    +public class ProposalStats {
    +    private final Histogram proposalSizes;
    +
    +    ProposalStats() {
    +        final MetricRegistry metrics = new MetricRegistry();
    +        Reservoir reservoir = new ExponentiallyDecayingReservoir();
    --- End diff --
    
    I'm not an expert either. According to the following article I think we've the following reservoir options:
    http://taint.org/2014/01/16/145944a.html
    
    SlidingTimeWindowReservoir: the standard one, but looks like the memory footprint could be potentially high in some cases.
    SlidingTimeWindowArrayReservoir: similar to the previous with much better memory handling
    ExponentiallyDecayingReservoir: good memory/cpu footprint and optimised for streaming data. The main difference is that if there's no data comes in within the configured number of samples, previously collected statistics will be presented on JMX. SlidingTimeWindow reservoirs will zero out, which sounds correct behaviour to me, but it might be less convenient.



---

[GitHub] zookeeper pull request #440: ZOOKEEPER-2939 Deal with maxbuffer as it relate...

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

    https://github.com/apache/zookeeper/pull/440#discussion_r160236050
  
    --- Diff: src/java/test/org/apache/zookeeper/server/quorum/LeaderBeanTest.java ---
    @@ -0,0 +1,119 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.server.quorum;
    +
    +import org.apache.jute.OutputArchive;
    --- End diff --
    
    nit: there are a couple unused imports here


---

[GitHub] zookeeper pull request #440: ZOOKEEPER-2939 Deal with maxbuffer as it relate...

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

    https://github.com/apache/zookeeper/pull/440#discussion_r159985176
  
    --- Diff: build.xml ---
    @@ -198,7 +198,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant">
         <!-- ====================================================== -->
         <!-- Dependency versions                                    -->
         <!-- ====================================================== -->
    -    <property name="slf4j.version" value="1.7.25"/>
    +    <property name="slf4j.version" value="1.7.22"/>
    --- End diff --
    
    what is the reason for the downgrade?


---

[GitHub] zookeeper issue #440: ZOOKEEPER-2979 Use dropwizard library histogram for pr...

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

    https://github.com/apache/zookeeper/pull/440
  
    @anmolnar I believe you misunderstood my comment above:
    
    > likely this change will go into master (3.6.0+) - impact on backports. I'm assuming we would want to take advantage of this library across the board. (can we do that? I assume yes?)
    
    > Sure, I'll have to backport it to branch-3.4, it will need some manual work for sure, but I don't expect it to be the end of the world.
    
    I meant that I was planning to only commit this change for 3.6.0+ - I am generally reticent to add a new dependency so late in the 3.5 train. Do you feel strongly otw?


---

[GitHub] zookeeper pull request #440: ZOOKEEPER-2939 Deal with maxbuffer as it relate...

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

    https://github.com/apache/zookeeper/pull/440#discussion_r159986909
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/ProposalStats.java ---
    @@ -0,0 +1,70 @@
    +/**
    + * 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.server.quorum;
    +
    +import com.codahale.metrics.ExponentiallyDecayingReservoir;
    --- End diff --
    
    I don't see much being pulled in from dropwizard, only codahale. Is codahale.metrics available independently? 
    
    It doesn't look like it but I only checked briefly.
      


---

[GitHub] zookeeper pull request #440: ZOOKEEPER-2939 Deal with maxbuffer as it relate...

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

    https://github.com/apache/zookeeper/pull/440#discussion_r160889741
  
    --- Diff: build.xml ---
    @@ -198,7 +198,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant">
         <!-- ====================================================== -->
         <!-- Dependency versions                                    -->
         <!-- ====================================================== -->
    -    <property name="slf4j.version" value="1.7.25"/>
    +    <property name="slf4j.version" value="1.7.22"/>
    --- End diff --
    
    Can we just exclude org.slf4j#slf4j-api;1.7.22 ?


---

[GitHub] zookeeper issue #440: ZOOKEEPER-2979 Use dropwizard library histogram for pr...

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

    https://github.com/apache/zookeeper/pull/440
  
    @aberghage Did you have a chance to take look?
    @afine I changed the implementation to use a configurable sliding window. Do you see any other thing that I should improve?


---

[GitHub] zookeeper pull request #440: ZOOKEEPER-2939 Deal with maxbuffer as it relate...

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

    https://github.com/apache/zookeeper/pull/440#discussion_r160233203
  
    --- Diff: ivy.xml ---
    @@ -133,6 +133,12 @@
         <dependency org="org.codehaus.jackson" name="jackson-mapper-asl"
                     rev="${jackson-mapper-asl.version}" conf="optional->default"/>
     
    +    <dependency org="io.dropwizard.metrics" name="metrics-core"
    +                rev="${dropwizard.version}" conf="default" />
    +
    +    <dependency org="org.hamcrest" name="hamcrest-all" rev="1.3"
    --- End diff --
    
    doesn't mockito pull these in already?


---

[GitHub] zookeeper pull request #440: ZOOKEEPER-2939 Deal with maxbuffer as it relate...

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

    https://github.com/apache/zookeeper/pull/440#discussion_r163458731
  
    --- Diff: src/java/main/org/apache/zookeeper/server/ZKDatabase.java ---
    @@ -264,19 +262,8 @@ public void addCommittedProposal(Request request) {
                     maxCommittedLog = request.zxid;
                 }
     
    -            ByteArrayOutputStream baos = new ByteArrayOutputStream();
    -            BinaryOutputArchive boa = BinaryOutputArchive.getArchive(baos);
    -            try {
    -                request.getHdr().serialize(boa, "hdr");
    -                if (request.getTxn() != null) {
    -                    request.getTxn().serialize(boa, "txn");
    -                }
    -                baos.close();
    -            } catch (IOException e) {
    -                LOG.error("This really should be impossible", e);
    -            }
    -            QuorumPacket pp = new QuorumPacket(Leader.PROPOSAL, request.zxid,
    -                    baos.toByteArray(), null);
    +            byte[] data = SerializeUtils.serializeRequest(request);
    --- End diff --
    
    Just a small refactoring. I'm happy to submit in another PR.


---

[GitHub] zookeeper pull request #440: ZOOKEEPER-2939 Deal with maxbuffer as it relate...

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

    https://github.com/apache/zookeeper/pull/440#discussion_r161661577
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/ProposalStats.java ---
    @@ -0,0 +1,70 @@
    +/**
    + * 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.server.quorum;
    +
    +import com.codahale.metrics.ExponentiallyDecayingReservoir;
    +import com.codahale.metrics.Histogram;
    +import com.codahale.metrics.JmxReporter;
    +import com.codahale.metrics.MetricRegistry;
    +import com.codahale.metrics.Reservoir;
    +import com.codahale.metrics.Snapshot;
    +import org.apache.zookeeper.jmx.CommonNames;
    +import org.apache.zookeeper.jmx.MBeanRegistry;
    +
    +import static com.codahale.metrics.MetricRegistry.name;
    +
    +/**
    + * Provides real-time metrics on Leader's proposal size.
    + * The class uses a histogram included in Dropwizard metrics library with ExponentiallyDecayingReservoir.
    + * It provides stats of proposal sizes from the last 5 minutes with acceptable cpu/memory footprint optimized for streaming data.
    + */
    +public class ProposalStats {
    +    private final Histogram proposalSizes;
    +
    +    ProposalStats() {
    +        final MetricRegistry metrics = new MetricRegistry();
    +        Reservoir reservoir = new ExponentiallyDecayingReservoir();
    --- End diff --
    
    @afine Would you approve the PR with existing reservoir or do you still think it'd be better to go with configurable sliding window?


---

[GitHub] zookeeper pull request #440: ZOOKEEPER-2939 Deal with maxbuffer as it relate...

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

    https://github.com/apache/zookeeper/pull/440#discussion_r160483835
  
    --- Diff: build.xml ---
    @@ -198,7 +198,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant">
         <!-- ====================================================== -->
         <!-- Dependency versions                                    -->
         <!-- ====================================================== -->
    -    <property name="slf4j.version" value="1.7.25"/>
    +    <property name="slf4j.version" value="1.7.22"/>
    --- End diff --
    
    So do we still need this lower version?


---

[GitHub] zookeeper pull request #440: ZOOKEEPER-2939 Deal with maxbuffer as it relate...

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

    https://github.com/apache/zookeeper/pull/440#discussion_r160398663
  
    --- Diff: build.xml ---
    @@ -198,7 +198,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant">
         <!-- ====================================================== -->
         <!-- Dependency versions                                    -->
         <!-- ====================================================== -->
    -    <property name="slf4j.version" value="1.7.25"/>
    +    <property name="slf4j.version" value="1.7.22"/>
    --- End diff --
    
    Not sure how to get an evidence, but if remember correctly it was dropwizard-core which forced me to use an earlier version of slf4j.


---

[GitHub] zookeeper issue #440: ZOOKEEPER-2939 Deal with maxbuffer as it relates to pr...

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

    https://github.com/apache/zookeeper/pull/440
  
    I like the idea of moving to dropwizard vs our own homebrew. Seems like it would allow us richer insight for metrics. Here are my concerns:
    
    1) performance. Do we know the impact of this library vs our homebrew? Both in the small and the large.
    
    2) likely this change will go into master (3.6.0+) - impact on backports. I'm assuming we would want to take advantage of this library across the board. (can we do that? I assume yes?)
    
    3) I believe I recommended on the original PR separating the new metric from introducing new dependencies. I still believe that's the quickest way to get the feature addressed, vs introducing new metrics functionality.  Can we commit #415 independently of this change?


---

[GitHub] zookeeper pull request #440: ZOOKEEPER-2939 Deal with maxbuffer as it relate...

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

    https://github.com/apache/zookeeper/pull/440#discussion_r160249680
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/ProposalStats.java ---
    @@ -0,0 +1,70 @@
    +/**
    + * 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.server.quorum;
    +
    +import com.codahale.metrics.ExponentiallyDecayingReservoir;
    +import com.codahale.metrics.Histogram;
    +import com.codahale.metrics.JmxReporter;
    +import com.codahale.metrics.MetricRegistry;
    +import com.codahale.metrics.Reservoir;
    +import com.codahale.metrics.Snapshot;
    +import org.apache.zookeeper.jmx.CommonNames;
    +import org.apache.zookeeper.jmx.MBeanRegistry;
    +
    +import static com.codahale.metrics.MetricRegistry.name;
    +
    +/**
    + * Provides real-time metrics on Leader's proposal size.
    + * The class uses a histogram included in Dropwizard metrics library with ExponentiallyDecayingReservoir.
    + * It provides stats of proposal sizes from the last 5 minutes with acceptable cpu/memory footprint optimized for streaming data.
    + */
    +public class ProposalStats {
    +    private final Histogram proposalSizes;
    +
    +    ProposalStats() {
    +        final MetricRegistry metrics = new MetricRegistry();
    +        Reservoir reservoir = new ExponentiallyDecayingReservoir();
    --- End diff --
    
    Perhaps a user configurable SlidingTimeWindowReservoir may be more appropriate?


---

[GitHub] zookeeper pull request #440: ZOOKEEPER-2939 Deal with maxbuffer as it relate...

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

    https://github.com/apache/zookeeper/pull/440#discussion_r163390292
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/ProposalStats.java ---
    @@ -0,0 +1,70 @@
    +/**
    + * 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.server.quorum;
    +
    +import com.codahale.metrics.ExponentiallyDecayingReservoir;
    +import com.codahale.metrics.Histogram;
    +import com.codahale.metrics.JmxReporter;
    +import com.codahale.metrics.MetricRegistry;
    +import com.codahale.metrics.Reservoir;
    +import com.codahale.metrics.Snapshot;
    +import org.apache.zookeeper.jmx.CommonNames;
    +import org.apache.zookeeper.jmx.MBeanRegistry;
    +
    +import static com.codahale.metrics.MetricRegistry.name;
    +
    +/**
    + * Provides real-time metrics on Leader's proposal size.
    + * The class uses a histogram included in Dropwizard metrics library with ExponentiallyDecayingReservoir.
    + * It provides stats of proposal sizes from the last 5 minutes with acceptable cpu/memory footprint optimized for streaming data.
    + */
    +public class ProposalStats {
    +    private final Histogram proposalSizes;
    +
    +    ProposalStats() {
    +        final MetricRegistry metrics = new MetricRegistry();
    +        Reservoir reservoir = new ExponentiallyDecayingReservoir();
    --- End diff --
    
    From: http://metrics.dropwizard.io/3.1.0/manual/core/#exponentially-decaying-reservoirs
    
    > A histogram with an exponentially decaying reservoir produces quantiles which are representative of (roughly) the last five minutes of data.
    
    My guess is that we would want more than data representing the last 5 minutes right?


---

[GitHub] zookeeper issue #440: ZOOKEEPER-2939 Deal with maxbuffer as it relates to pr...

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

    https://github.com/apache/zookeeper/pull/440
  
    @afine PR has been rebased on top of the previous jutemaxbuffer commit as discussed. Please review.


---

[GitHub] zookeeper pull request #440: ZOOKEEPER-2939 Deal with maxbuffer as it relate...

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

    https://github.com/apache/zookeeper/pull/440#discussion_r160399269
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/ProposalStats.java ---
    @@ -0,0 +1,70 @@
    +/**
    + * 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.server.quorum;
    +
    +import com.codahale.metrics.ExponentiallyDecayingReservoir;
    --- End diff --
    
    Doesn't look like it has separate package for that.


---

[GitHub] zookeeper pull request #440: ZOOKEEPER-2979 Use dropwizard library histogram...

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

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


---

[GitHub] zookeeper issue #440: ZOOKEEPER-2939 Deal with maxbuffer as it relates to pr...

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

    https://github.com/apache/zookeeper/pull/440
  
    @aberghage Thanks for reviewing this.
    @phunt 
    1. This library has some performance impact for sure, because it calculates more complicated statistics, while my original solution only exposes last / min / max proposal sizes and doesn't calculate anything. Not even average.
    
    2. Sure, I'll have to backport it to branch-3.4, it will need some manual work for sure, but I don't expect it to be the end of the world.
    
    3. Sure, I'd be more than happy if you could merge my previous PR first and later this one, because that would really mean some progress to be made on my side. Go ahead if you don't have concerns, I'm getting the build to be green asap. (the small refactoring you mentioned above is part of that PR too - be aware of that)


---

[GitHub] zookeeper issue #440: ZOOKEEPER-2979 Use dropwizard library histogram for pr...

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

    https://github.com/apache/zookeeper/pull/440
  
    @phunt right, I misunderstood your comment.
    
    I think not merging this patch to 3.4 is acceptable, but I'd argue with 3.5/3.6 a little bit. Given that how much time is needed to release a major version of ZooKeeper - even if we're successful at accelerating a little bit - I'm afraid this patch won't see a stable release in the foreseeable future.
    
    That's my only reason why I'd put this into 3.5 and keep the dumb implementation (last/min/max) in 3.4.


---

[GitHub] zookeeper pull request #440: ZOOKEEPER-2939 Deal with maxbuffer as it relate...

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

    https://github.com/apache/zookeeper/pull/440#discussion_r160398954
  
    --- Diff: ivy.xml ---
    @@ -133,6 +133,12 @@
         <dependency org="org.codehaus.jackson" name="jackson-mapper-asl"
                     rev="${jackson-mapper-asl.version}" conf="optional->default"/>
     
    +    <dependency org="io.dropwizard.metrics" name="metrics-core"
    +                rev="${dropwizard.version}" conf="default" />
    +
    +    <dependency org="org.hamcrest" name="hamcrest-all" rev="1.3"
    --- End diff --
    
    Unfortunately not. Mockito pulls in hamcrest-core only which contains the CoreMatchers namespaces. The matcher I use in the test is only included in hamcrest-all.
    
    I believe it'd be good start using more of Hamcrest matchers anyway.


---