Extending KeeperException could be a problem is current code expect an IOException (I don't know) or a specific KeeperException. Extending the current exception with a new type should be fine.
On Fri, Jan 8, 2021 at 3:01 PM Michael Han <h...@apache.org> wrote: > 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 <nkal...@cloudera.com.invalid > > > 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 <ihuizhi...@gmail.com> 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 <ihuizhi...@gmail.com> > 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 <ihuizhi...@gmail.com> > > 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 > > > > > > > > > > On Fri, Jan 8, 2021 at 12:01 AM Ted Dunning <ted.dunn...@gmail.com > > > > > wrote: > > > > > > > > > > > > Let's be clear from the start, storing large data objects in > > > Zookeeper is > > > > > > strongly discouraged. If you want to store large objects with > good > > > > > > consistency models, store the data in something else (like a > > > distributed > > > > > > file system or key value store), commit the data and then use ZK > to > > > provide > > > > > > a reference to that data. Zookeeper is intended for coordination, > > > not data > > > > > > storage. It is not a reasonable alternative to a noSQL database. > > > > > > > > > > > > That said, good and informative error messages are always useful > > and > > > are > > > > > > better than anonymous errors. Even if the connection is closed > > > related to > > > > > > the error, it would be nice to give some decent feedback. > > > > > > > > > > > > On the other hand, the error that you are seeing sounds like your > > > client > > > > > > and your server have inconsistent settings for the maximum jute > > > buffer > > > > > > size. If the client has a larger setting than the server, then > the > > > server > > > > > > will run out of buffer space before reading the entire request. > To > > > the > > > > > > server, this will look like a network error and there is little > > that > > > the > > > > > > server can do to recover the connection safely because some bytes > > may > > > > > > already have been lost due to the short read. As such closing the > > > > > > connection is pretty much all that can be done. > > > > > > > > > > > > If buffer lengths on client and server match and the client > tries a > > > long > > > > > > write, I believe the write will fail on the client side with a > much > > > more > > > > > > descriptive message. > > > > > > > > > > > > One thing that could plausibly be done would be to enhance the > > > initial > > > > > > handshake between client and server so mismatch in buffer sizes > are > > > > > > detected more aggressively. Since a length could be exchanged in > a > > > fixed > > > > > > size, this could be done while keeping the connection healthy > which > > > would, > > > > > > in turn, allow a useful error message that drives to the true > > source > > > of the > > > > > > error (i.e. the configuration). This would, however, require a > > > protocol > > > > > > change which is always a sensitive change. > > > > > > > > > > > > Can you determine if the root cause of your problem is > inconsistent > > > > > > settings between client and server? > > > > > > > > > > > > > > > > > > > > > > > > On Thu, Jan 7, 2021 at 10:27 PM Huizhi Lu <h...@apache.org> > wrote: > > > > > > > > > > > > > Hi ZK Experts, > > > > > > > > > > > > > > I would like to ask a quick question. As we know, assume we are > > > using > > > > > > > the default 1 MB jute.maxbuffer, if a zk client tries to write > a > > > large > > > > > > > znode > 1MB, the server will fail it. Server will log "Len > error" > > > and > > > > > > > close the connection. The client will receive a connection > loss. > > > In a > > > > > > > third party ZkClient lib (eg. I0IZkClient), it'll keep retrying > > the > > > > > > > operation upon connection loss. And this forever retrying might > > > have a > > > > > > > chance to take down the zk server. > > > > > > > > > > > > > > I believe the zk community must have considered such a > > situation. I > > > > > > > wonder why zk server does not handle the error a bit better and > > > send a > > > > > > > clearer response to the client, eg. > > KeeperException.PacketLenError > > > > > > > (and zk server does not really have to close the connection), > so > > > the > > > > > > > client knows the error is non retryable. I think there must be > > some > > > > > > > reasons I am not aware of that zk does not offer it, so I'd > like > > to > > > > > > > ask here. Or is there any ticket/email thread that has > discussed > > > this? > > > > > > > > > > > > > > Maybe zk would expect the app client to handle connection loss > > > > > > > appropriately, eg. by having a retry strategy(backoff retry, > > > limiting > > > > > > > the retry count, etc.). Is this what zk would expect, instead > of > > > > > > > returning a PacketLenError exception? > > > > > > > > > > > > > > Really appreciate any input. > > > > > > > > > > > > > > > > > > > > > Best, > > > > > > > -Huizhi > > > > > > > > > > > > >