You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@zookeeper.apache.org by Manos Kapritsos <ma...@cs.utexas.edu> on 2009/02/10 23:15:51 UTC

setACL semantics

Hi all,

I have a question about the way setACL functions. It seems that the 
PreRequestProcessor handles all kinds of requests the same, checks the 
validity of the corresponding ACL, and enqueues them to Sync and Final 
processors. Maybe I am missing something here, but this behaviour seems 
weird. What if a setACL request comes, setting the ACL of a path (e.g. / 
) to an IP (e.g. 128.62.32.32) , instead of its old value (e.g. World). 
This request will pass the ACL check, and will be enqueued to be 
processed by the next processors. Assume that the next request is a 
getData("/") from an IP other than 128.62.32.32. If this request is 
processed by the PreRequestProcessor before the setACL request is 
processed by the FinalRequestProcessor, then it will pass the ACL check 
(which it should not, since it came after the setACL request). It seems 
that there is a race condition here that should not exist.

Let me know if this is actually the case or I am missing something. I am 
using version 3.0.1 of the code.

Thank you,
Manos

RE: setACL semantics

Posted by Benjamin Reed <br...@yahoo-inc.com>.
you may not have noticed, but in the prep request processor we transform zookeeper requests into idempotent transactions. this helps alot for some of our recovery scenarios, but to pull it off we have to be able to look into the future and figure out what the state of the system will look like when the transaction executes. remember we could have hundreds if not thousands of requests in flight. so, for the requests that change state, we must do all the checks ahead of time in the prep request processor by looking into the future. check out the following:

"/foo" has "v0" version 1 with open acl.

If we get a pipeline of requests that all get through the prep request processor before the first requests hits the final request processor, what should happen? The pipelining must be completely transparent. Everything must act as if the requests are performed one by one without any processing.

1) setData("/foo", "v1", 1);
2) setData("/foo", "v2", -1);
3) setData("/foo", "v3", 1);
4) setACL("/foo", restricted permission);
5) setData("/foo", "v4", -1);

Step 1) this operation succeeds. "/foo" will be set to "v1" version 2.
Step 2) this operation succeeds. "/foo" will be set to "v2" version 3.
Step 3) this operation fails. even though /foo is still at version 1 in the data tree, we have the look ahead list that shows that when 3) executes /foo will be at version 3, so we fail due to a bad version
Step 4) this operation succeeds. /foo now has a restricted permission.
Step 5) this operation fails. even though /foo still has open acl in the data tree, we have the look ahead list that shows that when 5) executes /foo will have a restricted permission.

so in the end /foo will have "v2" version 3 with a restricted permission, which would be exactly the same as if the requests were not pipelined.

no waiting, no problem.

because we stage the result of the execution in the prep request processor, we have to do the check there. otherwise, we run into major race conditions.

ben

________________________________________
From: Manos Kapritsos [manos@cs.utexas.edu]
Sent: Tuesday, February 10, 2009 6:30 PM
To: zookeeper-dev@hadoop.apache.org
Subject: Re: setACL semantics

You are right about the getData, and it was probably a wrong choice.
Both the getData and getChildren have an extra ACL check in the
FinalRequestProcessor. My question is pretty much this: shouldn't the
other request types have such a check as well? What if client B has
issued a setData()? setData is only ACL-checked in PrepRequestProcessor,
so it may be the case that it succeeds, while client A's the setAcl()
has been ordered to be executed first.

It seems that an ACL check at the FinalRequestProcessor for most request
types would solve this issue. As it is there only for getData and
getChildren, I wonder why that is and if scenarios as the one above are
considered important.

By the way, does Zookeeper require clients to wait for the reply for
setAcl()? If not, then the "single client, single server" model would
face the problem even more obviously.

Thank you for the replies,
Manos

