Hey,

*Issue 1:*

I was going through the code to understand how BookiesChanged event is
handled for AuditTask (in Auditor, BookkeeperAdmin and BookieWatcher
classes). In BookieWatcher.notifyBookiesChanged they are making getChildren
call on "bookieRegistrationPath" and setting watcher (which listens for
NodeChildrenChanged event). Now as we know, Zookeeper watchers are one time
triggers, and hence in our code on consuming the watch event we reset
another watch to get notified of future changes.

But due to the limitation of ZK (which is well documented thing in ZK
https://zookeeper.apache.org/doc/r3.4.5/zookeeperProgrammers.html#ch_
zkWatches), "Because watches are one time triggers and there is latency
between getting the event and sending a new request to get a watch you
cannot reliably see every change that happens to a node in ZooKeeper. Be
prepared to handle the case where the znode changes multiple times between
getting the event and setting the watch again."

So now in the implementation - (Auditor.notifyBookiesChanged,
Auditor.availableBookiesChanged,
BookKeeperAdmin.notifyBookiesChanged and BookieWatcher.notifyBookiesChanged)
there is a possibility that if multiple bookies go off at a time, we might
miss handling BookiesChanged event in Auditor, which is a serious risk.

Disussed with my teammember Kishore briefly, that to overcome this issue we
should compare return value of getChildren("bookieRegistrationPath") (in
BookieWatcher.notifyBookiesChanged) with previous known state to recognize
the changes, if any happenend, during the transition period.

Am I missing something here? otherwise I think it is a serious flaw in the
implmentation.

*Issue 2:* Also in Auditor class, if any exception happens in
 notifyBookieChanges(), then we are just logging error and proceeding.
Which basically means that we may not successfully able to set watcher but
we are still proceeding. In this case aren't we missing BookiesChanged
event completely from then on?

    @Override
    public void availableBookiesChanged() {
        // since a watch is triggered, we need to watch again on the bookies
        try {
            notifyBookieChanges();
        } *catch (BKException bke) {*
*            LOG.error("Exception while registering for a bookie change
notification", bke);*
*        }*
        submitAuditTask();
    }

Thanks,
Charan

Reply via email to