Here's a first pass Curator recipe to handle the transaction part (note you add a check(path,
version) on the coordination node, not a setData).
https://gist.github.com/Randgalt/1a19dcd215e202936e5b92c121fc73de <https://gist.github.com/Randgalt/1a19dcd215e202936e5b92c121fc73de>
When you obtain leadership you call setLeader(). Then every time you need to coordinate you
include the operation returned by transactionOp().
I need to do more edge case testing. But, assuming this works I'll open a ticket to add this
as a new Curator recipe.
-JZ
> On Sep 21, 2019, at 10:05 AM, Jordan Zimmerman <jordan@jordanzimmerman.com> wrote:
>
> I took a quick look at "Rethink High-Availability Stores" and I think using the leader
path may not work. I think the best solution will be something akin to combining a leader
election with a common ZNode versioning scheme. i.e.
>
> Create a single ZNode to be used for coordination
> Elect a leader in the normal manner
> When an instance becomes leader it:
> Gets the version of the coordination ZNode
> Sets the data for that ZNode (the contents don't matter) using the retrieved version
number
> If the set succeeds you can be assured you are currently leader (otherwise release leadership
and re-contend)
> Save the new version
> Every time you need to perform a transaction, include a setData of the coordination ZNode
with the saved version number
> As long as this succeeds you should be guaranteed of consistency
> If it ever fails with an invalid version number, release leadership and re-contend
>
> I haven't completely thought this through so others should try to poke holes in it. When
I get a chance, I'll try to write a test to prove this.
>
> -Jordan
>
>> On Sep 20, 2019, at 3:30 AM, Zili Chen <wander4096@gmail.com <mailto:wander4096@gmail.com>>
wrote:
>>
>> Hi ZooKeepers,
>>
>> Recently there is an ongoing refactor[1] in Flink community aimed at
>> overcoming several inconsistent state issues on ZK we have met. I come
>> here to share our design of leader election and leader operation. For
>> leader operation, it is operation that should be committed only if the
>> contender is the leader. Also CC Curator mailing list because it also
>> contains the reason why we cannot JUST use Curator.
>>
>> The rule we want to keep is
>>
>> **Writes on ZK must be committed only if the contender is the leader**
>>
>> We represent contender by an individual ZK client. At the moment we use
>> Curator for leader election so the algorithm is the same as the
>> optimized version in this page[2].
>>
>> The problem is that this algorithm only take care of leader election but
>> is indifferent to subsequent operations. Consider the scenario below:
>>
>> 1. contender-1 becomes the leader
>> 2. contender-1 proposes a create txn-1
>> 3. sender thread suspended for full gc
>> 4. contender-1 lost leadership and contender-2 becomes the leader
>> 5. contender-1 recovers from full gc, before it reacts to revoke
>> leadership event, txn-1 retried and sent to ZK.
>>
>> Without other guard txn will success on ZK and thus contender-1 commit
>> a write operation even if it is no longer the leader. This issue is
>> also documented in this note[3].
>>
>> To overcome this issue instead of just saying that we're unfortunate,
>> we draft two possible solution.
>>
>> The first is document here[4]. Briefly, when the contender becomes the
>> leader, we memorize the latch path at that moment. And for
>> subsequent operations, we do in a transaction first checking the
>> existence of the latch path. Leadership is only switched if the latch
>> gone, and all operations will fail if the latch gone.
>>
>> The second is still rough. Basically it relies on session expire
>> mechanism in ZK. We will adopt the unoptimized version in the
>> recipe[2] given that in our scenario there are only few contenders
>> at the same time. Thus we create /leader node as ephemeral znode with
>> leader information and when session expired we think leadership is
>> revoked and terminate the contender. Asynchronous write operations
>> should not succeed because they will all fail on session expire.
>>
>> We cannot adopt 1 using Curator because it doesn't expose the latch
>> path(which is added recently, but not in the version we use); we
>> cannot adopt 2 using Curator because although we have to retry on
>> connection loss but we don't want to retry on session expire. Curator
>> always creates a new client on session expire and retry the operation.
>>
>> I'd like to learn from ZooKeeper community that 1. is there any
>> potential risk if we eventually adopt option 1 or option 2? 2. is
>> there any other solution we can adopt?
>>
>> Best,
>> tison.
>>
>> [1] https://issues.apache.org/jira/browse/FLINK-10333 <https://issues.apache.org/jira/browse/FLINK-10333>
>> [2] https://zookeeper.apache.org/doc/current/recipes.html#sc_leaderElection <https://zookeeper.apache.org/doc/current/recipes.html#sc_leaderElection>
>> [3] https://cwiki.apache.org/confluence/display/CURATOR/TN10 <https://cwiki.apache.org/confluence/display/CURATOR/TN10>
>> [4] https://docs.google.com/document/d/1cBY1t0k5g1xNqzyfZby3LcPu4t-wpx57G1xf-nmWrCo/edit?usp=sharing
<https://docs.google.com/document/d/1cBY1t0k5g1xNqzyfZby3LcPu4t-wpx57G1xf-nmWrCo/edit?usp=sharing>
>>
>
|