Re: write performance issue in 3.6.2

2021-05-03 Thread Michael Han
>> because the tests were run with Prometheus enabled,  which is new in 3.6
and has significant negative perf impact.

Interesting, let's see what the numbers are without Prometheus involved. It
could be that the increased latency we observed in CommitProcessor is just
a symptom rather than the cause if Prometheus is the culprit.

In my production environment we use an in house in-process metrics library
which hasn't caused any trouble for performance so far.

>> Does it measure how long a local write op takes in the commit processor
phase?

Yes.

>> I don't think the writes can be processed concurrently.

Correct. There can only be a single inflight write operation that's
committing at any given time. The concurrency is between a single write and
multiple reads where these reads belong to different sessions. Previously,
in 3.4, a single write will block processing of every other operation
including both writes and reads, and in 3.6, a single write will only block
reads that's from the same session; reads from other sessions can proceed
concurrently.

>> Is it correct to say that the new CommitProcessor works best for the
reads in the read/write workloads scenario,

Yes, as previously mentioned, a single write will now only
conditionally block other reads as opposed to unanimously block all reads
which increase overall throughput.


On Mon, May 3, 2021 at 5:06 PM Li Wang  wrote:

> Hi Michael,
>
> Thanks for your additional inputs.
>
> On Mon, May 3, 2021 at 3:13 PM Michael Han  wrote:
>
> > Hi Li,
> >
> > Thanks for following up.
> >
> > >> write_commitproc_time_ms were large
> >
> > This measures how long a local write op hears back from the leader. If
> it's
> > big, then either the leader is very busy acking the request, or your
> > network RTT is high.
> >
>
> Does it measure how long a local write op takes in the commit processor
> phase?
>
> ServerMetrics.getMetrics().WRITE_COMMITPROC_TIME.add(currentTime -
> request.commitProcQueueStartTime);
>
>
> > How does the local fsync time (fsynctime) look like between two tests?
> >
>
> The fsync time looks similar between two tests.
>
> >
> > >> We've found that increasing the maxCommitBatchSize
> >
> > Are you able to successfully tune this value so your benchmark against
> 3.6
> > is on par with 3.4 now? The original report mentioned lots of session
> > timeout and con loss. I am wondering if we can fix this first by tuning
> > this size.
> >
>
> We ran load tests with maxCommitBatchSize as 500 vs 1. 500 was used as we
> have 500 concurrent users in the load test.
> The connection loss error count was reduced about 40% and the session
> expired error was reduced about 45%.
> Tuning maxCommitBatchSize can significantly reduce the errors. We can not
> say the benchmark is on par with 3.4 (i.e. no errors)
> because the tests were run with Prometheus enabled,  which is new in 3.6
> and has significant negative perf impact.
> We will run tests with Prometheus disabled and maxCommitBatchSize as 500
> when we get a chance.
>
> >
> > The major difference of CommitProcessor between 3.4.14 and 3.6.2 is the
> > newly added per session commit queue such that reads and a write from
> > different sessions can be processed concurrently.
>
> Yes, I noticed that there is a queue per session in 3.6, but I don't think
> the writes can be processed concurrently.
> The CommitProcessor is single threaded and the CommitProc worker threads
> are only for reads. Did I miss anything?
>
> This works best for mixed
> > read / write workloads, but for pure write workloads, the new
> > CommitProcessor is not superior, as all writes still have to be
> serialized
> > due to global ordering, plus the per session queue has the overhead for
> > example in this test ZK has to manage 500 queues (and enqueue / dequeue
> and
> > so on cost cycles). Though, I didn't expect this overhead can create
> such a
> > big difference in your test..
> >
>
> Is it correct to say that the new CommitProcessor works best for the reads
> in the read/write workloads scenario,
> as only the reads can be processed concurrently?
>
>
> > Also this is obvious but just want to confirm if the benchmark for two
> > versions of ZK was done on exact same test environment including same OS
> > and networking configuration?
> >
>
> Yes, the benchmark for the two versions was done on the same test
> environment and configuration.
>
> >
> > On Mon, Apr 26, 2021 at 7:35 PM Li Wang  wrote:
> >
> > > Hi Michael,
> > >
> > > Thanks for your reply.
> > >
> > > 1. The

Re: write performance issue in 3.6.2

2021-05-03 Thread Michael Han
Hi Li,

Thanks for following up.

>> write_commitproc_time_ms were large

This measures how long a local write op hears back from the leader. If it's
big, then either the leader is very busy acking the request, or your
network RTT is high.

How does the local fsync time (fsynctime) look like between two tests?

>> We've found that increasing the maxCommitBatchSize

Are you able to successfully tune this value so your benchmark against 3.6
is on par with 3.4 now? The original report mentioned lots of session
timeout and con loss. I am wondering if we can fix this first by tuning
this size.

The major difference of CommitProcessor between 3.4.14 and 3.6.2 is the
newly added per session commit queue such that reads and a write from
different sessions can be processed concurrently. This works best for mixed
read / write workloads, but for pure write workloads, the new
CommitProcessor is not superior, as all writes still have to be serialized
due to global ordering, plus the per session queue has the overhead for
example in this test ZK has to manage 500 queues (and enqueue / dequeue and
so on cost cycles). Though, I didn't expect this overhead can create such a
big difference in your test..

Also this is obvious but just want to confirm if the benchmark for two
versions of ZK was done on exact same test environment including same OS
and networking configuration?

On Mon, Apr 26, 2021 at 7:35 PM Li Wang  wrote:

> Hi Michael,
>
> Thanks for your reply.
>
> 1. The workload is 500 concurrent users creating nodes with data size of 4
> bytes.
> 2. It's pure write
> 3. The perf issue is that under the same load, there were many session
> expired and connection loss errors when using ZK 3.6.2 but no such errors
> in ZK 3.4.14.
>
> The following are some updates on the issue.
>
> 1. We've checked the fine grained metrics and found that the
> CommitProcessor was the bottleneck. The commit_commit_proc_req_queued and
> the write_commitproc_time_ms were large.
> The errors were caused by too many commit requests queued up in the
> CommitProcessor and waiting to be processed.
> 2. We've found that increasing the maxCommitBatchSize can reduce both the
> session expired and connection loss errors.
> 3. We didn't observe any significant perf impact from the RequestThrottler.
>
>
> Please let me know if you or anyone has any questions.
>
> Thanks,
>
> Li
>
>
>
> On Tue, Apr 20, 2021 at 8:03 PM Michael Han  wrote:
>
> > What is the workload looking like? Is it pure write, or mixed read write?
> >
> > A couple of ideas to move this forward:
> > * Publish the performance benchmark so the community can help.
> > * Bisect git commit and find the bad commit that caused the regression.
> > * Use the fine grained metrics introduced in 3.6 (e.g per processor stage
> > metrics) to measure where time spends during writes. We might have to add
> > these metrics on 3.4 to get a fair comparison.
> >
> > For the throttling - the RequestThrottler introduced in 3.6 does
> introduce
> > latency, but should not impact throughput this much.
> >
> > On Thu, Mar 11, 2021 at 11:46 AM Li Wang  wrote:
> >
> > > The CPU usage of both server and client are normal (< 50%) during the
> > test.
> > >
> > > Based on the investigation, the server is too busy with the load.
> > >
> > > The issue doesn't exist in 3.4.14. I wonder why there is a significant
> > > write performance degradation from 3.4.14 to 3.6.2 and how we can
> address
> > > the issue.
> > >
> > > Best,
> > >
> > > Li
> > >
> > >
> > >
> > >
> > >
> > >
> > >
> > >
> > >
> > >
> > >
> > > On Thu, Mar 11, 2021 at 11:25 AM Andor Molnar 
> wrote:
> > >
> > > > What is the CPU usage of both server and client during the test?
> > > >
> > > > Looks like server is dropping the clients because either the server
> or
> > > > both are too busy to deal with the load.
> > > > This log line is also concerning: "Too busy to snap, skipping”
> > > >
> > > > If that’s the case I believe you'll have to profile the server
> process
> > to
> > > > figure out where the perf bottleneck is.
> > > >
> > > > Andor
> > > >
> > > >
> > > >
> > > >
> > > > > On 2021. Feb 22., at 5:31, Li Wang  wrote:
> > > > >
> > > > > Thanks, Patrick.
> > > > >
> > > > > Yes, we are using the same JVM version and GC configurations when
> > &

Re: write performance issue in 3.6.2

2021-04-20 Thread Michael Han
What is the workload looking like? Is it pure write, or mixed read write?

A couple of ideas to move this forward:
* Publish the performance benchmark so the community can help.
* Bisect git commit and find the bad commit that caused the regression.
* Use the fine grained metrics introduced in 3.6 (e.g per processor stage
metrics) to measure where time spends during writes. We might have to add
these metrics on 3.4 to get a fair comparison.

For the throttling - the RequestThrottler introduced in 3.6 does introduce
latency, but should not impact throughput this much.

On Thu, Mar 11, 2021 at 11:46 AM Li Wang  wrote:

> The CPU usage of both server and client are normal (< 50%) during the test.
>
> Based on the investigation, the server is too busy with the load.
>
> The issue doesn't exist in 3.4.14. I wonder why there is a significant
> write performance degradation from 3.4.14 to 3.6.2 and how we can address
> the issue.
>
> Best,
>
> Li
>
>
>
>
>
>
>
>
>
>
>
> On Thu, Mar 11, 2021 at 11:25 AM Andor Molnar  wrote:
>
> > What is the CPU usage of both server and client during the test?
> >
> > Looks like server is dropping the clients because either the server or
> > both are too busy to deal with the load.
> > This log line is also concerning: "Too busy to snap, skipping”
> >
> > If that’s the case I believe you'll have to profile the server process to
> > figure out where the perf bottleneck is.
> >
> > Andor
> >
> >
> >
> >
> > > On 2021. Feb 22., at 5:31, Li Wang  wrote:
> > >
> > > Thanks, Patrick.
> > >
> > > Yes, we are using the same JVM version and GC configurations when
> > > running the two tests. I have checked the GC metrics and also the heap
> > dump
> > > of the 3.6, the GC pause and the memory usage look okay.
> > >
> > > Best,
> > >
> > > Li
> > >
> > > On Sun, Feb 21, 2021 at 3:34 PM Patrick Hunt  wrote:
> > >
> > >> On Sun, Feb 21, 2021 at 3:28 PM Li Wang  wrote:
> > >>
> > >>> Hi Enrico, Sushant,
> > >>>
> > >>> I re-run the perf test with the data consistency check feature
> disabled
> > >>> (i.e. -Dzookeeper.digest.enabled=false), the write performance issue
> of
> > >> 3.6
> > >>> is still there.
> > >>>
> > >>> With everything exactly the same, the throughput of 3.6 was only 1/2
> of
> > >> 3.4
> > >>> and the max latency was more than 8 times.
> > >>>
> > >>> Any other points or thoughts?
> > >>>
> > >>>
> > >> In the past I've noticed a big impact of GC when doing certain
> > performance
> > >> measurements. I assume you are using the same JVM version and GC when
> > >> running the two tests? Perhaps our memory footprint has expanded over
> > time.
> > >> You should rule out GC by running with gc logging turned on with both
> > >> versions and compare the impact.
> > >>
> > >> Regards,
> > >>
> > >> Patrick
> > >>
> > >>
> > >>> Cheers,
> > >>>
> > >>> Li
> > >>>
> > >>>
> > >>>
> > >>>
> > >>>
> > >>>
> > >>>
> > >>>
> > >>>
> > >>>
> > >>>
> > >>>
> > >>>
> > >>> On Sat, Feb 20, 2021 at 9:04 PM Li Wang  wrote:
> > >>>
> >  Thanks Sushant and Enrico!
> > 
> >  This is a really good point.  According to the 3.6 documentation,
> the
> >  feature is disabled by default.
> > 
> > >>>
> > >>
> >
> https://zookeeper.apache.org/doc/r3.6.2/zookeeperAdmin.html#ch_administration
> > >>> .
> >  However, checking the code, the default is enabled.
> > 
> >  Let me set the zookeeper.digest.enabled to false and see how the
> write
> >  operation performs.
> > 
> >  Best,
> > 
> >  Li
> > 
> > 
> > 
> > 
> >  On Fri, Feb 19, 2021 at 1:32 PM Sushant Mane <
> sushantma...@gmail.com>
> >  wrote:
> > 
> > > Hi Li,
> > >
> > > On 3.6.2 consistency checker (adhash based) is enabled by default:
> > >
> > >
> > >>>
> > >>
> >
> https://github.com/apache/zookeeper/blob/803c7f1a12f85978cb049af5e4ef23bd8b688715/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java#L136
> > > .
> > > It is not present in ZK 3.4.14.
> > >
> > > This feature does have some impact on write performance.
> > >
> > > Thanks,
> > > Sushant
> > >
> > >
> > > On Fri, Feb 19, 2021 at 12:50 PM Enrico Olivelli <
> > eolive...@gmail.com
> > >>>
> > > wrote:
> > >
> > >> Li,
> > >> I wonder of we have some new throttling/back pressure mechanisms
> > >> that
> > >>> is
> > >> enabled by default.
> > >>
> > >> Does anyone has some pointer to relevant implementations?
> > >>
> > >>
> > >> Enrico
> > >>
> > >> Il Ven 19 Feb 2021, 19:46 Li Wang  ha scritto:
> > >>
> > >>> Hi,
> > >>>
> > >>> We switched to Netty on both client side and server side and the
> > >>> performance issue is still there.  Anyone has any insights on
> what
> > > could
> > >> be
> > >>> the cause of higher latency?
> > >>>
> > >>> Thanks,
> > >>>
> > >>> Li
> > >>>
> > >>>
> > >>>
> > >>> On 

Re: ConnectionLoss Upon Length Error - Failed to Write > 1 MB

2021-01-08 Thread Michael Han
Server should really check the length of incoming buffer size against the
sum of jute.maxbuffer and an extra configurable padding (reserved for
packet / requests headers). The default padding value is 1024 bytes and now
it's configurable through Java properties. I believe we do use this
combined value for check when deserializing snapshot, but the check against
incoming request buffer is still against the pure jute maxbuffer size,
which creates the not so intuitive experience. I think it's worth having a
consistent buffer length check across server code.

Another aspect of the question is having dedicated error codes - I think
that's a good suggestion and adding a new KeeperException would make for a
better client side experience.

On Fri, Jan 8, 2021 at 5:45 AM Norbert Kalmar 
wrote:

> We see a lot of issues (even on prod systems) around jute.maxbuffer. I
> agree it is not the "cleanest" of errors. If ZK is involved in some issue,
> usually we always check first for signs of requests being too big (a.k.a.
> jute.maxbuffer issue).
>
> But if we wan't to improve on this, we have to make sure it is backward
> compatible. Lot of monitoring is placed around this IMHO. That's why I
> think there wasn't any refactoring around it.
>
> Just my two cents.
>
> - Norbert
>
>
>
> On Fri, Jan 8, 2021 at 10:37 AM Huizhi Lu  wrote:
>
> > OK, I think I get it. The rough sanity check is applied only when
> > deserializing, the len of incoming buffer is read. There is no check
> > for outgoing data when serializing. And there are 10s of bytes in the
> > serialization metadata, so if a client is writing just below 1 MB
> > (1024 * 1024 - 1 bytes), the final incoming buffer data would exceed 1
> > MB and the write would fail. So it's kind of inaccurate that by
> > default zk could store a znode just below 1 MB (1024 * 1024 - 1). To
> > make it accurate, maybe we could just check the bytes length before
> > serializing, and the server could add some extra bytes based on 1 MB.
> > I guess this is minor as it is just a rough sanity check. ZK just does
> > not expect a client would write that large data :)
> >
> > On Fri, Jan 8, 2021 at 12:49 AM Huizhi Lu  wrote:
> > >
> > > From what I've learned and also the doc:
> > >
> > > "jute.maxbuffer : (Java system property:jute.maxbuffer).
> > >
> > > When jute.maxbuffer in the client side is greater than the server
> > > side, the client wants to write the data exceeds jute.maxbuffer in the
> > > server side, the server side will get java.io.IOException: Len error
> > > When jute.maxbuffer in the client side is less than the server side,
> > > the client wants to read the data exceeds jute.maxbuffer in the client
> > > side, the client side will get java.io.IOException: Unreasonable
> > > length or Packet len is out of range!"
> > >
> > > So I assume: the client only honors jute.maxbuffer when reading. If
> > > the client tries to read the data > jute.maxbuffer, it fails. For
> > > writing, jute.maxbuffer is honored on server side, the client does not
> > > do the sanity check.
> > > Correct me if I am wrong. I would really expect the client can also
> > > fail the request if it's writing data > jute.maxbuffer.
> > >
> > > On Fri, Jan 8, 2021 at 12:40 AM Huizhi Lu 
> wrote:
> > > >
> > > > Hi Ted,
> > > >
> > > > Really appreciate your prompt response and detailed explanation!
> > > >
> > > > For some reason, ZK could be abused for writing large data objects.
> > > > I understand we should correctly use ZK for coordination that ZK is
> > best at.
> > > > It's definitely something we could improve how we use ZK. But maybe
> > > > it'd be a long run to arrive.
> > > > Thanks for the clarification :)
> > > >
> > > > Back to the jute maxbuffer setting. With the consistent values 1 MB
> on
> > > > both client and server,
> > > > I am still able to produce it: request is sent to server as it throws
> > > > IOException "Len error" and closes
> > > > the connection. The client log is below, which does not give
> > > > descriptive enough info like "Len error".
> > > > [main-SendThread(localhost:2181)] WARN
> > > > org.apache.zookeeper.ClientCnxn - Session 0x1003e7613ab0005 for sever
> > > > localhost:2181, Closing socket connection. Attempting reconnect
> except
> > > > it is a SessionExpiredException.
> > > > java.io.IOException: Connection reset by peer
> > > >
> > > > With this, can I assume the zk client does not fail the request?
> > > > I also dig into the code, it seems the request reaches the server and
> > > > the server fails the request.
> > > > I am actually expecting the request can be failed earlier on the
> > > > client side and then get descriptive info "the packet size is too
> > > > large".
> > > > Is this (when writing, client jute.maxbuffer is not honored)
> expected?
> > > > I think if the client side fails the request and gives more
> > > > descriptive info/specific exception, that'd be great and it's what I
> > > > would expect.
> > > >
> > > > -Huizhi
> > > >
> > > > 

[jira] [Created] (ZOOKEEPER-3981) Flaky test MultipleAddressTest::testGetValidAddressWithNotValid

2020-10-20 Thread Michael Han (Jira)
Michael Han created ZOOKEEPER-3981:
--

 Summary: Flaky test 
MultipleAddressTest::testGetValidAddressWithNotValid
 Key: ZOOKEEPER-3981
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3981
 Project: ZooKeeper
  Issue Type: Task
  Components: tests
Reporter: Michael Han
Assignee: Michael Han


Problem:

Test MultipleAddressTest::testGetValidAddressWithNotValid might fail 
deterministically when the address it's using, 10.0.0.1, is reachable, as per 
https://tools.ietf.org/html/rfc5735 10.0.0.1 might be allocatable to private 
network usage. In fact, the router address of my ISP is assigned this IP, 
leading to this test always failing for me. 

Solution:

Replace the address with 240.0.0.0, which is reserved for future use and less 
likely to be reachable.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (ZOOKEEPER-3970) Enable ZooKeeperServerController to expire session

2020-10-14 Thread Michael Han (Jira)
Michael Han created ZOOKEEPER-3970:
--

 Summary: Enable ZooKeeperServerController to expire session
 Key: ZOOKEEPER-3970
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3970
 Project: ZooKeeper
  Issue Type: Task
  Components: server, tests
Reporter: Michael Han
Assignee: Michael Han


This is a follow up of ZOOKEEPER-3948. Here we enable ZooKeeperServerController 
to be able to expire a global or local session. This is very useful in our 
experience in integration testing when we want a controlled session expiration 
mechanism. This is done by having session tracker exposing both global and 
local session stats, so a zookeeper server can expire the sessions in the 
controller. 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (ZOOKEEPER-3967) Jetty License Update

2020-10-12 Thread Michael Han (Jira)
Michael Han created ZOOKEEPER-3967:
--

 Summary: Jetty License Update
 Key: ZOOKEEPER-3967
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3967
 Project: ZooKeeper
  Issue Type: Task
  Components: license
Reporter: Michael Han


ZooKeeper server is using Jetty (apache license, v2) for admin server (and for 
more things in future), but we didn't include any of Jetty's copy right / 
notice / license file in ZooKeeper distribution. This ticket is to figure out 
if Jetty license is indeed missing and if so, fix it.

Some previous discussions on Jetty license in ZOOKEEPER-2235 but Jetty somehow 
was not ended up in the patch.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (ZOOKEEPER-3966) Model ZooKeeper data tree using RocksDB primitives to enable on disk data tree storage

2020-10-07 Thread Michael Han (Jira)
Michael Han created ZOOKEEPER-3966:
--

 Summary: Model ZooKeeper data tree using RocksDB primitives to 
enable on disk data tree storage
 Key: ZOOKEEPER-3966
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3966
 Project: ZooKeeper
  Issue Type: Sub-task
  Components: server
Reporter: Michael Han






--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (ZOOKEEPER-3965) Add documentation for RocksDB Snap feature

2020-10-07 Thread Michael Han (Jira)
Michael Han created ZOOKEEPER-3965:
--

 Summary: Add documentation for RocksDB Snap feature
 Key: ZOOKEEPER-3965
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3965
 Project: ZooKeeper
  Issue Type: Sub-task
  Components: documentation
Reporter: Michael Han






--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (ZOOKEEPER-3964) Introduce RocksDB snap and implement change data capture to enable incremental snapshot

2020-10-07 Thread Michael Han (Jira)
Michael Han created ZOOKEEPER-3964:
--

 Summary: Introduce RocksDB snap and implement change data capture 
to enable incremental snapshot
 Key: ZOOKEEPER-3964
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3964
 Project: ZooKeeper
  Issue Type: Sub-task
  Components: rocksdb, server
Reporter: Michael Han
Assignee: Michael Han


This is the first step of enabling on disk storage engine for ZooKeeper by 
extending the existing Snap interface and implement a RocksDB backed snapshot. 
Comparing to file based snapshot, RocksDB based snapshot is superior for big in 
memory data tree as it supports incremental snapshot by only serializing the 
changed data between snapshots. 

High level overview:
 * Extend Snap interface so every thing that's need serialize has a presence on 
the interface.
 * Implement RocksDB based snapshot, and bidirectional conversations between 
File based snapshot and RocksDB snapshot, for back / forward compatibility.
 * Change data capture is implemented by buffering transactions applied to data 
tree, and applied to RocksDB when processing each transaction. An incremental 
snapshot thus only requires RocksDB flush. ZK will always do a full snapshot 
when first loading the data tree during the start process.
 * By default, this feature is disabled. Users need opt in by explicitly 
specify a Java system property to instantiate RocksDBSnap at runtime.

This work is based on top of the patch attached to ZOOKEEPER-3783 (kudos to 
Fangmin and co at FB), with some bug / test fixes and adjustment so it can 
cleanly apply to master branch.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


Re: ApacheCon Bug Bash

2020-10-02 Thread Michael Han
Let's not close these pull requests as invalid as they are with very good
intentions. Can someone on this list from MuseDev please work with the
contributors so they are aware of our contribution guideline:
https://cwiki.apache.org/confluence/display/ZOOKEEPER/HowToContribute

For these pull requests, I think they can be merged as long as they are
reviewed and get green builds. Committers can either help create JIRA and
associate it with PR at merge time, or use their own judgement without a
JIRA if the fix is trivial. I still prefer all contributors to follow the
guidelines so they can get their contribution credits in the JIRA system.

On Fri, Oct 2, 2020 at 8:34 AM Rich Bowen  wrote:

> I know you're not asking me, but with my Community Development hat on, I
> strenuously encourage you to view this as an opportunity to bring on new
> contributors, and couch your response accordingly. Anything that comes
> across as scolding them for Doing It Wrong is going to leave a bad taste
> and possibly lose new contributors, particularly when we invited them to
> participate in this process. We did invite them, and we did point them
> to the issues, via Muse.dev. Perhaps Muse.dev can work with us to
> automate the process of creating tickets for the issues that were raised?
>
> On 10/2/20 11:26 AM, Enrico Olivelli wrote:
> > Hey !
> > it looks like the Bug bash has brought a few Pull Requests
> > https://github.com/apache/zookeeper/pulls
> >
> > Unfortunately they are not following the contribution guidelines (for
> > instance there is no associated JIRA)
> > https://cwiki.apache.org/confluence/display/ZOOKEEPER/HowToContribute -
> >
> > Most of the PR are about trivial fixes, I am not sure if a JIRA is
> deserved.
> >
> > What should we do?
> > My proposal is to ping the contributor in order to obey the guide and
> > then finally accept the patches, as Micheal Han did in this patch
> > https://github.com/apache/zookeeper/pull/1470
> >
> > I don't want to see that patches remaining on github as low hanging
> > fruit, so it is better that we decide how to work on them,
> > another option is to close them as invalid (It would be a pity IMHO)
> >
> > Enrico
> >
> >
> >
> > Il giorno lun 28 set 2020 alle ore 15:03 Tom DuBuisson  > > ha scritto:
> >
> > Enrico,
> > That sounds great.  We'll get the repo activated.
> >
> > Tom
> >
> >
> > On Sun, Sep 27, 2020, 11:11 PM Enrico Olivelli  > > wrote:
> >
> >  > Tom
> >  > Overall I think that we can move forward.
> >  >
> >  > This thread has been around for a while, there are no objections,
> > every
> >  > question has been answered.
> >  >
> >  > Thank you very much
> >  >
> >  > I hope this activity will help in growing Zookeeper project both
> > in code
> >  > quality and with more contributions, that is to help the
> > community to grow.
> >  >
> >  > Best regards
> >  >
> >  > Enrico
> >  >
> >  > Il Lun 28 Set 2020, 01:27 Tom DuBuisson  > > ha scritto:
> >  >
> >  > > Norbert,
> >  > >
> >  > > Yes, you understand that correctly.  And those analyzers are
> > FindSecBugs,
> >  > > Error Prone and Infer.  All open source and in moderate to wide
> use
> >  > > already.  Only find sec bugs is security specific - Infer and
> > Error Prone
> >  > > might find security bugs but they are more general purpose in
> > nature.
> >  > >
> >  > > -Tom
> >  > >
> >  > > On Sun, Sep 27, 2020 at 3:43 PM Norbert Kalmar
> >  > > 
> >  > > wrote:
> >  > >
> >  > > > Hello Tom,
> >  > > >
> >  > > > +1 on the initiative, thanks for bringing this to our
> attention.
> >  > > >
> >  > > > If I understand correctly, there will be no disclosed
> > security issues
> >  > > which
> >  > > > cannot be found with open source static analyzers.
> >  > > >
> >  > > > Regards,
> >  > > > Norbert
> >  > > >
> >  > > >
> >  > > > On Sun, Sep 27, 2020 at 8:23 AM Szalay-Bekő Máté <
> >  > > > szalay.beko.m...@gmail.com  >>
> >  > > > wrote:
> >  > > >
> >  > > > > Hello Guys,
> >  > > > >
> >  > > > > In general I like the idea, but unfortunately I can not
> really
> >  > > > participate
> >  > > > > (either in the coding or in the review) as I have a few
> > important
> >  > > > projects
> >  > > > > close to deadline at the moment.
> >  > > > >
> >  > > > > My only concern is with the security bugs, which I don't
> > like to be
> >  > > > openly
> >  > > > > reported before publishing a release with the fix. But for
> > any other
> >  > > kind
> >  > > > > of bugfixes / improvements, I am very positive with the
> > initiative.
> >  > > > >
> >  > > > >
> >  > > > > Best regards,
> 

Re: [jira] [Commented] (ZOOKEEPER-3945) poker

2020-09-25 Thread Michael Han
Thanks Christopher, just deleted this spam issue.

On Thu, Sep 24, 2020 at 11:40 PM Christopher Tubbs (Jira) 
wrote:

>
> [
> https://issues.apache.org/jira/browse/ZOOKEEPER-3945?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17201944#comment-17201944
> ]
>
> Christopher Tubbs commented on ZOOKEEPER-3945:
> --
>
> If somebody has permission, it might be best to delete this spam issue, in
> case any of the links are malicious.
>
> > poker
> > -
> >
> > Key: ZOOKEEPER-3945
> > URL:
> https://issues.apache.org/jira/browse/ZOOKEEPER-3945
> > Project: ZooKeeper
> >  Issue Type: New Feature
> >  Components: contrib-bindings
> >Affects Versions: 3.4.14
> > Environment: Popularitas poker dengan cepat menjadi permainan
> yang paling banyak dinikmati di dunia. Permainan populer ini dikatakan
> telah sepenuhnya direvolusi oleh Chris Moneymaker, ketika ia menang di
> World Series of Poker pada tahun 2003 setelah menggunakan situs poker
> online untuk memenuhi syarat. Media telah menciptakan sensasi ini dan
> akibatnya yang langgeng “efek Penghasil Uang”. Seorang akuntan rendahan,
> Moneymaker memainkan turnamen satelit tiga puluh sembilan dolar di situs
> poker online PokerStars, memenangkan kursi di acara utama World Series of
> Poker. Itu adalah turnamen langsung pertama Moneymaker, dan dia membawa
> pulang hadiah dua setengah juta dolar. Karena acara yang disiarkan secara
> luas dan dihebohkan ini, poker online meledak ke layar dengan ribuan orang
> lain yang bercita-cita menjadi besar.
> > Dengan ribuan pemain poker yang berharap bermimpi menjadi besar setelah
> menyaksikan kemenangan Chris Moneymaker, popularitas bermain poker online
> melonjak ke ketinggian baru.
> > Bentuk Popularitas poker yang paling umum adalah Texas Holdem.
> > Di Texas Holdem, yang dikatakan berasal dari Robstown, Texas, setiap
> pemain diberikan dua kartu tertutup sementara lima kartu komunitas
> diserahkan oleh dealer. Setiap pemain memiliki kesempatan untuk bertaruh,
> memeriksa, atau melipat setelah setiap kesepakatan. Popularitasnya telah
> meledak selama awal 2000-an, bahkan menggantikan stud tujuh kartu sebagai
> permainan kasino Amerika Serikat yang paling banyak dimainkan.
> > Dengan ribuan pemain poker yang berharap bermimpi menjadi besar setelah
> menyaksikan kemenangan Chris Moneymaker, popularitas bermain poker online
> melonjak ke ketinggian baru. Kemudahan permainan dapat dimainkan secara
> online sangat menarik bagi pemain pemula. Kasino mungkin menakutkan bagi
> pemula. Mereka mungkin juga berlokasi di daerah yang secara geografis tidak
> nyaman, hanya menarik penduduk lokal atau mereka yang mampu membayar
> liburan mahal. Situs poker online juga memiliki taruhan yang lebih rendah
> daripada kasino, karena tidak sebanding dengan waktu operasi kehidupan
> nyata untuk menjalankan permainan dengan pot kecil yang tidak akan membayar
> biaya overhead mereka. Menambahkan tabel untuk permainan dengan berbagai
> biaya dan jenis permainan juga tidak akan menghabiskan ruang online seperti
> di kasino kehidupan nyata.
> > Ada beberapa situs berbeda yang tersedia untuk bermain poker online,
> termasuk PokerStars, Party Poker, William Hill, Everest Poker, dan Bodog.
> Mungkin yang paling terkenal di antara situs untuk bermain poker online
> adalah PokerStars – ruang kartu online terbesar di dunia. Situs penting
> lainnya untuk bermain poker online adalah Bodog Poker. Dibuat oleh Calvin
> Ayre, Bodog adalah ruang poker online AS yang diperdagangkan terbesar dan
> salah satu situs judi online paling tepercaya.
> >Reporter: azziazah
> >Priority: Minor
> > Fix For: 3.7.0
> >
> > Attachments: promosi-welcomebonus-desktop-13-5-2020.jpg
> >
> >
> > Popularitas poker dengan cepat menjadi permainan yang paling banyak
> dinikmati di dunia. Permainan populer ini dikatakan telah sepenuhnya
> direvolusi oleh Chris Moneymaker, ketika ia menang di World Series of Poker
> pada tahun 2003 setelah menggunakan situs poker online untuk memenuhi
> syarat. Media telah menciptakan sensasi ini dan akibatnya yang langgeng
> “efek Penghasil Uang”. Seorang akuntan rendahan, Moneymaker memainkan
> turnamen satelit tiga puluh sembilan dolar di situs poker online
> PokerStars, memenangkan kursi di acara utama World Series of [dewapoker|
> https://linktr.ee/Poker_Online_Indonesia] . Itu adalah turnamen langsung
> pertama Moneymaker, dan dia membawa pulang hadiah dua setengah juta dolar.
> Karena acara yang disiarkan secara luas dan dihebohkan ini, poker online
> meledak ke layar dengan ribuan orang lain yang bercita-cita menjadi besar.
> > Dengan ribuan pemain poker yang berharap bermimpi menjadi besar setelah
> menyaksikan kemenangan Chris Moneymaker, popularitas [dewa poker|
> https://linktr.ee/Poker.Online.Terbesar] bermain poker online melonjak ke
> ketinggian baru.
> > 

[jira] [Created] (ZOOKEEPER-3948) Introduce a deterministic runtime behavior injection framework for ZooKeeperServer testing

2020-09-25 Thread Michael Han (Jira)
Michael Han created ZOOKEEPER-3948:
--

 Summary: Introduce a deterministic runtime behavior injection 
framework for ZooKeeperServer testing
 Key: ZOOKEEPER-3948
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3948
 Project: ZooKeeper
  Issue Type: New Feature
  Components: server, tests
Reporter: Michael Han
Assignee: Michael Han


We'd like to understand how applications built on top of ZooKeeper behave under 
various faulty conditions, which is important to build resilient end to end 
solutions and avoid ZooKeeper being single point of failure. We'd also like to 
achieve this in both unit tests (in process) and integration tests (in and out 
of process). Traditional methods of using external fault injection mechanisms 
are non deterministic and requires non trivial set up and hard to integrate 
with unit tests, so here we introduce the ZooKeeperController service which 
solves both.

The basic idea here is to create a controllable ZooKeeperServer which accepts 
various control commands (such as - delay request, drop request, eat request, 
expire session, shutdown, trigger leader election, and so on), and reacting 
based on incoming commands. The controllable server and production server share 
the same underlying machineries (quorum peers, ZooKeeper server, etc) but code 
paths are separate, thus this feature has no production impact.

This controller system is currently composed of following pieces:

* CommandClient: a convenient HTTP client to send control commands to 
controller service.
* CommandListener: an embed HTTP server listening incoming commands and 
dispatch to controller service.
* Controller Service: the service that's responsible to create controllable ZK 
server and the controller.
* ZooKeeperServerController: the controller that changes the behavior of ZK 
server runtime.
* Controllable Cnx / Factory: controllable connection that accept behavior 
change requests.

In future more control commands and controllable components can be added on top 
of this framework.

This can be used in either unit tests / integration tests as an in process 
embedded controllable ZooKeeper server, or as an out of process stand alone 
controllable ZooKeeper process.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


Re: CI build issues

2020-09-15 Thread Michael Han
Hi Andor,

Here are a set of concrete examples of the dead links of the pre-merge jobs:
https://github.com/apache/zookeeper/pull/1433
https://github.com/apache/zookeeper/pull/1438
https://github.com/apache/zookeeper/pull/1441

The old link I posted (
https://ci-hadoop.apache.org/view/ZooKeeper/job/zookeeper-precommit-github-pr/view/change-requests/job/PR-1380/)
is good now, because I manually triggered the job yesterday :]

I think these broken links are fixable as we can trigger pre-merge jobs for
them, it's just tedious through the UI one by one as I haven't found a good
way to batch trigger pre-commit for all pull requests.

Also, as I just tested, it seems a user can just close and reopen the pull
request to trigger the pre-merge job (although last time I tested this, it
failed to trigger). So this can be a work around for contributors before
the trigger phrase (if any) is ready.

Again thanks for taking time working on getting CI back to good shape!

On Tue, Sep 15, 2020 at 12:55 PM Andor Molnar  wrote:

> Hi Michael,
>
> I was working on the CI migration and there’re still a few things which is
> not available in the new system. I haven’t found any solution for the
> “retest” trigger, but I’ll take another look tomorrow. I need to dig the
> builds@ list if there’s anything happened since I’ve last checked e.g.
> new plugins installed, etc.
>
> I’m not sure I understand your concern about dead links. Here’s the link
> of the pre-commit job for your PR:
>
> https://ci-hadoop.apache.org/view/ZooKeeper/job/zookeeper-precommit-github-pr/view/change-requests/job/PR-1380/
>
> From the Github PR page I can see the following link:
>
> https://ci-hadoop.apache.org/blue/organizations/jenkins/zookeeper-precommit-github-pr/detail/PR-1380/4/pipeline
>
> Which takes me to the Pipeline Report and definitely not dead. (This must
> be some new thing, but looks quite cool.)
>
> OWASP Build - Good point Patrick, that’s still missing.
> I’ve created the branch and PreCommit jobs as Jenkins pipelines, nicely
> committed and tracked in Git. I believe that’s how we should do CI in the
> future. But I was reluctant do the same with flaky-test job which is just a
> copy-and-paste Jenkins job atm.
>
> Feel free to choose your way for the Owasp build, if you’re willing to
> migrate it, but I think at the end of the way we should have everything in
> source control to be perfect.
>
> We still don’t have Windows build either, but I’m not sure if
> Windows-based nodes are available.
>
> Andor
>
>
>
> > On 2020. Sep 13., at 23:02, Michael Han  wrote:
> >
> > Folks,
> >
> > I am seeing some CI build issues. Specifically:
> >
> > * Comment on github PR with "retest maven build" does not trigger a
> rebuild
> > of JenkinsMaven. This used to work. Is this a known issue?
> >
> > * Tons of pre-merge job links on PRs are broken: they actually link to a
> > deleted ci job I created a few days ago to test the new CI system. Here
> is
> > a broken link
> > <
> https://ci-hadoop.apache.org/job/zookeeper_hanm_tests/job/PR-1380/1/display/redirect
> >
> > for reference. Do we know how we can trigger a new pre-merge job on
> > existing PRs so these links can be fixed?
>
>


Re: CI build issues

2020-09-14 Thread Michael Han
Thanks, Enrico. I figured that committers / members who have Jenkins access
can manually trigger a pre-commit job (
https://ci-hadoop.apache.org/job/zookeeper-precommit-github-pr/view/change-requests/job/PR-NUMBER
-> BUILD NOW), so I am unblocked for the moment.

It would be great to get the trigger phrase (the magic) back, so PR
contributors can trigger a CI build when needed. Close and reopen a PR, for
some reasons can't trigger the pre-commit job.


On Sun, Sep 13, 2020 at 11:34 PM Enrico Olivelli 
wrote:

> Micheal, Patrick
>
>
>
>
> Il giorno lun 14 set 2020 alle ore 05:54 Patrick Hunt 
> ha
> scritto:
>
> > Also - how do we get the owasp check added back? Are we manually adding
> > jobs through the UI or is there a new process?
> >
>
> AFAIK Andor is working on the migration, we simply still haven't restored
> it.
>
> This is the Jenkins configuration file for daily builds:
> https://github.com/apache/zookeeper/blob/master/Jenkinsfile
>
> probably we should change that file or add a new like "Jenkinsfile_owasp"
> that runs the OWASP check
> automatically it can pick up all active branches
>
> For reference this one is for pre-commit
> https://github.com/apache/zookeeper/blob/master/Jenkinsfile-PreCommit
>
>
>
>
> >
> > Patrick
> >
> > On Sun, Sep 13, 2020 at 2:02 PM Michael Han  wrote:
> >
> > > Folks,
> > >
> > > I am seeing some CI build issues. Specifically:
> > >
> > > * Comment on github PR with "retest maven build" does not trigger a
> > rebuild
> > > of JenkinsMaven. This used to work. Is this a known issue?
> >
>
> Unfortunately we still haven't a new "retest maven build" magic phrase,
> AFAIK Andor was working on it.
>
>
> Enrico
>
>
>
> > >
> > > * Tons of pre-merge job links on PRs are broken: they actually link to
> a
> > > deleted ci job I created a few days ago to test the new CI system. Here
> > is
> > > a broken link
> > > <
> > >
> >
> https://ci-hadoop.apache.org/job/zookeeper_hanm_tests/job/PR-1380/1/display/redirect
> > > >
> > > for reference. Do we know how we can trigger a new pre-merge job on
> > > existing PRs so these links can be fixed?
> > >
> >
>


CI build issues

2020-09-13 Thread Michael Han
Folks,

I am seeing some CI build issues. Specifically:

* Comment on github PR with "retest maven build" does not trigger a rebuild
of JenkinsMaven. This used to work. Is this a known issue?

* Tons of pre-merge job links on PRs are broken: they actually link to a
deleted ci job I created a few days ago to test the new CI system. Here is
a broken link

for reference. Do we know how we can trigger a new pre-merge job on
existing PRs so these links can be fixed?


Re: [VOTE] Apache ZooKeeper 3.6.2 candidate 1

2020-09-04 Thread Michael Han
+1, with two minor issues:

There is one unit test deterministically failing for me locally on mac:
MultipleAddressesTest.testGetValidAddressWithNotValid Expected exception:
java.net.NoRouteToHostException

Also missing a couple of items in release notes: ZOOKEEPER-3794,
ZOOKEEPER-3797, ZOOKEEPER-3813

Neither is blocker IMO but would be nice to update the release note
considering one of the missing items is CVE related fix.

On Fri, Sep 4, 2020 at 1:04 PM Patrick Hunt  wrote:

> +1. xsum/sig validate. RAT ran clean. Was able to build and do manual
> testing with various ensemble sizes successfully. lgtm.
>
> Patrick
>
> On Fri, Sep 4, 2020 at 6:01 AM Enrico Olivelli 
> wrote:
>
> > This is a release candidate for 3.6.2.
> >
> > It is a minor release and it fixes a few critical issues and brings a few
> > dependencies upgrades.
> >
> > The full release notes is available at:
> >
> >
> https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12310801=12347809
> >
> > *** Please download, test and vote by September 7th 2020, 23:59 UTC+0.
> ***
> >
> > Source files:
> > https://people.apache.org/~eolivelli/zookeeper-3.6.2-candidate-1/
> >
> > Maven staging repo:
> >
> https://repository.apache.org/content/repositories/orgapachezookeeper-1061/
> >
> > The release candidate tag in git to be voted upon: release-3.6.2-1
> > https://github.com/apache/zookeeper/tree/release-3.6.2-1
> >
> > ZooKeeper's KEYS file containing PGP keys we use to sign the release:
> > https://www.apache.org/dist/zookeeper/KEYS
> >
> > The staging version of the website is:
> >
> https://people.apache.org/~eolivelli/zookeeper-3.6.2-candidate-1/website/
> >
> > Should we release this candidate?
> > Enrico Olivelli
> >
>


Re: [VOTE] Apache ZooKeeper release 3.6.2 candidate 0

2020-09-03 Thread Michael Han
Haven't fully tested the RC but I didn't experience any of the lag through
the cli.

On Thu, Sep 3, 2020 at 3:15 PM Ted Dunning  wrote:

> On Thu, Sep 3, 2020 at 1:58 PM Patrick Hunt  wrote:
>
> > On Thu, Sep 3, 2020 at 1:54 PM Ted Dunning 
> wrote:
> >
> > > OK. Did it with the correct version this time. I saw no typing delays
> in
> > > zkCli.sh.
> > ...
> > Hm, no idea - I tried the regular mac terminal (I use iterm2) and also
> > tried launching from sh vs bash but no changes. Very odd.
>
>
> I use the normal terminal on my mac so our environments are very similar.
>


Re: May violate the ZAB agreement -- version 3.6.1

2020-09-01 Thread Michael Han
I can confirm this is a bug now I have a unit test to reproduce the issue.
I will submit a pull request soon. Let's move discussions of this topic to
JIRA and github.

On Fri, Aug 28, 2020 at 10:02 PM li xun <274952...@qq.com> wrote:

> Hi hanm
>
>
>
> Thanks
>
> This is the issue in jira
> https://issues.apache.org/jira/browse/ZOOKEEPER-3911
>
> ———
>
> Below are my thoughts
>
> Before the server becomes the real leader, the follower needs to
> synchronize data with the leader. When encountering big data, it will be
> very slow, causing the server to be temporarily unavailable. Can the leader
> communicate with the follower before the synchronization starts, and
> calculate the maximum zxid_n [reference 1] in the proposal owned by the
> leader that has reached the quorum, and then allow the leader to
> immediately be able to access externally, but only access <=zxid_n Data
> (such as webapp, which can access the leader, which reduces the time that
> zk is inaccessible), there may be two solutions for follower
> 1) Since the follower has not synchronized the data, external webpp access
> is temporarily not allowed, so that even if the data that the follower
> needs to synchronize is large, it will not affect the external service
> provided by zk. But disadvantages: access pressure will be concentrated in
> the leader, at this time the entire cluster does not have the
> characteristics of distributed, prone to single point of failure
> 2) The follower immediately provides services to the outside world, but
> since the follower has not synchronized with the leader, if the follower
> has just experienced a restart, then the follower cannot confirm that it
> currently holds the largest zxid_x that has reached the quorum, and may
> need the follower to do it once Additional inquiry to confirm whether
> zxid_x reaches a quorum. (Or make a separate flag for zxid to indicate
> whether a certain zxid reaches a quorum) Then follower provides access to
> the outside, only access <=zxid_x
> Disadvantages: complex implementation and increased communication volume
>
>
> Reference 1: from  Leslie Lamport 01 Nov 2001
> "
> 2.3 Learning a Chosen Value
> To learn that a value has been chosen, a learner must find out that a pro-
> posal has been accepted by a majority of acceptors. The obvious algorithm
> is to have each acceptor, whenever it accepts a proposal, respond to all
> learners, sending them the proposal. This allows learners to find out about
> a chosen value as soon as possible, but it requires each acceptor to
> respond to each learner—a number of responses equal to the product of the
> number of acceptors and the number of learners.
> The assumption of non-Byzantine failures makes it easy for one learner to
> find out from another learner that a value has been accepted. We can have
> the acceptors respond with their acceptances to a distinguished learner,
> which in turn informs the other learners when a value has been chosen. This
> approach requires an extra round for all the learners to discover the
> chosen value. It is also less reliable, since the distinguished learner
> could fail. But it requires a number of responses equal only to the sum of
> the number of acceptors and the number of learners.
> More generally, the acceptors could respond with their acceptances to some
> set of distinguished learners, each of which can then inform all the
> learners when a value has been chosen. Using a larger set of distinguished
> learners provides greater reliability at the cost of greater communication
> complexity.
> Because of message loss, a value could be chosen with no learner ever
> finding out. The learner could ask the acceptors what proposals they have
> accepted, but failure of an acceptor could make it impossible to know
> whether or not a majority had accepted a particular proposal . In that
> case, learners will find out what value is chosen only when a new proposal
> is chosen. If a learner needs to know whether a value has been chosen, it
> can have a proposer issue a proposal, using the algorithm described above.
> “
>
>
>
> Best,
> li xun
>
>
>
> 2020年8月29日 10:59,Michael Han  写道:
>
> Hi Xun,
>
> I think this is a bug, your test case is sound to me. Do you mind
> creating a JIRA for this issue?
>
> Followers should not ACK NEWLEADER without ACK every transaction from the
> DIFF sync. To ACK every transaction, a follower either persists the
> transaction in log, or takes a snapshot before sending the ACK of the
> NEWLEADER (which we did, before ZOOKEEPER-2678 where the snapshot
> optimization was introduced).
>
> A potential fix I have in mind is to make sur

Re: May violate the ZAB agreement -- version 3.6.1

2020-08-28 Thread Michael Han
Hi Xun,

I think this is a bug, your test case is sound to me. Do you mind
creating a JIRA for this issue?

Followers should not ACK NEWLEADER without ACK every transaction from the
DIFF sync. To ACK every transaction, a follower either persists the
transaction in log, or takes a snapshot before sending the ACK of the
NEWLEADER (which we did, before ZOOKEEPER-2678 where the snapshot
optimization was introduced).

A potential fix I have in mind is to make sure to persist all DIFF sync
proposals from LEADER (similar to what we are already doing for proposals
coming between NEWLEADER and UPTODATE). By doing so, when the leader
receives NEWLEADER ACK from a quorum, it's guaranteed that
every transaction leader DIFF sync to follower is quorum committed. Thus
there will not be inconsistent views moving forward. Alternatively we can
take a snapshot before ACK NEWLEADER but that will be a big performance hit
for big data trees.

I am also interested to hear what others think about this.

On Fri, Aug 28, 2020 at 12:20 AM li xun <274952...@qq.com> wrote:

> There is a example in the link, would you understand what I mean?
>
>
>
> https://drive.google.com/file/d/1jy3kkVQTDYGb4iV1RaPMBbEWLZZltTQG/view?usp=sharing
>
> Since version 3.4, the quorum of followers and the leader did not
> synchronize the files immediately when the synchronization was completed,
> and the data was not persisted to the files in an instant, and at this time
> the zk server can provide external access, such as webapp access, if it
> appears at this time Failure, phantom reading may occur
>
>
> > 2020年8月28日 14:51,Justin Ling Mao  写道:
> >
> > @李珣The situation you describe may have conceptual deviations about how
> the consensus protocol works:---> Since the data of the follower when the
> follower uses the DIFF method to synchronize with the leader is still in
> the memory, it has not had time to persist1. The write path is: write
> transaction log(WAL) firstly, after reaching a consensus, then apply to
> memory, other than the opposite.
> > ---> but at this time, the latest zxid_n of the leader has not been
> supported by the quorum of the follower. At this time, if a client connects
> to the leader and sees zxid_n,2. If a write has not been supported by the
> quorum, it's not safe to apply to the state machine and the client is not
> able to see this write.
> > I guess that your question may be: how the system handles the
> uncommitted logs when leader changes?
> >
> >
> >
> > - Original Message -
> > From: Ted Dunning 
> > To: dev@zookeeper.apache.org
> > Subject: Re: May violate the ZAB agreement -- version 3.6.1
> > Date: 2020-08-28 01:25
> >
> > How is it that participant A would have a later zxid than the leader?
> > In particular, it seems to me that it should be impossible to have these
> > two facts be true:
> > 1) a transaction has been committed with zxid = z_0. This implies that a
> > quorum of the cluster has accepted this transaction and it has been
> > committed.
> > 2) a new leader election nominates a leader with latest zxid < z_0.
> > My reasoning is that any new leader election has to involve a quorum and
> at
> > least a sufficient number of that quorum must have accepted zxid >= z_0
> and
> > therefore would refuse to be part of the quorum (this is a
> contradiction).
> > Thus, no leader could be elected with zxid < z_0 if fact (1) is true.
> > What you are describing seems to require both of these facts.
> > Perhaps I am missing something about your suggested scenario. Could you
> > describe what you are thinking in more detail?
> > On Thu, Aug 27, 2020 at 2:08 AM 李珣 <274952...@qq.com> wrote:
> >> version 3.6.1
> >> org.apache.zookeeper.server.quorum.Learner.java line:605
> >> Suppose there is a situation
> >> zxid_n is the largest zxid of Participant A (the leader has just resumed
> >> from downtime). Zxid_n has not been recognized by the quorum. Assuming
> >> Participant A is elected as the Leader, then if a follower appears to
> use
> >> DIFF to synchronize data with the Leader, Leader After sending the
> >> UPTODATE, the leader can already provide external access, but at this
> time,
> >> the latest zxid_n of the leader has not been supported by the quorum of
> the
> >> follower. At this time, if a client connects to the leader and sees
> zxid_n,
> >> then at this time both the leader and the follower are down. For some
> >> reason, the leader cannot be started, and the follower can start
> normally.
> >> At this time, a new leader can only be elected from the follower. Since
> the
> >> data of the follower when the follower uses the DIFF method to
> synchronize
> >> with the leader is still in the memory, it has not had time to persist,
> >> then this The newly elected leader does not have the data of zxid_n, but
> >> before zxid_n has been seen by the client on the old leader, there will
> be
> >> inconsistencies in the data view.
> >> Is the above situation possible?
>
>


Re: Creating a public fork of ZooKeeper

2020-08-16 Thread Michael Han
Have you considered a private fork? That might be a better option here as
it provides additional flexibilities on what to develop (e.g. patented or
company specific tech), when and what to upstream and so on. The downside
of private fork I experienced was losing the incentive to upstream which
creates divergence between in house code base and open source, leading to
accumulated tech debt.

A public fork would also work based on the use case description with
caveats already pointed out by others commented in this thread. As long as
the community is united and does not diverge as a result of the fork (think
Presto).

On Thu, Aug 13, 2020 at 10:51 PM Enrico Olivelli 
wrote:

> Hunter,
> If you simply want to fork the repository and use it in your company,
> respecting the Apache license then in my opinion there is no issue.
> As Christopher said problems will arise if you redistribute that code or
> binaries outside the limits of Apache license, call it 'Zookeeper'...
>
> We are releasing source code in order to let users pick it up and use it.
>
> I suggest you to keep in touch with the community and do not diverge too
> much from the community code base.
>
> If you have new big features or changes the right place to discuss them is
> here in this list.
>
> I saw cases in other ASF projects in which companies decided to go their
> way and when they came back to the ASF to merge their fork it took much
> time.
>
> We are here and open to new features and changes
>
> Thanks for choosing Zookeeper
>
> Enrico
>
> Il Ven 14 Ago 2020, 03:35 Christopher  ha scritto:
>
> > Hi Hunter,
> >
> > A public fork is certainly an option under the Apache License. Many
> > contributors maintain their own public forks on GitHub to enable them
> > to contribute via a pull request on GitHub. While most contributors
> > don't build and reproduce artifacts from their forks, they could do
> > so, in theory.
> >
> > If you're generating and redistributing artifacts, the one thing I
> > would be very careful about is trademark infringement. Please bear in
> > mind that the ASF owns the ZooKeeper trademarks, and careless
> > redistribution under that name could lead to confusion in the
> > marketplace, and can harm the ZooKeeper community. I'm not a lawyer,
> > and you should seek your own legal advice, but I would take care to
> > clarify that any modified artifacts that are redistributed are marked
> > in a way that distances them from those produced by the ZooKeeper PMC
> > within the ASF, so as to avoid trademark infringement and user
> > confusion (e.g. by not calling your derivative work "ZooKeeper").
> >
> > And, while I can't speak for the ZooKeeper PMC (I'm not on that PMC),
> > from my perspective as an ASF member and committer on other projects,
> > I think most projects at Apache would prefer to encourage
> > contributions and engagement with their existing community, rather
> > than see it fractured by independent groups operating apart from their
> > community. There's nothing stopping you from creating your own,
> > independent community, of course... but creating and growing
> > communities is sort of what we do here at the ASF. So, contributing to
> > the existing community is strongly encouraged.
> >
> > On Thu, Aug 13, 2020 at 8:11 PM Hunter Lee 
> > wrote:
> > >
> > > Hi ZooKeeper devs -
> > >
> > > I wanted to get an opinion on public, open-source forks of ZooKeeper. I
> > am aware that there are companies maintaining private or public forks of
> > ZooKeeper to deploy & develop their own flavor of ZooKeeper (or
> ZooKeeper +
> > alpha), like to have a public fork with our own potential use
> case-specific
> > features or bug fixes, etc.
> > >
> > > Is the ZooKeeper PMC generally okay with the idea? I wanted to see if
> > there were any guidelines or advice regarding having a public fork.
> > >
> > > The idea is to give us the flexibility to generate artifacts on
> modified
> > source code (instead of solely relying on official artifacts from Maven
> > Central) and give ourselves an opportunity to contribute back to
> upstream &
> > collaborate in the future.
> > >
> > > Thanks,
> > > Hunter
> >
>


Re: ZK Transaction API multi()

2020-07-28 Thread Michael Han
I agree with Ted's comments on the philosophy of scaling and the need to
recheck your use case to justify if ZooKeeper is the long term solution or
not.

That said, I was in a similar position and had gone through similar scaling
challenges for ZooKeeper so I could probably provide some suggestions which
might serve as a short term solution.

* Obvious ones - more powerful hardware with better IOPS and bigger memory.
* Run a modern version of ZooKeeper (3.5.5+ or 3.6.0+).
* Don't use participants to serve traffic. Use observers only.
* Tune SyncRequestProcessor to allow more throughput at a cost of higher
latency - specifically max batch size and flush delay.
* Tune CommitProcessor to favor more writes instead of reads (depends on
your actual workload).
* Consider using response cache to reduce pressure on JVM Eden space.
* JVM tuning - hard to provide concrete advice but the session expiration
is likely caused by JVM GC. try different options based on profiling and
workload characteristics.
* Client auditing - making sure all traffic from your client is legitimate.
This is often overlooked, but surprisingly prevalent as root causes of ZK
meltdown in practice from time to time.

These are some general guidelines that might help. As with any performance
tuning, the general approach should scope your workload, do some profiling,
identify bottleneck(s), and apply tunings accordingly. Good luck.

On Mon, Jul 27, 2020 at 10:53 PM Enrico Olivelli 
wrote:

> Huizhi,
> If you want to achieve total atomic broadcast and have a greater throughput
> you can consider using Zookeeper brother Apache Bookkeeper, that is built
> over ZK, it is very lightweight and scalable (no central coordination
> servers).
>
> https://bookkeeper.apache.org
>
> Hope that helps
> Enrico
>
> Il Mar 28 Lug 2020, 01:43 Huizhi Lu  ha scritto:
>
> > Hi Ted,
> >
> > Thank you so much for the reply. Your suggestion is very valuable. I do
> > agree that we should migrate from ZK to a distributed DB for this high
> > number of writes. Due to legacy codebase and usage, it may not be that
> easy
> > for us to do that. So we are considering multi() as a short/mid term
> > solution. Finally we will move the excessive number of writes out of ZK
> to
> > achieve higher scalability.
> >
> > Lastly, I greatly appreciate your insightful explanation! FYI, I am very
> > happy to receive prompt replies from you, Ted!
> >
> > Best,
> > -Huizhi
> >
> > On Mon, Jul 27, 2020 at 4:30 PM Ted Dunning 
> wrote:
> >
> > >
> > > This sounds like you are using ZK outside of the intended design. The
> > idea
> > > is that ZK is a coordination engine. If you have such high write rates
> > that
> > > ZK is dropping connections, you probably want a distributed database of
> > > some kind, perhaps one that uses ZK to coordinate itself. ZK is a form
> of
> > > replicated database, not a distributed one and, as such, the write rate
> > > doesn't scale and that is intentional.
> > >
> > > Even if multi() solves your immediate problem, it leaves the same
> problem
> > > in place at just a slightly higher scale. My own philosophy of scaling
> is
> > > that when you hit a problem, you should increase your scale by a large
> > > enough factor to give you time to solve some other problems or build
> new
> > > stuff before you have to fix your scaling problem again. Increasing
> scale
> > > by a factor of 2 rarely does this. I prefer to increase my scaling
> bounds
> > > by a factor of 10 or more so that I have some breathing space. I
> > remember a
> > > time in one startup where our system was on the edge of breaking and
> our
> > > traffic was doubling roughly every week. We had to improve our
> situation
> > by
> > > at least a factor of 10 each time we upgraded our systems just to stay
> in
> > > the same place. I can only hope you will have similar problems.
> > >
> > >
> > >
> > > On Mon, Jul 27, 2020 at 3:47 PM Huizhi Lu  wrote:
> > >
> > >> Hi Ted,
> > >>
> > >> Thank you very much for the reply! I didn't receive the reply in my
> > email
> > >> but I found it in ZK dev mail thread. So I could not reply directly to
> > the
> > >> thread.
> > >>
> > >> I really appreciate a reply from the original author of multi()! And
> > your
> > >> blog (A Tour of the Multi-update For Zookeeper) is very helpful with
> > >> understanding of multi(). Your reply helps convince my team that it
> is a
> > >> real transaction.
> > >>
> > >> Regarding my 2nd question, maybe I should have described a bit of our
> > >> challenge. When we have a large number of ZK write requests that cause
> > high
> > >> ZK write QPS, ZK sessions are expired by ZK. And this affects the
> > >> application's connection to ZK. We wonder if we could apply multi() to
> > >> batch the ZK write requests to reduce ZK write QPS so ZK wouldn't
> expire
> > >> sessions. So in this case, do you think we could still not apply
> > multi() to
> > >> achieve the purpose?
> > >>
> > >> Thank you, Ted!!
> > >>
> > >> On Mon, Jul 

