Re: Zookeeper leader stop and restart question

2010-11-01 Thread Flavio Junqueira
Hi Ruifang, It is not clear to me if you verified that leader1 restarted correctly. Was it able to join the ensemble by following leader2?-FlavioOn Nov 1, 2010, at 8:09 PM, Ruifang Ge wrote:Hi,I started a 5-node zookeeper cluster, then killed the leader (leader1). One of the other server became

Re: Is it possible to read/write a ledger concurrently

2010-10-22 Thread Flavio Junqueira
I thought we had agreed at some point that the application should do it in the case it needs this feature. That is, every so often the app writer either writes to ZooKeeper its last confirmed write or it sends directly to the reader. Knowing a confirmed write x enables the reader to read up to

Re: Changing configuration

2010-10-07 Thread Flavio Junqueira
We don't have dynamic configuration yet, but it is on our todo list: http://wiki.apache.org/hadoop/ZooKeeper/ClusterMembershipso for now I believe you would have to reconfigure manually and restart the cluster. For Zab, you should be looking at org.apache.zookeeper.server.quorum.Cheers,-FlavioOn

Re: BookKeeper newbie question

2010-10-01 Thread Flavio Junqueira
Thanks for your questions, Amit. On Sep 28, 2010, at 6:37 PM, amit jaiswal wrote: Hi, I am experimenting with BookKeeper and have a question on LedgerHandler class. The readEntries(firstEntry, lastEntry) method takes the indexes of first and last entries. Also, the LedgerSequence object

Re: Achieving quorum with only half of the nodes

2010-07-15 Thread Flavio Junqueira
? Would it result in considerable performance impact due to network latency? I hope that at least in theory since quorum can be reached without ack from EC2 node performance impact might be manageable.Regards,SergeiOn 07/14/2010 04:52 PM, Flavio Junqueira wrote:Hi Sergei, I'm not sure what

Re: Achieving quorum with only half of the nodes

2010-07-14 Thread Flavio Junqueira
Hi Sergei, I'm not sure what the implementation of QuorumVerifier you have in mind would look like to make your setting work. Even if you don't have partitions, variation in message delays can cause inconsistencies in your ZooKeeper cluster. Keep in mind that we make the assumption that quorums

Re: Zookeeper outage recap questions

2010-07-01 Thread Flavio Junqueira
Hi Travis, Do you think it would be possible for you to open a jira and upload your logs?Thanks,-FlavioOn Jul 1, 2010, at 8:13 AM, Travis Crawford wrote:Hey zookeepers -We just experienced a total zookeeper outage, and here's a quickpost-mortem of the issue, and some questions about preventing it

Re: zookeeper crash

2010-06-02 Thread Flavio Junqueira
Hi Charity, This is certainly not expected. It would be very useful if you could provide us with as much information about your issue as possible. I would suggest that either you create a new jira and link it to ZOOKEEPER-335, or that you add to 335 directly. We'll be looking further into

BookKeeper Performance Figures

2010-05-18 Thread Flavio Junqueira
Just in case anyone is interested, I've posted some BookKeeper performance figures here: http://wiki.apache.org/hadoop/BookKeeperPerfPage I'll be adding more numbers soon. -Flavio

Re: Problems with ZooKeeper apache wiki.

2010-05-14 Thread Flavio Junqueira
Hi Sudipto, I'm sorry but I don't have slides I can share that include a description of leader election. I'll work on it when I have a chance. In any case, you may consider inspecting the code if you need it urgently. -Flavio On May 13, 2010, at 10:54 PM, Sudipto Das wrote: Thanks

Re: zookeeper consistency model?

2010-04-29 Thread Flavio Junqueira
Hi Chen, Let's say that the value of a znode /test is initially v and client A writes value v' to znode /test. If the server that client B is connected to has not persisted the update operation of A, it will read v. If it submits sync before the read, client B will read v'. -Flavio On

Re: Managing multi-site clusters with Zookeeper

2010-03-15 Thread Flavio Junqueira
On top of Ben's description, you probably need to set initLimit to several minutes to transfer 700MB (worst case). The value of syncLimit, however, does not need to be that large. -Flavio On Mar 15, 2010, at 7:24 PM, Benjamin Reed wrote: it is a bit confusing but initLimit is the timer

Re: Killing a zookeeper server

