> It seems Curator does not expose session id you can always access the ZooKeeper handle directly to get the session ID:
CuratorFramework curator = ... curator.getZookeeperClient().getZooKeeper() -JZ > On Sep 20, 2019, at 10:21 PM, Zili Chen <wander4...@gmail.com> wrote: > > >>I am assuming the "write operation" here is write to ZooKeeper > > Yes. > > >>Looks like contender-1 was not reusing same ZooKeeper client object, so > >>this explains how the previous supposed to be fail operation succeeds? > > Yes. Our communication to ZK is based on Curator, which will re-instance a > client and retry the operation. Due to asynchronously schedule the error > execute order is possible. > > >>record the session ID and don't commit any write operations if session ID > >>changes. > > Sounds reasonable. Currently in our ongoing design we treat the latch path as > "session id" so we use multi-op to atomically verify it. > It seems Curator does not expose session id. And in my option 2 above even I > think of falling back to zookeeper so that we just fail on > session expired and re-instance another contender, contending for leadership. > This will save us from maintaining mutable state during > leadership epoch(to be clear, Flink scope leadership, not ZK). > > Best, > tison. > > > Michael Han <h...@apache.org <mailto:h...@apache.org>> 于2019年9月21日周六 上午4:03写道: > >> thus contender-1 commit a write operation even if it is no longer the > >> leader > > I am assuming the "write operation" here is write to ZooKeeper (as opposed to > write to an external storage system)? If so: > > >> contender-1 recovers from full gc, before it reacts to revoke leadership > >> event, txn-1 retried and sent to ZK. > > contender-2 becomes the leader implies that the ephemeral node appertains to > contender-1 has been removed, which further implies that the session > appertains to contender-1 is either explicitly closed (by client), or > expired. So if contender-1 was still using same client ZooKeeper object, then > it's not possible for txn-1 succeeded as session expire was an event ordered > prior to txn-1, which wouldn't commit after an expired session. > > >> Curator always creates a new client on session expire and retry the > >> operation. > Looks like contender-1 was not reusing same ZooKeeper client object, so this > explains how the previous supposed to be fail operation succeeds? > > If my reasoning make sense, one idea might be on Flink side, once you finish > leader election with ZK, record the session ID and don't commit any write > operations if session ID changes. > > The fencing token + multi might also work, but that sounds a little bit > heavier. > > On Fri, Sep 20, 2019 at 1:31 AM Zili Chen <wander4...@gmail.com > <mailto:wander4...@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>