Re: Need Help with Maven Build

2020-05-19 Thread Michael Han
hi jun - which maven version you are using?

If it's 3.5.x, try upgrade to 3.6.x. I had the exact same issue a while
back and upgrade maven fixed this, so I didn't bother to debug. That said,
it's interesting to understand why we failed under specific version of
maven / env, so cc dev list where we have a few maven experts who might be
able to help debug.


On Tue, May 19, 2020 at 8:34 AM Jun Wang  wrote:

> Hi
>
> I got following build error with latest code from github.  But build is
> fine with downloaded source code.   Any suggestion is appreciated.
>
> [ERROR] Failed to execute goal
> org.apache.maven.plugins:maven-compiler-plugin:3.8.1:compile
> (default-compile) on project zookeeper: Fatal error compiling:
> java.lang.NullPointerException -> [Help 1]
>
>
> https://gist.githubusercontent.com/wj1918/b1bcea0473b9ff2096ffa22e3c387e8f/raw/8c2ccfb7919470e0e874abdec5633976720e3dca/zookeeper.build.error.txt
>
> Thanks
> Jun


Re: Cutting 3.6.1 HEADS UP !

2020-04-10 Thread Michael Han
would be good to get ZOOKEEPER-3793 in this release. Without ZOOKEEPER-3793
zk will not throttle requests at all, given RequestThrottler is disabled by
default (and requires some understanding to configure it correctly).

On Thu, Apr 9, 2020 at 10:04 PM Enrico Olivelli  wrote:

> Il Ven 10 Apr 2020, 04:35 Christopher  ha scritto:
>
> > I don't anticipate any issues, but I can test Accumulo with the
> > release candidate when it's ready.
> > Do you already have the binary tarball built and uploaded somewhere?
> >
>
> Not yet.
> I will do it today or in the weekend.
> Btw if you build from release-3.6.1 on Linux with JDK8 the results should
> be the same.
> You can use the docker env
> dev/docker/run.sh
>
> I will use a Fedora 31 box directly
>
> Enrico
>
>
> > On Thu, Apr 9, 2020 at 2:43 PM Enrico Olivelli 
> > wrote:
> > >
> > > Il Gio 9 Apr 2020, 20:33 Norbert Kalmar 
> > ha
> > > scritto:
> > >
> > > > Hi Enrico,
> > > >
> > > > Thanks for driving this!
> > > >
> > > > I managed to build HBase with ZooKeeper 3.5.7 having cherry-picked
> the
> > > > getRevision() patch. I know it's not 3.6.x, but I found the problem
> > with
> > > > this 3.5.7 and fixed it according to this on 3.6 as well. So it
> should
> > be
> > > > fine now.
> > > >
> > >
> > > Thank you
> > >
> > > Enrico
> > >
> > > >
> > > > - Norbert
> > > >
> > > > On Thu, Apr 9, 2020 at 11:00 AM Enrico Olivelli  >
> > > > wrote:
> > > >
> > > > > Hi,
> > > > > I am going to prepare a release candidate for ZooKeeper 3.6.1.
> > > > >
> > > > > There is no JIRA issue with fixVersion = 3.6.1 that is unresolved.
> > > > >
> > > > > I have tested a few projects that had compatibility issues and they
> > are
> > > > > resolved (like Apache BookKeeper and other non OS projects in my
> > > > company).
> > > > >
> > > > > I remember that Norbert pointed a problem with HBase client, that
> > uses
> > > > > getRevision() method, it would be super great to have some feedback
> > of
> > > > > compatibility of HBase with 3.6.1 client.
> > > > >
> > > > > I have deployed the snapshots to snapshots.apache.org, this way
> you
> > can
> > > > > easily test your project in CI, even on Travis.
> > > > >
> > > > > I have created the work branch for release
> > > > > https://github.com/apache/zookeeper/tree/release-3.6.1
> > > > >
> > > > > Please ping me if you have any questions or concerns or you need to
> > add
> > > > new
> > > > > items.
> > > > >
> > > > > I will start the release procedure once I have self validated the
> > status
> > > > of
> > > > > that branch
> > > > >
> > > > > Stay tuned
> > > > >
> > > > > Enrico
> > > > >
> > > >
> >
>


[jira] [Created] (ZOOKEEPER-3793) Request throttling is broken when RequestThrottler is disabled or configured incorrectly.

2020-04-10 Thread Michael Han (Jira)
Michael Han created ZOOKEEPER-3793:
--

 Summary: Request throttling is broken when RequestThrottler is 
disabled or configured incorrectly.
 Key: ZOOKEEPER-3793
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3793
 Project: ZooKeeper
  Issue Type: Bug
  Components: server
Affects Versions: 3.6.0
Reporter: Michael Han
Assignee: Michael Han


When RequestThrottler is not enabled or is enabled but configured incorrectly, 
ZooKeeper server will stop throttling. This is a serious bug as without request 
throttling, it's fairly easy to overwhelm ZooKeeper which leads to all sorts of 
issues. 

This is a regression introduced in ZOOKEEPER-3243, where the total number of 
queued requests in request processing pipeline is not taking into consideration 
when deciding whether to throttle or not, or only taken into consideration 
conditionally based on RequestThrottler's configurations. We should make sure 
always taking into account the number of queued requests in request processing 
pipeline before making throttling decisions.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


Re: Implement checksum against malformed packets

2020-04-08 Thread Michael Han
>> Why?

A packet can be captured, altered its payload, and recomputed its
checksum(s) while transmitting, that's why encryption is needed. This
example sounds pedantic but serving the purpose of the argument.
With encryption the data integrity is ensured during transmission at least,
but data can still be corrupted (after being decrypted and put in a payload
buffer and then bit flip happens). This is really an end to end problem and
the only way to guarantee integrity is to figure out what the "ends" are
and checksum accordingly. I feel there is no need to over work on this in
our case given the probabilities of this happening in practice and the
impact.

On Wed, Apr 8, 2020 at 1:49 AM Ted Dunning  wrote:

> The only way to corrupt a message transmitted on an encrypted link is to
> break the encryption.  There is a tiny argument to be made that messages
> should be checksummed for protection even without the encryption, but these
> links should always be encrypted.
>
> The chance of a 160 bit checksum being corrupted transparently by accident
> is simply not worth considering. The TCP checksum is definitely small
> enough to see accidental transparent corruption, but we are talking about
> very different situation with much longer checks.
>
> On Wed, Apr 8, 2020, 1:40 AM Andor Molnar  wrote:
>
> > Hi Michael,
> >
> > "Checksum itself (even cryptographic checksum) can’t guarantee data
> > integrity as packet in flight can be altered along with the new
> checksum.”
> >
> > Why?
> >
> > If either checksum or the payload was altered, the packet will be
> > retransmitted. Furthermore we could implement error correction codes like
> > Reed-Solomon in order to avoid retransmission, but I don’t have practical
> > experience with that, so might be a crazy idea. :)
> >
> > Andor
> >
> >
> >
> > > On 2020. Apr 8., at 3:17, Michael Han  wrote:
> > >
> > >>> but not bulletproof?
> > >
> > > tcp checksum is weak and can't detect all corrupt packet; and bit flip
> > can
> > > happen anywhere within the stack, and can be caused by malfunction of
> > ram,
> > > or say, cosmic rays.
> > > Statistically a corrupt packet survives all layers of check is low, and
> > > even a packet is corrupted, app layer behavior might still be ok.
> > >
> > >>> wire encryption probably resolves it for free.
> > >
> > > This would be my recommendation here as well to ensure data integrity
> > > during transmission. Checksum itself (even cryptographic checksum)
> can't
> > > guarantee data integrity as packet in flight can be altered along with
> > the
> > > new checksum.
> > >
> > >
> > >
> > > On Tue, Apr 7, 2020 at 1:37 AM Andor Molnar  wrote:
> > >
> > >> Hi Zk community,
> > >>
> > >> I’ve been looking at the following ticket for a while and made me
> > thinking:
> > >> ZOOKEEPER-2175 Checksum validation for malformed packets needs to
> > handle.
> > >> https://issues.apache.org/jira/browse/ZOOKEEPER-2175
> > >>
> > >> It has been found in 3.4.6 originally and not resolved since then
> saying
> > >> that wire encryption probably resolves it for free. I have doubts
> > against
> > >> the issue could actually happen at all, but the evidence in the jira
> is
> > >> quite convincing.
> > >>
> > >> 1) But what if somebody don’t want the overhead of using encryption?
> > >> 2) How often this issue could ever happen given that TCP packets have
> > >> their own checksum which also protects the payload, but not
> bulletproof?
> > >> 3) Found a suggestion in a comment to send session id in the payload
> of
> > >> Ping packets, but malformed packets could happen in any other
> > >> communication, not just session ids.
> > >> 4) Does GRPC have a built-in mechanism to protect the communication?
> > Maybe
> > >> this would be the best way forward.
> > >>
> > >> What do you think?
> > >>
> > >> Andor
> > >>
> > >>
> > >>
> >
> >
>


Re: Implement checksum against malformed packets

2020-04-07 Thread Michael Han
>> but not bulletproof?

tcp checksum is weak and can't detect all corrupt packet; and bit flip can
happen anywhere within the stack, and can be caused by malfunction of ram,
or say, cosmic rays.
Statistically a corrupt packet survives all layers of check is low, and
even a packet is corrupted, app layer behavior might still be ok.

>> wire encryption probably resolves it for free.

This would be my recommendation here as well to ensure data integrity
during transmission. Checksum itself (even cryptographic checksum) can't
guarantee data integrity as packet in flight can be altered along with the
new checksum.



On Tue, Apr 7, 2020 at 1:37 AM Andor Molnar  wrote:

> Hi Zk community,
>
> I’ve been looking at the following ticket for a while and made me thinking:
> ZOOKEEPER-2175 Checksum validation for malformed packets needs to handle.
> https://issues.apache.org/jira/browse/ZOOKEEPER-2175
>
> It has been found in 3.4.6 originally and not resolved since then saying
> that wire encryption probably resolves it for free. I have doubts against
> the issue could actually happen at all, but the evidence in the jira is
> quite convincing.
>
> 1) But what if somebody don’t want the overhead of using encryption?
> 2) How often this issue could ever happen given that TCP packets have
> their own checksum which also protects the payload, but not bulletproof?
> 3) Found a suggestion in a comment to send session id in the payload of
> Ping packets, but malformed packets could happen in any other
> communication, not just session ids.
> 4) Does GRPC have a built-in mechanism to protect the communication? Maybe
> this would be the best way forward.
>
> What do you think?
>
> Andor
>
>
>


Re: ZooKeeper snapCount Tuning

2020-04-03 Thread Michael Han
The workload is a more deciding factor than hardwares when tuning
zookeeper.snapCount and other config parameters, under current ZK
implementation. I am afraid there is no universal value that applicable to
every case although we can provide recommended settings by benchmarking
predictable and usual workloads. In any cases, larger snap count leading to
less frequent snapshotting, which should improve system performance but at
the cost of longer recovery time.

For new hardwares, I think most of time ZK just get the benefit for free.
For preallocation, I agree and think that'll still be useful, as that's a
file system thing and work regardless of underlying medium. To get optimal
usage of the new hardware would require more thought, and just borrow some
ideas from database world that might applicable to ZK:
* Off loading snapshot to dedicated hardware accelerator like FPGA.
* SyncRequestProcessor can flush transaction to NVRam without buffering and
group commit.
* Durable ZK data tree on NVRam that does not require WAL and snapshot.

I suspect not much going on here as ZK, unlike databases, never received
enough workloads (which is a by design) that justifies the investment.



On Fri, Apr 3, 2020 at 1:34 PM Ted Dunning  wrote:

> On Fri, Apr 3, 2020 at 10:01 AM Patrick Hunt  wrote:
>
> > ...
> > Makes sense. For eg. SSD characteristics are vastly diff from spinning
> > media.
>
>
> super true.
>
>
> > I suspect it would be worth looking into this in even more depth -
> > we pre-allocate certain files, perhaps that's no longer necessary, etc...
> >
>
> The preallocation still makes sense on most file systems since meta-data
> changes (i.e. changing file length) are much more expensive than data
> changes (overwriting previously allocated blocks).
>
> Makes sense. If we do something it would be great to have a set of tests
> > that could be used/reused to explore the various types even beyond SSD
> > itself.
> >
>
> Indeed. Storage class memory, for example, could make for an amazing ZK
> implementation. So could use of the upcoming SSD devices that implement
> key-value stores.
>
>
> >
> > Regards,
> >
> > Patrick
> >
> >
> > > My hypothesis is: with a larger snapCount value, ZK can have higher
> > > throughput because it is spending less time creating snapshots.
> > >
> > > Thanks!
> > >
> >
>


Re: [DISCUSS] Sending 3.4 release line to End-Of-Life status

2020-04-01 Thread Michael Han
+1.

For EOL policy statement, just to throw something out here that i can think
of:

* Define what EOL means (such as: not supported by community dev team
anymore, no future 3.4 releases .. still accessible at download page for X
years..) and a date of EOL.

* Provide guidelines for upgrading paths to 3.5 / 3.6.

* State interoperability guarantees another post pointed out previously ^

On Wed, Apr 1, 2020 at 2:04 AM Andor Molnar  wrote:

> Hi folks,
>
> Based on Enrico’s latest post about a 3.4 client problem I’d like to push
> this initiative.
> Asking more senior members of the community what communicated policy is
> needed exactly to say 3.4 is EoL?
>
> In terms of timing I’d like Patrick’s suggestion about 1st of June, 2020.
>
> Any objections?
>
> Andor
>
>
>
>
> > On 2020. Mar 4., at 18:45, Michael K. Edwards 
> wrote:
> >
> > I think it would be useful for an EOL statement about 3.4.x to include a
> > policy on interoperability of newer ZooKeeper servers with 3.4.x client
> > code.  Stacks that build on top of Kafka and Hadoop (I'm looking at you,
> > Spark) often wind up having an indirect dependency on a comically stale
> > ZooKeeper library.  Even if this library isn't really exercised by the
> > client side of the stack, it's there in the mountain of jars; and when
> > application code also wants to use ZooKeeper more directly, using a newer
> > client library can get kind of messy.  The approach I've taken has been
> to
> > rebuild large swathes of the stack around a consistent, recent ZooKeeper
> > build; but I think it would be relevant to a lot of people to know
> whether,
> > say, a 3.4.14 client will work reliably with a 3.6.x quorum.
> >
> > On Wed, Mar 4, 2020 at 9:28 AM Enrico Olivelli 
> wrote:
> >
> >> Il giorno mer 4 mar 2020 alle ore 17:23 Patrick Hunt
> >>  ha scritto:
> >>>
> >>> It seems like we should have a stated/communicated policy around
> release
> >>> lifecycles before sending an EOL message. That way folks have some
> runway
> >>> to plan for the event, both near term (3.4) as well as long term.
> >>
> >> Shall we set a deadline ?
> >> Something like "3.4 will be EOL by the end of 2020" ?
> >> At this point we are only "discussing" about sending 3.4 to EOL, no
> >> decision has been made yet
> >>
> >>
> >> Enrico
> >>
> >>
> >>>
> >>> Patrick
> >>>
> >>> On Wed, Mar 4, 2020 at 5:16 AM Szalay-Bekő Máté <
> >> szalay.beko.m...@gmail.com>
> >>> wrote:
> >>>
>  Also a minor thing to consider: we wanted to ask the HBase community
> to
>  upgrade to ZooKeeper 3.5 before, and the conclusion there was that
> they
>  will do so only when the EOL will be at least scheduled / announced on
> >> the
>  ZooKeeper 3.4 versions. Maybe there are other ZooKeeper users as well
> >> who
>  will not upgrade until they get 'an official' statement about the 3.4
>  versions.
> 
>  On Wed, Mar 4, 2020 at 1:44 PM Jordan Zimmerman <
>  jor...@jordanzimmerman.com>
>  wrote:
> 
> > I'm +1 on this. We're planning to drop support for 3.4.x in the next
> > release of Apache Curator, FYI.
> >
> > -Jordan
> >
> >> On Mar 4, 2020, at 7:36 AM, Enrico Olivelli 
>  wrote:
> >>
> >> Hi,
> >> we are releasing 3.6.0 (I am waiting for mirrors to sync before
> >> updating the website).
> >>
> >> In my opinion it is time to officially send 3.4 branch to EOL
> >> status,
> > that is:
> >> - we are not expecting new releases
> >> - drop 3.4 from download area (it will stay on archives as usual)
> >> - strongly encourage people to update to 3.5/3.6
> >>
> >> 3.4 is far away from master branch and even from 3.6.
> >> There is a clean upgrade path from 3.4.LATEST to 3.5.7 and to 3.6
> >> so
> >> users are able to upgrade.
> >>
> >> I am not sure we need a VOTE, if we simply agree I can drop 3.4
> >> from
> >> the "dist" are as long as I push the new website.
> >>
> >> Best regards
> >> Enrico
> >
> >
> 
> >>
>
>


[Job Ad] Twitter is hiring ZooKeepers

2020-03-09 Thread Michael Han
Hi fellow ZooKeeper developers,

I have open software engineering positions

in my team and would love to hear from anyone who might be interested. The
work is around ZooKeeper and new abstractions built on top of ZooKeeper.
Positions are available for San Francisco, Seattle, and remote (yes,
remote) within United States.

If you are interested, please don't hesitate to reach out to me directly,
or apply on our job site

.

Michael.


Re: [VOTE] Apache ZooKeeper 3.6.0 candidate 4

2020-03-03 Thread Michael Han
+1

- verified checksum/sig.
- verified release notes.
- verified regenerated documentations.
- verified both java and c unit tests pass (ubuntu 18 / java11).
- verified with a few jetty admin commands and zk cli commands.

On Tue, Mar 3, 2020 at 2:24 AM Flavio Junqueira  wrote:

> +1 (binding)
>
> - Built from sources (there are a good number of flaky tests, but it
> eventually built correctly)
> - Checked LICENSE and NOTICED
> - Checked release notes
> - Checked that the maven dependency resolve for the staging artifact
> - Ran some local smoke tests
>
> -Flavio
>
> > On 3 Mar 2020, at 11:01, Andor Molnar  wrote:
> >
> > +1 (binding)
> >
> > + verified signatures, checksums
> > + successful build on Mac and Centos 7.5 (including C tests)
> > + run various smoke tests and latency tests with 3-node cluster
> > + verified rolling upgrade from 3.5.7
> >
> > Thanks Enrico, I think you’re now good to go.
> >
> > Andor
> >
> >
> >
> >> On 2020. Mar 1., at 10:03, Enrico Olivelli  wrote:
> >>
> >> +1 (binding)
> >> verified signatures and checksums
> >> run a few smoke tests form binaries (standalone mode)
> >> tested Prometheus.io metrics endpoint
> >> build from sources, run automatic QA tests (rat, checkstyle,
> spotbugs...)
> >> all on Linux with Java 8 (AdoptOpenJDK)
> >>
> >> We need at least one more PMC to vote please
> >>
> >> Enrico
> >>
> >> Il giorno dom 1 mar 2020 alle ore 01:58 Patrick Hunt
> >>  ha scritto:
> >>>
> >>> +1. xsum/sig verified. rat ran clean. Compiled and ran some manual
> tests
> >>> with various ensemble sizes successfully.
> >>>
> >>> Regards,
> >>>
> >>> Patrick
> >>>
> >>> On Fri, Feb 28, 2020 at 6:53 AM Enrico Olivelli 
> wrote:
> >>>
>  Thank you guys for voting.
> 
>  We need more votes please
> 
>  Enrico
> 
>  Il giorno gio 27 feb 2020 alle ore 14:14 Norbert Kalmar
>   ha scritto:
> >
> > +1 (non-binding)
> >
> > - unit tests pass (PurgeTxnTest as well)
> > - source tarball: compiled and started ZK + run few commands from
> source
> > tarball
> > - bin tarball: license files checked, started ZK + run few commands
> > - signatures OK.
> > - compared source tarball with git repository checked out at RC tag
> using
> > meld. Found no divergence.
> >
> > Tested on MacOS and Ubuntu 16, using openJDK 1.8.242.
> >
> > - Norbert
> >
> > On Thu, Feb 27, 2020 at 11:17 AM Szalay-Bekő Máté <
> > szalay.beko.m...@gmail.com> wrote:
> >
> >> +1 (non-binding)
> >>
> >> - I built the code and executed the java/C unit tests using 8u242
> >> (everything passed, except
>  PurgeTxnTest.testPurgeWhenLogRollingInProgress
> >> what seems to never work on my machine.. I saw it before to be flaky
>  also
> >> on the apache jenkins, I created a Jira iticket for fixing it:
> >> https://issues.apache.org/jira/browse/ZOOKEEPER-3740)
> >> - Using https://github.com/symat/zk-rolling-upgrade-test
> >> - I tested rolling upgrade from 3.5.7 to 3.6.0
> >> - I tested rolling restart on 3.6.0 to enable the multi-address
>  feature
> >> with the new quorum protocol version
> >> - Using https://github.com/symat/zookeeper-docker-test I also
> tested
>  the
> >> multi-address feature (disabling and re-enabling different virtual
>  network
> >> interfaces to see that the cluster always recovers)
> >>
> >> On Tue, Feb 25, 2020 at 4:13 PM Enrico Olivelli <
> eolive...@gmail.com>
> >> wrote:
> >>
> >>> This is the fifth release candidate for 3.6.0.
> >>>
> >>> It is a major release and it introduces a lot of new features, most
> >>> notably:
> >>> - Built-in data consistency check inside ZooKeeper
> >>> - Allow Followers to host Observers
> >>> - A new feature proposal to ZooKeeper: authentication enforcement
> >>> - Pluggable metrics system for ZooKeeper (and Prometheus.io
>  integration)
> >>> - TLS Port unification
> >>> - Audit logging in ZooKeeper servers
> >>> - Improve resilience to network (advertise multiple addresses for
> >>> members of a Zookeeper cluster)
> >>> - Persistent Recursive Watch
> >>> - add an API and the corresponding CLI to get total count of
>  recursive
> >>> sub nodes under a specific path
> >>>
> >>> The full release notes is available at:
> >>>
> >>>
> >>>
> >>
> 
> https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12310801=12346617
> >>>
> >>> *** Please download, test and vote by February 28th 2020, 23:59
>  UTC+0.
> >> ***
> >>>
> >>> Source files:
> >>> https://people.apache.org/~eolivelli/zookeeper-3.6.0-candidate-4/
> >>>
> >>> Maven staging repo:
> >>>
> >>
> 
> https://repository.apache.org/content/repositories/orgapachezookeeper-1053/
> >>>
> >>> The release candidate tag in git to be voted upon: release-3.6.0-4
> >>> 

Re: Jepsen.io report

2020-02-17 Thread Michael Han
I believe the original report for ZooKeeper
https://aphyr.com/posts/291-call-me-maybe-zookeeper was done by the author
of Jepsen. Some of the conclusions were not very accurate (see
https://github.com/jepsen-io/jepsen/issues/399 - "insumity" was an intern
in my team fyi). Would be nice if we can integrate Jepsen as part of apache
zookeeper CI.

On Sat, Feb 15, 2020 at 11:19 PM Enrico Olivelli 
wrote:

> Hi community,
> I see that Jepsen.io has a very old analysis of Zookeeper  (2013)
>
> https://jepsen.io/analyses
>
> Did they do by themselves or did any company pay for such analysis?
>
> Do you have experience in using such tools?
>
>
> Enrico
>


Re: Rolling upgrade from 3.5 to 3.6 - expected behaviour

2020-02-11 Thread Michael Han
s import.
> > > >
> > > > I will send an email next days with a proposal.
> > > > btw my idea is very like Andor's one
> > > >
> > > > Once we have an automatic environment we can launch from Jenkins
> > > >
> > > > Enrico
> > > >
> > > >
> > > > > >
> > > > > > Patrick
> > > > > >
> > > > > > On Tue, Feb 11, 2020 at 12:40 AM Enrico Olivelli <
> > > eolive...@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > >> Il giorno mar 11 feb 2020 alle ore 09:12 Szalay-Bekő Máté
> > > > > >>  ha scritto:
> > > > > >>>
> > > > > >>> Hi All,
> > > > > >>>
> > > > > >>> about the question from Michael:
> > > > > >>>> Regarding the fix, can we just make 3.6.0 aware of the old
> > > protocol
> > > > and
> > > > > >>>> speak old message format when it's talking to old server?
> > > > > >>>
> > > > > >>> In this particular case, it might be enough. The protocol
> change
> > > > happened
> > > > > >>> now in the 'initial message' sent by the QuorumCnxManager.
> Maybe
> > it
> > > > is
> > > > > >> not
> > > > > >>> a problem if the new servers can not initiate channels to the
> old
> > > > > >> servers,
> > > > > >>> maybe it is enough if these channel gets initiated by the old
> > > servers
> > > > > >> only.
> > > > > >>> I will test it quickly.
> > > > > >>>
> > > > > >>> Although I have no idea if any other thing changed in the
> quorum
> > > > protocol
> > > > > >>> between 3.5 and 3.6. In other cases it might not be enough if
> the
> > > new
> > > > > >>> servers can understand the old messages, as the old servers can
> > > > break by
> > > > > >>> not understanding the messages from the new servers. Also, in
> the
> > > > code
> > > > > >>> currently (AFAIK) there is no generic knowledge of protocol
> > > > versions, the
> > > > > >>> servers are not storing that which protocol versions they
> > > can/should
> > > > use
> > > > > >> to
> > > > > >>> communicate to which particular other servers. Maybe we don't
> > even
> > > > need
> > > > > >>> this, but I would feel better if we would have more tests
> around
> > > > these
> > > > > >>> things.
> > > > > >>>
> > > > > >>> My suggestion for the long term:
> > > > > >>> - let's fix this particular issue now with 3.6.0 quickly (I
> start
> > > > doing
> > > > > >>> this today)
> > > > > >>> - let's do some automation (backed up with jenkins) that will
> > test
> > > a
> > > > > >> whole
> > > > > >>> combinations of different ZooKeeper upgrade paths by making
> > rolling
> > > > > >>> upgrades during some light traffic. Let's have a bit better
> > > > definition
> > > > > >>> about what we expect (e.g. the quorum is up, but some clients
> can
> > > get
> > > > > >>> disconnected? What will happen to the ephemeral nodes? Do we
> want
> > > to
> > > > > >>> gracefully close or transfer the user sessions before stopping
> > the
> > > > old
> > > > > >>> server?) and let's see where this broke. Just by checking the
> > > code, I
> > > > > >> don't
> > > > > >>> think the quorum will always be up (e.g. between older 3.4
> > versions
> > > > and
> > > > > >>> 3.5).
> > > > > >>
> > > > > >>
> > > > > >> I am happy to work on this topic
> > > > > >>
> > > > > >>> - we need to update the Wiki about the working rolling upgrade
> > > paths
> > > > and
> > > > > >>> maybe about workarounds if needed
> &

Re: Rolling upgrade from 3.5 to 3.6 - expected behaviour

2020-02-10 Thread Michael Han
Kudos Enrico, very thorough work as the final gate keeper of the release!

Now with this, I'd like to *vote a -1* on the 3.6.0 RC2.

I'd recommend we fix this issue for 3.6.0. ZooKeeper is one of the rare
piece of software that put so much emphasis on compatibilities thus it just
works when upgrade / downgrade, which is amazing. One guarantee we always
had is during rolling upgrade, the quorum will always be available, leading
to no service interruption. It would be sad we lose such capability given
this is still a tractable problem.

Regarding the fix, can we just make 3.6.0 aware of the old protocol and
speak old message format when it's talking to old server? Basically, an
ugly if else check against the protocol version should work and there is no
need to have multiple pass on rolling upgrade process.


On Mon, Feb 10, 2020 at 10:23 PM Enrico Olivelli 
wrote:

> I suggest this plan:
> - release 3.6.0 now
> - improve the migration story, the flow outlined by Mate is
> interesting, but it will take time
>
> 3.6.0rc2 got enough binding votes so I am going to finalize the
> release this evening (within 8-10 hours) if no one comes out in the
> VOTE thread with a -1
>
> Enrico
>
> Enrico
>
> Il giorno lun 10 feb 2020 alle ore 19:33 Patrick Hunt
>  ha scritto:
> >
> > On Mon, Feb 10, 2020 at 3:38 AM Andor Molnar  wrote:
> >
> > > Hi,
> > >
> > > Answers inline.
> > >
> > >
> > > > In my experience when you are close to a release it is better to to
> > > > make big changes. (I am among the approvers of that patch, so I am
> > > > responsible for this change)
> > >
> > >
> > >
> > > Although this statement is acceptable for me, I don’t feel this patch
> > > should not have been merged into 3.6.0. Submission has been preceded
> by a
> > > long argument with MAPR folks who originally wanted to be merged into
> 3.4
> > > branch (considering the pace how ZooKeeper community is moving
> forward) and
> > > we reached an agreement that release it with 3.6.0.
> > >
> > > Make a long story short, this patch has been outstanding for ages
> without
> > > much attention from the community and contributors made a lot of
> effort to
> > > get it done before the release.
> > >
> > >
> > > > I would like to ear from people that have been in the community for
> > > > long time, then I am ready to complete the release process for
> > > > 3.6.0rc2.
> > >
> > >
> > > Me too.
> > >
> > > I tend to accept the way rolling restart works now - as you described
> > > Enrico - and given that situation was pretty much the same between 3.4
> and
> > > 3.5, I don’t feel we have to make additional changes.
> > >
> > > On the other hand, the fix that Mate suggested sounds quite cool, I’m
> also
> > > happy to work on getting it in.
> > >
> > > Fyi, Release Management page says the following:
> > >
> https://cwiki.apache.org/confluence/display/ZOOKEEPER/ReleaseManagement
> > >
> > > "major.minor release of ZooKeeper must be backwards compatible with the
> > > previous minor release, major.(minor-1)"
> > >
> > >
> > Our users, direct and indirect, value the ability to migrate to newer
> > versions - esp as we drop support for older. Frictions such as this can
> be
> > a reason to go elsewhere. I'm "pro" b/w compact - esp given our published
> > guidelines.
> >
> > Patrick
> >
> >
> > > Andor
> > >
> > >
> > >
> > >
> > > > On 2020. Feb 10., at 11:32, Enrico Olivelli 
> wrote:
> > > >
> > > > Thank you Mate for checking and explaining this story.
> > > >
> > > > I find it very interesting that the cause is ZOOKEEPER-3188 as:
> > > > - it is the last "big patch" committed to 3.6 before starting the
> > > > release process
> > > > - it is the cause of the failure of the first RC
> > > >
> > > > In my experience when you are close to a release it is better to to
> > > > make big changes. (I am among the approvers of that patch, so I am
> > > > responsible for this change)
> > > >
> > > > This is a pointer to the change to whom who wants to understand
> better
> > > > the context
> > > >
> > >
> https://github.com/apache/zookeeper/pull/1048/files#diff-7a209d890686bcba351d758b64b22a7dR11
> > > >
> > > > IIUC even for the upgrade from 3.4 to 3.5 the story was the same and
> > > > if this statement holds then I feel we can continue
> > > > with this release.
> > > >
> > > > - Reverting ZOOKEEPER-3188 is not an option for me, it is too
> complex.
> > > > - Making 3.5 and 3.6 "compatible" can be very tricky and we do not
> > > > have tools to certify this compatibility (at least not in the short
> > > > term)
> > > >
> > > > I would like to ear from people that have been in the community for
> > > > long time, then I am ready to complete the release process for
> > > > 3.6.0rc2.
> > > >
> > > > I will update the website and the release notes with a specific
> > > > warning about the upgrade, we should also update the Wiki
> > > >
> > > > Enrico
> > > >
> > > >
> > > > Il giorno lun 10 feb 2020 alle ore 11:17 Szalay-Bekő Máté
> > > >  ha scritto:
> > > >>

Re: [VOTE] Apache ZooKeeper release 3.6.0 candidate 2

2020-02-08 Thread Michael Han
+1.

- verified checksum/sig
- verified release notes.
- verified regenerated documentations.
- verified both java and c unit tests pass.
- verified with a few jetty admin commands and zk cli commands.




On Sat, Feb 8, 2020 at 10:22 AM Flavio Junqueira  wrote:

> +1 binding
>
> - Built from sources and ran tests (some tests fail intermittently)
> - Checked SHA 512 and signature
> - Checked license and notice
> - Ran local smoke tests
>
> -Flavio
>
> > On 7 Feb 2020, at 10:04, Norbert Kalmar 
> wrote:
> >
> > +1 (non-binding)
> >
> > - unit tests pass
> > - source tarball: compiled and started ZK + run few commands from source
> > tarball
> > - bin tarball: license files checked, started ZK + run few commands
> > - signature OK.
> >
> > Tested on MacOS and Linux, openJDK 1.8.242.
> >
> > Thanks Enrico!
> >
> > - Norbert
> >
> > On Thu, Feb 6, 2020 at 11:20 AM Szalay-Bekő Máté <
> szalay.beko.m...@gmail.com>
> > wrote:
> >
> >> +1 (non-binding)
> >>
> >> - I compiled and run all the unit tests using Ubuntu 18.04, using maven
> >> 3.3.9 and OpenJDK 1.8.242 (the tests that failed with this JDK for the
> >> previous RC now run without a problem)
> >> - I compiled and tested the C client and the python client (we added SSL
> >> feature / tests in this release for the C and python clients)
> >> - I did some manual tests for the multi-address feature with multiple
> >> virtual networks (using https://github.com/symat/zookeeper-docker-test)
> >> and
> >> the cluster did recover quickly after I disabled / enabled various
> virtual
> >> network interfaces
> >>
> >>
> >> On Thu, Feb 6, 2020 at 4:36 AM Patrick Hunt  wrote:
> >>
> >>> +1 - sig/xsum verified, rat ran clean, I compiled and ran various tests
> >> and
> >>> they passed.
> >>>
> >>> Patrick
> >>>
> >>> On Wed, Feb 5, 2020 at 11:34 AM Enrico Olivelli 
> >>> wrote:
> >>>
>  This is the third release candidate for Apache ZooKeeper 3.6.0.
> 
>  It is a major release and it introduces a lot of new features, most
>  notably:
>  - Built-in data consistency check inside ZooKeeper
>  - Allow Followers to host Observers
>  - Authentication enforcement
>  - Pluggable metrics system for ZooKeeper (and Prometheus.io
> >> integration)
>  - TLS Port unification
>  - Audit logging in ZooKeeper servers
>  - Improve resilience to network (advertise multiple addresses for
>  members of a Zookeeper cluster)
>  - Persistent Recursive Watches
>  - add an API and the corresponding CLI to get total count of recursive
>  sub nodes under a specific path
> 
>  The full release notes is available at:
> 
> 
> 
> >>>
> >>
> https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12310801=12326518
> 
>  *** Please download, test and vote by February 8th 2020, 23:59 UTC+0.
> >> ***
> 
>  Source files:
>  https://people.apache.org/~eolivelli/zookeeper-3.6.0-candidate-2/
> 
>  Maven staging repo:
> 
> >>>
> >>
> https://repository.apache.org/content/repositories/orgapachezookeeper-1049/
> 
>  The staging version of the website is:
> 
> >>>
> >>
> https://people.apache.org/~eolivelli/zookeeper-3.6.0-candidate-2/website/
> 
>  The release candidate tag in git to be voted upon: release-3.6.0-2
>  https://github.com/apache/zookeeper/tree/release-3.6.0-2
> 
>  ZooKeeper's KEYS file containing PGP keys we use to sign the release:
>  https://www.apache.org/dist/zookeeper/KEYS
> 
>  Please note that we are adding a new jar to the dependency set for
>  clients: zookeeper-metrics-providers.
> 
>  Should we release this candidate?
> 
>  Enrico Olivelli
> 
> >>>
> >>
>
>


Re: [ANNOUNCE] Enrico Olivelli new ZooKeeper PMC Member

2020-01-21 Thread Michael Han
Congrats, Enrico!

On Tue, Jan 21, 2020 at 1:57 PM Jordan Zimmerman 
wrote:

> Well deserved. Congratulations.
>
> 
> Jordan Zimmerman
>
> > On Jan 21, 2020, at 4:40 PM, Flavio Junqueira  wrote:
> >
> > I'm pleased to announce that Enrico Olivelli recently became the newest
> ZooKeeper PMC member. Enrico has contributed immensely to this community;
> he became a ZooKeeper committer in May 2019 and now he joins the PMC.
> >
> > Join me in congratulating him on the achievement. Congrats, Enrico!
> >
> > -Flavio on behalf of the Apache ZooKeeper PMC
>


Re: Hadoop logo

2019-12-12 Thread Michael Han
+1

I do see a book keeper gif file in repo, should remove that one too.

On my wish list: redesign of the ZooKeeper logo :)

On Thu, Dec 12, 2019 at 7:44 AM Enrico Olivelli  wrote:

> +1
>
> Maybe we should also check if we have old pages about Bookkeeper project.
> It was a subproject of ZK but now it is a (great) top level independent
> project
>
> Enrico
>
> Il gio 12 dic 2019, 16:38 Flavio Junqueira  ha scritto:
>
> > ZooKeeper was a subproject of Hadoop in the early Apache days, and we
> > still carry that flag... ;-)
> >
> > -Flavio
> >
> > > On 12 Dec 2019, at 16:16, Norbert Kalmar  >
> > wrote:
> > >
> > > Oh, wow, I didn't even notice that until now.
> > > Makes sense, knowing a lot of the time ZK is used "standalone" (I mean
> > > outside of any hadoop ecosystem).
> > >
> > > Regards,
> > > Norbert
> > >
> > > On Thu, Dec 12, 2019 at 2:52 PM Flavio Junqueira 
> wrote:
> > >
> > >> Should we remove that Hadoop logo from the documentation? It has been
> a
> > >> while that we aren't a subproject of Hadoop any longer.
> > >>
> > >> -Flavio
> >
> >
>


Re: Drop ANT build pull request

2019-11-24 Thread Michael Han
>> ZooKeeper_UT_Stress (hanm’s private job - not sure if needed)

I just deleted it. Thanks guys for taking care of the build system.

On Fri, Nov 22, 2019 at 3:05 AM Enrico Olivelli  wrote:

> Il giorno ven 22 nov 2019 alle ore 11:16 Andor Molnar 
> ha
> scritto:
>
> > Here’s another thing. (Maybe I should open a new thread)
> >
> > I’m disabled Ant-based trunk jobs in Jenkins and found the following
> which
> > doesn’t have Maven alternative:
> >
> > - ZooKeeper-trunk-java13
> >
>
> I would keep this, in Maven version
>
>
> > - Zookeeper-trunk-single-thread
> >
>
> I think this is not really needed, it was created to the stability of the
> suite using single thread mode (IIRC)
>
>
> > - ZooKeeper-trunk-windows-cmake
> >
>
> Yes we need it
>
>
> > - ZooKeeper_UT_Stress (hanm’s private job - not sure if needed)
> >
>
> Leaving the word to Micheal H
>
>
> Thank you Andor for helping with this housekeeping work
>
> Enrico
>
>
>
> >
> > Would you please confirm?
> > I’ll open new Jira tickets to cover these and also another one for the
> > crash-not-reported issue.
> >
> > Thanks,
> > Andor
> >
> >
> >
> > > On 2019. Nov 22., at 7:17, Enrico Olivelli 
> wrote:
> > >
> > > Andor
> > > You are totally right.
> > > I will investigate today
> > >
> > >
> > > Enrico
> > >
> > > Il ven 22 nov 2019, 07:13 Andor Molnar  ha scritto:
> > >
> > >> Though the job is still running on master. This ran 13 hours ago:
> > >>
> > >>
> >
> https://builds.apache.org/view/S-Z/view/ZooKeeper/job/PreCommit-ZOOKEEPER-github-pr-build/4259/
> > >>
> > >> But the result is not reported on the pull request.
> > >> The latest 2 Maven builds are reported as “Jenkins” and
> “JenkinsMaven”.
> > >>
> > >> Andor
> > >>
> > >>
> > >>
> > >>> On 2019. Nov 22., at 7:09, Andor Molnar  wrote:
> > >>>
> > >>> Hi Enrico,
> > >>>
> > >>> Looks like a previous Maven build has been reported as “Jenkins” on
> > this
> > >> patch:
> > >>> https://github.com/apache/zookeeper/pull/1147
> > >>>
> > >>> So it’s not the Ant build which still running on master.
> > >>>
> > >>> Andor
> > >>>
> > >>>
> > >>>
> >  On 2019. Nov 15., at 16:41, Andor Molnar  wrote:
> > 
> >  Yes we do.
> > 
> >  We just discussed on the PR that Maven surefire plugin cannot be
> > >> configured to report crashed tests as build errors.
> >  As I highlighted although tests were crashed during the build
> process,
> > >> Maven still reports as a green build:
> > 
> > 
> > >>
> >
> https://builds.apache.org/view/S-Z/view/ZooKeeper/job/zookeeper-master-maven/521/
> > 
> >  Andor
> > 
> > 
> > 
> > 
> > > On 2019. Nov 15., at 16:01, Enrico Olivelli 
> > >> wrote:
> > >
> > > Il ven 15 nov 2019, 15:28 Andor Molnar  ha
> > scritto:
> > >
> > >> Hi,
> > >>
> > >> I noticed that PR#1139 has been merged. As a consequence I have to
> > >> remove/disable all of the Ant-based build jobs in this Jenkis
> > >> dashboard:
> > >> https://builds.apache.org/view/S-Z/view/ZooKeeper/
> > >>
> > >> Which also means that we don’t have a build job which is able to
> > alert
> > >> when a test has crashed.
> > >>
> > >
> > > I can't follow you.
> > > We do have the maven jobs, don't we?
> > >
> > > Enrico
> > >
> > >
> > >> Is that okay?
> > >>
> > >> Andor
> > >>
> > >>
> > >>
> > >>
> > >>
> > >>> On 2019. Nov 15., at 11:06, Andor Molnar 
> wrote:
> > >>>
> > >>> Original patch:
> > >>>
> > >>> "ZOOKEEPER-3598: Fix potential data inconsistency issue due to
> > >>>
> > >>> has been reverted.
> > >>>
> > >>> Fangmin, my apologies for the inconvenience.
> > >>> Would you please create another PR with different solution or
> with
> > >> Enrico’s workaround?
> > >>>
> > >>> Regards,
> > >>> Andor
> > >>>
> > >>>
> > >>>
> >  On 2019. Nov 14., at 7:54, Andor Molnar 
> wrote:
> > 
> >  Hi Enrico,
> > 
> >  +1 for the idea
> >  I think the best would be to do this before we cut the first
> > >> version of
> > >> 3.6.0.
> > 
> >  Btw I noticed that the trunk build is completely broken since
> > 
> > >>
> > >>
> >
> https://builds.apache.org/view/S-Z/view/ZooKeeper/job/ZooKeeper-trunk/769/
> > 
> >  It’s failing on Java versions since we introduced those 2
> commits.
> >  But looks like the Maven build is still fine.
> > 
> >  Andor
> > 
> > 
> > 
> > > On 2019. Nov 11., at 13:36, Enrico Olivelli <
> eolive...@gmail.com
> > >
> > >> wrote:
> > >
> > > Hi community,
> > > I have just sent a PR to drop ANT from main build (not from
> > > recipes/contrib) and keep only Maven build.
> > > We are already releasing 3.5 branch with Maven, this is only
> the
> > >> final
> > >> step
> > > 

Re: [VOTE] Apache ZooKeeper release 3.5.6 candidate 2

2019-10-01 Thread Michael Han
>> How about officially dropping netty support from 3.4 and asking people
to move to the new version
+1. This sounds a good opportunity to deprecate 3.4 branch.

On Tue, Oct 1, 2019 at 8:00 AM Enrico Olivelli  wrote:

> Il giorno mar 1 ott 2019 alle ore 16:15 Patrick Hunt  ha
> scritto:
>
> > Another option/solution: How about officially dropping netty support from
> > 3.4 and asking people to move to the new version (3.5 stable or later)?
> >
>
> Sounds good
>
> Enrico
>
>
> >
> > Patrick
> >
> > On Tue, Oct 1, 2019 at 4:22 AM Andor Molnar  wrote:
> >
> > > I agree with 3.4 should not be refactored in any way even for a
> security
> > > fix.
> > >
> > > What's wrong with the "alpha story"?
> > >
> > > I think releasing in an early stage with "-alpha", "-beta" modifiers is
> > > not a bad thing alone, as long as it doesn't take years to get to the
> > > stable release.
> > >
> > > Andor
> > >
> > >
> > > On Tue, 1 Oct 2019, Enrico Olivelli wrote:
> > >
> > > > Date: Tue, 1 Oct 2019 10:54:24 +0200
> > > > From: Enrico Olivelli 
> > > > Reply-To: dev@zookeeper.apache.org
> > > > To: dev@zookeeper.apache.org
> > > > Subject: Re: [VOTE] Apache ZooKeeper release 3.5.6 candidate 2
> > > >
> > > > Il mar 1 ott 2019, 10:38 Andor Molnar  ha scritto:
> > > >
> > > >> Backporting Netty 4 would be a huge, cumbersome task, I hope we
> don’t
> > > have
> > > >> to do it.
> > > >>
> > > >
> > > > Yes, 3.4 is mature and stable and closed for refactors.
> > > >
> > > >
> > > >> However I had a quick look at the details of this CVE and it seems
> to
> > me
> > > >> that it only affects the HTTP codec:
> > > >>
> > > >>
> > >
> >
> https://github.com/netty/netty/commit/39cafcb05c99f2aa9fce7e6597664c9ed6a63a95
> > > >>
> > > >> Can’t we just say 3.4.14 is not affected?
> > > >> We’re not running HTTP server inside ZooKeeper.
> > > >>
> > > >> Otherwise we might be able to release 3.6.0-alpha1 now, put a date
> for
> > > 3.4
> > > >> EOL and highlight on the webpage that this
> > > >>
> > > >
> > > > Please do not start an 'alpha' story like for 3.5
> > > >
> > > > CVE probably won’t be resolved on that branch, please upgrade to 3.5.
> > > >>
> > > >
> > > > +1
> > > >
> > > >
> > > > Enrico
> > > >
> > > >>
> > > >> As a third option we could ask Norman to kindly fix 3.10.6.Final as
> > > well…
> > > >> or submit a PR ourselves, it doesn’t seem to me a big deal.
> > > >>
> > > >
> > > > Not so useful
> > > >
> > > >>
> > > >> What do you think?
> > > >>
> > > >> Andor
> > > >>
> > > >>
> > > >>
> > > >>
> > > >>> On 2019. Oct 1., at 2:00, Patrick Hunt  wrote:
> > > >>>
> > > >>> I pushed patches for 3.5 and trunk and the tests passed on my mac.
> > > >> However
> > > >>> 3.4 is using netty 3.10.6.Final and as such it's not a simple
> > upgrade.
> > > >>> (there are no fixes against 3.10 for this CVE, at least not so far)
> > Not
> > > >>> sure what we want to do about this... someone would need to
> backport
> > > the
> > > >>> netty 4.1 changes into 3.4 afaict.
> > > >>>
> > > >>> Patrick
> > > >>>
> > > >>> On Mon, Sep 30, 2019 at 1:08 PM Patrick Hunt 
> > wrote:
> > > >>>
> > >  I'll work on it today.
> > > 
> > >  Patrick
> > > 
> > >  On Mon, Sep 30, 2019 at 11:59 AM Enrico Olivelli <
> > eolive...@gmail.com
> > > >
> > >  wrote:
> > > 
> > > > Okay
> > > >
> > > > I am cancelling the release.
> > > >
> > > > I have a problem with my box, I can't work on netty upgrade.
> > > >
> > > > Any volounteer?
> > > >
> > > > Enrico
> > > >
> > > > Il lun 30 set 2019, 20:32 Andor Molnar  ha
> > > scritto:
> > > >
> > > >> The good news is: we need to release 3.4.15 too. :)
> > > >>
> > > >> Andor
> > > >>
> > > >>
> > > >>
> > > >>> On 2019. Sep 30., at 20:26, Patrick Hunt 
> > wrote:
> > > >>>
> > > >>> created: https://issues.apache.org/jira/browse/ZOOKEEPER-3563
> > > >>>
> > > >>> On Mon, Sep 30, 2019 at 11:20 AM Patrick Hunt <
> ph...@apache.org>
> > > > wrote:
> > > >>>
> > >  -1 - when I run dependency check on the release candidate
> > artifact
> > > > it's
> > >  failing with:
> > > 
> > >  [ERROR] netty-transport-4.1.29.Final.jar: CVE-2019-16869
> > > 
> > >  I ran this on trunk and it's passing, as such it must be an
> > issue
> > > > with
> > > >> the
> > >  the 3.5.6 netty version specifically. It's listed as a high,
> we
> > > > should
> > >  patch this as well before releasing.
> > > 
> > >  Patrick
> > > 
> > > 
> > >  On Sun, Sep 29, 2019 at 7:29 AM Enrico Olivelli <
> > > >> eolive...@gmail.com
> > > >>
> > >  wrote:
> > > 
> > > > This is a bugfix release candidate for 3.5.6.
> > > >
> > > > It fixes 28 issues, including upgrade of third party
> libraries,
> > > > TTL Node APIs for C API, support for PCKS12 

Re: Releasing 3.6.0 - ALPHA or not ?

2019-10-01 Thread Michael Han
I am leaning towards release master as 3.6.0 as well, not with any suffix.
We don't have any pending unstable API for 3.6 (like dynamic
reconfiguration to 3.5) that justify the added overheads of using a non
standard, ZooKeeper specific versioning scheme for master branch.

See
http://zookeeper-user.578899.n2.nabble.com/Question-about-3-5-0-stability-and-versioning-td7580927.html
for
some context on why the decision was made and the complains.


On Tue, Oct 1, 2019 at 7:11 AM Patrick Hunt  wrote:

> Enrico these are good ideas, thoughts below:
>
> On Tue, Oct 1, 2019 at 6:09 AM Norbert Kalmar  >
> wrote:
>
> > Hi,
> >
> > 3.5 had a lot of new features that wasn't really finalized, so API
> changed
> > until stable 3.5 (3.5.5). But I don't think this is the case with 3.6.0,
> we
> > have complete and pretty much finalized features as far as I can tell.
> > Also, it did confuse me that with the beta and alpha releases on 3.5
> minor
> > version jumped as well. So if we want to stick with alpha/beta qualifier,
> > let's keep it at 3.6.0-alpha and 3.6.0-beta (not like 3.6.2-beta).
> >
> >
> That is a good point Norbert. We did try to say "alpha/beta is unstable"
> (apis/code/etc...). That worked fairly well, but we were in that state for
> so long that people started using it in production and then got upset when
> we did change the APIs (whatever). As such I would say this is only
> partially successful. Perhaps it would have been more successful if we had
> limited the beta time down more, however folks kept increasing the scope
> (by committing new features to 3.5 rather than trunk) and that ended up
> continually pushing out the dates.
>
>
> > I don't know any change that would justify an "alpha" version, so maybe a
> > beta would be better? But I'm also just fine releasing just "3.6.0".
> Bugfix
> > version is zero, everyone pretty much knows what that means :)
> >
>
> Perhaps a limited "beta" to allow folks to bang on it, then a planned move
> to "stable"? You could say we'll release it as beta for 3 months then move
> to stable if there are no major issues. The problem with just releasing
> straight to stable is that many folks won't try it out from source and
> would only try a binary.
>
> Patrick
>
>
> >
> > So I lean toward leaving alpha and beta out of the version.
> >
> > Regards,
> > Norbert
> >
> > On Tue, Oct 1, 2019 at 2:34 PM Enrico Olivelli 
> > wrote:
> >
> > > Hi,
> > > We are close to a release for 3.6.0, currently master branch is full of
> > > important features and important refactors.
> > >
> > > On the VOTE thread for 3.5.6 it came out that we could release 3.6.0 as
> > > "ALPHA", here are my thoughts.
> > >
> > > I think we have at least these kind of "users":
> > > - Companies that run the Server on the most recent "stable" release
> > > - Companies that running a ZooKeeper cluster just because another
> system
> > > depends on it (HBase, Kafka,Solr, Pulsar)
> > > - Library maintainers (Kafka, BookKeeper, HBase), they depend on a
> > version
> > > of the client or on some feature of the server
> > > - Application developers
> > > - Big companies that maintain their own forks and/or are using the
> > "master"
> > > version
> > >
> > > With my library maintainer hat I feel I cannot depend on some "ALPHA"
> > > version of ZooKeeper client and make my users setup  an ALPHA version
> of
> > > the server.
> > > It happened on BookKeeper for instance, we started to depend on ZK 3.5
> > but
> > > as it was BETA so we needed to revert back to 3.4.
> > > I think that some similar story happened in Kafka, now that we have 3.5
> > > with SSL support users are going to migrate.
> > >
> > > If there is no blocker issue on 3.6.0 I feel we should dare to release
> it
> > > as "stable", we can always suggest users and companies to try out
> current
> > > master and give feedback.
> > >
> > > I am new to this story of tagging as "ALPHA"/"BETA" on ZooKeeper, but
> as
> > an
> > > user and library maintainer I suffered a lot that '-ALPHA' and '-BETA'
> > > suffixes.
> > > I know that ZooKeeper is the core of most of the other systems and we
> > > should not suggest to use something that it is "experimental", but as
> far
> > > as I know we are taking great care about being backward compatible and
> > > about the quality of our code base.
> > >
> > > Other opinions ?
> > >
> > > Enrico
> > >
> >
>


[jira] [Created] (ZOOKEEPER-3561) Generalize target authentication scheme for ZooKeeper authentication enforcement.

2019-09-26 Thread Michael Han (Jira)
Michael Han created ZOOKEEPER-3561:
--

 Summary: Generalize target authentication scheme for ZooKeeper 
authentication enforcement.
 Key: ZOOKEEPER-3561
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3561
 Project: ZooKeeper
  Issue Type: Improvement
  Components: server
Affects Versions: 3.6.0
Reporter: Michael Han


ZOOKEEPER-1634 introduced an option to allow user enforce authentication for 
ZooKeeper clients, but the enforced authentication scheme in committed 
implementation was SASL only. 

This JIRA is to generalize the authentication scheme such that the 
authentication enforcement on ZooKeeper clients could work with any supported 
authentication scheme.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (ZOOKEEPER-3560) Add response cache to serve get children (2) requests.

2019-09-26 Thread Michael Han (Jira)
Michael Han created ZOOKEEPER-3560:
--

 Summary: Add response cache to serve get children (2) requests.
 Key: ZOOKEEPER-3560
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3560
 Project: ZooKeeper
  Issue Type: Improvement
  Components: server
Reporter: Michael Han
Assignee: Michael Han


ZOOKEEPER-3180 introduces response cache but it only covers getData requests. 
This JIRA is to extend the response cache based on the infrastructure set up by 
ZOOKEEPER-3180 to so the response of get children requests can also be served 
out of cache. Some design decisions:

* Only OpCode.getChildren2 is supported, as OpCode.getChildren does not have 
associated stats and current cache infra relies on stats to invalidate cache.

* The children list is stored in a separate response cache object so it does 
not pollute the existing data cache that's serving getData requests, and this 
separation also allows potential separate tuning of each cache based on 
workload characteristics.

* As a result of cache object separation, new server metrics is added to 
measure cache hit / miss for get children requests, that's separated from get 
data requests.




