*Issue 1:*

*Matteo's Response:*

*"The  way I understand this is that you might get less notifications than*







*the number of changes on the z-node, but when you set the watch, it
shouldbe tied to the z-node version that you have already read, so you
should beget notified for any change after that.Resuming, you might not see
all intermediate states, but you shouldguaranteed to converge to final
state.Probably Flavio can give more insight on this."*

>From what I understood and my local testing, it would trigger watcher only
for the change which happened after setting the watcher but not for the
changes which happen after the last read state. So my question is, aren't
we missing the change (if any happened) during the transition period (after
receiving the watcher event and before setting the next watcher). Need ZK
experts here.

*Issue 2:*

*Sijie's Response:*





*"If the zookeeper client is configured with retries, the getChildren will
beretried with transient failures before throwing any exceptions. I
believethe zookeeper client at bookie server is configured with infinite
retriesby default, no?"*

I checked the code, in AutoRecoveryMain(ServerConfiguration conf,
StatsLogger statsLogger) constructor, RetryPolicy policy is not set. If no
retry policy is set, "connectRetryPolicy" of ZooKeeperClient is set
to BoundExponentialBackoffRetryPolicy with Integer.MAX_VALUE retries and
"operationRetryPolicy" of ZooKeeperClient is set to
BoundExponentialBackoffRetryPolicy
with 0 retries. I'm not sure how each one of them works. Have to look into
the details.

Thanks,
Charan

On Tue, Mar 7, 2017 at 10:51 AM, Sijie Guo <si...@apache.org> wrote:

> On Tue, Mar 7, 2017 at 10:10 AM, Charan Reddy G <reddychara...@gmail.com>
> wrote:
>
> > 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?
> >
>
> If the zookeeper client is configured with retries, the getChildren will be
> retried with transient failures before throwing any exceptions. I believe
> the zookeeper client at bookie server is configured with infinite retries
> by default, no?
>
> >
> >     @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