Re: How to handle "Node does not exist" error?

2010-08-11 Thread Ted Dunning
Try running the server in non-embedded mode.

Also, you are assuming that you know everything about how to configure the
quorumPeer.  That is going to change and your code will break at that time.
 If you use a non-embedded cluster, this won't be a problem and you will be
able to upgrade ZK version without having to restart your service.

My own opinion is that running an embedded ZK is a serious architectural
error.  Since I don't know your particular situation, it might be different,
but there is an inherent contradiction involved in running a coordination
layer as part of the thing being coordinated.  Whatever your software does,
it isn't what ZK does.  As such, it is better to factor out the ZK
functionality and make it completely stable.  That gives you a much simpler
world and will make it easier for you to trouble shoot your system.  The
simple fact that you can't take down your service without affecting the
reliability of your ZK layer makes this a very bad idea.

The problems you are having now are only a preview of what this
architectural error leads to.  There will be more problems and many of them
are likely to be more subtle and lead to service interruptions and lots of
wasted time.

On Wed, Aug 11, 2010 at 8:49 PM, Dr Hao He  wrote:

> hi, Ted and Mahadev,
>
>
> Here are some more details about my setup:
>
> I run zookeeper in the embedded mode with the following code:
>
>quorumPeer = new QuorumPeer();
>
>  quorumPeer.setClientPort(getClientPort());
>quorumPeer.setTxnFactory(new
> FileTxnSnapLog(new File(getDataLogDir()), new File(getDataDir(;
>
>  quorumPeer.setQuorumPeers(getServers());
>
>  quorumPeer.setElectionType(getElectionAlg());
>quorumPeer.setMyid(getServerId());
>
>  quorumPeer.setTickTime(getTickTime());
>
>  quorumPeer.setInitLimit(getInitLimit());
>
>  quorumPeer.setSyncLimit(getSyncLimit());
>
>  quorumPeer.setQuorumVerifier(getQuorumVerifier());
>
>  quorumPeer.setCnxnFactory(cnxnFactory);
>quorumPeer.start();
>
>
> The configuration values are read from the following XML document for
> server 1:
>
>  serverId="1">
>  
>  
>  
> 
>
>
> The other servers have the same configurations except their ids being
> changed to 2 and 3.
>
> The error occurred on server 3 when I batch loaded some messages to server
> 1.  However, this error does not always happen.  I am not sure exactly what
> trigged this error yet.
>
> I also performed the "stat" operation on one of the "No exit" node and got:
>
> stat
> /xpe/queues/3bd7851e79381ef4bfd1a5857b5e34c04e5159e5/msgs/msg001583
> Exception in thread "main" java.lang.NullPointerException
>at
> org.apache.zookeeper.ZooKeeperMain.printStat(ZooKeeperMain.java:129)
>at
> org.apache.zookeeper.ZooKeeperMain.processZKCmd(ZooKeeperMain.java:715)
>at
> org.apache.zookeeper.ZooKeeperMain.processCmd(ZooKeeperMain.java:579)
>at
> org.apache.zookeeper.ZooKeeperMain.executeLine(ZooKeeperMain.java:351)
>at org.apache.zookeeper.ZooKeeperMain.run(ZooKeeperMain.java:309)
>at org.apache.zookeeper.ZooKeeperMain.main(ZooKeeperMain.java:268)
> [...@t43 zookeeper-3.2.2]$ bin/zkCli.sh
>
>
> Those message nodes are created as CreateMode.PERSISTENT_SEQUENTIAL and are
> deleted by the last server who has read them.
>
> If I remove the troubled server's zookeeper log directory and restart the
> server, then everything is ok.
>
> I will try to get the nc result next time I see this problem.
>
>
> Dr Hao He
>
> XPE - the truly SOA platform
>
> h...@softtouchit.com
> http://softtouchit.com
> http://itunes.com/apps/Scanmobile
>
> On 12/08/2010, at 12:32 AM, Mahadev Konar wrote:
>
> > HI Dr Hao,
> >  Can you please post the configuration of all the 3 zookeeper servers? I
> > suspect it might be misconfigured clusters and they might not belong to
> the
> > same ensemble.
> >
> > Just to be clear:
> > /xpe/queues/3bd7851e79381ef4bfd1a5857b5e34c04e5159e5/msgs/msg002807
> >
> > And other such nodes exist on one of the zookeeper servers and the same
> node
> > does not exist on other servers?
> >
> > Also, as ted pointed out, can you please post the output of echo ³stat² |
> nc
> > localhost 2181 (on all the 3 servers) to the list?
> >
> > Thanks
> > mahadev
> >
> >
> >
> > On 8/11/10 12:10 AM, "Dr Hao He"  wrote:
> >
> >> hi, Ted,
> >>
> >> Thanks for the reply.  Here is what I did:
> >>
> >> [zk: localhost:2181(CONNECTED) 0] ls
> >> /xpe/queues/3bd7851e79381ef4bfd1a5857b5e34c04e5159e5/msgs/msg002948
> >> []
> >> zk: localhost:2181(CONNECTED) 1] ls
> >> /xpe/queues/3bd7851e79381ef4bfd1a5857b5e34c04e5159e5/msgs
> >> [msg002807, msg002700, msg002701, msg002804,
> msg002704,
> >> msg002706, msg002601, msg001849, msg001847,
> msg002508,
> >> msg002609, msg001841, msg000

Re: How to handle "Node does not exist" error?

2010-08-11 Thread Dr Hao He
hi, Ted and Mahadev,


Here are some more details about my setup:

I run zookeeper in the embedded mode with the following code:

quorumPeer = new QuorumPeer();

quorumPeer.setClientPort(getClientPort());
quorumPeer.setTxnFactory(new 
FileTxnSnapLog(new File(getDataLogDir()), new File(getDataDir(;
quorumPeer.setQuorumPeers(getServers());

quorumPeer.setElectionType(getElectionAlg());
quorumPeer.setMyid(getServerId());
quorumPeer.setTickTime(getTickTime());
quorumPeer.setInitLimit(getInitLimit());
quorumPeer.setSyncLimit(getSyncLimit());

quorumPeer.setQuorumVerifier(getQuorumVerifier());
quorumPeer.setCnxnFactory(cnxnFactory);
quorumPeer.start();


The configuration values are read from the following XML document for server 1:


  
   
  



The other servers have the same configurations except their ids being changed 
to 2 and 3.

The error occurred on server 3 when I batch loaded some messages to server 1.  
However, this error does not always happen.  I am not sure exactly what trigged 
this error yet.

I also performed the "stat" operation on one of the "No exit" node and got:

stat /xpe/queues/3bd7851e79381ef4bfd1a5857b5e34c04e5159e5/msgs/msg001583   
Exception in thread "main" java.lang.NullPointerException
at org.apache.zookeeper.ZooKeeperMain.printStat(ZooKeeperMain.java:129)
at 
org.apache.zookeeper.ZooKeeperMain.processZKCmd(ZooKeeperMain.java:715)
at org.apache.zookeeper.ZooKeeperMain.processCmd(ZooKeeperMain.java:579)
at 
org.apache.zookeeper.ZooKeeperMain.executeLine(ZooKeeperMain.java:351)
at org.apache.zookeeper.ZooKeeperMain.run(ZooKeeperMain.java:309)
at org.apache.zookeeper.ZooKeeperMain.main(ZooKeeperMain.java:268)
[...@t43 zookeeper-3.2.2]$ bin/zkCli.sh 


Those message nodes are created as CreateMode.PERSISTENT_SEQUENTIAL and are 
deleted by the last server who has read them. 

If I remove the troubled server's zookeeper log directory and restart the 
server, then everything is ok.

I will try to get the nc result next time I see this problem.


Dr Hao He

XPE - the truly SOA platform

h...@softtouchit.com
http://softtouchit.com
http://itunes.com/apps/Scanmobile

On 12/08/2010, at 12:32 AM, Mahadev Konar wrote:

> HI Dr Hao,
>  Can you please post the configuration of all the 3 zookeeper servers? I
> suspect it might be misconfigured clusters and they might not belong to the
> same ensemble.
> 
> Just to be clear:
> /xpe/queues/3bd7851e79381ef4bfd1a5857b5e34c04e5159e5/msgs/msg002807
> 
> And other such nodes exist on one of the zookeeper servers and the same node
> does not exist on other servers?
> 
> Also, as ted pointed out, can you please post the output of echo ³stat² | nc
> localhost 2181 (on all the 3 servers) to the list?
> 
> Thanks
> mahadev
> 
> 
> 
> On 8/11/10 12:10 AM, "Dr Hao He"  wrote:
> 
>> hi, Ted,
>> 
>> Thanks for the reply.  Here is what I did:
>> 
>> [zk: localhost:2181(CONNECTED) 0] ls
>> /xpe/queues/3bd7851e79381ef4bfd1a5857b5e34c04e5159e5/msgs/msg002948
>> []
>> zk: localhost:2181(CONNECTED) 1] ls
>> /xpe/queues/3bd7851e79381ef4bfd1a5857b5e34c04e5159e5/msgs
>> [msg002807, msg002700, msg002701, msg002804, msg002704,
>> msg002706, msg002601, msg001849, msg001847, msg002508,
>> msg002609, msg001841, msg002607, msg002606, msg002604,
>> msg002809, msg002817, msg001633, msg002812, msg002814,
>> msg002711, msg002815, msg002713, msg002716, msg001772,
>> msg002811, msg001635, msg001774, msg002515, msg002610,
>> msg001838, msg002517, msg002612, msg002519, msg001973,
>> msg001835, msg001974, msg002619, msg001831, msg002510,
>> msg002512, msg002615, msg002614, msg002617, msg002104,
>> msg002106, msg001769, msg001768, msg002828, msg002822,
>> msg001760, msg002820, msg001963, msg001961, msg002110,
>> msg002118, msg002900, msg002836, msg001757, msg002907,
>> msg001753, msg001752, msg001755, msg001952, msg001958,
>> msg001852, msg001956, msg001854, msg002749, msg001608,
>> msg001609, msg002747, msg002882, msg001743, msg002888,
>> msg001605, msg002885, msg001487, msg001746, msg002330,
>> msg001749, msg001488, msg001489, msg001881, msg001491,
>> msg002890, msg

Backing up zk data files

2010-08-11 Thread Adam Rosien
http://hadoop.apache.org/zookeeper/docs/r3.3.1/zookeeperAdmin.html#sc_dataFileManagement
says that one can copy the contents of the data directory and use it
on another machine. The example states the other instance is not in
the server list; what would happen if one did copy it to an offline
member of the quorum that then starts up?

Do the docs imply that one can copy the data directory as-is as a
backup method? Is it "restorable" to any crashed/hosed server, or only
the one with the same server id?

What is a valid backup method for zk data?

.. Adam


Re: Clarification on async calls in a cluster

2010-08-11 Thread Patrick Hunt


On 08/11/2010 03:25 PM, Jordan Zimmerman wrote:

If I use an async version of a call in a cluster ("ensemble") what
happens if the server I'm connected to goes down? Does ZK
transparently resubmit the call to the next server in the cluster and
call my async callback or is there something I need to do? The docs
aren't clear on this and searching the archive didn't give me the
answer. Another source of confusion here is that the non-async
versions do not resubmit the call - I need to do that manually.

Thanks!


Hi Jordan, the callbacks have a "rc" parameter that details the result 
of the request (result code), this will be one of KeeperException.Code, 
in this case CONNECTIONLOSS. You receive a connection loss result when 
the client has sent a request to the server but loses the connection 
before the server responds. You must resubmit of this request manually 
(usually once you reconnect to the cluster), same as for sync calls.


See these sections in the faq:
http://wiki.apache.org/hadoop/ZooKeeper/FAQ#A2

also some detail in
http://hadoop.apache.org/zookeeper/docs/current/zookeeperProgrammers.html#ch_zkSessions

I agree the docs could be improved here. The java api for callback is 
esp. embarassing (there is none). Please enter JIRAs for any areas you'd 
like to see improved, including adding javadoc to the callbacks.


Regards,

Patrick


Re: Sequence Number Generation With Zookeeper

2010-08-11 Thread Adam Rosien
Ah thanks, I forgot the "majority-commit" property because I also
forgot that all servers know what the cluster should look like, rather
than act adaptively (which wouldn't make sense after all).

.. Adam

On Wed, Aug 11, 2010 at 3:23 PM, Ted Dunning  wrote:
> Can't happen.
>
> In a network partition, the side without a quorum can't update the file
> version.
>
> On Wed, Aug 11, 2010 at 3:11 PM, Adam Rosien  wrote:
>
>> What happens during a network partition and different clients are
>> incrementing "different" counters, and then the partition goes away?
>> Won't (potentially) the same sequence value be given out to two
>> clients?
>>
>> .. Adam
>>
>> On Thu, Aug 5, 2010 at 5:38 PM, Jonathan Holloway
>>  wrote:
>> > Hi Ted,
>> >
>> > Thanks for the comments.
>> >
>> > I might have overlooked something here, but is it also possible to do the
>> > following:
>> >
>> > 1. Create a PERSISTENT node
>> > 2. Have multiple clients set the data on the node, e.g.  Stat stat =
>> > zookeeper.setData(SEQUENCE, ArrayUtils.EMPTY_BYTE_ARRAY, -1);
>> > 3. Use the version number from stat.getVersion() as the sequence
>> (obviously
>> > I'm limited to Integer.MAX_VALUE)
>> >
>> > Are there any weird race conditions involved here which would mean that a
>> > client would receive the wrong Stat object back?
>> >
>> > Many thanks again,
>> > Jon.
>> >
>> > On 5 August 2010 16:09, Ted Dunning  wrote:
>> >
>> >> (b)
>> >>
>> >> BUT:
>> >>
>> >> Sequential numbering is a special case of "now".  In large diameters,
>> now
>> >> gets very expensive.  This is a special case of that assertion.  If
>> there
>> >> is
>> >> a way to get away from this presumption of the need for sequential
>> >> numbering, you will be miles better off.
>> >>
>> >> HOWEVER:
>> >>
>> >> ZK can do better than you suggest.  Incrementing a counter does involve
>> >> potential contention, but you will very likely be able to get to pretty
>> >> high
>> >> rates before the optimistic locking begins to fail.  If you code your
>> >> update
>> >> with a few tries at full speed followed by some form of retry back-off,
>> you
>> >> should get pretty close to the best possible performance.
>> >>
>> >> You might also try building a lock with an ephemeral file before
>> updating
>> >> the counter.  I would expect that this will be slower than the back-off
>> >> option if only because involves more transactions in ZK.  IF you wanted
>> to
>> >> get too complicated for your own good, you could have a secondary
>> strategy
>> >> flag that is only sampled by all clients every few seconds and is
>> updated
>> >> whenever a client needs to back-off more than say 5 steps.  If this flag
>> >> has
>> >> been updated recently, then clients should switch to the locking
>> protocol.
>> >>  You might even have several locks so that you don't exclude all other
>> >> updaters, merely thin them out a bit.  This flagged strategy would run
>> as
>> >> fast as optimistic locking as long as optimistic locking is fast and
>> then
>> >> would limit the total number of transactions needed under very high
>> load.
>> >>
>> >> On Thu, Aug 5, 2010 at 3:31 PM, Jonathan Holloway <
>> >> jonathan.hollo...@gmail.com> wrote:
>> >>
>> >> > My so far involve:
>> >> > a) Creating a node with PERSISTENT_SEQUENTIAL then deleting it - this
>> >> gives
>> >> > me the monotonically increasing number, but the sequence number isn't
>> >> > contiguous
>> >> > b) Storing the sequence number in the data portion of a persistent
>> node -
>> >> > then updating this (using the version number - aka optimistic
>> locking).
>> >> >  The
>> >> > problem with this is that under high load I'm assuming there'll be a
>> lot
>> >> of
>> >> > contention and hence failures with regards to updates.
>> >> >
>> >>
>> >
>>
>


Clarification on async calls in a cluster

2010-08-11 Thread Jordan Zimmerman
If I use an async version of a call in a cluster ("ensemble") what happens if 
the server I'm connected to goes down? Does ZK transparently resubmit the call 
to the next server in the cluster and call my async callback or is there 
something I need to do? The docs aren't clear on this and searching the archive 
didn't give me the answer. Another source of confusion here is that the 
non-async versions do not resubmit the call - I need to do that manually.

Thanks!

Re: Sequence Number Generation With Zookeeper

2010-08-11 Thread Ted Dunning
Can't happen.

In a network partition, the side without a quorum can't update the file
version.

On Wed, Aug 11, 2010 at 3:11 PM, Adam Rosien  wrote:

> What happens during a network partition and different clients are
> incrementing "different" counters, and then the partition goes away?
> Won't (potentially) the same sequence value be given out to two
> clients?
>
> .. Adam
>
> On Thu, Aug 5, 2010 at 5:38 PM, Jonathan Holloway
>  wrote:
> > Hi Ted,
> >
> > Thanks for the comments.
> >
> > I might have overlooked something here, but is it also possible to do the
> > following:
> >
> > 1. Create a PERSISTENT node
> > 2. Have multiple clients set the data on the node, e.g.  Stat stat =
> > zookeeper.setData(SEQUENCE, ArrayUtils.EMPTY_BYTE_ARRAY, -1);
> > 3. Use the version number from stat.getVersion() as the sequence
> (obviously
> > I'm limited to Integer.MAX_VALUE)
> >
> > Are there any weird race conditions involved here which would mean that a
> > client would receive the wrong Stat object back?
> >
> > Many thanks again,
> > Jon.
> >
> > On 5 August 2010 16:09, Ted Dunning  wrote:
> >
> >> (b)
> >>
> >> BUT:
> >>
> >> Sequential numbering is a special case of "now".  In large diameters,
> now
> >> gets very expensive.  This is a special case of that assertion.  If
> there
> >> is
> >> a way to get away from this presumption of the need for sequential
> >> numbering, you will be miles better off.
> >>
> >> HOWEVER:
> >>
> >> ZK can do better than you suggest.  Incrementing a counter does involve
> >> potential contention, but you will very likely be able to get to pretty
> >> high
> >> rates before the optimistic locking begins to fail.  If you code your
> >> update
> >> with a few tries at full speed followed by some form of retry back-off,
> you
> >> should get pretty close to the best possible performance.
> >>
> >> You might also try building a lock with an ephemeral file before
> updating
> >> the counter.  I would expect that this will be slower than the back-off
> >> option if only because involves more transactions in ZK.  IF you wanted
> to
> >> get too complicated for your own good, you could have a secondary
> strategy
> >> flag that is only sampled by all clients every few seconds and is
> updated
> >> whenever a client needs to back-off more than say 5 steps.  If this flag
> >> has
> >> been updated recently, then clients should switch to the locking
> protocol.
> >>  You might even have several locks so that you don't exclude all other
> >> updaters, merely thin them out a bit.  This flagged strategy would run
> as
> >> fast as optimistic locking as long as optimistic locking is fast and
> then
> >> would limit the total number of transactions needed under very high
> load.
> >>
> >> On Thu, Aug 5, 2010 at 3:31 PM, Jonathan Holloway <
> >> jonathan.hollo...@gmail.com> wrote:
> >>
> >> > My so far involve:
> >> > a) Creating a node with PERSISTENT_SEQUENTIAL then deleting it - this
> >> gives
> >> > me the monotonically increasing number, but the sequence number isn't
> >> > contiguous
> >> > b) Storing the sequence number in the data portion of a persistent
> node -
> >> > then updating this (using the version number - aka optimistic
> locking).
> >> >  The
> >> > problem with this is that under high load I'm assuming there'll be a
> lot
> >> of
> >> > contention and hence failures with regards to updates.
> >> >
> >>
> >
>


zookeeper seems to hang

2010-08-11 Thread Ted Yu
Hi,
Using HBase 0.20.6 (with HBASE-2473) we encountered a situation where
Regionserver
process was shutting down and seemed to hang.

Here is the bottom of region server log:
http://pastebin.com/YYawJ4jA

zookeeper-3.2.2 is used.

Your comment is welcome.

Here is relevant portion from jstack - I attempted to attach jstack twice in
my email to d...@hbase.apache.org but failed:

"DestroyJavaVM" prio=10 tid=0x2aabb849c800 nid=0x6c60 waiting on
condition [0x]
   java.lang.Thread.State: RUNNABLE

"regionserver/10.32.42.245:60020" prio=10 tid=0x2aabb84ce000 nid=0x6c81
in Object.wait() [0x43755000]
   java.lang.Thread.State: WAITING (on object monitor)
at java.lang.Object.wait(Native Method)
- waiting on <0x2aaab76633c0> (a
org.apache.zookeeper.ClientCnxn$Packet)
at java.lang.Object.wait(Object.java:485)
at
org.apache.zookeeper.ClientCnxn.submitRequest(ClientCnxn.java:1099)
- locked <0x2aaab76633c0> (a
org.apache.zookeeper.ClientCnxn$Packet)
at org.apache.zookeeper.ClientCnxn.close(ClientCnxn.java:1077)
at org.apache.zookeeper.ZooKeeper.close(ZooKeeper.java:505)
- locked <0x2aaabf5e0c30> (a org.apache.zookeeper.ZooKeeper)
at
org.apache.hadoop.hbase.zookeeper.ZooKeeperWrapper.close(ZooKeeperWrapper.java:681)
at
org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.java:654)
at java.lang.Thread.run(Thread.java:619)

"main-EventThread" daemon prio=10 tid=0x43474000 nid=0x6c80 waiting
on condition [0x413f3000]
   java.lang.Thread.State: WAITING (parking)
at sun.misc.Unsafe.park(Native Method)
- parking to wait for  <0x2aaabf6e9150> (a
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)
at
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1987)
at
java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:399)
at
org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:414)

"RMI TCP Accept-0" daemon prio=10 tid=0x2aabb822c800 nid=0x6c7d runnable
[0x40752000]
   java.lang.Thread.State: RUNNABLE
at java.net.PlainSocketImpl.socketAccept(Native Method)
at java.net.PlainSocketImpl.accept(PlainSocketImpl.java:390)
- locked <0x2aaabf585578> (a java.net.SocksSocketImpl)
at java.net.ServerSocket.implAccept(ServerSocket.java:453)
at java.net.ServerSocket.accept(ServerSocket.java:421)
at
sun.management.jmxremote.LocalRMIServerSocketFactory$1.accept(LocalRMIServerSocketFactory.java:34)
at
sun.rmi.transport.tcp.TCPTransport$AcceptLoop.executeAcceptLoop(TCPTransport.java:369)
at
sun.rmi.transport.tcp.TCPTransport$AcceptLoop.run(TCPTransport.java:341)
at java.lang.Thread.run(Thread.java:619)


Re: Sequence Number Generation With Zookeeper

2010-08-11 Thread Adam Rosien
What happens during a network partition and different clients are
incrementing "different" counters, and then the partition goes away?
Won't (potentially) the same sequence value be given out to two
clients?

.. Adam

On Thu, Aug 5, 2010 at 5:38 PM, Jonathan Holloway
 wrote:
> Hi Ted,
>
> Thanks for the comments.
>
> I might have overlooked something here, but is it also possible to do the
> following:
>
> 1. Create a PERSISTENT node
> 2. Have multiple clients set the data on the node, e.g.  Stat stat =
> zookeeper.setData(SEQUENCE, ArrayUtils.EMPTY_BYTE_ARRAY, -1);
> 3. Use the version number from stat.getVersion() as the sequence (obviously
> I'm limited to Integer.MAX_VALUE)
>
> Are there any weird race conditions involved here which would mean that a
> client would receive the wrong Stat object back?
>
> Many thanks again,
> Jon.
>
> On 5 August 2010 16:09, Ted Dunning  wrote:
>
>> (b)
>>
>> BUT:
>>
>> Sequential numbering is a special case of "now".  In large diameters, now
>> gets very expensive.  This is a special case of that assertion.  If there
>> is
>> a way to get away from this presumption of the need for sequential
>> numbering, you will be miles better off.
>>
>> HOWEVER:
>>
>> ZK can do better than you suggest.  Incrementing a counter does involve
>> potential contention, but you will very likely be able to get to pretty
>> high
>> rates before the optimistic locking begins to fail.  If you code your
>> update
>> with a few tries at full speed followed by some form of retry back-off, you
>> should get pretty close to the best possible performance.
>>
>> You might also try building a lock with an ephemeral file before updating
>> the counter.  I would expect that this will be slower than the back-off
>> option if only because involves more transactions in ZK.  IF you wanted to
>> get too complicated for your own good, you could have a secondary strategy
>> flag that is only sampled by all clients every few seconds and is updated
>> whenever a client needs to back-off more than say 5 steps.  If this flag
>> has
>> been updated recently, then clients should switch to the locking protocol.
>>  You might even have several locks so that you don't exclude all other
>> updaters, merely thin them out a bit.  This flagged strategy would run as
>> fast as optimistic locking as long as optimistic locking is fast and then
>> would limit the total number of transactions needed under very high load.
>>
>> On Thu, Aug 5, 2010 at 3:31 PM, Jonathan Holloway <
>> jonathan.hollo...@gmail.com> wrote:
>>
>> > My so far involve:
>> > a) Creating a node with PERSISTENT_SEQUENTIAL then deleting it - this
>> gives
>> > me the monotonically increasing number, but the sequence number isn't
>> > contiguous
>> > b) Storing the sequence number in the data portion of a persistent node -
>> > then updating this (using the version number - aka optimistic locking).
>> >  The
>> > problem with this is that under high load I'm assuming there'll be a lot
>> of
>> > contention and hence failures with regards to updates.
>> >
>>
>


Re: How to handle "Node does not exist" error?

2010-08-11 Thread Mahadev Konar
HI Dr Hao,
  Can you please post the configuration of all the 3 zookeeper servers? I
suspect it might be misconfigured clusters and they might not belong to the
same ensemble.

Just to be clear:
/xpe/queues/3bd7851e79381ef4bfd1a5857b5e34c04e5159e5/msgs/msg002807

And other such nodes exist on one of the zookeeper servers and the same node
does not exist on other servers?

Also, as ted pointed out, can you please post the output of echo ³stat² | nc
localhost 2181 (on all the 3 servers) to the list?

Thanks
mahadev



On 8/11/10 12:10 AM, "Dr Hao He"  wrote:

> hi, Ted,
> 
> Thanks for the reply.  Here is what I did:
> 
> [zk: localhost:2181(CONNECTED) 0] ls
> /xpe/queues/3bd7851e79381ef4bfd1a5857b5e34c04e5159e5/msgs/msg002948
> []
> zk: localhost:2181(CONNECTED) 1] ls
> /xpe/queues/3bd7851e79381ef4bfd1a5857b5e34c04e5159e5/msgs
> [msg002807, msg002700, msg002701, msg002804, msg002704,
> msg002706, msg002601, msg001849, msg001847, msg002508,
> msg002609, msg001841, msg002607, msg002606, msg002604,
> msg002809, msg002817, msg001633, msg002812, msg002814,
> msg002711, msg002815, msg002713, msg002716, msg001772,
> msg002811, msg001635, msg001774, msg002515, msg002610,
> msg001838, msg002517, msg002612, msg002519, msg001973,
> msg001835, msg001974, msg002619, msg001831, msg002510,
> msg002512, msg002615, msg002614, msg002617, msg002104,
> msg002106, msg001769, msg001768, msg002828, msg002822,
> msg001760, msg002820, msg001963, msg001961, msg002110,
> msg002118, msg002900, msg002836, msg001757, msg002907,
> msg001753, msg001752, msg001755, msg001952, msg001958,
> msg001852, msg001956, msg001854, msg002749, msg001608,
> msg001609, msg002747, msg002882, msg001743, msg002888,
> msg001605, msg002885, msg001487, msg001746, msg002330,
> msg001749, msg001488, msg001489, msg001881, msg001491,
> msg002890, msg001889, msg002758, msg002241, msg002892,
> msg002852, msg002759, msg002898, msg002850, msg001733,
> msg002751, msg001739, msg002753, msg002756, msg002332,
> msg001872, msg002233, msg001721, msg001627, msg001720,
> msg001625, msg001628, msg001629, msg001729, msg002350,
> msg001727, msg002352, msg001622, msg001726, msg001623,
> msg001723, msg001724, msg001621, msg002736, msg002738,
> msg002363, msg001717, msg002878, msg002362, msg002361,
> msg001611, msg001894, msg002357, msg002218, msg002358,
> msg002355, msg001895, msg002356, msg001898, msg002354,
> msg001996, msg001990, msg002093, msg002880, msg002576,
> msg002579, msg002267, msg002266, msg002366, msg001901,
> msg002365, msg001903, msg001799, msg001906, msg002368,
> msg001597, msg002679, msg002166, msg001595, msg002481,
> msg002482, msg002373, msg002374, msg002371, msg001599,
> msg002773, msg002274, msg002275, msg002270, msg002583,
> msg002271, msg002580, msg002067, msg002277, msg002278,
> msg002376, msg002180, msg002467, msg002378, msg002182,
> msg002377, msg002184, msg002379, msg002187, msg002186,
> msg002665, msg002666, msg002381, msg002382, msg002661,
> msg002662, msg002663, msg002385, msg002284, msg002766,
> msg002282, msg002190, msg002599, msg002054, msg002596,
> msg002453, msg002459, msg002457, msg002456, msg002191,
> msg002652, msg002395, msg002650, msg002656, msg002655,
> msg002189, msg002047, msg002658, msg002659, msg002796,
> msg002250, msg002255, msg002589, msg002257, msg002061,
> msg002064, msg002585, msg002258, msg002587, msg002444,
> msg002446, msg002447, msg002450, msg002646, msg001501,
> msg002591, msg002592, msg001503, msg001506, msg002260,
> msg002594, msg002262, msg002263, msg002264, msg002590,
> msg002132, msg002130, msg002530, msg002931, msg001559,
> msg001808, msg002024, msg001553, msg002939, msg002937,
> msg001556, msg002935, msg002933, msg002140, msg001937,
> msg002143, msg002520, msg002522, msg002429, msg002524,
> msg002920, msg002035, msg001561, msg002134, msg002138,
> msg002925, msg002151, msg002287, msg002555, msg002010,
> msg002002, msg002290, msg001537, msg002005, msg00

Re: How to handle "Node does not exist" error?

2010-08-11 Thread Ted Dunning
What do your nodes  have in their logs during startup?   Are you sure  
you have them configured correctly?  Are the file ephemeral? Could  
they have disappeared on their own?


Sent from my iPhone

On Aug 11, 2010, at 12:10 AM, Dr Hao He  wrote:


hi, Ted,

Thanks for the reply.  Here is what I did:

[zk: localhost:2181(CONNECTED) 0] ls /xpe/queues/ 
3bd7851e79381ef4bfd1a5857b5e34c04e5159e5/msgs/msg002948

[]
zk: localhost:2181(CONNECTED) 1] ls /xpe/queues/ 
3bd7851e79381ef4bfd1a5857b5e34c04e5159e5/msgs
[msg002807, msg002700, msg002701, msg002804,  
msg002704, msg002706, msg002601, msg001849,  
msg001847, msg002508, msg002609, msg001841,  
msg002607, msg002606, msg002604, msg002809,  
msg002817, msg001633, msg002812, msg002814,  
msg002711, msg002815, msg002713, msg002716,  
msg001772, msg002811, msg001635, msg001774,  
msg002515, msg002610, msg001838, msg002517,  
msg002612, msg002519, msg001973, msg001835,  
msg001974, msg002619, msg001831, msg002510,  
msg002512, msg002615, msg002614, msg002617,  
msg002104, msg002106, msg001769, msg001768,  
msg002828, msg002822, msg001760, msg002820,  
msg001963, msg001961, msg002110, msg002118,  
msg002900, msg002836, msg001757, msg002907,  
msg001753, msg001752, msg001755, msg001952,  
msg001958, msg001852, msg001956, msg001854,  
msg002749, msg001608, msg001609, msg002747,  
msg002882, msg001743, msg002888, msg001605,  
msg002885, msg001487, msg001746, msg002330,  
msg001749, msg001488, msg001489, msg001881,  
msg001491, msg002890, msg001889, msg002758,  
msg002241, msg002892, msg002852, msg002759,  
msg002898, msg002850, msg001733, msg002751,  
msg001739, msg002753, msg002756, msg002332,  
msg001872, msg002233, msg001721, msg001627,  
msg001720, msg001625, msg001628, msg001629,  
msg001729, msg002350, msg001727, msg002352,  
msg001622, msg001726, msg001623, msg001723,  
msg001724, msg001621, msg002736, msg002738,  
msg002363, msg001717, msg002878, msg002362,  
msg002361, msg001611, msg001894, msg002357,  
msg002218, msg002358, msg002355, msg001895,  
msg002356, msg001898, msg002354, msg001996,  
msg001990, msg002093, msg002880, msg002576,  
msg002579, msg002267, msg002266, msg002366,  
msg001901, msg002365, msg001903, msg001799,  
msg001906, msg002368, msg001597, msg002679,  
msg002166, msg001595, msg002481, msg002482,  
msg002373, msg002374, msg002371, msg001599,  
msg002773, msg002274, msg002275, msg002270,  
msg002583, msg002271, msg002580, msg002067,  
msg002277, msg002278, msg002376, msg002180,  
msg002467, msg002378, msg002182, msg002377,  
msg002184, msg002379, msg002187, msg002186,  
msg002665, msg002666, msg002381, msg002382,  
msg002661, msg002662, msg002663, msg002385,  
msg002284, msg002766, msg002282, msg002190,  
msg002599, msg002054, msg002596, msg002453,  
msg002459, msg002457, msg002456, msg002191,  
msg002652, msg002395, msg002650, msg002656,  
msg002655, msg002189, msg002047, msg002658,  
msg002659, msg002796, msg002250, msg002255,  
msg002589, msg002257, msg002061, msg002064,  
msg002585, msg002258, msg002587, msg002444,  
msg002446, msg002447, msg002450, msg002646,  
msg001501, msg002591, msg002592, msg001503,  
msg001506, msg002260, msg002594, msg002262,  
msg002263, msg002264, msg002590, msg002132,  
msg002130, msg002530, msg002931, msg001559,  
msg001808, msg002024, msg001553, msg002939,  
msg002937, msg001556, msg002935, msg002933,  
msg002140, msg001937, msg002143, msg002520,  
msg002522, msg002429, msg002524, msg002920,  
msg002035, msg001561, msg002134, msg002138,  
msg002925, msg002151, msg002287, msg002555,  
msg002010, msg002002, msg002290, msg001537,  
msg002005, msg002147, msg002145, msg002698,  
msg001592, msg001810, msg002690, msg002691,  
msg001911, msg001910, msg002693, msg001812,  
msg001817, msg001547, msg002012, msg002015,  
msg002941, msg001688, msg002018, msg002684,  
msg002944, msg0

Re: How to handle "Node does not exist" error?

2010-08-11 Thread Dr Hao He
hi, Ted,

Thanks for the reply.  Here is what I did:

[zk: localhost:2181(CONNECTED) 0] ls 
/xpe/queues/3bd7851e79381ef4bfd1a5857b5e34c04e5159e5/msgs/msg002948
[]
zk: localhost:2181(CONNECTED) 1] ls 
/xpe/queues/3bd7851e79381ef4bfd1a5857b5e34c04e5159e5/msgs  
[msg002807, msg002700, msg002701, msg002804, msg002704, 
msg002706, msg002601, msg001849, msg001847, msg002508, 
msg002609, msg001841, msg002607, msg002606, msg002604, 
msg002809, msg002817, msg001633, msg002812, msg002814, 
msg002711, msg002815, msg002713, msg002716, msg001772, 
msg002811, msg001635, msg001774, msg002515, msg002610, 
msg001838, msg002517, msg002612, msg002519, msg001973, 
msg001835, msg001974, msg002619, msg001831, msg002510, 
msg002512, msg002615, msg002614, msg002617, msg002104, 
msg002106, msg001769, msg001768, msg002828, msg002822, 
msg001760, msg002820, msg001963, msg001961, msg002110, 
msg002118, msg002900, msg002836, msg001757, msg002907, 
msg001753, msg001752, msg001755, msg001952, msg001958, 
msg001852, msg001956, msg001854, msg002749, msg001608, 
msg001609, msg002747, msg002882, msg001743, msg002888, 
msg001605, msg002885, msg001487, msg001746, msg002330, 
msg001749, msg001488, msg001489, msg001881, msg001491, 
msg002890, msg001889, msg002758, msg002241, msg002892, 
msg002852, msg002759, msg002898, msg002850, msg001733, 
msg002751, msg001739, msg002753, msg002756, msg002332, 
msg001872, msg002233, msg001721, msg001627, msg001720, 
msg001625, msg001628, msg001629, msg001729, msg002350, 
msg001727, msg002352, msg001622, msg001726, msg001623, 
msg001723, msg001724, msg001621, msg002736, msg002738, 
msg002363, msg001717, msg002878, msg002362, msg002361, 
msg001611, msg001894, msg002357, msg002218, msg002358, 
msg002355, msg001895, msg002356, msg001898, msg002354, 
msg001996, msg001990, msg002093, msg002880, msg002576, 
msg002579, msg002267, msg002266, msg002366, msg001901, 
msg002365, msg001903, msg001799, msg001906, msg002368, 
msg001597, msg002679, msg002166, msg001595, msg002481, 
msg002482, msg002373, msg002374, msg002371, msg001599, 
msg002773, msg002274, msg002275, msg002270, msg002583, 
msg002271, msg002580, msg002067, msg002277, msg002278, 
msg002376, msg002180, msg002467, msg002378, msg002182, 
msg002377, msg002184, msg002379, msg002187, msg002186, 
msg002665, msg002666, msg002381, msg002382, msg002661, 
msg002662, msg002663, msg002385, msg002284, msg002766, 
msg002282, msg002190, msg002599, msg002054, msg002596, 
msg002453, msg002459, msg002457, msg002456, msg002191, 
msg002652, msg002395, msg002650, msg002656, msg002655, 
msg002189, msg002047, msg002658, msg002659, msg002796, 
msg002250, msg002255, msg002589, msg002257, msg002061, 
msg002064, msg002585, msg002258, msg002587, msg002444, 
msg002446, msg002447, msg002450, msg002646, msg001501, 
msg002591, msg002592, msg001503, msg001506, msg002260, 
msg002594, msg002262, msg002263, msg002264, msg002590, 
msg002132, msg002130, msg002530, msg002931, msg001559, 
msg001808, msg002024, msg001553, msg002939, msg002937, 
msg001556, msg002935, msg002933, msg002140, msg001937, 
msg002143, msg002520, msg002522, msg002429, msg002524, 
msg002920, msg002035, msg001561, msg002134, msg002138, 
msg002925, msg002151, msg002287, msg002555, msg002010, 
msg002002, msg002290, msg001537, msg002005, msg002147, 
msg002145, msg002698, msg001592, msg001810, msg002690, 
msg002691, msg001911, msg001910, msg002693, msg001812, 
msg001817, msg001547, msg002012, msg002015, msg002941, 
msg001688, msg002018, msg002684, msg002944, msg001540, 
msg002686, msg001541, msg002946, msg002688, msg001584, 
msg002948]

[zk: localhost:2181(CONNECTED) 7] delete 
/xpe/queues/3bd7851e79381ef4bfd1a5857b5e34c04e5159e5/msgs/msg002948
Node does not exist: 
/xpe/queues/3bd7851e79381ef4bfd1a5857b5e34c04e5159e5/msgs/msg002948

When I perfor