Unusual exception
I started seeing a bunch of these exceptions. What do these mean? 2010-10-13 14:01:33,426 - WARN [NIOServerCxn.Factory: 0.0.0.0/0.0.0.0:5001:nioserverc...@606] - EndOfStreamException: Unable to read additional data from client sessionid 0x0, likely client has closed socket 2010-10-13 14:01:33,426 - INFO [NIOServerCxn.Factory: 0.0.0.0/0.0.0.0:5001:nioserverc...@1286] - Closed socket connection for client /10.138.34.195:55738 (no session established for client) 2010-10-13 14:01:33,426 - DEBUG [CommitProcessor:1:finalrequestproces...@78] - Processing request:: sessionid:0x12b9d1f8b907a44 type:closeSession cxid:0x0 zxid:0x600193996 txntype:-11 reqpath:n/a 2010-10-13 14:01:33,427 - WARN [NIOServerCxn.Factory: 0.0.0.0/0.0.0.0:5001:nioserverc...@606] - EndOfStreamException: Unable to read additional data from client sessionid 0x12b9d1f8b907a5d, likely client has closed socket 2010-10-13 14:01:33,427 - INFO [NIOServerCxn.Factory: 0.0.0.0/0.0.0.0:5001:nioserverc...@1286] - Closed socket connection for client /10.138.34.195:55979 which had sessionid 0x12b9d1f8b907a5d 2010-10-13 14:01:33,427 - DEBUG [QuorumPeer:/0.0.0.0:5001 :commitproces...@159] - Committing request:: sessionid:0x52b90ab45bd51af type:createSession cxid:0x0 zxid:0x600193cf9 txntype:-10 reqpath:n/a 2010-10-13 14:01:33,427 - DEBUG [NIOServerCxn.Factory: 0.0.0.0/0.0.0.0:5001:nioserverc...@1302] - ignoring exception during output shutdown java.net.SocketException: Transport endpoint is not connected at sun.nio.ch.SocketChannelImpl.shutdown(Native Method) at sun.nio.ch.SocketChannelImpl.shutdownOutput(SocketChannelImpl.java:651) at sun.nio.ch.SocketAdaptor.shutdownOutput(SocketAdaptor.java:368) at org.apache.zookeeper.server.NIOServerCnxn.closeSock(NIOServerCnxn.java:1298) at org.apache.zookeeper.server.NIOServerCnxn.close(NIOServerCnxn.java:1263) at org.apache.zookeeper.server.NIOServerCnxn.doIO(NIOServerCnxn.java:609) at org.apache.zookeeper.server.NIOServerCnxn$Factory.run(NIOServerCnxn.java:262) 2010-10-13 14:01:33,428 - DEBUG [NIOServerCxn.Factory: 0.0.0.0/0.0.0.0:5001:nioserverc...@1310] - ignoring exception during input shutdown java.net.SocketException: Transport endpoint is not connected at sun.nio.ch.SocketChannelImpl.shutdown(Native Method) at sun.nio.ch.SocketChannelImpl.shutdownInput(SocketChannelImpl.java:640) at sun.nio.ch.SocketAdaptor.shutdownInput(SocketAdaptor.java:360) at org.apache.zookeeper.server.NIOServerCnxn.closeSock(NIOServerCnxn.java:1306) at org.apache.zookeeper.server.NIOServerCnxn.close(NIOServerCnxn.java:1263) at org.apache.zookeeper.server.NIOServerCnxn.doIO(NIOServerCnxn.java:609) at org.apache.zookeeper.server.NIOServerCnxn$Factory.run(NIOServerCnxn.java:262) 2010-10-13 14:01:33,428 - WARN [NIOServerCxn.Factory: 0.0.0.0/0.0.0.0:5001:nioserverc...@606] - EndOfStreamException: Unable to read additional data from client sessionid 0x0, likely client has closed socket 2010-10-13 14:01:33,428 - INFO [NIOServerCxn.Factory: 0.0.0.0/0.0.0.0:5001:nioserverc...@1286] - Closed socket connection for client /10.138.34.195:55731 (no session established for client)
Re: Retrying sequential znode creation
Patrick, What are these hurdles? The last comment on ZK-22 was last winter. Back then, it didn't sound like it was going to be that hard. On Wed, Oct 13, 2010 at 12:08 PM, Patrick Hunt wrote: > 22 would help with this issue > https://issues.apache.org/jira/browse/ZOOKEEPER-22 > however there are some real hurdles to implementing 22 successfully. >
Re: Retrying sequential znode creation
On Wed, Oct 13, 2010 at 5:58 AM, Vishal K wrote: > > However, gets trickier because there is no explicit way (to my knowledge) > to > get CreateMode for a znode. As a result, we cannot tell whether a node is > sequential or not. > Sequentials are really just regular znodes with fancy naming applied by the cluster at create time, subsequently it makes no distinction. Using the format of the name would be the be only/best way I know if you want to distinguish yourself. (or put some data into the znode itself) 22 would help with this issue https://issues.apache.org/jira/browse/ZOOKEEPER-22 however there are some real hurdles to implementing 22 successfully. Patrick > > Thanks. > -Vishal > > > On Tue, Oct 12, 2010 at 5:36 PM, Ted Dunning > wrote: > > > Yes. This is indeed a problem. I generally try to avoid sequential > nodes > > unless they are ephemeral and if I get an error on > > creation, I generally have to either tear down the connection (losing all > > other ephemeral nodes in the process) or scan through > > all live nodes trying to determine if mine got created. Neither is a > very > > acceptable answer so I try to avoid the problem. > > > > Your UUID answer is one option. At least you know what file got created > > (or > > not) and with good naming you can pretty much guarantee no collisions. > You > > don't have to scan all children since you can simply check for the > > existence > > of the file of interest. > > > > There was a JIRA filed that was supposed to take care of this problem, > but > > I > > don't know the state of play there. > > > > On Tue, Oct 12, 2010 at 12:11 PM, Vishal K wrote: > > > > > Hi, > > > > > > What is the best approach to have an idempotent create() operation for > a > > > sequential node? > > > > > > Suppose a client is trying to create a sequential node and it gets a > > > ConnectionLoss KeeperException, it cannot know for sure whether the > > request > > > succeeded or not. If in the meantime, the client's session is > > > re-established, the client would like to create a sequential znode > again. > > > However, the client needs to know if its earlier request has succeeded > or > > > not. If it did, then the client does not need to retry. To my > > understanding > > > ZooKeeper does not provide this feature. Can someone confirm this? > > > > > > External to ZooKeeper, the client can either set a unique UUID in the > > path > > > to the create call or write the UUID as part of its data. Before > > retrying, > > > it can read back all the children of the parent znode and go through > the > > > list to determine if its earlier request had succeeded. This doesn't > > sound > > > that appealing to me. > > > > > > I am guessing this is a common problem that many would have faced. Can > > > folks > > > give a feedback on what their approach was? > > > > > > Thanks. > > > -Vishal > > > > > >
Re: Membership using ZK
FYI: http://wiki.apache.org/hadoop/ZooKeeper/Troubleshooting On Tue, Oct 12, 2010 at 2:23 PM, Benjamin Reed wrote: > yes, your watcher objects will get the connectionloss event and eventually > the session expired event. > > ben > > > On 10/12/2010 10:57 AM, Avinash Lakshman wrote: > >> Would my watcher get invoked on this ConnectionLoss event? If so I am >> thinking I will check for KeeperState.Disconnected and reset my state. Is >> my >> understanding correct? Please advice. >> >> Thanks >> Avinash >> >> On Tue, Oct 12, 2010 at 10:45 AM, Benjamin Reed >> wrote: >> >> ZooKeeper considers a client dead when it hasn't heard from that client >>> during the timeout period. clients make sure to communicate with >>> ZooKeeper >>> at least once in 1/3 the timeout period. if the client doesn't hear from >>> ZooKeeper in 2/3 the timeout period, the client will issue a >>> ConnectionLoss >>> event and cause outstanding requests to fail with a ConnectionLoss. >>> >>> So, if ZooKeeper decides a process is dead, the process will get a >>> ConnectionLoss event. Once ZooKeeper decides that a client is dead, if >>> the >>> client reconnects, the client will get a SessionExpired. Once a session >>> is >>> expired, the expired handle will become useless, so no new requests, no >>> watches, etc. >>> >>> The bottom line is if your process gets a process expired, you need to >>> treat that process as expired and recover by creating a new zookeeper >>> handle >>> (possibly by restarting the process) and resetup your state. >>> >>> ben >>> >>> >>> On 10/12/2010 09:54 AM, Avinash Lakshman wrote: >>> >>> This is what I have going: I have a bunch of 200 nodes come up and create an ephemeral entry under a znode names /Membership. When nodes are detected dead the node associated with the dead node under /Membership is deleted and watch delivered to the rest of the members. Now there are circumstances a node A is deemed dead while the process is still up and running on A. It is a false detection which I need to probably deal with. How do I deal with this situation? Over time false detections delete all the entries underneath the /Membership znode even though all processes are up and running. So my questions are: Would the watches be pushed out to the node that is falsely deemed dead? If so I can have that process recreate the ephemeral znode underneath /Membership. If a node leaves a watch and then truly crashes. When it comes back up would it get watches it missed during the interim period? In any case how do watches behave in the event of false/true failure detection? Thanks A >>> >
Re: What does this mean?
On Mon, Oct 11, 2010 at 4:16 PM, Avinash Lakshman < avinash.laksh...@gmail.com> wrote: > tickTime = 2000, initLimit = 3000 and the data is around 11GB this is log + > snapshot. So if I need to add a new observer can I transfer state from the > ensemble manually before starting it? If so which files do I need to > transfer? > > You can't really do it manually. As part of the "bring up" process for a server it communicates with the current leader and downloads the appropriate data (either a diff of the recent changes or a full snapshot if too far behind ). Try increasing your initLimit to 15 or so (btw, that' in ticks, not milliseconds, so if you have 3000 now that's probably not the issue ;-) ). You might also want to increase the syncLimit at the same time. Here's from the sample conf that ships with the release: # The number of ticks that the initial # synchronization phase can take initLimit=10 # The number of ticks that can pass between # sending a request and getting an acknowledgement syncLimit=5 Patrick > Thanks > > On Mon, Oct 11, 2010 at 10:16 AM, Benjamin Reed > wrote: > > > how big is your data? you may be running into the problem where it takes > > too long to do the state transfer and times out. check the initLimit and > the > > size of your data. > > > > ben > > > > > > On 10/10/2010 08:57 AM, Avinash Lakshman wrote: > > > >> Thanks Ben. I am not mixing processes of different clusters. I just > double > >> checked that. I have ZK deployed in a 5 node cluster and I have 20 > >> observers. I just started the 5 node cluster w/o starting the observers. > I > >> still the same issue. Now my cluster won't start up. So what is the > >> correct > >> workaround to get this going? How can I find out who the leader is and > who > >> the follower to get more insight? > >> > >> Thanks > >> A > >> > >> On Sun, Oct 10, 2010 at 8:33 AM, Benjamin Reed > >> wrote: > >> > >> this usually happens when a follower closes its connection to the > leader. > >>> it is usually caused by the follower shutting down or failing. you may > >>> get > >>> further insight by looking at the follower logs. you should really run > >>> with > >>> timestamps on so that you can correlate the logs of the leader and > >>> follower. > >>> > >>> on thing that is strange is the wide divergence between zxid of > follower > >>> and leader. are you mixing processes of different clusters? > >>> > >>> ben > >>> > >>> > >>> From: Avinash Lakshman [avinash.laksh...@gmail.com] > >>> Sent: Sunday, October 10, 2010 8:18 AM > >>> To: zookeeper-user > >>> Subject: What does this mean? > >>> > >>> I see this exception and the servers not doing anything. > >>> > >>> java.io.IOException: Channel eof > >>>at > >>> > >>> > >>> > org.apache.zookeeper.server.quorum.QuorumCnxManager$RecvWorker.run(QuorumCnxManager.java:630) > >>> ERROR - 124554051584(higestZxid)> 21477836646(next log) for type -11 > >>> WARN - Sending snapshot last zxid of peer is 0xe zxid of > leader > >>> is > >>> 0x1e > >>> WARN - Sending snapshot last zxid of peer is 0x18 zxid of > leader > >>> is > >>> 0x1eg > >>> WARN - Sending snapshot last zxid of peer is 0x5002dc766 zxid of > leader > >>> is > >>> 0x1e > >>> WARN - Sending snapshot last zxid of peer is 0x1c zxid of > leader > >>> is > >>> 0x1e > >>> ERROR - Unexpected exception causing shutdown while sock still open > >>> java.net.SocketException: Broken pipe > >>>at java.net.SocketOutputStream.socketWrite0(Native Method) > >>>at > >>> java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:92) > >>>at > java.net.SocketOutputStream.write(SocketOutputStream.java:136) > >>>at > >>> java.io.BufferedOutputStream.flushBuffer(BufferedOutputStream.java:65) > >>>at > >>> java.io.BufferedOutputStream.write(BufferedOutputStream.java:78) > >>>at java.io.DataOutputStream.writeInt(DataOutputStream.java:180) > >>>at > >>> > org.apache.jute.BinaryOutputArchive.writeInt(BinaryOutputArchive.java:55) > >>>at > >>> > org.apache.zookeeper.data.StatPersisted.serialize(StatPersisted.java:116) > >>>at > >>> org.apache.zookeeper.server.DataNode.serialize(DataNode.java:167) > >>>at > >>> > >>> > >>> > org.apache.jute.BinaryOutputArchive.writeRecord(BinaryOutputArchive.java:123) > >>>at > >>> org.apache.zookeeper.server.DataTree.serializeNode(DataTree.java:967) > >>>at > >>> org.apache.zookeeper.server.DataTree.serializeNode(DataTree.java:982) > >>>at > >>> org.apache.zookeeper.server.DataTree.serializeNode(DataTree.java:982) > >>>at > >>> org.apache.zookeeper.server.DataTree.serializeNode(DataTree.java:982) > >>>at > >>> org.apache.zookeeper.server.DataTree.serialize(DataTree.java:1031) > >>>at > >>> > >>> > >>> > org.apache.zookeeper.server.util.SerializeUtils.serializeSnapshot(SerializeUtils.java:104) >
Re: Retrying sequential znode creation
Hi Ted, Thanks for the reply. I prefer to store the UUID in the data itself. I thought about storing it in znode names, but the downside of this approach is that when a Watcher receives an event, it needs to strip off the UUID. Otherwise, it wont be possible to figure out the order znodes (e.g., using sort()). Also, in our case, there are very few sequential children. So listing and reading all children is not a big overhead. However, gets trickier because there is no explicit way (to my knowledge) to get CreateMode for a znode. As a result, we cannot tell whether a node is sequential or not. Thanks. -Vishal On Tue, Oct 12, 2010 at 5:36 PM, Ted Dunning wrote: > Yes. This is indeed a problem. I generally try to avoid sequential nodes > unless they are ephemeral and if I get an error on > creation, I generally have to either tear down the connection (losing all > other ephemeral nodes in the process) or scan through > all live nodes trying to determine if mine got created. Neither is a very > acceptable answer so I try to avoid the problem. > > Your UUID answer is one option. At least you know what file got created > (or > not) and with good naming you can pretty much guarantee no collisions. You > don't have to scan all children since you can simply check for the > existence > of the file of interest. > > There was a JIRA filed that was supposed to take care of this problem, but > I > don't know the state of play there. > > On Tue, Oct 12, 2010 at 12:11 PM, Vishal K wrote: > > > Hi, > > > > What is the best approach to have an idempotent create() operation for a > > sequential node? > > > > Suppose a client is trying to create a sequential node and it gets a > > ConnectionLoss KeeperException, it cannot know for sure whether the > request > > succeeded or not. If in the meantime, the client's session is > > re-established, the client would like to create a sequential znode again. > > However, the client needs to know if its earlier request has succeeded or > > not. If it did, then the client does not need to retry. To my > understanding > > ZooKeeper does not provide this feature. Can someone confirm this? > > > > External to ZooKeeper, the client can either set a unique UUID in the > path > > to the create call or write the UUID as part of its data. Before > retrying, > > it can read back all the children of the parent znode and go through the > > list to determine if its earlier request had succeeded. This doesn't > sound > > that appealing to me. > > > > I am guessing this is a common problem that many would have faced. Can > > folks > > give a feedback on what their approach was? > > > > Thanks. > > -Vishal > > >