Benjamin Reed wrote:
> Manos,
>
> in your scenario the getData will not succeed. the acl will be checked in the FinalRequestProcessor after the setAcl has passed through the final request processor. execution only happens in the final request processor and always happens in order.
>
> the only way a getData will succeed after a setAcl succeeds if it is done on a lagging follower, but in that case it will see a lagging view of the system. consider the following:
>
> /foo has an open ACL
>
> client A: setAcl("/foo", restricted ACL); setData("/foo", new data);
> client B: getData("/foo");
>
> no matter what happens, what the delays are, if B is faster or slower than A, client B will not see "new data". it will either pass the ACL check and see the old data or it will fail with a permission exception.
>
> ben
> ________________________________________
> From: Manos Kapritsos [manos@cs.utexas.edu]
> Sent: Tuesday, February 10, 2009 4:34 PM
> To: zookeeper-dev@hadoop.apache.org
> Subject: Re: setACL semantics
>
> More or less, yes.
> If the requests are from the same client, then you say that there will
> not be a problem? I guess that is true if you always wait for the
> response of the first request in order to execute the second. I am not
> sure if that is a requirement for all Zookeeper client implementations.
> As for two different clients (which was the case I was thinking about),
> this seems to be a problem. I will agree that (if clients only have one
> outstanding request) the two requests are concurrent and either order of
> execution is considered to be acceptable, but it could be that two
> different replicas receive the two requests in the same order, but
> effectively execute them in a different order. In any case, it feels
> wrong (at least to me) that a getData would succeed when a setACL that
> prevents it has already been accepted to be processed.
>
> Manos
>
> Mahadev Konar wrote:
>
>> Hi Manos,
>>  If the setAcl and getdata are from the same client then they are all
>> handled in order. So you would get an unauthroized exception when you do a
>> getdata.
>>
>>
>> If two diff clients do setacl and getdata it might be that the getdata in
>> your case will succeed before the setacl returns on the first client.
>>
>> Is that what you meant?
>>
>> mahadev
>>
>>
>> On 2/10/09 2:15 PM, "Manos Kapritsos" <ma...@cs.utexas.edu> wrote:
>>
>>
>>
>>> Hi all,
>>>
>>> I have a question about the way setACL functions. It seems that the
>>> PreRequestProcessor handles all kinds of requests the same, checks the
>>> validity of the corresponding ACL, and enqueues them to Sync and Final
>>> processors. Maybe I am missing something here, but this behaviour seems
>>> weird. What if a setACL request comes, setting the ACL of a path (e.g. /
>>> ) to an IP (e.g. 128.62.32.32) , instead of its old value (e.g. World).
>>> This request will pass the ACL check, and will be enqueued to be
>>> processed by the next processors. Assume that the next request is a
>>> getData("/") from an IP other than 128.62.32.32. If this request is
>>> processed by the PreRequestProcessor before the setACL request is
>>> processed by the FinalRequestProcessor, then it will pass the ACL check
>>> (which it should not, since it came after the setACL request). It seems
>>> that there is a race condition here that should not exist.
>>>
>>> Let me know if this is actually the case or I am missing something. I am
>>> using version 3.0.1 of the code.
>>>
>>> Thank you,
>>> Manos
>>>
>>>


Re: setACL semantics

Posted by Mahadev Konar <ma...@yahoo-inc.com>.
Hi Manos,


On 2/10/09 6:30 PM, "Manos Kapritsos" <ma...@cs.utexas.edu> wrote:

> You are right about the getData, and it was probably a wrong choice.
> Both the getData and getChildren have an extra ACL check in the
> FinalRequestProcessor. My question is pretty much this: shouldn't the
> other request types have such a check as well? What if client B has
> issued a setData()? setData is only ACL-checked in PrepRequestProcessor,
> so it may be the case that it succeeds, while client A's the setAcl()
> has been ordered to be executed first.
In this case both the setdata and setacl will go through the
preprequestporcesssor. Remember that all the updates are ordered in
preprequestprocessor.

Lets say this is the order:
Setdata by client B and then setACl by client A.
In this case, 
The data will be set by client B and then the acl will change by client A.

In the other case :
Setacl by client A and setdata by clientB

The account of acl being changed by client A is kept in memory at
preprequestporcessor, so the next request by clientB to setdata will fail
since the acl is expected to be changed by clientA.