--
This message was sent by Atlassian Jira
(v8.3.4#803005)


Re: [jira] [Commented] (ZOOKEEPER-3556) Dynamic configuration file can not be updated automatically after some zookeeper servers of zk cluster are down

2019-09-25 Thread Michael Han
>> There were recently a post here from someone who has implemented this

Maybe this one?
http://zookeeper-user.578899.n2.nabble.com/About-ZooKeeper-Dynamic-Reconfiguration-td7584271.html

On Wed, Sep 25, 2019 at 9:19 PM Alexander Shraer  wrote:

> There were recently a post here from someone who has implemented this, but
> I couldn't find it for some reason.
>
> Essentially I think that you'd need to monitor the "health" and
> connectivity of servers to the leader, and issue reconfig commands to
> remove them when you suspect that they're down or add them back when you
> think they're up.
> Notice that you always have to have at least a quorum of the ensemble, so
> issuing a reconfig command if a quorum is lost (or any other command) won't
> work.
> You could use the information exposed in ZK's 4 letter commands to decide
> whether you think a server is up and connected to the quorum or down.
> Ideally we could also use the leader's view on who is connected
> but it doesn't look like this is being exposed right now. You can also
> periodically issue test read/write operations on various servers to check
> if they're really operational
>
> https://github.com/apache/zookeeper/blob/1ca627b5a3105d80ed4d851c6e9f1a1e2ac7d64a/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md#sc_4lw
>
> As accurate failure detection is impossible in async. systems, you'll need
> to decide how sensitive you are to potential failures vs false suspicions.
>
> Hope this helps...
>
> Alex
>
> On Wed, Sep 25, 2019 at 6:00 PM Gao,Wei  wrote:
>
> > Hi Alexander Shraer,
> >  Could you please tell me how to implement automation on top?
> > Thank you very much!
> >
> > -Original Message-
> > From: Alexander Shraer (Jira) 
> > Sent: Thursday, September 26, 2019 1:27 AM
> > To: iss...@zookeeper.apache.org
> > Subject: [jira] [Commented] (ZOOKEEPER-3556) Dynamic configuration file
> > can not be updated automatically after some zookeeper servers of zk
> cluster
> > are down
> >
> >
> > [
> >
> https://urldefense.proofpoint.com/v2/url?u=https-3A__issues.apache.org_jira_browse_ZOOKEEPER-2D3556-3Fpage-3Dcom.atlassian.jira.plugin.system.issuetabpanels-3Acomment-2Dtabpanel-26focusedCommentId-3D16937925-23comment-2D16937925=DwIFaQ=ZmK7amRlbztwfC_NTU_hNw=bTmnMF5RGYcfg4qOcKQAYjkGGUtOB2jR22ryrk8hNWk=UNFnO3kfjtUL8Jievmh9VMXf_nTLKBCfuJsaxe6FshU=XxgusqUbHgFrxTfTTcYuxMWxol3W-1dJ7WVzUqh1HAE=
> > ]
> >
> > Alexander Shraer commented on ZOOKEEPER-3556:
> > -
> >
> > The described behavior is not a bug – currently reconfiguration requires
> > explicit action by an operator. One could implement automation on top. We
> > should consider this as a feature, since it sounds like several adopters
> > have implemented such automation. Perhaps one of them could contribute
> this
> > upstream.
> >
> > > Dynamic configuration file can not be updated automatically after some
> > > zookeeper servers of zk cluster are down
> > > --
> > > -
> > >
> > > Key: ZOOKEEPER-3556
> > > URL:
> >
> https://urldefense.proofpoint.com/v2/url?u=https-3A__issues.apache.org_jira_browse_ZOOKEEPER-2D3556=DwIFaQ=ZmK7amRlbztwfC_NTU_hNw=bTmnMF5RGYcfg4qOcKQAYjkGGUtOB2jR22ryrk8hNWk=UNFnO3kfjtUL8Jievmh9VMXf_nTLKBCfuJsaxe6FshU=NQvX26JbBDNMmEtQhirmYk7ELe46vCjn4kbm1VqcNsA=
> > > Project: ZooKeeper
> > >  Issue Type: Wish
> > >  Components: java client
> > >Affects Versions: 3.5.5
> > >Reporter: Steven Chan
> > >Priority: Major
> > >   Original Estimate: 12h
> > >  Remaining Estimate: 12h
> > >
> > > *I encountered a problem which blocks my development of load balance
> > > using ZooKeeper 3.5.5.*
> > >*Actually, I have a ZooKeeper cluster which comprises of five zk
> > > servers. And the dynamic configuration file is as follows:*
> > >  **
> > > {color:#FF}
> > > *server.1=zk1:2888:3888:participant;0.0.0.0:2181*{color}
> > > {color:#FF}
> > > *server.2=zk2:2888:3888:participant;0.0.0.0:2181*{color}
> > > {color:#FF}
> > > *server.3=zk3:2888:3888:participant;0.0.0.0:2181*{color}
> > > {color:#FF}
> > > *server.4=zk4:2888:3888:participant;0.0.0.0:2181*{color}
> > > {color:#FF}
> > > *server.5=zk5:2888:3888:participant;0.0.0.0:2181*{color}
> > >  **
> > >   *The zk cluster can work fine if every member works normally.
> > > However, if say two of them are suddenly down without previously being
> > > notified,* *the dynamic configuration file shown above will not be
> > > synchronized dynamically, which leads to the zk cluster fail to work
> > > normally.*
> > >   *As far as I am concerned, the dynamic configuration file should be
> > > modified to this if server 1 and server 5 are down suddenly as
> > > follows:* {color:#FF}
> > > *server.2=zk2:2888:3888:participant;0.0.0.0:2181*{color}
> > > {color:#FF}
> > > 

Re: PoweredBy Zookeeper

2019-09-24 Thread Michael Han
link to the doc:

https://github.com/apache/zookeeper/blob/master/zookeeper-docs/src/main/resources/markdown/zookeeperUseCases.md

On Tue, Sep 24, 2019 at 4:36 AM Enrico Olivelli  wrote:

> Cool
>
> I am not sure, do we have to wait for 3.6 release before updating the
> website?
>
> Enrico
>
> Il mar 24 set 2019, 12:37 Justin Ling Mao  ha
> scritto:
>
> > 1.Thanks Olivelli for this letter. ZOOKEEPER-3529 has now landed.   1.1
> > The references may be a good reading material and they're are public and
> > free. welcome scrutiny.   1.2 The documentation has a declaration about
> > intellectual property rights and privacy.2 If you have a use case, please
> > do not hesitate, submit a pull request or write an email to **
> > dev@zookeeper.apache.org**
> >
> > - Original Message -
> > From: Enrico Olivelli 
> > To: dev@zookeeper.apache.org
> > Subject: PoweredBy Zookeeper
> > Date: 2019-09-14 20:22
> >
> > Hi,
> > Maoling started this initiative to have a great Zookeeper use cases page:
> > https://github.com/apache/zookeeper/pull/1073
> > I think this is a good idea, but before 'approving' that change I feel we
> > must discuss more about it.
> > The major point for me is whether we should ask for consent for
> publishing
> > such information
> > Usually maintainers of projects actively adk for their project to be
> added,
> > with this page we are citing thid party productd and companies
> > Once we clear this point I will be happy to merge that change.
> > Thanks Maoling
> > Enrico
> >
>


Re: maven command to run a specific test in a test case.

2019-09-20 Thread Michael Han
Thanks Pat. You inspired me and here is another way of running single test:

mvn clean install -DskipTests
mvn test -Dtest=CreateTest#testCreate -pl zookeeper-server

I think the problem I had was I did not specify the module of the test.
Some module does not have test (like zookeeper-doc), so either we have to
skip these module (via -DfailIfNoTests=false), or specify the module (via
-pl module).

On Fri, Sep 20, 2019 at 2:44 PM Patrick Hunt  wrote:

> the following works fine for me and runs the specified test (passes)
>
> $ mvn clean install -DskipTests
> $ mvn test -Dtest=UnifiedServerSocketTest -pl zookeeper-server
>
> [INFO] Tests run: 72, Failures: 0, Errors: 0, Skipped: 0, Time elapsed:
> 8.178 s - in org.apache.zookeeper.server.quorum.UnifiedServerSocketTest
>
> ⌂76% [phunt:~/dev/zookeeper-trunk] master+ 2s 1 ± mvn -v
> Apache Maven 3.6.2 (40f52333136460af0dc0d7232c0dc0bcf0d9e117;
> 2019-08-27T08:06:16-07:00)
> Maven home: /usr/local/Cellar/maven/3.6.2/libexec
> Java version: 1.8.0_222, vendor: AdoptOpenJDK, runtime:
> /Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home/jre
> Default locale: en_US, platform encoding: UTF-8
> OS name: "mac os x", version: "10.14.6", arch: "x86_64", family: "mac"
>
> Patrick
>
>
> On Fri, Sep 20, 2019 at 2:27 PM Michael Han  wrote:
>
> > Thanks Andor, I tried that before but got some errors (still trying to
> > figure out what those errors mean). Any chance you can have a concrete
> > canonical example that execute a real single test via maven?
> >
> > On a side note, "mvn test" works for me which ran all tests, when
> executed
> > under root zookeeper folder.
> >
> > On Fri, Sep 20, 2019 at 2:11 PM Andor Molnar  >
> > wrote:
> >
> > > Even better to do a quick google search:
> > >
> > >
> > >
> >
> https://maven.apache.org/surefire/maven-surefire-plugin/examples/single-test.html
> > >
> > > mvn -Dtest=TestCircle#mytest test
> > >
> > > Andor
> > >
> > > On Fri, Sep 20, 2019 at 11:08 PM Andor Molnar 
> > wrote:
> > >
> > > > I usually use:
> > > >
> > > > mvn verify -Dtest=bar
> > > >
> > > > Which runs the entire testcase. Not sure about how to run single test
> > > > method.
> > > >
> > > > Andor
> > > >
> > > >
> > > >
> > > > On Fri, Sep 20, 2019 at 11:05 PM Michael Han 
> wrote:
> > > >
> > > >> In ant, we can execute a single test "foo" in a test case "bar" like
> > > this:
> > > >>
> > > >> *ant -Dtestcase=bar -Dtest.method=foo*
> > > >>
> > > >> What's the corresponding maven command?
> > > >>
> > > >> Cheers
> > > >> Michael.
> > > >>
> > > >
> > >
> >
>


Re: maven command to run a specific test in a test case.

2019-09-20 Thread Michael Han
just figured out, here is one concrete example:


mvn test -DfailIfNoTests=false -Dtest=CreateTest#testCreate

I have to set the "-DfailIfNoTests=false" to skip the modules that don't
have a test.

On Fri, Sep 20, 2019 at 2:27 PM Michael Han  wrote:

> Thanks Andor, I tried that before but got some errors (still trying to
> figure out what those errors mean). Any chance you can have a concrete
> canonical example that execute a real single test via maven?
>
> On a side note, "mvn test" works for me which ran all tests, when executed
> under root zookeeper folder.
>
> On Fri, Sep 20, 2019 at 2:11 PM Andor Molnar 
> wrote:
>
>> Even better to do a quick google search:
>>
>>
>> https://maven.apache.org/surefire/maven-surefire-plugin/examples/single-test.html
>>
>> mvn -Dtest=TestCircle#mytest test
>>
>> Andor
>>
>> On Fri, Sep 20, 2019 at 11:08 PM Andor Molnar  wrote:
>>
>> > I usually use:
>> >
>> > mvn verify -Dtest=bar
>> >
>> > Which runs the entire testcase. Not sure about how to run single test
>> > method.
>> >
>> > Andor
>> >
>> >
>> >
>> > On Fri, Sep 20, 2019 at 11:05 PM Michael Han  wrote:
>> >
>> >> In ant, we can execute a single test "foo" in a test case "bar" like
>> this:
>> >>
>> >> *ant -Dtestcase=bar -Dtest.method=foo*
>> >>
>> >> What's the corresponding maven command?
>> >>
>> >> Cheers
>> >> Michael.
>> >>
>> >
>>
>


Re: maven command to run a specific test in a test case.

2019-09-20 Thread Michael Han
Thanks Andor, I tried that before but got some errors (still trying to
figure out what those errors mean). Any chance you can have a concrete
canonical example that execute a real single test via maven?

On a side note, "mvn test" works for me which ran all tests, when executed
under root zookeeper folder.

On Fri, Sep 20, 2019 at 2:11 PM Andor Molnar 
wrote:

> Even better to do a quick google search:
>
>
> https://maven.apache.org/surefire/maven-surefire-plugin/examples/single-test.html
>
> mvn -Dtest=TestCircle#mytest test
>
> Andor
>
> On Fri, Sep 20, 2019 at 11:08 PM Andor Molnar  wrote:
>
> > I usually use:
> >
> > mvn verify -Dtest=bar
> >
> > Which runs the entire testcase. Not sure about how to run single test
> > method.
> >
> > Andor
> >
> >
> >
> > On Fri, Sep 20, 2019 at 11:05 PM Michael Han  wrote:
> >
> >> In ant, we can execute a single test "foo" in a test case "bar" like
> this:
> >>
> >> *ant -Dtestcase=bar -Dtest.method=foo*
> >>
> >> What's the corresponding maven command?
> >>
> >> Cheers
> >> Michael.
> >>
> >
>


maven command to run a specific test in a test case.

2019-09-20 Thread Michael Han
In ant, we can execute a single test "foo" in a test case "bar" like this:

*ant -Dtestcase=bar -Dtest.method=foo*

What's the corresponding maven command?

Cheers
Michael.


[jira] [Created] (ZOOKEEPER-3548) Redundant zxid check in SnapStream.isValidSnapshot

2019-09-16 Thread Michael Han (Jira)
Michael Han created ZOOKEEPER-3548:
--

 Summary: Redundant zxid check in SnapStream.isValidSnapshot
 Key: ZOOKEEPER-3548
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3548
 Project: ZooKeeper
  Issue Type: Improvement
  Components: server
Reporter: Michael Han
Assignee: Michael Han


getZxidFromName is called twice in isValidSnapshot, and the second call is 
redundant and should be removed.



--
This message was sent by Atlassian Jira
(v8.3.2#803003)


Re: PoweredBy Zookeeper

2019-09-13 Thread Michael Han
Nice list on the Curator wiki page!

The pull request already include contents from ZooKeeper wiki, and it added
additional contents. Probably we should also add contents to ZooKeeper
wiki? I feel the documentation might have a wider audience than the wiki
(which is less known).

>> Usually maintainers of projects actively adk for their project to be
added,

I left a comment on the pull request, for copy right materials I think we
definitely need consent agreement.

I am not very sure if we'd have to reach out and explicitly ask ACK for
general cases from legal perspective, so I'll leave for others to comment
here.

On Fri, Sep 13, 2019 at 2:03 PM Patrick Hunt  wrote:

> forever and a day :-)
> https://cwiki.apache.org/confluence/display/ZOOKEEPER/PoweredBy
>
> Patrick
>
> On Fri, Sep 13, 2019 at 1:40 PM Jordan Zimmerman <
> jor...@jordanzimmerman.com>
> wrote:
>
> > FYI - we've had this on our wiki forever. By extension all these are
> > powered by ZK:
> >
> > https://cwiki.apache.org/confluence/display/CURATOR/Powered+By <
> > https://cwiki.apache.org/confluence/display/CURATOR/Powered+By>
> >
> > -Jordan
> >
> > > On Sep 13, 2019, at 3:21 PM, Enrico Olivelli 
> > wrote:
> > >
> > > Hi,
> > > Maoling started this initiative to have a great Zookeeper use cases
> page:
> > >
> > > https://github.com/apache/zookeeper/pull/1073
> > >
> > > I think this is a good idea, but before 'approving' that change I feel
> we
> > > must discuss more about it.
> > > The major point for me is whether we should ask for consent for
> > publishing
> > > such information
> > >
> > > Usually maintainers of projects actively adk for their project to be
> > added,
> > > with this page we are citing thid party productd and companies
> > >
> > > Once we clear this point I will be happy to merge that change.
> > >
> > > Thanks Maoling
> > >
> > >
> > > Enrico
> >
> >
>


Re: Subscribing to security@

2019-08-30 Thread Michael Han
iiuc, I did through emailing security-subscr...@zookeeper.apache.org.

On Fri, Aug 30, 2019 at 11:16 AM Enrico Olivelli 
wrote:

> Hello,
> How can I subscribe to secur...@zookeeper.apache.org?
>
> There is no way to subscribe using committers self service app
>
>
> Enrico
>


Re: Cutting Zookeeper 3.5.6

2019-08-30 Thread Michael Han
I said I will prepare a patch, and I managed to find sometime today to put
up
https://github.com/apache/zookeeper/pull/1069

I don't want us to rush something in, so please take time to review this.
As Brian pointed out, there is not lots of code here, just some thinking
and consensus from community on the approach we should take here.


On Fri, Aug 30, 2019 at 4:43 AM Enrico Olivelli  wrote:

> Il ven 30 ago 2019, 02:56 Brian Nixon  ha
> scritto:
>
> > I agree that ZOOKEEPER-3056 would be super nice to have in 3.5.6.
> >
>
>
> Does anyway want to pick it up? I can want until early next week.
> Otherwise I will start tonight (within 6-7 hours)
>
> Enrico
>
>
> > Don't know what shape the solution should be but don't think it requires
> > that much code - just thinking and community agreement.
> >
> >
> > On Thu, Aug 29, 2019 at 4:31 PM Michael Han  wrote:
> >
> > > ZOOKEEPER-3056 would be nice to have, I'll try to have a patch ready
> > soon,
> > > but it's not a blocker given workarounds available (unless community
> > votes
> > > we really need this for 3.5.6).
> > >
> > > On Thu, Aug 29, 2019 at 8:16 AM Zili Chen 
> wrote:
> > >
> > > > Great!
> > > >
> > > > IIRC it is possible to add a customized kanban on JIRA site,
> > > specifically,
> > > > to track all issues marked as "blocker" and to the target release. If
> > we
> > > > make use of that we can ensure no blockers before released.
> > > >
> > > > Best,
> > > > tison.
> > > >
> > > >
> > > > Andor Molnar  于2019年8月29日周四 下午10:47写道:
> > > >
> > > > > Great +1
> > > > >
> > > > > On Thu, Aug 29, 2019 at 4:37 PM Enrico Olivelli <
> eolive...@gmail.com
> > >
> > > > > wrote:
> > > > >
> > > > > > Hello,
> > > > > > I am going to cut 3.5.6, if you have patches to cherry pick or
> you
> > > are
> > > > > > aware of blocker issues for such release please tell me as soon
> as
> > > > > > possible.
> > > > > >
> > > > > > I will start the procedure tomorrow
> > > > > >
> > > > > >
> > > > > > Enrico
> > > > > >
> > > > >
> > > >
> > >
> >
>


Re: Cutting Zookeeper 3.5.6

2019-08-29 Thread Michael Han
ZOOKEEPER-3056 would be nice to have, I'll try to have a patch ready soon,
but it's not a blocker given workarounds available (unless community votes
we really need this for 3.5.6).

On Thu, Aug 29, 2019 at 8:16 AM Zili Chen  wrote:

> Great!
>
> IIRC it is possible to add a customized kanban on JIRA site, specifically,
> to track all issues marked as "blocker" and to the target release. If we
> make use of that we can ensure no blockers before released.
>
> Best,
> tison.
>
>
> Andor Molnar  于2019年8月29日周四 下午10:47写道:
>
> > Great +1
> >
> > On Thu, Aug 29, 2019 at 4:37 PM Enrico Olivelli 
> > wrote:
> >
> > > Hello,
> > > I am going to cut 3.5.6, if you have patches to cherry pick or you are
> > > aware of blocker issues for such release please tell me as soon as
> > > possible.
> > >
> > > I will start the procedure tomorrow
> > >
> > >
> > > Enrico
> > >
> >
>


Re: thoughts about extension to multi semantics

2019-08-17 Thread Michael Han
>> I would guess that it actually gets very, very little

Can't speak for others, but for the zookeeper clusters I maintain
internally, multi was used extensively in certain use cases.

>> how much would an extension to multi actually be used

This looks like a chicken egg problem to me. I feel the improved
expressiveness of multi might lead to some use cases thus promote the multi
usage in a positive feedback loop.

On Fri, Aug 16, 2019 at 11:01 PM Ted Dunning  wrote:

> It definitely sounds like a nice feature.
>
> The important question is what is the actual importance after you multiply
> it by the amount of usage it gets.
>
> For instance, I know that multi gets a bit of usage, but I would guess that
> it actually gets very, very little. It might even most of the cases that
> you have in mind.
>
> If that is so, how much would an extension to multi actually be used?
>
>
>
> On Fri, Aug 16, 2019 at 8:28 PM Michael Han  wrote:
>
> > This sounds a nice feature to me as it enables user to do more without
> > obvious downside. It could be useful in cases like state management where
> > the state is stored in a fine grained approach across multiple zNode,
> > instead of in a single zNode.
> >
> > On Fri, Aug 16, 2019 at 11:52 AM Ted Dunning 
> > wrote:
> >
> > > The recent discussion about if/then/else idioms in ZK has raised the
> > > thought that it might be nice to have some extended semantics.
> > >
> > > One version that I could see would be to to extend the current multi-op
> > to
> > > allow multiple alternatives. The idea would be that there would
> > effectively
> > > be multiple branches to be tried. The first one that succeeds
> atomically
> > > (all or nothing) would be used. The returned value would need to
> somehow
> > > indicate which alternative succeeded and would need to return any data
> > > accessed. The testing of alternatives would also be atomic so it
> wouldn't
> > > be possible for things to change within a single operation.
> > >
> > > This extension would allow the previous question to be answered like
> > this:
> > >
> > >pick_first {
> > >  create(...)
> > >} {
> > >  set(...)
> > >}
> > >
> > > (the syntax here is just made up and wouldn't actually be supported ...
> > it
> > > is just for pseudo code purposes).
> > >
> > >
> > > My theory is that this would be relatively easy to implement based on
> the
> > > current multi operation. Risk due to the change is pretty low given
> that
> > > there is code to copy.
> > >
> > > My question is whether this would actually have all that much benefit.
> > >
> > > Does anybody have an opinion on that?
> > >
> >
>


Re: thoughts about extension to multi semantics

2019-08-16 Thread Michael Han
This sounds a nice feature to me as it enables user to do more without
obvious downside. It could be useful in cases like state management where
the state is stored in a fine grained approach across multiple zNode,
instead of in a single zNode.

On Fri, Aug 16, 2019 at 11:52 AM Ted Dunning  wrote:

> The recent discussion about if/then/else idioms in ZK has raised the
> thought that it might be nice to have some extended semantics.
>
> One version that I could see would be to to extend the current multi-op to
> allow multiple alternatives. The idea would be that there would effectively
> be multiple branches to be tried. The first one that succeeds atomically
> (all or nothing) would be used. The returned value would need to somehow
> indicate which alternative succeeded and would need to return any data
> accessed. The testing of alternatives would also be atomic so it wouldn't
> be possible for things to change within a single operation.
>
> This extension would allow the previous question to be answered like this:
>
>pick_first {
>  create(...)
>} {
>  set(...)
>}
>
> (the syntax here is just made up and wouldn't actually be supported ... it
> is just for pseudo code purposes).
>
>
> My theory is that this would be relatively easy to implement based on the
> current multi operation. Risk due to the change is pretty low given that
> there is code to copy.
>
> My question is whether this would actually have all that much benefit.
>
> Does anybody have an opinion on that?
>


Re: [Request For Review] ZOOKEEPER-3474 a.k.a GH-1028

2019-08-08 Thread Michael Han
hi tison, that sounds good to me, thanks

On Thu, Aug 8, 2019 at 6:40 PM Zili Chen  wrote:

> Hi Michael,
>
> After thinking about it, I think enable checkstyle configuration
> is only about formatting. And I would make sure that there is no
> refactor. And forked repo maintainers can rebase their work on the
> new master just by manually applying patches. If the forked repo
> has been quite diverged from master of apache/zookeeper, a better
> way for syncing is to cherry-pick commits from apache/zookeeper and
> just ignore this checkstyle stuff.
>
> Best,
> tison.
>
>
> Zili Chen  于2019年8月8日周四 上午9:20写道:
>
> > @Michael
> >
> > FYI, it is possible by properly setting and updating suppressing rules.
> >
> > Best,
> > tison.
> >
> >
> > Michael Han  于2019年8月8日周四 上午9:12写道:
> >
> >> I think a good approach is doing this incrementally. Doing this all at
> >> once
> >> over entire server code base will make life much harder for those who
> >> maintain their own ZooKeeper forks and has internal patches, and the
> >> change
> >> itself will be impossible to review (even though most changes are
> >> mechanical, we still need review the change).
> >>
> >> On Mon, Aug 5, 2019 at 10:57 PM Zili Chen  wrote:
> >>
> >> > >I see some javadoc related issues in your gist,
> >> > >didn't we disable that rule ? JavadocType
> >> >
> >> > The failures reported are "Missing javadoc", and
> >> > what we disabled is "Empty javadoc". Fair enough to
> >> > disable "Missing javadoc" for now and filed into
> >> > ZOOKEEPER-3469 since a empty javadoc is also a
> >> > placeholder.
> >> >
> >> > >Are you using some automatic tool ?
> >> >
> >> > Yes, IntelliJ helps a lot.
> >> > Best,
> >> > tison.
> >> >
> >> >
> >> > Enrico Olivelli  于2019年8月6日周二 下午1:13写道:
> >> >
> >> > > Il giorno mar 6 ago 2019 alle ore 03:51 Zili Chen <
> >> wander4...@gmail.com>
> >> > > ha
> >> > > scritto:
> >> > >
> >> > > > Hi Enrico,
> >> > > >
> >> > > > Thanks for your participant!
> >> > > >
> >> > > > Running after turn on checkstyle on zookeeper-server
> >> > > > it reports about 9000 checkstyle failures(see failures.txt[1])
> >> > > > among 596 files(see files.txt[1]). Most of them
> >> > > > are related to whitespace and import.
> >> > > >
> >> > >
> >> > > I see some javadoc related issues in your gist,
> >> > > didn't we disable that rule ? JavadocType
> >> > >
> >> > >
> >> > > > I think I can handle it with one or two whole days but
> >> > > > I'm afraid that I have no spare time until next weekend(08.17).
> >> > > >
> >> > >
> >> > > Are you using some automatic tool ?
> >> > > In the past few months in my company I have been working in adding
> >> > > checkstyle to other projects and
> >> > > we used Apache NetBeans to fix up the code, I guess IntelliJ can do
> >> the
> >> > > same.
> >> > >
> >> > > Enrico
> >> > >
> >> > >
> >> > > >
> >> > > > Also I would prefer make it into one big PR and separate
> >> > > > commits per file, i.e., resolving failures per file for
> >> > > > smoothly review.
> >> > > >
> >> > > > Best,
> >> > > > tison.
> >> > > >
> >> > > > [1]
> >> https://gist.github.com/TisonKun/ba69524defbf3e9d4c99eaf2de338e5a
> >> > > >
> >> > > >
> >> > > > Zili Chen  于2019年8月6日周二 上午9:43写道:
> >> > > >
> >> > > > > Hi Enrico,
> >> > > > >
> >> > > > > Thanks for your participant!
> >> > > > >
> >> > > > > Running after turn on checkstyle on zookeeper-server
> >> > > > > it reports about 9000 checkstyle failures(see out.txt)
> >> > > > > among 596 files(see conflicts.txt). Most of them
> >> > > > > are related to whitespace and import.
> >> > > &

Re: [Request For Review] ZOOKEEPER-3474 a.k.a GH-1028

2019-08-07 Thread Michael Han
I think a good approach is doing this incrementally. Doing this all at once
over entire server code base will make life much harder for those who
maintain their own ZooKeeper forks and has internal patches, and the change
itself will be impossible to review (even though most changes are
mechanical, we still need review the change).

On Mon, Aug 5, 2019 at 10:57 PM Zili Chen  wrote:

> >I see some javadoc related issues in your gist,
> >didn't we disable that rule ? JavadocType
>
> The failures reported are "Missing javadoc", and
> what we disabled is "Empty javadoc". Fair enough to
> disable "Missing javadoc" for now and filed into
> ZOOKEEPER-3469 since a empty javadoc is also a
> placeholder.
>
> >Are you using some automatic tool ?
>
> Yes, IntelliJ helps a lot.
> Best,
> tison.
>
>
> Enrico Olivelli  于2019年8月6日周二 下午1:13写道:
>
> > Il giorno mar 6 ago 2019 alle ore 03:51 Zili Chen 
> > ha
> > scritto:
> >
> > > Hi Enrico,
> > >
> > > Thanks for your participant!
> > >
> > > Running after turn on checkstyle on zookeeper-server
> > > it reports about 9000 checkstyle failures(see failures.txt[1])
> > > among 596 files(see files.txt[1]). Most of them
> > > are related to whitespace and import.
> > >
> >
> > I see some javadoc related issues in your gist,
> > didn't we disable that rule ? JavadocType
> >
> >
> > > I think I can handle it with one or two whole days but
> > > I'm afraid that I have no spare time until next weekend(08.17).
> > >
> >
> > Are you using some automatic tool ?
> > In the past few months in my company I have been working in adding
> > checkstyle to other projects and
> > we used Apache NetBeans to fix up the code, I guess IntelliJ can do the
> > same.
> >
> > Enrico
> >
> >
> > >
> > > Also I would prefer make it into one big PR and separate
> > > commits per file, i.e., resolving failures per file for
> > > smoothly review.
> > >
> > > Best,
> > > tison.
> > >
> > > [1] https://gist.github.com/TisonKun/ba69524defbf3e9d4c99eaf2de338e5a
> > >
> > >
> > > Zili Chen  于2019年8月6日周二 上午9:43写道:
> > >
> > > > Hi Enrico,
> > > >
> > > > Thanks for your participant!
> > > >
> > > > Running after turn on checkstyle on zookeeper-server
> > > > it reports about 9000 checkstyle failures(see out.txt)
> > > > among 596 files(see conflicts.txt). Most of them
> > > > are related to whitespace and import.
> > > >
> > > > I think I can handle it with one or two whole days but
> > > > I'm afraid that I have no spare time until next weekend(08.17).
> > > >
> > > > Also I would prefer make it into one big PR and separate
> > > > commits per file, i.e., resolving failures per file for
> > > > smoothly review.
> > > >
> > > > Best,
> > > > tison.
> > > >
> > > >
> > > > Enrico Olivelli  于2019年8月5日周一 下午9:37写道:
> > > >
> > > >> Il giorno lun 5 ago 2019 alle ore 14:03 Zili Chen <
> > wander4...@gmail.com
> > > >
> > > >> ha
> > > >> scritto:
> > > >>
> > > >> > Hi zookeepers,
> > > >> >
> > > >> > Recently I've sent a pr[1] on enable checkstyle configuration on
> > > >> > zookeeper-promethus-metrics, Enrico(committer) and
> > Dinesh(contributor)
> > > >> > kindly reviewed it but we still need another look from committer
> as
> > > >> > process required. Someone of you has spare time to have a look?
> > > >> >
> > > >>
> > > >> Tison,
> > > >> Norbert approved the PR and I have merged it.
> > > >> We have already begun this check style work.
> > > >> Thank you so much to you and Mao Ling for these efforts.
> > > >>
> > > >>
> > > >> >
> > > >> > Besides, as ZOOKEEPER-3431 described, we're going to enable this
> > > >> > configuration on zookeeper-server as following. This is really a
> > major
> > > >> > package which contains a lot of codes involved by multi prs. I'd
> > like
> > > to
> > > >> > hear from the community about how to proceed.
> > > >> >
> > > >>
> > > >> I think that if we do it quickly there will be as few troubles as
> > > >> possible.
> > > >> I am available for review, just ping me.
> > > >>
> > > >>
> > > >>
> > > >> >
> > > >> > (From where I stand, enable on the whole package at once
> > > >> > could solve the problem at once, but it would conflict quite a lot
> > > >> > ongoing prs. On the other hand, sending the pr without timely
> > reviews
> > > >> > also causes a burden to rebase it on the master and solve conflict
> > > >> > again and again(since it likely conflicts with a merged pr))
> > > >> >
> > > >>
> > > >> Did you check how many violations we have con zookeeper-server ?
> > > >> How much big is the work ?
> > > >> Do you have an estimate ?
> > > >>
> > > >> We could take into consideration to have only one big PR.
> > > >> The tradeoff is that it will be very difficult to perform a review,
> we
> > > >> will
> > > >> need more eyes on the diff
> > > >>
> > > >> Enrico
> > > >>
> > > >>
> > > >>
> > > >>
> > > >> >
> > > >> > Best,
> > > >> > tison.
> > > >> >
> > > >> > [1] https://github.com/apache/zookeeper/pull/1028
> > > >> >
> > > >>
> > > >
> > >
> >
>


Re: KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

2019-08-07 Thread Michael Han
Related discussions:
https://lists.apache.org/list.html?d...@kafka.apache.org:lte=1M:KIP-500

>> Why would this lead to any discrepancy? It seems to me that the
controller, will read an even newer state in such a scenario.

I think in this case the Kafka side of expectation is not just be able to
get latest state, but also not miss any of the state changes. See the Kafka
thread ^ which said: *"Treating metadata as a log avoids a lot of the
complex failure corner cases we have seen where a broker misses a single
update sent from the controller, but gets subsequent updates."*

We don't support this in ZK today - clients will always be able to get
latest state, but client might miss watcher events between the first
watcher fired and the reset of the watcher on client side (and any changes
happening in this period will not get notified). The subscribe API can
solve this.

One thing that the proposal only briefly mentioned but I think is a key
differentiator is scalability: managing metadata as event logs will provide
better scalability, as ZK is inherently limited by memory. One can shard a
cluster to work around the limits, but that creates other problems
(consistency, operation, etc). Having an on disk storage backend for ZK's
data tree might solve this.

On Wed, Aug 7, 2019 at 12:25 PM Ted Dunning  wrote:

> On Wed, Aug 7, 2019 at 11:31 AM Karolos Antoniadis 
> wrote:
>
> > In the paragraph that Michael mentioned, among others it is written: "For
> > example, when a partition leader changes its ISR in ZK, the controller
> will
> > typically not learn about these changes for many seconds." Why would it
> > take "many seconds"?
>
>
> I think that this is conflating the situation with partition from or of the
> ZK cluster with simple hand-offs.
>
> In the case of a partition leader crashing, it will be several seconds
> before the rest of the world hears about the event.
>
>
> > Sending a watch event to the controller should be
> > pretty fast.
> >
>
> Absolutely. If the ZK cluster has its act together. And if the cause of the
> watch is detected quickly. And if you don't have a watch storm happening
> due to huge numbers of listeners.
>
> But none of those problems are really helped by moving the consensus
> algorithms into a library.
>
> Also, in the same paragraph, Colin states "By the time the controller
> > re-reads the znode and sets up a new watch, the state may have changed
> from
> > what it was when the watch originally fired.  [...] only way to resolve
> the
> > discrepancy." Why would this lead to any discrepancy? It seems to me that
> > the controller, will read an even newer state in such a scenario.
> >
>
> You are correct and this has always been one of the selling points of ZK.
> The way that you can reset the watch as part of the read operation means
> that you can guarantee never to lose anything and if you are slow to
> respond, you always get data that is as up-to-date as possible. Load
> shedding tricks like that are really helpful. Getting notifications of
> every change is actually disastrous in many cases, partly because of the
> number of notifications and partly because the notifications can become
> very heavy-weight with the data they have to carry.
>
> Putting this into a library doesn't help at all, of course.
>
>
>
> >
> > Also, another argument mentioned in original KIP-500 proposal had to do
> > with speeding up the failover of a controller: "Because the controllers
> > will now all track the latest state, controller failover will not
> require a
> > lengthy reloading period where we transfer all the state to the new
> > controller." But this does not seem to be a problem with ZK per se and
> > could be solved by keeping a broker as a standby controller (briefly
> > mentioned here https://www.slideshare.net/ConfluentInc/a-deep-dive-into-
> > kafka-controller
> > <
> https://www.slideshare.net/ConfluentInc/a-deep-dive-into-kafka-controller>
> > as future work.)
> >
>
> Also, the state still has to move. Using an in-process library doesn't
> change that at all. It could move via ZK or it could move as part of quorum
> decisions or via some sort of follow-the-leader protocol. But it has to
> move. Whoever is leader has to write it out to the network and whoever is
> follower has to read it in. Whether the data is written/read directly or
> via ZK isn't really a big deal.
>
>
> >
>


Re: KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

2019-08-02 Thread Michael Han
Very well said, thank you Ted!

>> I would still opt for quorum outside rather than quorum as a library.

One observation on out side quorum vs library: for Raft, cockroach db and
TiDB both choose the library approach instead of depending on etcd, though
they all share the etcd's Raft implementation. ZooKeeper could be used in a
similar approach if we can abstract ZAB and provides a nice SMR interface
on top of it.

On Fri, Aug 2, 2019 at 12:44 PM Ted Dunning  wrote:

> The core issue in these situations in my experience is that having the
> quorum as a separate service can be a pain point. This misunderstanding
> about how watches work and why they don't provide the data is just a
> symptom of this. Having an integrated quorum is very attractive from the
> point of view of management and tighter integration with the record of
> state.
>
> If I had it all to do over again, though, I think I would still opt for
> quorum outside rather than quorum as a library. There are management
> burdens, but many of those management burdens are implicit in the fact that
> managing the state of the system is different from managing the system or
> doing the stuff the system does. Pulling the quorum system into the
> do-stuff system doesn't actually make life all that much easier even if it
> does simplify the installer.
>
> The countervailing risk that you are likely to get a quorum system wrong is
> really significant. Having a battle-tested (some might say battle-scarred)
> system like ZK is quite a virtue since you can have a different level of
> confidence in it than something you whipped up last week.
>
>
>
> On Fri, Aug 2, 2019 at 11:49 AM Patrick Hunt  wrote:
>
> > Michael I think you are describing subscribe - this?
> > https://issues.apache.org/jira/browse/ZOOKEEPER-153
> > wasn't there some work done to keep tlogs around for a while? Or am I
> miss
> > remembering? (fb folks?)
> >
> > I'll also add that we haven't done any benchmarking in quite some time.
> It
> > would be interesting to collect a few of these use cases from the
> > community, esp downstreams, and evaluate performance, see if we can
> > address.
> >
> > Patrick
> >
> > On Fri, Aug 2, 2019 at 11:03 AM Michael Han  wrote:
> >
> > > Folks,
> > >
> > > Some of you might already see this. Comments?
> > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-500%3A+Replace+ZooKeeper+with+a+Self-Managed+Metadata+Quorum
> > >
> > >
> > > What caught my eyes are:
> > >
> > > *Worse still, although ZooKeeper is the store of record, the state in
> > > ZooKeeper often doesn't match the state that is held in memory in the
> > > controller.  For example, when a partition leader changes its ISR in
> ZK,
> > > the controller will typically not learn about these changes for many
> > > seconds.  There is no generic way for the controller to follow the
> > > ZooKeeper event log.  Although the controller can set one-shot watches,
> > the
> > > number of watches is limited for performance reasons.  When a watch
> > > triggers, it doesn't tell the controller the current state-- only that
> > the
> > > state has changed.  By the time the controller re-reads the znode and
> > sets
> > > up a new watch, the state may have changed from what it was when the
> > watch
> > > originally fired.  If there is no watch set, the controller may not
> learn
> > > about the change at all.  In some cases, restarting the controller is
> the
> > > only way to resolve the discrepancy.*
> > >
> > > I've seen some similar zookeeper use cases that ended up like what's
> > > described here. How can ZooKeeper solve this? It seems to me that the
> > only
> > > solution is to provide linearizable read on watched operations.
> Thoughts?
> > >
> > > Michael.
> > >
> >
>


Re: KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

2019-08-02 Thread Michael Han
Hi Pat,

Yes, ZOOKEEPER-153 could help this case. The gist of the issue is reliable
change notification with data. The linearizable read I had in mind alone
might not solve this as it's missing the reliably capturing change
notification part.

>> I'll also add that we haven't done any benchmarking in quite some time.

I think this is a very good point. The existing public benchmarks are
either targeted old version, or not optimally set up. This creates a gap
between current scalability and performance of ZK and the existing (usually
negative) public perception. With many improvements on scale / perf in last
2 years the status quo is very different now.

On Fri, Aug 2, 2019 at 11:49 AM Patrick Hunt  wrote:

> Michael I think you are describing subscribe - this?
> https://issues.apache.org/jira/browse/ZOOKEEPER-153
> wasn't there some work done to keep tlogs around for a while? Or am I miss
> remembering? (fb folks?)
>
> I'll also add that we haven't done any benchmarking in quite some time. It
> would be interesting to collect a few of these use cases from the
> community, esp downstreams, and evaluate performance, see if we can
> address.
>
> Patrick
>
> On Fri, Aug 2, 2019 at 11:03 AM Michael Han  wrote:
>
> > Folks,
> >
> > Some of you might already see this. Comments?
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-500%3A+Replace+ZooKeeper+with+a+Self-Managed+Metadata+Quorum
> >
> >
> > What caught my eyes are:
> >
> > *Worse still, although ZooKeeper is the store of record, the state in
> > ZooKeeper often doesn't match the state that is held in memory in the
> > controller.  For example, when a partition leader changes its ISR in ZK,
> > the controller will typically not learn about these changes for many
> > seconds.  There is no generic way for the controller to follow the
> > ZooKeeper event log.  Although the controller can set one-shot watches,
> the
> > number of watches is limited for performance reasons.  When a watch
> > triggers, it doesn't tell the controller the current state-- only that
> the
> > state has changed.  By the time the controller re-reads the znode and
> sets
> > up a new watch, the state may have changed from what it was when the
> watch
> > originally fired.  If there is no watch set, the controller may not learn
> > about the change at all.  In some cases, restarting the controller is the
> > only way to resolve the discrepancy.*
> >
> > I've seen some similar zookeeper use cases that ended up like what's
> > described here. How can ZooKeeper solve this? It seems to me that the
> only
> > solution is to provide linearizable read on watched operations. Thoughts?
> >
> > Michael.
> >
>


KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

2019-08-02 Thread Michael Han
Folks,

Some of you might already see this. Comments?
https://cwiki.apache.org/confluence/display/KAFKA/KIP-500%3A+Replace+ZooKeeper+with+a+Self-Managed+Metadata+Quorum


What caught my eyes are:

*Worse still, although ZooKeeper is the store of record, the state in
ZooKeeper often doesn't match the state that is held in memory in the
controller.  For example, when a partition leader changes its ISR in ZK,
the controller will typically not learn about these changes for many
seconds.  There is no generic way for the controller to follow the
ZooKeeper event log.  Although the controller can set one-shot watches, the
number of watches is limited for performance reasons.  When a watch
triggers, it doesn't tell the controller the current state-- only that the
state has changed.  By the time the controller re-reads the znode and sets
up a new watch, the state may have changed from what it was when the watch
originally fired.  If there is no watch set, the controller may not learn
about the change at all.  In some cases, restarting the controller is the
only way to resolve the discrepancy.*

I've seen some similar zookeeper use cases that ended up like what's
described here. How can ZooKeeper solve this? It seems to me that the only
solution is to provide linearizable read on watched operations. Thoughts?

Michael.


[jira] [Created] (ZOOKEEPER-3483) Flaky test: org.apache.zookeeper.server.util.RequestPathMetricsCollectorTest.testCollectStats

2019-08-01 Thread Michael Han (JIRA)
Michael Han created ZOOKEEPER-3483:
--

 Summary: Flaky test: 
org.apache.zookeeper.server.util.RequestPathMetricsCollectorTest.testCollectStats
 Key: ZOOKEEPER-3483
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3483
 Project: ZooKeeper
  Issue Type: Test
Affects Versions: 3.6.0
Reporter: Michael Han
Assignee: Michael Han


Test 
org.apache.zookeeper.server.util.RequestPathMetricsCollectorTest.testCollectStats
 consistently pass on local dev environment but frequently failing on Jenkins 
pre-commit build.

For now disable the test to unblock a couple of pull request acquiring a green 
build, before it's completely addressed.

Error for reference:

{code:java}
Error Message
expected:<845466> but was:<111>
Stacktrace
java.lang.AssertionError: expected:<845466> but was:<111>
at 
org.apache.zookeeper.server.util.RequestPathMetricsCollectorTest.testCollectStats(RequestPathMetricsCollectorTest.java:248)
{code}




--
This message was sent by Atlassian JIRA
(v7.6.14#76016)


Re: Leader election logging during reconfiguration

2019-07-30 Thread Michael Han
>> we should measure the total time more accurately

+1 - it would be good to have a new metric to measure reconfiguration time,
and leaving existing LE time metric dedicated to measure the conventional
FLE time. Mixing both (as of today) will provide some confusing insights on
how long the conventional FLE actually took.

On Mon, Jul 29, 2019 at 7:13 PM Alexander Shraer  wrote:

> Please see comments inline.
>
> Thanks,
> Alex
>
> On Mon, Jul 29, 2019 at 5:29 PM Karolos Antoniadis 
> wrote:
>
> > Hi ZooKeeper developers,
> >
> > ZooKeeper seems to be logging a "*LEADER ELECTION TOOK*" message even
> > though no leader election takes place during a reconfiguration.
> >
> > This can be reproduced by following these steps:
> > 1) start a ZooKeeper cluster (e.g., 3 participants)
> > 2) start a client that connects to some follower
> > 3) perform a *reconfig* operation that removes the leader from the
> cluster
> >
> > After the reconfiguration takes place, we can see that the log files of
> the
> > remaining participants contain a "*LEADER ELECTION TOOK*" message. For
> > example, a line that contains
> >
> > *2019-07-29 23:07:38,518 [myid:2] - INFO
> >  [QuorumPeer[myid=2](plain=0.0.0.0:2792)(secure=disabled):Follower@75] -
> > FOLLOWING - LEADER ELECTION TOOK - 57 MS*
> >
> > However, no leader election took place. With that, I mean that no server
> > went *LOOKING *and then started voting and sending notifications to other
> > participants as would be in a normal leader election.
> > It seems, that before the *reconfig *is committed, the participant that
> is
> > going to be the next leader is already decided (see here:
> >
> >
> https://github.com/apache/zookeeper/blob/master/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java#L865
> > ).
> >
> > I think the above issue raises the following questions:
> > - Should we avoid logging LEADER ELECTION messages altogether in such
> > cases?
> >
>
> In the specific scenario you described, the leader has changed, but our
> heuristic for choosing the leader apparently worked and a new leader could
> be elected without running the full election.
> Notice that we could be unlucky and the designated leader could be offline,
> and then we'll fall back on election. It's useful to know how much time it
> takes to start following the new leader.
>
>
> > - Or, should there be some logging for the time it took for the
> > reconfiguration (e.g., the time between a participant gets a *reconfig*
> > operation till the operation is committed)? Would such a time value be
> > useful?
> >
>
> IIRC the LEADER ELECTION message is used for this purpose. if you just look
> on the time to commit the reconfig operation, you won't
> account for the work that happens when the commit message is received, such
> as leader re-election, role-change (follower->observer conversion and such)
> etc which is what takes most of the time.
> Committing a reconfig operation is usually not much more expensive than
> committing a normal operation. But perhaps you're right that we should
> measure the total time more accurately. Would you
> like to open a Jira and perhaps take a stab at improving this ?
>
> >
> > Best,
> > Karolos
> >
>


Re: C client reviews needed

2019-07-29 Thread Michael Han
I'll start looking this week.

On Sun, Jul 28, 2019 at 11:02 PM Enrico Olivelli 
wrote:

> Hello folks,
> We have a bunch of pending requests about the C client.
>
> is there any reviewer available to help merging them ?
>
> we are talking about SSL support (cool!) and a few bugfixes
>
> Enrico
>


Re: complete quota system proposal

2019-07-18 Thread Michael Han
Sounds good. I think we also need a plan on how to identify clients. If
auth is enabled, we might be able to reuse some of the auth information,
but a generalized client-id based approach sounds better.

On Thu, Jul 18, 2019 at 11:32 AM Mocheng Guo  wrote:

> Hi Michael, thanks for your feedback and I will create an epic for this. I
> will divide into following stories and please let me know if this makes
> sense
>
> 1. quota configuration - metric key and value, format, storage
> 2. metrics collection and export - storage/rate/watch/connection
> 3. throttling implementation based on metrics inside server/client
>
> On Wed, Jul 17, 2019 at 8:02 PM Michael Han  wrote:
>
> > >> a more complete quota/throttling system would be valuable here
> >
> > Absolutely. This will improve the stability of a shared zookeeper
> cluster,
> > which is a very common use case. Because the lack of enforce quota (and
> > other soft isolation mechanisms such as flow control / request limiting),
> > the usual practice of dealing with such excessive client usages is to
> > provision dedicated ZK cluster per customer which creates additional
> > operation overheads and is waste of resource.
> >
> > >> We'd be happy to submit the work for review
> >
> > Sounds great. I would suggest we create an epic around this and then
> break
> > down the stories to multiple sub tasks and move existing issues that's
> > relevant under the epic, so we can consolidate efforts and have a single
> > place to track the progress.
> >
> >
> > On Tue, Jul 16, 2019 at 12:46 PM Mocheng Guo  wrote:
> >
> > > Hi, currently Zookeeper has a storage quota feature which is
> > informational
> > > only and there are a few JIRAs to enforce throttling.
> > >
> > > ZOOKEEPER-451
> > > ZOOKEEPER-2593
> > > ZOOKEEPER-3301
> > >
> > > I am wondering if a more complete quota/throttling system covering
> > metrics
> > > such as storage/rate/watch/connection would be valuable here? We at FB
> > have
> > > been battling with excessive system usage from clients and did some
> work
> > in
> > > this area. We'd be happy to submit the work for review and consolidate
> > with
> > > existing efforts if people feel this is a good feature to add.
> > >
> > > thanks
> > > Mocheng
> > >
> >
>


Re: complete quota system proposal

2019-07-17 Thread Michael Han
>> a more complete quota/throttling system would be valuable here

Absolutely. This will improve the stability of a shared zookeeper cluster,
which is a very common use case. Because the lack of enforce quota (and
other soft isolation mechanisms such as flow control / request limiting),
the usual practice of dealing with such excessive client usages is to
provision dedicated ZK cluster per customer which creates additional
operation overheads and is waste of resource.

>> We'd be happy to submit the work for review

Sounds great. I would suggest we create an epic around this and then break
down the stories to multiple sub tasks and move existing issues that's
relevant under the epic, so we can consolidate efforts and have a single
place to track the progress.


On Tue, Jul 16, 2019 at 12:46 PM Mocheng Guo  wrote:

> Hi, currently Zookeeper has a storage quota feature which is informational
> only and there are a few JIRAs to enforce throttling.
>
> ZOOKEEPER-451
> ZOOKEEPER-2593
> ZOOKEEPER-3301
>
> I am wondering if a more complete quota/throttling system covering metrics
> such as storage/rate/watch/connection would be valuable here? We at FB have
> been battling with excessive system usage from clients and did some work in
> this area. We'd be happy to submit the work for review and consolidate with
> existing efforts if people feel this is a good feature to add.
>
> thanks
> Mocheng
>


[jira] [Created] (ZOOKEEPER-3448) Introduce MessageTracker to assist debug leader and leaner connectivity issues

2019-06-28 Thread Michael Han (JIRA)
Michael Han created ZOOKEEPER-3448:
--

 Summary: Introduce MessageTracker to assist debug leader and 
leaner connectivity issues
 Key: ZOOKEEPER-3448
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3448
 Project: ZooKeeper
  Issue Type: Improvement
  Components: server
Affects Versions: 3.6.0
Reporter: Michael Han
Assignee: Michael Han


We want to have better insight on the state of the world when learners lost 
connection with leader, so we need capture more information when that happens. 
We capture more information through MessageTracker which will record the last 
few sent and received messages at various protocol stage, and these information 
will be dumped to log files for further analysis.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Created] (ZOOKEEPER-3439) Observability improvements on client / server connection close

2019-06-21 Thread Michael Han (JIRA)
Michael Han created ZOOKEEPER-3439:
--

 Summary: Observability improvements on client / server connection 
close
 Key: ZOOKEEPER-3439
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3439
 Project: ZooKeeper
  Issue Type: Improvement
  Components: server
Affects Versions: 3.6.0
Reporter: Michael Han
Assignee: Michael Han


Currently when server close a client connection there is not enough information 
recorded (except few exception logs) which makes it hard to do postmortems. On 
the other side, having a complete view of the aggregated connection closing 
reason will provide more signals based on which we can better operate the 
clusters (e.g. predicate an incident might happen based on the trending of the 
connection closing reasons).



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Created] (ZOOKEEPER-3430) Observability improvement: provide top N read / write path queries

2019-06-17 Thread Michael Han (JIRA)
Michael Han created ZOOKEEPER-3430:
--

 Summary: Observability improvement: provide top N read / write 
path queries
 Key: ZOOKEEPER-3430
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3430
 Project: ZooKeeper
  Issue Type: Improvement
  Components: server
Affects Versions: 3.6.0
Reporter: Michael Han
Assignee: Michael Han


We would like to have a better understanding of the type of workloads hit ZK, 
and one aspect of such understanding is to be able to answer queries of top N 
read and top N write request path. Knowing the hot request paths will allow us 
better optimize for such workloads, for example, enabling path specific 
caching, or change the path structure (e.g. break a long path to hierarchical 
paths).



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


Re: PostCommit- CI job ? Where it is ?

2019-06-13 Thread Michael Han
I see now. This was a job I set up long time ago for my own testing purpose
only; it was not intended to be part of official commit workflow. I am not
sure how this job becomes the official post commit job though. If we have
daily builds (do we) - then probably we don't need this. Though keep it
does not hurt, either. I have no strong preference one way or the other.


On Wed, Jun 12, 2019 at 1:50 AM Andor Molnar 
wrote:

> It might give some additional feedback to the committer/contributor whether
> the patch has been successfully integrated into the target branch. We've
> seen issues when the merge build was outdated, showing green light on the
> pull request and the commit eventually broke master after merging.
>
> I'm not 100% sure we need it, maybe other folks can give some more color to
> this.
>
> Btw. do we already have a View in jenkins for Maven builds like this one
> for Ant: https://builds.apache.org/view/S-Z/view/ZooKeeper/
>
> Andor
>
>
>
> On Wed, Jun 12, 2019 at 10:33 AM Enrico Olivelli 
> wrote:
>
> > Michael
> > it seems that the job is trying to send emails to you at
> l...@twitter.com
> > but there is a typo.
> >
> > Do we still need this job?
> > As we are dropping ANT we have to migrate it, but I don't find much value
> > as we are already performing daily builds and precommits.
> > Having such comment in jira is not so important
> >
> > Enrico
> >
> > Enrico
> >
> > Il mer 12 giu 2019, 10:04 Enrico Olivelli  ha
> > scritto:
> >
> > > Self answered from the message on jira...
> > >
> > > It is zookeeper-trunk-single-thread
> > >
> > > Thanks Michael
> > >
> > > Enrico
> > >
> > > SUCCESS: Integrated in Jenkins build Zookeeper-trunk-single-thread #388
> > > (See [https://builds.apache.org/job/Zookeeper-trunk-single-thread/388/
> ])
> > >
> > > ZOOKEEPER-3366: Pluggable metrics system for ZooKeeper - move remaining
> > > (nkalmar: rev 6af151a7e44760c33165e58d3b33a4d2760283a4)
> > > * (edit) zookeeper-server/src/main/java/org/apache/zookeeper/metrics/
> > > MetricsContext.java
> > > * (edit) zookeeper-server/src/main/java/org/apache/zookeeper/server/
> > > command/MonitorCommand.java
> > > * (edit) zookeeper-server/src/main/java/org/apache/zookeeper/server/
> > >
> > >
> > >
> > > Il mer 12 giu 2019, 02:00 Michael Han  ha scritto:
> > >
> > >> Hi Enrico,
> > >>
> > >> I don't think it's a post commit job that adds the "Issue resolved by
> > pull
> > >> request" comment (if this is the comment you were referring to.).
> > This
> > >> was added by the merge script through JIRA api.
> > >>
> > >> On Mon, Jun 10, 2019 at 1:34 PM Enrico Olivelli 
> > >> wrote:
> > >>
> > >> > Hi ZooKeepers,
> > >> > I am looking for the Post-Commit job, the one that adds that JIRA
> > >> comment
> > >> > after we merge a PR.
> > >> >
> > >> > I would like to understand it and decide whether to drop it or to
> move
> > >> it
> > >> > to Maven.
> > >> > Personally I don't find much value in that task.
> > >> >
> > >> > This is my search [1] (I hope that the link works for you)
> > >> >
> > >> > Enrico
> > >> >
> > >> > [1]
> > >> >
> > >> >
> > >>
> >
> https://builds.apache.org/job/Zookeeper-trunk-single-thread/search/?q=zookeeper=1e1a1d3bbc6a541e903422747a16c759
> > >> >
> > >>
> > >
> >
>


Re: PostCommit- CI job ? Where it is ?

2019-06-11 Thread Michael Han
Hi Enrico,

I don't think it's a post commit job that adds the "Issue resolved by pull
request" comment (if this is the comment you were referring to.). This
was added by the merge script through JIRA api.

On Mon, Jun 10, 2019 at 1:34 PM Enrico Olivelli  wrote:

> Hi ZooKeepers,
> I am looking for the Post-Commit job, the one that adds that JIRA comment
> after we merge a PR.
>
> I would like to understand it and decide whether to drop it or to move it
> to Maven.
> Personally I don't find much value in that task.
>
> This is my search [1] (I hope that the link works for you)
>
> Enrico
>
> [1]
>
> https://builds.apache.org/job/Zookeeper-trunk-single-thread/search/?q=zookeeper=1e1a1d3bbc6a541e903422747a16c759
>


[jira] [Created] (ZOOKEEPER-3427) Introduce SnapshotComparer that assists debugging with snapshots.

2019-06-11 Thread Michael Han (JIRA)
Michael Han created ZOOKEEPER-3427:
--

 Summary: Introduce SnapshotComparer that assists debugging with 
snapshots.
 Key: ZOOKEEPER-3427
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3427
 Project: ZooKeeper
  Issue Type: Improvement
  Components: server
Affects Versions: 3.6.0
Reporter: Michael Han
Assignee: Michael Han


SnapshotComparer is a tool that loads and compares two snapshots, with 
configurable threshold and various filters. It's useful in use cases that 
involve snapshot analysis, such as offline data consistency checking, and data 
trending analysis (e.g. what's growing under which zNode path during when). 

A sample output of the tool (actual numbers removed, due to sensitivity).
{code:java}
Successfully parsed options!

Deserialized snapshot in snapshot.0 in  seconds

Processed data tree in seconds

Deserialized snapshot in snapshot.1 in  seconds

Processed data tree in seconds

Node count: 

Total size: 

Max depth: 

Count of nodes at depth 1: 

Count of nodes at depth 2: 

Count of nodes at depth 3: 

Count of nodes at depth 4: 

Count of nodes at depth 5: 

Count of nodes at depth 6: 

Count of nodes at depth 7: 

Count of nodes at depth 8: 

Count of nodes at depth 9: 

Count of nodes at depth 10: 

Count of nodes at depth 11: 


Node count: 

Total size: 

Max depth: 

Count of nodes at depth 1: 

Count of nodes at depth 2: 

Count of nodes at depth 3: 

Count of nodes at depth 4: 

Count of nodes at depth 5:

Count of nodes at depth 6:

Count of nodes at depth 7: 

Count of nodes at depth 8: 

Count of nodes at depth 9: 

Count of nodes at depth 10: 

Count of nodes at depth 11: 




Analysis for depth 0

Analysis for depth 1

Analysis for depth 2

Analysis for depth 3

Analysis for depth 4

Analysis for depth 5

Analysis for depth 6

Analysis for depth 7

Analysis for depth 8

Analysis for depth 9

Analysis for depth 10
{code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Created] (ZOOKEEPER-3419) Backup and recovery support

2019-06-06 Thread Michael Han (JIRA)
Michael Han created ZOOKEEPER-3419:
--

 Summary: Backup and recovery support
 Key: ZOOKEEPER-3419
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3419
 Project: ZooKeeper
  Issue Type: New Feature
  Components: server
Affects Versions: 3.6.0
Reporter: Michael Han
Assignee: Michael Han


Historically ZooKeeper has no intrinsic support for backup and restore. The 
usual approach of doing backup and restore is through customized scripts to 
copy data around, or through some 3rd party tools (exhibitor, etc), which 
introduces operation burden. 

This Jira will introduce another option: a direct support of backup and restore 
from ZooKeeper itself. It's completely built into ZooKeeper, support point in 
time recovery of an entire tree rooted after an oops event, support recovery 
partial tree for test/dev purpose, and can help replay history for bug 
investigation. It will try to provide a generic interface so the backups can be 
directed to different data storage systems (S3, Kafka, HDFS, etc).

This same system has been in production at Twitter for X years and proved to be 
quite helpful for various use cases mentioned earlier.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Created] (ZOOKEEPER-3418) Improve quorum throughput through eager ACL checks of requests on local servers

2019-06-06 Thread Michael Han (JIRA)
Michael Han created ZOOKEEPER-3418:
--

 Summary: Improve quorum throughput through eager ACL checks of 
requests on local servers
 Key: ZOOKEEPER-3418
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3418
 Project: ZooKeeper
  Issue Type: Improvement
  Components: server
Affects Versions: 3.6.0
Reporter: Michael Han
Assignee: Michael Han


Serving write requests that change the state of the system requires quorum 
operations, and in some cases, the quorum operations can be avoided if the 
requests are doomed to fail. ACL check failure is such a case. To optimize for 
this case, we elevate the ACL check logic and perform eager ACL check on local 
server (where the requests are received), and fail fast, before sending the 
requests to leader. 

As with any features, there is a feature flag that can control this feature on, 
or off (default). This feature is also forward compatible in that for new any 
new Op code (and some existing Op code we did not explicit check against), they 
will pass the check and (potentially) fail on leader side, instead of being 
prematurely filtered out on local server.

The end result is better throughput and stability of the quorum for certain 
workloads.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Created] (ZOOKEEPER-3416) Remove redundant ServerCnxnFactoryAccessor

2019-06-05 Thread Michael Han (JIRA)
Michael Han created ZOOKEEPER-3416:
--

 Summary: Remove redundant ServerCnxnFactoryAccessor
 Key: ZOOKEEPER-3416
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3416
 Project: ZooKeeper
  Issue Type: Improvement
  Components: tests
Affects Versions: 3.6.0
Reporter: Michael Han
Assignee: Michael Han


We have two ways to access the private zkServer inside ServerCnxnFactory, and 
there is really no need to keep maintaining both. We could remove 
ServerCnxnFactoryAccessor when we added the public accessor for 
ServerCnxnFactory in ZOOKEEPER-1346, but we did not.

The solution is to consolidate all access of the zkServer through the public 
accessor of ServerCnxnFactory. The end result is cleaner code base and less 
confusion.

 

 



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


Re: [ANNOUNCE] New ZooKeeper committer: Norbert Kalmar

2019-05-29 Thread Michael Han
Congratulations Norbert!

On Tue, May 28, 2019 at 2:29 PM Dinesh Appavoo 
wrote:

> Awesome. Congrats Norbert!
>
> On Tue, May 28, 2019 at 10:51 AM Brian Nixon 
> wrote:
>
> > Congrats, Norbert! Well earned.
> >
> > On Mon, May 27, 2019 at 3:49 AM Andor Molnar  >
> > wrote:
> >
> > > Congratulations Norbert!
> > >
> > > Andor
> > >
> > >
> > >
> > > On Mon, May 27, 2019 at 11:52 AM Flavio Junqueira 
> > wrote:
> > >
> > > > Congrats, Norbert!
> > > >
> > > > -Flavio
> > > >
> > > > > On 27 May 2019, at 11:01, Tamas Penzes  >
> > > > wrote:
> > > > >
> > > > > Congrats Norbert!
> > > > >
> > > > > On Sun, May 26, 2019 at 11:21 PM Patrick Hunt 
> > > wrote:
> > > > >
> > > > >> The Apache ZooKeeper PMC recently extended committer karma to
> > Norbert
> > > > >> and he has accepted. Norbert has made some great contributions and
> > we
> > > > >> are looking forward to even more :)
> > > > >>
> > > > >> Congratulations and welcome aboard, Norbert!
> > > > >>
> > > >
> > > >
> > >
> >
>