2010-01-13 Thread Flavio Junqueira
Hi Nick, Your assessment sounds correct, the issue seems to be caused by the bug described in ZOOKEEPER-427. Can't you upgrade to a newer release? Killing the leader should do it, but the bug will still be there, so I recommend upgrading. Thanks, -Flavio On Jan 12, 2010, at 10:52 PM, Nick

Re: The idea behind 'myid'

2009-09-30 Thread Flavio Junqueira
We just need a unique identifier for every server. If such an identifier magically appears somehow, then I believe our protocols will be equally happy. Now, a mechanism to assign ids would also have to take into consideration the group scheme we have for hierarchical quorums. To assign

Re: Watches

2009-08-31 Thread Flavio Junqueira
I forgot to mention this. You may also consider adding more zookeeper servers and setting the weight of such servers to zero. We will be introducing this possibility in 3.2.1 (the upcoming release). Zero- weight servers simulate observers, but they do not behave exactly as a observers,

Re: Unending Leader Elections in WAN deploy

2009-07-31 Thread Flavio Junqueira
: Flavio Junqueira [mailto:f...@yahoo-inc.com] Sent: Friday, July 31, 2009 7:48 PM To: zookeeper-user@hadoop.apache.org Subject: Re: Unending Leader Elections in WAN deploy It should be in 479. Perhaps you have a stale version of the patch. -Flavio On Jul 31, 2009, at 7:46 PM, Todd Greenwood

Re: test failures in branch-3.2

2009-07-30 Thread Flavio Junqueira
Todd, On Jul 30, 2009, at 5:08 PM, Todd Greenwood wrote: The build succeeds, but not the all of the tests. In previous test runs, I noticed an error in org.apache.zookeeper.test.FLETest. It was not able to bind to a port or something. Now, after a machine reboot, I'm getting different

Re: Zookeeper WAN Configuration

2009-07-24 Thread Flavio Junqueira
Servers in a quorum need to be able to talk to each other to elect a leader. Once a leader is elected, followers only talk to the leader. Of course, if the leader fails, servers in some quorum will need to talk to each other again. If no quorum can be formed, the system is stalled.

Re: Leader Elections

2009-07-20 Thread Flavio Junqueira
wrote: Can you submit updates via an observer? On Sat, Jul 18, 2009 at 6:38 AM, Flavio Junqueira f...@yahoo-inc.com wrote: 2- Observers: you could have one computing center containing an ensemble and observers around the edge just learning committed values. -- Ted Dunning, CTO DeepDyve

Re: [ANNOUNCE] Apache ZooKeeper 3.1.0

2009-02-20 Thread Flavio Junqueira
Hi Bill, I'm sorry, I missed this message initially. I'm sending below a table that gives you throughput figures for BookKeeper. The rows correspond to distinct BookKeeper configuration (ensemble size, quorum size, entry type), and the columns to different values for the length of an entry

Re: [ANNOUNCE] Apache ZooKeeper 3.1.0

2009-02-20 Thread Flavio Junqueira
Also, you may consider checking a graph that we posted comparing the performance of BookKeeper with the one of HDFS using a local file system and local+NFS in the jira issue 5189 (https://issues.apache.org/jira/browse/HADOOP-5189 ). -Flavio On Feb 20, 2009, at 10:05 AM, Flavio Junqueira

Re: Distributed queue: how to ensure no lost items?

2009-01-08 Thread Flavio Junqueira
You can't simply leave an element in the queue until a consumer finishes processing it, otherwise multiple consumers may end up processing it. What about the following: - Use a failure detector to detect which consumers are up; - Before removing an element from the queue, a consumer creates

RE: Dynamic server management?

2008-11-17 Thread Flavio Junqueira
- Von: Flavio Junqueira [mailto:[EMAIL PROTECTED] Gesendet: Montag, 17. November 2008 13:49 An: zookeeper-user@hadoop.apache.org Betreff: RE: Dynamic server management? Hi Thomas, We currently don't have such a feature of adding and removing servers dynamically, although we would like to, so

RE: Leader election stalled

2008-09-16 Thread Flavio Junqueira
Austin, Please check: https://issues.apache.org/jira/browse/ZOOKEEPER-140 Thanks, -Flavio -Original Message- From: Austin Shoemaker [mailto:[EMAIL PROTECTED] Sent: Tuesday, September 16, 2008 12:22 PM To: zookeeper-user@hadoop.apache.org Subject: Re: Leader election stalled Ben,