> 
> It seems that an ACL check at the FinalRequestProcessor for most request
> types would solve this issue. As it is there only for getData and
> getChildren, I wonder why that is and if scenarios as the one above are
> considered important.
The above explanation should answer this.

> 
> By the way, does Zookeeper require clients to wait for the reply for
> setAcl()? If not, then the "single client, single server" model would
> face the problem even more obviously.
Sorry, I didn't understand this. Whats your reasoning behind this?

mahadev
> 
> Thank you for the replies,
> Manos
> 
> Benjamin Reed wrote:
>> Manos,
>> 
>> in your scenario the getData will not succeed. the acl will be checked in the
>> FinalRequestProcessor after the setAcl has passed through the final request
>> processor. execution only happens in the final request processor and always
>> happens in order.
>> 
>> the only way a getData will succeed after a setAcl succeeds if it is done on
>> a lagging follower, but in that case it will see a lagging view of the
>> system. consider the following:
>> 
>> /foo has an open ACL
>> 
>> client A: setAcl("/foo", restricted ACL); setData("/foo", new data);
>> client B: getData("/foo");
>> 
>> no matter what happens, what the delays are, if B is faster or slower than A,
>> client B will not see "new data". it will either pass the ACL check and see
>> the old data or it will fail with a permission exception.
>> 
>> ben
>> ________________________________________
>> From: Manos Kapritsos [manos@cs.utexas.edu]
>> Sent: Tuesday, February 10, 2009 4:34 PM
>> To: zookeeper-dev@hadoop.apache.org
>> Subject: Re: setACL semantics
>> 
>> More or less, yes.
>> If the requests are from the same client, then you say that there will
>> not be a problem? I guess that is true if you always wait for the
>> response of the first request in order to execute the second. I am not
>> sure if that is a requirement for all Zookeeper client implementations.
>> As for two different clients (which was the case I was thinking about),
>> this seems to be a problem. I will agree that (if clients only have one
>> outstanding request) the two requests are concurrent and either order of
>> execution is considered to be acceptable, but it could be that two
>> different replicas receive the two requests in the same order, but
>> effectively execute them in a different order. In any case, it feels
>> wrong (at least to me) that a getData would succeed when a setACL that
>> prevents it has already been accepted to be processed.
>> 
>> Manos
>> 
>> Mahadev Konar wrote:
>>   
>>> Hi Manos,
>>>  If the setAcl and getdata are from the same client then they are all
>>> handled in order. So you would get an unauthroized exception when you do a
>>> getdata.
>>> 
>>> 
>>> If two diff clients do setacl and getdata it might be that the getdata in
>>> your case will succeed before the setacl returns on the first client.
>>> 
>>> Is that what you meant?
>>> 
>>> mahadev
>>> 
>>> 
>>> On 2/10/09 2:15 PM, "Manos Kapritsos" <ma...@cs.utexas.edu> wrote:
>>> 
>>> 
>>>     
>>>> Hi all,
>>>> 
>>>> I have a question about the way setACL functions. It seems that the
>>>> PreRequestProcessor handles all kinds of requests the same, checks the
>>>> validity of the corresponding ACL, and enqueues them to Sync and Final
>>>> processors. Maybe I am missing something here, but this behaviour seems
>>>> weird. What if a setACL request comes, setting the ACL of a path (e.g. /
>>>> ) to an IP (e.g. 128.62.32.32) , instead of its old value (e.g. World).
>>>> This request will pass the ACL check, and will be enqueued to be
>>>> processed by the next processors. Assume that the next request is a
>>>> getData("/") from an IP other than 128.62.32.32. If this request is
>>>> processed by the PreRequestProcessor before the setACL request is
>>>> processed by the FinalRequestProcessor, then it will pass the ACL check
>>>> (which it should not, since it came after the setACL request). It seems
>>>> that there is a race condition here that should not exist.
>>>> 
>>>> Let me know if this is actually the case or I am missing something. I am
>>>> using version 3.0.1 of the code.
>>>> 
>>>> Thank you,
>>>> Manos
>>>> 
>>>>       
> 


Re: setACL semantics