Re: [ANNOUNCE] New ZooKeeper committer: Enrico Olivelli

2019-05-29 Thread Michael Han
Congratulations Enrico!

On Tue, May 28, 2019 at 2:05 PM Dinesh Appavoo 
wrote:

> Great news. Congrats Enrico!
>
> On Tue, May 28, 2019 at 10:51 AM Brian Nixon 
> wrote:
>
> > Awesome! Congrats, Enrico!
> >
> > On Mon, May 27, 2019 at 3:56 AM Andor Molnar  >
> > wrote:
> >
> > > Congratulations Enrico.
> > >
> > > Andor
> > >
> > >
> > >
> > > On Mon, May 27, 2019 at 11:04 AM Tamas Penzes
> >  > > >
> > > wrote:
> > >
> > > > Congrats Enrico!
> > > >
> > > > On Sun, May 26, 2019 at 11:19 PM Patrick Hunt 
> > wrote:
> > > >
> > > > > The Apache ZooKeeper PMC recently extended committer karma to
> Enrico
> > > > > and he has accepted. Enrico has made some great contributions and
> we
> > > > > are looking forward to even more :)
> > > > >
> > > > > Congratulations and welcome aboard, Enrico!
> > > > >
> > > >
> > >
> >
>


[jira] [Resolved] (ZOOKEEPER-1000) Provide SSL in zookeeper to be able to run cross colos.

