zookeeper-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From David Capwell <dcapw...@gmail.com>
Subject Re: mixing persitent and persitent sequential throws NodeExists for sequential mode
Date Wed, 01 Jul 2015 18:23:05 GMT
Here is the jira for this: https://issues.apache.org/jira/browse/ZOOKEEPER-2226

On Wed, Jul 1, 2015 at 11:16 AM, David Capwell <dcapwell@gmail.com> wrote:
> Thanks, I was walking the ZK code and thats what it looked like:
>
> PrepRequestProcessor:
>
> String parentPath = path.substring(0, lastSlash);
> ChangeRecord parentRecord = getRecordForPath(parentPath);
> ...
>
> int parentCVersion = parentRecord.stat.getCversion();
> CreateMode createMode =
>     CreateMode.fromFlag(createRequest.getFlags());
> if (createMode.isSequential()) {
>     path = path + String.format(Locale.ENGLISH, "%010d", parentCVersion);
> }
>
> Because of that, I have to rethink what im doing, but ill file a jira
> because of the exception thrown.
>
>
> On Wed, Jul 1, 2015 at 11:14 AM, Ivan Kelly <ivank@apache.org> wrote:
>> This occurs because PERSISTENT_SEQUENTIAL uses the cxid of the parent znode
>> to generate the sequence number. The cxid is incremented every time a child
>> is created under the znode, so if you have already created 4 nodes, that
>> explains why the next sequential node you create has id 4, counting from 0.
>>
>> The fact that this throws a NodeExists exception could be considered a bug.
>> In some cases it would be better to keep incrementing the sequence until
>> you get a successful create.
>>
>> -Ivan
>>
>> On Wed, Jul 1, 2015 at 7:54 PM David Capwell <dcapwell@gmail.com> wrote:
>>
>>> I have the following code (in curator):
>>>
>>> int id =
>>> extractId(client.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT_SEQUENTIAL).forPath(prefix,
>>> data));
>>>
>>> and
>>>
>>>
>>> client.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT).forPath(path(id),
>>> data);
>>>
>>>
>>> The first part joins our cluster and gets a id from zookeeper.  The
>>> second call will create a znode that looks like a znode above.
>>>
>>> The reason I do this is that I would like for ops to be able to define
>>> the ids when they want and not always have to (other code will
>>> "setData" one of the paths defined above, leaving out since thats not
>>> having issues).
>>>
>>> I created several test cases and saw that this was working, but
>>> created another test and got a unexpected exception (I put the
>>> exception at the bottom)
>>>
>>> Here is the test:
>>>
>>> create 4 PERSISTENT znodes with ids 1, 2, 3, 4
>>> create 1 PERSISTENT_SEQUENTIAL znode (expecting id = 0)
>>>
>>> Here is the error I saw
>>>
>>> INFO 2015-07-01 10:46:46,349 [ProcessThread(sid:0 cport:-1):]
>>> [PrepRequestProcessor] [line 627] Got user-level KeeperException when
>>> processing sessionid:0x14e4aba4d490000 type:create cxid:0x25 zxid:0xe
>>> txntype:-1 reqpath:n/a Error
>>> Path:/test/MembershipTest/replaceFourRegisterOne/member-0000000004
>>> Error:KeeperErrorCode = NodeExists for
>>> /test/MembershipTest/replaceFourRegisterOne/member-0000000004
>>>
>>> org.apache.zookeeper.KeeperException$NodeExistsException:
>>> KeeperErrorCode = NodeExists for
>>> /test/MembershipTest/replaceFourRegisterOne/member-
>>> ...
>>> Caused by: org.apache.zookeeper.KeeperException$NodeExistsException:
>>> KeeperErrorCode = NodeExists for
>>> /test/MembershipTest/replaceFourRegisterOne/member-
>>> at org.apache.zookeeper.KeeperException.create(KeeperException.java:119)
>>> at org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
>>> at org.apache.zookeeper.ZooKeeper.create(ZooKeeper.java:783)
>>> at
>>> org.apache.curator.framework.imps.CreateBuilderImpl$11.call(CreateBuilderImpl.java:688)
>>> at
>>> org.apache.curator.framework.imps.CreateBuilderImpl$11.call(CreateBuilderImpl.java:672)
>>> at org.apache.curator.RetryLoop.callWithRetry(RetryLoop.java:107)
>>> at
>>> org.apache.curator.framework.imps.CreateBuilderImpl.pathInForeground(CreateBuilderImpl.java:668)
>>> at
>>> org.apache.curator.framework.imps.CreateBuilderImpl.protectedPathInForeground(CreateBuilderImpl.java:453)
>>> at
>>> org.apache.curator.framework.imps.CreateBuilderImpl.forPath(CreateBuilderImpl.java:443)
>>> at
>>> org.apache.curator.framework.imps.CreateBuilderImpl.forPath(CreateBuilderImpl.java:44)
>>>
>>> Why is the sequential mode failing (and why does the log say its
>>> trying with 4, but error doesn't include id)?
>>>
>>>
>>> Versions:
>>> Curator: 2.6.1
>>> Zookeeper: 3.4.5
>>>
>>> Thanks for your time reading this email.
>>>

Mime
View raw message