Posted by Manos Kapritsos <ma...@cs.utexas.edu>.
You are right about the getData, and it was probably a wrong choice. 
Both the getData and getChildren have an extra ACL check in the 
FinalRequestProcessor. My question is pretty much this: shouldn't the 
other request types have such a check as well? What if client B has 
issued a setData()? setData is only ACL-checked in PrepRequestProcessor, 
so it may be the case that it succeeds, while client A's the setAcl() 
has been ordered to be executed first.

It seems that an ACL check at the FinalRequestProcessor for most request 
types would solve this issue. As it is there only for getData and 
getChildren, I wonder why that is and if scenarios as the one above are 
considered important.

By the way, does Zookeeper require clients to wait for the reply for 
setAcl()? If not, then the "single client, single server" model would 
face the problem even more obviously.

Thank you for the replies,
Manos

Benjamin Reed wrote:
> Manos,
>
> in your scenario the getData will not succeed. the acl will be checked in the FinalRequestProcessor after the setAcl has passed through the final request processor. execution only happens in the final request processor and always happens in order.
>
> the only way a getData will succeed after a setAcl succeeds if it is done on a lagging follower, but in that case it will see a lagging view of the system. consider the following:
>
> /foo has an open ACL
>
> client A: setAcl("/foo", restricted ACL); setData("/foo", new data);
> client B: getData("/foo");
>
> no matter what happens, what the delays are, if B is faster or slower than A, client B will not see "new data". it will either pass the ACL check and see the old data or it will fail with a permission exception.
>
> ben
> ________________________________________
> From: Manos Kapritsos [manos@cs.utexas.edu]
> Sent: Tuesday, February 10, 2009 4:34 PM
> To: zookeeper-dev@hadoop.apache.org
> Subject: Re: setACL semantics
>
> More or less, yes.
> If the requests are from the same client, then you say that there will
> not be a problem? I guess that is true if you always wait for the
> response of the first request in order to execute the second. I am not
> sure if that is a requirement for all Zookeeper client implementations.
> As for two different clients (which was the case I was thinking about),
> this seems to be a problem. I will agree that (if clients only have one
> outstanding request) the two requests are concurrent and either order of
> execution is considered to be acceptable, but it could be that two
> different replicas receive the two requests in the same order, but
> effectively execute them in a different order. In any case, it feels
> wrong (at least to me) that a getData would succeed when a setACL that
> prevents it has already been accepted to be processed.
>
> Manos
>
> Mahadev Konar wrote:
>   
>> Hi Manos,
>>  If the setAcl and getdata are from the same client then they are all
>> handled in order. So you would get an unauthroized exception when you do a
>> getdata.
>>
>>
>> If two diff clients do setacl and getdata it might be that the getdata in
>> your case will succeed before the setacl returns on the first client.
>>
>> Is that what you meant?
>>
>> mahadev
>>
>>
>> On 2/10/09 2:15 PM, "Manos Kapritsos" <ma...@cs.utexas.edu> wrote:
>>
>>
>>     
>>> Hi all,
>>>
>>> I have a question about the way setACL functions. It seems that the
>>> PreRequestProcessor handles all kinds of requests the same, checks the
>>> validity of the corresponding ACL, and enqueues them to Sync and Final
>>> processors. Maybe I am missing something here, but this behaviour seems
>>> weird. What if a setACL request comes, setting the ACL of a path (e.g. /
>>> ) to an IP (e.g. 128.62.32.32) , instead of its old value (e.g. World).
>>> This request will pass the ACL check, and will be enqueued to be
>>> processed by the next processors. Assume that the next request is a
>>> getData("/") from an IP other than 128.62.32.32. If this request is
>>> processed by the PreRequestProcessor before the setACL request is
>>> processed by the FinalRequestProcessor, then it will pass the ACL check
>>> (which it should not, since it came after the setACL request). It seems
>>> that there is a race condition here that should not exist.
>>>
>>> Let me know if this is actually the case or I am missing something. I am
>>> using version 3.0.1 of the code.
>>>
>>> Thank you,
>>> Manos
>>>
>>>       


RE: setACL semantics

Posted by Benjamin Reed <br...@yahoo-inc.com>.
Manos,

