[ 
https://issues.apache.org/jira/browse/CURATOR-584?focusedWorklogId=524491&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-524491
 ]

ASF GitHub Bot logged work on CURATOR-584:
------------------------------------------

                Author: ASF GitHub Bot
            Created on: 15/Dec/20 14:54
            Start Date: 15/Dec/20 14:54
    Worklog Time Spent: 10m 
      Work Description: jmslocum16 commented on pull request #376:
URL: https://github.com/apache/curator/pull/376#issuecomment-745344702


   > @jmslocum16 the basic interface looks OK to me, as does the idempotent 
logic.
   > 
   > Just to confirm my understanding of your solution, your plan is to still 
have the connection loss exception propagate up to the client code, and they 
will retry create / update commands with the idempotent flag set, which will 
succeed if the versions / data match?
   
   Hi Cam, thanks for the quick response!
   I'm glad I submitted this patch early - having the client retry was actually 
not the intended behavior. I had assumed the retry loop inside pathInForeground 
would handle retrying upon connection loss. It sounds like my assumptions were 
not correct there and i need to dig more into how retrying works, to make sure 
the client doesn't have to explicitly retry, and write some tests to validate 
that behavior.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 524491)
    Time Spent: 0.5h  (was: 20m)

> Curator Client Fault Tolerance Extensions
> -----------------------------------------
>
>                 Key: CURATOR-584
>                 URL: https://issues.apache.org/jira/browse/CURATOR-584
>             Project: Apache Curator
>          Issue Type: Improvement
>            Reporter: Josh Slocum
>            Assignee: Jordan Zimmerman
>            Priority: Minor
>          Time Spent: 0.5h
>  Remaining Estimate: 0h
>
> Tl;dr My team at Indeed has developed ZooKeeper functionality to handle 
> stateful retrying of connectionloss for write operations, and we wanted to 
> reach out to discuss if this is something the Curator team may be interested 
> in incorporating.
> We initially reached out to the Zookeeper team 
> (https://issues.apache.org/jira/browse/ZOOKEEPER-3927) but were redirected to 
> Curator as the better place to contribute them. The changes could be 
> relatively easily added as additional parameters and/or extensions of the 
> existing retry behavior in Curator's write operations.
>  
> Hi Curator Devs,
> My team uses zookeeper extensively as part of a distributed key-value store 
> we've built at Indeed (think HBase replacement). Due to our deployment setup 
> co-locating our database daemons with our large hadoop cluster, and the 
> network-intensive nature of a lot of our compute jobs, we were experiencing a 
> large amount of transient ConnectionLoss issues. This was especially 
> problematic on important write operations, such as the creation deletion of 
> distributed locks/leases or updating distributed state in the cluster. 
> We saw that some existing zookeeper client wrappers handled retrying in the 
> presence of ConnectionLoss, but all of the ones we looked at (including 
> Curator) didn't allow for retrying writes wiith all of the proper state. 
> Consider the case of retrying a create. If the initial create had succeeded 
> on the server, but the client got connectionloss, the client would get a 
> NodeExists exception on the retried request, even though the znode was 
> created. This resulted in many issues. For the distributed lock/lease 
> example, to other nodes, it looked like the calling node had been successful 
> acquiring the "lock", and to the calling node, it appeared that it was not 
> able to acquire the "lock", which results in a deadlock.
> Curator has parameters that can modify the behavior upon retry, but those 
> were not sufficient. For example, create() has orSetData(), and delete() has 
> guaranteed().
> To solve this, we implemented a set of "connection-loss tolerant primitives" 
> for the main types of write operations. They handle a connection loss by 
> retrying the operation in a loop, but upon error cases in the retry, inspect 
> the current state to see if it matches the case where a previous round that 
> got connectionloss actually succeeded.
> * createRetriable(String path, byte[] data)
> * setDataRetriable(String path, byte[] newData, int currentVersion)
> * deleteRetriable(String path, int currentVersion)
> * compareAndDeleteRetriable(String path, byte[] currentData, int 
> currentVersion)
> For example, in createRetriable, it will retry the create again on connection 
> loss. If the retried call gets a NodeExists exception, it will check to see 
> if (getData(path) == data and dataVersion == 0). If it does, it assumes the 
> first create succeeded and returns success, otherwise it propagates the 
> NodeExists exception.
> These primitives have allowed us to program our ZooKeeper layer as if 
> ConnectionLoss isn't a transient state we have to worry about, since they 
> have essentially the same guarantees as the non-retriable functions in the 
> zookeeper api do (with a slight difference in semantics).
> Because these behaviors could be relatively easily added to Curator as 
> additional parameters to the existing mechanisms, and (to my knowledge) 
> aren't implemented anywhere else, we think it could be a useful contribution 
> to the Curator project. If this isn't something that Curator is interested in 
> incorporating, Indeed may also consider open sourcing it as a standalone 
> library.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to