2019-05-21 Thread Michael Han (JIRA)


 [ 
https://issues.apache.org/jira/browse/ZOOKEEPER-1000?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Michael Han resolved ZOOKEEPER-1000.

Resolution: Duplicate

> Provide SSL in zookeeper to be able to run cross colos.
> ---
>
> Key: ZOOKEEPER-1000
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1000
> Project: ZooKeeper
>  Issue Type: Improvement
>Reporter: Mahadev konar
>Assignee: Mahadev konar
>Priority: Major
> Fix For: 3.6.0, 3.5.6
>
>
> This jira is to track SSL for zookeeper. The inter zookeeper server 
> communication and the client to server communication should be over ssl so 
> that zookeeper can be deployed over WAN's. 



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (ZOOKEEPER-1000) Provide SSL in zookeeper to be able to run cross colos.

2019-05-21 Thread Michael Han (JIRA)


[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-1000?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16845426#comment-16845426
 ] 

Michael Han commented on ZOOKEEPER-1000:


Agreed, this sounds like a dup we can close for now. If one day we find the 
current plain socket based solution is not good enough feature / performance 
wise, we can revisit this issue which is based on SSL on top of Netty.

> Provide SSL in zookeeper to be able to run cross colos.
> ---
>
> Key: ZOOKEEPER-1000
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1000
> Project: ZooKeeper
>  Issue Type: Improvement
>Reporter: Mahadev konar
>Assignee: Mahadev konar
>Priority: Major
> Fix For: 3.6.0, 3.5.6
>
>
> This jira is to track SSL for zookeeper. The inter zookeeper server 
> communication and the client to server communication should be over ssl so 
> that zookeeper can be deployed over WAN's. 



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Created] (ZOOKEEPER-3399) Remove logging in getGlobalOutstandingLimit for optimal performance.

2019-05-21 Thread Michael Han (JIRA)
Michael Han created ZOOKEEPER-3399:
--

 Summary: Remove logging in getGlobalOutstandingLimit for optimal 
performance.
 Key: ZOOKEEPER-3399
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3399
 Project: ZooKeeper
  Issue Type: Bug
  Components: server
Affects Versions: 3.6.0
Reporter: Michael Han
Assignee: Michael Han


Recently we have moved some of our production clusters to the top of the trunk. 
One issue we found is a performance regression on read and write latency on the 
clusters where the quorum is also serving traffic. The average read latency 
increased by 50x, p99 read latency increased by 300x. 

The root cause is a log statement introduced in ZOOKEEPER-3177 (PR711), where 
we added a LOG.info statement in getGlobalOutstandingLimit. 
getGlobalOutstandingLimit is on the critical code path for request processing 
and for each request, it will be called twice (one at processing the packet, 
one at finalizing the request response). This not only degrades performance of 
the server, but also bloated the log file, when the QPS of a server is high.

This only impacts clusters when the quorum (leader + follower) is serving 
traffic. For clusters where only observers are serving traffic no impact is 
observed.

 

 



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


Re: [VOTE] Apache ZooKeeper release 3.5.5 candidate 6

2019-05-13 Thread Michael Han
+1.

- verified checksum/sig.
- source build, unit tests pass.
- rat tool output looks ok.
- verified cluster with different sizes with smoke tests.
- release notes, license, notice looks good.


On Sun, May 12, 2019 at 10:40 AM sandeep shrids 
wrote:

> Hi,
>
> +1
> Performed the following checks on my ubuntu box:
> - digests match
> - verified ZK server (used bin tgz) in standalone mode using the zkCli.
> - verified ZK servers(used bin tgz) in a replicated mode(3 instances) using
> zkCli. Brought down a ZK server and verified the behavior using zkCli
> - local build works fine on the source (mvn clean install)
>
> This time used Oracle jdk with java version "1.8.0_211"
>
> Regards,
> Sandeep
>
>
>
> On Fri, May 3, 2019 at 6:04 PM Andor Molnar  wrote:
>
> > This is the first stable release of 3.5 branch: 3.5.5. It resolves 117
> > issues, including Maven migration, Quorum TLS, TTL nodes and lots of
> other
> > performance and stability improvements.
> >
> > The full release notes is available at:
> >
> >
> >
> https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12310801=12343268
> >
> > *** Please download, test and vote by May 10th 2019, 23:59 UTC+0. ***
> >
> > Source files:
> > https://dist.apache.org/repos/dist/dev/zookeeper/zookeeper-3.5.5-rc6/
> >
> > Maven staging repos:
> >
> >
> https://repository.apache.org/content/groups/staging/org/apache/zookeeper/parent/3.5.5/
> >
> >
> https://repository.apache.org/content/groups/staging/org/apache/zookeeper/zookeeper-jute/3.5.5/
> >
> >
> https://repository.apache.org/content/groups/staging/org/apache/zookeeper/zookeeper/3.5.5/
> >
> > The release candidate tag in git to be voted upon: release-3.5.5-rc6
> >
> > ZooKeeper's KEYS file containing PGP keys we use to sign the release:
> > http://www.apache.org/dist/zookeeper/KEYS
> >
> > Should we release this candidate?
> >
> >
>


Re: why not disable sync API in libzkst.a?

2019-05-08 Thread Michael Han
Hi Shuxin,

What's the zk client version you are testing? Sync apis were removed from c
client in https://issues.apache.org/jira/browse/ZOOKEEPER-761, but it's
only done for 3.5.3 and master branch only. There was a discussion in JIRA
regarding whether or not to do the same for 3.4 branch which never
concluded. Feel free to reopen the JIRA and submit a patch for 3.4 branch.

On Mon, May 6, 2019 at 11:24 PM Shuxin Yang 
wrote:

> Hi,
>
>  I'm new to C API lib. I accidentally use single-thread lib (i.e.
> libzkst.a) and it took me quit a while before I realize all sync
> functions in the libzkst.a is not usable at all. I'm wondering why not
> just disable them in libztst.a. It would otherwise be pretty confusing
> and error-prone.
>
> Take zoo_get_children() as an example, it always return ZOK, and 0
> children.  This function in turn calls zoo_wget_children_() which is
> excerpted bellow. The wait_sync_completion() at line 3657 actually does
> nothing in the single-thread version, meaning this function simply
> return to the caller without waiting for the job done.
>
> Thanks
>
> Shuxin
>
>
> --
>
> 3646 static int zoo_wget_children_(...)
>
> 3650 struct sync_completion *sc = alloc_sync_completion();
>
> 
>
> 3655 rc= zoo_awget_children (zh, path, watcher, watcherCtx,
> SYNCHRONOUS_MARKER, sc);
> 3656 if(rc==ZOK){
> 3657 wait_sync_completion(sc);
> 3658 rc = sc->rc;
> 3659 if (rc == 0) {
> 3660 if (strings) {
> 3661 *strings = sc->u.strs2;
> 3662 } else {
> 3663 deallocate_String_vector(>u.strs2);
> 3664 }
> 3665 }
> 3666 }
> 3667 free_sync_completion(sc);
> 3668 return rc;
> 3669 }
>
> 
>
>
>


Re: [Suggestion] Use Co-authored-by in commit messages

2019-05-08 Thread Michael Han
>> My proposal is to use github's feature of Co-author
+1. The commit script would have to be updated to incorporate this feature.

>> if someone participate in the review of PR, no matter whether he/she is
a committer, we all need include his/her name
We already do this when commit a change so reviewers get credits as well
and we can keep it this way unless there is a better approach.


On Wed, May 8, 2019 at 11:23 AM Enrico Olivelli  wrote:

> Yes, it is a good idea to have a common practice for tracking the original
> author.
>
> IMHO this is up to the person who is picking up an old patch, it is his own
> responsibility.
>
> IIRC In Bookkeeper we keep the original author of the patch if the patch is
> a straight port from another private company fork with minimal changes.
>
> Having a Co-author is good from my side. I am not sure we can force it
>
> My 2cents
>
> Enrico
>
> Il mer 8 mag 2019, 19:31 Brian Nixon  ha
> scritto:
>
> > +1 to the idea of multiple authors, particularly for rescued code
> >
> > -1 to including all reviewers in the commit proper, this information is
> > easily enough found from poking at the mail archive where "original
> author"
> > requires studying a ticket on jira
> >
> > awesome idea!
> >
> >
> > On Wed, May 8, 2019 at 6:32 AM Norbert Kalmar
>  > >
> > wrote:
> >
> > > Sorry everyone for the multiple emails...
> > > So, I get your suggestion now Maoling, sorry for the confusion.
> > > We already indicate the reviewer if it's from an apache email, as it
> > looks
> > > to me. (Doesn't have to be ZooKeeper committer). We should add external
> > > emails as well.
> > >
> > > So I just clarified this with Andor, looks like this is a manual entry
> > (the
> > > names/emails itself) during the commit (script).
> > >
> > > Let's hear what others think :)
> > >
> > >
> > > On Wed, May 8, 2019 at 3:24 PM Norbert Kalmar 
> > > wrote:
> > >
> > > > Well, HBase does it for example, commits have a "Signed-off-by: ..."
> > > line.
> > > >
> > > > All right, votes on for co-author and signed-off-by :)
> > > >
> > > >
> > > >
> > > > On Wed, May 8, 2019 at 2:58 PM Norbert Kalmar 
> > > > wrote:
> > > >
> > > >> Thanks Maoling, I also think encouraging code review as well is a
> good
> > > >> idea, but, unfortunately I have a "but" :)
> > > >> I see two issues with including reviewers in the commit message.
> > > >> First, I don't think there is a method to automate this, although I
> > > think
> > > >> the commit script the committers are using can be modified to
> include
> > > it.
> > > >> Otherwise doing manually would complicate merging PRs for
> committers.
> > > >> My other, bigger issue is that there is nothing to track this
> > > >> information. At least I am not aware of anything. What I mean is
> > Github
> > > >> tracks authors of the commits. But what would we use the reviewers
> > > >> information? If you just want to check reviewers for whatever
> reason,
> > > there
> > > >> is a filter for that already on github, in the Pull Request view.
> And
> > > this
> > > >> would also make the commit message more "bloated".
> > > >>
> > > >> I'm not saying we shouldn't do this (not a -1 from my side), I just
> > have
> > > >> my concerns mentioned above.
> > > >>
> > > >> Is there any Apache project doing this? Just out of curiosity.
> > > >>
> > > >> Regards,
> > > >> Norbert
> > > >>
> > > >>
> > > >> On Wed, May 8, 2019 at 2:34 PM Justin Ling Mao <
> > > maoling199210...@sina.com>
> > > >> wrote:
> > > >>
> > > >>> +1,A very good Suggestion.Thanks Norbert.I also suggest about the
> > > >>> sign-off of the Reviewers' name.For the incentive, if someone
> > > participate
> > > >>> in the review of PR, no matter whether he/she is a committer, we
> all
> > > need
> > > >>> include his/her name?
> > > >>>
> > > >>> - Original Message -
> > > >>> From: Norbert Kalmar 
> > > >>> To: dev@zookeeper.apache.org
> > > >>> Subject: [Suggestion] Use Co-authored-by in commit messages
> > > >>> Date: 2019-05-08 17:36
> > > >>>
> > > >>> Hi Devs,
> > > >>> I've got this idea from HBase.
> > > >>> So: when there is a patch that is abandoned by its original author
> > for
> > > >>> any
> > > >>> reason, and it can no longer be merged, someone comes by, and asks
> to
> > > >>> continue to work on it. Usually the reply is to use the change
> freely
> > > or
> > > >>> no
> > > >>> reply at all. Either way, what people end up doing is a new pull
> > > request,
> > > >>> and (correct me if I'm wrong) we do not have a standardized method
> > how
> > > to
> > > >>> indicate, or even to indicate at all the original author.
> > > >>> My proposal is to use github's feature of Co-author, which is a way
> > of
> > > >>> attributing multiple authors of a given commit. See more details
> > here:
> > > >>>
> > > >>>
> > >
> >
> https://help.github.com/en/articles/creating-a-commit-with-multiple-authors
> > > >>> I wouldn't think this needs to be forced or anything on future PRs,
> > but
> > > >>> it's a 

Re: [ANNOUNCE] New ZooKeeper PMC member: Andor Molnar

2019-05-08 Thread Michael Han
Congratulations, Andor!

On Mon, Apr 29, 2019 at 9:10 AM Brian Nixon 
wrote:

> Awesome! Congrats, Andor, you've been doing great work!
>
>
> On Mon, Apr 29, 2019 at 7:31 AM Patrick Hunt  wrote:
>
> > Kudos Andor!
> >
> > Patrick
> >
> > On Mon, Apr 29, 2019 at 2:07 AM Enrico Olivelli 
> > wrote:
> >
> > > Congratulations!
> > >
> > > Enrico
> > >
> > > Il lun 29 apr 2019, 10:40 Norbert Kalmar  >
> > ha
> > > scritto:
> > >
> > > > Congratulations Andor, well deserved!
> > > >
> > > > On Mon, Apr 29, 2019 at 10:14 AM Flavio Junqueira 
> > > wrote:
> > > >
> > > > > The Apache ZooKeeper PMC recently invited Andor to join the PMC and
> > he
> > > > has
> > > > > accepted. Andor has made some significant contributions to the
> > project,
> > > > > including driving releases.  We are looking forward to even greater
> > > > > contributions from Andor now as part of the PMC.
> > > > >
> > > > > Congratulations and welcome aboard Andor!
> > > > >
> > > > > -Flavio on behalf of the Apache ZooKeeper PMC
> > > >
> > >
> >
>


[jira] [Commented] (ZOOKEEPER-3352) Use LevelDB For Backend

2019-04-09 Thread Michael Han (JIRA)


[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-3352?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16814023#comment-16814023
 ] 

Michael Han commented on ZOOKEEPER-3352:


I don't see an obvious gain of using a LSM tree backend just for snapshot and 
txn log. For zk clients, the read path will be served directly from memory 
(think zk data tree as a 'memtable' that never flushes); the write path is 
already sequential for both snapshot and txn log. Reading snapshot and txn log 
out of LSM tree instead of flat files might reduce recovery time, but I doubt 
the difference is substantial.

That said, having a LSM tree backend and build the zk data tree on top of it 
will make it possible to store a much larger scale of data set per single node, 
as we will not store all data in memory only.

> Use LevelDB For Backend
> ---
>
> Key: ZOOKEEPER-3352
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3352
> Project: ZooKeeper
>  Issue Type: Improvement
>  Components: server
>Reporter: David Mollitor
>Assignee: David Mollitor
>Priority: Major
> Fix For: 4.0.0
>
>
> Use LevelDB for managing data stored in ZK (transaction logs and snapshots).
> https://stackoverflow.com/questions/6779669/does-leveldb-support-java



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


Re: Zookeeper java client - a first step

2019-01-13 Thread Michael Han
+1. Good time to revisit https://issues.apache.org/jira/browse/ZOOKEEPER-233

On Sun, Jan 13, 2019 at 8:54 AM Enrico Olivelli  wrote:

> Hi guys,
> We tried to separate java client  from zookeeper-server package but
> currently it is not possible  without breaking changes.
>
> I think we can still start providing a simple zookeeper java client just by
> creating a dummy zookeeper-client module as a Maven 'bom' (bills of
> material)
>
> It will be a simple pom which adds the dependency to zookeeper-server jar
> but it 'excludes' all third party dependencies not needed for running the
> client.
>
> I see these advantages :
> 1) applications will start to depend on something called 'client ', which
> is more user friendly
> 2) clients won't  import unused dependecies, this is actually  a nuisance
>  for downstream application
> 3) it will be easier to change dependencies  in the future (at least for
> the server)
>
> Regards
> Enrico
> --
>
>
> -- Enrico Olivelli
>


[jira] [Commented] (ZOOKEEPER-3240) Close socket on Learner shutdown to avoid dangling socket

2019-01-13 Thread Michael Han (JIRA)


[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-3240?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16741714#comment-16741714
 ] 

Michael Han commented on ZOOKEEPER-3240:


[~nixon] :
bq.  so the Leader is unable to sense the change in Learner status through the 
status of the network connection

A plausible theory :)

The ping packet between leader and learners is designed to solve this exact 
problem - to be able to detect liveness of the other side. Basically for each 
learner, leader will constantly read packets out of the socket associated with 
the learner in the corresponding LearnerHandler thread. And this read has a 
timeout configured on the socket on leader side, so even if the sockets on both 
side are valid, but there is no traffic (such as in this case, where learner 
leaks sockets by not properly closing them after shutting down), leader's read 
should eventually time out after sync limit check. Unless:

* Leader's socket read time out has no effect. So leader will block on reading 
a socket indefinitely because there is no traffic from learner.
* Learner process, after restarted, somehow ended up with reusing the old 
Learner socket that's leaked so the corresponding LearnerHandler thread can't 
detect any difference (which is expected.). I am not sure how possible this 
case is in practice.

In any case, it seems that our Ping mechanism failed to detect the network 
change in this case.

bq. the learner queue size keeps growing

Do you mind elaborate a little bit on which exact queue this is and what caused 
it growing?





> Close socket on Learner shutdown to avoid dangling socket
> -
>
> Key: ZOOKEEPER-3240
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3240
> Project: ZooKeeper
>  Issue Type: Improvement
>  Components: server
>Affects Versions: 3.6.0
>Reporter: Brian Nixon
>Priority: Minor
>  Labels: pull-request-available
>  Time Spent: 50m
>  Remaining Estimate: 0h
>
> There was a Learner that had two connections to the Leader after that Learner 
> hit an unexpected exception during flush txn to disk, which will shutdown 
> previous follower instance and restart a new one.
>  
> {quote}2018-10-26 02:31:35,568 ERROR 
> [SyncThread:3:ZooKeeperCriticalThread@48] - Severe unrecoverable error, from 
> thread : SyncThread:3
> java.io.IOException: Input/output error
>     at java.base/sun.nio.ch.FileDispatcherImpl.force0(Native Method)
>     at 
> java.base/sun.nio.ch.FileDispatcherImpl.force(FileDispatcherImpl.java:72)
>     at 
> java.base/sun.nio.ch.FileChannelImpl.force(FileChannelImpl.java:395)
>     at 
> org.apache.zookeeper.server.persistence.FileTxnLog.commit(FileTxnLog.java:457)
>     at 
> org.apache.zookeeper.server.persistence.FileTxnSnapLog.commit(FileTxnSnapLog.java:548)
>     at org.apache.zookeeper.server.ZKDatabase.commit(ZKDatabase.java:769)
>     at 
> org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:246)
>     at 
> org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:172)
> 2018-10-26 02:31:35,568 INFO  [SyncThread:3:ZooKeeperServerListenerImpl@42] - 
> Thread SyncThread:3 exits, error code 1
> 2018-10-26 02:31:35,568 INFO [SyncThread:3:SyncRequestProcessor@234] - 
> SyncRequestProcessor exited!{quote}
>  
> It is supposed to close the previous socket, but it doesn't seem to be done 
> anywhere in the code. This leaves the socket open with no one reading from 
> it, and caused the queue full and blocked on sender.
>  
> Since the LearnerHandler didn't shutdown gracefully, the learner queue size 
> keeps growing, the JVM heap size on leader keeps growing and added pressure 
> to the GC, and cause high GC time and latency in the quorum.
>  
> The simple fix is to gracefully shutdown the socket.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (ZOOKEEPER-3240) Close socket on Learner shutdown to avoid dangling socket

2019-01-11 Thread Michael Han (JIRA)


[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-3240?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16740161#comment-16740161
 ] 

Michael Han commented on ZOOKEEPER-3240:


[~nixon] Good catch, the fix looks reasonable. 

I've seen similar issue in my production environment, the fix I did was on 
Leader side where I tracked the LearnerHandler threads associated with server 
ids, and make sure each server id only has a single LearnerHandler thread. This 
also work in cases where the learners don't have a chance to close their 
sockets, or they did but due to some reasons the TCP reset never made it to 
leader. But in any case, it's good to fix the resource leaking on learner side.

I also wonder why we could get into such case on Leader side in first place. On 
leader, we do have socket read timeout set via setSoTimeout for leaner handler 
threads (after the socket was created via serverSocket.accept), and each 
learner handler would constantly polling / trying read from the socket 
afterwards. If, on a learner it dies but left a valid socket open, I was 
expecting one leader side the LearnerHandler thread that trying to read from 
that died learner socket will eventually timeout, which, will throw 
SocketTimeOutException and cause the LearnerHandler thread on the leader kill 
itself. This though does not seem to be the case I observed. Do you have any 
insights on this?

> Close socket on Learner shutdown to avoid dangling socket
> -
>
> Key: ZOOKEEPER-3240
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3240
> Project: ZooKeeper
>  Issue Type: Improvement
>  Components: server
>Affects Versions: 3.6.0
>Reporter: Brian Nixon
>Priority: Minor
>  Labels: pull-request-available
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> There was a Learner that had two connections to the Leader after that Learner 
> hit an unexpected exception during flush txn to disk, which will shutdown 
> previous follower instance and restart a new one.
>  
> {quote}2018-10-26 02:31:35,568 ERROR 
> [SyncThread:3:ZooKeeperCriticalThread@48] - Severe unrecoverable error, from 
> thread : SyncThread:3
> java.io.IOException: Input/output error
>     at java.base/sun.nio.ch.FileDispatcherImpl.force0(Native Method)
>     at 
> java.base/sun.nio.ch.FileDispatcherImpl.force(FileDispatcherImpl.java:72)
>     at 
> java.base/sun.nio.ch.FileChannelImpl.force(FileChannelImpl.java:395)
>     at 
> org.apache.zookeeper.server.persistence.FileTxnLog.commit(FileTxnLog.java:457)
>     at 
> org.apache.zookeeper.server.persistence.FileTxnSnapLog.commit(FileTxnSnapLog.java:548)
>     at org.apache.zookeeper.server.ZKDatabase.commit(ZKDatabase.java:769)
>     at 
> org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:246)
>     at 
> org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:172)
> 2018-10-26 02:31:35,568 INFO  [SyncThread:3:ZooKeeperServerListenerImpl@42] - 
> Thread SyncThread:3 exits, error code 1
> 2018-10-26 02:31:35,568 INFO [SyncThread:3:SyncRequestProcessor@234] - 
> SyncRequestProcessor exited!{quote}
>  
> It is supposed to close the previous socket, but it doesn't seem to be done 
> anywhere in the code. This leaves the socket open with no one reading from 
> it, and caused the queue full and blocked on sender.
>  
> Since the LearnerHandler didn't shutdown gracefully, the learner queue size 
> keeps growing, the JVM heap size on leader keeps growing and added pressure 
> to the GC, and cause high GC time and latency in the quorum.
>  
> The simple fix is to gracefully shutdown the socket.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


Re: Question about the leader-based atomic broadcast

2019-01-09 Thread Michael Han
First of all, these questions are more related to implementation details
rather than the Zab protocol itself. My answer thus is a reflection of the
current ZK implementation. Zab itself could be implemented differently
though.

>> When will the leader delete the message from its FIFO queue?

A message is materialized as a Proposal in ZK's implementation. A leader
processes a (write) proposal in a similar way like a two phased commit:
first phase is sync phase where leader persists the proposal to local non
volatile storage, and meanwhile leader will ask followers to do the same. A
follower, upon receiving the request from leader, will send acknowledgement
to leader after the proposal persistent finishes. Leader will wait until it
receives acknowledgments from enough number of followers (such that the
number of followers plus leader itself forms a quorum - typically a
majority quorum which is ZK's default quorum implementation), then proceeds
to next phase, which is commit phase. In commit phase, leader will apply
the proposal to the in memory database, and asks followers to do the same.

The proposal is deleted from leader's queue (in implementation, it's called
outstandingProposals) during the commit phase. In other words, after sync
phase, it's safe to delete the proposal, because it's already persist on a
quorum of servers. And because how majority quorum works, it's safe to
recover in crash cases.

>> I assume the leader won't delete the message until the message is
delivered to all servers?

As previously stated, it's safe to delete as long as the message is persist
on a set of followers such that leader + these servers form a quorum.

>> If so, will the leader's FIFO queue keep increasing when there exists a
slow follower

The size of leader's proposal queue depends on both the throughput of the
system, and the workload. If the throughput is low due to one or more slow
followers, but the workload is also light, it's still possible to clear the
proposal queue before the next client comes in.

But in general, yes, under high workload if one or more followers are slow
(but not too slow so they still in quorum instead of getting restarted
because of failed health check), the leader's proposal queue will keep
grow. There is no way to work around this as long as we use majority quorum
and want to maintain the safety property of the system. There are recent
researches such as Flexible Paxos which weakened the quorum intersection
requirement, which might improve the performance in such case, but ZK does
not implement such optimization.