in your scenario the getData will not succeed. the acl will be checked in the FinalRequestProcessor after the setAcl has passed through the final request processor. execution only happens in the final request processor and always happens in order.

the only way a getData will succeed after a setAcl succeeds if it is done on a lagging follower, but in that case it will see a lagging view of the system. consider the following:

/foo has an open ACL

client A: setAcl("/foo", restricted ACL); setData("/foo", new data);
client B: getData("/foo");

no matter what happens, what the delays are, if B is faster or slower than A, client B will not see "new data". it will either pass the ACL check and see the old data or it will fail with a permission exception.

ben
________________________________________
From: Manos Kapritsos [manos@cs.utexas.edu]
Sent: Tuesday, February 10, 2009 4:34 PM
To: zookeeper-dev@hadoop.apache.org
Subject: Re: setACL semantics

More or less, yes.
If the requests are from the same client, then you say that there will
not be a problem? I guess that is true if you always wait for the
response of the first request in order to execute the second. I am not
sure if that is a requirement for all Zookeeper client implementations.
As for two different clients (which was the case I was thinking about),
this seems to be a problem. I will agree that (if clients only have one
outstanding request) the two requests are concurrent and either order of
execution is considered to be acceptable, but it could be that two
different replicas receive the two requests in the same order, but
effectively execute them in a different order. In any case, it feels
wrong (at least to me) that a getData would succeed when a setACL that
prevents it has already been accepted to be processed.

Manos

Mahadev Konar wrote:
> Hi Manos,
>  If the setAcl and getdata are from the same client then they are all
> handled in order. So you would get an unauthroized exception when you do a
> getdata.
>
>
> If two diff clients do setacl and getdata it might be that the getdata in
> your case will succeed before the setacl returns on the first client.
>
> Is that what you meant?
>
> mahadev
>
>
> On 2/10/09 2:15 PM, "Manos Kapritsos" <ma...@cs.utexas.edu> wrote:
>
>
>> Hi all,
>>
>> I have a question about the way setACL functions. It seems that the
>> PreRequestProcessor handles all kinds of requests the same, checks the
>> validity of the corresponding ACL, and enqueues them to Sync and Final
>> processors. Maybe I am missing something here, but this behaviour seems
>> weird. What if a setACL request comes, setting the ACL of a path (e.g. /
>> ) to an IP (e.g. 128.62.32.32) , instead of its old value (e.g. World).
>> This request will pass the ACL check, and will be enqueued to be
>> processed by the next processors. Assume that the next request is a
>> getData("/") from an IP other than 128.62.32.32. If this request is
>> processed by the PreRequestProcessor before the setACL request is
>> processed by the FinalRequestProcessor, then it will pass the ACL check
>> (which it should not, since it came after the setACL request). It seems
>> that there is a race condition here that should not exist.
>>
>> Let me know if this is actually the case or I am missing something. I am
>> using version 3.0.1 of the code.
>>
>> Thank you,
>> Manos
>>


Re: setACL semantics

Posted by Mahadev Konar <ma...@yahoo-inc.com>.
Yes that is true.

mahadev


On 2/10/09 4:54 PM, "Patrick Hunt" <ph...@apache.org> wrote:

> Mahadev, is it true that a lagging follower, participating in the
> ensemble but not part of the quorum, would continue to respond to read
> requests until it had received/processed the update from the leader?
> 
> Patrick
> 
> Mahadev Konar wrote:
>> HI Manos,
>> 
>>  The accepting of a txn for being processed does not mean that the txn has
>> been accpeted by the quorum and has been applied to the quorum servers. So,
>> it is very much possible that one of the client changes an acl on a node and
>> the other client unauthorized to read (with respect to the change by
>> client1) reads the data before the acl change is propagated and accepted by
>> the quorum. 
>> 
>> 
>> mahadev
>> 
>> 
>> On 2/10/09 4:34 PM, "Manos Kapritsos" <ma...@cs.utexas.edu> wrote:
>> 
>>> More or less, yes.
>>> If the requests are from the same client, then you say that there will
>>> not be a problem? I guess that is true if you always wait for the
>>> response of the first request in order to execute the second. I am not
>>> sure if that is a requirement for all Zookeeper client implementations.
>>> As for two different clients (which was the case I was thinking about),
>>> this seems to be a problem. I will agree that (if clients only have one
>>> outstanding request) the two requests are concurrent and either order of
>>> execution is considered to be acceptable, but it could be that two
>>> different replicas receive the two requests in the same order, but
>>> effectively execute them in a different order. In any case, it feels
>>> wrong (at least to me) that a getData would succeed when a setACL that
>>> prevents it has already been accepted to be processed.
>>> 
>>> Manos
>>> 
>>> Mahadev Konar wrote:
>>>> Hi Manos, 
>>>>  If the setAcl and getdata are from the same client then they are all
>>>> handled in order. So you would get an unauthroized exception when you do a
>>>> getdata. 
>>>> 
>>>> 
>>>> If two diff clients do setacl and getdata it might be that the getdata in
>>>> your case will succeed before the setacl returns on the first client.
>>>> 
>>>> Is that what you meant?
>>>> 
>>>> mahadev
>>>> 
>>>> 
>>>> On 2/10/09 2:15 PM, "Manos Kapritsos" <ma...@cs.utexas.edu> wrote:
>>>> 
>>>>   
>>>>> Hi all,
>>>>> 
>>>>> I have a question about the way setACL functions. It seems that the
>>>>> PreRequestProcessor handles all kinds of requests the same, checks the
>>>>> validity of the corresponding ACL, and enqueues them to Sync and Final
>>>>> processors. Maybe I am missing something here, but this behaviour seems
>>>>> weird. What if a setACL request comes, setting the ACL of a path (e.g. /
>>>>> ) to an IP (e.g. 128.62.32.32) , instead of its old value (e.g. World).
>>>>> This request will pass the ACL check, and will be enqueued to be
>>>>> processed by the next processors. Assume that the next request is a
>>>>> getData("/") from an IP other than 128.62.32.32. If this request is
>>>>> processed by the PreRequestProcessor before the setACL request is
>>>>> processed by the FinalRequestProcessor, then it will pass the ACL check
>>>>> (which it should not, since it came after the setACL request). It seems
>>>>> that there is a race condition here that should not exist.
>>>>> 
>>>>> Let me know if this is actually the case or I am missing something. I am
>>>>> using version 3.0.1 of the code.
>>>>> 
>>>>> Thank you,
>>>>> Manos
>>>>>     
>> 


Re: setACL semantics

Posted by Patrick Hunt <ph...@apache.org>.
Mahadev, is it true that a lagging follower, participating in the 
ensemble but not part of the quorum, would continue to respond to read 
requests until it had received/processed the update from the leader?

Patrick

