In this case, I guess we have to check the protocol version and depending on the version deserialize differently.
-Flavio > On 11 May 2016, at 13:21, Mohammad arshad <[email protected]> wrote: > > Thanks Flavio, > > I think CreateTxn kind of solution cannot be applied in this scenario > I can case of transaction the records are separable but in case of snapshot > records are continuous > In this scenario, cuser de-serialization eats into the next record which > causes bigger problem > > Here is exception I got after adding cuser and running the ZooKeeper on old > data > java.io.IOException: Unreasonable length = 796553071 > at > org.apache.jute.BinaryInputArchive.checkLength(BinaryInputArchive.java:127) > at > org.apache.jute.BinaryInputArchive.readString(BinaryInputArchive.java:81) > at > org.apache.zookeeper.data.StatPersisted.deserialize(StatPersisted.java:143) > at org.apache.zookeeper.server.DataNode.deserialize(DataNode.java:173) > at > org.apache.jute.BinaryInputArchive.readRecord(BinaryInputArchive.java:99) > at org.apache.zookeeper.server.DataTree.deserialize(DataTree.java:1278) > > > Best Regards > Mohammad Arshad > HUAWEI TECHNOLOGIES CO.LTD. > Huawei Tecnologies India Pvt. Ltd. > Near EPIP Industrial Area, Kundalahalli Village > Whitefield, Bangalore-560066 > www.huawei.com > ----------------------------------------------------------------------------------------------------------------- > This e-mail and its attachments contain confidential information from HUAWEI, > which > is intended only for the person or entity whose address is listed above. Any > use of the > information contained herein in any way (including, but not limited to, total > or partial > disclosure, reproduction, or dissemination) by persons other than the > intended > recipient(s) is prohibited. If you receive this e-mail in error, please > notify the sender by > phone or email immediately and delete it! > > -----Original Message----- > From: Flavio Junqueira [mailto:[email protected]] > Sent: 11 May 2016 16:02 > To: [email protected] > Cc: Zookeeper > Subject: Re: How to extend ZooKeeper data structure without breaking the > backward compatibility > > In the past, we simply created two versions of the data structure: > > class CreateTxnV0 { > ustring path; > buffer data; > vector<org.apache.zookeeper.data.ACL> acl; > boolean ephemeral; > } > class CreateTxn { > ustring path; > buffer data; > vector<org.apache.zookeeper.data.ACL> acl; > boolean ephemeral; > int parentCVersion; > } > > and deal with it in the code. It is not ideal and the serialization framework > is actually a pretty old one. > > -Flavio > >> On 11 May 2016, at 11:14, Mohammad arshad <[email protected]> wrote: >> >> Hello Everyone >> >> is there any way to extend ZooKeeper data structure without breaking the >> backward compatibility. >> Suppose I want to add cuser field in StatPersisted class StatPersisted >> { >> ........ >> ustring cuser; //user who create the node >> } >> This is fine with fresh ZooKeeper server installation, where ZooKeeper >> data is created freshly But this causes problem while de-serializing when >> server is new but points to old ZooKeeper data. >> Here by ZooKeeper data I mean snapshot and transaction log >> >> In protocol buffers a field can be optional but in jute optional field is >> not supported. >> is there any way to overcome this jute limitation? any work around? >> >> Best Regards >> Mohammad Arshad >> HUAWEI TECHNOLOGIES CO.LTD. >> Huawei Tecnologies India Pvt. Ltd. >> Near EPIP Industrial Area, Kundalahalli Village Whitefield, >> Bangalore-560066 www.huawei.com<http://www.huawei.com/> >> ---------------------------------------------------------------------- >> ------------------------------------------- >> This e-mail and its attachments contain confidential information from >> HUAWEI, which is intended only for the person or entity whose address >> is listed above. Any use of the information contained herein in any >> way (including, but not limited to, total or partial disclosure, >> reproduction, or dissemination) by persons other than the intended >> recipient(s) is prohibited. If you receive this e-mail in error, >> please notify the sender by phone or email immediately and delete it! >> >