Hope these help.


On Wed, Jan 9, 2019 at 2:03 PM Ted Dunning  wrote:

> All updates to data in Zookeeper are modified to be idempotent before they
> are accepted into the leader's queue. That means that items in the queue
> can be committed in groups and once each group is acknowledged by a quorum
> of servers, it can be deleted from the queue. Any server not in the
> acknowledging quorum can get up to date by duplicating the contents of the
> leader. As with snapshots, this doesn't require updates to be stopped. The
> idempotency of all updates means that the server can duplicate the leader
> even as updates are being applied and then can apply all updates that
> occurred after the start of the concurrence process. If some of the leader
> state was copied before it was affected by an update after the start of
> synchronization, it will be updated in the follower's image when replaying
> very recent events. If it was copied after being updated, then replaying
> recent events will merely write the same value.
>
> The effect of all of this is that there is no penalty for deleting
> transactions immediately after they are committed and acknowledged.
>
>
>
> On Wed, Jan 9, 2019 at 10:55 AM Meng Xu  wrote:
>
> > Hi,
> >
> > I have a question about the leader-based atomic broadcast used in
> > ZooKeeper.
> >
> > According to the Zab paper "A simple totally ordered broadcast
> > protocol"[1], the protocol has the requirement of reliable delivery:
> > If a message is delivered to one server, it will be eventually
> > delivered by all correct servers.
> >
> > ***My question is:
> > When will the leader delete the message from its FIFO queue?
> >
> > I assume the leader won't delete the message until the message is
> > delivered to all servers?
> >
> > If so, will the leader's FIFO queue keep increasing when there exists
> > a slow follower (or slow connection), where it takes a long time for
> > the slow follower to receive a message?
> >
> > Thank you very much!
> >
> > [1]
> >
> https://www.datadoghq.com/pdf/zab.totally-ordered-broadcast-protocol.2008.pdf
> >
> > Best,
> >
> > Meng
> >
>


[jira] [Commented] (ZOOKEEPER-3180) Add response cache to improve the throughput of read heavy traffic

2018-12-12 Thread Michael Han (JIRA)


[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-3180?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16719752#comment-16719752
 ] 

Michael Han commented on ZOOKEEPER-3180:


My experience with JVM GC and ZooKeeper is GC is rarely a real issue in 
production if tuned correctly (I ran fairly large ZK fleet which kind push ZK 
to its limit). Most GC issue I had is software bugs - such as leaking 
connections. For this cache case, the current implementation is good enough for 
my use case, though I do have interests on off heap solutions as well. My 
concern around off heap solution is it's probably going to be more complicated, 
and has overhead of serialization / deserialization between heap / off heap. 
I'd say we get this patch landed, have more people tested it out, then improve 
it with more options.

 

And for caching in general, obviously it depends a lot on workload and actual 
use case, so it's kind hard to provide a cache solution that works for everyone 
in first place...

> Add response cache to improve the throughput of read heavy traffic 
> ---
>
> Key: ZOOKEEPER-3180
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3180
> Project: ZooKeeper
>  Issue Type: Improvement
>  Components: server
>Reporter: Fangmin Lv
>Assignee: Brian Nixon
>Priority: Minor
>  Labels: pull-request-available
> Fix For: 3.6.0
>
>  Time Spent: 3h 50m
>  Remaining Estimate: 0h
>
> On read heavy use case with large response data size, the serialization of 
> response takes time and added overhead to the GC.
> Add response cache helps improving the throughput we can support, which also 
> reduces the latency in general.
> This Jira is going to implement a LRU cache for the response, which shows 
> some performance gain on some of our production ensembles.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


Re: Migrating ZooKeeper to GitBox

2018-12-12 Thread Michael Han
+1 on voluntarily request to move.

On Wed, Dec 12, 2018 at 11:36 AM Patrick Hunt  wrote:

> Probably minor, but note there are some website pages that will also need
> to be updated:
> e.g. https://zookeeper.apache.org/git.html
>
> Patrick
>
> On Wed, Dec 12, 2018 at 10:58 AM Enrico Olivelli 
> wrote:
>
> > +1  for moving to github now
> >
> > Enrico
> >
> > Il mer 12 dic 2018, 19:49 Tamas Penzes  ha
> > scritto:
> >
> > > Hi Patrick,
> > >
> > > Yes we have to upgrade the main pom.xml, just as the website and the
> > > documentation.
> > > Do we have to change git urls in old documentations as well? (People
> > might
> > > read them.)
> > >
> > > Thanks, Tamaas
> > >
> > > On Wed, Dec 12, 2018, 18:31 Patrick Hunt  > >
> > > > Proactive sounds good to me. Can we schedule something outside the
> > > > holidays?
> > > >
> > > > Aside from Jenkins is there anything else we need to worry about?
> e.g.
> > I
> > > > see a reference in pom.xml
> > > >
> > > > pom.xml
> > > > 63:scm:git:
> > > > https://git-wip-us.apache.org/repos/asf/zookeeper.git
> > > > 
> > > > 64:https://git-wip-us.apache.org/repos/asf?p=zookeeper.git
> > > 
> > > >
> > > > What's the impact wrt github.com and the Apache org? Will this
> enable
> > > use
> > > > to become admins at the same time? (greater control over the
> > issues/prs,
> > > > etc...)
> > > >
> > > > Patrick
> > > >
> > > > On Wed, Dec 12, 2018 at 9:07 AM Andor Molnar 
> wrote:
> > > >
> > > > > Hi dev team,
> > > > >
> > > > > Have you seen this?
> > > > >
> > > > >
> > > >
> > >
> >
> https://blogs.apache.org/infra/entry/relocation-of-apache-git-repositories
> > > > > <
> > > > >
> > > >
> > >
> >
> https://blogs.apache.org/infra/entry/relocation-of-apache-git-repositories
> > > > > >
> > > > >
> > > > > I think we should volunteer for a coordinated migration of
> ZooKeeper,
> > > > > although the project is not too big, so the impact of a forced
> > process
> > > > > probably wouldn’t be so much.
> > > > >
> > > > > There’re some benefits though:
> > > > > - We could move to GitBox earlier rather than later, so we could
> take
> > > > > advantage of GitHub write access for example,
> > > > > - Migration needs a write-lock in place during the process, so we
> can
> > > > > avoid some unfortunate surprises,
> > > > > - git-wip-us.apache.org  will have
> > > CNAMEs
> > > > > to gitbox, so it’ll be operational for some time, but the read-only
> > > > > git.apache.org  will go away (not sure
> about
> > > the
> > > > > timing though), that we use in some of our Jenkins builds. It would
> > be
> > > > > better to migrate Jenkins configs in a coordinated fashion rather
> > than
> > > > when
> > > > > the house is burning.
> > > > >
> > > > > There’re a bunch of unkowns in each project, hence Infra guys
> > recommend
> > > > > the coordinated way, but we need an agreement in the community, so
> > > please
> > > > > share your thoughts.
> > > > >
> > > > > Regards,
> > > > > Andor
> > > > >
> > > > >
> > > > >
> > > >
> > >
> > --
> >
> >
> > -- Enrico Olivelli
> >
>


[jira] [Commented] (ZOOKEEPER-3211) zookeeper standalone mode,found a high level bug in kernel of centos7.0 ,zookeeper Server's tcp/ip socket connections(default 60 ) are CLOSE_WAIT ,this lead to zk

2018-12-12 Thread Michael Han (JIRA)


[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-3211?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16719736#comment-16719736
 ] 

Michael Han commented on ZOOKEEPER-3211:


{quote}Have similar defects been solved in 3.4.13?
{quote}
Previously there were reports about CLOSE_WAIT, but if I remember correctly, 
most of those cases ended up no actions taken because it was hard to reproduce. 
{quote}It looks like zk Server is deadlocked
{quote}
The thread dump in 1.log file indicates some threads are blocked, but that 
seems a symptom rather than the cause. If we run out available sockets then 
some zookeeper threads that involves file IO / socket IO will be blocked. 

 
{quote}Does this cause CLOSE_WAIT for zk?
{quote}
Most of time, long living CLOSE_WAIT connections indicate an application side 
bug instead of kernel bug - that the connection should be closed but for some 
reasons the application, after receiving TCP reset from clients can't close the 
connection - which effectively leaks connections. The upgrade of kernel could 
be a trigger though. 

 

I am interested to know if any other folks can reproduce this. I currently 
don't have the environment to reproduce this.

 

Also, [~yss] can you please use zip file instead of rar file for uploading log 
files? 

Another thing to try is to increase your limit of open file descriptors - seems 
its currently set as 60? If you increase it (ulimit), you could end up still 
leaking connections but the server should be available before its running out 
of sockets.

> zookeeper standalone mode,found a high level bug in kernel of centos7.0 
> ,zookeeper Server's  tcp/ip socket connections(default 60 ) are CLOSE_WAIT 
> ,this lead to zk can't work for client any more
> --
>
> Key: ZOOKEEPER-3211
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3211
> Project: ZooKeeper
>  Issue Type: Bug
>  Components: server
>Affects Versions: 3.4.5
> Environment: 1.zoo.cfg
> server.1=127.0.0.1:2902:2903
> 2.kernel
> kernel:Linux localhost.localdomain 3.10.0-123.el7.x86_64 #1 SMP Tue Feb 12 
> 19:44:50 EST 2019 x86_64 x86_64 x86_64 GNU/Linux
> JDK:
> java version "1.7.0_181"
> OpenJDK Runtime Environment (rhel-2.6.14.5.el7-x86_64 u181-b00)
> OpenJDK 64-Bit Server VM (build 24.181-b00, mixed mode)
> zk: 3.4.5
>Reporter: yeshuangshuang
>Priority: Blocker
> Fix For: 3.4.5
>
> Attachments: 1.log, zklog.rar
>
>   Original Estimate: 168h
>  Remaining Estimate: 168h
>
> 1.config--zoo.cfg
> server.1=127.0.0.1:2902:2903
> 2.kernel version
> version:Linux localhost.localdomain 3.10.0-123.el7.x86_64 #1 SMP Tue Feb 12 
> 19:44:50 EST 2019 x86_64 x86_64 x86_64 GNU/Linux
> JDK:
> java version "1.7.0_181"
> OpenJDK Runtime Environment (rhel-2.6.14.5.el7-x86_64 u181-b00)
> OpenJDK 64-Bit Server VM (build 24.181-b00, mixed mode)
> zk: 3.4.5
> 3.bug details:
> Occasionally,But the recurrence probability is extremely high. At first, the 
> read-write timeout takes about 6s, and after a few minutes, all connections 
> (including long ones) will be CLOSE_WAIT state.
> 4.:Circumvention scheme: it is found that all connections become close_wait 
> to restart the zookeeper server side actively



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


Re: Leader election

2018-12-12 Thread Michael Han
>> Can we reduce this time by configuring "syncLimit" and "tickTime" to
let's say 5 seconds? Can we have a strong
guarantee on this time bound?

It's not possible to guarantee the time bound, because of FLP impossibility
(reliable failure detection is not possible in async environment). Though
it's certainly possible to tune the parameters to some reasonable value
that fits your environment (which would be the SLA of your service).

>> As describe above - you might use 'sync'+'read' to avoid this problem.

I am afraid sync + read would not be correct 100% in all cases here. The
state of the world (e.g. leaders) could change between sync and read
operation. What we need here is linearizable read, which means we need have
read operations also go through the quorum consensus, which might be a nice
feature to have for ZooKeeper (for reference, etcd implements linearizable
read). Also, note ZooKeeper sync has bugs (sync should be a quorum
operation itself, but it's not implemented that way).

On Fri, Dec 7, 2018 at 2:11 AM Maciej Smoleński  wrote:

> On Fri, Dec 7, 2018 at 3:03 AM Michael Borokhovich 
> wrote:
>
> > We are planning to run Zookeeper nodes embedded with the client nodes.
> > I.e., each client runs also a ZK node. So, network partition will
> > disconnect a ZK node and not only the client.
> > My concern is about the following statement from the ZK documentation:
> >
> > "Timeliness: The clients view of the system is guaranteed to be
> up-to-date
> > within a certain time bound. (*On the order of tens of seconds.*) Either
> > system changes will be seen by a client within this bound, or the client
> > will detect a service outage."
> >
>
> This is related to the fact that ZooKeeper server handles reads from its
> local state - without communicating with other ZooKeeper servers.
> This design ensures scalability for read dominated workloads.
> In this approach client might receive data which is not up to date (it
> might not contain updates from other ZooKeeper servers (quorum)).
> Parameter 'syncLimit' describes how often ZooKeeper server
> synchronizes/updates its local state to global state.
> Client read operation will retrieve data from state not older then
> described by 'syncLimit'.
>
> However ZooKeeper client can always force to retrieve data which is up to
> date.
> It needs to issue 'sync' command to ZooKeeper server before issueing
> 'read'.
> With 'sync' ZooKeeper server with synchronize its local state with global
> state.
> Later 'read' will be handled from updated state.
> Client should be careful here - so that it communicates with the same
> ZooKeeper server for both 'sync' and 'read'.
>
>
> > What are these "*tens of seconds*"? Can we reduce this time by
> configuring
> > "syncLimit" and "tickTime" to let's say 5 seconds? Can we have a strong
> > guarantee on this time bound?
> >
>
> As describe above - you might use 'sync'+'read' to avoid this problem.
>
>
> >
> >
> > On Thu, Dec 6, 2018 at 1:05 PM Jordan Zimmerman <
> > jor...@jordanzimmerman.com>
> > wrote:
> >
> > > > Old service leader will detect network partition max 15 seconds after
> > it
> > > > happened.
> > >
> > > If the old service leader is in a very long GC it will not detect the
> > > partition. In the face of VM pauses, etc. it's not possible to avoid 2
> > > leaders for a short period of time.
> > >
> > > -JZ
> >
>


[jira] [Commented] (ZOOKEEPER-3211) zookeeper standalone mode,found a high level bug in kernel of centos7.0 ,zookeeper Server's tcp/ip socket connections(default 60 ) are CLOSE_WAIT ,this lead to zk

2018-12-12 Thread Michael Han (JIRA)


[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-3211?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16719680#comment-16719680
 ] 

Michael Han commented on ZOOKEEPER-3211:


[~yss] Have you tried newer version of stable zookeeper release (e.g. 3.4.13), 
as well as different versions of OS? 3.4.5 is a pretty old version. 

> zookeeper standalone mode,found a high level bug in kernel of centos7.0 
> ,zookeeper Server's  tcp/ip socket connections(default 60 ) are CLOSE_WAIT 
> ,this lead to zk can't work for client any more
> --
>
> Key: ZOOKEEPER-3211
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3211
> Project: ZooKeeper
>  Issue Type: Bug
>  Components: server
>Affects Versions: 3.4.5
> Environment: 1.zoo.cfg
> server.1=127.0.0.1:2902:2903
> 2.kernel
> kernel:Linux localhost.localdomain 3.10.0-123.el7.x86_64 #1 SMP Tue Feb 12 
> 19:44:50 EST 2019 x86_64 x86_64 x86_64 GNU/Linux
> JDK:
> java version "1.7.0_181"
> OpenJDK Runtime Environment (rhel-2.6.14.5.el7-x86_64 u181-b00)
> OpenJDK 64-Bit Server VM (build 24.181-b00, mixed mode)
> zk: 3.4.5
>Reporter: yeshuangshuang
>Priority: Blocker
> Fix For: 3.4.5
>
> Attachments: 1.log, zklog.rar
>
>   Original Estimate: 168h
>  Remaining Estimate: 168h
>
> 1.config--zoo.cfg
> server.1=127.0.0.1:2902:2903
> 2.kernel version
> version:Linux localhost.localdomain 3.10.0-123.el7.x86_64 #1 SMP Tue Feb 12 
> 19:44:50 EST 2019 x86_64 x86_64 x86_64 GNU/Linux
> JDK:
> java version "1.7.0_181"
> OpenJDK Runtime Environment (rhel-2.6.14.5.el7-x86_64 u181-b00)
> OpenJDK 64-Bit Server VM (build 24.181-b00, mixed mode)
> zk: 3.4.5
> 3.bug details:
> Occasionally,But the recurrence probability is extremely high. At first, the 
> read-write timeout takes about 6s, and after a few minutes, all connections 
> (including long ones) will be CLOSE_WAIT state.
> 4.:Circumvention scheme: it is found that all connections become close_wait 
> to restart the zookeeper server side actively



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (ZOOKEEPER-3214) Flaky test: org.apache.zookeeper.server.quorum.QuorumPeerMainTest.testLeaderElectionWithDisloyalVoter

2018-12-12 Thread Michael Han (JIRA)


[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-3214?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16719671#comment-16719671
 ] 

Michael Han commented on ZOOKEEPER-3214:


Thanks for reporting the flaky test. It's important to keep eyes on flaky tests 
which is an important signal on quality.

 

For this specific issue, it was reported before, so I am resolving this Jira 
and move discussions in the original Jira.

> Flaky test: 
> org.apache.zookeeper.server.quorum.QuorumPeerMainTest.testLeaderElectionWithDisloyalVoter
> -
>
> Key: ZOOKEEPER-3214
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3214
> Project: ZooKeeper
>  Issue Type: Bug
>  Components: tests
>Reporter: maoling
>Priority: Minor
>
> more details in:
> https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/2901/testReport/junit/org.apache.zookeeper.server.quorum/QuorumPeerMainTest/testLeaderElectionWithDisloyalVoter/



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Reopened] (ZOOKEEPER-3141) testLeaderElectionWithDisloyalVoter is flaky

2018-12-12 Thread Michael Han (JIRA)


 [ 
https://issues.apache.org/jira/browse/ZOOKEEPER-3141?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Michael Han reopened ZOOKEEPER-3141:


Reopen this issue because this test was observed similar symptom recently as 
reported  in ZOOKEEPER-3124 
[https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/2901/testReport/junit/org.apache.zookeeper.server.quorum/QuorumPeerMainTest/testLeaderElectionWithDisloyalVoter/].
 

> testLeaderElectionWithDisloyalVoter is flaky
> 
>
> Key: ZOOKEEPER-3141
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3141
> Project: ZooKeeper
>  Issue Type: Sub-task
>  Components: leaderElection, server, tests
>Affects Versions: 3.6.0
>    Reporter: Michael Han
>Priority: Major
>
> The unit test added in ZOOKEEPER-3109 turns out to be quite flaky.
> See 
> [https://builds.apache.org/job/zOOkeeper-Find-Flaky-Tests/511/artifact/report.html]
> Recent failure builds:
> [https://builds.apache.org/job/ZooKeeper-trunk//181] 
> [https://builds.apache.org/job/ZooKeeper-trunk//179] 
> [https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/2123/testReport/junit/org.apache.zookeeper.server.quorum/QuorumPeerMainTest/testLeaderElectionWithDisloyalVoter_stillHasMajority/]
>  
>  
> Snapshot of the failure:
> {code:java}
> org.apache.zookeeper.server.quorum.QuorumPeerMainTest.testLeaderElectionWithDisloyalVoter_stillHasMajority
> Error Message
> Server 0 should have joined quorum by now
> Stacktrace
> junit.framework.AssertionFailedError: Server 0 should have joined quorum by 
> now
>   at 
> org.apache.zookeeper.server.quorum.QuorumPeerMainTest.testLeaderElection(QuorumPeerMainTest.java:1482)
>   at 
> org.apache.zookeeper.server.quorum.QuorumPeerMainTest.testLeaderElectionWithDisloyalVoter_stillHasMajority(QuorumPeerMainTest.java:1431)
>   at 
> org.apache.zookeeper.JUnit4ZKTestRunner$LoggedInvokeMethod.evaluate(JUnit4ZKTestRunner.java:79)
> {code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Resolved] (ZOOKEEPER-3214) Flaky test: org.apache.zookeeper.server.quorum.QuorumPeerMainTest.testLeaderElectionWithDisloyalVoter

2018-12-12 Thread Michael Han (JIRA)


 [ 
https://issues.apache.org/jira/browse/ZOOKEEPER-3214?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Michael Han resolved ZOOKEEPER-3214.

Resolution: Duplicate

> Flaky test: 
> org.apache.zookeeper.server.quorum.QuorumPeerMainTest.testLeaderElectionWithDisloyalVoter
> -
>
> Key: ZOOKEEPER-3214
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3214
> Project: ZooKeeper
>  Issue Type: Bug
>  Components: tests
>Reporter: maoling
>Priority: Minor
>
> more details in:
> https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/2901/testReport/junit/org.apache.zookeeper.server.quorum/QuorumPeerMainTest/testLeaderElectionWithDisloyalVoter/



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


Re: Leader election

2018-12-06 Thread Michael Han
Tweak timeout is tempting as your solution might work most of the time yet
fail in certain cases (which others have pointed out). If the goal is
absolute correctness then we should avoid timeout, which does not guarantee
correctness as it only makes the problem hard to manifest. Fencing is the
right solution here - the zxid and also znode cversion can be used as
fencing token if you use ZooKeeper. Fencing will guarantee at any single
point in time you will have one active leader in action (it does not
guarantee that at a single point of time there are multiple parties *think*
they are the leader). An alternative solution, depends on your use case, is
to instead of requiring a single active leader in action at any time, make
your workload idempotent so multiple active leaders don't do any damage.

On Thu, Dec 6, 2018 at 1:05 PM Jordan Zimmerman 
wrote:

> > Old service leader will detect network partition max 15 seconds after it
> > happened.
>
> If the old service leader is in a very long GC it will not detect the
> partition. In the face of VM pauses, etc. it's not possible to avoid 2
> leaders for a short period of time.
>
> -JZ


[jira] [Commented] (ZOOKEEPER-3188) Improve resilience to network

2018-12-06 Thread Michael Han (JIRA)


[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-3188?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16712162#comment-16712162
 ] 

Michael Han commented on ZOOKEEPER-3188:


Appreciate detailed reply, agree on replies on 1 and 2.

bq. Such changes should be handled exactly the way they are now and there 
should be no interactions with the changes to the networking stack. 

Agreed. I think I was just looking for more elaborated use cases around using 
reconfig to manipulate multiple server addresses, as the proposal does not go 
into details other than 'support dynamic reconfiguration.'. I expect dynamic 
reconfiguration will just work out of box with proper abstractions, without 
touching too much part of reconfiguration code path, but there are some 
subtleties to consider. A couple of examples:

* Proper rebalance client connections - this was discussed on dev mailing list.
* Avoid unnecessary leader elections during reconfig - this change will 
probably change the abstraction of server addresses (QuorumServer) and we 
should be careful how the QuorumServers will be compared, to avoid unnecessary 
leader elections in cases where the server set is the same but some servers 
have new server addresses.
There might be more cases to consider...

bq. The documentation, in particular, should be essentially identical except 
that an example of adding an address might be nice

I am thinking at least 
[this|https://zookeeper.apache.org/doc/r3.5.4-beta/zookeeperReconfig.html#sc_reconfig_clientport]
 should be updated to reflect the fact that 1. the config format is changed and 
2. the multiple server addresses can be manipulated via reconfig.


> Improve resilience to network
> -
>
> Key: ZOOKEEPER-3188
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3188
> Project: ZooKeeper
>  Issue Type: Bug
>Reporter: Ted Dunning
>Priority: Major
>  Labels: pull-request-available
>  Time Spent: 0.5h
>  Remaining Estimate: 0h
>
> We propose to add network level resiliency to Zookeeper. The ideas that we 
> have on the topic have been discussed on the mailing list and via a 
> specification document that is located at 
> [https://docs.google.com/document/d/1iGVwxeHp57qogwfdodCh9b32P2_kOQaJZ2GDo7j36fI/edit?usp=sharing]
> That document is copied to this issue which is being created to report the 
> results of experimental implementations.
> h1. Zookeeper Network Resilience
> h2. Background
> Zookeeper is designed to help in building distributed systems. It provides a 
> variety of operations for doing this and all of these operations have rather 
> strict guarantees on semantics. Zookeeper itself is a distributed system made 
> up of cluster containing a leader and a number of followers. The leader is 
> designated in a process known as leader election in which a majority of all 
> nodes in the cluster must agree on a leader. All subsequent operations are 
> initiated by the leader and completed when a majority of nodes have confirmed 
> the operation. Whenever an operation cannot be confirmed by a majority or 
> whenever the leader goes missing for a time, a new leader election is 
> conducted and normal operations proceed once a new leader is confirmed.
>  
> The details of this are not important relative to this discussion. What is 
> important is that the semantics of the operations conducted by a Zookeeper 
> cluster and the semantics of how client processes communicate with the 
> cluster depend only on the basic fact that messages sent over TCP connections 
> will never appear out of order or missing. Central to the design of ZK is 
> that a server to server network connection is used as long as it works to use 
> it and a new connection is made when it appears that the old connection isn't 
> working.
>  
> As currently implemented, however, each member of a Zookeeper cluster can 
> have only a single address as viewed from some other process. This means, 
> absent network link bonding, that the loss of a single switch or a few 
> network connections could completely stop the operations of a the Zookeeper 
> cluster. It is the goal of this work to address this issue by allowing each 
> server to listen on multiple network interfaces and to connect to other 
> servers any of several addresses. The effect will be to allow servers to 
> communicate over redundant network paths to improve resiliency to network 
> failures without changing any core algorithms.
> h2. Proposed Change
> Interestingly, the correct operations of a Zookeeper cluster do not depend on 
> _how_ a TCP connection was made. There is no reason at all not to advertise 
> multiple addresses for mem

[jira] [Commented] (ZOOKEEPER-2778) Potential server deadlock between follower sync with leader and follower receiving external connection requests.

2018-11-20 Thread Michael Han (JIRA)


[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-2778?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16693936#comment-16693936
 ] 

Michael Han commented on ZOOKEEPER-2778:


I have to refresh my memory on this issue, but now looking back I think the 
gist of the issue is:
We want to guarantee to get a consistent membership view of the ensemble thus 
we need to lock (QV_LOCK) on the quorum peer when we access (read/write) to it. 
Meanwhile we need another lock on QCM itself and the order of acquiring both 
locks in different path is not consistent, thus causing dead lock.

The fix I proposed earlier and PR 707 did it by removing the QV lock on the 
read path. The problem is I am not sure how to validate its correctness given 
the intertwined code path :) - I previously while working on this was convinced 
removing QV_LOCK on read path of the three addresses is sound, now I am not 
sure.

We could also try remove one or both synchronized on the connectOne of QCM - it 
seems OK at least for the first connectOne (with two parameters) and this 
should fix this specific dead lock.

Another idea is to avoid QV lock completely by abstract the quorum verifier as 
an AtomicrReference similar as 707 did for three address field, if it's 
feasible to do so.

> Potential server deadlock between follower sync with leader and follower 
> receiving external connection requests.
> 
>
> Key: ZOOKEEPER-2778
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2778
> Project: ZooKeeper
>  Issue Type: Bug
>  Components: quorum
>Affects Versions: 3.5.3
>Reporter: Michael Han
>Priority: Blocker
>  Labels: pull-request-available
> Fix For: 3.6.0, 3.5.5
>
>  Time Spent: 1h 20m
>  Remaining Estimate: 0h
>
> It's possible to have a deadlock during recovery phase. 
> Found this issue by analyzing thread dumps of "flaky" ReconfigRecoveryTest 
> [1]. . Here is a sample thread dump that illustrates the state of the 
> execution:
> {noformat}
> [junit]  java.lang.Thread.State: BLOCKED
> [junit] at  
> org.apache.zookeeper.server.quorum.QuorumPeer.getElectionAddress(QuorumPeer.java:686)
> [junit] at  
> org.apache.zookeeper.server.quorum.QuorumCnxManager.initiateConnection(QuorumCnxManager.java:265)
> [junit] at  
> org.apache.zookeeper.server.quorum.QuorumCnxManager.connectOne(QuorumCnxManager.java:445)
> [junit] at  
> org.apache.zookeeper.server.quorum.QuorumCnxManager.receiveConnection(QuorumCnxManager.java:369)
> [junit] at  
> org.apache.zookeeper.server.quorum.QuorumCnxManager$Listener.run(QuorumCnxManager.java:642)
> [junit] 
> [junit]  java.lang.Thread.State: BLOCKED
> [junit] at  
> org.apache.zookeeper.server.quorum.QuorumCnxManager.connectOne(QuorumCnxManager.java:472)
> [junit] at  
> org.apache.zookeeper.server.quorum.QuorumPeer.connectNewPeers(QuorumPeer.java:1438)
> [junit] at  
> org.apache.zookeeper.server.quorum.QuorumPeer.setLastSeenQuorumVerifier(QuorumPeer.java:1471)
> [junit] at  
> org.apache.zookeeper.server.quorum.Learner.syncWithLeader(Learner.java:520)
> [junit] at  
> org.apache.zookeeper.server.quorum.Follower.followLeader(Follower.java:88)
> [junit] at  
> org.apache.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:1133)
> {noformat}
> The dead lock happens between the quorum peer thread which running the 
> follower that doing sync with leader work, and the listener of the qcm of the 
> same quorum peer that doing the receiving connection work. Basically to 
> finish sync with leader, the follower needs to synchronize on both QV_LOCK 
> and the qmc object it owns; while in the receiver thread to finish setup an 
> incoming connection the thread needs to synchronize on both the qcm object 
> the quorum peer owns, and the same QV_LOCK. It's easy to see the problem here 
> is the order of acquiring two locks are different, thus depends on timing / 
> actual execution order, two threads might end up acquiring one lock while 
> holding another.
> [1] 
> org.apache.zookeeper.server.quorum.ReconfigRecoveryTest.testCurrentServersAreObserversInNextConfig



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


  1   2   3   4   5   6   7   8   9   10   >