Mahadev Konar wrote:
> HI Manos,
> 
>  The accepting of a txn for being processed does not mean that the txn has
> been accpeted by the quorum and has been applied to the quorum servers. So,
> it is very much possible that one of the client changes an acl on a node and
> the other client unauthorized to read (with respect to the change by
> client1) reads the data before the acl change is propagated and accepted by
> the quorum. 
> 
> 
> mahadev
> 
> 
> On 2/10/09 4:34 PM, "Manos Kapritsos" <ma...@cs.utexas.edu> wrote:
> 
>> More or less, yes.
>> If the requests are from the same client, then you say that there will
>> not be a problem? I guess that is true if you always wait for the
>> response of the first request in order to execute the second. I am not
>> sure if that is a requirement for all Zookeeper client implementations.
>> As for two different clients (which was the case I was thinking about),
>> this seems to be a problem. I will agree that (if clients only have one
>> outstanding request) the two requests are concurrent and either order of
>> execution is considered to be acceptable, but it could be that two
>> different replicas receive the two requests in the same order, but
>> effectively execute them in a different order. In any case, it feels
>> wrong (at least to me) that a getData would succeed when a setACL that
>> prevents it has already been accepted to be processed.
>>
>> Manos
>>
>> Mahadev Konar wrote:
>>> Hi Manos, 
>>>  If the setAcl and getdata are from the same client then they are all
>>> handled in order. So you would get an unauthroized exception when you do a
>>> getdata. 
>>>
>>>
>>> If two diff clients do setacl and getdata it might be that the getdata in
>>> your case will succeed before the setacl returns on the first client.
>>>
>>> Is that what you meant?
>>>
>>> mahadev
>>>
>>>
>>> On 2/10/09 2:15 PM, "Manos Kapritsos" <ma...@cs.utexas.edu> wrote:
>>>
>>>   
>>>> Hi all,
>>>>
>>>> I have a question about the way setACL functions. It seems that the
>>>> PreRequestProcessor handles all kinds of requests the same, checks the
>>>> validity of the corresponding ACL, and enqueues them to Sync and Final
>>>> processors. Maybe I am missing something here, but this behaviour seems
>>>> weird. What if a setACL request comes, setting the ACL of a path (e.g. /
>>>> ) to an IP (e.g. 128.62.32.32) , instead of its old value (e.g. World).
>>>> This request will pass the ACL check, and will be enqueued to be
>>>> processed by the next processors. Assume that the next request is a
>>>> getData("/") from an IP other than 128.62.32.32. If this request is
>>>> processed by the PreRequestProcessor before the setACL request is
>>>> processed by the FinalRequestProcessor, then it will pass the ACL check
>>>> (which it should not, since it came after the setACL request). It seems
>>>> that there is a race condition here that should not exist.
>>>>
>>>> Let me know if this is actually the case or I am missing something. I am
>>>> using version 3.0.1 of the code.
>>>>
>>>> Thank you,
>>>> Manos
>>>>     
> 

Re: setACL semantics

Posted by Mahadev Konar <ma...@yahoo-inc.com>.
HI Manos,

 The accepting of a txn for being processed does not mean that the txn has
been accpeted by the quorum and has been applied to the quorum servers. So,
it is very much possible that one of the client changes an acl on a node and
the other client unauthorized to read (with respect to the change by
client1) reads the data before the acl change is propagated and accepted by
the quorum. 


mahadev


On 2/10/09 4:34 PM, "Manos Kapritsos" <ma...@cs.utexas.edu> wrote:

> More or less, yes.
> If the requests are from the same client, then you say that there will
> not be a problem? I guess that is true if you always wait for the
> response of the first request in order to execute the second. I am not
> sure if that is a requirement for all Zookeeper client implementations.
> As for two different clients (which was the case I was thinking about),
> this seems to be a problem. I will agree that (if clients only have one
> outstanding request) the two requests are concurrent and either order of
> execution is considered to be acceptable, but it could be that two
> different replicas receive the two requests in the same order, but
> effectively execute them in a different order. In any case, it feels
> wrong (at least to me) that a getData would succeed when a setACL that
> prevents it has already been accepted to be processed.
> 
> Manos
> 
> Mahadev Konar wrote:
>> Hi Manos, 
>>  If the setAcl and getdata are from the same client then they are all
>> handled in order. So you would get an unauthroized exception when you do a
>> getdata. 
>> 
>> 
>> If two diff clients do setacl and getdata it might be that the getdata in
>> your case will succeed before the setacl returns on the first client.
>> 
>> Is that what you meant?
>> 
>> mahadev
>> 
>> 
>> On 2/10/09 2:15 PM, "Manos Kapritsos" <ma...@cs.utexas.edu> wrote:
>> 
>>   
>>> Hi all,
>>> 
>>> I have a question about the way setACL functions. It seems that the
>>> PreRequestProcessor handles all kinds of requests the same, checks the
>>> validity of the corresponding ACL, and enqueues them to Sync and Final
>>> processors. Maybe I am missing something here, but this behaviour seems
>>> weird. What if a setACL request comes, setting the ACL of a path (e.g. /
>>> ) to an IP (e.g. 128.62.32.32) , instead of its old value (e.g. World).
>>> This request will pass the ACL check, and will be enqueued to be
>>> processed by the next processors. Assume that the next request is a
>>> getData("/") from an IP other than 128.62.32.32. If this request is
>>> processed by the PreRequestProcessor before the setACL request is
>>> processed by the FinalRequestProcessor, then it will pass the ACL check
>>> (which it should not, since it came after the setACL request). It seems
>>> that there is a race condition here that should not exist.
>>> 
>>> Let me know if this is actually the case or I am missing something. I am
>>> using version 3.0.1 of the code.
>>> 
>>> Thank you,
>>> Manos
>>>     
> 


Re: setACL semantics

Posted by Manos Kapritsos <ma...@cs.utexas.edu>.
More or less, yes.
If the requests are from the same client, then you say that there will 
not be a problem? I guess that is true if you always wait for the 
response of the first request in order to execute the second. I am not 
sure if that is a requirement for all Zookeeper client implementations.
As for two different clients (which was the case I was thinking about), 
this seems to be a problem. I will agree that (if clients only have one 
outstanding request) the two requests are concurrent and either order of 
execution is considered to be acceptable, but it could be that two 
different replicas receive the two requests in the same order, but 
effectively execute them in a different order. In any case, it feels 
wrong (at least to me) that a getData would succeed when a setACL that 
prevents it has already been accepted to be processed.

Manos

Mahadev Konar wrote:
> Hi Manos, 
>  If the setAcl and getdata are from the same client then they are all
> handled in order. So you would get an unauthroized exception when you do a
> getdata. 
>
>
> If two diff clients do setacl and getdata it might be that the getdata in
> your case will succeed before the setacl returns on the first client.
>
> Is that what you meant?
>
> mahadev
>
>
> On 2/10/09 2:15 PM, "Manos Kapritsos" <ma...@cs.utexas.edu> wrote:
>
>   
>> Hi all,
>>
>> I have a question about the way setACL functions. It seems that the
>> PreRequestProcessor handles all kinds of requests the same, checks the
>> validity of the corresponding ACL, and enqueues them to Sync and Final
>> processors. Maybe I am missing something here, but this behaviour seems
>> weird. What if a setACL request comes, setting the ACL of a path (e.g. /
>> ) to an IP (e.g. 128.62.32.32) , instead of its old value (e.g. World).
>> This request will pass the ACL check, and will be enqueued to be
>> processed by the next processors. Assume that the next request is a
>> getData("/") from an IP other than 128.62.32.32. If this request is
>> processed by the PreRequestProcessor before the setACL request is
>> processed by the FinalRequestProcessor, then it will pass the ACL check
>> (which it should not, since it came after the setACL request). It seems
>> that there is a race condition here that should not exist.
>>
>> Let me know if this is actually the case or I am missing something. I am
>> using version 3.0.1 of the code.
>>
>> Thank you,
>> Manos
>>     


Re: setACL semantics

Posted by Mahadev Konar <ma...@yahoo-inc.com>.
Hi Manos, 
 If the setAcl and getdata are from the same client then they are all
handled in order. So you would get an unauthroized exception when you do a
getdata. 


If two diff clients do setacl and getdata it might be that the getdata in
your case will succeed before the setacl returns on the first client.

Is that what you meant?

mahadev


On 2/10/09 2:15 PM, "Manos Kapritsos" <ma...@cs.utexas.edu> wrote:

> Hi all,
> 
> I have a question about the way setACL functions. It seems that the
> PreRequestProcessor handles all kinds of requests the same, checks the
> validity of the corresponding ACL, and enqueues them to Sync and Final
> processors. Maybe I am missing something here, but this behaviour seems
> weird. What if a setACL request comes, setting the ACL of a path (e.g. /
> ) to an IP (e.g. 128.62.32.32) , instead of its old value (e.g. World).
> This request will pass the ACL check, and will be enqueued to be
> processed by the next processors. Assume that the next request is a
> getData("/") from an IP other than 128.62.32.32. If this request is
> processed by the PreRequestProcessor before the setACL request is
> processed by the FinalRequestProcessor, then it will pass the ACL check
> (which it should not, since it came after the setACL request). It seems
> that there is a race condition here that should not exist.
> 
> Let me know if this is actually the case or I am missing something. I am
> using version 3.0.1 of the code.
> 
> Thank you,
> Manos