You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Dong Lin <li...@gmail.com> on 2017/12/02 02:33:05 UTC

[DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Hi all,

I have created KIP-232: Detect outdated metadata by adding
ControllerMetadataEpoch field:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-232%3A+Detect+outdated+metadata+by+adding+ControllerMetadataEpoch+field
.

The KIP proposes to add fields in MetadataResponse and
UpdateMetadataRequest so that client can reject outdated metadata and avoid
unnecessary OffsetOutOfRangeException. Otherwise there is currently race
condition that can cause consumer to reset offset which negatively affect
the consumer's availability.

Feedback and suggestions are welcome!

Regards,
Dong

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Dong Lin <li...@gmail.com>.
Hey Jason,

Thanks much. Great question. I have considered topic deletion but I have
not considered the scenario that user creates topic very soon after topic
deletion.

After thinking through this scenario, I think the only option is to have a
global metadata_epoch that keeps increasing every time controller sends
UpdateMetadataRequest. Any other per-topic or per-partition epoch field
will not survive topic deletion followed by topic creation. I have updated
the KIP to use a new design to address all previous questions from you and
Jun. Can you take another look?

Thanks,
Dong

On Tue, Dec 19, 2017 at 2:15 PM, Jason Gustafson <ja...@confluent.io> wrote:

> Hey Dong,
>
> One more thought came to mind. Have you considered edge cases around topic
> deletion? I think currently if a topic is deleted and then re-created, the
> leader epoch will start back at the beginning. It seems like that could
> cause trouble for this solution. One thing that helps is that we have logic
> to remove committed offsets for deleted topics, but there may not be any
> guarantees on when that happens relative to when the metadata is updated on
> all brokers. It seems like it could even happen that the topic is deleted
> and recreated quickly enough that the consumer doesn't even "witness" the
> deletion.
>
> Thanks,
> Jason
>
> On Tue, Dec 19, 2017 at 11:40 AM, Jason Gustafson <ja...@confluent.io>
> wrote:
>
> > I think you're saying that depending on the bug, in the worst case, you
> > may have to downgrade the client. I think that's fair. Note that one
> > advantage of making this a fatal error is that we'll be more likely to
> hit
> > unexpected edge cases in system tests.
> >
> > -Jason
> >
> > On Tue, Dec 19, 2017 at 11:26 AM, Dong Lin <li...@gmail.com> wrote:
> >
> >> Hey Jason,
> >>
> >> Yeah this may sound a bit confusing. Let me explain my thoughts.
> >>
> >> If there is no bug in the client library, after consumer rebalance or
> >> consumer restart, consume will fetch the previously committed offset and
> >> fetch the committed metadata until the leader epoch in the metadata >=
> the
> >> leader epoch in the OffsetFetchResponse. Therefore, when consumer
> commits
> >> offset later, the leader epoch in the OffsetCommitRequest should be
> larger
> >> than the leader epoch from the previously committed offset. Does this
> >> sound
> >> correct?
> >>
> >> Given the above understanding, it seems to suggest that the only
> >> explanation for this exception is that there is bug in the client
> library.
> >> And due to this specific bug, I am not sure we can avoid this error by
> >> simply restarting consumer. And because this error is non-retriable,
> user
> >> may be forced to downgrade client library. Did I miss something here?
> >>
> >> Thanks,
> >> Dong
> >>
> >>
> >> On Tue, Dec 19, 2017 at 11:19 AM, Jason Gustafson <ja...@confluent.io>
> >> wrote:
> >>
> >> > Hey Dong,
> >> >
> >> > Thanks for the updates. Just one question:
> >> >
> >> > When application receives
> >> > > this exception, the only choice will be to revert Kafka client
> >> library to
> >> > > an earlier version.
> >> >
> >> >
> >> > Not sure I follow this. Wouldn't we just restart the consumer? That
> >> would
> >> > cause it to fetch the previous committed offset and then fetch the
> >> correct
> >> > metadata.
> >> >
> >> > Thanks,
> >> > Jason
> >> >
> >> > On Tue, Dec 19, 2017 at 10:36 AM, Dong Lin <li...@gmail.com>
> wrote:
> >> >
> >> > > Hey Jason,
> >> > >
> >> > > Thanks for the comments. These make sense. I have updated the KIP to
> >> > > include a new error INVALID_LEADER_EPOCH. This will be a
> non-retriable
> >> > > error which may be thrown from consumer's API. When application
> >> receives
> >> > > this exception, the only choice will be to revert Kafka client
> >> library to
> >> > > an earlier version.
> >> > >
> >> > > Previously I think it may be better to simply log an error because I
> >> am
> >> > not
> >> > > sure it is a good idea to force user to downgrade Kafka client
> library
> >> > when
> >> > > the error itself, e.g. smaller leader epoch, may not be that fatal.
> >> One
> >> > the
> >> > > other hand it could be argued that we don't know what else can go
> >> wrong
> >> > in
> >> > > the buggy client library and it may be a good reason to force user
> to
> >> > > downgrade library.
> >> > >
> >> > > Thanks,
> >> > > Dong
> >> > >
> >> > >
> >> > > On Tue, Dec 19, 2017 at 9:06 AM, Jason Gustafson <
> jason@confluent.io>
> >> > > wrote:
> >> > >
> >> > > > Hey Dong,
> >> > > >
> >> > > >
> >> > > > > I think it is a good idea to let coordinator do the additional
> >> sanity
> >> > > > check
> >> > > > > to ensure the leader epoch from OffsetCommitRequest never
> >> decreases.
> >> > > This
> >> > > > > can help us detect bug. The next question will be what should we
> >> do
> >> > if
> >> > > > > OffsetCommitRequest provides a smaller leader epoch. One
> possible
> >> > > > solution
> >> > > > > is to return a non-retriable error to consumer which will then
> be
> >> > > thrown
> >> > > > to
> >> > > > > user application. But I am not sure it is worth doing it given
> its
> >> > > impact
> >> > > > > on the user. Maybe it will be safer to simply have an error
> >> message
> >> > in
> >> > > > the
> >> > > > > server log and allow offset commit to succeed. What do you
> think?
> >> > > >
> >> > > >
> >> > > > I think the check would only have value if you return an error
> when
> >> it
> >> > > > fails. It seems primarily useful to detect buggy consumer logic,
> so
> >> a
> >> > > > non-retriable error makes sense to me. Clients which don't
> implement
> >> > this
> >> > > > capability can use the sentinel value and keep the current
> behavior.
> >> > > >
> >> > > > It seems that FetchResponse includes leader epoch via the path
> >> > > > > FetchResponse -> MemoryRecords -> MutableRecordBatch ->
> >> > > > DefaultRecordBatch
> >> > > > > -> partitionLeaderEpoch. Could this be an existing case where we
> >> > expose
> >> > > > the
> >> > > > > leader epoch to clients?
> >> > > >
> >> > > >
> >> > > > Right, in this case the client has no direct dependence on the
> >> field,
> >> > but
> >> > > > it could still be argued that it is exposed (I had actually
> >> considered
> >> > > > stuffing this field into an opaque blob of bytes in the message
> >> format
> >> > > > which the client wasn't allowed to touch, but it didn't happen in
> >> the
> >> > > end).
> >> > > > I'm not opposed to using the leader epoch field here, I was just
> >> > > mentioning
> >> > > > that it does tie clients a bit tighter to something which could be
> >> > > > considered a Kafka internal implementation detail. It makes the
> >> > protocol
> >> > > a
> >> > > > bit less intuitive as well since it is rather difficult to explain
> >> the
> >> > > edge
> >> > > > case it is protecting. That said, we've hit other scenarios where
> >> being
> >> > > > able to detect stale metadata in the client would be helpful, so I
> >> > think
> >> > > it
> >> > > > might be worth the tradeoff.
> >> > > >
> >> > > > -Jason
> >> > > >
> >> > > > On Mon, Dec 18, 2017 at 6:09 PM, Dong Lin <li...@gmail.com>
> >> wrote:
> >> > > >
> >> > > > > Hey Jason,
> >> > > > >
> >> > > > > Thanks much for reviewing the KIP.
> >> > > > >
> >> > > > > I think it is a good idea to let coordinator do the additional
> >> sanity
> >> > > > check
> >> > > > > to ensure the leader epoch from OffsetCommitRequest never
> >> decreases.
> >> > > This
> >> > > > > can help us detect bug. The next question will be what should we
> >> do
> >> > if
> >> > > > > OffsetCommitRequest provides a smaller leader epoch. One
> possible
> >> > > > solution
> >> > > > > is to return a non-retriable error to consumer which will then
> be
> >> > > thrown
> >> > > > to
> >> > > > > user application. But I am not sure it is worth doing it given
> its
> >> > > impact
> >> > > > > on the user. Maybe it will be safer to simply have an error
> >> message
> >> > in
> >> > > > the
> >> > > > > server log and allow offset commit to succeed. What do you
> think?
> >> > > > >
> >> > > > > It seems that FetchResponse includes leader epoch via the path
> >> > > > > FetchResponse -> MemoryRecords -> MutableRecordBatch ->
> >> > > > DefaultRecordBatch
> >> > > > > -> partitionLeaderEpoch. Could this be an existing case where we
> >> > expose
> >> > > > the
> >> > > > > leader epoch to clients?
> >> > > > >
> >> > > > > Thanks,
> >> > > > > Dong
> >> > > > >
> >> > > > >
> >> > > > >
> >> > > > > On Mon, Dec 18, 2017 at 3:27 PM, Jason Gustafson <
> >> jason@confluent.io
> >> > >
> >> > > > > wrote:
> >> > > > >
> >> > > > > > Hi Dong,
> >> > > > > >
> >> > > > > > Thanks for the KIP. Good job identifying the problem. One
> minor
> >> > > > question
> >> > > > > I
> >> > > > > > had is whether the coordinator should enforce that the leader
> >> epoch
> >> > > > > > associated with an offset commit can only go forward for each
> >> > > > partition?
> >> > > > > > Currently it looks like we just depend on the client for this,
> >> but
> >> > > > since
> >> > > > > > we're caching the leader epoch anyway, it seems like a cheap
> >> safety
> >> > > > > > condition. To support old clients, you can always allow the
> >> commit
> >> > if
> >> > > > the
> >> > > > > > leader epoch is unknown.
> >> > > > > >
> >> > > > > > I agree that we shouldn't expose the leader epoch in
> >> > > OffsetAndMetadata
> >> > > > in
> >> > > > > > the consumer API for what it's worth. As you have noted, it is
> >> more
> >> > > of
> >> > > > an
> >> > > > > > implementation detail. By the same argument, it's also a bit
> >> > > > unfortunate
> >> > > > > > that we have to expose it in the request API since that is
> >> nearly
> >> > as
> >> > > > > > binding in terms of how it limits future iterations. I could
> be
> >> > > wrong,
> >> > > > > but
> >> > > > > > this appears to be the first case where clients will depend on
> >> the
> >> > > > > concept
> >> > > > > > of leader epoch. Might not be a big deal considering how
> deeply
> >> > > > embedded
> >> > > > > > leader epochs already are in the inter-broker RPCs and the
> >> message
> >> > > > format
> >> > > > > > itself, but just wanted to mention the fact that good
> >> encapsulation
> >> > > > > applies
> >> > > > > > to the client request API as well.
> >> > > > > >
> >> > > > > > Thanks,
> >> > > > > > Jason
> >> > > > > >
> >> > > > > > On Mon, Dec 18, 2017 at 1:58 PM, Dong Lin <
> lindong28@gmail.com>
> >> > > wrote:
> >> > > > > >
> >> > > > > > > Hey Jun,
> >> > > > > > >
> >> > > > > > > Thanks much for your comments. These are very thoughtful
> >> ideas.
> >> > > > Please
> >> > > > > > see
> >> > > > > > > my comments below.
> >> > > > > > >
> >> > > > > > > On Thu, Dec 14, 2017 at 6:38 PM, Jun Rao <ju...@confluent.io>
> >> > wrote:
> >> > > > > > >
> >> > > > > > > > Hi, Dong,
> >> > > > > > > >
> >> > > > > > > > Thanks for the update. A few more comments below.
> >> > > > > > > >
> >> > > > > > > > 10. It seems that we need to return the leader epoch in
> the
> >> > fetch
> >> > > > > > > response
> >> > > > > > > > as well When fetching data, we could be fetching data
> from a
> >> > > leader
> >> > > > > > epoch
> >> > > > > > > > older than what's returned in the metadata response. So,
> we
> >> > want
> >> > > to
> >> > > > > use
> >> > > > > > > the
> >> > > > > > > > leader epoch associated with the offset being fetched for
> >> > > > committing
> >> > > > > > > > offsets.
> >> > > > > > > >
> >> > > > > > >
> >> > > > > > > It seems that we may have two separate issues here. The
> first
> >> > issue
> >> > > > is
> >> > > > > > that
> >> > > > > > > consumer uses metadata that is older than the one it uses
> >> before.
> >> > > The
> >> > > > > > > second issue is that consumer uses metadata which is newer
> >> than
> >> > the
> >> > > > > > > corresponding leader epoch in the leader broker. We know
> that
> >> the
> >> > > > > > > OffsetOutOfRangeException described in this KIP can be
> >> prevented
> >> > by
> >> > > > > > > avoiding the first issue. On the other hand, it seems that
> the
> >> > > > > > > OffsetOffsetOutOfRangeException can still happen even if we
> >> > avoid
> >> > > > the
> >> > > > > > > second issue -- if consumer uses an older version of
> metadata,
> >> > the
> >> > > > > leader
> >> > > > > > > epoch in its metadata may equal the leader epoch in the
> broker
> >> > even
> >> > > > if
> >> > > > > > the
> >> > > > > > > leader epoch in the broker is oudated.
> >> > > > > > >
> >> > > > > > > Given this understanding, I am not sure why we need to
> return
> >> the
> >> > > > > leader
> >> > > > > > > epoch in the fetch response. As long as consumer's metadata
> is
> >> > not
> >> > > > > going
> >> > > > > > > back in version, I think we are good. Did I miss something
> >> here?
> >> > > > > > >
> >> > > > > > >
> >> > > > > > > >
> >> > > > > > > > 11. Should we now extend OffsetAndMetadata used in the
> >> offset
> >> > > > commit
> >> > > > > > api
> >> > > > > > > in
> >> > > > > > > > KafkaConsumer to include leader epoch? Similarly, should
> we
> >> > > return
> >> > > > > > leader
> >> > > > > > > > epoch in endOffsets(), beginningOffsets() and position()?
> We
> >> > > > probably
> >> > > > > > > need
> >> > > > > > > > to think about how to make the api backward compatible.
> >> > > > > > > >
> >> > > > > > >
> >> > > > > > > After thinking through this carefully, I think we probably
> >> don't
> >> > > want
> >> > > > > to
> >> > > > > > > extend OffsetAndMetadata to include leader epoch because
> >> leader
> >> > > epoch
> >> > > > > is
> >> > > > > > > kind of implementation detail which ideally should be hidden
> >> from
> >> > > > user.
> >> > > > > > The
> >> > > > > > > consumer can include leader epoch in the OffsetCommitRequest
> >> > after
> >> > > > > taking
> >> > > > > > > offset from commitSync(final Map<TopicPartition,
> >> > OffsetAndMetadata>
> >> > > > > > > offsets). Similarly consumer can store leader epoch from
> >> > > > > > > OffsetFetchResponse and only provide offset to user via
> >> > > > > > > consumer.committed(topicPartition). This solution seems to
> >> work
> >> > > well
> >> > > > > and
> >> > > > > > > we
> >> > > > > > > don't have to make changes to consumer's public API. Does
> this
> >> > > sound
> >> > > > > OK?
> >> > > > > > >
> >> > > > > > >
> >> > > > > > > >
> >> > > > > > > > 12. It seems that we now need to store leader epoch in the
> >> > offset
> >> > > > > > topic.
> >> > > > > > > > Could you include the new schema for the value of the
> offset
> >> > > topic
> >> > > > > and
> >> > > > > > > add
> >> > > > > > > > upgrade notes?
> >> > > > > > >
> >> > > > > > >
> >> > > > > > > You are right. I have updated the KIP to specify the new
> >> schema
> >> > for
> >> > > > the
> >> > > > > > > value of the offset topic. Can you take another look?
> >> > > > > > >
> >> > > > > > > For existing messages in the offset topic, leader_epoch will
> >> be
> >> > > > > missing.
> >> > > > > > We
> >> > > > > > > will use leader_epoch = -1 to indicate the missing
> >> leader_epoch.
> >> > > Then
> >> > > > > the
> >> > > > > > > consumer behavior will be the same as it is now because any
> >> > > > > leader_epoch
> >> > > > > > in
> >> > > > > > > the MetadataResponse will be larger than the leader_epoch =
> >> -1 in
> >> > > the
> >> > > > > > > OffetFetchResponse. Thus we don't need specific procedure
> for
> >> > > > upgrades
> >> > > > > > due
> >> > > > > > > to this change in the offset topic schema. By "upgrade
> >> nodes", do
> >> > > you
> >> > > > > > mean
> >> > > > > > > the sentences we need to include in the upgrade.html in the
> PR
> >> > > later?
> >> > > > > > >
> >> > > > > > >
> >> > > > > > > >
> >> > > > > > > > Jun
> >> > > > > > > >
> >> > > > > > > >
> >> > > > > > > > On Tue, Dec 12, 2017 at 5:19 PM, Dong Lin <
> >> lindong28@gmail.com
> >> > >
> >> > > > > wrote:
> >> > > > > > > >
> >> > > > > > > > > Hey Jun,
> >> > > > > > > > >
> >> > > > > > > > > I see. Sounds good. Yeah it is probably simpler to leave
> >> this
> >> > > to
> >> > > > > > > another
> >> > > > > > > > > KIP in the future.
> >> > > > > > > > >
> >> > > > > > > > > Thanks for all the comments. Since there is no further
> >> > comment
> >> > > in
> >> > > > > the
> >> > > > > > > > > community, I will open the voting thread.
> >> > > > > > > > >
> >> > > > > > > > > Thanks,
> >> > > > > > > > > Dong
> >> > > > > > > > >
> >> > > > > > > > > On Mon, Dec 11, 2017 at 5:37 PM, Jun Rao <
> >> jun@confluent.io>
> >> > > > wrote:
> >> > > > > > > > >
> >> > > > > > > > > > Hi, Dong,
> >> > > > > > > > > >
> >> > > > > > > > > > The case that I am thinking is network partitioning.
> >> > Suppose
> >> > > > one
> >> > > > > > > > deploys
> >> > > > > > > > > a
> >> > > > > > > > > > stretched cluster across multiple AZs in the same
> >> region.
> >> > If
> >> > > > the
> >> > > > > > > > machines
> >> > > > > > > > > > in one AZ can't communicate to brokers in other AZs
> due
> >> to
> >> > a
> >> > > > > > network
> >> > > > > > > > > issue,
> >> > > > > > > > > > the brokers in that AZ won't get any new metadata.
> >> > > > > > > > > >
> >> > > > > > > > > > We can potentially solve this problem by requiring
> some
> >> > kind
> >> > > of
> >> > > > > > > regular
> >> > > > > > > > > > heartbeats between the controller and the broker. This
> >> may
> >> > > need
> >> > > > > > some
> >> > > > > > > > more
> >> > > > > > > > > > thoughts. So, it's probably fine to leave this to
> >> another
> >> > KIP
> >> > > > in
> >> > > > > > the
> >> > > > > > > > > > future.
> >> > > > > > > > > >
> >> > > > > > > > > > Thanks,
> >> > > > > > > > > >
> >> > > > > > > > > > Jun
> >> > > > > > > > > >
> >> > > > > > > > > > On Mon, Dec 11, 2017 at 2:55 PM, Dong Lin <
> >> > > lindong28@gmail.com
> >> > > > >
> >> > > > > > > wrote:
> >> > > > > > > > > >
> >> > > > > > > > > > > Hey Jun,
> >> > > > > > > > > > >
> >> > > > > > > > > > > Thanks for the comment. I am open to improve this
> KIP
> >> to
> >> > > > > address
> >> > > > > > > more
> >> > > > > > > > > > > problems. I probably need more help in understanding
> >> what
> >> > > is
> >> > > > > the
> >> > > > > > > > > current
> >> > > > > > > > > > > problem with consumer using outdated metadata and
> >> whether
> >> > > it
> >> > > > is
> >> > > > > > > > easier
> >> > > > > > > > > to
> >> > > > > > > > > > > address it together with this KIP.
> >> > > > > > > > > > >
> >> > > > > > > > > > > I agree that a consumer can potentially talk to old
> >> > leader
> >> > > > for
> >> > > > > a
> >> > > > > > > long
> >> > > > > > > > > > time
> >> > > > > > > > > > > even after this KIP. But after this KIP, the
> consumer
> >> > > > probably
> >> > > > > > > should
> >> > > > > > > > > not
> >> > > > > > > > > > > get OffetOutofRangeException and therefore will not
> >> cause
> >> > > > > offset
> >> > > > > > > > rewind
> >> > > > > > > > > > > issue. So the only problem is that consumer will not
> >> be
> >> > > able
> >> > > > to
> >> > > > > > > fetch
> >> > > > > > > > > > data
> >> > > > > > > > > > > until it has updated metadata. It seems that this
> >> > situation
> >> > > > can
> >> > > > > > > only
> >> > > > > > > > > > happen
> >> > > > > > > > > > > if the broker is too slow in processing
> >> > LeaderAndIsrRequest
> >> > > > > since
> >> > > > > > > > > > otherwise
> >> > > > > > > > > > > the consumer will be forced to update metadata due
> to
> >> > > > > > > > > > > NotLeaderForPartitionException. So the problem we
> are
> >> > > having
> >> > > > > > here
> >> > > > > > > is
> >> > > > > > > > > > that
> >> > > > > > > > > > > consumer will not be able to fetch data if some
> >> broker is
> >> > > too
> >> > > > > > slow
> >> > > > > > > in
> >> > > > > > > > > > > processing LeaderAndIsrRequest.
> >> > > > > > > > > > >
> >> > > > > > > > > > > Because Kafka propagates LeaderAndIsrRequest
> >> > asynchronously
> >> > > > to
> >> > > > > > all
> >> > > > > > > > > > brokers
> >> > > > > > > > > > > in the cluster, there will always be a period of
> time
> >> > when
> >> > > > > > consumer
> >> > > > > > > > can
> >> > > > > > > > > > not
> >> > > > > > > > > > > fetch data for the partition during the leadership
> >> > change.
> >> > > > Thus
> >> > > > > > it
> >> > > > > > > > > seems
> >> > > > > > > > > > > more like a broker-side performance issue instead of
> >> > > > > client-side
> >> > > > > > > > > > > correctness issue. My gut feel is that it is not
> >> causing
> >> > a
> >> > > > > much a
> >> > > > > > > > > problem
> >> > > > > > > > > > > as the problem to be fixed in this KIP. And if we
> >> were to
> >> > > > > address
> >> > > > > > > it,
> >> > > > > > > > > we
> >> > > > > > > > > > > probably need to make change in the broker side,
> e.g.
> >> > with
> >> > > > > > > > prioritized
> >> > > > > > > > > > > queue for controller-related requests, which may be
> >> kind
> >> > of
> >> > > > > > > > orthogonal
> >> > > > > > > > > to
> >> > > > > > > > > > > this KIP. I am not very sure it will be easier to
> >> address
> >> > > it
> >> > > > > with
> >> > > > > > > the
> >> > > > > > > > > > > change in this KIP. Do you have any recommendation?
> >> > > > > > > > > > >
> >> > > > > > > > > > > Thanks,
> >> > > > > > > > > > > Dong
> >> > > > > > > > > > >
> >> > > > > > > > > > >
> >> > > > > > > > > > > On Mon, Dec 11, 2017 at 1:51 PM, Jun Rao <
> >> > jun@confluent.io
> >> > > >
> >> > > > > > wrote:
> >> > > > > > > > > > >
> >> > > > > > > > > > > > Hi, Dong,
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > Thanks for the reply.
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > My suggestion of forcing the metadata refresh from
> >> the
> >> > > > > > controller
> >> > > > > > > > may
> >> > > > > > > > > > not
> >> > > > > > > > > > > > work in general since the cached controller could
> be
> >> > > > outdated
> >> > > > > > > too.
> >> > > > > > > > > The
> >> > > > > > > > > > > > general problem is that if a consumer's metadata
> is
> >> > > > outdated,
> >> > > > > > it
> >> > > > > > > > may
> >> > > > > > > > > > get
> >> > > > > > > > > > > > stuck with the old leader for a long time. We can
> >> > address
> >> > > > the
> >> > > > > > > issue
> >> > > > > > > > > of
> >> > > > > > > > > > > > detecting outdated metadata in a separate KIP in
> the
> >> > > future
> >> > > > > if
> >> > > > > > > you
> >> > > > > > > > > > didn't
> >> > > > > > > > > > > > intend to address it in this KIP.
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > Thanks,
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > Jun
> >> > > > > > > > > > > >
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > On Sat, Dec 9, 2017 at 10:12 PM, Dong Lin <
> >> > > > > lindong28@gmail.com
> >> > > > > > >
> >> > > > > > > > > wrote:
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > > Hey Jun,
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > Thanks much for your comments. Given that client
> >> > needs
> >> > > to
> >> > > > > > > > > > de-serialize
> >> > > > > > > > > > > > the
> >> > > > > > > > > > > > > metadata anyway, the extra overhead of checking
> >> the
> >> > > > > > > per-partition
> >> > > > > > > > > > > version
> >> > > > > > > > > > > > > for every partition should not be a big concern.
> >> Thus
> >> > > it
> >> > > > > > makes
> >> > > > > > > > > sense
> >> > > > > > > > > > to
> >> > > > > > > > > > > > use
> >> > > > > > > > > > > > > leader epoch as the per-partition version
> instead
> >> of
> >> > > > > > creating a
> >> > > > > > > > > > global
> >> > > > > > > > > > > > > metadata version. I will update the KIP to do
> >> that.
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > Regarding the detection of outdated metadata, I
> >> think
> >> > > it
> >> > > > is
> >> > > > > > > > > possible
> >> > > > > > > > > > to
> >> > > > > > > > > > > > > ensure that client gets latest metadata by
> >> fetching
> >> > > from
> >> > > > > > > > > controller.
> >> > > > > > > > > > > Note
> >> > > > > > > > > > > > > that this requires extra logic in the controller
> >> such
> >> > > > that
> >> > > > > > > > > controller
> >> > > > > > > > > > > > > updates metadata directly in memory without
> >> requiring
> >> > > > > > > > > > > > > UpdateMetadataRequest. But I am not sure the
> main
> >> > > > > motivation
> >> > > > > > of
> >> > > > > > > > > this
> >> > > > > > > > > > at
> >> > > > > > > > > > > > > this moment. But this makes controller more
> like a
> >> > > > > bottleneck
> >> > > > > > > in
> >> > > > > > > > > the
> >> > > > > > > > > > > > > cluster which we probably want to avoid.
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > I think we can probably keep the current way of
> >> > > ensuring
> >> > > > > > > metadata
> >> > > > > > > > > > > > > freshness. Currently client will be forced to
> >> refresh
> >> > > > > > metadata
> >> > > > > > > if
> >> > > > > > > > > > > broker
> >> > > > > > > > > > > > > returns error (e.g. NotLeaderForPartition) due
> to
> >> > > > outdated
> >> > > > > > > > metadata
> >> > > > > > > > > > or
> >> > > > > > > > > > > if
> >> > > > > > > > > > > > > the metadata does not contain the partition that
> >> the
> >> > > > client
> >> > > > > > > > needs.
> >> > > > > > > > > In
> >> > > > > > > > > > > the
> >> > > > > > > > > > > > > future, as you previously suggested, we can
> >> include
> >> > > > > > > per-partition
> >> > > > > > > > > > > > > leaderEpoch in the FetchRequest/ProduceRequest
> >> such
> >> > > that
> >> > > > > > broker
> >> > > > > > > > can
> >> > > > > > > > > > > > return
> >> > > > > > > > > > > > > error if the epoch is smaller than cached epoch
> in
> >> > the
> >> > > > > > broker.
> >> > > > > > > > > Given
> >> > > > > > > > > > > that
> >> > > > > > > > > > > > > this adds more complexity to Kafka, I think we
> can
> >> > > > probably
> >> > > > > > > think
> >> > > > > > > > > > about
> >> > > > > > > > > > > > > that leader when we have a specific use-case or
> >> > problem
> >> > > > to
> >> > > > > > > solve
> >> > > > > > > > > with
> >> > > > > > > > > > > > > up-to-date metadata. Does this sound OK?
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > Thanks,
> >> > > > > > > > > > > > > Dong
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > On Fri, Dec 8, 2017 at 3:53 PM, Jun Rao <
> >> > > > jun@confluent.io>
> >> > > > > > > > wrote:
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > > Hi, Dong,
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > Thanks for the reply. A few more points below.
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > For dealing with how to prevent a consumer
> >> > switching
> >> > > > > from a
> >> > > > > > > new
> >> > > > > > > > > > > leader
> >> > > > > > > > > > > > to
> >> > > > > > > > > > > > > > an old leader, you suggestion that refreshes
> >> > metadata
> >> > > > on
> >> > > > > > > > consumer
> >> > > > > > > > > > > > restart
> >> > > > > > > > > > > > > > until it sees a metadata version >= the one
> >> > > associated
> >> > > > > with
> >> > > > > > > the
> >> > > > > > > > > > > offset
> >> > > > > > > > > > > > > > works too, as long as we guarantee that the
> >> cached
> >> > > > > metadata
> >> > > > > > > > > > versions
> >> > > > > > > > > > > on
> >> > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > brokers only go up.
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > The second discussion point is on whether the
> >> > > metadata
> >> > > > > > > > versioning
> >> > > > > > > > > > > > should
> >> > > > > > > > > > > > > be
> >> > > > > > > > > > > > > > per partition or global. For the partition
> level
> >> > > > > > versioning,
> >> > > > > > > > you
> >> > > > > > > > > > were
> >> > > > > > > > > > > > > > concerned about the performance. Given that
> >> > metadata
> >> > > > > > updates
> >> > > > > > > > are
> >> > > > > > > > > > > rare,
> >> > > > > > > > > > > > I
> >> > > > > > > > > > > > > am
> >> > > > > > > > > > > > > > not sure if it's a big concern though. Doing a
> >> > > million
> >> > > > if
> >> > > > > > > tests
> >> > > > > > > > > is
> >> > > > > > > > > > > > > probably
> >> > > > > > > > > > > > > > going to take less than 1ms. Another thing is
> >> that
> >> > > the
> >> > > > > > > metadata
> >> > > > > > > > > > > version
> >> > > > > > > > > > > > > > seems to need to survive controller failover.
> In
> >> > your
> >> > > > > > current
> >> > > > > > > > > > > > approach, a
> >> > > > > > > > > > > > > > consumer may not be able to wait on the right
> >> > version
> >> > > > of
> >> > > > > > the
> >> > > > > > > > > > metadata
> >> > > > > > > > > > > > > after
> >> > > > > > > > > > > > > > the consumer restart since the metadata
> version
> >> may
> >> > > > have
> >> > > > > > been
> >> > > > > > > > > > > recycled
> >> > > > > > > > > > > > on
> >> > > > > > > > > > > > > > the server side due to a controller failover
> >> while
> >> > > the
> >> > > > > > > consumer
> >> > > > > > > > > is
> >> > > > > > > > > > > > down.
> >> > > > > > > > > > > > > > The partition level leaderEpoch survives
> >> controller
> >> > > > > failure
> >> > > > > > > and
> >> > > > > > > > > > won't
> >> > > > > > > > > > > > > have
> >> > > > > > > > > > > > > > this issue.
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > Lastly, neither your proposal nor mine
> addresses
> >> > the
> >> > > > > issue
> >> > > > > > > how
> >> > > > > > > > to
> >> > > > > > > > > > > > > guarantee
> >> > > > > > > > > > > > > > a consumer to detect that is metadata is
> >> outdated.
> >> > > > > > Currently,
> >> > > > > > > > the
> >> > > > > > > > > > > > > consumer
> >> > > > > > > > > > > > > > is not guaranteed to fetch metadata from every
> >> > broker
> >> > > > > > within
> >> > > > > > > > some
> >> > > > > > > > > > > > bounded
> >> > > > > > > > > > > > > > period of time. Maybe this is out of the scope
> >> of
> >> > > your
> >> > > > > KIP.
> >> > > > > > > But
> >> > > > > > > > > one
> >> > > > > > > > > > > > idea
> >> > > > > > > > > > > > > is
> >> > > > > > > > > > > > > > force the consumer to refresh metadata from
> the
> >> > > > > controller
> >> > > > > > > > > > > > periodically.
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > Jun
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > On Thu, Dec 7, 2017 at 11:25 AM, Dong Lin <
> >> > > > > > > lindong28@gmail.com
> >> > > > > > > > >
> >> > > > > > > > > > > wrote:
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > Hey Jun,
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > Thanks much for the comments. Great point
> >> > > > particularly
> >> > > > > > > > > regarding
> >> > > > > > > > > > > > (3). I
> >> > > > > > > > > > > > > > > haven't thought about this before.
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > It seems that there are two possible ways
> >> where
> >> > the
> >> > > > > > version
> >> > > > > > > > > > number
> >> > > > > > > > > > > > can
> >> > > > > > > > > > > > > be
> >> > > > > > > > > > > > > > > used. One solution is for client to check
> the
> >> > > version
> >> > > > > > > number
> >> > > > > > > > at
> >> > > > > > > > > > the
> >> > > > > > > > > > > > > time
> >> > > > > > > > > > > > > > it
> >> > > > > > > > > > > > > > > receives MetadataResponse. And if the
> version
> >> > > number
> >> > > > in
> >> > > > > > the
> >> > > > > > > > > > > > > > > MetadataResponse is smaller than the version
> >> > number
> >> > > > in
> >> > > > > > the
> >> > > > > > > > > > client's
> >> > > > > > > > > > > > > > cache,
> >> > > > > > > > > > > > > > > the client will be forced to fetch metadata
> >> > again.
> >> > > > > > Another
> >> > > > > > > > > > > solution,
> >> > > > > > > > > > > > > as
> >> > > > > > > > > > > > > > > you have suggested, is for broker to check
> the
> >> > > > version
> >> > > > > > > number
> >> > > > > > > > > at
> >> > > > > > > > > > > the
> >> > > > > > > > > > > > > time
> >> > > > > > > > > > > > > > > it receives a request from client. The
> broker
> >> > will
> >> > > > > reject
> >> > > > > > > the
> >> > > > > > > > > > > request
> >> > > > > > > > > > > > > if
> >> > > > > > > > > > > > > > > the version is smaller than the version in
> >> > broker's
> >> > > > > > cache.
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > I am not very sure that the second solution
> >> can
> >> > > > address
> >> > > > > > the
> >> > > > > > > > > > problem
> >> > > > > > > > > > > > > here.
> >> > > > > > > > > > > > > > > In the scenario described in the JIRA
> ticket,
> >> > > > broker's
> >> > > > > > > cache
> >> > > > > > > > > may
> >> > > > > > > > > > be
> >> > > > > > > > > > > > > > > outdated because it has not processed the
> >> > > > > > > LeaderAndIsrRequest
> >> > > > > > > > > > from
> >> > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > controller. Thus it may still process
> client's
> >> > > > request
> >> > > > > > even
> >> > > > > > > > if
> >> > > > > > > > > > the
> >> > > > > > > > > > > > > > version
> >> > > > > > > > > > > > > > > in client's request is actually outdated.
> Does
> >> > this
> >> > > > > make
> >> > > > > > > > sense?
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > IMO, it seems that we can address problem
> (3)
> >> by
> >> > > > saving
> >> > > > > > the
> >> > > > > > > > > > > metadata
> >> > > > > > > > > > > > > > > version together with the offset. After
> >> consumer
> >> > > > > starts,
> >> > > > > > it
> >> > > > > > > > > will
> >> > > > > > > > > > > keep
> >> > > > > > > > > > > > > > > fetching metadata until the metadata version
> >> >=
> >> > the
> >> > > > > > version
> >> > > > > > > > > saved
> >> > > > > > > > > > > > with
> >> > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > offset of this partition.
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > Regarding problems (1) and (2): Currently we
> >> use
> >> > > the
> >> > > > > > > version
> >> > > > > > > > > > number
> >> > > > > > > > > > > > in
> >> > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > MetadataResponse to ensure that the metadata
> >> does
> >> > > not
> >> > > > > go
> >> > > > > > > back
> >> > > > > > > > > in
> >> > > > > > > > > > > > time.
> >> > > > > > > > > > > > > > > There are two alternative solutions to
> address
> >> > > > problems
> >> > > > > > (1)
> >> > > > > > > > and
> >> > > > > > > > > > > (2).
> >> > > > > > > > > > > > > One
> >> > > > > > > > > > > > > > > solution is for client to enumerate all
> >> > partitions
> >> > > in
> >> > > > > the
> >> > > > > > > > > > > > > > MetadataResponse,
> >> > > > > > > > > > > > > > > compare their epoch with those in the cached
> >> > > > metadata,
> >> > > > > > and
> >> > > > > > > > > > rejects
> >> > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > MetadataResponse iff any leader epoch is
> >> smaller.
> >> > > The
> >> > > > > > main
> >> > > > > > > > > > concern
> >> > > > > > > > > > > is
> >> > > > > > > > > > > > > > that
> >> > > > > > > > > > > > > > > MetadataResponse currently cached
> information
> >> of
> >> > > all
> >> > > > > > > > partitions
> >> > > > > > > > > > in
> >> > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > entire cluster. It may slow down client's
> >> > > performance
> >> > > > > if
> >> > > > > > we
> >> > > > > > > > > were
> >> > > > > > > > > > to
> >> > > > > > > > > > > > do
> >> > > > > > > > > > > > > > it.
> >> > > > > > > > > > > > > > > The other solution is for client to
> enumerate
> >> > > > > partitions
> >> > > > > > > for
> >> > > > > > > > > only
> >> > > > > > > > > > > > > topics
> >> > > > > > > > > > > > > > > registered in the org.apache.kafka.clients.
> >> > > Metadata,
> >> > > > > > which
> >> > > > > > > > > will
> >> > > > > > > > > > be
> >> > > > > > > > > > > > an
> >> > > > > > > > > > > > > > > empty
> >> > > > > > > > > > > > > > > set for producer and the set of subscribed
> >> > > partitions
> >> > > > > for
> >> > > > > > > > > > consumer.
> >> > > > > > > > > > > > But
> >> > > > > > > > > > > > > > > this degrades to all topics if consumer
> >> > subscribes
> >> > > to
> >> > > > > > > topics
> >> > > > > > > > in
> >> > > > > > > > > > the
> >> > > > > > > > > > > > > > cluster
> >> > > > > > > > > > > > > > > by pattern.
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > Note that client will only be forced to
> update
> >> > > > metadata
> >> > > > > > if
> >> > > > > > > > the
> >> > > > > > > > > > > > version
> >> > > > > > > > > > > > > in
> >> > > > > > > > > > > > > > > the MetadataResponse is smaller than the
> >> version
> >> > in
> >> > > > the
> >> > > > > > > > cached
> >> > > > > > > > > > > > > metadata.
> >> > > > > > > > > > > > > > In
> >> > > > > > > > > > > > > > > general it should not be a problem. It can
> be
> >> a
> >> > > > problem
> >> > > > > > > only
> >> > > > > > > > if
> >> > > > > > > > > > > some
> >> > > > > > > > > > > > > > broker
> >> > > > > > > > > > > > > > > is particularly slower than other brokers in
> >> > > > processing
> >> > > > > > > > > > > > > > > UpdateMetadataRequest. When this is the
> case,
> >> it
> >> > > > means
> >> > > > > > that
> >> > > > > > > > the
> >> > > > > > > > > > > > broker
> >> > > > > > > > > > > > > is
> >> > > > > > > > > > > > > > > also particularly slower in processing
> >> > > > > > LeaderAndIsrRequest,
> >> > > > > > > > > which
> >> > > > > > > > > > > can
> >> > > > > > > > > > > > > > cause
> >> > > > > > > > > > > > > > > problem anyway because some partition will
> >> > probably
> >> > > > > have
> >> > > > > > no
> >> > > > > > > > > > leader
> >> > > > > > > > > > > > > during
> >> > > > > > > > > > > > > > > this period. I am not sure problems (1) and
> >> (2)
> >> > > cause
> >> > > > > > more
> >> > > > > > > > > > problem
> >> > > > > > > > > > > > than
> >> > > > > > > > > > > > > > > what we already have.
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > Thanks,
> >> > > > > > > > > > > > > > > Dong
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > On Wed, Dec 6, 2017 at 6:42 PM, Jun Rao <
> >> > > > > > jun@confluent.io>
> >> > > > > > > > > > wrote:
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > Hi, Dong,
> >> > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > Great finding on the issue. It's a real
> >> > problem.
> >> > > A
> >> > > > > few
> >> > > > > > > > > comments
> >> > > > > > > > > > > > about
> >> > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > KIP. (1) I am not sure about updating
> >> > > > > > > > > controller_metadata_epoch
> >> > > > > > > > > > > on
> >> > > > > > > > > > > > > > every
> >> > > > > > > > > > > > > > > > UpdateMetadataRequest. Currently, the
> >> > controller
> >> > > > can
> >> > > > > > send
> >> > > > > > > > > > > > > > > > UpdateMetadataRequest when there is no
> >> actual
> >> > > > > metadata
> >> > > > > > > > > change.
> >> > > > > > > > > > > > Doing
> >> > > > > > > > > > > > > > this
> >> > > > > > > > > > > > > > > > may require unnecessary metadata refresh
> on
> >> the
> >> > > > > client.
> >> > > > > > > (2)
> >> > > > > > > > > > > > > > > > controller_metadata_epoch is global across
> >> all
> >> > > > > topics.
> >> > > > > > > This
> >> > > > > > > > > > means
> >> > > > > > > > > > > > > that
> >> > > > > > > > > > > > > > a
> >> > > > > > > > > > > > > > > > client may be forced to update its
> metadata
> >> > even
> >> > > > when
> >> > > > > > the
> >> > > > > > > > > > > metadata
> >> > > > > > > > > > > > > for
> >> > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > topics that it cares haven't changed. (3)
> It
> >> > > > doesn't
> >> > > > > > seem
> >> > > > > > > > > that
> >> > > > > > > > > > > the
> >> > > > > > > > > > > > > KIP
> >> > > > > > > > > > > > > > > > handles the corner case when a consumer is
> >> > > > restarted.
> >> > > > > > > Say a
> >> > > > > > > > > > > > consumer
> >> > > > > > > > > > > > > > > reads
> >> > > > > > > > > > > > > > > > from the new leader, commits the offset
> and
> >> > then
> >> > > is
> >> > > > > > > > > restarted.
> >> > > > > > > > > > On
> >> > > > > > > > > > > > > > > restart,
> >> > > > > > > > > > > > > > > > the consumer gets an outdated metadata and
> >> > > fetches
> >> > > > > from
> >> > > > > > > the
> >> > > > > > > > > old
> >> > > > > > > > > > > > > leader.
> >> > > > > > > > > > > > > > > > Then, the consumer will get into the
> offset
> >> out
> >> > > of
> >> > > > > > range
> >> > > > > > > > > issue.
> >> > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > Given the above, I am thinking of the
> >> following
> >> > > > > > approach.
> >> > > > > > > > We
> >> > > > > > > > > > > > actually
> >> > > > > > > > > > > > > > > > already have metadata versioning at the
> >> > partition
> >> > > > > > level.
> >> > > > > > > > Each
> >> > > > > > > > > > > > leader
> >> > > > > > > > > > > > > > has
> >> > > > > > > > > > > > > > > a
> >> > > > > > > > > > > > > > > > leader epoch which is monotonically
> >> increasing.
> >> > > We
> >> > > > > can
> >> > > > > > > > > > > potentially
> >> > > > > > > > > > > > > > > > propagate leader epoch back in the
> metadata
> >> > > > response
> >> > > > > > and
> >> > > > > > > > the
> >> > > > > > > > > > > > clients
> >> > > > > > > > > > > > > > can
> >> > > > > > > > > > > > > > > > cache that. This solves the issue of (1)
> and
> >> > (2).
> >> > > > To
> >> > > > > > > solve
> >> > > > > > > > > (3),
> >> > > > > > > > > > > > when
> >> > > > > > > > > > > > > > > saving
> >> > > > > > > > > > > > > > > > an offset, we could save both an offset
> and
> >> the
> >> > > > > > > > corresponding
> >> > > > > > > > > > > > leader
> >> > > > > > > > > > > > > > > epoch.
> >> > > > > > > > > > > > > > > > When fetching the data, the consumer
> >> provides
> >> > > both
> >> > > > > the
> >> > > > > > > > offset
> >> > > > > > > > > > and
> >> > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > leader epoch. A leader will only serve the
> >> > > request
> >> > > > if
> >> > > > > > its
> >> > > > > > > > > > leader
> >> > > > > > > > > > > > > epoch
> >> > > > > > > > > > > > > > is
> >> > > > > > > > > > > > > > > > equal to or greater than the leader epoch
> >> from
> >> > > the
> >> > > > > > > > consumer.
> >> > > > > > > > > To
> >> > > > > > > > > > > > > achieve
> >> > > > > > > > > > > > > > > > this, we need to change the fetch request
> >> > > protocol
> >> > > > > and
> >> > > > > > > the
> >> > > > > > > > > > offset
> >> > > > > > > > > > > > > > commit
> >> > > > > > > > > > > > > > > > api, which requires some more thoughts.
> >> > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > Thanks,
> >> > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > Jun
> >> > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > On Wed, Dec 6, 2017 at 10:57 AM, Dong Lin
> <
> >> > > > > > > > > lindong28@gmail.com
> >> > > > > > > > > > >
> >> > > > > > > > > > > > > wrote:
> >> > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > Bump up the thread.
> >> > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > It will be great to have more comments
> on
> >> > > whether
> >> > > > > we
> >> > > > > > > > should
> >> > > > > > > > > > do
> >> > > > > > > > > > > it
> >> > > > > > > > > > > > > or
> >> > > > > > > > > > > > > > > > > whether there is better way to address
> the
> >> > > > > motivation
> >> > > > > > > of
> >> > > > > > > > > this
> >> > > > > > > > > > > > KIP.
> >> > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > On Mon, Dec 4, 2017 at 3:09 PM, Dong
> Lin <
> >> > > > > > > > > > lindong28@gmail.com>
> >> > > > > > > > > > > > > > wrote:
> >> > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > I don't have an interesting rejected
> >> > > > alternative
> >> > > > > > > > solution
> >> > > > > > > > > > to
> >> > > > > > > > > > > > put
> >> > > > > > > > > > > > > in
> >> > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > KIP. If there is good alternative
> >> solution
> >> > > from
> >> > > > > > > anyone
> >> > > > > > > > in
> >> > > > > > > > > > > this
> >> > > > > > > > > > > > > > > thread,
> >> > > > > > > > > > > > > > > > I
> >> > > > > > > > > > > > > > > > > am
> >> > > > > > > > > > > > > > > > > > happy to discuss this and update the
> KIP
> >> > > > > > accordingly.
> >> > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > Thanks,
> >> > > > > > > > > > > > > > > > > > Dong
> >> > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > On Mon, Dec 4, 2017 at 1:12 PM, Ted
> Yu <
> >> > > > > > > > > > yuzhihong@gmail.com>
> >> > > > > > > > > > > > > > wrote:
> >> > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > >> It is clearer now.
> >> > > > > > > > > > > > > > > > > >>
> >> > > > > > > > > > > > > > > > > >> I noticed that Rejected Alternatives
> >> > section
> >> > > > is
> >> > > > > > > empty.
> >> > > > > > > > > > > > > > > > > >> Have you considered any alternative ?
> >> > > > > > > > > > > > > > > > > >>
> >> > > > > > > > > > > > > > > > > >> Cheers
> >> > > > > > > > > > > > > > > > > >>
> >> > > > > > > > > > > > > > > > > >> On Mon, Dec 4, 2017 at 1:07 PM, Dong
> >> Lin <
> >> > > > > > > > > > > lindong28@gmail.com
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > wrote:
> >> > > > > > > > > > > > > > > > > >>
> >> > > > > > > > > > > > > > > > > >> > Ted, thanks for catching this. I
> have
> >> > > > updated
> >> > > > > > the
> >> > > > > > > > > > sentence
> >> > > > > > > > > > > > to
> >> > > > > > > > > > > > > > make
> >> > > > > > > > > > > > > > > > it
> >> > > > > > > > > > > > > > > > > >> > readable.
> >> > > > > > > > > > > > > > > > > >> >
> >> > > > > > > > > > > > > > > > > >> > Thanks,
> >> > > > > > > > > > > > > > > > > >> > Dong
> >> > > > > > > > > > > > > > > > > >> >
> >> > > > > > > > > > > > > > > > > >> > On Sat, Dec 2, 2017 at 3:05 PM, Ted
> >> Yu <
> >> > > > > > > > > > > yuzhihong@gmail.com
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > wrote:
> >> > > > > > > > > > > > > > > > > >> >
> >> > > > > > > > > > > > > > > > > >> > > bq. It the controller_epoch of
> the
> >> > > > incoming
> >> > > > > > > > > > > > > MetadataResponse,
> >> > > > > > > > > > > > > > or
> >> > > > > > > > > > > > > > > > if
> >> > > > > > > > > > > > > > > > > >> the
> >> > > > > > > > > > > > > > > > > >> > > controller_epoch is the same but
> >> the
> >> > > > > > > > > > > > > controller_metadata_epoch
> >> > > > > > > > > > > > > > > > > >> > >
> >> > > > > > > > > > > > > > > > > >> > > Can you update the above sentence
> >> so
> >> > > that
> >> > > > > the
> >> > > > > > > > > > intention
> >> > > > > > > > > > > is
> >> > > > > > > > > > > > > > > > clearer ?
> >> > > > > > > > > > > > > > > > > >> > >
> >> > > > > > > > > > > > > > > > > >> > > Thanks
> >> > > > > > > > > > > > > > > > > >> > >
> >> > > > > > > > > > > > > > > > > >> > > On Fri, Dec 1, 2017 at 6:33 PM,
> >> Dong
> >> > > Lin <
> >> > > > > > > > > > > > > lindong28@gmail.com
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > wrote:
> >> > > > > > > > > > > > > > > > > >> > >
> >> > > > > > > > > > > > > > > > > >> > > > Hi all,
> >> > > > > > > > > > > > > > > > > >> > > >
> >> > > > > > > > > > > > > > > > > >> > > > I have created KIP-232: Detect
> >> > > outdated
> >> > > > > > > metadata
> >> > > > > > > > > by
> >> > > > > > > > > > > > adding
> >> > > > > > > > > > > > > > > > > >> > > > ControllerMetadataEpoch field:
> >> > > > > > > > > > > > > > > > > >> > > > https://cwiki.apache.org/
> >> > > > > > > > > > > confluence/display/KAFKA/KIP-
> >> > > > > > > > > > > > > > > > > >> > > > 232%3A+Detect+outdated+
> >> > > > > metadata+by+adding+
> >> > > > > > > > > > > > > > > > > >> > ControllerMetadataEpoch+field
> >> > > > > > > > > > > > > > > > > >> > > > .
> >> > > > > > > > > > > > > > > > > >> > > >
> >> > > > > > > > > > > > > > > > > >> > > > The KIP proposes to add fields
> in
> >> > > > > > > > MetadataResponse
> >> > > > > > > > > > and
> >> > > > > > > > > > > > > > > > > >> > > > UpdateMetadataRequest so that
> >> client
> >> > > can
> >> > > > > > > reject
> >> > > > > > > > > > > outdated
> >> > > > > > > > > > > > > > > > metadata
> >> > > > > > > > > > > > > > > > > >> and
> >> > > > > > > > > > > > > > > > > >> > > avoid
> >> > > > > > > > > > > > > > > > > >> > > > unnecessary
> >> > OffsetOutOfRangeException.
> >> > > > > > > Otherwise
> >> > > > > > > > > > there
> >> > > > > > > > > > > > is
> >> > > > > > > > > > > > > > > > > currently
> >> > > > > > > > > > > > > > > > > >> > race
> >> > > > > > > > > > > > > > > > > >> > > > condition that can cause
> >> consumer to
> >> > > > reset
> >> > > > > > > > offset
> >> > > > > > > > > > > which
> >> > > > > > > > > > > > > > > > negatively
> >> > > > > > > > > > > > > > > > > >> > affect
> >> > > > > > > > > > > > > > > > > >> > > > the consumer's availability.
> >> > > > > > > > > > > > > > > > > >> > > >
> >> > > > > > > > > > > > > > > > > >> > > > Feedback and suggestions are
> >> > welcome!
> >> > > > > > > > > > > > > > > > > >> > > >
> >> > > > > > > > > > > > > > > > > >> > > > Regards,
> >> > > > > > > > > > > > > > > > > >> > > > Dong
> >> > > > > > > > > > > > > > > > > >> > > >
> >> > > > > > > > > > > > > > > > > >> > >
> >> > > > > > > > > > > > > > > > > >> >
> >> > > > > > > > > > > > > > > > > >>
> >> > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > >
> >> > > > > > > > > > >
> >> > > > > > > > > >
> >> > > > > > > > >
> >> > > > > > > >
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Jason Gustafson <ja...@confluent.io>.
Hey Dong,

One more thought came to mind. Have you considered edge cases around topic
deletion? I think currently if a topic is deleted and then re-created, the
leader epoch will start back at the beginning. It seems like that could
cause trouble for this solution. One thing that helps is that we have logic
to remove committed offsets for deleted topics, but there may not be any
guarantees on when that happens relative to when the metadata is updated on
all brokers. It seems like it could even happen that the topic is deleted
and recreated quickly enough that the consumer doesn't even "witness" the
deletion.

Thanks,
Jason

On Tue, Dec 19, 2017 at 11:40 AM, Jason Gustafson <ja...@confluent.io>
wrote:

> I think you're saying that depending on the bug, in the worst case, you
> may have to downgrade the client. I think that's fair. Note that one
> advantage of making this a fatal error is that we'll be more likely to hit
> unexpected edge cases in system tests.
>
> -Jason
>
> On Tue, Dec 19, 2017 at 11:26 AM, Dong Lin <li...@gmail.com> wrote:
>
>> Hey Jason,
>>
>> Yeah this may sound a bit confusing. Let me explain my thoughts.
>>
>> If there is no bug in the client library, after consumer rebalance or
>> consumer restart, consume will fetch the previously committed offset and
>> fetch the committed metadata until the leader epoch in the metadata >= the
>> leader epoch in the OffsetFetchResponse. Therefore, when consumer commits
>> offset later, the leader epoch in the OffsetCommitRequest should be larger
>> than the leader epoch from the previously committed offset. Does this
>> sound
>> correct?
>>
>> Given the above understanding, it seems to suggest that the only
>> explanation for this exception is that there is bug in the client library.
>> And due to this specific bug, I am not sure we can avoid this error by
>> simply restarting consumer. And because this error is non-retriable, user
>> may be forced to downgrade client library. Did I miss something here?
>>
>> Thanks,
>> Dong
>>
>>
>> On Tue, Dec 19, 2017 at 11:19 AM, Jason Gustafson <ja...@confluent.io>
>> wrote:
>>
>> > Hey Dong,
>> >
>> > Thanks for the updates. Just one question:
>> >
>> > When application receives
>> > > this exception, the only choice will be to revert Kafka client
>> library to
>> > > an earlier version.
>> >
>> >
>> > Not sure I follow this. Wouldn't we just restart the consumer? That
>> would
>> > cause it to fetch the previous committed offset and then fetch the
>> correct
>> > metadata.
>> >
>> > Thanks,
>> > Jason
>> >
>> > On Tue, Dec 19, 2017 at 10:36 AM, Dong Lin <li...@gmail.com> wrote:
>> >
>> > > Hey Jason,
>> > >
>> > > Thanks for the comments. These make sense. I have updated the KIP to
>> > > include a new error INVALID_LEADER_EPOCH. This will be a non-retriable
>> > > error which may be thrown from consumer's API. When application
>> receives
>> > > this exception, the only choice will be to revert Kafka client
>> library to
>> > > an earlier version.
>> > >
>> > > Previously I think it may be better to simply log an error because I
>> am
>> > not
>> > > sure it is a good idea to force user to downgrade Kafka client library
>> > when
>> > > the error itself, e.g. smaller leader epoch, may not be that fatal.
>> One
>> > the
>> > > other hand it could be argued that we don't know what else can go
>> wrong
>> > in
>> > > the buggy client library and it may be a good reason to force user to
>> > > downgrade library.
>> > >
>> > > Thanks,
>> > > Dong
>> > >
>> > >
>> > > On Tue, Dec 19, 2017 at 9:06 AM, Jason Gustafson <ja...@confluent.io>
>> > > wrote:
>> > >
>> > > > Hey Dong,
>> > > >
>> > > >
>> > > > > I think it is a good idea to let coordinator do the additional
>> sanity
>> > > > check
>> > > > > to ensure the leader epoch from OffsetCommitRequest never
>> decreases.
>> > > This
>> > > > > can help us detect bug. The next question will be what should we
>> do
>> > if
>> > > > > OffsetCommitRequest provides a smaller leader epoch. One possible
>> > > > solution
>> > > > > is to return a non-retriable error to consumer which will then be
>> > > thrown
>> > > > to
>> > > > > user application. But I am not sure it is worth doing it given its
>> > > impact
>> > > > > on the user. Maybe it will be safer to simply have an error
>> message
>> > in
>> > > > the
>> > > > > server log and allow offset commit to succeed. What do you think?
>> > > >
>> > > >
>> > > > I think the check would only have value if you return an error when
>> it
>> > > > fails. It seems primarily useful to detect buggy consumer logic, so
>> a
>> > > > non-retriable error makes sense to me. Clients which don't implement
>> > this
>> > > > capability can use the sentinel value and keep the current behavior.
>> > > >
>> > > > It seems that FetchResponse includes leader epoch via the path
>> > > > > FetchResponse -> MemoryRecords -> MutableRecordBatch ->
>> > > > DefaultRecordBatch
>> > > > > -> partitionLeaderEpoch. Could this be an existing case where we
>> > expose
>> > > > the
>> > > > > leader epoch to clients?
>> > > >
>> > > >
>> > > > Right, in this case the client has no direct dependence on the
>> field,
>> > but
>> > > > it could still be argued that it is exposed (I had actually
>> considered
>> > > > stuffing this field into an opaque blob of bytes in the message
>> format
>> > > > which the client wasn't allowed to touch, but it didn't happen in
>> the
>> > > end).
>> > > > I'm not opposed to using the leader epoch field here, I was just
>> > > mentioning
>> > > > that it does tie clients a bit tighter to something which could be
>> > > > considered a Kafka internal implementation detail. It makes the
>> > protocol
>> > > a
>> > > > bit less intuitive as well since it is rather difficult to explain
>> the
>> > > edge
>> > > > case it is protecting. That said, we've hit other scenarios where
>> being
>> > > > able to detect stale metadata in the client would be helpful, so I
>> > think
>> > > it
>> > > > might be worth the tradeoff.
>> > > >
>> > > > -Jason
>> > > >
>> > > > On Mon, Dec 18, 2017 at 6:09 PM, Dong Lin <li...@gmail.com>
>> wrote:
>> > > >
>> > > > > Hey Jason,
>> > > > >
>> > > > > Thanks much for reviewing the KIP.
>> > > > >
>> > > > > I think it is a good idea to let coordinator do the additional
>> sanity
>> > > > check
>> > > > > to ensure the leader epoch from OffsetCommitRequest never
>> decreases.
>> > > This
>> > > > > can help us detect bug. The next question will be what should we
>> do
>> > if
>> > > > > OffsetCommitRequest provides a smaller leader epoch. One possible
>> > > > solution
>> > > > > is to return a non-retriable error to consumer which will then be
>> > > thrown
>> > > > to
>> > > > > user application. But I am not sure it is worth doing it given its
>> > > impact
>> > > > > on the user. Maybe it will be safer to simply have an error
>> message
>> > in
>> > > > the
>> > > > > server log and allow offset commit to succeed. What do you think?
>> > > > >
>> > > > > It seems that FetchResponse includes leader epoch via the path
>> > > > > FetchResponse -> MemoryRecords -> MutableRecordBatch ->
>> > > > DefaultRecordBatch
>> > > > > -> partitionLeaderEpoch. Could this be an existing case where we
>> > expose
>> > > > the
>> > > > > leader epoch to clients?
>> > > > >
>> > > > > Thanks,
>> > > > > Dong
>> > > > >
>> > > > >
>> > > > >
>> > > > > On Mon, Dec 18, 2017 at 3:27 PM, Jason Gustafson <
>> jason@confluent.io
>> > >
>> > > > > wrote:
>> > > > >
>> > > > > > Hi Dong,
>> > > > > >
>> > > > > > Thanks for the KIP. Good job identifying the problem. One minor
>> > > > question
>> > > > > I
>> > > > > > had is whether the coordinator should enforce that the leader
>> epoch
>> > > > > > associated with an offset commit can only go forward for each
>> > > > partition?
>> > > > > > Currently it looks like we just depend on the client for this,
>> but
>> > > > since
>> > > > > > we're caching the leader epoch anyway, it seems like a cheap
>> safety
>> > > > > > condition. To support old clients, you can always allow the
>> commit
>> > if
>> > > > the
>> > > > > > leader epoch is unknown.
>> > > > > >
>> > > > > > I agree that we shouldn't expose the leader epoch in
>> > > OffsetAndMetadata
>> > > > in
>> > > > > > the consumer API for what it's worth. As you have noted, it is
>> more
>> > > of
>> > > > an
>> > > > > > implementation detail. By the same argument, it's also a bit
>> > > > unfortunate
>> > > > > > that we have to expose it in the request API since that is
>> nearly
>> > as
>> > > > > > binding in terms of how it limits future iterations. I could be
>> > > wrong,
>> > > > > but
>> > > > > > this appears to be the first case where clients will depend on
>> the
>> > > > > concept
>> > > > > > of leader epoch. Might not be a big deal considering how deeply
>> > > > embedded
>> > > > > > leader epochs already are in the inter-broker RPCs and the
>> message
>> > > > format
>> > > > > > itself, but just wanted to mention the fact that good
>> encapsulation
>> > > > > applies
>> > > > > > to the client request API as well.
>> > > > > >
>> > > > > > Thanks,
>> > > > > > Jason
>> > > > > >
>> > > > > > On Mon, Dec 18, 2017 at 1:58 PM, Dong Lin <li...@gmail.com>
>> > > wrote:
>> > > > > >
>> > > > > > > Hey Jun,
>> > > > > > >
>> > > > > > > Thanks much for your comments. These are very thoughtful
>> ideas.
>> > > > Please
>> > > > > > see
>> > > > > > > my comments below.
>> > > > > > >
>> > > > > > > On Thu, Dec 14, 2017 at 6:38 PM, Jun Rao <ju...@confluent.io>
>> > wrote:
>> > > > > > >
>> > > > > > > > Hi, Dong,
>> > > > > > > >
>> > > > > > > > Thanks for the update. A few more comments below.
>> > > > > > > >
>> > > > > > > > 10. It seems that we need to return the leader epoch in the
>> > fetch
>> > > > > > > response
>> > > > > > > > as well When fetching data, we could be fetching data from a
>> > > leader
>> > > > > > epoch
>> > > > > > > > older than what's returned in the metadata response. So, we
>> > want
>> > > to
>> > > > > use
>> > > > > > > the
>> > > > > > > > leader epoch associated with the offset being fetched for
>> > > > committing
>> > > > > > > > offsets.
>> > > > > > > >
>> > > > > > >
>> > > > > > > It seems that we may have two separate issues here. The first
>> > issue
>> > > > is
>> > > > > > that
>> > > > > > > consumer uses metadata that is older than the one it uses
>> before.
>> > > The
>> > > > > > > second issue is that consumer uses metadata which is newer
>> than
>> > the
>> > > > > > > corresponding leader epoch in the leader broker. We know that
>> the
>> > > > > > > OffsetOutOfRangeException described in this KIP can be
>> prevented
>> > by
>> > > > > > > avoiding the first issue. On the other hand, it seems that the
>> > > > > > > OffsetOffsetOutOfRangeException can still happen even if we
>> > avoid
>> > > > the
>> > > > > > > second issue -- if consumer uses an older version of metadata,
>> > the
>> > > > > leader
>> > > > > > > epoch in its metadata may equal the leader epoch in the broker
>> > even
>> > > > if
>> > > > > > the
>> > > > > > > leader epoch in the broker is oudated.
>> > > > > > >
>> > > > > > > Given this understanding, I am not sure why we need to return
>> the
>> > > > > leader
>> > > > > > > epoch in the fetch response. As long as consumer's metadata is
>> > not
>> > > > > going
>> > > > > > > back in version, I think we are good. Did I miss something
>> here?
>> > > > > > >
>> > > > > > >
>> > > > > > > >
>> > > > > > > > 11. Should we now extend OffsetAndMetadata used in the
>> offset
>> > > > commit
>> > > > > > api
>> > > > > > > in
>> > > > > > > > KafkaConsumer to include leader epoch? Similarly, should we
>> > > return
>> > > > > > leader
>> > > > > > > > epoch in endOffsets(), beginningOffsets() and position()? We
>> > > > probably
>> > > > > > > need
>> > > > > > > > to think about how to make the api backward compatible.
>> > > > > > > >
>> > > > > > >
>> > > > > > > After thinking through this carefully, I think we probably
>> don't
>> > > want
>> > > > > to
>> > > > > > > extend OffsetAndMetadata to include leader epoch because
>> leader
>> > > epoch
>> > > > > is
>> > > > > > > kind of implementation detail which ideally should be hidden
>> from
>> > > > user.
>> > > > > > The
>> > > > > > > consumer can include leader epoch in the OffsetCommitRequest
>> > after
>> > > > > taking
>> > > > > > > offset from commitSync(final Map<TopicPartition,
>> > OffsetAndMetadata>
>> > > > > > > offsets). Similarly consumer can store leader epoch from
>> > > > > > > OffsetFetchResponse and only provide offset to user via
>> > > > > > > consumer.committed(topicPartition). This solution seems to
>> work
>> > > well
>> > > > > and
>> > > > > > > we
>> > > > > > > don't have to make changes to consumer's public API. Does this
>> > > sound
>> > > > > OK?
>> > > > > > >
>> > > > > > >
>> > > > > > > >
>> > > > > > > > 12. It seems that we now need to store leader epoch in the
>> > offset
>> > > > > > topic.
>> > > > > > > > Could you include the new schema for the value of the offset
>> > > topic
>> > > > > and
>> > > > > > > add
>> > > > > > > > upgrade notes?
>> > > > > > >
>> > > > > > >
>> > > > > > > You are right. I have updated the KIP to specify the new
>> schema
>> > for
>> > > > the
>> > > > > > > value of the offset topic. Can you take another look?
>> > > > > > >
>> > > > > > > For existing messages in the offset topic, leader_epoch will
>> be
>> > > > > missing.
>> > > > > > We
>> > > > > > > will use leader_epoch = -1 to indicate the missing
>> leader_epoch.
>> > > Then
>> > > > > the
>> > > > > > > consumer behavior will be the same as it is now because any
>> > > > > leader_epoch
>> > > > > > in
>> > > > > > > the MetadataResponse will be larger than the leader_epoch =
>> -1 in
>> > > the
>> > > > > > > OffetFetchResponse. Thus we don't need specific procedure for
>> > > > upgrades
>> > > > > > due
>> > > > > > > to this change in the offset topic schema. By "upgrade
>> nodes", do
>> > > you
>> > > > > > mean
>> > > > > > > the sentences we need to include in the upgrade.html in the PR
>> > > later?
>> > > > > > >
>> > > > > > >
>> > > > > > > >
>> > > > > > > > Jun
>> > > > > > > >
>> > > > > > > >
>> > > > > > > > On Tue, Dec 12, 2017 at 5:19 PM, Dong Lin <
>> lindong28@gmail.com
>> > >
>> > > > > wrote:
>> > > > > > > >
>> > > > > > > > > Hey Jun,
>> > > > > > > > >
>> > > > > > > > > I see. Sounds good. Yeah it is probably simpler to leave
>> this
>> > > to
>> > > > > > > another
>> > > > > > > > > KIP in the future.
>> > > > > > > > >
>> > > > > > > > > Thanks for all the comments. Since there is no further
>> > comment
>> > > in
>> > > > > the
>> > > > > > > > > community, I will open the voting thread.
>> > > > > > > > >
>> > > > > > > > > Thanks,
>> > > > > > > > > Dong
>> > > > > > > > >
>> > > > > > > > > On Mon, Dec 11, 2017 at 5:37 PM, Jun Rao <
>> jun@confluent.io>
>> > > > wrote:
>> > > > > > > > >
>> > > > > > > > > > Hi, Dong,
>> > > > > > > > > >
>> > > > > > > > > > The case that I am thinking is network partitioning.
>> > Suppose
>> > > > one
>> > > > > > > > deploys
>> > > > > > > > > a
>> > > > > > > > > > stretched cluster across multiple AZs in the same
>> region.
>> > If
>> > > > the
>> > > > > > > > machines
>> > > > > > > > > > in one AZ can't communicate to brokers in other AZs due
>> to
>> > a
>> > > > > > network
>> > > > > > > > > issue,
>> > > > > > > > > > the brokers in that AZ won't get any new metadata.
>> > > > > > > > > >
>> > > > > > > > > > We can potentially solve this problem by requiring some
>> > kind
>> > > of
>> > > > > > > regular
>> > > > > > > > > > heartbeats between the controller and the broker. This
>> may
>> > > need
>> > > > > > some
>> > > > > > > > more
>> > > > > > > > > > thoughts. So, it's probably fine to leave this to
>> another
>> > KIP
>> > > > in
>> > > > > > the
>> > > > > > > > > > future.
>> > > > > > > > > >
>> > > > > > > > > > Thanks,
>> > > > > > > > > >
>> > > > > > > > > > Jun
>> > > > > > > > > >
>> > > > > > > > > > On Mon, Dec 11, 2017 at 2:55 PM, Dong Lin <
>> > > lindong28@gmail.com
>> > > > >
>> > > > > > > wrote:
>> > > > > > > > > >
>> > > > > > > > > > > Hey Jun,
>> > > > > > > > > > >
>> > > > > > > > > > > Thanks for the comment. I am open to improve this KIP
>> to
>> > > > > address
>> > > > > > > more
>> > > > > > > > > > > problems. I probably need more help in understanding
>> what
>> > > is
>> > > > > the
>> > > > > > > > > current
>> > > > > > > > > > > problem with consumer using outdated metadata and
>> whether
>> > > it
>> > > > is
>> > > > > > > > easier
>> > > > > > > > > to
>> > > > > > > > > > > address it together with this KIP.
>> > > > > > > > > > >
>> > > > > > > > > > > I agree that a consumer can potentially talk to old
>> > leader
>> > > > for
>> > > > > a
>> > > > > > > long
>> > > > > > > > > > time
>> > > > > > > > > > > even after this KIP. But after this KIP, the consumer
>> > > > probably
>> > > > > > > should
>> > > > > > > > > not
>> > > > > > > > > > > get OffetOutofRangeException and therefore will not
>> cause
>> > > > > offset
>> > > > > > > > rewind
>> > > > > > > > > > > issue. So the only problem is that consumer will not
>> be
>> > > able
>> > > > to
>> > > > > > > fetch
>> > > > > > > > > > data
>> > > > > > > > > > > until it has updated metadata. It seems that this
>> > situation
>> > > > can
>> > > > > > > only
>> > > > > > > > > > happen
>> > > > > > > > > > > if the broker is too slow in processing
>> > LeaderAndIsrRequest
>> > > > > since
>> > > > > > > > > > otherwise
>> > > > > > > > > > > the consumer will be forced to update metadata due to
>> > > > > > > > > > > NotLeaderForPartitionException. So the problem we are
>> > > having
>> > > > > > here
>> > > > > > > is
>> > > > > > > > > > that
>> > > > > > > > > > > consumer will not be able to fetch data if some
>> broker is
>> > > too
>> > > > > > slow
>> > > > > > > in
>> > > > > > > > > > > processing LeaderAndIsrRequest.
>> > > > > > > > > > >
>> > > > > > > > > > > Because Kafka propagates LeaderAndIsrRequest
>> > asynchronously
>> > > > to
>> > > > > > all
>> > > > > > > > > > brokers
>> > > > > > > > > > > in the cluster, there will always be a period of time
>> > when
>> > > > > > consumer
>> > > > > > > > can
>> > > > > > > > > > not
>> > > > > > > > > > > fetch data for the partition during the leadership
>> > change.
>> > > > Thus
>> > > > > > it
>> > > > > > > > > seems
>> > > > > > > > > > > more like a broker-side performance issue instead of
>> > > > > client-side
>> > > > > > > > > > > correctness issue. My gut feel is that it is not
>> causing
>> > a
>> > > > > much a
>> > > > > > > > > problem
>> > > > > > > > > > > as the problem to be fixed in this KIP. And if we
>> were to
>> > > > > address
>> > > > > > > it,
>> > > > > > > > > we
>> > > > > > > > > > > probably need to make change in the broker side, e.g.
>> > with
>> > > > > > > > prioritized
>> > > > > > > > > > > queue for controller-related requests, which may be
>> kind
>> > of
>> > > > > > > > orthogonal
>> > > > > > > > > to
>> > > > > > > > > > > this KIP. I am not very sure it will be easier to
>> address
>> > > it
>> > > > > with
>> > > > > > > the
>> > > > > > > > > > > change in this KIP. Do you have any recommendation?
>> > > > > > > > > > >
>> > > > > > > > > > > Thanks,
>> > > > > > > > > > > Dong
>> > > > > > > > > > >
>> > > > > > > > > > >
>> > > > > > > > > > > On Mon, Dec 11, 2017 at 1:51 PM, Jun Rao <
>> > jun@confluent.io
>> > > >
>> > > > > > wrote:
>> > > > > > > > > > >
>> > > > > > > > > > > > Hi, Dong,
>> > > > > > > > > > > >
>> > > > > > > > > > > > Thanks for the reply.
>> > > > > > > > > > > >
>> > > > > > > > > > > > My suggestion of forcing the metadata refresh from
>> the
>> > > > > > controller
>> > > > > > > > may
>> > > > > > > > > > not
>> > > > > > > > > > > > work in general since the cached controller could be
>> > > > outdated
>> > > > > > > too.
>> > > > > > > > > The
>> > > > > > > > > > > > general problem is that if a consumer's metadata is
>> > > > outdated,
>> > > > > > it
>> > > > > > > > may
>> > > > > > > > > > get
>> > > > > > > > > > > > stuck with the old leader for a long time. We can
>> > address
>> > > > the
>> > > > > > > issue
>> > > > > > > > > of
>> > > > > > > > > > > > detecting outdated metadata in a separate KIP in the
>> > > future
>> > > > > if
>> > > > > > > you
>> > > > > > > > > > didn't
>> > > > > > > > > > > > intend to address it in this KIP.
>> > > > > > > > > > > >
>> > > > > > > > > > > > Thanks,
>> > > > > > > > > > > >
>> > > > > > > > > > > > Jun
>> > > > > > > > > > > >
>> > > > > > > > > > > >
>> > > > > > > > > > > > On Sat, Dec 9, 2017 at 10:12 PM, Dong Lin <
>> > > > > lindong28@gmail.com
>> > > > > > >
>> > > > > > > > > wrote:
>> > > > > > > > > > > >
>> > > > > > > > > > > > > Hey Jun,
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > Thanks much for your comments. Given that client
>> > needs
>> > > to
>> > > > > > > > > > de-serialize
>> > > > > > > > > > > > the
>> > > > > > > > > > > > > metadata anyway, the extra overhead of checking
>> the
>> > > > > > > per-partition
>> > > > > > > > > > > version
>> > > > > > > > > > > > > for every partition should not be a big concern.
>> Thus
>> > > it
>> > > > > > makes
>> > > > > > > > > sense
>> > > > > > > > > > to
>> > > > > > > > > > > > use
>> > > > > > > > > > > > > leader epoch as the per-partition version instead
>> of
>> > > > > > creating a
>> > > > > > > > > > global
>> > > > > > > > > > > > > metadata version. I will update the KIP to do
>> that.
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > Regarding the detection of outdated metadata, I
>> think
>> > > it
>> > > > is
>> > > > > > > > > possible
>> > > > > > > > > > to
>> > > > > > > > > > > > > ensure that client gets latest metadata by
>> fetching
>> > > from
>> > > > > > > > > controller.
>> > > > > > > > > > > Note
>> > > > > > > > > > > > > that this requires extra logic in the controller
>> such
>> > > > that
>> > > > > > > > > controller
>> > > > > > > > > > > > > updates metadata directly in memory without
>> requiring
>> > > > > > > > > > > > > UpdateMetadataRequest. But I am not sure the main
>> > > > > motivation
>> > > > > > of
>> > > > > > > > > this
>> > > > > > > > > > at
>> > > > > > > > > > > > > this moment. But this makes controller more like a
>> > > > > bottleneck
>> > > > > > > in
>> > > > > > > > > the
>> > > > > > > > > > > > > cluster which we probably want to avoid.
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > I think we can probably keep the current way of
>> > > ensuring
>> > > > > > > metadata
>> > > > > > > > > > > > > freshness. Currently client will be forced to
>> refresh
>> > > > > > metadata
>> > > > > > > if
>> > > > > > > > > > > broker
>> > > > > > > > > > > > > returns error (e.g. NotLeaderForPartition) due to
>> > > > outdated
>> > > > > > > > metadata
>> > > > > > > > > > or
>> > > > > > > > > > > if
>> > > > > > > > > > > > > the metadata does not contain the partition that
>> the
>> > > > client
>> > > > > > > > needs.
>> > > > > > > > > In
>> > > > > > > > > > > the
>> > > > > > > > > > > > > future, as you previously suggested, we can
>> include
>> > > > > > > per-partition
>> > > > > > > > > > > > > leaderEpoch in the FetchRequest/ProduceRequest
>> such
>> > > that
>> > > > > > broker
>> > > > > > > > can
>> > > > > > > > > > > > return
>> > > > > > > > > > > > > error if the epoch is smaller than cached epoch in
>> > the
>> > > > > > broker.
>> > > > > > > > > Given
>> > > > > > > > > > > that
>> > > > > > > > > > > > > this adds more complexity to Kafka, I think we can
>> > > > probably
>> > > > > > > think
>> > > > > > > > > > about
>> > > > > > > > > > > > > that leader when we have a specific use-case or
>> > problem
>> > > > to
>> > > > > > > solve
>> > > > > > > > > with
>> > > > > > > > > > > > > up-to-date metadata. Does this sound OK?
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > Thanks,
>> > > > > > > > > > > > > Dong
>> > > > > > > > > > > > >
>> > > > > > > > > > > > >
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > On Fri, Dec 8, 2017 at 3:53 PM, Jun Rao <
>> > > > jun@confluent.io>
>> > > > > > > > wrote:
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > > Hi, Dong,
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > > Thanks for the reply. A few more points below.
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > > For dealing with how to prevent a consumer
>> > switching
>> > > > > from a
>> > > > > > > new
>> > > > > > > > > > > leader
>> > > > > > > > > > > > to
>> > > > > > > > > > > > > > an old leader, you suggestion that refreshes
>> > metadata
>> > > > on
>> > > > > > > > consumer
>> > > > > > > > > > > > restart
>> > > > > > > > > > > > > > until it sees a metadata version >= the one
>> > > associated
>> > > > > with
>> > > > > > > the
>> > > > > > > > > > > offset
>> > > > > > > > > > > > > > works too, as long as we guarantee that the
>> cached
>> > > > > metadata
>> > > > > > > > > > versions
>> > > > > > > > > > > on
>> > > > > > > > > > > > > the
>> > > > > > > > > > > > > > brokers only go up.
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > > The second discussion point is on whether the
>> > > metadata
>> > > > > > > > versioning
>> > > > > > > > > > > > should
>> > > > > > > > > > > > > be
>> > > > > > > > > > > > > > per partition or global. For the partition level
>> > > > > > versioning,
>> > > > > > > > you
>> > > > > > > > > > were
>> > > > > > > > > > > > > > concerned about the performance. Given that
>> > metadata
>> > > > > > updates
>> > > > > > > > are
>> > > > > > > > > > > rare,
>> > > > > > > > > > > > I
>> > > > > > > > > > > > > am
>> > > > > > > > > > > > > > not sure if it's a big concern though. Doing a
>> > > million
>> > > > if
>> > > > > > > tests
>> > > > > > > > > is
>> > > > > > > > > > > > > probably
>> > > > > > > > > > > > > > going to take less than 1ms. Another thing is
>> that
>> > > the
>> > > > > > > metadata
>> > > > > > > > > > > version
>> > > > > > > > > > > > > > seems to need to survive controller failover. In
>> > your
>> > > > > > current
>> > > > > > > > > > > > approach, a
>> > > > > > > > > > > > > > consumer may not be able to wait on the right
>> > version
>> > > > of
>> > > > > > the
>> > > > > > > > > > metadata
>> > > > > > > > > > > > > after
>> > > > > > > > > > > > > > the consumer restart since the metadata version
>> may
>> > > > have
>> > > > > > been
>> > > > > > > > > > > recycled
>> > > > > > > > > > > > on
>> > > > > > > > > > > > > > the server side due to a controller failover
>> while
>> > > the
>> > > > > > > consumer
>> > > > > > > > > is
>> > > > > > > > > > > > down.
>> > > > > > > > > > > > > > The partition level leaderEpoch survives
>> controller
>> > > > > failure
>> > > > > > > and
>> > > > > > > > > > won't
>> > > > > > > > > > > > > have
>> > > > > > > > > > > > > > this issue.
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > > Lastly, neither your proposal nor mine addresses
>> > the
>> > > > > issue
>> > > > > > > how
>> > > > > > > > to
>> > > > > > > > > > > > > guarantee
>> > > > > > > > > > > > > > a consumer to detect that is metadata is
>> outdated.
>> > > > > > Currently,
>> > > > > > > > the
>> > > > > > > > > > > > > consumer
>> > > > > > > > > > > > > > is not guaranteed to fetch metadata from every
>> > broker
>> > > > > > within
>> > > > > > > > some
>> > > > > > > > > > > > bounded
>> > > > > > > > > > > > > > period of time. Maybe this is out of the scope
>> of
>> > > your
>> > > > > KIP.
>> > > > > > > But
>> > > > > > > > > one
>> > > > > > > > > > > > idea
>> > > > > > > > > > > > > is
>> > > > > > > > > > > > > > force the consumer to refresh metadata from the
>> > > > > controller
>> > > > > > > > > > > > periodically.
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > > Jun
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > > On Thu, Dec 7, 2017 at 11:25 AM, Dong Lin <
>> > > > > > > lindong28@gmail.com
>> > > > > > > > >
>> > > > > > > > > > > wrote:
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > Hey Jun,
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > Thanks much for the comments. Great point
>> > > > particularly
>> > > > > > > > > regarding
>> > > > > > > > > > > > (3). I
>> > > > > > > > > > > > > > > haven't thought about this before.
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > It seems that there are two possible ways
>> where
>> > the
>> > > > > > version
>> > > > > > > > > > number
>> > > > > > > > > > > > can
>> > > > > > > > > > > > > be
>> > > > > > > > > > > > > > > used. One solution is for client to check the
>> > > version
>> > > > > > > number
>> > > > > > > > at
>> > > > > > > > > > the
>> > > > > > > > > > > > > time
>> > > > > > > > > > > > > > it
>> > > > > > > > > > > > > > > receives MetadataResponse. And if the version
>> > > number
>> > > > in
>> > > > > > the
>> > > > > > > > > > > > > > > MetadataResponse is smaller than the version
>> > number
>> > > > in
>> > > > > > the
>> > > > > > > > > > client's
>> > > > > > > > > > > > > > cache,
>> > > > > > > > > > > > > > > the client will be forced to fetch metadata
>> > again.
>> > > > > > Another
>> > > > > > > > > > > solution,
>> > > > > > > > > > > > > as
>> > > > > > > > > > > > > > > you have suggested, is for broker to check the
>> > > > version
>> > > > > > > number
>> > > > > > > > > at
>> > > > > > > > > > > the
>> > > > > > > > > > > > > time
>> > > > > > > > > > > > > > > it receives a request from client. The broker
>> > will
>> > > > > reject
>> > > > > > > the
>> > > > > > > > > > > request
>> > > > > > > > > > > > > if
>> > > > > > > > > > > > > > > the version is smaller than the version in
>> > broker's
>> > > > > > cache.
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > I am not very sure that the second solution
>> can
>> > > > address
>> > > > > > the
>> > > > > > > > > > problem
>> > > > > > > > > > > > > here.
>> > > > > > > > > > > > > > > In the scenario described in the JIRA ticket,
>> > > > broker's
>> > > > > > > cache
>> > > > > > > > > may
>> > > > > > > > > > be
>> > > > > > > > > > > > > > > outdated because it has not processed the
>> > > > > > > LeaderAndIsrRequest
>> > > > > > > > > > from
>> > > > > > > > > > > > the
>> > > > > > > > > > > > > > > controller. Thus it may still process client's
>> > > > request
>> > > > > > even
>> > > > > > > > if
>> > > > > > > > > > the
>> > > > > > > > > > > > > > version
>> > > > > > > > > > > > > > > in client's request is actually outdated. Does
>> > this
>> > > > > make
>> > > > > > > > sense?
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > IMO, it seems that we can address problem (3)
>> by
>> > > > saving
>> > > > > > the
>> > > > > > > > > > > metadata
>> > > > > > > > > > > > > > > version together with the offset. After
>> consumer
>> > > > > starts,
>> > > > > > it
>> > > > > > > > > will
>> > > > > > > > > > > keep
>> > > > > > > > > > > > > > > fetching metadata until the metadata version
>> >=
>> > the
>> > > > > > version
>> > > > > > > > > saved
>> > > > > > > > > > > > with
>> > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > offset of this partition.
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > Regarding problems (1) and (2): Currently we
>> use
>> > > the
>> > > > > > > version
>> > > > > > > > > > number
>> > > > > > > > > > > > in
>> > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > MetadataResponse to ensure that the metadata
>> does
>> > > not
>> > > > > go
>> > > > > > > back
>> > > > > > > > > in
>> > > > > > > > > > > > time.
>> > > > > > > > > > > > > > > There are two alternative solutions to address
>> > > > problems
>> > > > > > (1)
>> > > > > > > > and
>> > > > > > > > > > > (2).
>> > > > > > > > > > > > > One
>> > > > > > > > > > > > > > > solution is for client to enumerate all
>> > partitions
>> > > in
>> > > > > the
>> > > > > > > > > > > > > > MetadataResponse,
>> > > > > > > > > > > > > > > compare their epoch with those in the cached
>> > > > metadata,
>> > > > > > and
>> > > > > > > > > > rejects
>> > > > > > > > > > > > the
>> > > > > > > > > > > > > > > MetadataResponse iff any leader epoch is
>> smaller.
>> > > The
>> > > > > > main
>> > > > > > > > > > concern
>> > > > > > > > > > > is
>> > > > > > > > > > > > > > that
>> > > > > > > > > > > > > > > MetadataResponse currently cached information
>> of
>> > > all
>> > > > > > > > partitions
>> > > > > > > > > > in
>> > > > > > > > > > > > the
>> > > > > > > > > > > > > > > entire cluster. It may slow down client's
>> > > performance
>> > > > > if
>> > > > > > we
>> > > > > > > > > were
>> > > > > > > > > > to
>> > > > > > > > > > > > do
>> > > > > > > > > > > > > > it.
>> > > > > > > > > > > > > > > The other solution is for client to enumerate
>> > > > > partitions
>> > > > > > > for
>> > > > > > > > > only
>> > > > > > > > > > > > > topics
>> > > > > > > > > > > > > > > registered in the org.apache.kafka.clients.
>> > > Metadata,
>> > > > > > which
>> > > > > > > > > will
>> > > > > > > > > > be
>> > > > > > > > > > > > an
>> > > > > > > > > > > > > > > empty
>> > > > > > > > > > > > > > > set for producer and the set of subscribed
>> > > partitions
>> > > > > for
>> > > > > > > > > > consumer.
>> > > > > > > > > > > > But
>> > > > > > > > > > > > > > > this degrades to all topics if consumer
>> > subscribes
>> > > to
>> > > > > > > topics
>> > > > > > > > in
>> > > > > > > > > > the
>> > > > > > > > > > > > > > cluster
>> > > > > > > > > > > > > > > by pattern.
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > Note that client will only be forced to update
>> > > > metadata
>> > > > > > if
>> > > > > > > > the
>> > > > > > > > > > > > version
>> > > > > > > > > > > > > in
>> > > > > > > > > > > > > > > the MetadataResponse is smaller than the
>> version
>> > in
>> > > > the
>> > > > > > > > cached
>> > > > > > > > > > > > > metadata.
>> > > > > > > > > > > > > > In
>> > > > > > > > > > > > > > > general it should not be a problem. It can be
>> a
>> > > > problem
>> > > > > > > only
>> > > > > > > > if
>> > > > > > > > > > > some
>> > > > > > > > > > > > > > broker
>> > > > > > > > > > > > > > > is particularly slower than other brokers in
>> > > > processing
>> > > > > > > > > > > > > > > UpdateMetadataRequest. When this is the case,
>> it
>> > > > means
>> > > > > > that
>> > > > > > > > the
>> > > > > > > > > > > > broker
>> > > > > > > > > > > > > is
>> > > > > > > > > > > > > > > also particularly slower in processing
>> > > > > > LeaderAndIsrRequest,
>> > > > > > > > > which
>> > > > > > > > > > > can
>> > > > > > > > > > > > > > cause
>> > > > > > > > > > > > > > > problem anyway because some partition will
>> > probably
>> > > > > have
>> > > > > > no
>> > > > > > > > > > leader
>> > > > > > > > > > > > > during
>> > > > > > > > > > > > > > > this period. I am not sure problems (1) and
>> (2)
>> > > cause
>> > > > > > more
>> > > > > > > > > > problem
>> > > > > > > > > > > > than
>> > > > > > > > > > > > > > > what we already have.
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > Thanks,
>> > > > > > > > > > > > > > > Dong
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > On Wed, Dec 6, 2017 at 6:42 PM, Jun Rao <
>> > > > > > jun@confluent.io>
>> > > > > > > > > > wrote:
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > Hi, Dong,
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > Great finding on the issue. It's a real
>> > problem.
>> > > A
>> > > > > few
>> > > > > > > > > comments
>> > > > > > > > > > > > about
>> > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > KIP. (1) I am not sure about updating
>> > > > > > > > > controller_metadata_epoch
>> > > > > > > > > > > on
>> > > > > > > > > > > > > > every
>> > > > > > > > > > > > > > > > UpdateMetadataRequest. Currently, the
>> > controller
>> > > > can
>> > > > > > send
>> > > > > > > > > > > > > > > > UpdateMetadataRequest when there is no
>> actual
>> > > > > metadata
>> > > > > > > > > change.
>> > > > > > > > > > > > Doing
>> > > > > > > > > > > > > > this
>> > > > > > > > > > > > > > > > may require unnecessary metadata refresh on
>> the
>> > > > > client.
>> > > > > > > (2)
>> > > > > > > > > > > > > > > > controller_metadata_epoch is global across
>> all
>> > > > > topics.
>> > > > > > > This
>> > > > > > > > > > means
>> > > > > > > > > > > > > that
>> > > > > > > > > > > > > > a
>> > > > > > > > > > > > > > > > client may be forced to update its metadata
>> > even
>> > > > when
>> > > > > > the
>> > > > > > > > > > > metadata
>> > > > > > > > > > > > > for
>> > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > topics that it cares haven't changed. (3) It
>> > > > doesn't
>> > > > > > seem
>> > > > > > > > > that
>> > > > > > > > > > > the
>> > > > > > > > > > > > > KIP
>> > > > > > > > > > > > > > > > handles the corner case when a consumer is
>> > > > restarted.
>> > > > > > > Say a
>> > > > > > > > > > > > consumer
>> > > > > > > > > > > > > > > reads
>> > > > > > > > > > > > > > > > from the new leader, commits the offset and
>> > then
>> > > is
>> > > > > > > > > restarted.
>> > > > > > > > > > On
>> > > > > > > > > > > > > > > restart,
>> > > > > > > > > > > > > > > > the consumer gets an outdated metadata and
>> > > fetches
>> > > > > from
>> > > > > > > the
>> > > > > > > > > old
>> > > > > > > > > > > > > leader.
>> > > > > > > > > > > > > > > > Then, the consumer will get into the offset
>> out
>> > > of
>> > > > > > range
>> > > > > > > > > issue.
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > Given the above, I am thinking of the
>> following
>> > > > > > approach.
>> > > > > > > > We
>> > > > > > > > > > > > actually
>> > > > > > > > > > > > > > > > already have metadata versioning at the
>> > partition
>> > > > > > level.
>> > > > > > > > Each
>> > > > > > > > > > > > leader
>> > > > > > > > > > > > > > has
>> > > > > > > > > > > > > > > a
>> > > > > > > > > > > > > > > > leader epoch which is monotonically
>> increasing.
>> > > We
>> > > > > can
>> > > > > > > > > > > potentially
>> > > > > > > > > > > > > > > > propagate leader epoch back in the metadata
>> > > > response
>> > > > > > and
>> > > > > > > > the
>> > > > > > > > > > > > clients
>> > > > > > > > > > > > > > can
>> > > > > > > > > > > > > > > > cache that. This solves the issue of (1) and
>> > (2).
>> > > > To
>> > > > > > > solve
>> > > > > > > > > (3),
>> > > > > > > > > > > > when
>> > > > > > > > > > > > > > > saving
>> > > > > > > > > > > > > > > > an offset, we could save both an offset and
>> the
>> > > > > > > > corresponding
>> > > > > > > > > > > > leader
>> > > > > > > > > > > > > > > epoch.
>> > > > > > > > > > > > > > > > When fetching the data, the consumer
>> provides
>> > > both
>> > > > > the
>> > > > > > > > offset
>> > > > > > > > > > and
>> > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > leader epoch. A leader will only serve the
>> > > request
>> > > > if
>> > > > > > its
>> > > > > > > > > > leader
>> > > > > > > > > > > > > epoch
>> > > > > > > > > > > > > > is
>> > > > > > > > > > > > > > > > equal to or greater than the leader epoch
>> from
>> > > the
>> > > > > > > > consumer.
>> > > > > > > > > To
>> > > > > > > > > > > > > achieve
>> > > > > > > > > > > > > > > > this, we need to change the fetch request
>> > > protocol
>> > > > > and
>> > > > > > > the
>> > > > > > > > > > offset
>> > > > > > > > > > > > > > commit
>> > > > > > > > > > > > > > > > api, which requires some more thoughts.
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > Thanks,
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > Jun
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > On Wed, Dec 6, 2017 at 10:57 AM, Dong Lin <
>> > > > > > > > > lindong28@gmail.com
>> > > > > > > > > > >
>> > > > > > > > > > > > > wrote:
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > Bump up the thread.
>> > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > It will be great to have more comments on
>> > > whether
>> > > > > we
>> > > > > > > > should
>> > > > > > > > > > do
>> > > > > > > > > > > it
>> > > > > > > > > > > > > or
>> > > > > > > > > > > > > > > > > whether there is better way to address the
>> > > > > motivation
>> > > > > > > of
>> > > > > > > > > this
>> > > > > > > > > > > > KIP.
>> > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > On Mon, Dec 4, 2017 at 3:09 PM, Dong Lin <
>> > > > > > > > > > lindong28@gmail.com>
>> > > > > > > > > > > > > > wrote:
>> > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > I don't have an interesting rejected
>> > > > alternative
>> > > > > > > > solution
>> > > > > > > > > > to
>> > > > > > > > > > > > put
>> > > > > > > > > > > > > in
>> > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > KIP. If there is good alternative
>> solution
>> > > from
>> > > > > > > anyone
>> > > > > > > > in
>> > > > > > > > > > > this
>> > > > > > > > > > > > > > > thread,
>> > > > > > > > > > > > > > > > I
>> > > > > > > > > > > > > > > > > am
>> > > > > > > > > > > > > > > > > > happy to discuss this and update the KIP
>> > > > > > accordingly.
>> > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > Thanks,
>> > > > > > > > > > > > > > > > > > Dong
>> > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > On Mon, Dec 4, 2017 at 1:12 PM, Ted Yu <
>> > > > > > > > > > yuzhihong@gmail.com>
>> > > > > > > > > > > > > > wrote:
>> > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > >> It is clearer now.
>> > > > > > > > > > > > > > > > > >>
>> > > > > > > > > > > > > > > > > >> I noticed that Rejected Alternatives
>> > section
>> > > > is
>> > > > > > > empty.
>> > > > > > > > > > > > > > > > > >> Have you considered any alternative ?
>> > > > > > > > > > > > > > > > > >>
>> > > > > > > > > > > > > > > > > >> Cheers
>> > > > > > > > > > > > > > > > > >>
>> > > > > > > > > > > > > > > > > >> On Mon, Dec 4, 2017 at 1:07 PM, Dong
>> Lin <
>> > > > > > > > > > > lindong28@gmail.com
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > > > wrote:
>> > > > > > > > > > > > > > > > > >>
>> > > > > > > > > > > > > > > > > >> > Ted, thanks for catching this. I have
>> > > > updated
>> > > > > > the
>> > > > > > > > > > sentence
>> > > > > > > > > > > > to
>> > > > > > > > > > > > > > make
>> > > > > > > > > > > > > > > > it
>> > > > > > > > > > > > > > > > > >> > readable.
>> > > > > > > > > > > > > > > > > >> >
>> > > > > > > > > > > > > > > > > >> > Thanks,
>> > > > > > > > > > > > > > > > > >> > Dong
>> > > > > > > > > > > > > > > > > >> >
>> > > > > > > > > > > > > > > > > >> > On Sat, Dec 2, 2017 at 3:05 PM, Ted
>> Yu <
>> > > > > > > > > > > yuzhihong@gmail.com
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > > > wrote:
>> > > > > > > > > > > > > > > > > >> >
>> > > > > > > > > > > > > > > > > >> > > bq. It the controller_epoch of the
>> > > > incoming
>> > > > > > > > > > > > > MetadataResponse,
>> > > > > > > > > > > > > > or
>> > > > > > > > > > > > > > > > if
>> > > > > > > > > > > > > > > > > >> the
>> > > > > > > > > > > > > > > > > >> > > controller_epoch is the same but
>> the
>> > > > > > > > > > > > > controller_metadata_epoch
>> > > > > > > > > > > > > > > > > >> > >
>> > > > > > > > > > > > > > > > > >> > > Can you update the above sentence
>> so
>> > > that
>> > > > > the
>> > > > > > > > > > intention
>> > > > > > > > > > > is
>> > > > > > > > > > > > > > > > clearer ?
>> > > > > > > > > > > > > > > > > >> > >
>> > > > > > > > > > > > > > > > > >> > > Thanks
>> > > > > > > > > > > > > > > > > >> > >
>> > > > > > > > > > > > > > > > > >> > > On Fri, Dec 1, 2017 at 6:33 PM,
>> Dong
>> > > Lin <
>> > > > > > > > > > > > > lindong28@gmail.com
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > wrote:
>> > > > > > > > > > > > > > > > > >> > >
>> > > > > > > > > > > > > > > > > >> > > > Hi all,
>> > > > > > > > > > > > > > > > > >> > > >
>> > > > > > > > > > > > > > > > > >> > > > I have created KIP-232: Detect
>> > > outdated
>> > > > > > > metadata
>> > > > > > > > > by
>> > > > > > > > > > > > adding
>> > > > > > > > > > > > > > > > > >> > > > ControllerMetadataEpoch field:
>> > > > > > > > > > > > > > > > > >> > > > https://cwiki.apache.org/
>> > > > > > > > > > > confluence/display/KAFKA/KIP-
>> > > > > > > > > > > > > > > > > >> > > > 232%3A+Detect+outdated+
>> > > > > metadata+by+adding+
>> > > > > > > > > > > > > > > > > >> > ControllerMetadataEpoch+field
>> > > > > > > > > > > > > > > > > >> > > > .
>> > > > > > > > > > > > > > > > > >> > > >
>> > > > > > > > > > > > > > > > > >> > > > The KIP proposes to add fields in
>> > > > > > > > MetadataResponse
>> > > > > > > > > > and
>> > > > > > > > > > > > > > > > > >> > > > UpdateMetadataRequest so that
>> client
>> > > can
>> > > > > > > reject
>> > > > > > > > > > > outdated
>> > > > > > > > > > > > > > > > metadata
>> > > > > > > > > > > > > > > > > >> and
>> > > > > > > > > > > > > > > > > >> > > avoid
>> > > > > > > > > > > > > > > > > >> > > > unnecessary
>> > OffsetOutOfRangeException.
>> > > > > > > Otherwise
>> > > > > > > > > > there
>> > > > > > > > > > > > is
>> > > > > > > > > > > > > > > > > currently
>> > > > > > > > > > > > > > > > > >> > race
>> > > > > > > > > > > > > > > > > >> > > > condition that can cause
>> consumer to
>> > > > reset
>> > > > > > > > offset
>> > > > > > > > > > > which
>> > > > > > > > > > > > > > > > negatively
>> > > > > > > > > > > > > > > > > >> > affect
>> > > > > > > > > > > > > > > > > >> > > > the consumer's availability.
>> > > > > > > > > > > > > > > > > >> > > >
>> > > > > > > > > > > > > > > > > >> > > > Feedback and suggestions are
>> > welcome!
>> > > > > > > > > > > > > > > > > >> > > >
>> > > > > > > > > > > > > > > > > >> > > > Regards,
>> > > > > > > > > > > > > > > > > >> > > > Dong
>> > > > > > > > > > > > > > > > > >> > > >
>> > > > > > > > > > > > > > > > > >> > >
>> > > > > > > > > > > > > > > > > >> >
>> > > > > > > > > > > > > > > > > >>
>> > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > >
>> > > > > > > > > > > >
>> > > > > > > > > > >
>> > > > > > > > > >
>> > > > > > > > >
>> > > > > > > >
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>>
>
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Jason Gustafson <ja...@confluent.io>.
I think you're saying that depending on the bug, in the worst case, you may
have to downgrade the client. I think that's fair. Note that one advantage
of making this a fatal error is that we'll be more likely to hit unexpected
edge cases in system tests.

-Jason

On Tue, Dec 19, 2017 at 11:26 AM, Dong Lin <li...@gmail.com> wrote:

> Hey Jason,
>
> Yeah this may sound a bit confusing. Let me explain my thoughts.
>
> If there is no bug in the client library, after consumer rebalance or
> consumer restart, consume will fetch the previously committed offset and
> fetch the committed metadata until the leader epoch in the metadata >= the
> leader epoch in the OffsetFetchResponse. Therefore, when consumer commits
> offset later, the leader epoch in the OffsetCommitRequest should be larger
> than the leader epoch from the previously committed offset. Does this sound
> correct?
>
> Given the above understanding, it seems to suggest that the only
> explanation for this exception is that there is bug in the client library.
> And due to this specific bug, I am not sure we can avoid this error by
> simply restarting consumer. And because this error is non-retriable, user
> may be forced to downgrade client library. Did I miss something here?
>
> Thanks,
> Dong
>
>
> On Tue, Dec 19, 2017 at 11:19 AM, Jason Gustafson <ja...@confluent.io>
> wrote:
>
> > Hey Dong,
> >
> > Thanks for the updates. Just one question:
> >
> > When application receives
> > > this exception, the only choice will be to revert Kafka client library
> to
> > > an earlier version.
> >
> >
> > Not sure I follow this. Wouldn't we just restart the consumer? That would
> > cause it to fetch the previous committed offset and then fetch the
> correct
> > metadata.
> >
> > Thanks,
> > Jason
> >
> > On Tue, Dec 19, 2017 at 10:36 AM, Dong Lin <li...@gmail.com> wrote:
> >
> > > Hey Jason,
> > >
> > > Thanks for the comments. These make sense. I have updated the KIP to
> > > include a new error INVALID_LEADER_EPOCH. This will be a non-retriable
> > > error which may be thrown from consumer's API. When application
> receives
> > > this exception, the only choice will be to revert Kafka client library
> to
> > > an earlier version.
> > >
> > > Previously I think it may be better to simply log an error because I am
> > not
> > > sure it is a good idea to force user to downgrade Kafka client library
> > when
> > > the error itself, e.g. smaller leader epoch, may not be that fatal. One
> > the
> > > other hand it could be argued that we don't know what else can go wrong
> > in
> > > the buggy client library and it may be a good reason to force user to
> > > downgrade library.
> > >
> > > Thanks,
> > > Dong
> > >
> > >
> > > On Tue, Dec 19, 2017 at 9:06 AM, Jason Gustafson <ja...@confluent.io>
> > > wrote:
> > >
> > > > Hey Dong,
> > > >
> > > >
> > > > > I think it is a good idea to let coordinator do the additional
> sanity
> > > > check
> > > > > to ensure the leader epoch from OffsetCommitRequest never
> decreases.
> > > This
> > > > > can help us detect bug. The next question will be what should we do
> > if
> > > > > OffsetCommitRequest provides a smaller leader epoch. One possible
> > > > solution
> > > > > is to return a non-retriable error to consumer which will then be
> > > thrown
> > > > to
> > > > > user application. But I am not sure it is worth doing it given its
> > > impact
> > > > > on the user. Maybe it will be safer to simply have an error message
> > in
> > > > the
> > > > > server log and allow offset commit to succeed. What do you think?
> > > >
> > > >
> > > > I think the check would only have value if you return an error when
> it
> > > > fails. It seems primarily useful to detect buggy consumer logic, so a
> > > > non-retriable error makes sense to me. Clients which don't implement
> > this
> > > > capability can use the sentinel value and keep the current behavior.
> > > >
> > > > It seems that FetchResponse includes leader epoch via the path
> > > > > FetchResponse -> MemoryRecords -> MutableRecordBatch ->
> > > > DefaultRecordBatch
> > > > > -> partitionLeaderEpoch. Could this be an existing case where we
> > expose
> > > > the
> > > > > leader epoch to clients?
> > > >
> > > >
> > > > Right, in this case the client has no direct dependence on the field,
> > but
> > > > it could still be argued that it is exposed (I had actually
> considered
> > > > stuffing this field into an opaque blob of bytes in the message
> format
> > > > which the client wasn't allowed to touch, but it didn't happen in the
> > > end).
> > > > I'm not opposed to using the leader epoch field here, I was just
> > > mentioning
> > > > that it does tie clients a bit tighter to something which could be
> > > > considered a Kafka internal implementation detail. It makes the
> > protocol
> > > a
> > > > bit less intuitive as well since it is rather difficult to explain
> the
> > > edge
> > > > case it is protecting. That said, we've hit other scenarios where
> being
> > > > able to detect stale metadata in the client would be helpful, so I
> > think
> > > it
> > > > might be worth the tradeoff.
> > > >
> > > > -Jason
> > > >
> > > > On Mon, Dec 18, 2017 at 6:09 PM, Dong Lin <li...@gmail.com>
> wrote:
> > > >
> > > > > Hey Jason,
> > > > >
> > > > > Thanks much for reviewing the KIP.
> > > > >
> > > > > I think it is a good idea to let coordinator do the additional
> sanity
> > > > check
> > > > > to ensure the leader epoch from OffsetCommitRequest never
> decreases.
> > > This
> > > > > can help us detect bug. The next question will be what should we do
> > if
> > > > > OffsetCommitRequest provides a smaller leader epoch. One possible
> > > > solution
> > > > > is to return a non-retriable error to consumer which will then be
> > > thrown
> > > > to
> > > > > user application. But I am not sure it is worth doing it given its
> > > impact
> > > > > on the user. Maybe it will be safer to simply have an error message
> > in
> > > > the
> > > > > server log and allow offset commit to succeed. What do you think?
> > > > >
> > > > > It seems that FetchResponse includes leader epoch via the path
> > > > > FetchResponse -> MemoryRecords -> MutableRecordBatch ->
> > > > DefaultRecordBatch
> > > > > -> partitionLeaderEpoch. Could this be an existing case where we
> > expose
> > > > the
> > > > > leader epoch to clients?
> > > > >
> > > > > Thanks,
> > > > > Dong
> > > > >
> > > > >
> > > > >
> > > > > On Mon, Dec 18, 2017 at 3:27 PM, Jason Gustafson <
> jason@confluent.io
> > >
> > > > > wrote:
> > > > >
> > > > > > Hi Dong,
> > > > > >
> > > > > > Thanks for the KIP. Good job identifying the problem. One minor
> > > > question
> > > > > I
> > > > > > had is whether the coordinator should enforce that the leader
> epoch
> > > > > > associated with an offset commit can only go forward for each
> > > > partition?
> > > > > > Currently it looks like we just depend on the client for this,
> but
> > > > since
> > > > > > we're caching the leader epoch anyway, it seems like a cheap
> safety
> > > > > > condition. To support old clients, you can always allow the
> commit
> > if
> > > > the
> > > > > > leader epoch is unknown.
> > > > > >
> > > > > > I agree that we shouldn't expose the leader epoch in
> > > OffsetAndMetadata
> > > > in
> > > > > > the consumer API for what it's worth. As you have noted, it is
> more
> > > of
> > > > an
> > > > > > implementation detail. By the same argument, it's also a bit
> > > > unfortunate
> > > > > > that we have to expose it in the request API since that is nearly
> > as
> > > > > > binding in terms of how it limits future iterations. I could be
> > > wrong,
> > > > > but
> > > > > > this appears to be the first case where clients will depend on
> the
> > > > > concept
> > > > > > of leader epoch. Might not be a big deal considering how deeply
> > > > embedded
> > > > > > leader epochs already are in the inter-broker RPCs and the
> message
> > > > format
> > > > > > itself, but just wanted to mention the fact that good
> encapsulation
> > > > > applies
> > > > > > to the client request API as well.
> > > > > >
> > > > > > Thanks,
> > > > > > Jason
> > > > > >
> > > > > > On Mon, Dec 18, 2017 at 1:58 PM, Dong Lin <li...@gmail.com>
> > > wrote:
> > > > > >
> > > > > > > Hey Jun,
> > > > > > >
> > > > > > > Thanks much for your comments. These are very thoughtful ideas.
> > > > Please
> > > > > > see
> > > > > > > my comments below.
> > > > > > >
> > > > > > > On Thu, Dec 14, 2017 at 6:38 PM, Jun Rao <ju...@confluent.io>
> > wrote:
> > > > > > >
> > > > > > > > Hi, Dong,
> > > > > > > >
> > > > > > > > Thanks for the update. A few more comments below.
> > > > > > > >
> > > > > > > > 10. It seems that we need to return the leader epoch in the
> > fetch
> > > > > > > response
> > > > > > > > as well When fetching data, we could be fetching data from a
> > > leader
> > > > > > epoch
> > > > > > > > older than what's returned in the metadata response. So, we
> > want
> > > to
> > > > > use
> > > > > > > the
> > > > > > > > leader epoch associated with the offset being fetched for
> > > > committing
> > > > > > > > offsets.
> > > > > > > >
> > > > > > >
> > > > > > > It seems that we may have two separate issues here. The first
> > issue
> > > > is
> > > > > > that
> > > > > > > consumer uses metadata that is older than the one it uses
> before.
> > > The
> > > > > > > second issue is that consumer uses metadata which is newer than
> > the
> > > > > > > corresponding leader epoch in the leader broker. We know that
> the
> > > > > > > OffsetOutOfRangeException described in this KIP can be
> prevented
> > by
> > > > > > > avoiding the first issue. On the other hand, it seems that the
> > > > > > > OffsetOffsetOutOfRangeException can still happen even if we
> > avoid
> > > > the
> > > > > > > second issue -- if consumer uses an older version of metadata,
> > the
> > > > > leader
> > > > > > > epoch in its metadata may equal the leader epoch in the broker
> > even
> > > > if
> > > > > > the
> > > > > > > leader epoch in the broker is oudated.
> > > > > > >
> > > > > > > Given this understanding, I am not sure why we need to return
> the
> > > > > leader
> > > > > > > epoch in the fetch response. As long as consumer's metadata is
> > not
> > > > > going
> > > > > > > back in version, I think we are good. Did I miss something
> here?
> > > > > > >
> > > > > > >
> > > > > > > >
> > > > > > > > 11. Should we now extend OffsetAndMetadata used in the offset
> > > > commit
> > > > > > api
> > > > > > > in
> > > > > > > > KafkaConsumer to include leader epoch? Similarly, should we
> > > return
> > > > > > leader
> > > > > > > > epoch in endOffsets(), beginningOffsets() and position()? We
> > > > probably
> > > > > > > need
> > > > > > > > to think about how to make the api backward compatible.
> > > > > > > >
> > > > > > >
> > > > > > > After thinking through this carefully, I think we probably
> don't
> > > want
> > > > > to
> > > > > > > extend OffsetAndMetadata to include leader epoch because leader
> > > epoch
> > > > > is
> > > > > > > kind of implementation detail which ideally should be hidden
> from
> > > > user.
> > > > > > The
> > > > > > > consumer can include leader epoch in the OffsetCommitRequest
> > after
> > > > > taking
> > > > > > > offset from commitSync(final Map<TopicPartition,
> > OffsetAndMetadata>
> > > > > > > offsets). Similarly consumer can store leader epoch from
> > > > > > > OffsetFetchResponse and only provide offset to user via
> > > > > > > consumer.committed(topicPartition). This solution seems to
> work
> > > well
> > > > > and
> > > > > > > we
> > > > > > > don't have to make changes to consumer's public API. Does this
> > > sound
> > > > > OK?
> > > > > > >
> > > > > > >
> > > > > > > >
> > > > > > > > 12. It seems that we now need to store leader epoch in the
> > offset
> > > > > > topic.
> > > > > > > > Could you include the new schema for the value of the offset
> > > topic
> > > > > and
> > > > > > > add
> > > > > > > > upgrade notes?
> > > > > > >
> > > > > > >
> > > > > > > You are right. I have updated the KIP to specify the new schema
> > for
> > > > the
> > > > > > > value of the offset topic. Can you take another look?
> > > > > > >
> > > > > > > For existing messages in the offset topic, leader_epoch will be
> > > > > missing.
> > > > > > We
> > > > > > > will use leader_epoch = -1 to indicate the missing
> leader_epoch.
> > > Then
> > > > > the
> > > > > > > consumer behavior will be the same as it is now because any
> > > > > leader_epoch
> > > > > > in
> > > > > > > the MetadataResponse will be larger than the leader_epoch = -1
> in
> > > the
> > > > > > > OffetFetchResponse. Thus we don't need specific procedure for
> > > > upgrades
> > > > > > due
> > > > > > > to this change in the offset topic schema. By "upgrade nodes",
> do
> > > you
> > > > > > mean
> > > > > > > the sentences we need to include in the upgrade.html in the PR
> > > later?
> > > > > > >
> > > > > > >
> > > > > > > >
> > > > > > > > Jun
> > > > > > > >
> > > > > > > >
> > > > > > > > On Tue, Dec 12, 2017 at 5:19 PM, Dong Lin <
> lindong28@gmail.com
> > >
> > > > > wrote:
> > > > > > > >
> > > > > > > > > Hey Jun,
> > > > > > > > >
> > > > > > > > > I see. Sounds good. Yeah it is probably simpler to leave
> this
> > > to
> > > > > > > another
> > > > > > > > > KIP in the future.
> > > > > > > > >
> > > > > > > > > Thanks for all the comments. Since there is no further
> > comment
> > > in
> > > > > the
> > > > > > > > > community, I will open the voting thread.
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > > Dong
> > > > > > > > >
> > > > > > > > > On Mon, Dec 11, 2017 at 5:37 PM, Jun Rao <jun@confluent.io
> >
> > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hi, Dong,
> > > > > > > > > >
> > > > > > > > > > The case that I am thinking is network partitioning.
> > Suppose
> > > > one
> > > > > > > > deploys
> > > > > > > > > a
> > > > > > > > > > stretched cluster across multiple AZs in the same region.
> > If
> > > > the
> > > > > > > > machines
> > > > > > > > > > in one AZ can't communicate to brokers in other AZs due
> to
> > a
> > > > > > network
> > > > > > > > > issue,
> > > > > > > > > > the brokers in that AZ won't get any new metadata.
> > > > > > > > > >
> > > > > > > > > > We can potentially solve this problem by requiring some
> > kind
> > > of
> > > > > > > regular
> > > > > > > > > > heartbeats between the controller and the broker. This
> may
> > > need
> > > > > > some
> > > > > > > > more
> > > > > > > > > > thoughts. So, it's probably fine to leave this to another
> > KIP
> > > > in
> > > > > > the
> > > > > > > > > > future.
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > >
> > > > > > > > > > Jun
> > > > > > > > > >
> > > > > > > > > > On Mon, Dec 11, 2017 at 2:55 PM, Dong Lin <
> > > lindong28@gmail.com
> > > > >
> > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Hey Jun,
> > > > > > > > > > >
> > > > > > > > > > > Thanks for the comment. I am open to improve this KIP
> to
> > > > > address
> > > > > > > more
> > > > > > > > > > > problems. I probably need more help in understanding
> what
> > > is
> > > > > the
> > > > > > > > > current
> > > > > > > > > > > problem with consumer using outdated metadata and
> whether
> > > it
> > > > is
> > > > > > > > easier
> > > > > > > > > to
> > > > > > > > > > > address it together with this KIP.
> > > > > > > > > > >
> > > > > > > > > > > I agree that a consumer can potentially talk to old
> > leader
> > > > for
> > > > > a
> > > > > > > long
> > > > > > > > > > time
> > > > > > > > > > > even after this KIP. But after this KIP, the consumer
> > > > probably
> > > > > > > should
> > > > > > > > > not
> > > > > > > > > > > get OffetOutofRangeException and therefore will not
> cause
> > > > > offset
> > > > > > > > rewind
> > > > > > > > > > > issue. So the only problem is that consumer will not be
> > > able
> > > > to
> > > > > > > fetch
> > > > > > > > > > data
> > > > > > > > > > > until it has updated metadata. It seems that this
> > situation
> > > > can
> > > > > > > only
> > > > > > > > > > happen
> > > > > > > > > > > if the broker is too slow in processing
> > LeaderAndIsrRequest
> > > > > since
> > > > > > > > > > otherwise
> > > > > > > > > > > the consumer will be forced to update metadata due to
> > > > > > > > > > > NotLeaderForPartitionException. So the problem we are
> > > having
> > > > > > here
> > > > > > > is
> > > > > > > > > > that
> > > > > > > > > > > consumer will not be able to fetch data if some broker
> is
> > > too
> > > > > > slow
> > > > > > > in
> > > > > > > > > > > processing LeaderAndIsrRequest.
> > > > > > > > > > >
> > > > > > > > > > > Because Kafka propagates LeaderAndIsrRequest
> > asynchronously
> > > > to
> > > > > > all
> > > > > > > > > > brokers
> > > > > > > > > > > in the cluster, there will always be a period of time
> > when
> > > > > > consumer
> > > > > > > > can
> > > > > > > > > > not
> > > > > > > > > > > fetch data for the partition during the leadership
> > change.
> > > > Thus
> > > > > > it
> > > > > > > > > seems
> > > > > > > > > > > more like a broker-side performance issue instead of
> > > > > client-side
> > > > > > > > > > > correctness issue. My gut feel is that it is not
> causing
> > a
> > > > > much a
> > > > > > > > > problem
> > > > > > > > > > > as the problem to be fixed in this KIP. And if we were
> to
> > > > > address
> > > > > > > it,
> > > > > > > > > we
> > > > > > > > > > > probably need to make change in the broker side, e.g.
> > with
> > > > > > > > prioritized
> > > > > > > > > > > queue for controller-related requests, which may be
> kind
> > of
> > > > > > > > orthogonal
> > > > > > > > > to
> > > > > > > > > > > this KIP. I am not very sure it will be easier to
> address
> > > it
> > > > > with
> > > > > > > the
> > > > > > > > > > > change in this KIP. Do you have any recommendation?
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > > Dong
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Mon, Dec 11, 2017 at 1:51 PM, Jun Rao <
> > jun@confluent.io
> > > >
> > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Hi, Dong,
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks for the reply.
> > > > > > > > > > > >
> > > > > > > > > > > > My suggestion of forcing the metadata refresh from
> the
> > > > > > controller
> > > > > > > > may
> > > > > > > > > > not
> > > > > > > > > > > > work in general since the cached controller could be
> > > > outdated
> > > > > > > too.
> > > > > > > > > The
> > > > > > > > > > > > general problem is that if a consumer's metadata is
> > > > outdated,
> > > > > > it
> > > > > > > > may
> > > > > > > > > > get
> > > > > > > > > > > > stuck with the old leader for a long time. We can
> > address
> > > > the
> > > > > > > issue
> > > > > > > > > of
> > > > > > > > > > > > detecting outdated metadata in a separate KIP in the
> > > future
> > > > > if
> > > > > > > you
> > > > > > > > > > didn't
> > > > > > > > > > > > intend to address it in this KIP.
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks,
> > > > > > > > > > > >
> > > > > > > > > > > > Jun
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On Sat, Dec 9, 2017 at 10:12 PM, Dong Lin <
> > > > > lindong28@gmail.com
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Hey Jun,
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks much for your comments. Given that client
> > needs
> > > to
> > > > > > > > > > de-serialize
> > > > > > > > > > > > the
> > > > > > > > > > > > > metadata anyway, the extra overhead of checking the
> > > > > > > per-partition
> > > > > > > > > > > version
> > > > > > > > > > > > > for every partition should not be a big concern.
> Thus
> > > it
> > > > > > makes
> > > > > > > > > sense
> > > > > > > > > > to
> > > > > > > > > > > > use
> > > > > > > > > > > > > leader epoch as the per-partition version instead
> of
> > > > > > creating a
> > > > > > > > > > global
> > > > > > > > > > > > > metadata version. I will update the KIP to do that.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Regarding the detection of outdated metadata, I
> think
> > > it
> > > > is
> > > > > > > > > possible
> > > > > > > > > > to
> > > > > > > > > > > > > ensure that client gets latest metadata by fetching
> > > from
> > > > > > > > > controller.
> > > > > > > > > > > Note
> > > > > > > > > > > > > that this requires extra logic in the controller
> such
> > > > that
> > > > > > > > > controller
> > > > > > > > > > > > > updates metadata directly in memory without
> requiring
> > > > > > > > > > > > > UpdateMetadataRequest. But I am not sure the main
> > > > > motivation
> > > > > > of
> > > > > > > > > this
> > > > > > > > > > at
> > > > > > > > > > > > > this moment. But this makes controller more like a
> > > > > bottleneck
> > > > > > > in
> > > > > > > > > the
> > > > > > > > > > > > > cluster which we probably want to avoid.
> > > > > > > > > > > > >
> > > > > > > > > > > > > I think we can probably keep the current way of
> > > ensuring
> > > > > > > metadata
> > > > > > > > > > > > > freshness. Currently client will be forced to
> refresh
> > > > > > metadata
> > > > > > > if
> > > > > > > > > > > broker
> > > > > > > > > > > > > returns error (e.g. NotLeaderForPartition) due to
> > > > outdated
> > > > > > > > metadata
> > > > > > > > > > or
> > > > > > > > > > > if
> > > > > > > > > > > > > the metadata does not contain the partition that
> the
> > > > client
> > > > > > > > needs.
> > > > > > > > > In
> > > > > > > > > > > the
> > > > > > > > > > > > > future, as you previously suggested, we can include
> > > > > > > per-partition
> > > > > > > > > > > > > leaderEpoch in the FetchRequest/ProduceRequest such
> > > that
> > > > > > broker
> > > > > > > > can
> > > > > > > > > > > > return
> > > > > > > > > > > > > error if the epoch is smaller than cached epoch in
> > the
> > > > > > broker.
> > > > > > > > > Given
> > > > > > > > > > > that
> > > > > > > > > > > > > this adds more complexity to Kafka, I think we can
> > > > probably
> > > > > > > think
> > > > > > > > > > about
> > > > > > > > > > > > > that leader when we have a specific use-case or
> > problem
> > > > to
> > > > > > > solve
> > > > > > > > > with
> > > > > > > > > > > > > up-to-date metadata. Does this sound OK?
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > Dong
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Fri, Dec 8, 2017 at 3:53 PM, Jun Rao <
> > > > jun@confluent.io>
> > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Hi, Dong,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks for the reply. A few more points below.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > For dealing with how to prevent a consumer
> > switching
> > > > > from a
> > > > > > > new
> > > > > > > > > > > leader
> > > > > > > > > > > > to
> > > > > > > > > > > > > > an old leader, you suggestion that refreshes
> > metadata
> > > > on
> > > > > > > > consumer
> > > > > > > > > > > > restart
> > > > > > > > > > > > > > until it sees a metadata version >= the one
> > > associated
> > > > > with
> > > > > > > the
> > > > > > > > > > > offset
> > > > > > > > > > > > > > works too, as long as we guarantee that the
> cached
> > > > > metadata
> > > > > > > > > > versions
> > > > > > > > > > > on
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > brokers only go up.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > The second discussion point is on whether the
> > > metadata
> > > > > > > > versioning
> > > > > > > > > > > > should
> > > > > > > > > > > > > be
> > > > > > > > > > > > > > per partition or global. For the partition level
> > > > > > versioning,
> > > > > > > > you
> > > > > > > > > > were
> > > > > > > > > > > > > > concerned about the performance. Given that
> > metadata
> > > > > > updates
> > > > > > > > are
> > > > > > > > > > > rare,
> > > > > > > > > > > > I
> > > > > > > > > > > > > am
> > > > > > > > > > > > > > not sure if it's a big concern though. Doing a
> > > million
> > > > if
> > > > > > > tests
> > > > > > > > > is
> > > > > > > > > > > > > probably
> > > > > > > > > > > > > > going to take less than 1ms. Another thing is
> that
> > > the
> > > > > > > metadata
> > > > > > > > > > > version
> > > > > > > > > > > > > > seems to need to survive controller failover. In
> > your
> > > > > > current
> > > > > > > > > > > > approach, a
> > > > > > > > > > > > > > consumer may not be able to wait on the right
> > version
> > > > of
> > > > > > the
> > > > > > > > > > metadata
> > > > > > > > > > > > > after
> > > > > > > > > > > > > > the consumer restart since the metadata version
> may
> > > > have
> > > > > > been
> > > > > > > > > > > recycled
> > > > > > > > > > > > on
> > > > > > > > > > > > > > the server side due to a controller failover
> while
> > > the
> > > > > > > consumer
> > > > > > > > > is
> > > > > > > > > > > > down.
> > > > > > > > > > > > > > The partition level leaderEpoch survives
> controller
> > > > > failure
> > > > > > > and
> > > > > > > > > > won't
> > > > > > > > > > > > > have
> > > > > > > > > > > > > > this issue.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Lastly, neither your proposal nor mine addresses
> > the
> > > > > issue
> > > > > > > how
> > > > > > > > to
> > > > > > > > > > > > > guarantee
> > > > > > > > > > > > > > a consumer to detect that is metadata is
> outdated.
> > > > > > Currently,
> > > > > > > > the
> > > > > > > > > > > > > consumer
> > > > > > > > > > > > > > is not guaranteed to fetch metadata from every
> > broker
> > > > > > within
> > > > > > > > some
> > > > > > > > > > > > bounded
> > > > > > > > > > > > > > period of time. Maybe this is out of the scope of
> > > your
> > > > > KIP.
> > > > > > > But
> > > > > > > > > one
> > > > > > > > > > > > idea
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > force the consumer to refresh metadata from the
> > > > > controller
> > > > > > > > > > > > periodically.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Thu, Dec 7, 2017 at 11:25 AM, Dong Lin <
> > > > > > > lindong28@gmail.com
> > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Hey Jun,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks much for the comments. Great point
> > > > particularly
> > > > > > > > > regarding
> > > > > > > > > > > > (3). I
> > > > > > > > > > > > > > > haven't thought about this before.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > It seems that there are two possible ways where
> > the
> > > > > > version
> > > > > > > > > > number
> > > > > > > > > > > > can
> > > > > > > > > > > > > be
> > > > > > > > > > > > > > > used. One solution is for client to check the
> > > version
> > > > > > > number
> > > > > > > > at
> > > > > > > > > > the
> > > > > > > > > > > > > time
> > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > receives MetadataResponse. And if the version
> > > number
> > > > in
> > > > > > the
> > > > > > > > > > > > > > > MetadataResponse is smaller than the version
> > number
> > > > in
> > > > > > the
> > > > > > > > > > client's
> > > > > > > > > > > > > > cache,
> > > > > > > > > > > > > > > the client will be forced to fetch metadata
> > again.
> > > > > > Another
> > > > > > > > > > > solution,
> > > > > > > > > > > > > as
> > > > > > > > > > > > > > > you have suggested, is for broker to check the
> > > > version
> > > > > > > number
> > > > > > > > > at
> > > > > > > > > > > the
> > > > > > > > > > > > > time
> > > > > > > > > > > > > > > it receives a request from client. The broker
> > will
> > > > > reject
> > > > > > > the
> > > > > > > > > > > request
> > > > > > > > > > > > > if
> > > > > > > > > > > > > > > the version is smaller than the version in
> > broker's
> > > > > > cache.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > I am not very sure that the second solution can
> > > > address
> > > > > > the
> > > > > > > > > > problem
> > > > > > > > > > > > > here.
> > > > > > > > > > > > > > > In the scenario described in the JIRA ticket,
> > > > broker's
> > > > > > > cache
> > > > > > > > > may
> > > > > > > > > > be
> > > > > > > > > > > > > > > outdated because it has not processed the
> > > > > > > LeaderAndIsrRequest
> > > > > > > > > > from
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > controller. Thus it may still process client's
> > > > request
> > > > > > even
> > > > > > > > if
> > > > > > > > > > the
> > > > > > > > > > > > > > version
> > > > > > > > > > > > > > > in client's request is actually outdated. Does
> > this
> > > > > make
> > > > > > > > sense?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > IMO, it seems that we can address problem (3)
> by
> > > > saving
> > > > > > the
> > > > > > > > > > > metadata
> > > > > > > > > > > > > > > version together with the offset. After
> consumer
> > > > > starts,
> > > > > > it
> > > > > > > > > will
> > > > > > > > > > > keep
> > > > > > > > > > > > > > > fetching metadata until the metadata version >=
> > the
> > > > > > version
> > > > > > > > > saved
> > > > > > > > > > > > with
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > offset of this partition.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Regarding problems (1) and (2): Currently we
> use
> > > the
> > > > > > > version
> > > > > > > > > > number
> > > > > > > > > > > > in
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > MetadataResponse to ensure that the metadata
> does
> > > not
> > > > > go
> > > > > > > back
> > > > > > > > > in
> > > > > > > > > > > > time.
> > > > > > > > > > > > > > > There are two alternative solutions to address
> > > > problems
> > > > > > (1)
> > > > > > > > and
> > > > > > > > > > > (2).
> > > > > > > > > > > > > One
> > > > > > > > > > > > > > > solution is for client to enumerate all
> > partitions
> > > in
> > > > > the
> > > > > > > > > > > > > > MetadataResponse,
> > > > > > > > > > > > > > > compare their epoch with those in the cached
> > > > metadata,
> > > > > > and
> > > > > > > > > > rejects
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > MetadataResponse iff any leader epoch is
> smaller.
> > > The
> > > > > > main
> > > > > > > > > > concern
> > > > > > > > > > > is
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > MetadataResponse currently cached information
> of
> > > all
> > > > > > > > partitions
> > > > > > > > > > in
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > entire cluster. It may slow down client's
> > > performance
> > > > > if
> > > > > > we
> > > > > > > > > were
> > > > > > > > > > to
> > > > > > > > > > > > do
> > > > > > > > > > > > > > it.
> > > > > > > > > > > > > > > The other solution is for client to enumerate
> > > > > partitions
> > > > > > > for
> > > > > > > > > only
> > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > registered in the org.apache.kafka.clients.
> > > Metadata,
> > > > > > which
> > > > > > > > > will
> > > > > > > > > > be
> > > > > > > > > > > > an
> > > > > > > > > > > > > > > empty
> > > > > > > > > > > > > > > set for producer and the set of subscribed
> > > partitions
> > > > > for
> > > > > > > > > > consumer.
> > > > > > > > > > > > But
> > > > > > > > > > > > > > > this degrades to all topics if consumer
> > subscribes
> > > to
> > > > > > > topics
> > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > > > > > cluster
> > > > > > > > > > > > > > > by pattern.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Note that client will only be forced to update
> > > > metadata
> > > > > > if
> > > > > > > > the
> > > > > > > > > > > > version
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > the MetadataResponse is smaller than the
> version
> > in
> > > > the
> > > > > > > > cached
> > > > > > > > > > > > > metadata.
> > > > > > > > > > > > > > In
> > > > > > > > > > > > > > > general it should not be a problem. It can be a
> > > > problem
> > > > > > > only
> > > > > > > > if
> > > > > > > > > > > some
> > > > > > > > > > > > > > broker
> > > > > > > > > > > > > > > is particularly slower than other brokers in
> > > > processing
> > > > > > > > > > > > > > > UpdateMetadataRequest. When this is the case,
> it
> > > > means
> > > > > > that
> > > > > > > > the
> > > > > > > > > > > > broker
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > also particularly slower in processing
> > > > > > LeaderAndIsrRequest,
> > > > > > > > > which
> > > > > > > > > > > can
> > > > > > > > > > > > > > cause
> > > > > > > > > > > > > > > problem anyway because some partition will
> > probably
> > > > > have
> > > > > > no
> > > > > > > > > > leader
> > > > > > > > > > > > > during
> > > > > > > > > > > > > > > this period. I am not sure problems (1) and (2)
> > > cause
> > > > > > more
> > > > > > > > > > problem
> > > > > > > > > > > > than
> > > > > > > > > > > > > > > what we already have.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > Dong
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Wed, Dec 6, 2017 at 6:42 PM, Jun Rao <
> > > > > > jun@confluent.io>
> > > > > > > > > > wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Hi, Dong,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Great finding on the issue. It's a real
> > problem.
> > > A
> > > > > few
> > > > > > > > > comments
> > > > > > > > > > > > about
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > KIP. (1) I am not sure about updating
> > > > > > > > > controller_metadata_epoch
> > > > > > > > > > > on
> > > > > > > > > > > > > > every
> > > > > > > > > > > > > > > > UpdateMetadataRequest. Currently, the
> > controller
> > > > can
> > > > > > send
> > > > > > > > > > > > > > > > UpdateMetadataRequest when there is no actual
> > > > > metadata
> > > > > > > > > change.
> > > > > > > > > > > > Doing
> > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > may require unnecessary metadata refresh on
> the
> > > > > client.
> > > > > > > (2)
> > > > > > > > > > > > > > > > controller_metadata_epoch is global across
> all
> > > > > topics.
> > > > > > > This
> > > > > > > > > > means
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > client may be forced to update its metadata
> > even
> > > > when
> > > > > > the
> > > > > > > > > > > metadata
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > topics that it cares haven't changed. (3) It
> > > > doesn't
> > > > > > seem
> > > > > > > > > that
> > > > > > > > > > > the
> > > > > > > > > > > > > KIP
> > > > > > > > > > > > > > > > handles the corner case when a consumer is
> > > > restarted.
> > > > > > > Say a
> > > > > > > > > > > > consumer
> > > > > > > > > > > > > > > reads
> > > > > > > > > > > > > > > > from the new leader, commits the offset and
> > then
> > > is
> > > > > > > > > restarted.
> > > > > > > > > > On
> > > > > > > > > > > > > > > restart,
> > > > > > > > > > > > > > > > the consumer gets an outdated metadata and
> > > fetches
> > > > > from
> > > > > > > the
> > > > > > > > > old
> > > > > > > > > > > > > leader.
> > > > > > > > > > > > > > > > Then, the consumer will get into the offset
> out
> > > of
> > > > > > range
> > > > > > > > > issue.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Given the above, I am thinking of the
> following
> > > > > > approach.
> > > > > > > > We
> > > > > > > > > > > > actually
> > > > > > > > > > > > > > > > already have metadata versioning at the
> > partition
> > > > > > level.
> > > > > > > > Each
> > > > > > > > > > > > leader
> > > > > > > > > > > > > > has
> > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > leader epoch which is monotonically
> increasing.
> > > We
> > > > > can
> > > > > > > > > > > potentially
> > > > > > > > > > > > > > > > propagate leader epoch back in the metadata
> > > > response
> > > > > > and
> > > > > > > > the
> > > > > > > > > > > > clients
> > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > cache that. This solves the issue of (1) and
> > (2).
> > > > To
> > > > > > > solve
> > > > > > > > > (3),
> > > > > > > > > > > > when
> > > > > > > > > > > > > > > saving
> > > > > > > > > > > > > > > > an offset, we could save both an offset and
> the
> > > > > > > > corresponding
> > > > > > > > > > > > leader
> > > > > > > > > > > > > > > epoch.
> > > > > > > > > > > > > > > > When fetching the data, the consumer provides
> > > both
> > > > > the
> > > > > > > > offset
> > > > > > > > > > and
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > leader epoch. A leader will only serve the
> > > request
> > > > if
> > > > > > its
> > > > > > > > > > leader
> > > > > > > > > > > > > epoch
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > equal to or greater than the leader epoch
> from
> > > the
> > > > > > > > consumer.
> > > > > > > > > To
> > > > > > > > > > > > > achieve
> > > > > > > > > > > > > > > > this, we need to change the fetch request
> > > protocol
> > > > > and
> > > > > > > the
> > > > > > > > > > offset
> > > > > > > > > > > > > > commit
> > > > > > > > > > > > > > > > api, which requires some more thoughts.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Wed, Dec 6, 2017 at 10:57 AM, Dong Lin <
> > > > > > > > > lindong28@gmail.com
> > > > > > > > > > >
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Bump up the thread.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > It will be great to have more comments on
> > > whether
> > > > > we
> > > > > > > > should
> > > > > > > > > > do
> > > > > > > > > > > it
> > > > > > > > > > > > > or
> > > > > > > > > > > > > > > > > whether there is better way to address the
> > > > > motivation
> > > > > > > of
> > > > > > > > > this
> > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Mon, Dec 4, 2017 at 3:09 PM, Dong Lin <
> > > > > > > > > > lindong28@gmail.com>
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > I don't have an interesting rejected
> > > > alternative
> > > > > > > > solution
> > > > > > > > > > to
> > > > > > > > > > > > put
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > KIP. If there is good alternative
> solution
> > > from
> > > > > > > anyone
> > > > > > > > in
> > > > > > > > > > > this
> > > > > > > > > > > > > > > thread,
> > > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > am
> > > > > > > > > > > > > > > > > > happy to discuss this and update the KIP
> > > > > > accordingly.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > Dong
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > On Mon, Dec 4, 2017 at 1:12 PM, Ted Yu <
> > > > > > > > > > yuzhihong@gmail.com>
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >> It is clearer now.
> > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > >> I noticed that Rejected Alternatives
> > section
> > > > is
> > > > > > > empty.
> > > > > > > > > > > > > > > > > >> Have you considered any alternative ?
> > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > >> Cheers
> > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > >> On Mon, Dec 4, 2017 at 1:07 PM, Dong
> Lin <
> > > > > > > > > > > lindong28@gmail.com
> > > > > > > > > > > > >
> > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > >> > Ted, thanks for catching this. I have
> > > > updated
> > > > > > the
> > > > > > > > > > sentence
> > > > > > > > > > > > to
> > > > > > > > > > > > > > make
> > > > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > >> > readable.
> > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > >> > Thanks,
> > > > > > > > > > > > > > > > > >> > Dong
> > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > >> > On Sat, Dec 2, 2017 at 3:05 PM, Ted
> Yu <
> > > > > > > > > > > yuzhihong@gmail.com
> > > > > > > > > > > > >
> > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > >> > > bq. It the controller_epoch of the
> > > > incoming
> > > > > > > > > > > > > MetadataResponse,
> > > > > > > > > > > > > > or
> > > > > > > > > > > > > > > > if
> > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > >> > > controller_epoch is the same but the
> > > > > > > > > > > > > controller_metadata_epoch
> > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > >> > > Can you update the above sentence so
> > > that
> > > > > the
> > > > > > > > > > intention
> > > > > > > > > > > is
> > > > > > > > > > > > > > > > clearer ?
> > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > >> > > Thanks
> > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > >> > > On Fri, Dec 1, 2017 at 6:33 PM, Dong
> > > Lin <
> > > > > > > > > > > > > lindong28@gmail.com
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > >> > > > Hi all,
> > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > >> > > > I have created KIP-232: Detect
> > > outdated
> > > > > > > metadata
> > > > > > > > > by
> > > > > > > > > > > > adding
> > > > > > > > > > > > > > > > > >> > > > ControllerMetadataEpoch field:
> > > > > > > > > > > > > > > > > >> > > > https://cwiki.apache.org/
> > > > > > > > > > > confluence/display/KAFKA/KIP-
> > > > > > > > > > > > > > > > > >> > > > 232%3A+Detect+outdated+
> > > > > metadata+by+adding+
> > > > > > > > > > > > > > > > > >> > ControllerMetadataEpoch+field
> > > > > > > > > > > > > > > > > >> > > > .
> > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > >> > > > The KIP proposes to add fields in
> > > > > > > > MetadataResponse
> > > > > > > > > > and
> > > > > > > > > > > > > > > > > >> > > > UpdateMetadataRequest so that
> client
> > > can
> > > > > > > reject
> > > > > > > > > > > outdated
> > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > >> and
> > > > > > > > > > > > > > > > > >> > > avoid
> > > > > > > > > > > > > > > > > >> > > > unnecessary
> > OffsetOutOfRangeException.
> > > > > > > Otherwise
> > > > > > > > > > there
> > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > currently
> > > > > > > > > > > > > > > > > >> > race
> > > > > > > > > > > > > > > > > >> > > > condition that can cause consumer
> to
> > > > reset
> > > > > > > > offset
> > > > > > > > > > > which
> > > > > > > > > > > > > > > > negatively
> > > > > > > > > > > > > > > > > >> > affect
> > > > > > > > > > > > > > > > > >> > > > the consumer's availability.
> > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > >> > > > Feedback and suggestions are
> > welcome!
> > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > >> > > > Regards,
> > > > > > > > > > > > > > > > > >> > > > Dong
> > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Dong Lin <li...@gmail.com>.
Hey Jason,

Yeah this may sound a bit confusing. Let me explain my thoughts.

If there is no bug in the client library, after consumer rebalance or
consumer restart, consume will fetch the previously committed offset and
fetch the committed metadata until the leader epoch in the metadata >= the
leader epoch in the OffsetFetchResponse. Therefore, when consumer commits
offset later, the leader epoch in the OffsetCommitRequest should be larger
than the leader epoch from the previously committed offset. Does this sound
correct?

Given the above understanding, it seems to suggest that the only
explanation for this exception is that there is bug in the client library.
And due to this specific bug, I am not sure we can avoid this error by
simply restarting consumer. And because this error is non-retriable, user
may be forced to downgrade client library. Did I miss something here?

Thanks,
Dong


On Tue, Dec 19, 2017 at 11:19 AM, Jason Gustafson <ja...@confluent.io>
wrote:

> Hey Dong,
>
> Thanks for the updates. Just one question:
>
> When application receives
> > this exception, the only choice will be to revert Kafka client library to
> > an earlier version.
>
>
> Not sure I follow this. Wouldn't we just restart the consumer? That would
> cause it to fetch the previous committed offset and then fetch the correct
> metadata.
>
> Thanks,
> Jason
>
> On Tue, Dec 19, 2017 at 10:36 AM, Dong Lin <li...@gmail.com> wrote:
>
> > Hey Jason,
> >
> > Thanks for the comments. These make sense. I have updated the KIP to
> > include a new error INVALID_LEADER_EPOCH. This will be a non-retriable
> > error which may be thrown from consumer's API. When application receives
> > this exception, the only choice will be to revert Kafka client library to
> > an earlier version.
> >
> > Previously I think it may be better to simply log an error because I am
> not
> > sure it is a good idea to force user to downgrade Kafka client library
> when
> > the error itself, e.g. smaller leader epoch, may not be that fatal. One
> the
> > other hand it could be argued that we don't know what else can go wrong
> in
> > the buggy client library and it may be a good reason to force user to
> > downgrade library.
> >
> > Thanks,
> > Dong
> >
> >
> > On Tue, Dec 19, 2017 at 9:06 AM, Jason Gustafson <ja...@confluent.io>
> > wrote:
> >
> > > Hey Dong,
> > >
> > >
> > > > I think it is a good idea to let coordinator do the additional sanity
> > > check
> > > > to ensure the leader epoch from OffsetCommitRequest never decreases.
> > This
> > > > can help us detect bug. The next question will be what should we do
> if
> > > > OffsetCommitRequest provides a smaller leader epoch. One possible
> > > solution
> > > > is to return a non-retriable error to consumer which will then be
> > thrown
> > > to
> > > > user application. But I am not sure it is worth doing it given its
> > impact
> > > > on the user. Maybe it will be safer to simply have an error message
> in
> > > the
> > > > server log and allow offset commit to succeed. What do you think?
> > >
> > >
> > > I think the check would only have value if you return an error when it
> > > fails. It seems primarily useful to detect buggy consumer logic, so a
> > > non-retriable error makes sense to me. Clients which don't implement
> this
> > > capability can use the sentinel value and keep the current behavior.
> > >
> > > It seems that FetchResponse includes leader epoch via the path
> > > > FetchResponse -> MemoryRecords -> MutableRecordBatch ->
> > > DefaultRecordBatch
> > > > -> partitionLeaderEpoch. Could this be an existing case where we
> expose
> > > the
> > > > leader epoch to clients?
> > >
> > >
> > > Right, in this case the client has no direct dependence on the field,
> but
> > > it could still be argued that it is exposed (I had actually considered
> > > stuffing this field into an opaque blob of bytes in the message format
> > > which the client wasn't allowed to touch, but it didn't happen in the
> > end).
> > > I'm not opposed to using the leader epoch field here, I was just
> > mentioning
> > > that it does tie clients a bit tighter to something which could be
> > > considered a Kafka internal implementation detail. It makes the
> protocol
> > a
> > > bit less intuitive as well since it is rather difficult to explain the
> > edge
> > > case it is protecting. That said, we've hit other scenarios where being
> > > able to detect stale metadata in the client would be helpful, so I
> think
> > it
> > > might be worth the tradeoff.
> > >
> > > -Jason
> > >
> > > On Mon, Dec 18, 2017 at 6:09 PM, Dong Lin <li...@gmail.com> wrote:
> > >
> > > > Hey Jason,
> > > >
> > > > Thanks much for reviewing the KIP.
> > > >
> > > > I think it is a good idea to let coordinator do the additional sanity
> > > check
> > > > to ensure the leader epoch from OffsetCommitRequest never decreases.
> > This
> > > > can help us detect bug. The next question will be what should we do
> if
> > > > OffsetCommitRequest provides a smaller leader epoch. One possible
> > > solution
> > > > is to return a non-retriable error to consumer which will then be
> > thrown
> > > to
> > > > user application. But I am not sure it is worth doing it given its
> > impact
> > > > on the user. Maybe it will be safer to simply have an error message
> in
> > > the
> > > > server log and allow offset commit to succeed. What do you think?
> > > >
> > > > It seems that FetchResponse includes leader epoch via the path
> > > > FetchResponse -> MemoryRecords -> MutableRecordBatch ->
> > > DefaultRecordBatch
> > > > -> partitionLeaderEpoch. Could this be an existing case where we
> expose
> > > the
> > > > leader epoch to clients?
> > > >
> > > > Thanks,
> > > > Dong
> > > >
> > > >
> > > >
> > > > On Mon, Dec 18, 2017 at 3:27 PM, Jason Gustafson <jason@confluent.io
> >
> > > > wrote:
> > > >
> > > > > Hi Dong,
> > > > >
> > > > > Thanks for the KIP. Good job identifying the problem. One minor
> > > question
> > > > I
> > > > > had is whether the coordinator should enforce that the leader epoch
> > > > > associated with an offset commit can only go forward for each
> > > partition?
> > > > > Currently it looks like we just depend on the client for this, but
> > > since
> > > > > we're caching the leader epoch anyway, it seems like a cheap safety
> > > > > condition. To support old clients, you can always allow the commit
> if
> > > the
> > > > > leader epoch is unknown.
> > > > >
> > > > > I agree that we shouldn't expose the leader epoch in
> > OffsetAndMetadata
> > > in
> > > > > the consumer API for what it's worth. As you have noted, it is more
> > of
> > > an
> > > > > implementation detail. By the same argument, it's also a bit
> > > unfortunate
> > > > > that we have to expose it in the request API since that is nearly
> as
> > > > > binding in terms of how it limits future iterations. I could be
> > wrong,
> > > > but
> > > > > this appears to be the first case where clients will depend on the
> > > > concept
> > > > > of leader epoch. Might not be a big deal considering how deeply
> > > embedded
> > > > > leader epochs already are in the inter-broker RPCs and the message
> > > format
> > > > > itself, but just wanted to mention the fact that good encapsulation
> > > > applies
> > > > > to the client request API as well.
> > > > >
> > > > > Thanks,
> > > > > Jason
> > > > >
> > > > > On Mon, Dec 18, 2017 at 1:58 PM, Dong Lin <li...@gmail.com>
> > wrote:
> > > > >
> > > > > > Hey Jun,
> > > > > >
> > > > > > Thanks much for your comments. These are very thoughtful ideas.
> > > Please
> > > > > see
> > > > > > my comments below.
> > > > > >
> > > > > > On Thu, Dec 14, 2017 at 6:38 PM, Jun Rao <ju...@confluent.io>
> wrote:
> > > > > >
> > > > > > > Hi, Dong,
> > > > > > >
> > > > > > > Thanks for the update. A few more comments below.
> > > > > > >
> > > > > > > 10. It seems that we need to return the leader epoch in the
> fetch
> > > > > > response
> > > > > > > as well When fetching data, we could be fetching data from a
> > leader
> > > > > epoch
> > > > > > > older than what's returned in the metadata response. So, we
> want
> > to
> > > > use
> > > > > > the
> > > > > > > leader epoch associated with the offset being fetched for
> > > committing
> > > > > > > offsets.
> > > > > > >
> > > > > >
> > > > > > It seems that we may have two separate issues here. The first
> issue
> > > is
> > > > > that
> > > > > > consumer uses metadata that is older than the one it uses before.
> > The
> > > > > > second issue is that consumer uses metadata which is newer than
> the
> > > > > > corresponding leader epoch in the leader broker. We know that the
> > > > > > OffsetOutOfRangeException described in this KIP can be prevented
> by
> > > > > > avoiding the first issue. On the other hand, it seems that the
> > > > > > OffsetOffsetOutOfRangeException can still happen even if we
> avoid
> > > the
> > > > > > second issue -- if consumer uses an older version of metadata,
> the
> > > > leader
> > > > > > epoch in its metadata may equal the leader epoch in the broker
> even
> > > if
> > > > > the
> > > > > > leader epoch in the broker is oudated.
> > > > > >
> > > > > > Given this understanding, I am not sure why we need to return the
> > > > leader
> > > > > > epoch in the fetch response. As long as consumer's metadata is
> not
> > > > going
> > > > > > back in version, I think we are good. Did I miss something here?
> > > > > >
> > > > > >
> > > > > > >
> > > > > > > 11. Should we now extend OffsetAndMetadata used in the offset
> > > commit
> > > > > api
> > > > > > in
> > > > > > > KafkaConsumer to include leader epoch? Similarly, should we
> > return
> > > > > leader
> > > > > > > epoch in endOffsets(), beginningOffsets() and position()? We
> > > probably
> > > > > > need
> > > > > > > to think about how to make the api backward compatible.
> > > > > > >
> > > > > >
> > > > > > After thinking through this carefully, I think we probably don't
> > want
> > > > to
> > > > > > extend OffsetAndMetadata to include leader epoch because leader
> > epoch
> > > > is
> > > > > > kind of implementation detail which ideally should be hidden from
> > > user.
> > > > > The
> > > > > > consumer can include leader epoch in the OffsetCommitRequest
> after
> > > > taking
> > > > > > offset from commitSync(final Map<TopicPartition,
> OffsetAndMetadata>
> > > > > > offsets). Similarly consumer can store leader epoch from
> > > > > > OffsetFetchResponse and only provide offset to user via
> > > > > > consumer.committed(topicPartition). This solution seems to work
> > well
> > > > and
> > > > > > we
> > > > > > don't have to make changes to consumer's public API. Does this
> > sound
> > > > OK?
> > > > > >
> > > > > >
> > > > > > >
> > > > > > > 12. It seems that we now need to store leader epoch in the
> offset
> > > > > topic.
> > > > > > > Could you include the new schema for the value of the offset
> > topic
> > > > and
> > > > > > add
> > > > > > > upgrade notes?
> > > > > >
> > > > > >
> > > > > > You are right. I have updated the KIP to specify the new schema
> for
> > > the
> > > > > > value of the offset topic. Can you take another look?
> > > > > >
> > > > > > For existing messages in the offset topic, leader_epoch will be
> > > > missing.
> > > > > We
> > > > > > will use leader_epoch = -1 to indicate the missing leader_epoch.
> > Then
> > > > the
> > > > > > consumer behavior will be the same as it is now because any
> > > > leader_epoch
> > > > > in
> > > > > > the MetadataResponse will be larger than the leader_epoch = -1 in
> > the
> > > > > > OffetFetchResponse. Thus we don't need specific procedure for
> > > upgrades
> > > > > due
> > > > > > to this change in the offset topic schema. By "upgrade nodes", do
> > you
> > > > > mean
> > > > > > the sentences we need to include in the upgrade.html in the PR
> > later?
> > > > > >
> > > > > >
> > > > > > >
> > > > > > > Jun
> > > > > > >
> > > > > > >
> > > > > > > On Tue, Dec 12, 2017 at 5:19 PM, Dong Lin <lindong28@gmail.com
> >
> > > > wrote:
> > > > > > >
> > > > > > > > Hey Jun,
> > > > > > > >
> > > > > > > > I see. Sounds good. Yeah it is probably simpler to leave this
> > to
> > > > > > another
> > > > > > > > KIP in the future.
> > > > > > > >
> > > > > > > > Thanks for all the comments. Since there is no further
> comment
> > in
> > > > the
> > > > > > > > community, I will open the voting thread.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Dong
> > > > > > > >
> > > > > > > > On Mon, Dec 11, 2017 at 5:37 PM, Jun Rao <ju...@confluent.io>
> > > wrote:
> > > > > > > >
> > > > > > > > > Hi, Dong,
> > > > > > > > >
> > > > > > > > > The case that I am thinking is network partitioning.
> Suppose
> > > one
> > > > > > > deploys
> > > > > > > > a
> > > > > > > > > stretched cluster across multiple AZs in the same region.
> If
> > > the
> > > > > > > machines
> > > > > > > > > in one AZ can't communicate to brokers in other AZs due to
> a
> > > > > network
> > > > > > > > issue,
> > > > > > > > > the brokers in that AZ won't get any new metadata.
> > > > > > > > >
> > > > > > > > > We can potentially solve this problem by requiring some
> kind
> > of
> > > > > > regular
> > > > > > > > > heartbeats between the controller and the broker. This may
> > need
> > > > > some
> > > > > > > more
> > > > > > > > > thoughts. So, it's probably fine to leave this to another
> KIP
> > > in
> > > > > the
> > > > > > > > > future.
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > >
> > > > > > > > > Jun
> > > > > > > > >
> > > > > > > > > On Mon, Dec 11, 2017 at 2:55 PM, Dong Lin <
> > lindong28@gmail.com
> > > >
> > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hey Jun,
> > > > > > > > > >
> > > > > > > > > > Thanks for the comment. I am open to improve this KIP to
> > > > address
> > > > > > more
> > > > > > > > > > problems. I probably need more help in understanding what
> > is
> > > > the
> > > > > > > > current
> > > > > > > > > > problem with consumer using outdated metadata and whether
> > it
> > > is
> > > > > > > easier
> > > > > > > > to
> > > > > > > > > > address it together with this KIP.
> > > > > > > > > >
> > > > > > > > > > I agree that a consumer can potentially talk to old
> leader
> > > for
> > > > a
> > > > > > long
> > > > > > > > > time
> > > > > > > > > > even after this KIP. But after this KIP, the consumer
> > > probably
> > > > > > should
> > > > > > > > not
> > > > > > > > > > get OffetOutofRangeException and therefore will not cause
> > > > offset
> > > > > > > rewind
> > > > > > > > > > issue. So the only problem is that consumer will not be
> > able
> > > to
> > > > > > fetch
> > > > > > > > > data
> > > > > > > > > > until it has updated metadata. It seems that this
> situation
> > > can
> > > > > > only
> > > > > > > > > happen
> > > > > > > > > > if the broker is too slow in processing
> LeaderAndIsrRequest
> > > > since
> > > > > > > > > otherwise
> > > > > > > > > > the consumer will be forced to update metadata due to
> > > > > > > > > > NotLeaderForPartitionException. So the problem we are
> > having
> > > > > here
> > > > > > is
> > > > > > > > > that
> > > > > > > > > > consumer will not be able to fetch data if some broker is
> > too
> > > > > slow
> > > > > > in
> > > > > > > > > > processing LeaderAndIsrRequest.
> > > > > > > > > >
> > > > > > > > > > Because Kafka propagates LeaderAndIsrRequest
> asynchronously
> > > to
> > > > > all
> > > > > > > > > brokers
> > > > > > > > > > in the cluster, there will always be a period of time
> when
> > > > > consumer
> > > > > > > can
> > > > > > > > > not
> > > > > > > > > > fetch data for the partition during the leadership
> change.
> > > Thus
> > > > > it
> > > > > > > > seems
> > > > > > > > > > more like a broker-side performance issue instead of
> > > > client-side
> > > > > > > > > > correctness issue. My gut feel is that it is not causing
> a
> > > > much a
> > > > > > > > problem
> > > > > > > > > > as the problem to be fixed in this KIP. And if we were to
> > > > address
> > > > > > it,
> > > > > > > > we
> > > > > > > > > > probably need to make change in the broker side, e.g.
> with
> > > > > > > prioritized
> > > > > > > > > > queue for controller-related requests, which may be kind
> of
> > > > > > > orthogonal
> > > > > > > > to
> > > > > > > > > > this KIP. I am not very sure it will be easier to address
> > it
> > > > with
> > > > > > the
> > > > > > > > > > change in this KIP. Do you have any recommendation?
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > > Dong
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Mon, Dec 11, 2017 at 1:51 PM, Jun Rao <
> jun@confluent.io
> > >
> > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Hi, Dong,
> > > > > > > > > > >
> > > > > > > > > > > Thanks for the reply.
> > > > > > > > > > >
> > > > > > > > > > > My suggestion of forcing the metadata refresh from the
> > > > > controller
> > > > > > > may
> > > > > > > > > not
> > > > > > > > > > > work in general since the cached controller could be
> > > outdated
> > > > > > too.
> > > > > > > > The
> > > > > > > > > > > general problem is that if a consumer's metadata is
> > > outdated,
> > > > > it
> > > > > > > may
> > > > > > > > > get
> > > > > > > > > > > stuck with the old leader for a long time. We can
> address
> > > the
> > > > > > issue
> > > > > > > > of
> > > > > > > > > > > detecting outdated metadata in a separate KIP in the
> > future
> > > > if
> > > > > > you
> > > > > > > > > didn't
> > > > > > > > > > > intend to address it in this KIP.
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > >
> > > > > > > > > > > Jun
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Sat, Dec 9, 2017 at 10:12 PM, Dong Lin <
> > > > lindong28@gmail.com
> > > > > >
> > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Hey Jun,
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks much for your comments. Given that client
> needs
> > to
> > > > > > > > > de-serialize
> > > > > > > > > > > the
> > > > > > > > > > > > metadata anyway, the extra overhead of checking the
> > > > > > per-partition
> > > > > > > > > > version
> > > > > > > > > > > > for every partition should not be a big concern. Thus
> > it
> > > > > makes
> > > > > > > > sense
> > > > > > > > > to
> > > > > > > > > > > use
> > > > > > > > > > > > leader epoch as the per-partition version instead of
> > > > > creating a
> > > > > > > > > global
> > > > > > > > > > > > metadata version. I will update the KIP to do that.
> > > > > > > > > > > >
> > > > > > > > > > > > Regarding the detection of outdated metadata, I think
> > it
> > > is
> > > > > > > > possible
> > > > > > > > > to
> > > > > > > > > > > > ensure that client gets latest metadata by fetching
> > from
> > > > > > > > controller.
> > > > > > > > > > Note
> > > > > > > > > > > > that this requires extra logic in the controller such
> > > that
> > > > > > > > controller
> > > > > > > > > > > > updates metadata directly in memory without requiring
> > > > > > > > > > > > UpdateMetadataRequest. But I am not sure the main
> > > > motivation
> > > > > of
> > > > > > > > this
> > > > > > > > > at
> > > > > > > > > > > > this moment. But this makes controller more like a
> > > > bottleneck
> > > > > > in
> > > > > > > > the
> > > > > > > > > > > > cluster which we probably want to avoid.
> > > > > > > > > > > >
> > > > > > > > > > > > I think we can probably keep the current way of
> > ensuring
> > > > > > metadata
> > > > > > > > > > > > freshness. Currently client will be forced to refresh
> > > > > metadata
> > > > > > if
> > > > > > > > > > broker
> > > > > > > > > > > > returns error (e.g. NotLeaderForPartition) due to
> > > outdated
> > > > > > > metadata
> > > > > > > > > or
> > > > > > > > > > if
> > > > > > > > > > > > the metadata does not contain the partition that the
> > > client
> > > > > > > needs.
> > > > > > > > In
> > > > > > > > > > the
> > > > > > > > > > > > future, as you previously suggested, we can include
> > > > > > per-partition
> > > > > > > > > > > > leaderEpoch in the FetchRequest/ProduceRequest such
> > that
> > > > > broker
> > > > > > > can
> > > > > > > > > > > return
> > > > > > > > > > > > error if the epoch is smaller than cached epoch in
> the
> > > > > broker.
> > > > > > > > Given
> > > > > > > > > > that
> > > > > > > > > > > > this adds more complexity to Kafka, I think we can
> > > probably
> > > > > > think
> > > > > > > > > about
> > > > > > > > > > > > that leader when we have a specific use-case or
> problem
> > > to
> > > > > > solve
> > > > > > > > with
> > > > > > > > > > > > up-to-date metadata. Does this sound OK?
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks,
> > > > > > > > > > > > Dong
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On Fri, Dec 8, 2017 at 3:53 PM, Jun Rao <
> > > jun@confluent.io>
> > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Hi, Dong,
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks for the reply. A few more points below.
> > > > > > > > > > > > >
> > > > > > > > > > > > > For dealing with how to prevent a consumer
> switching
> > > > from a
> > > > > > new
> > > > > > > > > > leader
> > > > > > > > > > > to
> > > > > > > > > > > > > an old leader, you suggestion that refreshes
> metadata
> > > on
> > > > > > > consumer
> > > > > > > > > > > restart
> > > > > > > > > > > > > until it sees a metadata version >= the one
> > associated
> > > > with
> > > > > > the
> > > > > > > > > > offset
> > > > > > > > > > > > > works too, as long as we guarantee that the cached
> > > > metadata
> > > > > > > > > versions
> > > > > > > > > > on
> > > > > > > > > > > > the
> > > > > > > > > > > > > brokers only go up.
> > > > > > > > > > > > >
> > > > > > > > > > > > > The second discussion point is on whether the
> > metadata
> > > > > > > versioning
> > > > > > > > > > > should
> > > > > > > > > > > > be
> > > > > > > > > > > > > per partition or global. For the partition level
> > > > > versioning,
> > > > > > > you
> > > > > > > > > were
> > > > > > > > > > > > > concerned about the performance. Given that
> metadata
> > > > > updates
> > > > > > > are
> > > > > > > > > > rare,
> > > > > > > > > > > I
> > > > > > > > > > > > am
> > > > > > > > > > > > > not sure if it's a big concern though. Doing a
> > million
> > > if
> > > > > > tests
> > > > > > > > is
> > > > > > > > > > > > probably
> > > > > > > > > > > > > going to take less than 1ms. Another thing is that
> > the
> > > > > > metadata
> > > > > > > > > > version
> > > > > > > > > > > > > seems to need to survive controller failover. In
> your
> > > > > current
> > > > > > > > > > > approach, a
> > > > > > > > > > > > > consumer may not be able to wait on the right
> version
> > > of
> > > > > the
> > > > > > > > > metadata
> > > > > > > > > > > > after
> > > > > > > > > > > > > the consumer restart since the metadata version may
> > > have
> > > > > been
> > > > > > > > > > recycled
> > > > > > > > > > > on
> > > > > > > > > > > > > the server side due to a controller failover while
> > the
> > > > > > consumer
> > > > > > > > is
> > > > > > > > > > > down.
> > > > > > > > > > > > > The partition level leaderEpoch survives controller
> > > > failure
> > > > > > and
> > > > > > > > > won't
> > > > > > > > > > > > have
> > > > > > > > > > > > > this issue.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Lastly, neither your proposal nor mine addresses
> the
> > > > issue
> > > > > > how
> > > > > > > to
> > > > > > > > > > > > guarantee
> > > > > > > > > > > > > a consumer to detect that is metadata is outdated.
> > > > > Currently,
> > > > > > > the
> > > > > > > > > > > > consumer
> > > > > > > > > > > > > is not guaranteed to fetch metadata from every
> broker
> > > > > within
> > > > > > > some
> > > > > > > > > > > bounded
> > > > > > > > > > > > > period of time. Maybe this is out of the scope of
> > your
> > > > KIP.
> > > > > > But
> > > > > > > > one
> > > > > > > > > > > idea
> > > > > > > > > > > > is
> > > > > > > > > > > > > force the consumer to refresh metadata from the
> > > > controller
> > > > > > > > > > > periodically.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Jun
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Thu, Dec 7, 2017 at 11:25 AM, Dong Lin <
> > > > > > lindong28@gmail.com
> > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Hey Jun,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks much for the comments. Great point
> > > particularly
> > > > > > > > regarding
> > > > > > > > > > > (3). I
> > > > > > > > > > > > > > haven't thought about this before.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > It seems that there are two possible ways where
> the
> > > > > version
> > > > > > > > > number
> > > > > > > > > > > can
> > > > > > > > > > > > be
> > > > > > > > > > > > > > used. One solution is for client to check the
> > version
> > > > > > number
> > > > > > > at
> > > > > > > > > the
> > > > > > > > > > > > time
> > > > > > > > > > > > > it
> > > > > > > > > > > > > > receives MetadataResponse. And if the version
> > number
> > > in
> > > > > the
> > > > > > > > > > > > > > MetadataResponse is smaller than the version
> number
> > > in
> > > > > the
> > > > > > > > > client's
> > > > > > > > > > > > > cache,
> > > > > > > > > > > > > > the client will be forced to fetch metadata
> again.
> > > > > Another
> > > > > > > > > > solution,
> > > > > > > > > > > > as
> > > > > > > > > > > > > > you have suggested, is for broker to check the
> > > version
> > > > > > number
> > > > > > > > at
> > > > > > > > > > the
> > > > > > > > > > > > time
> > > > > > > > > > > > > > it receives a request from client. The broker
> will
> > > > reject
> > > > > > the
> > > > > > > > > > request
> > > > > > > > > > > > if
> > > > > > > > > > > > > > the version is smaller than the version in
> broker's
> > > > > cache.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > I am not very sure that the second solution can
> > > address
> > > > > the
> > > > > > > > > problem
> > > > > > > > > > > > here.
> > > > > > > > > > > > > > In the scenario described in the JIRA ticket,
> > > broker's
> > > > > > cache
> > > > > > > > may
> > > > > > > > > be
> > > > > > > > > > > > > > outdated because it has not processed the
> > > > > > LeaderAndIsrRequest
> > > > > > > > > from
> > > > > > > > > > > the
> > > > > > > > > > > > > > controller. Thus it may still process client's
> > > request
> > > > > even
> > > > > > > if
> > > > > > > > > the
> > > > > > > > > > > > > version
> > > > > > > > > > > > > > in client's request is actually outdated. Does
> this
> > > > make
> > > > > > > sense?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > IMO, it seems that we can address problem (3) by
> > > saving
> > > > > the
> > > > > > > > > > metadata
> > > > > > > > > > > > > > version together with the offset. After consumer
> > > > starts,
> > > > > it
> > > > > > > > will
> > > > > > > > > > keep
> > > > > > > > > > > > > > fetching metadata until the metadata version >=
> the
> > > > > version
> > > > > > > > saved
> > > > > > > > > > > with
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > offset of this partition.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Regarding problems (1) and (2): Currently we use
> > the
> > > > > > version
> > > > > > > > > number
> > > > > > > > > > > in
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > MetadataResponse to ensure that the metadata does
> > not
> > > > go
> > > > > > back
> > > > > > > > in
> > > > > > > > > > > time.
> > > > > > > > > > > > > > There are two alternative solutions to address
> > > problems
> > > > > (1)
> > > > > > > and
> > > > > > > > > > (2).
> > > > > > > > > > > > One
> > > > > > > > > > > > > > solution is for client to enumerate all
> partitions
> > in
> > > > the
> > > > > > > > > > > > > MetadataResponse,
> > > > > > > > > > > > > > compare their epoch with those in the cached
> > > metadata,
> > > > > and
> > > > > > > > > rejects
> > > > > > > > > > > the
> > > > > > > > > > > > > > MetadataResponse iff any leader epoch is smaller.
> > The
> > > > > main
> > > > > > > > > concern
> > > > > > > > > > is
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > MetadataResponse currently cached information of
> > all
> > > > > > > partitions
> > > > > > > > > in
> > > > > > > > > > > the
> > > > > > > > > > > > > > entire cluster. It may slow down client's
> > performance
> > > > if
> > > > > we
> > > > > > > > were
> > > > > > > > > to
> > > > > > > > > > > do
> > > > > > > > > > > > > it.
> > > > > > > > > > > > > > The other solution is for client to enumerate
> > > > partitions
> > > > > > for
> > > > > > > > only
> > > > > > > > > > > > topics
> > > > > > > > > > > > > > registered in the org.apache.kafka.clients.
> > Metadata,
> > > > > which
> > > > > > > > will
> > > > > > > > > be
> > > > > > > > > > > an
> > > > > > > > > > > > > > empty
> > > > > > > > > > > > > > set for producer and the set of subscribed
> > partitions
> > > > for
> > > > > > > > > consumer.
> > > > > > > > > > > But
> > > > > > > > > > > > > > this degrades to all topics if consumer
> subscribes
> > to
> > > > > > topics
> > > > > > > in
> > > > > > > > > the
> > > > > > > > > > > > > cluster
> > > > > > > > > > > > > > by pattern.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Note that client will only be forced to update
> > > metadata
> > > > > if
> > > > > > > the
> > > > > > > > > > > version
> > > > > > > > > > > > in
> > > > > > > > > > > > > > the MetadataResponse is smaller than the version
> in
> > > the
> > > > > > > cached
> > > > > > > > > > > > metadata.
> > > > > > > > > > > > > In
> > > > > > > > > > > > > > general it should not be a problem. It can be a
> > > problem
> > > > > > only
> > > > > > > if
> > > > > > > > > > some
> > > > > > > > > > > > > broker
> > > > > > > > > > > > > > is particularly slower than other brokers in
> > > processing
> > > > > > > > > > > > > > UpdateMetadataRequest. When this is the case, it
> > > means
> > > > > that
> > > > > > > the
> > > > > > > > > > > broker
> > > > > > > > > > > > is
> > > > > > > > > > > > > > also particularly slower in processing
> > > > > LeaderAndIsrRequest,
> > > > > > > > which
> > > > > > > > > > can
> > > > > > > > > > > > > cause
> > > > > > > > > > > > > > problem anyway because some partition will
> probably
> > > > have
> > > > > no
> > > > > > > > > leader
> > > > > > > > > > > > during
> > > > > > > > > > > > > > this period. I am not sure problems (1) and (2)
> > cause
> > > > > more
> > > > > > > > > problem
> > > > > > > > > > > than
> > > > > > > > > > > > > > what we already have.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > Dong
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Wed, Dec 6, 2017 at 6:42 PM, Jun Rao <
> > > > > jun@confluent.io>
> > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Hi, Dong,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Great finding on the issue. It's a real
> problem.
> > A
> > > > few
> > > > > > > > comments
> > > > > > > > > > > about
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > KIP. (1) I am not sure about updating
> > > > > > > > controller_metadata_epoch
> > > > > > > > > > on
> > > > > > > > > > > > > every
> > > > > > > > > > > > > > > UpdateMetadataRequest. Currently, the
> controller
> > > can
> > > > > send
> > > > > > > > > > > > > > > UpdateMetadataRequest when there is no actual
> > > > metadata
> > > > > > > > change.
> > > > > > > > > > > Doing
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > may require unnecessary metadata refresh on the
> > > > client.
> > > > > > (2)
> > > > > > > > > > > > > > > controller_metadata_epoch is global across all
> > > > topics.
> > > > > > This
> > > > > > > > > means
> > > > > > > > > > > > that
> > > > > > > > > > > > > a
> > > > > > > > > > > > > > > client may be forced to update its metadata
> even
> > > when
> > > > > the
> > > > > > > > > > metadata
> > > > > > > > > > > > for
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > topics that it cares haven't changed. (3) It
> > > doesn't
> > > > > seem
> > > > > > > > that
> > > > > > > > > > the
> > > > > > > > > > > > KIP
> > > > > > > > > > > > > > > handles the corner case when a consumer is
> > > restarted.
> > > > > > Say a
> > > > > > > > > > > consumer
> > > > > > > > > > > > > > reads
> > > > > > > > > > > > > > > from the new leader, commits the offset and
> then
> > is
> > > > > > > > restarted.
> > > > > > > > > On
> > > > > > > > > > > > > > restart,
> > > > > > > > > > > > > > > the consumer gets an outdated metadata and
> > fetches
> > > > from
> > > > > > the
> > > > > > > > old
> > > > > > > > > > > > leader.
> > > > > > > > > > > > > > > Then, the consumer will get into the offset out
> > of
> > > > > range
> > > > > > > > issue.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Given the above, I am thinking of the following
> > > > > approach.
> > > > > > > We
> > > > > > > > > > > actually
> > > > > > > > > > > > > > > already have metadata versioning at the
> partition
> > > > > level.
> > > > > > > Each
> > > > > > > > > > > leader
> > > > > > > > > > > > > has
> > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > leader epoch which is monotonically increasing.
> > We
> > > > can
> > > > > > > > > > potentially
> > > > > > > > > > > > > > > propagate leader epoch back in the metadata
> > > response
> > > > > and
> > > > > > > the
> > > > > > > > > > > clients
> > > > > > > > > > > > > can
> > > > > > > > > > > > > > > cache that. This solves the issue of (1) and
> (2).
> > > To
> > > > > > solve
> > > > > > > > (3),
> > > > > > > > > > > when
> > > > > > > > > > > > > > saving
> > > > > > > > > > > > > > > an offset, we could save both an offset and the
> > > > > > > corresponding
> > > > > > > > > > > leader
> > > > > > > > > > > > > > epoch.
> > > > > > > > > > > > > > > When fetching the data, the consumer provides
> > both
> > > > the
> > > > > > > offset
> > > > > > > > > and
> > > > > > > > > > > the
> > > > > > > > > > > > > > > leader epoch. A leader will only serve the
> > request
> > > if
> > > > > its
> > > > > > > > > leader
> > > > > > > > > > > > epoch
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > equal to or greater than the leader epoch from
> > the
> > > > > > > consumer.
> > > > > > > > To
> > > > > > > > > > > > achieve
> > > > > > > > > > > > > > > this, we need to change the fetch request
> > protocol
> > > > and
> > > > > > the
> > > > > > > > > offset
> > > > > > > > > > > > > commit
> > > > > > > > > > > > > > > api, which requires some more thoughts.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Wed, Dec 6, 2017 at 10:57 AM, Dong Lin <
> > > > > > > > lindong28@gmail.com
> > > > > > > > > >
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Bump up the thread.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > It will be great to have more comments on
> > whether
> > > > we
> > > > > > > should
> > > > > > > > > do
> > > > > > > > > > it
> > > > > > > > > > > > or
> > > > > > > > > > > > > > > > whether there is better way to address the
> > > > motivation
> > > > > > of
> > > > > > > > this
> > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Mon, Dec 4, 2017 at 3:09 PM, Dong Lin <
> > > > > > > > > lindong28@gmail.com>
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > I don't have an interesting rejected
> > > alternative
> > > > > > > solution
> > > > > > > > > to
> > > > > > > > > > > put
> > > > > > > > > > > > in
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > KIP. If there is good alternative solution
> > from
> > > > > > anyone
> > > > > > > in
> > > > > > > > > > this
> > > > > > > > > > > > > > thread,
> > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > am
> > > > > > > > > > > > > > > > > happy to discuss this and update the KIP
> > > > > accordingly.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > Dong
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Mon, Dec 4, 2017 at 1:12 PM, Ted Yu <
> > > > > > > > > yuzhihong@gmail.com>
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >> It is clearer now.
> > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > >> I noticed that Rejected Alternatives
> section
> > > is
> > > > > > empty.
> > > > > > > > > > > > > > > > >> Have you considered any alternative ?
> > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > >> Cheers
> > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > >> On Mon, Dec 4, 2017 at 1:07 PM, Dong Lin <
> > > > > > > > > > lindong28@gmail.com
> > > > > > > > > > > >
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > >> > Ted, thanks for catching this. I have
> > > updated
> > > > > the
> > > > > > > > > sentence
> > > > > > > > > > > to
> > > > > > > > > > > > > make
> > > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > >> > readable.
> > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > >> > Thanks,
> > > > > > > > > > > > > > > > >> > Dong
> > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > >> > On Sat, Dec 2, 2017 at 3:05 PM, Ted Yu <
> > > > > > > > > > yuzhihong@gmail.com
> > > > > > > > > > > >
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > >> > > bq. It the controller_epoch of the
> > > incoming
> > > > > > > > > > > > MetadataResponse,
> > > > > > > > > > > > > or
> > > > > > > > > > > > > > > if
> > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > >> > > controller_epoch is the same but the
> > > > > > > > > > > > controller_metadata_epoch
> > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > >> > > Can you update the above sentence so
> > that
> > > > the
> > > > > > > > > intention
> > > > > > > > > > is
> > > > > > > > > > > > > > > clearer ?
> > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > >> > > Thanks
> > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > >> > > On Fri, Dec 1, 2017 at 6:33 PM, Dong
> > Lin <
> > > > > > > > > > > > lindong28@gmail.com
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > >> > > > Hi all,
> > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > >> > > > I have created KIP-232: Detect
> > outdated
> > > > > > metadata
> > > > > > > > by
> > > > > > > > > > > adding
> > > > > > > > > > > > > > > > >> > > > ControllerMetadataEpoch field:
> > > > > > > > > > > > > > > > >> > > > https://cwiki.apache.org/
> > > > > > > > > > confluence/display/KAFKA/KIP-
> > > > > > > > > > > > > > > > >> > > > 232%3A+Detect+outdated+
> > > > metadata+by+adding+
> > > > > > > > > > > > > > > > >> > ControllerMetadataEpoch+field
> > > > > > > > > > > > > > > > >> > > > .
> > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > >> > > > The KIP proposes to add fields in
> > > > > > > MetadataResponse
> > > > > > > > > and
> > > > > > > > > > > > > > > > >> > > > UpdateMetadataRequest so that client
> > can
> > > > > > reject
> > > > > > > > > > outdated
> > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > >> and
> > > > > > > > > > > > > > > > >> > > avoid
> > > > > > > > > > > > > > > > >> > > > unnecessary
> OffsetOutOfRangeException.
> > > > > > Otherwise
> > > > > > > > > there
> > > > > > > > > > > is
> > > > > > > > > > > > > > > > currently
> > > > > > > > > > > > > > > > >> > race
> > > > > > > > > > > > > > > > >> > > > condition that can cause consumer to
> > > reset
> > > > > > > offset
> > > > > > > > > > which
> > > > > > > > > > > > > > > negatively
> > > > > > > > > > > > > > > > >> > affect
> > > > > > > > > > > > > > > > >> > > > the consumer's availability.
> > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > >> > > > Feedback and suggestions are
> welcome!
> > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > >> > > > Regards,
> > > > > > > > > > > > > > > > >> > > > Dong
> > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Jason Gustafson <ja...@confluent.io>.
Hey Dong,

Thanks for the updates. Just one question:

When application receives
> this exception, the only choice will be to revert Kafka client library to
> an earlier version.


Not sure I follow this. Wouldn't we just restart the consumer? That would
cause it to fetch the previous committed offset and then fetch the correct
metadata.

Thanks,
Jason

On Tue, Dec 19, 2017 at 10:36 AM, Dong Lin <li...@gmail.com> wrote:

> Hey Jason,
>
> Thanks for the comments. These make sense. I have updated the KIP to
> include a new error INVALID_LEADER_EPOCH. This will be a non-retriable
> error which may be thrown from consumer's API. When application receives
> this exception, the only choice will be to revert Kafka client library to
> an earlier version.
>
> Previously I think it may be better to simply log an error because I am not
> sure it is a good idea to force user to downgrade Kafka client library when
> the error itself, e.g. smaller leader epoch, may not be that fatal. One the
> other hand it could be argued that we don't know what else can go wrong in
> the buggy client library and it may be a good reason to force user to
> downgrade library.
>
> Thanks,
> Dong
>
>
> On Tue, Dec 19, 2017 at 9:06 AM, Jason Gustafson <ja...@confluent.io>
> wrote:
>
> > Hey Dong,
> >
> >
> > > I think it is a good idea to let coordinator do the additional sanity
> > check
> > > to ensure the leader epoch from OffsetCommitRequest never decreases.
> This
> > > can help us detect bug. The next question will be what should we do if
> > > OffsetCommitRequest provides a smaller leader epoch. One possible
> > solution
> > > is to return a non-retriable error to consumer which will then be
> thrown
> > to
> > > user application. But I am not sure it is worth doing it given its
> impact
> > > on the user. Maybe it will be safer to simply have an error message in
> > the
> > > server log and allow offset commit to succeed. What do you think?
> >
> >
> > I think the check would only have value if you return an error when it
> > fails. It seems primarily useful to detect buggy consumer logic, so a
> > non-retriable error makes sense to me. Clients which don't implement this
> > capability can use the sentinel value and keep the current behavior.
> >
> > It seems that FetchResponse includes leader epoch via the path
> > > FetchResponse -> MemoryRecords -> MutableRecordBatch ->
> > DefaultRecordBatch
> > > -> partitionLeaderEpoch. Could this be an existing case where we expose
> > the
> > > leader epoch to clients?
> >
> >
> > Right, in this case the client has no direct dependence on the field, but
> > it could still be argued that it is exposed (I had actually considered
> > stuffing this field into an opaque blob of bytes in the message format
> > which the client wasn't allowed to touch, but it didn't happen in the
> end).
> > I'm not opposed to using the leader epoch field here, I was just
> mentioning
> > that it does tie clients a bit tighter to something which could be
> > considered a Kafka internal implementation detail. It makes the protocol
> a
> > bit less intuitive as well since it is rather difficult to explain the
> edge
> > case it is protecting. That said, we've hit other scenarios where being
> > able to detect stale metadata in the client would be helpful, so I think
> it
> > might be worth the tradeoff.
> >
> > -Jason
> >
> > On Mon, Dec 18, 2017 at 6:09 PM, Dong Lin <li...@gmail.com> wrote:
> >
> > > Hey Jason,
> > >
> > > Thanks much for reviewing the KIP.
> > >
> > > I think it is a good idea to let coordinator do the additional sanity
> > check
> > > to ensure the leader epoch from OffsetCommitRequest never decreases.
> This
> > > can help us detect bug. The next question will be what should we do if
> > > OffsetCommitRequest provides a smaller leader epoch. One possible
> > solution
> > > is to return a non-retriable error to consumer which will then be
> thrown
> > to
> > > user application. But I am not sure it is worth doing it given its
> impact
> > > on the user. Maybe it will be safer to simply have an error message in
> > the
> > > server log and allow offset commit to succeed. What do you think?
> > >
> > > It seems that FetchResponse includes leader epoch via the path
> > > FetchResponse -> MemoryRecords -> MutableRecordBatch ->
> > DefaultRecordBatch
> > > -> partitionLeaderEpoch. Could this be an existing case where we expose
> > the
> > > leader epoch to clients?
> > >
> > > Thanks,
> > > Dong
> > >
> > >
> > >
> > > On Mon, Dec 18, 2017 at 3:27 PM, Jason Gustafson <ja...@confluent.io>
> > > wrote:
> > >
> > > > Hi Dong,
> > > >
> > > > Thanks for the KIP. Good job identifying the problem. One minor
> > question
> > > I
> > > > had is whether the coordinator should enforce that the leader epoch
> > > > associated with an offset commit can only go forward for each
> > partition?
> > > > Currently it looks like we just depend on the client for this, but
> > since
> > > > we're caching the leader epoch anyway, it seems like a cheap safety
> > > > condition. To support old clients, you can always allow the commit if
> > the
> > > > leader epoch is unknown.
> > > >
> > > > I agree that we shouldn't expose the leader epoch in
> OffsetAndMetadata
> > in
> > > > the consumer API for what it's worth. As you have noted, it is more
> of
> > an
> > > > implementation detail. By the same argument, it's also a bit
> > unfortunate
> > > > that we have to expose it in the request API since that is nearly as
> > > > binding in terms of how it limits future iterations. I could be
> wrong,
> > > but
> > > > this appears to be the first case where clients will depend on the
> > > concept
> > > > of leader epoch. Might not be a big deal considering how deeply
> > embedded
> > > > leader epochs already are in the inter-broker RPCs and the message
> > format
> > > > itself, but just wanted to mention the fact that good encapsulation
> > > applies
> > > > to the client request API as well.
> > > >
> > > > Thanks,
> > > > Jason
> > > >
> > > > On Mon, Dec 18, 2017 at 1:58 PM, Dong Lin <li...@gmail.com>
> wrote:
> > > >
> > > > > Hey Jun,
> > > > >
> > > > > Thanks much for your comments. These are very thoughtful ideas.
> > Please
> > > > see
> > > > > my comments below.
> > > > >
> > > > > On Thu, Dec 14, 2017 at 6:38 PM, Jun Rao <ju...@confluent.io> wrote:
> > > > >
> > > > > > Hi, Dong,
> > > > > >
> > > > > > Thanks for the update. A few more comments below.
> > > > > >
> > > > > > 10. It seems that we need to return the leader epoch in the fetch
> > > > > response
> > > > > > as well When fetching data, we could be fetching data from a
> leader
> > > > epoch
> > > > > > older than what's returned in the metadata response. So, we want
> to
> > > use
> > > > > the
> > > > > > leader epoch associated with the offset being fetched for
> > committing
> > > > > > offsets.
> > > > > >
> > > > >
> > > > > It seems that we may have two separate issues here. The first issue
> > is
> > > > that
> > > > > consumer uses metadata that is older than the one it uses before.
> The
> > > > > second issue is that consumer uses metadata which is newer than the
> > > > > corresponding leader epoch in the leader broker. We know that the
> > > > > OffsetOutOfRangeException described in this KIP can be prevented by
> > > > > avoiding the first issue. On the other hand, it seems that the
> > > > > OffsetOffsetOutOfRangeException can still happen even if we avoid
> > the
> > > > > second issue -- if consumer uses an older version of metadata, the
> > > leader
> > > > > epoch in its metadata may equal the leader epoch in the broker even
> > if
> > > > the
> > > > > leader epoch in the broker is oudated.
> > > > >
> > > > > Given this understanding, I am not sure why we need to return the
> > > leader
> > > > > epoch in the fetch response. As long as consumer's metadata is not
> > > going
> > > > > back in version, I think we are good. Did I miss something here?
> > > > >
> > > > >
> > > > > >
> > > > > > 11. Should we now extend OffsetAndMetadata used in the offset
> > commit
> > > > api
> > > > > in
> > > > > > KafkaConsumer to include leader epoch? Similarly, should we
> return
> > > > leader
> > > > > > epoch in endOffsets(), beginningOffsets() and position()? We
> > probably
> > > > > need
> > > > > > to think about how to make the api backward compatible.
> > > > > >
> > > > >
> > > > > After thinking through this carefully, I think we probably don't
> want
> > > to
> > > > > extend OffsetAndMetadata to include leader epoch because leader
> epoch
> > > is
> > > > > kind of implementation detail which ideally should be hidden from
> > user.
> > > > The
> > > > > consumer can include leader epoch in the OffsetCommitRequest after
> > > taking
> > > > > offset from commitSync(final Map<TopicPartition, OffsetAndMetadata>
> > > > > offsets). Similarly consumer can store leader epoch from
> > > > > OffsetFetchResponse and only provide offset to user via
> > > > > consumer.committed(topicPartition). This solution seems to work
> well
> > > and
> > > > > we
> > > > > don't have to make changes to consumer's public API. Does this
> sound
> > > OK?
> > > > >
> > > > >
> > > > > >
> > > > > > 12. It seems that we now need to store leader epoch in the offset
> > > > topic.
> > > > > > Could you include the new schema for the value of the offset
> topic
> > > and
> > > > > add
> > > > > > upgrade notes?
> > > > >
> > > > >
> > > > > You are right. I have updated the KIP to specify the new schema for
> > the
> > > > > value of the offset topic. Can you take another look?
> > > > >
> > > > > For existing messages in the offset topic, leader_epoch will be
> > > missing.
> > > > We
> > > > > will use leader_epoch = -1 to indicate the missing leader_epoch.
> Then
> > > the
> > > > > consumer behavior will be the same as it is now because any
> > > leader_epoch
> > > > in
> > > > > the MetadataResponse will be larger than the leader_epoch = -1 in
> the
> > > > > OffetFetchResponse. Thus we don't need specific procedure for
> > upgrades
> > > > due
> > > > > to this change in the offset topic schema. By "upgrade nodes", do
> you
> > > > mean
> > > > > the sentences we need to include in the upgrade.html in the PR
> later?
> > > > >
> > > > >
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > >
> > > > > > On Tue, Dec 12, 2017 at 5:19 PM, Dong Lin <li...@gmail.com>
> > > wrote:
> > > > > >
> > > > > > > Hey Jun,
> > > > > > >
> > > > > > > I see. Sounds good. Yeah it is probably simpler to leave this
> to
> > > > > another
> > > > > > > KIP in the future.
> > > > > > >
> > > > > > > Thanks for all the comments. Since there is no further comment
> in
> > > the
> > > > > > > community, I will open the voting thread.
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Dong
> > > > > > >
> > > > > > > On Mon, Dec 11, 2017 at 5:37 PM, Jun Rao <ju...@confluent.io>
> > wrote:
> > > > > > >
> > > > > > > > Hi, Dong,
> > > > > > > >
> > > > > > > > The case that I am thinking is network partitioning. Suppose
> > one
> > > > > > deploys
> > > > > > > a
> > > > > > > > stretched cluster across multiple AZs in the same region. If
> > the
> > > > > > machines
> > > > > > > > in one AZ can't communicate to brokers in other AZs due to a
> > > > network
> > > > > > > issue,
> > > > > > > > the brokers in that AZ won't get any new metadata.
> > > > > > > >
> > > > > > > > We can potentially solve this problem by requiring some kind
> of
> > > > > regular
> > > > > > > > heartbeats between the controller and the broker. This may
> need
> > > > some
> > > > > > more
> > > > > > > > thoughts. So, it's probably fine to leave this to another KIP
> > in
> > > > the
> > > > > > > > future.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > >
> > > > > > > > Jun
> > > > > > > >
> > > > > > > > On Mon, Dec 11, 2017 at 2:55 PM, Dong Lin <
> lindong28@gmail.com
> > >
> > > > > wrote:
> > > > > > > >
> > > > > > > > > Hey Jun,
> > > > > > > > >
> > > > > > > > > Thanks for the comment. I am open to improve this KIP to
> > > address
> > > > > more
> > > > > > > > > problems. I probably need more help in understanding what
> is
> > > the
> > > > > > > current
> > > > > > > > > problem with consumer using outdated metadata and whether
> it
> > is
> > > > > > easier
> > > > > > > to
> > > > > > > > > address it together with this KIP.
> > > > > > > > >
> > > > > > > > > I agree that a consumer can potentially talk to old leader
> > for
> > > a
> > > > > long
> > > > > > > > time
> > > > > > > > > even after this KIP. But after this KIP, the consumer
> > probably
> > > > > should
> > > > > > > not
> > > > > > > > > get OffetOutofRangeException and therefore will not cause
> > > offset
> > > > > > rewind
> > > > > > > > > issue. So the only problem is that consumer will not be
> able
> > to
> > > > > fetch
> > > > > > > > data
> > > > > > > > > until it has updated metadata. It seems that this situation
> > can
> > > > > only
> > > > > > > > happen
> > > > > > > > > if the broker is too slow in processing LeaderAndIsrRequest
> > > since
> > > > > > > > otherwise
> > > > > > > > > the consumer will be forced to update metadata due to
> > > > > > > > > NotLeaderForPartitionException. So the problem we are
> having
> > > > here
> > > > > is
> > > > > > > > that
> > > > > > > > > consumer will not be able to fetch data if some broker is
> too
> > > > slow
> > > > > in
> > > > > > > > > processing LeaderAndIsrRequest.
> > > > > > > > >
> > > > > > > > > Because Kafka propagates LeaderAndIsrRequest asynchronously
> > to
> > > > all
> > > > > > > > brokers
> > > > > > > > > in the cluster, there will always be a period of time when
> > > > consumer
> > > > > > can
> > > > > > > > not
> > > > > > > > > fetch data for the partition during the leadership change.
> > Thus
> > > > it
> > > > > > > seems
> > > > > > > > > more like a broker-side performance issue instead of
> > > client-side
> > > > > > > > > correctness issue. My gut feel is that it is not causing a
> > > much a
> > > > > > > problem
> > > > > > > > > as the problem to be fixed in this KIP. And if we were to
> > > address
> > > > > it,
> > > > > > > we
> > > > > > > > > probably need to make change in the broker side, e.g. with
> > > > > > prioritized
> > > > > > > > > queue for controller-related requests, which may be kind of
> > > > > > orthogonal
> > > > > > > to
> > > > > > > > > this KIP. I am not very sure it will be easier to address
> it
> > > with
> > > > > the
> > > > > > > > > change in this KIP. Do you have any recommendation?
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > > Dong
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Mon, Dec 11, 2017 at 1:51 PM, Jun Rao <jun@confluent.io
> >
> > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hi, Dong,
> > > > > > > > > >
> > > > > > > > > > Thanks for the reply.
> > > > > > > > > >
> > > > > > > > > > My suggestion of forcing the metadata refresh from the
> > > > controller
> > > > > > may
> > > > > > > > not
> > > > > > > > > > work in general since the cached controller could be
> > outdated
> > > > > too.
> > > > > > > The
> > > > > > > > > > general problem is that if a consumer's metadata is
> > outdated,
> > > > it
> > > > > > may
> > > > > > > > get
> > > > > > > > > > stuck with the old leader for a long time. We can address
> > the
> > > > > issue
> > > > > > > of
> > > > > > > > > > detecting outdated metadata in a separate KIP in the
> future
> > > if
> > > > > you
> > > > > > > > didn't
> > > > > > > > > > intend to address it in this KIP.
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > >
> > > > > > > > > > Jun
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Sat, Dec 9, 2017 at 10:12 PM, Dong Lin <
> > > lindong28@gmail.com
> > > > >
> > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Hey Jun,
> > > > > > > > > > >
> > > > > > > > > > > Thanks much for your comments. Given that client needs
> to
> > > > > > > > de-serialize
> > > > > > > > > > the
> > > > > > > > > > > metadata anyway, the extra overhead of checking the
> > > > > per-partition
> > > > > > > > > version
> > > > > > > > > > > for every partition should not be a big concern. Thus
> it
> > > > makes
> > > > > > > sense
> > > > > > > > to
> > > > > > > > > > use
> > > > > > > > > > > leader epoch as the per-partition version instead of
> > > > creating a
> > > > > > > > global
> > > > > > > > > > > metadata version. I will update the KIP to do that.
> > > > > > > > > > >
> > > > > > > > > > > Regarding the detection of outdated metadata, I think
> it
> > is
> > > > > > > possible
> > > > > > > > to
> > > > > > > > > > > ensure that client gets latest metadata by fetching
> from
> > > > > > > controller.
> > > > > > > > > Note
> > > > > > > > > > > that this requires extra logic in the controller such
> > that
> > > > > > > controller
> > > > > > > > > > > updates metadata directly in memory without requiring
> > > > > > > > > > > UpdateMetadataRequest. But I am not sure the main
> > > motivation
> > > > of
> > > > > > > this
> > > > > > > > at
> > > > > > > > > > > this moment. But this makes controller more like a
> > > bottleneck
> > > > > in
> > > > > > > the
> > > > > > > > > > > cluster which we probably want to avoid.
> > > > > > > > > > >
> > > > > > > > > > > I think we can probably keep the current way of
> ensuring
> > > > > metadata
> > > > > > > > > > > freshness. Currently client will be forced to refresh
> > > > metadata
> > > > > if
> > > > > > > > > broker
> > > > > > > > > > > returns error (e.g. NotLeaderForPartition) due to
> > outdated
> > > > > > metadata
> > > > > > > > or
> > > > > > > > > if
> > > > > > > > > > > the metadata does not contain the partition that the
> > client
> > > > > > needs.
> > > > > > > In
> > > > > > > > > the
> > > > > > > > > > > future, as you previously suggested, we can include
> > > > > per-partition
> > > > > > > > > > > leaderEpoch in the FetchRequest/ProduceRequest such
> that
> > > > broker
> > > > > > can
> > > > > > > > > > return
> > > > > > > > > > > error if the epoch is smaller than cached epoch in the
> > > > broker.
> > > > > > > Given
> > > > > > > > > that
> > > > > > > > > > > this adds more complexity to Kafka, I think we can
> > probably
> > > > > think
> > > > > > > > about
> > > > > > > > > > > that leader when we have a specific use-case or problem
> > to
> > > > > solve
> > > > > > > with
> > > > > > > > > > > up-to-date metadata. Does this sound OK?
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > > Dong
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Fri, Dec 8, 2017 at 3:53 PM, Jun Rao <
> > jun@confluent.io>
> > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Hi, Dong,
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks for the reply. A few more points below.
> > > > > > > > > > > >
> > > > > > > > > > > > For dealing with how to prevent a consumer switching
> > > from a
> > > > > new
> > > > > > > > > leader
> > > > > > > > > > to
> > > > > > > > > > > > an old leader, you suggestion that refreshes metadata
> > on
> > > > > > consumer
> > > > > > > > > > restart
> > > > > > > > > > > > until it sees a metadata version >= the one
> associated
> > > with
> > > > > the
> > > > > > > > > offset
> > > > > > > > > > > > works too, as long as we guarantee that the cached
> > > metadata
> > > > > > > > versions
> > > > > > > > > on
> > > > > > > > > > > the
> > > > > > > > > > > > brokers only go up.
> > > > > > > > > > > >
> > > > > > > > > > > > The second discussion point is on whether the
> metadata
> > > > > > versioning
> > > > > > > > > > should
> > > > > > > > > > > be
> > > > > > > > > > > > per partition or global. For the partition level
> > > > versioning,
> > > > > > you
> > > > > > > > were
> > > > > > > > > > > > concerned about the performance. Given that metadata
> > > > updates
> > > > > > are
> > > > > > > > > rare,
> > > > > > > > > > I
> > > > > > > > > > > am
> > > > > > > > > > > > not sure if it's a big concern though. Doing a
> million
> > if
> > > > > tests
> > > > > > > is
> > > > > > > > > > > probably
> > > > > > > > > > > > going to take less than 1ms. Another thing is that
> the
> > > > > metadata
> > > > > > > > > version
> > > > > > > > > > > > seems to need to survive controller failover. In your
> > > > current
> > > > > > > > > > approach, a
> > > > > > > > > > > > consumer may not be able to wait on the right version
> > of
> > > > the
> > > > > > > > metadata
> > > > > > > > > > > after
> > > > > > > > > > > > the consumer restart since the metadata version may
> > have
> > > > been
> > > > > > > > > recycled
> > > > > > > > > > on
> > > > > > > > > > > > the server side due to a controller failover while
> the
> > > > > consumer
> > > > > > > is
> > > > > > > > > > down.
> > > > > > > > > > > > The partition level leaderEpoch survives controller
> > > failure
> > > > > and
> > > > > > > > won't
> > > > > > > > > > > have
> > > > > > > > > > > > this issue.
> > > > > > > > > > > >
> > > > > > > > > > > > Lastly, neither your proposal nor mine addresses the
> > > issue
> > > > > how
> > > > > > to
> > > > > > > > > > > guarantee
> > > > > > > > > > > > a consumer to detect that is metadata is outdated.
> > > > Currently,
> > > > > > the
> > > > > > > > > > > consumer
> > > > > > > > > > > > is not guaranteed to fetch metadata from every broker
> > > > within
> > > > > > some
> > > > > > > > > > bounded
> > > > > > > > > > > > period of time. Maybe this is out of the scope of
> your
> > > KIP.
> > > > > But
> > > > > > > one
> > > > > > > > > > idea
> > > > > > > > > > > is
> > > > > > > > > > > > force the consumer to refresh metadata from the
> > > controller
> > > > > > > > > > periodically.
> > > > > > > > > > > >
> > > > > > > > > > > > Jun
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On Thu, Dec 7, 2017 at 11:25 AM, Dong Lin <
> > > > > lindong28@gmail.com
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Hey Jun,
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks much for the comments. Great point
> > particularly
> > > > > > > regarding
> > > > > > > > > > (3). I
> > > > > > > > > > > > > haven't thought about this before.
> > > > > > > > > > > > >
> > > > > > > > > > > > > It seems that there are two possible ways where the
> > > > version
> > > > > > > > number
> > > > > > > > > > can
> > > > > > > > > > > be
> > > > > > > > > > > > > used. One solution is for client to check the
> version
> > > > > number
> > > > > > at
> > > > > > > > the
> > > > > > > > > > > time
> > > > > > > > > > > > it
> > > > > > > > > > > > > receives MetadataResponse. And if the version
> number
> > in
> > > > the
> > > > > > > > > > > > > MetadataResponse is smaller than the version number
> > in
> > > > the
> > > > > > > > client's
> > > > > > > > > > > > cache,
> > > > > > > > > > > > > the client will be forced to fetch metadata again.
> > > > Another
> > > > > > > > > solution,
> > > > > > > > > > > as
> > > > > > > > > > > > > you have suggested, is for broker to check the
> > version
> > > > > number
> > > > > > > at
> > > > > > > > > the
> > > > > > > > > > > time
> > > > > > > > > > > > > it receives a request from client. The broker will
> > > reject
> > > > > the
> > > > > > > > > request
> > > > > > > > > > > if
> > > > > > > > > > > > > the version is smaller than the version in broker's
> > > > cache.
> > > > > > > > > > > > >
> > > > > > > > > > > > > I am not very sure that the second solution can
> > address
> > > > the
> > > > > > > > problem
> > > > > > > > > > > here.
> > > > > > > > > > > > > In the scenario described in the JIRA ticket,
> > broker's
> > > > > cache
> > > > > > > may
> > > > > > > > be
> > > > > > > > > > > > > outdated because it has not processed the
> > > > > LeaderAndIsrRequest
> > > > > > > > from
> > > > > > > > > > the
> > > > > > > > > > > > > controller. Thus it may still process client's
> > request
> > > > even
> > > > > > if
> > > > > > > > the
> > > > > > > > > > > > version
> > > > > > > > > > > > > in client's request is actually outdated. Does this
> > > make
> > > > > > sense?
> > > > > > > > > > > > >
> > > > > > > > > > > > > IMO, it seems that we can address problem (3) by
> > saving
> > > > the
> > > > > > > > > metadata
> > > > > > > > > > > > > version together with the offset. After consumer
> > > starts,
> > > > it
> > > > > > > will
> > > > > > > > > keep
> > > > > > > > > > > > > fetching metadata until the metadata version >= the
> > > > version
> > > > > > > saved
> > > > > > > > > > with
> > > > > > > > > > > > the
> > > > > > > > > > > > > offset of this partition.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Regarding problems (1) and (2): Currently we use
> the
> > > > > version
> > > > > > > > number
> > > > > > > > > > in
> > > > > > > > > > > > the
> > > > > > > > > > > > > MetadataResponse to ensure that the metadata does
> not
> > > go
> > > > > back
> > > > > > > in
> > > > > > > > > > time.
> > > > > > > > > > > > > There are two alternative solutions to address
> > problems
> > > > (1)
> > > > > > and
> > > > > > > > > (2).
> > > > > > > > > > > One
> > > > > > > > > > > > > solution is for client to enumerate all partitions
> in
> > > the
> > > > > > > > > > > > MetadataResponse,
> > > > > > > > > > > > > compare their epoch with those in the cached
> > metadata,
> > > > and
> > > > > > > > rejects
> > > > > > > > > > the
> > > > > > > > > > > > > MetadataResponse iff any leader epoch is smaller.
> The
> > > > main
> > > > > > > > concern
> > > > > > > > > is
> > > > > > > > > > > > that
> > > > > > > > > > > > > MetadataResponse currently cached information of
> all
> > > > > > partitions
> > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > > > > entire cluster. It may slow down client's
> performance
> > > if
> > > > we
> > > > > > > were
> > > > > > > > to
> > > > > > > > > > do
> > > > > > > > > > > > it.
> > > > > > > > > > > > > The other solution is for client to enumerate
> > > partitions
> > > > > for
> > > > > > > only
> > > > > > > > > > > topics
> > > > > > > > > > > > > registered in the org.apache.kafka.clients.
> Metadata,
> > > > which
> > > > > > > will
> > > > > > > > be
> > > > > > > > > > an
> > > > > > > > > > > > > empty
> > > > > > > > > > > > > set for producer and the set of subscribed
> partitions
> > > for
> > > > > > > > consumer.
> > > > > > > > > > But
> > > > > > > > > > > > > this degrades to all topics if consumer subscribes
> to
> > > > > topics
> > > > > > in
> > > > > > > > the
> > > > > > > > > > > > cluster
> > > > > > > > > > > > > by pattern.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Note that client will only be forced to update
> > metadata
> > > > if
> > > > > > the
> > > > > > > > > > version
> > > > > > > > > > > in
> > > > > > > > > > > > > the MetadataResponse is smaller than the version in
> > the
> > > > > > cached
> > > > > > > > > > > metadata.
> > > > > > > > > > > > In
> > > > > > > > > > > > > general it should not be a problem. It can be a
> > problem
> > > > > only
> > > > > > if
> > > > > > > > > some
> > > > > > > > > > > > broker
> > > > > > > > > > > > > is particularly slower than other brokers in
> > processing
> > > > > > > > > > > > > UpdateMetadataRequest. When this is the case, it
> > means
> > > > that
> > > > > > the
> > > > > > > > > > broker
> > > > > > > > > > > is
> > > > > > > > > > > > > also particularly slower in processing
> > > > LeaderAndIsrRequest,
> > > > > > > which
> > > > > > > > > can
> > > > > > > > > > > > cause
> > > > > > > > > > > > > problem anyway because some partition will probably
> > > have
> > > > no
> > > > > > > > leader
> > > > > > > > > > > during
> > > > > > > > > > > > > this period. I am not sure problems (1) and (2)
> cause
> > > > more
> > > > > > > > problem
> > > > > > > > > > than
> > > > > > > > > > > > > what we already have.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > Dong
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Wed, Dec 6, 2017 at 6:42 PM, Jun Rao <
> > > > jun@confluent.io>
> > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Hi, Dong,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Great finding on the issue. It's a real problem.
> A
> > > few
> > > > > > > comments
> > > > > > > > > > about
> > > > > > > > > > > > the
> > > > > > > > > > > > > > KIP. (1) I am not sure about updating
> > > > > > > controller_metadata_epoch
> > > > > > > > > on
> > > > > > > > > > > > every
> > > > > > > > > > > > > > UpdateMetadataRequest. Currently, the controller
> > can
> > > > send
> > > > > > > > > > > > > > UpdateMetadataRequest when there is no actual
> > > metadata
> > > > > > > change.
> > > > > > > > > > Doing
> > > > > > > > > > > > this
> > > > > > > > > > > > > > may require unnecessary metadata refresh on the
> > > client.
> > > > > (2)
> > > > > > > > > > > > > > controller_metadata_epoch is global across all
> > > topics.
> > > > > This
> > > > > > > > means
> > > > > > > > > > > that
> > > > > > > > > > > > a
> > > > > > > > > > > > > > client may be forced to update its metadata even
> > when
> > > > the
> > > > > > > > > metadata
> > > > > > > > > > > for
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > topics that it cares haven't changed. (3) It
> > doesn't
> > > > seem
> > > > > > > that
> > > > > > > > > the
> > > > > > > > > > > KIP
> > > > > > > > > > > > > > handles the corner case when a consumer is
> > restarted.
> > > > > Say a
> > > > > > > > > > consumer
> > > > > > > > > > > > > reads
> > > > > > > > > > > > > > from the new leader, commits the offset and then
> is
> > > > > > > restarted.
> > > > > > > > On
> > > > > > > > > > > > > restart,
> > > > > > > > > > > > > > the consumer gets an outdated metadata and
> fetches
> > > from
> > > > > the
> > > > > > > old
> > > > > > > > > > > leader.
> > > > > > > > > > > > > > Then, the consumer will get into the offset out
> of
> > > > range
> > > > > > > issue.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Given the above, I am thinking of the following
> > > > approach.
> > > > > > We
> > > > > > > > > > actually
> > > > > > > > > > > > > > already have metadata versioning at the partition
> > > > level.
> > > > > > Each
> > > > > > > > > > leader
> > > > > > > > > > > > has
> > > > > > > > > > > > > a
> > > > > > > > > > > > > > leader epoch which is monotonically increasing.
> We
> > > can
> > > > > > > > > potentially
> > > > > > > > > > > > > > propagate leader epoch back in the metadata
> > response
> > > > and
> > > > > > the
> > > > > > > > > > clients
> > > > > > > > > > > > can
> > > > > > > > > > > > > > cache that. This solves the issue of (1) and (2).
> > To
> > > > > solve
> > > > > > > (3),
> > > > > > > > > > when
> > > > > > > > > > > > > saving
> > > > > > > > > > > > > > an offset, we could save both an offset and the
> > > > > > corresponding
> > > > > > > > > > leader
> > > > > > > > > > > > > epoch.
> > > > > > > > > > > > > > When fetching the data, the consumer provides
> both
> > > the
> > > > > > offset
> > > > > > > > and
> > > > > > > > > > the
> > > > > > > > > > > > > > leader epoch. A leader will only serve the
> request
> > if
> > > > its
> > > > > > > > leader
> > > > > > > > > > > epoch
> > > > > > > > > > > > is
> > > > > > > > > > > > > > equal to or greater than the leader epoch from
> the
> > > > > > consumer.
> > > > > > > To
> > > > > > > > > > > achieve
> > > > > > > > > > > > > > this, we need to change the fetch request
> protocol
> > > and
> > > > > the
> > > > > > > > offset
> > > > > > > > > > > > commit
> > > > > > > > > > > > > > api, which requires some more thoughts.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Wed, Dec 6, 2017 at 10:57 AM, Dong Lin <
> > > > > > > lindong28@gmail.com
> > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Bump up the thread.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > It will be great to have more comments on
> whether
> > > we
> > > > > > should
> > > > > > > > do
> > > > > > > > > it
> > > > > > > > > > > or
> > > > > > > > > > > > > > > whether there is better way to address the
> > > motivation
> > > > > of
> > > > > > > this
> > > > > > > > > > KIP.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Mon, Dec 4, 2017 at 3:09 PM, Dong Lin <
> > > > > > > > lindong28@gmail.com>
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > I don't have an interesting rejected
> > alternative
> > > > > > solution
> > > > > > > > to
> > > > > > > > > > put
> > > > > > > > > > > in
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > KIP. If there is good alternative solution
> from
> > > > > anyone
> > > > > > in
> > > > > > > > > this
> > > > > > > > > > > > > thread,
> > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > am
> > > > > > > > > > > > > > > > happy to discuss this and update the KIP
> > > > accordingly.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > Dong
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Mon, Dec 4, 2017 at 1:12 PM, Ted Yu <
> > > > > > > > yuzhihong@gmail.com>
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >> It is clearer now.
> > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > >> I noticed that Rejected Alternatives section
> > is
> > > > > empty.
> > > > > > > > > > > > > > > >> Have you considered any alternative ?
> > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > >> Cheers
> > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > >> On Mon, Dec 4, 2017 at 1:07 PM, Dong Lin <
> > > > > > > > > lindong28@gmail.com
> > > > > > > > > > >
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > >> > Ted, thanks for catching this. I have
> > updated
> > > > the
> > > > > > > > sentence
> > > > > > > > > > to
> > > > > > > > > > > > make
> > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > >> > readable.
> > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > >> > Thanks,
> > > > > > > > > > > > > > > >> > Dong
> > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > >> > On Sat, Dec 2, 2017 at 3:05 PM, Ted Yu <
> > > > > > > > > yuzhihong@gmail.com
> > > > > > > > > > >
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > >> > > bq. It the controller_epoch of the
> > incoming
> > > > > > > > > > > MetadataResponse,
> > > > > > > > > > > > or
> > > > > > > > > > > > > > if
> > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > >> > > controller_epoch is the same but the
> > > > > > > > > > > controller_metadata_epoch
> > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > >> > > Can you update the above sentence so
> that
> > > the
> > > > > > > > intention
> > > > > > > > > is
> > > > > > > > > > > > > > clearer ?
> > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > >> > > Thanks
> > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > >> > > On Fri, Dec 1, 2017 at 6:33 PM, Dong
> Lin <
> > > > > > > > > > > lindong28@gmail.com
> > > > > > > > > > > > >
> > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > >> > > > Hi all,
> > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > >> > > > I have created KIP-232: Detect
> outdated
> > > > > metadata
> > > > > > > by
> > > > > > > > > > adding
> > > > > > > > > > > > > > > >> > > > ControllerMetadataEpoch field:
> > > > > > > > > > > > > > > >> > > > https://cwiki.apache.org/
> > > > > > > > > confluence/display/KAFKA/KIP-
> > > > > > > > > > > > > > > >> > > > 232%3A+Detect+outdated+
> > > metadata+by+adding+
> > > > > > > > > > > > > > > >> > ControllerMetadataEpoch+field
> > > > > > > > > > > > > > > >> > > > .
> > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > >> > > > The KIP proposes to add fields in
> > > > > > MetadataResponse
> > > > > > > > and
> > > > > > > > > > > > > > > >> > > > UpdateMetadataRequest so that client
> can
> > > > > reject
> > > > > > > > > outdated
> > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > >> and
> > > > > > > > > > > > > > > >> > > avoid
> > > > > > > > > > > > > > > >> > > > unnecessary OffsetOutOfRangeException.
> > > > > Otherwise
> > > > > > > > there
> > > > > > > > > > is
> > > > > > > > > > > > > > > currently
> > > > > > > > > > > > > > > >> > race
> > > > > > > > > > > > > > > >> > > > condition that can cause consumer to
> > reset
> > > > > > offset
> > > > > > > > > which
> > > > > > > > > > > > > > negatively
> > > > > > > > > > > > > > > >> > affect
> > > > > > > > > > > > > > > >> > > > the consumer's availability.
> > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > >> > > > Feedback and suggestions are welcome!
> > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > >> > > > Regards,
> > > > > > > > > > > > > > > >> > > > Dong
> > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Dong Lin <li...@gmail.com>.
Hey Jason,

Thanks for the comments. These make sense. I have updated the KIP to
include a new error INVALID_LEADER_EPOCH. This will be a non-retriable
error which may be thrown from consumer's API. When application receives
this exception, the only choice will be to revert Kafka client library to
an earlier version.

Previously I think it may be better to simply log an error because I am not
sure it is a good idea to force user to downgrade Kafka client library when
the error itself, e.g. smaller leader epoch, may not be that fatal. One the
other hand it could be argued that we don't know what else can go wrong in
the buggy client library and it may be a good reason to force user to
downgrade library.

Thanks,
Dong


On Tue, Dec 19, 2017 at 9:06 AM, Jason Gustafson <ja...@confluent.io> wrote:

> Hey Dong,
>
>
> > I think it is a good idea to let coordinator do the additional sanity
> check
> > to ensure the leader epoch from OffsetCommitRequest never decreases. This
> > can help us detect bug. The next question will be what should we do if
> > OffsetCommitRequest provides a smaller leader epoch. One possible
> solution
> > is to return a non-retriable error to consumer which will then be thrown
> to
> > user application. But I am not sure it is worth doing it given its impact
> > on the user. Maybe it will be safer to simply have an error message in
> the
> > server log and allow offset commit to succeed. What do you think?
>
>
> I think the check would only have value if you return an error when it
> fails. It seems primarily useful to detect buggy consumer logic, so a
> non-retriable error makes sense to me. Clients which don't implement this
> capability can use the sentinel value and keep the current behavior.
>
> It seems that FetchResponse includes leader epoch via the path
> > FetchResponse -> MemoryRecords -> MutableRecordBatch ->
> DefaultRecordBatch
> > -> partitionLeaderEpoch. Could this be an existing case where we expose
> the
> > leader epoch to clients?
>
>
> Right, in this case the client has no direct dependence on the field, but
> it could still be argued that it is exposed (I had actually considered
> stuffing this field into an opaque blob of bytes in the message format
> which the client wasn't allowed to touch, but it didn't happen in the end).
> I'm not opposed to using the leader epoch field here, I was just mentioning
> that it does tie clients a bit tighter to something which could be
> considered a Kafka internal implementation detail. It makes the protocol a
> bit less intuitive as well since it is rather difficult to explain the edge
> case it is protecting. That said, we've hit other scenarios where being
> able to detect stale metadata in the client would be helpful, so I think it
> might be worth the tradeoff.
>
> -Jason
>
> On Mon, Dec 18, 2017 at 6:09 PM, Dong Lin <li...@gmail.com> wrote:
>
> > Hey Jason,
> >
> > Thanks much for reviewing the KIP.
> >
> > I think it is a good idea to let coordinator do the additional sanity
> check
> > to ensure the leader epoch from OffsetCommitRequest never decreases. This
> > can help us detect bug. The next question will be what should we do if
> > OffsetCommitRequest provides a smaller leader epoch. One possible
> solution
> > is to return a non-retriable error to consumer which will then be thrown
> to
> > user application. But I am not sure it is worth doing it given its impact
> > on the user. Maybe it will be safer to simply have an error message in
> the
> > server log and allow offset commit to succeed. What do you think?
> >
> > It seems that FetchResponse includes leader epoch via the path
> > FetchResponse -> MemoryRecords -> MutableRecordBatch ->
> DefaultRecordBatch
> > -> partitionLeaderEpoch. Could this be an existing case where we expose
> the
> > leader epoch to clients?
> >
> > Thanks,
> > Dong
> >
> >
> >
> > On Mon, Dec 18, 2017 at 3:27 PM, Jason Gustafson <ja...@confluent.io>
> > wrote:
> >
> > > Hi Dong,
> > >
> > > Thanks for the KIP. Good job identifying the problem. One minor
> question
> > I
> > > had is whether the coordinator should enforce that the leader epoch
> > > associated with an offset commit can only go forward for each
> partition?
> > > Currently it looks like we just depend on the client for this, but
> since
> > > we're caching the leader epoch anyway, it seems like a cheap safety
> > > condition. To support old clients, you can always allow the commit if
> the
> > > leader epoch is unknown.
> > >
> > > I agree that we shouldn't expose the leader epoch in OffsetAndMetadata
> in
> > > the consumer API for what it's worth. As you have noted, it is more of
> an
> > > implementation detail. By the same argument, it's also a bit
> unfortunate
> > > that we have to expose it in the request API since that is nearly as
> > > binding in terms of how it limits future iterations. I could be wrong,
> > but
> > > this appears to be the first case where clients will depend on the
> > concept
> > > of leader epoch. Might not be a big deal considering how deeply
> embedded
> > > leader epochs already are in the inter-broker RPCs and the message
> format
> > > itself, but just wanted to mention the fact that good encapsulation
> > applies
> > > to the client request API as well.
> > >
> > > Thanks,
> > > Jason
> > >
> > > On Mon, Dec 18, 2017 at 1:58 PM, Dong Lin <li...@gmail.com> wrote:
> > >
> > > > Hey Jun,
> > > >
> > > > Thanks much for your comments. These are very thoughtful ideas.
> Please
> > > see
> > > > my comments below.
> > > >
> > > > On Thu, Dec 14, 2017 at 6:38 PM, Jun Rao <ju...@confluent.io> wrote:
> > > >
> > > > > Hi, Dong,
> > > > >
> > > > > Thanks for the update. A few more comments below.
> > > > >
> > > > > 10. It seems that we need to return the leader epoch in the fetch
> > > > response
> > > > > as well When fetching data, we could be fetching data from a leader
> > > epoch
> > > > > older than what's returned in the metadata response. So, we want to
> > use
> > > > the
> > > > > leader epoch associated with the offset being fetched for
> committing
> > > > > offsets.
> > > > >
> > > >
> > > > It seems that we may have two separate issues here. The first issue
> is
> > > that
> > > > consumer uses metadata that is older than the one it uses before. The
> > > > second issue is that consumer uses metadata which is newer than the
> > > > corresponding leader epoch in the leader broker. We know that the
> > > > OffsetOutOfRangeException described in this KIP can be prevented by
> > > > avoiding the first issue. On the other hand, it seems that the
> > > > OffsetOffsetOutOfRangeException can still happen even if we avoid
> the
> > > > second issue -- if consumer uses an older version of metadata, the
> > leader
> > > > epoch in its metadata may equal the leader epoch in the broker even
> if
> > > the
> > > > leader epoch in the broker is oudated.
> > > >
> > > > Given this understanding, I am not sure why we need to return the
> > leader
> > > > epoch in the fetch response. As long as consumer's metadata is not
> > going
> > > > back in version, I think we are good. Did I miss something here?
> > > >
> > > >
> > > > >
> > > > > 11. Should we now extend OffsetAndMetadata used in the offset
> commit
> > > api
> > > > in
> > > > > KafkaConsumer to include leader epoch? Similarly, should we return
> > > leader
> > > > > epoch in endOffsets(), beginningOffsets() and position()? We
> probably
> > > > need
> > > > > to think about how to make the api backward compatible.
> > > > >
> > > >
> > > > After thinking through this carefully, I think we probably don't want
> > to
> > > > extend OffsetAndMetadata to include leader epoch because leader epoch
> > is
> > > > kind of implementation detail which ideally should be hidden from
> user.
> > > The
> > > > consumer can include leader epoch in the OffsetCommitRequest after
> > taking
> > > > offset from commitSync(final Map<TopicPartition, OffsetAndMetadata>
> > > > offsets). Similarly consumer can store leader epoch from
> > > > OffsetFetchResponse and only provide offset to user via
> > > > consumer.committed(topicPartition). This solution seems to work well
> > and
> > > > we
> > > > don't have to make changes to consumer's public API. Does this sound
> > OK?
> > > >
> > > >
> > > > >
> > > > > 12. It seems that we now need to store leader epoch in the offset
> > > topic.
> > > > > Could you include the new schema for the value of the offset topic
> > and
> > > > add
> > > > > upgrade notes?
> > > >
> > > >
> > > > You are right. I have updated the KIP to specify the new schema for
> the
> > > > value of the offset topic. Can you take another look?
> > > >
> > > > For existing messages in the offset topic, leader_epoch will be
> > missing.
> > > We
> > > > will use leader_epoch = -1 to indicate the missing leader_epoch. Then
> > the
> > > > consumer behavior will be the same as it is now because any
> > leader_epoch
> > > in
> > > > the MetadataResponse will be larger than the leader_epoch = -1 in the
> > > > OffetFetchResponse. Thus we don't need specific procedure for
> upgrades
> > > due
> > > > to this change in the offset topic schema. By "upgrade nodes", do you
> > > mean
> > > > the sentences we need to include in the upgrade.html in the PR later?
> > > >
> > > >
> > > > >
> > > > > Jun
> > > > >
> > > > >
> > > > > On Tue, Dec 12, 2017 at 5:19 PM, Dong Lin <li...@gmail.com>
> > wrote:
> > > > >
> > > > > > Hey Jun,
> > > > > >
> > > > > > I see. Sounds good. Yeah it is probably simpler to leave this to
> > > > another
> > > > > > KIP in the future.
> > > > > >
> > > > > > Thanks for all the comments. Since there is no further comment in
> > the
> > > > > > community, I will open the voting thread.
> > > > > >
> > > > > > Thanks,
> > > > > > Dong
> > > > > >
> > > > > > On Mon, Dec 11, 2017 at 5:37 PM, Jun Rao <ju...@confluent.io>
> wrote:
> > > > > >
> > > > > > > Hi, Dong,
> > > > > > >
> > > > > > > The case that I am thinking is network partitioning. Suppose
> one
> > > > > deploys
> > > > > > a
> > > > > > > stretched cluster across multiple AZs in the same region. If
> the
> > > > > machines
> > > > > > > in one AZ can't communicate to brokers in other AZs due to a
> > > network
> > > > > > issue,
> > > > > > > the brokers in that AZ won't get any new metadata.
> > > > > > >
> > > > > > > We can potentially solve this problem by requiring some kind of
> > > > regular
> > > > > > > heartbeats between the controller and the broker. This may need
> > > some
> > > > > more
> > > > > > > thoughts. So, it's probably fine to leave this to another KIP
> in
> > > the
> > > > > > > future.
> > > > > > >
> > > > > > > Thanks,
> > > > > > >
> > > > > > > Jun
> > > > > > >
> > > > > > > On Mon, Dec 11, 2017 at 2:55 PM, Dong Lin <lindong28@gmail.com
> >
> > > > wrote:
> > > > > > >
> > > > > > > > Hey Jun,
> > > > > > > >
> > > > > > > > Thanks for the comment. I am open to improve this KIP to
> > address
> > > > more
> > > > > > > > problems. I probably need more help in understanding what is
> > the
> > > > > > current
> > > > > > > > problem with consumer using outdated metadata and whether it
> is
> > > > > easier
> > > > > > to
> > > > > > > > address it together with this KIP.
> > > > > > > >
> > > > > > > > I agree that a consumer can potentially talk to old leader
> for
> > a
> > > > long
> > > > > > > time
> > > > > > > > even after this KIP. But after this KIP, the consumer
> probably
> > > > should
> > > > > > not
> > > > > > > > get OffetOutofRangeException and therefore will not cause
> > offset
> > > > > rewind
> > > > > > > > issue. So the only problem is that consumer will not be able
> to
> > > > fetch
> > > > > > > data
> > > > > > > > until it has updated metadata. It seems that this situation
> can
> > > > only
> > > > > > > happen
> > > > > > > > if the broker is too slow in processing LeaderAndIsrRequest
> > since
> > > > > > > otherwise
> > > > > > > > the consumer will be forced to update metadata due to
> > > > > > > > NotLeaderForPartitionException. So the problem we are having
> > > here
> > > > is
> > > > > > > that
> > > > > > > > consumer will not be able to fetch data if some broker is too
> > > slow
> > > > in
> > > > > > > > processing LeaderAndIsrRequest.
> > > > > > > >
> > > > > > > > Because Kafka propagates LeaderAndIsrRequest asynchronously
> to
> > > all
> > > > > > > brokers
> > > > > > > > in the cluster, there will always be a period of time when
> > > consumer
> > > > > can
> > > > > > > not
> > > > > > > > fetch data for the partition during the leadership change.
> Thus
> > > it
> > > > > > seems
> > > > > > > > more like a broker-side performance issue instead of
> > client-side
> > > > > > > > correctness issue. My gut feel is that it is not causing a
> > much a
> > > > > > problem
> > > > > > > > as the problem to be fixed in this KIP. And if we were to
> > address
> > > > it,
> > > > > > we
> > > > > > > > probably need to make change in the broker side, e.g. with
> > > > > prioritized
> > > > > > > > queue for controller-related requests, which may be kind of
> > > > > orthogonal
> > > > > > to
> > > > > > > > this KIP. I am not very sure it will be easier to address it
> > with
> > > > the
> > > > > > > > change in this KIP. Do you have any recommendation?
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Dong
> > > > > > > >
> > > > > > > >
> > > > > > > > On Mon, Dec 11, 2017 at 1:51 PM, Jun Rao <ju...@confluent.io>
> > > wrote:
> > > > > > > >
> > > > > > > > > Hi, Dong,
> > > > > > > > >
> > > > > > > > > Thanks for the reply.
> > > > > > > > >
> > > > > > > > > My suggestion of forcing the metadata refresh from the
> > > controller
> > > > > may
> > > > > > > not
> > > > > > > > > work in general since the cached controller could be
> outdated
> > > > too.
> > > > > > The
> > > > > > > > > general problem is that if a consumer's metadata is
> outdated,
> > > it
> > > > > may
> > > > > > > get
> > > > > > > > > stuck with the old leader for a long time. We can address
> the
> > > > issue
> > > > > > of
> > > > > > > > > detecting outdated metadata in a separate KIP in the future
> > if
> > > > you
> > > > > > > didn't
> > > > > > > > > intend to address it in this KIP.
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > >
> > > > > > > > > Jun
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Sat, Dec 9, 2017 at 10:12 PM, Dong Lin <
> > lindong28@gmail.com
> > > >
> > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hey Jun,
> > > > > > > > > >
> > > > > > > > > > Thanks much for your comments. Given that client needs to
> > > > > > > de-serialize
> > > > > > > > > the
> > > > > > > > > > metadata anyway, the extra overhead of checking the
> > > > per-partition
> > > > > > > > version
> > > > > > > > > > for every partition should not be a big concern. Thus it
> > > makes
> > > > > > sense
> > > > > > > to
> > > > > > > > > use
> > > > > > > > > > leader epoch as the per-partition version instead of
> > > creating a
> > > > > > > global
> > > > > > > > > > metadata version. I will update the KIP to do that.
> > > > > > > > > >
> > > > > > > > > > Regarding the detection of outdated metadata, I think it
> is
> > > > > > possible
> > > > > > > to
> > > > > > > > > > ensure that client gets latest metadata by fetching from
> > > > > > controller.
> > > > > > > > Note
> > > > > > > > > > that this requires extra logic in the controller such
> that
> > > > > > controller
> > > > > > > > > > updates metadata directly in memory without requiring
> > > > > > > > > > UpdateMetadataRequest. But I am not sure the main
> > motivation
> > > of
> > > > > > this
> > > > > > > at
> > > > > > > > > > this moment. But this makes controller more like a
> > bottleneck
> > > > in
> > > > > > the
> > > > > > > > > > cluster which we probably want to avoid.
> > > > > > > > > >
> > > > > > > > > > I think we can probably keep the current way of ensuring
> > > > metadata
> > > > > > > > > > freshness. Currently client will be forced to refresh
> > > metadata
> > > > if
> > > > > > > > broker
> > > > > > > > > > returns error (e.g. NotLeaderForPartition) due to
> outdated
> > > > > metadata
> > > > > > > or
> > > > > > > > if
> > > > > > > > > > the metadata does not contain the partition that the
> client
> > > > > needs.
> > > > > > In
> > > > > > > > the
> > > > > > > > > > future, as you previously suggested, we can include
> > > > per-partition
> > > > > > > > > > leaderEpoch in the FetchRequest/ProduceRequest such that
> > > broker
> > > > > can
> > > > > > > > > return
> > > > > > > > > > error if the epoch is smaller than cached epoch in the
> > > broker.
> > > > > > Given
> > > > > > > > that
> > > > > > > > > > this adds more complexity to Kafka, I think we can
> probably
> > > > think
> > > > > > > about
> > > > > > > > > > that leader when we have a specific use-case or problem
> to
> > > > solve
> > > > > > with
> > > > > > > > > > up-to-date metadata. Does this sound OK?
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > > Dong
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Fri, Dec 8, 2017 at 3:53 PM, Jun Rao <
> jun@confluent.io>
> > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Hi, Dong,
> > > > > > > > > > >
> > > > > > > > > > > Thanks for the reply. A few more points below.
> > > > > > > > > > >
> > > > > > > > > > > For dealing with how to prevent a consumer switching
> > from a
> > > > new
> > > > > > > > leader
> > > > > > > > > to
> > > > > > > > > > > an old leader, you suggestion that refreshes metadata
> on
> > > > > consumer
> > > > > > > > > restart
> > > > > > > > > > > until it sees a metadata version >= the one associated
> > with
> > > > the
> > > > > > > > offset
> > > > > > > > > > > works too, as long as we guarantee that the cached
> > metadata
> > > > > > > versions
> > > > > > > > on
> > > > > > > > > > the
> > > > > > > > > > > brokers only go up.
> > > > > > > > > > >
> > > > > > > > > > > The second discussion point is on whether the metadata
> > > > > versioning
> > > > > > > > > should
> > > > > > > > > > be
> > > > > > > > > > > per partition or global. For the partition level
> > > versioning,
> > > > > you
> > > > > > > were
> > > > > > > > > > > concerned about the performance. Given that metadata
> > > updates
> > > > > are
> > > > > > > > rare,
> > > > > > > > > I
> > > > > > > > > > am
> > > > > > > > > > > not sure if it's a big concern though. Doing a million
> if
> > > > tests
> > > > > > is
> > > > > > > > > > probably
> > > > > > > > > > > going to take less than 1ms. Another thing is that the
> > > > metadata
> > > > > > > > version
> > > > > > > > > > > seems to need to survive controller failover. In your
> > > current
> > > > > > > > > approach, a
> > > > > > > > > > > consumer may not be able to wait on the right version
> of
> > > the
> > > > > > > metadata
> > > > > > > > > > after
> > > > > > > > > > > the consumer restart since the metadata version may
> have
> > > been
> > > > > > > > recycled
> > > > > > > > > on
> > > > > > > > > > > the server side due to a controller failover while the
> > > > consumer
> > > > > > is
> > > > > > > > > down.
> > > > > > > > > > > The partition level leaderEpoch survives controller
> > failure
> > > > and
> > > > > > > won't
> > > > > > > > > > have
> > > > > > > > > > > this issue.
> > > > > > > > > > >
> > > > > > > > > > > Lastly, neither your proposal nor mine addresses the
> > issue
> > > > how
> > > > > to
> > > > > > > > > > guarantee
> > > > > > > > > > > a consumer to detect that is metadata is outdated.
> > > Currently,
> > > > > the
> > > > > > > > > > consumer
> > > > > > > > > > > is not guaranteed to fetch metadata from every broker
> > > within
> > > > > some
> > > > > > > > > bounded
> > > > > > > > > > > period of time. Maybe this is out of the scope of your
> > KIP.
> > > > But
> > > > > > one
> > > > > > > > > idea
> > > > > > > > > > is
> > > > > > > > > > > force the consumer to refresh metadata from the
> > controller
> > > > > > > > > periodically.
> > > > > > > > > > >
> > > > > > > > > > > Jun
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Thu, Dec 7, 2017 at 11:25 AM, Dong Lin <
> > > > lindong28@gmail.com
> > > > > >
> > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Hey Jun,
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks much for the comments. Great point
> particularly
> > > > > > regarding
> > > > > > > > > (3). I
> > > > > > > > > > > > haven't thought about this before.
> > > > > > > > > > > >
> > > > > > > > > > > > It seems that there are two possible ways where the
> > > version
> > > > > > > number
> > > > > > > > > can
> > > > > > > > > > be
> > > > > > > > > > > > used. One solution is for client to check the version
> > > > number
> > > > > at
> > > > > > > the
> > > > > > > > > > time
> > > > > > > > > > > it
> > > > > > > > > > > > receives MetadataResponse. And if the version number
> in
> > > the
> > > > > > > > > > > > MetadataResponse is smaller than the version number
> in
> > > the
> > > > > > > client's
> > > > > > > > > > > cache,
> > > > > > > > > > > > the client will be forced to fetch metadata again.
> > > Another
> > > > > > > > solution,
> > > > > > > > > > as
> > > > > > > > > > > > you have suggested, is for broker to check the
> version
> > > > number
> > > > > > at
> > > > > > > > the
> > > > > > > > > > time
> > > > > > > > > > > > it receives a request from client. The broker will
> > reject
> > > > the
> > > > > > > > request
> > > > > > > > > > if
> > > > > > > > > > > > the version is smaller than the version in broker's
> > > cache.
> > > > > > > > > > > >
> > > > > > > > > > > > I am not very sure that the second solution can
> address
> > > the
> > > > > > > problem
> > > > > > > > > > here.
> > > > > > > > > > > > In the scenario described in the JIRA ticket,
> broker's
> > > > cache
> > > > > > may
> > > > > > > be
> > > > > > > > > > > > outdated because it has not processed the
> > > > LeaderAndIsrRequest
> > > > > > > from
> > > > > > > > > the
> > > > > > > > > > > > controller. Thus it may still process client's
> request
> > > even
> > > > > if
> > > > > > > the
> > > > > > > > > > > version
> > > > > > > > > > > > in client's request is actually outdated. Does this
> > make
> > > > > sense?
> > > > > > > > > > > >
> > > > > > > > > > > > IMO, it seems that we can address problem (3) by
> saving
> > > the
> > > > > > > > metadata
> > > > > > > > > > > > version together with the offset. After consumer
> > starts,
> > > it
> > > > > > will
> > > > > > > > keep
> > > > > > > > > > > > fetching metadata until the metadata version >= the
> > > version
> > > > > > saved
> > > > > > > > > with
> > > > > > > > > > > the
> > > > > > > > > > > > offset of this partition.
> > > > > > > > > > > >
> > > > > > > > > > > > Regarding problems (1) and (2): Currently we use the
> > > > version
> > > > > > > number
> > > > > > > > > in
> > > > > > > > > > > the
> > > > > > > > > > > > MetadataResponse to ensure that the metadata does not
> > go
> > > > back
> > > > > > in
> > > > > > > > > time.
> > > > > > > > > > > > There are two alternative solutions to address
> problems
> > > (1)
> > > > > and
> > > > > > > > (2).
> > > > > > > > > > One
> > > > > > > > > > > > solution is for client to enumerate all partitions in
> > the
> > > > > > > > > > > MetadataResponse,
> > > > > > > > > > > > compare their epoch with those in the cached
> metadata,
> > > and
> > > > > > > rejects
> > > > > > > > > the
> > > > > > > > > > > > MetadataResponse iff any leader epoch is smaller. The
> > > main
> > > > > > > concern
> > > > > > > > is
> > > > > > > > > > > that
> > > > > > > > > > > > MetadataResponse currently cached information of all
> > > > > partitions
> > > > > > > in
> > > > > > > > > the
> > > > > > > > > > > > entire cluster. It may slow down client's performance
> > if
> > > we
> > > > > > were
> > > > > > > to
> > > > > > > > > do
> > > > > > > > > > > it.
> > > > > > > > > > > > The other solution is for client to enumerate
> > partitions
> > > > for
> > > > > > only
> > > > > > > > > > topics
> > > > > > > > > > > > registered in the org.apache.kafka.clients.Metadata,
> > > which
> > > > > > will
> > > > > > > be
> > > > > > > > > an
> > > > > > > > > > > > empty
> > > > > > > > > > > > set for producer and the set of subscribed partitions
> > for
> > > > > > > consumer.
> > > > > > > > > But
> > > > > > > > > > > > this degrades to all topics if consumer subscribes to
> > > > topics
> > > > > in
> > > > > > > the
> > > > > > > > > > > cluster
> > > > > > > > > > > > by pattern.
> > > > > > > > > > > >
> > > > > > > > > > > > Note that client will only be forced to update
> metadata
> > > if
> > > > > the
> > > > > > > > > version
> > > > > > > > > > in
> > > > > > > > > > > > the MetadataResponse is smaller than the version in
> the
> > > > > cached
> > > > > > > > > > metadata.
> > > > > > > > > > > In
> > > > > > > > > > > > general it should not be a problem. It can be a
> problem
> > > > only
> > > > > if
> > > > > > > > some
> > > > > > > > > > > broker
> > > > > > > > > > > > is particularly slower than other brokers in
> processing
> > > > > > > > > > > > UpdateMetadataRequest. When this is the case, it
> means
> > > that
> > > > > the
> > > > > > > > > broker
> > > > > > > > > > is
> > > > > > > > > > > > also particularly slower in processing
> > > LeaderAndIsrRequest,
> > > > > > which
> > > > > > > > can
> > > > > > > > > > > cause
> > > > > > > > > > > > problem anyway because some partition will probably
> > have
> > > no
> > > > > > > leader
> > > > > > > > > > during
> > > > > > > > > > > > this period. I am not sure problems (1) and (2) cause
> > > more
> > > > > > > problem
> > > > > > > > > than
> > > > > > > > > > > > what we already have.
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks,
> > > > > > > > > > > > Dong
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On Wed, Dec 6, 2017 at 6:42 PM, Jun Rao <
> > > jun@confluent.io>
> > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Hi, Dong,
> > > > > > > > > > > > >
> > > > > > > > > > > > > Great finding on the issue. It's a real problem. A
> > few
> > > > > > comments
> > > > > > > > > about
> > > > > > > > > > > the
> > > > > > > > > > > > > KIP. (1) I am not sure about updating
> > > > > > controller_metadata_epoch
> > > > > > > > on
> > > > > > > > > > > every
> > > > > > > > > > > > > UpdateMetadataRequest. Currently, the controller
> can
> > > send
> > > > > > > > > > > > > UpdateMetadataRequest when there is no actual
> > metadata
> > > > > > change.
> > > > > > > > > Doing
> > > > > > > > > > > this
> > > > > > > > > > > > > may require unnecessary metadata refresh on the
> > client.
> > > > (2)
> > > > > > > > > > > > > controller_metadata_epoch is global across all
> > topics.
> > > > This
> > > > > > > means
> > > > > > > > > > that
> > > > > > > > > > > a
> > > > > > > > > > > > > client may be forced to update its metadata even
> when
> > > the
> > > > > > > > metadata
> > > > > > > > > > for
> > > > > > > > > > > > the
> > > > > > > > > > > > > topics that it cares haven't changed. (3) It
> doesn't
> > > seem
> > > > > > that
> > > > > > > > the
> > > > > > > > > > KIP
> > > > > > > > > > > > > handles the corner case when a consumer is
> restarted.
> > > > Say a
> > > > > > > > > consumer
> > > > > > > > > > > > reads
> > > > > > > > > > > > > from the new leader, commits the offset and then is
> > > > > > restarted.
> > > > > > > On
> > > > > > > > > > > > restart,
> > > > > > > > > > > > > the consumer gets an outdated metadata and fetches
> > from
> > > > the
> > > > > > old
> > > > > > > > > > leader.
> > > > > > > > > > > > > Then, the consumer will get into the offset out of
> > > range
> > > > > > issue.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Given the above, I am thinking of the following
> > > approach.
> > > > > We
> > > > > > > > > actually
> > > > > > > > > > > > > already have metadata versioning at the partition
> > > level.
> > > > > Each
> > > > > > > > > leader
> > > > > > > > > > > has
> > > > > > > > > > > > a
> > > > > > > > > > > > > leader epoch which is monotonically increasing. We
> > can
> > > > > > > > potentially
> > > > > > > > > > > > > propagate leader epoch back in the metadata
> response
> > > and
> > > > > the
> > > > > > > > > clients
> > > > > > > > > > > can
> > > > > > > > > > > > > cache that. This solves the issue of (1) and (2).
> To
> > > > solve
> > > > > > (3),
> > > > > > > > > when
> > > > > > > > > > > > saving
> > > > > > > > > > > > > an offset, we could save both an offset and the
> > > > > corresponding
> > > > > > > > > leader
> > > > > > > > > > > > epoch.
> > > > > > > > > > > > > When fetching the data, the consumer provides both
> > the
> > > > > offset
> > > > > > > and
> > > > > > > > > the
> > > > > > > > > > > > > leader epoch. A leader will only serve the request
> if
> > > its
> > > > > > > leader
> > > > > > > > > > epoch
> > > > > > > > > > > is
> > > > > > > > > > > > > equal to or greater than the leader epoch from the
> > > > > consumer.
> > > > > > To
> > > > > > > > > > achieve
> > > > > > > > > > > > > this, we need to change the fetch request protocol
> > and
> > > > the
> > > > > > > offset
> > > > > > > > > > > commit
> > > > > > > > > > > > > api, which requires some more thoughts.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > >
> > > > > > > > > > > > > Jun
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Wed, Dec 6, 2017 at 10:57 AM, Dong Lin <
> > > > > > lindong28@gmail.com
> > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Bump up the thread.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > It will be great to have more comments on whether
> > we
> > > > > should
> > > > > > > do
> > > > > > > > it
> > > > > > > > > > or
> > > > > > > > > > > > > > whether there is better way to address the
> > motivation
> > > > of
> > > > > > this
> > > > > > > > > KIP.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Mon, Dec 4, 2017 at 3:09 PM, Dong Lin <
> > > > > > > lindong28@gmail.com>
> > > > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > I don't have an interesting rejected
> alternative
> > > > > solution
> > > > > > > to
> > > > > > > > > put
> > > > > > > > > > in
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > KIP. If there is good alternative solution from
> > > > anyone
> > > > > in
> > > > > > > > this
> > > > > > > > > > > > thread,
> > > > > > > > > > > > > I
> > > > > > > > > > > > > > am
> > > > > > > > > > > > > > > happy to discuss this and update the KIP
> > > accordingly.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > Dong
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Mon, Dec 4, 2017 at 1:12 PM, Ted Yu <
> > > > > > > yuzhihong@gmail.com>
> > > > > > > > > > > wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >> It is clearer now.
> > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > >> I noticed that Rejected Alternatives section
> is
> > > > empty.
> > > > > > > > > > > > > > >> Have you considered any alternative ?
> > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > >> Cheers
> > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > >> On Mon, Dec 4, 2017 at 1:07 PM, Dong Lin <
> > > > > > > > lindong28@gmail.com
> > > > > > > > > >
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > >> > Ted, thanks for catching this. I have
> updated
> > > the
> > > > > > > sentence
> > > > > > > > > to
> > > > > > > > > > > make
> > > > > > > > > > > > > it
> > > > > > > > > > > > > > >> > readable.
> > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > >> > Thanks,
> > > > > > > > > > > > > > >> > Dong
> > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > >> > On Sat, Dec 2, 2017 at 3:05 PM, Ted Yu <
> > > > > > > > yuzhihong@gmail.com
> > > > > > > > > >
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > >> > > bq. It the controller_epoch of the
> incoming
> > > > > > > > > > MetadataResponse,
> > > > > > > > > > > or
> > > > > > > > > > > > > if
> > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > >> > > controller_epoch is the same but the
> > > > > > > > > > controller_metadata_epoch
> > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > >> > > Can you update the above sentence so that
> > the
> > > > > > > intention
> > > > > > > > is
> > > > > > > > > > > > > clearer ?
> > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > >> > > Thanks
> > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > >> > > On Fri, Dec 1, 2017 at 6:33 PM, Dong Lin <
> > > > > > > > > > lindong28@gmail.com
> > > > > > > > > > > >
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > >> > > > Hi all,
> > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > >> > > > I have created KIP-232: Detect outdated
> > > > metadata
> > > > > > by
> > > > > > > > > adding
> > > > > > > > > > > > > > >> > > > ControllerMetadataEpoch field:
> > > > > > > > > > > > > > >> > > > https://cwiki.apache.org/
> > > > > > > > confluence/display/KAFKA/KIP-
> > > > > > > > > > > > > > >> > > > 232%3A+Detect+outdated+
> > metadata+by+adding+
> > > > > > > > > > > > > > >> > ControllerMetadataEpoch+field
> > > > > > > > > > > > > > >> > > > .
> > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > >> > > > The KIP proposes to add fields in
> > > > > MetadataResponse
> > > > > > > and
> > > > > > > > > > > > > > >> > > > UpdateMetadataRequest so that client can
> > > > reject
> > > > > > > > outdated
> > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > >> and
> > > > > > > > > > > > > > >> > > avoid
> > > > > > > > > > > > > > >> > > > unnecessary OffsetOutOfRangeException.
> > > > Otherwise
> > > > > > > there
> > > > > > > > > is
> > > > > > > > > > > > > > currently
> > > > > > > > > > > > > > >> > race
> > > > > > > > > > > > > > >> > > > condition that can cause consumer to
> reset
> > > > > offset
> > > > > > > > which
> > > > > > > > > > > > > negatively
> > > > > > > > > > > > > > >> > affect
> > > > > > > > > > > > > > >> > > > the consumer's availability.
> > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > >> > > > Feedback and suggestions are welcome!
> > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > >> > > > Regards,
> > > > > > > > > > > > > > >> > > > Dong
> > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Jason Gustafson <ja...@confluent.io>.
Hey Dong,


> I think it is a good idea to let coordinator do the additional sanity check
> to ensure the leader epoch from OffsetCommitRequest never decreases. This
> can help us detect bug. The next question will be what should we do if
> OffsetCommitRequest provides a smaller leader epoch. One possible solution
> is to return a non-retriable error to consumer which will then be thrown to
> user application. But I am not sure it is worth doing it given its impact
> on the user. Maybe it will be safer to simply have an error message in the
> server log and allow offset commit to succeed. What do you think?


I think the check would only have value if you return an error when it
fails. It seems primarily useful to detect buggy consumer logic, so a
non-retriable error makes sense to me. Clients which don't implement this
capability can use the sentinel value and keep the current behavior.

It seems that FetchResponse includes leader epoch via the path
> FetchResponse -> MemoryRecords -> MutableRecordBatch -> DefaultRecordBatch
> -> partitionLeaderEpoch. Could this be an existing case where we expose the
> leader epoch to clients?


Right, in this case the client has no direct dependence on the field, but
it could still be argued that it is exposed (I had actually considered
stuffing this field into an opaque blob of bytes in the message format
which the client wasn't allowed to touch, but it didn't happen in the end).
I'm not opposed to using the leader epoch field here, I was just mentioning
that it does tie clients a bit tighter to something which could be
considered a Kafka internal implementation detail. It makes the protocol a
bit less intuitive as well since it is rather difficult to explain the edge
case it is protecting. That said, we've hit other scenarios where being
able to detect stale metadata in the client would be helpful, so I think it
might be worth the tradeoff.

-Jason

On Mon, Dec 18, 2017 at 6:09 PM, Dong Lin <li...@gmail.com> wrote:

> Hey Jason,
>
> Thanks much for reviewing the KIP.
>
> I think it is a good idea to let coordinator do the additional sanity check
> to ensure the leader epoch from OffsetCommitRequest never decreases. This
> can help us detect bug. The next question will be what should we do if
> OffsetCommitRequest provides a smaller leader epoch. One possible solution
> is to return a non-retriable error to consumer which will then be thrown to
> user application. But I am not sure it is worth doing it given its impact
> on the user. Maybe it will be safer to simply have an error message in the
> server log and allow offset commit to succeed. What do you think?
>
> It seems that FetchResponse includes leader epoch via the path
> FetchResponse -> MemoryRecords -> MutableRecordBatch -> DefaultRecordBatch
> -> partitionLeaderEpoch. Could this be an existing case where we expose the
> leader epoch to clients?
>
> Thanks,
> Dong
>
>
>
> On Mon, Dec 18, 2017 at 3:27 PM, Jason Gustafson <ja...@confluent.io>
> wrote:
>
> > Hi Dong,
> >
> > Thanks for the KIP. Good job identifying the problem. One minor question
> I
> > had is whether the coordinator should enforce that the leader epoch
> > associated with an offset commit can only go forward for each partition?
> > Currently it looks like we just depend on the client for this, but since
> > we're caching the leader epoch anyway, it seems like a cheap safety
> > condition. To support old clients, you can always allow the commit if the
> > leader epoch is unknown.
> >
> > I agree that we shouldn't expose the leader epoch in OffsetAndMetadata in
> > the consumer API for what it's worth. As you have noted, it is more of an
> > implementation detail. By the same argument, it's also a bit unfortunate
> > that we have to expose it in the request API since that is nearly as
> > binding in terms of how it limits future iterations. I could be wrong,
> but
> > this appears to be the first case where clients will depend on the
> concept
> > of leader epoch. Might not be a big deal considering how deeply embedded
> > leader epochs already are in the inter-broker RPCs and the message format
> > itself, but just wanted to mention the fact that good encapsulation
> applies
> > to the client request API as well.
> >
> > Thanks,
> > Jason
> >
> > On Mon, Dec 18, 2017 at 1:58 PM, Dong Lin <li...@gmail.com> wrote:
> >
> > > Hey Jun,
> > >
> > > Thanks much for your comments. These are very thoughtful ideas. Please
> > see
> > > my comments below.
> > >
> > > On Thu, Dec 14, 2017 at 6:38 PM, Jun Rao <ju...@confluent.io> wrote:
> > >
> > > > Hi, Dong,
> > > >
> > > > Thanks for the update. A few more comments below.
> > > >
> > > > 10. It seems that we need to return the leader epoch in the fetch
> > > response
> > > > as well When fetching data, we could be fetching data from a leader
> > epoch
> > > > older than what's returned in the metadata response. So, we want to
> use
> > > the
> > > > leader epoch associated with the offset being fetched for committing
> > > > offsets.
> > > >
> > >
> > > It seems that we may have two separate issues here. The first issue is
> > that
> > > consumer uses metadata that is older than the one it uses before. The
> > > second issue is that consumer uses metadata which is newer than the
> > > corresponding leader epoch in the leader broker. We know that the
> > > OffsetOutOfRangeException described in this KIP can be prevented by
> > > avoiding the first issue. On the other hand, it seems that the
> > > OffsetOffsetOutOfRangeException can still happen even if we avoid the
> > > second issue -- if consumer uses an older version of metadata, the
> leader
> > > epoch in its metadata may equal the leader epoch in the broker even if
> > the
> > > leader epoch in the broker is oudated.
> > >
> > > Given this understanding, I am not sure why we need to return the
> leader
> > > epoch in the fetch response. As long as consumer's metadata is not
> going
> > > back in version, I think we are good. Did I miss something here?
> > >
> > >
> > > >
> > > > 11. Should we now extend OffsetAndMetadata used in the offset commit
> > api
> > > in
> > > > KafkaConsumer to include leader epoch? Similarly, should we return
> > leader
> > > > epoch in endOffsets(), beginningOffsets() and position()? We probably
> > > need
> > > > to think about how to make the api backward compatible.
> > > >
> > >
> > > After thinking through this carefully, I think we probably don't want
> to
> > > extend OffsetAndMetadata to include leader epoch because leader epoch
> is
> > > kind of implementation detail which ideally should be hidden from user.
> > The
> > > consumer can include leader epoch in the OffsetCommitRequest after
> taking
> > > offset from commitSync(final Map<TopicPartition, OffsetAndMetadata>
> > > offsets). Similarly consumer can store leader epoch from
> > > OffsetFetchResponse and only provide offset to user via
> > > consumer.committed(topicPartition). This solution seems to work well
> and
> > > we
> > > don't have to make changes to consumer's public API. Does this sound
> OK?
> > >
> > >
> > > >
> > > > 12. It seems that we now need to store leader epoch in the offset
> > topic.
> > > > Could you include the new schema for the value of the offset topic
> and
> > > add
> > > > upgrade notes?
> > >
> > >
> > > You are right. I have updated the KIP to specify the new schema for the
> > > value of the offset topic. Can you take another look?
> > >
> > > For existing messages in the offset topic, leader_epoch will be
> missing.
> > We
> > > will use leader_epoch = -1 to indicate the missing leader_epoch. Then
> the
> > > consumer behavior will be the same as it is now because any
> leader_epoch
> > in
> > > the MetadataResponse will be larger than the leader_epoch = -1 in the
> > > OffetFetchResponse. Thus we don't need specific procedure for upgrades
> > due
> > > to this change in the offset topic schema. By "upgrade nodes", do you
> > mean
> > > the sentences we need to include in the upgrade.html in the PR later?
> > >
> > >
> > > >
> > > > Jun
> > > >
> > > >
> > > > On Tue, Dec 12, 2017 at 5:19 PM, Dong Lin <li...@gmail.com>
> wrote:
> > > >
> > > > > Hey Jun,
> > > > >
> > > > > I see. Sounds good. Yeah it is probably simpler to leave this to
> > > another
> > > > > KIP in the future.
> > > > >
> > > > > Thanks for all the comments. Since there is no further comment in
> the
> > > > > community, I will open the voting thread.
> > > > >
> > > > > Thanks,
> > > > > Dong
> > > > >
> > > > > On Mon, Dec 11, 2017 at 5:37 PM, Jun Rao <ju...@confluent.io> wrote:
> > > > >
> > > > > > Hi, Dong,
> > > > > >
> > > > > > The case that I am thinking is network partitioning. Suppose one
> > > > deploys
> > > > > a
> > > > > > stretched cluster across multiple AZs in the same region. If the
> > > > machines
> > > > > > in one AZ can't communicate to brokers in other AZs due to a
> > network
> > > > > issue,
> > > > > > the brokers in that AZ won't get any new metadata.
> > > > > >
> > > > > > We can potentially solve this problem by requiring some kind of
> > > regular
> > > > > > heartbeats between the controller and the broker. This may need
> > some
> > > > more
> > > > > > thoughts. So, it's probably fine to leave this to another KIP in
> > the
> > > > > > future.
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > > On Mon, Dec 11, 2017 at 2:55 PM, Dong Lin <li...@gmail.com>
> > > wrote:
> > > > > >
> > > > > > > Hey Jun,
> > > > > > >
> > > > > > > Thanks for the comment. I am open to improve this KIP to
> address
> > > more
> > > > > > > problems. I probably need more help in understanding what is
> the
> > > > > current
> > > > > > > problem with consumer using outdated metadata and whether it is
> > > > easier
> > > > > to
> > > > > > > address it together with this KIP.
> > > > > > >
> > > > > > > I agree that a consumer can potentially talk to old leader for
> a
> > > long
> > > > > > time
> > > > > > > even after this KIP. But after this KIP, the consumer probably
> > > should
> > > > > not
> > > > > > > get OffetOutofRangeException and therefore will not cause
> offset
> > > > rewind
> > > > > > > issue. So the only problem is that consumer will not be able to
> > > fetch
> > > > > > data
> > > > > > > until it has updated metadata. It seems that this situation can
> > > only
> > > > > > happen
> > > > > > > if the broker is too slow in processing LeaderAndIsrRequest
> since
> > > > > > otherwise
> > > > > > > the consumer will be forced to update metadata due to
> > > > > > > NotLeaderForPartitionException. So the problem we are having
> > here
> > > is
> > > > > > that
> > > > > > > consumer will not be able to fetch data if some broker is too
> > slow
> > > in
> > > > > > > processing LeaderAndIsrRequest.
> > > > > > >
> > > > > > > Because Kafka propagates LeaderAndIsrRequest asynchronously to
> > all
> > > > > > brokers
> > > > > > > in the cluster, there will always be a period of time when
> > consumer
> > > > can
> > > > > > not
> > > > > > > fetch data for the partition during the leadership change. Thus
> > it
> > > > > seems
> > > > > > > more like a broker-side performance issue instead of
> client-side
> > > > > > > correctness issue. My gut feel is that it is not causing a
> much a
> > > > > problem
> > > > > > > as the problem to be fixed in this KIP. And if we were to
> address
> > > it,
> > > > > we
> > > > > > > probably need to make change in the broker side, e.g. with
> > > > prioritized
> > > > > > > queue for controller-related requests, which may be kind of
> > > > orthogonal
> > > > > to
> > > > > > > this KIP. I am not very sure it will be easier to address it
> with
> > > the
> > > > > > > change in this KIP. Do you have any recommendation?
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Dong
> > > > > > >
> > > > > > >
> > > > > > > On Mon, Dec 11, 2017 at 1:51 PM, Jun Rao <ju...@confluent.io>
> > wrote:
> > > > > > >
> > > > > > > > Hi, Dong,
> > > > > > > >
> > > > > > > > Thanks for the reply.
> > > > > > > >
> > > > > > > > My suggestion of forcing the metadata refresh from the
> > controller
> > > > may
> > > > > > not
> > > > > > > > work in general since the cached controller could be outdated
> > > too.
> > > > > The
> > > > > > > > general problem is that if a consumer's metadata is outdated,
> > it
> > > > may
> > > > > > get
> > > > > > > > stuck with the old leader for a long time. We can address the
> > > issue
> > > > > of
> > > > > > > > detecting outdated metadata in a separate KIP in the future
> if
> > > you
> > > > > > didn't
> > > > > > > > intend to address it in this KIP.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > >
> > > > > > > > Jun
> > > > > > > >
> > > > > > > >
> > > > > > > > On Sat, Dec 9, 2017 at 10:12 PM, Dong Lin <
> lindong28@gmail.com
> > >
> > > > > wrote:
> > > > > > > >
> > > > > > > > > Hey Jun,
> > > > > > > > >
> > > > > > > > > Thanks much for your comments. Given that client needs to
> > > > > > de-serialize
> > > > > > > > the
> > > > > > > > > metadata anyway, the extra overhead of checking the
> > > per-partition
> > > > > > > version
> > > > > > > > > for every partition should not be a big concern. Thus it
> > makes
> > > > > sense
> > > > > > to
> > > > > > > > use
> > > > > > > > > leader epoch as the per-partition version instead of
> > creating a
> > > > > > global
> > > > > > > > > metadata version. I will update the KIP to do that.
> > > > > > > > >
> > > > > > > > > Regarding the detection of outdated metadata, I think it is
> > > > > possible
> > > > > > to
> > > > > > > > > ensure that client gets latest metadata by fetching from
> > > > > controller.
> > > > > > > Note
> > > > > > > > > that this requires extra logic in the controller such that
> > > > > controller
> > > > > > > > > updates metadata directly in memory without requiring
> > > > > > > > > UpdateMetadataRequest. But I am not sure the main
> motivation
> > of
> > > > > this
> > > > > > at
> > > > > > > > > this moment. But this makes controller more like a
> bottleneck
> > > in
> > > > > the
> > > > > > > > > cluster which we probably want to avoid.
> > > > > > > > >
> > > > > > > > > I think we can probably keep the current way of ensuring
> > > metadata
> > > > > > > > > freshness. Currently client will be forced to refresh
> > metadata
> > > if
> > > > > > > broker
> > > > > > > > > returns error (e.g. NotLeaderForPartition) due to outdated
> > > > metadata
> > > > > > or
> > > > > > > if
> > > > > > > > > the metadata does not contain the partition that the client
> > > > needs.
> > > > > In
> > > > > > > the
> > > > > > > > > future, as you previously suggested, we can include
> > > per-partition
> > > > > > > > > leaderEpoch in the FetchRequest/ProduceRequest such that
> > broker
> > > > can
> > > > > > > > return
> > > > > > > > > error if the epoch is smaller than cached epoch in the
> > broker.
> > > > > Given
> > > > > > > that
> > > > > > > > > this adds more complexity to Kafka, I think we can probably
> > > think
> > > > > > about
> > > > > > > > > that leader when we have a specific use-case or problem to
> > > solve
> > > > > with
> > > > > > > > > up-to-date metadata. Does this sound OK?
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > > Dong
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Fri, Dec 8, 2017 at 3:53 PM, Jun Rao <ju...@confluent.io>
> > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hi, Dong,
> > > > > > > > > >
> > > > > > > > > > Thanks for the reply. A few more points below.
> > > > > > > > > >
> > > > > > > > > > For dealing with how to prevent a consumer switching
> from a
> > > new
> > > > > > > leader
> > > > > > > > to
> > > > > > > > > > an old leader, you suggestion that refreshes metadata on
> > > > consumer
> > > > > > > > restart
> > > > > > > > > > until it sees a metadata version >= the one associated
> with
> > > the
> > > > > > > offset
> > > > > > > > > > works too, as long as we guarantee that the cached
> metadata
> > > > > > versions
> > > > > > > on
> > > > > > > > > the
> > > > > > > > > > brokers only go up.
> > > > > > > > > >
> > > > > > > > > > The second discussion point is on whether the metadata
> > > > versioning
> > > > > > > > should
> > > > > > > > > be
> > > > > > > > > > per partition or global. For the partition level
> > versioning,
> > > > you
> > > > > > were
> > > > > > > > > > concerned about the performance. Given that metadata
> > updates
> > > > are
> > > > > > > rare,
> > > > > > > > I
> > > > > > > > > am
> > > > > > > > > > not sure if it's a big concern though. Doing a million if
> > > tests
> > > > > is
> > > > > > > > > probably
> > > > > > > > > > going to take less than 1ms. Another thing is that the
> > > metadata
> > > > > > > version
> > > > > > > > > > seems to need to survive controller failover. In your
> > current
> > > > > > > > approach, a
> > > > > > > > > > consumer may not be able to wait on the right version of
> > the
> > > > > > metadata
> > > > > > > > > after
> > > > > > > > > > the consumer restart since the metadata version may have
> > been
> > > > > > > recycled
> > > > > > > > on
> > > > > > > > > > the server side due to a controller failover while the
> > > consumer
> > > > > is
> > > > > > > > down.
> > > > > > > > > > The partition level leaderEpoch survives controller
> failure
> > > and
> > > > > > won't
> > > > > > > > > have
> > > > > > > > > > this issue.
> > > > > > > > > >
> > > > > > > > > > Lastly, neither your proposal nor mine addresses the
> issue
> > > how
> > > > to
> > > > > > > > > guarantee
> > > > > > > > > > a consumer to detect that is metadata is outdated.
> > Currently,
> > > > the
> > > > > > > > > consumer
> > > > > > > > > > is not guaranteed to fetch metadata from every broker
> > within
> > > > some
> > > > > > > > bounded
> > > > > > > > > > period of time. Maybe this is out of the scope of your
> KIP.
> > > But
> > > > > one
> > > > > > > > idea
> > > > > > > > > is
> > > > > > > > > > force the consumer to refresh metadata from the
> controller
> > > > > > > > periodically.
> > > > > > > > > >
> > > > > > > > > > Jun
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Thu, Dec 7, 2017 at 11:25 AM, Dong Lin <
> > > lindong28@gmail.com
> > > > >
> > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Hey Jun,
> > > > > > > > > > >
> > > > > > > > > > > Thanks much for the comments. Great point particularly
> > > > > regarding
> > > > > > > > (3). I
> > > > > > > > > > > haven't thought about this before.
> > > > > > > > > > >
> > > > > > > > > > > It seems that there are two possible ways where the
> > version
> > > > > > number
> > > > > > > > can
> > > > > > > > > be
> > > > > > > > > > > used. One solution is for client to check the version
> > > number
> > > > at
> > > > > > the
> > > > > > > > > time
> > > > > > > > > > it
> > > > > > > > > > > receives MetadataResponse. And if the version number in
> > the
> > > > > > > > > > > MetadataResponse is smaller than the version number in
> > the
> > > > > > client's
> > > > > > > > > > cache,
> > > > > > > > > > > the client will be forced to fetch metadata again.
> > Another
> > > > > > > solution,
> > > > > > > > > as
> > > > > > > > > > > you have suggested, is for broker to check the version
> > > number
> > > > > at
> > > > > > > the
> > > > > > > > > time
> > > > > > > > > > > it receives a request from client. The broker will
> reject
> > > the
> > > > > > > request
> > > > > > > > > if
> > > > > > > > > > > the version is smaller than the version in broker's
> > cache.
> > > > > > > > > > >
> > > > > > > > > > > I am not very sure that the second solution can address
> > the
> > > > > > problem
> > > > > > > > > here.
> > > > > > > > > > > In the scenario described in the JIRA ticket, broker's
> > > cache
> > > > > may
> > > > > > be
> > > > > > > > > > > outdated because it has not processed the
> > > LeaderAndIsrRequest
> > > > > > from
> > > > > > > > the
> > > > > > > > > > > controller. Thus it may still process client's request
> > even
> > > > if
> > > > > > the
> > > > > > > > > > version
> > > > > > > > > > > in client's request is actually outdated. Does this
> make
> > > > sense?
> > > > > > > > > > >
> > > > > > > > > > > IMO, it seems that we can address problem (3) by saving
> > the
> > > > > > > metadata
> > > > > > > > > > > version together with the offset. After consumer
> starts,
> > it
> > > > > will
> > > > > > > keep
> > > > > > > > > > > fetching metadata until the metadata version >= the
> > version
> > > > > saved
> > > > > > > > with
> > > > > > > > > > the
> > > > > > > > > > > offset of this partition.
> > > > > > > > > > >
> > > > > > > > > > > Regarding problems (1) and (2): Currently we use the
> > > version
> > > > > > number
> > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > > MetadataResponse to ensure that the metadata does not
> go
> > > back
> > > > > in
> > > > > > > > time.
> > > > > > > > > > > There are two alternative solutions to address problems
> > (1)
> > > > and
> > > > > > > (2).
> > > > > > > > > One
> > > > > > > > > > > solution is for client to enumerate all partitions in
> the
> > > > > > > > > > MetadataResponse,
> > > > > > > > > > > compare their epoch with those in the cached metadata,
> > and
> > > > > > rejects
> > > > > > > > the
> > > > > > > > > > > MetadataResponse iff any leader epoch is smaller. The
> > main
> > > > > > concern
> > > > > > > is
> > > > > > > > > > that
> > > > > > > > > > > MetadataResponse currently cached information of all
> > > > partitions
> > > > > > in
> > > > > > > > the
> > > > > > > > > > > entire cluster. It may slow down client's performance
> if
> > we
> > > > > were
> > > > > > to
> > > > > > > > do
> > > > > > > > > > it.
> > > > > > > > > > > The other solution is for client to enumerate
> partitions
> > > for
> > > > > only
> > > > > > > > > topics
> > > > > > > > > > > registered in the org.apache.kafka.clients.Metadata,
> > which
> > > > > will
> > > > > > be
> > > > > > > > an
> > > > > > > > > > > empty
> > > > > > > > > > > set for producer and the set of subscribed partitions
> for
> > > > > > consumer.
> > > > > > > > But
> > > > > > > > > > > this degrades to all topics if consumer subscribes to
> > > topics
> > > > in
> > > > > > the
> > > > > > > > > > cluster
> > > > > > > > > > > by pattern.
> > > > > > > > > > >
> > > > > > > > > > > Note that client will only be forced to update metadata
> > if
> > > > the
> > > > > > > > version
> > > > > > > > > in
> > > > > > > > > > > the MetadataResponse is smaller than the version in the
> > > > cached
> > > > > > > > > metadata.
> > > > > > > > > > In
> > > > > > > > > > > general it should not be a problem. It can be a problem
> > > only
> > > > if
> > > > > > > some
> > > > > > > > > > broker
> > > > > > > > > > > is particularly slower than other brokers in processing
> > > > > > > > > > > UpdateMetadataRequest. When this is the case, it means
> > that
> > > > the
> > > > > > > > broker
> > > > > > > > > is
> > > > > > > > > > > also particularly slower in processing
> > LeaderAndIsrRequest,
> > > > > which
> > > > > > > can
> > > > > > > > > > cause
> > > > > > > > > > > problem anyway because some partition will probably
> have
> > no
> > > > > > leader
> > > > > > > > > during
> > > > > > > > > > > this period. I am not sure problems (1) and (2) cause
> > more
> > > > > > problem
> > > > > > > > than
> > > > > > > > > > > what we already have.
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > > Dong
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Wed, Dec 6, 2017 at 6:42 PM, Jun Rao <
> > jun@confluent.io>
> > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Hi, Dong,
> > > > > > > > > > > >
> > > > > > > > > > > > Great finding on the issue. It's a real problem. A
> few
> > > > > comments
> > > > > > > > about
> > > > > > > > > > the
> > > > > > > > > > > > KIP. (1) I am not sure about updating
> > > > > controller_metadata_epoch
> > > > > > > on
> > > > > > > > > > every
> > > > > > > > > > > > UpdateMetadataRequest. Currently, the controller can
> > send
> > > > > > > > > > > > UpdateMetadataRequest when there is no actual
> metadata
> > > > > change.
> > > > > > > > Doing
> > > > > > > > > > this
> > > > > > > > > > > > may require unnecessary metadata refresh on the
> client.
> > > (2)
> > > > > > > > > > > > controller_metadata_epoch is global across all
> topics.
> > > This
> > > > > > means
> > > > > > > > > that
> > > > > > > > > > a
> > > > > > > > > > > > client may be forced to update its metadata even when
> > the
> > > > > > > metadata
> > > > > > > > > for
> > > > > > > > > > > the
> > > > > > > > > > > > topics that it cares haven't changed. (3) It doesn't
> > seem
> > > > > that
> > > > > > > the
> > > > > > > > > KIP
> > > > > > > > > > > > handles the corner case when a consumer is restarted.
> > > Say a
> > > > > > > > consumer
> > > > > > > > > > > reads
> > > > > > > > > > > > from the new leader, commits the offset and then is
> > > > > restarted.
> > > > > > On
> > > > > > > > > > > restart,
> > > > > > > > > > > > the consumer gets an outdated metadata and fetches
> from
> > > the
> > > > > old
> > > > > > > > > leader.
> > > > > > > > > > > > Then, the consumer will get into the offset out of
> > range
> > > > > issue.
> > > > > > > > > > > >
> > > > > > > > > > > > Given the above, I am thinking of the following
> > approach.
> > > > We
> > > > > > > > actually
> > > > > > > > > > > > already have metadata versioning at the partition
> > level.
> > > > Each
> > > > > > > > leader
> > > > > > > > > > has
> > > > > > > > > > > a
> > > > > > > > > > > > leader epoch which is monotonically increasing. We
> can
> > > > > > > potentially
> > > > > > > > > > > > propagate leader epoch back in the metadata response
> > and
> > > > the
> > > > > > > > clients
> > > > > > > > > > can
> > > > > > > > > > > > cache that. This solves the issue of (1) and (2). To
> > > solve
> > > > > (3),
> > > > > > > > when
> > > > > > > > > > > saving
> > > > > > > > > > > > an offset, we could save both an offset and the
> > > > corresponding
> > > > > > > > leader
> > > > > > > > > > > epoch.
> > > > > > > > > > > > When fetching the data, the consumer provides both
> the
> > > > offset
> > > > > > and
> > > > > > > > the
> > > > > > > > > > > > leader epoch. A leader will only serve the request if
> > its
> > > > > > leader
> > > > > > > > > epoch
> > > > > > > > > > is
> > > > > > > > > > > > equal to or greater than the leader epoch from the
> > > > consumer.
> > > > > To
> > > > > > > > > achieve
> > > > > > > > > > > > this, we need to change the fetch request protocol
> and
> > > the
> > > > > > offset
> > > > > > > > > > commit
> > > > > > > > > > > > api, which requires some more thoughts.
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks,
> > > > > > > > > > > >
> > > > > > > > > > > > Jun
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On Wed, Dec 6, 2017 at 10:57 AM, Dong Lin <
> > > > > lindong28@gmail.com
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Bump up the thread.
> > > > > > > > > > > > >
> > > > > > > > > > > > > It will be great to have more comments on whether
> we
> > > > should
> > > > > > do
> > > > > > > it
> > > > > > > > > or
> > > > > > > > > > > > > whether there is better way to address the
> motivation
> > > of
> > > > > this
> > > > > > > > KIP.
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Mon, Dec 4, 2017 at 3:09 PM, Dong Lin <
> > > > > > lindong28@gmail.com>
> > > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > I don't have an interesting rejected alternative
> > > > solution
> > > > > > to
> > > > > > > > put
> > > > > > > > > in
> > > > > > > > > > > the
> > > > > > > > > > > > > > KIP. If there is good alternative solution from
> > > anyone
> > > > in
> > > > > > > this
> > > > > > > > > > > thread,
> > > > > > > > > > > > I
> > > > > > > > > > > > > am
> > > > > > > > > > > > > > happy to discuss this and update the KIP
> > accordingly.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > Dong
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Mon, Dec 4, 2017 at 1:12 PM, Ted Yu <
> > > > > > yuzhihong@gmail.com>
> > > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >> It is clearer now.
> > > > > > > > > > > > > >>
> > > > > > > > > > > > > >> I noticed that Rejected Alternatives section is
> > > empty.
> > > > > > > > > > > > > >> Have you considered any alternative ?
> > > > > > > > > > > > > >>
> > > > > > > > > > > > > >> Cheers
> > > > > > > > > > > > > >>
> > > > > > > > > > > > > >> On Mon, Dec 4, 2017 at 1:07 PM, Dong Lin <
> > > > > > > lindong28@gmail.com
> > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > > > > > >>
> > > > > > > > > > > > > >> > Ted, thanks for catching this. I have updated
> > the
> > > > > > sentence
> > > > > > > > to
> > > > > > > > > > make
> > > > > > > > > > > > it
> > > > > > > > > > > > > >> > readable.
> > > > > > > > > > > > > >> >
> > > > > > > > > > > > > >> > Thanks,
> > > > > > > > > > > > > >> > Dong
> > > > > > > > > > > > > >> >
> > > > > > > > > > > > > >> > On Sat, Dec 2, 2017 at 3:05 PM, Ted Yu <
> > > > > > > yuzhihong@gmail.com
> > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > > > > > >> >
> > > > > > > > > > > > > >> > > bq. It the controller_epoch of the incoming
> > > > > > > > > MetadataResponse,
> > > > > > > > > > or
> > > > > > > > > > > > if
> > > > > > > > > > > > > >> the
> > > > > > > > > > > > > >> > > controller_epoch is the same but the
> > > > > > > > > controller_metadata_epoch
> > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > >> > > Can you update the above sentence so that
> the
> > > > > > intention
> > > > > > > is
> > > > > > > > > > > > clearer ?
> > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > >> > > Thanks
> > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > >> > > On Fri, Dec 1, 2017 at 6:33 PM, Dong Lin <
> > > > > > > > > lindong28@gmail.com
> > > > > > > > > > >
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > >> > > > Hi all,
> > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > >> > > > I have created KIP-232: Detect outdated
> > > metadata
> > > > > by
> > > > > > > > adding
> > > > > > > > > > > > > >> > > > ControllerMetadataEpoch field:
> > > > > > > > > > > > > >> > > > https://cwiki.apache.org/
> > > > > > > confluence/display/KAFKA/KIP-
> > > > > > > > > > > > > >> > > > 232%3A+Detect+outdated+
> metadata+by+adding+
> > > > > > > > > > > > > >> > ControllerMetadataEpoch+field
> > > > > > > > > > > > > >> > > > .
> > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > >> > > > The KIP proposes to add fields in
> > > > MetadataResponse
> > > > > > and
> > > > > > > > > > > > > >> > > > UpdateMetadataRequest so that client can
> > > reject
> > > > > > > outdated
> > > > > > > > > > > > metadata
> > > > > > > > > > > > > >> and
> > > > > > > > > > > > > >> > > avoid
> > > > > > > > > > > > > >> > > > unnecessary OffsetOutOfRangeException.
> > > Otherwise
> > > > > > there
> > > > > > > > is
> > > > > > > > > > > > > currently
> > > > > > > > > > > > > >> > race
> > > > > > > > > > > > > >> > > > condition that can cause consumer to reset
> > > > offset
> > > > > > > which
> > > > > > > > > > > > negatively
> > > > > > > > > > > > > >> > affect
> > > > > > > > > > > > > >> > > > the consumer's availability.
> > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > >> > > > Feedback and suggestions are welcome!
> > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > >> > > > Regards,
> > > > > > > > > > > > > >> > > > Dong
> > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > >> >
> > > > > > > > > > > > > >>
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Dong Lin <li...@gmail.com>.
Hey Jason,

Thanks much for reviewing the KIP.

I think it is a good idea to let coordinator do the additional sanity check
to ensure the leader epoch from OffsetCommitRequest never decreases. This
can help us detect bug. The next question will be what should we do if
OffsetCommitRequest provides a smaller leader epoch. One possible solution
is to return a non-retriable error to consumer which will then be thrown to
user application. But I am not sure it is worth doing it given its impact
on the user. Maybe it will be safer to simply have an error message in the
server log and allow offset commit to succeed. What do you think?

It seems that FetchResponse includes leader epoch via the path
FetchResponse -> MemoryRecords -> MutableRecordBatch -> DefaultRecordBatch
-> partitionLeaderEpoch. Could this be an existing case where we expose the
leader epoch to clients?

Thanks,
Dong



On Mon, Dec 18, 2017 at 3:27 PM, Jason Gustafson <ja...@confluent.io> wrote:

> Hi Dong,
>
> Thanks for the KIP. Good job identifying the problem. One minor question I
> had is whether the coordinator should enforce that the leader epoch
> associated with an offset commit can only go forward for each partition?
> Currently it looks like we just depend on the client for this, but since
> we're caching the leader epoch anyway, it seems like a cheap safety
> condition. To support old clients, you can always allow the commit if the
> leader epoch is unknown.
>
> I agree that we shouldn't expose the leader epoch in OffsetAndMetadata in
> the consumer API for what it's worth. As you have noted, it is more of an
> implementation detail. By the same argument, it's also a bit unfortunate
> that we have to expose it in the request API since that is nearly as
> binding in terms of how it limits future iterations. I could be wrong, but
> this appears to be the first case where clients will depend on the concept
> of leader epoch. Might not be a big deal considering how deeply embedded
> leader epochs already are in the inter-broker RPCs and the message format
> itself, but just wanted to mention the fact that good encapsulation applies
> to the client request API as well.
>
> Thanks,
> Jason
>
> On Mon, Dec 18, 2017 at 1:58 PM, Dong Lin <li...@gmail.com> wrote:
>
> > Hey Jun,
> >
> > Thanks much for your comments. These are very thoughtful ideas. Please
> see
> > my comments below.
> >
> > On Thu, Dec 14, 2017 at 6:38 PM, Jun Rao <ju...@confluent.io> wrote:
> >
> > > Hi, Dong,
> > >
> > > Thanks for the update. A few more comments below.
> > >
> > > 10. It seems that we need to return the leader epoch in the fetch
> > response
> > > as well When fetching data, we could be fetching data from a leader
> epoch
> > > older than what's returned in the metadata response. So, we want to use
> > the
> > > leader epoch associated with the offset being fetched for committing
> > > offsets.
> > >
> >
> > It seems that we may have two separate issues here. The first issue is
> that
> > consumer uses metadata that is older than the one it uses before. The
> > second issue is that consumer uses metadata which is newer than the
> > corresponding leader epoch in the leader broker. We know that the
> > OffsetOutOfRangeException described in this KIP can be prevented by
> > avoiding the first issue. On the other hand, it seems that the
> > OffsetOffsetOutOfRangeException can still happen even if we avoid the
> > second issue -- if consumer uses an older version of metadata, the leader
> > epoch in its metadata may equal the leader epoch in the broker even if
> the
> > leader epoch in the broker is oudated.
> >
> > Given this understanding, I am not sure why we need to return the leader
> > epoch in the fetch response. As long as consumer's metadata is not going
> > back in version, I think we are good. Did I miss something here?
> >
> >
> > >
> > > 11. Should we now extend OffsetAndMetadata used in the offset commit
> api
> > in
> > > KafkaConsumer to include leader epoch? Similarly, should we return
> leader
> > > epoch in endOffsets(), beginningOffsets() and position()? We probably
> > need
> > > to think about how to make the api backward compatible.
> > >
> >
> > After thinking through this carefully, I think we probably don't want to
> > extend OffsetAndMetadata to include leader epoch because leader epoch is
> > kind of implementation detail which ideally should be hidden from user.
> The
> > consumer can include leader epoch in the OffsetCommitRequest after taking
> > offset from commitSync(final Map<TopicPartition, OffsetAndMetadata>
> > offsets). Similarly consumer can store leader epoch from
> > OffsetFetchResponse and only provide offset to user via
> > consumer.committed(topicPartition). This solution seems to work well and
> > we
> > don't have to make changes to consumer's public API. Does this sound OK?
> >
> >
> > >
> > > 12. It seems that we now need to store leader epoch in the offset
> topic.
> > > Could you include the new schema for the value of the offset topic and
> > add
> > > upgrade notes?
> >
> >
> > You are right. I have updated the KIP to specify the new schema for the
> > value of the offset topic. Can you take another look?
> >
> > For existing messages in the offset topic, leader_epoch will be missing.
> We
> > will use leader_epoch = -1 to indicate the missing leader_epoch. Then the
> > consumer behavior will be the same as it is now because any leader_epoch
> in
> > the MetadataResponse will be larger than the leader_epoch = -1 in the
> > OffetFetchResponse. Thus we don't need specific procedure for upgrades
> due
> > to this change in the offset topic schema. By "upgrade nodes", do you
> mean
> > the sentences we need to include in the upgrade.html in the PR later?
> >
> >
> > >
> > > Jun
> > >
> > >
> > > On Tue, Dec 12, 2017 at 5:19 PM, Dong Lin <li...@gmail.com> wrote:
> > >
> > > > Hey Jun,
> > > >
> > > > I see. Sounds good. Yeah it is probably simpler to leave this to
> > another
> > > > KIP in the future.
> > > >
> > > > Thanks for all the comments. Since there is no further comment in the
> > > > community, I will open the voting thread.
> > > >
> > > > Thanks,
> > > > Dong
> > > >
> > > > On Mon, Dec 11, 2017 at 5:37 PM, Jun Rao <ju...@confluent.io> wrote:
> > > >
> > > > > Hi, Dong,
> > > > >
> > > > > The case that I am thinking is network partitioning. Suppose one
> > > deploys
> > > > a
> > > > > stretched cluster across multiple AZs in the same region. If the
> > > machines
> > > > > in one AZ can't communicate to brokers in other AZs due to a
> network
> > > > issue,
> > > > > the brokers in that AZ won't get any new metadata.
> > > > >
> > > > > We can potentially solve this problem by requiring some kind of
> > regular
> > > > > heartbeats between the controller and the broker. This may need
> some
> > > more
> > > > > thoughts. So, it's probably fine to leave this to another KIP in
> the
> > > > > future.
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Jun
> > > > >
> > > > > On Mon, Dec 11, 2017 at 2:55 PM, Dong Lin <li...@gmail.com>
> > wrote:
> > > > >
> > > > > > Hey Jun,
> > > > > >
> > > > > > Thanks for the comment. I am open to improve this KIP to address
> > more
> > > > > > problems. I probably need more help in understanding what is the
> > > > current
> > > > > > problem with consumer using outdated metadata and whether it is
> > > easier
> > > > to
> > > > > > address it together with this KIP.
> > > > > >
> > > > > > I agree that a consumer can potentially talk to old leader for a
> > long
> > > > > time
> > > > > > even after this KIP. But after this KIP, the consumer probably
> > should
> > > > not
> > > > > > get OffetOutofRangeException and therefore will not cause offset
> > > rewind
> > > > > > issue. So the only problem is that consumer will not be able to
> > fetch
> > > > > data
> > > > > > until it has updated metadata. It seems that this situation can
> > only
> > > > > happen
> > > > > > if the broker is too slow in processing LeaderAndIsrRequest since
> > > > > otherwise
> > > > > > the consumer will be forced to update metadata due to
> > > > > > NotLeaderForPartitionException. So the problem we are having
> here
> > is
> > > > > that
> > > > > > consumer will not be able to fetch data if some broker is too
> slow
> > in
> > > > > > processing LeaderAndIsrRequest.
> > > > > >
> > > > > > Because Kafka propagates LeaderAndIsrRequest asynchronously to
> all
> > > > > brokers
> > > > > > in the cluster, there will always be a period of time when
> consumer
> > > can
> > > > > not
> > > > > > fetch data for the partition during the leadership change. Thus
> it
> > > > seems
> > > > > > more like a broker-side performance issue instead of client-side
> > > > > > correctness issue. My gut feel is that it is not causing a much a
> > > > problem
> > > > > > as the problem to be fixed in this KIP. And if we were to address
> > it,
> > > > we
> > > > > > probably need to make change in the broker side, e.g. with
> > > prioritized
> > > > > > queue for controller-related requests, which may be kind of
> > > orthogonal
> > > > to
> > > > > > this KIP. I am not very sure it will be easier to address it with
> > the
> > > > > > change in this KIP. Do you have any recommendation?
> > > > > >
> > > > > > Thanks,
> > > > > > Dong
> > > > > >
> > > > > >
> > > > > > On Mon, Dec 11, 2017 at 1:51 PM, Jun Rao <ju...@confluent.io>
> wrote:
> > > > > >
> > > > > > > Hi, Dong,
> > > > > > >
> > > > > > > Thanks for the reply.
> > > > > > >
> > > > > > > My suggestion of forcing the metadata refresh from the
> controller
> > > may
> > > > > not
> > > > > > > work in general since the cached controller could be outdated
> > too.
> > > > The
> > > > > > > general problem is that if a consumer's metadata is outdated,
> it
> > > may
> > > > > get
> > > > > > > stuck with the old leader for a long time. We can address the
> > issue
> > > > of
> > > > > > > detecting outdated metadata in a separate KIP in the future if
> > you
> > > > > didn't
> > > > > > > intend to address it in this KIP.
> > > > > > >
> > > > > > > Thanks,
> > > > > > >
> > > > > > > Jun
> > > > > > >
> > > > > > >
> > > > > > > On Sat, Dec 9, 2017 at 10:12 PM, Dong Lin <lindong28@gmail.com
> >
> > > > wrote:
> > > > > > >
> > > > > > > > Hey Jun,
> > > > > > > >
> > > > > > > > Thanks much for your comments. Given that client needs to
> > > > > de-serialize
> > > > > > > the
> > > > > > > > metadata anyway, the extra overhead of checking the
> > per-partition
> > > > > > version
> > > > > > > > for every partition should not be a big concern. Thus it
> makes
> > > > sense
> > > > > to
> > > > > > > use
> > > > > > > > leader epoch as the per-partition version instead of
> creating a
> > > > > global
> > > > > > > > metadata version. I will update the KIP to do that.
> > > > > > > >
> > > > > > > > Regarding the detection of outdated metadata, I think it is
> > > > possible
> > > > > to
> > > > > > > > ensure that client gets latest metadata by fetching from
> > > > controller.
> > > > > > Note
> > > > > > > > that this requires extra logic in the controller such that
> > > > controller
> > > > > > > > updates metadata directly in memory without requiring
> > > > > > > > UpdateMetadataRequest. But I am not sure the main motivation
> of
> > > > this
> > > > > at
> > > > > > > > this moment. But this makes controller more like a bottleneck
> > in
> > > > the
> > > > > > > > cluster which we probably want to avoid.
> > > > > > > >
> > > > > > > > I think we can probably keep the current way of ensuring
> > metadata
> > > > > > > > freshness. Currently client will be forced to refresh
> metadata
> > if
> > > > > > broker
> > > > > > > > returns error (e.g. NotLeaderForPartition) due to outdated
> > > metadata
> > > > > or
> > > > > > if
> > > > > > > > the metadata does not contain the partition that the client
> > > needs.
> > > > In
> > > > > > the
> > > > > > > > future, as you previously suggested, we can include
> > per-partition
> > > > > > > > leaderEpoch in the FetchRequest/ProduceRequest such that
> broker
> > > can
> > > > > > > return
> > > > > > > > error if the epoch is smaller than cached epoch in the
> broker.
> > > > Given
> > > > > > that
> > > > > > > > this adds more complexity to Kafka, I think we can probably
> > think
> > > > > about
> > > > > > > > that leader when we have a specific use-case or problem to
> > solve
> > > > with
> > > > > > > > up-to-date metadata. Does this sound OK?
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Dong
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > On Fri, Dec 8, 2017 at 3:53 PM, Jun Rao <ju...@confluent.io>
> > > wrote:
> > > > > > > >
> > > > > > > > > Hi, Dong,
> > > > > > > > >
> > > > > > > > > Thanks for the reply. A few more points below.
> > > > > > > > >
> > > > > > > > > For dealing with how to prevent a consumer switching from a
> > new
> > > > > > leader
> > > > > > > to
> > > > > > > > > an old leader, you suggestion that refreshes metadata on
> > > consumer
> > > > > > > restart
> > > > > > > > > until it sees a metadata version >= the one associated with
> > the
> > > > > > offset
> > > > > > > > > works too, as long as we guarantee that the cached metadata
> > > > > versions
> > > > > > on
> > > > > > > > the
> > > > > > > > > brokers only go up.
> > > > > > > > >
> > > > > > > > > The second discussion point is on whether the metadata
> > > versioning
> > > > > > > should
> > > > > > > > be
> > > > > > > > > per partition or global. For the partition level
> versioning,
> > > you
> > > > > were
> > > > > > > > > concerned about the performance. Given that metadata
> updates
> > > are
> > > > > > rare,
> > > > > > > I
> > > > > > > > am
> > > > > > > > > not sure if it's a big concern though. Doing a million if
> > tests
> > > > is
> > > > > > > > probably
> > > > > > > > > going to take less than 1ms. Another thing is that the
> > metadata
> > > > > > version
> > > > > > > > > seems to need to survive controller failover. In your
> current
> > > > > > > approach, a
> > > > > > > > > consumer may not be able to wait on the right version of
> the
> > > > > metadata
> > > > > > > > after
> > > > > > > > > the consumer restart since the metadata version may have
> been
> > > > > > recycled
> > > > > > > on
> > > > > > > > > the server side due to a controller failover while the
> > consumer
> > > > is
> > > > > > > down.
> > > > > > > > > The partition level leaderEpoch survives controller failure
> > and
> > > > > won't
> > > > > > > > have
> > > > > > > > > this issue.
> > > > > > > > >
> > > > > > > > > Lastly, neither your proposal nor mine addresses the issue
> > how
> > > to
> > > > > > > > guarantee
> > > > > > > > > a consumer to detect that is metadata is outdated.
> Currently,
> > > the
> > > > > > > > consumer
> > > > > > > > > is not guaranteed to fetch metadata from every broker
> within
> > > some
> > > > > > > bounded
> > > > > > > > > period of time. Maybe this is out of the scope of your KIP.
> > But
> > > > one
> > > > > > > idea
> > > > > > > > is
> > > > > > > > > force the consumer to refresh metadata from the controller
> > > > > > > periodically.
> > > > > > > > >
> > > > > > > > > Jun
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Thu, Dec 7, 2017 at 11:25 AM, Dong Lin <
> > lindong28@gmail.com
> > > >
> > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hey Jun,
> > > > > > > > > >
> > > > > > > > > > Thanks much for the comments. Great point particularly
> > > > regarding
> > > > > > > (3). I
> > > > > > > > > > haven't thought about this before.
> > > > > > > > > >
> > > > > > > > > > It seems that there are two possible ways where the
> version
> > > > > number
> > > > > > > can
> > > > > > > > be
> > > > > > > > > > used. One solution is for client to check the version
> > number
> > > at
> > > > > the
> > > > > > > > time
> > > > > > > > > it
> > > > > > > > > > receives MetadataResponse. And if the version number in
> the
> > > > > > > > > > MetadataResponse is smaller than the version number in
> the
> > > > > client's
> > > > > > > > > cache,
> > > > > > > > > > the client will be forced to fetch metadata again.
> Another
> > > > > > solution,
> > > > > > > > as
> > > > > > > > > > you have suggested, is for broker to check the version
> > number
> > > > at
> > > > > > the
> > > > > > > > time
> > > > > > > > > > it receives a request from client. The broker will reject
> > the
> > > > > > request
> > > > > > > > if
> > > > > > > > > > the version is smaller than the version in broker's
> cache.
> > > > > > > > > >
> > > > > > > > > > I am not very sure that the second solution can address
> the
> > > > > problem
> > > > > > > > here.
> > > > > > > > > > In the scenario described in the JIRA ticket, broker's
> > cache
> > > > may
> > > > > be
> > > > > > > > > > outdated because it has not processed the
> > LeaderAndIsrRequest
> > > > > from
> > > > > > > the
> > > > > > > > > > controller. Thus it may still process client's request
> even
> > > if
> > > > > the
> > > > > > > > > version
> > > > > > > > > > in client's request is actually outdated. Does this make
> > > sense?
> > > > > > > > > >
> > > > > > > > > > IMO, it seems that we can address problem (3) by saving
> the
> > > > > > metadata
> > > > > > > > > > version together with the offset. After consumer starts,
> it
> > > > will
> > > > > > keep
> > > > > > > > > > fetching metadata until the metadata version >= the
> version
> > > > saved
> > > > > > > with
> > > > > > > > > the
> > > > > > > > > > offset of this partition.
> > > > > > > > > >
> > > > > > > > > > Regarding problems (1) and (2): Currently we use the
> > version
> > > > > number
> > > > > > > in
> > > > > > > > > the
> > > > > > > > > > MetadataResponse to ensure that the metadata does not go
> > back
> > > > in
> > > > > > > time.
> > > > > > > > > > There are two alternative solutions to address problems
> (1)
> > > and
> > > > > > (2).
> > > > > > > > One
> > > > > > > > > > solution is for client to enumerate all partitions in the
> > > > > > > > > MetadataResponse,
> > > > > > > > > > compare their epoch with those in the cached metadata,
> and
> > > > > rejects
> > > > > > > the
> > > > > > > > > > MetadataResponse iff any leader epoch is smaller. The
> main
> > > > > concern
> > > > > > is
> > > > > > > > > that
> > > > > > > > > > MetadataResponse currently cached information of all
> > > partitions
> > > > > in
> > > > > > > the
> > > > > > > > > > entire cluster. It may slow down client's performance if
> we
> > > > were
> > > > > to
> > > > > > > do
> > > > > > > > > it.
> > > > > > > > > > The other solution is for client to enumerate partitions
> > for
> > > > only
> > > > > > > > topics
> > > > > > > > > > registered in the org.apache.kafka.clients.Metadata,
> which
> > > > will
> > > > > be
> > > > > > > an
> > > > > > > > > > empty
> > > > > > > > > > set for producer and the set of subscribed partitions for
> > > > > consumer.
> > > > > > > But
> > > > > > > > > > this degrades to all topics if consumer subscribes to
> > topics
> > > in
> > > > > the
> > > > > > > > > cluster
> > > > > > > > > > by pattern.
> > > > > > > > > >
> > > > > > > > > > Note that client will only be forced to update metadata
> if
> > > the
> > > > > > > version
> > > > > > > > in
> > > > > > > > > > the MetadataResponse is smaller than the version in the
> > > cached
> > > > > > > > metadata.
> > > > > > > > > In
> > > > > > > > > > general it should not be a problem. It can be a problem
> > only
> > > if
> > > > > > some
> > > > > > > > > broker
> > > > > > > > > > is particularly slower than other brokers in processing
> > > > > > > > > > UpdateMetadataRequest. When this is the case, it means
> that
> > > the
> > > > > > > broker
> > > > > > > > is
> > > > > > > > > > also particularly slower in processing
> LeaderAndIsrRequest,
> > > > which
> > > > > > can
> > > > > > > > > cause
> > > > > > > > > > problem anyway because some partition will probably have
> no
> > > > > leader
> > > > > > > > during
> > > > > > > > > > this period. I am not sure problems (1) and (2) cause
> more
> > > > > problem
> > > > > > > than
> > > > > > > > > > what we already have.
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > > Dong
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Wed, Dec 6, 2017 at 6:42 PM, Jun Rao <
> jun@confluent.io>
> > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Hi, Dong,
> > > > > > > > > > >
> > > > > > > > > > > Great finding on the issue. It's a real problem. A few
> > > > comments
> > > > > > > about
> > > > > > > > > the
> > > > > > > > > > > KIP. (1) I am not sure about updating
> > > > controller_metadata_epoch
> > > > > > on
> > > > > > > > > every
> > > > > > > > > > > UpdateMetadataRequest. Currently, the controller can
> send
> > > > > > > > > > > UpdateMetadataRequest when there is no actual metadata
> > > > change.
> > > > > > > Doing
> > > > > > > > > this
> > > > > > > > > > > may require unnecessary metadata refresh on the client.
> > (2)
> > > > > > > > > > > controller_metadata_epoch is global across all topics.
> > This
> > > > > means
> > > > > > > > that
> > > > > > > > > a
> > > > > > > > > > > client may be forced to update its metadata even when
> the
> > > > > > metadata
> > > > > > > > for
> > > > > > > > > > the
> > > > > > > > > > > topics that it cares haven't changed. (3) It doesn't
> seem
> > > > that
> > > > > > the
> > > > > > > > KIP
> > > > > > > > > > > handles the corner case when a consumer is restarted.
> > Say a
> > > > > > > consumer
> > > > > > > > > > reads
> > > > > > > > > > > from the new leader, commits the offset and then is
> > > > restarted.
> > > > > On
> > > > > > > > > > restart,
> > > > > > > > > > > the consumer gets an outdated metadata and fetches from
> > the
> > > > old
> > > > > > > > leader.
> > > > > > > > > > > Then, the consumer will get into the offset out of
> range
> > > > issue.
> > > > > > > > > > >
> > > > > > > > > > > Given the above, I am thinking of the following
> approach.
> > > We
> > > > > > > actually
> > > > > > > > > > > already have metadata versioning at the partition
> level.
> > > Each
> > > > > > > leader
> > > > > > > > > has
> > > > > > > > > > a
> > > > > > > > > > > leader epoch which is monotonically increasing. We can
> > > > > > potentially
> > > > > > > > > > > propagate leader epoch back in the metadata response
> and
> > > the
> > > > > > > clients
> > > > > > > > > can
> > > > > > > > > > > cache that. This solves the issue of (1) and (2). To
> > solve
> > > > (3),
> > > > > > > when
> > > > > > > > > > saving
> > > > > > > > > > > an offset, we could save both an offset and the
> > > corresponding
> > > > > > > leader
> > > > > > > > > > epoch.
> > > > > > > > > > > When fetching the data, the consumer provides both the
> > > offset
> > > > > and
> > > > > > > the
> > > > > > > > > > > leader epoch. A leader will only serve the request if
> its
> > > > > leader
> > > > > > > > epoch
> > > > > > > > > is
> > > > > > > > > > > equal to or greater than the leader epoch from the
> > > consumer.
> > > > To
> > > > > > > > achieve
> > > > > > > > > > > this, we need to change the fetch request protocol and
> > the
> > > > > offset
> > > > > > > > > commit
> > > > > > > > > > > api, which requires some more thoughts.
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > >
> > > > > > > > > > > Jun
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Wed, Dec 6, 2017 at 10:57 AM, Dong Lin <
> > > > lindong28@gmail.com
> > > > > >
> > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Bump up the thread.
> > > > > > > > > > > >
> > > > > > > > > > > > It will be great to have more comments on whether we
> > > should
> > > > > do
> > > > > > it
> > > > > > > > or
> > > > > > > > > > > > whether there is better way to address the motivation
> > of
> > > > this
> > > > > > > KIP.
> > > > > > > > > > > >
> > > > > > > > > > > > On Mon, Dec 4, 2017 at 3:09 PM, Dong Lin <
> > > > > lindong28@gmail.com>
> > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > I don't have an interesting rejected alternative
> > > solution
> > > > > to
> > > > > > > put
> > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > > > > KIP. If there is good alternative solution from
> > anyone
> > > in
> > > > > > this
> > > > > > > > > > thread,
> > > > > > > > > > > I
> > > > > > > > > > > > am
> > > > > > > > > > > > > happy to discuss this and update the KIP
> accordingly.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > Dong
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Mon, Dec 4, 2017 at 1:12 PM, Ted Yu <
> > > > > yuzhihong@gmail.com>
> > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > >> It is clearer now.
> > > > > > > > > > > > >>
> > > > > > > > > > > > >> I noticed that Rejected Alternatives section is
> > empty.
> > > > > > > > > > > > >> Have you considered any alternative ?
> > > > > > > > > > > > >>
> > > > > > > > > > > > >> Cheers
> > > > > > > > > > > > >>
> > > > > > > > > > > > >> On Mon, Dec 4, 2017 at 1:07 PM, Dong Lin <
> > > > > > lindong28@gmail.com
> > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > > > > >>
> > > > > > > > > > > > >> > Ted, thanks for catching this. I have updated
> the
> > > > > sentence
> > > > > > > to
> > > > > > > > > make
> > > > > > > > > > > it
> > > > > > > > > > > > >> > readable.
> > > > > > > > > > > > >> >
> > > > > > > > > > > > >> > Thanks,
> > > > > > > > > > > > >> > Dong
> > > > > > > > > > > > >> >
> > > > > > > > > > > > >> > On Sat, Dec 2, 2017 at 3:05 PM, Ted Yu <
> > > > > > yuzhihong@gmail.com
> > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > > > > >> >
> > > > > > > > > > > > >> > > bq. It the controller_epoch of the incoming
> > > > > > > > MetadataResponse,
> > > > > > > > > or
> > > > > > > > > > > if
> > > > > > > > > > > > >> the
> > > > > > > > > > > > >> > > controller_epoch is the same but the
> > > > > > > > controller_metadata_epoch
> > > > > > > > > > > > >> > >
> > > > > > > > > > > > >> > > Can you update the above sentence so that the
> > > > > intention
> > > > > > is
> > > > > > > > > > > clearer ?
> > > > > > > > > > > > >> > >
> > > > > > > > > > > > >> > > Thanks
> > > > > > > > > > > > >> > >
> > > > > > > > > > > > >> > > On Fri, Dec 1, 2017 at 6:33 PM, Dong Lin <
> > > > > > > > lindong28@gmail.com
> > > > > > > > > >
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > >> > >
> > > > > > > > > > > > >> > > > Hi all,
> > > > > > > > > > > > >> > > >
> > > > > > > > > > > > >> > > > I have created KIP-232: Detect outdated
> > metadata
> > > > by
> > > > > > > adding
> > > > > > > > > > > > >> > > > ControllerMetadataEpoch field:
> > > > > > > > > > > > >> > > > https://cwiki.apache.org/
> > > > > > confluence/display/KAFKA/KIP-
> > > > > > > > > > > > >> > > > 232%3A+Detect+outdated+metadata+by+adding+
> > > > > > > > > > > > >> > ControllerMetadataEpoch+field
> > > > > > > > > > > > >> > > > .
> > > > > > > > > > > > >> > > >
> > > > > > > > > > > > >> > > > The KIP proposes to add fields in
> > > MetadataResponse
> > > > > and
> > > > > > > > > > > > >> > > > UpdateMetadataRequest so that client can
> > reject
> > > > > > outdated
> > > > > > > > > > > metadata
> > > > > > > > > > > > >> and
> > > > > > > > > > > > >> > > avoid
> > > > > > > > > > > > >> > > > unnecessary OffsetOutOfRangeException.
> > Otherwise
> > > > > there
> > > > > > > is
> > > > > > > > > > > > currently
> > > > > > > > > > > > >> > race
> > > > > > > > > > > > >> > > > condition that can cause consumer to reset
> > > offset
> > > > > > which
> > > > > > > > > > > negatively
> > > > > > > > > > > > >> > affect
> > > > > > > > > > > > >> > > > the consumer's availability.
> > > > > > > > > > > > >> > > >
> > > > > > > > > > > > >> > > > Feedback and suggestions are welcome!
> > > > > > > > > > > > >> > > >
> > > > > > > > > > > > >> > > > Regards,
> > > > > > > > > > > > >> > > > Dong
> > > > > > > > > > > > >> > > >
> > > > > > > > > > > > >> > >
> > > > > > > > > > > > >> >
> > > > > > > > > > > > >>
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Jason Gustafson <ja...@confluent.io>.
Hi Dong,

Thanks for the KIP. Good job identifying the problem. One minor question I
had is whether the coordinator should enforce that the leader epoch
associated with an offset commit can only go forward for each partition?
Currently it looks like we just depend on the client for this, but since
we're caching the leader epoch anyway, it seems like a cheap safety
condition. To support old clients, you can always allow the commit if the
leader epoch is unknown.

I agree that we shouldn't expose the leader epoch in OffsetAndMetadata in
the consumer API for what it's worth. As you have noted, it is more of an
implementation detail. By the same argument, it's also a bit unfortunate
that we have to expose it in the request API since that is nearly as
binding in terms of how it limits future iterations. I could be wrong, but
this appears to be the first case where clients will depend on the concept
of leader epoch. Might not be a big deal considering how deeply embedded
leader epochs already are in the inter-broker RPCs and the message format
itself, but just wanted to mention the fact that good encapsulation applies
to the client request API as well.

Thanks,
Jason

On Mon, Dec 18, 2017 at 1:58 PM, Dong Lin <li...@gmail.com> wrote:

> Hey Jun,
>
> Thanks much for your comments. These are very thoughtful ideas. Please see
> my comments below.
>
> On Thu, Dec 14, 2017 at 6:38 PM, Jun Rao <ju...@confluent.io> wrote:
>
> > Hi, Dong,
> >
> > Thanks for the update. A few more comments below.
> >
> > 10. It seems that we need to return the leader epoch in the fetch
> response
> > as well When fetching data, we could be fetching data from a leader epoch
> > older than what's returned in the metadata response. So, we want to use
> the
> > leader epoch associated with the offset being fetched for committing
> > offsets.
> >
>
> It seems that we may have two separate issues here. The first issue is that
> consumer uses metadata that is older than the one it uses before. The
> second issue is that consumer uses metadata which is newer than the
> corresponding leader epoch in the leader broker. We know that the
> OffsetOutOfRangeException described in this KIP can be prevented by
> avoiding the first issue. On the other hand, it seems that the
> OffsetOffsetOutOfRangeException can still happen even if we avoid the
> second issue -- if consumer uses an older version of metadata, the leader
> epoch in its metadata may equal the leader epoch in the broker even if the
> leader epoch in the broker is oudated.
>
> Given this understanding, I am not sure why we need to return the leader
> epoch in the fetch response. As long as consumer's metadata is not going
> back in version, I think we are good. Did I miss something here?
>
>
> >
> > 11. Should we now extend OffsetAndMetadata used in the offset commit api
> in
> > KafkaConsumer to include leader epoch? Similarly, should we return leader
> > epoch in endOffsets(), beginningOffsets() and position()? We probably
> need
> > to think about how to make the api backward compatible.
> >
>
> After thinking through this carefully, I think we probably don't want to
> extend OffsetAndMetadata to include leader epoch because leader epoch is
> kind of implementation detail which ideally should be hidden from user. The
> consumer can include leader epoch in the OffsetCommitRequest after taking
> offset from commitSync(final Map<TopicPartition, OffsetAndMetadata>
> offsets). Similarly consumer can store leader epoch from
> OffsetFetchResponse and only provide offset to user via
> consumer.committed(topicPartition). This solution seems to work well and
> we
> don't have to make changes to consumer's public API. Does this sound OK?
>
>
> >
> > 12. It seems that we now need to store leader epoch in the offset topic.
> > Could you include the new schema for the value of the offset topic and
> add
> > upgrade notes?
>
>
> You are right. I have updated the KIP to specify the new schema for the
> value of the offset topic. Can you take another look?
>
> For existing messages in the offset topic, leader_epoch will be missing. We
> will use leader_epoch = -1 to indicate the missing leader_epoch. Then the
> consumer behavior will be the same as it is now because any leader_epoch in
> the MetadataResponse will be larger than the leader_epoch = -1 in the
> OffetFetchResponse. Thus we don't need specific procedure for upgrades due
> to this change in the offset topic schema. By "upgrade nodes", do you mean
> the sentences we need to include in the upgrade.html in the PR later?
>
>
> >
> > Jun
> >
> >
> > On Tue, Dec 12, 2017 at 5:19 PM, Dong Lin <li...@gmail.com> wrote:
> >
> > > Hey Jun,
> > >
> > > I see. Sounds good. Yeah it is probably simpler to leave this to
> another
> > > KIP in the future.
> > >
> > > Thanks for all the comments. Since there is no further comment in the
> > > community, I will open the voting thread.
> > >
> > > Thanks,
> > > Dong
> > >
> > > On Mon, Dec 11, 2017 at 5:37 PM, Jun Rao <ju...@confluent.io> wrote:
> > >
> > > > Hi, Dong,
> > > >
> > > > The case that I am thinking is network partitioning. Suppose one
> > deploys
> > > a
> > > > stretched cluster across multiple AZs in the same region. If the
> > machines
> > > > in one AZ can't communicate to brokers in other AZs due to a network
> > > issue,
> > > > the brokers in that AZ won't get any new metadata.
> > > >
> > > > We can potentially solve this problem by requiring some kind of
> regular
> > > > heartbeats between the controller and the broker. This may need some
> > more
> > > > thoughts. So, it's probably fine to leave this to another KIP in the
> > > > future.
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > > On Mon, Dec 11, 2017 at 2:55 PM, Dong Lin <li...@gmail.com>
> wrote:
> > > >
> > > > > Hey Jun,
> > > > >
> > > > > Thanks for the comment. I am open to improve this KIP to address
> more
> > > > > problems. I probably need more help in understanding what is the
> > > current
> > > > > problem with consumer using outdated metadata and whether it is
> > easier
> > > to
> > > > > address it together with this KIP.
> > > > >
> > > > > I agree that a consumer can potentially talk to old leader for a
> long
> > > > time
> > > > > even after this KIP. But after this KIP, the consumer probably
> should
> > > not
> > > > > get OffetOutofRangeException and therefore will not cause offset
> > rewind
> > > > > issue. So the only problem is that consumer will not be able to
> fetch
> > > > data
> > > > > until it has updated metadata. It seems that this situation can
> only
> > > > happen
> > > > > if the broker is too slow in processing LeaderAndIsrRequest since
> > > > otherwise
> > > > > the consumer will be forced to update metadata due to
> > > > > NotLeaderForPartitionException. So the problem we are having here
> is
> > > > that
> > > > > consumer will not be able to fetch data if some broker is too slow
> in
> > > > > processing LeaderAndIsrRequest.
> > > > >
> > > > > Because Kafka propagates LeaderAndIsrRequest asynchronously to all
> > > > brokers
> > > > > in the cluster, there will always be a period of time when consumer
> > can
> > > > not
> > > > > fetch data for the partition during the leadership change. Thus it
> > > seems
> > > > > more like a broker-side performance issue instead of client-side
> > > > > correctness issue. My gut feel is that it is not causing a much a
> > > problem
> > > > > as the problem to be fixed in this KIP. And if we were to address
> it,
> > > we
> > > > > probably need to make change in the broker side, e.g. with
> > prioritized
> > > > > queue for controller-related requests, which may be kind of
> > orthogonal
> > > to
> > > > > this KIP. I am not very sure it will be easier to address it with
> the
> > > > > change in this KIP. Do you have any recommendation?
> > > > >
> > > > > Thanks,
> > > > > Dong
> > > > >
> > > > >
> > > > > On Mon, Dec 11, 2017 at 1:51 PM, Jun Rao <ju...@confluent.io> wrote:
> > > > >
> > > > > > Hi, Dong,
> > > > > >
> > > > > > Thanks for the reply.
> > > > > >
> > > > > > My suggestion of forcing the metadata refresh from the controller
> > may
> > > > not
> > > > > > work in general since the cached controller could be outdated
> too.
> > > The
> > > > > > general problem is that if a consumer's metadata is outdated, it
> > may
> > > > get
> > > > > > stuck with the old leader for a long time. We can address the
> issue
> > > of
> > > > > > detecting outdated metadata in a separate KIP in the future if
> you
> > > > didn't
> > > > > > intend to address it in this KIP.
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > >
> > > > > > On Sat, Dec 9, 2017 at 10:12 PM, Dong Lin <li...@gmail.com>
> > > wrote:
> > > > > >
> > > > > > > Hey Jun,
> > > > > > >
> > > > > > > Thanks much for your comments. Given that client needs to
> > > > de-serialize
> > > > > > the
> > > > > > > metadata anyway, the extra overhead of checking the
> per-partition
> > > > > version
> > > > > > > for every partition should not be a big concern. Thus it makes
> > > sense
> > > > to
> > > > > > use
> > > > > > > leader epoch as the per-partition version instead of creating a
> > > > global
> > > > > > > metadata version. I will update the KIP to do that.
> > > > > > >
> > > > > > > Regarding the detection of outdated metadata, I think it is
> > > possible
> > > > to
> > > > > > > ensure that client gets latest metadata by fetching from
> > > controller.
> > > > > Note
> > > > > > > that this requires extra logic in the controller such that
> > > controller
> > > > > > > updates metadata directly in memory without requiring
> > > > > > > UpdateMetadataRequest. But I am not sure the main motivation of
> > > this
> > > > at
> > > > > > > this moment. But this makes controller more like a bottleneck
> in
> > > the
> > > > > > > cluster which we probably want to avoid.
> > > > > > >
> > > > > > > I think we can probably keep the current way of ensuring
> metadata
> > > > > > > freshness. Currently client will be forced to refresh metadata
> if
> > > > > broker
> > > > > > > returns error (e.g. NotLeaderForPartition) due to outdated
> > metadata
> > > > or
> > > > > if
> > > > > > > the metadata does not contain the partition that the client
> > needs.
> > > In
> > > > > the
> > > > > > > future, as you previously suggested, we can include
> per-partition
> > > > > > > leaderEpoch in the FetchRequest/ProduceRequest such that broker
> > can
> > > > > > return
> > > > > > > error if the epoch is smaller than cached epoch in the broker.
> > > Given
> > > > > that
> > > > > > > this adds more complexity to Kafka, I think we can probably
> think
> > > > about
> > > > > > > that leader when we have a specific use-case or problem to
> solve
> > > with
> > > > > > > up-to-date metadata. Does this sound OK?
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Dong
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > On Fri, Dec 8, 2017 at 3:53 PM, Jun Rao <ju...@confluent.io>
> > wrote:
> > > > > > >
> > > > > > > > Hi, Dong,
> > > > > > > >
> > > > > > > > Thanks for the reply. A few more points below.
> > > > > > > >
> > > > > > > > For dealing with how to prevent a consumer switching from a
> new
> > > > > leader
> > > > > > to
> > > > > > > > an old leader, you suggestion that refreshes metadata on
> > consumer
> > > > > > restart
> > > > > > > > until it sees a metadata version >= the one associated with
> the
> > > > > offset
> > > > > > > > works too, as long as we guarantee that the cached metadata
> > > > versions
> > > > > on
> > > > > > > the
> > > > > > > > brokers only go up.
> > > > > > > >
> > > > > > > > The second discussion point is on whether the metadata
> > versioning
> > > > > > should
> > > > > > > be
> > > > > > > > per partition or global. For the partition level versioning,
> > you
> > > > were
> > > > > > > > concerned about the performance. Given that metadata updates
> > are
> > > > > rare,
> > > > > > I
> > > > > > > am
> > > > > > > > not sure if it's a big concern though. Doing a million if
> tests
> > > is
> > > > > > > probably
> > > > > > > > going to take less than 1ms. Another thing is that the
> metadata
> > > > > version
> > > > > > > > seems to need to survive controller failover. In your current
> > > > > > approach, a
> > > > > > > > consumer may not be able to wait on the right version of the
> > > > metadata
> > > > > > > after
> > > > > > > > the consumer restart since the metadata version may have been
> > > > > recycled
> > > > > > on
> > > > > > > > the server side due to a controller failover while the
> consumer
> > > is
> > > > > > down.
> > > > > > > > The partition level leaderEpoch survives controller failure
> and
> > > > won't
> > > > > > > have
> > > > > > > > this issue.
> > > > > > > >
> > > > > > > > Lastly, neither your proposal nor mine addresses the issue
> how
> > to
> > > > > > > guarantee
> > > > > > > > a consumer to detect that is metadata is outdated. Currently,
> > the
> > > > > > > consumer
> > > > > > > > is not guaranteed to fetch metadata from every broker within
> > some
> > > > > > bounded
> > > > > > > > period of time. Maybe this is out of the scope of your KIP.
> But
> > > one
> > > > > > idea
> > > > > > > is
> > > > > > > > force the consumer to refresh metadata from the controller
> > > > > > periodically.
> > > > > > > >
> > > > > > > > Jun
> > > > > > > >
> > > > > > > >
> > > > > > > > On Thu, Dec 7, 2017 at 11:25 AM, Dong Lin <
> lindong28@gmail.com
> > >
> > > > > wrote:
> > > > > > > >
> > > > > > > > > Hey Jun,
> > > > > > > > >
> > > > > > > > > Thanks much for the comments. Great point particularly
> > > regarding
> > > > > > (3). I
> > > > > > > > > haven't thought about this before.
> > > > > > > > >
> > > > > > > > > It seems that there are two possible ways where the version
> > > > number
> > > > > > can
> > > > > > > be
> > > > > > > > > used. One solution is for client to check the version
> number
> > at
> > > > the
> > > > > > > time
> > > > > > > > it
> > > > > > > > > receives MetadataResponse. And if the version number in the
> > > > > > > > > MetadataResponse is smaller than the version number in the
> > > > client's
> > > > > > > > cache,
> > > > > > > > > the client will be forced to fetch metadata again.  Another
> > > > > solution,
> > > > > > > as
> > > > > > > > > you have suggested, is for broker to check the version
> number
> > > at
> > > > > the
> > > > > > > time
> > > > > > > > > it receives a request from client. The broker will reject
> the
> > > > > request
> > > > > > > if
> > > > > > > > > the version is smaller than the version in broker's cache.
> > > > > > > > >
> > > > > > > > > I am not very sure that the second solution can address the
> > > > problem
> > > > > > > here.
> > > > > > > > > In the scenario described in the JIRA ticket, broker's
> cache
> > > may
> > > > be
> > > > > > > > > outdated because it has not processed the
> LeaderAndIsrRequest
> > > > from
> > > > > > the
> > > > > > > > > controller. Thus it may still process client's request even
> > if
> > > > the
> > > > > > > > version
> > > > > > > > > in client's request is actually outdated. Does this make
> > sense?
> > > > > > > > >
> > > > > > > > > IMO, it seems that we can address problem (3) by saving the
> > > > > metadata
> > > > > > > > > version together with the offset. After consumer starts, it
> > > will
> > > > > keep
> > > > > > > > > fetching metadata until the metadata version >= the version
> > > saved
> > > > > > with
> > > > > > > > the
> > > > > > > > > offset of this partition.
> > > > > > > > >
> > > > > > > > > Regarding problems (1) and (2): Currently we use the
> version
> > > > number
> > > > > > in
> > > > > > > > the
> > > > > > > > > MetadataResponse to ensure that the metadata does not go
> back
> > > in
> > > > > > time.
> > > > > > > > > There are two alternative solutions to address problems (1)
> > and
> > > > > (2).
> > > > > > > One
> > > > > > > > > solution is for client to enumerate all partitions in the
> > > > > > > > MetadataResponse,
> > > > > > > > > compare their epoch with those in the cached metadata, and
> > > > rejects
> > > > > > the
> > > > > > > > > MetadataResponse iff any leader epoch is smaller. The main
> > > > concern
> > > > > is
> > > > > > > > that
> > > > > > > > > MetadataResponse currently cached information of all
> > partitions
> > > > in
> > > > > > the
> > > > > > > > > entire cluster. It may slow down client's performance if we
> > > were
> > > > to
> > > > > > do
> > > > > > > > it.
> > > > > > > > > The other solution is for client to enumerate partitions
> for
> > > only
> > > > > > > topics
> > > > > > > > > registered in the org.apache.kafka.clients.Metadata, which
> > > will
> > > > be
> > > > > > an
> > > > > > > > > empty
> > > > > > > > > set for producer and the set of subscribed partitions for
> > > > consumer.
> > > > > > But
> > > > > > > > > this degrades to all topics if consumer subscribes to
> topics
> > in
> > > > the
> > > > > > > > cluster
> > > > > > > > > by pattern.
> > > > > > > > >
> > > > > > > > > Note that client will only be forced to update metadata if
> > the
> > > > > > version
> > > > > > > in
> > > > > > > > > the MetadataResponse is smaller than the version in the
> > cached
> > > > > > > metadata.
> > > > > > > > In
> > > > > > > > > general it should not be a problem. It can be a problem
> only
> > if
> > > > > some
> > > > > > > > broker
> > > > > > > > > is particularly slower than other brokers in processing
> > > > > > > > > UpdateMetadataRequest. When this is the case, it means that
> > the
> > > > > > broker
> > > > > > > is
> > > > > > > > > also particularly slower in processing LeaderAndIsrRequest,
> > > which
> > > > > can
> > > > > > > > cause
> > > > > > > > > problem anyway because some partition will probably have no
> > > > leader
> > > > > > > during
> > > > > > > > > this period. I am not sure problems (1) and (2) cause more
> > > > problem
> > > > > > than
> > > > > > > > > what we already have.
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > > Dong
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Wed, Dec 6, 2017 at 6:42 PM, Jun Rao <ju...@confluent.io>
> > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hi, Dong,
> > > > > > > > > >
> > > > > > > > > > Great finding on the issue. It's a real problem. A few
> > > comments
> > > > > > about
> > > > > > > > the
> > > > > > > > > > KIP. (1) I am not sure about updating
> > > controller_metadata_epoch
> > > > > on
> > > > > > > > every
> > > > > > > > > > UpdateMetadataRequest. Currently, the controller can send
> > > > > > > > > > UpdateMetadataRequest when there is no actual metadata
> > > change.
> > > > > > Doing
> > > > > > > > this
> > > > > > > > > > may require unnecessary metadata refresh on the client.
> (2)
> > > > > > > > > > controller_metadata_epoch is global across all topics.
> This
> > > > means
> > > > > > > that
> > > > > > > > a
> > > > > > > > > > client may be forced to update its metadata even when the
> > > > > metadata
> > > > > > > for
> > > > > > > > > the
> > > > > > > > > > topics that it cares haven't changed. (3) It doesn't seem
> > > that
> > > > > the
> > > > > > > KIP
> > > > > > > > > > handles the corner case when a consumer is restarted.
> Say a
> > > > > > consumer
> > > > > > > > > reads
> > > > > > > > > > from the new leader, commits the offset and then is
> > > restarted.
> > > > On
> > > > > > > > > restart,
> > > > > > > > > > the consumer gets an outdated metadata and fetches from
> the
> > > old
> > > > > > > leader.
> > > > > > > > > > Then, the consumer will get into the offset out of range
> > > issue.
> > > > > > > > > >
> > > > > > > > > > Given the above, I am thinking of the following approach.
> > We
> > > > > > actually
> > > > > > > > > > already have metadata versioning at the partition level.
> > Each
> > > > > > leader
> > > > > > > > has
> > > > > > > > > a
> > > > > > > > > > leader epoch which is monotonically increasing. We can
> > > > > potentially
> > > > > > > > > > propagate leader epoch back in the metadata response and
> > the
> > > > > > clients
> > > > > > > > can
> > > > > > > > > > cache that. This solves the issue of (1) and (2). To
> solve
> > > (3),
> > > > > > when
> > > > > > > > > saving
> > > > > > > > > > an offset, we could save both an offset and the
> > corresponding
> > > > > > leader
> > > > > > > > > epoch.
> > > > > > > > > > When fetching the data, the consumer provides both the
> > offset
> > > > and
> > > > > > the
> > > > > > > > > > leader epoch. A leader will only serve the request if its
> > > > leader
> > > > > > > epoch
> > > > > > > > is
> > > > > > > > > > equal to or greater than the leader epoch from the
> > consumer.
> > > To
> > > > > > > achieve
> > > > > > > > > > this, we need to change the fetch request protocol and
> the
> > > > offset
> > > > > > > > commit
> > > > > > > > > > api, which requires some more thoughts.
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > >
> > > > > > > > > > Jun
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Wed, Dec 6, 2017 at 10:57 AM, Dong Lin <
> > > lindong28@gmail.com
> > > > >
> > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Bump up the thread.
> > > > > > > > > > >
> > > > > > > > > > > It will be great to have more comments on whether we
> > should
> > > > do
> > > > > it
> > > > > > > or
> > > > > > > > > > > whether there is better way to address the motivation
> of
> > > this
> > > > > > KIP.
> > > > > > > > > > >
> > > > > > > > > > > On Mon, Dec 4, 2017 at 3:09 PM, Dong Lin <
> > > > lindong28@gmail.com>
> > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > I don't have an interesting rejected alternative
> > solution
> > > > to
> > > > > > put
> > > > > > > in
> > > > > > > > > the
> > > > > > > > > > > > KIP. If there is good alternative solution from
> anyone
> > in
> > > > > this
> > > > > > > > > thread,
> > > > > > > > > > I
> > > > > > > > > > > am
> > > > > > > > > > > > happy to discuss this and update the KIP accordingly.
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks,
> > > > > > > > > > > > Dong
> > > > > > > > > > > >
> > > > > > > > > > > > On Mon, Dec 4, 2017 at 1:12 PM, Ted Yu <
> > > > yuzhihong@gmail.com>
> > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > >> It is clearer now.
> > > > > > > > > > > >>
> > > > > > > > > > > >> I noticed that Rejected Alternatives section is
> empty.
> > > > > > > > > > > >> Have you considered any alternative ?
> > > > > > > > > > > >>
> > > > > > > > > > > >> Cheers
> > > > > > > > > > > >>
> > > > > > > > > > > >> On Mon, Dec 4, 2017 at 1:07 PM, Dong Lin <
> > > > > lindong28@gmail.com
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > > > >>
> > > > > > > > > > > >> > Ted, thanks for catching this. I have updated the
> > > > sentence
> > > > > > to
> > > > > > > > make
> > > > > > > > > > it
> > > > > > > > > > > >> > readable.
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > Thanks,
> > > > > > > > > > > >> > Dong
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > On Sat, Dec 2, 2017 at 3:05 PM, Ted Yu <
> > > > > yuzhihong@gmail.com
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > > bq. It the controller_epoch of the incoming
> > > > > > > MetadataResponse,
> > > > > > > > or
> > > > > > > > > > if
> > > > > > > > > > > >> the
> > > > > > > > > > > >> > > controller_epoch is the same but the
> > > > > > > controller_metadata_epoch
> > > > > > > > > > > >> > >
> > > > > > > > > > > >> > > Can you update the above sentence so that the
> > > > intention
> > > > > is
> > > > > > > > > > clearer ?
> > > > > > > > > > > >> > >
> > > > > > > > > > > >> > > Thanks
> > > > > > > > > > > >> > >
> > > > > > > > > > > >> > > On Fri, Dec 1, 2017 at 6:33 PM, Dong Lin <
> > > > > > > lindong28@gmail.com
> > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > > > >> > >
> > > > > > > > > > > >> > > > Hi all,
> > > > > > > > > > > >> > > >
> > > > > > > > > > > >> > > > I have created KIP-232: Detect outdated
> metadata
> > > by
> > > > > > adding
> > > > > > > > > > > >> > > > ControllerMetadataEpoch field:
> > > > > > > > > > > >> > > > https://cwiki.apache.org/
> > > > > confluence/display/KAFKA/KIP-
> > > > > > > > > > > >> > > > 232%3A+Detect+outdated+metadata+by+adding+
> > > > > > > > > > > >> > ControllerMetadataEpoch+field
> > > > > > > > > > > >> > > > .
> > > > > > > > > > > >> > > >
> > > > > > > > > > > >> > > > The KIP proposes to add fields in
> > MetadataResponse
> > > > and
> > > > > > > > > > > >> > > > UpdateMetadataRequest so that client can
> reject
> > > > > outdated
> > > > > > > > > > metadata
> > > > > > > > > > > >> and
> > > > > > > > > > > >> > > avoid
> > > > > > > > > > > >> > > > unnecessary OffsetOutOfRangeException.
> Otherwise
> > > > there
> > > > > > is
> > > > > > > > > > > currently
> > > > > > > > > > > >> > race
> > > > > > > > > > > >> > > > condition that can cause consumer to reset
> > offset
> > > > > which
> > > > > > > > > > negatively
> > > > > > > > > > > >> > affect
> > > > > > > > > > > >> > > > the consumer's availability.
> > > > > > > > > > > >> > > >
> > > > > > > > > > > >> > > > Feedback and suggestions are welcome!
> > > > > > > > > > > >> > > >
> > > > > > > > > > > >> > > > Regards,
> > > > > > > > > > > >> > > > Dong
> > > > > > > > > > > >> > > >
> > > > > > > > > > > >> > >
> > > > > > > > > > > >> >
> > > > > > > > > > > >>
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Dong Lin <li...@gmail.com>.
Hey Jason,

Thanks for the comments.

On Mon, Jan 8, 2018 at 2:27 PM, Jason Gustafson <ja...@confluent.io> wrote:

> >
> > I am not sure I understand the benefit of incrementing this epoch after
> > topic deletion. At a high level, client can not differentiate between
> topic
> > deletion and topic creation when the global epoch is incremented. Can you
> > provide more specific use-case?
>
>
> Say you send two metadata requests to two separate brokers. In the
> responses, one of them says a certain topic exists and one says it does
> not. Who is right? My suggestion is to bump the topic epoch on deletion and
> include it in the metadata response when returning
> UNKNOWN_TOPIC_OR_PARTITION. Then the client always knows which metadata is
> more current (if not necessarily up to date). Because of this ambiguity,
> Kafka clients currently have no choice but retry on unknown topic errors.
> Yes, you can say it is already handled, but this gives us some better
> options in the future. In the consumer, users are often asking to be
> notified when they attempt to fetch from unknown topics, for example,
> because it could indicate a configuration problem. We have difficulty
> supporting this at the moment.
>
> Currently when broker returns UNKNOWN_TOPIC_OR_PARTITION, it means that the
> > topic is not follower or leader of this partition. Note that
> > UNKNOWN_TOPIC_OR_PARTITION does not necessarily tell client whether this
> > partition exists on other broker or not. UNKNOWN_TOPIC_OR_PARTITION can
> be
> > caused either when the broker has not yet processed the latest
> > LeaderAndIsrRequest, or the client is using outdated metadata.
>

My bad. I was thinking about the UNKNOWN_TOPIC_OR_PARTITION from
ProduceResponse or FetchReponse but we are actually discussing the error
from MetadataResponse.



>
>
> I don't think this is right. Metadata is propagated through the
> UpdateMetadata request which the controller sends to all brokers. Brokers
> will return UNKNOWN_TOPIC_OR_PARTITION in a metadata response if they don't
> have metadata cached for the requested topic.
>
> There is one problem though which I think might be what you're getting at.
> After a topic is deleted, the controller will leave it out of future
> UpdateMetadata requests, which means the deleted epoch would not be
> propagated to all brokers and we'd be stuck in the current state. Suppose
> instead that when a topic is deleted, we 1) bump the topic epoch, and 2)
> set an expiration time (say several hours). When the expiration time is
> reached, we delete the topic metadata in zookeeper; until then, the
> controller continues to propagate it as usual with a flag indicating it no
> longer exists. The point of the epoch is solving edge cases around topic
> deletion and recreation, so the expiration timer gives clients a window to
> observe the deletion before the metadata is removed. It also ensures that
> topic metadata is eventually cleaned up following deletion.
>
> What do you think?
>

Yeah that is what I am thinking. As of current proposal, when a topic is
deleted, all topic-specific information will be immediately removed and
client will not see this topic in the latest MetadataResponse. This makes
it hard to distinguish between the metadata before and after the topic
deletion even if we bump up the global topic_epoch in the zookeeper.

I think your approach will allow user to distinguish between the metadata
before and after the topic deletion. I also agree that this can be
potentially be useful to user. I am just not very sure whether we already
have a good use-case to make the additional complexity worthwhile. It seems
that this feature is kind of independent of the main problem of this KIP.
Could we add this as a future work?



> In comparison to byte[], String has the benefit of being more readable and
> > it is also the same type of the existing metadata field, which is used
> for
> > a similar purpose by user. Do you think this is reasonable?
>
>
> I don't have too strong of a feeling about it. I'm not sure how important
> readability is since it's intended to be opaque to the user. To clarify a
> little bit, I think we should continue to send the topic and leader epochs
> in the protocol directly as separate fields. It's only when we surface it
> through the consumer API that we add some obscurity since we don't want
> users to depend on the fields directly and we don't want to make API
> changes in the future if we need to add something else which is also
> internal. In fact, rather than using byte[] or String directly, perhaps we
> could just expose it as an object and give it a readable toString()?
>

I am also not very strong about this. String just seems like a nature
choice given that we already have similar json formatted strong in
zookeeper and we use String for the existing metadata in OffsetAndMetadata.
I just feel it is probably better to use some Java class than using raw
byte array in our Consumer API. But I don't have a strong reason.

If we expose it as an object, I am not very sure how user is going to store
and retrieve this value from an external store. If user stores the
object.toString() in the external store, how can user convert the string
back to the object?

And yes, this only matters to the consumer API. The current KIP continues
to send leader_epoch and topic_epoch as separate fields in request/response
and the offset topic schema.

Thanks much,
Dong



>
> Thanks,
> Jason
>
>
> On Fri, Jan 5, 2018 at 5:12 PM, Dong Lin <li...@gmail.com> wrote:
>
> > Hey Jason,
> >
> > Thanks a lot for the comments. I will comment inline. And I have updated
> > the KIP accordingly. Could you take another look?
> >
> > On Fri, Jan 5, 2018 at 11:15 AM, Jason Gustafson <ja...@confluent.io>
> > wrote:
> >
> > > Hi Dong,
> > >
> > > Sorry for the late reply. I think the latest revision is looking good.
> I
> > > have a few minor suggestions:
> > >
> > > 1. The name "partition_epoch" makes me think it changes independently
> at
> > > the partition level, but all partitions for a topic should have the
> same
> > > epoch. Maybe "topic_epoch" is nearer the mark?
> > >
> >
> > Actually, in the current proposal, partitions of the same topic will have
> > different epoch. Every time a new partition is created, either due to
> topic
> > creation or partition expansion, the global epoch is incremented by 1 and
> > is assigned to that partition. This is probably why we currently call it
> > partition_epoch.
> >
> > Thinking about your idea more, one alternative approach following your
> idea
> > is to use a topic_epoch is that incremented by 1 whenever we create a
> > topic. We should store a single topic_epoch in
> > znode /brokers/topics/[topic] without storing the list of partition_epoch
> > for all partitions. This same epoch will be used for the new partitions
> > after partition expansion of the existing topic. This approach has more
> > simpler znode format than the existing KIP and it still allows us to
> detect
> > topic created after topic deletion. I think this is better. I have
> updated
> > the KIP with this approach.
> >
> >
> > > 2. Should we increment this epoch when a topic is deleted also? When
> the
> > > broker returns an UNKNOWN_TOPIC_OR_PARTITION error in a metadata
> > response,
> > > we can also include the latest partition epoch, which would allow the
> > > client to disambiguate the error if it has seen more recent metadata.
> > >
> >
> > I am not sure I understand the benefit of incrementing this epoch after
> > topic deletion. At a high level, client can not differentiate between
> topic
> > deletion and topic creation when the global epoch is incremented. Can you
> > provide more specific use-case?
> >
> > Currently when broker returns UNKNOWN_TOPIC_OR_PARTITION, it means that
> the
> > topic is not follower or leader of this partition. Note that
> > UNKNOWN_TOPIC_OR_PARTITION does not necessarily tell client whether this
> > partition exists on other broker or not. UNKNOWN_TOPIC_OR_PARTITION can
> be
> > caused either when the broker has not yet processed the latest
> > LeaderAndIsrRequest, or the client is using outdated metadata. In either
> > case, the client needs to retry and possibly refresh metadata, which is
> > already done by client with the current Kafka implementation. So it seems
> > that we don't have a problem to fix here?
> >
> >
> >
> > > 3. I am still wondering whether it is a good idea to expose these
> epochs
> > in
> > > the consumer API. As an alternative, have you considered representing
> the
> > > data as an opaque blob of bytes? For example:
> > >
> > > class OffsetAndMetadata {
> > >   long offset;
> > >   byte[] offsetMetadata;
> > >   String metadata;
> > > }
> > >
> > > Admittedly, the naming is a bit annoying, but we can probably come up
> > with
> > > something better. Internally the byte array would have a version. If in
> > the
> > > future we have anything else we need to add, we can update the version
> > and
> > > we wouldn't need any new APIs.
> > >
> > > The corresponding seek() and position() APIs might look something like
> > > this:
> > >
> > > void seek(TopicPartition partition, long offset, byte[]
> offsetMetadata);
> > > byte[] positionMetadata(TopicPartition partition);
> > >
> > > What do you think?
> > >
> >
> > I think it is a very good idea to consolidate the new information into a
> > single field rather than explicitly listing them with specific types.
> This
> > provides us the advantage of evolving the information in this field in
> the
> > future.
> >
> > But I probably would prefer to use String rather than byte[] as the type
> of
> > this new field. This string can probably have the following json format:
> >
> > {
> >   "version": 1,
> >   "topic_epoch": int,
> >   "leader_epoch": int.
> > }
> >
> > In comparison to byte[], String has the benefit of being more readable
> and
> > it is also the same type of the existing metadata field, which is used
> for
> > a similar purpose by user. Do you think this is reasonable?
> >
> >
> >
> > >
> > > Thanks,
> > > Jason
> > >
> > > On Thu, Jan 4, 2018 at 7:04 PM, Dong Lin <li...@gmail.com> wrote:
> > >
> > > > Hey Jun, Jason,
> > > >
> > > > Thanks much for all the feedback. I have updated the KIP based on the
> > > > latest discussion. Can you help check whether it looks good?
> > > >
> > > > Thanks,
> > > > Dong
> > > >
> > > > On Thu, Jan 4, 2018 at 5:36 PM, Dong Lin <li...@gmail.com>
> wrote:
> > > >
> > > > > Hey Jun,
> > > > >
> > > > > Hmm... thinking about this more, I am not sure that the proposed
> API
> > is
> > > > > sufficient. For users that store offset externally, we probably
> need
> > > > extra
> > > > > API to return the leader_epoch and partition_epoch for all
> partitions
> > > > that
> > > > > consumers are consuming. I suppose these users currently use
> > position()
> > > > to
> > > > > get the offset. Thus we probably need a new method
> > > positionWithEpoch(..)
> > > > to
> > > > > return <offset, partition_epoch, leader_epoch>. Does this sound
> > > > reasonable?
> > > > >
> > > > > Thanks,
> > > > > Dong
> > > > >
> > > > >
> > > > > On Thu, Jan 4, 2018 at 5:26 PM, Jun Rao <ju...@confluent.io> wrote:
> > > > >
> > > > >> Hi, Dong,
> > > > >>
> > > > >> Yes, that's what I am thinking. OffsetEpoch will be composed of
> > > > >> (partition_epoch,
> > > > >> leader_epoch).
> > > > >>
> > > > >> Thanks,
> > > > >>
> > > > >> Jun
> > > > >>
> > > > >>
> > > > >> On Thu, Jan 4, 2018 at 4:22 PM, Dong Lin <li...@gmail.com>
> > wrote:
> > > > >>
> > > > >> > Hey Jun,
> > > > >> >
> > > > >> > Thanks much. I like the the new API that you proposed. I am not
> > sure
> > > > >> what
> > > > >> > you exactly mean by offset_epoch. I suppose that we can use the
> > pair
> > > > of
> > > > >> > (partition_epoch, leader_epoch) as the offset_epoch, right?
> > > > >> >
> > > > >> > Thanks,
> > > > >> > Dong
> > > > >> >
> > > > >> > On Thu, Jan 4, 2018 at 4:02 PM, Jun Rao <ju...@confluent.io>
> wrote:
> > > > >> >
> > > > >> > > Hi, Dong,
> > > > >> > >
> > > > >> > > Got it. The api that you proposed works. The question is
> whether
> > > > >> that's
> > > > >> > the
> > > > >> > > api that we want to have in the long term. My concern is that
> > > while
> > > > >> the
> > > > >> > api
> > > > >> > > change is simple, the new api seems harder to explain and use.
> > For
> > > > >> > example,
> > > > >> > > a consumer storing offsets externally now needs to call
> > > > >> > > waitForMetadataUpdate() after calling seek().
> > > > >> > >
> > > > >> > > An alternative approach is to make the following compatible
> api
> > > > >> changes
> > > > >> > in
> > > > >> > > Consumer.
> > > > >> > > * Add an additional OffsetEpoch field in OffsetAndMetadata.
> (no
> > > need
> > > > >> to
> > > > >> > > change the CommitSync() api)
> > > > >> > > * Add a new api seek(TopicPartition partition, long offset,
> > > > >> OffsetEpoch
> > > > >> > > offsetEpoch). We can potentially deprecate the old api
> > > > >> > seek(TopicPartition
> > > > >> > > partition, long offset) in the future.
> > > > >> > >
> > > > >> > > The alternative approach has similar amount of api changes as
> > > yours
> > > > >> but
> > > > >> > has
> > > > >> > > the following benefits.
> > > > >> > > 1. The api works in a similar way as how offset management
> works
> > > now
> > > > >> and
> > > > >> > is
> > > > >> > > probably what we want in the long term.
> > > > >> > > 2. It can reset offsets better when there is data loss due to
> > > > unclean
> > > > >> > > leader election or correlated replica failure.
> > > > >> > > 3. It can reset offsets better when topic is recreated.
> > > > >> > >
> > > > >> > > Thanks,
> > > > >> > >
> > > > >> > > Jun
> > > > >> > >
> > > > >> > >
> > > > >> > > On Thu, Jan 4, 2018 at 2:05 PM, Dong Lin <lindong28@gmail.com
> >
> > > > wrote:
> > > > >> > >
> > > > >> > > > Hey Jun,
> > > > >> > > >
> > > > >> > > > Yeah I agree that ideally we don't want an ever growing
> global
> > > > >> metadata
> > > > >> > > > version. I just think it may be more desirable to keep the
> > > > consumer
> > > > >> API
> > > > >> > > > simple.
> > > > >> > > >
> > > > >> > > > In my current proposal, metadata version returned in the
> fetch
> > > > >> response
> > > > >> > > > will be stored with the offset together. More specifically,
> > the
> > > > >> > > > metadata_epoch in the new offset topic schema will be the
> > > largest
> > > > >> > > > metadata_epoch from all the MetadataResponse and
> FetchResponse
> > > > ever
> > > > >> > > > received by this consumer.
> > > > >> > > >
> > > > >> > > > We probably don't have to change the consumer API for
> > > > >> > > > commitSync(Map<TopicPartition, OffsetAndMetadata>). If user
> > > calls
> > > > >> > > > commitSync(...) to commit offset 10 for a given partition,
> for
> > > > most
> > > > >> > > > use-cases, this consumer instance should have consumed
> message
> > > > with
> > > > >> > > offset
> > > > >> > > > 9 from this partition, in which case the consumer can
> remember
> > > and
> > > > >> use
> > > > >> > > the
> > > > >> > > > metadata_epoch from the corresponding FetchResponse when
> > > > committing
> > > > >> > > offset.
> > > > >> > > > If user calls commitSync(..) to commit offset 10 for a given
> > > > >> partition
> > > > >> > > > without having consumed the message with offset 9 using this
> > > > >> consumer
> > > > >> > > > instance, this is probably an advanced use-case. In this
> case
> > > the
> > > > >> > > advanced
> > > > >> > > > user can retrieve the metadata_epoch using the newly added
> > > > >> > > metadataEpoch()
> > > > >> > > > API after it fetches the message with offset 9 (probably
> from
> > > > >> another
> > > > >> > > > consumer instance) and encode this metadata_epoch in the
> > > > >> > > > string OffsetAndMetadata.metadata. Do you think this
> solution
> > > > would
> > > > >> > work?
> > > > >> > > >
> > > > >> > > > By "not sure that I fully understand your latest
> suggestion",
> > > are
> > > > >> you
> > > > >> > > > referring to solution related to unclean leader election
> using
> > > > >> > > leader_epoch
> > > > >> > > > in my previous email?
> > > > >> > > >
> > > > >> > > > Thanks,
> > > > >> > > > Dong
> > > > >> > > >
> > > > >> > > > On Thu, Jan 4, 2018 at 1:33 PM, Jun Rao <ju...@confluent.io>
> > > wrote:
> > > > >> > > >
> > > > >> > > > > Hi, Dong,
> > > > >> > > > >
> > > > >> > > > > Not sure that I fully understand your latest suggestion.
> > > > >> Returning an
> > > > >> > > > ever
> > > > >> > > > > growing global metadata version itself is no ideal, but is
> > > fine.
> > > > >> My
> > > > >> > > > > question is whether the metadata version returned in the
> > fetch
> > > > >> > response
> > > > >> > > > > needs to be stored with the offset together if offsets are
> > > > stored
> > > > >> > > > > externally. If so, we also have to change the consumer API
> > for
> > > > >> > > > commitSync()
> > > > >> > > > > and need to worry about compatibility. If we don't store
> the
> > > > >> metadata
> > > > >> > > > > version together with the offset, on a consumer restart,
> > it's
> > > > not
> > > > >> > clear
> > > > >> > > > how
> > > > >> > > > > we can ensure the metadata in the consumer is high enough
> > > since
> > > > >> there
> > > > >> > > is
> > > > >> > > > no
> > > > >> > > > > metadata version to compare with.
> > > > >> > > > >
> > > > >> > > > > Thanks,
> > > > >> > > > >
> > > > >> > > > > Jun
> > > > >> > > > >
> > > > >> > > > >
> > > > >> > > > > On Wed, Jan 3, 2018 at 6:43 PM, Dong Lin <
> > lindong28@gmail.com
> > > >
> > > > >> > wrote:
> > > > >> > > > >
> > > > >> > > > > > Hey Jun,
> > > > >> > > > > >
> > > > >> > > > > > Thanks much for the explanation.
> > > > >> > > > > >
> > > > >> > > > > > I understand the advantage of partition_epoch over
> > > > >> metadata_epoch.
> > > > >> > My
> > > > >> > > > > > current concern is that the use of leader_epoch and the
> > > > >> > > partition_epoch
> > > > >> > > > > > requires us considerable change to consumer's public API
> > to
> > > > take
> > > > >> > care
> > > > >> > > > of
> > > > >> > > > > > the case where user stores offset externally. For
> example,
> > > > >> > > *consumer*.
> > > > >> > > > > > *commitSync*(..) would have to take a map whose value is
> > > > >> <offset,
> > > > >> > > > > metadata,
> > > > >> > > > > > leader epoch, partition epoch>. *consumer*.*seek*(...)
> > would
> > > > >> also
> > > > >> > > need
> > > > >> > > > > > leader_epoch and partition_epoch as parameter.
> Technically
> > > we
> > > > >> can
> > > > >> > > > > probably
> > > > >> > > > > > still make it work in a backward compatible manner after
> > > > careful
> > > > >> > > design
> > > > >> > > > > and
> > > > >> > > > > > discussion. But these changes can make the consumer's
> > > > interface
> > > > >> > > > > > unnecessarily complex for more users who do not store
> > offset
> > > > >> > > > externally.
> > > > >> > > > > >
> > > > >> > > > > > After thinking more about it, we can address all
> problems
> > > > >> discussed
> > > > >> > > by
> > > > >> > > > > only
> > > > >> > > > > > using the metadata_epoch without introducing
> leader_epoch
> > or
> > > > the
> > > > >> > > > > > partition_epoch. The current KIP describes the changes
> to
> > > the
> > > > >> > > consumer
> > > > >> > > > > API
> > > > >> > > > > > and how the new API can be used if user stores offset
> > > > >> externally.
> > > > >> > In
> > > > >> > > > > order
> > > > >> > > > > > to address the scenario you described earlier, we can
> > > include
> > > > >> > > > > > metadata_epoch in the FetchResponse and the
> > > > LeaderAndIsrRequest.
> > > > >> > > > Consumer
> > > > >> > > > > > remembers the largest metadata_epoch from all the
> > > > FetchResponse
> > > > >> it
> > > > >> > > has
> > > > >> > > > > > received. The metadata_epoch committed with the offset,
> > > either
> > > > >> > within
> > > > >> > > > or
> > > > >> > > > > > outside Kafka, should be the largest metadata_epoch
> across
> > > all
> > > > >> > > > > > FetchResponse and MetadataResponse ever received by this
> > > > >> consumer.
> > > > >> > > > > >
> > > > >> > > > > > The drawback of using only the metadata_epoch is that we
> > can
> > > > not
> > > > >> > > always
> > > > >> > > > > do
> > > > >> > > > > > the smart offset reset in case of unclean leader
> election
> > > > which
> > > > >> you
> > > > >> > > > > > mentioned earlier. But in most case, unclean leader
> > election
> > > > >> > probably
> > > > >> > > > > > happens when consumer is not rebalancing/restarting. In
> > > these
> > > > >> > cases,
> > > > >> > > > > either
> > > > >> > > > > > consumer is not directly affected by unclean leader
> > election
> > > > >> since
> > > > >> > it
> > > > >> > > > is
> > > > >> > > > > > not consuming from the end of the log, or consumer can
> > > derive
> > > > >> the
> > > > >> > > > > > leader_epoch from the most recent message received
> before
> > it
> > > > >> sees
> > > > >> > > > > > OffsetOutOfRangeException. So I am not sure it is worth
> > > adding
> > > > >> the
> > > > >> > > > > > leader_epoch to consumer API to address the remaining
> > corner
> > > > >> case.
> > > > >> > > What
> > > > >> > > > > do
> > > > >> > > > > > you think?
> > > > >> > > > > >
> > > > >> > > > > > Thanks,
> > > > >> > > > > > Dong
> > > > >> > > > > >
> > > > >> > > > > >
> > > > >> > > > > >
> > > > >> > > > > > On Tue, Jan 2, 2018 at 6:28 PM, Jun Rao <
> jun@confluent.io
> > >
> > > > >> wrote:
> > > > >> > > > > >
> > > > >> > > > > > > Hi, Dong,
> > > > >> > > > > > >
> > > > >> > > > > > > Thanks for the reply.
> > > > >> > > > > > >
> > > > >> > > > > > > To solve the topic recreation issue, we could use
> > either a
> > > > >> global
> > > > >> > > > > > metadata
> > > > >> > > > > > > version or a partition level epoch. But either one
> will
> > > be a
> > > > >> new
> > > > >> > > > > concept,
> > > > >> > > > > > > right? To me, the latter seems more natural. It also
> > makes
> > > > it
> > > > >> > > easier
> > > > >> > > > to
> > > > >> > > > > > > detect if a consumer's offset is still valid after a
> > topic
> > > > is
> > > > >> > > > > recreated.
> > > > >> > > > > > As
> > > > >> > > > > > > you pointed out, we don't need to store the partition
> > > epoch
> > > > in
> > > > >> > the
> > > > >> > > > > > message.
> > > > >> > > > > > > The following is what I am thinking. When a partition
> is
> > > > >> created,
> > > > >> > > we
> > > > >> > > > > can
> > > > >> > > > > > > assign a partition epoch from an ever-increasing
> global
> > > > >> counter
> > > > >> > and
> > > > >> > > > > store
> > > > >> > > > > > > it in /brokers/topics/[topic]/
> partitions/[partitionId]
> > in
> > > > ZK.
> > > > >> > The
> > > > >> > > > > > > partition
> > > > >> > > > > > > epoch is propagated to every broker. The consumer will
> > be
> > > > >> > tracking
> > > > >> > > a
> > > > >> > > > > > tuple
> > > > >> > > > > > > of <offset, leader epoch, partition epoch> for
> offsets.
> > > If a
> > > > >> > topic
> > > > >> > > is
> > > > >> > > > > > > recreated, it's possible that a consumer's offset and
> > > leader
> > > > >> > epoch
> > > > >> > > > > still
> > > > >> > > > > > > match that in the broker, but partition epoch won't
> be.
> > In
> > > > >> this
> > > > >> > > case,
> > > > >> > > > > we
> > > > >> > > > > > > can potentially still treat the consumer's offset as
> out
> > > of
> > > > >> range
> > > > >> > > and
> > > > >> > > > > > reset
> > > > >> > > > > > > the offset based on the offset reset policy in the
> > > consumer.
> > > > >> This
> > > > >> > > > seems
> > > > >> > > > > > > harder to do with a global metadata version.
> > > > >> > > > > > >
> > > > >> > > > > > > Jun
> > > > >> > > > > > >
> > > > >> > > > > > >
> > > > >> > > > > > >
> > > > >> > > > > > > On Mon, Dec 25, 2017 at 6:56 AM, Dong Lin <
> > > > >> lindong28@gmail.com>
> > > > >> > > > wrote:
> > > > >> > > > > > >
> > > > >> > > > > > > > Hey Jun,
> > > > >> > > > > > > >
> > > > >> > > > > > > > This is a very good example. After thinking through
> > this
> > > > in
> > > > >> > > > detail, I
> > > > >> > > > > > > agree
> > > > >> > > > > > > > that we need to commit offset with leader epoch in
> > order
> > > > to
> > > > >> > > address
> > > > >> > > > > > this
> > > > >> > > > > > > > example.
> > > > >> > > > > > > >
> > > > >> > > > > > > > I think the remaining question is how to address the
> > > > >> scenario
> > > > >> > > that
> > > > >> > > > > the
> > > > >> > > > > > > > topic is deleted and re-created. One possible
> solution
> > > is
> > > > to
> > > > >> > > commit
> > > > >> > > > > > > offset
> > > > >> > > > > > > > with both the leader epoch and the metadata version.
> > The
> > > > >> logic
> > > > >> > > and
> > > > >> > > > > the
> > > > >> > > > > > > > implementation of this solution does not require a
> new
> > > > >> concept
> > > > >> > > > (e.g.
> > > > >> > > > > > > > partition epoch) and it does not require any change
> to
> > > the
> > > > >> > > message
> > > > >> > > > > > format
> > > > >> > > > > > > > or leader epoch. It also allows us to order the
> > metadata
> > > > in
> > > > >> a
> > > > >> > > > > > > > straightforward manner which may be useful in the
> > > future.
> > > > >> So it
> > > > >> > > may
> > > > >> > > > > be
> > > > >> > > > > > a
> > > > >> > > > > > > > better solution than generating a random partition
> > epoch
> > > > >> every
> > > > >> > > time
> > > > >> > > > > we
> > > > >> > > > > > > > create a partition. Does this sound reasonable?
> > > > >> > > > > > > >
> > > > >> > > > > > > > Previously one concern with using the metadata
> version
> > > is
> > > > >> that
> > > > >> > > > > consumer
> > > > >> > > > > > > > will be forced to refresh metadata even if metadata
> > > > version
> > > > >> is
> > > > >> > > > > > increased
> > > > >> > > > > > > > due to topics that the consumer is not interested
> in.
> > > Now
> > > > I
> > > > >> > > > realized
> > > > >> > > > > > that
> > > > >> > > > > > > > this is probably not a problem. Currently client
> will
> > > > >> refresh
> > > > >> > > > > metadata
> > > > >> > > > > > > > either due to InvalidMetadataException in the
> response
> > > > from
> > > > >> > > broker
> > > > >> > > > or
> > > > >> > > > > > due
> > > > >> > > > > > > > to metadata expiry. The addition of the metadata
> > version
> > > > >> should
> > > > >> > > > > > increase
> > > > >> > > > > > > > the overhead of metadata refresh caused by
> > > > >> > > > InvalidMetadataException.
> > > > >> > > > > If
> > > > >> > > > > > > > client refresh metadata due to expiry and it
> receives
> > a
> > > > >> > metadata
> > > > >> > > > > whose
> > > > >> > > > > > > > version is lower than the current metadata version,
> we
> > > can
> > > > >> > reject
> > > > >> > > > the
> > > > >> > > > > > > > metadata but still reset the metadata age, which
> > > > essentially
> > > > >> > keep
> > > > >> > > > the
> > > > >> > > > > > > > existing behavior in the client.
> > > > >> > > > > > > >
> > > > >> > > > > > > > Thanks much,
> > > > >> > > > > > > > Dong
> > > > >> > > > > > > >
> > > > >> > > > > > >
> > > > >> > > > > >
> > > > >> > > > >
> > > > >> > > >
> > > > >> > >
> > > > >> >
> > > > >>
> > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Jason Gustafson <ja...@confluent.io>.
>
> I am not sure I understand the benefit of incrementing this epoch after
> topic deletion. At a high level, client can not differentiate between topic
> deletion and topic creation when the global epoch is incremented. Can you
> provide more specific use-case?


Say you send two metadata requests to two separate brokers. In the
responses, one of them says a certain topic exists and one says it does
not. Who is right? My suggestion is to bump the topic epoch on deletion and
include it in the metadata response when returning
UNKNOWN_TOPIC_OR_PARTITION. Then the client always knows which metadata is
more current (if not necessarily up to date). Because of this ambiguity,
Kafka clients currently have no choice but retry on unknown topic errors.
Yes, you can say it is already handled, but this gives us some better
options in the future. In the consumer, users are often asking to be
notified when they attempt to fetch from unknown topics, for example,
because it could indicate a configuration problem. We have difficulty
supporting this at the moment.

Currently when broker returns UNKNOWN_TOPIC_OR_PARTITION, it means that the
> topic is not follower or leader of this partition. Note that
> UNKNOWN_TOPIC_OR_PARTITION does not necessarily tell client whether this
> partition exists on other broker or not. UNKNOWN_TOPIC_OR_PARTITION can be
> caused either when the broker has not yet processed the latest
> LeaderAndIsrRequest, or the client is using outdated metadata.


I don't think this is right. Metadata is propagated through the
UpdateMetadata request which the controller sends to all brokers. Brokers
will return UNKNOWN_TOPIC_OR_PARTITION in a metadata response if they don't
have metadata cached for the requested topic.

There is one problem though which I think might be what you're getting at.
After a topic is deleted, the controller will leave it out of future
UpdateMetadata requests, which means the deleted epoch would not be
propagated to all brokers and we'd be stuck in the current state. Suppose
instead that when a topic is deleted, we 1) bump the topic epoch, and 2)
set an expiration time (say several hours). When the expiration time is
reached, we delete the topic metadata in zookeeper; until then, the
controller continues to propagate it as usual with a flag indicating it no
longer exists. The point of the epoch is solving edge cases around topic
deletion and recreation, so the expiration timer gives clients a window to
observe the deletion before the metadata is removed. It also ensures that
topic metadata is eventually cleaned up following deletion.

What do you think?

In comparison to byte[], String has the benefit of being more readable and
> it is also the same type of the existing metadata field, which is used for
> a similar purpose by user. Do you think this is reasonable?


I don't have too strong of a feeling about it. I'm not sure how important
readability is since it's intended to be opaque to the user. To clarify a
little bit, I think we should continue to send the topic and leader epochs
in the protocol directly as separate fields. It's only when we surface it
through the consumer API that we add some obscurity since we don't want
users to depend on the fields directly and we don't want to make API
changes in the future if we need to add something else which is also
internal. In fact, rather than using byte[] or String directly, perhaps we
could just expose it as an object and give it a readable toString()?


Thanks,
Jason


On Fri, Jan 5, 2018 at 5:12 PM, Dong Lin <li...@gmail.com> wrote:

> Hey Jason,
>
> Thanks a lot for the comments. I will comment inline. And I have updated
> the KIP accordingly. Could you take another look?
>
> On Fri, Jan 5, 2018 at 11:15 AM, Jason Gustafson <ja...@confluent.io>
> wrote:
>
> > Hi Dong,
> >
> > Sorry for the late reply. I think the latest revision is looking good. I
> > have a few minor suggestions:
> >
> > 1. The name "partition_epoch" makes me think it changes independently at
> > the partition level, but all partitions for a topic should have the same
> > epoch. Maybe "topic_epoch" is nearer the mark?
> >
>
> Actually, in the current proposal, partitions of the same topic will have
> different epoch. Every time a new partition is created, either due to topic
> creation or partition expansion, the global epoch is incremented by 1 and
> is assigned to that partition. This is probably why we currently call it
> partition_epoch.
>
> Thinking about your idea more, one alternative approach following your idea
> is to use a topic_epoch is that incremented by 1 whenever we create a
> topic. We should store a single topic_epoch in
> znode /brokers/topics/[topic] without storing the list of partition_epoch
> for all partitions. This same epoch will be used for the new partitions
> after partition expansion of the existing topic. This approach has more
> simpler znode format than the existing KIP and it still allows us to detect
> topic created after topic deletion. I think this is better. I have updated
> the KIP with this approach.
>
>
> > 2. Should we increment this epoch when a topic is deleted also? When the
> > broker returns an UNKNOWN_TOPIC_OR_PARTITION error in a metadata
> response,
> > we can also include the latest partition epoch, which would allow the
> > client to disambiguate the error if it has seen more recent metadata.
> >
>
> I am not sure I understand the benefit of incrementing this epoch after
> topic deletion. At a high level, client can not differentiate between topic
> deletion and topic creation when the global epoch is incremented. Can you
> provide more specific use-case?
>
> Currently when broker returns UNKNOWN_TOPIC_OR_PARTITION, it means that the
> topic is not follower or leader of this partition. Note that
> UNKNOWN_TOPIC_OR_PARTITION does not necessarily tell client whether this
> partition exists on other broker or not. UNKNOWN_TOPIC_OR_PARTITION can be
> caused either when the broker has not yet processed the latest
> LeaderAndIsrRequest, or the client is using outdated metadata. In either
> case, the client needs to retry and possibly refresh metadata, which is
> already done by client with the current Kafka implementation. So it seems
> that we don't have a problem to fix here?
>
>
>
> > 3. I am still wondering whether it is a good idea to expose these epochs
> in
> > the consumer API. As an alternative, have you considered representing the
> > data as an opaque blob of bytes? For example:
> >
> > class OffsetAndMetadata {
> >   long offset;
> >   byte[] offsetMetadata;
> >   String metadata;
> > }
> >
> > Admittedly, the naming is a bit annoying, but we can probably come up
> with
> > something better. Internally the byte array would have a version. If in
> the
> > future we have anything else we need to add, we can update the version
> and
> > we wouldn't need any new APIs.
> >
> > The corresponding seek() and position() APIs might look something like
> > this:
> >
> > void seek(TopicPartition partition, long offset, byte[] offsetMetadata);
> > byte[] positionMetadata(TopicPartition partition);
> >
> > What do you think?
> >
>
> I think it is a very good idea to consolidate the new information into a
> single field rather than explicitly listing them with specific types. This
> provides us the advantage of evolving the information in this field in the
> future.
>
> But I probably would prefer to use String rather than byte[] as the type of
> this new field. This string can probably have the following json format:
>
> {
>   "version": 1,
>   "topic_epoch": int,
>   "leader_epoch": int.
> }
>
> In comparison to byte[], String has the benefit of being more readable and
> it is also the same type of the existing metadata field, which is used for
> a similar purpose by user. Do you think this is reasonable?
>
>
>
> >
> > Thanks,
> > Jason
> >
> > On Thu, Jan 4, 2018 at 7:04 PM, Dong Lin <li...@gmail.com> wrote:
> >
> > > Hey Jun, Jason,
> > >
> > > Thanks much for all the feedback. I have updated the KIP based on the
> > > latest discussion. Can you help check whether it looks good?
> > >
> > > Thanks,
> > > Dong
> > >
> > > On Thu, Jan 4, 2018 at 5:36 PM, Dong Lin <li...@gmail.com> wrote:
> > >
> > > > Hey Jun,
> > > >
> > > > Hmm... thinking about this more, I am not sure that the proposed API
> is
> > > > sufficient. For users that store offset externally, we probably need
> > > extra
> > > > API to return the leader_epoch and partition_epoch for all partitions
> > > that
> > > > consumers are consuming. I suppose these users currently use
> position()
> > > to
> > > > get the offset. Thus we probably need a new method
> > positionWithEpoch(..)
> > > to
> > > > return <offset, partition_epoch, leader_epoch>. Does this sound
> > > reasonable?
> > > >
> > > > Thanks,
> > > > Dong
> > > >
> > > >
> > > > On Thu, Jan 4, 2018 at 5:26 PM, Jun Rao <ju...@confluent.io> wrote:
> > > >
> > > >> Hi, Dong,
> > > >>
> > > >> Yes, that's what I am thinking. OffsetEpoch will be composed of
> > > >> (partition_epoch,
> > > >> leader_epoch).
> > > >>
> > > >> Thanks,
> > > >>
> > > >> Jun
> > > >>
> > > >>
> > > >> On Thu, Jan 4, 2018 at 4:22 PM, Dong Lin <li...@gmail.com>
> wrote:
> > > >>
> > > >> > Hey Jun,
> > > >> >
> > > >> > Thanks much. I like the the new API that you proposed. I am not
> sure
> > > >> what
> > > >> > you exactly mean by offset_epoch. I suppose that we can use the
> pair
> > > of
> > > >> > (partition_epoch, leader_epoch) as the offset_epoch, right?
> > > >> >
> > > >> > Thanks,
> > > >> > Dong
> > > >> >
> > > >> > On Thu, Jan 4, 2018 at 4:02 PM, Jun Rao <ju...@confluent.io> wrote:
> > > >> >
> > > >> > > Hi, Dong,
> > > >> > >
> > > >> > > Got it. The api that you proposed works. The question is whether
> > > >> that's
> > > >> > the
> > > >> > > api that we want to have in the long term. My concern is that
> > while
> > > >> the
> > > >> > api
> > > >> > > change is simple, the new api seems harder to explain and use.
> For
> > > >> > example,
> > > >> > > a consumer storing offsets externally now needs to call
> > > >> > > waitForMetadataUpdate() after calling seek().
> > > >> > >
> > > >> > > An alternative approach is to make the following compatible api
> > > >> changes
> > > >> > in
> > > >> > > Consumer.
> > > >> > > * Add an additional OffsetEpoch field in OffsetAndMetadata. (no
> > need
> > > >> to
> > > >> > > change the CommitSync() api)
> > > >> > > * Add a new api seek(TopicPartition partition, long offset,
> > > >> OffsetEpoch
> > > >> > > offsetEpoch). We can potentially deprecate the old api
> > > >> > seek(TopicPartition
> > > >> > > partition, long offset) in the future.
> > > >> > >
> > > >> > > The alternative approach has similar amount of api changes as
> > yours
> > > >> but
> > > >> > has
> > > >> > > the following benefits.
> > > >> > > 1. The api works in a similar way as how offset management works
> > now
> > > >> and
> > > >> > is
> > > >> > > probably what we want in the long term.
> > > >> > > 2. It can reset offsets better when there is data loss due to
> > > unclean
> > > >> > > leader election or correlated replica failure.
> > > >> > > 3. It can reset offsets better when topic is recreated.
> > > >> > >
> > > >> > > Thanks,
> > > >> > >
> > > >> > > Jun
> > > >> > >
> > > >> > >
> > > >> > > On Thu, Jan 4, 2018 at 2:05 PM, Dong Lin <li...@gmail.com>
> > > wrote:
> > > >> > >
> > > >> > > > Hey Jun,
> > > >> > > >
> > > >> > > > Yeah I agree that ideally we don't want an ever growing global
> > > >> metadata
> > > >> > > > version. I just think it may be more desirable to keep the
> > > consumer
> > > >> API
> > > >> > > > simple.
> > > >> > > >
> > > >> > > > In my current proposal, metadata version returned in the fetch
> > > >> response
> > > >> > > > will be stored with the offset together. More specifically,
> the
> > > >> > > > metadata_epoch in the new offset topic schema will be the
> > largest
> > > >> > > > metadata_epoch from all the MetadataResponse and FetchResponse
> > > ever
> > > >> > > > received by this consumer.
> > > >> > > >
> > > >> > > > We probably don't have to change the consumer API for
> > > >> > > > commitSync(Map<TopicPartition, OffsetAndMetadata>). If user
> > calls
> > > >> > > > commitSync(...) to commit offset 10 for a given partition, for
> > > most
> > > >> > > > use-cases, this consumer instance should have consumed message
> > > with
> > > >> > > offset
> > > >> > > > 9 from this partition, in which case the consumer can remember
> > and
> > > >> use
> > > >> > > the
> > > >> > > > metadata_epoch from the corresponding FetchResponse when
> > > committing
> > > >> > > offset.
> > > >> > > > If user calls commitSync(..) to commit offset 10 for a given
> > > >> partition
> > > >> > > > without having consumed the message with offset 9 using this
> > > >> consumer
> > > >> > > > instance, this is probably an advanced use-case. In this case
> > the
> > > >> > > advanced
> > > >> > > > user can retrieve the metadata_epoch using the newly added
> > > >> > > metadataEpoch()
> > > >> > > > API after it fetches the message with offset 9 (probably from
> > > >> another
> > > >> > > > consumer instance) and encode this metadata_epoch in the
> > > >> > > > string OffsetAndMetadata.metadata. Do you think this solution
> > > would
> > > >> > work?
> > > >> > > >
> > > >> > > > By "not sure that I fully understand your latest suggestion",
> > are
> > > >> you
> > > >> > > > referring to solution related to unclean leader election using
> > > >> > > leader_epoch
> > > >> > > > in my previous email?
> > > >> > > >
> > > >> > > > Thanks,
> > > >> > > > Dong
> > > >> > > >
> > > >> > > > On Thu, Jan 4, 2018 at 1:33 PM, Jun Rao <ju...@confluent.io>
> > wrote:
> > > >> > > >
> > > >> > > > > Hi, Dong,
> > > >> > > > >
> > > >> > > > > Not sure that I fully understand your latest suggestion.
> > > >> Returning an
> > > >> > > > ever
> > > >> > > > > growing global metadata version itself is no ideal, but is
> > fine.
> > > >> My
> > > >> > > > > question is whether the metadata version returned in the
> fetch
> > > >> > response
> > > >> > > > > needs to be stored with the offset together if offsets are
> > > stored
> > > >> > > > > externally. If so, we also have to change the consumer API
> for
> > > >> > > > commitSync()
> > > >> > > > > and need to worry about compatibility. If we don't store the
> > > >> metadata
> > > >> > > > > version together with the offset, on a consumer restart,
> it's
> > > not
> > > >> > clear
> > > >> > > > how
> > > >> > > > > we can ensure the metadata in the consumer is high enough
> > since
> > > >> there
> > > >> > > is
> > > >> > > > no
> > > >> > > > > metadata version to compare with.
> > > >> > > > >
> > > >> > > > > Thanks,
> > > >> > > > >
> > > >> > > > > Jun
> > > >> > > > >
> > > >> > > > >
> > > >> > > > > On Wed, Jan 3, 2018 at 6:43 PM, Dong Lin <
> lindong28@gmail.com
> > >
> > > >> > wrote:
> > > >> > > > >
> > > >> > > > > > Hey Jun,
> > > >> > > > > >
> > > >> > > > > > Thanks much for the explanation.
> > > >> > > > > >
> > > >> > > > > > I understand the advantage of partition_epoch over
> > > >> metadata_epoch.
> > > >> > My
> > > >> > > > > > current concern is that the use of leader_epoch and the
> > > >> > > partition_epoch
> > > >> > > > > > requires us considerable change to consumer's public API
> to
> > > take
> > > >> > care
> > > >> > > > of
> > > >> > > > > > the case where user stores offset externally. For example,
> > > >> > > *consumer*.
> > > >> > > > > > *commitSync*(..) would have to take a map whose value is
> > > >> <offset,
> > > >> > > > > metadata,
> > > >> > > > > > leader epoch, partition epoch>. *consumer*.*seek*(...)
> would
> > > >> also
> > > >> > > need
> > > >> > > > > > leader_epoch and partition_epoch as parameter. Technically
> > we
> > > >> can
> > > >> > > > > probably
> > > >> > > > > > still make it work in a backward compatible manner after
> > > careful
> > > >> > > design
> > > >> > > > > and
> > > >> > > > > > discussion. But these changes can make the consumer's
> > > interface
> > > >> > > > > > unnecessarily complex for more users who do not store
> offset
> > > >> > > > externally.
> > > >> > > > > >
> > > >> > > > > > After thinking more about it, we can address all problems
> > > >> discussed
> > > >> > > by
> > > >> > > > > only
> > > >> > > > > > using the metadata_epoch without introducing leader_epoch
> or
> > > the
> > > >> > > > > > partition_epoch. The current KIP describes the changes to
> > the
> > > >> > > consumer
> > > >> > > > > API
> > > >> > > > > > and how the new API can be used if user stores offset
> > > >> externally.
> > > >> > In
> > > >> > > > > order
> > > >> > > > > > to address the scenario you described earlier, we can
> > include
> > > >> > > > > > metadata_epoch in the FetchResponse and the
> > > LeaderAndIsrRequest.
> > > >> > > > Consumer
> > > >> > > > > > remembers the largest metadata_epoch from all the
> > > FetchResponse
> > > >> it
> > > >> > > has
> > > >> > > > > > received. The metadata_epoch committed with the offset,
> > either
> > > >> > within
> > > >> > > > or
> > > >> > > > > > outside Kafka, should be the largest metadata_epoch across
> > all
> > > >> > > > > > FetchResponse and MetadataResponse ever received by this
> > > >> consumer.
> > > >> > > > > >
> > > >> > > > > > The drawback of using only the metadata_epoch is that we
> can
> > > not
> > > >> > > always
> > > >> > > > > do
> > > >> > > > > > the smart offset reset in case of unclean leader election
> > > which
> > > >> you
> > > >> > > > > > mentioned earlier. But in most case, unclean leader
> election
> > > >> > probably
> > > >> > > > > > happens when consumer is not rebalancing/restarting. In
> > these
> > > >> > cases,
> > > >> > > > > either
> > > >> > > > > > consumer is not directly affected by unclean leader
> election
> > > >> since
> > > >> > it
> > > >> > > > is
> > > >> > > > > > not consuming from the end of the log, or consumer can
> > derive
> > > >> the
> > > >> > > > > > leader_epoch from the most recent message received before
> it
> > > >> sees
> > > >> > > > > > OffsetOutOfRangeException. So I am not sure it is worth
> > adding
> > > >> the
> > > >> > > > > > leader_epoch to consumer API to address the remaining
> corner
> > > >> case.
> > > >> > > What
> > > >> > > > > do
> > > >> > > > > > you think?
> > > >> > > > > >
> > > >> > > > > > Thanks,
> > > >> > > > > > Dong
> > > >> > > > > >
> > > >> > > > > >
> > > >> > > > > >
> > > >> > > > > > On Tue, Jan 2, 2018 at 6:28 PM, Jun Rao <jun@confluent.io
> >
> > > >> wrote:
> > > >> > > > > >
> > > >> > > > > > > Hi, Dong,
> > > >> > > > > > >
> > > >> > > > > > > Thanks for the reply.
> > > >> > > > > > >
> > > >> > > > > > > To solve the topic recreation issue, we could use
> either a
> > > >> global
> > > >> > > > > > metadata
> > > >> > > > > > > version or a partition level epoch. But either one will
> > be a
> > > >> new
> > > >> > > > > concept,
> > > >> > > > > > > right? To me, the latter seems more natural. It also
> makes
> > > it
> > > >> > > easier
> > > >> > > > to
> > > >> > > > > > > detect if a consumer's offset is still valid after a
> topic
> > > is
> > > >> > > > > recreated.
> > > >> > > > > > As
> > > >> > > > > > > you pointed out, we don't need to store the partition
> > epoch
> > > in
> > > >> > the
> > > >> > > > > > message.
> > > >> > > > > > > The following is what I am thinking. When a partition is
> > > >> created,
> > > >> > > we
> > > >> > > > > can
> > > >> > > > > > > assign a partition epoch from an ever-increasing global
> > > >> counter
> > > >> > and
> > > >> > > > > store
> > > >> > > > > > > it in /brokers/topics/[topic]/partitions/[partitionId]
> in
> > > ZK.
> > > >> > The
> > > >> > > > > > > partition
> > > >> > > > > > > epoch is propagated to every broker. The consumer will
> be
> > > >> > tracking
> > > >> > > a
> > > >> > > > > > tuple
> > > >> > > > > > > of <offset, leader epoch, partition epoch> for offsets.
> > If a
> > > >> > topic
> > > >> > > is
> > > >> > > > > > > recreated, it's possible that a consumer's offset and
> > leader
> > > >> > epoch
> > > >> > > > > still
> > > >> > > > > > > match that in the broker, but partition epoch won't be.
> In
> > > >> this
> > > >> > > case,
> > > >> > > > > we
> > > >> > > > > > > can potentially still treat the consumer's offset as out
> > of
> > > >> range
> > > >> > > and
> > > >> > > > > > reset
> > > >> > > > > > > the offset based on the offset reset policy in the
> > consumer.
> > > >> This
> > > >> > > > seems
> > > >> > > > > > > harder to do with a global metadata version.
> > > >> > > > > > >
> > > >> > > > > > > Jun
> > > >> > > > > > >
> > > >> > > > > > >
> > > >> > > > > > >
> > > >> > > > > > > On Mon, Dec 25, 2017 at 6:56 AM, Dong Lin <
> > > >> lindong28@gmail.com>
> > > >> > > > wrote:
> > > >> > > > > > >
> > > >> > > > > > > > Hey Jun,
> > > >> > > > > > > >
> > > >> > > > > > > > This is a very good example. After thinking through
> this
> > > in
> > > >> > > > detail, I
> > > >> > > > > > > agree
> > > >> > > > > > > > that we need to commit offset with leader epoch in
> order
> > > to
> > > >> > > address
> > > >> > > > > > this
> > > >> > > > > > > > example.
> > > >> > > > > > > >
> > > >> > > > > > > > I think the remaining question is how to address the
> > > >> scenario
> > > >> > > that
> > > >> > > > > the
> > > >> > > > > > > > topic is deleted and re-created. One possible solution
> > is
> > > to
> > > >> > > commit
> > > >> > > > > > > offset
> > > >> > > > > > > > with both the leader epoch and the metadata version.
> The
> > > >> logic
> > > >> > > and
> > > >> > > > > the
> > > >> > > > > > > > implementation of this solution does not require a new
> > > >> concept
> > > >> > > > (e.g.
> > > >> > > > > > > > partition epoch) and it does not require any change to
> > the
> > > >> > > message
> > > >> > > > > > format
> > > >> > > > > > > > or leader epoch. It also allows us to order the
> metadata
> > > in
> > > >> a
> > > >> > > > > > > > straightforward manner which may be useful in the
> > future.
> > > >> So it
> > > >> > > may
> > > >> > > > > be
> > > >> > > > > > a
> > > >> > > > > > > > better solution than generating a random partition
> epoch
> > > >> every
> > > >> > > time
> > > >> > > > > we
> > > >> > > > > > > > create a partition. Does this sound reasonable?
> > > >> > > > > > > >
> > > >> > > > > > > > Previously one concern with using the metadata version
> > is
> > > >> that
> > > >> > > > > consumer
> > > >> > > > > > > > will be forced to refresh metadata even if metadata
> > > version
> > > >> is
> > > >> > > > > > increased
> > > >> > > > > > > > due to topics that the consumer is not interested in.
> > Now
> > > I
> > > >> > > > realized
> > > >> > > > > > that
> > > >> > > > > > > > this is probably not a problem. Currently client will
> > > >> refresh
> > > >> > > > > metadata
> > > >> > > > > > > > either due to InvalidMetadataException in the response
> > > from
> > > >> > > broker
> > > >> > > > or
> > > >> > > > > > due
> > > >> > > > > > > > to metadata expiry. The addition of the metadata
> version
> > > >> should
> > > >> > > > > > increase
> > > >> > > > > > > > the overhead of metadata refresh caused by
> > > >> > > > InvalidMetadataException.
> > > >> > > > > If
> > > >> > > > > > > > client refresh metadata due to expiry and it receives
> a
> > > >> > metadata
> > > >> > > > > whose
> > > >> > > > > > > > version is lower than the current metadata version, we
> > can
> > > >> > reject
> > > >> > > > the
> > > >> > > > > > > > metadata but still reset the metadata age, which
> > > essentially
> > > >> > keep
> > > >> > > > the
> > > >> > > > > > > > existing behavior in the client.
> > > >> > > > > > > >
> > > >> > > > > > > > Thanks much,
> > > >> > > > > > > > Dong
> > > >> > > > > > > >
> > > >> > > > > > >
> > > >> > > > > >
> > > >> > > > >
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Dong Lin <li...@gmail.com>.
Hey Jason,

Thanks a lot for the comments. I will comment inline. And I have updated
the KIP accordingly. Could you take another look?

On Fri, Jan 5, 2018 at 11:15 AM, Jason Gustafson <ja...@confluent.io> wrote:

> Hi Dong,
>
> Sorry for the late reply. I think the latest revision is looking good. I
> have a few minor suggestions:
>
> 1. The name "partition_epoch" makes me think it changes independently at
> the partition level, but all partitions for a topic should have the same
> epoch. Maybe "topic_epoch" is nearer the mark?
>

Actually, in the current proposal, partitions of the same topic will have
different epoch. Every time a new partition is created, either due to topic
creation or partition expansion, the global epoch is incremented by 1 and
is assigned to that partition. This is probably why we currently call it
partition_epoch.

Thinking about your idea more, one alternative approach following your idea
is to use a topic_epoch is that incremented by 1 whenever we create a
topic. We should store a single topic_epoch in
znode /brokers/topics/[topic] without storing the list of partition_epoch
for all partitions. This same epoch will be used for the new partitions
after partition expansion of the existing topic. This approach has more
simpler znode format than the existing KIP and it still allows us to detect
topic created after topic deletion. I think this is better. I have updated
the KIP with this approach.


> 2. Should we increment this epoch when a topic is deleted also? When the
> broker returns an UNKNOWN_TOPIC_OR_PARTITION error in a metadata response,
> we can also include the latest partition epoch, which would allow the
> client to disambiguate the error if it has seen more recent metadata.
>

I am not sure I understand the benefit of incrementing this epoch after
topic deletion. At a high level, client can not differentiate between topic
deletion and topic creation when the global epoch is incremented. Can you
provide more specific use-case?

Currently when broker returns UNKNOWN_TOPIC_OR_PARTITION, it means that the
topic is not follower or leader of this partition. Note that
UNKNOWN_TOPIC_OR_PARTITION does not necessarily tell client whether this
partition exists on other broker or not. UNKNOWN_TOPIC_OR_PARTITION can be
caused either when the broker has not yet processed the latest
LeaderAndIsrRequest, or the client is using outdated metadata. In either
case, the client needs to retry and possibly refresh metadata, which is
already done by client with the current Kafka implementation. So it seems
that we don't have a problem to fix here?



> 3. I am still wondering whether it is a good idea to expose these epochs in
> the consumer API. As an alternative, have you considered representing the
> data as an opaque blob of bytes? For example:
>
> class OffsetAndMetadata {
>   long offset;
>   byte[] offsetMetadata;
>   String metadata;
> }
>
> Admittedly, the naming is a bit annoying, but we can probably come up with
> something better. Internally the byte array would have a version. If in the
> future we have anything else we need to add, we can update the version and
> we wouldn't need any new APIs.
>
> The corresponding seek() and position() APIs might look something like
> this:
>
> void seek(TopicPartition partition, long offset, byte[] offsetMetadata);
> byte[] positionMetadata(TopicPartition partition);
>
> What do you think?
>

I think it is a very good idea to consolidate the new information into a
single field rather than explicitly listing them with specific types. This
provides us the advantage of evolving the information in this field in the
future.

But I probably would prefer to use String rather than byte[] as the type of
this new field. This string can probably have the following json format:

{
  "version": 1,
  "topic_epoch": int,
  "leader_epoch": int.
}

In comparison to byte[], String has the benefit of being more readable and
it is also the same type of the existing metadata field, which is used for
a similar purpose by user. Do you think this is reasonable?



>
> Thanks,
> Jason
>
> On Thu, Jan 4, 2018 at 7:04 PM, Dong Lin <li...@gmail.com> wrote:
>
> > Hey Jun, Jason,
> >
> > Thanks much for all the feedback. I have updated the KIP based on the
> > latest discussion. Can you help check whether it looks good?
> >
> > Thanks,
> > Dong
> >
> > On Thu, Jan 4, 2018 at 5:36 PM, Dong Lin <li...@gmail.com> wrote:
> >
> > > Hey Jun,
> > >
> > > Hmm... thinking about this more, I am not sure that the proposed API is
> > > sufficient. For users that store offset externally, we probably need
> > extra
> > > API to return the leader_epoch and partition_epoch for all partitions
> > that
> > > consumers are consuming. I suppose these users currently use position()
> > to
> > > get the offset. Thus we probably need a new method
> positionWithEpoch(..)
> > to
> > > return <offset, partition_epoch, leader_epoch>. Does this sound
> > reasonable?
> > >
> > > Thanks,
> > > Dong
> > >
> > >
> > > On Thu, Jan 4, 2018 at 5:26 PM, Jun Rao <ju...@confluent.io> wrote:
> > >
> > >> Hi, Dong,
> > >>
> > >> Yes, that's what I am thinking. OffsetEpoch will be composed of
> > >> (partition_epoch,
> > >> leader_epoch).
> > >>
> > >> Thanks,
> > >>
> > >> Jun
> > >>
> > >>
> > >> On Thu, Jan 4, 2018 at 4:22 PM, Dong Lin <li...@gmail.com> wrote:
> > >>
> > >> > Hey Jun,
> > >> >
> > >> > Thanks much. I like the the new API that you proposed. I am not sure
> > >> what
> > >> > you exactly mean by offset_epoch. I suppose that we can use the pair
> > of
> > >> > (partition_epoch, leader_epoch) as the offset_epoch, right?
> > >> >
> > >> > Thanks,
> > >> > Dong
> > >> >
> > >> > On Thu, Jan 4, 2018 at 4:02 PM, Jun Rao <ju...@confluent.io> wrote:
> > >> >
> > >> > > Hi, Dong,
> > >> > >
> > >> > > Got it. The api that you proposed works. The question is whether
> > >> that's
> > >> > the
> > >> > > api that we want to have in the long term. My concern is that
> while
> > >> the
> > >> > api
> > >> > > change is simple, the new api seems harder to explain and use. For
> > >> > example,
> > >> > > a consumer storing offsets externally now needs to call
> > >> > > waitForMetadataUpdate() after calling seek().
> > >> > >
> > >> > > An alternative approach is to make the following compatible api
> > >> changes
> > >> > in
> > >> > > Consumer.
> > >> > > * Add an additional OffsetEpoch field in OffsetAndMetadata. (no
> need
> > >> to
> > >> > > change the CommitSync() api)
> > >> > > * Add a new api seek(TopicPartition partition, long offset,
> > >> OffsetEpoch
> > >> > > offsetEpoch). We can potentially deprecate the old api
> > >> > seek(TopicPartition
> > >> > > partition, long offset) in the future.
> > >> > >
> > >> > > The alternative approach has similar amount of api changes as
> yours
> > >> but
> > >> > has
> > >> > > the following benefits.
> > >> > > 1. The api works in a similar way as how offset management works
> now
> > >> and
> > >> > is
> > >> > > probably what we want in the long term.
> > >> > > 2. It can reset offsets better when there is data loss due to
> > unclean
> > >> > > leader election or correlated replica failure.
> > >> > > 3. It can reset offsets better when topic is recreated.
> > >> > >
> > >> > > Thanks,
> > >> > >
> > >> > > Jun
> > >> > >
> > >> > >
> > >> > > On Thu, Jan 4, 2018 at 2:05 PM, Dong Lin <li...@gmail.com>
> > wrote:
> > >> > >
> > >> > > > Hey Jun,
> > >> > > >
> > >> > > > Yeah I agree that ideally we don't want an ever growing global
> > >> metadata
> > >> > > > version. I just think it may be more desirable to keep the
> > consumer
> > >> API
> > >> > > > simple.
> > >> > > >
> > >> > > > In my current proposal, metadata version returned in the fetch
> > >> response
> > >> > > > will be stored with the offset together. More specifically, the
> > >> > > > metadata_epoch in the new offset topic schema will be the
> largest
> > >> > > > metadata_epoch from all the MetadataResponse and FetchResponse
> > ever
> > >> > > > received by this consumer.
> > >> > > >
> > >> > > > We probably don't have to change the consumer API for
> > >> > > > commitSync(Map<TopicPartition, OffsetAndMetadata>). If user
> calls
> > >> > > > commitSync(...) to commit offset 10 for a given partition, for
> > most
> > >> > > > use-cases, this consumer instance should have consumed message
> > with
> > >> > > offset
> > >> > > > 9 from this partition, in which case the consumer can remember
> and
> > >> use
> > >> > > the
> > >> > > > metadata_epoch from the corresponding FetchResponse when
> > committing
> > >> > > offset.
> > >> > > > If user calls commitSync(..) to commit offset 10 for a given
> > >> partition
> > >> > > > without having consumed the message with offset 9 using this
> > >> consumer
> > >> > > > instance, this is probably an advanced use-case. In this case
> the
> > >> > > advanced
> > >> > > > user can retrieve the metadata_epoch using the newly added
> > >> > > metadataEpoch()
> > >> > > > API after it fetches the message with offset 9 (probably from
> > >> another
> > >> > > > consumer instance) and encode this metadata_epoch in the
> > >> > > > string OffsetAndMetadata.metadata. Do you think this solution
> > would
> > >> > work?
> > >> > > >
> > >> > > > By "not sure that I fully understand your latest suggestion",
> are
> > >> you
> > >> > > > referring to solution related to unclean leader election using
> > >> > > leader_epoch
> > >> > > > in my previous email?
> > >> > > >
> > >> > > > Thanks,
> > >> > > > Dong
> > >> > > >
> > >> > > > On Thu, Jan 4, 2018 at 1:33 PM, Jun Rao <ju...@confluent.io>
> wrote:
> > >> > > >
> > >> > > > > Hi, Dong,
> > >> > > > >
> > >> > > > > Not sure that I fully understand your latest suggestion.
> > >> Returning an
> > >> > > > ever
> > >> > > > > growing global metadata version itself is no ideal, but is
> fine.
> > >> My
> > >> > > > > question is whether the metadata version returned in the fetch
> > >> > response
> > >> > > > > needs to be stored with the offset together if offsets are
> > stored
> > >> > > > > externally. If so, we also have to change the consumer API for
> > >> > > > commitSync()
> > >> > > > > and need to worry about compatibility. If we don't store the
> > >> metadata
> > >> > > > > version together with the offset, on a consumer restart, it's
> > not
> > >> > clear
> > >> > > > how
> > >> > > > > we can ensure the metadata in the consumer is high enough
> since
> > >> there
> > >> > > is
> > >> > > > no
> > >> > > > > metadata version to compare with.
> > >> > > > >
> > >> > > > > Thanks,
> > >> > > > >
> > >> > > > > Jun
> > >> > > > >
> > >> > > > >
> > >> > > > > On Wed, Jan 3, 2018 at 6:43 PM, Dong Lin <lindong28@gmail.com
> >
> > >> > wrote:
> > >> > > > >
> > >> > > > > > Hey Jun,
> > >> > > > > >
> > >> > > > > > Thanks much for the explanation.
> > >> > > > > >
> > >> > > > > > I understand the advantage of partition_epoch over
> > >> metadata_epoch.
> > >> > My
> > >> > > > > > current concern is that the use of leader_epoch and the
> > >> > > partition_epoch
> > >> > > > > > requires us considerable change to consumer's public API to
> > take
> > >> > care
> > >> > > > of
> > >> > > > > > the case where user stores offset externally. For example,
> > >> > > *consumer*.
> > >> > > > > > *commitSync*(..) would have to take a map whose value is
> > >> <offset,
> > >> > > > > metadata,
> > >> > > > > > leader epoch, partition epoch>. *consumer*.*seek*(...) would
> > >> also
> > >> > > need
> > >> > > > > > leader_epoch and partition_epoch as parameter. Technically
> we
> > >> can
> > >> > > > > probably
> > >> > > > > > still make it work in a backward compatible manner after
> > careful
> > >> > > design
> > >> > > > > and
> > >> > > > > > discussion. But these changes can make the consumer's
> > interface
> > >> > > > > > unnecessarily complex for more users who do not store offset
> > >> > > > externally.
> > >> > > > > >
> > >> > > > > > After thinking more about it, we can address all problems
> > >> discussed
> > >> > > by
> > >> > > > > only
> > >> > > > > > using the metadata_epoch without introducing leader_epoch or
> > the
> > >> > > > > > partition_epoch. The current KIP describes the changes to
> the
> > >> > > consumer
> > >> > > > > API
> > >> > > > > > and how the new API can be used if user stores offset
> > >> externally.
> > >> > In
> > >> > > > > order
> > >> > > > > > to address the scenario you described earlier, we can
> include
> > >> > > > > > metadata_epoch in the FetchResponse and the
> > LeaderAndIsrRequest.
> > >> > > > Consumer
> > >> > > > > > remembers the largest metadata_epoch from all the
> > FetchResponse
> > >> it
> > >> > > has
> > >> > > > > > received. The metadata_epoch committed with the offset,
> either
> > >> > within
> > >> > > > or
> > >> > > > > > outside Kafka, should be the largest metadata_epoch across
> all
> > >> > > > > > FetchResponse and MetadataResponse ever received by this
> > >> consumer.
> > >> > > > > >
> > >> > > > > > The drawback of using only the metadata_epoch is that we can
> > not
> > >> > > always
> > >> > > > > do
> > >> > > > > > the smart offset reset in case of unclean leader election
> > which
> > >> you
> > >> > > > > > mentioned earlier. But in most case, unclean leader election
> > >> > probably
> > >> > > > > > happens when consumer is not rebalancing/restarting. In
> these
> > >> > cases,
> > >> > > > > either
> > >> > > > > > consumer is not directly affected by unclean leader election
> > >> since
> > >> > it
> > >> > > > is
> > >> > > > > > not consuming from the end of the log, or consumer can
> derive
> > >> the
> > >> > > > > > leader_epoch from the most recent message received before it
> > >> sees
> > >> > > > > > OffsetOutOfRangeException. So I am not sure it is worth
> adding
> > >> the
> > >> > > > > > leader_epoch to consumer API to address the remaining corner
> > >> case.
> > >> > > What
> > >> > > > > do
> > >> > > > > > you think?
> > >> > > > > >
> > >> > > > > > Thanks,
> > >> > > > > > Dong
> > >> > > > > >
> > >> > > > > >
> > >> > > > > >
> > >> > > > > > On Tue, Jan 2, 2018 at 6:28 PM, Jun Rao <ju...@confluent.io>
> > >> wrote:
> > >> > > > > >
> > >> > > > > > > Hi, Dong,
> > >> > > > > > >
> > >> > > > > > > Thanks for the reply.
> > >> > > > > > >
> > >> > > > > > > To solve the topic recreation issue, we could use either a
> > >> global
> > >> > > > > > metadata
> > >> > > > > > > version or a partition level epoch. But either one will
> be a
> > >> new
> > >> > > > > concept,
> > >> > > > > > > right? To me, the latter seems more natural. It also makes
> > it
> > >> > > easier
> > >> > > > to
> > >> > > > > > > detect if a consumer's offset is still valid after a topic
> > is
> > >> > > > > recreated.
> > >> > > > > > As
> > >> > > > > > > you pointed out, we don't need to store the partition
> epoch
> > in
> > >> > the
> > >> > > > > > message.
> > >> > > > > > > The following is what I am thinking. When a partition is
> > >> created,
> > >> > > we
> > >> > > > > can
> > >> > > > > > > assign a partition epoch from an ever-increasing global
> > >> counter
> > >> > and
> > >> > > > > store
> > >> > > > > > > it in /brokers/topics/[topic]/partitions/[partitionId] in
> > ZK.
> > >> > The
> > >> > > > > > > partition
> > >> > > > > > > epoch is propagated to every broker. The consumer will be
> > >> > tracking
> > >> > > a
> > >> > > > > > tuple
> > >> > > > > > > of <offset, leader epoch, partition epoch> for offsets.
> If a
> > >> > topic
> > >> > > is
> > >> > > > > > > recreated, it's possible that a consumer's offset and
> leader
> > >> > epoch
> > >> > > > > still
> > >> > > > > > > match that in the broker, but partition epoch won't be. In
> > >> this
> > >> > > case,
> > >> > > > > we
> > >> > > > > > > can potentially still treat the consumer's offset as out
> of
> > >> range
> > >> > > and
> > >> > > > > > reset
> > >> > > > > > > the offset based on the offset reset policy in the
> consumer.
> > >> This
> > >> > > > seems
> > >> > > > > > > harder to do with a global metadata version.
> > >> > > > > > >
> > >> > > > > > > Jun
> > >> > > > > > >
> > >> > > > > > >
> > >> > > > > > >
> > >> > > > > > > On Mon, Dec 25, 2017 at 6:56 AM, Dong Lin <
> > >> lindong28@gmail.com>
> > >> > > > wrote:
> > >> > > > > > >
> > >> > > > > > > > Hey Jun,
> > >> > > > > > > >
> > >> > > > > > > > This is a very good example. After thinking through this
> > in
> > >> > > > detail, I
> > >> > > > > > > agree
> > >> > > > > > > > that we need to commit offset with leader epoch in order
> > to
> > >> > > address
> > >> > > > > > this
> > >> > > > > > > > example.
> > >> > > > > > > >
> > >> > > > > > > > I think the remaining question is how to address the
> > >> scenario
> > >> > > that
> > >> > > > > the
> > >> > > > > > > > topic is deleted and re-created. One possible solution
> is
> > to
> > >> > > commit
> > >> > > > > > > offset
> > >> > > > > > > > with both the leader epoch and the metadata version. The
> > >> logic
> > >> > > and
> > >> > > > > the
> > >> > > > > > > > implementation of this solution does not require a new
> > >> concept
> > >> > > > (e.g.
> > >> > > > > > > > partition epoch) and it does not require any change to
> the
> > >> > > message
> > >> > > > > > format
> > >> > > > > > > > or leader epoch. It also allows us to order the metadata
> > in
> > >> a
> > >> > > > > > > > straightforward manner which may be useful in the
> future.
> > >> So it
> > >> > > may
> > >> > > > > be
> > >> > > > > > a
> > >> > > > > > > > better solution than generating a random partition epoch
> > >> every
> > >> > > time
> > >> > > > > we
> > >> > > > > > > > create a partition. Does this sound reasonable?
> > >> > > > > > > >
> > >> > > > > > > > Previously one concern with using the metadata version
> is
> > >> that
> > >> > > > > consumer
> > >> > > > > > > > will be forced to refresh metadata even if metadata
> > version
> > >> is
> > >> > > > > > increased
> > >> > > > > > > > due to topics that the consumer is not interested in.
> Now
> > I
> > >> > > > realized
> > >> > > > > > that
> > >> > > > > > > > this is probably not a problem. Currently client will
> > >> refresh
> > >> > > > > metadata
> > >> > > > > > > > either due to InvalidMetadataException in the response
> > from
> > >> > > broker
> > >> > > > or
> > >> > > > > > due
> > >> > > > > > > > to metadata expiry. The addition of the metadata version
> > >> should
> > >> > > > > > increase
> > >> > > > > > > > the overhead of metadata refresh caused by
> > >> > > > InvalidMetadataException.
> > >> > > > > If
> > >> > > > > > > > client refresh metadata due to expiry and it receives a
> > >> > metadata
> > >> > > > > whose
> > >> > > > > > > > version is lower than the current metadata version, we
> can
> > >> > reject
> > >> > > > the
> > >> > > > > > > > metadata but still reset the metadata age, which
> > essentially
> > >> > keep
> > >> > > > the
> > >> > > > > > > > existing behavior in the client.
> > >> > > > > > > >
> > >> > > > > > > > Thanks much,
> > >> > > > > > > > Dong
> > >> > > > > > > >
> > >> > > > > > >
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Dong Lin <li...@gmail.com>.
For record purpose, this KIP is closed as its design has been merged into
KIP-320. See
https://cwiki.apache.org/confluence/display/KAFKA/KIP-320%3A+Allow+fetchers+to+detect+and+handle+log+truncation
.

On Wed, Jan 31, 2018 at 12:16 AM Dong Lin <li...@gmail.com> wrote:

> Hey Jun, Jason,
>
> Thanks for all the comments. Could you see if you can give +1 for the KIP?
> I am open to make further improvements for the KIP.
>
> Thanks,
> Dong
>
> On Tue, Jan 23, 2018 at 3:44 PM, Dong Lin <li...@gmail.com> wrote:
>
>> Hey Jun, Jason,
>>
>> Thanks much for all the review! I will open the voting thread.
>>
>> Regards,
>> Dong
>>
>> On Tue, Jan 23, 2018 at 3:37 PM, Jun Rao <ju...@confluent.io> wrote:
>>
>>> Hi, Dong,
>>>
>>> The current KIP looks good to me.
>>>
>>> Thanks,
>>>
>>> Jun
>>>
>>> On Tue, Jan 23, 2018 at 12:29 PM, Dong Lin <li...@gmail.com> wrote:
>>>
>>> > Hey Jun,
>>> >
>>> > Do you think the current KIP looks OK? I am wondering if we can open
>>> the
>>> > voting thread.
>>> >
>>> > Thanks!
>>> > Dong
>>> >
>>> > On Fri, Jan 19, 2018 at 3:08 PM, Dong Lin <li...@gmail.com> wrote:
>>> >
>>> > > Hey Jun,
>>> > >
>>> > > I think we can probably have a static method in Util class to decode
>>> the
>>> > > byte[]. Both KafkaConsumer implementation and the user application
>>> will
>>> > be
>>> > > able to decode the byte array and log its content for debug purpose.
>>> So
>>> > it
>>> > > seems that we can still print the information we want. It is just not
>>> > > explicitly exposed in the consumer interface. Would this address the
>>> > > problem here?
>>> > >
>>> > > Yeah we can include OffsetEpoch in AdminClient. This can be added in
>>> > > KIP-222? Is there something you would like me to add in this KIP?
>>> > >
>>> > > Thanks!
>>> > > Dong
>>> > >
>>> > > On Fri, Jan 19, 2018 at 3:00 PM, Jun Rao <ju...@confluent.io> wrote:
>>> > >
>>> > >> Hi, Dong,
>>> > >>
>>> > >> The issue with using just byte[] for OffsetEpoch is that it won't be
>>> > >> printable, which makes debugging harder.
>>> > >>
>>> > >> Also, KIP-222 proposes a listGroupOffset() method in AdminClient. If
>>> > that
>>> > >> gets adopted before this KIP, we probably want to include
>>> OffsetEpoch in
>>> > >> the AdminClient too.
>>> > >>
>>> > >> Thanks,
>>> > >>
>>> > >> Jun
>>> > >>
>>> > >>
>>> > >> On Thu, Jan 18, 2018 at 6:30 PM, Dong Lin <li...@gmail.com>
>>> wrote:
>>> > >>
>>> > >> > Hey Jun,
>>> > >> >
>>> > >> > I agree. I have updated the KIP to remove the class OffetEpoch and
>>> > >> replace
>>> > >> > OffsetEpoch with byte[] in APIs that use it. Can you see if it
>>> looks
>>> > >> good?
>>> > >> >
>>> > >> > Thanks!
>>> > >> > Dong
>>> > >> >
>>> > >> > On Thu, Jan 18, 2018 at 6:07 PM, Jun Rao <ju...@confluent.io>
>>> wrote:
>>> > >> >
>>> > >> > > Hi, Dong,
>>> > >> > >
>>> > >> > > Thanks for the updated KIP. It looks good to me now. The only
>>> thing
>>> > is
>>> > >> > > for OffsetEpoch.
>>> > >> > > If we expose the individual fields in the class, we probably
>>> don't
>>> > >> need
>>> > >> > the
>>> > >> > > encode/decode methods. If we want to hide the details of
>>> > OffsetEpoch,
>>> > >> we
>>> > >> > > probably don't want expose the individual fields.
>>> > >> > >
>>> > >> > > Jun
>>> > >> > >
>>> > >> > > On Wed, Jan 17, 2018 at 10:10 AM, Dong Lin <lindong28@gmail.com
>>> >
>>> > >> wrote:
>>> > >> > >
>>> > >> > > > Thinking about point 61 more, I realize that the async
>>> zookeeper
>>> > >> read
>>> > >> > may
>>> > >> > > > make it less of an issue for controller to read more zookeeper
>>> > >> nodes.
>>> > >> > > > Writing partition_epoch in the per-partition znode makes it
>>> > simpler
>>> > >> to
>>> > >> > > > handle the broker failure between zookeeper writes for a topic
>>> > >> > creation.
>>> > >> > > I
>>> > >> > > > have updated the KIP to use the suggested approach.
>>> > >> > > >
>>> > >> > > >
>>> > >> > > > On Wed, Jan 17, 2018 at 9:57 AM, Dong Lin <
>>> lindong28@gmail.com>
>>> > >> wrote:
>>> > >> > > >
>>> > >> > > > > Hey Jun,
>>> > >> > > > >
>>> > >> > > > > Thanks much for the comments. Please see my comments inline.
>>> > >> > > > >
>>> > >> > > > > On Tue, Jan 16, 2018 at 4:38 PM, Jun Rao <ju...@confluent.io>
>>> > >> wrote:
>>> > >> > > > >
>>> > >> > > > >> Hi, Dong,
>>> > >> > > > >>
>>> > >> > > > >> Thanks for the updated KIP. Looks good to me overall. Just
>>> a
>>> > few
>>> > >> > minor
>>> > >> > > > >> comments.
>>> > >> > > > >>
>>> > >> > > > >> 60. OffsetAndMetadata positionAndOffsetEpoch(TopicPartition
>>> > >> > > partition):
>>> > >> > > > >> It
>>> > >> > > > >> seems that there is no need to return metadata. We probably
>>> > want
>>> > >> to
>>> > >> > > > return
>>> > >> > > > >> sth like OffsetAndEpoch.
>>> > >> > > > >>
>>> > >> > > > >
>>> > >> > > > > Previously I think we may want to re-use the existing class
>>> to
>>> > >> keep
>>> > >> > our
>>> > >> > > > > consumer interface simpler. I have updated the KIP to add
>>> class
>>> > >> > > > > OffsetAndOffsetEpoch. I didn't use OffsetAndEpoch because
>>> user
>>> > may
>>> > >> > > > confuse
>>> > >> > > > > this name with OffsetEpoch. Does this sound OK?
>>> > >> > > > >
>>> > >> > > > >
>>> > >> > > > >>
>>> > >> > > > >> 61. Should we store partition_epoch in
>>> > >> > > > >> /brokers/topics/[topic]/partitions/[partitionId] in ZK?
>>> > >> > > > >>
>>> > >> > > > >
>>> > >> > > > > I have considered this. I think the advantage of adding the
>>> > >> > > > > partition->partition_epoch map in the existing
>>> > >> > > > > znode /brokers/topics/[topic]/partitions is that controller
>>> > only
>>> > >> > needs
>>> > >> > > > to
>>> > >> > > > > read one znode per topic to gets its partition_epoch
>>> > information.
>>> > >> > > > Otherwise
>>> > >> > > > > controller may need to read one extra znode per partition
>>> to get
>>> > >> the
>>> > >> > > same
>>> > >> > > > > information.
>>> > >> > > > >
>>> > >> > > > > When we delete partition or expand partition of a topic,
>>> someone
>>> > >> > needs
>>> > >> > > to
>>> > >> > > > > modify partition->partition_epoch map in znode
>>> > >> > > > > /brokers/topics/[topic]/partitions. This may seem a bit more
>>> > >> > > complicated
>>> > >> > > > > than simply adding or deleting znode
>>> /brokers/topics/[topic]/
>>> > >> > > > partitions/[partitionId].
>>> > >> > > > > But the complexity is probably similar to the existing
>>> operation
>>> > >> of
>>> > >> > > > > modifying the partition->replica_list mapping in znode
>>> > >> > > > > /brokers/topics/[topic]. So not sure it is better to store
>>> the
>>> > >> > > > > partition_epoch in /brokers/topics/[topic]/partit
>>> > >> ions/[partitionId].
>>> > >> > > > What
>>> > >> > > > > do you think?
>>> > >> > > > >
>>> > >> > > > >
>>> > >> > > > >>
>>> > >> > > > >> 62. For checking outdated metadata in the client, we
>>> probably
>>> > >> want
>>> > >> > to
>>> > >> > > > add
>>> > >> > > > >> when max_partition_epoch will be used.
>>> > >> > > > >>
>>> > >> > > > >
>>> > >> > > > > The max_partition_epoch is used in the Proposed Changes ->
>>> > >> Client's
>>> > >> > > > > metadata refresh section to determine whether a metadata is
>>> > >> outdated.
>>> > >> > > And
>>> > >> > > > > this formula is referenced and re-used in other sections to
>>> > >> determine
>>> > >> > > > > whether a metadata is outdated. Does this formula look OK?
>>> > >> > > > >
>>> > >> > > > >
>>> > >> > > > >>
>>> > >> > > > >> 63. "The leader_epoch should be the largest leader_epoch of
>>> > >> messages
>>> > >> > > > whose
>>> > >> > > > >> offset < the commit offset. If no message has been consumed
>>> > since
>>> > >> > > > consumer
>>> > >> > > > >> initialization, the leader_epoch from seek(...) or
>>> > >> > OffsetFetchResponse
>>> > >> > > > >> should be used. The partition_epoch should be read from the
>>> > last
>>> > >> > > > >> FetchResponse corresponding to the given partition and
>>> commit
>>> > >> > offset.
>>> > >> > > ":
>>> > >> > > > >> leader_epoch and partition_epoch are associated with an
>>> offset.
>>> > >> So,
>>> > >> > if
>>> > >> > > > no
>>> > >> > > > >> message is consumed, there is no offset and therefore
>>> there is
>>> > no
>>> > >> > need
>>> > >> > > > to
>>> > >> > > > >> read leader_epoch and partition_epoch. Also, the
>>> leader_epoch
>>> > >> > > associated
>>> > >> > > > >> with the offset should just come from the messages
>>> returned in
>>> > >> the
>>> > >> > > fetch
>>> > >> > > > >> response.
>>> > >> > > > >>
>>> > >> > > > >
>>> > >> > > > > I am thinking that, if user calls seek(..) and
>>> commitSync(...)
>>> > >> > without
>>> > >> > > > > consuming any messages, we should re-use the leader_epoch
>>> and
>>> > >> > > > > partition_epoch provided by the seek(...) in the
>>> > >> OffsetCommitRequest.
>>> > >> > > And
>>> > >> > > > > if messages have been successfully consumed, then
>>> leader_epoch
>>> > >> will
>>> > >> > > come
>>> > >> > > > > from the messages returned in the fetch response. The
>>> condition
>>> > >> > > "messages
>>> > >> > > > > whose offset < the commit offset" is needed to take care of
>>> the
>>> > >> log
>>> > >> > > > > compacted topic which may have offset gap due to log
>>> cleaning.
>>> > >> > > > >
>>> > >> > > > > Did I miss something here? Or should I rephrase the
>>> paragraph to
>>> > >> make
>>> > >> > > it
>>> > >> > > > > less confusing?
>>> > >> > > > >
>>> > >> > > > >
>>> > >> > > > >> 64. Could you include the public methods in the OffsetEpoch
>>> > >> class?
>>> > >> > > > >>
>>> > >> > > > >
>>> > >> > > > > I mistakenly deleted the definition of OffsetEpoch class
>>> from
>>> > the
>>> > >> > KIP.
>>> > >> > > I
>>> > >> > > > > just added it back with the public methods. Could you take
>>> > another
>>> > >> > > look?
>>> > >> > > > >
>>> > >> > > > >
>>> > >> > > > >>
>>> > >> > > > >> Jun
>>> > >> > > > >>
>>> > >> > > > >>
>>> > >> > > > >> On Thu, Jan 11, 2018 at 5:43 PM, Dong Lin <
>>> lindong28@gmail.com
>>> > >
>>> > >> > > wrote:
>>> > >> > > > >>
>>> > >> > > > >> > Hey Jun,
>>> > >> > > > >> >
>>> > >> > > > >> > Thanks much. I agree that we can not rely on committed
>>> > offsets
>>> > >> to
>>> > >> > be
>>> > >> > > > >> always
>>> > >> > > > >> > deleted when we delete topic. So it is necessary to use a
>>> > >> > > > per-partition
>>> > >> > > > >> > epoch that does not change unless this partition is
>>> deleted.
>>> > I
>>> > >> > also
>>> > >> > > > >> agree
>>> > >> > > > >> > that it is very nice to be able to uniquely identify a
>>> > message
>>> > >> > with
>>> > >> > > > >> > (offset, leader_epoch, partition_epoch) in face of
>>> potential
>>> > >> topic
>>> > >> > > > >> deletion
>>> > >> > > > >> > and unclean leader election.
>>> > >> > > > >> >
>>> > >> > > > >> > I agree with all your comments. And I have updated the
>>> KIP
>>> > >> based
>>> > >> > on
>>> > >> > > > our
>>> > >> > > > >> > latest discussion. In addition, I added
>>> > >> > > InvalidPartitionEpochException
>>> > >> > > > >> > which will be thrown by consumer.poll() if the
>>> > partition_epoch
>>> > >> > > > >> associated
>>> > >> > > > >> > with the partition, which can be given to consumer using
>>> > >> > seek(...),
>>> > >> > > is
>>> > >> > > > >> > different from the partition_epoch in the FetchResponse.
>>> > >> > > > >> >
>>> > >> > > > >> > Can you take another look at the latest KIP?
>>> > >> > > > >> >
>>> > >> > > > >> > Thanks!
>>> > >> > > > >> > Dong
>>> > >> > > > >> >
>>> > >> > > > >> >
>>> > >> > > > >> >
>>> > >> > > > >> > On Wed, Jan 10, 2018 at 2:24 PM, Jun Rao <
>>> jun@confluent.io>
>>> > >> > wrote:
>>> > >> > > > >> >
>>> > >> > > > >> > > Hi, Dong,
>>> > >> > > > >> > >
>>> > >> > > > >> > > My replies are the following.
>>> > >> > > > >> > >
>>> > >> > > > >> > > 60. What you described could also work. The drawback is
>>> > that
>>> > >> we
>>> > >> > > will
>>> > >> > > > >> be
>>> > >> > > > >> > > unnecessarily changing the partition epoch when a
>>> partition
>>> > >> > hasn't
>>> > >> > > > >> really
>>> > >> > > > >> > > changed. I was imagining that the partition epoch will
>>> be
>>> > >> stored
>>> > >> > > in
>>> > >> > > > >> > > /brokers/topics/[topic]/partitions/[partitionId],
>>> instead
>>> > >> of at
>>> > >> > > the
>>> > >> > > > >> > topic
>>> > >> > > > >> > > level. So, not sure if ZK size limit is an issue.
>>> > >> > > > >> > >
>>> > >> > > > >> > > 61, 62 and 65. To me, the offset + offset_epoch is a
>>> unique
>>> > >> > > > identifier
>>> > >> > > > >> > for
>>> > >> > > > >> > > a message. So, if a message hasn't changed, the offset
>>> and
>>> > >> the
>>> > >> > > > >> associated
>>> > >> > > > >> > > offset_epoch ideally should remain the same (it will be
>>> > kind
>>> > >> of
>>> > >> > > > weird
>>> > >> > > > >> if
>>> > >> > > > >> > > two consumer apps save the offset on the same message,
>>> but
>>> > >> the
>>> > >> > > > >> > offset_epoch
>>> > >> > > > >> > > are different). partition_epoch + leader_epoch give us
>>> > that.
>>> > >> > > > >> > global_epoch +
>>> > >> > > > >> > > leader_epoch don't. If we use this approach, we can
>>> solve
>>> > not
>>> > >> > only
>>> > >> > > > the
>>> > >> > > > >> > > problem that you have identified, but also other
>>> problems
>>> > >> when
>>> > >> > > there
>>> > >> > > > >> is
>>> > >> > > > >> > > data loss or topic re-creation more reliably. For
>>> example,
>>> > in
>>> > >> > the
>>> > >> > > > >> future,
>>> > >> > > > >> > > if we include the partition_epoch and leader_epoch in
>>> the
>>> > >> fetch
>>> > >> > > > >> request,
>>> > >> > > > >> > > the server can do a more reliable check of whether that
>>> > >> offset
>>> > >> > is
>>> > >> > > > >> valid
>>> > >> > > > >> > or
>>> > >> > > > >> > > not. I am not sure that we can rely upon all external
>>> > >> offsets to
>>> > >> > > be
>>> > >> > > > >> > removed
>>> > >> > > > >> > > on topic deletion. For example, a topic may be deleted
>>> by
>>> > an
>>> > >> > admin
>>> > >> > > > who
>>> > >> > > > >> > may
>>> > >> > > > >> > > not know all the applications.
>>> > >> > > > >> > >
>>> > >> > > > >> > > If we agree on the above, the second question is then
>>> how
>>> > to
>>> > >> > > > reliably
>>> > >> > > > >> > > propagate the partition_epoch and the leader_epoch to
>>> the
>>> > >> > consumer
>>> > >> > > > >> when
>>> > >> > > > >> > > there are leader or partition changes. The leader_epoch
>>> > comes
>>> > >> > from
>>> > >> > > > the
>>> > >> > > > >> > > message, which is reliable. So, I was suggesting that
>>> when
>>> > we
>>> > >> > > store
>>> > >> > > > an
>>> > >> > > > >> > > offset, we can just store the leader_epoch from the
>>> message
>>> > >> set
>>> > >> > > > >> > containing
>>> > >> > > > >> > > that offset. Similarly, I was thinking that if the
>>> > >> > partition_epoch
>>> > >> > > > is
>>> > >> > > > >> in
>>> > >> > > > >> > > the fetch response, we can propagate partition_epoch
>>> > reliably
>>> > >> > > where
>>> > >> > > > is
>>> > >> > > > >> > > partition_epoch change.
>>> > >> > > > >> > >
>>> > >> > > > >> > > 63. My point is that once a leader is producing a
>>> message
>>> > in
>>> > >> the
>>> > >> > > new
>>> > >> > > > >> > > partition_epoch, ideally, we should associate the new
>>> > offsets
>>> > >> > with
>>> > >> > > > the
>>> > >> > > > >> > new
>>> > >> > > > >> > > partition_epoch. Otherwise, the offset_epoch won't be
>>> the
>>> > >> > correct
>>> > >> > > > >> unique
>>> > >> > > > >> > > identifier (useful for solving other problems mentioned
>>> > >> above).
>>> > >> > I
>>> > >> > > > was
>>> > >> > > > >> > > originally thinking that the leader will include the
>>> > >> > > partition_epoch
>>> > >> > > > >> in
>>> > >> > > > >> > the
>>> > >> > > > >> > > metadata cache in the fetch response. It's just that
>>> right
>>> > >> now,
>>> > >> > > > >> metadata
>>> > >> > > > >> > > cache is updated on UpdateMetadataRequest, which
>>> typically
>>> > >> > happens
>>> > >> > > > >> after
>>> > >> > > > >> > > the LeaderAndIsrRequest. Another approach is for the
>>> leader
>>> > >> to
>>> > >> > > cache
>>> > >> > > > >> the
>>> > >> > > > >> > > partition_epoch in the Partition object and return that
>>> > >> (instead
>>> > >> > > of
>>> > >> > > > >> the
>>> > >> > > > >> > one
>>> > >> > > > >> > > in metadata cache) in the fetch response.
>>> > >> > > > >> > >
>>> > >> > > > >> > > 65. It seems to me that the global_epoch and the
>>> > >> partition_epoch
>>> > >> > > > have
>>> > >> > > > >> > > different purposes. A partition_epoch has the benefit
>>> that
>>> > it
>>> > >> > (1)
>>> > >> > > > can
>>> > >> > > > >> be
>>> > >> > > > >> > > used to form a unique identifier for a message and (2)
>>> can
>>> > be
>>> > >> > used
>>> > >> > > > to
>>> > >> > > > >> > > solve other
>>> > >> > > > >> > > corner case problems in the future. I am not sure
>>> having
>>> > >> just a
>>> > >> > > > >> > > global_epoch can achieve these. global_epoch is useful
>>> to
>>> > >> > > determine
>>> > >> > > > >> which
>>> > >> > > > >> > > version of the metadata is newer, especially with topic
>>> > >> > deletion.
>>> > >> > > > >> > >
>>> > >> > > > >> > > Thanks,
>>> > >> > > > >> > >
>>> > >> > > > >> > > Jun
>>> > >> > > > >> > >
>>> > >> > > > >> > > On Tue, Jan 9, 2018 at 11:34 PM, Dong Lin <
>>> > >> lindong28@gmail.com>
>>> > >> > > > >> wrote:
>>> > >> > > > >> > >
>>> > >> > > > >> > > > Regarding the use of the global epoch in 65), it is
>>> very
>>> > >> > similar
>>> > >> > > > to
>>> > >> > > > >> the
>>> > >> > > > >> > > > proposal of the metadata_epoch we discussed earlier.
>>> The
>>> > >> main
>>> > >> > > > >> > difference
>>> > >> > > > >> > > is
>>> > >> > > > >> > > > that this epoch is incremented when we
>>> > create/expand/delete
>>> > >> > > topic
>>> > >> > > > >> and
>>> > >> > > > >> > > does
>>> > >> > > > >> > > > not change when controller re-send metadata.
>>> > >> > > > >> > > >
>>> > >> > > > >> > > > I looked at our previous discussion. It seems that we
>>> > >> prefer
>>> > >> > > > >> > > > partition_epoch over the metadata_epoch because 1) we
>>> > >> prefer
>>> > >> > not
>>> > >> > > > to
>>> > >> > > > >> > have
>>> > >> > > > >> > > an
>>> > >> > > > >> > > > ever growing metadata_epoch and 2) we can reset
>>> offset
>>> > >> better
>>> > >> > > when
>>> > >> > > > >> > topic
>>> > >> > > > >> > > is
>>> > >> > > > >> > > > re-created. The use of global topic_epoch avoids the
>>> > >> drawback
>>> > >> > of
>>> > >> > > > an
>>> > >> > > > >> > ever
>>> > >> > > > >> > > > quickly ever growing metadata_epoch. Though the
>>> global
>>> > >> epoch
>>> > >> > > does
>>> > >> > > > >> not
>>> > >> > > > >> > > allow
>>> > >> > > > >> > > > us to recognize the invalid offset committed before
>>> the
>>> > >> topic
>>> > >> > > > >> > > re-creation,
>>> > >> > > > >> > > > we can probably just delete the offset when we
>>> delete a
>>> > >> topic.
>>> > >> > > > Thus
>>> > >> > > > >> I
>>> > >> > > > >> > am
>>> > >> > > > >> > > > not very sure whether it is still worthwhile to have
>>> a
>>> > >> > > > per-partition
>>> > >> > > > >> > > > partition_epoch if the metadata already has the
>>> global
>>> > >> epoch.
>>> > >> > > > >> > > >
>>> > >> > > > >> > > >
>>> > >> > > > >> > > > On Tue, Jan 9, 2018 at 6:58 PM, Dong Lin <
>>> > >> lindong28@gmail.com
>>> > >> > >
>>> > >> > > > >> wrote:
>>> > >> > > > >> > > >
>>> > >> > > > >> > > > > Hey Jun,
>>> > >> > > > >> > > > >
>>> > >> > > > >> > > > > Thanks so much. These comments very useful. Please
>>> see
>>> > >> below
>>> > >> > > my
>>> > >> > > > >> > > comments.
>>> > >> > > > >> > > > >
>>> > >> > > > >> > > > > On Mon, Jan 8, 2018 at 5:52 PM, Jun Rao <
>>> > >> jun@confluent.io>
>>> > >> > > > wrote:
>>> > >> > > > >> > > > >
>>> > >> > > > >> > > > >> Hi, Dong,
>>> > >> > > > >> > > > >>
>>> > >> > > > >> > > > >> Thanks for the updated KIP. A few more comments.
>>> > >> > > > >> > > > >>
>>> > >> > > > >> > > > >> 60. Perhaps having a partition epoch is more
>>> flexible
>>> > >> since
>>> > >> > > in
>>> > >> > > > >> the
>>> > >> > > > >> > > > future,
>>> > >> > > > >> > > > >> we may support deleting a partition as well.
>>> > >> > > > >> > > > >>
>>> > >> > > > >> > > > >
>>> > >> > > > >> > > > > Yeah I have considered this. I think we can
>>> probably
>>> > >> still
>>> > >> > > > support
>>> > >> > > > >> > > > > deleting a partition by using the topic_epoch --
>>> when
>>> > >> > > partition
>>> > >> > > > >> of a
>>> > >> > > > >> > > > topic
>>> > >> > > > >> > > > > is deleted or created, epoch of all partitions of
>>> this
>>> > >> topic
>>> > >> > > > will
>>> > >> > > > >> be
>>> > >> > > > >> > > > > incremented by 1. Therefore, if that partition is
>>> > >> re-created
>>> > >> > > > >> later,
>>> > >> > > > >> > the
>>> > >> > > > >> > > > > epoch of that partition will still be larger than
>>> its
>>> > >> epoch
>>> > >> > > > before
>>> > >> > > > >> > the
>>> > >> > > > >> > > > > deletion, which still allows the client to order
>>> the
>>> > >> > metadata
>>> > >> > > > for
>>> > >> > > > >> the
>>> > >> > > > >> > > > > purpose of this KIP. Does this sound reasonable?
>>> > >> > > > >> > > > >
>>> > >> > > > >> > > > > The advantage of using topic_epoch instead of
>>> > >> > partition_epoch
>>> > >> > > is
>>> > >> > > > >> that
>>> > >> > > > >> > > the
>>> > >> > > > >> > > > > size of the /brokers/topics/[topic] znode and
>>> > >> > request/response
>>> > >> > > > >> size
>>> > >> > > > >> > can
>>> > >> > > > >> > > > be
>>> > >> > > > >> > > > > smaller. We have a limit on the maximum size of
>>> znode
>>> > >> > > (typically
>>> > >> > > > >> > 1MB).
>>> > >> > > > >> > > > Use
>>> > >> > > > >> > > > > partition epoch can effectively reduce the number
>>> of
>>> > >> > > partitions
>>> > >> > > > >> that
>>> > >> > > > >> > > can
>>> > >> > > > >> > > > be
>>> > >> > > > >> > > > > described by the /brokers/topics/[topic] znode.
>>> > >> > > > >> > > > >
>>> > >> > > > >> > > > > One use-case of partition_epoch for client to
>>> detect
>>> > that
>>> > >> > the
>>> > >> > > > >> > committed
>>> > >> > > > >> > > > > offset, either from kafka offset topic or from the
>>> > >> external
>>> > >> > > > store
>>> > >> > > > >> is
>>> > >> > > > >> > > > > invalid after partition deletion and re-creation.
>>> > >> However,
>>> > >> > it
>>> > >> > > > >> seems
>>> > >> > > > >> > > that
>>> > >> > > > >> > > > we
>>> > >> > > > >> > > > > can also address this use-case with other
>>> approaches.
>>> > For
>>> > >> > > > example,
>>> > >> > > > >> > when
>>> > >> > > > >> > > > > AdminClient deletes partitions, it can also delete
>>> the
>>> > >> > > committed
>>> > >> > > > >> > > offsets
>>> > >> > > > >> > > > > for those partitions from the offset topic. If user
>>> > >> stores
>>> > >> > > > offset
>>> > >> > > > >> > > > > externally, it might make sense for user to
>>> similarly
>>> > >> remove
>>> > >> > > > >> offsets
>>> > >> > > > >> > of
>>> > >> > > > >> > > > > related partitions after these partitions are
>>> deleted.
>>> > >> So I
>>> > >> > am
>>> > >> > > > not
>>> > >> > > > >> > sure
>>> > >> > > > >> > > > > that we should use partition_epoch in this KIP.
>>> > >> > > > >> > > > >
>>> > >> > > > >> > > > >
>>> > >> > > > >> > > > >>
>>> > >> > > > >> > > > >> 61. It seems that the leader epoch returned in the
>>> > >> > position()
>>> > >> > > > >> call
>>> > >> > > > >> > > > should
>>> > >> > > > >> > > > >> the the leader epoch returned in the fetch
>>> response,
>>> > not
>>> > >> > the
>>> > >> > > > one
>>> > >> > > > >> in
>>> > >> > > > >> > > the
>>> > >> > > > >> > > > >> metadata cache of the client.
>>> > >> > > > >> > > > >
>>> > >> > > > >> > > > >
>>> > >> > > > >> > > > > I think this is a good idea. Just to double check,
>>> this
>>> > >> > change
>>> > >> > > > >> does
>>> > >> > > > >> > not
>>> > >> > > > >> > > > > affect the correctness or performance of this KIP.
>>> But
>>> > it
>>> > >> > can
>>> > >> > > be
>>> > >> > > > >> > useful
>>> > >> > > > >> > > > if
>>> > >> > > > >> > > > > we want to use the leader_epoch to better handle
>>> the
>>> > >> offset
>>> > >> > > rest
>>> > >> > > > >> in
>>> > >> > > > >> > > case
>>> > >> > > > >> > > > of
>>> > >> > > > >> > > > > unclean leader election, which is listed in the
>>> future
>>> > >> work.
>>> > >> > > Is
>>> > >> > > > >> this
>>> > >> > > > >> > > > > understanding correct?
>>> > >> > > > >> > > > >
>>> > >> > > > >> > > > > I have updated the KIP to specify that the
>>> leader_epoch
>>> > >> > > returned
>>> > >> > > > >> by
>>> > >> > > > >> > > > > position() should be the largest leader_epoch of
>>> those
>>> > >> > already
>>> > >> > > > >> > consumed
>>> > >> > > > >> > > > > messages whose offset < position. If no message has
>>> > been
>>> > >> > > > consumed
>>> > >> > > > >> > since
>>> > >> > > > >> > > > > consumer initialization, the leader_epoch from
>>> seek()
>>> > or
>>> > >> > > > >> > > > > OffsetFetchResponse should be used. The offset
>>> included
>>> > >> in
>>> > >> > the
>>> > >> > > > >> > > > > OffsetCommitRequest will also be determined in the
>>> > >> similar
>>> > >> > > > manner.
>>> > >> > > > >> > > > >
>>> > >> > > > >> > > > >
>>> > >> > > > >> > > > >>
>>> > >> > > > >> > > > >> 62. I am wondering if we should return the
>>> partition
>>> > >> epoch
>>> > >> > in
>>> > >> > > > the
>>> > >> > > > >> > > fetch
>>> > >> > > > >> > > > >> response as well. In the current proposal, if a
>>> topic
>>> > is
>>> > >> > > > >> recreated
>>> > >> > > > >> > and
>>> > >> > > > >> > > > the
>>> > >> > > > >> > > > >> new leader is on the same broker as the old one,
>>> there
>>> > >> is
>>> > >> > > > >> nothing to
>>> > >> > > > >> > > > force
>>> > >> > > > >> > > > >> the metadata refresh in the client. So, the
>>> client may
>>> > >> > still
>>> > >> > > > >> > associate
>>> > >> > > > >> > > > the
>>> > >> > > > >> > > > >> offset with the old partition epoch.
>>> > >> > > > >> > > > >>
>>> > >> > > > >> > > > >
>>> > >> > > > >> > > > > Could you help me understand the problem if a
>>> client
>>> > >> > > associates
>>> > >> > > > >> old
>>> > >> > > > >> > > > > partition_epoch (or the topic_epoch as of the
>>> current
>>> > >> KIP)
>>> > >> > > with
>>> > >> > > > >> the
>>> > >> > > > >> > > > offset?
>>> > >> > > > >> > > > > The main purpose of the topic_epoch is to be able
>>> to
>>> > drop
>>> > >> > > > >> > leader_epoch
>>> > >> > > > >> > > > to 0
>>> > >> > > > >> > > > > after a partition is deleted and re-created. I
>>> guess
>>> > you
>>> > >> may
>>> > >> > > be
>>> > >> > > > >> > > thinking
>>> > >> > > > >> > > > > about using the partition_epoch to detect that the
>>> > >> committed
>>> > >> > > > >> offset
>>> > >> > > > >> > is
>>> > >> > > > >> > > > > invalid? In that case, I am wondering if the
>>> > alternative
>>> > >> > > > approach
>>> > >> > > > >> > > > described
>>> > >> > > > >> > > > > in 60) would be reasonable.
>>> > >> > > > >> > > > >
>>> > >> > > > >> > > > >
>>> > >> > > > >> > > > >>
>>> > >> > > > >> > > > >> 63. There is some subtle coordination between the
>>> > >> > > > >> > LeaderAndIsrRequest
>>> > >> > > > >> > > > and
>>> > >> > > > >> > > > >> UpdateMetadataRequest. Currently, when a leader
>>> > changes,
>>> > >> > the
>>> > >> > > > >> > > controller
>>> > >> > > > >> > > > >> first sends the LeaderAndIsrRequest to the
>>> assigned
>>> > >> > replicas
>>> > >> > > > and
>>> > >> > > > >> the
>>> > >> > > > >> > > > >> UpdateMetadataRequest to every broker. So, there
>>> could
>>> > >> be a
>>> > >> > > > small
>>> > >> > > > >> > > window
>>> > >> > > > >> > > > >> when the leader already receives the new partition
>>> > >> epoch in
>>> > >> > > the
>>> > >> > > > >> > > > >> LeaderAndIsrRequest, but the metadata cache in the
>>> > >> broker
>>> > >> > > > hasn't
>>> > >> > > > >> > been
>>> > >> > > > >> > > > >> updated with the latest partition epoch. Not sure
>>> > what's
>>> > >> > the
>>> > >> > > > best
>>> > >> > > > >> > way
>>> > >> > > > >> > > to
>>> > >> > > > >> > > > >> address this issue. Perhaps we can update the
>>> metadata
>>> > >> > cache
>>> > >> > > on
>>> > >> > > > >> the
>>> > >> > > > >> > > > broker
>>> > >> > > > >> > > > >> with both LeaderAndIsrRequest and
>>> > UpdateMetadataRequest.
>>> > >> > The
>>> > >> > > > >> > challenge
>>> > >> > > > >> > > > is
>>> > >> > > > >> > > > >> that the two have slightly different data. For
>>> > example,
>>> > >> > only
>>> > >> > > > the
>>> > >> > > > >> > > latter
>>> > >> > > > >> > > > >> has
>>> > >> > > > >> > > > >> all endpoints.
>>> > >> > > > >> > > > >>
>>> > >> > > > >> > > > >
>>> > >> > > > >> > > > > I am not sure whether this is a problem. Could you
>>> > >> explain a
>>> > >> > > bit
>>> > >> > > > >> more
>>> > >> > > > >> > > > what
>>> > >> > > > >> > > > > specific problem this small window can cause?
>>> > >> > > > >> > > > >
>>> > >> > > > >> > > > > Since client can fetch metadata from any broker in
>>> the
>>> > >> > > cluster,
>>> > >> > > > >> and
>>> > >> > > > >> > > given
>>> > >> > > > >> > > > > that different brokers receive request (e.g.
>>> > >> > > LeaderAndIsrRequest
>>> > >> > > > >> and
>>> > >> > > > >> > > > > UpdateMetadataRequest) in arbitrary order, the
>>> metadata
>>> > >> > > received
>>> > >> > > > >> by
>>> > >> > > > >> > > > client
>>> > >> > > > >> > > > > can be in arbitrary order (either newer or older)
>>> > >> compared
>>> > >> > to
>>> > >> > > > the
>>> > >> > > > >> > > > broker's
>>> > >> > > > >> > > > > leadership state even if a given broker receives
>>> > >> > > > >> LeaderAndIsrRequest
>>> > >> > > > >> > > and
>>> > >> > > > >> > > > > UpdateMetadataRequest simultaneously. So I am not
>>> sure
>>> > >> it is
>>> > >> > > > >> useful
>>> > >> > > > >> > to
>>> > >> > > > >> > > > > update broker's cache with LeaderAndIsrRequest.
>>> > >> > > > >> > > > >
>>> > >> > > > >> > > > >
>>> > >> > > > >> > > > >> 64. The enforcement of leader epoch in Offset
>>> commit:
>>> > We
>>> > >> > > allow
>>> > >> > > > a
>>> > >> > > > >> > > > consumer
>>> > >> > > > >> > > > >> to set an arbitrary offset. So it's possible for
>>> > >> offsets or
>>> > >> > > > >> leader
>>> > >> > > > >> > > epoch
>>> > >> > > > >> > > > >> to
>>> > >> > > > >> > > > >> go backwards. I am not sure if we could always
>>> enforce
>>> > >> that
>>> > >> > > the
>>> > >> > > > >> > leader
>>> > >> > > > >> > > > >> epoch only goes up on the broker.
>>> > >> > > > >> > > > >>
>>> > >> > > > >> > > > >
>>> > >> > > > >> > > > > Sure. I have removed this check from the KIP.
>>> > >> > > > >> > > > >
>>> > >> > > > >> > > > > BTW, we can probably still ensure that the
>>> leader_epoch
>>> > >> > always
>>> > >> > > > >> > increase
>>> > >> > > > >> > > > if
>>> > >> > > > >> > > > > the leader_epoch used with offset commit is the
>>> > >> > > max(leader_epoch
>>> > >> > > > >> of
>>> > >> > > > >> > the
>>> > >> > > > >> > > > > message with offset = the committed offset - 1, the
>>> > >> largest
>>> > >> > > > known
>>> > >> > > > >> > > > > leader_epoch from the metadata). But I don't have a
>>> > good
>>> > >> > > > use-case
>>> > >> > > > >> for
>>> > >> > > > >> > > > this
>>> > >> > > > >> > > > > alternative definition. So I choose the keep the
>>> KIP
>>> > >> simple
>>> > >> > by
>>> > >> > > > >> > > requiring
>>> > >> > > > >> > > > > leader_epoch to always increase.
>>> > >> > > > >> > > > >
>>> > >> > > > >> > > > >
>>> > >> > > > >> > > > >> 65. Good point on handling missing partition
>>> epoch due
>>> > >> to
>>> > >> > > topic
>>> > >> > > > >> > > > deletion.
>>> > >> > > > >> > > > >> Another potential way to address this is to
>>> > additionally
>>> > >> > > > >> propagate
>>> > >> > > > >> > the
>>> > >> > > > >> > > > >> global partition epoch to brokers and the clients.
>>> > This
>>> > >> > way,
>>> > >> > > > >> when a
>>> > >> > > > >> > > > >> partition epoch is missing, we can use the global
>>> > >> partition
>>> > >> > > > >> epoch to
>>> > >> > > > >> > > > >> reason
>>> > >> > > > >> > > > >> about which metadata is more recent.
>>> > >> > > > >> > > > >>
>>> > >> > > > >> > > > >
>>> > >> > > > >> > > > > This is a great idea. The global epoch can be used
>>> to
>>> > >> order
>>> > >> > > the
>>> > >> > > > >> > > metadata
>>> > >> > > > >> > > > > and help us recognize the more recent metadata if a
>>> > topic
>>> > >> > (or
>>> > >> > > > >> > > partition)
>>> > >> > > > >> > > > is
>>> > >> > > > >> > > > > deleted and re-created.
>>> > >> > > > >> > > > >
>>> > >> > > > >> > > > > Actually, it seems we only need to propagate the
>>> global
>>> > >> > epoch
>>> > >> > > to
>>> > >> > > > >> > > brokers
>>> > >> > > > >> > > > > and clients without propagating this epoch on a
>>> > >> per-topic or
>>> > >> > > > >> > > > per-partition
>>> > >> > > > >> > > > > basic. Doing so would simply interface changes made
>>> > this
>>> > >> > KIP.
>>> > >> > > > Does
>>> > >> > > > >> > this
>>> > >> > > > >> > > > > approach sound reasonable?
>>> > >> > > > >> > > > >
>>> > >> > > > >> > > > >
>>> > >> > > > >> > > > >> 66. A client may also get an offset by time using
>>> the
>>> > >> > > > >> > offsetForTimes()
>>> > >> > > > >> > > > >> api.
>>> > >> > > > >> > > > >> So, we probably want to include
>>> offsetInternalMetadata
>>> > >> in
>>> > >> > > > >> > > > >> OffsetAndTimestamp
>>> > >> > > > >> > > > >> as well.
>>> > >> > > > >> > > > >>
>>> > >> > > > >> > > > >
>>> > >> > > > >> > > > > You are right. This probably also requires us to
>>> change
>>> > >> the
>>> > >> > > > >> > > > > ListOffsetRequest as well. I will update the KIP
>>> after
>>> > we
>>> > >> > > agree
>>> > >> > > > on
>>> > >> > > > >> > the
>>> > >> > > > >> > > > > solution for 65).
>>> > >> > > > >> > > > >
>>> > >> > > > >> > > > >
>>> > >> > > > >> > > > >>
>>> > >> > > > >> > > > >> 67. InteralMetadata can be a bit confusing with
>>> the
>>> > >> > metadata
>>> > >> > > > >> field
>>> > >> > > > >> > > > already
>>> > >> > > > >> > > > >> there. Perhaps we can just call it OffsetEpoch. It
>>> > >> might be
>>> > >> > > > >> useful
>>> > >> > > > >> > to
>>> > >> > > > >> > > > make
>>> > >> > > > >> > > > >> OffsetEpoch printable at least for debugging
>>> purpose.
>>> > >> Once
>>> > >> > > you
>>> > >> > > > do
>>> > >> > > > >> > > that,
>>> > >> > > > >> > > > we
>>> > >> > > > >> > > > >> are already exposing the internal fields. So, not
>>> sure
>>> > >> if
>>> > >> > > it's
>>> > >> > > > >> worth
>>> > >> > > > >> > > > >> hiding
>>> > >> > > > >> > > > >> them. If we do want to hide them, perhaps we can
>>> have
>>> > >> sth
>>> > >> > > like
>>> > >> > > > >> the
>>> > >> > > > >> > > > >> following. The binary encoding is probably more
>>> > >> efficient
>>> > >> > > than
>>> > >> > > > >> JSON
>>> > >> > > > >> > > for
>>> > >> > > > >> > > > >> external storage.
>>> > >> > > > >> > > > >>
>>> > >> > > > >> > > > >> OffsetEpoch {
>>> > >> > > > >> > > > >>  static OffsetEpoch decode(byte[]);
>>> > >> > > > >> > > > >>
>>> > >> > > > >> > > > >>   public byte[] encode();
>>> > >> > > > >> > > > >>
>>> > >> > > > >> > > > >>   public String toString();
>>> > >> > > > >> > > > >> }
>>> > >> > > > >> > > > >>
>>> > >> > > > >> > > > >
>>> > >> > > > >> > > > > Thanks much. I like this solution. I have updated
>>> the
>>> > KIP
>>> > >> > > > >> > accordingly.
>>> > >> > > > >> > > > >
>>> > >> > > > >> > > > >
>>> > >> > > > >> > > > >
>>> > >> > > > >> > > > >>
>>> > >> > > > >> > > > >> Jun
>>> > >> > > > >> > > > >>
>>> > >> > > > >> > > > >> On Mon, Jan 8, 2018 at 4:22 PM, Dong Lin <
>>> > >> > > lindong28@gmail.com>
>>> > >> > > > >> > wrote:
>>> > >> > > > >> > > > >>
>>> > >> > > > >> > > > >> > Hey Jason,
>>> > >> > > > >> > > > >> >
>>> > >> > > > >> > > > >> > Certainly. This sounds good. I have updated the
>>> KIP
>>> > to
>>> > >> > > > clarity
>>> > >> > > > >> > that
>>> > >> > > > >> > > > the
>>> > >> > > > >> > > > >> > global epoch will be incremented by 1 each time
>>> a
>>> > >> topic
>>> > >> > is
>>> > >> > > > >> > deleted.
>>> > >> > > > >> > > > >> >
>>> > >> > > > >> > > > >> > Thanks,
>>> > >> > > > >> > > > >> > Dong
>>> > >> > > > >> > > > >> >
>>> > >> > > > >> > > > >> > On Mon, Jan 8, 2018 at 4:09 PM, Jason Gustafson
>>> <
>>> > >> > > > >> > jason@confluent.io
>>> > >> > > > >> > > >
>>> > >> > > > >> > > > >> > wrote:
>>> > >> > > > >> > > > >> >
>>> > >> > > > >> > > > >> > > Hi Dong,
>>> > >> > > > >> > > > >> > >
>>> > >> > > > >> > > > >> > >
>>> > >> > > > >> > > > >> > > I think your approach will allow user to
>>> > distinguish
>>> > >> > > > between
>>> > >> > > > >> the
>>> > >> > > > >> > > > >> metadata
>>> > >> > > > >> > > > >> > > > before and after the topic deletion. I also
>>> > agree
>>> > >> > that
>>> > >> > > > this
>>> > >> > > > >> > can
>>> > >> > > > >> > > be
>>> > >> > > > >> > > > >> > > > potentially be useful to user. I am just not
>>> > very
>>> > >> > sure
>>> > >> > > > >> whether
>>> > >> > > > >> > > we
>>> > >> > > > >> > > > >> > already
>>> > >> > > > >> > > > >> > > > have a good use-case to make the additional
>>> > >> > complexity
>>> > >> > > > >> > > worthwhile.
>>> > >> > > > >> > > > >> It
>>> > >> > > > >> > > > >> > > seems
>>> > >> > > > >> > > > >> > > > that this feature is kind of independent of
>>> the
>>> > >> main
>>> > >> > > > >> problem
>>> > >> > > > >> > of
>>> > >> > > > >> > > > this
>>> > >> > > > >> > > > >> > KIP.
>>> > >> > > > >> > > > >> > > > Could we add this as a future work?
>>> > >> > > > >> > > > >> > >
>>> > >> > > > >> > > > >> > >
>>> > >> > > > >> > > > >> > > Do you think it's fair if we bump the topic
>>> epoch
>>> > on
>>> > >> > > > deletion
>>> > >> > > > >> > and
>>> > >> > > > >> > > > >> leave
>>> > >> > > > >> > > > >> > > propagation of the epoch for deleted topics
>>> for
>>> > >> future
>>> > >> > > > work?
>>> > >> > > > >> I
>>> > >> > > > >> > > don't
>>> > >> > > > >> > > > >> > think
>>> > >> > > > >> > > > >> > > this adds much complexity and it makes the
>>> > behavior
>>> > >> > > > >> consistent:
>>> > >> > > > >> > > > every
>>> > >> > > > >> > > > >> > topic
>>> > >> > > > >> > > > >> > > mutation results in an epoch bump.
>>> > >> > > > >> > > > >> > >
>>> > >> > > > >> > > > >> > > Thanks,
>>> > >> > > > >> > > > >> > > Jason
>>> > >> > > > >> > > > >> > >
>>> > >> > > > >> > > > >> > > On Mon, Jan 8, 2018 at 3:14 PM, Dong Lin <
>>> > >> > > > >> lindong28@gmail.com>
>>> > >> > > > >> > > > wrote:
>>> > >> > > > >> > > > >> > >
>>> > >> > > > >> > > > >> > > > Hey Ismael,
>>> > >> > > > >> > > > >> > > >
>>> > >> > > > >> > > > >> > > > I guess we actually need user to see this
>>> field
>>> > so
>>> > >> > that
>>> > >> > > > >> user
>>> > >> > > > >> > can
>>> > >> > > > >> > > > >> store
>>> > >> > > > >> > > > >> > > this
>>> > >> > > > >> > > > >> > > > value in the external store together with
>>> the
>>> > >> offset.
>>> > >> > > We
>>> > >> > > > >> just
>>> > >> > > > >> > > > prefer
>>> > >> > > > >> > > > >> > the
>>> > >> > > > >> > > > >> > > > value to be opaque to discourage most users
>>> from
>>> > >> > > > >> interpreting
>>> > >> > > > >> > > this
>>> > >> > > > >> > > > >> > value.
>>> > >> > > > >> > > > >> > > > One more advantage of using such an opaque
>>> field
>>> > >> is
>>> > >> > to
>>> > >> > > be
>>> > >> > > > >> able
>>> > >> > > > >> > > to
>>> > >> > > > >> > > > >> > evolve
>>> > >> > > > >> > > > >> > > > the information (or schema) of this value
>>> > without
>>> > >> > > > changing
>>> > >> > > > >> > > > consumer
>>> > >> > > > >> > > > >> API
>>> > >> > > > >> > > > >> > > in
>>> > >> > > > >> > > > >> > > > the future.
>>> > >> > > > >> > > > >> > > >
>>> > >> > > > >> > > > >> > > > I also thinking it is probably OK for user
>>> to be
>>> > >> able
>>> > >> > > to
>>> > >> > > > >> > > interpret
>>> > >> > > > >> > > > >> this
>>> > >> > > > >> > > > >> > > > value, particularly for those advanced
>>> users.
>>> > >> > > > >> > > > >> > > >
>>> > >> > > > >> > > > >> > > > Thanks,
>>> > >> > > > >> > > > >> > > > Dong
>>> > >> > > > >> > > > >> > > >
>>> > >> > > > >> > > > >> > > > On Mon, Jan 8, 2018 at 2:34 PM, Ismael Juma
>>> <
>>> > >> > > > >> > ismael@juma.me.uk>
>>> > >> > > > >> > > > >> wrote:
>>> > >> > > > >> > > > >> > > >
>>> > >> > > > >> > > > >> > > > > On Fri, Jan 5, 2018 at 7:15 PM, Jason
>>> > Gustafson
>>> > >> <
>>> > >> > > > >> > > > >> jason@confluent.io>
>>> > >> > > > >> > > > >> > > > > wrote:
>>> > >> > > > >> > > > >> > > > > >
>>> > >> > > > >> > > > >> > > > > > class OffsetAndMetadata {
>>> > >> > > > >> > > > >> > > > > >   long offset;
>>> > >> > > > >> > > > >> > > > > >   byte[] offsetMetadata;
>>> > >> > > > >> > > > >> > > > > >   String metadata;
>>> > >> > > > >> > > > >> > > > > > }
>>> > >> > > > >> > > > >> > > > >
>>> > >> > > > >> > > > >> > > > >
>>> > >> > > > >> > > > >> > > > > > Admittedly, the naming is a bit
>>> annoying,
>>> > but
>>> > >> we
>>> > >> > > can
>>> > >> > > > >> > > probably
>>> > >> > > > >> > > > >> come
>>> > >> > > > >> > > > >> > up
>>> > >> > > > >> > > > >> > > > > with
>>> > >> > > > >> > > > >> > > > > > something better. Internally the byte
>>> array
>>> > >> would
>>> > >> > > > have
>>> > >> > > > >> a
>>> > >> > > > >> > > > >> version.
>>> > >> > > > >> > > > >> > If
>>> > >> > > > >> > > > >> > > in
>>> > >> > > > >> > > > >> > > > > the
>>> > >> > > > >> > > > >> > > > > > future we have anything else we need to
>>> add,
>>> > >> we
>>> > >> > can
>>> > >> > > > >> update
>>> > >> > > > >> > > the
>>> > >> > > > >> > > > >> > > version
>>> > >> > > > >> > > > >> > > > > and
>>> > >> > > > >> > > > >> > > > > > we wouldn't need any new APIs.
>>> > >> > > > >> > > > >> > > > > >
>>> > >> > > > >> > > > >> > > > >
>>> > >> > > > >> > > > >> > > > > We can also add fields to a class in a
>>> > >> compatible
>>> > >> > > way.
>>> > >> > > > >> So,
>>> > >> > > > >> > it
>>> > >> > > > >> > > > >> seems
>>> > >> > > > >> > > > >> > to
>>> > >> > > > >> > > > >> > > me
>>> > >> > > > >> > > > >> > > > > that the main advantage of the byte array
>>> is
>>> > >> that
>>> > >> > > it's
>>> > >> > > > >> > opaque
>>> > >> > > > >> > > to
>>> > >> > > > >> > > > >> the
>>> > >> > > > >> > > > >> > > > user.
>>> > >> > > > >> > > > >> > > > > Is that correct? If so, we could also add
>>> any
>>> > >> > opaque
>>> > >> > > > >> > metadata
>>> > >> > > > >> > > > in a
>>> > >> > > > >> > > > >> > > > subclass
>>> > >> > > > >> > > > >> > > > > so that users don't even see it (unless
>>> they
>>> > >> cast
>>> > >> > it,
>>> > >> > > > but
>>> > >> > > > >> > then
>>> > >> > > > >> > > > >> > they're
>>> > >> > > > >> > > > >> > > on
>>> > >> > > > >> > > > >> > > > > their own).
>>> > >> > > > >> > > > >> > > > >
>>> > >> > > > >> > > > >> > > > > Ismael
>>> > >> > > > >> > > > >> > > > >
>>> > >> > > > >> > > > >> > > > > The corresponding seek() and position()
>>> APIs
>>> > >> might
>>> > >> > > look
>>> > >> > > > >> > > > something
>>> > >> > > > >> > > > >> > like
>>> > >> > > > >> > > > >> > > > > this:
>>> > >> > > > >> > > > >> > > > > >
>>> > >> > > > >> > > > >> > > > > > void seek(TopicPartition partition, long
>>> > >> offset,
>>> > >> > > > byte[]
>>> > >> > > > >> > > > >> > > > offsetMetadata);
>>> > >> > > > >> > > > >> > > > > > byte[] positionMetadata(TopicPartition
>>> > >> > partition);
>>> > >> > > > >> > > > >> > > > > >
>>> > >> > > > >> > > > >> > > > > > What do you think?
>>> > >> > > > >> > > > >> > > > > >
>>> > >> > > > >> > > > >> > > > > > Thanks,
>>> > >> > > > >> > > > >> > > > > > Jason
>>> > >> > > > >> > > > >> > > > > >
>>> > >> > > > >> > > > >> > > > > > On Thu, Jan 4, 2018 at 7:04 PM, Dong
>>> Lin <
>>> > >> > > > >> > > lindong28@gmail.com
>>> > >> > > > >> > > > >
>>> > >> > > > >> > > > >> > > wrote:
>>> > >> > > > >> > > > >> > > > > >
>>> > >> > > > >> > > > >> > > > > > > Hey Jun, Jason,
>>> > >> > > > >> > > > >> > > > > > >
>>> > >> > > > >> > > > >> > > > > > > Thanks much for all the feedback. I
>>> have
>>> > >> > updated
>>> > >> > > > the
>>> > >> > > > >> KIP
>>> > >> > > > >> > > > >> based on
>>> > >> > > > >> > > > >> > > the
>>> > >> > > > >> > > > >> > > > > > > latest discussion. Can you help check
>>> > >> whether
>>> > >> > it
>>> > >> > > > >> looks
>>> > >> > > > >> > > good?
>>> > >> > > > >> > > > >> > > > > > >
>>> > >> > > > >> > > > >> > > > > > > Thanks,
>>> > >> > > > >> > > > >> > > > > > > Dong
>>> > >> > > > >> > > > >> > > > > > >
>>> > >> > > > >> > > > >> > > > > > > On Thu, Jan 4, 2018 at 5:36 PM, Dong
>>> Lin <
>>> > >> > > > >> > > > lindong28@gmail.com
>>> > >> > > > >> > > > >> >
>>> > >> > > > >> > > > >> > > > wrote:
>>> > >> > > > >> > > > >> > > > > > >
>>> > >> > > > >> > > > >> > > > > > > > Hey Jun,
>>> > >> > > > >> > > > >> > > > > > > >
>>> > >> > > > >> > > > >> > > > > > > > Hmm... thinking about this more, I
>>> am
>>> > not
>>> > >> > sure
>>> > >> > > > that
>>> > >> > > > >> > the
>>> > >> > > > >> > > > >> > proposed
>>> > >> > > > >> > > > >> > > > API
>>> > >> > > > >> > > > >> > > > > is
>>> > >> > > > >> > > > >> > > > > > > > sufficient. For users that store
>>> offset
>>> > >> > > > >> externally, we
>>> > >> > > > >> > > > >> probably
>>> > >> > > > >> > > > >> > > > need
>>> > >> > > > >> > > > >> > > > > > > extra
>>> > >> > > > >> > > > >> > > > > > > > API to return the leader_epoch and
>>> > >> > > > partition_epoch
>>> > >> > > > >> for
>>> > >> > > > >> > > all
>>> > >> > > > >> > > > >> > > > partitions
>>> > >> > > > >> > > > >> > > > > > > that
>>> > >> > > > >> > > > >> > > > > > > > consumers are consuming. I suppose
>>> these
>>> > >> > users
>>> > >> > > > >> > currently
>>> > >> > > > >> > > > use
>>> > >> > > > >> > > > >> > > > > position()
>>> > >> > > > >> > > > >> > > > > > > to
>>> > >> > > > >> > > > >> > > > > > > > get the offset. Thus we probably
>>> need a
>>> > >> new
>>> > >> > > > method
>>> > >> > > > >> > > > >> > > > > > positionWithEpoch(..)
>>> > >> > > > >> > > > >> > > > > > > to
>>> > >> > > > >> > > > >> > > > > > > > return <offset, partition_epoch,
>>> > >> > leader_epoch>.
>>> > >> > > > >> Does
>>> > >> > > > >> > > this
>>> > >> > > > >> > > > >> sound
>>> > >> > > > >> > > > >> > > > > > > reasonable?
>>> > >> > > > >> > > > >> > > > > > > >
>>> > >> > > > >> > > > >> > > > > > > > Thanks,
>>> > >> > > > >> > > > >> > > > > > > > Dong
>>> > >> > > > >> > > > >> > > > > > > >
>>> > >> > > > >> > > > >> > > > > > > >
>>> > >> > > > >> > > > >> > > > > > > > On Thu, Jan 4, 2018 at 5:26 PM, Jun
>>> Rao
>>> > <
>>> > >> > > > >> > > jun@confluent.io
>>> > >> > > > >> > > > >
>>> > >> > > > >> > > > >> > > wrote:
>>> > >> > > > >> > > > >> > > > > > > >
>>> > >> > > > >> > > > >> > > > > > > >> Hi, Dong,
>>> > >> > > > >> > > > >> > > > > > > >>
>>> > >> > > > >> > > > >> > > > > > > >> Yes, that's what I am thinking.
>>> > >> OffsetEpoch
>>> > >> > > will
>>> > >> > > > >> be
>>> > >> > > > >> > > > >> composed
>>> > >> > > > >> > > > >> > of
>>> > >> > > > >> > > > >> > > > > > > >> (partition_epoch,
>>> > >> > > > >> > > > >> > > > > > > >> leader_epoch).
>>> > >> > > > >> > > > >> > > > > > > >>
>>> > >> > > > >> > > > >> > > > > > > >> Thanks,
>>> > >> > > > >> > > > >> > > > > > > >>
>>> > >> > > > >> > > > >> > > > > > > >> Jun
>>> > >> > > > >> > > > >> > > > > > > >>
>>> > >> > > > >> > > > >> > > > > > > >>
>>> > >> > > > >> > > > >> > > > > > > >> On Thu, Jan 4, 2018 at 4:22 PM,
>>> Dong
>>> > Lin
>>> > >> <
>>> > >> > > > >> > > > >> lindong28@gmail.com
>>> > >> > > > >> > > > >> > >
>>> > >> > > > >> > > > >> > > > > wrote:
>>> > >> > > > >> > > > >> > > > > > > >>
>>> > >> > > > >> > > > >> > > > > > > >> > Hey Jun,
>>> > >> > > > >> > > > >> > > > > > > >> >
>>> > >> > > > >> > > > >> > > > > > > >> > Thanks much. I like the the new
>>> API
>>> > >> that
>>> > >> > you
>>> > >> > > > >> > > proposed.
>>> > >> > > > >> > > > I
>>> > >> > > > >> > > > >> am
>>> > >> > > > >> > > > >> > > not
>>> > >> > > > >> > > > >> > > > > sure
>>> > >> > > > >> > > > >> > > > > > > >> what
>>> > >> > > > >> > > > >> > > > > > > >> > you exactly mean by
>>> offset_epoch. I
>>> > >> > suppose
>>> > >> > > > >> that we
>>> > >> > > > >> > > can
>>> > >> > > > >> > > > >> use
>>> > >> > > > >> > > > >> > > the
>>> > >> > > > >> > > > >> > > > > pair
>>> > >> > > > >> > > > >> > > > > > > of
>>> > >> > > > >> > > > >> > > > > > > >> > (partition_epoch, leader_epoch)
>>> as
>>> > the
>>> > >> > > > >> > offset_epoch,
>>> > >> > > > >> > > > >> right?
>>> > >> > > > >> > > > >> > > > > > > >> >
>>> > >> > > > >> > > > >> > > > > > > >> > Thanks,
>>> > >> > > > >> > > > >> > > > > > > >> > Dong
>>> > >> > > > >> > > > >> > > > > > > >> >
>>> > >> > > > >> > > > >> > > > > > > >> > On Thu, Jan 4, 2018 at 4:02 PM,
>>> Jun
>>> > >> Rao <
>>> > >> > > > >> > > > >> jun@confluent.io>
>>> > >> > > > >> > > > >> > > > wrote:
>>> > >> > > > >> > > > >> > > > > > > >> >
>>> > >> > > > >> > > > >> > > > > > > >> > > Hi, Dong,
>>> > >> > > > >> > > > >> > > > > > > >> > >
>>> > >> > > > >> > > > >> > > > > > > >> > > Got it. The api that you
>>> proposed
>>> > >> works.
>>> > >> > > The
>>> > >> > > > >> > > question
>>> > >> > > > >> > > > >> is
>>> > >> > > > >> > > > >> > > > whether
>>> > >> > > > >> > > > >> > > > > > > >> that's
>>> > >> > > > >> > > > >> > > > > > > >> > the
>>> > >> > > > >> > > > >> > > > > > > >> > > api that we want to have in the
>>> > long
>>> > >> > term.
>>> > >> > > > My
>>> > >> > > > >> > > concern
>>> > >> > > > >> > > > >> is
>>> > >> > > > >> > > > >> > > that
>>> > >> > > > >> > > > >> > > > > > while
>>> > >> > > > >> > > > >> > > > > > > >> the
>>> > >> > > > >> > > > >> > > > > > > >> > api
>>> > >> > > > >> > > > >> > > > > > > >> > > change is simple, the new api
>>> seems
>>> > >> > harder
>>> > >> > > > to
>>> > >> > > > >> > > explain
>>> > >> > > > >> > > > >> and
>>> > >> > > > >> > > > >> > > use.
>>> > >> > > > >> > > > >> > > > > For
>>> > >> > > > >> > > > >> > > > > > > >> > example,
>>> > >> > > > >> > > > >> > > > > > > >> > > a consumer storing offsets
>>> > externally
>>> > >> > now
>>> > >> > > > >> needs
>>> > >> > > > >> > to
>>> > >> > > > >> > > > call
>>> > >> > > > >> > > > >> > > > > > > >> > > waitForMetadataUpdate() after
>>> > calling
>>> > >> > > > seek().
>>> > >> > > > >> > > > >> > > > > > > >> > >
>>> > >> > > > >> > > > >> > > > > > > >> > > An alternative approach is to
>>> make
>>> > >> the
>>> > >> > > > >> following
>>> > >> > > > >> > > > >> > compatible
>>> > >> > > > >> > > > >> > > > api
>>> > >> > > > >> > > > >> > > > > > > >> changes
>>> > >> > > > >> > > > >> > > > > > > >> > in
>>> > >> > > > >> > > > >> > > > > > > >> > > Consumer.
>>> > >> > > > >> > > > >> > > > > > > >> > > * Add an additional OffsetEpoch
>>> > >> field in
>>> > >> > > > >> > > > >> > OffsetAndMetadata.
>>> > >> > > > >> > > > >> > > > (no
>>> > >> > > > >> > > > >> > > > > > need
>>> > >> > > > >> > > > >> > > > > > > >> to
>>> > >> > > > >> > > > >> > > > > > > >> > > change the CommitSync() api)
>>> > >> > > > >> > > > >> > > > > > > >> > > * Add a new api
>>> seek(TopicPartition
>>> > >> > > > partition,
>>> > >> > > > >> > long
>>> > >> > > > >> > > > >> > offset,
>>> > >> > > > >> > > > >> > > > > > > >> OffsetEpoch
>>> > >> > > > >> > > > >> > > > > > > >> > > offsetEpoch). We can
>>> potentially
>>> > >> > deprecate
>>> > >> > > > the
>>> > >> > > > >> > old
>>> > >> > > > >> > > > api
>>> > >> > > > >> > > > >> > > > > > > >> > seek(TopicPartition
>>> > >> > > > >> > > > >> > > > > > > >> > > partition, long offset) in the
>>> > >> future.
>>> > >> > > > >> > > > >> > > > > > > >> > >
>>> > >> > > > >> > > > >> > > > > > > >> > > The alternative approach has
>>> > similar
>>> > >> > > amount
>>> > >> > > > of
>>> > >> > > > >> > api
>>> > >> > > > >> > > > >> changes
>>> > >> > > > >> > > > >> > > as
>>> > >> > > > >> > > > >> > > > > > yours
>>> > >> > > > >> > > > >> > > > > > > >> but
>>> > >> > > > >> > > > >> > > > > > > >> > has
>>> > >> > > > >> > > > >> > > > > > > >> > > the following benefits.
>>> > >> > > > >> > > > >> > > > > > > >> > > 1. The api works in a similar
>>> way
>>> > as
>>> > >> how
>>> > >> > > > >> offset
>>> > >> > > > >> > > > >> management
>>> > >> > > > >> > > > >> > > > works
>>> > >> > > > >> > > > >> > > > > > now
>>> > >> > > > >> > > > >> > > > > > > >> and
>>> > >> > > > >> > > > >> > > > > > > >> > is
>>> > >> > > > >> > > > >> > > > > > > >> > > probably what we want in the
>>> long
>>> > >> term.
>>> > >> > > > >> > > > >> > > > > > > >> > > 2. It can reset offsets better
>>> when
>>> > >> > there
>>> > >> > > is
>>> > >> > > > >> data
>>> > >> > > > >> > > > loss
>>> > >> > > > >> > > > >> due
>>> > >> > > > >> > > > >> > > to
>>> > >> > > > >> > > > >> > > > > > > unclean
>>> > >> > > > >> > > > >> > > > > > > >> > > leader election or correlated
>>> > replica
>>> > >> > > > failure.
>>> > >> > > > >> > > > >> > > > > > > >> > > 3. It can reset offsets better
>>> when
>>> > >> > topic
>>> > >> > > is
>>> > >> > > > >> > > > recreated.
>>> > >> > > > >> > > > >> > > > > > > >> > >
>>> > >> > > > >> > > > >> > > > > > > >> > > Thanks,
>>> > >> > > > >> > > > >> > > > > > > >> > >
>>> > >> > > > >> > > > >> > > > > > > >> > > Jun
>>> > >> > > > >> > > > >> > > > > > > >> > >
>>> > >> > > > >> > > > >> > > > > > > >> > >
>>> > >> > > > >> > > > >> > > > > > > >> > > On Thu, Jan 4, 2018 at 2:05 PM,
>>> > Dong
>>> > >> > Lin <
>>> > >> > > > >> > > > >> > > lindong28@gmail.com
>>> > >> > > > >> > > > >> > > > >
>>> > >> > > > >> > > > >> > > > > > > wrote:
>>> > >> > > > >> > > > >> > > > > > > >> > >
>>> > >> > > > >> > > > >> > > > > > > >> > > > Hey Jun,
>>> > >> > > > >> > > > >> > > > > > > >> > > >
>>> > >> > > > >> > > > >> > > > > > > >> > > > Yeah I agree that ideally we
>>> > don't
>>> > >> > want
>>> > >> > > an
>>> > >> > > > >> ever
>>> > >> > > > >> > > > >> growing
>>> > >> > > > >> > > > >> > > > global
>>> > >> > > > >> > > > >> > > > > > > >> metadata
>>> > >> > > > >> > > > >> > > > > > > >> > > > version. I just think it may
>>> be
>>> > >> more
>>> > >> > > > >> desirable
>>> > >> > > > >> > to
>>> > >> > > > >> > > > >> keep
>>> > >> > > > >> > > > >> > the
>>> > >> > > > >> > > > >> > > > > > > consumer
>>> > >> > > > >> > > > >> > > > > > > >> API
>>> > >> > > > >> > > > >> > > > > > > >> > > > simple.
>>> > >> > > > >> > > > >> > > > > > > >> > > >
>>> > >> > > > >> > > > >> > > > > > > >> > > > In my current proposal,
>>> metadata
>>> > >> > version
>>> > >> > > > >> > returned
>>> > >> > > > >> > > > in
>>> > >> > > > >> > > > >> the
>>> > >> > > > >> > > > >> > > > fetch
>>> > >> > > > >> > > > >> > > > > > > >> response
>>> > >> > > > >> > > > >> > > > > > > >> > > > will be stored with the
>>> offset
>>> > >> > together.
>>> > >> > > > >> More
>>> > >> > > > >> > > > >> > > specifically,
>>> > >> > > > >> > > > >> > > > > the
>>> > >> > > > >> > > > >> > > > > > > >> > > > metadata_epoch in the new
>>> offset
>>> > >> topic
>>> > >> > > > >> schema
>>> > >> > > > >> > > will
>>> > >> > > > >> > > > be
>>> > >> > > > >> > > > >> > the
>>> > >> > > > >> > > > >> > > > > > largest
>>> > >> > > > >> > > > >> > > > > > > >> > > > metadata_epoch from all the
>>> > >> > > > MetadataResponse
>>> > >> > > > >> > and
>>> > >> > > > >> > > > >> > > > FetchResponse
>>> > >> > > > >> > > > >> > > > > > > ever
>>> > >> > > > >> > > > >> > > > > > > >> > > > received by this consumer.
>>> > >> > > > >> > > > >> > > > > > > >> > > >
>>> > >> > > > >> > > > >> > > > > > > >> > > > We probably don't have to
>>> change
>>> > >> the
>>> > >> > > > >> consumer
>>> > >> > > > >> > API
>>> > >> > > > >> > > > for
>>> > >> > > > >> > > > >> > > > > > > >> > > >
>>> commitSync(Map<TopicPartition,
>>> > >> > > > >> > > OffsetAndMetadata>).
>>> > >> > > > >> > > > >> If
>>> > >> > > > >> > > > >> > > user
>>> > >> > > > >> > > > >> > > > > > calls
>>> > >> > > > >> > > > >> > > > > > > >> > > > commitSync(...) to commit
>>> offset
>>> > 10
>>> > >> > for
>>> > >> > > a
>>> > >> > > > >> given
>>> > >> > > > >> > > > >> > partition,
>>> > >> > > > >> > > > >> > > > for
>>> > >> > > > >> > > > >> > > > > > > most
>>> > >> > > > >> > > > >> > > > > > > >> > > > use-cases, this consumer
>>> instance
>>> > >> > should
>>> > >> > > > >> have
>>> > >> > > > >> > > > >> consumed
>>> > >> > > > >> > > > >> > > > message
>>> > >> > > > >> > > > >> > > > > > > with
>>> > >> > > > >> > > > >> > > > > > > >> > > offset
>>> > >> > > > >> > > > >> > > > > > > >> > > > 9 from this partition, in
>>> which
>>> > >> case
>>> > >> > the
>>> > >> > > > >> > consumer
>>> > >> > > > >> > > > can
>>> > >> > > > >> > > > >> > > > remember
>>> > >> > > > >> > > > >> > > > > > and
>>> > >
>>>
>>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Dong Lin <li...@gmail.com>.
Hey Jun, Jason,

Thanks for all the comments. Could you see if you can give +1 for the KIP?
I am open to make further improvements for the KIP.

Thanks,
Dong

On Tue, Jan 23, 2018 at 3:44 PM, Dong Lin <li...@gmail.com> wrote:

> Hey Jun, Jason,
>
> Thanks much for all the review! I will open the voting thread.
>
> Regards,
> Dong
>
> On Tue, Jan 23, 2018 at 3:37 PM, Jun Rao <ju...@confluent.io> wrote:
>
>> Hi, Dong,
>>
>> The current KIP looks good to me.
>>
>> Thanks,
>>
>> Jun
>>
>> On Tue, Jan 23, 2018 at 12:29 PM, Dong Lin <li...@gmail.com> wrote:
>>
>> > Hey Jun,
>> >
>> > Do you think the current KIP looks OK? I am wondering if we can open the
>> > voting thread.
>> >
>> > Thanks!
>> > Dong
>> >
>> > On Fri, Jan 19, 2018 at 3:08 PM, Dong Lin <li...@gmail.com> wrote:
>> >
>> > > Hey Jun,
>> > >
>> > > I think we can probably have a static method in Util class to decode
>> the
>> > > byte[]. Both KafkaConsumer implementation and the user application
>> will
>> > be
>> > > able to decode the byte array and log its content for debug purpose.
>> So
>> > it
>> > > seems that we can still print the information we want. It is just not
>> > > explicitly exposed in the consumer interface. Would this address the
>> > > problem here?
>> > >
>> > > Yeah we can include OffsetEpoch in AdminClient. This can be added in
>> > > KIP-222? Is there something you would like me to add in this KIP?
>> > >
>> > > Thanks!
>> > > Dong
>> > >
>> > > On Fri, Jan 19, 2018 at 3:00 PM, Jun Rao <ju...@confluent.io> wrote:
>> > >
>> > >> Hi, Dong,
>> > >>
>> > >> The issue with using just byte[] for OffsetEpoch is that it won't be
>> > >> printable, which makes debugging harder.
>> > >>
>> > >> Also, KIP-222 proposes a listGroupOffset() method in AdminClient. If
>> > that
>> > >> gets adopted before this KIP, we probably want to include
>> OffsetEpoch in
>> > >> the AdminClient too.
>> > >>
>> > >> Thanks,
>> > >>
>> > >> Jun
>> > >>
>> > >>
>> > >> On Thu, Jan 18, 2018 at 6:30 PM, Dong Lin <li...@gmail.com>
>> wrote:
>> > >>
>> > >> > Hey Jun,
>> > >> >
>> > >> > I agree. I have updated the KIP to remove the class OffetEpoch and
>> > >> replace
>> > >> > OffsetEpoch with byte[] in APIs that use it. Can you see if it
>> looks
>> > >> good?
>> > >> >
>> > >> > Thanks!
>> > >> > Dong
>> > >> >
>> > >> > On Thu, Jan 18, 2018 at 6:07 PM, Jun Rao <ju...@confluent.io> wrote:
>> > >> >
>> > >> > > Hi, Dong,
>> > >> > >
>> > >> > > Thanks for the updated KIP. It looks good to me now. The only
>> thing
>> > is
>> > >> > > for OffsetEpoch.
>> > >> > > If we expose the individual fields in the class, we probably
>> don't
>> > >> need
>> > >> > the
>> > >> > > encode/decode methods. If we want to hide the details of
>> > OffsetEpoch,
>> > >> we
>> > >> > > probably don't want expose the individual fields.
>> > >> > >
>> > >> > > Jun
>> > >> > >
>> > >> > > On Wed, Jan 17, 2018 at 10:10 AM, Dong Lin <li...@gmail.com>
>> > >> wrote:
>> > >> > >
>> > >> > > > Thinking about point 61 more, I realize that the async
>> zookeeper
>> > >> read
>> > >> > may
>> > >> > > > make it less of an issue for controller to read more zookeeper
>> > >> nodes.
>> > >> > > > Writing partition_epoch in the per-partition znode makes it
>> > simpler
>> > >> to
>> > >> > > > handle the broker failure between zookeeper writes for a topic
>> > >> > creation.
>> > >> > > I
>> > >> > > > have updated the KIP to use the suggested approach.
>> > >> > > >
>> > >> > > >
>> > >> > > > On Wed, Jan 17, 2018 at 9:57 AM, Dong Lin <lindong28@gmail.com
>> >
>> > >> wrote:
>> > >> > > >
>> > >> > > > > Hey Jun,
>> > >> > > > >
>> > >> > > > > Thanks much for the comments. Please see my comments inline.
>> > >> > > > >
>> > >> > > > > On Tue, Jan 16, 2018 at 4:38 PM, Jun Rao <ju...@confluent.io>
>> > >> wrote:
>> > >> > > > >
>> > >> > > > >> Hi, Dong,
>> > >> > > > >>
>> > >> > > > >> Thanks for the updated KIP. Looks good to me overall. Just a
>> > few
>> > >> > minor
>> > >> > > > >> comments.
>> > >> > > > >>
>> > >> > > > >> 60. OffsetAndMetadata positionAndOffsetEpoch(TopicPartition
>> > >> > > partition):
>> > >> > > > >> It
>> > >> > > > >> seems that there is no need to return metadata. We probably
>> > want
>> > >> to
>> > >> > > > return
>> > >> > > > >> sth like OffsetAndEpoch.
>> > >> > > > >>
>> > >> > > > >
>> > >> > > > > Previously I think we may want to re-use the existing class
>> to
>> > >> keep
>> > >> > our
>> > >> > > > > consumer interface simpler. I have updated the KIP to add
>> class
>> > >> > > > > OffsetAndOffsetEpoch. I didn't use OffsetAndEpoch because
>> user
>> > may
>> > >> > > > confuse
>> > >> > > > > this name with OffsetEpoch. Does this sound OK?
>> > >> > > > >
>> > >> > > > >
>> > >> > > > >>
>> > >> > > > >> 61. Should we store partition_epoch in
>> > >> > > > >> /brokers/topics/[topic]/partitions/[partitionId] in ZK?
>> > >> > > > >>
>> > >> > > > >
>> > >> > > > > I have considered this. I think the advantage of adding the
>> > >> > > > > partition->partition_epoch map in the existing
>> > >> > > > > znode /brokers/topics/[topic]/partitions is that controller
>> > only
>> > >> > needs
>> > >> > > > to
>> > >> > > > > read one znode per topic to gets its partition_epoch
>> > information.
>> > >> > > > Otherwise
>> > >> > > > > controller may need to read one extra znode per partition to
>> get
>> > >> the
>> > >> > > same
>> > >> > > > > information.
>> > >> > > > >
>> > >> > > > > When we delete partition or expand partition of a topic,
>> someone
>> > >> > needs
>> > >> > > to
>> > >> > > > > modify partition->partition_epoch map in znode
>> > >> > > > > /brokers/topics/[topic]/partitions. This may seem a bit more
>> > >> > > complicated
>> > >> > > > > than simply adding or deleting znode /brokers/topics/[topic]/
>> > >> > > > partitions/[partitionId].
>> > >> > > > > But the complexity is probably similar to the existing
>> operation
>> > >> of
>> > >> > > > > modifying the partition->replica_list mapping in znode
>> > >> > > > > /brokers/topics/[topic]. So not sure it is better to store
>> the
>> > >> > > > > partition_epoch in /brokers/topics/[topic]/partit
>> > >> ions/[partitionId].
>> > >> > > > What
>> > >> > > > > do you think?
>> > >> > > > >
>> > >> > > > >
>> > >> > > > >>
>> > >> > > > >> 62. For checking outdated metadata in the client, we
>> probably
>> > >> want
>> > >> > to
>> > >> > > > add
>> > >> > > > >> when max_partition_epoch will be used.
>> > >> > > > >>
>> > >> > > > >
>> > >> > > > > The max_partition_epoch is used in the Proposed Changes ->
>> > >> Client's
>> > >> > > > > metadata refresh section to determine whether a metadata is
>> > >> outdated.
>> > >> > > And
>> > >> > > > > this formula is referenced and re-used in other sections to
>> > >> determine
>> > >> > > > > whether a metadata is outdated. Does this formula look OK?
>> > >> > > > >
>> > >> > > > >
>> > >> > > > >>
>> > >> > > > >> 63. "The leader_epoch should be the largest leader_epoch of
>> > >> messages
>> > >> > > > whose
>> > >> > > > >> offset < the commit offset. If no message has been consumed
>> > since
>> > >> > > > consumer
>> > >> > > > >> initialization, the leader_epoch from seek(...) or
>> > >> > OffsetFetchResponse
>> > >> > > > >> should be used. The partition_epoch should be read from the
>> > last
>> > >> > > > >> FetchResponse corresponding to the given partition and
>> commit
>> > >> > offset.
>> > >> > > ":
>> > >> > > > >> leader_epoch and partition_epoch are associated with an
>> offset.
>> > >> So,
>> > >> > if
>> > >> > > > no
>> > >> > > > >> message is consumed, there is no offset and therefore there
>> is
>> > no
>> > >> > need
>> > >> > > > to
>> > >> > > > >> read leader_epoch and partition_epoch. Also, the
>> leader_epoch
>> > >> > > associated
>> > >> > > > >> with the offset should just come from the messages returned
>> in
>> > >> the
>> > >> > > fetch
>> > >> > > > >> response.
>> > >> > > > >>
>> > >> > > > >
>> > >> > > > > I am thinking that, if user calls seek(..) and
>> commitSync(...)
>> > >> > without
>> > >> > > > > consuming any messages, we should re-use the leader_epoch and
>> > >> > > > > partition_epoch provided by the seek(...) in the
>> > >> OffsetCommitRequest.
>> > >> > > And
>> > >> > > > > if messages have been successfully consumed, then
>> leader_epoch
>> > >> will
>> > >> > > come
>> > >> > > > > from the messages returned in the fetch response. The
>> condition
>> > >> > > "messages
>> > >> > > > > whose offset < the commit offset" is needed to take care of
>> the
>> > >> log
>> > >> > > > > compacted topic which may have offset gap due to log
>> cleaning.
>> > >> > > > >
>> > >> > > > > Did I miss something here? Or should I rephrase the
>> paragraph to
>> > >> make
>> > >> > > it
>> > >> > > > > less confusing?
>> > >> > > > >
>> > >> > > > >
>> > >> > > > >> 64. Could you include the public methods in the OffsetEpoch
>> > >> class?
>> > >> > > > >>
>> > >> > > > >
>> > >> > > > > I mistakenly deleted the definition of OffsetEpoch class from
>> > the
>> > >> > KIP.
>> > >> > > I
>> > >> > > > > just added it back with the public methods. Could you take
>> > another
>> > >> > > look?
>> > >> > > > >
>> > >> > > > >
>> > >> > > > >>
>> > >> > > > >> Jun
>> > >> > > > >>
>> > >> > > > >>
>> > >> > > > >> On Thu, Jan 11, 2018 at 5:43 PM, Dong Lin <
>> lindong28@gmail.com
>> > >
>> > >> > > wrote:
>> > >> > > > >>
>> > >> > > > >> > Hey Jun,
>> > >> > > > >> >
>> > >> > > > >> > Thanks much. I agree that we can not rely on committed
>> > offsets
>> > >> to
>> > >> > be
>> > >> > > > >> always
>> > >> > > > >> > deleted when we delete topic. So it is necessary to use a
>> > >> > > > per-partition
>> > >> > > > >> > epoch that does not change unless this partition is
>> deleted.
>> > I
>> > >> > also
>> > >> > > > >> agree
>> > >> > > > >> > that it is very nice to be able to uniquely identify a
>> > message
>> > >> > with
>> > >> > > > >> > (offset, leader_epoch, partition_epoch) in face of
>> potential
>> > >> topic
>> > >> > > > >> deletion
>> > >> > > > >> > and unclean leader election.
>> > >> > > > >> >
>> > >> > > > >> > I agree with all your comments. And I have updated the KIP
>> > >> based
>> > >> > on
>> > >> > > > our
>> > >> > > > >> > latest discussion. In addition, I added
>> > >> > > InvalidPartitionEpochException
>> > >> > > > >> > which will be thrown by consumer.poll() if the
>> > partition_epoch
>> > >> > > > >> associated
>> > >> > > > >> > with the partition, which can be given to consumer using
>> > >> > seek(...),
>> > >> > > is
>> > >> > > > >> > different from the partition_epoch in the FetchResponse.
>> > >> > > > >> >
>> > >> > > > >> > Can you take another look at the latest KIP?
>> > >> > > > >> >
>> > >> > > > >> > Thanks!
>> > >> > > > >> > Dong
>> > >> > > > >> >
>> > >> > > > >> >
>> > >> > > > >> >
>> > >> > > > >> > On Wed, Jan 10, 2018 at 2:24 PM, Jun Rao <
>> jun@confluent.io>
>> > >> > wrote:
>> > >> > > > >> >
>> > >> > > > >> > > Hi, Dong,
>> > >> > > > >> > >
>> > >> > > > >> > > My replies are the following.
>> > >> > > > >> > >
>> > >> > > > >> > > 60. What you described could also work. The drawback is
>> > that
>> > >> we
>> > >> > > will
>> > >> > > > >> be
>> > >> > > > >> > > unnecessarily changing the partition epoch when a
>> partition
>> > >> > hasn't
>> > >> > > > >> really
>> > >> > > > >> > > changed. I was imagining that the partition epoch will
>> be
>> > >> stored
>> > >> > > in
>> > >> > > > >> > > /brokers/topics/[topic]/partitions/[partitionId],
>> instead
>> > >> of at
>> > >> > > the
>> > >> > > > >> > topic
>> > >> > > > >> > > level. So, not sure if ZK size limit is an issue.
>> > >> > > > >> > >
>> > >> > > > >> > > 61, 62 and 65. To me, the offset + offset_epoch is a
>> unique
>> > >> > > > identifier
>> > >> > > > >> > for
>> > >> > > > >> > > a message. So, if a message hasn't changed, the offset
>> and
>> > >> the
>> > >> > > > >> associated
>> > >> > > > >> > > offset_epoch ideally should remain the same (it will be
>> > kind
>> > >> of
>> > >> > > > weird
>> > >> > > > >> if
>> > >> > > > >> > > two consumer apps save the offset on the same message,
>> but
>> > >> the
>> > >> > > > >> > offset_epoch
>> > >> > > > >> > > are different). partition_epoch + leader_epoch give us
>> > that.
>> > >> > > > >> > global_epoch +
>> > >> > > > >> > > leader_epoch don't. If we use this approach, we can
>> solve
>> > not
>> > >> > only
>> > >> > > > the
>> > >> > > > >> > > problem that you have identified, but also other
>> problems
>> > >> when
>> > >> > > there
>> > >> > > > >> is
>> > >> > > > >> > > data loss or topic re-creation more reliably. For
>> example,
>> > in
>> > >> > the
>> > >> > > > >> future,
>> > >> > > > >> > > if we include the partition_epoch and leader_epoch in
>> the
>> > >> fetch
>> > >> > > > >> request,
>> > >> > > > >> > > the server can do a more reliable check of whether that
>> > >> offset
>> > >> > is
>> > >> > > > >> valid
>> > >> > > > >> > or
>> > >> > > > >> > > not. I am not sure that we can rely upon all external
>> > >> offsets to
>> > >> > > be
>> > >> > > > >> > removed
>> > >> > > > >> > > on topic deletion. For example, a topic may be deleted
>> by
>> > an
>> > >> > admin
>> > >> > > > who
>> > >> > > > >> > may
>> > >> > > > >> > > not know all the applications.
>> > >> > > > >> > >
>> > >> > > > >> > > If we agree on the above, the second question is then
>> how
>> > to
>> > >> > > > reliably
>> > >> > > > >> > > propagate the partition_epoch and the leader_epoch to
>> the
>> > >> > consumer
>> > >> > > > >> when
>> > >> > > > >> > > there are leader or partition changes. The leader_epoch
>> > comes
>> > >> > from
>> > >> > > > the
>> > >> > > > >> > > message, which is reliable. So, I was suggesting that
>> when
>> > we
>> > >> > > store
>> > >> > > > an
>> > >> > > > >> > > offset, we can just store the leader_epoch from the
>> message
>> > >> set
>> > >> > > > >> > containing
>> > >> > > > >> > > that offset. Similarly, I was thinking that if the
>> > >> > partition_epoch
>> > >> > > > is
>> > >> > > > >> in
>> > >> > > > >> > > the fetch response, we can propagate partition_epoch
>> > reliably
>> > >> > > where
>> > >> > > > is
>> > >> > > > >> > > partition_epoch change.
>> > >> > > > >> > >
>> > >> > > > >> > > 63. My point is that once a leader is producing a
>> message
>> > in
>> > >> the
>> > >> > > new
>> > >> > > > >> > > partition_epoch, ideally, we should associate the new
>> > offsets
>> > >> > with
>> > >> > > > the
>> > >> > > > >> > new
>> > >> > > > >> > > partition_epoch. Otherwise, the offset_epoch won't be
>> the
>> > >> > correct
>> > >> > > > >> unique
>> > >> > > > >> > > identifier (useful for solving other problems mentioned
>> > >> above).
>> > >> > I
>> > >> > > > was
>> > >> > > > >> > > originally thinking that the leader will include the
>> > >> > > partition_epoch
>> > >> > > > >> in
>> > >> > > > >> > the
>> > >> > > > >> > > metadata cache in the fetch response. It's just that
>> right
>> > >> now,
>> > >> > > > >> metadata
>> > >> > > > >> > > cache is updated on UpdateMetadataRequest, which
>> typically
>> > >> > happens
>> > >> > > > >> after
>> > >> > > > >> > > the LeaderAndIsrRequest. Another approach is for the
>> leader
>> > >> to
>> > >> > > cache
>> > >> > > > >> the
>> > >> > > > >> > > partition_epoch in the Partition object and return that
>> > >> (instead
>> > >> > > of
>> > >> > > > >> the
>> > >> > > > >> > one
>> > >> > > > >> > > in metadata cache) in the fetch response.
>> > >> > > > >> > >
>> > >> > > > >> > > 65. It seems to me that the global_epoch and the
>> > >> partition_epoch
>> > >> > > > have
>> > >> > > > >> > > different purposes. A partition_epoch has the benefit
>> that
>> > it
>> > >> > (1)
>> > >> > > > can
>> > >> > > > >> be
>> > >> > > > >> > > used to form a unique identifier for a message and (2)
>> can
>> > be
>> > >> > used
>> > >> > > > to
>> > >> > > > >> > > solve other
>> > >> > > > >> > > corner case problems in the future. I am not sure having
>> > >> just a
>> > >> > > > >> > > global_epoch can achieve these. global_epoch is useful
>> to
>> > >> > > determine
>> > >> > > > >> which
>> > >> > > > >> > > version of the metadata is newer, especially with topic
>> > >> > deletion.
>> > >> > > > >> > >
>> > >> > > > >> > > Thanks,
>> > >> > > > >> > >
>> > >> > > > >> > > Jun
>> > >> > > > >> > >
>> > >> > > > >> > > On Tue, Jan 9, 2018 at 11:34 PM, Dong Lin <
>> > >> lindong28@gmail.com>
>> > >> > > > >> wrote:
>> > >> > > > >> > >
>> > >> > > > >> > > > Regarding the use of the global epoch in 65), it is
>> very
>> > >> > similar
>> > >> > > > to
>> > >> > > > >> the
>> > >> > > > >> > > > proposal of the metadata_epoch we discussed earlier.
>> The
>> > >> main
>> > >> > > > >> > difference
>> > >> > > > >> > > is
>> > >> > > > >> > > > that this epoch is incremented when we
>> > create/expand/delete
>> > >> > > topic
>> > >> > > > >> and
>> > >> > > > >> > > does
>> > >> > > > >> > > > not change when controller re-send metadata.
>> > >> > > > >> > > >
>> > >> > > > >> > > > I looked at our previous discussion. It seems that we
>> > >> prefer
>> > >> > > > >> > > > partition_epoch over the metadata_epoch because 1) we
>> > >> prefer
>> > >> > not
>> > >> > > > to
>> > >> > > > >> > have
>> > >> > > > >> > > an
>> > >> > > > >> > > > ever growing metadata_epoch and 2) we can reset offset
>> > >> better
>> > >> > > when
>> > >> > > > >> > topic
>> > >> > > > >> > > is
>> > >> > > > >> > > > re-created. The use of global topic_epoch avoids the
>> > >> drawback
>> > >> > of
>> > >> > > > an
>> > >> > > > >> > ever
>> > >> > > > >> > > > quickly ever growing metadata_epoch. Though the global
>> > >> epoch
>> > >> > > does
>> > >> > > > >> not
>> > >> > > > >> > > allow
>> > >> > > > >> > > > us to recognize the invalid offset committed before
>> the
>> > >> topic
>> > >> > > > >> > > re-creation,
>> > >> > > > >> > > > we can probably just delete the offset when we delete
>> a
>> > >> topic.
>> > >> > > > Thus
>> > >> > > > >> I
>> > >> > > > >> > am
>> > >> > > > >> > > > not very sure whether it is still worthwhile to have a
>> > >> > > > per-partition
>> > >> > > > >> > > > partition_epoch if the metadata already has the global
>> > >> epoch.
>> > >> > > > >> > > >
>> > >> > > > >> > > >
>> > >> > > > >> > > > On Tue, Jan 9, 2018 at 6:58 PM, Dong Lin <
>> > >> lindong28@gmail.com
>> > >> > >
>> > >> > > > >> wrote:
>> > >> > > > >> > > >
>> > >> > > > >> > > > > Hey Jun,
>> > >> > > > >> > > > >
>> > >> > > > >> > > > > Thanks so much. These comments very useful. Please
>> see
>> > >> below
>> > >> > > my
>> > >> > > > >> > > comments.
>> > >> > > > >> > > > >
>> > >> > > > >> > > > > On Mon, Jan 8, 2018 at 5:52 PM, Jun Rao <
>> > >> jun@confluent.io>
>> > >> > > > wrote:
>> > >> > > > >> > > > >
>> > >> > > > >> > > > >> Hi, Dong,
>> > >> > > > >> > > > >>
>> > >> > > > >> > > > >> Thanks for the updated KIP. A few more comments.
>> > >> > > > >> > > > >>
>> > >> > > > >> > > > >> 60. Perhaps having a partition epoch is more
>> flexible
>> > >> since
>> > >> > > in
>> > >> > > > >> the
>> > >> > > > >> > > > future,
>> > >> > > > >> > > > >> we may support deleting a partition as well.
>> > >> > > > >> > > > >>
>> > >> > > > >> > > > >
>> > >> > > > >> > > > > Yeah I have considered this. I think we can probably
>> > >> still
>> > >> > > > support
>> > >> > > > >> > > > > deleting a partition by using the topic_epoch --
>> when
>> > >> > > partition
>> > >> > > > >> of a
>> > >> > > > >> > > > topic
>> > >> > > > >> > > > > is deleted or created, epoch of all partitions of
>> this
>> > >> topic
>> > >> > > > will
>> > >> > > > >> be
>> > >> > > > >> > > > > incremented by 1. Therefore, if that partition is
>> > >> re-created
>> > >> > > > >> later,
>> > >> > > > >> > the
>> > >> > > > >> > > > > epoch of that partition will still be larger than
>> its
>> > >> epoch
>> > >> > > > before
>> > >> > > > >> > the
>> > >> > > > >> > > > > deletion, which still allows the client to order the
>> > >> > metadata
>> > >> > > > for
>> > >> > > > >> the
>> > >> > > > >> > > > > purpose of this KIP. Does this sound reasonable?
>> > >> > > > >> > > > >
>> > >> > > > >> > > > > The advantage of using topic_epoch instead of
>> > >> > partition_epoch
>> > >> > > is
>> > >> > > > >> that
>> > >> > > > >> > > the
>> > >> > > > >> > > > > size of the /brokers/topics/[topic] znode and
>> > >> > request/response
>> > >> > > > >> size
>> > >> > > > >> > can
>> > >> > > > >> > > > be
>> > >> > > > >> > > > > smaller. We have a limit on the maximum size of
>> znode
>> > >> > > (typically
>> > >> > > > >> > 1MB).
>> > >> > > > >> > > > Use
>> > >> > > > >> > > > > partition epoch can effectively reduce the number of
>> > >> > > partitions
>> > >> > > > >> that
>> > >> > > > >> > > can
>> > >> > > > >> > > > be
>> > >> > > > >> > > > > described by the /brokers/topics/[topic] znode.
>> > >> > > > >> > > > >
>> > >> > > > >> > > > > One use-case of partition_epoch for client to detect
>> > that
>> > >> > the
>> > >> > > > >> > committed
>> > >> > > > >> > > > > offset, either from kafka offset topic or from the
>> > >> external
>> > >> > > > store
>> > >> > > > >> is
>> > >> > > > >> > > > > invalid after partition deletion and re-creation.
>> > >> However,
>> > >> > it
>> > >> > > > >> seems
>> > >> > > > >> > > that
>> > >> > > > >> > > > we
>> > >> > > > >> > > > > can also address this use-case with other
>> approaches.
>> > For
>> > >> > > > example,
>> > >> > > > >> > when
>> > >> > > > >> > > > > AdminClient deletes partitions, it can also delete
>> the
>> > >> > > committed
>> > >> > > > >> > > offsets
>> > >> > > > >> > > > > for those partitions from the offset topic. If user
>> > >> stores
>> > >> > > > offset
>> > >> > > > >> > > > > externally, it might make sense for user to
>> similarly
>> > >> remove
>> > >> > > > >> offsets
>> > >> > > > >> > of
>> > >> > > > >> > > > > related partitions after these partitions are
>> deleted.
>> > >> So I
>> > >> > am
>> > >> > > > not
>> > >> > > > >> > sure
>> > >> > > > >> > > > > that we should use partition_epoch in this KIP.
>> > >> > > > >> > > > >
>> > >> > > > >> > > > >
>> > >> > > > >> > > > >>
>> > >> > > > >> > > > >> 61. It seems that the leader epoch returned in the
>> > >> > position()
>> > >> > > > >> call
>> > >> > > > >> > > > should
>> > >> > > > >> > > > >> the the leader epoch returned in the fetch
>> response,
>> > not
>> > >> > the
>> > >> > > > one
>> > >> > > > >> in
>> > >> > > > >> > > the
>> > >> > > > >> > > > >> metadata cache of the client.
>> > >> > > > >> > > > >
>> > >> > > > >> > > > >
>> > >> > > > >> > > > > I think this is a good idea. Just to double check,
>> this
>> > >> > change
>> > >> > > > >> does
>> > >> > > > >> > not
>> > >> > > > >> > > > > affect the correctness or performance of this KIP.
>> But
>> > it
>> > >> > can
>> > >> > > be
>> > >> > > > >> > useful
>> > >> > > > >> > > > if
>> > >> > > > >> > > > > we want to use the leader_epoch to better handle the
>> > >> offset
>> > >> > > rest
>> > >> > > > >> in
>> > >> > > > >> > > case
>> > >> > > > >> > > > of
>> > >> > > > >> > > > > unclean leader election, which is listed in the
>> future
>> > >> work.
>> > >> > > Is
>> > >> > > > >> this
>> > >> > > > >> > > > > understanding correct?
>> > >> > > > >> > > > >
>> > >> > > > >> > > > > I have updated the KIP to specify that the
>> leader_epoch
>> > >> > > returned
>> > >> > > > >> by
>> > >> > > > >> > > > > position() should be the largest leader_epoch of
>> those
>> > >> > already
>> > >> > > > >> > consumed
>> > >> > > > >> > > > > messages whose offset < position. If no message has
>> > been
>> > >> > > > consumed
>> > >> > > > >> > since
>> > >> > > > >> > > > > consumer initialization, the leader_epoch from
>> seek()
>> > or
>> > >> > > > >> > > > > OffsetFetchResponse should be used. The offset
>> included
>> > >> in
>> > >> > the
>> > >> > > > >> > > > > OffsetCommitRequest will also be determined in the
>> > >> similar
>> > >> > > > manner.
>> > >> > > > >> > > > >
>> > >> > > > >> > > > >
>> > >> > > > >> > > > >>
>> > >> > > > >> > > > >> 62. I am wondering if we should return the
>> partition
>> > >> epoch
>> > >> > in
>> > >> > > > the
>> > >> > > > >> > > fetch
>> > >> > > > >> > > > >> response as well. In the current proposal, if a
>> topic
>> > is
>> > >> > > > >> recreated
>> > >> > > > >> > and
>> > >> > > > >> > > > the
>> > >> > > > >> > > > >> new leader is on the same broker as the old one,
>> there
>> > >> is
>> > >> > > > >> nothing to
>> > >> > > > >> > > > force
>> > >> > > > >> > > > >> the metadata refresh in the client. So, the client
>> may
>> > >> > still
>> > >> > > > >> > associate
>> > >> > > > >> > > > the
>> > >> > > > >> > > > >> offset with the old partition epoch.
>> > >> > > > >> > > > >>
>> > >> > > > >> > > > >
>> > >> > > > >> > > > > Could you help me understand the problem if a client
>> > >> > > associates
>> > >> > > > >> old
>> > >> > > > >> > > > > partition_epoch (or the topic_epoch as of the
>> current
>> > >> KIP)
>> > >> > > with
>> > >> > > > >> the
>> > >> > > > >> > > > offset?
>> > >> > > > >> > > > > The main purpose of the topic_epoch is to be able to
>> > drop
>> > >> > > > >> > leader_epoch
>> > >> > > > >> > > > to 0
>> > >> > > > >> > > > > after a partition is deleted and re-created. I guess
>> > you
>> > >> may
>> > >> > > be
>> > >> > > > >> > > thinking
>> > >> > > > >> > > > > about using the partition_epoch to detect that the
>> > >> committed
>> > >> > > > >> offset
>> > >> > > > >> > is
>> > >> > > > >> > > > > invalid? In that case, I am wondering if the
>> > alternative
>> > >> > > > approach
>> > >> > > > >> > > > described
>> > >> > > > >> > > > > in 60) would be reasonable.
>> > >> > > > >> > > > >
>> > >> > > > >> > > > >
>> > >> > > > >> > > > >>
>> > >> > > > >> > > > >> 63. There is some subtle coordination between the
>> > >> > > > >> > LeaderAndIsrRequest
>> > >> > > > >> > > > and
>> > >> > > > >> > > > >> UpdateMetadataRequest. Currently, when a leader
>> > changes,
>> > >> > the
>> > >> > > > >> > > controller
>> > >> > > > >> > > > >> first sends the LeaderAndIsrRequest to the assigned
>> > >> > replicas
>> > >> > > > and
>> > >> > > > >> the
>> > >> > > > >> > > > >> UpdateMetadataRequest to every broker. So, there
>> could
>> > >> be a
>> > >> > > > small
>> > >> > > > >> > > window
>> > >> > > > >> > > > >> when the leader already receives the new partition
>> > >> epoch in
>> > >> > > the
>> > >> > > > >> > > > >> LeaderAndIsrRequest, but the metadata cache in the
>> > >> broker
>> > >> > > > hasn't
>> > >> > > > >> > been
>> > >> > > > >> > > > >> updated with the latest partition epoch. Not sure
>> > what's
>> > >> > the
>> > >> > > > best
>> > >> > > > >> > way
>> > >> > > > >> > > to
>> > >> > > > >> > > > >> address this issue. Perhaps we can update the
>> metadata
>> > >> > cache
>> > >> > > on
>> > >> > > > >> the
>> > >> > > > >> > > > broker
>> > >> > > > >> > > > >> with both LeaderAndIsrRequest and
>> > UpdateMetadataRequest.
>> > >> > The
>> > >> > > > >> > challenge
>> > >> > > > >> > > > is
>> > >> > > > >> > > > >> that the two have slightly different data. For
>> > example,
>> > >> > only
>> > >> > > > the
>> > >> > > > >> > > latter
>> > >> > > > >> > > > >> has
>> > >> > > > >> > > > >> all endpoints.
>> > >> > > > >> > > > >>
>> > >> > > > >> > > > >
>> > >> > > > >> > > > > I am not sure whether this is a problem. Could you
>> > >> explain a
>> > >> > > bit
>> > >> > > > >> more
>> > >> > > > >> > > > what
>> > >> > > > >> > > > > specific problem this small window can cause?
>> > >> > > > >> > > > >
>> > >> > > > >> > > > > Since client can fetch metadata from any broker in
>> the
>> > >> > > cluster,
>> > >> > > > >> and
>> > >> > > > >> > > given
>> > >> > > > >> > > > > that different brokers receive request (e.g.
>> > >> > > LeaderAndIsrRequest
>> > >> > > > >> and
>> > >> > > > >> > > > > UpdateMetadataRequest) in arbitrary order, the
>> metadata
>> > >> > > received
>> > >> > > > >> by
>> > >> > > > >> > > > client
>> > >> > > > >> > > > > can be in arbitrary order (either newer or older)
>> > >> compared
>> > >> > to
>> > >> > > > the
>> > >> > > > >> > > > broker's
>> > >> > > > >> > > > > leadership state even if a given broker receives
>> > >> > > > >> LeaderAndIsrRequest
>> > >> > > > >> > > and
>> > >> > > > >> > > > > UpdateMetadataRequest simultaneously. So I am not
>> sure
>> > >> it is
>> > >> > > > >> useful
>> > >> > > > >> > to
>> > >> > > > >> > > > > update broker's cache with LeaderAndIsrRequest.
>> > >> > > > >> > > > >
>> > >> > > > >> > > > >
>> > >> > > > >> > > > >> 64. The enforcement of leader epoch in Offset
>> commit:
>> > We
>> > >> > > allow
>> > >> > > > a
>> > >> > > > >> > > > consumer
>> > >> > > > >> > > > >> to set an arbitrary offset. So it's possible for
>> > >> offsets or
>> > >> > > > >> leader
>> > >> > > > >> > > epoch
>> > >> > > > >> > > > >> to
>> > >> > > > >> > > > >> go backwards. I am not sure if we could always
>> enforce
>> > >> that
>> > >> > > the
>> > >> > > > >> > leader
>> > >> > > > >> > > > >> epoch only goes up on the broker.
>> > >> > > > >> > > > >>
>> > >> > > > >> > > > >
>> > >> > > > >> > > > > Sure. I have removed this check from the KIP.
>> > >> > > > >> > > > >
>> > >> > > > >> > > > > BTW, we can probably still ensure that the
>> leader_epoch
>> > >> > always
>> > >> > > > >> > increase
>> > >> > > > >> > > > if
>> > >> > > > >> > > > > the leader_epoch used with offset commit is the
>> > >> > > max(leader_epoch
>> > >> > > > >> of
>> > >> > > > >> > the
>> > >> > > > >> > > > > message with offset = the committed offset - 1, the
>> > >> largest
>> > >> > > > known
>> > >> > > > >> > > > > leader_epoch from the metadata). But I don't have a
>> > good
>> > >> > > > use-case
>> > >> > > > >> for
>> > >> > > > >> > > > this
>> > >> > > > >> > > > > alternative definition. So I choose the keep the KIP
>> > >> simple
>> > >> > by
>> > >> > > > >> > > requiring
>> > >> > > > >> > > > > leader_epoch to always increase.
>> > >> > > > >> > > > >
>> > >> > > > >> > > > >
>> > >> > > > >> > > > >> 65. Good point on handling missing partition epoch
>> due
>> > >> to
>> > >> > > topic
>> > >> > > > >> > > > deletion.
>> > >> > > > >> > > > >> Another potential way to address this is to
>> > additionally
>> > >> > > > >> propagate
>> > >> > > > >> > the
>> > >> > > > >> > > > >> global partition epoch to brokers and the clients.
>> > This
>> > >> > way,
>> > >> > > > >> when a
>> > >> > > > >> > > > >> partition epoch is missing, we can use the global
>> > >> partition
>> > >> > > > >> epoch to
>> > >> > > > >> > > > >> reason
>> > >> > > > >> > > > >> about which metadata is more recent.
>> > >> > > > >> > > > >>
>> > >> > > > >> > > > >
>> > >> > > > >> > > > > This is a great idea. The global epoch can be used
>> to
>> > >> order
>> > >> > > the
>> > >> > > > >> > > metadata
>> > >> > > > >> > > > > and help us recognize the more recent metadata if a
>> > topic
>> > >> > (or
>> > >> > > > >> > > partition)
>> > >> > > > >> > > > is
>> > >> > > > >> > > > > deleted and re-created.
>> > >> > > > >> > > > >
>> > >> > > > >> > > > > Actually, it seems we only need to propagate the
>> global
>> > >> > epoch
>> > >> > > to
>> > >> > > > >> > > brokers
>> > >> > > > >> > > > > and clients without propagating this epoch on a
>> > >> per-topic or
>> > >> > > > >> > > > per-partition
>> > >> > > > >> > > > > basic. Doing so would simply interface changes made
>> > this
>> > >> > KIP.
>> > >> > > > Does
>> > >> > > > >> > this
>> > >> > > > >> > > > > approach sound reasonable?
>> > >> > > > >> > > > >
>> > >> > > > >> > > > >
>> > >> > > > >> > > > >> 66. A client may also get an offset by time using
>> the
>> > >> > > > >> > offsetForTimes()
>> > >> > > > >> > > > >> api.
>> > >> > > > >> > > > >> So, we probably want to include
>> offsetInternalMetadata
>> > >> in
>> > >> > > > >> > > > >> OffsetAndTimestamp
>> > >> > > > >> > > > >> as well.
>> > >> > > > >> > > > >>
>> > >> > > > >> > > > >
>> > >> > > > >> > > > > You are right. This probably also requires us to
>> change
>> > >> the
>> > >> > > > >> > > > > ListOffsetRequest as well. I will update the KIP
>> after
>> > we
>> > >> > > agree
>> > >> > > > on
>> > >> > > > >> > the
>> > >> > > > >> > > > > solution for 65).
>> > >> > > > >> > > > >
>> > >> > > > >> > > > >
>> > >> > > > >> > > > >>
>> > >> > > > >> > > > >> 67. InteralMetadata can be a bit confusing with the
>> > >> > metadata
>> > >> > > > >> field
>> > >> > > > >> > > > already
>> > >> > > > >> > > > >> there. Perhaps we can just call it OffsetEpoch. It
>> > >> might be
>> > >> > > > >> useful
>> > >> > > > >> > to
>> > >> > > > >> > > > make
>> > >> > > > >> > > > >> OffsetEpoch printable at least for debugging
>> purpose.
>> > >> Once
>> > >> > > you
>> > >> > > > do
>> > >> > > > >> > > that,
>> > >> > > > >> > > > we
>> > >> > > > >> > > > >> are already exposing the internal fields. So, not
>> sure
>> > >> if
>> > >> > > it's
>> > >> > > > >> worth
>> > >> > > > >> > > > >> hiding
>> > >> > > > >> > > > >> them. If we do want to hide them, perhaps we can
>> have
>> > >> sth
>> > >> > > like
>> > >> > > > >> the
>> > >> > > > >> > > > >> following. The binary encoding is probably more
>> > >> efficient
>> > >> > > than
>> > >> > > > >> JSON
>> > >> > > > >> > > for
>> > >> > > > >> > > > >> external storage.
>> > >> > > > >> > > > >>
>> > >> > > > >> > > > >> OffsetEpoch {
>> > >> > > > >> > > > >>  static OffsetEpoch decode(byte[]);
>> > >> > > > >> > > > >>
>> > >> > > > >> > > > >>   public byte[] encode();
>> > >> > > > >> > > > >>
>> > >> > > > >> > > > >>   public String toString();
>> > >> > > > >> > > > >> }
>> > >> > > > >> > > > >>
>> > >> > > > >> > > > >
>> > >> > > > >> > > > > Thanks much. I like this solution. I have updated
>> the
>> > KIP
>> > >> > > > >> > accordingly.
>> > >> > > > >> > > > >
>> > >> > > > >> > > > >
>> > >> > > > >> > > > >
>> > >> > > > >> > > > >>
>> > >> > > > >> > > > >> Jun
>> > >> > > > >> > > > >>
>> > >> > > > >> > > > >> On Mon, Jan 8, 2018 at 4:22 PM, Dong Lin <
>> > >> > > lindong28@gmail.com>
>> > >> > > > >> > wrote:
>> > >> > > > >> > > > >>
>> > >> > > > >> > > > >> > Hey Jason,
>> > >> > > > >> > > > >> >
>> > >> > > > >> > > > >> > Certainly. This sounds good. I have updated the
>> KIP
>> > to
>> > >> > > > clarity
>> > >> > > > >> > that
>> > >> > > > >> > > > the
>> > >> > > > >> > > > >> > global epoch will be incremented by 1 each time a
>> > >> topic
>> > >> > is
>> > >> > > > >> > deleted.
>> > >> > > > >> > > > >> >
>> > >> > > > >> > > > >> > Thanks,
>> > >> > > > >> > > > >> > Dong
>> > >> > > > >> > > > >> >
>> > >> > > > >> > > > >> > On Mon, Jan 8, 2018 at 4:09 PM, Jason Gustafson <
>> > >> > > > >> > jason@confluent.io
>> > >> > > > >> > > >
>> > >> > > > >> > > > >> > wrote:
>> > >> > > > >> > > > >> >
>> > >> > > > >> > > > >> > > Hi Dong,
>> > >> > > > >> > > > >> > >
>> > >> > > > >> > > > >> > >
>> > >> > > > >> > > > >> > > I think your approach will allow user to
>> > distinguish
>> > >> > > > between
>> > >> > > > >> the
>> > >> > > > >> > > > >> metadata
>> > >> > > > >> > > > >> > > > before and after the topic deletion. I also
>> > agree
>> > >> > that
>> > >> > > > this
>> > >> > > > >> > can
>> > >> > > > >> > > be
>> > >> > > > >> > > > >> > > > potentially be useful to user. I am just not
>> > very
>> > >> > sure
>> > >> > > > >> whether
>> > >> > > > >> > > we
>> > >> > > > >> > > > >> > already
>> > >> > > > >> > > > >> > > > have a good use-case to make the additional
>> > >> > complexity
>> > >> > > > >> > > worthwhile.
>> > >> > > > >> > > > >> It
>> > >> > > > >> > > > >> > > seems
>> > >> > > > >> > > > >> > > > that this feature is kind of independent of
>> the
>> > >> main
>> > >> > > > >> problem
>> > >> > > > >> > of
>> > >> > > > >> > > > this
>> > >> > > > >> > > > >> > KIP.
>> > >> > > > >> > > > >> > > > Could we add this as a future work?
>> > >> > > > >> > > > >> > >
>> > >> > > > >> > > > >> > >
>> > >> > > > >> > > > >> > > Do you think it's fair if we bump the topic
>> epoch
>> > on
>> > >> > > > deletion
>> > >> > > > >> > and
>> > >> > > > >> > > > >> leave
>> > >> > > > >> > > > >> > > propagation of the epoch for deleted topics for
>> > >> future
>> > >> > > > work?
>> > >> > > > >> I
>> > >> > > > >> > > don't
>> > >> > > > >> > > > >> > think
>> > >> > > > >> > > > >> > > this adds much complexity and it makes the
>> > behavior
>> > >> > > > >> consistent:
>> > >> > > > >> > > > every
>> > >> > > > >> > > > >> > topic
>> > >> > > > >> > > > >> > > mutation results in an epoch bump.
>> > >> > > > >> > > > >> > >
>> > >> > > > >> > > > >> > > Thanks,
>> > >> > > > >> > > > >> > > Jason
>> > >> > > > >> > > > >> > >
>> > >> > > > >> > > > >> > > On Mon, Jan 8, 2018 at 3:14 PM, Dong Lin <
>> > >> > > > >> lindong28@gmail.com>
>> > >> > > > >> > > > wrote:
>> > >> > > > >> > > > >> > >
>> > >> > > > >> > > > >> > > > Hey Ismael,
>> > >> > > > >> > > > >> > > >
>> > >> > > > >> > > > >> > > > I guess we actually need user to see this
>> field
>> > so
>> > >> > that
>> > >> > > > >> user
>> > >> > > > >> > can
>> > >> > > > >> > > > >> store
>> > >> > > > >> > > > >> > > this
>> > >> > > > >> > > > >> > > > value in the external store together with the
>> > >> offset.
>> > >> > > We
>> > >> > > > >> just
>> > >> > > > >> > > > prefer
>> > >> > > > >> > > > >> > the
>> > >> > > > >> > > > >> > > > value to be opaque to discourage most users
>> from
>> > >> > > > >> interpreting
>> > >> > > > >> > > this
>> > >> > > > >> > > > >> > value.
>> > >> > > > >> > > > >> > > > One more advantage of using such an opaque
>> field
>> > >> is
>> > >> > to
>> > >> > > be
>> > >> > > > >> able
>> > >> > > > >> > > to
>> > >> > > > >> > > > >> > evolve
>> > >> > > > >> > > > >> > > > the information (or schema) of this value
>> > without
>> > >> > > > changing
>> > >> > > > >> > > > consumer
>> > >> > > > >> > > > >> API
>> > >> > > > >> > > > >> > > in
>> > >> > > > >> > > > >> > > > the future.
>> > >> > > > >> > > > >> > > >
>> > >> > > > >> > > > >> > > > I also thinking it is probably OK for user
>> to be
>> > >> able
>> > >> > > to
>> > >> > > > >> > > interpret
>> > >> > > > >> > > > >> this
>> > >> > > > >> > > > >> > > > value, particularly for those advanced users.
>> > >> > > > >> > > > >> > > >
>> > >> > > > >> > > > >> > > > Thanks,
>> > >> > > > >> > > > >> > > > Dong
>> > >> > > > >> > > > >> > > >
>> > >> > > > >> > > > >> > > > On Mon, Jan 8, 2018 at 2:34 PM, Ismael Juma <
>> > >> > > > >> > ismael@juma.me.uk>
>> > >> > > > >> > > > >> wrote:
>> > >> > > > >> > > > >> > > >
>> > >> > > > >> > > > >> > > > > On Fri, Jan 5, 2018 at 7:15 PM, Jason
>> > Gustafson
>> > >> <
>> > >> > > > >> > > > >> jason@confluent.io>
>> > >> > > > >> > > > >> > > > > wrote:
>> > >> > > > >> > > > >> > > > > >
>> > >> > > > >> > > > >> > > > > > class OffsetAndMetadata {
>> > >> > > > >> > > > >> > > > > >   long offset;
>> > >> > > > >> > > > >> > > > > >   byte[] offsetMetadata;
>> > >> > > > >> > > > >> > > > > >   String metadata;
>> > >> > > > >> > > > >> > > > > > }
>> > >> > > > >> > > > >> > > > >
>> > >> > > > >> > > > >> > > > >
>> > >> > > > >> > > > >> > > > > > Admittedly, the naming is a bit annoying,
>> > but
>> > >> we
>> > >> > > can
>> > >> > > > >> > > probably
>> > >> > > > >> > > > >> come
>> > >> > > > >> > > > >> > up
>> > >> > > > >> > > > >> > > > > with
>> > >> > > > >> > > > >> > > > > > something better. Internally the byte
>> array
>> > >> would
>> > >> > > > have
>> > >> > > > >> a
>> > >> > > > >> > > > >> version.
>> > >> > > > >> > > > >> > If
>> > >> > > > >> > > > >> > > in
>> > >> > > > >> > > > >> > > > > the
>> > >> > > > >> > > > >> > > > > > future we have anything else we need to
>> add,
>> > >> we
>> > >> > can
>> > >> > > > >> update
>> > >> > > > >> > > the
>> > >> > > > >> > > > >> > > version
>> > >> > > > >> > > > >> > > > > and
>> > >> > > > >> > > > >> > > > > > we wouldn't need any new APIs.
>> > >> > > > >> > > > >> > > > > >
>> > >> > > > >> > > > >> > > > >
>> > >> > > > >> > > > >> > > > > We can also add fields to a class in a
>> > >> compatible
>> > >> > > way.
>> > >> > > > >> So,
>> > >> > > > >> > it
>> > >> > > > >> > > > >> seems
>> > >> > > > >> > > > >> > to
>> > >> > > > >> > > > >> > > me
>> > >> > > > >> > > > >> > > > > that the main advantage of the byte array
>> is
>> > >> that
>> > >> > > it's
>> > >> > > > >> > opaque
>> > >> > > > >> > > to
>> > >> > > > >> > > > >> the
>> > >> > > > >> > > > >> > > > user.
>> > >> > > > >> > > > >> > > > > Is that correct? If so, we could also add
>> any
>> > >> > opaque
>> > >> > > > >> > metadata
>> > >> > > > >> > > > in a
>> > >> > > > >> > > > >> > > > subclass
>> > >> > > > >> > > > >> > > > > so that users don't even see it (unless
>> they
>> > >> cast
>> > >> > it,
>> > >> > > > but
>> > >> > > > >> > then
>> > >> > > > >> > > > >> > they're
>> > >> > > > >> > > > >> > > on
>> > >> > > > >> > > > >> > > > > their own).
>> > >> > > > >> > > > >> > > > >
>> > >> > > > >> > > > >> > > > > Ismael
>> > >> > > > >> > > > >> > > > >
>> > >> > > > >> > > > >> > > > > The corresponding seek() and position()
>> APIs
>> > >> might
>> > >> > > look
>> > >> > > > >> > > > something
>> > >> > > > >> > > > >> > like
>> > >> > > > >> > > > >> > > > > this:
>> > >> > > > >> > > > >> > > > > >
>> > >> > > > >> > > > >> > > > > > void seek(TopicPartition partition, long
>> > >> offset,
>> > >> > > > byte[]
>> > >> > > > >> > > > >> > > > offsetMetadata);
>> > >> > > > >> > > > >> > > > > > byte[] positionMetadata(TopicPartition
>> > >> > partition);
>> > >> > > > >> > > > >> > > > > >
>> > >> > > > >> > > > >> > > > > > What do you think?
>> > >> > > > >> > > > >> > > > > >
>> > >> > > > >> > > > >> > > > > > Thanks,
>> > >> > > > >> > > > >> > > > > > Jason
>> > >> > > > >> > > > >> > > > > >
>> > >> > > > >> > > > >> > > > > > On Thu, Jan 4, 2018 at 7:04 PM, Dong Lin
>> <
>> > >> > > > >> > > lindong28@gmail.com
>> > >> > > > >> > > > >
>> > >> > > > >> > > > >> > > wrote:
>> > >> > > > >> > > > >> > > > > >
>> > >> > > > >> > > > >> > > > > > > Hey Jun, Jason,
>> > >> > > > >> > > > >> > > > > > >
>> > >> > > > >> > > > >> > > > > > > Thanks much for all the feedback. I
>> have
>> > >> > updated
>> > >> > > > the
>> > >> > > > >> KIP
>> > >> > > > >> > > > >> based on
>> > >> > > > >> > > > >> > > the
>> > >> > > > >> > > > >> > > > > > > latest discussion. Can you help check
>> > >> whether
>> > >> > it
>> > >> > > > >> looks
>> > >> > > > >> > > good?
>> > >> > > > >> > > > >> > > > > > >
>> > >> > > > >> > > > >> > > > > > > Thanks,
>> > >> > > > >> > > > >> > > > > > > Dong
>> > >> > > > >> > > > >> > > > > > >
>> > >> > > > >> > > > >> > > > > > > On Thu, Jan 4, 2018 at 5:36 PM, Dong
>> Lin <
>> > >> > > > >> > > > lindong28@gmail.com
>> > >> > > > >> > > > >> >
>> > >> > > > >> > > > >> > > > wrote:
>> > >> > > > >> > > > >> > > > > > >
>> > >> > > > >> > > > >> > > > > > > > Hey Jun,
>> > >> > > > >> > > > >> > > > > > > >
>> > >> > > > >> > > > >> > > > > > > > Hmm... thinking about this more, I am
>> > not
>> > >> > sure
>> > >> > > > that
>> > >> > > > >> > the
>> > >> > > > >> > > > >> > proposed
>> > >> > > > >> > > > >> > > > API
>> > >> > > > >> > > > >> > > > > is
>> > >> > > > >> > > > >> > > > > > > > sufficient. For users that store
>> offset
>> > >> > > > >> externally, we
>> > >> > > > >> > > > >> probably
>> > >> > > > >> > > > >> > > > need
>> > >> > > > >> > > > >> > > > > > > extra
>> > >> > > > >> > > > >> > > > > > > > API to return the leader_epoch and
>> > >> > > > partition_epoch
>> > >> > > > >> for
>> > >> > > > >> > > all
>> > >> > > > >> > > > >> > > > partitions
>> > >> > > > >> > > > >> > > > > > > that
>> > >> > > > >> > > > >> > > > > > > > consumers are consuming. I suppose
>> these
>> > >> > users
>> > >> > > > >> > currently
>> > >> > > > >> > > > use
>> > >> > > > >> > > > >> > > > > position()
>> > >> > > > >> > > > >> > > > > > > to
>> > >> > > > >> > > > >> > > > > > > > get the offset. Thus we probably
>> need a
>> > >> new
>> > >> > > > method
>> > >> > > > >> > > > >> > > > > > positionWithEpoch(..)
>> > >> > > > >> > > > >> > > > > > > to
>> > >> > > > >> > > > >> > > > > > > > return <offset, partition_epoch,
>> > >> > leader_epoch>.
>> > >> > > > >> Does
>> > >> > > > >> > > this
>> > >> > > > >> > > > >> sound
>> > >> > > > >> > > > >> > > > > > > reasonable?
>> > >> > > > >> > > > >> > > > > > > >
>> > >> > > > >> > > > >> > > > > > > > Thanks,
>> > >> > > > >> > > > >> > > > > > > > Dong
>> > >> > > > >> > > > >> > > > > > > >
>> > >> > > > >> > > > >> > > > > > > >
>> > >> > > > >> > > > >> > > > > > > > On Thu, Jan 4, 2018 at 5:26 PM, Jun
>> Rao
>> > <
>> > >> > > > >> > > jun@confluent.io
>> > >> > > > >> > > > >
>> > >> > > > >> > > > >> > > wrote:
>> > >> > > > >> > > > >> > > > > > > >
>> > >> > > > >> > > > >> > > > > > > >> Hi, Dong,
>> > >> > > > >> > > > >> > > > > > > >>
>> > >> > > > >> > > > >> > > > > > > >> Yes, that's what I am thinking.
>> > >> OffsetEpoch
>> > >> > > will
>> > >> > > > >> be
>> > >> > > > >> > > > >> composed
>> > >> > > > >> > > > >> > of
>> > >> > > > >> > > > >> > > > > > > >> (partition_epoch,
>> > >> > > > >> > > > >> > > > > > > >> leader_epoch).
>> > >> > > > >> > > > >> > > > > > > >>
>> > >> > > > >> > > > >> > > > > > > >> Thanks,
>> > >> > > > >> > > > >> > > > > > > >>
>> > >> > > > >> > > > >> > > > > > > >> Jun
>> > >> > > > >> > > > >> > > > > > > >>
>> > >> > > > >> > > > >> > > > > > > >>
>> > >> > > > >> > > > >> > > > > > > >> On Thu, Jan 4, 2018 at 4:22 PM, Dong
>> > Lin
>> > >> <
>> > >> > > > >> > > > >> lindong28@gmail.com
>> > >> > > > >> > > > >> > >
>> > >> > > > >> > > > >> > > > > wrote:
>> > >> > > > >> > > > >> > > > > > > >>
>> > >> > > > >> > > > >> > > > > > > >> > Hey Jun,
>> > >> > > > >> > > > >> > > > > > > >> >
>> > >> > > > >> > > > >> > > > > > > >> > Thanks much. I like the the new
>> API
>> > >> that
>> > >> > you
>> > >> > > > >> > > proposed.
>> > >> > > > >> > > > I
>> > >> > > > >> > > > >> am
>> > >> > > > >> > > > >> > > not
>> > >> > > > >> > > > >> > > > > sure
>> > >> > > > >> > > > >> > > > > > > >> what
>> > >> > > > >> > > > >> > > > > > > >> > you exactly mean by offset_epoch.
>> I
>> > >> > suppose
>> > >> > > > >> that we
>> > >> > > > >> > > can
>> > >> > > > >> > > > >> use
>> > >> > > > >> > > > >> > > the
>> > >> > > > >> > > > >> > > > > pair
>> > >> > > > >> > > > >> > > > > > > of
>> > >> > > > >> > > > >> > > > > > > >> > (partition_epoch, leader_epoch) as
>> > the
>> > >> > > > >> > offset_epoch,
>> > >> > > > >> > > > >> right?
>> > >> > > > >> > > > >> > > > > > > >> >
>> > >> > > > >> > > > >> > > > > > > >> > Thanks,
>> > >> > > > >> > > > >> > > > > > > >> > Dong
>> > >> > > > >> > > > >> > > > > > > >> >
>> > >> > > > >> > > > >> > > > > > > >> > On Thu, Jan 4, 2018 at 4:02 PM,
>> Jun
>> > >> Rao <
>> > >> > > > >> > > > >> jun@confluent.io>
>> > >> > > > >> > > > >> > > > wrote:
>> > >> > > > >> > > > >> > > > > > > >> >
>> > >> > > > >> > > > >> > > > > > > >> > > Hi, Dong,
>> > >> > > > >> > > > >> > > > > > > >> > >
>> > >> > > > >> > > > >> > > > > > > >> > > Got it. The api that you
>> proposed
>> > >> works.
>> > >> > > The
>> > >> > > > >> > > question
>> > >> > > > >> > > > >> is
>> > >> > > > >> > > > >> > > > whether
>> > >> > > > >> > > > >> > > > > > > >> that's
>> > >> > > > >> > > > >> > > > > > > >> > the
>> > >> > > > >> > > > >> > > > > > > >> > > api that we want to have in the
>> > long
>> > >> > term.
>> > >> > > > My
>> > >> > > > >> > > concern
>> > >> > > > >> > > > >> is
>> > >> > > > >> > > > >> > > that
>> > >> > > > >> > > > >> > > > > > while
>> > >> > > > >> > > > >> > > > > > > >> the
>> > >> > > > >> > > > >> > > > > > > >> > api
>> > >> > > > >> > > > >> > > > > > > >> > > change is simple, the new api
>> seems
>> > >> > harder
>> > >> > > > to
>> > >> > > > >> > > explain
>> > >> > > > >> > > > >> and
>> > >> > > > >> > > > >> > > use.
>> > >> > > > >> > > > >> > > > > For
>> > >> > > > >> > > > >> > > > > > > >> > example,
>> > >> > > > >> > > > >> > > > > > > >> > > a consumer storing offsets
>> > externally
>> > >> > now
>> > >> > > > >> needs
>> > >> > > > >> > to
>> > >> > > > >> > > > call
>> > >> > > > >> > > > >> > > > > > > >> > > waitForMetadataUpdate() after
>> > calling
>> > >> > > > seek().
>> > >> > > > >> > > > >> > > > > > > >> > >
>> > >> > > > >> > > > >> > > > > > > >> > > An alternative approach is to
>> make
>> > >> the
>> > >> > > > >> following
>> > >> > > > >> > > > >> > compatible
>> > >> > > > >> > > > >> > > > api
>> > >> > > > >> > > > >> > > > > > > >> changes
>> > >> > > > >> > > > >> > > > > > > >> > in
>> > >> > > > >> > > > >> > > > > > > >> > > Consumer.
>> > >> > > > >> > > > >> > > > > > > >> > > * Add an additional OffsetEpoch
>> > >> field in
>> > >> > > > >> > > > >> > OffsetAndMetadata.
>> > >> > > > >> > > > >> > > > (no
>> > >> > > > >> > > > >> > > > > > need
>> > >> > > > >> > > > >> > > > > > > >> to
>> > >> > > > >> > > > >> > > > > > > >> > > change the CommitSync() api)
>> > >> > > > >> > > > >> > > > > > > >> > > * Add a new api
>> seek(TopicPartition
>> > >> > > > partition,
>> > >> > > > >> > long
>> > >> > > > >> > > > >> > offset,
>> > >> > > > >> > > > >> > > > > > > >> OffsetEpoch
>> > >> > > > >> > > > >> > > > > > > >> > > offsetEpoch). We can potentially
>> > >> > deprecate
>> > >> > > > the
>> > >> > > > >> > old
>> > >> > > > >> > > > api
>> > >> > > > >> > > > >> > > > > > > >> > seek(TopicPartition
>> > >> > > > >> > > > >> > > > > > > >> > > partition, long offset) in the
>> > >> future.
>> > >> > > > >> > > > >> > > > > > > >> > >
>> > >> > > > >> > > > >> > > > > > > >> > > The alternative approach has
>> > similar
>> > >> > > amount
>> > >> > > > of
>> > >> > > > >> > api
>> > >> > > > >> > > > >> changes
>> > >> > > > >> > > > >> > > as
>> > >> > > > >> > > > >> > > > > > yours
>> > >> > > > >> > > > >> > > > > > > >> but
>> > >> > > > >> > > > >> > > > > > > >> > has
>> > >> > > > >> > > > >> > > > > > > >> > > the following benefits.
>> > >> > > > >> > > > >> > > > > > > >> > > 1. The api works in a similar
>> way
>> > as
>> > >> how
>> > >> > > > >> offset
>> > >> > > > >> > > > >> management
>> > >> > > > >> > > > >> > > > works
>> > >> > > > >> > > > >> > > > > > now
>> > >> > > > >> > > > >> > > > > > > >> and
>> > >> > > > >> > > > >> > > > > > > >> > is
>> > >> > > > >> > > > >> > > > > > > >> > > probably what we want in the
>> long
>> > >> term.
>> > >> > > > >> > > > >> > > > > > > >> > > 2. It can reset offsets better
>> when
>> > >> > there
>> > >> > > is
>> > >> > > > >> data
>> > >> > > > >> > > > loss
>> > >> > > > >> > > > >> due
>> > >> > > > >> > > > >> > > to
>> > >> > > > >> > > > >> > > > > > > unclean
>> > >> > > > >> > > > >> > > > > > > >> > > leader election or correlated
>> > replica
>> > >> > > > failure.
>> > >> > > > >> > > > >> > > > > > > >> > > 3. It can reset offsets better
>> when
>> > >> > topic
>> > >> > > is
>> > >> > > > >> > > > recreated.
>> > >> > > > >> > > > >> > > > > > > >> > >
>> > >> > > > >> > > > >> > > > > > > >> > > Thanks,
>> > >> > > > >> > > > >> > > > > > > >> > >
>> > >> > > > >> > > > >> > > > > > > >> > > Jun
>> > >> > > > >> > > > >> > > > > > > >> > >
>> > >> > > > >> > > > >> > > > > > > >> > >
>> > >> > > > >> > > > >> > > > > > > >> > > On Thu, Jan 4, 2018 at 2:05 PM,
>> > Dong
>> > >> > Lin <
>> > >> > > > >> > > > >> > > lindong28@gmail.com
>> > >> > > > >> > > > >> > > > >
>> > >> > > > >> > > > >> > > > > > > wrote:
>> > >> > > > >> > > > >> > > > > > > >> > >
>> > >> > > > >> > > > >> > > > > > > >> > > > Hey Jun,
>> > >> > > > >> > > > >> > > > > > > >> > > >
>> > >> > > > >> > > > >> > > > > > > >> > > > Yeah I agree that ideally we
>> > don't
>> > >> > want
>> > >> > > an
>> > >> > > > >> ever
>> > >> > > > >> > > > >> growing
>> > >> > > > >> > > > >> > > > global
>> > >> > > > >> > > > >> > > > > > > >> metadata
>> > >> > > > >> > > > >> > > > > > > >> > > > version. I just think it may
>> be
>> > >> more
>> > >> > > > >> desirable
>> > >> > > > >> > to
>> > >> > > > >> > > > >> keep
>> > >> > > > >> > > > >> > the
>> > >> > > > >> > > > >> > > > > > > consumer
>> > >> > > > >> > > > >> > > > > > > >> API
>> > >> > > > >> > > > >> > > > > > > >> > > > simple.
>> > >> > > > >> > > > >> > > > > > > >> > > >
>> > >> > > > >> > > > >> > > > > > > >> > > > In my current proposal,
>> metadata
>> > >> > version
>> > >> > > > >> > returned
>> > >> > > > >> > > > in
>> > >> > > > >> > > > >> the
>> > >> > > > >> > > > >> > > > fetch
>> > >> > > > >> > > > >> > > > > > > >> response
>> > >> > > > >> > > > >> > > > > > > >> > > > will be stored with the offset
>> > >> > together.
>> > >> > > > >> More
>> > >> > > > >> > > > >> > > specifically,
>> > >> > > > >> > > > >> > > > > the
>> > >> > > > >> > > > >> > > > > > > >> > > > metadata_epoch in the new
>> offset
>> > >> topic
>> > >> > > > >> schema
>> > >> > > > >> > > will
>> > >> > > > >> > > > be
>> > >> > > > >> > > > >> > the
>> > >> > > > >> > > > >> > > > > > largest
>> > >> > > > >> > > > >> > > > > > > >> > > > metadata_epoch from all the
>> > >> > > > MetadataResponse
>> > >> > > > >> > and
>> > >> > > > >> > > > >> > > > FetchResponse
>> > >> > > > >> > > > >> > > > > > > ever
>> > >> > > > >> > > > >> > > > > > > >> > > > received by this consumer.
>> > >> > > > >> > > > >> > > > > > > >> > > >
>> > >> > > > >> > > > >> > > > > > > >> > > > We probably don't have to
>> change
>> > >> the
>> > >> > > > >> consumer
>> > >> > > > >> > API
>> > >> > > > >> > > > for
>> > >> > > > >> > > > >> > > > > > > >> > > > commitSync(Map<TopicPartition,
>> > >> > > > >> > > OffsetAndMetadata>).
>> > >> > > > >> > > > >> If
>> > >> > > > >> > > > >> > > user
>> > >> > > > >> > > > >> > > > > > calls
>> > >> > > > >> > > > >> > > > > > > >> > > > commitSync(...) to commit
>> offset
>> > 10
>> > >> > for
>> > >> > > a
>> > >> > > > >> given
>> > >> > > > >> > > > >> > partition,
>> > >> > > > >> > > > >> > > > for
>> > >> > > > >> > > > >> > > > > > > most
>> > >> > > > >> > > > >> > > > > > > >> > > > use-cases, this consumer
>> instance
>> > >> > should
>> > >> > > > >> have
>> > >> > > > >> > > > >> consumed
>> > >> > > > >> > > > >> > > > message
>> > >> > > > >> > > > >> > > > > > > with
>> > >> > > > >> > > > >> > > > > > > >> > > offset
>> > >> > > > >> > > > >> > > > > > > >> > > > 9 from this partition, in
>> which
>> > >> case
>> > >> > the
>> > >> > > > >> > consumer
>> > >> > > > >> > > > can
>> > >> > > > >> > > > >> > > > remember
>> > >> > > > >> > > > >> > > > > > and
>> > >> > > > >> > > > >> > > > > > > >> use
>> > >> > > > >> > > > >> > > > > > > >> > > the
>> > >> > > > >> > > > >> > > > > > > >> > > > metadata_epoch from the
>> > >> corresponding
>> > >> > > > >> > > FetchResponse
>> > >> > > > >> > > > >> when
>> > >> > > > >> > > > >> > > > > > > committing
>> > >> > > > >> > > > >> > > > > > > >> > > offset.
>> > >> > > > >> > > > >> > > > > > > >> > > > If user calls commitSync(..)
>> to
>> > >> commit
>> > >> > > > >> offset
>> > >> > > > >> > 10
>> > >> > > > >> > > > for
>> > >> > > > >> > > > >> a
>> > >> > > > >> > > > >> > > given
>> > >> > > > >> > > > >> > > > > > > >> partition
>> > >> > > > >> > > > >> > > > > > > >> > > > without having consumed the
>> > message
>> > >> > with
>> > >> > > > >> > offset 9
>> > >> > > > >> > > > >> using
>> > >> > > > >> > > > >> > > this
>> > >> > > > >> > > > >> > > > > > > >> consumer
>> > >> > > > >> > > > >> > > > > > > >> > > > instance, this is probably an
>> > >> advanced
>> > >> > > > >> > use-case.
>> > >> > > > >> > > In
>> > >> > > > >> > > > >> this
>> > >> > > > >> > > > >> > > > case
>> > >> > > > >> > > > >> > > > > > the
>> > >> > > > >> > > > >> > > > > > > >> > > advanced
>> > >> > > > >> > > > >> > > > > > > >> > > > user can retrieve the
>> > >> metadata_epoch
>> > >> > > using
>> > >> > > > >> the
>> > >> > > > >> > > > newly
>> > >> > > > >> > > > >> > added
>> > >> > > > >> > > > >> > > > > > > >> > > metadataEpoch()
>> > >> > > > >> > > > >> > > > > > > >> > > > API after it fetches the
>> message
>> > >> with
>> > >> > > > >> offset 9
>> > >> > > > >> > > > >> (probably
>> > >> > > > >> > > > >> > > > from
>> > >> > > > >> > > > >> > > > > > > >> another
>> > >> > > > >> > > > >> > > > > > > >> > > > consumer instance) and encode
>> > this
>> > >> > > > >> > metadata_epoch
>> > >> > > > >> > > > in
>> > >> > > > >> > > > >> the
>> > >> > > > >> > > > >> > > > > > > >> > > > string
>> > OffsetAndMetadata.metadata.
>> > >> Do
>> > >> > > you
>> > >> > > > >> think
>> > >> > > > >> > > > this
>> > >> > > > >> > > > >> > > > solution
>> > >> > > > >> > > > >> > > > > > > would
>> > >> > > > >> > > > >> > > > > > > >> > work?
>> > >> > > > >> > > > >> > > > > > > >> > > >
>> > >> > > > >> > > > >> > > > > > > >> > > > By "not sure that I fully
>> > >> understand
>> > >> > > your
>> > >> > > > >> > latest
>> > >> > > > >> > > > >> > > > suggestion",
>> > >> > > > >> > > > >> > > > > > are
>> > >> > > > >> > > > >> > > > > > > >> you
>> > >> > > > >> > > > >> > > > > > > >> > > > referring to solution related
>> to
>> > >> > unclean
>> > >> > > > >> leader
>> > >> > > > >> > > > >> election
>> > >> > > > >> > > > >> > > > using
>> > >> > > > >> > > > >> > > > > > > >> > > leader_epoch
>> > >> > > > >> > > > >> > > > > > > >> > > > in my previous email?
>> > >> > > > >> > > > >> > > > > > > >> > > >
>> > >> > > > >> > > > >> > > > > > > >> > > > Thanks,
>> > >> > > > >> > > > >> > > > > > > >> > > > Dong
>> > >> > > > >> > > > >> > > > > > > >> > > >
>> > >> > > > >> > > > >> > > > > > > >> > > > On Thu, Jan 4, 2018 at 1:33
>> PM,
>> > Jun
>> > >> > Rao
>> > >> > > <
>> > >> > > > >> > > > >> > jun@confluent.io
>> > >> > > > >> > > > >> > > >
>> > >> > > > >> > > > >> > > > > > wrote:
>> > >> > > > >> > > > >> > > > > > > >> > > >
>> > >> > > > >> > > > >> > > > > > > >> > > > > Hi, Dong,
>> > >> > > > >> > > > >> > > > > > > >> > > > >
>> > >> > > > >> > > > >> > > > > > > >> > > > > Not sure that I fully
>> > understand
>> > >> > your
>> > >> > > > >> latest
>> > >> > > > >> > > > >> > suggestion.
>> > >> > > > >> > > > >> > > > > > > >> Returning an
>> > >> > > > >> > > > >> > > > > > > >> > > > ever
>> > >> > > > >> > > > >> > > > > > > >> > > > > growing global metadata
>> version
>> > >> > itself
>> > >> > > > is
>> > >> > > > >> no
>> > >> > > > >> > > > ideal,
>> > >> > > > >> > > > >> > but
>> > >> > > > >> > > > >> > > is
>> > >> > > > >> > > > >> > > > > > fine.
>> > >> > > > >> > > > >> > > > > > > >> My
>> > >> > > > >> > > > >> > > > > > > >> > > > > question is whether the
>> > metadata
>> > >> > > version
>> > >> > > > >> > > returned
>> > >> > > > >> > > > >> in
>> > >> > > > >> > > > >> > the
>> > >> > > > >> > > > >> > > > > fetch
>> > >> > > > >> > > > >> > > > > > > >> > response
>> > >> > > > >> > > > >> > > > > > > >> > > > > needs to be stored with the
>> > >> offset
>> > >> > > > >> together
>> > >> > > > >> > if
>> > >> > > > >> > > > >> offsets
>> > >> > > > >> > > > >> > > are
>> > >> > > > >> > > > >> > > > > > > stored
>> > >> > > > >> > > > >> > > > > > > >> > > > > externally. If so, we also
>> have
>> > >> to
>> > >> > > > change
>> > >> > > > >> the
>> > >> > > > >> > > > >> consumer
>> > >> > > > >> > > > >> > > API
>> > >> > > > >> > > > >> > > > > for
>> > >> > > > >> > > > >> > > > > > > >> > > > commitSync()
>> > >> > > > >> > > > >> > > > > > > >> > > > > and need to worry about
>> > >> > compatibility.
>> > >> > > > If
>> > >> > > > >> we
>> > >> > > > >> > > > don't
>> > >> > > > >> > > > >> > store
>> > >> > > > >> > > > >> > > > the
>> > >> > > > >> > > > >> > > > > > > >> metadata
>> > >> > > > >> > > > >> > > > > > > >> > > > > version together with the
>> > offset,
>> > >> > on a
>> > >> > > > >> > consumer
>> > >> > > > >> > > > >> > restart,
>> > >> > > > >> > > > >> > > > > it's
>> > >> > > > >> > > > >> > > > > > > not
>> > >> > > > >> > > > >> > > > > > > >> > clear
>> > >> > > > >> > > > >> > > > > > > >> > > > how
>> > >> > > > >> > > > >> > > > > > > >> > > > > we can ensure the metadata
>> in
>> > the
>> > >> > > > >> consumer is
>> > >> > > > >> > > > high
>> > >> > > > >> > > > >> > > enough
>> > >> > > > >> > > > >> > > > > > since
>> > >> > > > >> > > > >> > > > > > > >> there
>> > >> > > > >> > > > >> > > > > > > >> > > is
>> > >> > > > >> > > > >> > > > > > > >> > > > no
>> > >> > > > >> > > > >> > > > > > > >> > > > > metadata version to compare
>> > with.
>> > >> > > > >> > > > >> > > > > > > >> > > > >
>> > >> > > > >> > > > >> > > > > > > >> > > > > Thanks,
>> > >> > > > >> > > > >> > > > > > > >> > > > >
>> > >> > > > >> > > > >> > > > > > > >> > > > > Jun
>> > >> > > > >> > > > >> > > > > > > >> > > > >
>> > >> > > > >> > > > >> > > > > > > >> > > > >
>> > >> > > > >> > > > >> > > > > > > >> > > > > On Wed, Jan 3, 2018 at 6:43
>> PM,
>> > >> Dong
>> > >> > > > Lin <
>> > >> > > > >> > > > >> > > > > lindong28@gmail.com
>> > >> > > > >> > > > >> > > > > > >
>> > >> > > > >> > > > >> > > > > > > >> > wrote:
>> > >> > > > >> > > > >> > > > > > > >> > > > >
>> > >> > > > >> > > > >> > > > > > > >> > > > > > Hey Jun,
>> > >> > > > >> > > > >> > > > > > > >> > > > > >
>> > >> > > > >> > > > >> > > > > > > >> > > > > > Thanks much for the
>> > >> explanation.
>> > >> > > > >> > > > >> > > > > > > >> > > > > >
>> > >> > > > >> > > > >> > > > > > > >> > > > > > I understand the
>> advantage of
>> > >> > > > >> > partition_epoch
>> > >> > > > >> > > > >> over
>> > >> > > > >> > > > >> > > > > > > >> metadata_epoch.
>> > >> > > > >> > > > >> > > > > > > >> > My
>> > >> > > > >> > > > >> > > > > > > >> > > > > > current concern is that
>> the
>> > >> use of
>> > >> > > > >> > > leader_epoch
>> > >> > > > >> > > > >> and
>> > >> > > > >> > > > >> > > the
>> > >> > > > >> > > > >> > > > > > > >> > > partition_epoch
>> > >> > > > >> > > > >> > > > > > > >> > > > > > requires us considerable
>> > >> change to
>> > >> > > > >> > consumer's
>> > >> > > > >> > > > >> public
>> > >> > > > >> > > > >> > > API
>> > >> > > > >> > > > >> > > > > to
>> > >> > > > >> > > > >> > > > > > > take
>> > >> > > > >> > > > >> > > > > > > >> > care
>> > >> > > > >> > > > >> > > > > > > >> > > > of
>> > >> > > > >> > > > >> > > > > > > >> > > > > > the case where user stores
>> > >> offset
>> > >> > > > >> > externally.
>> > >> > > > >> > > > For
>> > >> > > > >> > > > >> > > > example,
>> > >> > > > >> > > > >> > > > > > > >> > > *consumer*.
>> > >> > > > >> > > > >> > > > > > > >> > > > > > *commitSync*(..) would
>> have
>> > to
>> > >> > take
>> > >> > > a
>> > >> > > > >> map
>> > >> > > > >> > > whose
>> > >> > > > >> > > > >> > value
>> > >> > > > >> > > > >> > > is
>> > >> > > > >> > > > >> > > > > > > >> <offset,
>> > >> > > > >> > > > >> > > > > > > >> > > > > metadata,
>> > >> > > > >> > > > >> > > > > > > >> > > > > > leader epoch, partition
>> > epoch>.
>> > >> > > > >> > > > >> > *consumer*.*seek*(...)
>> > >> > > > >> > > > >> > > > > would
>> > >> > > > >> > > > >> > > > > > > >> also
>> > >> > > > >> > > > >> > > > > > > >> > > need
>> > >> > > > >> > > > >> > > > > > > >> > > > > > leader_epoch and
>> > >> partition_epoch
>> > >> > as
>> > >> > > > >> > > parameter.
>> > >> > > > >> > > > >> > > > Technically
>> > >> > > > >> > > > >> > > > > > we
>> > >> > > > >> > > > >> > > > > > > >> can
>> > >> > > > >> > > > >> > > > > > > >> > > > > probably
>> > >> > > > >> > > > >> > > > > > > >> > > > > > still make it work in a
>> > >> backward
>> > >> > > > >> compatible
>> > >> > > > >> > > > >> manner
>> > >> > > > >> > > > >> > > after
>> > >> > > > >> > > > >> > > > > > > careful
>> > >> > > > >> > > > >> > > > > > > >> > > design
>> > >> > > > >> > > > >> > > > > > > >> > > > > and
>> > >> > > > >> > > > >> > > > > > > >> > > > > > discussion. But these
>> changes
>> > >> can
>> > >> > > make
>> > >> > > > >> the
>> > >> > > > >> > > > >> > consumer's
>> > >> > > > >> > > > >> > > > > > > interface
>> > >> > > > >> > > > >> > > > > > > >> > > > > > unnecessarily complex for
>> > more
>> > >> > users
>> > >> > > > >> who do
>> > >> > > > >> > > not
>> > >> > > > >> > > > >> > store
>> > >> > > > >> > > > >> > > > > offset
>> > >> > > > >> > > > >> > > > > > > >> > > > externally.
>> > >> > > > >> > > > >> > > > > > > >> > > > > >
>> > >> > > > >> > > > >> > > > > > > >> > > > > > After thinking more about
>> it,
>> > >> we
>> > >> > can
>> > >> > > > >> > address
>> > >> > > > >> > > > all
>> > >> > > > >> > > > >> > > > problems
>> > >> > > > >> > > > >> > > > > > > >> discussed
>> > >> > > > >> > > > >> > > > > > > >> > > by
>> > >> > > > >> > > > >> > > > > > > >> > > > > only
>> > >> > > > >> > > > >> > > > > > > >> > > > > > using the metadata_epoch
>> > >> without
>> > >> > > > >> > introducing
>> > >> > > > >> > > > >> > > > leader_epoch
>> > >> > > > >> > > > >> > > > > or
>> > >> > > > >> > > > >> > > > > > > the
>> > >> > > > >> > > > >> > > > > > > >> > > > > > partition_epoch. The
>> current
>> > >> KIP
>> > >> > > > >> describes
>> > >> > > > >> > > the
>> > >> > > > >> > > > >> > changes
>> > >> > > > >> > > > >> > > > to
>> > >> > > > >> > > > >> > > > > > the
>> > >> > > > >> > > > >> > > > > > > >> > > consumer
>> > >> > > > >> > > > >> > > > > > > >> > > > > API
>> > >> > > > >> > > > >> > > > > > > >> > > > > > and how the new API can be
>> > >> used if
>> > >> > > > user
>> > >> > > > >> > > stores
>> > >> > > > >> > > > >> > offset
>> > >> > > > >> > > > >> > > > > > > >> externally.
>> > >> > > > >> > > > >> > > > > > > >> > In
>> > >> > > > >> > > > >> > > > > > > >> > > > > order
>> > >> > > > >> > > > >> > > > > > > >> > > > > > to address the scenario
>> you
>> > >> > > described
>> > >> > > > >> > > earlier,
>> > >> > > > >> > > > we
>> > >> > > > >> > > > >> > can
>> > >> > > > >> > > > >> > > > > > include
>> > >> > > > >> > > > >> > > > > > > >> > > > > > metadata_epoch in the
>> > >> > FetchResponse
>> > >> > > > and
>> > >> > > > >> the
>> > >> > > > >> > > > >> > > > > > > LeaderAndIsrRequest.
>> > >> > > > >> > > > >> > > > > > > >> > > > Consumer
>> > >> > > > >> > > > >> > > > > > > >> > > > > > remembers the largest
>> > >> > metadata_epoch
>> > >> > > > >> from
>> > >> > > > >> > all
>> > >> > > > >> > > > the
>> > >> > > > >> > > > >> > > > > > > FetchResponse
>> > >> > > > >> > > > >> > > > > > > >> it
>> > >> > > > >> > > > >> > > > > > > >> > > has
>> > >> > > > >> > > > >> > > > > > > >> > > > > > received. The
>> metadata_epoch
>> > >> > > committed
>> > >> > > > >> with
>> > >> > > > >> > > the
>> > >> > > > >> > > > >> > > offset,
>> > >> > > > >> > > > >> > > > > > either
>> > >> > > > >> > > > >> > > > > > > >> > within
>> > >> > > > >> > > > >> > > > > > > >> > > > or
>> > >> > > > >> > > > >> > > > > > > >> > > > > > outside Kafka, should be
>> the
>> > >> > largest
>> > >> > > > >> > > > >> metadata_epoch
>> > >> > > > >> > > > >> > > > across
>> > >> > > > >> > > > >> > > > > > all
>> > >> > > > >> > > > >> > > > > > > >> > > > > > FetchResponse and
>> > >> MetadataResponse
>> > >> > > > ever
>> > >> > > > >> > > > received
>> > >> > > > >> > > > >> by
>> > >> > > > >> > > > >> > > this
>> > >> > > > >> > > > >> > > > > > > >> consumer.
>> > >> > > > >> > > > >> > > > > > > >> > > > > >
>> > >> > > > >> > > > >> > > > > > > >> > > > > > The drawback of using only
>> > the
>> > >> > > > >> > metadata_epoch
>> > >> > > > >> > > > is
>> > >> > > > >> > > > >> > that
>> > >> > > > >> > > > >> > > we
>> > >> > > > >> > > > >> > > > > can
>> > >> > > > >> > > > >> > > > > > > not
>> > >> > > > >> > > > >> > > > > > > >> > > always
>> > >> > > > >> > > > >> > > > > > > >> > > > > do
>> > >> > > > >> > > > >> > > > > > > >> > > > > > the smart offset reset in
>> > case
>> > >> of
>> > >> > > > >> unclean
>> > >> > > > >> > > > leader
>> > >> > > > >> > > > >> > > > election
>> > >> > > > >> > > > >> > > > > > > which
>> > >> > > > >> > > > >> > > > > > > >> you
>> > >> > > > >> > > > >> > > > > > > >> > > > > > mentioned earlier. But in
>> > most
>> > >> > case,
>> > >> > > > >> > unclean
>> > >> > > > >> > > > >> leader
>> > >> > > > >> > > > >> > > > > election
>> > >> > > > >> > > > >> > > > > > > >> > probably
>> > >> > > > >> > > > >> > > > > > > >> > > > > > happens when consumer is
>> not
>> > >> > > > >> > > > >> rebalancing/restarting.
>> > >> > > > >> > > > >> > > In
>> > >> > > > >> > > > >> > > > > > these
>> > >> > > > >> > > > >> > > > > > > >> > cases,
>> > >> > > > >> > > > >> > > > > > > >> > > > > either
>> > >> > > > >> > > > >> > > > > > > >> > > > > > consumer is not directly
>> > >> affected
>> > >> > by
>> > >> > > > >> > unclean
>> > >> > > > >> > > > >> leader
>> > >> > > > >> > > > >> > > > > election
>> > >> > > > >> > > > >> > > > > > > >> since
>> > >> > > > >> > > > >> > > > > > > >> > it
>> > >> > > > >> > > > >> > > > > > > >> > > > is
>> > >> > > > >> > > > >> > > > > > > >> > > > > > not consuming from the
>> end of
>> > >> the
>> > >> > > log,
>> > >> > > > >> or
>> > >> > > > >> > > > >> consumer
>> > >> > > > >> > > > >> > can
>> > >> > > > >> > > > >> > > > > > derive
>> > >> > > > >> > > > >> > > > > > > >> the
>> > >> > > > >> > > > >> > > > > > > >> > > > > > leader_epoch from the most
>> > >> recent
>> > >> > > > >> message
>> > >> > > > >> > > > >> received
>> > >> > > > >> > > > >> > > > before
>> > >> > > > >> > > > >> > > > > it
>> > >> > > > >> > > > >> > > > > > > >> sees
>> > >> > > > >> > > > >> > > > > > > >> > > > > >
>> OffsetOutOfRangeException. So
>> > >> I am
>> > >> > > not
>> > >> > > > >> sure
>> > >> > > > >> > > it
>> > >> > > > >> > > > is
>> > >> > > > >> > > > >> > > worth
>> > >> > > > >> > > > >> > > > > > adding
>> > >> > > > >> > > > >> > > > > > > >> the
>> > >> > > > >> > > > >> > > > > > > >> > > > > > leader_epoch to consumer
>> API
>> > to
>> > >> > > > address
>> > >> > > > >> the
>> > >> > > > >> > > > >> > remaining
>> > >> > > > >> > > > >> > > > > corner
>> > >> > > > >> > > > >> > > > > > > >> case.
>> > >> > > > >> > > > >> > > > > > > >> > > What
>> > >> > > > >> > > > >> > > > > > > >> > > > > do
>> > >> > > > >> > > > >> > > > > > > >> > > > > > you think?
>> > >> > > > >> > > > >> > > > > > > >> > > > > >
>> > >> > > > >> > > > >> > > > > > > >> > > > > > Thanks,
>> > >> > > > >> > > > >> > > > > > > >> > > > > > Dong
>> > >> > > > >> > > > >> > > > > > > >> > > > > >
>> > >> > > > >> > > > >> > > > > > > >> > > > > >
>> > >> > > > >> > > > >> > > > > > > >> > > > > >
>> > >> > > > >> > > > >> > > > > > > >> > > > > > On Tue, Jan 2, 2018 at
>> 6:28
>> > PM,
>> > >> > Jun
>> > >> > > > Rao
>> > >> > > > >> <
>> > >> > > > >> > > > >> > > > jun@confluent.io
>> > >> > > > >> > > > >> > > > > >
>> > >> > > > >> > > > >> > > > > > > >> wrote:
>> > >> > > > >> > > > >> > > > > > > >> > > > > >
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > Hi, Dong,
>> > >> > > > >> > > > >> > > > > > > >> > > > > > >
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > Thanks for the reply.
>> > >> > > > >> > > > >> > > > > > > >> > > > > > >
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > To solve the topic
>> > recreation
>> > >> > > issue,
>> > >> > > > >> we
>> > >> > > > >> > > could
>> > >> > > > >> > > > >> use
>> > >> > > > >> > > > >> > > > > either a
>> > >> > > > >> > > > >> > > > > > > >> global
>> > >> > > > >> > > > >> > > > > > > >> > > > > > metadata
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > version or a partition
>> > level
>> > >> > > epoch.
>> > >> > > > >> But
>> > >> > > > >> > > > either
>> > >> > > > >> > > > >> one
>> > >> > > > >> > > > >> > > > will
>> > >> > > > >> > > > >> > > > > > be a
>> > >> > > > >> > > > >> > > > > > > >> new
>> > >> > > > >> > > > >> > > > > > > >> > > > > concept,
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > right? To me, the latter
>> > >> seems
>> > >> > > more
>> > >> > > > >> > > natural.
>> > >> > > > >> > > > It
>> > >> > > > >> > > > >> > also
>> > >> > > > >> > > > >> > > > > makes
>> > >> > > > >> > > > >> > > > > > > it
>> > >> > > > >> > > > >> > > > > > > >> > > easier
>> > >> > > > >> > > > >> > > > > > > >> > > > to
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > detect if a consumer's
>> > >> offset is
>> > >> > > > still
>> > >> > > > >> > > valid
>> > >> > > > >> > > > >> > after a
>> > >> > > > >> > > > >> > > > > topic
>> > >> > > > >> > > > >> > > > > > > is
>> > >> > > > >> > > > >> > > > > > > >> > > > > recreated.
>> > >> > > > >> > > > >> > > > > > > >> > > > > > As
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > you pointed out, we
>> don't
>> > >> need
>> > >> > to
>> > >> > > > >> store
>> > >> > > > >> > the
>> > >> > > > >> > > > >> > > partition
>> > >> > > > >> > > > >> > > > > > epoch
>> > >> > > > >> > > > >> > > > > > > in
>> > >> > > > >> > > > >> > > > > > > >> > the
>> > >> > > > >> > > > >> > > > > > > >> > > > > > message.
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > The following is what I
>> am
>> > >> > > thinking.
>> > >> > > > >> > When a
>> > >> > > > >> > > > >> > > partition
>> > >> > > > >> > > > >> > > > is
>> > >> > > > >> > > > >> > > > > > > >> created,
>> > >> > > > >> > > > >> > > > > > > >> > > we
>> > >> > > > >> > > > >> > > > > > > >> > > > > can
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > assign a partition epoch
>> > >> from an
>> > >> > > > >> > > > >> ever-increasing
>> > >> > > > >> > > > >> > > > global
>> > >> > > > >> > > > >> > > > > > > >> counter
>> > >> > > > >> > > > >> > > > > > > >> > and
>> > >> > > > >> > > > >> > > > > > > >> > > > > store
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > it in
>> > >> /brokers/topics/[topic]/
>> > >> > > > >> > > > >> > > > partitions/[partitionId]
>> > >> > > > >> > > > >> > > > > in
>> > >> > > > >> > > > >> > > > > > > ZK.
>> > >> > > > >> > > > >> > > > > > > >> > The
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > partition
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > epoch is propagated to
>> > every
>> > >> > > broker.
>> > >> > > > >> The
>> > >> > > > >> > > > >> consumer
>> > >> > > > >> > > > >> > > will
>> > >> > > > >> > > > >> > > > > be
>> > >> > > > >> > > > >> > > > > > > >> > tracking
>> > >> > > > >> > > > >> > > > > > > >> > > a
>> > >> > > > >> > > > >> > > > > > > >> > > > > > tuple
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > of <offset, leader
>> epoch,
>> > >> > > partition
>> > >> > > > >> > epoch>
>> > >> > > > >> > > > for
>> > >> > > > >> > > > >> > > > offsets.
>> > >> > > > >> > > > >> > > > > > If a
>> > >> > > > >> > > > >> > > > > > > >> > topic
>> > >> > > > >> > > > >> > > > > > > >> > > is
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > recreated, it's possible
>> > >> that a
>> > >> > > > >> > consumer's
>> > >> > > > >> > > > >> offset
>> > >> > > > >> > > > >> > > and
>> > >> > > > >> > > > >> > > > > > leader
>> > >> > > > >> > > > >> > > > > > > >> > epoch
>> > >> > > > >> > > > >> > > > > > > >> > > > > still
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > match that in the
>> broker,
>> > but
>> > >> > > > >> partition
>> > >> > > > >> > > epoch
>> > >> > > > >> > > > >> > won't
>> > >> > > > >> > > > >> > > > be.
>> > >> > > > >> > > > >> > > > > In
>> > >> > > > >> > > > >> > > > > > > >> this
>> > >> > > > >> > > > >> > > > > > > >> > > case,
>> > >> > > > >> > > > >> > > > > > > >> > > > > we
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > can potentially still
>> treat
>> > >> the
>> > >> > > > >> > consumer's
>> > >> > > > >> > > > >> offset
>> > >> > > > >> > > > >> > as
>> > >> > > > >> > > > >> > > > out
>> > >> > > > >> > > > >> > > > > > of
>> > >> > > > >> > > > >> > > > > > > >> range
>> > >> > > > >> > > > >> > > > > > > >> > > and
>> > >> > > > >> > > > >> > > > > > > >> > > > > > reset
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > the offset based on the
>> > >> offset
>> > >> > > reset
>> > >> > > > >> > policy
>> > >> > > > >> > > > in
>> > >> > > > >> > > > >> the
>> > >> > > > >> > > > >> > > > > > consumer.
>> > >> > > > >> > > > >> > > > > > > >> This
>> > >> > > > >> > > > >> > > > > > > >> > > > seems
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > harder to do with a
>> global
>> > >> > > metadata
>> > >> > > > >> > > version.
>> > >> > > > >> > > > >> > > > > > > >> > > > > > >
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > Jun
>> > >> > > > >> > > > >> > > > > > > >> > > > > > >
>> > >> > > > >> > > > >> > > > > > > >> > > > > > >
>> > >> > > > >> > > > >> > > > > > > >> > > > > > >
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > On Mon, Dec 25, 2017 at
>> > 6:56
>> > >> AM,
>> > >> > > > Dong
>> > >> > > > >> > Lin <
>> > >> > > > >> > > > >> > > > > > > >> lindong28@gmail.com>
>> > >> > > > >> > > > >> > > > > > > >> > > > wrote:
>> > >> > > > >> > > > >> > > > > > > >> > > > > > >
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > > Hey Jun,
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > >
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > > This is a very good
>> > >> example.
>> > >> > > After
>> > >> > > > >> > > thinking
>> > >> > > > >> > > > >> > > through
>> > >> > > > >> > > > >> > > > > this
>> > >> > > > >> > > > >> > > > > > > in
>> > >> > > > >> > > > >> > > > > > > >> > > > detail, I
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > agree
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > > that we need to commit
>> > >> offset
>> > >> > > with
>> > >> > > > >> > leader
>> > >> > > > >> > > > >> epoch
>> > >> > > > >> > > > >> > in
>> > >> > > > >> > > > >> > > > > order
>> > >> > > > >> > > > >> > > > > > > to
>> > >> > > > >> > > > >> > > > > > > >> > > address
>> > >> > > > >> > > > >> > > > > > > >> > > > > > this
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > > example.
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > >
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > > I think the remaining
>> > >> question
>> > >> > > is
>> > >> > > > >> how
>> > >> > > > >> > to
>> > >> > > > >> > > > >> address
>> > >> > > > >> > > > >> > > the
>> > >> > > > >> > > > >> > > > > > > >> scenario
>> > >> > > > >> > > > >> > > > > > > >> > > that
>> > >> > > > >> > > > >> > > > > > > >> > > > > the
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > > topic is deleted and
>> > >> > re-created.
>> > >> > > > One
>> > >> > > > >> > > > possible
>> > >> > > > >> > > > >> > > > solution
>> > >> > > > >> > > > >> > > > > > is
>> > >> > > > >> > > > >> > > > > > > to
>> > >> > > > >> > > > >> > > > > > > >> > > commit
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > offset
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > > with both the leader
>> > epoch
>> > >> and
>> > >> > > the
>> > >> > > > >> > > metadata
>> > >> > > > >> > > > >> > > version.
>> > >> > > > >> > > > >> > > > > The
>> > >> > > > >> > > > >> > > > > > > >> logic
>> > >> > > > >> > > > >> > > > > > > >> > > and
>> > >> > > > >> > > > >> > > > > > > >> > > > > the
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > > implementation of this
>> > >> > solution
>> > >> > > > does
>> > >> > > > >> > not
>> > >> > > > >> > > > >> > require a
>> > >> > > > >> > > > >> > > > new
>> > >> > > > >> > > > >> > > > > > > >> concept
>> > >> > > > >> > > > >> > > > > > > >> > > > (e.g.
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > > partition epoch) and
>> it
>> > >> does
>> > >> > not
>> > >> > > > >> > require
>> > >> > > > >> > > > any
>> > >> > > > >> > > > >> > > change
>> > >> > > > >> > > > >> > > > to
>> > >> > > > >> > > > >> > > > > > the
>> > >> > > > >> > > > >> > > > > > > >> > > message
>> > >> > > > >> > > > >> > > > > > > >> > > > > > format
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > > or leader epoch. It
>> also
>> > >> > allows
>> > >> > > us
>> > >> > > > >> to
>> > >> > > > >> > > order
>> > >> > > > >> > > > >> the
>> > >> > > > >> > > > >> > > > > metadata
>> > >> > > > >> > > > >> > > > > > > in
>> > >> > > > >> > > > >> > > > > > > >> a
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > > straightforward manner
>> > >> which
>> > >> > may
>> > >> > > > be
>> > >> > > > >> > > useful
>> > >> > > > >> > > > in
>> > >> > > > >> > > > >> > the
>> > >> > > > >> > > > >> > > > > > future.
>> > >> > > > >> > > > >> > > > > > > >> So it
>> > >> > > > >> > > > >> > > > > > > >> > > may
>> > >> > > > >> > > > >> > > > > > > >> > > > > be
>> > >> > > > >> > > > >> > > > > > > >> > > > > > a
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > > better solution than
>> > >> > generating
>> > >> > > a
>> > >> > > > >> > random
>> > >> > > > >> > > > >> > partition
>> > >> > > > >> > > > >> > > > > epoch
>> > >> > > > >> > > > >> > > > > > > >> every
>> > >> > > > >> > > > >> > > > > > > >> > > time
>> > >> > > > >> > > > >> > > > > > > >> > > > > we
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > > create a partition.
>> Does
>> > >> this
>> > >> > > > sound
>> > >> > > > >> > > > >> reasonable?
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > >
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > > Previously one concern
>> > with
>> > >> > > using
>> > >> > > > >> the
>> > >> > > > >> > > > >> metadata
>> > >> > > > >> > > > >> > > > version
>> > >> > > > >> > > > >> > > > > > is
>> > >> > > > >> > > > >> > > > > > > >> that
>> > >> > > > >> > > > >> > > > > > > >> > > > > consumer
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > > will be forced to
>> refresh
>> > >> > > metadata
>> > >> > > > >> even
>> > >> > > > >> > > if
>> > >> > > > >> > > > >> > > metadata
>> > >> > > > >> > > > >> > > > > > > version
>> > >> > > > >> > > > >> > > > > > > >> is
>> > >> > > > >> > > > >> > > > > > > >> > > > > > increased
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > > due to topics that the
>> > >> > consumer
>> > >> > > is
>> > >> > > > >> not
>> > >> > > > >> > > > >> > interested
>> > >> > > > >> > > > >> > > > in.
>> > >> > > > >> > > > >> > > > > > Now
>> > >> > > > >> > > > >> > > > > > > I
>> > >> > > > >> > > > >> > > > > > > >> > > > realized
>> > >> > > > >> > > > >> > > > > > > >> > > > > > that
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > > this is probably not a
>> > >> > problem.
>> > >> > > > >> > Currently
>> > >> > > > >> > > > >> client
>> > >> > > > >> > > > >> > > > will
>> > >> > > > >> > > > >> > > > > > > >> refresh
>> > >> > > > >> > > > >> > > > > > > >> > > > > metadata
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > > either due to
>> > >> > > > >> InvalidMetadataException
>> > >> > > > >> > in
>> > >> > > > >> > > > the
>> > >> > > > >> > > > >> > > > response
>> > >> > > > >> > > > >> > > > > > > from
>> > >> > > > >> > > > >> > > > > > > >> > > broker
>> > >> > > > >> > > > >> > > > > > > >> > > > or
>> > >> > > > >> > > > >> > > > > > > >> > > > > > due
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > > to metadata expiry.
>> The
>> > >> > addition
>> > >> > > > of
>> > >> > > > >> the
>> > >> > > > >> > > > >> metadata
>> > >> > > > >> > > > >> > > > > version
>> > >> > > > >> > > > >> > > > > > > >> should
>> > >> > > > >> > > > >> > > > > > > >> > > > > > increase
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > > the overhead of
>> metadata
>> > >> > refresh
>> > >> > > > >> caused
>> > >> > > > >> > > by
>> > >> > > > >> > > > >> > > > > > > >> > > > InvalidMetadataException.
>> > >> > > > >> > > > >> > > > > > > >> > > > > If
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > > client refresh
>> metadata
>> > >> due to
>> > >> > > > >> expiry
>> > >> > > > >> > and
>> > >> > > > >> > > > it
>> > >> > > > >> > > > >> > > > receives
>> > >> > > > >> > > > >> > > > > a
>> > >> > > > >> > > > >> > > > > > > >> > metadata
>> > >> > > > >> > > > >> > > > > > > >> > > > > whose
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > > version is lower than
>> the
>> > >> > > current
>> > >> > > > >> > > metadata
>> > >> > > > >> > > > >> > > version,
>> > >> > > > >> > > > >> > > > we
>> > >> > > > >> > > > >> > > > > > can
>> > >> > > > >> > > > >> > > > > > > >> > reject
>> > >> > > > >> > > > >> > > > > > > >> > > > the
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > > metadata but still
>> reset
>> > >> the
>> > >> > > > >> metadata
>> > >> > > > >> > > age,
>> > >> > > > >> > > > >> which
>> > >> > > > >> > > > >> > > > > > > essentially
>> > >> > > > >> > > > >> > > > > > > >> > keep
>> > >> > > > >> > > > >> > > > > > > >> > > > the
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > > existing behavior in
>> the
>> > >> > client.
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > >
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > > Thanks much,
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > > Dong
>> > >> > > > >> > > > >> > > > > > > >> > > > > > > >
>> > >> > > > >> > > > >> > > > > > > >> > > > > > >
>> > >> > > > >> > > > >> > > > > > > >> > > > > >
>> > >> > > > >> > > > >> > > > > > > >> > > > >
>> > >> > > > >> > > > >> > > > > > > >> > > >
>> > >> > > > >> > > > >> > > > > > > >> > >
>> > >> > > > >> > > > >> > > > > > > >> >
>> > >> > > > >> > > > >> > > > > > > >>
>> > >> > > > >> > > > >> > > > > > > >
>> > >> > > > >> > > > >> > > > > > > >
>> > >> > > > >> > > > >> > > > > > >
>> > >> > > > >> > > > >> > > > > >
>> > >> > > > >> > > > >> > > > >
>> > >> > > > >> > > > >> > > >
>> > >> > > > >> > > > >> > >
>> > >> > > > >> > > > >> >
>> > >> > > > >> > > > >>
>> > >> > > > >> > > > >
>> > >> > > > >> > > > >
>> > >> > > > >> > > >
>> > >> > > > >> > >
>> > >> > > > >> >
>> > >> > > > >>
>> > >> > > > >
>> > >> > > > >
>> > >> > > >
>> > >> > >
>> > >> >
>> > >>
>> > >
>> > >
>> >
>>
>
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Dong Lin <li...@gmail.com>.
Hey Jun, Jason,

Thanks much for all the review! I will open the voting thread.

Regards,
Dong

On Tue, Jan 23, 2018 at 3:37 PM, Jun Rao <ju...@confluent.io> wrote:

> Hi, Dong,
>
> The current KIP looks good to me.
>
> Thanks,
>
> Jun
>
> On Tue, Jan 23, 2018 at 12:29 PM, Dong Lin <li...@gmail.com> wrote:
>
> > Hey Jun,
> >
> > Do you think the current KIP looks OK? I am wondering if we can open the
> > voting thread.
> >
> > Thanks!
> > Dong
> >
> > On Fri, Jan 19, 2018 at 3:08 PM, Dong Lin <li...@gmail.com> wrote:
> >
> > > Hey Jun,
> > >
> > > I think we can probably have a static method in Util class to decode
> the
> > > byte[]. Both KafkaConsumer implementation and the user application will
> > be
> > > able to decode the byte array and log its content for debug purpose. So
> > it
> > > seems that we can still print the information we want. It is just not
> > > explicitly exposed in the consumer interface. Would this address the
> > > problem here?
> > >
> > > Yeah we can include OffsetEpoch in AdminClient. This can be added in
> > > KIP-222? Is there something you would like me to add in this KIP?
> > >
> > > Thanks!
> > > Dong
> > >
> > > On Fri, Jan 19, 2018 at 3:00 PM, Jun Rao <ju...@confluent.io> wrote:
> > >
> > >> Hi, Dong,
> > >>
> > >> The issue with using just byte[] for OffsetEpoch is that it won't be
> > >> printable, which makes debugging harder.
> > >>
> > >> Also, KIP-222 proposes a listGroupOffset() method in AdminClient. If
> > that
> > >> gets adopted before this KIP, we probably want to include OffsetEpoch
> in
> > >> the AdminClient too.
> > >>
> > >> Thanks,
> > >>
> > >> Jun
> > >>
> > >>
> > >> On Thu, Jan 18, 2018 at 6:30 PM, Dong Lin <li...@gmail.com>
> wrote:
> > >>
> > >> > Hey Jun,
> > >> >
> > >> > I agree. I have updated the KIP to remove the class OffetEpoch and
> > >> replace
> > >> > OffsetEpoch with byte[] in APIs that use it. Can you see if it looks
> > >> good?
> > >> >
> > >> > Thanks!
> > >> > Dong
> > >> >
> > >> > On Thu, Jan 18, 2018 at 6:07 PM, Jun Rao <ju...@confluent.io> wrote:
> > >> >
> > >> > > Hi, Dong,
> > >> > >
> > >> > > Thanks for the updated KIP. It looks good to me now. The only
> thing
> > is
> > >> > > for OffsetEpoch.
> > >> > > If we expose the individual fields in the class, we probably don't
> > >> need
> > >> > the
> > >> > > encode/decode methods. If we want to hide the details of
> > OffsetEpoch,
> > >> we
> > >> > > probably don't want expose the individual fields.
> > >> > >
> > >> > > Jun
> > >> > >
> > >> > > On Wed, Jan 17, 2018 at 10:10 AM, Dong Lin <li...@gmail.com>
> > >> wrote:
> > >> > >
> > >> > > > Thinking about point 61 more, I realize that the async zookeeper
> > >> read
> > >> > may
> > >> > > > make it less of an issue for controller to read more zookeeper
> > >> nodes.
> > >> > > > Writing partition_epoch in the per-partition znode makes it
> > simpler
> > >> to
> > >> > > > handle the broker failure between zookeeper writes for a topic
> > >> > creation.
> > >> > > I
> > >> > > > have updated the KIP to use the suggested approach.
> > >> > > >
> > >> > > >
> > >> > > > On Wed, Jan 17, 2018 at 9:57 AM, Dong Lin <li...@gmail.com>
> > >> wrote:
> > >> > > >
> > >> > > > > Hey Jun,
> > >> > > > >
> > >> > > > > Thanks much for the comments. Please see my comments inline.
> > >> > > > >
> > >> > > > > On Tue, Jan 16, 2018 at 4:38 PM, Jun Rao <ju...@confluent.io>
> > >> wrote:
> > >> > > > >
> > >> > > > >> Hi, Dong,
> > >> > > > >>
> > >> > > > >> Thanks for the updated KIP. Looks good to me overall. Just a
> > few
> > >> > minor
> > >> > > > >> comments.
> > >> > > > >>
> > >> > > > >> 60. OffsetAndMetadata positionAndOffsetEpoch(TopicPartition
> > >> > > partition):
> > >> > > > >> It
> > >> > > > >> seems that there is no need to return metadata. We probably
> > want
> > >> to
> > >> > > > return
> > >> > > > >> sth like OffsetAndEpoch.
> > >> > > > >>
> > >> > > > >
> > >> > > > > Previously I think we may want to re-use the existing class to
> > >> keep
> > >> > our
> > >> > > > > consumer interface simpler. I have updated the KIP to add
> class
> > >> > > > > OffsetAndOffsetEpoch. I didn't use OffsetAndEpoch because user
> > may
> > >> > > > confuse
> > >> > > > > this name with OffsetEpoch. Does this sound OK?
> > >> > > > >
> > >> > > > >
> > >> > > > >>
> > >> > > > >> 61. Should we store partition_epoch in
> > >> > > > >> /brokers/topics/[topic]/partitions/[partitionId] in ZK?
> > >> > > > >>
> > >> > > > >
> > >> > > > > I have considered this. I think the advantage of adding the
> > >> > > > > partition->partition_epoch map in the existing
> > >> > > > > znode /brokers/topics/[topic]/partitions is that controller
> > only
> > >> > needs
> > >> > > > to
> > >> > > > > read one znode per topic to gets its partition_epoch
> > information.
> > >> > > > Otherwise
> > >> > > > > controller may need to read one extra znode per partition to
> get
> > >> the
> > >> > > same
> > >> > > > > information.
> > >> > > > >
> > >> > > > > When we delete partition or expand partition of a topic,
> someone
> > >> > needs
> > >> > > to
> > >> > > > > modify partition->partition_epoch map in znode
> > >> > > > > /brokers/topics/[topic]/partitions. This may seem a bit more
> > >> > > complicated
> > >> > > > > than simply adding or deleting znode /brokers/topics/[topic]/
> > >> > > > partitions/[partitionId].
> > >> > > > > But the complexity is probably similar to the existing
> operation
> > >> of
> > >> > > > > modifying the partition->replica_list mapping in znode
> > >> > > > > /brokers/topics/[topic]. So not sure it is better to store the
> > >> > > > > partition_epoch in /brokers/topics/[topic]/partit
> > >> ions/[partitionId].
> > >> > > > What
> > >> > > > > do you think?
> > >> > > > >
> > >> > > > >
> > >> > > > >>
> > >> > > > >> 62. For checking outdated metadata in the client, we probably
> > >> want
> > >> > to
> > >> > > > add
> > >> > > > >> when max_partition_epoch will be used.
> > >> > > > >>
> > >> > > > >
> > >> > > > > The max_partition_epoch is used in the Proposed Changes ->
> > >> Client's
> > >> > > > > metadata refresh section to determine whether a metadata is
> > >> outdated.
> > >> > > And
> > >> > > > > this formula is referenced and re-used in other sections to
> > >> determine
> > >> > > > > whether a metadata is outdated. Does this formula look OK?
> > >> > > > >
> > >> > > > >
> > >> > > > >>
> > >> > > > >> 63. "The leader_epoch should be the largest leader_epoch of
> > >> messages
> > >> > > > whose
> > >> > > > >> offset < the commit offset. If no message has been consumed
> > since
> > >> > > > consumer
> > >> > > > >> initialization, the leader_epoch from seek(...) or
> > >> > OffsetFetchResponse
> > >> > > > >> should be used. The partition_epoch should be read from the
> > last
> > >> > > > >> FetchResponse corresponding to the given partition and commit
> > >> > offset.
> > >> > > ":
> > >> > > > >> leader_epoch and partition_epoch are associated with an
> offset.
> > >> So,
> > >> > if
> > >> > > > no
> > >> > > > >> message is consumed, there is no offset and therefore there
> is
> > no
> > >> > need
> > >> > > > to
> > >> > > > >> read leader_epoch and partition_epoch. Also, the leader_epoch
> > >> > > associated
> > >> > > > >> with the offset should just come from the messages returned
> in
> > >> the
> > >> > > fetch
> > >> > > > >> response.
> > >> > > > >>
> > >> > > > >
> > >> > > > > I am thinking that, if user calls seek(..) and commitSync(...)
> > >> > without
> > >> > > > > consuming any messages, we should re-use the leader_epoch and
> > >> > > > > partition_epoch provided by the seek(...) in the
> > >> OffsetCommitRequest.
> > >> > > And
> > >> > > > > if messages have been successfully consumed, then leader_epoch
> > >> will
> > >> > > come
> > >> > > > > from the messages returned in the fetch response. The
> condition
> > >> > > "messages
> > >> > > > > whose offset < the commit offset" is needed to take care of
> the
> > >> log
> > >> > > > > compacted topic which may have offset gap due to log cleaning.
> > >> > > > >
> > >> > > > > Did I miss something here? Or should I rephrase the paragraph
> to
> > >> make
> > >> > > it
> > >> > > > > less confusing?
> > >> > > > >
> > >> > > > >
> > >> > > > >> 64. Could you include the public methods in the OffsetEpoch
> > >> class?
> > >> > > > >>
> > >> > > > >
> > >> > > > > I mistakenly deleted the definition of OffsetEpoch class from
> > the
> > >> > KIP.
> > >> > > I
> > >> > > > > just added it back with the public methods. Could you take
> > another
> > >> > > look?
> > >> > > > >
> > >> > > > >
> > >> > > > >>
> > >> > > > >> Jun
> > >> > > > >>
> > >> > > > >>
> > >> > > > >> On Thu, Jan 11, 2018 at 5:43 PM, Dong Lin <
> lindong28@gmail.com
> > >
> > >> > > wrote:
> > >> > > > >>
> > >> > > > >> > Hey Jun,
> > >> > > > >> >
> > >> > > > >> > Thanks much. I agree that we can not rely on committed
> > offsets
> > >> to
> > >> > be
> > >> > > > >> always
> > >> > > > >> > deleted when we delete topic. So it is necessary to use a
> > >> > > > per-partition
> > >> > > > >> > epoch that does not change unless this partition is
> deleted.
> > I
> > >> > also
> > >> > > > >> agree
> > >> > > > >> > that it is very nice to be able to uniquely identify a
> > message
> > >> > with
> > >> > > > >> > (offset, leader_epoch, partition_epoch) in face of
> potential
> > >> topic
> > >> > > > >> deletion
> > >> > > > >> > and unclean leader election.
> > >> > > > >> >
> > >> > > > >> > I agree with all your comments. And I have updated the KIP
> > >> based
> > >> > on
> > >> > > > our
> > >> > > > >> > latest discussion. In addition, I added
> > >> > > InvalidPartitionEpochException
> > >> > > > >> > which will be thrown by consumer.poll() if the
> > partition_epoch
> > >> > > > >> associated
> > >> > > > >> > with the partition, which can be given to consumer using
> > >> > seek(...),
> > >> > > is
> > >> > > > >> > different from the partition_epoch in the FetchResponse.
> > >> > > > >> >
> > >> > > > >> > Can you take another look at the latest KIP?
> > >> > > > >> >
> > >> > > > >> > Thanks!
> > >> > > > >> > Dong
> > >> > > > >> >
> > >> > > > >> >
> > >> > > > >> >
> > >> > > > >> > On Wed, Jan 10, 2018 at 2:24 PM, Jun Rao <jun@confluent.io
> >
> > >> > wrote:
> > >> > > > >> >
> > >> > > > >> > > Hi, Dong,
> > >> > > > >> > >
> > >> > > > >> > > My replies are the following.
> > >> > > > >> > >
> > >> > > > >> > > 60. What you described could also work. The drawback is
> > that
> > >> we
> > >> > > will
> > >> > > > >> be
> > >> > > > >> > > unnecessarily changing the partition epoch when a
> partition
> > >> > hasn't
> > >> > > > >> really
> > >> > > > >> > > changed. I was imagining that the partition epoch will be
> > >> stored
> > >> > > in
> > >> > > > >> > > /brokers/topics/[topic]/partitions/[partitionId],
> instead
> > >> of at
> > >> > > the
> > >> > > > >> > topic
> > >> > > > >> > > level. So, not sure if ZK size limit is an issue.
> > >> > > > >> > >
> > >> > > > >> > > 61, 62 and 65. To me, the offset + offset_epoch is a
> unique
> > >> > > > identifier
> > >> > > > >> > for
> > >> > > > >> > > a message. So, if a message hasn't changed, the offset
> and
> > >> the
> > >> > > > >> associated
> > >> > > > >> > > offset_epoch ideally should remain the same (it will be
> > kind
> > >> of
> > >> > > > weird
> > >> > > > >> if
> > >> > > > >> > > two consumer apps save the offset on the same message,
> but
> > >> the
> > >> > > > >> > offset_epoch
> > >> > > > >> > > are different). partition_epoch + leader_epoch give us
> > that.
> > >> > > > >> > global_epoch +
> > >> > > > >> > > leader_epoch don't. If we use this approach, we can solve
> > not
> > >> > only
> > >> > > > the
> > >> > > > >> > > problem that you have identified, but also other problems
> > >> when
> > >> > > there
> > >> > > > >> is
> > >> > > > >> > > data loss or topic re-creation more reliably. For
> example,
> > in
> > >> > the
> > >> > > > >> future,
> > >> > > > >> > > if we include the partition_epoch and leader_epoch in the
> > >> fetch
> > >> > > > >> request,
> > >> > > > >> > > the server can do a more reliable check of whether that
> > >> offset
> > >> > is
> > >> > > > >> valid
> > >> > > > >> > or
> > >> > > > >> > > not. I am not sure that we can rely upon all external
> > >> offsets to
> > >> > > be
> > >> > > > >> > removed
> > >> > > > >> > > on topic deletion. For example, a topic may be deleted by
> > an
> > >> > admin
> > >> > > > who
> > >> > > > >> > may
> > >> > > > >> > > not know all the applications.
> > >> > > > >> > >
> > >> > > > >> > > If we agree on the above, the second question is then how
> > to
> > >> > > > reliably
> > >> > > > >> > > propagate the partition_epoch and the leader_epoch to the
> > >> > consumer
> > >> > > > >> when
> > >> > > > >> > > there are leader or partition changes. The leader_epoch
> > comes
> > >> > from
> > >> > > > the
> > >> > > > >> > > message, which is reliable. So, I was suggesting that
> when
> > we
> > >> > > store
> > >> > > > an
> > >> > > > >> > > offset, we can just store the leader_epoch from the
> message
> > >> set
> > >> > > > >> > containing
> > >> > > > >> > > that offset. Similarly, I was thinking that if the
> > >> > partition_epoch
> > >> > > > is
> > >> > > > >> in
> > >> > > > >> > > the fetch response, we can propagate partition_epoch
> > reliably
> > >> > > where
> > >> > > > is
> > >> > > > >> > > partition_epoch change.
> > >> > > > >> > >
> > >> > > > >> > > 63. My point is that once a leader is producing a message
> > in
> > >> the
> > >> > > new
> > >> > > > >> > > partition_epoch, ideally, we should associate the new
> > offsets
> > >> > with
> > >> > > > the
> > >> > > > >> > new
> > >> > > > >> > > partition_epoch. Otherwise, the offset_epoch won't be the
> > >> > correct
> > >> > > > >> unique
> > >> > > > >> > > identifier (useful for solving other problems mentioned
> > >> above).
> > >> > I
> > >> > > > was
> > >> > > > >> > > originally thinking that the leader will include the
> > >> > > partition_epoch
> > >> > > > >> in
> > >> > > > >> > the
> > >> > > > >> > > metadata cache in the fetch response. It's just that
> right
> > >> now,
> > >> > > > >> metadata
> > >> > > > >> > > cache is updated on UpdateMetadataRequest, which
> typically
> > >> > happens
> > >> > > > >> after
> > >> > > > >> > > the LeaderAndIsrRequest. Another approach is for the
> leader
> > >> to
> > >> > > cache
> > >> > > > >> the
> > >> > > > >> > > partition_epoch in the Partition object and return that
> > >> (instead
> > >> > > of
> > >> > > > >> the
> > >> > > > >> > one
> > >> > > > >> > > in metadata cache) in the fetch response.
> > >> > > > >> > >
> > >> > > > >> > > 65. It seems to me that the global_epoch and the
> > >> partition_epoch
> > >> > > > have
> > >> > > > >> > > different purposes. A partition_epoch has the benefit
> that
> > it
> > >> > (1)
> > >> > > > can
> > >> > > > >> be
> > >> > > > >> > > used to form a unique identifier for a message and (2)
> can
> > be
> > >> > used
> > >> > > > to
> > >> > > > >> > > solve other
> > >> > > > >> > > corner case problems in the future. I am not sure having
> > >> just a
> > >> > > > >> > > global_epoch can achieve these. global_epoch is useful to
> > >> > > determine
> > >> > > > >> which
> > >> > > > >> > > version of the metadata is newer, especially with topic
> > >> > deletion.
> > >> > > > >> > >
> > >> > > > >> > > Thanks,
> > >> > > > >> > >
> > >> > > > >> > > Jun
> > >> > > > >> > >
> > >> > > > >> > > On Tue, Jan 9, 2018 at 11:34 PM, Dong Lin <
> > >> lindong28@gmail.com>
> > >> > > > >> wrote:
> > >> > > > >> > >
> > >> > > > >> > > > Regarding the use of the global epoch in 65), it is
> very
> > >> > similar
> > >> > > > to
> > >> > > > >> the
> > >> > > > >> > > > proposal of the metadata_epoch we discussed earlier.
> The
> > >> main
> > >> > > > >> > difference
> > >> > > > >> > > is
> > >> > > > >> > > > that this epoch is incremented when we
> > create/expand/delete
> > >> > > topic
> > >> > > > >> and
> > >> > > > >> > > does
> > >> > > > >> > > > not change when controller re-send metadata.
> > >> > > > >> > > >
> > >> > > > >> > > > I looked at our previous discussion. It seems that we
> > >> prefer
> > >> > > > >> > > > partition_epoch over the metadata_epoch because 1) we
> > >> prefer
> > >> > not
> > >> > > > to
> > >> > > > >> > have
> > >> > > > >> > > an
> > >> > > > >> > > > ever growing metadata_epoch and 2) we can reset offset
> > >> better
> > >> > > when
> > >> > > > >> > topic
> > >> > > > >> > > is
> > >> > > > >> > > > re-created. The use of global topic_epoch avoids the
> > >> drawback
> > >> > of
> > >> > > > an
> > >> > > > >> > ever
> > >> > > > >> > > > quickly ever growing metadata_epoch. Though the global
> > >> epoch
> > >> > > does
> > >> > > > >> not
> > >> > > > >> > > allow
> > >> > > > >> > > > us to recognize the invalid offset committed before the
> > >> topic
> > >> > > > >> > > re-creation,
> > >> > > > >> > > > we can probably just delete the offset when we delete a
> > >> topic.
> > >> > > > Thus
> > >> > > > >> I
> > >> > > > >> > am
> > >> > > > >> > > > not very sure whether it is still worthwhile to have a
> > >> > > > per-partition
> > >> > > > >> > > > partition_epoch if the metadata already has the global
> > >> epoch.
> > >> > > > >> > > >
> > >> > > > >> > > >
> > >> > > > >> > > > On Tue, Jan 9, 2018 at 6:58 PM, Dong Lin <
> > >> lindong28@gmail.com
> > >> > >
> > >> > > > >> wrote:
> > >> > > > >> > > >
> > >> > > > >> > > > > Hey Jun,
> > >> > > > >> > > > >
> > >> > > > >> > > > > Thanks so much. These comments very useful. Please
> see
> > >> below
> > >> > > my
> > >> > > > >> > > comments.
> > >> > > > >> > > > >
> > >> > > > >> > > > > On Mon, Jan 8, 2018 at 5:52 PM, Jun Rao <
> > >> jun@confluent.io>
> > >> > > > wrote:
> > >> > > > >> > > > >
> > >> > > > >> > > > >> Hi, Dong,
> > >> > > > >> > > > >>
> > >> > > > >> > > > >> Thanks for the updated KIP. A few more comments.
> > >> > > > >> > > > >>
> > >> > > > >> > > > >> 60. Perhaps having a partition epoch is more
> flexible
> > >> since
> > >> > > in
> > >> > > > >> the
> > >> > > > >> > > > future,
> > >> > > > >> > > > >> we may support deleting a partition as well.
> > >> > > > >> > > > >>
> > >> > > > >> > > > >
> > >> > > > >> > > > > Yeah I have considered this. I think we can probably
> > >> still
> > >> > > > support
> > >> > > > >> > > > > deleting a partition by using the topic_epoch -- when
> > >> > > partition
> > >> > > > >> of a
> > >> > > > >> > > > topic
> > >> > > > >> > > > > is deleted or created, epoch of all partitions of
> this
> > >> topic
> > >> > > > will
> > >> > > > >> be
> > >> > > > >> > > > > incremented by 1. Therefore, if that partition is
> > >> re-created
> > >> > > > >> later,
> > >> > > > >> > the
> > >> > > > >> > > > > epoch of that partition will still be larger than its
> > >> epoch
> > >> > > > before
> > >> > > > >> > the
> > >> > > > >> > > > > deletion, which still allows the client to order the
> > >> > metadata
> > >> > > > for
> > >> > > > >> the
> > >> > > > >> > > > > purpose of this KIP. Does this sound reasonable?
> > >> > > > >> > > > >
> > >> > > > >> > > > > The advantage of using topic_epoch instead of
> > >> > partition_epoch
> > >> > > is
> > >> > > > >> that
> > >> > > > >> > > the
> > >> > > > >> > > > > size of the /brokers/topics/[topic] znode and
> > >> > request/response
> > >> > > > >> size
> > >> > > > >> > can
> > >> > > > >> > > > be
> > >> > > > >> > > > > smaller. We have a limit on the maximum size of znode
> > >> > > (typically
> > >> > > > >> > 1MB).
> > >> > > > >> > > > Use
> > >> > > > >> > > > > partition epoch can effectively reduce the number of
> > >> > > partitions
> > >> > > > >> that
> > >> > > > >> > > can
> > >> > > > >> > > > be
> > >> > > > >> > > > > described by the /brokers/topics/[topic] znode.
> > >> > > > >> > > > >
> > >> > > > >> > > > > One use-case of partition_epoch for client to detect
> > that
> > >> > the
> > >> > > > >> > committed
> > >> > > > >> > > > > offset, either from kafka offset topic or from the
> > >> external
> > >> > > > store
> > >> > > > >> is
> > >> > > > >> > > > > invalid after partition deletion and re-creation.
> > >> However,
> > >> > it
> > >> > > > >> seems
> > >> > > > >> > > that
> > >> > > > >> > > > we
> > >> > > > >> > > > > can also address this use-case with other approaches.
> > For
> > >> > > > example,
> > >> > > > >> > when
> > >> > > > >> > > > > AdminClient deletes partitions, it can also delete
> the
> > >> > > committed
> > >> > > > >> > > offsets
> > >> > > > >> > > > > for those partitions from the offset topic. If user
> > >> stores
> > >> > > > offset
> > >> > > > >> > > > > externally, it might make sense for user to similarly
> > >> remove
> > >> > > > >> offsets
> > >> > > > >> > of
> > >> > > > >> > > > > related partitions after these partitions are
> deleted.
> > >> So I
> > >> > am
> > >> > > > not
> > >> > > > >> > sure
> > >> > > > >> > > > > that we should use partition_epoch in this KIP.
> > >> > > > >> > > > >
> > >> > > > >> > > > >
> > >> > > > >> > > > >>
> > >> > > > >> > > > >> 61. It seems that the leader epoch returned in the
> > >> > position()
> > >> > > > >> call
> > >> > > > >> > > > should
> > >> > > > >> > > > >> the the leader epoch returned in the fetch response,
> > not
> > >> > the
> > >> > > > one
> > >> > > > >> in
> > >> > > > >> > > the
> > >> > > > >> > > > >> metadata cache of the client.
> > >> > > > >> > > > >
> > >> > > > >> > > > >
> > >> > > > >> > > > > I think this is a good idea. Just to double check,
> this
> > >> > change
> > >> > > > >> does
> > >> > > > >> > not
> > >> > > > >> > > > > affect the correctness or performance of this KIP.
> But
> > it
> > >> > can
> > >> > > be
> > >> > > > >> > useful
> > >> > > > >> > > > if
> > >> > > > >> > > > > we want to use the leader_epoch to better handle the
> > >> offset
> > >> > > rest
> > >> > > > >> in
> > >> > > > >> > > case
> > >> > > > >> > > > of
> > >> > > > >> > > > > unclean leader election, which is listed in the
> future
> > >> work.
> > >> > > Is
> > >> > > > >> this
> > >> > > > >> > > > > understanding correct?
> > >> > > > >> > > > >
> > >> > > > >> > > > > I have updated the KIP to specify that the
> leader_epoch
> > >> > > returned
> > >> > > > >> by
> > >> > > > >> > > > > position() should be the largest leader_epoch of
> those
> > >> > already
> > >> > > > >> > consumed
> > >> > > > >> > > > > messages whose offset < position. If no message has
> > been
> > >> > > > consumed
> > >> > > > >> > since
> > >> > > > >> > > > > consumer initialization, the leader_epoch from seek()
> > or
> > >> > > > >> > > > > OffsetFetchResponse should be used. The offset
> included
> > >> in
> > >> > the
> > >> > > > >> > > > > OffsetCommitRequest will also be determined in the
> > >> similar
> > >> > > > manner.
> > >> > > > >> > > > >
> > >> > > > >> > > > >
> > >> > > > >> > > > >>
> > >> > > > >> > > > >> 62. I am wondering if we should return the partition
> > >> epoch
> > >> > in
> > >> > > > the
> > >> > > > >> > > fetch
> > >> > > > >> > > > >> response as well. In the current proposal, if a
> topic
> > is
> > >> > > > >> recreated
> > >> > > > >> > and
> > >> > > > >> > > > the
> > >> > > > >> > > > >> new leader is on the same broker as the old one,
> there
> > >> is
> > >> > > > >> nothing to
> > >> > > > >> > > > force
> > >> > > > >> > > > >> the metadata refresh in the client. So, the client
> may
> > >> > still
> > >> > > > >> > associate
> > >> > > > >> > > > the
> > >> > > > >> > > > >> offset with the old partition epoch.
> > >> > > > >> > > > >>
> > >> > > > >> > > > >
> > >> > > > >> > > > > Could you help me understand the problem if a client
> > >> > > associates
> > >> > > > >> old
> > >> > > > >> > > > > partition_epoch (or the topic_epoch as of the current
> > >> KIP)
> > >> > > with
> > >> > > > >> the
> > >> > > > >> > > > offset?
> > >> > > > >> > > > > The main purpose of the topic_epoch is to be able to
> > drop
> > >> > > > >> > leader_epoch
> > >> > > > >> > > > to 0
> > >> > > > >> > > > > after a partition is deleted and re-created. I guess
> > you
> > >> may
> > >> > > be
> > >> > > > >> > > thinking
> > >> > > > >> > > > > about using the partition_epoch to detect that the
> > >> committed
> > >> > > > >> offset
> > >> > > > >> > is
> > >> > > > >> > > > > invalid? In that case, I am wondering if the
> > alternative
> > >> > > > approach
> > >> > > > >> > > > described
> > >> > > > >> > > > > in 60) would be reasonable.
> > >> > > > >> > > > >
> > >> > > > >> > > > >
> > >> > > > >> > > > >>
> > >> > > > >> > > > >> 63. There is some subtle coordination between the
> > >> > > > >> > LeaderAndIsrRequest
> > >> > > > >> > > > and
> > >> > > > >> > > > >> UpdateMetadataRequest. Currently, when a leader
> > changes,
> > >> > the
> > >> > > > >> > > controller
> > >> > > > >> > > > >> first sends the LeaderAndIsrRequest to the assigned
> > >> > replicas
> > >> > > > and
> > >> > > > >> the
> > >> > > > >> > > > >> UpdateMetadataRequest to every broker. So, there
> could
> > >> be a
> > >> > > > small
> > >> > > > >> > > window
> > >> > > > >> > > > >> when the leader already receives the new partition
> > >> epoch in
> > >> > > the
> > >> > > > >> > > > >> LeaderAndIsrRequest, but the metadata cache in the
> > >> broker
> > >> > > > hasn't
> > >> > > > >> > been
> > >> > > > >> > > > >> updated with the latest partition epoch. Not sure
> > what's
> > >> > the
> > >> > > > best
> > >> > > > >> > way
> > >> > > > >> > > to
> > >> > > > >> > > > >> address this issue. Perhaps we can update the
> metadata
> > >> > cache
> > >> > > on
> > >> > > > >> the
> > >> > > > >> > > > broker
> > >> > > > >> > > > >> with both LeaderAndIsrRequest and
> > UpdateMetadataRequest.
> > >> > The
> > >> > > > >> > challenge
> > >> > > > >> > > > is
> > >> > > > >> > > > >> that the two have slightly different data. For
> > example,
> > >> > only
> > >> > > > the
> > >> > > > >> > > latter
> > >> > > > >> > > > >> has
> > >> > > > >> > > > >> all endpoints.
> > >> > > > >> > > > >>
> > >> > > > >> > > > >
> > >> > > > >> > > > > I am not sure whether this is a problem. Could you
> > >> explain a
> > >> > > bit
> > >> > > > >> more
> > >> > > > >> > > > what
> > >> > > > >> > > > > specific problem this small window can cause?
> > >> > > > >> > > > >
> > >> > > > >> > > > > Since client can fetch metadata from any broker in
> the
> > >> > > cluster,
> > >> > > > >> and
> > >> > > > >> > > given
> > >> > > > >> > > > > that different brokers receive request (e.g.
> > >> > > LeaderAndIsrRequest
> > >> > > > >> and
> > >> > > > >> > > > > UpdateMetadataRequest) in arbitrary order, the
> metadata
> > >> > > received
> > >> > > > >> by
> > >> > > > >> > > > client
> > >> > > > >> > > > > can be in arbitrary order (either newer or older)
> > >> compared
> > >> > to
> > >> > > > the
> > >> > > > >> > > > broker's
> > >> > > > >> > > > > leadership state even if a given broker receives
> > >> > > > >> LeaderAndIsrRequest
> > >> > > > >> > > and
> > >> > > > >> > > > > UpdateMetadataRequest simultaneously. So I am not
> sure
> > >> it is
> > >> > > > >> useful
> > >> > > > >> > to
> > >> > > > >> > > > > update broker's cache with LeaderAndIsrRequest.
> > >> > > > >> > > > >
> > >> > > > >> > > > >
> > >> > > > >> > > > >> 64. The enforcement of leader epoch in Offset
> commit:
> > We
> > >> > > allow
> > >> > > > a
> > >> > > > >> > > > consumer
> > >> > > > >> > > > >> to set an arbitrary offset. So it's possible for
> > >> offsets or
> > >> > > > >> leader
> > >> > > > >> > > epoch
> > >> > > > >> > > > >> to
> > >> > > > >> > > > >> go backwards. I am not sure if we could always
> enforce
> > >> that
> > >> > > the
> > >> > > > >> > leader
> > >> > > > >> > > > >> epoch only goes up on the broker.
> > >> > > > >> > > > >>
> > >> > > > >> > > > >
> > >> > > > >> > > > > Sure. I have removed this check from the KIP.
> > >> > > > >> > > > >
> > >> > > > >> > > > > BTW, we can probably still ensure that the
> leader_epoch
> > >> > always
> > >> > > > >> > increase
> > >> > > > >> > > > if
> > >> > > > >> > > > > the leader_epoch used with offset commit is the
> > >> > > max(leader_epoch
> > >> > > > >> of
> > >> > > > >> > the
> > >> > > > >> > > > > message with offset = the committed offset - 1, the
> > >> largest
> > >> > > > known
> > >> > > > >> > > > > leader_epoch from the metadata). But I don't have a
> > good
> > >> > > > use-case
> > >> > > > >> for
> > >> > > > >> > > > this
> > >> > > > >> > > > > alternative definition. So I choose the keep the KIP
> > >> simple
> > >> > by
> > >> > > > >> > > requiring
> > >> > > > >> > > > > leader_epoch to always increase.
> > >> > > > >> > > > >
> > >> > > > >> > > > >
> > >> > > > >> > > > >> 65. Good point on handling missing partition epoch
> due
> > >> to
> > >> > > topic
> > >> > > > >> > > > deletion.
> > >> > > > >> > > > >> Another potential way to address this is to
> > additionally
> > >> > > > >> propagate
> > >> > > > >> > the
> > >> > > > >> > > > >> global partition epoch to brokers and the clients.
> > This
> > >> > way,
> > >> > > > >> when a
> > >> > > > >> > > > >> partition epoch is missing, we can use the global
> > >> partition
> > >> > > > >> epoch to
> > >> > > > >> > > > >> reason
> > >> > > > >> > > > >> about which metadata is more recent.
> > >> > > > >> > > > >>
> > >> > > > >> > > > >
> > >> > > > >> > > > > This is a great idea. The global epoch can be used to
> > >> order
> > >> > > the
> > >> > > > >> > > metadata
> > >> > > > >> > > > > and help us recognize the more recent metadata if a
> > topic
> > >> > (or
> > >> > > > >> > > partition)
> > >> > > > >> > > > is
> > >> > > > >> > > > > deleted and re-created.
> > >> > > > >> > > > >
> > >> > > > >> > > > > Actually, it seems we only need to propagate the
> global
> > >> > epoch
> > >> > > to
> > >> > > > >> > > brokers
> > >> > > > >> > > > > and clients without propagating this epoch on a
> > >> per-topic or
> > >> > > > >> > > > per-partition
> > >> > > > >> > > > > basic. Doing so would simply interface changes made
> > this
> > >> > KIP.
> > >> > > > Does
> > >> > > > >> > this
> > >> > > > >> > > > > approach sound reasonable?
> > >> > > > >> > > > >
> > >> > > > >> > > > >
> > >> > > > >> > > > >> 66. A client may also get an offset by time using
> the
> > >> > > > >> > offsetForTimes()
> > >> > > > >> > > > >> api.
> > >> > > > >> > > > >> So, we probably want to include
> offsetInternalMetadata
> > >> in
> > >> > > > >> > > > >> OffsetAndTimestamp
> > >> > > > >> > > > >> as well.
> > >> > > > >> > > > >>
> > >> > > > >> > > > >
> > >> > > > >> > > > > You are right. This probably also requires us to
> change
> > >> the
> > >> > > > >> > > > > ListOffsetRequest as well. I will update the KIP
> after
> > we
> > >> > > agree
> > >> > > > on
> > >> > > > >> > the
> > >> > > > >> > > > > solution for 65).
> > >> > > > >> > > > >
> > >> > > > >> > > > >
> > >> > > > >> > > > >>
> > >> > > > >> > > > >> 67. InteralMetadata can be a bit confusing with the
> > >> > metadata
> > >> > > > >> field
> > >> > > > >> > > > already
> > >> > > > >> > > > >> there. Perhaps we can just call it OffsetEpoch. It
> > >> might be
> > >> > > > >> useful
> > >> > > > >> > to
> > >> > > > >> > > > make
> > >> > > > >> > > > >> OffsetEpoch printable at least for debugging
> purpose.
> > >> Once
> > >> > > you
> > >> > > > do
> > >> > > > >> > > that,
> > >> > > > >> > > > we
> > >> > > > >> > > > >> are already exposing the internal fields. So, not
> sure
> > >> if
> > >> > > it's
> > >> > > > >> worth
> > >> > > > >> > > > >> hiding
> > >> > > > >> > > > >> them. If we do want to hide them, perhaps we can
> have
> > >> sth
> > >> > > like
> > >> > > > >> the
> > >> > > > >> > > > >> following. The binary encoding is probably more
> > >> efficient
> > >> > > than
> > >> > > > >> JSON
> > >> > > > >> > > for
> > >> > > > >> > > > >> external storage.
> > >> > > > >> > > > >>
> > >> > > > >> > > > >> OffsetEpoch {
> > >> > > > >> > > > >>  static OffsetEpoch decode(byte[]);
> > >> > > > >> > > > >>
> > >> > > > >> > > > >>   public byte[] encode();
> > >> > > > >> > > > >>
> > >> > > > >> > > > >>   public String toString();
> > >> > > > >> > > > >> }
> > >> > > > >> > > > >>
> > >> > > > >> > > > >
> > >> > > > >> > > > > Thanks much. I like this solution. I have updated the
> > KIP
> > >> > > > >> > accordingly.
> > >> > > > >> > > > >
> > >> > > > >> > > > >
> > >> > > > >> > > > >
> > >> > > > >> > > > >>
> > >> > > > >> > > > >> Jun
> > >> > > > >> > > > >>
> > >> > > > >> > > > >> On Mon, Jan 8, 2018 at 4:22 PM, Dong Lin <
> > >> > > lindong28@gmail.com>
> > >> > > > >> > wrote:
> > >> > > > >> > > > >>
> > >> > > > >> > > > >> > Hey Jason,
> > >> > > > >> > > > >> >
> > >> > > > >> > > > >> > Certainly. This sounds good. I have updated the
> KIP
> > to
> > >> > > > clarity
> > >> > > > >> > that
> > >> > > > >> > > > the
> > >> > > > >> > > > >> > global epoch will be incremented by 1 each time a
> > >> topic
> > >> > is
> > >> > > > >> > deleted.
> > >> > > > >> > > > >> >
> > >> > > > >> > > > >> > Thanks,
> > >> > > > >> > > > >> > Dong
> > >> > > > >> > > > >> >
> > >> > > > >> > > > >> > On Mon, Jan 8, 2018 at 4:09 PM, Jason Gustafson <
> > >> > > > >> > jason@confluent.io
> > >> > > > >> > > >
> > >> > > > >> > > > >> > wrote:
> > >> > > > >> > > > >> >
> > >> > > > >> > > > >> > > Hi Dong,
> > >> > > > >> > > > >> > >
> > >> > > > >> > > > >> > >
> > >> > > > >> > > > >> > > I think your approach will allow user to
> > distinguish
> > >> > > > between
> > >> > > > >> the
> > >> > > > >> > > > >> metadata
> > >> > > > >> > > > >> > > > before and after the topic deletion. I also
> > agree
> > >> > that
> > >> > > > this
> > >> > > > >> > can
> > >> > > > >> > > be
> > >> > > > >> > > > >> > > > potentially be useful to user. I am just not
> > very
> > >> > sure
> > >> > > > >> whether
> > >> > > > >> > > we
> > >> > > > >> > > > >> > already
> > >> > > > >> > > > >> > > > have a good use-case to make the additional
> > >> > complexity
> > >> > > > >> > > worthwhile.
> > >> > > > >> > > > >> It
> > >> > > > >> > > > >> > > seems
> > >> > > > >> > > > >> > > > that this feature is kind of independent of
> the
> > >> main
> > >> > > > >> problem
> > >> > > > >> > of
> > >> > > > >> > > > this
> > >> > > > >> > > > >> > KIP.
> > >> > > > >> > > > >> > > > Could we add this as a future work?
> > >> > > > >> > > > >> > >
> > >> > > > >> > > > >> > >
> > >> > > > >> > > > >> > > Do you think it's fair if we bump the topic
> epoch
> > on
> > >> > > > deletion
> > >> > > > >> > and
> > >> > > > >> > > > >> leave
> > >> > > > >> > > > >> > > propagation of the epoch for deleted topics for
> > >> future
> > >> > > > work?
> > >> > > > >> I
> > >> > > > >> > > don't
> > >> > > > >> > > > >> > think
> > >> > > > >> > > > >> > > this adds much complexity and it makes the
> > behavior
> > >> > > > >> consistent:
> > >> > > > >> > > > every
> > >> > > > >> > > > >> > topic
> > >> > > > >> > > > >> > > mutation results in an epoch bump.
> > >> > > > >> > > > >> > >
> > >> > > > >> > > > >> > > Thanks,
> > >> > > > >> > > > >> > > Jason
> > >> > > > >> > > > >> > >
> > >> > > > >> > > > >> > > On Mon, Jan 8, 2018 at 3:14 PM, Dong Lin <
> > >> > > > >> lindong28@gmail.com>
> > >> > > > >> > > > wrote:
> > >> > > > >> > > > >> > >
> > >> > > > >> > > > >> > > > Hey Ismael,
> > >> > > > >> > > > >> > > >
> > >> > > > >> > > > >> > > > I guess we actually need user to see this
> field
> > so
> > >> > that
> > >> > > > >> user
> > >> > > > >> > can
> > >> > > > >> > > > >> store
> > >> > > > >> > > > >> > > this
> > >> > > > >> > > > >> > > > value in the external store together with the
> > >> offset.
> > >> > > We
> > >> > > > >> just
> > >> > > > >> > > > prefer
> > >> > > > >> > > > >> > the
> > >> > > > >> > > > >> > > > value to be opaque to discourage most users
> from
> > >> > > > >> interpreting
> > >> > > > >> > > this
> > >> > > > >> > > > >> > value.
> > >> > > > >> > > > >> > > > One more advantage of using such an opaque
> field
> > >> is
> > >> > to
> > >> > > be
> > >> > > > >> able
> > >> > > > >> > > to
> > >> > > > >> > > > >> > evolve
> > >> > > > >> > > > >> > > > the information (or schema) of this value
> > without
> > >> > > > changing
> > >> > > > >> > > > consumer
> > >> > > > >> > > > >> API
> > >> > > > >> > > > >> > > in
> > >> > > > >> > > > >> > > > the future.
> > >> > > > >> > > > >> > > >
> > >> > > > >> > > > >> > > > I also thinking it is probably OK for user to
> be
> > >> able
> > >> > > to
> > >> > > > >> > > interpret
> > >> > > > >> > > > >> this
> > >> > > > >> > > > >> > > > value, particularly for those advanced users.
> > >> > > > >> > > > >> > > >
> > >> > > > >> > > > >> > > > Thanks,
> > >> > > > >> > > > >> > > > Dong
> > >> > > > >> > > > >> > > >
> > >> > > > >> > > > >> > > > On Mon, Jan 8, 2018 at 2:34 PM, Ismael Juma <
> > >> > > > >> > ismael@juma.me.uk>
> > >> > > > >> > > > >> wrote:
> > >> > > > >> > > > >> > > >
> > >> > > > >> > > > >> > > > > On Fri, Jan 5, 2018 at 7:15 PM, Jason
> > Gustafson
> > >> <
> > >> > > > >> > > > >> jason@confluent.io>
> > >> > > > >> > > > >> > > > > wrote:
> > >> > > > >> > > > >> > > > > >
> > >> > > > >> > > > >> > > > > > class OffsetAndMetadata {
> > >> > > > >> > > > >> > > > > >   long offset;
> > >> > > > >> > > > >> > > > > >   byte[] offsetMetadata;
> > >> > > > >> > > > >> > > > > >   String metadata;
> > >> > > > >> > > > >> > > > > > }
> > >> > > > >> > > > >> > > > >
> > >> > > > >> > > > >> > > > >
> > >> > > > >> > > > >> > > > > > Admittedly, the naming is a bit annoying,
> > but
> > >> we
> > >> > > can
> > >> > > > >> > > probably
> > >> > > > >> > > > >> come
> > >> > > > >> > > > >> > up
> > >> > > > >> > > > >> > > > > with
> > >> > > > >> > > > >> > > > > > something better. Internally the byte
> array
> > >> would
> > >> > > > have
> > >> > > > >> a
> > >> > > > >> > > > >> version.
> > >> > > > >> > > > >> > If
> > >> > > > >> > > > >> > > in
> > >> > > > >> > > > >> > > > > the
> > >> > > > >> > > > >> > > > > > future we have anything else we need to
> add,
> > >> we
> > >> > can
> > >> > > > >> update
> > >> > > > >> > > the
> > >> > > > >> > > > >> > > version
> > >> > > > >> > > > >> > > > > and
> > >> > > > >> > > > >> > > > > > we wouldn't need any new APIs.
> > >> > > > >> > > > >> > > > > >
> > >> > > > >> > > > >> > > > >
> > >> > > > >> > > > >> > > > > We can also add fields to a class in a
> > >> compatible
> > >> > > way.
> > >> > > > >> So,
> > >> > > > >> > it
> > >> > > > >> > > > >> seems
> > >> > > > >> > > > >> > to
> > >> > > > >> > > > >> > > me
> > >> > > > >> > > > >> > > > > that the main advantage of the byte array is
> > >> that
> > >> > > it's
> > >> > > > >> > opaque
> > >> > > > >> > > to
> > >> > > > >> > > > >> the
> > >> > > > >> > > > >> > > > user.
> > >> > > > >> > > > >> > > > > Is that correct? If so, we could also add
> any
> > >> > opaque
> > >> > > > >> > metadata
> > >> > > > >> > > > in a
> > >> > > > >> > > > >> > > > subclass
> > >> > > > >> > > > >> > > > > so that users don't even see it (unless they
> > >> cast
> > >> > it,
> > >> > > > but
> > >> > > > >> > then
> > >> > > > >> > > > >> > they're
> > >> > > > >> > > > >> > > on
> > >> > > > >> > > > >> > > > > their own).
> > >> > > > >> > > > >> > > > >
> > >> > > > >> > > > >> > > > > Ismael
> > >> > > > >> > > > >> > > > >
> > >> > > > >> > > > >> > > > > The corresponding seek() and position() APIs
> > >> might
> > >> > > look
> > >> > > > >> > > > something
> > >> > > > >> > > > >> > like
> > >> > > > >> > > > >> > > > > this:
> > >> > > > >> > > > >> > > > > >
> > >> > > > >> > > > >> > > > > > void seek(TopicPartition partition, long
> > >> offset,
> > >> > > > byte[]
> > >> > > > >> > > > >> > > > offsetMetadata);
> > >> > > > >> > > > >> > > > > > byte[] positionMetadata(TopicPartition
> > >> > partition);
> > >> > > > >> > > > >> > > > > >
> > >> > > > >> > > > >> > > > > > What do you think?
> > >> > > > >> > > > >> > > > > >
> > >> > > > >> > > > >> > > > > > Thanks,
> > >> > > > >> > > > >> > > > > > Jason
> > >> > > > >> > > > >> > > > > >
> > >> > > > >> > > > >> > > > > > On Thu, Jan 4, 2018 at 7:04 PM, Dong Lin <
> > >> > > > >> > > lindong28@gmail.com
> > >> > > > >> > > > >
> > >> > > > >> > > > >> > > wrote:
> > >> > > > >> > > > >> > > > > >
> > >> > > > >> > > > >> > > > > > > Hey Jun, Jason,
> > >> > > > >> > > > >> > > > > > >
> > >> > > > >> > > > >> > > > > > > Thanks much for all the feedback. I have
> > >> > updated
> > >> > > > the
> > >> > > > >> KIP
> > >> > > > >> > > > >> based on
> > >> > > > >> > > > >> > > the
> > >> > > > >> > > > >> > > > > > > latest discussion. Can you help check
> > >> whether
> > >> > it
> > >> > > > >> looks
> > >> > > > >> > > good?
> > >> > > > >> > > > >> > > > > > >
> > >> > > > >> > > > >> > > > > > > Thanks,
> > >> > > > >> > > > >> > > > > > > Dong
> > >> > > > >> > > > >> > > > > > >
> > >> > > > >> > > > >> > > > > > > On Thu, Jan 4, 2018 at 5:36 PM, Dong
> Lin <
> > >> > > > >> > > > lindong28@gmail.com
> > >> > > > >> > > > >> >
> > >> > > > >> > > > >> > > > wrote:
> > >> > > > >> > > > >> > > > > > >
> > >> > > > >> > > > >> > > > > > > > Hey Jun,
> > >> > > > >> > > > >> > > > > > > >
> > >> > > > >> > > > >> > > > > > > > Hmm... thinking about this more, I am
> > not
> > >> > sure
> > >> > > > that
> > >> > > > >> > the
> > >> > > > >> > > > >> > proposed
> > >> > > > >> > > > >> > > > API
> > >> > > > >> > > > >> > > > > is
> > >> > > > >> > > > >> > > > > > > > sufficient. For users that store
> offset
> > >> > > > >> externally, we
> > >> > > > >> > > > >> probably
> > >> > > > >> > > > >> > > > need
> > >> > > > >> > > > >> > > > > > > extra
> > >> > > > >> > > > >> > > > > > > > API to return the leader_epoch and
> > >> > > > partition_epoch
> > >> > > > >> for
> > >> > > > >> > > all
> > >> > > > >> > > > >> > > > partitions
> > >> > > > >> > > > >> > > > > > > that
> > >> > > > >> > > > >> > > > > > > > consumers are consuming. I suppose
> these
> > >> > users
> > >> > > > >> > currently
> > >> > > > >> > > > use
> > >> > > > >> > > > >> > > > > position()
> > >> > > > >> > > > >> > > > > > > to
> > >> > > > >> > > > >> > > > > > > > get the offset. Thus we probably need
> a
> > >> new
> > >> > > > method
> > >> > > > >> > > > >> > > > > > positionWithEpoch(..)
> > >> > > > >> > > > >> > > > > > > to
> > >> > > > >> > > > >> > > > > > > > return <offset, partition_epoch,
> > >> > leader_epoch>.
> > >> > > > >> Does
> > >> > > > >> > > this
> > >> > > > >> > > > >> sound
> > >> > > > >> > > > >> > > > > > > reasonable?
> > >> > > > >> > > > >> > > > > > > >
> > >> > > > >> > > > >> > > > > > > > Thanks,
> > >> > > > >> > > > >> > > > > > > > Dong
> > >> > > > >> > > > >> > > > > > > >
> > >> > > > >> > > > >> > > > > > > >
> > >> > > > >> > > > >> > > > > > > > On Thu, Jan 4, 2018 at 5:26 PM, Jun
> Rao
> > <
> > >> > > > >> > > jun@confluent.io
> > >> > > > >> > > > >
> > >> > > > >> > > > >> > > wrote:
> > >> > > > >> > > > >> > > > > > > >
> > >> > > > >> > > > >> > > > > > > >> Hi, Dong,
> > >> > > > >> > > > >> > > > > > > >>
> > >> > > > >> > > > >> > > > > > > >> Yes, that's what I am thinking.
> > >> OffsetEpoch
> > >> > > will
> > >> > > > >> be
> > >> > > > >> > > > >> composed
> > >> > > > >> > > > >> > of
> > >> > > > >> > > > >> > > > > > > >> (partition_epoch,
> > >> > > > >> > > > >> > > > > > > >> leader_epoch).
> > >> > > > >> > > > >> > > > > > > >>
> > >> > > > >> > > > >> > > > > > > >> Thanks,
> > >> > > > >> > > > >> > > > > > > >>
> > >> > > > >> > > > >> > > > > > > >> Jun
> > >> > > > >> > > > >> > > > > > > >>
> > >> > > > >> > > > >> > > > > > > >>
> > >> > > > >> > > > >> > > > > > > >> On Thu, Jan 4, 2018 at 4:22 PM, Dong
> > Lin
> > >> <
> > >> > > > >> > > > >> lindong28@gmail.com
> > >> > > > >> > > > >> > >
> > >> > > > >> > > > >> > > > > wrote:
> > >> > > > >> > > > >> > > > > > > >>
> > >> > > > >> > > > >> > > > > > > >> > Hey Jun,
> > >> > > > >> > > > >> > > > > > > >> >
> > >> > > > >> > > > >> > > > > > > >> > Thanks much. I like the the new API
> > >> that
> > >> > you
> > >> > > > >> > > proposed.
> > >> > > > >> > > > I
> > >> > > > >> > > > >> am
> > >> > > > >> > > > >> > > not
> > >> > > > >> > > > >> > > > > sure
> > >> > > > >> > > > >> > > > > > > >> what
> > >> > > > >> > > > >> > > > > > > >> > you exactly mean by offset_epoch. I
> > >> > suppose
> > >> > > > >> that we
> > >> > > > >> > > can
> > >> > > > >> > > > >> use
> > >> > > > >> > > > >> > > the
> > >> > > > >> > > > >> > > > > pair
> > >> > > > >> > > > >> > > > > > > of
> > >> > > > >> > > > >> > > > > > > >> > (partition_epoch, leader_epoch) as
> > the
> > >> > > > >> > offset_epoch,
> > >> > > > >> > > > >> right?
> > >> > > > >> > > > >> > > > > > > >> >
> > >> > > > >> > > > >> > > > > > > >> > Thanks,
> > >> > > > >> > > > >> > > > > > > >> > Dong
> > >> > > > >> > > > >> > > > > > > >> >
> > >> > > > >> > > > >> > > > > > > >> > On Thu, Jan 4, 2018 at 4:02 PM, Jun
> > >> Rao <
> > >> > > > >> > > > >> jun@confluent.io>
> > >> > > > >> > > > >> > > > wrote:
> > >> > > > >> > > > >> > > > > > > >> >
> > >> > > > >> > > > >> > > > > > > >> > > Hi, Dong,
> > >> > > > >> > > > >> > > > > > > >> > >
> > >> > > > >> > > > >> > > > > > > >> > > Got it. The api that you proposed
> > >> works.
> > >> > > The
> > >> > > > >> > > question
> > >> > > > >> > > > >> is
> > >> > > > >> > > > >> > > > whether
> > >> > > > >> > > > >> > > > > > > >> that's
> > >> > > > >> > > > >> > > > > > > >> > the
> > >> > > > >> > > > >> > > > > > > >> > > api that we want to have in the
> > long
> > >> > term.
> > >> > > > My
> > >> > > > >> > > concern
> > >> > > > >> > > > >> is
> > >> > > > >> > > > >> > > that
> > >> > > > >> > > > >> > > > > > while
> > >> > > > >> > > > >> > > > > > > >> the
> > >> > > > >> > > > >> > > > > > > >> > api
> > >> > > > >> > > > >> > > > > > > >> > > change is simple, the new api
> seems
> > >> > harder
> > >> > > > to
> > >> > > > >> > > explain
> > >> > > > >> > > > >> and
> > >> > > > >> > > > >> > > use.
> > >> > > > >> > > > >> > > > > For
> > >> > > > >> > > > >> > > > > > > >> > example,
> > >> > > > >> > > > >> > > > > > > >> > > a consumer storing offsets
> > externally
> > >> > now
> > >> > > > >> needs
> > >> > > > >> > to
> > >> > > > >> > > > call
> > >> > > > >> > > > >> > > > > > > >> > > waitForMetadataUpdate() after
> > calling
> > >> > > > seek().
> > >> > > > >> > > > >> > > > > > > >> > >
> > >> > > > >> > > > >> > > > > > > >> > > An alternative approach is to
> make
> > >> the
> > >> > > > >> following
> > >> > > > >> > > > >> > compatible
> > >> > > > >> > > > >> > > > api
> > >> > > > >> > > > >> > > > > > > >> changes
> > >> > > > >> > > > >> > > > > > > >> > in
> > >> > > > >> > > > >> > > > > > > >> > > Consumer.
> > >> > > > >> > > > >> > > > > > > >> > > * Add an additional OffsetEpoch
> > >> field in
> > >> > > > >> > > > >> > OffsetAndMetadata.
> > >> > > > >> > > > >> > > > (no
> > >> > > > >> > > > >> > > > > > need
> > >> > > > >> > > > >> > > > > > > >> to
> > >> > > > >> > > > >> > > > > > > >> > > change the CommitSync() api)
> > >> > > > >> > > > >> > > > > > > >> > > * Add a new api
> seek(TopicPartition
> > >> > > > partition,
> > >> > > > >> > long
> > >> > > > >> > > > >> > offset,
> > >> > > > >> > > > >> > > > > > > >> OffsetEpoch
> > >> > > > >> > > > >> > > > > > > >> > > offsetEpoch). We can potentially
> > >> > deprecate
> > >> > > > the
> > >> > > > >> > old
> > >> > > > >> > > > api
> > >> > > > >> > > > >> > > > > > > >> > seek(TopicPartition
> > >> > > > >> > > > >> > > > > > > >> > > partition, long offset) in the
> > >> future.
> > >> > > > >> > > > >> > > > > > > >> > >
> > >> > > > >> > > > >> > > > > > > >> > > The alternative approach has
> > similar
> > >> > > amount
> > >> > > > of
> > >> > > > >> > api
> > >> > > > >> > > > >> changes
> > >> > > > >> > > > >> > > as
> > >> > > > >> > > > >> > > > > > yours
> > >> > > > >> > > > >> > > > > > > >> but
> > >> > > > >> > > > >> > > > > > > >> > has
> > >> > > > >> > > > >> > > > > > > >> > > the following benefits.
> > >> > > > >> > > > >> > > > > > > >> > > 1. The api works in a similar way
> > as
> > >> how
> > >> > > > >> offset
> > >> > > > >> > > > >> management
> > >> > > > >> > > > >> > > > works
> > >> > > > >> > > > >> > > > > > now
> > >> > > > >> > > > >> > > > > > > >> and
> > >> > > > >> > > > >> > > > > > > >> > is
> > >> > > > >> > > > >> > > > > > > >> > > probably what we want in the long
> > >> term.
> > >> > > > >> > > > >> > > > > > > >> > > 2. It can reset offsets better
> when
> > >> > there
> > >> > > is
> > >> > > > >> data
> > >> > > > >> > > > loss
> > >> > > > >> > > > >> due
> > >> > > > >> > > > >> > > to
> > >> > > > >> > > > >> > > > > > > unclean
> > >> > > > >> > > > >> > > > > > > >> > > leader election or correlated
> > replica
> > >> > > > failure.
> > >> > > > >> > > > >> > > > > > > >> > > 3. It can reset offsets better
> when
> > >> > topic
> > >> > > is
> > >> > > > >> > > > recreated.
> > >> > > > >> > > > >> > > > > > > >> > >
> > >> > > > >> > > > >> > > > > > > >> > > Thanks,
> > >> > > > >> > > > >> > > > > > > >> > >
> > >> > > > >> > > > >> > > > > > > >> > > Jun
> > >> > > > >> > > > >> > > > > > > >> > >
> > >> > > > >> > > > >> > > > > > > >> > >
> > >> > > > >> > > > >> > > > > > > >> > > On Thu, Jan 4, 2018 at 2:05 PM,
> > Dong
> > >> > Lin <
> > >> > > > >> > > > >> > > lindong28@gmail.com
> > >> > > > >> > > > >> > > > >
> > >> > > > >> > > > >> > > > > > > wrote:
> > >> > > > >> > > > >> > > > > > > >> > >
> > >> > > > >> > > > >> > > > > > > >> > > > Hey Jun,
> > >> > > > >> > > > >> > > > > > > >> > > >
> > >> > > > >> > > > >> > > > > > > >> > > > Yeah I agree that ideally we
> > don't
> > >> > want
> > >> > > an
> > >> > > > >> ever
> > >> > > > >> > > > >> growing
> > >> > > > >> > > > >> > > > global
> > >> > > > >> > > > >> > > > > > > >> metadata
> > >> > > > >> > > > >> > > > > > > >> > > > version. I just think it may be
> > >> more
> > >> > > > >> desirable
> > >> > > > >> > to
> > >> > > > >> > > > >> keep
> > >> > > > >> > > > >> > the
> > >> > > > >> > > > >> > > > > > > consumer
> > >> > > > >> > > > >> > > > > > > >> API
> > >> > > > >> > > > >> > > > > > > >> > > > simple.
> > >> > > > >> > > > >> > > > > > > >> > > >
> > >> > > > >> > > > >> > > > > > > >> > > > In my current proposal,
> metadata
> > >> > version
> > >> > > > >> > returned
> > >> > > > >> > > > in
> > >> > > > >> > > > >> the
> > >> > > > >> > > > >> > > > fetch
> > >> > > > >> > > > >> > > > > > > >> response
> > >> > > > >> > > > >> > > > > > > >> > > > will be stored with the offset
> > >> > together.
> > >> > > > >> More
> > >> > > > >> > > > >> > > specifically,
> > >> > > > >> > > > >> > > > > the
> > >> > > > >> > > > >> > > > > > > >> > > > metadata_epoch in the new
> offset
> > >> topic
> > >> > > > >> schema
> > >> > > > >> > > will
> > >> > > > >> > > > be
> > >> > > > >> > > > >> > the
> > >> > > > >> > > > >> > > > > > largest
> > >> > > > >> > > > >> > > > > > > >> > > > metadata_epoch from all the
> > >> > > > MetadataResponse
> > >> > > > >> > and
> > >> > > > >> > > > >> > > > FetchResponse
> > >> > > > >> > > > >> > > > > > > ever
> > >> > > > >> > > > >> > > > > > > >> > > > received by this consumer.
> > >> > > > >> > > > >> > > > > > > >> > > >
> > >> > > > >> > > > >> > > > > > > >> > > > We probably don't have to
> change
> > >> the
> > >> > > > >> consumer
> > >> > > > >> > API
> > >> > > > >> > > > for
> > >> > > > >> > > > >> > > > > > > >> > > > commitSync(Map<TopicPartition,
> > >> > > > >> > > OffsetAndMetadata>).
> > >> > > > >> > > > >> If
> > >> > > > >> > > > >> > > user
> > >> > > > >> > > > >> > > > > > calls
> > >> > > > >> > > > >> > > > > > > >> > > > commitSync(...) to commit
> offset
> > 10
> > >> > for
> > >> > > a
> > >> > > > >> given
> > >> > > > >> > > > >> > partition,
> > >> > > > >> > > > >> > > > for
> > >> > > > >> > > > >> > > > > > > most
> > >> > > > >> > > > >> > > > > > > >> > > > use-cases, this consumer
> instance
> > >> > should
> > >> > > > >> have
> > >> > > > >> > > > >> consumed
> > >> > > > >> > > > >> > > > message
> > >> > > > >> > > > >> > > > > > > with
> > >> > > > >> > > > >> > > > > > > >> > > offset
> > >> > > > >> > > > >> > > > > > > >> > > > 9 from this partition, in which
> > >> case
> > >> > the
> > >> > > > >> > consumer
> > >> > > > >> > > > can
> > >> > > > >> > > > >> > > > remember
> > >> > > > >> > > > >> > > > > > and
> > >> > > > >> > > > >> > > > > > > >> use
> > >> > > > >> > > > >> > > > > > > >> > > the
> > >> > > > >> > > > >> > > > > > > >> > > > metadata_epoch from the
> > >> corresponding
> > >> > > > >> > > FetchResponse
> > >> > > > >> > > > >> when
> > >> > > > >> > > > >> > > > > > > committing
> > >> > > > >> > > > >> > > > > > > >> > > offset.
> > >> > > > >> > > > >> > > > > > > >> > > > If user calls commitSync(..) to
> > >> commit
> > >> > > > >> offset
> > >> > > > >> > 10
> > >> > > > >> > > > for
> > >> > > > >> > > > >> a
> > >> > > > >> > > > >> > > given
> > >> > > > >> > > > >> > > > > > > >> partition
> > >> > > > >> > > > >> > > > > > > >> > > > without having consumed the
> > message
> > >> > with
> > >> > > > >> > offset 9
> > >> > > > >> > > > >> using
> > >> > > > >> > > > >> > > this
> > >> > > > >> > > > >> > > > > > > >> consumer
> > >> > > > >> > > > >> > > > > > > >> > > > instance, this is probably an
> > >> advanced
> > >> > > > >> > use-case.
> > >> > > > >> > > In
> > >> > > > >> > > > >> this
> > >> > > > >> > > > >> > > > case
> > >> > > > >> > > > >> > > > > > the
> > >> > > > >> > > > >> > > > > > > >> > > advanced
> > >> > > > >> > > > >> > > > > > > >> > > > user can retrieve the
> > >> metadata_epoch
> > >> > > using
> > >> > > > >> the
> > >> > > > >> > > > newly
> > >> > > > >> > > > >> > added
> > >> > > > >> > > > >> > > > > > > >> > > metadataEpoch()
> > >> > > > >> > > > >> > > > > > > >> > > > API after it fetches the
> message
> > >> with
> > >> > > > >> offset 9
> > >> > > > >> > > > >> (probably
> > >> > > > >> > > > >> > > > from
> > >> > > > >> > > > >> > > > > > > >> another
> > >> > > > >> > > > >> > > > > > > >> > > > consumer instance) and encode
> > this
> > >> > > > >> > metadata_epoch
> > >> > > > >> > > > in
> > >> > > > >> > > > >> the
> > >> > > > >> > > > >> > > > > > > >> > > > string
> > OffsetAndMetadata.metadata.
> > >> Do
> > >> > > you
> > >> > > > >> think
> > >> > > > >> > > > this
> > >> > > > >> > > > >> > > > solution
> > >> > > > >> > > > >> > > > > > > would
> > >> > > > >> > > > >> > > > > > > >> > work?
> > >> > > > >> > > > >> > > > > > > >> > > >
> > >> > > > >> > > > >> > > > > > > >> > > > By "not sure that I fully
> > >> understand
> > >> > > your
> > >> > > > >> > latest
> > >> > > > >> > > > >> > > > suggestion",
> > >> > > > >> > > > >> > > > > > are
> > >> > > > >> > > > >> > > > > > > >> you
> > >> > > > >> > > > >> > > > > > > >> > > > referring to solution related
> to
> > >> > unclean
> > >> > > > >> leader
> > >> > > > >> > > > >> election
> > >> > > > >> > > > >> > > > using
> > >> > > > >> > > > >> > > > > > > >> > > leader_epoch
> > >> > > > >> > > > >> > > > > > > >> > > > in my previous email?
> > >> > > > >> > > > >> > > > > > > >> > > >
> > >> > > > >> > > > >> > > > > > > >> > > > Thanks,
> > >> > > > >> > > > >> > > > > > > >> > > > Dong
> > >> > > > >> > > > >> > > > > > > >> > > >
> > >> > > > >> > > > >> > > > > > > >> > > > On Thu, Jan 4, 2018 at 1:33 PM,
> > Jun
> > >> > Rao
> > >> > > <
> > >> > > > >> > > > >> > jun@confluent.io
> > >> > > > >> > > > >> > > >
> > >> > > > >> > > > >> > > > > > wrote:
> > >> > > > >> > > > >> > > > > > > >> > > >
> > >> > > > >> > > > >> > > > > > > >> > > > > Hi, Dong,
> > >> > > > >> > > > >> > > > > > > >> > > > >
> > >> > > > >> > > > >> > > > > > > >> > > > > Not sure that I fully
> > understand
> > >> > your
> > >> > > > >> latest
> > >> > > > >> > > > >> > suggestion.
> > >> > > > >> > > > >> > > > > > > >> Returning an
> > >> > > > >> > > > >> > > > > > > >> > > > ever
> > >> > > > >> > > > >> > > > > > > >> > > > > growing global metadata
> version
> > >> > itself
> > >> > > > is
> > >> > > > >> no
> > >> > > > >> > > > ideal,
> > >> > > > >> > > > >> > but
> > >> > > > >> > > > >> > > is
> > >> > > > >> > > > >> > > > > > fine.
> > >> > > > >> > > > >> > > > > > > >> My
> > >> > > > >> > > > >> > > > > > > >> > > > > question is whether the
> > metadata
> > >> > > version
> > >> > > > >> > > returned
> > >> > > > >> > > > >> in
> > >> > > > >> > > > >> > the
> > >> > > > >> > > > >> > > > > fetch
> > >> > > > >> > > > >> > > > > > > >> > response
> > >> > > > >> > > > >> > > > > > > >> > > > > needs to be stored with the
> > >> offset
> > >> > > > >> together
> > >> > > > >> > if
> > >> > > > >> > > > >> offsets
> > >> > > > >> > > > >> > > are
> > >> > > > >> > > > >> > > > > > > stored
> > >> > > > >> > > > >> > > > > > > >> > > > > externally. If so, we also
> have
> > >> to
> > >> > > > change
> > >> > > > >> the
> > >> > > > >> > > > >> consumer
> > >> > > > >> > > > >> > > API
> > >> > > > >> > > > >> > > > > for
> > >> > > > >> > > > >> > > > > > > >> > > > commitSync()
> > >> > > > >> > > > >> > > > > > > >> > > > > and need to worry about
> > >> > compatibility.
> > >> > > > If
> > >> > > > >> we
> > >> > > > >> > > > don't
> > >> > > > >> > > > >> > store
> > >> > > > >> > > > >> > > > the
> > >> > > > >> > > > >> > > > > > > >> metadata
> > >> > > > >> > > > >> > > > > > > >> > > > > version together with the
> > offset,
> > >> > on a
> > >> > > > >> > consumer
> > >> > > > >> > > > >> > restart,
> > >> > > > >> > > > >> > > > > it's
> > >> > > > >> > > > >> > > > > > > not
> > >> > > > >> > > > >> > > > > > > >> > clear
> > >> > > > >> > > > >> > > > > > > >> > > > how
> > >> > > > >> > > > >> > > > > > > >> > > > > we can ensure the metadata in
> > the
> > >> > > > >> consumer is
> > >> > > > >> > > > high
> > >> > > > >> > > > >> > > enough
> > >> > > > >> > > > >> > > > > > since
> > >> > > > >> > > > >> > > > > > > >> there
> > >> > > > >> > > > >> > > > > > > >> > > is
> > >> > > > >> > > > >> > > > > > > >> > > > no
> > >> > > > >> > > > >> > > > > > > >> > > > > metadata version to compare
> > with.
> > >> > > > >> > > > >> > > > > > > >> > > > >
> > >> > > > >> > > > >> > > > > > > >> > > > > Thanks,
> > >> > > > >> > > > >> > > > > > > >> > > > >
> > >> > > > >> > > > >> > > > > > > >> > > > > Jun
> > >> > > > >> > > > >> > > > > > > >> > > > >
> > >> > > > >> > > > >> > > > > > > >> > > > >
> > >> > > > >> > > > >> > > > > > > >> > > > > On Wed, Jan 3, 2018 at 6:43
> PM,
> > >> Dong
> > >> > > > Lin <
> > >> > > > >> > > > >> > > > > lindong28@gmail.com
> > >> > > > >> > > > >> > > > > > >
> > >> > > > >> > > > >> > > > > > > >> > wrote:
> > >> > > > >> > > > >> > > > > > > >> > > > >
> > >> > > > >> > > > >> > > > > > > >> > > > > > Hey Jun,
> > >> > > > >> > > > >> > > > > > > >> > > > > >
> > >> > > > >> > > > >> > > > > > > >> > > > > > Thanks much for the
> > >> explanation.
> > >> > > > >> > > > >> > > > > > > >> > > > > >
> > >> > > > >> > > > >> > > > > > > >> > > > > > I understand the advantage
> of
> > >> > > > >> > partition_epoch
> > >> > > > >> > > > >> over
> > >> > > > >> > > > >> > > > > > > >> metadata_epoch.
> > >> > > > >> > > > >> > > > > > > >> > My
> > >> > > > >> > > > >> > > > > > > >> > > > > > current concern is that the
> > >> use of
> > >> > > > >> > > leader_epoch
> > >> > > > >> > > > >> and
> > >> > > > >> > > > >> > > the
> > >> > > > >> > > > >> > > > > > > >> > > partition_epoch
> > >> > > > >> > > > >> > > > > > > >> > > > > > requires us considerable
> > >> change to
> > >> > > > >> > consumer's
> > >> > > > >> > > > >> public
> > >> > > > >> > > > >> > > API
> > >> > > > >> > > > >> > > > > to
> > >> > > > >> > > > >> > > > > > > take
> > >> > > > >> > > > >> > > > > > > >> > care
> > >> > > > >> > > > >> > > > > > > >> > > > of
> > >> > > > >> > > > >> > > > > > > >> > > > > > the case where user stores
> > >> offset
> > >> > > > >> > externally.
> > >> > > > >> > > > For
> > >> > > > >> > > > >> > > > example,
> > >> > > > >> > > > >> > > > > > > >> > > *consumer*.
> > >> > > > >> > > > >> > > > > > > >> > > > > > *commitSync*(..) would have
> > to
> > >> > take
> > >> > > a
> > >> > > > >> map
> > >> > > > >> > > whose
> > >> > > > >> > > > >> > value
> > >> > > > >> > > > >> > > is
> > >> > > > >> > > > >> > > > > > > >> <offset,
> > >> > > > >> > > > >> > > > > > > >> > > > > metadata,
> > >> > > > >> > > > >> > > > > > > >> > > > > > leader epoch, partition
> > epoch>.
> > >> > > > >> > > > >> > *consumer*.*seek*(...)
> > >> > > > >> > > > >> > > > > would
> > >> > > > >> > > > >> > > > > > > >> also
> > >> > > > >> > > > >> > > > > > > >> > > need
> > >> > > > >> > > > >> > > > > > > >> > > > > > leader_epoch and
> > >> partition_epoch
> > >> > as
> > >> > > > >> > > parameter.
> > >> > > > >> > > > >> > > > Technically
> > >> > > > >> > > > >> > > > > > we
> > >> > > > >> > > > >> > > > > > > >> can
> > >> > > > >> > > > >> > > > > > > >> > > > > probably
> > >> > > > >> > > > >> > > > > > > >> > > > > > still make it work in a
> > >> backward
> > >> > > > >> compatible
> > >> > > > >> > > > >> manner
> > >> > > > >> > > > >> > > after
> > >> > > > >> > > > >> > > > > > > careful
> > >> > > > >> > > > >> > > > > > > >> > > design
> > >> > > > >> > > > >> > > > > > > >> > > > > and
> > >> > > > >> > > > >> > > > > > > >> > > > > > discussion. But these
> changes
> > >> can
> > >> > > make
> > >> > > > >> the
> > >> > > > >> > > > >> > consumer's
> > >> > > > >> > > > >> > > > > > > interface
> > >> > > > >> > > > >> > > > > > > >> > > > > > unnecessarily complex for
> > more
> > >> > users
> > >> > > > >> who do
> > >> > > > >> > > not
> > >> > > > >> > > > >> > store
> > >> > > > >> > > > >> > > > > offset
> > >> > > > >> > > > >> > > > > > > >> > > > externally.
> > >> > > > >> > > > >> > > > > > > >> > > > > >
> > >> > > > >> > > > >> > > > > > > >> > > > > > After thinking more about
> it,
> > >> we
> > >> > can
> > >> > > > >> > address
> > >> > > > >> > > > all
> > >> > > > >> > > > >> > > > problems
> > >> > > > >> > > > >> > > > > > > >> discussed
> > >> > > > >> > > > >> > > > > > > >> > > by
> > >> > > > >> > > > >> > > > > > > >> > > > > only
> > >> > > > >> > > > >> > > > > > > >> > > > > > using the metadata_epoch
> > >> without
> > >> > > > >> > introducing
> > >> > > > >> > > > >> > > > leader_epoch
> > >> > > > >> > > > >> > > > > or
> > >> > > > >> > > > >> > > > > > > the
> > >> > > > >> > > > >> > > > > > > >> > > > > > partition_epoch. The
> current
> > >> KIP
> > >> > > > >> describes
> > >> > > > >> > > the
> > >> > > > >> > > > >> > changes
> > >> > > > >> > > > >> > > > to
> > >> > > > >> > > > >> > > > > > the
> > >> > > > >> > > > >> > > > > > > >> > > consumer
> > >> > > > >> > > > >> > > > > > > >> > > > > API
> > >> > > > >> > > > >> > > > > > > >> > > > > > and how the new API can be
> > >> used if
> > >> > > > user
> > >> > > > >> > > stores
> > >> > > > >> > > > >> > offset
> > >> > > > >> > > > >> > > > > > > >> externally.
> > >> > > > >> > > > >> > > > > > > >> > In
> > >> > > > >> > > > >> > > > > > > >> > > > > order
> > >> > > > >> > > > >> > > > > > > >> > > > > > to address the scenario you
> > >> > > described
> > >> > > > >> > > earlier,
> > >> > > > >> > > > we
> > >> > > > >> > > > >> > can
> > >> > > > >> > > > >> > > > > > include
> > >> > > > >> > > > >> > > > > > > >> > > > > > metadata_epoch in the
> > >> > FetchResponse
> > >> > > > and
> > >> > > > >> the
> > >> > > > >> > > > >> > > > > > > LeaderAndIsrRequest.
> > >> > > > >> > > > >> > > > > > > >> > > > Consumer
> > >> > > > >> > > > >> > > > > > > >> > > > > > remembers the largest
> > >> > metadata_epoch
> > >> > > > >> from
> > >> > > > >> > all
> > >> > > > >> > > > the
> > >> > > > >> > > > >> > > > > > > FetchResponse
> > >> > > > >> > > > >> > > > > > > >> it
> > >> > > > >> > > > >> > > > > > > >> > > has
> > >> > > > >> > > > >> > > > > > > >> > > > > > received. The
> metadata_epoch
> > >> > > committed
> > >> > > > >> with
> > >> > > > >> > > the
> > >> > > > >> > > > >> > > offset,
> > >> > > > >> > > > >> > > > > > either
> > >> > > > >> > > > >> > > > > > > >> > within
> > >> > > > >> > > > >> > > > > > > >> > > > or
> > >> > > > >> > > > >> > > > > > > >> > > > > > outside Kafka, should be
> the
> > >> > largest
> > >> > > > >> > > > >> metadata_epoch
> > >> > > > >> > > > >> > > > across
> > >> > > > >> > > > >> > > > > > all
> > >> > > > >> > > > >> > > > > > > >> > > > > > FetchResponse and
> > >> MetadataResponse
> > >> > > > ever
> > >> > > > >> > > > received
> > >> > > > >> > > > >> by
> > >> > > > >> > > > >> > > this
> > >> > > > >> > > > >> > > > > > > >> consumer.
> > >> > > > >> > > > >> > > > > > > >> > > > > >
> > >> > > > >> > > > >> > > > > > > >> > > > > > The drawback of using only
> > the
> > >> > > > >> > metadata_epoch
> > >> > > > >> > > > is
> > >> > > > >> > > > >> > that
> > >> > > > >> > > > >> > > we
> > >> > > > >> > > > >> > > > > can
> > >> > > > >> > > > >> > > > > > > not
> > >> > > > >> > > > >> > > > > > > >> > > always
> > >> > > > >> > > > >> > > > > > > >> > > > > do
> > >> > > > >> > > > >> > > > > > > >> > > > > > the smart offset reset in
> > case
> > >> of
> > >> > > > >> unclean
> > >> > > > >> > > > leader
> > >> > > > >> > > > >> > > > election
> > >> > > > >> > > > >> > > > > > > which
> > >> > > > >> > > > >> > > > > > > >> you
> > >> > > > >> > > > >> > > > > > > >> > > > > > mentioned earlier. But in
> > most
> > >> > case,
> > >> > > > >> > unclean
> > >> > > > >> > > > >> leader
> > >> > > > >> > > > >> > > > > election
> > >> > > > >> > > > >> > > > > > > >> > probably
> > >> > > > >> > > > >> > > > > > > >> > > > > > happens when consumer is
> not
> > >> > > > >> > > > >> rebalancing/restarting.
> > >> > > > >> > > > >> > > In
> > >> > > > >> > > > >> > > > > > these
> > >> > > > >> > > > >> > > > > > > >> > cases,
> > >> > > > >> > > > >> > > > > > > >> > > > > either
> > >> > > > >> > > > >> > > > > > > >> > > > > > consumer is not directly
> > >> affected
> > >> > by
> > >> > > > >> > unclean
> > >> > > > >> > > > >> leader
> > >> > > > >> > > > >> > > > > election
> > >> > > > >> > > > >> > > > > > > >> since
> > >> > > > >> > > > >> > > > > > > >> > it
> > >> > > > >> > > > >> > > > > > > >> > > > is
> > >> > > > >> > > > >> > > > > > > >> > > > > > not consuming from the end
> of
> > >> the
> > >> > > log,
> > >> > > > >> or
> > >> > > > >> > > > >> consumer
> > >> > > > >> > > > >> > can
> > >> > > > >> > > > >> > > > > > derive
> > >> > > > >> > > > >> > > > > > > >> the
> > >> > > > >> > > > >> > > > > > > >> > > > > > leader_epoch from the most
> > >> recent
> > >> > > > >> message
> > >> > > > >> > > > >> received
> > >> > > > >> > > > >> > > > before
> > >> > > > >> > > > >> > > > > it
> > >> > > > >> > > > >> > > > > > > >> sees
> > >> > > > >> > > > >> > > > > > > >> > > > > > OffsetOutOfRangeException.
> So
> > >> I am
> > >> > > not
> > >> > > > >> sure
> > >> > > > >> > > it
> > >> > > > >> > > > is
> > >> > > > >> > > > >> > > worth
> > >> > > > >> > > > >> > > > > > adding
> > >> > > > >> > > > >> > > > > > > >> the
> > >> > > > >> > > > >> > > > > > > >> > > > > > leader_epoch to consumer
> API
> > to
> > >> > > > address
> > >> > > > >> the
> > >> > > > >> > > > >> > remaining
> > >> > > > >> > > > >> > > > > corner
> > >> > > > >> > > > >> > > > > > > >> case.
> > >> > > > >> > > > >> > > > > > > >> > > What
> > >> > > > >> > > > >> > > > > > > >> > > > > do
> > >> > > > >> > > > >> > > > > > > >> > > > > > you think?
> > >> > > > >> > > > >> > > > > > > >> > > > > >
> > >> > > > >> > > > >> > > > > > > >> > > > > > Thanks,
> > >> > > > >> > > > >> > > > > > > >> > > > > > Dong
> > >> > > > >> > > > >> > > > > > > >> > > > > >
> > >> > > > >> > > > >> > > > > > > >> > > > > >
> > >> > > > >> > > > >> > > > > > > >> > > > > >
> > >> > > > >> > > > >> > > > > > > >> > > > > > On Tue, Jan 2, 2018 at 6:28
> > PM,
> > >> > Jun
> > >> > > > Rao
> > >> > > > >> <
> > >> > > > >> > > > >> > > > jun@confluent.io
> > >> > > > >> > > > >> > > > > >
> > >> > > > >> > > > >> > > > > > > >> wrote:
> > >> > > > >> > > > >> > > > > > > >> > > > > >
> > >> > > > >> > > > >> > > > > > > >> > > > > > > Hi, Dong,
> > >> > > > >> > > > >> > > > > > > >> > > > > > >
> > >> > > > >> > > > >> > > > > > > >> > > > > > > Thanks for the reply.
> > >> > > > >> > > > >> > > > > > > >> > > > > > >
> > >> > > > >> > > > >> > > > > > > >> > > > > > > To solve the topic
> > recreation
> > >> > > issue,
> > >> > > > >> we
> > >> > > > >> > > could
> > >> > > > >> > > > >> use
> > >> > > > >> > > > >> > > > > either a
> > >> > > > >> > > > >> > > > > > > >> global
> > >> > > > >> > > > >> > > > > > > >> > > > > > metadata
> > >> > > > >> > > > >> > > > > > > >> > > > > > > version or a partition
> > level
> > >> > > epoch.
> > >> > > > >> But
> > >> > > > >> > > > either
> > >> > > > >> > > > >> one
> > >> > > > >> > > > >> > > > will
> > >> > > > >> > > > >> > > > > > be a
> > >> > > > >> > > > >> > > > > > > >> new
> > >> > > > >> > > > >> > > > > > > >> > > > > concept,
> > >> > > > >> > > > >> > > > > > > >> > > > > > > right? To me, the latter
> > >> seems
> > >> > > more
> > >> > > > >> > > natural.
> > >> > > > >> > > > It
> > >> > > > >> > > > >> > also
> > >> > > > >> > > > >> > > > > makes
> > >> > > > >> > > > >> > > > > > > it
> > >> > > > >> > > > >> > > > > > > >> > > easier
> > >> > > > >> > > > >> > > > > > > >> > > > to
> > >> > > > >> > > > >> > > > > > > >> > > > > > > detect if a consumer's
> > >> offset is
> > >> > > > still
> > >> > > > >> > > valid
> > >> > > > >> > > > >> > after a
> > >> > > > >> > > > >> > > > > topic
> > >> > > > >> > > > >> > > > > > > is
> > >> > > > >> > > > >> > > > > > > >> > > > > recreated.
> > >> > > > >> > > > >> > > > > > > >> > > > > > As
> > >> > > > >> > > > >> > > > > > > >> > > > > > > you pointed out, we don't
> > >> need
> > >> > to
> > >> > > > >> store
> > >> > > > >> > the
> > >> > > > >> > > > >> > > partition
> > >> > > > >> > > > >> > > > > > epoch
> > >> > > > >> > > > >> > > > > > > in
> > >> > > > >> > > > >> > > > > > > >> > the
> > >> > > > >> > > > >> > > > > > > >> > > > > > message.
> > >> > > > >> > > > >> > > > > > > >> > > > > > > The following is what I
> am
> > >> > > thinking.
> > >> > > > >> > When a
> > >> > > > >> > > > >> > > partition
> > >> > > > >> > > > >> > > > is
> > >> > > > >> > > > >> > > > > > > >> created,
> > >> > > > >> > > > >> > > > > > > >> > > we
> > >> > > > >> > > > >> > > > > > > >> > > > > can
> > >> > > > >> > > > >> > > > > > > >> > > > > > > assign a partition epoch
> > >> from an
> > >> > > > >> > > > >> ever-increasing
> > >> > > > >> > > > >> > > > global
> > >> > > > >> > > > >> > > > > > > >> counter
> > >> > > > >> > > > >> > > > > > > >> > and
> > >> > > > >> > > > >> > > > > > > >> > > > > store
> > >> > > > >> > > > >> > > > > > > >> > > > > > > it in
> > >> /brokers/topics/[topic]/
> > >> > > > >> > > > >> > > > partitions/[partitionId]
> > >> > > > >> > > > >> > > > > in
> > >> > > > >> > > > >> > > > > > > ZK.
> > >> > > > >> > > > >> > > > > > > >> > The
> > >> > > > >> > > > >> > > > > > > >> > > > > > > partition
> > >> > > > >> > > > >> > > > > > > >> > > > > > > epoch is propagated to
> > every
> > >> > > broker.
> > >> > > > >> The
> > >> > > > >> > > > >> consumer
> > >> > > > >> > > > >> > > will
> > >> > > > >> > > > >> > > > > be
> > >> > > > >> > > > >> > > > > > > >> > tracking
> > >> > > > >> > > > >> > > > > > > >> > > a
> > >> > > > >> > > > >> > > > > > > >> > > > > > tuple
> > >> > > > >> > > > >> > > > > > > >> > > > > > > of <offset, leader epoch,
> > >> > > partition
> > >> > > > >> > epoch>
> > >> > > > >> > > > for
> > >> > > > >> > > > >> > > > offsets.
> > >> > > > >> > > > >> > > > > > If a
> > >> > > > >> > > > >> > > > > > > >> > topic
> > >> > > > >> > > > >> > > > > > > >> > > is
> > >> > > > >> > > > >> > > > > > > >> > > > > > > recreated, it's possible
> > >> that a
> > >> > > > >> > consumer's
> > >> > > > >> > > > >> offset
> > >> > > > >> > > > >> > > and
> > >> > > > >> > > > >> > > > > > leader
> > >> > > > >> > > > >> > > > > > > >> > epoch
> > >> > > > >> > > > >> > > > > > > >> > > > > still
> > >> > > > >> > > > >> > > > > > > >> > > > > > > match that in the broker,
> > but
> > >> > > > >> partition
> > >> > > > >> > > epoch
> > >> > > > >> > > > >> > won't
> > >> > > > >> > > > >> > > > be.
> > >> > > > >> > > > >> > > > > In
> > >> > > > >> > > > >> > > > > > > >> this
> > >> > > > >> > > > >> > > > > > > >> > > case,
> > >> > > > >> > > > >> > > > > > > >> > > > > we
> > >> > > > >> > > > >> > > > > > > >> > > > > > > can potentially still
> treat
> > >> the
> > >> > > > >> > consumer's
> > >> > > > >> > > > >> offset
> > >> > > > >> > > > >> > as
> > >> > > > >> > > > >> > > > out
> > >> > > > >> > > > >> > > > > > of
> > >> > > > >> > > > >> > > > > > > >> range
> > >> > > > >> > > > >> > > > > > > >> > > and
> > >> > > > >> > > > >> > > > > > > >> > > > > > reset
> > >> > > > >> > > > >> > > > > > > >> > > > > > > the offset based on the
> > >> offset
> > >> > > reset
> > >> > > > >> > policy
> > >> > > > >> > > > in
> > >> > > > >> > > > >> the
> > >> > > > >> > > > >> > > > > > consumer.
> > >> > > > >> > > > >> > > > > > > >> This
> > >> > > > >> > > > >> > > > > > > >> > > > seems
> > >> > > > >> > > > >> > > > > > > >> > > > > > > harder to do with a
> global
> > >> > > metadata
> > >> > > > >> > > version.
> > >> > > > >> > > > >> > > > > > > >> > > > > > >
> > >> > > > >> > > > >> > > > > > > >> > > > > > > Jun
> > >> > > > >> > > > >> > > > > > > >> > > > > > >
> > >> > > > >> > > > >> > > > > > > >> > > > > > >
> > >> > > > >> > > > >> > > > > > > >> > > > > > >
> > >> > > > >> > > > >> > > > > > > >> > > > > > > On Mon, Dec 25, 2017 at
> > 6:56
> > >> AM,
> > >> > > > Dong
> > >> > > > >> > Lin <
> > >> > > > >> > > > >> > > > > > > >> lindong28@gmail.com>
> > >> > > > >> > > > >> > > > > > > >> > > > wrote:
> > >> > > > >> > > > >> > > > > > > >> > > > > > >
> > >> > > > >> > > > >> > > > > > > >> > > > > > > > Hey Jun,
> > >> > > > >> > > > >> > > > > > > >> > > > > > > >
> > >> > > > >> > > > >> > > > > > > >> > > > > > > > This is a very good
> > >> example.
> > >> > > After
> > >> > > > >> > > thinking
> > >> > > > >> > > > >> > > through
> > >> > > > >> > > > >> > > > > this
> > >> > > > >> > > > >> > > > > > > in
> > >> > > > >> > > > >> > > > > > > >> > > > detail, I
> > >> > > > >> > > > >> > > > > > > >> > > > > > > agree
> > >> > > > >> > > > >> > > > > > > >> > > > > > > > that we need to commit
> > >> offset
> > >> > > with
> > >> > > > >> > leader
> > >> > > > >> > > > >> epoch
> > >> > > > >> > > > >> > in
> > >> > > > >> > > > >> > > > > order
> > >> > > > >> > > > >> > > > > > > to
> > >> > > > >> > > > >> > > > > > > >> > > address
> > >> > > > >> > > > >> > > > > > > >> > > > > > this
> > >> > > > >> > > > >> > > > > > > >> > > > > > > > example.
> > >> > > > >> > > > >> > > > > > > >> > > > > > > >
> > >> > > > >> > > > >> > > > > > > >> > > > > > > > I think the remaining
> > >> question
> > >> > > is
> > >> > > > >> how
> > >> > > > >> > to
> > >> > > > >> > > > >> address
> > >> > > > >> > > > >> > > the
> > >> > > > >> > > > >> > > > > > > >> scenario
> > >> > > > >> > > > >> > > > > > > >> > > that
> > >> > > > >> > > > >> > > > > > > >> > > > > the
> > >> > > > >> > > > >> > > > > > > >> > > > > > > > topic is deleted and
> > >> > re-created.
> > >> > > > One
> > >> > > > >> > > > possible
> > >> > > > >> > > > >> > > > solution
> > >> > > > >> > > > >> > > > > > is
> > >> > > > >> > > > >> > > > > > > to
> > >> > > > >> > > > >> > > > > > > >> > > commit
> > >> > > > >> > > > >> > > > > > > >> > > > > > > offset
> > >> > > > >> > > > >> > > > > > > >> > > > > > > > with both the leader
> > epoch
> > >> and
> > >> > > the
> > >> > > > >> > > metadata
> > >> > > > >> > > > >> > > version.
> > >> > > > >> > > > >> > > > > The
> > >> > > > >> > > > >> > > > > > > >> logic
> > >> > > > >> > > > >> > > > > > > >> > > and
> > >> > > > >> > > > >> > > > > > > >> > > > > the
> > >> > > > >> > > > >> > > > > > > >> > > > > > > > implementation of this
> > >> > solution
> > >> > > > does
> > >> > > > >> > not
> > >> > > > >> > > > >> > require a
> > >> > > > >> > > > >> > > > new
> > >> > > > >> > > > >> > > > > > > >> concept
> > >> > > > >> > > > >> > > > > > > >> > > > (e.g.
> > >> > > > >> > > > >> > > > > > > >> > > > > > > > partition epoch) and it
> > >> does
> > >> > not
> > >> > > > >> > require
> > >> > > > >> > > > any
> > >> > > > >> > > > >> > > change
> > >> > > > >> > > > >> > > > to
> > >> > > > >> > > > >> > > > > > the
> > >> > > > >> > > > >> > > > > > > >> > > message
> > >> > > > >> > > > >> > > > > > > >> > > > > > format
> > >> > > > >> > > > >> > > > > > > >> > > > > > > > or leader epoch. It
> also
> > >> > allows
> > >> > > us
> > >> > > > >> to
> > >> > > > >> > > order
> > >> > > > >> > > > >> the
> > >> > > > >> > > > >> > > > > metadata
> > >> > > > >> > > > >> > > > > > > in
> > >> > > > >> > > > >> > > > > > > >> a
> > >> > > > >> > > > >> > > > > > > >> > > > > > > > straightforward manner
> > >> which
> > >> > may
> > >> > > > be
> > >> > > > >> > > useful
> > >> > > > >> > > > in
> > >> > > > >> > > > >> > the
> > >> > > > >> > > > >> > > > > > future.
> > >> > > > >> > > > >> > > > > > > >> So it
> > >> > > > >> > > > >> > > > > > > >> > > may
> > >> > > > >> > > > >> > > > > > > >> > > > > be
> > >> > > > >> > > > >> > > > > > > >> > > > > > a
> > >> > > > >> > > > >> > > > > > > >> > > > > > > > better solution than
> > >> > generating
> > >> > > a
> > >> > > > >> > random
> > >> > > > >> > > > >> > partition
> > >> > > > >> > > > >> > > > > epoch
> > >> > > > >> > > > >> > > > > > > >> every
> > >> > > > >> > > > >> > > > > > > >> > > time
> > >> > > > >> > > > >> > > > > > > >> > > > > we
> > >> > > > >> > > > >> > > > > > > >> > > > > > > > create a partition.
> Does
> > >> this
> > >> > > > sound
> > >> > > > >> > > > >> reasonable?
> > >> > > > >> > > > >> > > > > > > >> > > > > > > >
> > >> > > > >> > > > >> > > > > > > >> > > > > > > > Previously one concern
> > with
> > >> > > using
> > >> > > > >> the
> > >> > > > >> > > > >> metadata
> > >> > > > >> > > > >> > > > version
> > >> > > > >> > > > >> > > > > > is
> > >> > > > >> > > > >> > > > > > > >> that
> > >> > > > >> > > > >> > > > > > > >> > > > > consumer
> > >> > > > >> > > > >> > > > > > > >> > > > > > > > will be forced to
> refresh
> > >> > > metadata
> > >> > > > >> even
> > >> > > > >> > > if
> > >> > > > >> > > > >> > > metadata
> > >> > > > >> > > > >> > > > > > > version
> > >> > > > >> > > > >> > > > > > > >> is
> > >> > > > >> > > > >> > > > > > > >> > > > > > increased
> > >> > > > >> > > > >> > > > > > > >> > > > > > > > due to topics that the
> > >> > consumer
> > >> > > is
> > >> > > > >> not
> > >> > > > >> > > > >> > interested
> > >> > > > >> > > > >> > > > in.
> > >> > > > >> > > > >> > > > > > Now
> > >> > > > >> > > > >> > > > > > > I
> > >> > > > >> > > > >> > > > > > > >> > > > realized
> > >> > > > >> > > > >> > > > > > > >> > > > > > that
> > >> > > > >> > > > >> > > > > > > >> > > > > > > > this is probably not a
> > >> > problem.
> > >> > > > >> > Currently
> > >> > > > >> > > > >> client
> > >> > > > >> > > > >> > > > will
> > >> > > > >> > > > >> > > > > > > >> refresh
> > >> > > > >> > > > >> > > > > > > >> > > > > metadata
> > >> > > > >> > > > >> > > > > > > >> > > > > > > > either due to
> > >> > > > >> InvalidMetadataException
> > >> > > > >> > in
> > >> > > > >> > > > the
> > >> > > > >> > > > >> > > > response
> > >> > > > >> > > > >> > > > > > > from
> > >> > > > >> > > > >> > > > > > > >> > > broker
> > >> > > > >> > > > >> > > > > > > >> > > > or
> > >> > > > >> > > > >> > > > > > > >> > > > > > due
> > >> > > > >> > > > >> > > > > > > >> > > > > > > > to metadata expiry. The
> > >> > addition
> > >> > > > of
> > >> > > > >> the
> > >> > > > >> > > > >> metadata
> > >> > > > >> > > > >> > > > > version
> > >> > > > >> > > > >> > > > > > > >> should
> > >> > > > >> > > > >> > > > > > > >> > > > > > increase
> > >> > > > >> > > > >> > > > > > > >> > > > > > > > the overhead of
> metadata
> > >> > refresh
> > >> > > > >> caused
> > >> > > > >> > > by
> > >> > > > >> > > > >> > > > > > > >> > > > InvalidMetadataException.
> > >> > > > >> > > > >> > > > > > > >> > > > > If
> > >> > > > >> > > > >> > > > > > > >> > > > > > > > client refresh metadata
> > >> due to
> > >> > > > >> expiry
> > >> > > > >> > and
> > >> > > > >> > > > it
> > >> > > > >> > > > >> > > > receives
> > >> > > > >> > > > >> > > > > a
> > >> > > > >> > > > >> > > > > > > >> > metadata
> > >> > > > >> > > > >> > > > > > > >> > > > > whose
> > >> > > > >> > > > >> > > > > > > >> > > > > > > > version is lower than
> the
> > >> > > current
> > >> > > > >> > > metadata
> > >> > > > >> > > > >> > > version,
> > >> > > > >> > > > >> > > > we
> > >> > > > >> > > > >> > > > > > can
> > >> > > > >> > > > >> > > > > > > >> > reject
> > >> > > > >> > > > >> > > > > > > >> > > > the
> > >> > > > >> > > > >> > > > > > > >> > > > > > > > metadata but still
> reset
> > >> the
> > >> > > > >> metadata
> > >> > > > >> > > age,
> > >> > > > >> > > > >> which
> > >> > > > >> > > > >> > > > > > > essentially
> > >> > > > >> > > > >> > > > > > > >> > keep
> > >> > > > >> > > > >> > > > > > > >> > > > the
> > >> > > > >> > > > >> > > > > > > >> > > > > > > > existing behavior in
> the
> > >> > client.
> > >> > > > >> > > > >> > > > > > > >> > > > > > > >
> > >> > > > >> > > > >> > > > > > > >> > > > > > > > Thanks much,
> > >> > > > >> > > > >> > > > > > > >> > > > > > > > Dong
> > >> > > > >> > > > >> > > > > > > >> > > > > > > >
> > >> > > > >> > > > >> > > > > > > >> > > > > > >
> > >> > > > >> > > > >> > > > > > > >> > > > > >
> > >> > > > >> > > > >> > > > > > > >> > > > >
> > >> > > > >> > > > >> > > > > > > >> > > >
> > >> > > > >> > > > >> > > > > > > >> > >
> > >> > > > >> > > > >> > > > > > > >> >
> > >> > > > >> > > > >> > > > > > > >>
> > >> > > > >> > > > >> > > > > > > >
> > >> > > > >> > > > >> > > > > > > >
> > >> > > > >> > > > >> > > > > > >
> > >> > > > >> > > > >> > > > > >
> > >> > > > >> > > > >> > > > >
> > >> > > > >> > > > >> > > >
> > >> > > > >> > > > >> > >
> > >> > > > >> > > > >> >
> > >> > > > >> > > > >>
> > >> > > > >> > > > >
> > >> > > > >> > > > >
> > >> > > > >> > > >
> > >> > > > >> > >
> > >> > > > >> >
> > >> > > > >>
> > >> > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Jun Rao <ju...@confluent.io>.
Hi, Dong,

The current KIP looks good to me.

Thanks,

Jun

On Tue, Jan 23, 2018 at 12:29 PM, Dong Lin <li...@gmail.com> wrote:

> Hey Jun,
>
> Do you think the current KIP looks OK? I am wondering if we can open the
> voting thread.
>
> Thanks!
> Dong
>
> On Fri, Jan 19, 2018 at 3:08 PM, Dong Lin <li...@gmail.com> wrote:
>
> > Hey Jun,
> >
> > I think we can probably have a static method in Util class to decode the
> > byte[]. Both KafkaConsumer implementation and the user application will
> be
> > able to decode the byte array and log its content for debug purpose. So
> it
> > seems that we can still print the information we want. It is just not
> > explicitly exposed in the consumer interface. Would this address the
> > problem here?
> >
> > Yeah we can include OffsetEpoch in AdminClient. This can be added in
> > KIP-222? Is there something you would like me to add in this KIP?
> >
> > Thanks!
> > Dong
> >
> > On Fri, Jan 19, 2018 at 3:00 PM, Jun Rao <ju...@confluent.io> wrote:
> >
> >> Hi, Dong,
> >>
> >> The issue with using just byte[] for OffsetEpoch is that it won't be
> >> printable, which makes debugging harder.
> >>
> >> Also, KIP-222 proposes a listGroupOffset() method in AdminClient. If
> that
> >> gets adopted before this KIP, we probably want to include OffsetEpoch in
> >> the AdminClient too.
> >>
> >> Thanks,
> >>
> >> Jun
> >>
> >>
> >> On Thu, Jan 18, 2018 at 6:30 PM, Dong Lin <li...@gmail.com> wrote:
> >>
> >> > Hey Jun,
> >> >
> >> > I agree. I have updated the KIP to remove the class OffetEpoch and
> >> replace
> >> > OffsetEpoch with byte[] in APIs that use it. Can you see if it looks
> >> good?
> >> >
> >> > Thanks!
> >> > Dong
> >> >
> >> > On Thu, Jan 18, 2018 at 6:07 PM, Jun Rao <ju...@confluent.io> wrote:
> >> >
> >> > > Hi, Dong,
> >> > >
> >> > > Thanks for the updated KIP. It looks good to me now. The only thing
> is
> >> > > for OffsetEpoch.
> >> > > If we expose the individual fields in the class, we probably don't
> >> need
> >> > the
> >> > > encode/decode methods. If we want to hide the details of
> OffsetEpoch,
> >> we
> >> > > probably don't want expose the individual fields.
> >> > >
> >> > > Jun
> >> > >
> >> > > On Wed, Jan 17, 2018 at 10:10 AM, Dong Lin <li...@gmail.com>
> >> wrote:
> >> > >
> >> > > > Thinking about point 61 more, I realize that the async zookeeper
> >> read
> >> > may
> >> > > > make it less of an issue for controller to read more zookeeper
> >> nodes.
> >> > > > Writing partition_epoch in the per-partition znode makes it
> simpler
> >> to
> >> > > > handle the broker failure between zookeeper writes for a topic
> >> > creation.
> >> > > I
> >> > > > have updated the KIP to use the suggested approach.
> >> > > >
> >> > > >
> >> > > > On Wed, Jan 17, 2018 at 9:57 AM, Dong Lin <li...@gmail.com>
> >> wrote:
> >> > > >
> >> > > > > Hey Jun,
> >> > > > >
> >> > > > > Thanks much for the comments. Please see my comments inline.
> >> > > > >
> >> > > > > On Tue, Jan 16, 2018 at 4:38 PM, Jun Rao <ju...@confluent.io>
> >> wrote:
> >> > > > >
> >> > > > >> Hi, Dong,
> >> > > > >>
> >> > > > >> Thanks for the updated KIP. Looks good to me overall. Just a
> few
> >> > minor
> >> > > > >> comments.
> >> > > > >>
> >> > > > >> 60. OffsetAndMetadata positionAndOffsetEpoch(TopicPartition
> >> > > partition):
> >> > > > >> It
> >> > > > >> seems that there is no need to return metadata. We probably
> want
> >> to
> >> > > > return
> >> > > > >> sth like OffsetAndEpoch.
> >> > > > >>
> >> > > > >
> >> > > > > Previously I think we may want to re-use the existing class to
> >> keep
> >> > our
> >> > > > > consumer interface simpler. I have updated the KIP to add class
> >> > > > > OffsetAndOffsetEpoch. I didn't use OffsetAndEpoch because user
> may
> >> > > > confuse
> >> > > > > this name with OffsetEpoch. Does this sound OK?
> >> > > > >
> >> > > > >
> >> > > > >>
> >> > > > >> 61. Should we store partition_epoch in
> >> > > > >> /brokers/topics/[topic]/partitions/[partitionId] in ZK?
> >> > > > >>
> >> > > > >
> >> > > > > I have considered this. I think the advantage of adding the
> >> > > > > partition->partition_epoch map in the existing
> >> > > > > znode /brokers/topics/[topic]/partitions is that controller
> only
> >> > needs
> >> > > > to
> >> > > > > read one znode per topic to gets its partition_epoch
> information.
> >> > > > Otherwise
> >> > > > > controller may need to read one extra znode per partition to get
> >> the
> >> > > same
> >> > > > > information.
> >> > > > >
> >> > > > > When we delete partition or expand partition of a topic, someone
> >> > needs
> >> > > to
> >> > > > > modify partition->partition_epoch map in znode
> >> > > > > /brokers/topics/[topic]/partitions. This may seem a bit more
> >> > > complicated
> >> > > > > than simply adding or deleting znode /brokers/topics/[topic]/
> >> > > > partitions/[partitionId].
> >> > > > > But the complexity is probably similar to the existing operation
> >> of
> >> > > > > modifying the partition->replica_list mapping in znode
> >> > > > > /brokers/topics/[topic]. So not sure it is better to store the
> >> > > > > partition_epoch in /brokers/topics/[topic]/partit
> >> ions/[partitionId].
> >> > > > What
> >> > > > > do you think?
> >> > > > >
> >> > > > >
> >> > > > >>
> >> > > > >> 62. For checking outdated metadata in the client, we probably
> >> want
> >> > to
> >> > > > add
> >> > > > >> when max_partition_epoch will be used.
> >> > > > >>
> >> > > > >
> >> > > > > The max_partition_epoch is used in the Proposed Changes ->
> >> Client's
> >> > > > > metadata refresh section to determine whether a metadata is
> >> outdated.
> >> > > And
> >> > > > > this formula is referenced and re-used in other sections to
> >> determine
> >> > > > > whether a metadata is outdated. Does this formula look OK?
> >> > > > >
> >> > > > >
> >> > > > >>
> >> > > > >> 63. "The leader_epoch should be the largest leader_epoch of
> >> messages
> >> > > > whose
> >> > > > >> offset < the commit offset. If no message has been consumed
> since
> >> > > > consumer
> >> > > > >> initialization, the leader_epoch from seek(...) or
> >> > OffsetFetchResponse
> >> > > > >> should be used. The partition_epoch should be read from the
> last
> >> > > > >> FetchResponse corresponding to the given partition and commit
> >> > offset.
> >> > > ":
> >> > > > >> leader_epoch and partition_epoch are associated with an offset.
> >> So,
> >> > if
> >> > > > no
> >> > > > >> message is consumed, there is no offset and therefore there is
> no
> >> > need
> >> > > > to
> >> > > > >> read leader_epoch and partition_epoch. Also, the leader_epoch
> >> > > associated
> >> > > > >> with the offset should just come from the messages returned in
> >> the
> >> > > fetch
> >> > > > >> response.
> >> > > > >>
> >> > > > >
> >> > > > > I am thinking that, if user calls seek(..) and commitSync(...)
> >> > without
> >> > > > > consuming any messages, we should re-use the leader_epoch and
> >> > > > > partition_epoch provided by the seek(...) in the
> >> OffsetCommitRequest.
> >> > > And
> >> > > > > if messages have been successfully consumed, then leader_epoch
> >> will
> >> > > come
> >> > > > > from the messages returned in the fetch response. The condition
> >> > > "messages
> >> > > > > whose offset < the commit offset" is needed to take care of the
> >> log
> >> > > > > compacted topic which may have offset gap due to log cleaning.
> >> > > > >
> >> > > > > Did I miss something here? Or should I rephrase the paragraph to
> >> make
> >> > > it
> >> > > > > less confusing?
> >> > > > >
> >> > > > >
> >> > > > >> 64. Could you include the public methods in the OffsetEpoch
> >> class?
> >> > > > >>
> >> > > > >
> >> > > > > I mistakenly deleted the definition of OffsetEpoch class from
> the
> >> > KIP.
> >> > > I
> >> > > > > just added it back with the public methods. Could you take
> another
> >> > > look?
> >> > > > >
> >> > > > >
> >> > > > >>
> >> > > > >> Jun
> >> > > > >>
> >> > > > >>
> >> > > > >> On Thu, Jan 11, 2018 at 5:43 PM, Dong Lin <lindong28@gmail.com
> >
> >> > > wrote:
> >> > > > >>
> >> > > > >> > Hey Jun,
> >> > > > >> >
> >> > > > >> > Thanks much. I agree that we can not rely on committed
> offsets
> >> to
> >> > be
> >> > > > >> always
> >> > > > >> > deleted when we delete topic. So it is necessary to use a
> >> > > > per-partition
> >> > > > >> > epoch that does not change unless this partition is deleted.
> I
> >> > also
> >> > > > >> agree
> >> > > > >> > that it is very nice to be able to uniquely identify a
> message
> >> > with
> >> > > > >> > (offset, leader_epoch, partition_epoch) in face of potential
> >> topic
> >> > > > >> deletion
> >> > > > >> > and unclean leader election.
> >> > > > >> >
> >> > > > >> > I agree with all your comments. And I have updated the KIP
> >> based
> >> > on
> >> > > > our
> >> > > > >> > latest discussion. In addition, I added
> >> > > InvalidPartitionEpochException
> >> > > > >> > which will be thrown by consumer.poll() if the
> partition_epoch
> >> > > > >> associated
> >> > > > >> > with the partition, which can be given to consumer using
> >> > seek(...),
> >> > > is
> >> > > > >> > different from the partition_epoch in the FetchResponse.
> >> > > > >> >
> >> > > > >> > Can you take another look at the latest KIP?
> >> > > > >> >
> >> > > > >> > Thanks!
> >> > > > >> > Dong
> >> > > > >> >
> >> > > > >> >
> >> > > > >> >
> >> > > > >> > On Wed, Jan 10, 2018 at 2:24 PM, Jun Rao <ju...@confluent.io>
> >> > wrote:
> >> > > > >> >
> >> > > > >> > > Hi, Dong,
> >> > > > >> > >
> >> > > > >> > > My replies are the following.
> >> > > > >> > >
> >> > > > >> > > 60. What you described could also work. The drawback is
> that
> >> we
> >> > > will
> >> > > > >> be
> >> > > > >> > > unnecessarily changing the partition epoch when a partition
> >> > hasn't
> >> > > > >> really
> >> > > > >> > > changed. I was imagining that the partition epoch will be
> >> stored
> >> > > in
> >> > > > >> > > /brokers/topics/[topic]/partitions/[partitionId], instead
> >> of at
> >> > > the
> >> > > > >> > topic
> >> > > > >> > > level. So, not sure if ZK size limit is an issue.
> >> > > > >> > >
> >> > > > >> > > 61, 62 and 65. To me, the offset + offset_epoch is a unique
> >> > > > identifier
> >> > > > >> > for
> >> > > > >> > > a message. So, if a message hasn't changed, the offset and
> >> the
> >> > > > >> associated
> >> > > > >> > > offset_epoch ideally should remain the same (it will be
> kind
> >> of
> >> > > > weird
> >> > > > >> if
> >> > > > >> > > two consumer apps save the offset on the same message, but
> >> the
> >> > > > >> > offset_epoch
> >> > > > >> > > are different). partition_epoch + leader_epoch give us
> that.
> >> > > > >> > global_epoch +
> >> > > > >> > > leader_epoch don't. If we use this approach, we can solve
> not
> >> > only
> >> > > > the
> >> > > > >> > > problem that you have identified, but also other problems
> >> when
> >> > > there
> >> > > > >> is
> >> > > > >> > > data loss or topic re-creation more reliably. For example,
> in
> >> > the
> >> > > > >> future,
> >> > > > >> > > if we include the partition_epoch and leader_epoch in the
> >> fetch
> >> > > > >> request,
> >> > > > >> > > the server can do a more reliable check of whether that
> >> offset
> >> > is
> >> > > > >> valid
> >> > > > >> > or
> >> > > > >> > > not. I am not sure that we can rely upon all external
> >> offsets to
> >> > > be
> >> > > > >> > removed
> >> > > > >> > > on topic deletion. For example, a topic may be deleted by
> an
> >> > admin
> >> > > > who
> >> > > > >> > may
> >> > > > >> > > not know all the applications.
> >> > > > >> > >
> >> > > > >> > > If we agree on the above, the second question is then how
> to
> >> > > > reliably
> >> > > > >> > > propagate the partition_epoch and the leader_epoch to the
> >> > consumer
> >> > > > >> when
> >> > > > >> > > there are leader or partition changes. The leader_epoch
> comes
> >> > from
> >> > > > the
> >> > > > >> > > message, which is reliable. So, I was suggesting that when
> we
> >> > > store
> >> > > > an
> >> > > > >> > > offset, we can just store the leader_epoch from the message
> >> set
> >> > > > >> > containing
> >> > > > >> > > that offset. Similarly, I was thinking that if the
> >> > partition_epoch
> >> > > > is
> >> > > > >> in
> >> > > > >> > > the fetch response, we can propagate partition_epoch
> reliably
> >> > > where
> >> > > > is
> >> > > > >> > > partition_epoch change.
> >> > > > >> > >
> >> > > > >> > > 63. My point is that once a leader is producing a message
> in
> >> the
> >> > > new
> >> > > > >> > > partition_epoch, ideally, we should associate the new
> offsets
> >> > with
> >> > > > the
> >> > > > >> > new
> >> > > > >> > > partition_epoch. Otherwise, the offset_epoch won't be the
> >> > correct
> >> > > > >> unique
> >> > > > >> > > identifier (useful for solving other problems mentioned
> >> above).
> >> > I
> >> > > > was
> >> > > > >> > > originally thinking that the leader will include the
> >> > > partition_epoch
> >> > > > >> in
> >> > > > >> > the
> >> > > > >> > > metadata cache in the fetch response. It's just that right
> >> now,
> >> > > > >> metadata
> >> > > > >> > > cache is updated on UpdateMetadataRequest, which typically
> >> > happens
> >> > > > >> after
> >> > > > >> > > the LeaderAndIsrRequest. Another approach is for the leader
> >> to
> >> > > cache
> >> > > > >> the
> >> > > > >> > > partition_epoch in the Partition object and return that
> >> (instead
> >> > > of
> >> > > > >> the
> >> > > > >> > one
> >> > > > >> > > in metadata cache) in the fetch response.
> >> > > > >> > >
> >> > > > >> > > 65. It seems to me that the global_epoch and the
> >> partition_epoch
> >> > > > have
> >> > > > >> > > different purposes. A partition_epoch has the benefit that
> it
> >> > (1)
> >> > > > can
> >> > > > >> be
> >> > > > >> > > used to form a unique identifier for a message and (2) can
> be
> >> > used
> >> > > > to
> >> > > > >> > > solve other
> >> > > > >> > > corner case problems in the future. I am not sure having
> >> just a
> >> > > > >> > > global_epoch can achieve these. global_epoch is useful to
> >> > > determine
> >> > > > >> which
> >> > > > >> > > version of the metadata is newer, especially with topic
> >> > deletion.
> >> > > > >> > >
> >> > > > >> > > Thanks,
> >> > > > >> > >
> >> > > > >> > > Jun
> >> > > > >> > >
> >> > > > >> > > On Tue, Jan 9, 2018 at 11:34 PM, Dong Lin <
> >> lindong28@gmail.com>
> >> > > > >> wrote:
> >> > > > >> > >
> >> > > > >> > > > Regarding the use of the global epoch in 65), it is very
> >> > similar
> >> > > > to
> >> > > > >> the
> >> > > > >> > > > proposal of the metadata_epoch we discussed earlier. The
> >> main
> >> > > > >> > difference
> >> > > > >> > > is
> >> > > > >> > > > that this epoch is incremented when we
> create/expand/delete
> >> > > topic
> >> > > > >> and
> >> > > > >> > > does
> >> > > > >> > > > not change when controller re-send metadata.
> >> > > > >> > > >
> >> > > > >> > > > I looked at our previous discussion. It seems that we
> >> prefer
> >> > > > >> > > > partition_epoch over the metadata_epoch because 1) we
> >> prefer
> >> > not
> >> > > > to
> >> > > > >> > have
> >> > > > >> > > an
> >> > > > >> > > > ever growing metadata_epoch and 2) we can reset offset
> >> better
> >> > > when
> >> > > > >> > topic
> >> > > > >> > > is
> >> > > > >> > > > re-created. The use of global topic_epoch avoids the
> >> drawback
> >> > of
> >> > > > an
> >> > > > >> > ever
> >> > > > >> > > > quickly ever growing metadata_epoch. Though the global
> >> epoch
> >> > > does
> >> > > > >> not
> >> > > > >> > > allow
> >> > > > >> > > > us to recognize the invalid offset committed before the
> >> topic
> >> > > > >> > > re-creation,
> >> > > > >> > > > we can probably just delete the offset when we delete a
> >> topic.
> >> > > > Thus
> >> > > > >> I
> >> > > > >> > am
> >> > > > >> > > > not very sure whether it is still worthwhile to have a
> >> > > > per-partition
> >> > > > >> > > > partition_epoch if the metadata already has the global
> >> epoch.
> >> > > > >> > > >
> >> > > > >> > > >
> >> > > > >> > > > On Tue, Jan 9, 2018 at 6:58 PM, Dong Lin <
> >> lindong28@gmail.com
> >> > >
> >> > > > >> wrote:
> >> > > > >> > > >
> >> > > > >> > > > > Hey Jun,
> >> > > > >> > > > >
> >> > > > >> > > > > Thanks so much. These comments very useful. Please see
> >> below
> >> > > my
> >> > > > >> > > comments.
> >> > > > >> > > > >
> >> > > > >> > > > > On Mon, Jan 8, 2018 at 5:52 PM, Jun Rao <
> >> jun@confluent.io>
> >> > > > wrote:
> >> > > > >> > > > >
> >> > > > >> > > > >> Hi, Dong,
> >> > > > >> > > > >>
> >> > > > >> > > > >> Thanks for the updated KIP. A few more comments.
> >> > > > >> > > > >>
> >> > > > >> > > > >> 60. Perhaps having a partition epoch is more flexible
> >> since
> >> > > in
> >> > > > >> the
> >> > > > >> > > > future,
> >> > > > >> > > > >> we may support deleting a partition as well.
> >> > > > >> > > > >>
> >> > > > >> > > > >
> >> > > > >> > > > > Yeah I have considered this. I think we can probably
> >> still
> >> > > > support
> >> > > > >> > > > > deleting a partition by using the topic_epoch -- when
> >> > > partition
> >> > > > >> of a
> >> > > > >> > > > topic
> >> > > > >> > > > > is deleted or created, epoch of all partitions of this
> >> topic
> >> > > > will
> >> > > > >> be
> >> > > > >> > > > > incremented by 1. Therefore, if that partition is
> >> re-created
> >> > > > >> later,
> >> > > > >> > the
> >> > > > >> > > > > epoch of that partition will still be larger than its
> >> epoch
> >> > > > before
> >> > > > >> > the
> >> > > > >> > > > > deletion, which still allows the client to order the
> >> > metadata
> >> > > > for
> >> > > > >> the
> >> > > > >> > > > > purpose of this KIP. Does this sound reasonable?
> >> > > > >> > > > >
> >> > > > >> > > > > The advantage of using topic_epoch instead of
> >> > partition_epoch
> >> > > is
> >> > > > >> that
> >> > > > >> > > the
> >> > > > >> > > > > size of the /brokers/topics/[topic] znode and
> >> > request/response
> >> > > > >> size
> >> > > > >> > can
> >> > > > >> > > > be
> >> > > > >> > > > > smaller. We have a limit on the maximum size of znode
> >> > > (typically
> >> > > > >> > 1MB).
> >> > > > >> > > > Use
> >> > > > >> > > > > partition epoch can effectively reduce the number of
> >> > > partitions
> >> > > > >> that
> >> > > > >> > > can
> >> > > > >> > > > be
> >> > > > >> > > > > described by the /brokers/topics/[topic] znode.
> >> > > > >> > > > >
> >> > > > >> > > > > One use-case of partition_epoch for client to detect
> that
> >> > the
> >> > > > >> > committed
> >> > > > >> > > > > offset, either from kafka offset topic or from the
> >> external
> >> > > > store
> >> > > > >> is
> >> > > > >> > > > > invalid after partition deletion and re-creation.
> >> However,
> >> > it
> >> > > > >> seems
> >> > > > >> > > that
> >> > > > >> > > > we
> >> > > > >> > > > > can also address this use-case with other approaches.
> For
> >> > > > example,
> >> > > > >> > when
> >> > > > >> > > > > AdminClient deletes partitions, it can also delete the
> >> > > committed
> >> > > > >> > > offsets
> >> > > > >> > > > > for those partitions from the offset topic. If user
> >> stores
> >> > > > offset
> >> > > > >> > > > > externally, it might make sense for user to similarly
> >> remove
> >> > > > >> offsets
> >> > > > >> > of
> >> > > > >> > > > > related partitions after these partitions are deleted.
> >> So I
> >> > am
> >> > > > not
> >> > > > >> > sure
> >> > > > >> > > > > that we should use partition_epoch in this KIP.
> >> > > > >> > > > >
> >> > > > >> > > > >
> >> > > > >> > > > >>
> >> > > > >> > > > >> 61. It seems that the leader epoch returned in the
> >> > position()
> >> > > > >> call
> >> > > > >> > > > should
> >> > > > >> > > > >> the the leader epoch returned in the fetch response,
> not
> >> > the
> >> > > > one
> >> > > > >> in
> >> > > > >> > > the
> >> > > > >> > > > >> metadata cache of the client.
> >> > > > >> > > > >
> >> > > > >> > > > >
> >> > > > >> > > > > I think this is a good idea. Just to double check, this
> >> > change
> >> > > > >> does
> >> > > > >> > not
> >> > > > >> > > > > affect the correctness or performance of this KIP. But
> it
> >> > can
> >> > > be
> >> > > > >> > useful
> >> > > > >> > > > if
> >> > > > >> > > > > we want to use the leader_epoch to better handle the
> >> offset
> >> > > rest
> >> > > > >> in
> >> > > > >> > > case
> >> > > > >> > > > of
> >> > > > >> > > > > unclean leader election, which is listed in the future
> >> work.
> >> > > Is
> >> > > > >> this
> >> > > > >> > > > > understanding correct?
> >> > > > >> > > > >
> >> > > > >> > > > > I have updated the KIP to specify that the leader_epoch
> >> > > returned
> >> > > > >> by
> >> > > > >> > > > > position() should be the largest leader_epoch of those
> >> > already
> >> > > > >> > consumed
> >> > > > >> > > > > messages whose offset < position. If no message has
> been
> >> > > > consumed
> >> > > > >> > since
> >> > > > >> > > > > consumer initialization, the leader_epoch from seek()
> or
> >> > > > >> > > > > OffsetFetchResponse should be used. The offset included
> >> in
> >> > the
> >> > > > >> > > > > OffsetCommitRequest will also be determined in the
> >> similar
> >> > > > manner.
> >> > > > >> > > > >
> >> > > > >> > > > >
> >> > > > >> > > > >>
> >> > > > >> > > > >> 62. I am wondering if we should return the partition
> >> epoch
> >> > in
> >> > > > the
> >> > > > >> > > fetch
> >> > > > >> > > > >> response as well. In the current proposal, if a topic
> is
> >> > > > >> recreated
> >> > > > >> > and
> >> > > > >> > > > the
> >> > > > >> > > > >> new leader is on the same broker as the old one, there
> >> is
> >> > > > >> nothing to
> >> > > > >> > > > force
> >> > > > >> > > > >> the metadata refresh in the client. So, the client may
> >> > still
> >> > > > >> > associate
> >> > > > >> > > > the
> >> > > > >> > > > >> offset with the old partition epoch.
> >> > > > >> > > > >>
> >> > > > >> > > > >
> >> > > > >> > > > > Could you help me understand the problem if a client
> >> > > associates
> >> > > > >> old
> >> > > > >> > > > > partition_epoch (or the topic_epoch as of the current
> >> KIP)
> >> > > with
> >> > > > >> the
> >> > > > >> > > > offset?
> >> > > > >> > > > > The main purpose of the topic_epoch is to be able to
> drop
> >> > > > >> > leader_epoch
> >> > > > >> > > > to 0
> >> > > > >> > > > > after a partition is deleted and re-created. I guess
> you
> >> may
> >> > > be
> >> > > > >> > > thinking
> >> > > > >> > > > > about using the partition_epoch to detect that the
> >> committed
> >> > > > >> offset
> >> > > > >> > is
> >> > > > >> > > > > invalid? In that case, I am wondering if the
> alternative
> >> > > > approach
> >> > > > >> > > > described
> >> > > > >> > > > > in 60) would be reasonable.
> >> > > > >> > > > >
> >> > > > >> > > > >
> >> > > > >> > > > >>
> >> > > > >> > > > >> 63. There is some subtle coordination between the
> >> > > > >> > LeaderAndIsrRequest
> >> > > > >> > > > and
> >> > > > >> > > > >> UpdateMetadataRequest. Currently, when a leader
> changes,
> >> > the
> >> > > > >> > > controller
> >> > > > >> > > > >> first sends the LeaderAndIsrRequest to the assigned
> >> > replicas
> >> > > > and
> >> > > > >> the
> >> > > > >> > > > >> UpdateMetadataRequest to every broker. So, there could
> >> be a
> >> > > > small
> >> > > > >> > > window
> >> > > > >> > > > >> when the leader already receives the new partition
> >> epoch in
> >> > > the
> >> > > > >> > > > >> LeaderAndIsrRequest, but the metadata cache in the
> >> broker
> >> > > > hasn't
> >> > > > >> > been
> >> > > > >> > > > >> updated with the latest partition epoch. Not sure
> what's
> >> > the
> >> > > > best
> >> > > > >> > way
> >> > > > >> > > to
> >> > > > >> > > > >> address this issue. Perhaps we can update the metadata
> >> > cache
> >> > > on
> >> > > > >> the
> >> > > > >> > > > broker
> >> > > > >> > > > >> with both LeaderAndIsrRequest and
> UpdateMetadataRequest.
> >> > The
> >> > > > >> > challenge
> >> > > > >> > > > is
> >> > > > >> > > > >> that the two have slightly different data. For
> example,
> >> > only
> >> > > > the
> >> > > > >> > > latter
> >> > > > >> > > > >> has
> >> > > > >> > > > >> all endpoints.
> >> > > > >> > > > >>
> >> > > > >> > > > >
> >> > > > >> > > > > I am not sure whether this is a problem. Could you
> >> explain a
> >> > > bit
> >> > > > >> more
> >> > > > >> > > > what
> >> > > > >> > > > > specific problem this small window can cause?
> >> > > > >> > > > >
> >> > > > >> > > > > Since client can fetch metadata from any broker in the
> >> > > cluster,
> >> > > > >> and
> >> > > > >> > > given
> >> > > > >> > > > > that different brokers receive request (e.g.
> >> > > LeaderAndIsrRequest
> >> > > > >> and
> >> > > > >> > > > > UpdateMetadataRequest) in arbitrary order, the metadata
> >> > > received
> >> > > > >> by
> >> > > > >> > > > client
> >> > > > >> > > > > can be in arbitrary order (either newer or older)
> >> compared
> >> > to
> >> > > > the
> >> > > > >> > > > broker's
> >> > > > >> > > > > leadership state even if a given broker receives
> >> > > > >> LeaderAndIsrRequest
> >> > > > >> > > and
> >> > > > >> > > > > UpdateMetadataRequest simultaneously. So I am not sure
> >> it is
> >> > > > >> useful
> >> > > > >> > to
> >> > > > >> > > > > update broker's cache with LeaderAndIsrRequest.
> >> > > > >> > > > >
> >> > > > >> > > > >
> >> > > > >> > > > >> 64. The enforcement of leader epoch in Offset commit:
> We
> >> > > allow
> >> > > > a
> >> > > > >> > > > consumer
> >> > > > >> > > > >> to set an arbitrary offset. So it's possible for
> >> offsets or
> >> > > > >> leader
> >> > > > >> > > epoch
> >> > > > >> > > > >> to
> >> > > > >> > > > >> go backwards. I am not sure if we could always enforce
> >> that
> >> > > the
> >> > > > >> > leader
> >> > > > >> > > > >> epoch only goes up on the broker.
> >> > > > >> > > > >>
> >> > > > >> > > > >
> >> > > > >> > > > > Sure. I have removed this check from the KIP.
> >> > > > >> > > > >
> >> > > > >> > > > > BTW, we can probably still ensure that the leader_epoch
> >> > always
> >> > > > >> > increase
> >> > > > >> > > > if
> >> > > > >> > > > > the leader_epoch used with offset commit is the
> >> > > max(leader_epoch
> >> > > > >> of
> >> > > > >> > the
> >> > > > >> > > > > message with offset = the committed offset - 1, the
> >> largest
> >> > > > known
> >> > > > >> > > > > leader_epoch from the metadata). But I don't have a
> good
> >> > > > use-case
> >> > > > >> for
> >> > > > >> > > > this
> >> > > > >> > > > > alternative definition. So I choose the keep the KIP
> >> simple
> >> > by
> >> > > > >> > > requiring
> >> > > > >> > > > > leader_epoch to always increase.
> >> > > > >> > > > >
> >> > > > >> > > > >
> >> > > > >> > > > >> 65. Good point on handling missing partition epoch due
> >> to
> >> > > topic
> >> > > > >> > > > deletion.
> >> > > > >> > > > >> Another potential way to address this is to
> additionally
> >> > > > >> propagate
> >> > > > >> > the
> >> > > > >> > > > >> global partition epoch to brokers and the clients.
> This
> >> > way,
> >> > > > >> when a
> >> > > > >> > > > >> partition epoch is missing, we can use the global
> >> partition
> >> > > > >> epoch to
> >> > > > >> > > > >> reason
> >> > > > >> > > > >> about which metadata is more recent.
> >> > > > >> > > > >>
> >> > > > >> > > > >
> >> > > > >> > > > > This is a great idea. The global epoch can be used to
> >> order
> >> > > the
> >> > > > >> > > metadata
> >> > > > >> > > > > and help us recognize the more recent metadata if a
> topic
> >> > (or
> >> > > > >> > > partition)
> >> > > > >> > > > is
> >> > > > >> > > > > deleted and re-created.
> >> > > > >> > > > >
> >> > > > >> > > > > Actually, it seems we only need to propagate the global
> >> > epoch
> >> > > to
> >> > > > >> > > brokers
> >> > > > >> > > > > and clients without propagating this epoch on a
> >> per-topic or
> >> > > > >> > > > per-partition
> >> > > > >> > > > > basic. Doing so would simply interface changes made
> this
> >> > KIP.
> >> > > > Does
> >> > > > >> > this
> >> > > > >> > > > > approach sound reasonable?
> >> > > > >> > > > >
> >> > > > >> > > > >
> >> > > > >> > > > >> 66. A client may also get an offset by time using the
> >> > > > >> > offsetForTimes()
> >> > > > >> > > > >> api.
> >> > > > >> > > > >> So, we probably want to include offsetInternalMetadata
> >> in
> >> > > > >> > > > >> OffsetAndTimestamp
> >> > > > >> > > > >> as well.
> >> > > > >> > > > >>
> >> > > > >> > > > >
> >> > > > >> > > > > You are right. This probably also requires us to change
> >> the
> >> > > > >> > > > > ListOffsetRequest as well. I will update the KIP after
> we
> >> > > agree
> >> > > > on
> >> > > > >> > the
> >> > > > >> > > > > solution for 65).
> >> > > > >> > > > >
> >> > > > >> > > > >
> >> > > > >> > > > >>
> >> > > > >> > > > >> 67. InteralMetadata can be a bit confusing with the
> >> > metadata
> >> > > > >> field
> >> > > > >> > > > already
> >> > > > >> > > > >> there. Perhaps we can just call it OffsetEpoch. It
> >> might be
> >> > > > >> useful
> >> > > > >> > to
> >> > > > >> > > > make
> >> > > > >> > > > >> OffsetEpoch printable at least for debugging purpose.
> >> Once
> >> > > you
> >> > > > do
> >> > > > >> > > that,
> >> > > > >> > > > we
> >> > > > >> > > > >> are already exposing the internal fields. So, not sure
> >> if
> >> > > it's
> >> > > > >> worth
> >> > > > >> > > > >> hiding
> >> > > > >> > > > >> them. If we do want to hide them, perhaps we can have
> >> sth
> >> > > like
> >> > > > >> the
> >> > > > >> > > > >> following. The binary encoding is probably more
> >> efficient
> >> > > than
> >> > > > >> JSON
> >> > > > >> > > for
> >> > > > >> > > > >> external storage.
> >> > > > >> > > > >>
> >> > > > >> > > > >> OffsetEpoch {
> >> > > > >> > > > >>  static OffsetEpoch decode(byte[]);
> >> > > > >> > > > >>
> >> > > > >> > > > >>   public byte[] encode();
> >> > > > >> > > > >>
> >> > > > >> > > > >>   public String toString();
> >> > > > >> > > > >> }
> >> > > > >> > > > >>
> >> > > > >> > > > >
> >> > > > >> > > > > Thanks much. I like this solution. I have updated the
> KIP
> >> > > > >> > accordingly.
> >> > > > >> > > > >
> >> > > > >> > > > >
> >> > > > >> > > > >
> >> > > > >> > > > >>
> >> > > > >> > > > >> Jun
> >> > > > >> > > > >>
> >> > > > >> > > > >> On Mon, Jan 8, 2018 at 4:22 PM, Dong Lin <
> >> > > lindong28@gmail.com>
> >> > > > >> > wrote:
> >> > > > >> > > > >>
> >> > > > >> > > > >> > Hey Jason,
> >> > > > >> > > > >> >
> >> > > > >> > > > >> > Certainly. This sounds good. I have updated the KIP
> to
> >> > > > clarity
> >> > > > >> > that
> >> > > > >> > > > the
> >> > > > >> > > > >> > global epoch will be incremented by 1 each time a
> >> topic
> >> > is
> >> > > > >> > deleted.
> >> > > > >> > > > >> >
> >> > > > >> > > > >> > Thanks,
> >> > > > >> > > > >> > Dong
> >> > > > >> > > > >> >
> >> > > > >> > > > >> > On Mon, Jan 8, 2018 at 4:09 PM, Jason Gustafson <
> >> > > > >> > jason@confluent.io
> >> > > > >> > > >
> >> > > > >> > > > >> > wrote:
> >> > > > >> > > > >> >
> >> > > > >> > > > >> > > Hi Dong,
> >> > > > >> > > > >> > >
> >> > > > >> > > > >> > >
> >> > > > >> > > > >> > > I think your approach will allow user to
> distinguish
> >> > > > between
> >> > > > >> the
> >> > > > >> > > > >> metadata
> >> > > > >> > > > >> > > > before and after the topic deletion. I also
> agree
> >> > that
> >> > > > this
> >> > > > >> > can
> >> > > > >> > > be
> >> > > > >> > > > >> > > > potentially be useful to user. I am just not
> very
> >> > sure
> >> > > > >> whether
> >> > > > >> > > we
> >> > > > >> > > > >> > already
> >> > > > >> > > > >> > > > have a good use-case to make the additional
> >> > complexity
> >> > > > >> > > worthwhile.
> >> > > > >> > > > >> It
> >> > > > >> > > > >> > > seems
> >> > > > >> > > > >> > > > that this feature is kind of independent of the
> >> main
> >> > > > >> problem
> >> > > > >> > of
> >> > > > >> > > > this
> >> > > > >> > > > >> > KIP.
> >> > > > >> > > > >> > > > Could we add this as a future work?
> >> > > > >> > > > >> > >
> >> > > > >> > > > >> > >
> >> > > > >> > > > >> > > Do you think it's fair if we bump the topic epoch
> on
> >> > > > deletion
> >> > > > >> > and
> >> > > > >> > > > >> leave
> >> > > > >> > > > >> > > propagation of the epoch for deleted topics for
> >> future
> >> > > > work?
> >> > > > >> I
> >> > > > >> > > don't
> >> > > > >> > > > >> > think
> >> > > > >> > > > >> > > this adds much complexity and it makes the
> behavior
> >> > > > >> consistent:
> >> > > > >> > > > every
> >> > > > >> > > > >> > topic
> >> > > > >> > > > >> > > mutation results in an epoch bump.
> >> > > > >> > > > >> > >
> >> > > > >> > > > >> > > Thanks,
> >> > > > >> > > > >> > > Jason
> >> > > > >> > > > >> > >
> >> > > > >> > > > >> > > On Mon, Jan 8, 2018 at 3:14 PM, Dong Lin <
> >> > > > >> lindong28@gmail.com>
> >> > > > >> > > > wrote:
> >> > > > >> > > > >> > >
> >> > > > >> > > > >> > > > Hey Ismael,
> >> > > > >> > > > >> > > >
> >> > > > >> > > > >> > > > I guess we actually need user to see this field
> so
> >> > that
> >> > > > >> user
> >> > > > >> > can
> >> > > > >> > > > >> store
> >> > > > >> > > > >> > > this
> >> > > > >> > > > >> > > > value in the external store together with the
> >> offset.
> >> > > We
> >> > > > >> just
> >> > > > >> > > > prefer
> >> > > > >> > > > >> > the
> >> > > > >> > > > >> > > > value to be opaque to discourage most users from
> >> > > > >> interpreting
> >> > > > >> > > this
> >> > > > >> > > > >> > value.
> >> > > > >> > > > >> > > > One more advantage of using such an opaque field
> >> is
> >> > to
> >> > > be
> >> > > > >> able
> >> > > > >> > > to
> >> > > > >> > > > >> > evolve
> >> > > > >> > > > >> > > > the information (or schema) of this value
> without
> >> > > > changing
> >> > > > >> > > > consumer
> >> > > > >> > > > >> API
> >> > > > >> > > > >> > > in
> >> > > > >> > > > >> > > > the future.
> >> > > > >> > > > >> > > >
> >> > > > >> > > > >> > > > I also thinking it is probably OK for user to be
> >> able
> >> > > to
> >> > > > >> > > interpret
> >> > > > >> > > > >> this
> >> > > > >> > > > >> > > > value, particularly for those advanced users.
> >> > > > >> > > > >> > > >
> >> > > > >> > > > >> > > > Thanks,
> >> > > > >> > > > >> > > > Dong
> >> > > > >> > > > >> > > >
> >> > > > >> > > > >> > > > On Mon, Jan 8, 2018 at 2:34 PM, Ismael Juma <
> >> > > > >> > ismael@juma.me.uk>
> >> > > > >> > > > >> wrote:
> >> > > > >> > > > >> > > >
> >> > > > >> > > > >> > > > > On Fri, Jan 5, 2018 at 7:15 PM, Jason
> Gustafson
> >> <
> >> > > > >> > > > >> jason@confluent.io>
> >> > > > >> > > > >> > > > > wrote:
> >> > > > >> > > > >> > > > > >
> >> > > > >> > > > >> > > > > > class OffsetAndMetadata {
> >> > > > >> > > > >> > > > > >   long offset;
> >> > > > >> > > > >> > > > > >   byte[] offsetMetadata;
> >> > > > >> > > > >> > > > > >   String metadata;
> >> > > > >> > > > >> > > > > > }
> >> > > > >> > > > >> > > > >
> >> > > > >> > > > >> > > > >
> >> > > > >> > > > >> > > > > > Admittedly, the naming is a bit annoying,
> but
> >> we
> >> > > can
> >> > > > >> > > probably
> >> > > > >> > > > >> come
> >> > > > >> > > > >> > up
> >> > > > >> > > > >> > > > > with
> >> > > > >> > > > >> > > > > > something better. Internally the byte array
> >> would
> >> > > > have
> >> > > > >> a
> >> > > > >> > > > >> version.
> >> > > > >> > > > >> > If
> >> > > > >> > > > >> > > in
> >> > > > >> > > > >> > > > > the
> >> > > > >> > > > >> > > > > > future we have anything else we need to add,
> >> we
> >> > can
> >> > > > >> update
> >> > > > >> > > the
> >> > > > >> > > > >> > > version
> >> > > > >> > > > >> > > > > and
> >> > > > >> > > > >> > > > > > we wouldn't need any new APIs.
> >> > > > >> > > > >> > > > > >
> >> > > > >> > > > >> > > > >
> >> > > > >> > > > >> > > > > We can also add fields to a class in a
> >> compatible
> >> > > way.
> >> > > > >> So,
> >> > > > >> > it
> >> > > > >> > > > >> seems
> >> > > > >> > > > >> > to
> >> > > > >> > > > >> > > me
> >> > > > >> > > > >> > > > > that the main advantage of the byte array is
> >> that
> >> > > it's
> >> > > > >> > opaque
> >> > > > >> > > to
> >> > > > >> > > > >> the
> >> > > > >> > > > >> > > > user.
> >> > > > >> > > > >> > > > > Is that correct? If so, we could also add any
> >> > opaque
> >> > > > >> > metadata
> >> > > > >> > > > in a
> >> > > > >> > > > >> > > > subclass
> >> > > > >> > > > >> > > > > so that users don't even see it (unless they
> >> cast
> >> > it,
> >> > > > but
> >> > > > >> > then
> >> > > > >> > > > >> > they're
> >> > > > >> > > > >> > > on
> >> > > > >> > > > >> > > > > their own).
> >> > > > >> > > > >> > > > >
> >> > > > >> > > > >> > > > > Ismael
> >> > > > >> > > > >> > > > >
> >> > > > >> > > > >> > > > > The corresponding seek() and position() APIs
> >> might
> >> > > look
> >> > > > >> > > > something
> >> > > > >> > > > >> > like
> >> > > > >> > > > >> > > > > this:
> >> > > > >> > > > >> > > > > >
> >> > > > >> > > > >> > > > > > void seek(TopicPartition partition, long
> >> offset,
> >> > > > byte[]
> >> > > > >> > > > >> > > > offsetMetadata);
> >> > > > >> > > > >> > > > > > byte[] positionMetadata(TopicPartition
> >> > partition);
> >> > > > >> > > > >> > > > > >
> >> > > > >> > > > >> > > > > > What do you think?
> >> > > > >> > > > >> > > > > >
> >> > > > >> > > > >> > > > > > Thanks,
> >> > > > >> > > > >> > > > > > Jason
> >> > > > >> > > > >> > > > > >
> >> > > > >> > > > >> > > > > > On Thu, Jan 4, 2018 at 7:04 PM, Dong Lin <
> >> > > > >> > > lindong28@gmail.com
> >> > > > >> > > > >
> >> > > > >> > > > >> > > wrote:
> >> > > > >> > > > >> > > > > >
> >> > > > >> > > > >> > > > > > > Hey Jun, Jason,
> >> > > > >> > > > >> > > > > > >
> >> > > > >> > > > >> > > > > > > Thanks much for all the feedback. I have
> >> > updated
> >> > > > the
> >> > > > >> KIP
> >> > > > >> > > > >> based on
> >> > > > >> > > > >> > > the
> >> > > > >> > > > >> > > > > > > latest discussion. Can you help check
> >> whether
> >> > it
> >> > > > >> looks
> >> > > > >> > > good?
> >> > > > >> > > > >> > > > > > >
> >> > > > >> > > > >> > > > > > > Thanks,
> >> > > > >> > > > >> > > > > > > Dong
> >> > > > >> > > > >> > > > > > >
> >> > > > >> > > > >> > > > > > > On Thu, Jan 4, 2018 at 5:36 PM, Dong Lin <
> >> > > > >> > > > lindong28@gmail.com
> >> > > > >> > > > >> >
> >> > > > >> > > > >> > > > wrote:
> >> > > > >> > > > >> > > > > > >
> >> > > > >> > > > >> > > > > > > > Hey Jun,
> >> > > > >> > > > >> > > > > > > >
> >> > > > >> > > > >> > > > > > > > Hmm... thinking about this more, I am
> not
> >> > sure
> >> > > > that
> >> > > > >> > the
> >> > > > >> > > > >> > proposed
> >> > > > >> > > > >> > > > API
> >> > > > >> > > > >> > > > > is
> >> > > > >> > > > >> > > > > > > > sufficient. For users that store offset
> >> > > > >> externally, we
> >> > > > >> > > > >> probably
> >> > > > >> > > > >> > > > need
> >> > > > >> > > > >> > > > > > > extra
> >> > > > >> > > > >> > > > > > > > API to return the leader_epoch and
> >> > > > partition_epoch
> >> > > > >> for
> >> > > > >> > > all
> >> > > > >> > > > >> > > > partitions
> >> > > > >> > > > >> > > > > > > that
> >> > > > >> > > > >> > > > > > > > consumers are consuming. I suppose these
> >> > users
> >> > > > >> > currently
> >> > > > >> > > > use
> >> > > > >> > > > >> > > > > position()
> >> > > > >> > > > >> > > > > > > to
> >> > > > >> > > > >> > > > > > > > get the offset. Thus we probably need a
> >> new
> >> > > > method
> >> > > > >> > > > >> > > > > > positionWithEpoch(..)
> >> > > > >> > > > >> > > > > > > to
> >> > > > >> > > > >> > > > > > > > return <offset, partition_epoch,
> >> > leader_epoch>.
> >> > > > >> Does
> >> > > > >> > > this
> >> > > > >> > > > >> sound
> >> > > > >> > > > >> > > > > > > reasonable?
> >> > > > >> > > > >> > > > > > > >
> >> > > > >> > > > >> > > > > > > > Thanks,
> >> > > > >> > > > >> > > > > > > > Dong
> >> > > > >> > > > >> > > > > > > >
> >> > > > >> > > > >> > > > > > > >
> >> > > > >> > > > >> > > > > > > > On Thu, Jan 4, 2018 at 5:26 PM, Jun Rao
> <
> >> > > > >> > > jun@confluent.io
> >> > > > >> > > > >
> >> > > > >> > > > >> > > wrote:
> >> > > > >> > > > >> > > > > > > >
> >> > > > >> > > > >> > > > > > > >> Hi, Dong,
> >> > > > >> > > > >> > > > > > > >>
> >> > > > >> > > > >> > > > > > > >> Yes, that's what I am thinking.
> >> OffsetEpoch
> >> > > will
> >> > > > >> be
> >> > > > >> > > > >> composed
> >> > > > >> > > > >> > of
> >> > > > >> > > > >> > > > > > > >> (partition_epoch,
> >> > > > >> > > > >> > > > > > > >> leader_epoch).
> >> > > > >> > > > >> > > > > > > >>
> >> > > > >> > > > >> > > > > > > >> Thanks,
> >> > > > >> > > > >> > > > > > > >>
> >> > > > >> > > > >> > > > > > > >> Jun
> >> > > > >> > > > >> > > > > > > >>
> >> > > > >> > > > >> > > > > > > >>
> >> > > > >> > > > >> > > > > > > >> On Thu, Jan 4, 2018 at 4:22 PM, Dong
> Lin
> >> <
> >> > > > >> > > > >> lindong28@gmail.com
> >> > > > >> > > > >> > >
> >> > > > >> > > > >> > > > > wrote:
> >> > > > >> > > > >> > > > > > > >>
> >> > > > >> > > > >> > > > > > > >> > Hey Jun,
> >> > > > >> > > > >> > > > > > > >> >
> >> > > > >> > > > >> > > > > > > >> > Thanks much. I like the the new API
> >> that
> >> > you
> >> > > > >> > > proposed.
> >> > > > >> > > > I
> >> > > > >> > > > >> am
> >> > > > >> > > > >> > > not
> >> > > > >> > > > >> > > > > sure
> >> > > > >> > > > >> > > > > > > >> what
> >> > > > >> > > > >> > > > > > > >> > you exactly mean by offset_epoch. I
> >> > suppose
> >> > > > >> that we
> >> > > > >> > > can
> >> > > > >> > > > >> use
> >> > > > >> > > > >> > > the
> >> > > > >> > > > >> > > > > pair
> >> > > > >> > > > >> > > > > > > of
> >> > > > >> > > > >> > > > > > > >> > (partition_epoch, leader_epoch) as
> the
> >> > > > >> > offset_epoch,
> >> > > > >> > > > >> right?
> >> > > > >> > > > >> > > > > > > >> >
> >> > > > >> > > > >> > > > > > > >> > Thanks,
> >> > > > >> > > > >> > > > > > > >> > Dong
> >> > > > >> > > > >> > > > > > > >> >
> >> > > > >> > > > >> > > > > > > >> > On Thu, Jan 4, 2018 at 4:02 PM, Jun
> >> Rao <
> >> > > > >> > > > >> jun@confluent.io>
> >> > > > >> > > > >> > > > wrote:
> >> > > > >> > > > >> > > > > > > >> >
> >> > > > >> > > > >> > > > > > > >> > > Hi, Dong,
> >> > > > >> > > > >> > > > > > > >> > >
> >> > > > >> > > > >> > > > > > > >> > > Got it. The api that you proposed
> >> works.
> >> > > The
> >> > > > >> > > question
> >> > > > >> > > > >> is
> >> > > > >> > > > >> > > > whether
> >> > > > >> > > > >> > > > > > > >> that's
> >> > > > >> > > > >> > > > > > > >> > the
> >> > > > >> > > > >> > > > > > > >> > > api that we want to have in the
> long
> >> > term.
> >> > > > My
> >> > > > >> > > concern
> >> > > > >> > > > >> is
> >> > > > >> > > > >> > > that
> >> > > > >> > > > >> > > > > > while
> >> > > > >> > > > >> > > > > > > >> the
> >> > > > >> > > > >> > > > > > > >> > api
> >> > > > >> > > > >> > > > > > > >> > > change is simple, the new api seems
> >> > harder
> >> > > > to
> >> > > > >> > > explain
> >> > > > >> > > > >> and
> >> > > > >> > > > >> > > use.
> >> > > > >> > > > >> > > > > For
> >> > > > >> > > > >> > > > > > > >> > example,
> >> > > > >> > > > >> > > > > > > >> > > a consumer storing offsets
> externally
> >> > now
> >> > > > >> needs
> >> > > > >> > to
> >> > > > >> > > > call
> >> > > > >> > > > >> > > > > > > >> > > waitForMetadataUpdate() after
> calling
> >> > > > seek().
> >> > > > >> > > > >> > > > > > > >> > >
> >> > > > >> > > > >> > > > > > > >> > > An alternative approach is to make
> >> the
> >> > > > >> following
> >> > > > >> > > > >> > compatible
> >> > > > >> > > > >> > > > api
> >> > > > >> > > > >> > > > > > > >> changes
> >> > > > >> > > > >> > > > > > > >> > in
> >> > > > >> > > > >> > > > > > > >> > > Consumer.
> >> > > > >> > > > >> > > > > > > >> > > * Add an additional OffsetEpoch
> >> field in
> >> > > > >> > > > >> > OffsetAndMetadata.
> >> > > > >> > > > >> > > > (no
> >> > > > >> > > > >> > > > > > need
> >> > > > >> > > > >> > > > > > > >> to
> >> > > > >> > > > >> > > > > > > >> > > change the CommitSync() api)
> >> > > > >> > > > >> > > > > > > >> > > * Add a new api seek(TopicPartition
> >> > > > partition,
> >> > > > >> > long
> >> > > > >> > > > >> > offset,
> >> > > > >> > > > >> > > > > > > >> OffsetEpoch
> >> > > > >> > > > >> > > > > > > >> > > offsetEpoch). We can potentially
> >> > deprecate
> >> > > > the
> >> > > > >> > old
> >> > > > >> > > > api
> >> > > > >> > > > >> > > > > > > >> > seek(TopicPartition
> >> > > > >> > > > >> > > > > > > >> > > partition, long offset) in the
> >> future.
> >> > > > >> > > > >> > > > > > > >> > >
> >> > > > >> > > > >> > > > > > > >> > > The alternative approach has
> similar
> >> > > amount
> >> > > > of
> >> > > > >> > api
> >> > > > >> > > > >> changes
> >> > > > >> > > > >> > > as
> >> > > > >> > > > >> > > > > > yours
> >> > > > >> > > > >> > > > > > > >> but
> >> > > > >> > > > >> > > > > > > >> > has
> >> > > > >> > > > >> > > > > > > >> > > the following benefits.
> >> > > > >> > > > >> > > > > > > >> > > 1. The api works in a similar way
> as
> >> how
> >> > > > >> offset
> >> > > > >> > > > >> management
> >> > > > >> > > > >> > > > works
> >> > > > >> > > > >> > > > > > now
> >> > > > >> > > > >> > > > > > > >> and
> >> > > > >> > > > >> > > > > > > >> > is
> >> > > > >> > > > >> > > > > > > >> > > probably what we want in the long
> >> term.
> >> > > > >> > > > >> > > > > > > >> > > 2. It can reset offsets better when
> >> > there
> >> > > is
> >> > > > >> data
> >> > > > >> > > > loss
> >> > > > >> > > > >> due
> >> > > > >> > > > >> > > to
> >> > > > >> > > > >> > > > > > > unclean
> >> > > > >> > > > >> > > > > > > >> > > leader election or correlated
> replica
> >> > > > failure.
> >> > > > >> > > > >> > > > > > > >> > > 3. It can reset offsets better when
> >> > topic
> >> > > is
> >> > > > >> > > > recreated.
> >> > > > >> > > > >> > > > > > > >> > >
> >> > > > >> > > > >> > > > > > > >> > > Thanks,
> >> > > > >> > > > >> > > > > > > >> > >
> >> > > > >> > > > >> > > > > > > >> > > Jun
> >> > > > >> > > > >> > > > > > > >> > >
> >> > > > >> > > > >> > > > > > > >> > >
> >> > > > >> > > > >> > > > > > > >> > > On Thu, Jan 4, 2018 at 2:05 PM,
> Dong
> >> > Lin <
> >> > > > >> > > > >> > > lindong28@gmail.com
> >> > > > >> > > > >> > > > >
> >> > > > >> > > > >> > > > > > > wrote:
> >> > > > >> > > > >> > > > > > > >> > >
> >> > > > >> > > > >> > > > > > > >> > > > Hey Jun,
> >> > > > >> > > > >> > > > > > > >> > > >
> >> > > > >> > > > >> > > > > > > >> > > > Yeah I agree that ideally we
> don't
> >> > want
> >> > > an
> >> > > > >> ever
> >> > > > >> > > > >> growing
> >> > > > >> > > > >> > > > global
> >> > > > >> > > > >> > > > > > > >> metadata
> >> > > > >> > > > >> > > > > > > >> > > > version. I just think it may be
> >> more
> >> > > > >> desirable
> >> > > > >> > to
> >> > > > >> > > > >> keep
> >> > > > >> > > > >> > the
> >> > > > >> > > > >> > > > > > > consumer
> >> > > > >> > > > >> > > > > > > >> API
> >> > > > >> > > > >> > > > > > > >> > > > simple.
> >> > > > >> > > > >> > > > > > > >> > > >
> >> > > > >> > > > >> > > > > > > >> > > > In my current proposal, metadata
> >> > version
> >> > > > >> > returned
> >> > > > >> > > > in
> >> > > > >> > > > >> the
> >> > > > >> > > > >> > > > fetch
> >> > > > >> > > > >> > > > > > > >> response
> >> > > > >> > > > >> > > > > > > >> > > > will be stored with the offset
> >> > together.
> >> > > > >> More
> >> > > > >> > > > >> > > specifically,
> >> > > > >> > > > >> > > > > the
> >> > > > >> > > > >> > > > > > > >> > > > metadata_epoch in the new offset
> >> topic
> >> > > > >> schema
> >> > > > >> > > will
> >> > > > >> > > > be
> >> > > > >> > > > >> > the
> >> > > > >> > > > >> > > > > > largest
> >> > > > >> > > > >> > > > > > > >> > > > metadata_epoch from all the
> >> > > > MetadataResponse
> >> > > > >> > and
> >> > > > >> > > > >> > > > FetchResponse
> >> > > > >> > > > >> > > > > > > ever
> >> > > > >> > > > >> > > > > > > >> > > > received by this consumer.
> >> > > > >> > > > >> > > > > > > >> > > >
> >> > > > >> > > > >> > > > > > > >> > > > We probably don't have to change
> >> the
> >> > > > >> consumer
> >> > > > >> > API
> >> > > > >> > > > for
> >> > > > >> > > > >> > > > > > > >> > > > commitSync(Map<TopicPartition,
> >> > > > >> > > OffsetAndMetadata>).
> >> > > > >> > > > >> If
> >> > > > >> > > > >> > > user
> >> > > > >> > > > >> > > > > > calls
> >> > > > >> > > > >> > > > > > > >> > > > commitSync(...) to commit offset
> 10
> >> > for
> >> > > a
> >> > > > >> given
> >> > > > >> > > > >> > partition,
> >> > > > >> > > > >> > > > for
> >> > > > >> > > > >> > > > > > > most
> >> > > > >> > > > >> > > > > > > >> > > > use-cases, this consumer instance
> >> > should
> >> > > > >> have
> >> > > > >> > > > >> consumed
> >> > > > >> > > > >> > > > message
> >> > > > >> > > > >> > > > > > > with
> >> > > > >> > > > >> > > > > > > >> > > offset
> >> > > > >> > > > >> > > > > > > >> > > > 9 from this partition, in which
> >> case
> >> > the
> >> > > > >> > consumer
> >> > > > >> > > > can
> >> > > > >> > > > >> > > > remember
> >> > > > >> > > > >> > > > > > and
> >> > > > >> > > > >> > > > > > > >> use
> >> > > > >> > > > >> > > > > > > >> > > the
> >> > > > >> > > > >> > > > > > > >> > > > metadata_epoch from the
> >> corresponding
> >> > > > >> > > FetchResponse
> >> > > > >> > > > >> when
> >> > > > >> > > > >> > > > > > > committing
> >> > > > >> > > > >> > > > > > > >> > > offset.
> >> > > > >> > > > >> > > > > > > >> > > > If user calls commitSync(..) to
> >> commit
> >> > > > >> offset
> >> > > > >> > 10
> >> > > > >> > > > for
> >> > > > >> > > > >> a
> >> > > > >> > > > >> > > given
> >> > > > >> > > > >> > > > > > > >> partition
> >> > > > >> > > > >> > > > > > > >> > > > without having consumed the
> message
> >> > with
> >> > > > >> > offset 9
> >> > > > >> > > > >> using
> >> > > > >> > > > >> > > this
> >> > > > >> > > > >> > > > > > > >> consumer
> >> > > > >> > > > >> > > > > > > >> > > > instance, this is probably an
> >> advanced
> >> > > > >> > use-case.
> >> > > > >> > > In
> >> > > > >> > > > >> this
> >> > > > >> > > > >> > > > case
> >> > > > >> > > > >> > > > > > the
> >> > > > >> > > > >> > > > > > > >> > > advanced
> >> > > > >> > > > >> > > > > > > >> > > > user can retrieve the
> >> metadata_epoch
> >> > > using
> >> > > > >> the
> >> > > > >> > > > newly
> >> > > > >> > > > >> > added
> >> > > > >> > > > >> > > > > > > >> > > metadataEpoch()
> >> > > > >> > > > >> > > > > > > >> > > > API after it fetches the message
> >> with
> >> > > > >> offset 9
> >> > > > >> > > > >> (probably
> >> > > > >> > > > >> > > > from
> >> > > > >> > > > >> > > > > > > >> another
> >> > > > >> > > > >> > > > > > > >> > > > consumer instance) and encode
> this
> >> > > > >> > metadata_epoch
> >> > > > >> > > > in
> >> > > > >> > > > >> the
> >> > > > >> > > > >> > > > > > > >> > > > string
> OffsetAndMetadata.metadata.
> >> Do
> >> > > you
> >> > > > >> think
> >> > > > >> > > > this
> >> > > > >> > > > >> > > > solution
> >> > > > >> > > > >> > > > > > > would
> >> > > > >> > > > >> > > > > > > >> > work?
> >> > > > >> > > > >> > > > > > > >> > > >
> >> > > > >> > > > >> > > > > > > >> > > > By "not sure that I fully
> >> understand
> >> > > your
> >> > > > >> > latest
> >> > > > >> > > > >> > > > suggestion",
> >> > > > >> > > > >> > > > > > are
> >> > > > >> > > > >> > > > > > > >> you
> >> > > > >> > > > >> > > > > > > >> > > > referring to solution related to
> >> > unclean
> >> > > > >> leader
> >> > > > >> > > > >> election
> >> > > > >> > > > >> > > > using
> >> > > > >> > > > >> > > > > > > >> > > leader_epoch
> >> > > > >> > > > >> > > > > > > >> > > > in my previous email?
> >> > > > >> > > > >> > > > > > > >> > > >
> >> > > > >> > > > >> > > > > > > >> > > > Thanks,
> >> > > > >> > > > >> > > > > > > >> > > > Dong
> >> > > > >> > > > >> > > > > > > >> > > >
> >> > > > >> > > > >> > > > > > > >> > > > On Thu, Jan 4, 2018 at 1:33 PM,
> Jun
> >> > Rao
> >> > > <
> >> > > > >> > > > >> > jun@confluent.io
> >> > > > >> > > > >> > > >
> >> > > > >> > > > >> > > > > > wrote:
> >> > > > >> > > > >> > > > > > > >> > > >
> >> > > > >> > > > >> > > > > > > >> > > > > Hi, Dong,
> >> > > > >> > > > >> > > > > > > >> > > > >
> >> > > > >> > > > >> > > > > > > >> > > > > Not sure that I fully
> understand
> >> > your
> >> > > > >> latest
> >> > > > >> > > > >> > suggestion.
> >> > > > >> > > > >> > > > > > > >> Returning an
> >> > > > >> > > > >> > > > > > > >> > > > ever
> >> > > > >> > > > >> > > > > > > >> > > > > growing global metadata version
> >> > itself
> >> > > > is
> >> > > > >> no
> >> > > > >> > > > ideal,
> >> > > > >> > > > >> > but
> >> > > > >> > > > >> > > is
> >> > > > >> > > > >> > > > > > fine.
> >> > > > >> > > > >> > > > > > > >> My
> >> > > > >> > > > >> > > > > > > >> > > > > question is whether the
> metadata
> >> > > version
> >> > > > >> > > returned
> >> > > > >> > > > >> in
> >> > > > >> > > > >> > the
> >> > > > >> > > > >> > > > > fetch
> >> > > > >> > > > >> > > > > > > >> > response
> >> > > > >> > > > >> > > > > > > >> > > > > needs to be stored with the
> >> offset
> >> > > > >> together
> >> > > > >> > if
> >> > > > >> > > > >> offsets
> >> > > > >> > > > >> > > are
> >> > > > >> > > > >> > > > > > > stored
> >> > > > >> > > > >> > > > > > > >> > > > > externally. If so, we also have
> >> to
> >> > > > change
> >> > > > >> the
> >> > > > >> > > > >> consumer
> >> > > > >> > > > >> > > API
> >> > > > >> > > > >> > > > > for
> >> > > > >> > > > >> > > > > > > >> > > > commitSync()
> >> > > > >> > > > >> > > > > > > >> > > > > and need to worry about
> >> > compatibility.
> >> > > > If
> >> > > > >> we
> >> > > > >> > > > don't
> >> > > > >> > > > >> > store
> >> > > > >> > > > >> > > > the
> >> > > > >> > > > >> > > > > > > >> metadata
> >> > > > >> > > > >> > > > > > > >> > > > > version together with the
> offset,
> >> > on a
> >> > > > >> > consumer
> >> > > > >> > > > >> > restart,
> >> > > > >> > > > >> > > > > it's
> >> > > > >> > > > >> > > > > > > not
> >> > > > >> > > > >> > > > > > > >> > clear
> >> > > > >> > > > >> > > > > > > >> > > > how
> >> > > > >> > > > >> > > > > > > >> > > > > we can ensure the metadata in
> the
> >> > > > >> consumer is
> >> > > > >> > > > high
> >> > > > >> > > > >> > > enough
> >> > > > >> > > > >> > > > > > since
> >> > > > >> > > > >> > > > > > > >> there
> >> > > > >> > > > >> > > > > > > >> > > is
> >> > > > >> > > > >> > > > > > > >> > > > no
> >> > > > >> > > > >> > > > > > > >> > > > > metadata version to compare
> with.
> >> > > > >> > > > >> > > > > > > >> > > > >
> >> > > > >> > > > >> > > > > > > >> > > > > Thanks,
> >> > > > >> > > > >> > > > > > > >> > > > >
> >> > > > >> > > > >> > > > > > > >> > > > > Jun
> >> > > > >> > > > >> > > > > > > >> > > > >
> >> > > > >> > > > >> > > > > > > >> > > > >
> >> > > > >> > > > >> > > > > > > >> > > > > On Wed, Jan 3, 2018 at 6:43 PM,
> >> Dong
> >> > > > Lin <
> >> > > > >> > > > >> > > > > lindong28@gmail.com
> >> > > > >> > > > >> > > > > > >
> >> > > > >> > > > >> > > > > > > >> > wrote:
> >> > > > >> > > > >> > > > > > > >> > > > >
> >> > > > >> > > > >> > > > > > > >> > > > > > Hey Jun,
> >> > > > >> > > > >> > > > > > > >> > > > > >
> >> > > > >> > > > >> > > > > > > >> > > > > > Thanks much for the
> >> explanation.
> >> > > > >> > > > >> > > > > > > >> > > > > >
> >> > > > >> > > > >> > > > > > > >> > > > > > I understand the advantage of
> >> > > > >> > partition_epoch
> >> > > > >> > > > >> over
> >> > > > >> > > > >> > > > > > > >> metadata_epoch.
> >> > > > >> > > > >> > > > > > > >> > My
> >> > > > >> > > > >> > > > > > > >> > > > > > current concern is that the
> >> use of
> >> > > > >> > > leader_epoch
> >> > > > >> > > > >> and
> >> > > > >> > > > >> > > the
> >> > > > >> > > > >> > > > > > > >> > > partition_epoch
> >> > > > >> > > > >> > > > > > > >> > > > > > requires us considerable
> >> change to
> >> > > > >> > consumer's
> >> > > > >> > > > >> public
> >> > > > >> > > > >> > > API
> >> > > > >> > > > >> > > > > to
> >> > > > >> > > > >> > > > > > > take
> >> > > > >> > > > >> > > > > > > >> > care
> >> > > > >> > > > >> > > > > > > >> > > > of
> >> > > > >> > > > >> > > > > > > >> > > > > > the case where user stores
> >> offset
> >> > > > >> > externally.
> >> > > > >> > > > For
> >> > > > >> > > > >> > > > example,
> >> > > > >> > > > >> > > > > > > >> > > *consumer*.
> >> > > > >> > > > >> > > > > > > >> > > > > > *commitSync*(..) would have
> to
> >> > take
> >> > > a
> >> > > > >> map
> >> > > > >> > > whose
> >> > > > >> > > > >> > value
> >> > > > >> > > > >> > > is
> >> > > > >> > > > >> > > > > > > >> <offset,
> >> > > > >> > > > >> > > > > > > >> > > > > metadata,
> >> > > > >> > > > >> > > > > > > >> > > > > > leader epoch, partition
> epoch>.
> >> > > > >> > > > >> > *consumer*.*seek*(...)
> >> > > > >> > > > >> > > > > would
> >> > > > >> > > > >> > > > > > > >> also
> >> > > > >> > > > >> > > > > > > >> > > need
> >> > > > >> > > > >> > > > > > > >> > > > > > leader_epoch and
> >> partition_epoch
> >> > as
> >> > > > >> > > parameter.
> >> > > > >> > > > >> > > > Technically
> >> > > > >> > > > >> > > > > > we
> >> > > > >> > > > >> > > > > > > >> can
> >> > > > >> > > > >> > > > > > > >> > > > > probably
> >> > > > >> > > > >> > > > > > > >> > > > > > still make it work in a
> >> backward
> >> > > > >> compatible
> >> > > > >> > > > >> manner
> >> > > > >> > > > >> > > after
> >> > > > >> > > > >> > > > > > > careful
> >> > > > >> > > > >> > > > > > > >> > > design
> >> > > > >> > > > >> > > > > > > >> > > > > and
> >> > > > >> > > > >> > > > > > > >> > > > > > discussion. But these changes
> >> can
> >> > > make
> >> > > > >> the
> >> > > > >> > > > >> > consumer's
> >> > > > >> > > > >> > > > > > > interface
> >> > > > >> > > > >> > > > > > > >> > > > > > unnecessarily complex for
> more
> >> > users
> >> > > > >> who do
> >> > > > >> > > not
> >> > > > >> > > > >> > store
> >> > > > >> > > > >> > > > > offset
> >> > > > >> > > > >> > > > > > > >> > > > externally.
> >> > > > >> > > > >> > > > > > > >> > > > > >
> >> > > > >> > > > >> > > > > > > >> > > > > > After thinking more about it,
> >> we
> >> > can
> >> > > > >> > address
> >> > > > >> > > > all
> >> > > > >> > > > >> > > > problems
> >> > > > >> > > > >> > > > > > > >> discussed
> >> > > > >> > > > >> > > > > > > >> > > by
> >> > > > >> > > > >> > > > > > > >> > > > > only
> >> > > > >> > > > >> > > > > > > >> > > > > > using the metadata_epoch
> >> without
> >> > > > >> > introducing
> >> > > > >> > > > >> > > > leader_epoch
> >> > > > >> > > > >> > > > > or
> >> > > > >> > > > >> > > > > > > the
> >> > > > >> > > > >> > > > > > > >> > > > > > partition_epoch. The current
> >> KIP
> >> > > > >> describes
> >> > > > >> > > the
> >> > > > >> > > > >> > changes
> >> > > > >> > > > >> > > > to
> >> > > > >> > > > >> > > > > > the
> >> > > > >> > > > >> > > > > > > >> > > consumer
> >> > > > >> > > > >> > > > > > > >> > > > > API
> >> > > > >> > > > >> > > > > > > >> > > > > > and how the new API can be
> >> used if
> >> > > > user
> >> > > > >> > > stores
> >> > > > >> > > > >> > offset
> >> > > > >> > > > >> > > > > > > >> externally.
> >> > > > >> > > > >> > > > > > > >> > In
> >> > > > >> > > > >> > > > > > > >> > > > > order
> >> > > > >> > > > >> > > > > > > >> > > > > > to address the scenario you
> >> > > described
> >> > > > >> > > earlier,
> >> > > > >> > > > we
> >> > > > >> > > > >> > can
> >> > > > >> > > > >> > > > > > include
> >> > > > >> > > > >> > > > > > > >> > > > > > metadata_epoch in the
> >> > FetchResponse
> >> > > > and
> >> > > > >> the
> >> > > > >> > > > >> > > > > > > LeaderAndIsrRequest.
> >> > > > >> > > > >> > > > > > > >> > > > Consumer
> >> > > > >> > > > >> > > > > > > >> > > > > > remembers the largest
> >> > metadata_epoch
> >> > > > >> from
> >> > > > >> > all
> >> > > > >> > > > the
> >> > > > >> > > > >> > > > > > > FetchResponse
> >> > > > >> > > > >> > > > > > > >> it
> >> > > > >> > > > >> > > > > > > >> > > has
> >> > > > >> > > > >> > > > > > > >> > > > > > received. The metadata_epoch
> >> > > committed
> >> > > > >> with
> >> > > > >> > > the
> >> > > > >> > > > >> > > offset,
> >> > > > >> > > > >> > > > > > either
> >> > > > >> > > > >> > > > > > > >> > within
> >> > > > >> > > > >> > > > > > > >> > > > or
> >> > > > >> > > > >> > > > > > > >> > > > > > outside Kafka, should be the
> >> > largest
> >> > > > >> > > > >> metadata_epoch
> >> > > > >> > > > >> > > > across
> >> > > > >> > > > >> > > > > > all
> >> > > > >> > > > >> > > > > > > >> > > > > > FetchResponse and
> >> MetadataResponse
> >> > > > ever
> >> > > > >> > > > received
> >> > > > >> > > > >> by
> >> > > > >> > > > >> > > this
> >> > > > >> > > > >> > > > > > > >> consumer.
> >> > > > >> > > > >> > > > > > > >> > > > > >
> >> > > > >> > > > >> > > > > > > >> > > > > > The drawback of using only
> the
> >> > > > >> > metadata_epoch
> >> > > > >> > > > is
> >> > > > >> > > > >> > that
> >> > > > >> > > > >> > > we
> >> > > > >> > > > >> > > > > can
> >> > > > >> > > > >> > > > > > > not
> >> > > > >> > > > >> > > > > > > >> > > always
> >> > > > >> > > > >> > > > > > > >> > > > > do
> >> > > > >> > > > >> > > > > > > >> > > > > > the smart offset reset in
> case
> >> of
> >> > > > >> unclean
> >> > > > >> > > > leader
> >> > > > >> > > > >> > > > election
> >> > > > >> > > > >> > > > > > > which
> >> > > > >> > > > >> > > > > > > >> you
> >> > > > >> > > > >> > > > > > > >> > > > > > mentioned earlier. But in
> most
> >> > case,
> >> > > > >> > unclean
> >> > > > >> > > > >> leader
> >> > > > >> > > > >> > > > > election
> >> > > > >> > > > >> > > > > > > >> > probably
> >> > > > >> > > > >> > > > > > > >> > > > > > happens when consumer is not
> >> > > > >> > > > >> rebalancing/restarting.
> >> > > > >> > > > >> > > In
> >> > > > >> > > > >> > > > > > these
> >> > > > >> > > > >> > > > > > > >> > cases,
> >> > > > >> > > > >> > > > > > > >> > > > > either
> >> > > > >> > > > >> > > > > > > >> > > > > > consumer is not directly
> >> affected
> >> > by
> >> > > > >> > unclean
> >> > > > >> > > > >> leader
> >> > > > >> > > > >> > > > > election
> >> > > > >> > > > >> > > > > > > >> since
> >> > > > >> > > > >> > > > > > > >> > it
> >> > > > >> > > > >> > > > > > > >> > > > is
> >> > > > >> > > > >> > > > > > > >> > > > > > not consuming from the end of
> >> the
> >> > > log,
> >> > > > >> or
> >> > > > >> > > > >> consumer
> >> > > > >> > > > >> > can
> >> > > > >> > > > >> > > > > > derive
> >> > > > >> > > > >> > > > > > > >> the
> >> > > > >> > > > >> > > > > > > >> > > > > > leader_epoch from the most
> >> recent
> >> > > > >> message
> >> > > > >> > > > >> received
> >> > > > >> > > > >> > > > before
> >> > > > >> > > > >> > > > > it
> >> > > > >> > > > >> > > > > > > >> sees
> >> > > > >> > > > >> > > > > > > >> > > > > > OffsetOutOfRangeException. So
> >> I am
> >> > > not
> >> > > > >> sure
> >> > > > >> > > it
> >> > > > >> > > > is
> >> > > > >> > > > >> > > worth
> >> > > > >> > > > >> > > > > > adding
> >> > > > >> > > > >> > > > > > > >> the
> >> > > > >> > > > >> > > > > > > >> > > > > > leader_epoch to consumer API
> to
> >> > > > address
> >> > > > >> the
> >> > > > >> > > > >> > remaining
> >> > > > >> > > > >> > > > > corner
> >> > > > >> > > > >> > > > > > > >> case.
> >> > > > >> > > > >> > > > > > > >> > > What
> >> > > > >> > > > >> > > > > > > >> > > > > do
> >> > > > >> > > > >> > > > > > > >> > > > > > you think?
> >> > > > >> > > > >> > > > > > > >> > > > > >
> >> > > > >> > > > >> > > > > > > >> > > > > > Thanks,
> >> > > > >> > > > >> > > > > > > >> > > > > > Dong
> >> > > > >> > > > >> > > > > > > >> > > > > >
> >> > > > >> > > > >> > > > > > > >> > > > > >
> >> > > > >> > > > >> > > > > > > >> > > > > >
> >> > > > >> > > > >> > > > > > > >> > > > > > On Tue, Jan 2, 2018 at 6:28
> PM,
> >> > Jun
> >> > > > Rao
> >> > > > >> <
> >> > > > >> > > > >> > > > jun@confluent.io
> >> > > > >> > > > >> > > > > >
> >> > > > >> > > > >> > > > > > > >> wrote:
> >> > > > >> > > > >> > > > > > > >> > > > > >
> >> > > > >> > > > >> > > > > > > >> > > > > > > Hi, Dong,
> >> > > > >> > > > >> > > > > > > >> > > > > > >
> >> > > > >> > > > >> > > > > > > >> > > > > > > Thanks for the reply.
> >> > > > >> > > > >> > > > > > > >> > > > > > >
> >> > > > >> > > > >> > > > > > > >> > > > > > > To solve the topic
> recreation
> >> > > issue,
> >> > > > >> we
> >> > > > >> > > could
> >> > > > >> > > > >> use
> >> > > > >> > > > >> > > > > either a
> >> > > > >> > > > >> > > > > > > >> global
> >> > > > >> > > > >> > > > > > > >> > > > > > metadata
> >> > > > >> > > > >> > > > > > > >> > > > > > > version or a partition
> level
> >> > > epoch.
> >> > > > >> But
> >> > > > >> > > > either
> >> > > > >> > > > >> one
> >> > > > >> > > > >> > > > will
> >> > > > >> > > > >> > > > > > be a
> >> > > > >> > > > >> > > > > > > >> new
> >> > > > >> > > > >> > > > > > > >> > > > > concept,
> >> > > > >> > > > >> > > > > > > >> > > > > > > right? To me, the latter
> >> seems
> >> > > more
> >> > > > >> > > natural.
> >> > > > >> > > > It
> >> > > > >> > > > >> > also
> >> > > > >> > > > >> > > > > makes
> >> > > > >> > > > >> > > > > > > it
> >> > > > >> > > > >> > > > > > > >> > > easier
> >> > > > >> > > > >> > > > > > > >> > > > to
> >> > > > >> > > > >> > > > > > > >> > > > > > > detect if a consumer's
> >> offset is
> >> > > > still
> >> > > > >> > > valid
> >> > > > >> > > > >> > after a
> >> > > > >> > > > >> > > > > topic
> >> > > > >> > > > >> > > > > > > is
> >> > > > >> > > > >> > > > > > > >> > > > > recreated.
> >> > > > >> > > > >> > > > > > > >> > > > > > As
> >> > > > >> > > > >> > > > > > > >> > > > > > > you pointed out, we don't
> >> need
> >> > to
> >> > > > >> store
> >> > > > >> > the
> >> > > > >> > > > >> > > partition
> >> > > > >> > > > >> > > > > > epoch
> >> > > > >> > > > >> > > > > > > in
> >> > > > >> > > > >> > > > > > > >> > the
> >> > > > >> > > > >> > > > > > > >> > > > > > message.
> >> > > > >> > > > >> > > > > > > >> > > > > > > The following is what I am
> >> > > thinking.
> >> > > > >> > When a
> >> > > > >> > > > >> > > partition
> >> > > > >> > > > >> > > > is
> >> > > > >> > > > >> > > > > > > >> created,
> >> > > > >> > > > >> > > > > > > >> > > we
> >> > > > >> > > > >> > > > > > > >> > > > > can
> >> > > > >> > > > >> > > > > > > >> > > > > > > assign a partition epoch
> >> from an
> >> > > > >> > > > >> ever-increasing
> >> > > > >> > > > >> > > > global
> >> > > > >> > > > >> > > > > > > >> counter
> >> > > > >> > > > >> > > > > > > >> > and
> >> > > > >> > > > >> > > > > > > >> > > > > store
> >> > > > >> > > > >> > > > > > > >> > > > > > > it in
> >> /brokers/topics/[topic]/
> >> > > > >> > > > >> > > > partitions/[partitionId]
> >> > > > >> > > > >> > > > > in
> >> > > > >> > > > >> > > > > > > ZK.
> >> > > > >> > > > >> > > > > > > >> > The
> >> > > > >> > > > >> > > > > > > >> > > > > > > partition
> >> > > > >> > > > >> > > > > > > >> > > > > > > epoch is propagated to
> every
> >> > > broker.
> >> > > > >> The
> >> > > > >> > > > >> consumer
> >> > > > >> > > > >> > > will
> >> > > > >> > > > >> > > > > be
> >> > > > >> > > > >> > > > > > > >> > tracking
> >> > > > >> > > > >> > > > > > > >> > > a
> >> > > > >> > > > >> > > > > > > >> > > > > > tuple
> >> > > > >> > > > >> > > > > > > >> > > > > > > of <offset, leader epoch,
> >> > > partition
> >> > > > >> > epoch>
> >> > > > >> > > > for
> >> > > > >> > > > >> > > > offsets.
> >> > > > >> > > > >> > > > > > If a
> >> > > > >> > > > >> > > > > > > >> > topic
> >> > > > >> > > > >> > > > > > > >> > > is
> >> > > > >> > > > >> > > > > > > >> > > > > > > recreated, it's possible
> >> that a
> >> > > > >> > consumer's
> >> > > > >> > > > >> offset
> >> > > > >> > > > >> > > and
> >> > > > >> > > > >> > > > > > leader
> >> > > > >> > > > >> > > > > > > >> > epoch
> >> > > > >> > > > >> > > > > > > >> > > > > still
> >> > > > >> > > > >> > > > > > > >> > > > > > > match that in the broker,
> but
> >> > > > >> partition
> >> > > > >> > > epoch
> >> > > > >> > > > >> > won't
> >> > > > >> > > > >> > > > be.
> >> > > > >> > > > >> > > > > In
> >> > > > >> > > > >> > > > > > > >> this
> >> > > > >> > > > >> > > > > > > >> > > case,
> >> > > > >> > > > >> > > > > > > >> > > > > we
> >> > > > >> > > > >> > > > > > > >> > > > > > > can potentially still treat
> >> the
> >> > > > >> > consumer's
> >> > > > >> > > > >> offset
> >> > > > >> > > > >> > as
> >> > > > >> > > > >> > > > out
> >> > > > >> > > > >> > > > > > of
> >> > > > >> > > > >> > > > > > > >> range
> >> > > > >> > > > >> > > > > > > >> > > and
> >> > > > >> > > > >> > > > > > > >> > > > > > reset
> >> > > > >> > > > >> > > > > > > >> > > > > > > the offset based on the
> >> offset
> >> > > reset
> >> > > > >> > policy
> >> > > > >> > > > in
> >> > > > >> > > > >> the
> >> > > > >> > > > >> > > > > > consumer.
> >> > > > >> > > > >> > > > > > > >> This
> >> > > > >> > > > >> > > > > > > >> > > > seems
> >> > > > >> > > > >> > > > > > > >> > > > > > > harder to do with a global
> >> > > metadata
> >> > > > >> > > version.
> >> > > > >> > > > >> > > > > > > >> > > > > > >
> >> > > > >> > > > >> > > > > > > >> > > > > > > Jun
> >> > > > >> > > > >> > > > > > > >> > > > > > >
> >> > > > >> > > > >> > > > > > > >> > > > > > >
> >> > > > >> > > > >> > > > > > > >> > > > > > >
> >> > > > >> > > > >> > > > > > > >> > > > > > > On Mon, Dec 25, 2017 at
> 6:56
> >> AM,
> >> > > > Dong
> >> > > > >> > Lin <
> >> > > > >> > > > >> > > > > > > >> lindong28@gmail.com>
> >> > > > >> > > > >> > > > > > > >> > > > wrote:
> >> > > > >> > > > >> > > > > > > >> > > > > > >
> >> > > > >> > > > >> > > > > > > >> > > > > > > > Hey Jun,
> >> > > > >> > > > >> > > > > > > >> > > > > > > >
> >> > > > >> > > > >> > > > > > > >> > > > > > > > This is a very good
> >> example.
> >> > > After
> >> > > > >> > > thinking
> >> > > > >> > > > >> > > through
> >> > > > >> > > > >> > > > > this
> >> > > > >> > > > >> > > > > > > in
> >> > > > >> > > > >> > > > > > > >> > > > detail, I
> >> > > > >> > > > >> > > > > > > >> > > > > > > agree
> >> > > > >> > > > >> > > > > > > >> > > > > > > > that we need to commit
> >> offset
> >> > > with
> >> > > > >> > leader
> >> > > > >> > > > >> epoch
> >> > > > >> > > > >> > in
> >> > > > >> > > > >> > > > > order
> >> > > > >> > > > >> > > > > > > to
> >> > > > >> > > > >> > > > > > > >> > > address
> >> > > > >> > > > >> > > > > > > >> > > > > > this
> >> > > > >> > > > >> > > > > > > >> > > > > > > > example.
> >> > > > >> > > > >> > > > > > > >> > > > > > > >
> >> > > > >> > > > >> > > > > > > >> > > > > > > > I think the remaining
> >> question
> >> > > is
> >> > > > >> how
> >> > > > >> > to
> >> > > > >> > > > >> address
> >> > > > >> > > > >> > > the
> >> > > > >> > > > >> > > > > > > >> scenario
> >> > > > >> > > > >> > > > > > > >> > > that
> >> > > > >> > > > >> > > > > > > >> > > > > the
> >> > > > >> > > > >> > > > > > > >> > > > > > > > topic is deleted and
> >> > re-created.
> >> > > > One
> >> > > > >> > > > possible
> >> > > > >> > > > >> > > > solution
> >> > > > >> > > > >> > > > > > is
> >> > > > >> > > > >> > > > > > > to
> >> > > > >> > > > >> > > > > > > >> > > commit
> >> > > > >> > > > >> > > > > > > >> > > > > > > offset
> >> > > > >> > > > >> > > > > > > >> > > > > > > > with both the leader
> epoch
> >> and
> >> > > the
> >> > > > >> > > metadata
> >> > > > >> > > > >> > > version.
> >> > > > >> > > > >> > > > > The
> >> > > > >> > > > >> > > > > > > >> logic
> >> > > > >> > > > >> > > > > > > >> > > and
> >> > > > >> > > > >> > > > > > > >> > > > > the
> >> > > > >> > > > >> > > > > > > >> > > > > > > > implementation of this
> >> > solution
> >> > > > does
> >> > > > >> > not
> >> > > > >> > > > >> > require a
> >> > > > >> > > > >> > > > new
> >> > > > >> > > > >> > > > > > > >> concept
> >> > > > >> > > > >> > > > > > > >> > > > (e.g.
> >> > > > >> > > > >> > > > > > > >> > > > > > > > partition epoch) and it
> >> does
> >> > not
> >> > > > >> > require
> >> > > > >> > > > any
> >> > > > >> > > > >> > > change
> >> > > > >> > > > >> > > > to
> >> > > > >> > > > >> > > > > > the
> >> > > > >> > > > >> > > > > > > >> > > message
> >> > > > >> > > > >> > > > > > > >> > > > > > format
> >> > > > >> > > > >> > > > > > > >> > > > > > > > or leader epoch. It also
> >> > allows
> >> > > us
> >> > > > >> to
> >> > > > >> > > order
> >> > > > >> > > > >> the
> >> > > > >> > > > >> > > > > metadata
> >> > > > >> > > > >> > > > > > > in
> >> > > > >> > > > >> > > > > > > >> a
> >> > > > >> > > > >> > > > > > > >> > > > > > > > straightforward manner
> >> which
> >> > may
> >> > > > be
> >> > > > >> > > useful
> >> > > > >> > > > in
> >> > > > >> > > > >> > the
> >> > > > >> > > > >> > > > > > future.
> >> > > > >> > > > >> > > > > > > >> So it
> >> > > > >> > > > >> > > > > > > >> > > may
> >> > > > >> > > > >> > > > > > > >> > > > > be
> >> > > > >> > > > >> > > > > > > >> > > > > > a
> >> > > > >> > > > >> > > > > > > >> > > > > > > > better solution than
> >> > generating
> >> > > a
> >> > > > >> > random
> >> > > > >> > > > >> > partition
> >> > > > >> > > > >> > > > > epoch
> >> > > > >> > > > >> > > > > > > >> every
> >> > > > >> > > > >> > > > > > > >> > > time
> >> > > > >> > > > >> > > > > > > >> > > > > we
> >> > > > >> > > > >> > > > > > > >> > > > > > > > create a partition. Does
> >> this
> >> > > > sound
> >> > > > >> > > > >> reasonable?
> >> > > > >> > > > >> > > > > > > >> > > > > > > >
> >> > > > >> > > > >> > > > > > > >> > > > > > > > Previously one concern
> with
> >> > > using
> >> > > > >> the
> >> > > > >> > > > >> metadata
> >> > > > >> > > > >> > > > version
> >> > > > >> > > > >> > > > > > is
> >> > > > >> > > > >> > > > > > > >> that
> >> > > > >> > > > >> > > > > > > >> > > > > consumer
> >> > > > >> > > > >> > > > > > > >> > > > > > > > will be forced to refresh
> >> > > metadata
> >> > > > >> even
> >> > > > >> > > if
> >> > > > >> > > > >> > > metadata
> >> > > > >> > > > >> > > > > > > version
> >> > > > >> > > > >> > > > > > > >> is
> >> > > > >> > > > >> > > > > > > >> > > > > > increased
> >> > > > >> > > > >> > > > > > > >> > > > > > > > due to topics that the
> >> > consumer
> >> > > is
> >> > > > >> not
> >> > > > >> > > > >> > interested
> >> > > > >> > > > >> > > > in.
> >> > > > >> > > > >> > > > > > Now
> >> > > > >> > > > >> > > > > > > I
> >> > > > >> > > > >> > > > > > > >> > > > realized
> >> > > > >> > > > >> > > > > > > >> > > > > > that
> >> > > > >> > > > >> > > > > > > >> > > > > > > > this is probably not a
> >> > problem.
> >> > > > >> > Currently
> >> > > > >> > > > >> client
> >> > > > >> > > > >> > > > will
> >> > > > >> > > > >> > > > > > > >> refresh
> >> > > > >> > > > >> > > > > > > >> > > > > metadata
> >> > > > >> > > > >> > > > > > > >> > > > > > > > either due to
> >> > > > >> InvalidMetadataException
> >> > > > >> > in
> >> > > > >> > > > the
> >> > > > >> > > > >> > > > response
> >> > > > >> > > > >> > > > > > > from
> >> > > > >> > > > >> > > > > > > >> > > broker
> >> > > > >> > > > >> > > > > > > >> > > > or
> >> > > > >> > > > >> > > > > > > >> > > > > > due
> >> > > > >> > > > >> > > > > > > >> > > > > > > > to metadata expiry. The
> >> > addition
> >> > > > of
> >> > > > >> the
> >> > > > >> > > > >> metadata
> >> > > > >> > > > >> > > > > version
> >> > > > >> > > > >> > > > > > > >> should
> >> > > > >> > > > >> > > > > > > >> > > > > > increase
> >> > > > >> > > > >> > > > > > > >> > > > > > > > the overhead of metadata
> >> > refresh
> >> > > > >> caused
> >> > > > >> > > by
> >> > > > >> > > > >> > > > > > > >> > > > InvalidMetadataException.
> >> > > > >> > > > >> > > > > > > >> > > > > If
> >> > > > >> > > > >> > > > > > > >> > > > > > > > client refresh metadata
> >> due to
> >> > > > >> expiry
> >> > > > >> > and
> >> > > > >> > > > it
> >> > > > >> > > > >> > > > receives
> >> > > > >> > > > >> > > > > a
> >> > > > >> > > > >> > > > > > > >> > metadata
> >> > > > >> > > > >> > > > > > > >> > > > > whose
> >> > > > >> > > > >> > > > > > > >> > > > > > > > version is lower than the
> >> > > current
> >> > > > >> > > metadata
> >> > > > >> > > > >> > > version,
> >> > > > >> > > > >> > > > we
> >> > > > >> > > > >> > > > > > can
> >> > > > >> > > > >> > > > > > > >> > reject
> >> > > > >> > > > >> > > > > > > >> > > > the
> >> > > > >> > > > >> > > > > > > >> > > > > > > > metadata but still reset
> >> the
> >> > > > >> metadata
> >> > > > >> > > age,
> >> > > > >> > > > >> which
> >> > > > >> > > > >> > > > > > > essentially
> >> > > > >> > > > >> > > > > > > >> > keep
> >> > > > >> > > > >> > > > > > > >> > > > the
> >> > > > >> > > > >> > > > > > > >> > > > > > > > existing behavior in the
> >> > client.
> >> > > > >> > > > >> > > > > > > >> > > > > > > >
> >> > > > >> > > > >> > > > > > > >> > > > > > > > Thanks much,
> >> > > > >> > > > >> > > > > > > >> > > > > > > > Dong
> >> > > > >> > > > >> > > > > > > >> > > > > > > >
> >> > > > >> > > > >> > > > > > > >> > > > > > >
> >> > > > >> > > > >> > > > > > > >> > > > > >
> >> > > > >> > > > >> > > > > > > >> > > > >
> >> > > > >> > > > >> > > > > > > >> > > >
> >> > > > >> > > > >> > > > > > > >> > >
> >> > > > >> > > > >> > > > > > > >> >
> >> > > > >> > > > >> > > > > > > >>
> >> > > > >> > > > >> > > > > > > >
> >> > > > >> > > > >> > > > > > > >
> >> > > > >> > > > >> > > > > > >
> >> > > > >> > > > >> > > > > >
> >> > > > >> > > > >> > > > >
> >> > > > >> > > > >> > > >
> >> > > > >> > > > >> > >
> >> > > > >> > > > >> >
> >> > > > >> > > > >>
> >> > > > >> > > > >
> >> > > > >> > > > >
> >> > > > >> > > >
> >> > > > >> > >
> >> > > > >> >
> >> > > > >>
> >> > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Dong Lin <li...@gmail.com>.
Hey Jun,

Do you think the current KIP looks OK? I am wondering if we can open the
voting thread.

Thanks!
Dong

On Fri, Jan 19, 2018 at 3:08 PM, Dong Lin <li...@gmail.com> wrote:

> Hey Jun,
>
> I think we can probably have a static method in Util class to decode the
> byte[]. Both KafkaConsumer implementation and the user application will be
> able to decode the byte array and log its content for debug purpose. So it
> seems that we can still print the information we want. It is just not
> explicitly exposed in the consumer interface. Would this address the
> problem here?
>
> Yeah we can include OffsetEpoch in AdminClient. This can be added in
> KIP-222? Is there something you would like me to add in this KIP?
>
> Thanks!
> Dong
>
> On Fri, Jan 19, 2018 at 3:00 PM, Jun Rao <ju...@confluent.io> wrote:
>
>> Hi, Dong,
>>
>> The issue with using just byte[] for OffsetEpoch is that it won't be
>> printable, which makes debugging harder.
>>
>> Also, KIP-222 proposes a listGroupOffset() method in AdminClient. If that
>> gets adopted before this KIP, we probably want to include OffsetEpoch in
>> the AdminClient too.
>>
>> Thanks,
>>
>> Jun
>>
>>
>> On Thu, Jan 18, 2018 at 6:30 PM, Dong Lin <li...@gmail.com> wrote:
>>
>> > Hey Jun,
>> >
>> > I agree. I have updated the KIP to remove the class OffetEpoch and
>> replace
>> > OffsetEpoch with byte[] in APIs that use it. Can you see if it looks
>> good?
>> >
>> > Thanks!
>> > Dong
>> >
>> > On Thu, Jan 18, 2018 at 6:07 PM, Jun Rao <ju...@confluent.io> wrote:
>> >
>> > > Hi, Dong,
>> > >
>> > > Thanks for the updated KIP. It looks good to me now. The only thing is
>> > > for OffsetEpoch.
>> > > If we expose the individual fields in the class, we probably don't
>> need
>> > the
>> > > encode/decode methods. If we want to hide the details of OffsetEpoch,
>> we
>> > > probably don't want expose the individual fields.
>> > >
>> > > Jun
>> > >
>> > > On Wed, Jan 17, 2018 at 10:10 AM, Dong Lin <li...@gmail.com>
>> wrote:
>> > >
>> > > > Thinking about point 61 more, I realize that the async zookeeper
>> read
>> > may
>> > > > make it less of an issue for controller to read more zookeeper
>> nodes.
>> > > > Writing partition_epoch in the per-partition znode makes it simpler
>> to
>> > > > handle the broker failure between zookeeper writes for a topic
>> > creation.
>> > > I
>> > > > have updated the KIP to use the suggested approach.
>> > > >
>> > > >
>> > > > On Wed, Jan 17, 2018 at 9:57 AM, Dong Lin <li...@gmail.com>
>> wrote:
>> > > >
>> > > > > Hey Jun,
>> > > > >
>> > > > > Thanks much for the comments. Please see my comments inline.
>> > > > >
>> > > > > On Tue, Jan 16, 2018 at 4:38 PM, Jun Rao <ju...@confluent.io>
>> wrote:
>> > > > >
>> > > > >> Hi, Dong,
>> > > > >>
>> > > > >> Thanks for the updated KIP. Looks good to me overall. Just a few
>> > minor
>> > > > >> comments.
>> > > > >>
>> > > > >> 60. OffsetAndMetadata positionAndOffsetEpoch(TopicPartition
>> > > partition):
>> > > > >> It
>> > > > >> seems that there is no need to return metadata. We probably want
>> to
>> > > > return
>> > > > >> sth like OffsetAndEpoch.
>> > > > >>
>> > > > >
>> > > > > Previously I think we may want to re-use the existing class to
>> keep
>> > our
>> > > > > consumer interface simpler. I have updated the KIP to add class
>> > > > > OffsetAndOffsetEpoch. I didn't use OffsetAndEpoch because user may
>> > > > confuse
>> > > > > this name with OffsetEpoch. Does this sound OK?
>> > > > >
>> > > > >
>> > > > >>
>> > > > >> 61. Should we store partition_epoch in
>> > > > >> /brokers/topics/[topic]/partitions/[partitionId] in ZK?
>> > > > >>
>> > > > >
>> > > > > I have considered this. I think the advantage of adding the
>> > > > > partition->partition_epoch map in the existing
>> > > > > znode /brokers/topics/[topic]/partitions is that controller only
>> > needs
>> > > > to
>> > > > > read one znode per topic to gets its partition_epoch information.
>> > > > Otherwise
>> > > > > controller may need to read one extra znode per partition to get
>> the
>> > > same
>> > > > > information.
>> > > > >
>> > > > > When we delete partition or expand partition of a topic, someone
>> > needs
>> > > to
>> > > > > modify partition->partition_epoch map in znode
>> > > > > /brokers/topics/[topic]/partitions. This may seem a bit more
>> > > complicated
>> > > > > than simply adding or deleting znode /brokers/topics/[topic]/
>> > > > partitions/[partitionId].
>> > > > > But the complexity is probably similar to the existing operation
>> of
>> > > > > modifying the partition->replica_list mapping in znode
>> > > > > /brokers/topics/[topic]. So not sure it is better to store the
>> > > > > partition_epoch in /brokers/topics/[topic]/partit
>> ions/[partitionId].
>> > > > What
>> > > > > do you think?
>> > > > >
>> > > > >
>> > > > >>
>> > > > >> 62. For checking outdated metadata in the client, we probably
>> want
>> > to
>> > > > add
>> > > > >> when max_partition_epoch will be used.
>> > > > >>
>> > > > >
>> > > > > The max_partition_epoch is used in the Proposed Changes ->
>> Client's
>> > > > > metadata refresh section to determine whether a metadata is
>> outdated.
>> > > And
>> > > > > this formula is referenced and re-used in other sections to
>> determine
>> > > > > whether a metadata is outdated. Does this formula look OK?
>> > > > >
>> > > > >
>> > > > >>
>> > > > >> 63. "The leader_epoch should be the largest leader_epoch of
>> messages
>> > > > whose
>> > > > >> offset < the commit offset. If no message has been consumed since
>> > > > consumer
>> > > > >> initialization, the leader_epoch from seek(...) or
>> > OffsetFetchResponse
>> > > > >> should be used. The partition_epoch should be read from the last
>> > > > >> FetchResponse corresponding to the given partition and commit
>> > offset.
>> > > ":
>> > > > >> leader_epoch and partition_epoch are associated with an offset.
>> So,
>> > if
>> > > > no
>> > > > >> message is consumed, there is no offset and therefore there is no
>> > need
>> > > > to
>> > > > >> read leader_epoch and partition_epoch. Also, the leader_epoch
>> > > associated
>> > > > >> with the offset should just come from the messages returned in
>> the
>> > > fetch
>> > > > >> response.
>> > > > >>
>> > > > >
>> > > > > I am thinking that, if user calls seek(..) and commitSync(...)
>> > without
>> > > > > consuming any messages, we should re-use the leader_epoch and
>> > > > > partition_epoch provided by the seek(...) in the
>> OffsetCommitRequest.
>> > > And
>> > > > > if messages have been successfully consumed, then leader_epoch
>> will
>> > > come
>> > > > > from the messages returned in the fetch response. The condition
>> > > "messages
>> > > > > whose offset < the commit offset" is needed to take care of the
>> log
>> > > > > compacted topic which may have offset gap due to log cleaning.
>> > > > >
>> > > > > Did I miss something here? Or should I rephrase the paragraph to
>> make
>> > > it
>> > > > > less confusing?
>> > > > >
>> > > > >
>> > > > >> 64. Could you include the public methods in the OffsetEpoch
>> class?
>> > > > >>
>> > > > >
>> > > > > I mistakenly deleted the definition of OffsetEpoch class from the
>> > KIP.
>> > > I
>> > > > > just added it back with the public methods. Could you take another
>> > > look?
>> > > > >
>> > > > >
>> > > > >>
>> > > > >> Jun
>> > > > >>
>> > > > >>
>> > > > >> On Thu, Jan 11, 2018 at 5:43 PM, Dong Lin <li...@gmail.com>
>> > > wrote:
>> > > > >>
>> > > > >> > Hey Jun,
>> > > > >> >
>> > > > >> > Thanks much. I agree that we can not rely on committed offsets
>> to
>> > be
>> > > > >> always
>> > > > >> > deleted when we delete topic. So it is necessary to use a
>> > > > per-partition
>> > > > >> > epoch that does not change unless this partition is deleted. I
>> > also
>> > > > >> agree
>> > > > >> > that it is very nice to be able to uniquely identify a message
>> > with
>> > > > >> > (offset, leader_epoch, partition_epoch) in face of potential
>> topic
>> > > > >> deletion
>> > > > >> > and unclean leader election.
>> > > > >> >
>> > > > >> > I agree with all your comments. And I have updated the KIP
>> based
>> > on
>> > > > our
>> > > > >> > latest discussion. In addition, I added
>> > > InvalidPartitionEpochException
>> > > > >> > which will be thrown by consumer.poll() if the partition_epoch
>> > > > >> associated
>> > > > >> > with the partition, which can be given to consumer using
>> > seek(...),
>> > > is
>> > > > >> > different from the partition_epoch in the FetchResponse.
>> > > > >> >
>> > > > >> > Can you take another look at the latest KIP?
>> > > > >> >
>> > > > >> > Thanks!
>> > > > >> > Dong
>> > > > >> >
>> > > > >> >
>> > > > >> >
>> > > > >> > On Wed, Jan 10, 2018 at 2:24 PM, Jun Rao <ju...@confluent.io>
>> > wrote:
>> > > > >> >
>> > > > >> > > Hi, Dong,
>> > > > >> > >
>> > > > >> > > My replies are the following.
>> > > > >> > >
>> > > > >> > > 60. What you described could also work. The drawback is that
>> we
>> > > will
>> > > > >> be
>> > > > >> > > unnecessarily changing the partition epoch when a partition
>> > hasn't
>> > > > >> really
>> > > > >> > > changed. I was imagining that the partition epoch will be
>> stored
>> > > in
>> > > > >> > > /brokers/topics/[topic]/partitions/[partitionId], instead
>> of at
>> > > the
>> > > > >> > topic
>> > > > >> > > level. So, not sure if ZK size limit is an issue.
>> > > > >> > >
>> > > > >> > > 61, 62 and 65. To me, the offset + offset_epoch is a unique
>> > > > identifier
>> > > > >> > for
>> > > > >> > > a message. So, if a message hasn't changed, the offset and
>> the
>> > > > >> associated
>> > > > >> > > offset_epoch ideally should remain the same (it will be kind
>> of
>> > > > weird
>> > > > >> if
>> > > > >> > > two consumer apps save the offset on the same message, but
>> the
>> > > > >> > offset_epoch
>> > > > >> > > are different). partition_epoch + leader_epoch give us that.
>> > > > >> > global_epoch +
>> > > > >> > > leader_epoch don't. If we use this approach, we can solve not
>> > only
>> > > > the
>> > > > >> > > problem that you have identified, but also other problems
>> when
>> > > there
>> > > > >> is
>> > > > >> > > data loss or topic re-creation more reliably. For example, in
>> > the
>> > > > >> future,
>> > > > >> > > if we include the partition_epoch and leader_epoch in the
>> fetch
>> > > > >> request,
>> > > > >> > > the server can do a more reliable check of whether that
>> offset
>> > is
>> > > > >> valid
>> > > > >> > or
>> > > > >> > > not. I am not sure that we can rely upon all external
>> offsets to
>> > > be
>> > > > >> > removed
>> > > > >> > > on topic deletion. For example, a topic may be deleted by an
>> > admin
>> > > > who
>> > > > >> > may
>> > > > >> > > not know all the applications.
>> > > > >> > >
>> > > > >> > > If we agree on the above, the second question is then how to
>> > > > reliably
>> > > > >> > > propagate the partition_epoch and the leader_epoch to the
>> > consumer
>> > > > >> when
>> > > > >> > > there are leader or partition changes. The leader_epoch comes
>> > from
>> > > > the
>> > > > >> > > message, which is reliable. So, I was suggesting that when we
>> > > store
>> > > > an
>> > > > >> > > offset, we can just store the leader_epoch from the message
>> set
>> > > > >> > containing
>> > > > >> > > that offset. Similarly, I was thinking that if the
>> > partition_epoch
>> > > > is
>> > > > >> in
>> > > > >> > > the fetch response, we can propagate partition_epoch reliably
>> > > where
>> > > > is
>> > > > >> > > partition_epoch change.
>> > > > >> > >
>> > > > >> > > 63. My point is that once a leader is producing a message in
>> the
>> > > new
>> > > > >> > > partition_epoch, ideally, we should associate the new offsets
>> > with
>> > > > the
>> > > > >> > new
>> > > > >> > > partition_epoch. Otherwise, the offset_epoch won't be the
>> > correct
>> > > > >> unique
>> > > > >> > > identifier (useful for solving other problems mentioned
>> above).
>> > I
>> > > > was
>> > > > >> > > originally thinking that the leader will include the
>> > > partition_epoch
>> > > > >> in
>> > > > >> > the
>> > > > >> > > metadata cache in the fetch response. It's just that right
>> now,
>> > > > >> metadata
>> > > > >> > > cache is updated on UpdateMetadataRequest, which typically
>> > happens
>> > > > >> after
>> > > > >> > > the LeaderAndIsrRequest. Another approach is for the leader
>> to
>> > > cache
>> > > > >> the
>> > > > >> > > partition_epoch in the Partition object and return that
>> (instead
>> > > of
>> > > > >> the
>> > > > >> > one
>> > > > >> > > in metadata cache) in the fetch response.
>> > > > >> > >
>> > > > >> > > 65. It seems to me that the global_epoch and the
>> partition_epoch
>> > > > have
>> > > > >> > > different purposes. A partition_epoch has the benefit that it
>> > (1)
>> > > > can
>> > > > >> be
>> > > > >> > > used to form a unique identifier for a message and (2) can be
>> > used
>> > > > to
>> > > > >> > > solve other
>> > > > >> > > corner case problems in the future. I am not sure having
>> just a
>> > > > >> > > global_epoch can achieve these. global_epoch is useful to
>> > > determine
>> > > > >> which
>> > > > >> > > version of the metadata is newer, especially with topic
>> > deletion.
>> > > > >> > >
>> > > > >> > > Thanks,
>> > > > >> > >
>> > > > >> > > Jun
>> > > > >> > >
>> > > > >> > > On Tue, Jan 9, 2018 at 11:34 PM, Dong Lin <
>> lindong28@gmail.com>
>> > > > >> wrote:
>> > > > >> > >
>> > > > >> > > > Regarding the use of the global epoch in 65), it is very
>> > similar
>> > > > to
>> > > > >> the
>> > > > >> > > > proposal of the metadata_epoch we discussed earlier. The
>> main
>> > > > >> > difference
>> > > > >> > > is
>> > > > >> > > > that this epoch is incremented when we create/expand/delete
>> > > topic
>> > > > >> and
>> > > > >> > > does
>> > > > >> > > > not change when controller re-send metadata.
>> > > > >> > > >
>> > > > >> > > > I looked at our previous discussion. It seems that we
>> prefer
>> > > > >> > > > partition_epoch over the metadata_epoch because 1) we
>> prefer
>> > not
>> > > > to
>> > > > >> > have
>> > > > >> > > an
>> > > > >> > > > ever growing metadata_epoch and 2) we can reset offset
>> better
>> > > when
>> > > > >> > topic
>> > > > >> > > is
>> > > > >> > > > re-created. The use of global topic_epoch avoids the
>> drawback
>> > of
>> > > > an
>> > > > >> > ever
>> > > > >> > > > quickly ever growing metadata_epoch. Though the global
>> epoch
>> > > does
>> > > > >> not
>> > > > >> > > allow
>> > > > >> > > > us to recognize the invalid offset committed before the
>> topic
>> > > > >> > > re-creation,
>> > > > >> > > > we can probably just delete the offset when we delete a
>> topic.
>> > > > Thus
>> > > > >> I
>> > > > >> > am
>> > > > >> > > > not very sure whether it is still worthwhile to have a
>> > > > per-partition
>> > > > >> > > > partition_epoch if the metadata already has the global
>> epoch.
>> > > > >> > > >
>> > > > >> > > >
>> > > > >> > > > On Tue, Jan 9, 2018 at 6:58 PM, Dong Lin <
>> lindong28@gmail.com
>> > >
>> > > > >> wrote:
>> > > > >> > > >
>> > > > >> > > > > Hey Jun,
>> > > > >> > > > >
>> > > > >> > > > > Thanks so much. These comments very useful. Please see
>> below
>> > > my
>> > > > >> > > comments.
>> > > > >> > > > >
>> > > > >> > > > > On Mon, Jan 8, 2018 at 5:52 PM, Jun Rao <
>> jun@confluent.io>
>> > > > wrote:
>> > > > >> > > > >
>> > > > >> > > > >> Hi, Dong,
>> > > > >> > > > >>
>> > > > >> > > > >> Thanks for the updated KIP. A few more comments.
>> > > > >> > > > >>
>> > > > >> > > > >> 60. Perhaps having a partition epoch is more flexible
>> since
>> > > in
>> > > > >> the
>> > > > >> > > > future,
>> > > > >> > > > >> we may support deleting a partition as well.
>> > > > >> > > > >>
>> > > > >> > > > >
>> > > > >> > > > > Yeah I have considered this. I think we can probably
>> still
>> > > > support
>> > > > >> > > > > deleting a partition by using the topic_epoch -- when
>> > > partition
>> > > > >> of a
>> > > > >> > > > topic
>> > > > >> > > > > is deleted or created, epoch of all partitions of this
>> topic
>> > > > will
>> > > > >> be
>> > > > >> > > > > incremented by 1. Therefore, if that partition is
>> re-created
>> > > > >> later,
>> > > > >> > the
>> > > > >> > > > > epoch of that partition will still be larger than its
>> epoch
>> > > > before
>> > > > >> > the
>> > > > >> > > > > deletion, which still allows the client to order the
>> > metadata
>> > > > for
>> > > > >> the
>> > > > >> > > > > purpose of this KIP. Does this sound reasonable?
>> > > > >> > > > >
>> > > > >> > > > > The advantage of using topic_epoch instead of
>> > partition_epoch
>> > > is
>> > > > >> that
>> > > > >> > > the
>> > > > >> > > > > size of the /brokers/topics/[topic] znode and
>> > request/response
>> > > > >> size
>> > > > >> > can
>> > > > >> > > > be
>> > > > >> > > > > smaller. We have a limit on the maximum size of znode
>> > > (typically
>> > > > >> > 1MB).
>> > > > >> > > > Use
>> > > > >> > > > > partition epoch can effectively reduce the number of
>> > > partitions
>> > > > >> that
>> > > > >> > > can
>> > > > >> > > > be
>> > > > >> > > > > described by the /brokers/topics/[topic] znode.
>> > > > >> > > > >
>> > > > >> > > > > One use-case of partition_epoch for client to detect that
>> > the
>> > > > >> > committed
>> > > > >> > > > > offset, either from kafka offset topic or from the
>> external
>> > > > store
>> > > > >> is
>> > > > >> > > > > invalid after partition deletion and re-creation.
>> However,
>> > it
>> > > > >> seems
>> > > > >> > > that
>> > > > >> > > > we
>> > > > >> > > > > can also address this use-case with other approaches. For
>> > > > example,
>> > > > >> > when
>> > > > >> > > > > AdminClient deletes partitions, it can also delete the
>> > > committed
>> > > > >> > > offsets
>> > > > >> > > > > for those partitions from the offset topic. If user
>> stores
>> > > > offset
>> > > > >> > > > > externally, it might make sense for user to similarly
>> remove
>> > > > >> offsets
>> > > > >> > of
>> > > > >> > > > > related partitions after these partitions are deleted.
>> So I
>> > am
>> > > > not
>> > > > >> > sure
>> > > > >> > > > > that we should use partition_epoch in this KIP.
>> > > > >> > > > >
>> > > > >> > > > >
>> > > > >> > > > >>
>> > > > >> > > > >> 61. It seems that the leader epoch returned in the
>> > position()
>> > > > >> call
>> > > > >> > > > should
>> > > > >> > > > >> the the leader epoch returned in the fetch response, not
>> > the
>> > > > one
>> > > > >> in
>> > > > >> > > the
>> > > > >> > > > >> metadata cache of the client.
>> > > > >> > > > >
>> > > > >> > > > >
>> > > > >> > > > > I think this is a good idea. Just to double check, this
>> > change
>> > > > >> does
>> > > > >> > not
>> > > > >> > > > > affect the correctness or performance of this KIP. But it
>> > can
>> > > be
>> > > > >> > useful
>> > > > >> > > > if
>> > > > >> > > > > we want to use the leader_epoch to better handle the
>> offset
>> > > rest
>> > > > >> in
>> > > > >> > > case
>> > > > >> > > > of
>> > > > >> > > > > unclean leader election, which is listed in the future
>> work.
>> > > Is
>> > > > >> this
>> > > > >> > > > > understanding correct?
>> > > > >> > > > >
>> > > > >> > > > > I have updated the KIP to specify that the leader_epoch
>> > > returned
>> > > > >> by
>> > > > >> > > > > position() should be the largest leader_epoch of those
>> > already
>> > > > >> > consumed
>> > > > >> > > > > messages whose offset < position. If no message has been
>> > > > consumed
>> > > > >> > since
>> > > > >> > > > > consumer initialization, the leader_epoch from seek() or
>> > > > >> > > > > OffsetFetchResponse should be used. The offset included
>> in
>> > the
>> > > > >> > > > > OffsetCommitRequest will also be determined in the
>> similar
>> > > > manner.
>> > > > >> > > > >
>> > > > >> > > > >
>> > > > >> > > > >>
>> > > > >> > > > >> 62. I am wondering if we should return the partition
>> epoch
>> > in
>> > > > the
>> > > > >> > > fetch
>> > > > >> > > > >> response as well. In the current proposal, if a topic is
>> > > > >> recreated
>> > > > >> > and
>> > > > >> > > > the
>> > > > >> > > > >> new leader is on the same broker as the old one, there
>> is
>> > > > >> nothing to
>> > > > >> > > > force
>> > > > >> > > > >> the metadata refresh in the client. So, the client may
>> > still
>> > > > >> > associate
>> > > > >> > > > the
>> > > > >> > > > >> offset with the old partition epoch.
>> > > > >> > > > >>
>> > > > >> > > > >
>> > > > >> > > > > Could you help me understand the problem if a client
>> > > associates
>> > > > >> old
>> > > > >> > > > > partition_epoch (or the topic_epoch as of the current
>> KIP)
>> > > with
>> > > > >> the
>> > > > >> > > > offset?
>> > > > >> > > > > The main purpose of the topic_epoch is to be able to drop
>> > > > >> > leader_epoch
>> > > > >> > > > to 0
>> > > > >> > > > > after a partition is deleted and re-created. I guess you
>> may
>> > > be
>> > > > >> > > thinking
>> > > > >> > > > > about using the partition_epoch to detect that the
>> committed
>> > > > >> offset
>> > > > >> > is
>> > > > >> > > > > invalid? In that case, I am wondering if the alternative
>> > > > approach
>> > > > >> > > > described
>> > > > >> > > > > in 60) would be reasonable.
>> > > > >> > > > >
>> > > > >> > > > >
>> > > > >> > > > >>
>> > > > >> > > > >> 63. There is some subtle coordination between the
>> > > > >> > LeaderAndIsrRequest
>> > > > >> > > > and
>> > > > >> > > > >> UpdateMetadataRequest. Currently, when a leader changes,
>> > the
>> > > > >> > > controller
>> > > > >> > > > >> first sends the LeaderAndIsrRequest to the assigned
>> > replicas
>> > > > and
>> > > > >> the
>> > > > >> > > > >> UpdateMetadataRequest to every broker. So, there could
>> be a
>> > > > small
>> > > > >> > > window
>> > > > >> > > > >> when the leader already receives the new partition
>> epoch in
>> > > the
>> > > > >> > > > >> LeaderAndIsrRequest, but the metadata cache in the
>> broker
>> > > > hasn't
>> > > > >> > been
>> > > > >> > > > >> updated with the latest partition epoch. Not sure what's
>> > the
>> > > > best
>> > > > >> > way
>> > > > >> > > to
>> > > > >> > > > >> address this issue. Perhaps we can update the metadata
>> > cache
>> > > on
>> > > > >> the
>> > > > >> > > > broker
>> > > > >> > > > >> with both LeaderAndIsrRequest and UpdateMetadataRequest.
>> > The
>> > > > >> > challenge
>> > > > >> > > > is
>> > > > >> > > > >> that the two have slightly different data. For example,
>> > only
>> > > > the
>> > > > >> > > latter
>> > > > >> > > > >> has
>> > > > >> > > > >> all endpoints.
>> > > > >> > > > >>
>> > > > >> > > > >
>> > > > >> > > > > I am not sure whether this is a problem. Could you
>> explain a
>> > > bit
>> > > > >> more
>> > > > >> > > > what
>> > > > >> > > > > specific problem this small window can cause?
>> > > > >> > > > >
>> > > > >> > > > > Since client can fetch metadata from any broker in the
>> > > cluster,
>> > > > >> and
>> > > > >> > > given
>> > > > >> > > > > that different brokers receive request (e.g.
>> > > LeaderAndIsrRequest
>> > > > >> and
>> > > > >> > > > > UpdateMetadataRequest) in arbitrary order, the metadata
>> > > received
>> > > > >> by
>> > > > >> > > > client
>> > > > >> > > > > can be in arbitrary order (either newer or older)
>> compared
>> > to
>> > > > the
>> > > > >> > > > broker's
>> > > > >> > > > > leadership state even if a given broker receives
>> > > > >> LeaderAndIsrRequest
>> > > > >> > > and
>> > > > >> > > > > UpdateMetadataRequest simultaneously. So I am not sure
>> it is
>> > > > >> useful
>> > > > >> > to
>> > > > >> > > > > update broker's cache with LeaderAndIsrRequest.
>> > > > >> > > > >
>> > > > >> > > > >
>> > > > >> > > > >> 64. The enforcement of leader epoch in Offset commit: We
>> > > allow
>> > > > a
>> > > > >> > > > consumer
>> > > > >> > > > >> to set an arbitrary offset. So it's possible for
>> offsets or
>> > > > >> leader
>> > > > >> > > epoch
>> > > > >> > > > >> to
>> > > > >> > > > >> go backwards. I am not sure if we could always enforce
>> that
>> > > the
>> > > > >> > leader
>> > > > >> > > > >> epoch only goes up on the broker.
>> > > > >> > > > >>
>> > > > >> > > > >
>> > > > >> > > > > Sure. I have removed this check from the KIP.
>> > > > >> > > > >
>> > > > >> > > > > BTW, we can probably still ensure that the leader_epoch
>> > always
>> > > > >> > increase
>> > > > >> > > > if
>> > > > >> > > > > the leader_epoch used with offset commit is the
>> > > max(leader_epoch
>> > > > >> of
>> > > > >> > the
>> > > > >> > > > > message with offset = the committed offset - 1, the
>> largest
>> > > > known
>> > > > >> > > > > leader_epoch from the metadata). But I don't have a good
>> > > > use-case
>> > > > >> for
>> > > > >> > > > this
>> > > > >> > > > > alternative definition. So I choose the keep the KIP
>> simple
>> > by
>> > > > >> > > requiring
>> > > > >> > > > > leader_epoch to always increase.
>> > > > >> > > > >
>> > > > >> > > > >
>> > > > >> > > > >> 65. Good point on handling missing partition epoch due
>> to
>> > > topic
>> > > > >> > > > deletion.
>> > > > >> > > > >> Another potential way to address this is to additionally
>> > > > >> propagate
>> > > > >> > the
>> > > > >> > > > >> global partition epoch to brokers and the clients. This
>> > way,
>> > > > >> when a
>> > > > >> > > > >> partition epoch is missing, we can use the global
>> partition
>> > > > >> epoch to
>> > > > >> > > > >> reason
>> > > > >> > > > >> about which metadata is more recent.
>> > > > >> > > > >>
>> > > > >> > > > >
>> > > > >> > > > > This is a great idea. The global epoch can be used to
>> order
>> > > the
>> > > > >> > > metadata
>> > > > >> > > > > and help us recognize the more recent metadata if a topic
>> > (or
>> > > > >> > > partition)
>> > > > >> > > > is
>> > > > >> > > > > deleted and re-created.
>> > > > >> > > > >
>> > > > >> > > > > Actually, it seems we only need to propagate the global
>> > epoch
>> > > to
>> > > > >> > > brokers
>> > > > >> > > > > and clients without propagating this epoch on a
>> per-topic or
>> > > > >> > > > per-partition
>> > > > >> > > > > basic. Doing so would simply interface changes made this
>> > KIP.
>> > > > Does
>> > > > >> > this
>> > > > >> > > > > approach sound reasonable?
>> > > > >> > > > >
>> > > > >> > > > >
>> > > > >> > > > >> 66. A client may also get an offset by time using the
>> > > > >> > offsetForTimes()
>> > > > >> > > > >> api.
>> > > > >> > > > >> So, we probably want to include offsetInternalMetadata
>> in
>> > > > >> > > > >> OffsetAndTimestamp
>> > > > >> > > > >> as well.
>> > > > >> > > > >>
>> > > > >> > > > >
>> > > > >> > > > > You are right. This probably also requires us to change
>> the
>> > > > >> > > > > ListOffsetRequest as well. I will update the KIP after we
>> > > agree
>> > > > on
>> > > > >> > the
>> > > > >> > > > > solution for 65).
>> > > > >> > > > >
>> > > > >> > > > >
>> > > > >> > > > >>
>> > > > >> > > > >> 67. InteralMetadata can be a bit confusing with the
>> > metadata
>> > > > >> field
>> > > > >> > > > already
>> > > > >> > > > >> there. Perhaps we can just call it OffsetEpoch. It
>> might be
>> > > > >> useful
>> > > > >> > to
>> > > > >> > > > make
>> > > > >> > > > >> OffsetEpoch printable at least for debugging purpose.
>> Once
>> > > you
>> > > > do
>> > > > >> > > that,
>> > > > >> > > > we
>> > > > >> > > > >> are already exposing the internal fields. So, not sure
>> if
>> > > it's
>> > > > >> worth
>> > > > >> > > > >> hiding
>> > > > >> > > > >> them. If we do want to hide them, perhaps we can have
>> sth
>> > > like
>> > > > >> the
>> > > > >> > > > >> following. The binary encoding is probably more
>> efficient
>> > > than
>> > > > >> JSON
>> > > > >> > > for
>> > > > >> > > > >> external storage.
>> > > > >> > > > >>
>> > > > >> > > > >> OffsetEpoch {
>> > > > >> > > > >>  static OffsetEpoch decode(byte[]);
>> > > > >> > > > >>
>> > > > >> > > > >>   public byte[] encode();
>> > > > >> > > > >>
>> > > > >> > > > >>   public String toString();
>> > > > >> > > > >> }
>> > > > >> > > > >>
>> > > > >> > > > >
>> > > > >> > > > > Thanks much. I like this solution. I have updated the KIP
>> > > > >> > accordingly.
>> > > > >> > > > >
>> > > > >> > > > >
>> > > > >> > > > >
>> > > > >> > > > >>
>> > > > >> > > > >> Jun
>> > > > >> > > > >>
>> > > > >> > > > >> On Mon, Jan 8, 2018 at 4:22 PM, Dong Lin <
>> > > lindong28@gmail.com>
>> > > > >> > wrote:
>> > > > >> > > > >>
>> > > > >> > > > >> > Hey Jason,
>> > > > >> > > > >> >
>> > > > >> > > > >> > Certainly. This sounds good. I have updated the KIP to
>> > > > clarity
>> > > > >> > that
>> > > > >> > > > the
>> > > > >> > > > >> > global epoch will be incremented by 1 each time a
>> topic
>> > is
>> > > > >> > deleted.
>> > > > >> > > > >> >
>> > > > >> > > > >> > Thanks,
>> > > > >> > > > >> > Dong
>> > > > >> > > > >> >
>> > > > >> > > > >> > On Mon, Jan 8, 2018 at 4:09 PM, Jason Gustafson <
>> > > > >> > jason@confluent.io
>> > > > >> > > >
>> > > > >> > > > >> > wrote:
>> > > > >> > > > >> >
>> > > > >> > > > >> > > Hi Dong,
>> > > > >> > > > >> > >
>> > > > >> > > > >> > >
>> > > > >> > > > >> > > I think your approach will allow user to distinguish
>> > > > between
>> > > > >> the
>> > > > >> > > > >> metadata
>> > > > >> > > > >> > > > before and after the topic deletion. I also agree
>> > that
>> > > > this
>> > > > >> > can
>> > > > >> > > be
>> > > > >> > > > >> > > > potentially be useful to user. I am just not very
>> > sure
>> > > > >> whether
>> > > > >> > > we
>> > > > >> > > > >> > already
>> > > > >> > > > >> > > > have a good use-case to make the additional
>> > complexity
>> > > > >> > > worthwhile.
>> > > > >> > > > >> It
>> > > > >> > > > >> > > seems
>> > > > >> > > > >> > > > that this feature is kind of independent of the
>> main
>> > > > >> problem
>> > > > >> > of
>> > > > >> > > > this
>> > > > >> > > > >> > KIP.
>> > > > >> > > > >> > > > Could we add this as a future work?
>> > > > >> > > > >> > >
>> > > > >> > > > >> > >
>> > > > >> > > > >> > > Do you think it's fair if we bump the topic epoch on
>> > > > deletion
>> > > > >> > and
>> > > > >> > > > >> leave
>> > > > >> > > > >> > > propagation of the epoch for deleted topics for
>> future
>> > > > work?
>> > > > >> I
>> > > > >> > > don't
>> > > > >> > > > >> > think
>> > > > >> > > > >> > > this adds much complexity and it makes the behavior
>> > > > >> consistent:
>> > > > >> > > > every
>> > > > >> > > > >> > topic
>> > > > >> > > > >> > > mutation results in an epoch bump.
>> > > > >> > > > >> > >
>> > > > >> > > > >> > > Thanks,
>> > > > >> > > > >> > > Jason
>> > > > >> > > > >> > >
>> > > > >> > > > >> > > On Mon, Jan 8, 2018 at 3:14 PM, Dong Lin <
>> > > > >> lindong28@gmail.com>
>> > > > >> > > > wrote:
>> > > > >> > > > >> > >
>> > > > >> > > > >> > > > Hey Ismael,
>> > > > >> > > > >> > > >
>> > > > >> > > > >> > > > I guess we actually need user to see this field so
>> > that
>> > > > >> user
>> > > > >> > can
>> > > > >> > > > >> store
>> > > > >> > > > >> > > this
>> > > > >> > > > >> > > > value in the external store together with the
>> offset.
>> > > We
>> > > > >> just
>> > > > >> > > > prefer
>> > > > >> > > > >> > the
>> > > > >> > > > >> > > > value to be opaque to discourage most users from
>> > > > >> interpreting
>> > > > >> > > this
>> > > > >> > > > >> > value.
>> > > > >> > > > >> > > > One more advantage of using such an opaque field
>> is
>> > to
>> > > be
>> > > > >> able
>> > > > >> > > to
>> > > > >> > > > >> > evolve
>> > > > >> > > > >> > > > the information (or schema) of this value without
>> > > > changing
>> > > > >> > > > consumer
>> > > > >> > > > >> API
>> > > > >> > > > >> > > in
>> > > > >> > > > >> > > > the future.
>> > > > >> > > > >> > > >
>> > > > >> > > > >> > > > I also thinking it is probably OK for user to be
>> able
>> > > to
>> > > > >> > > interpret
>> > > > >> > > > >> this
>> > > > >> > > > >> > > > value, particularly for those advanced users.
>> > > > >> > > > >> > > >
>> > > > >> > > > >> > > > Thanks,
>> > > > >> > > > >> > > > Dong
>> > > > >> > > > >> > > >
>> > > > >> > > > >> > > > On Mon, Jan 8, 2018 at 2:34 PM, Ismael Juma <
>> > > > >> > ismael@juma.me.uk>
>> > > > >> > > > >> wrote:
>> > > > >> > > > >> > > >
>> > > > >> > > > >> > > > > On Fri, Jan 5, 2018 at 7:15 PM, Jason Gustafson
>> <
>> > > > >> > > > >> jason@confluent.io>
>> > > > >> > > > >> > > > > wrote:
>> > > > >> > > > >> > > > > >
>> > > > >> > > > >> > > > > > class OffsetAndMetadata {
>> > > > >> > > > >> > > > > >   long offset;
>> > > > >> > > > >> > > > > >   byte[] offsetMetadata;
>> > > > >> > > > >> > > > > >   String metadata;
>> > > > >> > > > >> > > > > > }
>> > > > >> > > > >> > > > >
>> > > > >> > > > >> > > > >
>> > > > >> > > > >> > > > > > Admittedly, the naming is a bit annoying, but
>> we
>> > > can
>> > > > >> > > probably
>> > > > >> > > > >> come
>> > > > >> > > > >> > up
>> > > > >> > > > >> > > > > with
>> > > > >> > > > >> > > > > > something better. Internally the byte array
>> would
>> > > > have
>> > > > >> a
>> > > > >> > > > >> version.
>> > > > >> > > > >> > If
>> > > > >> > > > >> > > in
>> > > > >> > > > >> > > > > the
>> > > > >> > > > >> > > > > > future we have anything else we need to add,
>> we
>> > can
>> > > > >> update
>> > > > >> > > the
>> > > > >> > > > >> > > version
>> > > > >> > > > >> > > > > and
>> > > > >> > > > >> > > > > > we wouldn't need any new APIs.
>> > > > >> > > > >> > > > > >
>> > > > >> > > > >> > > > >
>> > > > >> > > > >> > > > > We can also add fields to a class in a
>> compatible
>> > > way.
>> > > > >> So,
>> > > > >> > it
>> > > > >> > > > >> seems
>> > > > >> > > > >> > to
>> > > > >> > > > >> > > me
>> > > > >> > > > >> > > > > that the main advantage of the byte array is
>> that
>> > > it's
>> > > > >> > opaque
>> > > > >> > > to
>> > > > >> > > > >> the
>> > > > >> > > > >> > > > user.
>> > > > >> > > > >> > > > > Is that correct? If so, we could also add any
>> > opaque
>> > > > >> > metadata
>> > > > >> > > > in a
>> > > > >> > > > >> > > > subclass
>> > > > >> > > > >> > > > > so that users don't even see it (unless they
>> cast
>> > it,
>> > > > but
>> > > > >> > then
>> > > > >> > > > >> > they're
>> > > > >> > > > >> > > on
>> > > > >> > > > >> > > > > their own).
>> > > > >> > > > >> > > > >
>> > > > >> > > > >> > > > > Ismael
>> > > > >> > > > >> > > > >
>> > > > >> > > > >> > > > > The corresponding seek() and position() APIs
>> might
>> > > look
>> > > > >> > > > something
>> > > > >> > > > >> > like
>> > > > >> > > > >> > > > > this:
>> > > > >> > > > >> > > > > >
>> > > > >> > > > >> > > > > > void seek(TopicPartition partition, long
>> offset,
>> > > > byte[]
>> > > > >> > > > >> > > > offsetMetadata);
>> > > > >> > > > >> > > > > > byte[] positionMetadata(TopicPartition
>> > partition);
>> > > > >> > > > >> > > > > >
>> > > > >> > > > >> > > > > > What do you think?
>> > > > >> > > > >> > > > > >
>> > > > >> > > > >> > > > > > Thanks,
>> > > > >> > > > >> > > > > > Jason
>> > > > >> > > > >> > > > > >
>> > > > >> > > > >> > > > > > On Thu, Jan 4, 2018 at 7:04 PM, Dong Lin <
>> > > > >> > > lindong28@gmail.com
>> > > > >> > > > >
>> > > > >> > > > >> > > wrote:
>> > > > >> > > > >> > > > > >
>> > > > >> > > > >> > > > > > > Hey Jun, Jason,
>> > > > >> > > > >> > > > > > >
>> > > > >> > > > >> > > > > > > Thanks much for all the feedback. I have
>> > updated
>> > > > the
>> > > > >> KIP
>> > > > >> > > > >> based on
>> > > > >> > > > >> > > the
>> > > > >> > > > >> > > > > > > latest discussion. Can you help check
>> whether
>> > it
>> > > > >> looks
>> > > > >> > > good?
>> > > > >> > > > >> > > > > > >
>> > > > >> > > > >> > > > > > > Thanks,
>> > > > >> > > > >> > > > > > > Dong
>> > > > >> > > > >> > > > > > >
>> > > > >> > > > >> > > > > > > On Thu, Jan 4, 2018 at 5:36 PM, Dong Lin <
>> > > > >> > > > lindong28@gmail.com
>> > > > >> > > > >> >
>> > > > >> > > > >> > > > wrote:
>> > > > >> > > > >> > > > > > >
>> > > > >> > > > >> > > > > > > > Hey Jun,
>> > > > >> > > > >> > > > > > > >
>> > > > >> > > > >> > > > > > > > Hmm... thinking about this more, I am not
>> > sure
>> > > > that
>> > > > >> > the
>> > > > >> > > > >> > proposed
>> > > > >> > > > >> > > > API
>> > > > >> > > > >> > > > > is
>> > > > >> > > > >> > > > > > > > sufficient. For users that store offset
>> > > > >> externally, we
>> > > > >> > > > >> probably
>> > > > >> > > > >> > > > need
>> > > > >> > > > >> > > > > > > extra
>> > > > >> > > > >> > > > > > > > API to return the leader_epoch and
>> > > > partition_epoch
>> > > > >> for
>> > > > >> > > all
>> > > > >> > > > >> > > > partitions
>> > > > >> > > > >> > > > > > > that
>> > > > >> > > > >> > > > > > > > consumers are consuming. I suppose these
>> > users
>> > > > >> > currently
>> > > > >> > > > use
>> > > > >> > > > >> > > > > position()
>> > > > >> > > > >> > > > > > > to
>> > > > >> > > > >> > > > > > > > get the offset. Thus we probably need a
>> new
>> > > > method
>> > > > >> > > > >> > > > > > positionWithEpoch(..)
>> > > > >> > > > >> > > > > > > to
>> > > > >> > > > >> > > > > > > > return <offset, partition_epoch,
>> > leader_epoch>.
>> > > > >> Does
>> > > > >> > > this
>> > > > >> > > > >> sound
>> > > > >> > > > >> > > > > > > reasonable?
>> > > > >> > > > >> > > > > > > >
>> > > > >> > > > >> > > > > > > > Thanks,
>> > > > >> > > > >> > > > > > > > Dong
>> > > > >> > > > >> > > > > > > >
>> > > > >> > > > >> > > > > > > >
>> > > > >> > > > >> > > > > > > > On Thu, Jan 4, 2018 at 5:26 PM, Jun Rao <
>> > > > >> > > jun@confluent.io
>> > > > >> > > > >
>> > > > >> > > > >> > > wrote:
>> > > > >> > > > >> > > > > > > >
>> > > > >> > > > >> > > > > > > >> Hi, Dong,
>> > > > >> > > > >> > > > > > > >>
>> > > > >> > > > >> > > > > > > >> Yes, that's what I am thinking.
>> OffsetEpoch
>> > > will
>> > > > >> be
>> > > > >> > > > >> composed
>> > > > >> > > > >> > of
>> > > > >> > > > >> > > > > > > >> (partition_epoch,
>> > > > >> > > > >> > > > > > > >> leader_epoch).
>> > > > >> > > > >> > > > > > > >>
>> > > > >> > > > >> > > > > > > >> Thanks,
>> > > > >> > > > >> > > > > > > >>
>> > > > >> > > > >> > > > > > > >> Jun
>> > > > >> > > > >> > > > > > > >>
>> > > > >> > > > >> > > > > > > >>
>> > > > >> > > > >> > > > > > > >> On Thu, Jan 4, 2018 at 4:22 PM, Dong Lin
>> <
>> > > > >> > > > >> lindong28@gmail.com
>> > > > >> > > > >> > >
>> > > > >> > > > >> > > > > wrote:
>> > > > >> > > > >> > > > > > > >>
>> > > > >> > > > >> > > > > > > >> > Hey Jun,
>> > > > >> > > > >> > > > > > > >> >
>> > > > >> > > > >> > > > > > > >> > Thanks much. I like the the new API
>> that
>> > you
>> > > > >> > > proposed.
>> > > > >> > > > I
>> > > > >> > > > >> am
>> > > > >> > > > >> > > not
>> > > > >> > > > >> > > > > sure
>> > > > >> > > > >> > > > > > > >> what
>> > > > >> > > > >> > > > > > > >> > you exactly mean by offset_epoch. I
>> > suppose
>> > > > >> that we
>> > > > >> > > can
>> > > > >> > > > >> use
>> > > > >> > > > >> > > the
>> > > > >> > > > >> > > > > pair
>> > > > >> > > > >> > > > > > > of
>> > > > >> > > > >> > > > > > > >> > (partition_epoch, leader_epoch) as the
>> > > > >> > offset_epoch,
>> > > > >> > > > >> right?
>> > > > >> > > > >> > > > > > > >> >
>> > > > >> > > > >> > > > > > > >> > Thanks,
>> > > > >> > > > >> > > > > > > >> > Dong
>> > > > >> > > > >> > > > > > > >> >
>> > > > >> > > > >> > > > > > > >> > On Thu, Jan 4, 2018 at 4:02 PM, Jun
>> Rao <
>> > > > >> > > > >> jun@confluent.io>
>> > > > >> > > > >> > > > wrote:
>> > > > >> > > > >> > > > > > > >> >
>> > > > >> > > > >> > > > > > > >> > > Hi, Dong,
>> > > > >> > > > >> > > > > > > >> > >
>> > > > >> > > > >> > > > > > > >> > > Got it. The api that you proposed
>> works.
>> > > The
>> > > > >> > > question
>> > > > >> > > > >> is
>> > > > >> > > > >> > > > whether
>> > > > >> > > > >> > > > > > > >> that's
>> > > > >> > > > >> > > > > > > >> > the
>> > > > >> > > > >> > > > > > > >> > > api that we want to have in the long
>> > term.
>> > > > My
>> > > > >> > > concern
>> > > > >> > > > >> is
>> > > > >> > > > >> > > that
>> > > > >> > > > >> > > > > > while
>> > > > >> > > > >> > > > > > > >> the
>> > > > >> > > > >> > > > > > > >> > api
>> > > > >> > > > >> > > > > > > >> > > change is simple, the new api seems
>> > harder
>> > > > to
>> > > > >> > > explain
>> > > > >> > > > >> and
>> > > > >> > > > >> > > use.
>> > > > >> > > > >> > > > > For
>> > > > >> > > > >> > > > > > > >> > example,
>> > > > >> > > > >> > > > > > > >> > > a consumer storing offsets externally
>> > now
>> > > > >> needs
>> > > > >> > to
>> > > > >> > > > call
>> > > > >> > > > >> > > > > > > >> > > waitForMetadataUpdate() after calling
>> > > > seek().
>> > > > >> > > > >> > > > > > > >> > >
>> > > > >> > > > >> > > > > > > >> > > An alternative approach is to make
>> the
>> > > > >> following
>> > > > >> > > > >> > compatible
>> > > > >> > > > >> > > > api
>> > > > >> > > > >> > > > > > > >> changes
>> > > > >> > > > >> > > > > > > >> > in
>> > > > >> > > > >> > > > > > > >> > > Consumer.
>> > > > >> > > > >> > > > > > > >> > > * Add an additional OffsetEpoch
>> field in
>> > > > >> > > > >> > OffsetAndMetadata.
>> > > > >> > > > >> > > > (no
>> > > > >> > > > >> > > > > > need
>> > > > >> > > > >> > > > > > > >> to
>> > > > >> > > > >> > > > > > > >> > > change the CommitSync() api)
>> > > > >> > > > >> > > > > > > >> > > * Add a new api seek(TopicPartition
>> > > > partition,
>> > > > >> > long
>> > > > >> > > > >> > offset,
>> > > > >> > > > >> > > > > > > >> OffsetEpoch
>> > > > >> > > > >> > > > > > > >> > > offsetEpoch). We can potentially
>> > deprecate
>> > > > the
>> > > > >> > old
>> > > > >> > > > api
>> > > > >> > > > >> > > > > > > >> > seek(TopicPartition
>> > > > >> > > > >> > > > > > > >> > > partition, long offset) in the
>> future.
>> > > > >> > > > >> > > > > > > >> > >
>> > > > >> > > > >> > > > > > > >> > > The alternative approach has similar
>> > > amount
>> > > > of
>> > > > >> > api
>> > > > >> > > > >> changes
>> > > > >> > > > >> > > as
>> > > > >> > > > >> > > > > > yours
>> > > > >> > > > >> > > > > > > >> but
>> > > > >> > > > >> > > > > > > >> > has
>> > > > >> > > > >> > > > > > > >> > > the following benefits.
>> > > > >> > > > >> > > > > > > >> > > 1. The api works in a similar way as
>> how
>> > > > >> offset
>> > > > >> > > > >> management
>> > > > >> > > > >> > > > works
>> > > > >> > > > >> > > > > > now
>> > > > >> > > > >> > > > > > > >> and
>> > > > >> > > > >> > > > > > > >> > is
>> > > > >> > > > >> > > > > > > >> > > probably what we want in the long
>> term.
>> > > > >> > > > >> > > > > > > >> > > 2. It can reset offsets better when
>> > there
>> > > is
>> > > > >> data
>> > > > >> > > > loss
>> > > > >> > > > >> due
>> > > > >> > > > >> > > to
>> > > > >> > > > >> > > > > > > unclean
>> > > > >> > > > >> > > > > > > >> > > leader election or correlated replica
>> > > > failure.
>> > > > >> > > > >> > > > > > > >> > > 3. It can reset offsets better when
>> > topic
>> > > is
>> > > > >> > > > recreated.
>> > > > >> > > > >> > > > > > > >> > >
>> > > > >> > > > >> > > > > > > >> > > Thanks,
>> > > > >> > > > >> > > > > > > >> > >
>> > > > >> > > > >> > > > > > > >> > > Jun
>> > > > >> > > > >> > > > > > > >> > >
>> > > > >> > > > >> > > > > > > >> > >
>> > > > >> > > > >> > > > > > > >> > > On Thu, Jan 4, 2018 at 2:05 PM, Dong
>> > Lin <
>> > > > >> > > > >> > > lindong28@gmail.com
>> > > > >> > > > >> > > > >
>> > > > >> > > > >> > > > > > > wrote:
>> > > > >> > > > >> > > > > > > >> > >
>> > > > >> > > > >> > > > > > > >> > > > Hey Jun,
>> > > > >> > > > >> > > > > > > >> > > >
>> > > > >> > > > >> > > > > > > >> > > > Yeah I agree that ideally we don't
>> > want
>> > > an
>> > > > >> ever
>> > > > >> > > > >> growing
>> > > > >> > > > >> > > > global
>> > > > >> > > > >> > > > > > > >> metadata
>> > > > >> > > > >> > > > > > > >> > > > version. I just think it may be
>> more
>> > > > >> desirable
>> > > > >> > to
>> > > > >> > > > >> keep
>> > > > >> > > > >> > the
>> > > > >> > > > >> > > > > > > consumer
>> > > > >> > > > >> > > > > > > >> API
>> > > > >> > > > >> > > > > > > >> > > > simple.
>> > > > >> > > > >> > > > > > > >> > > >
>> > > > >> > > > >> > > > > > > >> > > > In my current proposal, metadata
>> > version
>> > > > >> > returned
>> > > > >> > > > in
>> > > > >> > > > >> the
>> > > > >> > > > >> > > > fetch
>> > > > >> > > > >> > > > > > > >> response
>> > > > >> > > > >> > > > > > > >> > > > will be stored with the offset
>> > together.
>> > > > >> More
>> > > > >> > > > >> > > specifically,
>> > > > >> > > > >> > > > > the
>> > > > >> > > > >> > > > > > > >> > > > metadata_epoch in the new offset
>> topic
>> > > > >> schema
>> > > > >> > > will
>> > > > >> > > > be
>> > > > >> > > > >> > the
>> > > > >> > > > >> > > > > > largest
>> > > > >> > > > >> > > > > > > >> > > > metadata_epoch from all the
>> > > > MetadataResponse
>> > > > >> > and
>> > > > >> > > > >> > > > FetchResponse
>> > > > >> > > > >> > > > > > > ever
>> > > > >> > > > >> > > > > > > >> > > > received by this consumer.
>> > > > >> > > > >> > > > > > > >> > > >
>> > > > >> > > > >> > > > > > > >> > > > We probably don't have to change
>> the
>> > > > >> consumer
>> > > > >> > API
>> > > > >> > > > for
>> > > > >> > > > >> > > > > > > >> > > > commitSync(Map<TopicPartition,
>> > > > >> > > OffsetAndMetadata>).
>> > > > >> > > > >> If
>> > > > >> > > > >> > > user
>> > > > >> > > > >> > > > > > calls
>> > > > >> > > > >> > > > > > > >> > > > commitSync(...) to commit offset 10
>> > for
>> > > a
>> > > > >> given
>> > > > >> > > > >> > partition,
>> > > > >> > > > >> > > > for
>> > > > >> > > > >> > > > > > > most
>> > > > >> > > > >> > > > > > > >> > > > use-cases, this consumer instance
>> > should
>> > > > >> have
>> > > > >> > > > >> consumed
>> > > > >> > > > >> > > > message
>> > > > >> > > > >> > > > > > > with
>> > > > >> > > > >> > > > > > > >> > > offset
>> > > > >> > > > >> > > > > > > >> > > > 9 from this partition, in which
>> case
>> > the
>> > > > >> > consumer
>> > > > >> > > > can
>> > > > >> > > > >> > > > remember
>> > > > >> > > > >> > > > > > and
>> > > > >> > > > >> > > > > > > >> use
>> > > > >> > > > >> > > > > > > >> > > the
>> > > > >> > > > >> > > > > > > >> > > > metadata_epoch from the
>> corresponding
>> > > > >> > > FetchResponse
>> > > > >> > > > >> when
>> > > > >> > > > >> > > > > > > committing
>> > > > >> > > > >> > > > > > > >> > > offset.
>> > > > >> > > > >> > > > > > > >> > > > If user calls commitSync(..) to
>> commit
>> > > > >> offset
>> > > > >> > 10
>> > > > >> > > > for
>> > > > >> > > > >> a
>> > > > >> > > > >> > > given
>> > > > >> > > > >> > > > > > > >> partition
>> > > > >> > > > >> > > > > > > >> > > > without having consumed the message
>> > with
>> > > > >> > offset 9
>> > > > >> > > > >> using
>> > > > >> > > > >> > > this
>> > > > >> > > > >> > > > > > > >> consumer
>> > > > >> > > > >> > > > > > > >> > > > instance, this is probably an
>> advanced
>> > > > >> > use-case.
>> > > > >> > > In
>> > > > >> > > > >> this
>> > > > >> > > > >> > > > case
>> > > > >> > > > >> > > > > > the
>> > > > >> > > > >> > > > > > > >> > > advanced
>> > > > >> > > > >> > > > > > > >> > > > user can retrieve the
>> metadata_epoch
>> > > using
>> > > > >> the
>> > > > >> > > > newly
>> > > > >> > > > >> > added
>> > > > >> > > > >> > > > > > > >> > > metadataEpoch()
>> > > > >> > > > >> > > > > > > >> > > > API after it fetches the message
>> with
>> > > > >> offset 9
>> > > > >> > > > >> (probably
>> > > > >> > > > >> > > > from
>> > > > >> > > > >> > > > > > > >> another
>> > > > >> > > > >> > > > > > > >> > > > consumer instance) and encode this
>> > > > >> > metadata_epoch
>> > > > >> > > > in
>> > > > >> > > > >> the
>> > > > >> > > > >> > > > > > > >> > > > string OffsetAndMetadata.metadata.
>> Do
>> > > you
>> > > > >> think
>> > > > >> > > > this
>> > > > >> > > > >> > > > solution
>> > > > >> > > > >> > > > > > > would
>> > > > >> > > > >> > > > > > > >> > work?
>> > > > >> > > > >> > > > > > > >> > > >
>> > > > >> > > > >> > > > > > > >> > > > By "not sure that I fully
>> understand
>> > > your
>> > > > >> > latest
>> > > > >> > > > >> > > > suggestion",
>> > > > >> > > > >> > > > > > are
>> > > > >> > > > >> > > > > > > >> you
>> > > > >> > > > >> > > > > > > >> > > > referring to solution related to
>> > unclean
>> > > > >> leader
>> > > > >> > > > >> election
>> > > > >> > > > >> > > > using
>> > > > >> > > > >> > > > > > > >> > > leader_epoch
>> > > > >> > > > >> > > > > > > >> > > > in my previous email?
>> > > > >> > > > >> > > > > > > >> > > >
>> > > > >> > > > >> > > > > > > >> > > > Thanks,
>> > > > >> > > > >> > > > > > > >> > > > Dong
>> > > > >> > > > >> > > > > > > >> > > >
>> > > > >> > > > >> > > > > > > >> > > > On Thu, Jan 4, 2018 at 1:33 PM, Jun
>> > Rao
>> > > <
>> > > > >> > > > >> > jun@confluent.io
>> > > > >> > > > >> > > >
>> > > > >> > > > >> > > > > > wrote:
>> > > > >> > > > >> > > > > > > >> > > >
>> > > > >> > > > >> > > > > > > >> > > > > Hi, Dong,
>> > > > >> > > > >> > > > > > > >> > > > >
>> > > > >> > > > >> > > > > > > >> > > > > Not sure that I fully understand
>> > your
>> > > > >> latest
>> > > > >> > > > >> > suggestion.
>> > > > >> > > > >> > > > > > > >> Returning an
>> > > > >> > > > >> > > > > > > >> > > > ever
>> > > > >> > > > >> > > > > > > >> > > > > growing global metadata version
>> > itself
>> > > > is
>> > > > >> no
>> > > > >> > > > ideal,
>> > > > >> > > > >> > but
>> > > > >> > > > >> > > is
>> > > > >> > > > >> > > > > > fine.
>> > > > >> > > > >> > > > > > > >> My
>> > > > >> > > > >> > > > > > > >> > > > > question is whether the metadata
>> > > version
>> > > > >> > > returned
>> > > > >> > > > >> in
>> > > > >> > > > >> > the
>> > > > >> > > > >> > > > > fetch
>> > > > >> > > > >> > > > > > > >> > response
>> > > > >> > > > >> > > > > > > >> > > > > needs to be stored with the
>> offset
>> > > > >> together
>> > > > >> > if
>> > > > >> > > > >> offsets
>> > > > >> > > > >> > > are
>> > > > >> > > > >> > > > > > > stored
>> > > > >> > > > >> > > > > > > >> > > > > externally. If so, we also have
>> to
>> > > > change
>> > > > >> the
>> > > > >> > > > >> consumer
>> > > > >> > > > >> > > API
>> > > > >> > > > >> > > > > for
>> > > > >> > > > >> > > > > > > >> > > > commitSync()
>> > > > >> > > > >> > > > > > > >> > > > > and need to worry about
>> > compatibility.
>> > > > If
>> > > > >> we
>> > > > >> > > > don't
>> > > > >> > > > >> > store
>> > > > >> > > > >> > > > the
>> > > > >> > > > >> > > > > > > >> metadata
>> > > > >> > > > >> > > > > > > >> > > > > version together with the offset,
>> > on a
>> > > > >> > consumer
>> > > > >> > > > >> > restart,
>> > > > >> > > > >> > > > > it's
>> > > > >> > > > >> > > > > > > not
>> > > > >> > > > >> > > > > > > >> > clear
>> > > > >> > > > >> > > > > > > >> > > > how
>> > > > >> > > > >> > > > > > > >> > > > > we can ensure the metadata in the
>> > > > >> consumer is
>> > > > >> > > > high
>> > > > >> > > > >> > > enough
>> > > > >> > > > >> > > > > > since
>> > > > >> > > > >> > > > > > > >> there
>> > > > >> > > > >> > > > > > > >> > > is
>> > > > >> > > > >> > > > > > > >> > > > no
>> > > > >> > > > >> > > > > > > >> > > > > metadata version to compare with.
>> > > > >> > > > >> > > > > > > >> > > > >
>> > > > >> > > > >> > > > > > > >> > > > > Thanks,
>> > > > >> > > > >> > > > > > > >> > > > >
>> > > > >> > > > >> > > > > > > >> > > > > Jun
>> > > > >> > > > >> > > > > > > >> > > > >
>> > > > >> > > > >> > > > > > > >> > > > >
>> > > > >> > > > >> > > > > > > >> > > > > On Wed, Jan 3, 2018 at 6:43 PM,
>> Dong
>> > > > Lin <
>> > > > >> > > > >> > > > > lindong28@gmail.com
>> > > > >> > > > >> > > > > > >
>> > > > >> > > > >> > > > > > > >> > wrote:
>> > > > >> > > > >> > > > > > > >> > > > >
>> > > > >> > > > >> > > > > > > >> > > > > > Hey Jun,
>> > > > >> > > > >> > > > > > > >> > > > > >
>> > > > >> > > > >> > > > > > > >> > > > > > Thanks much for the
>> explanation.
>> > > > >> > > > >> > > > > > > >> > > > > >
>> > > > >> > > > >> > > > > > > >> > > > > > I understand the advantage of
>> > > > >> > partition_epoch
>> > > > >> > > > >> over
>> > > > >> > > > >> > > > > > > >> metadata_epoch.
>> > > > >> > > > >> > > > > > > >> > My
>> > > > >> > > > >> > > > > > > >> > > > > > current concern is that the
>> use of
>> > > > >> > > leader_epoch
>> > > > >> > > > >> and
>> > > > >> > > > >> > > the
>> > > > >> > > > >> > > > > > > >> > > partition_epoch
>> > > > >> > > > >> > > > > > > >> > > > > > requires us considerable
>> change to
>> > > > >> > consumer's
>> > > > >> > > > >> public
>> > > > >> > > > >> > > API
>> > > > >> > > > >> > > > > to
>> > > > >> > > > >> > > > > > > take
>> > > > >> > > > >> > > > > > > >> > care
>> > > > >> > > > >> > > > > > > >> > > > of
>> > > > >> > > > >> > > > > > > >> > > > > > the case where user stores
>> offset
>> > > > >> > externally.
>> > > > >> > > > For
>> > > > >> > > > >> > > > example,
>> > > > >> > > > >> > > > > > > >> > > *consumer*.
>> > > > >> > > > >> > > > > > > >> > > > > > *commitSync*(..) would have to
>> > take
>> > > a
>> > > > >> map
>> > > > >> > > whose
>> > > > >> > > > >> > value
>> > > > >> > > > >> > > is
>> > > > >> > > > >> > > > > > > >> <offset,
>> > > > >> > > > >> > > > > > > >> > > > > metadata,
>> > > > >> > > > >> > > > > > > >> > > > > > leader epoch, partition epoch>.
>> > > > >> > > > >> > *consumer*.*seek*(...)
>> > > > >> > > > >> > > > > would
>> > > > >> > > > >> > > > > > > >> also
>> > > > >> > > > >> > > > > > > >> > > need
>> > > > >> > > > >> > > > > > > >> > > > > > leader_epoch and
>> partition_epoch
>> > as
>> > > > >> > > parameter.
>> > > > >> > > > >> > > > Technically
>> > > > >> > > > >> > > > > > we
>> > > > >> > > > >> > > > > > > >> can
>> > > > >> > > > >> > > > > > > >> > > > > probably
>> > > > >> > > > >> > > > > > > >> > > > > > still make it work in a
>> backward
>> > > > >> compatible
>> > > > >> > > > >> manner
>> > > > >> > > > >> > > after
>> > > > >> > > > >> > > > > > > careful
>> > > > >> > > > >> > > > > > > >> > > design
>> > > > >> > > > >> > > > > > > >> > > > > and
>> > > > >> > > > >> > > > > > > >> > > > > > discussion. But these changes
>> can
>> > > make
>> > > > >> the
>> > > > >> > > > >> > consumer's
>> > > > >> > > > >> > > > > > > interface
>> > > > >> > > > >> > > > > > > >> > > > > > unnecessarily complex for more
>> > users
>> > > > >> who do
>> > > > >> > > not
>> > > > >> > > > >> > store
>> > > > >> > > > >> > > > > offset
>> > > > >> > > > >> > > > > > > >> > > > externally.
>> > > > >> > > > >> > > > > > > >> > > > > >
>> > > > >> > > > >> > > > > > > >> > > > > > After thinking more about it,
>> we
>> > can
>> > > > >> > address
>> > > > >> > > > all
>> > > > >> > > > >> > > > problems
>> > > > >> > > > >> > > > > > > >> discussed
>> > > > >> > > > >> > > > > > > >> > > by
>> > > > >> > > > >> > > > > > > >> > > > > only
>> > > > >> > > > >> > > > > > > >> > > > > > using the metadata_epoch
>> without
>> > > > >> > introducing
>> > > > >> > > > >> > > > leader_epoch
>> > > > >> > > > >> > > > > or
>> > > > >> > > > >> > > > > > > the
>> > > > >> > > > >> > > > > > > >> > > > > > partition_epoch. The current
>> KIP
>> > > > >> describes
>> > > > >> > > the
>> > > > >> > > > >> > changes
>> > > > >> > > > >> > > > to
>> > > > >> > > > >> > > > > > the
>> > > > >> > > > >> > > > > > > >> > > consumer
>> > > > >> > > > >> > > > > > > >> > > > > API
>> > > > >> > > > >> > > > > > > >> > > > > > and how the new API can be
>> used if
>> > > > user
>> > > > >> > > stores
>> > > > >> > > > >> > offset
>> > > > >> > > > >> > > > > > > >> externally.
>> > > > >> > > > >> > > > > > > >> > In
>> > > > >> > > > >> > > > > > > >> > > > > order
>> > > > >> > > > >> > > > > > > >> > > > > > to address the scenario you
>> > > described
>> > > > >> > > earlier,
>> > > > >> > > > we
>> > > > >> > > > >> > can
>> > > > >> > > > >> > > > > > include
>> > > > >> > > > >> > > > > > > >> > > > > > metadata_epoch in the
>> > FetchResponse
>> > > > and
>> > > > >> the
>> > > > >> > > > >> > > > > > > LeaderAndIsrRequest.
>> > > > >> > > > >> > > > > > > >> > > > Consumer
>> > > > >> > > > >> > > > > > > >> > > > > > remembers the largest
>> > metadata_epoch
>> > > > >> from
>> > > > >> > all
>> > > > >> > > > the
>> > > > >> > > > >> > > > > > > FetchResponse
>> > > > >> > > > >> > > > > > > >> it
>> > > > >> > > > >> > > > > > > >> > > has
>> > > > >> > > > >> > > > > > > >> > > > > > received. The metadata_epoch
>> > > committed
>> > > > >> with
>> > > > >> > > the
>> > > > >> > > > >> > > offset,
>> > > > >> > > > >> > > > > > either
>> > > > >> > > > >> > > > > > > >> > within
>> > > > >> > > > >> > > > > > > >> > > > or
>> > > > >> > > > >> > > > > > > >> > > > > > outside Kafka, should be the
>> > largest
>> > > > >> > > > >> metadata_epoch
>> > > > >> > > > >> > > > across
>> > > > >> > > > >> > > > > > all
>> > > > >> > > > >> > > > > > > >> > > > > > FetchResponse and
>> MetadataResponse
>> > > > ever
>> > > > >> > > > received
>> > > > >> > > > >> by
>> > > > >> > > > >> > > this
>> > > > >> > > > >> > > > > > > >> consumer.
>> > > > >> > > > >> > > > > > > >> > > > > >
>> > > > >> > > > >> > > > > > > >> > > > > > The drawback of using only the
>> > > > >> > metadata_epoch
>> > > > >> > > > is
>> > > > >> > > > >> > that
>> > > > >> > > > >> > > we
>> > > > >> > > > >> > > > > can
>> > > > >> > > > >> > > > > > > not
>> > > > >> > > > >> > > > > > > >> > > always
>> > > > >> > > > >> > > > > > > >> > > > > do
>> > > > >> > > > >> > > > > > > >> > > > > > the smart offset reset in case
>> of
>> > > > >> unclean
>> > > > >> > > > leader
>> > > > >> > > > >> > > > election
>> > > > >> > > > >> > > > > > > which
>> > > > >> > > > >> > > > > > > >> you
>> > > > >> > > > >> > > > > > > >> > > > > > mentioned earlier. But in most
>> > case,
>> > > > >> > unclean
>> > > > >> > > > >> leader
>> > > > >> > > > >> > > > > election
>> > > > >> > > > >> > > > > > > >> > probably
>> > > > >> > > > >> > > > > > > >> > > > > > happens when consumer is not
>> > > > >> > > > >> rebalancing/restarting.
>> > > > >> > > > >> > > In
>> > > > >> > > > >> > > > > > these
>> > > > >> > > > >> > > > > > > >> > cases,
>> > > > >> > > > >> > > > > > > >> > > > > either
>> > > > >> > > > >> > > > > > > >> > > > > > consumer is not directly
>> affected
>> > by
>> > > > >> > unclean
>> > > > >> > > > >> leader
>> > > > >> > > > >> > > > > election
>> > > > >> > > > >> > > > > > > >> since
>> > > > >> > > > >> > > > > > > >> > it
>> > > > >> > > > >> > > > > > > >> > > > is
>> > > > >> > > > >> > > > > > > >> > > > > > not consuming from the end of
>> the
>> > > log,
>> > > > >> or
>> > > > >> > > > >> consumer
>> > > > >> > > > >> > can
>> > > > >> > > > >> > > > > > derive
>> > > > >> > > > >> > > > > > > >> the
>> > > > >> > > > >> > > > > > > >> > > > > > leader_epoch from the most
>> recent
>> > > > >> message
>> > > > >> > > > >> received
>> > > > >> > > > >> > > > before
>> > > > >> > > > >> > > > > it
>> > > > >> > > > >> > > > > > > >> sees
>> > > > >> > > > >> > > > > > > >> > > > > > OffsetOutOfRangeException. So
>> I am
>> > > not
>> > > > >> sure
>> > > > >> > > it
>> > > > >> > > > is
>> > > > >> > > > >> > > worth
>> > > > >> > > > >> > > > > > adding
>> > > > >> > > > >> > > > > > > >> the
>> > > > >> > > > >> > > > > > > >> > > > > > leader_epoch to consumer API to
>> > > > address
>> > > > >> the
>> > > > >> > > > >> > remaining
>> > > > >> > > > >> > > > > corner
>> > > > >> > > > >> > > > > > > >> case.
>> > > > >> > > > >> > > > > > > >> > > What
>> > > > >> > > > >> > > > > > > >> > > > > do
>> > > > >> > > > >> > > > > > > >> > > > > > you think?
>> > > > >> > > > >> > > > > > > >> > > > > >
>> > > > >> > > > >> > > > > > > >> > > > > > Thanks,
>> > > > >> > > > >> > > > > > > >> > > > > > Dong
>> > > > >> > > > >> > > > > > > >> > > > > >
>> > > > >> > > > >> > > > > > > >> > > > > >
>> > > > >> > > > >> > > > > > > >> > > > > >
>> > > > >> > > > >> > > > > > > >> > > > > > On Tue, Jan 2, 2018 at 6:28 PM,
>> > Jun
>> > > > Rao
>> > > > >> <
>> > > > >> > > > >> > > > jun@confluent.io
>> > > > >> > > > >> > > > > >
>> > > > >> > > > >> > > > > > > >> wrote:
>> > > > >> > > > >> > > > > > > >> > > > > >
>> > > > >> > > > >> > > > > > > >> > > > > > > Hi, Dong,
>> > > > >> > > > >> > > > > > > >> > > > > > >
>> > > > >> > > > >> > > > > > > >> > > > > > > Thanks for the reply.
>> > > > >> > > > >> > > > > > > >> > > > > > >
>> > > > >> > > > >> > > > > > > >> > > > > > > To solve the topic recreation
>> > > issue,
>> > > > >> we
>> > > > >> > > could
>> > > > >> > > > >> use
>> > > > >> > > > >> > > > > either a
>> > > > >> > > > >> > > > > > > >> global
>> > > > >> > > > >> > > > > > > >> > > > > > metadata
>> > > > >> > > > >> > > > > > > >> > > > > > > version or a partition level
>> > > epoch.
>> > > > >> But
>> > > > >> > > > either
>> > > > >> > > > >> one
>> > > > >> > > > >> > > > will
>> > > > >> > > > >> > > > > > be a
>> > > > >> > > > >> > > > > > > >> new
>> > > > >> > > > >> > > > > > > >> > > > > concept,
>> > > > >> > > > >> > > > > > > >> > > > > > > right? To me, the latter
>> seems
>> > > more
>> > > > >> > > natural.
>> > > > >> > > > It
>> > > > >> > > > >> > also
>> > > > >> > > > >> > > > > makes
>> > > > >> > > > >> > > > > > > it
>> > > > >> > > > >> > > > > > > >> > > easier
>> > > > >> > > > >> > > > > > > >> > > > to
>> > > > >> > > > >> > > > > > > >> > > > > > > detect if a consumer's
>> offset is
>> > > > still
>> > > > >> > > valid
>> > > > >> > > > >> > after a
>> > > > >> > > > >> > > > > topic
>> > > > >> > > > >> > > > > > > is
>> > > > >> > > > >> > > > > > > >> > > > > recreated.
>> > > > >> > > > >> > > > > > > >> > > > > > As
>> > > > >> > > > >> > > > > > > >> > > > > > > you pointed out, we don't
>> need
>> > to
>> > > > >> store
>> > > > >> > the
>> > > > >> > > > >> > > partition
>> > > > >> > > > >> > > > > > epoch
>> > > > >> > > > >> > > > > > > in
>> > > > >> > > > >> > > > > > > >> > the
>> > > > >> > > > >> > > > > > > >> > > > > > message.
>> > > > >> > > > >> > > > > > > >> > > > > > > The following is what I am
>> > > thinking.
>> > > > >> > When a
>> > > > >> > > > >> > > partition
>> > > > >> > > > >> > > > is
>> > > > >> > > > >> > > > > > > >> created,
>> > > > >> > > > >> > > > > > > >> > > we
>> > > > >> > > > >> > > > > > > >> > > > > can
>> > > > >> > > > >> > > > > > > >> > > > > > > assign a partition epoch
>> from an
>> > > > >> > > > >> ever-increasing
>> > > > >> > > > >> > > > global
>> > > > >> > > > >> > > > > > > >> counter
>> > > > >> > > > >> > > > > > > >> > and
>> > > > >> > > > >> > > > > > > >> > > > > store
>> > > > >> > > > >> > > > > > > >> > > > > > > it in
>> /brokers/topics/[topic]/
>> > > > >> > > > >> > > > partitions/[partitionId]
>> > > > >> > > > >> > > > > in
>> > > > >> > > > >> > > > > > > ZK.
>> > > > >> > > > >> > > > > > > >> > The
>> > > > >> > > > >> > > > > > > >> > > > > > > partition
>> > > > >> > > > >> > > > > > > >> > > > > > > epoch is propagated to every
>> > > broker.
>> > > > >> The
>> > > > >> > > > >> consumer
>> > > > >> > > > >> > > will
>> > > > >> > > > >> > > > > be
>> > > > >> > > > >> > > > > > > >> > tracking
>> > > > >> > > > >> > > > > > > >> > > a
>> > > > >> > > > >> > > > > > > >> > > > > > tuple
>> > > > >> > > > >> > > > > > > >> > > > > > > of <offset, leader epoch,
>> > > partition
>> > > > >> > epoch>
>> > > > >> > > > for
>> > > > >> > > > >> > > > offsets.
>> > > > >> > > > >> > > > > > If a
>> > > > >> > > > >> > > > > > > >> > topic
>> > > > >> > > > >> > > > > > > >> > > is
>> > > > >> > > > >> > > > > > > >> > > > > > > recreated, it's possible
>> that a
>> > > > >> > consumer's
>> > > > >> > > > >> offset
>> > > > >> > > > >> > > and
>> > > > >> > > > >> > > > > > leader
>> > > > >> > > > >> > > > > > > >> > epoch
>> > > > >> > > > >> > > > > > > >> > > > > still
>> > > > >> > > > >> > > > > > > >> > > > > > > match that in the broker, but
>> > > > >> partition
>> > > > >> > > epoch
>> > > > >> > > > >> > won't
>> > > > >> > > > >> > > > be.
>> > > > >> > > > >> > > > > In
>> > > > >> > > > >> > > > > > > >> this
>> > > > >> > > > >> > > > > > > >> > > case,
>> > > > >> > > > >> > > > > > > >> > > > > we
>> > > > >> > > > >> > > > > > > >> > > > > > > can potentially still treat
>> the
>> > > > >> > consumer's
>> > > > >> > > > >> offset
>> > > > >> > > > >> > as
>> > > > >> > > > >> > > > out
>> > > > >> > > > >> > > > > > of
>> > > > >> > > > >> > > > > > > >> range
>> > > > >> > > > >> > > > > > > >> > > and
>> > > > >> > > > >> > > > > > > >> > > > > > reset
>> > > > >> > > > >> > > > > > > >> > > > > > > the offset based on the
>> offset
>> > > reset
>> > > > >> > policy
>> > > > >> > > > in
>> > > > >> > > > >> the
>> > > > >> > > > >> > > > > > consumer.
>> > > > >> > > > >> > > > > > > >> This
>> > > > >> > > > >> > > > > > > >> > > > seems
>> > > > >> > > > >> > > > > > > >> > > > > > > harder to do with a global
>> > > metadata
>> > > > >> > > version.
>> > > > >> > > > >> > > > > > > >> > > > > > >
>> > > > >> > > > >> > > > > > > >> > > > > > > Jun
>> > > > >> > > > >> > > > > > > >> > > > > > >
>> > > > >> > > > >> > > > > > > >> > > > > > >
>> > > > >> > > > >> > > > > > > >> > > > > > >
>> > > > >> > > > >> > > > > > > >> > > > > > > On Mon, Dec 25, 2017 at 6:56
>> AM,
>> > > > Dong
>> > > > >> > Lin <
>> > > > >> > > > >> > > > > > > >> lindong28@gmail.com>
>> > > > >> > > > >> > > > > > > >> > > > wrote:
>> > > > >> > > > >> > > > > > > >> > > > > > >
>> > > > >> > > > >> > > > > > > >> > > > > > > > Hey Jun,
>> > > > >> > > > >> > > > > > > >> > > > > > > >
>> > > > >> > > > >> > > > > > > >> > > > > > > > This is a very good
>> example.
>> > > After
>> > > > >> > > thinking
>> > > > >> > > > >> > > through
>> > > > >> > > > >> > > > > this
>> > > > >> > > > >> > > > > > > in
>> > > > >> > > > >> > > > > > > >> > > > detail, I
>> > > > >> > > > >> > > > > > > >> > > > > > > agree
>> > > > >> > > > >> > > > > > > >> > > > > > > > that we need to commit
>> offset
>> > > with
>> > > > >> > leader
>> > > > >> > > > >> epoch
>> > > > >> > > > >> > in
>> > > > >> > > > >> > > > > order
>> > > > >> > > > >> > > > > > > to
>> > > > >> > > > >> > > > > > > >> > > address
>> > > > >> > > > >> > > > > > > >> > > > > > this
>> > > > >> > > > >> > > > > > > >> > > > > > > > example.
>> > > > >> > > > >> > > > > > > >> > > > > > > >
>> > > > >> > > > >> > > > > > > >> > > > > > > > I think the remaining
>> question
>> > > is
>> > > > >> how
>> > > > >> > to
>> > > > >> > > > >> address
>> > > > >> > > > >> > > the
>> > > > >> > > > >> > > > > > > >> scenario
>> > > > >> > > > >> > > > > > > >> > > that
>> > > > >> > > > >> > > > > > > >> > > > > the
>> > > > >> > > > >> > > > > > > >> > > > > > > > topic is deleted and
>> > re-created.
>> > > > One
>> > > > >> > > > possible
>> > > > >> > > > >> > > > solution
>> > > > >> > > > >> > > > > > is
>> > > > >> > > > >> > > > > > > to
>> > > > >> > > > >> > > > > > > >> > > commit
>> > > > >> > > > >> > > > > > > >> > > > > > > offset
>> > > > >> > > > >> > > > > > > >> > > > > > > > with both the leader epoch
>> and
>> > > the
>> > > > >> > > metadata
>> > > > >> > > > >> > > version.
>> > > > >> > > > >> > > > > The
>> > > > >> > > > >> > > > > > > >> logic
>> > > > >> > > > >> > > > > > > >> > > and
>> > > > >> > > > >> > > > > > > >> > > > > the
>> > > > >> > > > >> > > > > > > >> > > > > > > > implementation of this
>> > solution
>> > > > does
>> > > > >> > not
>> > > > >> > > > >> > require a
>> > > > >> > > > >> > > > new
>> > > > >> > > > >> > > > > > > >> concept
>> > > > >> > > > >> > > > > > > >> > > > (e.g.
>> > > > >> > > > >> > > > > > > >> > > > > > > > partition epoch) and it
>> does
>> > not
>> > > > >> > require
>> > > > >> > > > any
>> > > > >> > > > >> > > change
>> > > > >> > > > >> > > > to
>> > > > >> > > > >> > > > > > the
>> > > > >> > > > >> > > > > > > >> > > message
>> > > > >> > > > >> > > > > > > >> > > > > > format
>> > > > >> > > > >> > > > > > > >> > > > > > > > or leader epoch. It also
>> > allows
>> > > us
>> > > > >> to
>> > > > >> > > order
>> > > > >> > > > >> the
>> > > > >> > > > >> > > > > metadata
>> > > > >> > > > >> > > > > > > in
>> > > > >> > > > >> > > > > > > >> a
>> > > > >> > > > >> > > > > > > >> > > > > > > > straightforward manner
>> which
>> > may
>> > > > be
>> > > > >> > > useful
>> > > > >> > > > in
>> > > > >> > > > >> > the
>> > > > >> > > > >> > > > > > future.
>> > > > >> > > > >> > > > > > > >> So it
>> > > > >> > > > >> > > > > > > >> > > may
>> > > > >> > > > >> > > > > > > >> > > > > be
>> > > > >> > > > >> > > > > > > >> > > > > > a
>> > > > >> > > > >> > > > > > > >> > > > > > > > better solution than
>> > generating
>> > > a
>> > > > >> > random
>> > > > >> > > > >> > partition
>> > > > >> > > > >> > > > > epoch
>> > > > >> > > > >> > > > > > > >> every
>> > > > >> > > > >> > > > > > > >> > > time
>> > > > >> > > > >> > > > > > > >> > > > > we
>> > > > >> > > > >> > > > > > > >> > > > > > > > create a partition. Does
>> this
>> > > > sound
>> > > > >> > > > >> reasonable?
>> > > > >> > > > >> > > > > > > >> > > > > > > >
>> > > > >> > > > >> > > > > > > >> > > > > > > > Previously one concern with
>> > > using
>> > > > >> the
>> > > > >> > > > >> metadata
>> > > > >> > > > >> > > > version
>> > > > >> > > > >> > > > > > is
>> > > > >> > > > >> > > > > > > >> that
>> > > > >> > > > >> > > > > > > >> > > > > consumer
>> > > > >> > > > >> > > > > > > >> > > > > > > > will be forced to refresh
>> > > metadata
>> > > > >> even
>> > > > >> > > if
>> > > > >> > > > >> > > metadata
>> > > > >> > > > >> > > > > > > version
>> > > > >> > > > >> > > > > > > >> is
>> > > > >> > > > >> > > > > > > >> > > > > > increased
>> > > > >> > > > >> > > > > > > >> > > > > > > > due to topics that the
>> > consumer
>> > > is
>> > > > >> not
>> > > > >> > > > >> > interested
>> > > > >> > > > >> > > > in.
>> > > > >> > > > >> > > > > > Now
>> > > > >> > > > >> > > > > > > I
>> > > > >> > > > >> > > > > > > >> > > > realized
>> > > > >> > > > >> > > > > > > >> > > > > > that
>> > > > >> > > > >> > > > > > > >> > > > > > > > this is probably not a
>> > problem.
>> > > > >> > Currently
>> > > > >> > > > >> client
>> > > > >> > > > >> > > > will
>> > > > >> > > > >> > > > > > > >> refresh
>> > > > >> > > > >> > > > > > > >> > > > > metadata
>> > > > >> > > > >> > > > > > > >> > > > > > > > either due to
>> > > > >> InvalidMetadataException
>> > > > >> > in
>> > > > >> > > > the
>> > > > >> > > > >> > > > response
>> > > > >> > > > >> > > > > > > from
>> > > > >> > > > >> > > > > > > >> > > broker
>> > > > >> > > > >> > > > > > > >> > > > or
>> > > > >> > > > >> > > > > > > >> > > > > > due
>> > > > >> > > > >> > > > > > > >> > > > > > > > to metadata expiry. The
>> > addition
>> > > > of
>> > > > >> the
>> > > > >> > > > >> metadata
>> > > > >> > > > >> > > > > version
>> > > > >> > > > >> > > > > > > >> should
>> > > > >> > > > >> > > > > > > >> > > > > > increase
>> > > > >> > > > >> > > > > > > >> > > > > > > > the overhead of metadata
>> > refresh
>> > > > >> caused
>> > > > >> > > by
>> > > > >> > > > >> > > > > > > >> > > > InvalidMetadataException.
>> > > > >> > > > >> > > > > > > >> > > > > If
>> > > > >> > > > >> > > > > > > >> > > > > > > > client refresh metadata
>> due to
>> > > > >> expiry
>> > > > >> > and
>> > > > >> > > > it
>> > > > >> > > > >> > > > receives
>> > > > >> > > > >> > > > > a
>> > > > >> > > > >> > > > > > > >> > metadata
>> > > > >> > > > >> > > > > > > >> > > > > whose
>> > > > >> > > > >> > > > > > > >> > > > > > > > version is lower than the
>> > > current
>> > > > >> > > metadata
>> > > > >> > > > >> > > version,
>> > > > >> > > > >> > > > we
>> > > > >> > > > >> > > > > > can
>> > > > >> > > > >> > > > > > > >> > reject
>> > > > >> > > > >> > > > > > > >> > > > the
>> > > > >> > > > >> > > > > > > >> > > > > > > > metadata but still reset
>> the
>> > > > >> metadata
>> > > > >> > > age,
>> > > > >> > > > >> which
>> > > > >> > > > >> > > > > > > essentially
>> > > > >> > > > >> > > > > > > >> > keep
>> > > > >> > > > >> > > > > > > >> > > > the
>> > > > >> > > > >> > > > > > > >> > > > > > > > existing behavior in the
>> > client.
>> > > > >> > > > >> > > > > > > >> > > > > > > >
>> > > > >> > > > >> > > > > > > >> > > > > > > > Thanks much,
>> > > > >> > > > >> > > > > > > >> > > > > > > > Dong
>> > > > >> > > > >> > > > > > > >> > > > > > > >
>> > > > >> > > > >> > > > > > > >> > > > > > >
>> > > > >> > > > >> > > > > > > >> > > > > >
>> > > > >> > > > >> > > > > > > >> > > > >
>> > > > >> > > > >> > > > > > > >> > > >
>> > > > >> > > > >> > > > > > > >> > >
>> > > > >> > > > >> > > > > > > >> >
>> > > > >> > > > >> > > > > > > >>
>> > > > >> > > > >> > > > > > > >
>> > > > >> > > > >> > > > > > > >
>> > > > >> > > > >> > > > > > >
>> > > > >> > > > >> > > > > >
>> > > > >> > > > >> > > > >
>> > > > >> > > > >> > > >
>> > > > >> > > > >> > >
>> > > > >> > > > >> >
>> > > > >> > > > >>
>> > > > >> > > > >
>> > > > >> > > > >
>> > > > >> > > >
>> > > > >> > >
>> > > > >> >
>> > > > >>
>> > > > >
>> > > > >
>> > > >
>> > >
>> >
>>
>
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Dong Lin <li...@gmail.com>.
Hey Jun,

I think we can probably have a static method in Util class to decode the
byte[]. Both KafkaConsumer implementation and the user application will be
able to decode the byte array and log its content for debug purpose. So it
seems that we can still print the information we want. It is just not
explicitly exposed in the consumer interface. Would this address the
problem here?

Yeah we can include OffsetEpoch in AdminClient. This can be added in
KIP-222? Is there something you would like me to add in this KIP?

Thanks!
Dong

On Fri, Jan 19, 2018 at 3:00 PM, Jun Rao <ju...@confluent.io> wrote:

> Hi, Dong,
>
> The issue with using just byte[] for OffsetEpoch is that it won't be
> printable, which makes debugging harder.
>
> Also, KIP-222 proposes a listGroupOffset() method in AdminClient. If that
> gets adopted before this KIP, we probably want to include OffsetEpoch in
> the AdminClient too.
>
> Thanks,
>
> Jun
>
>
> On Thu, Jan 18, 2018 at 6:30 PM, Dong Lin <li...@gmail.com> wrote:
>
> > Hey Jun,
> >
> > I agree. I have updated the KIP to remove the class OffetEpoch and
> replace
> > OffsetEpoch with byte[] in APIs that use it. Can you see if it looks
> good?
> >
> > Thanks!
> > Dong
> >
> > On Thu, Jan 18, 2018 at 6:07 PM, Jun Rao <ju...@confluent.io> wrote:
> >
> > > Hi, Dong,
> > >
> > > Thanks for the updated KIP. It looks good to me now. The only thing is
> > > for OffsetEpoch.
> > > If we expose the individual fields in the class, we probably don't need
> > the
> > > encode/decode methods. If we want to hide the details of OffsetEpoch,
> we
> > > probably don't want expose the individual fields.
> > >
> > > Jun
> > >
> > > On Wed, Jan 17, 2018 at 10:10 AM, Dong Lin <li...@gmail.com>
> wrote:
> > >
> > > > Thinking about point 61 more, I realize that the async zookeeper read
> > may
> > > > make it less of an issue for controller to read more zookeeper nodes.
> > > > Writing partition_epoch in the per-partition znode makes it simpler
> to
> > > > handle the broker failure between zookeeper writes for a topic
> > creation.
> > > I
> > > > have updated the KIP to use the suggested approach.
> > > >
> > > >
> > > > On Wed, Jan 17, 2018 at 9:57 AM, Dong Lin <li...@gmail.com>
> wrote:
> > > >
> > > > > Hey Jun,
> > > > >
> > > > > Thanks much for the comments. Please see my comments inline.
> > > > >
> > > > > On Tue, Jan 16, 2018 at 4:38 PM, Jun Rao <ju...@confluent.io> wrote:
> > > > >
> > > > >> Hi, Dong,
> > > > >>
> > > > >> Thanks for the updated KIP. Looks good to me overall. Just a few
> > minor
> > > > >> comments.
> > > > >>
> > > > >> 60. OffsetAndMetadata positionAndOffsetEpoch(TopicPartition
> > > partition):
> > > > >> It
> > > > >> seems that there is no need to return metadata. We probably want
> to
> > > > return
> > > > >> sth like OffsetAndEpoch.
> > > > >>
> > > > >
> > > > > Previously I think we may want to re-use the existing class to keep
> > our
> > > > > consumer interface simpler. I have updated the KIP to add class
> > > > > OffsetAndOffsetEpoch. I didn't use OffsetAndEpoch because user may
> > > > confuse
> > > > > this name with OffsetEpoch. Does this sound OK?
> > > > >
> > > > >
> > > > >>
> > > > >> 61. Should we store partition_epoch in
> > > > >> /brokers/topics/[topic]/partitions/[partitionId] in ZK?
> > > > >>
> > > > >
> > > > > I have considered this. I think the advantage of adding the
> > > > > partition->partition_epoch map in the existing
> > > > > znode /brokers/topics/[topic]/partitions is that controller only
> > needs
> > > > to
> > > > > read one znode per topic to gets its partition_epoch information.
> > > > Otherwise
> > > > > controller may need to read one extra znode per partition to get
> the
> > > same
> > > > > information.
> > > > >
> > > > > When we delete partition or expand partition of a topic, someone
> > needs
> > > to
> > > > > modify partition->partition_epoch map in znode
> > > > > /brokers/topics/[topic]/partitions. This may seem a bit more
> > > complicated
> > > > > than simply adding or deleting znode /brokers/topics/[topic]/
> > > > partitions/[partitionId].
> > > > > But the complexity is probably similar to the existing operation of
> > > > > modifying the partition->replica_list mapping in znode
> > > > > /brokers/topics/[topic]. So not sure it is better to store the
> > > > > partition_epoch in /brokers/topics/[topic]/
> partitions/[partitionId].
> > > > What
> > > > > do you think?
> > > > >
> > > > >
> > > > >>
> > > > >> 62. For checking outdated metadata in the client, we probably want
> > to
> > > > add
> > > > >> when max_partition_epoch will be used.
> > > > >>
> > > > >
> > > > > The max_partition_epoch is used in the Proposed Changes -> Client's
> > > > > metadata refresh section to determine whether a metadata is
> outdated.
> > > And
> > > > > this formula is referenced and re-used in other sections to
> determine
> > > > > whether a metadata is outdated. Does this formula look OK?
> > > > >
> > > > >
> > > > >>
> > > > >> 63. "The leader_epoch should be the largest leader_epoch of
> messages
> > > > whose
> > > > >> offset < the commit offset. If no message has been consumed since
> > > > consumer
> > > > >> initialization, the leader_epoch from seek(...) or
> > OffsetFetchResponse
> > > > >> should be used. The partition_epoch should be read from the last
> > > > >> FetchResponse corresponding to the given partition and commit
> > offset.
> > > ":
> > > > >> leader_epoch and partition_epoch are associated with an offset.
> So,
> > if
> > > > no
> > > > >> message is consumed, there is no offset and therefore there is no
> > need
> > > > to
> > > > >> read leader_epoch and partition_epoch. Also, the leader_epoch
> > > associated
> > > > >> with the offset should just come from the messages returned in the
> > > fetch
> > > > >> response.
> > > > >>
> > > > >
> > > > > I am thinking that, if user calls seek(..) and commitSync(...)
> > without
> > > > > consuming any messages, we should re-use the leader_epoch and
> > > > > partition_epoch provided by the seek(...) in the
> OffsetCommitRequest.
> > > And
> > > > > if messages have been successfully consumed, then leader_epoch will
> > > come
> > > > > from the messages returned in the fetch response. The condition
> > > "messages
> > > > > whose offset < the commit offset" is needed to take care of the log
> > > > > compacted topic which may have offset gap due to log cleaning.
> > > > >
> > > > > Did I miss something here? Or should I rephrase the paragraph to
> make
> > > it
> > > > > less confusing?
> > > > >
> > > > >
> > > > >> 64. Could you include the public methods in the OffsetEpoch class?
> > > > >>
> > > > >
> > > > > I mistakenly deleted the definition of OffsetEpoch class from the
> > KIP.
> > > I
> > > > > just added it back with the public methods. Could you take another
> > > look?
> > > > >
> > > > >
> > > > >>
> > > > >> Jun
> > > > >>
> > > > >>
> > > > >> On Thu, Jan 11, 2018 at 5:43 PM, Dong Lin <li...@gmail.com>
> > > wrote:
> > > > >>
> > > > >> > Hey Jun,
> > > > >> >
> > > > >> > Thanks much. I agree that we can not rely on committed offsets
> to
> > be
> > > > >> always
> > > > >> > deleted when we delete topic. So it is necessary to use a
> > > > per-partition
> > > > >> > epoch that does not change unless this partition is deleted. I
> > also
> > > > >> agree
> > > > >> > that it is very nice to be able to uniquely identify a message
> > with
> > > > >> > (offset, leader_epoch, partition_epoch) in face of potential
> topic
> > > > >> deletion
> > > > >> > and unclean leader election.
> > > > >> >
> > > > >> > I agree with all your comments. And I have updated the KIP based
> > on
> > > > our
> > > > >> > latest discussion. In addition, I added
> > > InvalidPartitionEpochException
> > > > >> > which will be thrown by consumer.poll() if the partition_epoch
> > > > >> associated
> > > > >> > with the partition, which can be given to consumer using
> > seek(...),
> > > is
> > > > >> > different from the partition_epoch in the FetchResponse.
> > > > >> >
> > > > >> > Can you take another look at the latest KIP?
> > > > >> >
> > > > >> > Thanks!
> > > > >> > Dong
> > > > >> >
> > > > >> >
> > > > >> >
> > > > >> > On Wed, Jan 10, 2018 at 2:24 PM, Jun Rao <ju...@confluent.io>
> > wrote:
> > > > >> >
> > > > >> > > Hi, Dong,
> > > > >> > >
> > > > >> > > My replies are the following.
> > > > >> > >
> > > > >> > > 60. What you described could also work. The drawback is that
> we
> > > will
> > > > >> be
> > > > >> > > unnecessarily changing the partition epoch when a partition
> > hasn't
> > > > >> really
> > > > >> > > changed. I was imagining that the partition epoch will be
> stored
> > > in
> > > > >> > > /brokers/topics/[topic]/partitions/[partitionId], instead of
> at
> > > the
> > > > >> > topic
> > > > >> > > level. So, not sure if ZK size limit is an issue.
> > > > >> > >
> > > > >> > > 61, 62 and 65. To me, the offset + offset_epoch is a unique
> > > > identifier
> > > > >> > for
> > > > >> > > a message. So, if a message hasn't changed, the offset and the
> > > > >> associated
> > > > >> > > offset_epoch ideally should remain the same (it will be kind
> of
> > > > weird
> > > > >> if
> > > > >> > > two consumer apps save the offset on the same message, but the
> > > > >> > offset_epoch
> > > > >> > > are different). partition_epoch + leader_epoch give us that.
> > > > >> > global_epoch +
> > > > >> > > leader_epoch don't. If we use this approach, we can solve not
> > only
> > > > the
> > > > >> > > problem that you have identified, but also other problems when
> > > there
> > > > >> is
> > > > >> > > data loss or topic re-creation more reliably. For example, in
> > the
> > > > >> future,
> > > > >> > > if we include the partition_epoch and leader_epoch in the
> fetch
> > > > >> request,
> > > > >> > > the server can do a more reliable check of whether that offset
> > is
> > > > >> valid
> > > > >> > or
> > > > >> > > not. I am not sure that we can rely upon all external offsets
> to
> > > be
> > > > >> > removed
> > > > >> > > on topic deletion. For example, a topic may be deleted by an
> > admin
> > > > who
> > > > >> > may
> > > > >> > > not know all the applications.
> > > > >> > >
> > > > >> > > If we agree on the above, the second question is then how to
> > > > reliably
> > > > >> > > propagate the partition_epoch and the leader_epoch to the
> > consumer
> > > > >> when
> > > > >> > > there are leader or partition changes. The leader_epoch comes
> > from
> > > > the
> > > > >> > > message, which is reliable. So, I was suggesting that when we
> > > store
> > > > an
> > > > >> > > offset, we can just store the leader_epoch from the message
> set
> > > > >> > containing
> > > > >> > > that offset. Similarly, I was thinking that if the
> > partition_epoch
> > > > is
> > > > >> in
> > > > >> > > the fetch response, we can propagate partition_epoch reliably
> > > where
> > > > is
> > > > >> > > partition_epoch change.
> > > > >> > >
> > > > >> > > 63. My point is that once a leader is producing a message in
> the
> > > new
> > > > >> > > partition_epoch, ideally, we should associate the new offsets
> > with
> > > > the
> > > > >> > new
> > > > >> > > partition_epoch. Otherwise, the offset_epoch won't be the
> > correct
> > > > >> unique
> > > > >> > > identifier (useful for solving other problems mentioned
> above).
> > I
> > > > was
> > > > >> > > originally thinking that the leader will include the
> > > partition_epoch
> > > > >> in
> > > > >> > the
> > > > >> > > metadata cache in the fetch response. It's just that right
> now,
> > > > >> metadata
> > > > >> > > cache is updated on UpdateMetadataRequest, which typically
> > happens
> > > > >> after
> > > > >> > > the LeaderAndIsrRequest. Another approach is for the leader to
> > > cache
> > > > >> the
> > > > >> > > partition_epoch in the Partition object and return that
> (instead
> > > of
> > > > >> the
> > > > >> > one
> > > > >> > > in metadata cache) in the fetch response.
> > > > >> > >
> > > > >> > > 65. It seems to me that the global_epoch and the
> partition_epoch
> > > > have
> > > > >> > > different purposes. A partition_epoch has the benefit that it
> > (1)
> > > > can
> > > > >> be
> > > > >> > > used to form a unique identifier for a message and (2) can be
> > used
> > > > to
> > > > >> > > solve other
> > > > >> > > corner case problems in the future. I am not sure having just
> a
> > > > >> > > global_epoch can achieve these. global_epoch is useful to
> > > determine
> > > > >> which
> > > > >> > > version of the metadata is newer, especially with topic
> > deletion.
> > > > >> > >
> > > > >> > > Thanks,
> > > > >> > >
> > > > >> > > Jun
> > > > >> > >
> > > > >> > > On Tue, Jan 9, 2018 at 11:34 PM, Dong Lin <
> lindong28@gmail.com>
> > > > >> wrote:
> > > > >> > >
> > > > >> > > > Regarding the use of the global epoch in 65), it is very
> > similar
> > > > to
> > > > >> the
> > > > >> > > > proposal of the metadata_epoch we discussed earlier. The
> main
> > > > >> > difference
> > > > >> > > is
> > > > >> > > > that this epoch is incremented when we create/expand/delete
> > > topic
> > > > >> and
> > > > >> > > does
> > > > >> > > > not change when controller re-send metadata.
> > > > >> > > >
> > > > >> > > > I looked at our previous discussion. It seems that we prefer
> > > > >> > > > partition_epoch over the metadata_epoch because 1) we prefer
> > not
> > > > to
> > > > >> > have
> > > > >> > > an
> > > > >> > > > ever growing metadata_epoch and 2) we can reset offset
> better
> > > when
> > > > >> > topic
> > > > >> > > is
> > > > >> > > > re-created. The use of global topic_epoch avoids the
> drawback
> > of
> > > > an
> > > > >> > ever
> > > > >> > > > quickly ever growing metadata_epoch. Though the global epoch
> > > does
> > > > >> not
> > > > >> > > allow
> > > > >> > > > us to recognize the invalid offset committed before the
> topic
> > > > >> > > re-creation,
> > > > >> > > > we can probably just delete the offset when we delete a
> topic.
> > > > Thus
> > > > >> I
> > > > >> > am
> > > > >> > > > not very sure whether it is still worthwhile to have a
> > > > per-partition
> > > > >> > > > partition_epoch if the metadata already has the global
> epoch.
> > > > >> > > >
> > > > >> > > >
> > > > >> > > > On Tue, Jan 9, 2018 at 6:58 PM, Dong Lin <
> lindong28@gmail.com
> > >
> > > > >> wrote:
> > > > >> > > >
> > > > >> > > > > Hey Jun,
> > > > >> > > > >
> > > > >> > > > > Thanks so much. These comments very useful. Please see
> below
> > > my
> > > > >> > > comments.
> > > > >> > > > >
> > > > >> > > > > On Mon, Jan 8, 2018 at 5:52 PM, Jun Rao <jun@confluent.io
> >
> > > > wrote:
> > > > >> > > > >
> > > > >> > > > >> Hi, Dong,
> > > > >> > > > >>
> > > > >> > > > >> Thanks for the updated KIP. A few more comments.
> > > > >> > > > >>
> > > > >> > > > >> 60. Perhaps having a partition epoch is more flexible
> since
> > > in
> > > > >> the
> > > > >> > > > future,
> > > > >> > > > >> we may support deleting a partition as well.
> > > > >> > > > >>
> > > > >> > > > >
> > > > >> > > > > Yeah I have considered this. I think we can probably still
> > > > support
> > > > >> > > > > deleting a partition by using the topic_epoch -- when
> > > partition
> > > > >> of a
> > > > >> > > > topic
> > > > >> > > > > is deleted or created, epoch of all partitions of this
> topic
> > > > will
> > > > >> be
> > > > >> > > > > incremented by 1. Therefore, if that partition is
> re-created
> > > > >> later,
> > > > >> > the
> > > > >> > > > > epoch of that partition will still be larger than its
> epoch
> > > > before
> > > > >> > the
> > > > >> > > > > deletion, which still allows the client to order the
> > metadata
> > > > for
> > > > >> the
> > > > >> > > > > purpose of this KIP. Does this sound reasonable?
> > > > >> > > > >
> > > > >> > > > > The advantage of using topic_epoch instead of
> > partition_epoch
> > > is
> > > > >> that
> > > > >> > > the
> > > > >> > > > > size of the /brokers/topics/[topic] znode and
> > request/response
> > > > >> size
> > > > >> > can
> > > > >> > > > be
> > > > >> > > > > smaller. We have a limit on the maximum size of znode
> > > (typically
> > > > >> > 1MB).
> > > > >> > > > Use
> > > > >> > > > > partition epoch can effectively reduce the number of
> > > partitions
> > > > >> that
> > > > >> > > can
> > > > >> > > > be
> > > > >> > > > > described by the /brokers/topics/[topic] znode.
> > > > >> > > > >
> > > > >> > > > > One use-case of partition_epoch for client to detect that
> > the
> > > > >> > committed
> > > > >> > > > > offset, either from kafka offset topic or from the
> external
> > > > store
> > > > >> is
> > > > >> > > > > invalid after partition deletion and re-creation. However,
> > it
> > > > >> seems
> > > > >> > > that
> > > > >> > > > we
> > > > >> > > > > can also address this use-case with other approaches. For
> > > > example,
> > > > >> > when
> > > > >> > > > > AdminClient deletes partitions, it can also delete the
> > > committed
> > > > >> > > offsets
> > > > >> > > > > for those partitions from the offset topic. If user stores
> > > > offset
> > > > >> > > > > externally, it might make sense for user to similarly
> remove
> > > > >> offsets
> > > > >> > of
> > > > >> > > > > related partitions after these partitions are deleted. So
> I
> > am
> > > > not
> > > > >> > sure
> > > > >> > > > > that we should use partition_epoch in this KIP.
> > > > >> > > > >
> > > > >> > > > >
> > > > >> > > > >>
> > > > >> > > > >> 61. It seems that the leader epoch returned in the
> > position()
> > > > >> call
> > > > >> > > > should
> > > > >> > > > >> the the leader epoch returned in the fetch response, not
> > the
> > > > one
> > > > >> in
> > > > >> > > the
> > > > >> > > > >> metadata cache of the client.
> > > > >> > > > >
> > > > >> > > > >
> > > > >> > > > > I think this is a good idea. Just to double check, this
> > change
> > > > >> does
> > > > >> > not
> > > > >> > > > > affect the correctness or performance of this KIP. But it
> > can
> > > be
> > > > >> > useful
> > > > >> > > > if
> > > > >> > > > > we want to use the leader_epoch to better handle the
> offset
> > > rest
> > > > >> in
> > > > >> > > case
> > > > >> > > > of
> > > > >> > > > > unclean leader election, which is listed in the future
> work.
> > > Is
> > > > >> this
> > > > >> > > > > understanding correct?
> > > > >> > > > >
> > > > >> > > > > I have updated the KIP to specify that the leader_epoch
> > > returned
> > > > >> by
> > > > >> > > > > position() should be the largest leader_epoch of those
> > already
> > > > >> > consumed
> > > > >> > > > > messages whose offset < position. If no message has been
> > > > consumed
> > > > >> > since
> > > > >> > > > > consumer initialization, the leader_epoch from seek() or
> > > > >> > > > > OffsetFetchResponse should be used. The offset included in
> > the
> > > > >> > > > > OffsetCommitRequest will also be determined in the similar
> > > > manner.
> > > > >> > > > >
> > > > >> > > > >
> > > > >> > > > >>
> > > > >> > > > >> 62. I am wondering if we should return the partition
> epoch
> > in
> > > > the
> > > > >> > > fetch
> > > > >> > > > >> response as well. In the current proposal, if a topic is
> > > > >> recreated
> > > > >> > and
> > > > >> > > > the
> > > > >> > > > >> new leader is on the same broker as the old one, there is
> > > > >> nothing to
> > > > >> > > > force
> > > > >> > > > >> the metadata refresh in the client. So, the client may
> > still
> > > > >> > associate
> > > > >> > > > the
> > > > >> > > > >> offset with the old partition epoch.
> > > > >> > > > >>
> > > > >> > > > >
> > > > >> > > > > Could you help me understand the problem if a client
> > > associates
> > > > >> old
> > > > >> > > > > partition_epoch (or the topic_epoch as of the current KIP)
> > > with
> > > > >> the
> > > > >> > > > offset?
> > > > >> > > > > The main purpose of the topic_epoch is to be able to drop
> > > > >> > leader_epoch
> > > > >> > > > to 0
> > > > >> > > > > after a partition is deleted and re-created. I guess you
> may
> > > be
> > > > >> > > thinking
> > > > >> > > > > about using the partition_epoch to detect that the
> committed
> > > > >> offset
> > > > >> > is
> > > > >> > > > > invalid? In that case, I am wondering if the alternative
> > > > approach
> > > > >> > > > described
> > > > >> > > > > in 60) would be reasonable.
> > > > >> > > > >
> > > > >> > > > >
> > > > >> > > > >>
> > > > >> > > > >> 63. There is some subtle coordination between the
> > > > >> > LeaderAndIsrRequest
> > > > >> > > > and
> > > > >> > > > >> UpdateMetadataRequest. Currently, when a leader changes,
> > the
> > > > >> > > controller
> > > > >> > > > >> first sends the LeaderAndIsrRequest to the assigned
> > replicas
> > > > and
> > > > >> the
> > > > >> > > > >> UpdateMetadataRequest to every broker. So, there could
> be a
> > > > small
> > > > >> > > window
> > > > >> > > > >> when the leader already receives the new partition epoch
> in
> > > the
> > > > >> > > > >> LeaderAndIsrRequest, but the metadata cache in the broker
> > > > hasn't
> > > > >> > been
> > > > >> > > > >> updated with the latest partition epoch. Not sure what's
> > the
> > > > best
> > > > >> > way
> > > > >> > > to
> > > > >> > > > >> address this issue. Perhaps we can update the metadata
> > cache
> > > on
> > > > >> the
> > > > >> > > > broker
> > > > >> > > > >> with both LeaderAndIsrRequest and UpdateMetadataRequest.
> > The
> > > > >> > challenge
> > > > >> > > > is
> > > > >> > > > >> that the two have slightly different data. For example,
> > only
> > > > the
> > > > >> > > latter
> > > > >> > > > >> has
> > > > >> > > > >> all endpoints.
> > > > >> > > > >>
> > > > >> > > > >
> > > > >> > > > > I am not sure whether this is a problem. Could you
> explain a
> > > bit
> > > > >> more
> > > > >> > > > what
> > > > >> > > > > specific problem this small window can cause?
> > > > >> > > > >
> > > > >> > > > > Since client can fetch metadata from any broker in the
> > > cluster,
> > > > >> and
> > > > >> > > given
> > > > >> > > > > that different brokers receive request (e.g.
> > > LeaderAndIsrRequest
> > > > >> and
> > > > >> > > > > UpdateMetadataRequest) in arbitrary order, the metadata
> > > received
> > > > >> by
> > > > >> > > > client
> > > > >> > > > > can be in arbitrary order (either newer or older) compared
> > to
> > > > the
> > > > >> > > > broker's
> > > > >> > > > > leadership state even if a given broker receives
> > > > >> LeaderAndIsrRequest
> > > > >> > > and
> > > > >> > > > > UpdateMetadataRequest simultaneously. So I am not sure it
> is
> > > > >> useful
> > > > >> > to
> > > > >> > > > > update broker's cache with LeaderAndIsrRequest.
> > > > >> > > > >
> > > > >> > > > >
> > > > >> > > > >> 64. The enforcement of leader epoch in Offset commit: We
> > > allow
> > > > a
> > > > >> > > > consumer
> > > > >> > > > >> to set an arbitrary offset. So it's possible for offsets
> or
> > > > >> leader
> > > > >> > > epoch
> > > > >> > > > >> to
> > > > >> > > > >> go backwards. I am not sure if we could always enforce
> that
> > > the
> > > > >> > leader
> > > > >> > > > >> epoch only goes up on the broker.
> > > > >> > > > >>
> > > > >> > > > >
> > > > >> > > > > Sure. I have removed this check from the KIP.
> > > > >> > > > >
> > > > >> > > > > BTW, we can probably still ensure that the leader_epoch
> > always
> > > > >> > increase
> > > > >> > > > if
> > > > >> > > > > the leader_epoch used with offset commit is the
> > > max(leader_epoch
> > > > >> of
> > > > >> > the
> > > > >> > > > > message with offset = the committed offset - 1, the
> largest
> > > > known
> > > > >> > > > > leader_epoch from the metadata). But I don't have a good
> > > > use-case
> > > > >> for
> > > > >> > > > this
> > > > >> > > > > alternative definition. So I choose the keep the KIP
> simple
> > by
> > > > >> > > requiring
> > > > >> > > > > leader_epoch to always increase.
> > > > >> > > > >
> > > > >> > > > >
> > > > >> > > > >> 65. Good point on handling missing partition epoch due to
> > > topic
> > > > >> > > > deletion.
> > > > >> > > > >> Another potential way to address this is to additionally
> > > > >> propagate
> > > > >> > the
> > > > >> > > > >> global partition epoch to brokers and the clients. This
> > way,
> > > > >> when a
> > > > >> > > > >> partition epoch is missing, we can use the global
> partition
> > > > >> epoch to
> > > > >> > > > >> reason
> > > > >> > > > >> about which metadata is more recent.
> > > > >> > > > >>
> > > > >> > > > >
> > > > >> > > > > This is a great idea. The global epoch can be used to
> order
> > > the
> > > > >> > > metadata
> > > > >> > > > > and help us recognize the more recent metadata if a topic
> > (or
> > > > >> > > partition)
> > > > >> > > > is
> > > > >> > > > > deleted and re-created.
> > > > >> > > > >
> > > > >> > > > > Actually, it seems we only need to propagate the global
> > epoch
> > > to
> > > > >> > > brokers
> > > > >> > > > > and clients without propagating this epoch on a per-topic
> or
> > > > >> > > > per-partition
> > > > >> > > > > basic. Doing so would simply interface changes made this
> > KIP.
> > > > Does
> > > > >> > this
> > > > >> > > > > approach sound reasonable?
> > > > >> > > > >
> > > > >> > > > >
> > > > >> > > > >> 66. A client may also get an offset by time using the
> > > > >> > offsetForTimes()
> > > > >> > > > >> api.
> > > > >> > > > >> So, we probably want to include offsetInternalMetadata in
> > > > >> > > > >> OffsetAndTimestamp
> > > > >> > > > >> as well.
> > > > >> > > > >>
> > > > >> > > > >
> > > > >> > > > > You are right. This probably also requires us to change
> the
> > > > >> > > > > ListOffsetRequest as well. I will update the KIP after we
> > > agree
> > > > on
> > > > >> > the
> > > > >> > > > > solution for 65).
> > > > >> > > > >
> > > > >> > > > >
> > > > >> > > > >>
> > > > >> > > > >> 67. InteralMetadata can be a bit confusing with the
> > metadata
> > > > >> field
> > > > >> > > > already
> > > > >> > > > >> there. Perhaps we can just call it OffsetEpoch. It might
> be
> > > > >> useful
> > > > >> > to
> > > > >> > > > make
> > > > >> > > > >> OffsetEpoch printable at least for debugging purpose.
> Once
> > > you
> > > > do
> > > > >> > > that,
> > > > >> > > > we
> > > > >> > > > >> are already exposing the internal fields. So, not sure if
> > > it's
> > > > >> worth
> > > > >> > > > >> hiding
> > > > >> > > > >> them. If we do want to hide them, perhaps we can have sth
> > > like
> > > > >> the
> > > > >> > > > >> following. The binary encoding is probably more efficient
> > > than
> > > > >> JSON
> > > > >> > > for
> > > > >> > > > >> external storage.
> > > > >> > > > >>
> > > > >> > > > >> OffsetEpoch {
> > > > >> > > > >>  static OffsetEpoch decode(byte[]);
> > > > >> > > > >>
> > > > >> > > > >>   public byte[] encode();
> > > > >> > > > >>
> > > > >> > > > >>   public String toString();
> > > > >> > > > >> }
> > > > >> > > > >>
> > > > >> > > > >
> > > > >> > > > > Thanks much. I like this solution. I have updated the KIP
> > > > >> > accordingly.
> > > > >> > > > >
> > > > >> > > > >
> > > > >> > > > >
> > > > >> > > > >>
> > > > >> > > > >> Jun
> > > > >> > > > >>
> > > > >> > > > >> On Mon, Jan 8, 2018 at 4:22 PM, Dong Lin <
> > > lindong28@gmail.com>
> > > > >> > wrote:
> > > > >> > > > >>
> > > > >> > > > >> > Hey Jason,
> > > > >> > > > >> >
> > > > >> > > > >> > Certainly. This sounds good. I have updated the KIP to
> > > > clarity
> > > > >> > that
> > > > >> > > > the
> > > > >> > > > >> > global epoch will be incremented by 1 each time a topic
> > is
> > > > >> > deleted.
> > > > >> > > > >> >
> > > > >> > > > >> > Thanks,
> > > > >> > > > >> > Dong
> > > > >> > > > >> >
> > > > >> > > > >> > On Mon, Jan 8, 2018 at 4:09 PM, Jason Gustafson <
> > > > >> > jason@confluent.io
> > > > >> > > >
> > > > >> > > > >> > wrote:
> > > > >> > > > >> >
> > > > >> > > > >> > > Hi Dong,
> > > > >> > > > >> > >
> > > > >> > > > >> > >
> > > > >> > > > >> > > I think your approach will allow user to distinguish
> > > > between
> > > > >> the
> > > > >> > > > >> metadata
> > > > >> > > > >> > > > before and after the topic deletion. I also agree
> > that
> > > > this
> > > > >> > can
> > > > >> > > be
> > > > >> > > > >> > > > potentially be useful to user. I am just not very
> > sure
> > > > >> whether
> > > > >> > > we
> > > > >> > > > >> > already
> > > > >> > > > >> > > > have a good use-case to make the additional
> > complexity
> > > > >> > > worthwhile.
> > > > >> > > > >> It
> > > > >> > > > >> > > seems
> > > > >> > > > >> > > > that this feature is kind of independent of the
> main
> > > > >> problem
> > > > >> > of
> > > > >> > > > this
> > > > >> > > > >> > KIP.
> > > > >> > > > >> > > > Could we add this as a future work?
> > > > >> > > > >> > >
> > > > >> > > > >> > >
> > > > >> > > > >> > > Do you think it's fair if we bump the topic epoch on
> > > > deletion
> > > > >> > and
> > > > >> > > > >> leave
> > > > >> > > > >> > > propagation of the epoch for deleted topics for
> future
> > > > work?
> > > > >> I
> > > > >> > > don't
> > > > >> > > > >> > think
> > > > >> > > > >> > > this adds much complexity and it makes the behavior
> > > > >> consistent:
> > > > >> > > > every
> > > > >> > > > >> > topic
> > > > >> > > > >> > > mutation results in an epoch bump.
> > > > >> > > > >> > >
> > > > >> > > > >> > > Thanks,
> > > > >> > > > >> > > Jason
> > > > >> > > > >> > >
> > > > >> > > > >> > > On Mon, Jan 8, 2018 at 3:14 PM, Dong Lin <
> > > > >> lindong28@gmail.com>
> > > > >> > > > wrote:
> > > > >> > > > >> > >
> > > > >> > > > >> > > > Hey Ismael,
> > > > >> > > > >> > > >
> > > > >> > > > >> > > > I guess we actually need user to see this field so
> > that
> > > > >> user
> > > > >> > can
> > > > >> > > > >> store
> > > > >> > > > >> > > this
> > > > >> > > > >> > > > value in the external store together with the
> offset.
> > > We
> > > > >> just
> > > > >> > > > prefer
> > > > >> > > > >> > the
> > > > >> > > > >> > > > value to be opaque to discourage most users from
> > > > >> interpreting
> > > > >> > > this
> > > > >> > > > >> > value.
> > > > >> > > > >> > > > One more advantage of using such an opaque field is
> > to
> > > be
> > > > >> able
> > > > >> > > to
> > > > >> > > > >> > evolve
> > > > >> > > > >> > > > the information (or schema) of this value without
> > > > changing
> > > > >> > > > consumer
> > > > >> > > > >> API
> > > > >> > > > >> > > in
> > > > >> > > > >> > > > the future.
> > > > >> > > > >> > > >
> > > > >> > > > >> > > > I also thinking it is probably OK for user to be
> able
> > > to
> > > > >> > > interpret
> > > > >> > > > >> this
> > > > >> > > > >> > > > value, particularly for those advanced users.
> > > > >> > > > >> > > >
> > > > >> > > > >> > > > Thanks,
> > > > >> > > > >> > > > Dong
> > > > >> > > > >> > > >
> > > > >> > > > >> > > > On Mon, Jan 8, 2018 at 2:34 PM, Ismael Juma <
> > > > >> > ismael@juma.me.uk>
> > > > >> > > > >> wrote:
> > > > >> > > > >> > > >
> > > > >> > > > >> > > > > On Fri, Jan 5, 2018 at 7:15 PM, Jason Gustafson <
> > > > >> > > > >> jason@confluent.io>
> > > > >> > > > >> > > > > wrote:
> > > > >> > > > >> > > > > >
> > > > >> > > > >> > > > > > class OffsetAndMetadata {
> > > > >> > > > >> > > > > >   long offset;
> > > > >> > > > >> > > > > >   byte[] offsetMetadata;
> > > > >> > > > >> > > > > >   String metadata;
> > > > >> > > > >> > > > > > }
> > > > >> > > > >> > > > >
> > > > >> > > > >> > > > >
> > > > >> > > > >> > > > > > Admittedly, the naming is a bit annoying, but
> we
> > > can
> > > > >> > > probably
> > > > >> > > > >> come
> > > > >> > > > >> > up
> > > > >> > > > >> > > > > with
> > > > >> > > > >> > > > > > something better. Internally the byte array
> would
> > > > have
> > > > >> a
> > > > >> > > > >> version.
> > > > >> > > > >> > If
> > > > >> > > > >> > > in
> > > > >> > > > >> > > > > the
> > > > >> > > > >> > > > > > future we have anything else we need to add, we
> > can
> > > > >> update
> > > > >> > > the
> > > > >> > > > >> > > version
> > > > >> > > > >> > > > > and
> > > > >> > > > >> > > > > > we wouldn't need any new APIs.
> > > > >> > > > >> > > > > >
> > > > >> > > > >> > > > >
> > > > >> > > > >> > > > > We can also add fields to a class in a compatible
> > > way.
> > > > >> So,
> > > > >> > it
> > > > >> > > > >> seems
> > > > >> > > > >> > to
> > > > >> > > > >> > > me
> > > > >> > > > >> > > > > that the main advantage of the byte array is that
> > > it's
> > > > >> > opaque
> > > > >> > > to
> > > > >> > > > >> the
> > > > >> > > > >> > > > user.
> > > > >> > > > >> > > > > Is that correct? If so, we could also add any
> > opaque
> > > > >> > metadata
> > > > >> > > > in a
> > > > >> > > > >> > > > subclass
> > > > >> > > > >> > > > > so that users don't even see it (unless they cast
> > it,
> > > > but
> > > > >> > then
> > > > >> > > > >> > they're
> > > > >> > > > >> > > on
> > > > >> > > > >> > > > > their own).
> > > > >> > > > >> > > > >
> > > > >> > > > >> > > > > Ismael
> > > > >> > > > >> > > > >
> > > > >> > > > >> > > > > The corresponding seek() and position() APIs
> might
> > > look
> > > > >> > > > something
> > > > >> > > > >> > like
> > > > >> > > > >> > > > > this:
> > > > >> > > > >> > > > > >
> > > > >> > > > >> > > > > > void seek(TopicPartition partition, long
> offset,
> > > > byte[]
> > > > >> > > > >> > > > offsetMetadata);
> > > > >> > > > >> > > > > > byte[] positionMetadata(TopicPartition
> > partition);
> > > > >> > > > >> > > > > >
> > > > >> > > > >> > > > > > What do you think?
> > > > >> > > > >> > > > > >
> > > > >> > > > >> > > > > > Thanks,
> > > > >> > > > >> > > > > > Jason
> > > > >> > > > >> > > > > >
> > > > >> > > > >> > > > > > On Thu, Jan 4, 2018 at 7:04 PM, Dong Lin <
> > > > >> > > lindong28@gmail.com
> > > > >> > > > >
> > > > >> > > > >> > > wrote:
> > > > >> > > > >> > > > > >
> > > > >> > > > >> > > > > > > Hey Jun, Jason,
> > > > >> > > > >> > > > > > >
> > > > >> > > > >> > > > > > > Thanks much for all the feedback. I have
> > updated
> > > > the
> > > > >> KIP
> > > > >> > > > >> based on
> > > > >> > > > >> > > the
> > > > >> > > > >> > > > > > > latest discussion. Can you help check whether
> > it
> > > > >> looks
> > > > >> > > good?
> > > > >> > > > >> > > > > > >
> > > > >> > > > >> > > > > > > Thanks,
> > > > >> > > > >> > > > > > > Dong
> > > > >> > > > >> > > > > > >
> > > > >> > > > >> > > > > > > On Thu, Jan 4, 2018 at 5:36 PM, Dong Lin <
> > > > >> > > > lindong28@gmail.com
> > > > >> > > > >> >
> > > > >> > > > >> > > > wrote:
> > > > >> > > > >> > > > > > >
> > > > >> > > > >> > > > > > > > Hey Jun,
> > > > >> > > > >> > > > > > > >
> > > > >> > > > >> > > > > > > > Hmm... thinking about this more, I am not
> > sure
> > > > that
> > > > >> > the
> > > > >> > > > >> > proposed
> > > > >> > > > >> > > > API
> > > > >> > > > >> > > > > is
> > > > >> > > > >> > > > > > > > sufficient. For users that store offset
> > > > >> externally, we
> > > > >> > > > >> probably
> > > > >> > > > >> > > > need
> > > > >> > > > >> > > > > > > extra
> > > > >> > > > >> > > > > > > > API to return the leader_epoch and
> > > > partition_epoch
> > > > >> for
> > > > >> > > all
> > > > >> > > > >> > > > partitions
> > > > >> > > > >> > > > > > > that
> > > > >> > > > >> > > > > > > > consumers are consuming. I suppose these
> > users
> > > > >> > currently
> > > > >> > > > use
> > > > >> > > > >> > > > > position()
> > > > >> > > > >> > > > > > > to
> > > > >> > > > >> > > > > > > > get the offset. Thus we probably need a new
> > > > method
> > > > >> > > > >> > > > > > positionWithEpoch(..)
> > > > >> > > > >> > > > > > > to
> > > > >> > > > >> > > > > > > > return <offset, partition_epoch,
> > leader_epoch>.
> > > > >> Does
> > > > >> > > this
> > > > >> > > > >> sound
> > > > >> > > > >> > > > > > > reasonable?
> > > > >> > > > >> > > > > > > >
> > > > >> > > > >> > > > > > > > Thanks,
> > > > >> > > > >> > > > > > > > Dong
> > > > >> > > > >> > > > > > > >
> > > > >> > > > >> > > > > > > >
> > > > >> > > > >> > > > > > > > On Thu, Jan 4, 2018 at 5:26 PM, Jun Rao <
> > > > >> > > jun@confluent.io
> > > > >> > > > >
> > > > >> > > > >> > > wrote:
> > > > >> > > > >> > > > > > > >
> > > > >> > > > >> > > > > > > >> Hi, Dong,
> > > > >> > > > >> > > > > > > >>
> > > > >> > > > >> > > > > > > >> Yes, that's what I am thinking.
> OffsetEpoch
> > > will
> > > > >> be
> > > > >> > > > >> composed
> > > > >> > > > >> > of
> > > > >> > > > >> > > > > > > >> (partition_epoch,
> > > > >> > > > >> > > > > > > >> leader_epoch).
> > > > >> > > > >> > > > > > > >>
> > > > >> > > > >> > > > > > > >> Thanks,
> > > > >> > > > >> > > > > > > >>
> > > > >> > > > >> > > > > > > >> Jun
> > > > >> > > > >> > > > > > > >>
> > > > >> > > > >> > > > > > > >>
> > > > >> > > > >> > > > > > > >> On Thu, Jan 4, 2018 at 4:22 PM, Dong Lin <
> > > > >> > > > >> lindong28@gmail.com
> > > > >> > > > >> > >
> > > > >> > > > >> > > > > wrote:
> > > > >> > > > >> > > > > > > >>
> > > > >> > > > >> > > > > > > >> > Hey Jun,
> > > > >> > > > >> > > > > > > >> >
> > > > >> > > > >> > > > > > > >> > Thanks much. I like the the new API that
> > you
> > > > >> > > proposed.
> > > > >> > > > I
> > > > >> > > > >> am
> > > > >> > > > >> > > not
> > > > >> > > > >> > > > > sure
> > > > >> > > > >> > > > > > > >> what
> > > > >> > > > >> > > > > > > >> > you exactly mean by offset_epoch. I
> > suppose
> > > > >> that we
> > > > >> > > can
> > > > >> > > > >> use
> > > > >> > > > >> > > the
> > > > >> > > > >> > > > > pair
> > > > >> > > > >> > > > > > > of
> > > > >> > > > >> > > > > > > >> > (partition_epoch, leader_epoch) as the
> > > > >> > offset_epoch,
> > > > >> > > > >> right?
> > > > >> > > > >> > > > > > > >> >
> > > > >> > > > >> > > > > > > >> > Thanks,
> > > > >> > > > >> > > > > > > >> > Dong
> > > > >> > > > >> > > > > > > >> >
> > > > >> > > > >> > > > > > > >> > On Thu, Jan 4, 2018 at 4:02 PM, Jun Rao
> <
> > > > >> > > > >> jun@confluent.io>
> > > > >> > > > >> > > > wrote:
> > > > >> > > > >> > > > > > > >> >
> > > > >> > > > >> > > > > > > >> > > Hi, Dong,
> > > > >> > > > >> > > > > > > >> > >
> > > > >> > > > >> > > > > > > >> > > Got it. The api that you proposed
> works.
> > > The
> > > > >> > > question
> > > > >> > > > >> is
> > > > >> > > > >> > > > whether
> > > > >> > > > >> > > > > > > >> that's
> > > > >> > > > >> > > > > > > >> > the
> > > > >> > > > >> > > > > > > >> > > api that we want to have in the long
> > term.
> > > > My
> > > > >> > > concern
> > > > >> > > > >> is
> > > > >> > > > >> > > that
> > > > >> > > > >> > > > > > while
> > > > >> > > > >> > > > > > > >> the
> > > > >> > > > >> > > > > > > >> > api
> > > > >> > > > >> > > > > > > >> > > change is simple, the new api seems
> > harder
> > > > to
> > > > >> > > explain
> > > > >> > > > >> and
> > > > >> > > > >> > > use.
> > > > >> > > > >> > > > > For
> > > > >> > > > >> > > > > > > >> > example,
> > > > >> > > > >> > > > > > > >> > > a consumer storing offsets externally
> > now
> > > > >> needs
> > > > >> > to
> > > > >> > > > call
> > > > >> > > > >> > > > > > > >> > > waitForMetadataUpdate() after calling
> > > > seek().
> > > > >> > > > >> > > > > > > >> > >
> > > > >> > > > >> > > > > > > >> > > An alternative approach is to make the
> > > > >> following
> > > > >> > > > >> > compatible
> > > > >> > > > >> > > > api
> > > > >> > > > >> > > > > > > >> changes
> > > > >> > > > >> > > > > > > >> > in
> > > > >> > > > >> > > > > > > >> > > Consumer.
> > > > >> > > > >> > > > > > > >> > > * Add an additional OffsetEpoch field
> in
> > > > >> > > > >> > OffsetAndMetadata.
> > > > >> > > > >> > > > (no
> > > > >> > > > >> > > > > > need
> > > > >> > > > >> > > > > > > >> to
> > > > >> > > > >> > > > > > > >> > > change the CommitSync() api)
> > > > >> > > > >> > > > > > > >> > > * Add a new api seek(TopicPartition
> > > > partition,
> > > > >> > long
> > > > >> > > > >> > offset,
> > > > >> > > > >> > > > > > > >> OffsetEpoch
> > > > >> > > > >> > > > > > > >> > > offsetEpoch). We can potentially
> > deprecate
> > > > the
> > > > >> > old
> > > > >> > > > api
> > > > >> > > > >> > > > > > > >> > seek(TopicPartition
> > > > >> > > > >> > > > > > > >> > > partition, long offset) in the future.
> > > > >> > > > >> > > > > > > >> > >
> > > > >> > > > >> > > > > > > >> > > The alternative approach has similar
> > > amount
> > > > of
> > > > >> > api
> > > > >> > > > >> changes
> > > > >> > > > >> > > as
> > > > >> > > > >> > > > > > yours
> > > > >> > > > >> > > > > > > >> but
> > > > >> > > > >> > > > > > > >> > has
> > > > >> > > > >> > > > > > > >> > > the following benefits.
> > > > >> > > > >> > > > > > > >> > > 1. The api works in a similar way as
> how
> > > > >> offset
> > > > >> > > > >> management
> > > > >> > > > >> > > > works
> > > > >> > > > >> > > > > > now
> > > > >> > > > >> > > > > > > >> and
> > > > >> > > > >> > > > > > > >> > is
> > > > >> > > > >> > > > > > > >> > > probably what we want in the long
> term.
> > > > >> > > > >> > > > > > > >> > > 2. It can reset offsets better when
> > there
> > > is
> > > > >> data
> > > > >> > > > loss
> > > > >> > > > >> due
> > > > >> > > > >> > > to
> > > > >> > > > >> > > > > > > unclean
> > > > >> > > > >> > > > > > > >> > > leader election or correlated replica
> > > > failure.
> > > > >> > > > >> > > > > > > >> > > 3. It can reset offsets better when
> > topic
> > > is
> > > > >> > > > recreated.
> > > > >> > > > >> > > > > > > >> > >
> > > > >> > > > >> > > > > > > >> > > Thanks,
> > > > >> > > > >> > > > > > > >> > >
> > > > >> > > > >> > > > > > > >> > > Jun
> > > > >> > > > >> > > > > > > >> > >
> > > > >> > > > >> > > > > > > >> > >
> > > > >> > > > >> > > > > > > >> > > On Thu, Jan 4, 2018 at 2:05 PM, Dong
> > Lin <
> > > > >> > > > >> > > lindong28@gmail.com
> > > > >> > > > >> > > > >
> > > > >> > > > >> > > > > > > wrote:
> > > > >> > > > >> > > > > > > >> > >
> > > > >> > > > >> > > > > > > >> > > > Hey Jun,
> > > > >> > > > >> > > > > > > >> > > >
> > > > >> > > > >> > > > > > > >> > > > Yeah I agree that ideally we don't
> > want
> > > an
> > > > >> ever
> > > > >> > > > >> growing
> > > > >> > > > >> > > > global
> > > > >> > > > >> > > > > > > >> metadata
> > > > >> > > > >> > > > > > > >> > > > version. I just think it may be more
> > > > >> desirable
> > > > >> > to
> > > > >> > > > >> keep
> > > > >> > > > >> > the
> > > > >> > > > >> > > > > > > consumer
> > > > >> > > > >> > > > > > > >> API
> > > > >> > > > >> > > > > > > >> > > > simple.
> > > > >> > > > >> > > > > > > >> > > >
> > > > >> > > > >> > > > > > > >> > > > In my current proposal, metadata
> > version
> > > > >> > returned
> > > > >> > > > in
> > > > >> > > > >> the
> > > > >> > > > >> > > > fetch
> > > > >> > > > >> > > > > > > >> response
> > > > >> > > > >> > > > > > > >> > > > will be stored with the offset
> > together.
> > > > >> More
> > > > >> > > > >> > > specifically,
> > > > >> > > > >> > > > > the
> > > > >> > > > >> > > > > > > >> > > > metadata_epoch in the new offset
> topic
> > > > >> schema
> > > > >> > > will
> > > > >> > > > be
> > > > >> > > > >> > the
> > > > >> > > > >> > > > > > largest
> > > > >> > > > >> > > > > > > >> > > > metadata_epoch from all the
> > > > MetadataResponse
> > > > >> > and
> > > > >> > > > >> > > > FetchResponse
> > > > >> > > > >> > > > > > > ever
> > > > >> > > > >> > > > > > > >> > > > received by this consumer.
> > > > >> > > > >> > > > > > > >> > > >
> > > > >> > > > >> > > > > > > >> > > > We probably don't have to change the
> > > > >> consumer
> > > > >> > API
> > > > >> > > > for
> > > > >> > > > >> > > > > > > >> > > > commitSync(Map<TopicPartition,
> > > > >> > > OffsetAndMetadata>).
> > > > >> > > > >> If
> > > > >> > > > >> > > user
> > > > >> > > > >> > > > > > calls
> > > > >> > > > >> > > > > > > >> > > > commitSync(...) to commit offset 10
> > for
> > > a
> > > > >> given
> > > > >> > > > >> > partition,
> > > > >> > > > >> > > > for
> > > > >> > > > >> > > > > > > most
> > > > >> > > > >> > > > > > > >> > > > use-cases, this consumer instance
> > should
> > > > >> have
> > > > >> > > > >> consumed
> > > > >> > > > >> > > > message
> > > > >> > > > >> > > > > > > with
> > > > >> > > > >> > > > > > > >> > > offset
> > > > >> > > > >> > > > > > > >> > > > 9 from this partition, in which case
> > the
> > > > >> > consumer
> > > > >> > > > can
> > > > >> > > > >> > > > remember
> > > > >> > > > >> > > > > > and
> > > > >> > > > >> > > > > > > >> use
> > > > >> > > > >> > > > > > > >> > > the
> > > > >> > > > >> > > > > > > >> > > > metadata_epoch from the
> corresponding
> > > > >> > > FetchResponse
> > > > >> > > > >> when
> > > > >> > > > >> > > > > > > committing
> > > > >> > > > >> > > > > > > >> > > offset.
> > > > >> > > > >> > > > > > > >> > > > If user calls commitSync(..) to
> commit
> > > > >> offset
> > > > >> > 10
> > > > >> > > > for
> > > > >> > > > >> a
> > > > >> > > > >> > > given
> > > > >> > > > >> > > > > > > >> partition
> > > > >> > > > >> > > > > > > >> > > > without having consumed the message
> > with
> > > > >> > offset 9
> > > > >> > > > >> using
> > > > >> > > > >> > > this
> > > > >> > > > >> > > > > > > >> consumer
> > > > >> > > > >> > > > > > > >> > > > instance, this is probably an
> advanced
> > > > >> > use-case.
> > > > >> > > In
> > > > >> > > > >> this
> > > > >> > > > >> > > > case
> > > > >> > > > >> > > > > > the
> > > > >> > > > >> > > > > > > >> > > advanced
> > > > >> > > > >> > > > > > > >> > > > user can retrieve the metadata_epoch
> > > using
> > > > >> the
> > > > >> > > > newly
> > > > >> > > > >> > added
> > > > >> > > > >> > > > > > > >> > > metadataEpoch()
> > > > >> > > > >> > > > > > > >> > > > API after it fetches the message
> with
> > > > >> offset 9
> > > > >> > > > >> (probably
> > > > >> > > > >> > > > from
> > > > >> > > > >> > > > > > > >> another
> > > > >> > > > >> > > > > > > >> > > > consumer instance) and encode this
> > > > >> > metadata_epoch
> > > > >> > > > in
> > > > >> > > > >> the
> > > > >> > > > >> > > > > > > >> > > > string OffsetAndMetadata.metadata.
> Do
> > > you
> > > > >> think
> > > > >> > > > this
> > > > >> > > > >> > > > solution
> > > > >> > > > >> > > > > > > would
> > > > >> > > > >> > > > > > > >> > work?
> > > > >> > > > >> > > > > > > >> > > >
> > > > >> > > > >> > > > > > > >> > > > By "not sure that I fully understand
> > > your
> > > > >> > latest
> > > > >> > > > >> > > > suggestion",
> > > > >> > > > >> > > > > > are
> > > > >> > > > >> > > > > > > >> you
> > > > >> > > > >> > > > > > > >> > > > referring to solution related to
> > unclean
> > > > >> leader
> > > > >> > > > >> election
> > > > >> > > > >> > > > using
> > > > >> > > > >> > > > > > > >> > > leader_epoch
> > > > >> > > > >> > > > > > > >> > > > in my previous email?
> > > > >> > > > >> > > > > > > >> > > >
> > > > >> > > > >> > > > > > > >> > > > Thanks,
> > > > >> > > > >> > > > > > > >> > > > Dong
> > > > >> > > > >> > > > > > > >> > > >
> > > > >> > > > >> > > > > > > >> > > > On Thu, Jan 4, 2018 at 1:33 PM, Jun
> > Rao
> > > <
> > > > >> > > > >> > jun@confluent.io
> > > > >> > > > >> > > >
> > > > >> > > > >> > > > > > wrote:
> > > > >> > > > >> > > > > > > >> > > >
> > > > >> > > > >> > > > > > > >> > > > > Hi, Dong,
> > > > >> > > > >> > > > > > > >> > > > >
> > > > >> > > > >> > > > > > > >> > > > > Not sure that I fully understand
> > your
> > > > >> latest
> > > > >> > > > >> > suggestion.
> > > > >> > > > >> > > > > > > >> Returning an
> > > > >> > > > >> > > > > > > >> > > > ever
> > > > >> > > > >> > > > > > > >> > > > > growing global metadata version
> > itself
> > > > is
> > > > >> no
> > > > >> > > > ideal,
> > > > >> > > > >> > but
> > > > >> > > > >> > > is
> > > > >> > > > >> > > > > > fine.
> > > > >> > > > >> > > > > > > >> My
> > > > >> > > > >> > > > > > > >> > > > > question is whether the metadata
> > > version
> > > > >> > > returned
> > > > >> > > > >> in
> > > > >> > > > >> > the
> > > > >> > > > >> > > > > fetch
> > > > >> > > > >> > > > > > > >> > response
> > > > >> > > > >> > > > > > > >> > > > > needs to be stored with the offset
> > > > >> together
> > > > >> > if
> > > > >> > > > >> offsets
> > > > >> > > > >> > > are
> > > > >> > > > >> > > > > > > stored
> > > > >> > > > >> > > > > > > >> > > > > externally. If so, we also have to
> > > > change
> > > > >> the
> > > > >> > > > >> consumer
> > > > >> > > > >> > > API
> > > > >> > > > >> > > > > for
> > > > >> > > > >> > > > > > > >> > > > commitSync()
> > > > >> > > > >> > > > > > > >> > > > > and need to worry about
> > compatibility.
> > > > If
> > > > >> we
> > > > >> > > > don't
> > > > >> > > > >> > store
> > > > >> > > > >> > > > the
> > > > >> > > > >> > > > > > > >> metadata
> > > > >> > > > >> > > > > > > >> > > > > version together with the offset,
> > on a
> > > > >> > consumer
> > > > >> > > > >> > restart,
> > > > >> > > > >> > > > > it's
> > > > >> > > > >> > > > > > > not
> > > > >> > > > >> > > > > > > >> > clear
> > > > >> > > > >> > > > > > > >> > > > how
> > > > >> > > > >> > > > > > > >> > > > > we can ensure the metadata in the
> > > > >> consumer is
> > > > >> > > > high
> > > > >> > > > >> > > enough
> > > > >> > > > >> > > > > > since
> > > > >> > > > >> > > > > > > >> there
> > > > >> > > > >> > > > > > > >> > > is
> > > > >> > > > >> > > > > > > >> > > > no
> > > > >> > > > >> > > > > > > >> > > > > metadata version to compare with.
> > > > >> > > > >> > > > > > > >> > > > >
> > > > >> > > > >> > > > > > > >> > > > > Thanks,
> > > > >> > > > >> > > > > > > >> > > > >
> > > > >> > > > >> > > > > > > >> > > > > Jun
> > > > >> > > > >> > > > > > > >> > > > >
> > > > >> > > > >> > > > > > > >> > > > >
> > > > >> > > > >> > > > > > > >> > > > > On Wed, Jan 3, 2018 at 6:43 PM,
> Dong
> > > > Lin <
> > > > >> > > > >> > > > > lindong28@gmail.com
> > > > >> > > > >> > > > > > >
> > > > >> > > > >> > > > > > > >> > wrote:
> > > > >> > > > >> > > > > > > >> > > > >
> > > > >> > > > >> > > > > > > >> > > > > > Hey Jun,
> > > > >> > > > >> > > > > > > >> > > > > >
> > > > >> > > > >> > > > > > > >> > > > > > Thanks much for the explanation.
> > > > >> > > > >> > > > > > > >> > > > > >
> > > > >> > > > >> > > > > > > >> > > > > > I understand the advantage of
> > > > >> > partition_epoch
> > > > >> > > > >> over
> > > > >> > > > >> > > > > > > >> metadata_epoch.
> > > > >> > > > >> > > > > > > >> > My
> > > > >> > > > >> > > > > > > >> > > > > > current concern is that the use
> of
> > > > >> > > leader_epoch
> > > > >> > > > >> and
> > > > >> > > > >> > > the
> > > > >> > > > >> > > > > > > >> > > partition_epoch
> > > > >> > > > >> > > > > > > >> > > > > > requires us considerable change
> to
> > > > >> > consumer's
> > > > >> > > > >> public
> > > > >> > > > >> > > API
> > > > >> > > > >> > > > > to
> > > > >> > > > >> > > > > > > take
> > > > >> > > > >> > > > > > > >> > care
> > > > >> > > > >> > > > > > > >> > > > of
> > > > >> > > > >> > > > > > > >> > > > > > the case where user stores
> offset
> > > > >> > externally.
> > > > >> > > > For
> > > > >> > > > >> > > > example,
> > > > >> > > > >> > > > > > > >> > > *consumer*.
> > > > >> > > > >> > > > > > > >> > > > > > *commitSync*(..) would have to
> > take
> > > a
> > > > >> map
> > > > >> > > whose
> > > > >> > > > >> > value
> > > > >> > > > >> > > is
> > > > >> > > > >> > > > > > > >> <offset,
> > > > >> > > > >> > > > > > > >> > > > > metadata,
> > > > >> > > > >> > > > > > > >> > > > > > leader epoch, partition epoch>.
> > > > >> > > > >> > *consumer*.*seek*(...)
> > > > >> > > > >> > > > > would
> > > > >> > > > >> > > > > > > >> also
> > > > >> > > > >> > > > > > > >> > > need
> > > > >> > > > >> > > > > > > >> > > > > > leader_epoch and partition_epoch
> > as
> > > > >> > > parameter.
> > > > >> > > > >> > > > Technically
> > > > >> > > > >> > > > > > we
> > > > >> > > > >> > > > > > > >> can
> > > > >> > > > >> > > > > > > >> > > > > probably
> > > > >> > > > >> > > > > > > >> > > > > > still make it work in a backward
> > > > >> compatible
> > > > >> > > > >> manner
> > > > >> > > > >> > > after
> > > > >> > > > >> > > > > > > careful
> > > > >> > > > >> > > > > > > >> > > design
> > > > >> > > > >> > > > > > > >> > > > > and
> > > > >> > > > >> > > > > > > >> > > > > > discussion. But these changes
> can
> > > make
> > > > >> the
> > > > >> > > > >> > consumer's
> > > > >> > > > >> > > > > > > interface
> > > > >> > > > >> > > > > > > >> > > > > > unnecessarily complex for more
> > users
> > > > >> who do
> > > > >> > > not
> > > > >> > > > >> > store
> > > > >> > > > >> > > > > offset
> > > > >> > > > >> > > > > > > >> > > > externally.
> > > > >> > > > >> > > > > > > >> > > > > >
> > > > >> > > > >> > > > > > > >> > > > > > After thinking more about it, we
> > can
> > > > >> > address
> > > > >> > > > all
> > > > >> > > > >> > > > problems
> > > > >> > > > >> > > > > > > >> discussed
> > > > >> > > > >> > > > > > > >> > > by
> > > > >> > > > >> > > > > > > >> > > > > only
> > > > >> > > > >> > > > > > > >> > > > > > using the metadata_epoch without
> > > > >> > introducing
> > > > >> > > > >> > > > leader_epoch
> > > > >> > > > >> > > > > or
> > > > >> > > > >> > > > > > > the
> > > > >> > > > >> > > > > > > >> > > > > > partition_epoch. The current KIP
> > > > >> describes
> > > > >> > > the
> > > > >> > > > >> > changes
> > > > >> > > > >> > > > to
> > > > >> > > > >> > > > > > the
> > > > >> > > > >> > > > > > > >> > > consumer
> > > > >> > > > >> > > > > > > >> > > > > API
> > > > >> > > > >> > > > > > > >> > > > > > and how the new API can be used
> if
> > > > user
> > > > >> > > stores
> > > > >> > > > >> > offset
> > > > >> > > > >> > > > > > > >> externally.
> > > > >> > > > >> > > > > > > >> > In
> > > > >> > > > >> > > > > > > >> > > > > order
> > > > >> > > > >> > > > > > > >> > > > > > to address the scenario you
> > > described
> > > > >> > > earlier,
> > > > >> > > > we
> > > > >> > > > >> > can
> > > > >> > > > >> > > > > > include
> > > > >> > > > >> > > > > > > >> > > > > > metadata_epoch in the
> > FetchResponse
> > > > and
> > > > >> the
> > > > >> > > > >> > > > > > > LeaderAndIsrRequest.
> > > > >> > > > >> > > > > > > >> > > > Consumer
> > > > >> > > > >> > > > > > > >> > > > > > remembers the largest
> > metadata_epoch
> > > > >> from
> > > > >> > all
> > > > >> > > > the
> > > > >> > > > >> > > > > > > FetchResponse
> > > > >> > > > >> > > > > > > >> it
> > > > >> > > > >> > > > > > > >> > > has
> > > > >> > > > >> > > > > > > >> > > > > > received. The metadata_epoch
> > > committed
> > > > >> with
> > > > >> > > the
> > > > >> > > > >> > > offset,
> > > > >> > > > >> > > > > > either
> > > > >> > > > >> > > > > > > >> > within
> > > > >> > > > >> > > > > > > >> > > > or
> > > > >> > > > >> > > > > > > >> > > > > > outside Kafka, should be the
> > largest
> > > > >> > > > >> metadata_epoch
> > > > >> > > > >> > > > across
> > > > >> > > > >> > > > > > all
> > > > >> > > > >> > > > > > > >> > > > > > FetchResponse and
> MetadataResponse
> > > > ever
> > > > >> > > > received
> > > > >> > > > >> by
> > > > >> > > > >> > > this
> > > > >> > > > >> > > > > > > >> consumer.
> > > > >> > > > >> > > > > > > >> > > > > >
> > > > >> > > > >> > > > > > > >> > > > > > The drawback of using only the
> > > > >> > metadata_epoch
> > > > >> > > > is
> > > > >> > > > >> > that
> > > > >> > > > >> > > we
> > > > >> > > > >> > > > > can
> > > > >> > > > >> > > > > > > not
> > > > >> > > > >> > > > > > > >> > > always
> > > > >> > > > >> > > > > > > >> > > > > do
> > > > >> > > > >> > > > > > > >> > > > > > the smart offset reset in case
> of
> > > > >> unclean
> > > > >> > > > leader
> > > > >> > > > >> > > > election
> > > > >> > > > >> > > > > > > which
> > > > >> > > > >> > > > > > > >> you
> > > > >> > > > >> > > > > > > >> > > > > > mentioned earlier. But in most
> > case,
> > > > >> > unclean
> > > > >> > > > >> leader
> > > > >> > > > >> > > > > election
> > > > >> > > > >> > > > > > > >> > probably
> > > > >> > > > >> > > > > > > >> > > > > > happens when consumer is not
> > > > >> > > > >> rebalancing/restarting.
> > > > >> > > > >> > > In
> > > > >> > > > >> > > > > > these
> > > > >> > > > >> > > > > > > >> > cases,
> > > > >> > > > >> > > > > > > >> > > > > either
> > > > >> > > > >> > > > > > > >> > > > > > consumer is not directly
> affected
> > by
> > > > >> > unclean
> > > > >> > > > >> leader
> > > > >> > > > >> > > > > election
> > > > >> > > > >> > > > > > > >> since
> > > > >> > > > >> > > > > > > >> > it
> > > > >> > > > >> > > > > > > >> > > > is
> > > > >> > > > >> > > > > > > >> > > > > > not consuming from the end of
> the
> > > log,
> > > > >> or
> > > > >> > > > >> consumer
> > > > >> > > > >> > can
> > > > >> > > > >> > > > > > derive
> > > > >> > > > >> > > > > > > >> the
> > > > >> > > > >> > > > > > > >> > > > > > leader_epoch from the most
> recent
> > > > >> message
> > > > >> > > > >> received
> > > > >> > > > >> > > > before
> > > > >> > > > >> > > > > it
> > > > >> > > > >> > > > > > > >> sees
> > > > >> > > > >> > > > > > > >> > > > > > OffsetOutOfRangeException. So I
> am
> > > not
> > > > >> sure
> > > > >> > > it
> > > > >> > > > is
> > > > >> > > > >> > > worth
> > > > >> > > > >> > > > > > adding
> > > > >> > > > >> > > > > > > >> the
> > > > >> > > > >> > > > > > > >> > > > > > leader_epoch to consumer API to
> > > > address
> > > > >> the
> > > > >> > > > >> > remaining
> > > > >> > > > >> > > > > corner
> > > > >> > > > >> > > > > > > >> case.
> > > > >> > > > >> > > > > > > >> > > What
> > > > >> > > > >> > > > > > > >> > > > > do
> > > > >> > > > >> > > > > > > >> > > > > > you think?
> > > > >> > > > >> > > > > > > >> > > > > >
> > > > >> > > > >> > > > > > > >> > > > > > Thanks,
> > > > >> > > > >> > > > > > > >> > > > > > Dong
> > > > >> > > > >> > > > > > > >> > > > > >
> > > > >> > > > >> > > > > > > >> > > > > >
> > > > >> > > > >> > > > > > > >> > > > > >
> > > > >> > > > >> > > > > > > >> > > > > > On Tue, Jan 2, 2018 at 6:28 PM,
> > Jun
> > > > Rao
> > > > >> <
> > > > >> > > > >> > > > jun@confluent.io
> > > > >> > > > >> > > > > >
> > > > >> > > > >> > > > > > > >> wrote:
> > > > >> > > > >> > > > > > > >> > > > > >
> > > > >> > > > >> > > > > > > >> > > > > > > Hi, Dong,
> > > > >> > > > >> > > > > > > >> > > > > > >
> > > > >> > > > >> > > > > > > >> > > > > > > Thanks for the reply.
> > > > >> > > > >> > > > > > > >> > > > > > >
> > > > >> > > > >> > > > > > > >> > > > > > > To solve the topic recreation
> > > issue,
> > > > >> we
> > > > >> > > could
> > > > >> > > > >> use
> > > > >> > > > >> > > > > either a
> > > > >> > > > >> > > > > > > >> global
> > > > >> > > > >> > > > > > > >> > > > > > metadata
> > > > >> > > > >> > > > > > > >> > > > > > > version or a partition level
> > > epoch.
> > > > >> But
> > > > >> > > > either
> > > > >> > > > >> one
> > > > >> > > > >> > > > will
> > > > >> > > > >> > > > > > be a
> > > > >> > > > >> > > > > > > >> new
> > > > >> > > > >> > > > > > > >> > > > > concept,
> > > > >> > > > >> > > > > > > >> > > > > > > right? To me, the latter seems
> > > more
> > > > >> > > natural.
> > > > >> > > > It
> > > > >> > > > >> > also
> > > > >> > > > >> > > > > makes
> > > > >> > > > >> > > > > > > it
> > > > >> > > > >> > > > > > > >> > > easier
> > > > >> > > > >> > > > > > > >> > > > to
> > > > >> > > > >> > > > > > > >> > > > > > > detect if a consumer's offset
> is
> > > > still
> > > > >> > > valid
> > > > >> > > > >> > after a
> > > > >> > > > >> > > > > topic
> > > > >> > > > >> > > > > > > is
> > > > >> > > > >> > > > > > > >> > > > > recreated.
> > > > >> > > > >> > > > > > > >> > > > > > As
> > > > >> > > > >> > > > > > > >> > > > > > > you pointed out, we don't need
> > to
> > > > >> store
> > > > >> > the
> > > > >> > > > >> > > partition
> > > > >> > > > >> > > > > > epoch
> > > > >> > > > >> > > > > > > in
> > > > >> > > > >> > > > > > > >> > the
> > > > >> > > > >> > > > > > > >> > > > > > message.
> > > > >> > > > >> > > > > > > >> > > > > > > The following is what I am
> > > thinking.
> > > > >> > When a
> > > > >> > > > >> > > partition
> > > > >> > > > >> > > > is
> > > > >> > > > >> > > > > > > >> created,
> > > > >> > > > >> > > > > > > >> > > we
> > > > >> > > > >> > > > > > > >> > > > > can
> > > > >> > > > >> > > > > > > >> > > > > > > assign a partition epoch from
> an
> > > > >> > > > >> ever-increasing
> > > > >> > > > >> > > > global
> > > > >> > > > >> > > > > > > >> counter
> > > > >> > > > >> > > > > > > >> > and
> > > > >> > > > >> > > > > > > >> > > > > store
> > > > >> > > > >> > > > > > > >> > > > > > > it in /brokers/topics/[topic]/
> > > > >> > > > >> > > > partitions/[partitionId]
> > > > >> > > > >> > > > > in
> > > > >> > > > >> > > > > > > ZK.
> > > > >> > > > >> > > > > > > >> > The
> > > > >> > > > >> > > > > > > >> > > > > > > partition
> > > > >> > > > >> > > > > > > >> > > > > > > epoch is propagated to every
> > > broker.
> > > > >> The
> > > > >> > > > >> consumer
> > > > >> > > > >> > > will
> > > > >> > > > >> > > > > be
> > > > >> > > > >> > > > > > > >> > tracking
> > > > >> > > > >> > > > > > > >> > > a
> > > > >> > > > >> > > > > > > >> > > > > > tuple
> > > > >> > > > >> > > > > > > >> > > > > > > of <offset, leader epoch,
> > > partition
> > > > >> > epoch>
> > > > >> > > > for
> > > > >> > > > >> > > > offsets.
> > > > >> > > > >> > > > > > If a
> > > > >> > > > >> > > > > > > >> > topic
> > > > >> > > > >> > > > > > > >> > > is
> > > > >> > > > >> > > > > > > >> > > > > > > recreated, it's possible that
> a
> > > > >> > consumer's
> > > > >> > > > >> offset
> > > > >> > > > >> > > and
> > > > >> > > > >> > > > > > leader
> > > > >> > > > >> > > > > > > >> > epoch
> > > > >> > > > >> > > > > > > >> > > > > still
> > > > >> > > > >> > > > > > > >> > > > > > > match that in the broker, but
> > > > >> partition
> > > > >> > > epoch
> > > > >> > > > >> > won't
> > > > >> > > > >> > > > be.
> > > > >> > > > >> > > > > In
> > > > >> > > > >> > > > > > > >> this
> > > > >> > > > >> > > > > > > >> > > case,
> > > > >> > > > >> > > > > > > >> > > > > we
> > > > >> > > > >> > > > > > > >> > > > > > > can potentially still treat
> the
> > > > >> > consumer's
> > > > >> > > > >> offset
> > > > >> > > > >> > as
> > > > >> > > > >> > > > out
> > > > >> > > > >> > > > > > of
> > > > >> > > > >> > > > > > > >> range
> > > > >> > > > >> > > > > > > >> > > and
> > > > >> > > > >> > > > > > > >> > > > > > reset
> > > > >> > > > >> > > > > > > >> > > > > > > the offset based on the offset
> > > reset
> > > > >> > policy
> > > > >> > > > in
> > > > >> > > > >> the
> > > > >> > > > >> > > > > > consumer.
> > > > >> > > > >> > > > > > > >> This
> > > > >> > > > >> > > > > > > >> > > > seems
> > > > >> > > > >> > > > > > > >> > > > > > > harder to do with a global
> > > metadata
> > > > >> > > version.
> > > > >> > > > >> > > > > > > >> > > > > > >
> > > > >> > > > >> > > > > > > >> > > > > > > Jun
> > > > >> > > > >> > > > > > > >> > > > > > >
> > > > >> > > > >> > > > > > > >> > > > > > >
> > > > >> > > > >> > > > > > > >> > > > > > >
> > > > >> > > > >> > > > > > > >> > > > > > > On Mon, Dec 25, 2017 at 6:56
> AM,
> > > > Dong
> > > > >> > Lin <
> > > > >> > > > >> > > > > > > >> lindong28@gmail.com>
> > > > >> > > > >> > > > > > > >> > > > wrote:
> > > > >> > > > >> > > > > > > >> > > > > > >
> > > > >> > > > >> > > > > > > >> > > > > > > > Hey Jun,
> > > > >> > > > >> > > > > > > >> > > > > > > >
> > > > >> > > > >> > > > > > > >> > > > > > > > This is a very good example.
> > > After
> > > > >> > > thinking
> > > > >> > > > >> > > through
> > > > >> > > > >> > > > > this
> > > > >> > > > >> > > > > > > in
> > > > >> > > > >> > > > > > > >> > > > detail, I
> > > > >> > > > >> > > > > > > >> > > > > > > agree
> > > > >> > > > >> > > > > > > >> > > > > > > > that we need to commit
> offset
> > > with
> > > > >> > leader
> > > > >> > > > >> epoch
> > > > >> > > > >> > in
> > > > >> > > > >> > > > > order
> > > > >> > > > >> > > > > > > to
> > > > >> > > > >> > > > > > > >> > > address
> > > > >> > > > >> > > > > > > >> > > > > > this
> > > > >> > > > >> > > > > > > >> > > > > > > > example.
> > > > >> > > > >> > > > > > > >> > > > > > > >
> > > > >> > > > >> > > > > > > >> > > > > > > > I think the remaining
> question
> > > is
> > > > >> how
> > > > >> > to
> > > > >> > > > >> address
> > > > >> > > > >> > > the
> > > > >> > > > >> > > > > > > >> scenario
> > > > >> > > > >> > > > > > > >> > > that
> > > > >> > > > >> > > > > > > >> > > > > the
> > > > >> > > > >> > > > > > > >> > > > > > > > topic is deleted and
> > re-created.
> > > > One
> > > > >> > > > possible
> > > > >> > > > >> > > > solution
> > > > >> > > > >> > > > > > is
> > > > >> > > > >> > > > > > > to
> > > > >> > > > >> > > > > > > >> > > commit
> > > > >> > > > >> > > > > > > >> > > > > > > offset
> > > > >> > > > >> > > > > > > >> > > > > > > > with both the leader epoch
> and
> > > the
> > > > >> > > metadata
> > > > >> > > > >> > > version.
> > > > >> > > > >> > > > > The
> > > > >> > > > >> > > > > > > >> logic
> > > > >> > > > >> > > > > > > >> > > and
> > > > >> > > > >> > > > > > > >> > > > > the
> > > > >> > > > >> > > > > > > >> > > > > > > > implementation of this
> > solution
> > > > does
> > > > >> > not
> > > > >> > > > >> > require a
> > > > >> > > > >> > > > new
> > > > >> > > > >> > > > > > > >> concept
> > > > >> > > > >> > > > > > > >> > > > (e.g.
> > > > >> > > > >> > > > > > > >> > > > > > > > partition epoch) and it does
> > not
> > > > >> > require
> > > > >> > > > any
> > > > >> > > > >> > > change
> > > > >> > > > >> > > > to
> > > > >> > > > >> > > > > > the
> > > > >> > > > >> > > > > > > >> > > message
> > > > >> > > > >> > > > > > > >> > > > > > format
> > > > >> > > > >> > > > > > > >> > > > > > > > or leader epoch. It also
> > allows
> > > us
> > > > >> to
> > > > >> > > order
> > > > >> > > > >> the
> > > > >> > > > >> > > > > metadata
> > > > >> > > > >> > > > > > > in
> > > > >> > > > >> > > > > > > >> a
> > > > >> > > > >> > > > > > > >> > > > > > > > straightforward manner which
> > may
> > > > be
> > > > >> > > useful
> > > > >> > > > in
> > > > >> > > > >> > the
> > > > >> > > > >> > > > > > future.
> > > > >> > > > >> > > > > > > >> So it
> > > > >> > > > >> > > > > > > >> > > may
> > > > >> > > > >> > > > > > > >> > > > > be
> > > > >> > > > >> > > > > > > >> > > > > > a
> > > > >> > > > >> > > > > > > >> > > > > > > > better solution than
> > generating
> > > a
> > > > >> > random
> > > > >> > > > >> > partition
> > > > >> > > > >> > > > > epoch
> > > > >> > > > >> > > > > > > >> every
> > > > >> > > > >> > > > > > > >> > > time
> > > > >> > > > >> > > > > > > >> > > > > we
> > > > >> > > > >> > > > > > > >> > > > > > > > create a partition. Does
> this
> > > > sound
> > > > >> > > > >> reasonable?
> > > > >> > > > >> > > > > > > >> > > > > > > >
> > > > >> > > > >> > > > > > > >> > > > > > > > Previously one concern with
> > > using
> > > > >> the
> > > > >> > > > >> metadata
> > > > >> > > > >> > > > version
> > > > >> > > > >> > > > > > is
> > > > >> > > > >> > > > > > > >> that
> > > > >> > > > >> > > > > > > >> > > > > consumer
> > > > >> > > > >> > > > > > > >> > > > > > > > will be forced to refresh
> > > metadata
> > > > >> even
> > > > >> > > if
> > > > >> > > > >> > > metadata
> > > > >> > > > >> > > > > > > version
> > > > >> > > > >> > > > > > > >> is
> > > > >> > > > >> > > > > > > >> > > > > > increased
> > > > >> > > > >> > > > > > > >> > > > > > > > due to topics that the
> > consumer
> > > is
> > > > >> not
> > > > >> > > > >> > interested
> > > > >> > > > >> > > > in.
> > > > >> > > > >> > > > > > Now
> > > > >> > > > >> > > > > > > I
> > > > >> > > > >> > > > > > > >> > > > realized
> > > > >> > > > >> > > > > > > >> > > > > > that
> > > > >> > > > >> > > > > > > >> > > > > > > > this is probably not a
> > problem.
> > > > >> > Currently
> > > > >> > > > >> client
> > > > >> > > > >> > > > will
> > > > >> > > > >> > > > > > > >> refresh
> > > > >> > > > >> > > > > > > >> > > > > metadata
> > > > >> > > > >> > > > > > > >> > > > > > > > either due to
> > > > >> InvalidMetadataException
> > > > >> > in
> > > > >> > > > the
> > > > >> > > > >> > > > response
> > > > >> > > > >> > > > > > > from
> > > > >> > > > >> > > > > > > >> > > broker
> > > > >> > > > >> > > > > > > >> > > > or
> > > > >> > > > >> > > > > > > >> > > > > > due
> > > > >> > > > >> > > > > > > >> > > > > > > > to metadata expiry. The
> > addition
> > > > of
> > > > >> the
> > > > >> > > > >> metadata
> > > > >> > > > >> > > > > version
> > > > >> > > > >> > > > > > > >> should
> > > > >> > > > >> > > > > > > >> > > > > > increase
> > > > >> > > > >> > > > > > > >> > > > > > > > the overhead of metadata
> > refresh
> > > > >> caused
> > > > >> > > by
> > > > >> > > > >> > > > > > > >> > > > InvalidMetadataException.
> > > > >> > > > >> > > > > > > >> > > > > If
> > > > >> > > > >> > > > > > > >> > > > > > > > client refresh metadata due
> to
> > > > >> expiry
> > > > >> > and
> > > > >> > > > it
> > > > >> > > > >> > > > receives
> > > > >> > > > >> > > > > a
> > > > >> > > > >> > > > > > > >> > metadata
> > > > >> > > > >> > > > > > > >> > > > > whose
> > > > >> > > > >> > > > > > > >> > > > > > > > version is lower than the
> > > current
> > > > >> > > metadata
> > > > >> > > > >> > > version,
> > > > >> > > > >> > > > we
> > > > >> > > > >> > > > > > can
> > > > >> > > > >> > > > > > > >> > reject
> > > > >> > > > >> > > > > > > >> > > > the
> > > > >> > > > >> > > > > > > >> > > > > > > > metadata but still reset the
> > > > >> metadata
> > > > >> > > age,
> > > > >> > > > >> which
> > > > >> > > > >> > > > > > > essentially
> > > > >> > > > >> > > > > > > >> > keep
> > > > >> > > > >> > > > > > > >> > > > the
> > > > >> > > > >> > > > > > > >> > > > > > > > existing behavior in the
> > client.
> > > > >> > > > >> > > > > > > >> > > > > > > >
> > > > >> > > > >> > > > > > > >> > > > > > > > Thanks much,
> > > > >> > > > >> > > > > > > >> > > > > > > > Dong
> > > > >> > > > >> > > > > > > >> > > > > > > >
> > > > >> > > > >> > > > > > > >> > > > > > >
> > > > >> > > > >> > > > > > > >> > > > > >
> > > > >> > > > >> > > > > > > >> > > > >
> > > > >> > > > >> > > > > > > >> > > >
> > > > >> > > > >> > > > > > > >> > >
> > > > >> > > > >> > > > > > > >> >
> > > > >> > > > >> > > > > > > >>
> > > > >> > > > >> > > > > > > >
> > > > >> > > > >> > > > > > > >
> > > > >> > > > >> > > > > > >
> > > > >> > > > >> > > > > >
> > > > >> > > > >> > > > >
> > > > >> > > > >> > > >
> > > > >> > > > >> > >
> > > > >> > > > >> >
> > > > >> > > > >>
> > > > >> > > > >
> > > > >> > > > >
> > > > >> > > >
> > > > >> > >
> > > > >> >
> > > > >>
> > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Jun Rao <ju...@confluent.io>.
Hi, Dong,

The issue with using just byte[] for OffsetEpoch is that it won't be
printable, which makes debugging harder.

Also, KIP-222 proposes a listGroupOffset() method in AdminClient. If that
gets adopted before this KIP, we probably want to include OffsetEpoch in
the AdminClient too.

Thanks,

Jun


On Thu, Jan 18, 2018 at 6:30 PM, Dong Lin <li...@gmail.com> wrote:

> Hey Jun,
>
> I agree. I have updated the KIP to remove the class OffetEpoch and replace
> OffsetEpoch with byte[] in APIs that use it. Can you see if it looks good?
>
> Thanks!
> Dong
>
> On Thu, Jan 18, 2018 at 6:07 PM, Jun Rao <ju...@confluent.io> wrote:
>
> > Hi, Dong,
> >
> > Thanks for the updated KIP. It looks good to me now. The only thing is
> > for OffsetEpoch.
> > If we expose the individual fields in the class, we probably don't need
> the
> > encode/decode methods. If we want to hide the details of OffsetEpoch, we
> > probably don't want expose the individual fields.
> >
> > Jun
> >
> > On Wed, Jan 17, 2018 at 10:10 AM, Dong Lin <li...@gmail.com> wrote:
> >
> > > Thinking about point 61 more, I realize that the async zookeeper read
> may
> > > make it less of an issue for controller to read more zookeeper nodes.
> > > Writing partition_epoch in the per-partition znode makes it simpler to
> > > handle the broker failure between zookeeper writes for a topic
> creation.
> > I
> > > have updated the KIP to use the suggested approach.
> > >
> > >
> > > On Wed, Jan 17, 2018 at 9:57 AM, Dong Lin <li...@gmail.com> wrote:
> > >
> > > > Hey Jun,
> > > >
> > > > Thanks much for the comments. Please see my comments inline.
> > > >
> > > > On Tue, Jan 16, 2018 at 4:38 PM, Jun Rao <ju...@confluent.io> wrote:
> > > >
> > > >> Hi, Dong,
> > > >>
> > > >> Thanks for the updated KIP. Looks good to me overall. Just a few
> minor
> > > >> comments.
> > > >>
> > > >> 60. OffsetAndMetadata positionAndOffsetEpoch(TopicPartition
> > partition):
> > > >> It
> > > >> seems that there is no need to return metadata. We probably want to
> > > return
> > > >> sth like OffsetAndEpoch.
> > > >>
> > > >
> > > > Previously I think we may want to re-use the existing class to keep
> our
> > > > consumer interface simpler. I have updated the KIP to add class
> > > > OffsetAndOffsetEpoch. I didn't use OffsetAndEpoch because user may
> > > confuse
> > > > this name with OffsetEpoch. Does this sound OK?
> > > >
> > > >
> > > >>
> > > >> 61. Should we store partition_epoch in
> > > >> /brokers/topics/[topic]/partitions/[partitionId] in ZK?
> > > >>
> > > >
> > > > I have considered this. I think the advantage of adding the
> > > > partition->partition_epoch map in the existing
> > > > znode /brokers/topics/[topic]/partitions is that controller only
> needs
> > > to
> > > > read one znode per topic to gets its partition_epoch information.
> > > Otherwise
> > > > controller may need to read one extra znode per partition to get the
> > same
> > > > information.
> > > >
> > > > When we delete partition or expand partition of a topic, someone
> needs
> > to
> > > > modify partition->partition_epoch map in znode
> > > > /brokers/topics/[topic]/partitions. This may seem a bit more
> > complicated
> > > > than simply adding or deleting znode /brokers/topics/[topic]/
> > > partitions/[partitionId].
> > > > But the complexity is probably similar to the existing operation of
> > > > modifying the partition->replica_list mapping in znode
> > > > /brokers/topics/[topic]. So not sure it is better to store the
> > > > partition_epoch in /brokers/topics/[topic]/partitions/[partitionId].
> > > What
> > > > do you think?
> > > >
> > > >
> > > >>
> > > >> 62. For checking outdated metadata in the client, we probably want
> to
> > > add
> > > >> when max_partition_epoch will be used.
> > > >>
> > > >
> > > > The max_partition_epoch is used in the Proposed Changes -> Client's
> > > > metadata refresh section to determine whether a metadata is outdated.
> > And
> > > > this formula is referenced and re-used in other sections to determine
> > > > whether a metadata is outdated. Does this formula look OK?
> > > >
> > > >
> > > >>
> > > >> 63. "The leader_epoch should be the largest leader_epoch of messages
> > > whose
> > > >> offset < the commit offset. If no message has been consumed since
> > > consumer
> > > >> initialization, the leader_epoch from seek(...) or
> OffsetFetchResponse
> > > >> should be used. The partition_epoch should be read from the last
> > > >> FetchResponse corresponding to the given partition and commit
> offset.
> > ":
> > > >> leader_epoch and partition_epoch are associated with an offset. So,
> if
> > > no
> > > >> message is consumed, there is no offset and therefore there is no
> need
> > > to
> > > >> read leader_epoch and partition_epoch. Also, the leader_epoch
> > associated
> > > >> with the offset should just come from the messages returned in the
> > fetch
> > > >> response.
> > > >>
> > > >
> > > > I am thinking that, if user calls seek(..) and commitSync(...)
> without
> > > > consuming any messages, we should re-use the leader_epoch and
> > > > partition_epoch provided by the seek(...) in the OffsetCommitRequest.
> > And
> > > > if messages have been successfully consumed, then leader_epoch will
> > come
> > > > from the messages returned in the fetch response. The condition
> > "messages
> > > > whose offset < the commit offset" is needed to take care of the log
> > > > compacted topic which may have offset gap due to log cleaning.
> > > >
> > > > Did I miss something here? Or should I rephrase the paragraph to make
> > it
> > > > less confusing?
> > > >
> > > >
> > > >> 64. Could you include the public methods in the OffsetEpoch class?
> > > >>
> > > >
> > > > I mistakenly deleted the definition of OffsetEpoch class from the
> KIP.
> > I
> > > > just added it back with the public methods. Could you take another
> > look?
> > > >
> > > >
> > > >>
> > > >> Jun
> > > >>
> > > >>
> > > >> On Thu, Jan 11, 2018 at 5:43 PM, Dong Lin <li...@gmail.com>
> > wrote:
> > > >>
> > > >> > Hey Jun,
> > > >> >
> > > >> > Thanks much. I agree that we can not rely on committed offsets to
> be
> > > >> always
> > > >> > deleted when we delete topic. So it is necessary to use a
> > > per-partition
> > > >> > epoch that does not change unless this partition is deleted. I
> also
> > > >> agree
> > > >> > that it is very nice to be able to uniquely identify a message
> with
> > > >> > (offset, leader_epoch, partition_epoch) in face of potential topic
> > > >> deletion
> > > >> > and unclean leader election.
> > > >> >
> > > >> > I agree with all your comments. And I have updated the KIP based
> on
> > > our
> > > >> > latest discussion. In addition, I added
> > InvalidPartitionEpochException
> > > >> > which will be thrown by consumer.poll() if the partition_epoch
> > > >> associated
> > > >> > with the partition, which can be given to consumer using
> seek(...),
> > is
> > > >> > different from the partition_epoch in the FetchResponse.
> > > >> >
> > > >> > Can you take another look at the latest KIP?
> > > >> >
> > > >> > Thanks!
> > > >> > Dong
> > > >> >
> > > >> >
> > > >> >
> > > >> > On Wed, Jan 10, 2018 at 2:24 PM, Jun Rao <ju...@confluent.io>
> wrote:
> > > >> >
> > > >> > > Hi, Dong,
> > > >> > >
> > > >> > > My replies are the following.
> > > >> > >
> > > >> > > 60. What you described could also work. The drawback is that we
> > will
> > > >> be
> > > >> > > unnecessarily changing the partition epoch when a partition
> hasn't
> > > >> really
> > > >> > > changed. I was imagining that the partition epoch will be stored
> > in
> > > >> > > /brokers/topics/[topic]/partitions/[partitionId], instead of at
> > the
> > > >> > topic
> > > >> > > level. So, not sure if ZK size limit is an issue.
> > > >> > >
> > > >> > > 61, 62 and 65. To me, the offset + offset_epoch is a unique
> > > identifier
> > > >> > for
> > > >> > > a message. So, if a message hasn't changed, the offset and the
> > > >> associated
> > > >> > > offset_epoch ideally should remain the same (it will be kind of
> > > weird
> > > >> if
> > > >> > > two consumer apps save the offset on the same message, but the
> > > >> > offset_epoch
> > > >> > > are different). partition_epoch + leader_epoch give us that.
> > > >> > global_epoch +
> > > >> > > leader_epoch don't. If we use this approach, we can solve not
> only
> > > the
> > > >> > > problem that you have identified, but also other problems when
> > there
> > > >> is
> > > >> > > data loss or topic re-creation more reliably. For example, in
> the
> > > >> future,
> > > >> > > if we include the partition_epoch and leader_epoch in the fetch
> > > >> request,
> > > >> > > the server can do a more reliable check of whether that offset
> is
> > > >> valid
> > > >> > or
> > > >> > > not. I am not sure that we can rely upon all external offsets to
> > be
> > > >> > removed
> > > >> > > on topic deletion. For example, a topic may be deleted by an
> admin
> > > who
> > > >> > may
> > > >> > > not know all the applications.
> > > >> > >
> > > >> > > If we agree on the above, the second question is then how to
> > > reliably
> > > >> > > propagate the partition_epoch and the leader_epoch to the
> consumer
> > > >> when
> > > >> > > there are leader or partition changes. The leader_epoch comes
> from
> > > the
> > > >> > > message, which is reliable. So, I was suggesting that when we
> > store
> > > an
> > > >> > > offset, we can just store the leader_epoch from the message set
> > > >> > containing
> > > >> > > that offset. Similarly, I was thinking that if the
> partition_epoch
> > > is
> > > >> in
> > > >> > > the fetch response, we can propagate partition_epoch reliably
> > where
> > > is
> > > >> > > partition_epoch change.
> > > >> > >
> > > >> > > 63. My point is that once a leader is producing a message in the
> > new
> > > >> > > partition_epoch, ideally, we should associate the new offsets
> with
> > > the
> > > >> > new
> > > >> > > partition_epoch. Otherwise, the offset_epoch won't be the
> correct
> > > >> unique
> > > >> > > identifier (useful for solving other problems mentioned above).
> I
> > > was
> > > >> > > originally thinking that the leader will include the
> > partition_epoch
> > > >> in
> > > >> > the
> > > >> > > metadata cache in the fetch response. It's just that right now,
> > > >> metadata
> > > >> > > cache is updated on UpdateMetadataRequest, which typically
> happens
> > > >> after
> > > >> > > the LeaderAndIsrRequest. Another approach is for the leader to
> > cache
> > > >> the
> > > >> > > partition_epoch in the Partition object and return that (instead
> > of
> > > >> the
> > > >> > one
> > > >> > > in metadata cache) in the fetch response.
> > > >> > >
> > > >> > > 65. It seems to me that the global_epoch and the partition_epoch
> > > have
> > > >> > > different purposes. A partition_epoch has the benefit that it
> (1)
> > > can
> > > >> be
> > > >> > > used to form a unique identifier for a message and (2) can be
> used
> > > to
> > > >> > > solve other
> > > >> > > corner case problems in the future. I am not sure having just a
> > > >> > > global_epoch can achieve these. global_epoch is useful to
> > determine
> > > >> which
> > > >> > > version of the metadata is newer, especially with topic
> deletion.
> > > >> > >
> > > >> > > Thanks,
> > > >> > >
> > > >> > > Jun
> > > >> > >
> > > >> > > On Tue, Jan 9, 2018 at 11:34 PM, Dong Lin <li...@gmail.com>
> > > >> wrote:
> > > >> > >
> > > >> > > > Regarding the use of the global epoch in 65), it is very
> similar
> > > to
> > > >> the
> > > >> > > > proposal of the metadata_epoch we discussed earlier. The main
> > > >> > difference
> > > >> > > is
> > > >> > > > that this epoch is incremented when we create/expand/delete
> > topic
> > > >> and
> > > >> > > does
> > > >> > > > not change when controller re-send metadata.
> > > >> > > >
> > > >> > > > I looked at our previous discussion. It seems that we prefer
> > > >> > > > partition_epoch over the metadata_epoch because 1) we prefer
> not
> > > to
> > > >> > have
> > > >> > > an
> > > >> > > > ever growing metadata_epoch and 2) we can reset offset better
> > when
> > > >> > topic
> > > >> > > is
> > > >> > > > re-created. The use of global topic_epoch avoids the drawback
> of
> > > an
> > > >> > ever
> > > >> > > > quickly ever growing metadata_epoch. Though the global epoch
> > does
> > > >> not
> > > >> > > allow
> > > >> > > > us to recognize the invalid offset committed before the topic
> > > >> > > re-creation,
> > > >> > > > we can probably just delete the offset when we delete a topic.
> > > Thus
> > > >> I
> > > >> > am
> > > >> > > > not very sure whether it is still worthwhile to have a
> > > per-partition
> > > >> > > > partition_epoch if the metadata already has the global epoch.
> > > >> > > >
> > > >> > > >
> > > >> > > > On Tue, Jan 9, 2018 at 6:58 PM, Dong Lin <lindong28@gmail.com
> >
> > > >> wrote:
> > > >> > > >
> > > >> > > > > Hey Jun,
> > > >> > > > >
> > > >> > > > > Thanks so much. These comments very useful. Please see below
> > my
> > > >> > > comments.
> > > >> > > > >
> > > >> > > > > On Mon, Jan 8, 2018 at 5:52 PM, Jun Rao <ju...@confluent.io>
> > > wrote:
> > > >> > > > >
> > > >> > > > >> Hi, Dong,
> > > >> > > > >>
> > > >> > > > >> Thanks for the updated KIP. A few more comments.
> > > >> > > > >>
> > > >> > > > >> 60. Perhaps having a partition epoch is more flexible since
> > in
> > > >> the
> > > >> > > > future,
> > > >> > > > >> we may support deleting a partition as well.
> > > >> > > > >>
> > > >> > > > >
> > > >> > > > > Yeah I have considered this. I think we can probably still
> > > support
> > > >> > > > > deleting a partition by using the topic_epoch -- when
> > partition
> > > >> of a
> > > >> > > > topic
> > > >> > > > > is deleted or created, epoch of all partitions of this topic
> > > will
> > > >> be
> > > >> > > > > incremented by 1. Therefore, if that partition is re-created
> > > >> later,
> > > >> > the
> > > >> > > > > epoch of that partition will still be larger than its epoch
> > > before
> > > >> > the
> > > >> > > > > deletion, which still allows the client to order the
> metadata
> > > for
> > > >> the
> > > >> > > > > purpose of this KIP. Does this sound reasonable?
> > > >> > > > >
> > > >> > > > > The advantage of using topic_epoch instead of
> partition_epoch
> > is
> > > >> that
> > > >> > > the
> > > >> > > > > size of the /brokers/topics/[topic] znode and
> request/response
> > > >> size
> > > >> > can
> > > >> > > > be
> > > >> > > > > smaller. We have a limit on the maximum size of znode
> > (typically
> > > >> > 1MB).
> > > >> > > > Use
> > > >> > > > > partition epoch can effectively reduce the number of
> > partitions
> > > >> that
> > > >> > > can
> > > >> > > > be
> > > >> > > > > described by the /brokers/topics/[topic] znode.
> > > >> > > > >
> > > >> > > > > One use-case of partition_epoch for client to detect that
> the
> > > >> > committed
> > > >> > > > > offset, either from kafka offset topic or from the external
> > > store
> > > >> is
> > > >> > > > > invalid after partition deletion and re-creation. However,
> it
> > > >> seems
> > > >> > > that
> > > >> > > > we
> > > >> > > > > can also address this use-case with other approaches. For
> > > example,
> > > >> > when
> > > >> > > > > AdminClient deletes partitions, it can also delete the
> > committed
> > > >> > > offsets
> > > >> > > > > for those partitions from the offset topic. If user stores
> > > offset
> > > >> > > > > externally, it might make sense for user to similarly remove
> > > >> offsets
> > > >> > of
> > > >> > > > > related partitions after these partitions are deleted. So I
> am
> > > not
> > > >> > sure
> > > >> > > > > that we should use partition_epoch in this KIP.
> > > >> > > > >
> > > >> > > > >
> > > >> > > > >>
> > > >> > > > >> 61. It seems that the leader epoch returned in the
> position()
> > > >> call
> > > >> > > > should
> > > >> > > > >> the the leader epoch returned in the fetch response, not
> the
> > > one
> > > >> in
> > > >> > > the
> > > >> > > > >> metadata cache of the client.
> > > >> > > > >
> > > >> > > > >
> > > >> > > > > I think this is a good idea. Just to double check, this
> change
> > > >> does
> > > >> > not
> > > >> > > > > affect the correctness or performance of this KIP. But it
> can
> > be
> > > >> > useful
> > > >> > > > if
> > > >> > > > > we want to use the leader_epoch to better handle the offset
> > rest
> > > >> in
> > > >> > > case
> > > >> > > > of
> > > >> > > > > unclean leader election, which is listed in the future work.
> > Is
> > > >> this
> > > >> > > > > understanding correct?
> > > >> > > > >
> > > >> > > > > I have updated the KIP to specify that the leader_epoch
> > returned
> > > >> by
> > > >> > > > > position() should be the largest leader_epoch of those
> already
> > > >> > consumed
> > > >> > > > > messages whose offset < position. If no message has been
> > > consumed
> > > >> > since
> > > >> > > > > consumer initialization, the leader_epoch from seek() or
> > > >> > > > > OffsetFetchResponse should be used. The offset included in
> the
> > > >> > > > > OffsetCommitRequest will also be determined in the similar
> > > manner.
> > > >> > > > >
> > > >> > > > >
> > > >> > > > >>
> > > >> > > > >> 62. I am wondering if we should return the partition epoch
> in
> > > the
> > > >> > > fetch
> > > >> > > > >> response as well. In the current proposal, if a topic is
> > > >> recreated
> > > >> > and
> > > >> > > > the
> > > >> > > > >> new leader is on the same broker as the old one, there is
> > > >> nothing to
> > > >> > > > force
> > > >> > > > >> the metadata refresh in the client. So, the client may
> still
> > > >> > associate
> > > >> > > > the
> > > >> > > > >> offset with the old partition epoch.
> > > >> > > > >>
> > > >> > > > >
> > > >> > > > > Could you help me understand the problem if a client
> > associates
> > > >> old
> > > >> > > > > partition_epoch (or the topic_epoch as of the current KIP)
> > with
> > > >> the
> > > >> > > > offset?
> > > >> > > > > The main purpose of the topic_epoch is to be able to drop
> > > >> > leader_epoch
> > > >> > > > to 0
> > > >> > > > > after a partition is deleted and re-created. I guess you may
> > be
> > > >> > > thinking
> > > >> > > > > about using the partition_epoch to detect that the committed
> > > >> offset
> > > >> > is
> > > >> > > > > invalid? In that case, I am wondering if the alternative
> > > approach
> > > >> > > > described
> > > >> > > > > in 60) would be reasonable.
> > > >> > > > >
> > > >> > > > >
> > > >> > > > >>
> > > >> > > > >> 63. There is some subtle coordination between the
> > > >> > LeaderAndIsrRequest
> > > >> > > > and
> > > >> > > > >> UpdateMetadataRequest. Currently, when a leader changes,
> the
> > > >> > > controller
> > > >> > > > >> first sends the LeaderAndIsrRequest to the assigned
> replicas
> > > and
> > > >> the
> > > >> > > > >> UpdateMetadataRequest to every broker. So, there could be a
> > > small
> > > >> > > window
> > > >> > > > >> when the leader already receives the new partition epoch in
> > the
> > > >> > > > >> LeaderAndIsrRequest, but the metadata cache in the broker
> > > hasn't
> > > >> > been
> > > >> > > > >> updated with the latest partition epoch. Not sure what's
> the
> > > best
> > > >> > way
> > > >> > > to
> > > >> > > > >> address this issue. Perhaps we can update the metadata
> cache
> > on
> > > >> the
> > > >> > > > broker
> > > >> > > > >> with both LeaderAndIsrRequest and UpdateMetadataRequest.
> The
> > > >> > challenge
> > > >> > > > is
> > > >> > > > >> that the two have slightly different data. For example,
> only
> > > the
> > > >> > > latter
> > > >> > > > >> has
> > > >> > > > >> all endpoints.
> > > >> > > > >>
> > > >> > > > >
> > > >> > > > > I am not sure whether this is a problem. Could you explain a
> > bit
> > > >> more
> > > >> > > > what
> > > >> > > > > specific problem this small window can cause?
> > > >> > > > >
> > > >> > > > > Since client can fetch metadata from any broker in the
> > cluster,
> > > >> and
> > > >> > > given
> > > >> > > > > that different brokers receive request (e.g.
> > LeaderAndIsrRequest
> > > >> and
> > > >> > > > > UpdateMetadataRequest) in arbitrary order, the metadata
> > received
> > > >> by
> > > >> > > > client
> > > >> > > > > can be in arbitrary order (either newer or older) compared
> to
> > > the
> > > >> > > > broker's
> > > >> > > > > leadership state even if a given broker receives
> > > >> LeaderAndIsrRequest
> > > >> > > and
> > > >> > > > > UpdateMetadataRequest simultaneously. So I am not sure it is
> > > >> useful
> > > >> > to
> > > >> > > > > update broker's cache with LeaderAndIsrRequest.
> > > >> > > > >
> > > >> > > > >
> > > >> > > > >> 64. The enforcement of leader epoch in Offset commit: We
> > allow
> > > a
> > > >> > > > consumer
> > > >> > > > >> to set an arbitrary offset. So it's possible for offsets or
> > > >> leader
> > > >> > > epoch
> > > >> > > > >> to
> > > >> > > > >> go backwards. I am not sure if we could always enforce that
> > the
> > > >> > leader
> > > >> > > > >> epoch only goes up on the broker.
> > > >> > > > >>
> > > >> > > > >
> > > >> > > > > Sure. I have removed this check from the KIP.
> > > >> > > > >
> > > >> > > > > BTW, we can probably still ensure that the leader_epoch
> always
> > > >> > increase
> > > >> > > > if
> > > >> > > > > the leader_epoch used with offset commit is the
> > max(leader_epoch
> > > >> of
> > > >> > the
> > > >> > > > > message with offset = the committed offset - 1, the largest
> > > known
> > > >> > > > > leader_epoch from the metadata). But I don't have a good
> > > use-case
> > > >> for
> > > >> > > > this
> > > >> > > > > alternative definition. So I choose the keep the KIP simple
> by
> > > >> > > requiring
> > > >> > > > > leader_epoch to always increase.
> > > >> > > > >
> > > >> > > > >
> > > >> > > > >> 65. Good point on handling missing partition epoch due to
> > topic
> > > >> > > > deletion.
> > > >> > > > >> Another potential way to address this is to additionally
> > > >> propagate
> > > >> > the
> > > >> > > > >> global partition epoch to brokers and the clients. This
> way,
> > > >> when a
> > > >> > > > >> partition epoch is missing, we can use the global partition
> > > >> epoch to
> > > >> > > > >> reason
> > > >> > > > >> about which metadata is more recent.
> > > >> > > > >>
> > > >> > > > >
> > > >> > > > > This is a great idea. The global epoch can be used to order
> > the
> > > >> > > metadata
> > > >> > > > > and help us recognize the more recent metadata if a topic
> (or
> > > >> > > partition)
> > > >> > > > is
> > > >> > > > > deleted and re-created.
> > > >> > > > >
> > > >> > > > > Actually, it seems we only need to propagate the global
> epoch
> > to
> > > >> > > brokers
> > > >> > > > > and clients without propagating this epoch on a per-topic or
> > > >> > > > per-partition
> > > >> > > > > basic. Doing so would simply interface changes made this
> KIP.
> > > Does
> > > >> > this
> > > >> > > > > approach sound reasonable?
> > > >> > > > >
> > > >> > > > >
> > > >> > > > >> 66. A client may also get an offset by time using the
> > > >> > offsetForTimes()
> > > >> > > > >> api.
> > > >> > > > >> So, we probably want to include offsetInternalMetadata in
> > > >> > > > >> OffsetAndTimestamp
> > > >> > > > >> as well.
> > > >> > > > >>
> > > >> > > > >
> > > >> > > > > You are right. This probably also requires us to change the
> > > >> > > > > ListOffsetRequest as well. I will update the KIP after we
> > agree
> > > on
> > > >> > the
> > > >> > > > > solution for 65).
> > > >> > > > >
> > > >> > > > >
> > > >> > > > >>
> > > >> > > > >> 67. InteralMetadata can be a bit confusing with the
> metadata
> > > >> field
> > > >> > > > already
> > > >> > > > >> there. Perhaps we can just call it OffsetEpoch. It might be
> > > >> useful
> > > >> > to
> > > >> > > > make
> > > >> > > > >> OffsetEpoch printable at least for debugging purpose. Once
> > you
> > > do
> > > >> > > that,
> > > >> > > > we
> > > >> > > > >> are already exposing the internal fields. So, not sure if
> > it's
> > > >> worth
> > > >> > > > >> hiding
> > > >> > > > >> them. If we do want to hide them, perhaps we can have sth
> > like
> > > >> the
> > > >> > > > >> following. The binary encoding is probably more efficient
> > than
> > > >> JSON
> > > >> > > for
> > > >> > > > >> external storage.
> > > >> > > > >>
> > > >> > > > >> OffsetEpoch {
> > > >> > > > >>  static OffsetEpoch decode(byte[]);
> > > >> > > > >>
> > > >> > > > >>   public byte[] encode();
> > > >> > > > >>
> > > >> > > > >>   public String toString();
> > > >> > > > >> }
> > > >> > > > >>
> > > >> > > > >
> > > >> > > > > Thanks much. I like this solution. I have updated the KIP
> > > >> > accordingly.
> > > >> > > > >
> > > >> > > > >
> > > >> > > > >
> > > >> > > > >>
> > > >> > > > >> Jun
> > > >> > > > >>
> > > >> > > > >> On Mon, Jan 8, 2018 at 4:22 PM, Dong Lin <
> > lindong28@gmail.com>
> > > >> > wrote:
> > > >> > > > >>
> > > >> > > > >> > Hey Jason,
> > > >> > > > >> >
> > > >> > > > >> > Certainly. This sounds good. I have updated the KIP to
> > > clarity
> > > >> > that
> > > >> > > > the
> > > >> > > > >> > global epoch will be incremented by 1 each time a topic
> is
> > > >> > deleted.
> > > >> > > > >> >
> > > >> > > > >> > Thanks,
> > > >> > > > >> > Dong
> > > >> > > > >> >
> > > >> > > > >> > On Mon, Jan 8, 2018 at 4:09 PM, Jason Gustafson <
> > > >> > jason@confluent.io
> > > >> > > >
> > > >> > > > >> > wrote:
> > > >> > > > >> >
> > > >> > > > >> > > Hi Dong,
> > > >> > > > >> > >
> > > >> > > > >> > >
> > > >> > > > >> > > I think your approach will allow user to distinguish
> > > between
> > > >> the
> > > >> > > > >> metadata
> > > >> > > > >> > > > before and after the topic deletion. I also agree
> that
> > > this
> > > >> > can
> > > >> > > be
> > > >> > > > >> > > > potentially be useful to user. I am just not very
> sure
> > > >> whether
> > > >> > > we
> > > >> > > > >> > already
> > > >> > > > >> > > > have a good use-case to make the additional
> complexity
> > > >> > > worthwhile.
> > > >> > > > >> It
> > > >> > > > >> > > seems
> > > >> > > > >> > > > that this feature is kind of independent of the main
> > > >> problem
> > > >> > of
> > > >> > > > this
> > > >> > > > >> > KIP.
> > > >> > > > >> > > > Could we add this as a future work?
> > > >> > > > >> > >
> > > >> > > > >> > >
> > > >> > > > >> > > Do you think it's fair if we bump the topic epoch on
> > > deletion
> > > >> > and
> > > >> > > > >> leave
> > > >> > > > >> > > propagation of the epoch for deleted topics for future
> > > work?
> > > >> I
> > > >> > > don't
> > > >> > > > >> > think
> > > >> > > > >> > > this adds much complexity and it makes the behavior
> > > >> consistent:
> > > >> > > > every
> > > >> > > > >> > topic
> > > >> > > > >> > > mutation results in an epoch bump.
> > > >> > > > >> > >
> > > >> > > > >> > > Thanks,
> > > >> > > > >> > > Jason
> > > >> > > > >> > >
> > > >> > > > >> > > On Mon, Jan 8, 2018 at 3:14 PM, Dong Lin <
> > > >> lindong28@gmail.com>
> > > >> > > > wrote:
> > > >> > > > >> > >
> > > >> > > > >> > > > Hey Ismael,
> > > >> > > > >> > > >
> > > >> > > > >> > > > I guess we actually need user to see this field so
> that
> > > >> user
> > > >> > can
> > > >> > > > >> store
> > > >> > > > >> > > this
> > > >> > > > >> > > > value in the external store together with the offset.
> > We
> > > >> just
> > > >> > > > prefer
> > > >> > > > >> > the
> > > >> > > > >> > > > value to be opaque to discourage most users from
> > > >> interpreting
> > > >> > > this
> > > >> > > > >> > value.
> > > >> > > > >> > > > One more advantage of using such an opaque field is
> to
> > be
> > > >> able
> > > >> > > to
> > > >> > > > >> > evolve
> > > >> > > > >> > > > the information (or schema) of this value without
> > > changing
> > > >> > > > consumer
> > > >> > > > >> API
> > > >> > > > >> > > in
> > > >> > > > >> > > > the future.
> > > >> > > > >> > > >
> > > >> > > > >> > > > I also thinking it is probably OK for user to be able
> > to
> > > >> > > interpret
> > > >> > > > >> this
> > > >> > > > >> > > > value, particularly for those advanced users.
> > > >> > > > >> > > >
> > > >> > > > >> > > > Thanks,
> > > >> > > > >> > > > Dong
> > > >> > > > >> > > >
> > > >> > > > >> > > > On Mon, Jan 8, 2018 at 2:34 PM, Ismael Juma <
> > > >> > ismael@juma.me.uk>
> > > >> > > > >> wrote:
> > > >> > > > >> > > >
> > > >> > > > >> > > > > On Fri, Jan 5, 2018 at 7:15 PM, Jason Gustafson <
> > > >> > > > >> jason@confluent.io>
> > > >> > > > >> > > > > wrote:
> > > >> > > > >> > > > > >
> > > >> > > > >> > > > > > class OffsetAndMetadata {
> > > >> > > > >> > > > > >   long offset;
> > > >> > > > >> > > > > >   byte[] offsetMetadata;
> > > >> > > > >> > > > > >   String metadata;
> > > >> > > > >> > > > > > }
> > > >> > > > >> > > > >
> > > >> > > > >> > > > >
> > > >> > > > >> > > > > > Admittedly, the naming is a bit annoying, but we
> > can
> > > >> > > probably
> > > >> > > > >> come
> > > >> > > > >> > up
> > > >> > > > >> > > > > with
> > > >> > > > >> > > > > > something better. Internally the byte array would
> > > have
> > > >> a
> > > >> > > > >> version.
> > > >> > > > >> > If
> > > >> > > > >> > > in
> > > >> > > > >> > > > > the
> > > >> > > > >> > > > > > future we have anything else we need to add, we
> can
> > > >> update
> > > >> > > the
> > > >> > > > >> > > version
> > > >> > > > >> > > > > and
> > > >> > > > >> > > > > > we wouldn't need any new APIs.
> > > >> > > > >> > > > > >
> > > >> > > > >> > > > >
> > > >> > > > >> > > > > We can also add fields to a class in a compatible
> > way.
> > > >> So,
> > > >> > it
> > > >> > > > >> seems
> > > >> > > > >> > to
> > > >> > > > >> > > me
> > > >> > > > >> > > > > that the main advantage of the byte array is that
> > it's
> > > >> > opaque
> > > >> > > to
> > > >> > > > >> the
> > > >> > > > >> > > > user.
> > > >> > > > >> > > > > Is that correct? If so, we could also add any
> opaque
> > > >> > metadata
> > > >> > > > in a
> > > >> > > > >> > > > subclass
> > > >> > > > >> > > > > so that users don't even see it (unless they cast
> it,
> > > but
> > > >> > then
> > > >> > > > >> > they're
> > > >> > > > >> > > on
> > > >> > > > >> > > > > their own).
> > > >> > > > >> > > > >
> > > >> > > > >> > > > > Ismael
> > > >> > > > >> > > > >
> > > >> > > > >> > > > > The corresponding seek() and position() APIs might
> > look
> > > >> > > > something
> > > >> > > > >> > like
> > > >> > > > >> > > > > this:
> > > >> > > > >> > > > > >
> > > >> > > > >> > > > > > void seek(TopicPartition partition, long offset,
> > > byte[]
> > > >> > > > >> > > > offsetMetadata);
> > > >> > > > >> > > > > > byte[] positionMetadata(TopicPartition
> partition);
> > > >> > > > >> > > > > >
> > > >> > > > >> > > > > > What do you think?
> > > >> > > > >> > > > > >
> > > >> > > > >> > > > > > Thanks,
> > > >> > > > >> > > > > > Jason
> > > >> > > > >> > > > > >
> > > >> > > > >> > > > > > On Thu, Jan 4, 2018 at 7:04 PM, Dong Lin <
> > > >> > > lindong28@gmail.com
> > > >> > > > >
> > > >> > > > >> > > wrote:
> > > >> > > > >> > > > > >
> > > >> > > > >> > > > > > > Hey Jun, Jason,
> > > >> > > > >> > > > > > >
> > > >> > > > >> > > > > > > Thanks much for all the feedback. I have
> updated
> > > the
> > > >> KIP
> > > >> > > > >> based on
> > > >> > > > >> > > the
> > > >> > > > >> > > > > > > latest discussion. Can you help check whether
> it
> > > >> looks
> > > >> > > good?
> > > >> > > > >> > > > > > >
> > > >> > > > >> > > > > > > Thanks,
> > > >> > > > >> > > > > > > Dong
> > > >> > > > >> > > > > > >
> > > >> > > > >> > > > > > > On Thu, Jan 4, 2018 at 5:36 PM, Dong Lin <
> > > >> > > > lindong28@gmail.com
> > > >> > > > >> >
> > > >> > > > >> > > > wrote:
> > > >> > > > >> > > > > > >
> > > >> > > > >> > > > > > > > Hey Jun,
> > > >> > > > >> > > > > > > >
> > > >> > > > >> > > > > > > > Hmm... thinking about this more, I am not
> sure
> > > that
> > > >> > the
> > > >> > > > >> > proposed
> > > >> > > > >> > > > API
> > > >> > > > >> > > > > is
> > > >> > > > >> > > > > > > > sufficient. For users that store offset
> > > >> externally, we
> > > >> > > > >> probably
> > > >> > > > >> > > > need
> > > >> > > > >> > > > > > > extra
> > > >> > > > >> > > > > > > > API to return the leader_epoch and
> > > partition_epoch
> > > >> for
> > > >> > > all
> > > >> > > > >> > > > partitions
> > > >> > > > >> > > > > > > that
> > > >> > > > >> > > > > > > > consumers are consuming. I suppose these
> users
> > > >> > currently
> > > >> > > > use
> > > >> > > > >> > > > > position()
> > > >> > > > >> > > > > > > to
> > > >> > > > >> > > > > > > > get the offset. Thus we probably need a new
> > > method
> > > >> > > > >> > > > > > positionWithEpoch(..)
> > > >> > > > >> > > > > > > to
> > > >> > > > >> > > > > > > > return <offset, partition_epoch,
> leader_epoch>.
> > > >> Does
> > > >> > > this
> > > >> > > > >> sound
> > > >> > > > >> > > > > > > reasonable?
> > > >> > > > >> > > > > > > >
> > > >> > > > >> > > > > > > > Thanks,
> > > >> > > > >> > > > > > > > Dong
> > > >> > > > >> > > > > > > >
> > > >> > > > >> > > > > > > >
> > > >> > > > >> > > > > > > > On Thu, Jan 4, 2018 at 5:26 PM, Jun Rao <
> > > >> > > jun@confluent.io
> > > >> > > > >
> > > >> > > > >> > > wrote:
> > > >> > > > >> > > > > > > >
> > > >> > > > >> > > > > > > >> Hi, Dong,
> > > >> > > > >> > > > > > > >>
> > > >> > > > >> > > > > > > >> Yes, that's what I am thinking. OffsetEpoch
> > will
> > > >> be
> > > >> > > > >> composed
> > > >> > > > >> > of
> > > >> > > > >> > > > > > > >> (partition_epoch,
> > > >> > > > >> > > > > > > >> leader_epoch).
> > > >> > > > >> > > > > > > >>
> > > >> > > > >> > > > > > > >> Thanks,
> > > >> > > > >> > > > > > > >>
> > > >> > > > >> > > > > > > >> Jun
> > > >> > > > >> > > > > > > >>
> > > >> > > > >> > > > > > > >>
> > > >> > > > >> > > > > > > >> On Thu, Jan 4, 2018 at 4:22 PM, Dong Lin <
> > > >> > > > >> lindong28@gmail.com
> > > >> > > > >> > >
> > > >> > > > >> > > > > wrote:
> > > >> > > > >> > > > > > > >>
> > > >> > > > >> > > > > > > >> > Hey Jun,
> > > >> > > > >> > > > > > > >> >
> > > >> > > > >> > > > > > > >> > Thanks much. I like the the new API that
> you
> > > >> > > proposed.
> > > >> > > > I
> > > >> > > > >> am
> > > >> > > > >> > > not
> > > >> > > > >> > > > > sure
> > > >> > > > >> > > > > > > >> what
> > > >> > > > >> > > > > > > >> > you exactly mean by offset_epoch. I
> suppose
> > > >> that we
> > > >> > > can
> > > >> > > > >> use
> > > >> > > > >> > > the
> > > >> > > > >> > > > > pair
> > > >> > > > >> > > > > > > of
> > > >> > > > >> > > > > > > >> > (partition_epoch, leader_epoch) as the
> > > >> > offset_epoch,
> > > >> > > > >> right?
> > > >> > > > >> > > > > > > >> >
> > > >> > > > >> > > > > > > >> > Thanks,
> > > >> > > > >> > > > > > > >> > Dong
> > > >> > > > >> > > > > > > >> >
> > > >> > > > >> > > > > > > >> > On Thu, Jan 4, 2018 at 4:02 PM, Jun Rao <
> > > >> > > > >> jun@confluent.io>
> > > >> > > > >> > > > wrote:
> > > >> > > > >> > > > > > > >> >
> > > >> > > > >> > > > > > > >> > > Hi, Dong,
> > > >> > > > >> > > > > > > >> > >
> > > >> > > > >> > > > > > > >> > > Got it. The api that you proposed works.
> > The
> > > >> > > question
> > > >> > > > >> is
> > > >> > > > >> > > > whether
> > > >> > > > >> > > > > > > >> that's
> > > >> > > > >> > > > > > > >> > the
> > > >> > > > >> > > > > > > >> > > api that we want to have in the long
> term.
> > > My
> > > >> > > concern
> > > >> > > > >> is
> > > >> > > > >> > > that
> > > >> > > > >> > > > > > while
> > > >> > > > >> > > > > > > >> the
> > > >> > > > >> > > > > > > >> > api
> > > >> > > > >> > > > > > > >> > > change is simple, the new api seems
> harder
> > > to
> > > >> > > explain
> > > >> > > > >> and
> > > >> > > > >> > > use.
> > > >> > > > >> > > > > For
> > > >> > > > >> > > > > > > >> > example,
> > > >> > > > >> > > > > > > >> > > a consumer storing offsets externally
> now
> > > >> needs
> > > >> > to
> > > >> > > > call
> > > >> > > > >> > > > > > > >> > > waitForMetadataUpdate() after calling
> > > seek().
> > > >> > > > >> > > > > > > >> > >
> > > >> > > > >> > > > > > > >> > > An alternative approach is to make the
> > > >> following
> > > >> > > > >> > compatible
> > > >> > > > >> > > > api
> > > >> > > > >> > > > > > > >> changes
> > > >> > > > >> > > > > > > >> > in
> > > >> > > > >> > > > > > > >> > > Consumer.
> > > >> > > > >> > > > > > > >> > > * Add an additional OffsetEpoch field in
> > > >> > > > >> > OffsetAndMetadata.
> > > >> > > > >> > > > (no
> > > >> > > > >> > > > > > need
> > > >> > > > >> > > > > > > >> to
> > > >> > > > >> > > > > > > >> > > change the CommitSync() api)
> > > >> > > > >> > > > > > > >> > > * Add a new api seek(TopicPartition
> > > partition,
> > > >> > long
> > > >> > > > >> > offset,
> > > >> > > > >> > > > > > > >> OffsetEpoch
> > > >> > > > >> > > > > > > >> > > offsetEpoch). We can potentially
> deprecate
> > > the
> > > >> > old
> > > >> > > > api
> > > >> > > > >> > > > > > > >> > seek(TopicPartition
> > > >> > > > >> > > > > > > >> > > partition, long offset) in the future.
> > > >> > > > >> > > > > > > >> > >
> > > >> > > > >> > > > > > > >> > > The alternative approach has similar
> > amount
> > > of
> > > >> > api
> > > >> > > > >> changes
> > > >> > > > >> > > as
> > > >> > > > >> > > > > > yours
> > > >> > > > >> > > > > > > >> but
> > > >> > > > >> > > > > > > >> > has
> > > >> > > > >> > > > > > > >> > > the following benefits.
> > > >> > > > >> > > > > > > >> > > 1. The api works in a similar way as how
> > > >> offset
> > > >> > > > >> management
> > > >> > > > >> > > > works
> > > >> > > > >> > > > > > now
> > > >> > > > >> > > > > > > >> and
> > > >> > > > >> > > > > > > >> > is
> > > >> > > > >> > > > > > > >> > > probably what we want in the long term.
> > > >> > > > >> > > > > > > >> > > 2. It can reset offsets better when
> there
> > is
> > > >> data
> > > >> > > > loss
> > > >> > > > >> due
> > > >> > > > >> > > to
> > > >> > > > >> > > > > > > unclean
> > > >> > > > >> > > > > > > >> > > leader election or correlated replica
> > > failure.
> > > >> > > > >> > > > > > > >> > > 3. It can reset offsets better when
> topic
> > is
> > > >> > > > recreated.
> > > >> > > > >> > > > > > > >> > >
> > > >> > > > >> > > > > > > >> > > Thanks,
> > > >> > > > >> > > > > > > >> > >
> > > >> > > > >> > > > > > > >> > > Jun
> > > >> > > > >> > > > > > > >> > >
> > > >> > > > >> > > > > > > >> > >
> > > >> > > > >> > > > > > > >> > > On Thu, Jan 4, 2018 at 2:05 PM, Dong
> Lin <
> > > >> > > > >> > > lindong28@gmail.com
> > > >> > > > >> > > > >
> > > >> > > > >> > > > > > > wrote:
> > > >> > > > >> > > > > > > >> > >
> > > >> > > > >> > > > > > > >> > > > Hey Jun,
> > > >> > > > >> > > > > > > >> > > >
> > > >> > > > >> > > > > > > >> > > > Yeah I agree that ideally we don't
> want
> > an
> > > >> ever
> > > >> > > > >> growing
> > > >> > > > >> > > > global
> > > >> > > > >> > > > > > > >> metadata
> > > >> > > > >> > > > > > > >> > > > version. I just think it may be more
> > > >> desirable
> > > >> > to
> > > >> > > > >> keep
> > > >> > > > >> > the
> > > >> > > > >> > > > > > > consumer
> > > >> > > > >> > > > > > > >> API
> > > >> > > > >> > > > > > > >> > > > simple.
> > > >> > > > >> > > > > > > >> > > >
> > > >> > > > >> > > > > > > >> > > > In my current proposal, metadata
> version
> > > >> > returned
> > > >> > > > in
> > > >> > > > >> the
> > > >> > > > >> > > > fetch
> > > >> > > > >> > > > > > > >> response
> > > >> > > > >> > > > > > > >> > > > will be stored with the offset
> together.
> > > >> More
> > > >> > > > >> > > specifically,
> > > >> > > > >> > > > > the
> > > >> > > > >> > > > > > > >> > > > metadata_epoch in the new offset topic
> > > >> schema
> > > >> > > will
> > > >> > > > be
> > > >> > > > >> > the
> > > >> > > > >> > > > > > largest
> > > >> > > > >> > > > > > > >> > > > metadata_epoch from all the
> > > MetadataResponse
> > > >> > and
> > > >> > > > >> > > > FetchResponse
> > > >> > > > >> > > > > > > ever
> > > >> > > > >> > > > > > > >> > > > received by this consumer.
> > > >> > > > >> > > > > > > >> > > >
> > > >> > > > >> > > > > > > >> > > > We probably don't have to change the
> > > >> consumer
> > > >> > API
> > > >> > > > for
> > > >> > > > >> > > > > > > >> > > > commitSync(Map<TopicPartition,
> > > >> > > OffsetAndMetadata>).
> > > >> > > > >> If
> > > >> > > > >> > > user
> > > >> > > > >> > > > > > calls
> > > >> > > > >> > > > > > > >> > > > commitSync(...) to commit offset 10
> for
> > a
> > > >> given
> > > >> > > > >> > partition,
> > > >> > > > >> > > > for
> > > >> > > > >> > > > > > > most
> > > >> > > > >> > > > > > > >> > > > use-cases, this consumer instance
> should
> > > >> have
> > > >> > > > >> consumed
> > > >> > > > >> > > > message
> > > >> > > > >> > > > > > > with
> > > >> > > > >> > > > > > > >> > > offset
> > > >> > > > >> > > > > > > >> > > > 9 from this partition, in which case
> the
> > > >> > consumer
> > > >> > > > can
> > > >> > > > >> > > > remember
> > > >> > > > >> > > > > > and
> > > >> > > > >> > > > > > > >> use
> > > >> > > > >> > > > > > > >> > > the
> > > >> > > > >> > > > > > > >> > > > metadata_epoch from the corresponding
> > > >> > > FetchResponse
> > > >> > > > >> when
> > > >> > > > >> > > > > > > committing
> > > >> > > > >> > > > > > > >> > > offset.
> > > >> > > > >> > > > > > > >> > > > If user calls commitSync(..) to commit
> > > >> offset
> > > >> > 10
> > > >> > > > for
> > > >> > > > >> a
> > > >> > > > >> > > given
> > > >> > > > >> > > > > > > >> partition
> > > >> > > > >> > > > > > > >> > > > without having consumed the message
> with
> > > >> > offset 9
> > > >> > > > >> using
> > > >> > > > >> > > this
> > > >> > > > >> > > > > > > >> consumer
> > > >> > > > >> > > > > > > >> > > > instance, this is probably an advanced
> > > >> > use-case.
> > > >> > > In
> > > >> > > > >> this
> > > >> > > > >> > > > case
> > > >> > > > >> > > > > > the
> > > >> > > > >> > > > > > > >> > > advanced
> > > >> > > > >> > > > > > > >> > > > user can retrieve the metadata_epoch
> > using
> > > >> the
> > > >> > > > newly
> > > >> > > > >> > added
> > > >> > > > >> > > > > > > >> > > metadataEpoch()
> > > >> > > > >> > > > > > > >> > > > API after it fetches the message with
> > > >> offset 9
> > > >> > > > >> (probably
> > > >> > > > >> > > > from
> > > >> > > > >> > > > > > > >> another
> > > >> > > > >> > > > > > > >> > > > consumer instance) and encode this
> > > >> > metadata_epoch
> > > >> > > > in
> > > >> > > > >> the
> > > >> > > > >> > > > > > > >> > > > string OffsetAndMetadata.metadata. Do
> > you
> > > >> think
> > > >> > > > this
> > > >> > > > >> > > > solution
> > > >> > > > >> > > > > > > would
> > > >> > > > >> > > > > > > >> > work?
> > > >> > > > >> > > > > > > >> > > >
> > > >> > > > >> > > > > > > >> > > > By "not sure that I fully understand
> > your
> > > >> > latest
> > > >> > > > >> > > > suggestion",
> > > >> > > > >> > > > > > are
> > > >> > > > >> > > > > > > >> you
> > > >> > > > >> > > > > > > >> > > > referring to solution related to
> unclean
> > > >> leader
> > > >> > > > >> election
> > > >> > > > >> > > > using
> > > >> > > > >> > > > > > > >> > > leader_epoch
> > > >> > > > >> > > > > > > >> > > > in my previous email?
> > > >> > > > >> > > > > > > >> > > >
> > > >> > > > >> > > > > > > >> > > > Thanks,
> > > >> > > > >> > > > > > > >> > > > Dong
> > > >> > > > >> > > > > > > >> > > >
> > > >> > > > >> > > > > > > >> > > > On Thu, Jan 4, 2018 at 1:33 PM, Jun
> Rao
> > <
> > > >> > > > >> > jun@confluent.io
> > > >> > > > >> > > >
> > > >> > > > >> > > > > > wrote:
> > > >> > > > >> > > > > > > >> > > >
> > > >> > > > >> > > > > > > >> > > > > Hi, Dong,
> > > >> > > > >> > > > > > > >> > > > >
> > > >> > > > >> > > > > > > >> > > > > Not sure that I fully understand
> your
> > > >> latest
> > > >> > > > >> > suggestion.
> > > >> > > > >> > > > > > > >> Returning an
> > > >> > > > >> > > > > > > >> > > > ever
> > > >> > > > >> > > > > > > >> > > > > growing global metadata version
> itself
> > > is
> > > >> no
> > > >> > > > ideal,
> > > >> > > > >> > but
> > > >> > > > >> > > is
> > > >> > > > >> > > > > > fine.
> > > >> > > > >> > > > > > > >> My
> > > >> > > > >> > > > > > > >> > > > > question is whether the metadata
> > version
> > > >> > > returned
> > > >> > > > >> in
> > > >> > > > >> > the
> > > >> > > > >> > > > > fetch
> > > >> > > > >> > > > > > > >> > response
> > > >> > > > >> > > > > > > >> > > > > needs to be stored with the offset
> > > >> together
> > > >> > if
> > > >> > > > >> offsets
> > > >> > > > >> > > are
> > > >> > > > >> > > > > > > stored
> > > >> > > > >> > > > > > > >> > > > > externally. If so, we also have to
> > > change
> > > >> the
> > > >> > > > >> consumer
> > > >> > > > >> > > API
> > > >> > > > >> > > > > for
> > > >> > > > >> > > > > > > >> > > > commitSync()
> > > >> > > > >> > > > > > > >> > > > > and need to worry about
> compatibility.
> > > If
> > > >> we
> > > >> > > > don't
> > > >> > > > >> > store
> > > >> > > > >> > > > the
> > > >> > > > >> > > > > > > >> metadata
> > > >> > > > >> > > > > > > >> > > > > version together with the offset,
> on a
> > > >> > consumer
> > > >> > > > >> > restart,
> > > >> > > > >> > > > > it's
> > > >> > > > >> > > > > > > not
> > > >> > > > >> > > > > > > >> > clear
> > > >> > > > >> > > > > > > >> > > > how
> > > >> > > > >> > > > > > > >> > > > > we can ensure the metadata in the
> > > >> consumer is
> > > >> > > > high
> > > >> > > > >> > > enough
> > > >> > > > >> > > > > > since
> > > >> > > > >> > > > > > > >> there
> > > >> > > > >> > > > > > > >> > > is
> > > >> > > > >> > > > > > > >> > > > no
> > > >> > > > >> > > > > > > >> > > > > metadata version to compare with.
> > > >> > > > >> > > > > > > >> > > > >
> > > >> > > > >> > > > > > > >> > > > > Thanks,
> > > >> > > > >> > > > > > > >> > > > >
> > > >> > > > >> > > > > > > >> > > > > Jun
> > > >> > > > >> > > > > > > >> > > > >
> > > >> > > > >> > > > > > > >> > > > >
> > > >> > > > >> > > > > > > >> > > > > On Wed, Jan 3, 2018 at 6:43 PM, Dong
> > > Lin <
> > > >> > > > >> > > > > lindong28@gmail.com
> > > >> > > > >> > > > > > >
> > > >> > > > >> > > > > > > >> > wrote:
> > > >> > > > >> > > > > > > >> > > > >
> > > >> > > > >> > > > > > > >> > > > > > Hey Jun,
> > > >> > > > >> > > > > > > >> > > > > >
> > > >> > > > >> > > > > > > >> > > > > > Thanks much for the explanation.
> > > >> > > > >> > > > > > > >> > > > > >
> > > >> > > > >> > > > > > > >> > > > > > I understand the advantage of
> > > >> > partition_epoch
> > > >> > > > >> over
> > > >> > > > >> > > > > > > >> metadata_epoch.
> > > >> > > > >> > > > > > > >> > My
> > > >> > > > >> > > > > > > >> > > > > > current concern is that the use of
> > > >> > > leader_epoch
> > > >> > > > >> and
> > > >> > > > >> > > the
> > > >> > > > >> > > > > > > >> > > partition_epoch
> > > >> > > > >> > > > > > > >> > > > > > requires us considerable change to
> > > >> > consumer's
> > > >> > > > >> public
> > > >> > > > >> > > API
> > > >> > > > >> > > > > to
> > > >> > > > >> > > > > > > take
> > > >> > > > >> > > > > > > >> > care
> > > >> > > > >> > > > > > > >> > > > of
> > > >> > > > >> > > > > > > >> > > > > > the case where user stores offset
> > > >> > externally.
> > > >> > > > For
> > > >> > > > >> > > > example,
> > > >> > > > >> > > > > > > >> > > *consumer*.
> > > >> > > > >> > > > > > > >> > > > > > *commitSync*(..) would have to
> take
> > a
> > > >> map
> > > >> > > whose
> > > >> > > > >> > value
> > > >> > > > >> > > is
> > > >> > > > >> > > > > > > >> <offset,
> > > >> > > > >> > > > > > > >> > > > > metadata,
> > > >> > > > >> > > > > > > >> > > > > > leader epoch, partition epoch>.
> > > >> > > > >> > *consumer*.*seek*(...)
> > > >> > > > >> > > > > would
> > > >> > > > >> > > > > > > >> also
> > > >> > > > >> > > > > > > >> > > need
> > > >> > > > >> > > > > > > >> > > > > > leader_epoch and partition_epoch
> as
> > > >> > > parameter.
> > > >> > > > >> > > > Technically
> > > >> > > > >> > > > > > we
> > > >> > > > >> > > > > > > >> can
> > > >> > > > >> > > > > > > >> > > > > probably
> > > >> > > > >> > > > > > > >> > > > > > still make it work in a backward
> > > >> compatible
> > > >> > > > >> manner
> > > >> > > > >> > > after
> > > >> > > > >> > > > > > > careful
> > > >> > > > >> > > > > > > >> > > design
> > > >> > > > >> > > > > > > >> > > > > and
> > > >> > > > >> > > > > > > >> > > > > > discussion. But these changes can
> > make
> > > >> the
> > > >> > > > >> > consumer's
> > > >> > > > >> > > > > > > interface
> > > >> > > > >> > > > > > > >> > > > > > unnecessarily complex for more
> users
> > > >> who do
> > > >> > > not
> > > >> > > > >> > store
> > > >> > > > >> > > > > offset
> > > >> > > > >> > > > > > > >> > > > externally.
> > > >> > > > >> > > > > > > >> > > > > >
> > > >> > > > >> > > > > > > >> > > > > > After thinking more about it, we
> can
> > > >> > address
> > > >> > > > all
> > > >> > > > >> > > > problems
> > > >> > > > >> > > > > > > >> discussed
> > > >> > > > >> > > > > > > >> > > by
> > > >> > > > >> > > > > > > >> > > > > only
> > > >> > > > >> > > > > > > >> > > > > > using the metadata_epoch without
> > > >> > introducing
> > > >> > > > >> > > > leader_epoch
> > > >> > > > >> > > > > or
> > > >> > > > >> > > > > > > the
> > > >> > > > >> > > > > > > >> > > > > > partition_epoch. The current KIP
> > > >> describes
> > > >> > > the
> > > >> > > > >> > changes
> > > >> > > > >> > > > to
> > > >> > > > >> > > > > > the
> > > >> > > > >> > > > > > > >> > > consumer
> > > >> > > > >> > > > > > > >> > > > > API
> > > >> > > > >> > > > > > > >> > > > > > and how the new API can be used if
> > > user
> > > >> > > stores
> > > >> > > > >> > offset
> > > >> > > > >> > > > > > > >> externally.
> > > >> > > > >> > > > > > > >> > In
> > > >> > > > >> > > > > > > >> > > > > order
> > > >> > > > >> > > > > > > >> > > > > > to address the scenario you
> > described
> > > >> > > earlier,
> > > >> > > > we
> > > >> > > > >> > can
> > > >> > > > >> > > > > > include
> > > >> > > > >> > > > > > > >> > > > > > metadata_epoch in the
> FetchResponse
> > > and
> > > >> the
> > > >> > > > >> > > > > > > LeaderAndIsrRequest.
> > > >> > > > >> > > > > > > >> > > > Consumer
> > > >> > > > >> > > > > > > >> > > > > > remembers the largest
> metadata_epoch
> > > >> from
> > > >> > all
> > > >> > > > the
> > > >> > > > >> > > > > > > FetchResponse
> > > >> > > > >> > > > > > > >> it
> > > >> > > > >> > > > > > > >> > > has
> > > >> > > > >> > > > > > > >> > > > > > received. The metadata_epoch
> > committed
> > > >> with
> > > >> > > the
> > > >> > > > >> > > offset,
> > > >> > > > >> > > > > > either
> > > >> > > > >> > > > > > > >> > within
> > > >> > > > >> > > > > > > >> > > > or
> > > >> > > > >> > > > > > > >> > > > > > outside Kafka, should be the
> largest
> > > >> > > > >> metadata_epoch
> > > >> > > > >> > > > across
> > > >> > > > >> > > > > > all
> > > >> > > > >> > > > > > > >> > > > > > FetchResponse and MetadataResponse
> > > ever
> > > >> > > > received
> > > >> > > > >> by
> > > >> > > > >> > > this
> > > >> > > > >> > > > > > > >> consumer.
> > > >> > > > >> > > > > > > >> > > > > >
> > > >> > > > >> > > > > > > >> > > > > > The drawback of using only the
> > > >> > metadata_epoch
> > > >> > > > is
> > > >> > > > >> > that
> > > >> > > > >> > > we
> > > >> > > > >> > > > > can
> > > >> > > > >> > > > > > > not
> > > >> > > > >> > > > > > > >> > > always
> > > >> > > > >> > > > > > > >> > > > > do
> > > >> > > > >> > > > > > > >> > > > > > the smart offset reset in case of
> > > >> unclean
> > > >> > > > leader
> > > >> > > > >> > > > election
> > > >> > > > >> > > > > > > which
> > > >> > > > >> > > > > > > >> you
> > > >> > > > >> > > > > > > >> > > > > > mentioned earlier. But in most
> case,
> > > >> > unclean
> > > >> > > > >> leader
> > > >> > > > >> > > > > election
> > > >> > > > >> > > > > > > >> > probably
> > > >> > > > >> > > > > > > >> > > > > > happens when consumer is not
> > > >> > > > >> rebalancing/restarting.
> > > >> > > > >> > > In
> > > >> > > > >> > > > > > these
> > > >> > > > >> > > > > > > >> > cases,
> > > >> > > > >> > > > > > > >> > > > > either
> > > >> > > > >> > > > > > > >> > > > > > consumer is not directly affected
> by
> > > >> > unclean
> > > >> > > > >> leader
> > > >> > > > >> > > > > election
> > > >> > > > >> > > > > > > >> since
> > > >> > > > >> > > > > > > >> > it
> > > >> > > > >> > > > > > > >> > > > is
> > > >> > > > >> > > > > > > >> > > > > > not consuming from the end of the
> > log,
> > > >> or
> > > >> > > > >> consumer
> > > >> > > > >> > can
> > > >> > > > >> > > > > > derive
> > > >> > > > >> > > > > > > >> the
> > > >> > > > >> > > > > > > >> > > > > > leader_epoch from the most recent
> > > >> message
> > > >> > > > >> received
> > > >> > > > >> > > > before
> > > >> > > > >> > > > > it
> > > >> > > > >> > > > > > > >> sees
> > > >> > > > >> > > > > > > >> > > > > > OffsetOutOfRangeException. So I am
> > not
> > > >> sure
> > > >> > > it
> > > >> > > > is
> > > >> > > > >> > > worth
> > > >> > > > >> > > > > > adding
> > > >> > > > >> > > > > > > >> the
> > > >> > > > >> > > > > > > >> > > > > > leader_epoch to consumer API to
> > > address
> > > >> the
> > > >> > > > >> > remaining
> > > >> > > > >> > > > > corner
> > > >> > > > >> > > > > > > >> case.
> > > >> > > > >> > > > > > > >> > > What
> > > >> > > > >> > > > > > > >> > > > > do
> > > >> > > > >> > > > > > > >> > > > > > you think?
> > > >> > > > >> > > > > > > >> > > > > >
> > > >> > > > >> > > > > > > >> > > > > > Thanks,
> > > >> > > > >> > > > > > > >> > > > > > Dong
> > > >> > > > >> > > > > > > >> > > > > >
> > > >> > > > >> > > > > > > >> > > > > >
> > > >> > > > >> > > > > > > >> > > > > >
> > > >> > > > >> > > > > > > >> > > > > > On Tue, Jan 2, 2018 at 6:28 PM,
> Jun
> > > Rao
> > > >> <
> > > >> > > > >> > > > jun@confluent.io
> > > >> > > > >> > > > > >
> > > >> > > > >> > > > > > > >> wrote:
> > > >> > > > >> > > > > > > >> > > > > >
> > > >> > > > >> > > > > > > >> > > > > > > Hi, Dong,
> > > >> > > > >> > > > > > > >> > > > > > >
> > > >> > > > >> > > > > > > >> > > > > > > Thanks for the reply.
> > > >> > > > >> > > > > > > >> > > > > > >
> > > >> > > > >> > > > > > > >> > > > > > > To solve the topic recreation
> > issue,
> > > >> we
> > > >> > > could
> > > >> > > > >> use
> > > >> > > > >> > > > > either a
> > > >> > > > >> > > > > > > >> global
> > > >> > > > >> > > > > > > >> > > > > > metadata
> > > >> > > > >> > > > > > > >> > > > > > > version or a partition level
> > epoch.
> > > >> But
> > > >> > > > either
> > > >> > > > >> one
> > > >> > > > >> > > > will
> > > >> > > > >> > > > > > be a
> > > >> > > > >> > > > > > > >> new
> > > >> > > > >> > > > > > > >> > > > > concept,
> > > >> > > > >> > > > > > > >> > > > > > > right? To me, the latter seems
> > more
> > > >> > > natural.
> > > >> > > > It
> > > >> > > > >> > also
> > > >> > > > >> > > > > makes
> > > >> > > > >> > > > > > > it
> > > >> > > > >> > > > > > > >> > > easier
> > > >> > > > >> > > > > > > >> > > > to
> > > >> > > > >> > > > > > > >> > > > > > > detect if a consumer's offset is
> > > still
> > > >> > > valid
> > > >> > > > >> > after a
> > > >> > > > >> > > > > topic
> > > >> > > > >> > > > > > > is
> > > >> > > > >> > > > > > > >> > > > > recreated.
> > > >> > > > >> > > > > > > >> > > > > > As
> > > >> > > > >> > > > > > > >> > > > > > > you pointed out, we don't need
> to
> > > >> store
> > > >> > the
> > > >> > > > >> > > partition
> > > >> > > > >> > > > > > epoch
> > > >> > > > >> > > > > > > in
> > > >> > > > >> > > > > > > >> > the
> > > >> > > > >> > > > > > > >> > > > > > message.
> > > >> > > > >> > > > > > > >> > > > > > > The following is what I am
> > thinking.
> > > >> > When a
> > > >> > > > >> > > partition
> > > >> > > > >> > > > is
> > > >> > > > >> > > > > > > >> created,
> > > >> > > > >> > > > > > > >> > > we
> > > >> > > > >> > > > > > > >> > > > > can
> > > >> > > > >> > > > > > > >> > > > > > > assign a partition epoch from an
> > > >> > > > >> ever-increasing
> > > >> > > > >> > > > global
> > > >> > > > >> > > > > > > >> counter
> > > >> > > > >> > > > > > > >> > and
> > > >> > > > >> > > > > > > >> > > > > store
> > > >> > > > >> > > > > > > >> > > > > > > it in /brokers/topics/[topic]/
> > > >> > > > >> > > > partitions/[partitionId]
> > > >> > > > >> > > > > in
> > > >> > > > >> > > > > > > ZK.
> > > >> > > > >> > > > > > > >> > The
> > > >> > > > >> > > > > > > >> > > > > > > partition
> > > >> > > > >> > > > > > > >> > > > > > > epoch is propagated to every
> > broker.
> > > >> The
> > > >> > > > >> consumer
> > > >> > > > >> > > will
> > > >> > > > >> > > > > be
> > > >> > > > >> > > > > > > >> > tracking
> > > >> > > > >> > > > > > > >> > > a
> > > >> > > > >> > > > > > > >> > > > > > tuple
> > > >> > > > >> > > > > > > >> > > > > > > of <offset, leader epoch,
> > partition
> > > >> > epoch>
> > > >> > > > for
> > > >> > > > >> > > > offsets.
> > > >> > > > >> > > > > > If a
> > > >> > > > >> > > > > > > >> > topic
> > > >> > > > >> > > > > > > >> > > is
> > > >> > > > >> > > > > > > >> > > > > > > recreated, it's possible that a
> > > >> > consumer's
> > > >> > > > >> offset
> > > >> > > > >> > > and
> > > >> > > > >> > > > > > leader
> > > >> > > > >> > > > > > > >> > epoch
> > > >> > > > >> > > > > > > >> > > > > still
> > > >> > > > >> > > > > > > >> > > > > > > match that in the broker, but
> > > >> partition
> > > >> > > epoch
> > > >> > > > >> > won't
> > > >> > > > >> > > > be.
> > > >> > > > >> > > > > In
> > > >> > > > >> > > > > > > >> this
> > > >> > > > >> > > > > > > >> > > case,
> > > >> > > > >> > > > > > > >> > > > > we
> > > >> > > > >> > > > > > > >> > > > > > > can potentially still treat the
> > > >> > consumer's
> > > >> > > > >> offset
> > > >> > > > >> > as
> > > >> > > > >> > > > out
> > > >> > > > >> > > > > > of
> > > >> > > > >> > > > > > > >> range
> > > >> > > > >> > > > > > > >> > > and
> > > >> > > > >> > > > > > > >> > > > > > reset
> > > >> > > > >> > > > > > > >> > > > > > > the offset based on the offset
> > reset
> > > >> > policy
> > > >> > > > in
> > > >> > > > >> the
> > > >> > > > >> > > > > > consumer.
> > > >> > > > >> > > > > > > >> This
> > > >> > > > >> > > > > > > >> > > > seems
> > > >> > > > >> > > > > > > >> > > > > > > harder to do with a global
> > metadata
> > > >> > > version.
> > > >> > > > >> > > > > > > >> > > > > > >
> > > >> > > > >> > > > > > > >> > > > > > > Jun
> > > >> > > > >> > > > > > > >> > > > > > >
> > > >> > > > >> > > > > > > >> > > > > > >
> > > >> > > > >> > > > > > > >> > > > > > >
> > > >> > > > >> > > > > > > >> > > > > > > On Mon, Dec 25, 2017 at 6:56 AM,
> > > Dong
> > > >> > Lin <
> > > >> > > > >> > > > > > > >> lindong28@gmail.com>
> > > >> > > > >> > > > > > > >> > > > wrote:
> > > >> > > > >> > > > > > > >> > > > > > >
> > > >> > > > >> > > > > > > >> > > > > > > > Hey Jun,
> > > >> > > > >> > > > > > > >> > > > > > > >
> > > >> > > > >> > > > > > > >> > > > > > > > This is a very good example.
> > After
> > > >> > > thinking
> > > >> > > > >> > > through
> > > >> > > > >> > > > > this
> > > >> > > > >> > > > > > > in
> > > >> > > > >> > > > > > > >> > > > detail, I
> > > >> > > > >> > > > > > > >> > > > > > > agree
> > > >> > > > >> > > > > > > >> > > > > > > > that we need to commit offset
> > with
> > > >> > leader
> > > >> > > > >> epoch
> > > >> > > > >> > in
> > > >> > > > >> > > > > order
> > > >> > > > >> > > > > > > to
> > > >> > > > >> > > > > > > >> > > address
> > > >> > > > >> > > > > > > >> > > > > > this
> > > >> > > > >> > > > > > > >> > > > > > > > example.
> > > >> > > > >> > > > > > > >> > > > > > > >
> > > >> > > > >> > > > > > > >> > > > > > > > I think the remaining question
> > is
> > > >> how
> > > >> > to
> > > >> > > > >> address
> > > >> > > > >> > > the
> > > >> > > > >> > > > > > > >> scenario
> > > >> > > > >> > > > > > > >> > > that
> > > >> > > > >> > > > > > > >> > > > > the
> > > >> > > > >> > > > > > > >> > > > > > > > topic is deleted and
> re-created.
> > > One
> > > >> > > > possible
> > > >> > > > >> > > > solution
> > > >> > > > >> > > > > > is
> > > >> > > > >> > > > > > > to
> > > >> > > > >> > > > > > > >> > > commit
> > > >> > > > >> > > > > > > >> > > > > > > offset
> > > >> > > > >> > > > > > > >> > > > > > > > with both the leader epoch and
> > the
> > > >> > > metadata
> > > >> > > > >> > > version.
> > > >> > > > >> > > > > The
> > > >> > > > >> > > > > > > >> logic
> > > >> > > > >> > > > > > > >> > > and
> > > >> > > > >> > > > > > > >> > > > > the
> > > >> > > > >> > > > > > > >> > > > > > > > implementation of this
> solution
> > > does
> > > >> > not
> > > >> > > > >> > require a
> > > >> > > > >> > > > new
> > > >> > > > >> > > > > > > >> concept
> > > >> > > > >> > > > > > > >> > > > (e.g.
> > > >> > > > >> > > > > > > >> > > > > > > > partition epoch) and it does
> not
> > > >> > require
> > > >> > > > any
> > > >> > > > >> > > change
> > > >> > > > >> > > > to
> > > >> > > > >> > > > > > the
> > > >> > > > >> > > > > > > >> > > message
> > > >> > > > >> > > > > > > >> > > > > > format
> > > >> > > > >> > > > > > > >> > > > > > > > or leader epoch. It also
> allows
> > us
> > > >> to
> > > >> > > order
> > > >> > > > >> the
> > > >> > > > >> > > > > metadata
> > > >> > > > >> > > > > > > in
> > > >> > > > >> > > > > > > >> a
> > > >> > > > >> > > > > > > >> > > > > > > > straightforward manner which
> may
> > > be
> > > >> > > useful
> > > >> > > > in
> > > >> > > > >> > the
> > > >> > > > >> > > > > > future.
> > > >> > > > >> > > > > > > >> So it
> > > >> > > > >> > > > > > > >> > > may
> > > >> > > > >> > > > > > > >> > > > > be
> > > >> > > > >> > > > > > > >> > > > > > a
> > > >> > > > >> > > > > > > >> > > > > > > > better solution than
> generating
> > a
> > > >> > random
> > > >> > > > >> > partition
> > > >> > > > >> > > > > epoch
> > > >> > > > >> > > > > > > >> every
> > > >> > > > >> > > > > > > >> > > time
> > > >> > > > >> > > > > > > >> > > > > we
> > > >> > > > >> > > > > > > >> > > > > > > > create a partition. Does this
> > > sound
> > > >> > > > >> reasonable?
> > > >> > > > >> > > > > > > >> > > > > > > >
> > > >> > > > >> > > > > > > >> > > > > > > > Previously one concern with
> > using
> > > >> the
> > > >> > > > >> metadata
> > > >> > > > >> > > > version
> > > >> > > > >> > > > > > is
> > > >> > > > >> > > > > > > >> that
> > > >> > > > >> > > > > > > >> > > > > consumer
> > > >> > > > >> > > > > > > >> > > > > > > > will be forced to refresh
> > metadata
> > > >> even
> > > >> > > if
> > > >> > > > >> > > metadata
> > > >> > > > >> > > > > > > version
> > > >> > > > >> > > > > > > >> is
> > > >> > > > >> > > > > > > >> > > > > > increased
> > > >> > > > >> > > > > > > >> > > > > > > > due to topics that the
> consumer
> > is
> > > >> not
> > > >> > > > >> > interested
> > > >> > > > >> > > > in.
> > > >> > > > >> > > > > > Now
> > > >> > > > >> > > > > > > I
> > > >> > > > >> > > > > > > >> > > > realized
> > > >> > > > >> > > > > > > >> > > > > > that
> > > >> > > > >> > > > > > > >> > > > > > > > this is probably not a
> problem.
> > > >> > Currently
> > > >> > > > >> client
> > > >> > > > >> > > > will
> > > >> > > > >> > > > > > > >> refresh
> > > >> > > > >> > > > > > > >> > > > > metadata
> > > >> > > > >> > > > > > > >> > > > > > > > either due to
> > > >> InvalidMetadataException
> > > >> > in
> > > >> > > > the
> > > >> > > > >> > > > response
> > > >> > > > >> > > > > > > from
> > > >> > > > >> > > > > > > >> > > broker
> > > >> > > > >> > > > > > > >> > > > or
> > > >> > > > >> > > > > > > >> > > > > > due
> > > >> > > > >> > > > > > > >> > > > > > > > to metadata expiry. The
> addition
> > > of
> > > >> the
> > > >> > > > >> metadata
> > > >> > > > >> > > > > version
> > > >> > > > >> > > > > > > >> should
> > > >> > > > >> > > > > > > >> > > > > > increase
> > > >> > > > >> > > > > > > >> > > > > > > > the overhead of metadata
> refresh
> > > >> caused
> > > >> > > by
> > > >> > > > >> > > > > > > >> > > > InvalidMetadataException.
> > > >> > > > >> > > > > > > >> > > > > If
> > > >> > > > >> > > > > > > >> > > > > > > > client refresh metadata due to
> > > >> expiry
> > > >> > and
> > > >> > > > it
> > > >> > > > >> > > > receives
> > > >> > > > >> > > > > a
> > > >> > > > >> > > > > > > >> > metadata
> > > >> > > > >> > > > > > > >> > > > > whose
> > > >> > > > >> > > > > > > >> > > > > > > > version is lower than the
> > current
> > > >> > > metadata
> > > >> > > > >> > > version,
> > > >> > > > >> > > > we
> > > >> > > > >> > > > > > can
> > > >> > > > >> > > > > > > >> > reject
> > > >> > > > >> > > > > > > >> > > > the
> > > >> > > > >> > > > > > > >> > > > > > > > metadata but still reset the
> > > >> metadata
> > > >> > > age,
> > > >> > > > >> which
> > > >> > > > >> > > > > > > essentially
> > > >> > > > >> > > > > > > >> > keep
> > > >> > > > >> > > > > > > >> > > > the
> > > >> > > > >> > > > > > > >> > > > > > > > existing behavior in the
> client.
> > > >> > > > >> > > > > > > >> > > > > > > >
> > > >> > > > >> > > > > > > >> > > > > > > > Thanks much,
> > > >> > > > >> > > > > > > >> > > > > > > > Dong
> > > >> > > > >> > > > > > > >> > > > > > > >
> > > >> > > > >> > > > > > > >> > > > > > >
> > > >> > > > >> > > > > > > >> > > > > >
> > > >> > > > >> > > > > > > >> > > > >
> > > >> > > > >> > > > > > > >> > > >
> > > >> > > > >> > > > > > > >> > >
> > > >> > > > >> > > > > > > >> >
> > > >> > > > >> > > > > > > >>
> > > >> > > > >> > > > > > > >
> > > >> > > > >> > > > > > > >
> > > >> > > > >> > > > > > >
> > > >> > > > >> > > > > >
> > > >> > > > >> > > > >
> > > >> > > > >> > > >
> > > >> > > > >> > >
> > > >> > > > >> >
> > > >> > > > >>
> > > >> > > > >
> > > >> > > > >
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Dong Lin <li...@gmail.com>.
Hey Jun,

I agree. I have updated the KIP to remove the class OffetEpoch and replace
OffsetEpoch with byte[] in APIs that use it. Can you see if it looks good?

Thanks!
Dong

On Thu, Jan 18, 2018 at 6:07 PM, Jun Rao <ju...@confluent.io> wrote:

> Hi, Dong,
>
> Thanks for the updated KIP. It looks good to me now. The only thing is
> for OffsetEpoch.
> If we expose the individual fields in the class, we probably don't need the
> encode/decode methods. If we want to hide the details of OffsetEpoch, we
> probably don't want expose the individual fields.
>
> Jun
>
> On Wed, Jan 17, 2018 at 10:10 AM, Dong Lin <li...@gmail.com> wrote:
>
> > Thinking about point 61 more, I realize that the async zookeeper read may
> > make it less of an issue for controller to read more zookeeper nodes.
> > Writing partition_epoch in the per-partition znode makes it simpler to
> > handle the broker failure between zookeeper writes for a topic creation.
> I
> > have updated the KIP to use the suggested approach.
> >
> >
> > On Wed, Jan 17, 2018 at 9:57 AM, Dong Lin <li...@gmail.com> wrote:
> >
> > > Hey Jun,
> > >
> > > Thanks much for the comments. Please see my comments inline.
> > >
> > > On Tue, Jan 16, 2018 at 4:38 PM, Jun Rao <ju...@confluent.io> wrote:
> > >
> > >> Hi, Dong,
> > >>
> > >> Thanks for the updated KIP. Looks good to me overall. Just a few minor
> > >> comments.
> > >>
> > >> 60. OffsetAndMetadata positionAndOffsetEpoch(TopicPartition
> partition):
> > >> It
> > >> seems that there is no need to return metadata. We probably want to
> > return
> > >> sth like OffsetAndEpoch.
> > >>
> > >
> > > Previously I think we may want to re-use the existing class to keep our
> > > consumer interface simpler. I have updated the KIP to add class
> > > OffsetAndOffsetEpoch. I didn't use OffsetAndEpoch because user may
> > confuse
> > > this name with OffsetEpoch. Does this sound OK?
> > >
> > >
> > >>
> > >> 61. Should we store partition_epoch in
> > >> /brokers/topics/[topic]/partitions/[partitionId] in ZK?
> > >>
> > >
> > > I have considered this. I think the advantage of adding the
> > > partition->partition_epoch map in the existing
> > > znode /brokers/topics/[topic]/partitions is that controller only needs
> > to
> > > read one znode per topic to gets its partition_epoch information.
> > Otherwise
> > > controller may need to read one extra znode per partition to get the
> same
> > > information.
> > >
> > > When we delete partition or expand partition of a topic, someone needs
> to
> > > modify partition->partition_epoch map in znode
> > > /brokers/topics/[topic]/partitions. This may seem a bit more
> complicated
> > > than simply adding or deleting znode /brokers/topics/[topic]/
> > partitions/[partitionId].
> > > But the complexity is probably similar to the existing operation of
> > > modifying the partition->replica_list mapping in znode
> > > /brokers/topics/[topic]. So not sure it is better to store the
> > > partition_epoch in /brokers/topics/[topic]/partitions/[partitionId].
> > What
> > > do you think?
> > >
> > >
> > >>
> > >> 62. For checking outdated metadata in the client, we probably want to
> > add
> > >> when max_partition_epoch will be used.
> > >>
> > >
> > > The max_partition_epoch is used in the Proposed Changes -> Client's
> > > metadata refresh section to determine whether a metadata is outdated.
> And
> > > this formula is referenced and re-used in other sections to determine
> > > whether a metadata is outdated. Does this formula look OK?
> > >
> > >
> > >>
> > >> 63. "The leader_epoch should be the largest leader_epoch of messages
> > whose
> > >> offset < the commit offset. If no message has been consumed since
> > consumer
> > >> initialization, the leader_epoch from seek(...) or OffsetFetchResponse
> > >> should be used. The partition_epoch should be read from the last
> > >> FetchResponse corresponding to the given partition and commit offset.
> ":
> > >> leader_epoch and partition_epoch are associated with an offset. So, if
> > no
> > >> message is consumed, there is no offset and therefore there is no need
> > to
> > >> read leader_epoch and partition_epoch. Also, the leader_epoch
> associated
> > >> with the offset should just come from the messages returned in the
> fetch
> > >> response.
> > >>
> > >
> > > I am thinking that, if user calls seek(..) and commitSync(...) without
> > > consuming any messages, we should re-use the leader_epoch and
> > > partition_epoch provided by the seek(...) in the OffsetCommitRequest.
> And
> > > if messages have been successfully consumed, then leader_epoch will
> come
> > > from the messages returned in the fetch response. The condition
> "messages
> > > whose offset < the commit offset" is needed to take care of the log
> > > compacted topic which may have offset gap due to log cleaning.
> > >
> > > Did I miss something here? Or should I rephrase the paragraph to make
> it
> > > less confusing?
> > >
> > >
> > >> 64. Could you include the public methods in the OffsetEpoch class?
> > >>
> > >
> > > I mistakenly deleted the definition of OffsetEpoch class from the KIP.
> I
> > > just added it back with the public methods. Could you take another
> look?
> > >
> > >
> > >>
> > >> Jun
> > >>
> > >>
> > >> On Thu, Jan 11, 2018 at 5:43 PM, Dong Lin <li...@gmail.com>
> wrote:
> > >>
> > >> > Hey Jun,
> > >> >
> > >> > Thanks much. I agree that we can not rely on committed offsets to be
> > >> always
> > >> > deleted when we delete topic. So it is necessary to use a
> > per-partition
> > >> > epoch that does not change unless this partition is deleted. I also
> > >> agree
> > >> > that it is very nice to be able to uniquely identify a message with
> > >> > (offset, leader_epoch, partition_epoch) in face of potential topic
> > >> deletion
> > >> > and unclean leader election.
> > >> >
> > >> > I agree with all your comments. And I have updated the KIP based on
> > our
> > >> > latest discussion. In addition, I added
> InvalidPartitionEpochException
> > >> > which will be thrown by consumer.poll() if the partition_epoch
> > >> associated
> > >> > with the partition, which can be given to consumer using seek(...),
> is
> > >> > different from the partition_epoch in the FetchResponse.
> > >> >
> > >> > Can you take another look at the latest KIP?
> > >> >
> > >> > Thanks!
> > >> > Dong
> > >> >
> > >> >
> > >> >
> > >> > On Wed, Jan 10, 2018 at 2:24 PM, Jun Rao <ju...@confluent.io> wrote:
> > >> >
> > >> > > Hi, Dong,
> > >> > >
> > >> > > My replies are the following.
> > >> > >
> > >> > > 60. What you described could also work. The drawback is that we
> will
> > >> be
> > >> > > unnecessarily changing the partition epoch when a partition hasn't
> > >> really
> > >> > > changed. I was imagining that the partition epoch will be stored
> in
> > >> > > /brokers/topics/[topic]/partitions/[partitionId], instead of at
> the
> > >> > topic
> > >> > > level. So, not sure if ZK size limit is an issue.
> > >> > >
> > >> > > 61, 62 and 65. To me, the offset + offset_epoch is a unique
> > identifier
> > >> > for
> > >> > > a message. So, if a message hasn't changed, the offset and the
> > >> associated
> > >> > > offset_epoch ideally should remain the same (it will be kind of
> > weird
> > >> if
> > >> > > two consumer apps save the offset on the same message, but the
> > >> > offset_epoch
> > >> > > are different). partition_epoch + leader_epoch give us that.
> > >> > global_epoch +
> > >> > > leader_epoch don't. If we use this approach, we can solve not only
> > the
> > >> > > problem that you have identified, but also other problems when
> there
> > >> is
> > >> > > data loss or topic re-creation more reliably. For example, in the
> > >> future,
> > >> > > if we include the partition_epoch and leader_epoch in the fetch
> > >> request,
> > >> > > the server can do a more reliable check of whether that offset is
> > >> valid
> > >> > or
> > >> > > not. I am not sure that we can rely upon all external offsets to
> be
> > >> > removed
> > >> > > on topic deletion. For example, a topic may be deleted by an admin
> > who
> > >> > may
> > >> > > not know all the applications.
> > >> > >
> > >> > > If we agree on the above, the second question is then how to
> > reliably
> > >> > > propagate the partition_epoch and the leader_epoch to the consumer
> > >> when
> > >> > > there are leader or partition changes. The leader_epoch comes from
> > the
> > >> > > message, which is reliable. So, I was suggesting that when we
> store
> > an
> > >> > > offset, we can just store the leader_epoch from the message set
> > >> > containing
> > >> > > that offset. Similarly, I was thinking that if the partition_epoch
> > is
> > >> in
> > >> > > the fetch response, we can propagate partition_epoch reliably
> where
> > is
> > >> > > partition_epoch change.
> > >> > >
> > >> > > 63. My point is that once a leader is producing a message in the
> new
> > >> > > partition_epoch, ideally, we should associate the new offsets with
> > the
> > >> > new
> > >> > > partition_epoch. Otherwise, the offset_epoch won't be the correct
> > >> unique
> > >> > > identifier (useful for solving other problems mentioned above). I
> > was
> > >> > > originally thinking that the leader will include the
> partition_epoch
> > >> in
> > >> > the
> > >> > > metadata cache in the fetch response. It's just that right now,
> > >> metadata
> > >> > > cache is updated on UpdateMetadataRequest, which typically happens
> > >> after
> > >> > > the LeaderAndIsrRequest. Another approach is for the leader to
> cache
> > >> the
> > >> > > partition_epoch in the Partition object and return that (instead
> of
> > >> the
> > >> > one
> > >> > > in metadata cache) in the fetch response.
> > >> > >
> > >> > > 65. It seems to me that the global_epoch and the partition_epoch
> > have
> > >> > > different purposes. A partition_epoch has the benefit that it (1)
> > can
> > >> be
> > >> > > used to form a unique identifier for a message and (2) can be used
> > to
> > >> > > solve other
> > >> > > corner case problems in the future. I am not sure having just a
> > >> > > global_epoch can achieve these. global_epoch is useful to
> determine
> > >> which
> > >> > > version of the metadata is newer, especially with topic deletion.
> > >> > >
> > >> > > Thanks,
> > >> > >
> > >> > > Jun
> > >> > >
> > >> > > On Tue, Jan 9, 2018 at 11:34 PM, Dong Lin <li...@gmail.com>
> > >> wrote:
> > >> > >
> > >> > > > Regarding the use of the global epoch in 65), it is very similar
> > to
> > >> the
> > >> > > > proposal of the metadata_epoch we discussed earlier. The main
> > >> > difference
> > >> > > is
> > >> > > > that this epoch is incremented when we create/expand/delete
> topic
> > >> and
> > >> > > does
> > >> > > > not change when controller re-send metadata.
> > >> > > >
> > >> > > > I looked at our previous discussion. It seems that we prefer
> > >> > > > partition_epoch over the metadata_epoch because 1) we prefer not
> > to
> > >> > have
> > >> > > an
> > >> > > > ever growing metadata_epoch and 2) we can reset offset better
> when
> > >> > topic
> > >> > > is
> > >> > > > re-created. The use of global topic_epoch avoids the drawback of
> > an
> > >> > ever
> > >> > > > quickly ever growing metadata_epoch. Though the global epoch
> does
> > >> not
> > >> > > allow
> > >> > > > us to recognize the invalid offset committed before the topic
> > >> > > re-creation,
> > >> > > > we can probably just delete the offset when we delete a topic.
> > Thus
> > >> I
> > >> > am
> > >> > > > not very sure whether it is still worthwhile to have a
> > per-partition
> > >> > > > partition_epoch if the metadata already has the global epoch.
> > >> > > >
> > >> > > >
> > >> > > > On Tue, Jan 9, 2018 at 6:58 PM, Dong Lin <li...@gmail.com>
> > >> wrote:
> > >> > > >
> > >> > > > > Hey Jun,
> > >> > > > >
> > >> > > > > Thanks so much. These comments very useful. Please see below
> my
> > >> > > comments.
> > >> > > > >
> > >> > > > > On Mon, Jan 8, 2018 at 5:52 PM, Jun Rao <ju...@confluent.io>
> > wrote:
> > >> > > > >
> > >> > > > >> Hi, Dong,
> > >> > > > >>
> > >> > > > >> Thanks for the updated KIP. A few more comments.
> > >> > > > >>
> > >> > > > >> 60. Perhaps having a partition epoch is more flexible since
> in
> > >> the
> > >> > > > future,
> > >> > > > >> we may support deleting a partition as well.
> > >> > > > >>
> > >> > > > >
> > >> > > > > Yeah I have considered this. I think we can probably still
> > support
> > >> > > > > deleting a partition by using the topic_epoch -- when
> partition
> > >> of a
> > >> > > > topic
> > >> > > > > is deleted or created, epoch of all partitions of this topic
> > will
> > >> be
> > >> > > > > incremented by 1. Therefore, if that partition is re-created
> > >> later,
> > >> > the
> > >> > > > > epoch of that partition will still be larger than its epoch
> > before
> > >> > the
> > >> > > > > deletion, which still allows the client to order the metadata
> > for
> > >> the
> > >> > > > > purpose of this KIP. Does this sound reasonable?
> > >> > > > >
> > >> > > > > The advantage of using topic_epoch instead of partition_epoch
> is
> > >> that
> > >> > > the
> > >> > > > > size of the /brokers/topics/[topic] znode and request/response
> > >> size
> > >> > can
> > >> > > > be
> > >> > > > > smaller. We have a limit on the maximum size of znode
> (typically
> > >> > 1MB).
> > >> > > > Use
> > >> > > > > partition epoch can effectively reduce the number of
> partitions
> > >> that
> > >> > > can
> > >> > > > be
> > >> > > > > described by the /brokers/topics/[topic] znode.
> > >> > > > >
> > >> > > > > One use-case of partition_epoch for client to detect that the
> > >> > committed
> > >> > > > > offset, either from kafka offset topic or from the external
> > store
> > >> is
> > >> > > > > invalid after partition deletion and re-creation. However, it
> > >> seems
> > >> > > that
> > >> > > > we
> > >> > > > > can also address this use-case with other approaches. For
> > example,
> > >> > when
> > >> > > > > AdminClient deletes partitions, it can also delete the
> committed
> > >> > > offsets
> > >> > > > > for those partitions from the offset topic. If user stores
> > offset
> > >> > > > > externally, it might make sense for user to similarly remove
> > >> offsets
> > >> > of
> > >> > > > > related partitions after these partitions are deleted. So I am
> > not
> > >> > sure
> > >> > > > > that we should use partition_epoch in this KIP.
> > >> > > > >
> > >> > > > >
> > >> > > > >>
> > >> > > > >> 61. It seems that the leader epoch returned in the position()
> > >> call
> > >> > > > should
> > >> > > > >> the the leader epoch returned in the fetch response, not the
> > one
> > >> in
> > >> > > the
> > >> > > > >> metadata cache of the client.
> > >> > > > >
> > >> > > > >
> > >> > > > > I think this is a good idea. Just to double check, this change
> > >> does
> > >> > not
> > >> > > > > affect the correctness or performance of this KIP. But it can
> be
> > >> > useful
> > >> > > > if
> > >> > > > > we want to use the leader_epoch to better handle the offset
> rest
> > >> in
> > >> > > case
> > >> > > > of
> > >> > > > > unclean leader election, which is listed in the future work.
> Is
> > >> this
> > >> > > > > understanding correct?
> > >> > > > >
> > >> > > > > I have updated the KIP to specify that the leader_epoch
> returned
> > >> by
> > >> > > > > position() should be the largest leader_epoch of those already
> > >> > consumed
> > >> > > > > messages whose offset < position. If no message has been
> > consumed
> > >> > since
> > >> > > > > consumer initialization, the leader_epoch from seek() or
> > >> > > > > OffsetFetchResponse should be used. The offset included in the
> > >> > > > > OffsetCommitRequest will also be determined in the similar
> > manner.
> > >> > > > >
> > >> > > > >
> > >> > > > >>
> > >> > > > >> 62. I am wondering if we should return the partition epoch in
> > the
> > >> > > fetch
> > >> > > > >> response as well. In the current proposal, if a topic is
> > >> recreated
> > >> > and
> > >> > > > the
> > >> > > > >> new leader is on the same broker as the old one, there is
> > >> nothing to
> > >> > > > force
> > >> > > > >> the metadata refresh in the client. So, the client may still
> > >> > associate
> > >> > > > the
> > >> > > > >> offset with the old partition epoch.
> > >> > > > >>
> > >> > > > >
> > >> > > > > Could you help me understand the problem if a client
> associates
> > >> old
> > >> > > > > partition_epoch (or the topic_epoch as of the current KIP)
> with
> > >> the
> > >> > > > offset?
> > >> > > > > The main purpose of the topic_epoch is to be able to drop
> > >> > leader_epoch
> > >> > > > to 0
> > >> > > > > after a partition is deleted and re-created. I guess you may
> be
> > >> > > thinking
> > >> > > > > about using the partition_epoch to detect that the committed
> > >> offset
> > >> > is
> > >> > > > > invalid? In that case, I am wondering if the alternative
> > approach
> > >> > > > described
> > >> > > > > in 60) would be reasonable.
> > >> > > > >
> > >> > > > >
> > >> > > > >>
> > >> > > > >> 63. There is some subtle coordination between the
> > >> > LeaderAndIsrRequest
> > >> > > > and
> > >> > > > >> UpdateMetadataRequest. Currently, when a leader changes, the
> > >> > > controller
> > >> > > > >> first sends the LeaderAndIsrRequest to the assigned replicas
> > and
> > >> the
> > >> > > > >> UpdateMetadataRequest to every broker. So, there could be a
> > small
> > >> > > window
> > >> > > > >> when the leader already receives the new partition epoch in
> the
> > >> > > > >> LeaderAndIsrRequest, but the metadata cache in the broker
> > hasn't
> > >> > been
> > >> > > > >> updated with the latest partition epoch. Not sure what's the
> > best
> > >> > way
> > >> > > to
> > >> > > > >> address this issue. Perhaps we can update the metadata cache
> on
> > >> the
> > >> > > > broker
> > >> > > > >> with both LeaderAndIsrRequest and UpdateMetadataRequest. The
> > >> > challenge
> > >> > > > is
> > >> > > > >> that the two have slightly different data. For example, only
> > the
> > >> > > latter
> > >> > > > >> has
> > >> > > > >> all endpoints.
> > >> > > > >>
> > >> > > > >
> > >> > > > > I am not sure whether this is a problem. Could you explain a
> bit
> > >> more
> > >> > > > what
> > >> > > > > specific problem this small window can cause?
> > >> > > > >
> > >> > > > > Since client can fetch metadata from any broker in the
> cluster,
> > >> and
> > >> > > given
> > >> > > > > that different brokers receive request (e.g.
> LeaderAndIsrRequest
> > >> and
> > >> > > > > UpdateMetadataRequest) in arbitrary order, the metadata
> received
> > >> by
> > >> > > > client
> > >> > > > > can be in arbitrary order (either newer or older) compared to
> > the
> > >> > > > broker's
> > >> > > > > leadership state even if a given broker receives
> > >> LeaderAndIsrRequest
> > >> > > and
> > >> > > > > UpdateMetadataRequest simultaneously. So I am not sure it is
> > >> useful
> > >> > to
> > >> > > > > update broker's cache with LeaderAndIsrRequest.
> > >> > > > >
> > >> > > > >
> > >> > > > >> 64. The enforcement of leader epoch in Offset commit: We
> allow
> > a
> > >> > > > consumer
> > >> > > > >> to set an arbitrary offset. So it's possible for offsets or
> > >> leader
> > >> > > epoch
> > >> > > > >> to
> > >> > > > >> go backwards. I am not sure if we could always enforce that
> the
> > >> > leader
> > >> > > > >> epoch only goes up on the broker.
> > >> > > > >>
> > >> > > > >
> > >> > > > > Sure. I have removed this check from the KIP.
> > >> > > > >
> > >> > > > > BTW, we can probably still ensure that the leader_epoch always
> > >> > increase
> > >> > > > if
> > >> > > > > the leader_epoch used with offset commit is the
> max(leader_epoch
> > >> of
> > >> > the
> > >> > > > > message with offset = the committed offset - 1, the largest
> > known
> > >> > > > > leader_epoch from the metadata). But I don't have a good
> > use-case
> > >> for
> > >> > > > this
> > >> > > > > alternative definition. So I choose the keep the KIP simple by
> > >> > > requiring
> > >> > > > > leader_epoch to always increase.
> > >> > > > >
> > >> > > > >
> > >> > > > >> 65. Good point on handling missing partition epoch due to
> topic
> > >> > > > deletion.
> > >> > > > >> Another potential way to address this is to additionally
> > >> propagate
> > >> > the
> > >> > > > >> global partition epoch to brokers and the clients. This way,
> > >> when a
> > >> > > > >> partition epoch is missing, we can use the global partition
> > >> epoch to
> > >> > > > >> reason
> > >> > > > >> about which metadata is more recent.
> > >> > > > >>
> > >> > > > >
> > >> > > > > This is a great idea. The global epoch can be used to order
> the
> > >> > > metadata
> > >> > > > > and help us recognize the more recent metadata if a topic (or
> > >> > > partition)
> > >> > > > is
> > >> > > > > deleted and re-created.
> > >> > > > >
> > >> > > > > Actually, it seems we only need to propagate the global epoch
> to
> > >> > > brokers
> > >> > > > > and clients without propagating this epoch on a per-topic or
> > >> > > > per-partition
> > >> > > > > basic. Doing so would simply interface changes made this KIP.
> > Does
> > >> > this
> > >> > > > > approach sound reasonable?
> > >> > > > >
> > >> > > > >
> > >> > > > >> 66. A client may also get an offset by time using the
> > >> > offsetForTimes()
> > >> > > > >> api.
> > >> > > > >> So, we probably want to include offsetInternalMetadata in
> > >> > > > >> OffsetAndTimestamp
> > >> > > > >> as well.
> > >> > > > >>
> > >> > > > >
> > >> > > > > You are right. This probably also requires us to change the
> > >> > > > > ListOffsetRequest as well. I will update the KIP after we
> agree
> > on
> > >> > the
> > >> > > > > solution for 65).
> > >> > > > >
> > >> > > > >
> > >> > > > >>
> > >> > > > >> 67. InteralMetadata can be a bit confusing with the metadata
> > >> field
> > >> > > > already
> > >> > > > >> there. Perhaps we can just call it OffsetEpoch. It might be
> > >> useful
> > >> > to
> > >> > > > make
> > >> > > > >> OffsetEpoch printable at least for debugging purpose. Once
> you
> > do
> > >> > > that,
> > >> > > > we
> > >> > > > >> are already exposing the internal fields. So, not sure if
> it's
> > >> worth
> > >> > > > >> hiding
> > >> > > > >> them. If we do want to hide them, perhaps we can have sth
> like
> > >> the
> > >> > > > >> following. The binary encoding is probably more efficient
> than
> > >> JSON
> > >> > > for
> > >> > > > >> external storage.
> > >> > > > >>
> > >> > > > >> OffsetEpoch {
> > >> > > > >>  static OffsetEpoch decode(byte[]);
> > >> > > > >>
> > >> > > > >>   public byte[] encode();
> > >> > > > >>
> > >> > > > >>   public String toString();
> > >> > > > >> }
> > >> > > > >>
> > >> > > > >
> > >> > > > > Thanks much. I like this solution. I have updated the KIP
> > >> > accordingly.
> > >> > > > >
> > >> > > > >
> > >> > > > >
> > >> > > > >>
> > >> > > > >> Jun
> > >> > > > >>
> > >> > > > >> On Mon, Jan 8, 2018 at 4:22 PM, Dong Lin <
> lindong28@gmail.com>
> > >> > wrote:
> > >> > > > >>
> > >> > > > >> > Hey Jason,
> > >> > > > >> >
> > >> > > > >> > Certainly. This sounds good. I have updated the KIP to
> > clarity
> > >> > that
> > >> > > > the
> > >> > > > >> > global epoch will be incremented by 1 each time a topic is
> > >> > deleted.
> > >> > > > >> >
> > >> > > > >> > Thanks,
> > >> > > > >> > Dong
> > >> > > > >> >
> > >> > > > >> > On Mon, Jan 8, 2018 at 4:09 PM, Jason Gustafson <
> > >> > jason@confluent.io
> > >> > > >
> > >> > > > >> > wrote:
> > >> > > > >> >
> > >> > > > >> > > Hi Dong,
> > >> > > > >> > >
> > >> > > > >> > >
> > >> > > > >> > > I think your approach will allow user to distinguish
> > between
> > >> the
> > >> > > > >> metadata
> > >> > > > >> > > > before and after the topic deletion. I also agree that
> > this
> > >> > can
> > >> > > be
> > >> > > > >> > > > potentially be useful to user. I am just not very sure
> > >> whether
> > >> > > we
> > >> > > > >> > already
> > >> > > > >> > > > have a good use-case to make the additional complexity
> > >> > > worthwhile.
> > >> > > > >> It
> > >> > > > >> > > seems
> > >> > > > >> > > > that this feature is kind of independent of the main
> > >> problem
> > >> > of
> > >> > > > this
> > >> > > > >> > KIP.
> > >> > > > >> > > > Could we add this as a future work?
> > >> > > > >> > >
> > >> > > > >> > >
> > >> > > > >> > > Do you think it's fair if we bump the topic epoch on
> > deletion
> > >> > and
> > >> > > > >> leave
> > >> > > > >> > > propagation of the epoch for deleted topics for future
> > work?
> > >> I
> > >> > > don't
> > >> > > > >> > think
> > >> > > > >> > > this adds much complexity and it makes the behavior
> > >> consistent:
> > >> > > > every
> > >> > > > >> > topic
> > >> > > > >> > > mutation results in an epoch bump.
> > >> > > > >> > >
> > >> > > > >> > > Thanks,
> > >> > > > >> > > Jason
> > >> > > > >> > >
> > >> > > > >> > > On Mon, Jan 8, 2018 at 3:14 PM, Dong Lin <
> > >> lindong28@gmail.com>
> > >> > > > wrote:
> > >> > > > >> > >
> > >> > > > >> > > > Hey Ismael,
> > >> > > > >> > > >
> > >> > > > >> > > > I guess we actually need user to see this field so that
> > >> user
> > >> > can
> > >> > > > >> store
> > >> > > > >> > > this
> > >> > > > >> > > > value in the external store together with the offset.
> We
> > >> just
> > >> > > > prefer
> > >> > > > >> > the
> > >> > > > >> > > > value to be opaque to discourage most users from
> > >> interpreting
> > >> > > this
> > >> > > > >> > value.
> > >> > > > >> > > > One more advantage of using such an opaque field is to
> be
> > >> able
> > >> > > to
> > >> > > > >> > evolve
> > >> > > > >> > > > the information (or schema) of this value without
> > changing
> > >> > > > consumer
> > >> > > > >> API
> > >> > > > >> > > in
> > >> > > > >> > > > the future.
> > >> > > > >> > > >
> > >> > > > >> > > > I also thinking it is probably OK for user to be able
> to
> > >> > > interpret
> > >> > > > >> this
> > >> > > > >> > > > value, particularly for those advanced users.
> > >> > > > >> > > >
> > >> > > > >> > > > Thanks,
> > >> > > > >> > > > Dong
> > >> > > > >> > > >
> > >> > > > >> > > > On Mon, Jan 8, 2018 at 2:34 PM, Ismael Juma <
> > >> > ismael@juma.me.uk>
> > >> > > > >> wrote:
> > >> > > > >> > > >
> > >> > > > >> > > > > On Fri, Jan 5, 2018 at 7:15 PM, Jason Gustafson <
> > >> > > > >> jason@confluent.io>
> > >> > > > >> > > > > wrote:
> > >> > > > >> > > > > >
> > >> > > > >> > > > > > class OffsetAndMetadata {
> > >> > > > >> > > > > >   long offset;
> > >> > > > >> > > > > >   byte[] offsetMetadata;
> > >> > > > >> > > > > >   String metadata;
> > >> > > > >> > > > > > }
> > >> > > > >> > > > >
> > >> > > > >> > > > >
> > >> > > > >> > > > > > Admittedly, the naming is a bit annoying, but we
> can
> > >> > > probably
> > >> > > > >> come
> > >> > > > >> > up
> > >> > > > >> > > > > with
> > >> > > > >> > > > > > something better. Internally the byte array would
> > have
> > >> a
> > >> > > > >> version.
> > >> > > > >> > If
> > >> > > > >> > > in
> > >> > > > >> > > > > the
> > >> > > > >> > > > > > future we have anything else we need to add, we can
> > >> update
> > >> > > the
> > >> > > > >> > > version
> > >> > > > >> > > > > and
> > >> > > > >> > > > > > we wouldn't need any new APIs.
> > >> > > > >> > > > > >
> > >> > > > >> > > > >
> > >> > > > >> > > > > We can also add fields to a class in a compatible
> way.
> > >> So,
> > >> > it
> > >> > > > >> seems
> > >> > > > >> > to
> > >> > > > >> > > me
> > >> > > > >> > > > > that the main advantage of the byte array is that
> it's
> > >> > opaque
> > >> > > to
> > >> > > > >> the
> > >> > > > >> > > > user.
> > >> > > > >> > > > > Is that correct? If so, we could also add any opaque
> > >> > metadata
> > >> > > > in a
> > >> > > > >> > > > subclass
> > >> > > > >> > > > > so that users don't even see it (unless they cast it,
> > but
> > >> > then
> > >> > > > >> > they're
> > >> > > > >> > > on
> > >> > > > >> > > > > their own).
> > >> > > > >> > > > >
> > >> > > > >> > > > > Ismael
> > >> > > > >> > > > >
> > >> > > > >> > > > > The corresponding seek() and position() APIs might
> look
> > >> > > > something
> > >> > > > >> > like
> > >> > > > >> > > > > this:
> > >> > > > >> > > > > >
> > >> > > > >> > > > > > void seek(TopicPartition partition, long offset,
> > byte[]
> > >> > > > >> > > > offsetMetadata);
> > >> > > > >> > > > > > byte[] positionMetadata(TopicPartition partition);
> > >> > > > >> > > > > >
> > >> > > > >> > > > > > What do you think?
> > >> > > > >> > > > > >
> > >> > > > >> > > > > > Thanks,
> > >> > > > >> > > > > > Jason
> > >> > > > >> > > > > >
> > >> > > > >> > > > > > On Thu, Jan 4, 2018 at 7:04 PM, Dong Lin <
> > >> > > lindong28@gmail.com
> > >> > > > >
> > >> > > > >> > > wrote:
> > >> > > > >> > > > > >
> > >> > > > >> > > > > > > Hey Jun, Jason,
> > >> > > > >> > > > > > >
> > >> > > > >> > > > > > > Thanks much for all the feedback. I have updated
> > the
> > >> KIP
> > >> > > > >> based on
> > >> > > > >> > > the
> > >> > > > >> > > > > > > latest discussion. Can you help check whether it
> > >> looks
> > >> > > good?
> > >> > > > >> > > > > > >
> > >> > > > >> > > > > > > Thanks,
> > >> > > > >> > > > > > > Dong
> > >> > > > >> > > > > > >
> > >> > > > >> > > > > > > On Thu, Jan 4, 2018 at 5:36 PM, Dong Lin <
> > >> > > > lindong28@gmail.com
> > >> > > > >> >
> > >> > > > >> > > > wrote:
> > >> > > > >> > > > > > >
> > >> > > > >> > > > > > > > Hey Jun,
> > >> > > > >> > > > > > > >
> > >> > > > >> > > > > > > > Hmm... thinking about this more, I am not sure
> > that
> > >> > the
> > >> > > > >> > proposed
> > >> > > > >> > > > API
> > >> > > > >> > > > > is
> > >> > > > >> > > > > > > > sufficient. For users that store offset
> > >> externally, we
> > >> > > > >> probably
> > >> > > > >> > > > need
> > >> > > > >> > > > > > > extra
> > >> > > > >> > > > > > > > API to return the leader_epoch and
> > partition_epoch
> > >> for
> > >> > > all
> > >> > > > >> > > > partitions
> > >> > > > >> > > > > > > that
> > >> > > > >> > > > > > > > consumers are consuming. I suppose these users
> > >> > currently
> > >> > > > use
> > >> > > > >> > > > > position()
> > >> > > > >> > > > > > > to
> > >> > > > >> > > > > > > > get the offset. Thus we probably need a new
> > method
> > >> > > > >> > > > > > positionWithEpoch(..)
> > >> > > > >> > > > > > > to
> > >> > > > >> > > > > > > > return <offset, partition_epoch, leader_epoch>.
> > >> Does
> > >> > > this
> > >> > > > >> sound
> > >> > > > >> > > > > > > reasonable?
> > >> > > > >> > > > > > > >
> > >> > > > >> > > > > > > > Thanks,
> > >> > > > >> > > > > > > > Dong
> > >> > > > >> > > > > > > >
> > >> > > > >> > > > > > > >
> > >> > > > >> > > > > > > > On Thu, Jan 4, 2018 at 5:26 PM, Jun Rao <
> > >> > > jun@confluent.io
> > >> > > > >
> > >> > > > >> > > wrote:
> > >> > > > >> > > > > > > >
> > >> > > > >> > > > > > > >> Hi, Dong,
> > >> > > > >> > > > > > > >>
> > >> > > > >> > > > > > > >> Yes, that's what I am thinking. OffsetEpoch
> will
> > >> be
> > >> > > > >> composed
> > >> > > > >> > of
> > >> > > > >> > > > > > > >> (partition_epoch,
> > >> > > > >> > > > > > > >> leader_epoch).
> > >> > > > >> > > > > > > >>
> > >> > > > >> > > > > > > >> Thanks,
> > >> > > > >> > > > > > > >>
> > >> > > > >> > > > > > > >> Jun
> > >> > > > >> > > > > > > >>
> > >> > > > >> > > > > > > >>
> > >> > > > >> > > > > > > >> On Thu, Jan 4, 2018 at 4:22 PM, Dong Lin <
> > >> > > > >> lindong28@gmail.com
> > >> > > > >> > >
> > >> > > > >> > > > > wrote:
> > >> > > > >> > > > > > > >>
> > >> > > > >> > > > > > > >> > Hey Jun,
> > >> > > > >> > > > > > > >> >
> > >> > > > >> > > > > > > >> > Thanks much. I like the the new API that you
> > >> > > proposed.
> > >> > > > I
> > >> > > > >> am
> > >> > > > >> > > not
> > >> > > > >> > > > > sure
> > >> > > > >> > > > > > > >> what
> > >> > > > >> > > > > > > >> > you exactly mean by offset_epoch. I suppose
> > >> that we
> > >> > > can
> > >> > > > >> use
> > >> > > > >> > > the
> > >> > > > >> > > > > pair
> > >> > > > >> > > > > > > of
> > >> > > > >> > > > > > > >> > (partition_epoch, leader_epoch) as the
> > >> > offset_epoch,
> > >> > > > >> right?
> > >> > > > >> > > > > > > >> >
> > >> > > > >> > > > > > > >> > Thanks,
> > >> > > > >> > > > > > > >> > Dong
> > >> > > > >> > > > > > > >> >
> > >> > > > >> > > > > > > >> > On Thu, Jan 4, 2018 at 4:02 PM, Jun Rao <
> > >> > > > >> jun@confluent.io>
> > >> > > > >> > > > wrote:
> > >> > > > >> > > > > > > >> >
> > >> > > > >> > > > > > > >> > > Hi, Dong,
> > >> > > > >> > > > > > > >> > >
> > >> > > > >> > > > > > > >> > > Got it. The api that you proposed works.
> The
> > >> > > question
> > >> > > > >> is
> > >> > > > >> > > > whether
> > >> > > > >> > > > > > > >> that's
> > >> > > > >> > > > > > > >> > the
> > >> > > > >> > > > > > > >> > > api that we want to have in the long term.
> > My
> > >> > > concern
> > >> > > > >> is
> > >> > > > >> > > that
> > >> > > > >> > > > > > while
> > >> > > > >> > > > > > > >> the
> > >> > > > >> > > > > > > >> > api
> > >> > > > >> > > > > > > >> > > change is simple, the new api seems harder
> > to
> > >> > > explain
> > >> > > > >> and
> > >> > > > >> > > use.
> > >> > > > >> > > > > For
> > >> > > > >> > > > > > > >> > example,
> > >> > > > >> > > > > > > >> > > a consumer storing offsets externally now
> > >> needs
> > >> > to
> > >> > > > call
> > >> > > > >> > > > > > > >> > > waitForMetadataUpdate() after calling
> > seek().
> > >> > > > >> > > > > > > >> > >
> > >> > > > >> > > > > > > >> > > An alternative approach is to make the
> > >> following
> > >> > > > >> > compatible
> > >> > > > >> > > > api
> > >> > > > >> > > > > > > >> changes
> > >> > > > >> > > > > > > >> > in
> > >> > > > >> > > > > > > >> > > Consumer.
> > >> > > > >> > > > > > > >> > > * Add an additional OffsetEpoch field in
> > >> > > > >> > OffsetAndMetadata.
> > >> > > > >> > > > (no
> > >> > > > >> > > > > > need
> > >> > > > >> > > > > > > >> to
> > >> > > > >> > > > > > > >> > > change the CommitSync() api)
> > >> > > > >> > > > > > > >> > > * Add a new api seek(TopicPartition
> > partition,
> > >> > long
> > >> > > > >> > offset,
> > >> > > > >> > > > > > > >> OffsetEpoch
> > >> > > > >> > > > > > > >> > > offsetEpoch). We can potentially deprecate
> > the
> > >> > old
> > >> > > > api
> > >> > > > >> > > > > > > >> > seek(TopicPartition
> > >> > > > >> > > > > > > >> > > partition, long offset) in the future.
> > >> > > > >> > > > > > > >> > >
> > >> > > > >> > > > > > > >> > > The alternative approach has similar
> amount
> > of
> > >> > api
> > >> > > > >> changes
> > >> > > > >> > > as
> > >> > > > >> > > > > > yours
> > >> > > > >> > > > > > > >> but
> > >> > > > >> > > > > > > >> > has
> > >> > > > >> > > > > > > >> > > the following benefits.
> > >> > > > >> > > > > > > >> > > 1. The api works in a similar way as how
> > >> offset
> > >> > > > >> management
> > >> > > > >> > > > works
> > >> > > > >> > > > > > now
> > >> > > > >> > > > > > > >> and
> > >> > > > >> > > > > > > >> > is
> > >> > > > >> > > > > > > >> > > probably what we want in the long term.
> > >> > > > >> > > > > > > >> > > 2. It can reset offsets better when there
> is
> > >> data
> > >> > > > loss
> > >> > > > >> due
> > >> > > > >> > > to
> > >> > > > >> > > > > > > unclean
> > >> > > > >> > > > > > > >> > > leader election or correlated replica
> > failure.
> > >> > > > >> > > > > > > >> > > 3. It can reset offsets better when topic
> is
> > >> > > > recreated.
> > >> > > > >> > > > > > > >> > >
> > >> > > > >> > > > > > > >> > > Thanks,
> > >> > > > >> > > > > > > >> > >
> > >> > > > >> > > > > > > >> > > Jun
> > >> > > > >> > > > > > > >> > >
> > >> > > > >> > > > > > > >> > >
> > >> > > > >> > > > > > > >> > > On Thu, Jan 4, 2018 at 2:05 PM, Dong Lin <
> > >> > > > >> > > lindong28@gmail.com
> > >> > > > >> > > > >
> > >> > > > >> > > > > > > wrote:
> > >> > > > >> > > > > > > >> > >
> > >> > > > >> > > > > > > >> > > > Hey Jun,
> > >> > > > >> > > > > > > >> > > >
> > >> > > > >> > > > > > > >> > > > Yeah I agree that ideally we don't want
> an
> > >> ever
> > >> > > > >> growing
> > >> > > > >> > > > global
> > >> > > > >> > > > > > > >> metadata
> > >> > > > >> > > > > > > >> > > > version. I just think it may be more
> > >> desirable
> > >> > to
> > >> > > > >> keep
> > >> > > > >> > the
> > >> > > > >> > > > > > > consumer
> > >> > > > >> > > > > > > >> API
> > >> > > > >> > > > > > > >> > > > simple.
> > >> > > > >> > > > > > > >> > > >
> > >> > > > >> > > > > > > >> > > > In my current proposal, metadata version
> > >> > returned
> > >> > > > in
> > >> > > > >> the
> > >> > > > >> > > > fetch
> > >> > > > >> > > > > > > >> response
> > >> > > > >> > > > > > > >> > > > will be stored with the offset together.
> > >> More
> > >> > > > >> > > specifically,
> > >> > > > >> > > > > the
> > >> > > > >> > > > > > > >> > > > metadata_epoch in the new offset topic
> > >> schema
> > >> > > will
> > >> > > > be
> > >> > > > >> > the
> > >> > > > >> > > > > > largest
> > >> > > > >> > > > > > > >> > > > metadata_epoch from all the
> > MetadataResponse
> > >> > and
> > >> > > > >> > > > FetchResponse
> > >> > > > >> > > > > > > ever
> > >> > > > >> > > > > > > >> > > > received by this consumer.
> > >> > > > >> > > > > > > >> > > >
> > >> > > > >> > > > > > > >> > > > We probably don't have to change the
> > >> consumer
> > >> > API
> > >> > > > for
> > >> > > > >> > > > > > > >> > > > commitSync(Map<TopicPartition,
> > >> > > OffsetAndMetadata>).
> > >> > > > >> If
> > >> > > > >> > > user
> > >> > > > >> > > > > > calls
> > >> > > > >> > > > > > > >> > > > commitSync(...) to commit offset 10 for
> a
> > >> given
> > >> > > > >> > partition,
> > >> > > > >> > > > for
> > >> > > > >> > > > > > > most
> > >> > > > >> > > > > > > >> > > > use-cases, this consumer instance should
> > >> have
> > >> > > > >> consumed
> > >> > > > >> > > > message
> > >> > > > >> > > > > > > with
> > >> > > > >> > > > > > > >> > > offset
> > >> > > > >> > > > > > > >> > > > 9 from this partition, in which case the
> > >> > consumer
> > >> > > > can
> > >> > > > >> > > > remember
> > >> > > > >> > > > > > and
> > >> > > > >> > > > > > > >> use
> > >> > > > >> > > > > > > >> > > the
> > >> > > > >> > > > > > > >> > > > metadata_epoch from the corresponding
> > >> > > FetchResponse
> > >> > > > >> when
> > >> > > > >> > > > > > > committing
> > >> > > > >> > > > > > > >> > > offset.
> > >> > > > >> > > > > > > >> > > > If user calls commitSync(..) to commit
> > >> offset
> > >> > 10
> > >> > > > for
> > >> > > > >> a
> > >> > > > >> > > given
> > >> > > > >> > > > > > > >> partition
> > >> > > > >> > > > > > > >> > > > without having consumed the message with
> > >> > offset 9
> > >> > > > >> using
> > >> > > > >> > > this
> > >> > > > >> > > > > > > >> consumer
> > >> > > > >> > > > > > > >> > > > instance, this is probably an advanced
> > >> > use-case.
> > >> > > In
> > >> > > > >> this
> > >> > > > >> > > > case
> > >> > > > >> > > > > > the
> > >> > > > >> > > > > > > >> > > advanced
> > >> > > > >> > > > > > > >> > > > user can retrieve the metadata_epoch
> using
> > >> the
> > >> > > > newly
> > >> > > > >> > added
> > >> > > > >> > > > > > > >> > > metadataEpoch()
> > >> > > > >> > > > > > > >> > > > API after it fetches the message with
> > >> offset 9
> > >> > > > >> (probably
> > >> > > > >> > > > from
> > >> > > > >> > > > > > > >> another
> > >> > > > >> > > > > > > >> > > > consumer instance) and encode this
> > >> > metadata_epoch
> > >> > > > in
> > >> > > > >> the
> > >> > > > >> > > > > > > >> > > > string OffsetAndMetadata.metadata. Do
> you
> > >> think
> > >> > > > this
> > >> > > > >> > > > solution
> > >> > > > >> > > > > > > would
> > >> > > > >> > > > > > > >> > work?
> > >> > > > >> > > > > > > >> > > >
> > >> > > > >> > > > > > > >> > > > By "not sure that I fully understand
> your
> > >> > latest
> > >> > > > >> > > > suggestion",
> > >> > > > >> > > > > > are
> > >> > > > >> > > > > > > >> you
> > >> > > > >> > > > > > > >> > > > referring to solution related to unclean
> > >> leader
> > >> > > > >> election
> > >> > > > >> > > > using
> > >> > > > >> > > > > > > >> > > leader_epoch
> > >> > > > >> > > > > > > >> > > > in my previous email?
> > >> > > > >> > > > > > > >> > > >
> > >> > > > >> > > > > > > >> > > > Thanks,
> > >> > > > >> > > > > > > >> > > > Dong
> > >> > > > >> > > > > > > >> > > >
> > >> > > > >> > > > > > > >> > > > On Thu, Jan 4, 2018 at 1:33 PM, Jun Rao
> <
> > >> > > > >> > jun@confluent.io
> > >> > > > >> > > >
> > >> > > > >> > > > > > wrote:
> > >> > > > >> > > > > > > >> > > >
> > >> > > > >> > > > > > > >> > > > > Hi, Dong,
> > >> > > > >> > > > > > > >> > > > >
> > >> > > > >> > > > > > > >> > > > > Not sure that I fully understand your
> > >> latest
> > >> > > > >> > suggestion.
> > >> > > > >> > > > > > > >> Returning an
> > >> > > > >> > > > > > > >> > > > ever
> > >> > > > >> > > > > > > >> > > > > growing global metadata version itself
> > is
> > >> no
> > >> > > > ideal,
> > >> > > > >> > but
> > >> > > > >> > > is
> > >> > > > >> > > > > > fine.
> > >> > > > >> > > > > > > >> My
> > >> > > > >> > > > > > > >> > > > > question is whether the metadata
> version
> > >> > > returned
> > >> > > > >> in
> > >> > > > >> > the
> > >> > > > >> > > > > fetch
> > >> > > > >> > > > > > > >> > response
> > >> > > > >> > > > > > > >> > > > > needs to be stored with the offset
> > >> together
> > >> > if
> > >> > > > >> offsets
> > >> > > > >> > > are
> > >> > > > >> > > > > > > stored
> > >> > > > >> > > > > > > >> > > > > externally. If so, we also have to
> > change
> > >> the
> > >> > > > >> consumer
> > >> > > > >> > > API
> > >> > > > >> > > > > for
> > >> > > > >> > > > > > > >> > > > commitSync()
> > >> > > > >> > > > > > > >> > > > > and need to worry about compatibility.
> > If
> > >> we
> > >> > > > don't
> > >> > > > >> > store
> > >> > > > >> > > > the
> > >> > > > >> > > > > > > >> metadata
> > >> > > > >> > > > > > > >> > > > > version together with the offset, on a
> > >> > consumer
> > >> > > > >> > restart,
> > >> > > > >> > > > > it's
> > >> > > > >> > > > > > > not
> > >> > > > >> > > > > > > >> > clear
> > >> > > > >> > > > > > > >> > > > how
> > >> > > > >> > > > > > > >> > > > > we can ensure the metadata in the
> > >> consumer is
> > >> > > > high
> > >> > > > >> > > enough
> > >> > > > >> > > > > > since
> > >> > > > >> > > > > > > >> there
> > >> > > > >> > > > > > > >> > > is
> > >> > > > >> > > > > > > >> > > > no
> > >> > > > >> > > > > > > >> > > > > metadata version to compare with.
> > >> > > > >> > > > > > > >> > > > >
> > >> > > > >> > > > > > > >> > > > > Thanks,
> > >> > > > >> > > > > > > >> > > > >
> > >> > > > >> > > > > > > >> > > > > Jun
> > >> > > > >> > > > > > > >> > > > >
> > >> > > > >> > > > > > > >> > > > >
> > >> > > > >> > > > > > > >> > > > > On Wed, Jan 3, 2018 at 6:43 PM, Dong
> > Lin <
> > >> > > > >> > > > > lindong28@gmail.com
> > >> > > > >> > > > > > >
> > >> > > > >> > > > > > > >> > wrote:
> > >> > > > >> > > > > > > >> > > > >
> > >> > > > >> > > > > > > >> > > > > > Hey Jun,
> > >> > > > >> > > > > > > >> > > > > >
> > >> > > > >> > > > > > > >> > > > > > Thanks much for the explanation.
> > >> > > > >> > > > > > > >> > > > > >
> > >> > > > >> > > > > > > >> > > > > > I understand the advantage of
> > >> > partition_epoch
> > >> > > > >> over
> > >> > > > >> > > > > > > >> metadata_epoch.
> > >> > > > >> > > > > > > >> > My
> > >> > > > >> > > > > > > >> > > > > > current concern is that the use of
> > >> > > leader_epoch
> > >> > > > >> and
> > >> > > > >> > > the
> > >> > > > >> > > > > > > >> > > partition_epoch
> > >> > > > >> > > > > > > >> > > > > > requires us considerable change to
> > >> > consumer's
> > >> > > > >> public
> > >> > > > >> > > API
> > >> > > > >> > > > > to
> > >> > > > >> > > > > > > take
> > >> > > > >> > > > > > > >> > care
> > >> > > > >> > > > > > > >> > > > of
> > >> > > > >> > > > > > > >> > > > > > the case where user stores offset
> > >> > externally.
> > >> > > > For
> > >> > > > >> > > > example,
> > >> > > > >> > > > > > > >> > > *consumer*.
> > >> > > > >> > > > > > > >> > > > > > *commitSync*(..) would have to take
> a
> > >> map
> > >> > > whose
> > >> > > > >> > value
> > >> > > > >> > > is
> > >> > > > >> > > > > > > >> <offset,
> > >> > > > >> > > > > > > >> > > > > metadata,
> > >> > > > >> > > > > > > >> > > > > > leader epoch, partition epoch>.
> > >> > > > >> > *consumer*.*seek*(...)
> > >> > > > >> > > > > would
> > >> > > > >> > > > > > > >> also
> > >> > > > >> > > > > > > >> > > need
> > >> > > > >> > > > > > > >> > > > > > leader_epoch and partition_epoch as
> > >> > > parameter.
> > >> > > > >> > > > Technically
> > >> > > > >> > > > > > we
> > >> > > > >> > > > > > > >> can
> > >> > > > >> > > > > > > >> > > > > probably
> > >> > > > >> > > > > > > >> > > > > > still make it work in a backward
> > >> compatible
> > >> > > > >> manner
> > >> > > > >> > > after
> > >> > > > >> > > > > > > careful
> > >> > > > >> > > > > > > >> > > design
> > >> > > > >> > > > > > > >> > > > > and
> > >> > > > >> > > > > > > >> > > > > > discussion. But these changes can
> make
> > >> the
> > >> > > > >> > consumer's
> > >> > > > >> > > > > > > interface
> > >> > > > >> > > > > > > >> > > > > > unnecessarily complex for more users
> > >> who do
> > >> > > not
> > >> > > > >> > store
> > >> > > > >> > > > > offset
> > >> > > > >> > > > > > > >> > > > externally.
> > >> > > > >> > > > > > > >> > > > > >
> > >> > > > >> > > > > > > >> > > > > > After thinking more about it, we can
> > >> > address
> > >> > > > all
> > >> > > > >> > > > problems
> > >> > > > >> > > > > > > >> discussed
> > >> > > > >> > > > > > > >> > > by
> > >> > > > >> > > > > > > >> > > > > only
> > >> > > > >> > > > > > > >> > > > > > using the metadata_epoch without
> > >> > introducing
> > >> > > > >> > > > leader_epoch
> > >> > > > >> > > > > or
> > >> > > > >> > > > > > > the
> > >> > > > >> > > > > > > >> > > > > > partition_epoch. The current KIP
> > >> describes
> > >> > > the
> > >> > > > >> > changes
> > >> > > > >> > > > to
> > >> > > > >> > > > > > the
> > >> > > > >> > > > > > > >> > > consumer
> > >> > > > >> > > > > > > >> > > > > API
> > >> > > > >> > > > > > > >> > > > > > and how the new API can be used if
> > user
> > >> > > stores
> > >> > > > >> > offset
> > >> > > > >> > > > > > > >> externally.
> > >> > > > >> > > > > > > >> > In
> > >> > > > >> > > > > > > >> > > > > order
> > >> > > > >> > > > > > > >> > > > > > to address the scenario you
> described
> > >> > > earlier,
> > >> > > > we
> > >> > > > >> > can
> > >> > > > >> > > > > > include
> > >> > > > >> > > > > > > >> > > > > > metadata_epoch in the FetchResponse
> > and
> > >> the
> > >> > > > >> > > > > > > LeaderAndIsrRequest.
> > >> > > > >> > > > > > > >> > > > Consumer
> > >> > > > >> > > > > > > >> > > > > > remembers the largest metadata_epoch
> > >> from
> > >> > all
> > >> > > > the
> > >> > > > >> > > > > > > FetchResponse
> > >> > > > >> > > > > > > >> it
> > >> > > > >> > > > > > > >> > > has
> > >> > > > >> > > > > > > >> > > > > > received. The metadata_epoch
> committed
> > >> with
> > >> > > the
> > >> > > > >> > > offset,
> > >> > > > >> > > > > > either
> > >> > > > >> > > > > > > >> > within
> > >> > > > >> > > > > > > >> > > > or
> > >> > > > >> > > > > > > >> > > > > > outside Kafka, should be the largest
> > >> > > > >> metadata_epoch
> > >> > > > >> > > > across
> > >> > > > >> > > > > > all
> > >> > > > >> > > > > > > >> > > > > > FetchResponse and MetadataResponse
> > ever
> > >> > > > received
> > >> > > > >> by
> > >> > > > >> > > this
> > >> > > > >> > > > > > > >> consumer.
> > >> > > > >> > > > > > > >> > > > > >
> > >> > > > >> > > > > > > >> > > > > > The drawback of using only the
> > >> > metadata_epoch
> > >> > > > is
> > >> > > > >> > that
> > >> > > > >> > > we
> > >> > > > >> > > > > can
> > >> > > > >> > > > > > > not
> > >> > > > >> > > > > > > >> > > always
> > >> > > > >> > > > > > > >> > > > > do
> > >> > > > >> > > > > > > >> > > > > > the smart offset reset in case of
> > >> unclean
> > >> > > > leader
> > >> > > > >> > > > election
> > >> > > > >> > > > > > > which
> > >> > > > >> > > > > > > >> you
> > >> > > > >> > > > > > > >> > > > > > mentioned earlier. But in most case,
> > >> > unclean
> > >> > > > >> leader
> > >> > > > >> > > > > election
> > >> > > > >> > > > > > > >> > probably
> > >> > > > >> > > > > > > >> > > > > > happens when consumer is not
> > >> > > > >> rebalancing/restarting.
> > >> > > > >> > > In
> > >> > > > >> > > > > > these
> > >> > > > >> > > > > > > >> > cases,
> > >> > > > >> > > > > > > >> > > > > either
> > >> > > > >> > > > > > > >> > > > > > consumer is not directly affected by
> > >> > unclean
> > >> > > > >> leader
> > >> > > > >> > > > > election
> > >> > > > >> > > > > > > >> since
> > >> > > > >> > > > > > > >> > it
> > >> > > > >> > > > > > > >> > > > is
> > >> > > > >> > > > > > > >> > > > > > not consuming from the end of the
> log,
> > >> or
> > >> > > > >> consumer
> > >> > > > >> > can
> > >> > > > >> > > > > > derive
> > >> > > > >> > > > > > > >> the
> > >> > > > >> > > > > > > >> > > > > > leader_epoch from the most recent
> > >> message
> > >> > > > >> received
> > >> > > > >> > > > before
> > >> > > > >> > > > > it
> > >> > > > >> > > > > > > >> sees
> > >> > > > >> > > > > > > >> > > > > > OffsetOutOfRangeException. So I am
> not
> > >> sure
> > >> > > it
> > >> > > > is
> > >> > > > >> > > worth
> > >> > > > >> > > > > > adding
> > >> > > > >> > > > > > > >> the
> > >> > > > >> > > > > > > >> > > > > > leader_epoch to consumer API to
> > address
> > >> the
> > >> > > > >> > remaining
> > >> > > > >> > > > > corner
> > >> > > > >> > > > > > > >> case.
> > >> > > > >> > > > > > > >> > > What
> > >> > > > >> > > > > > > >> > > > > do
> > >> > > > >> > > > > > > >> > > > > > you think?
> > >> > > > >> > > > > > > >> > > > > >
> > >> > > > >> > > > > > > >> > > > > > Thanks,
> > >> > > > >> > > > > > > >> > > > > > Dong
> > >> > > > >> > > > > > > >> > > > > >
> > >> > > > >> > > > > > > >> > > > > >
> > >> > > > >> > > > > > > >> > > > > >
> > >> > > > >> > > > > > > >> > > > > > On Tue, Jan 2, 2018 at 6:28 PM, Jun
> > Rao
> > >> <
> > >> > > > >> > > > jun@confluent.io
> > >> > > > >> > > > > >
> > >> > > > >> > > > > > > >> wrote:
> > >> > > > >> > > > > > > >> > > > > >
> > >> > > > >> > > > > > > >> > > > > > > Hi, Dong,
> > >> > > > >> > > > > > > >> > > > > > >
> > >> > > > >> > > > > > > >> > > > > > > Thanks for the reply.
> > >> > > > >> > > > > > > >> > > > > > >
> > >> > > > >> > > > > > > >> > > > > > > To solve the topic recreation
> issue,
> > >> we
> > >> > > could
> > >> > > > >> use
> > >> > > > >> > > > > either a
> > >> > > > >> > > > > > > >> global
> > >> > > > >> > > > > > > >> > > > > > metadata
> > >> > > > >> > > > > > > >> > > > > > > version or a partition level
> epoch.
> > >> But
> > >> > > > either
> > >> > > > >> one
> > >> > > > >> > > > will
> > >> > > > >> > > > > > be a
> > >> > > > >> > > > > > > >> new
> > >> > > > >> > > > > > > >> > > > > concept,
> > >> > > > >> > > > > > > >> > > > > > > right? To me, the latter seems
> more
> > >> > > natural.
> > >> > > > It
> > >> > > > >> > also
> > >> > > > >> > > > > makes
> > >> > > > >> > > > > > > it
> > >> > > > >> > > > > > > >> > > easier
> > >> > > > >> > > > > > > >> > > > to
> > >> > > > >> > > > > > > >> > > > > > > detect if a consumer's offset is
> > still
> > >> > > valid
> > >> > > > >> > after a
> > >> > > > >> > > > > topic
> > >> > > > >> > > > > > > is
> > >> > > > >> > > > > > > >> > > > > recreated.
> > >> > > > >> > > > > > > >> > > > > > As
> > >> > > > >> > > > > > > >> > > > > > > you pointed out, we don't need to
> > >> store
> > >> > the
> > >> > > > >> > > partition
> > >> > > > >> > > > > > epoch
> > >> > > > >> > > > > > > in
> > >> > > > >> > > > > > > >> > the
> > >> > > > >> > > > > > > >> > > > > > message.
> > >> > > > >> > > > > > > >> > > > > > > The following is what I am
> thinking.
> > >> > When a
> > >> > > > >> > > partition
> > >> > > > >> > > > is
> > >> > > > >> > > > > > > >> created,
> > >> > > > >> > > > > > > >> > > we
> > >> > > > >> > > > > > > >> > > > > can
> > >> > > > >> > > > > > > >> > > > > > > assign a partition epoch from an
> > >> > > > >> ever-increasing
> > >> > > > >> > > > global
> > >> > > > >> > > > > > > >> counter
> > >> > > > >> > > > > > > >> > and
> > >> > > > >> > > > > > > >> > > > > store
> > >> > > > >> > > > > > > >> > > > > > > it in /brokers/topics/[topic]/
> > >> > > > >> > > > partitions/[partitionId]
> > >> > > > >> > > > > in
> > >> > > > >> > > > > > > ZK.
> > >> > > > >> > > > > > > >> > The
> > >> > > > >> > > > > > > >> > > > > > > partition
> > >> > > > >> > > > > > > >> > > > > > > epoch is propagated to every
> broker.
> > >> The
> > >> > > > >> consumer
> > >> > > > >> > > will
> > >> > > > >> > > > > be
> > >> > > > >> > > > > > > >> > tracking
> > >> > > > >> > > > > > > >> > > a
> > >> > > > >> > > > > > > >> > > > > > tuple
> > >> > > > >> > > > > > > >> > > > > > > of <offset, leader epoch,
> partition
> > >> > epoch>
> > >> > > > for
> > >> > > > >> > > > offsets.
> > >> > > > >> > > > > > If a
> > >> > > > >> > > > > > > >> > topic
> > >> > > > >> > > > > > > >> > > is
> > >> > > > >> > > > > > > >> > > > > > > recreated, it's possible that a
> > >> > consumer's
> > >> > > > >> offset
> > >> > > > >> > > and
> > >> > > > >> > > > > > leader
> > >> > > > >> > > > > > > >> > epoch
> > >> > > > >> > > > > > > >> > > > > still
> > >> > > > >> > > > > > > >> > > > > > > match that in the broker, but
> > >> partition
> > >> > > epoch
> > >> > > > >> > won't
> > >> > > > >> > > > be.
> > >> > > > >> > > > > In
> > >> > > > >> > > > > > > >> this
> > >> > > > >> > > > > > > >> > > case,
> > >> > > > >> > > > > > > >> > > > > we
> > >> > > > >> > > > > > > >> > > > > > > can potentially still treat the
> > >> > consumer's
> > >> > > > >> offset
> > >> > > > >> > as
> > >> > > > >> > > > out
> > >> > > > >> > > > > > of
> > >> > > > >> > > > > > > >> range
> > >> > > > >> > > > > > > >> > > and
> > >> > > > >> > > > > > > >> > > > > > reset
> > >> > > > >> > > > > > > >> > > > > > > the offset based on the offset
> reset
> > >> > policy
> > >> > > > in
> > >> > > > >> the
> > >> > > > >> > > > > > consumer.
> > >> > > > >> > > > > > > >> This
> > >> > > > >> > > > > > > >> > > > seems
> > >> > > > >> > > > > > > >> > > > > > > harder to do with a global
> metadata
> > >> > > version.
> > >> > > > >> > > > > > > >> > > > > > >
> > >> > > > >> > > > > > > >> > > > > > > Jun
> > >> > > > >> > > > > > > >> > > > > > >
> > >> > > > >> > > > > > > >> > > > > > >
> > >> > > > >> > > > > > > >> > > > > > >
> > >> > > > >> > > > > > > >> > > > > > > On Mon, Dec 25, 2017 at 6:56 AM,
> > Dong
> > >> > Lin <
> > >> > > > >> > > > > > > >> lindong28@gmail.com>
> > >> > > > >> > > > > > > >> > > > wrote:
> > >> > > > >> > > > > > > >> > > > > > >
> > >> > > > >> > > > > > > >> > > > > > > > Hey Jun,
> > >> > > > >> > > > > > > >> > > > > > > >
> > >> > > > >> > > > > > > >> > > > > > > > This is a very good example.
> After
> > >> > > thinking
> > >> > > > >> > > through
> > >> > > > >> > > > > this
> > >> > > > >> > > > > > > in
> > >> > > > >> > > > > > > >> > > > detail, I
> > >> > > > >> > > > > > > >> > > > > > > agree
> > >> > > > >> > > > > > > >> > > > > > > > that we need to commit offset
> with
> > >> > leader
> > >> > > > >> epoch
> > >> > > > >> > in
> > >> > > > >> > > > > order
> > >> > > > >> > > > > > > to
> > >> > > > >> > > > > > > >> > > address
> > >> > > > >> > > > > > > >> > > > > > this
> > >> > > > >> > > > > > > >> > > > > > > > example.
> > >> > > > >> > > > > > > >> > > > > > > >
> > >> > > > >> > > > > > > >> > > > > > > > I think the remaining question
> is
> > >> how
> > >> > to
> > >> > > > >> address
> > >> > > > >> > > the
> > >> > > > >> > > > > > > >> scenario
> > >> > > > >> > > > > > > >> > > that
> > >> > > > >> > > > > > > >> > > > > the
> > >> > > > >> > > > > > > >> > > > > > > > topic is deleted and re-created.
> > One
> > >> > > > possible
> > >> > > > >> > > > solution
> > >> > > > >> > > > > > is
> > >> > > > >> > > > > > > to
> > >> > > > >> > > > > > > >> > > commit
> > >> > > > >> > > > > > > >> > > > > > > offset
> > >> > > > >> > > > > > > >> > > > > > > > with both the leader epoch and
> the
> > >> > > metadata
> > >> > > > >> > > version.
> > >> > > > >> > > > > The
> > >> > > > >> > > > > > > >> logic
> > >> > > > >> > > > > > > >> > > and
> > >> > > > >> > > > > > > >> > > > > the
> > >> > > > >> > > > > > > >> > > > > > > > implementation of this solution
> > does
> > >> > not
> > >> > > > >> > require a
> > >> > > > >> > > > new
> > >> > > > >> > > > > > > >> concept
> > >> > > > >> > > > > > > >> > > > (e.g.
> > >> > > > >> > > > > > > >> > > > > > > > partition epoch) and it does not
> > >> > require
> > >> > > > any
> > >> > > > >> > > change
> > >> > > > >> > > > to
> > >> > > > >> > > > > > the
> > >> > > > >> > > > > > > >> > > message
> > >> > > > >> > > > > > > >> > > > > > format
> > >> > > > >> > > > > > > >> > > > > > > > or leader epoch. It also allows
> us
> > >> to
> > >> > > order
> > >> > > > >> the
> > >> > > > >> > > > > metadata
> > >> > > > >> > > > > > > in
> > >> > > > >> > > > > > > >> a
> > >> > > > >> > > > > > > >> > > > > > > > straightforward manner which may
> > be
> > >> > > useful
> > >> > > > in
> > >> > > > >> > the
> > >> > > > >> > > > > > future.
> > >> > > > >> > > > > > > >> So it
> > >> > > > >> > > > > > > >> > > may
> > >> > > > >> > > > > > > >> > > > > be
> > >> > > > >> > > > > > > >> > > > > > a
> > >> > > > >> > > > > > > >> > > > > > > > better solution than generating
> a
> > >> > random
> > >> > > > >> > partition
> > >> > > > >> > > > > epoch
> > >> > > > >> > > > > > > >> every
> > >> > > > >> > > > > > > >> > > time
> > >> > > > >> > > > > > > >> > > > > we
> > >> > > > >> > > > > > > >> > > > > > > > create a partition. Does this
> > sound
> > >> > > > >> reasonable?
> > >> > > > >> > > > > > > >> > > > > > > >
> > >> > > > >> > > > > > > >> > > > > > > > Previously one concern with
> using
> > >> the
> > >> > > > >> metadata
> > >> > > > >> > > > version
> > >> > > > >> > > > > > is
> > >> > > > >> > > > > > > >> that
> > >> > > > >> > > > > > > >> > > > > consumer
> > >> > > > >> > > > > > > >> > > > > > > > will be forced to refresh
> metadata
> > >> even
> > >> > > if
> > >> > > > >> > > metadata
> > >> > > > >> > > > > > > version
> > >> > > > >> > > > > > > >> is
> > >> > > > >> > > > > > > >> > > > > > increased
> > >> > > > >> > > > > > > >> > > > > > > > due to topics that the consumer
> is
> > >> not
> > >> > > > >> > interested
> > >> > > > >> > > > in.
> > >> > > > >> > > > > > Now
> > >> > > > >> > > > > > > I
> > >> > > > >> > > > > > > >> > > > realized
> > >> > > > >> > > > > > > >> > > > > > that
> > >> > > > >> > > > > > > >> > > > > > > > this is probably not a problem.
> > >> > Currently
> > >> > > > >> client
> > >> > > > >> > > > will
> > >> > > > >> > > > > > > >> refresh
> > >> > > > >> > > > > > > >> > > > > metadata
> > >> > > > >> > > > > > > >> > > > > > > > either due to
> > >> InvalidMetadataException
> > >> > in
> > >> > > > the
> > >> > > > >> > > > response
> > >> > > > >> > > > > > > from
> > >> > > > >> > > > > > > >> > > broker
> > >> > > > >> > > > > > > >> > > > or
> > >> > > > >> > > > > > > >> > > > > > due
> > >> > > > >> > > > > > > >> > > > > > > > to metadata expiry. The addition
> > of
> > >> the
> > >> > > > >> metadata
> > >> > > > >> > > > > version
> > >> > > > >> > > > > > > >> should
> > >> > > > >> > > > > > > >> > > > > > increase
> > >> > > > >> > > > > > > >> > > > > > > > the overhead of metadata refresh
> > >> caused
> > >> > > by
> > >> > > > >> > > > > > > >> > > > InvalidMetadataException.
> > >> > > > >> > > > > > > >> > > > > If
> > >> > > > >> > > > > > > >> > > > > > > > client refresh metadata due to
> > >> expiry
> > >> > and
> > >> > > > it
> > >> > > > >> > > > receives
> > >> > > > >> > > > > a
> > >> > > > >> > > > > > > >> > metadata
> > >> > > > >> > > > > > > >> > > > > whose
> > >> > > > >> > > > > > > >> > > > > > > > version is lower than the
> current
> > >> > > metadata
> > >> > > > >> > > version,
> > >> > > > >> > > > we
> > >> > > > >> > > > > > can
> > >> > > > >> > > > > > > >> > reject
> > >> > > > >> > > > > > > >> > > > the
> > >> > > > >> > > > > > > >> > > > > > > > metadata but still reset the
> > >> metadata
> > >> > > age,
> > >> > > > >> which
> > >> > > > >> > > > > > > essentially
> > >> > > > >> > > > > > > >> > keep
> > >> > > > >> > > > > > > >> > > > the
> > >> > > > >> > > > > > > >> > > > > > > > existing behavior in the client.
> > >> > > > >> > > > > > > >> > > > > > > >
> > >> > > > >> > > > > > > >> > > > > > > > Thanks much,
> > >> > > > >> > > > > > > >> > > > > > > > Dong
> > >> > > > >> > > > > > > >> > > > > > > >
> > >> > > > >> > > > > > > >> > > > > > >
> > >> > > > >> > > > > > > >> > > > > >
> > >> > > > >> > > > > > > >> > > > >
> > >> > > > >> > > > > > > >> > > >
> > >> > > > >> > > > > > > >> > >
> > >> > > > >> > > > > > > >> >
> > >> > > > >> > > > > > > >>
> > >> > > > >> > > > > > > >
> > >> > > > >> > > > > > > >
> > >> > > > >> > > > > > >
> > >> > > > >> > > > > >
> > >> > > > >> > > > >
> > >> > > > >> > > >
> > >> > > > >> > >
> > >> > > > >> >
> > >> > > > >>
> > >> > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Jun Rao <ju...@confluent.io>.
Hi, Dong,

Thanks for the updated KIP. It looks good to me now. The only thing is
for OffsetEpoch.
If we expose the individual fields in the class, we probably don't need the
encode/decode methods. If we want to hide the details of OffsetEpoch, we
probably don't want expose the individual fields.

Jun

On Wed, Jan 17, 2018 at 10:10 AM, Dong Lin <li...@gmail.com> wrote:

> Thinking about point 61 more, I realize that the async zookeeper read may
> make it less of an issue for controller to read more zookeeper nodes.
> Writing partition_epoch in the per-partition znode makes it simpler to
> handle the broker failure between zookeeper writes for a topic creation. I
> have updated the KIP to use the suggested approach.
>
>
> On Wed, Jan 17, 2018 at 9:57 AM, Dong Lin <li...@gmail.com> wrote:
>
> > Hey Jun,
> >
> > Thanks much for the comments. Please see my comments inline.
> >
> > On Tue, Jan 16, 2018 at 4:38 PM, Jun Rao <ju...@confluent.io> wrote:
> >
> >> Hi, Dong,
> >>
> >> Thanks for the updated KIP. Looks good to me overall. Just a few minor
> >> comments.
> >>
> >> 60. OffsetAndMetadata positionAndOffsetEpoch(TopicPartition partition):
> >> It
> >> seems that there is no need to return metadata. We probably want to
> return
> >> sth like OffsetAndEpoch.
> >>
> >
> > Previously I think we may want to re-use the existing class to keep our
> > consumer interface simpler. I have updated the KIP to add class
> > OffsetAndOffsetEpoch. I didn't use OffsetAndEpoch because user may
> confuse
> > this name with OffsetEpoch. Does this sound OK?
> >
> >
> >>
> >> 61. Should we store partition_epoch in
> >> /brokers/topics/[topic]/partitions/[partitionId] in ZK?
> >>
> >
> > I have considered this. I think the advantage of adding the
> > partition->partition_epoch map in the existing
> > znode /brokers/topics/[topic]/partitions is that controller only needs
> to
> > read one znode per topic to gets its partition_epoch information.
> Otherwise
> > controller may need to read one extra znode per partition to get the same
> > information.
> >
> > When we delete partition or expand partition of a topic, someone needs to
> > modify partition->partition_epoch map in znode
> > /brokers/topics/[topic]/partitions. This may seem a bit more complicated
> > than simply adding or deleting znode /brokers/topics/[topic]/
> partitions/[partitionId].
> > But the complexity is probably similar to the existing operation of
> > modifying the partition->replica_list mapping in znode
> > /brokers/topics/[topic]. So not sure it is better to store the
> > partition_epoch in /brokers/topics/[topic]/partitions/[partitionId].
> What
> > do you think?
> >
> >
> >>
> >> 62. For checking outdated metadata in the client, we probably want to
> add
> >> when max_partition_epoch will be used.
> >>
> >
> > The max_partition_epoch is used in the Proposed Changes -> Client's
> > metadata refresh section to determine whether a metadata is outdated. And
> > this formula is referenced and re-used in other sections to determine
> > whether a metadata is outdated. Does this formula look OK?
> >
> >
> >>
> >> 63. "The leader_epoch should be the largest leader_epoch of messages
> whose
> >> offset < the commit offset. If no message has been consumed since
> consumer
> >> initialization, the leader_epoch from seek(...) or OffsetFetchResponse
> >> should be used. The partition_epoch should be read from the last
> >> FetchResponse corresponding to the given partition and commit offset. ":
> >> leader_epoch and partition_epoch are associated with an offset. So, if
> no
> >> message is consumed, there is no offset and therefore there is no need
> to
> >> read leader_epoch and partition_epoch. Also, the leader_epoch associated
> >> with the offset should just come from the messages returned in the fetch
> >> response.
> >>
> >
> > I am thinking that, if user calls seek(..) and commitSync(...) without
> > consuming any messages, we should re-use the leader_epoch and
> > partition_epoch provided by the seek(...) in the OffsetCommitRequest. And
> > if messages have been successfully consumed, then leader_epoch will come
> > from the messages returned in the fetch response. The condition "messages
> > whose offset < the commit offset" is needed to take care of the log
> > compacted topic which may have offset gap due to log cleaning.
> >
> > Did I miss something here? Or should I rephrase the paragraph to make it
> > less confusing?
> >
> >
> >> 64. Could you include the public methods in the OffsetEpoch class?
> >>
> >
> > I mistakenly deleted the definition of OffsetEpoch class from the KIP. I
> > just added it back with the public methods. Could you take another look?
> >
> >
> >>
> >> Jun
> >>
> >>
> >> On Thu, Jan 11, 2018 at 5:43 PM, Dong Lin <li...@gmail.com> wrote:
> >>
> >> > Hey Jun,
> >> >
> >> > Thanks much. I agree that we can not rely on committed offsets to be
> >> always
> >> > deleted when we delete topic. So it is necessary to use a
> per-partition
> >> > epoch that does not change unless this partition is deleted. I also
> >> agree
> >> > that it is very nice to be able to uniquely identify a message with
> >> > (offset, leader_epoch, partition_epoch) in face of potential topic
> >> deletion
> >> > and unclean leader election.
> >> >
> >> > I agree with all your comments. And I have updated the KIP based on
> our
> >> > latest discussion. In addition, I added InvalidPartitionEpochException
> >> > which will be thrown by consumer.poll() if the partition_epoch
> >> associated
> >> > with the partition, which can be given to consumer using seek(...), is
> >> > different from the partition_epoch in the FetchResponse.
> >> >
> >> > Can you take another look at the latest KIP?
> >> >
> >> > Thanks!
> >> > Dong
> >> >
> >> >
> >> >
> >> > On Wed, Jan 10, 2018 at 2:24 PM, Jun Rao <ju...@confluent.io> wrote:
> >> >
> >> > > Hi, Dong,
> >> > >
> >> > > My replies are the following.
> >> > >
> >> > > 60. What you described could also work. The drawback is that we will
> >> be
> >> > > unnecessarily changing the partition epoch when a partition hasn't
> >> really
> >> > > changed. I was imagining that the partition epoch will be stored in
> >> > > /brokers/topics/[topic]/partitions/[partitionId], instead of at the
> >> > topic
> >> > > level. So, not sure if ZK size limit is an issue.
> >> > >
> >> > > 61, 62 and 65. To me, the offset + offset_epoch is a unique
> identifier
> >> > for
> >> > > a message. So, if a message hasn't changed, the offset and the
> >> associated
> >> > > offset_epoch ideally should remain the same (it will be kind of
> weird
> >> if
> >> > > two consumer apps save the offset on the same message, but the
> >> > offset_epoch
> >> > > are different). partition_epoch + leader_epoch give us that.
> >> > global_epoch +
> >> > > leader_epoch don't. If we use this approach, we can solve not only
> the
> >> > > problem that you have identified, but also other problems when there
> >> is
> >> > > data loss or topic re-creation more reliably. For example, in the
> >> future,
> >> > > if we include the partition_epoch and leader_epoch in the fetch
> >> request,
> >> > > the server can do a more reliable check of whether that offset is
> >> valid
> >> > or
> >> > > not. I am not sure that we can rely upon all external offsets to be
> >> > removed
> >> > > on topic deletion. For example, a topic may be deleted by an admin
> who
> >> > may
> >> > > not know all the applications.
> >> > >
> >> > > If we agree on the above, the second question is then how to
> reliably
> >> > > propagate the partition_epoch and the leader_epoch to the consumer
> >> when
> >> > > there are leader or partition changes. The leader_epoch comes from
> the
> >> > > message, which is reliable. So, I was suggesting that when we store
> an
> >> > > offset, we can just store the leader_epoch from the message set
> >> > containing
> >> > > that offset. Similarly, I was thinking that if the partition_epoch
> is
> >> in
> >> > > the fetch response, we can propagate partition_epoch reliably where
> is
> >> > > partition_epoch change.
> >> > >
> >> > > 63. My point is that once a leader is producing a message in the new
> >> > > partition_epoch, ideally, we should associate the new offsets with
> the
> >> > new
> >> > > partition_epoch. Otherwise, the offset_epoch won't be the correct
> >> unique
> >> > > identifier (useful for solving other problems mentioned above). I
> was
> >> > > originally thinking that the leader will include the partition_epoch
> >> in
> >> > the
> >> > > metadata cache in the fetch response. It's just that right now,
> >> metadata
> >> > > cache is updated on UpdateMetadataRequest, which typically happens
> >> after
> >> > > the LeaderAndIsrRequest. Another approach is for the leader to cache
> >> the
> >> > > partition_epoch in the Partition object and return that (instead of
> >> the
> >> > one
> >> > > in metadata cache) in the fetch response.
> >> > >
> >> > > 65. It seems to me that the global_epoch and the partition_epoch
> have
> >> > > different purposes. A partition_epoch has the benefit that it (1)
> can
> >> be
> >> > > used to form a unique identifier for a message and (2) can be used
> to
> >> > > solve other
> >> > > corner case problems in the future. I am not sure having just a
> >> > > global_epoch can achieve these. global_epoch is useful to determine
> >> which
> >> > > version of the metadata is newer, especially with topic deletion.
> >> > >
> >> > > Thanks,
> >> > >
> >> > > Jun
> >> > >
> >> > > On Tue, Jan 9, 2018 at 11:34 PM, Dong Lin <li...@gmail.com>
> >> wrote:
> >> > >
> >> > > > Regarding the use of the global epoch in 65), it is very similar
> to
> >> the
> >> > > > proposal of the metadata_epoch we discussed earlier. The main
> >> > difference
> >> > > is
> >> > > > that this epoch is incremented when we create/expand/delete topic
> >> and
> >> > > does
> >> > > > not change when controller re-send metadata.
> >> > > >
> >> > > > I looked at our previous discussion. It seems that we prefer
> >> > > > partition_epoch over the metadata_epoch because 1) we prefer not
> to
> >> > have
> >> > > an
> >> > > > ever growing metadata_epoch and 2) we can reset offset better when
> >> > topic
> >> > > is
> >> > > > re-created. The use of global topic_epoch avoids the drawback of
> an
> >> > ever
> >> > > > quickly ever growing metadata_epoch. Though the global epoch does
> >> not
> >> > > allow
> >> > > > us to recognize the invalid offset committed before the topic
> >> > > re-creation,
> >> > > > we can probably just delete the offset when we delete a topic.
> Thus
> >> I
> >> > am
> >> > > > not very sure whether it is still worthwhile to have a
> per-partition
> >> > > > partition_epoch if the metadata already has the global epoch.
> >> > > >
> >> > > >
> >> > > > On Tue, Jan 9, 2018 at 6:58 PM, Dong Lin <li...@gmail.com>
> >> wrote:
> >> > > >
> >> > > > > Hey Jun,
> >> > > > >
> >> > > > > Thanks so much. These comments very useful. Please see below my
> >> > > comments.
> >> > > > >
> >> > > > > On Mon, Jan 8, 2018 at 5:52 PM, Jun Rao <ju...@confluent.io>
> wrote:
> >> > > > >
> >> > > > >> Hi, Dong,
> >> > > > >>
> >> > > > >> Thanks for the updated KIP. A few more comments.
> >> > > > >>
> >> > > > >> 60. Perhaps having a partition epoch is more flexible since in
> >> the
> >> > > > future,
> >> > > > >> we may support deleting a partition as well.
> >> > > > >>
> >> > > > >
> >> > > > > Yeah I have considered this. I think we can probably still
> support
> >> > > > > deleting a partition by using the topic_epoch -- when partition
> >> of a
> >> > > > topic
> >> > > > > is deleted or created, epoch of all partitions of this topic
> will
> >> be
> >> > > > > incremented by 1. Therefore, if that partition is re-created
> >> later,
> >> > the
> >> > > > > epoch of that partition will still be larger than its epoch
> before
> >> > the
> >> > > > > deletion, which still allows the client to order the metadata
> for
> >> the
> >> > > > > purpose of this KIP. Does this sound reasonable?
> >> > > > >
> >> > > > > The advantage of using topic_epoch instead of partition_epoch is
> >> that
> >> > > the
> >> > > > > size of the /brokers/topics/[topic] znode and request/response
> >> size
> >> > can
> >> > > > be
> >> > > > > smaller. We have a limit on the maximum size of znode (typically
> >> > 1MB).
> >> > > > Use
> >> > > > > partition epoch can effectively reduce the number of partitions
> >> that
> >> > > can
> >> > > > be
> >> > > > > described by the /brokers/topics/[topic] znode.
> >> > > > >
> >> > > > > One use-case of partition_epoch for client to detect that the
> >> > committed
> >> > > > > offset, either from kafka offset topic or from the external
> store
> >> is
> >> > > > > invalid after partition deletion and re-creation. However, it
> >> seems
> >> > > that
> >> > > > we
> >> > > > > can also address this use-case with other approaches. For
> example,
> >> > when
> >> > > > > AdminClient deletes partitions, it can also delete the committed
> >> > > offsets
> >> > > > > for those partitions from the offset topic. If user stores
> offset
> >> > > > > externally, it might make sense for user to similarly remove
> >> offsets
> >> > of
> >> > > > > related partitions after these partitions are deleted. So I am
> not
> >> > sure
> >> > > > > that we should use partition_epoch in this KIP.
> >> > > > >
> >> > > > >
> >> > > > >>
> >> > > > >> 61. It seems that the leader epoch returned in the position()
> >> call
> >> > > > should
> >> > > > >> the the leader epoch returned in the fetch response, not the
> one
> >> in
> >> > > the
> >> > > > >> metadata cache of the client.
> >> > > > >
> >> > > > >
> >> > > > > I think this is a good idea. Just to double check, this change
> >> does
> >> > not
> >> > > > > affect the correctness or performance of this KIP. But it can be
> >> > useful
> >> > > > if
> >> > > > > we want to use the leader_epoch to better handle the offset rest
> >> in
> >> > > case
> >> > > > of
> >> > > > > unclean leader election, which is listed in the future work. Is
> >> this
> >> > > > > understanding correct?
> >> > > > >
> >> > > > > I have updated the KIP to specify that the leader_epoch returned
> >> by
> >> > > > > position() should be the largest leader_epoch of those already
> >> > consumed
> >> > > > > messages whose offset < position. If no message has been
> consumed
> >> > since
> >> > > > > consumer initialization, the leader_epoch from seek() or
> >> > > > > OffsetFetchResponse should be used. The offset included in the
> >> > > > > OffsetCommitRequest will also be determined in the similar
> manner.
> >> > > > >
> >> > > > >
> >> > > > >>
> >> > > > >> 62. I am wondering if we should return the partition epoch in
> the
> >> > > fetch
> >> > > > >> response as well. In the current proposal, if a topic is
> >> recreated
> >> > and
> >> > > > the
> >> > > > >> new leader is on the same broker as the old one, there is
> >> nothing to
> >> > > > force
> >> > > > >> the metadata refresh in the client. So, the client may still
> >> > associate
> >> > > > the
> >> > > > >> offset with the old partition epoch.
> >> > > > >>
> >> > > > >
> >> > > > > Could you help me understand the problem if a client associates
> >> old
> >> > > > > partition_epoch (or the topic_epoch as of the current KIP) with
> >> the
> >> > > > offset?
> >> > > > > The main purpose of the topic_epoch is to be able to drop
> >> > leader_epoch
> >> > > > to 0
> >> > > > > after a partition is deleted and re-created. I guess you may be
> >> > > thinking
> >> > > > > about using the partition_epoch to detect that the committed
> >> offset
> >> > is
> >> > > > > invalid? In that case, I am wondering if the alternative
> approach
> >> > > > described
> >> > > > > in 60) would be reasonable.
> >> > > > >
> >> > > > >
> >> > > > >>
> >> > > > >> 63. There is some subtle coordination between the
> >> > LeaderAndIsrRequest
> >> > > > and
> >> > > > >> UpdateMetadataRequest. Currently, when a leader changes, the
> >> > > controller
> >> > > > >> first sends the LeaderAndIsrRequest to the assigned replicas
> and
> >> the
> >> > > > >> UpdateMetadataRequest to every broker. So, there could be a
> small
> >> > > window
> >> > > > >> when the leader already receives the new partition epoch in the
> >> > > > >> LeaderAndIsrRequest, but the metadata cache in the broker
> hasn't
> >> > been
> >> > > > >> updated with the latest partition epoch. Not sure what's the
> best
> >> > way
> >> > > to
> >> > > > >> address this issue. Perhaps we can update the metadata cache on
> >> the
> >> > > > broker
> >> > > > >> with both LeaderAndIsrRequest and UpdateMetadataRequest. The
> >> > challenge
> >> > > > is
> >> > > > >> that the two have slightly different data. For example, only
> the
> >> > > latter
> >> > > > >> has
> >> > > > >> all endpoints.
> >> > > > >>
> >> > > > >
> >> > > > > I am not sure whether this is a problem. Could you explain a bit
> >> more
> >> > > > what
> >> > > > > specific problem this small window can cause?
> >> > > > >
> >> > > > > Since client can fetch metadata from any broker in the cluster,
> >> and
> >> > > given
> >> > > > > that different brokers receive request (e.g. LeaderAndIsrRequest
> >> and
> >> > > > > UpdateMetadataRequest) in arbitrary order, the metadata received
> >> by
> >> > > > client
> >> > > > > can be in arbitrary order (either newer or older) compared to
> the
> >> > > > broker's
> >> > > > > leadership state even if a given broker receives
> >> LeaderAndIsrRequest
> >> > > and
> >> > > > > UpdateMetadataRequest simultaneously. So I am not sure it is
> >> useful
> >> > to
> >> > > > > update broker's cache with LeaderAndIsrRequest.
> >> > > > >
> >> > > > >
> >> > > > >> 64. The enforcement of leader epoch in Offset commit: We allow
> a
> >> > > > consumer
> >> > > > >> to set an arbitrary offset. So it's possible for offsets or
> >> leader
> >> > > epoch
> >> > > > >> to
> >> > > > >> go backwards. I am not sure if we could always enforce that the
> >> > leader
> >> > > > >> epoch only goes up on the broker.
> >> > > > >>
> >> > > > >
> >> > > > > Sure. I have removed this check from the KIP.
> >> > > > >
> >> > > > > BTW, we can probably still ensure that the leader_epoch always
> >> > increase
> >> > > > if
> >> > > > > the leader_epoch used with offset commit is the max(leader_epoch
> >> of
> >> > the
> >> > > > > message with offset = the committed offset - 1, the largest
> known
> >> > > > > leader_epoch from the metadata). But I don't have a good
> use-case
> >> for
> >> > > > this
> >> > > > > alternative definition. So I choose the keep the KIP simple by
> >> > > requiring
> >> > > > > leader_epoch to always increase.
> >> > > > >
> >> > > > >
> >> > > > >> 65. Good point on handling missing partition epoch due to topic
> >> > > > deletion.
> >> > > > >> Another potential way to address this is to additionally
> >> propagate
> >> > the
> >> > > > >> global partition epoch to brokers and the clients. This way,
> >> when a
> >> > > > >> partition epoch is missing, we can use the global partition
> >> epoch to
> >> > > > >> reason
> >> > > > >> about which metadata is more recent.
> >> > > > >>
> >> > > > >
> >> > > > > This is a great idea. The global epoch can be used to order the
> >> > > metadata
> >> > > > > and help us recognize the more recent metadata if a topic (or
> >> > > partition)
> >> > > > is
> >> > > > > deleted and re-created.
> >> > > > >
> >> > > > > Actually, it seems we only need to propagate the global epoch to
> >> > > brokers
> >> > > > > and clients without propagating this epoch on a per-topic or
> >> > > > per-partition
> >> > > > > basic. Doing so would simply interface changes made this KIP.
> Does
> >> > this
> >> > > > > approach sound reasonable?
> >> > > > >
> >> > > > >
> >> > > > >> 66. A client may also get an offset by time using the
> >> > offsetForTimes()
> >> > > > >> api.
> >> > > > >> So, we probably want to include offsetInternalMetadata in
> >> > > > >> OffsetAndTimestamp
> >> > > > >> as well.
> >> > > > >>
> >> > > > >
> >> > > > > You are right. This probably also requires us to change the
> >> > > > > ListOffsetRequest as well. I will update the KIP after we agree
> on
> >> > the
> >> > > > > solution for 65).
> >> > > > >
> >> > > > >
> >> > > > >>
> >> > > > >> 67. InteralMetadata can be a bit confusing with the metadata
> >> field
> >> > > > already
> >> > > > >> there. Perhaps we can just call it OffsetEpoch. It might be
> >> useful
> >> > to
> >> > > > make
> >> > > > >> OffsetEpoch printable at least for debugging purpose. Once you
> do
> >> > > that,
> >> > > > we
> >> > > > >> are already exposing the internal fields. So, not sure if it's
> >> worth
> >> > > > >> hiding
> >> > > > >> them. If we do want to hide them, perhaps we can have sth like
> >> the
> >> > > > >> following. The binary encoding is probably more efficient than
> >> JSON
> >> > > for
> >> > > > >> external storage.
> >> > > > >>
> >> > > > >> OffsetEpoch {
> >> > > > >>  static OffsetEpoch decode(byte[]);
> >> > > > >>
> >> > > > >>   public byte[] encode();
> >> > > > >>
> >> > > > >>   public String toString();
> >> > > > >> }
> >> > > > >>
> >> > > > >
> >> > > > > Thanks much. I like this solution. I have updated the KIP
> >> > accordingly.
> >> > > > >
> >> > > > >
> >> > > > >
> >> > > > >>
> >> > > > >> Jun
> >> > > > >>
> >> > > > >> On Mon, Jan 8, 2018 at 4:22 PM, Dong Lin <li...@gmail.com>
> >> > wrote:
> >> > > > >>
> >> > > > >> > Hey Jason,
> >> > > > >> >
> >> > > > >> > Certainly. This sounds good. I have updated the KIP to
> clarity
> >> > that
> >> > > > the
> >> > > > >> > global epoch will be incremented by 1 each time a topic is
> >> > deleted.
> >> > > > >> >
> >> > > > >> > Thanks,
> >> > > > >> > Dong
> >> > > > >> >
> >> > > > >> > On Mon, Jan 8, 2018 at 4:09 PM, Jason Gustafson <
> >> > jason@confluent.io
> >> > > >
> >> > > > >> > wrote:
> >> > > > >> >
> >> > > > >> > > Hi Dong,
> >> > > > >> > >
> >> > > > >> > >
> >> > > > >> > > I think your approach will allow user to distinguish
> between
> >> the
> >> > > > >> metadata
> >> > > > >> > > > before and after the topic deletion. I also agree that
> this
> >> > can
> >> > > be
> >> > > > >> > > > potentially be useful to user. I am just not very sure
> >> whether
> >> > > we
> >> > > > >> > already
> >> > > > >> > > > have a good use-case to make the additional complexity
> >> > > worthwhile.
> >> > > > >> It
> >> > > > >> > > seems
> >> > > > >> > > > that this feature is kind of independent of the main
> >> problem
> >> > of
> >> > > > this
> >> > > > >> > KIP.
> >> > > > >> > > > Could we add this as a future work?
> >> > > > >> > >
> >> > > > >> > >
> >> > > > >> > > Do you think it's fair if we bump the topic epoch on
> deletion
> >> > and
> >> > > > >> leave
> >> > > > >> > > propagation of the epoch for deleted topics for future
> work?
> >> I
> >> > > don't
> >> > > > >> > think
> >> > > > >> > > this adds much complexity and it makes the behavior
> >> consistent:
> >> > > > every
> >> > > > >> > topic
> >> > > > >> > > mutation results in an epoch bump.
> >> > > > >> > >
> >> > > > >> > > Thanks,
> >> > > > >> > > Jason
> >> > > > >> > >
> >> > > > >> > > On Mon, Jan 8, 2018 at 3:14 PM, Dong Lin <
> >> lindong28@gmail.com>
> >> > > > wrote:
> >> > > > >> > >
> >> > > > >> > > > Hey Ismael,
> >> > > > >> > > >
> >> > > > >> > > > I guess we actually need user to see this field so that
> >> user
> >> > can
> >> > > > >> store
> >> > > > >> > > this
> >> > > > >> > > > value in the external store together with the offset. We
> >> just
> >> > > > prefer
> >> > > > >> > the
> >> > > > >> > > > value to be opaque to discourage most users from
> >> interpreting
> >> > > this
> >> > > > >> > value.
> >> > > > >> > > > One more advantage of using such an opaque field is to be
> >> able
> >> > > to
> >> > > > >> > evolve
> >> > > > >> > > > the information (or schema) of this value without
> changing
> >> > > > consumer
> >> > > > >> API
> >> > > > >> > > in
> >> > > > >> > > > the future.
> >> > > > >> > > >
> >> > > > >> > > > I also thinking it is probably OK for user to be able to
> >> > > interpret
> >> > > > >> this
> >> > > > >> > > > value, particularly for those advanced users.
> >> > > > >> > > >
> >> > > > >> > > > Thanks,
> >> > > > >> > > > Dong
> >> > > > >> > > >
> >> > > > >> > > > On Mon, Jan 8, 2018 at 2:34 PM, Ismael Juma <
> >> > ismael@juma.me.uk>
> >> > > > >> wrote:
> >> > > > >> > > >
> >> > > > >> > > > > On Fri, Jan 5, 2018 at 7:15 PM, Jason Gustafson <
> >> > > > >> jason@confluent.io>
> >> > > > >> > > > > wrote:
> >> > > > >> > > > > >
> >> > > > >> > > > > > class OffsetAndMetadata {
> >> > > > >> > > > > >   long offset;
> >> > > > >> > > > > >   byte[] offsetMetadata;
> >> > > > >> > > > > >   String metadata;
> >> > > > >> > > > > > }
> >> > > > >> > > > >
> >> > > > >> > > > >
> >> > > > >> > > > > > Admittedly, the naming is a bit annoying, but we can
> >> > > probably
> >> > > > >> come
> >> > > > >> > up
> >> > > > >> > > > > with
> >> > > > >> > > > > > something better. Internally the byte array would
> have
> >> a
> >> > > > >> version.
> >> > > > >> > If
> >> > > > >> > > in
> >> > > > >> > > > > the
> >> > > > >> > > > > > future we have anything else we need to add, we can
> >> update
> >> > > the
> >> > > > >> > > version
> >> > > > >> > > > > and
> >> > > > >> > > > > > we wouldn't need any new APIs.
> >> > > > >> > > > > >
> >> > > > >> > > > >
> >> > > > >> > > > > We can also add fields to a class in a compatible way.
> >> So,
> >> > it
> >> > > > >> seems
> >> > > > >> > to
> >> > > > >> > > me
> >> > > > >> > > > > that the main advantage of the byte array is that it's
> >> > opaque
> >> > > to
> >> > > > >> the
> >> > > > >> > > > user.
> >> > > > >> > > > > Is that correct? If so, we could also add any opaque
> >> > metadata
> >> > > > in a
> >> > > > >> > > > subclass
> >> > > > >> > > > > so that users don't even see it (unless they cast it,
> but
> >> > then
> >> > > > >> > they're
> >> > > > >> > > on
> >> > > > >> > > > > their own).
> >> > > > >> > > > >
> >> > > > >> > > > > Ismael
> >> > > > >> > > > >
> >> > > > >> > > > > The corresponding seek() and position() APIs might look
> >> > > > something
> >> > > > >> > like
> >> > > > >> > > > > this:
> >> > > > >> > > > > >
> >> > > > >> > > > > > void seek(TopicPartition partition, long offset,
> byte[]
> >> > > > >> > > > offsetMetadata);
> >> > > > >> > > > > > byte[] positionMetadata(TopicPartition partition);
> >> > > > >> > > > > >
> >> > > > >> > > > > > What do you think?
> >> > > > >> > > > > >
> >> > > > >> > > > > > Thanks,
> >> > > > >> > > > > > Jason
> >> > > > >> > > > > >
> >> > > > >> > > > > > On Thu, Jan 4, 2018 at 7:04 PM, Dong Lin <
> >> > > lindong28@gmail.com
> >> > > > >
> >> > > > >> > > wrote:
> >> > > > >> > > > > >
> >> > > > >> > > > > > > Hey Jun, Jason,
> >> > > > >> > > > > > >
> >> > > > >> > > > > > > Thanks much for all the feedback. I have updated
> the
> >> KIP
> >> > > > >> based on
> >> > > > >> > > the
> >> > > > >> > > > > > > latest discussion. Can you help check whether it
> >> looks
> >> > > good?
> >> > > > >> > > > > > >
> >> > > > >> > > > > > > Thanks,
> >> > > > >> > > > > > > Dong
> >> > > > >> > > > > > >
> >> > > > >> > > > > > > On Thu, Jan 4, 2018 at 5:36 PM, Dong Lin <
> >> > > > lindong28@gmail.com
> >> > > > >> >
> >> > > > >> > > > wrote:
> >> > > > >> > > > > > >
> >> > > > >> > > > > > > > Hey Jun,
> >> > > > >> > > > > > > >
> >> > > > >> > > > > > > > Hmm... thinking about this more, I am not sure
> that
> >> > the
> >> > > > >> > proposed
> >> > > > >> > > > API
> >> > > > >> > > > > is
> >> > > > >> > > > > > > > sufficient. For users that store offset
> >> externally, we
> >> > > > >> probably
> >> > > > >> > > > need
> >> > > > >> > > > > > > extra
> >> > > > >> > > > > > > > API to return the leader_epoch and
> partition_epoch
> >> for
> >> > > all
> >> > > > >> > > > partitions
> >> > > > >> > > > > > > that
> >> > > > >> > > > > > > > consumers are consuming. I suppose these users
> >> > currently
> >> > > > use
> >> > > > >> > > > > position()
> >> > > > >> > > > > > > to
> >> > > > >> > > > > > > > get the offset. Thus we probably need a new
> method
> >> > > > >> > > > > > positionWithEpoch(..)
> >> > > > >> > > > > > > to
> >> > > > >> > > > > > > > return <offset, partition_epoch, leader_epoch>.
> >> Does
> >> > > this
> >> > > > >> sound
> >> > > > >> > > > > > > reasonable?
> >> > > > >> > > > > > > >
> >> > > > >> > > > > > > > Thanks,
> >> > > > >> > > > > > > > Dong
> >> > > > >> > > > > > > >
> >> > > > >> > > > > > > >
> >> > > > >> > > > > > > > On Thu, Jan 4, 2018 at 5:26 PM, Jun Rao <
> >> > > jun@confluent.io
> >> > > > >
> >> > > > >> > > wrote:
> >> > > > >> > > > > > > >
> >> > > > >> > > > > > > >> Hi, Dong,
> >> > > > >> > > > > > > >>
> >> > > > >> > > > > > > >> Yes, that's what I am thinking. OffsetEpoch will
> >> be
> >> > > > >> composed
> >> > > > >> > of
> >> > > > >> > > > > > > >> (partition_epoch,
> >> > > > >> > > > > > > >> leader_epoch).
> >> > > > >> > > > > > > >>
> >> > > > >> > > > > > > >> Thanks,
> >> > > > >> > > > > > > >>
> >> > > > >> > > > > > > >> Jun
> >> > > > >> > > > > > > >>
> >> > > > >> > > > > > > >>
> >> > > > >> > > > > > > >> On Thu, Jan 4, 2018 at 4:22 PM, Dong Lin <
> >> > > > >> lindong28@gmail.com
> >> > > > >> > >
> >> > > > >> > > > > wrote:
> >> > > > >> > > > > > > >>
> >> > > > >> > > > > > > >> > Hey Jun,
> >> > > > >> > > > > > > >> >
> >> > > > >> > > > > > > >> > Thanks much. I like the the new API that you
> >> > > proposed.
> >> > > > I
> >> > > > >> am
> >> > > > >> > > not
> >> > > > >> > > > > sure
> >> > > > >> > > > > > > >> what
> >> > > > >> > > > > > > >> > you exactly mean by offset_epoch. I suppose
> >> that we
> >> > > can
> >> > > > >> use
> >> > > > >> > > the
> >> > > > >> > > > > pair
> >> > > > >> > > > > > > of
> >> > > > >> > > > > > > >> > (partition_epoch, leader_epoch) as the
> >> > offset_epoch,
> >> > > > >> right?
> >> > > > >> > > > > > > >> >
> >> > > > >> > > > > > > >> > Thanks,
> >> > > > >> > > > > > > >> > Dong
> >> > > > >> > > > > > > >> >
> >> > > > >> > > > > > > >> > On Thu, Jan 4, 2018 at 4:02 PM, Jun Rao <
> >> > > > >> jun@confluent.io>
> >> > > > >> > > > wrote:
> >> > > > >> > > > > > > >> >
> >> > > > >> > > > > > > >> > > Hi, Dong,
> >> > > > >> > > > > > > >> > >
> >> > > > >> > > > > > > >> > > Got it. The api that you proposed works. The
> >> > > question
> >> > > > >> is
> >> > > > >> > > > whether
> >> > > > >> > > > > > > >> that's
> >> > > > >> > > > > > > >> > the
> >> > > > >> > > > > > > >> > > api that we want to have in the long term.
> My
> >> > > concern
> >> > > > >> is
> >> > > > >> > > that
> >> > > > >> > > > > > while
> >> > > > >> > > > > > > >> the
> >> > > > >> > > > > > > >> > api
> >> > > > >> > > > > > > >> > > change is simple, the new api seems harder
> to
> >> > > explain
> >> > > > >> and
> >> > > > >> > > use.
> >> > > > >> > > > > For
> >> > > > >> > > > > > > >> > example,
> >> > > > >> > > > > > > >> > > a consumer storing offsets externally now
> >> needs
> >> > to
> >> > > > call
> >> > > > >> > > > > > > >> > > waitForMetadataUpdate() after calling
> seek().
> >> > > > >> > > > > > > >> > >
> >> > > > >> > > > > > > >> > > An alternative approach is to make the
> >> following
> >> > > > >> > compatible
> >> > > > >> > > > api
> >> > > > >> > > > > > > >> changes
> >> > > > >> > > > > > > >> > in
> >> > > > >> > > > > > > >> > > Consumer.
> >> > > > >> > > > > > > >> > > * Add an additional OffsetEpoch field in
> >> > > > >> > OffsetAndMetadata.
> >> > > > >> > > > (no
> >> > > > >> > > > > > need
> >> > > > >> > > > > > > >> to
> >> > > > >> > > > > > > >> > > change the CommitSync() api)
> >> > > > >> > > > > > > >> > > * Add a new api seek(TopicPartition
> partition,
> >> > long
> >> > > > >> > offset,
> >> > > > >> > > > > > > >> OffsetEpoch
> >> > > > >> > > > > > > >> > > offsetEpoch). We can potentially deprecate
> the
> >> > old
> >> > > > api
> >> > > > >> > > > > > > >> > seek(TopicPartition
> >> > > > >> > > > > > > >> > > partition, long offset) in the future.
> >> > > > >> > > > > > > >> > >
> >> > > > >> > > > > > > >> > > The alternative approach has similar amount
> of
> >> > api
> >> > > > >> changes
> >> > > > >> > > as
> >> > > > >> > > > > > yours
> >> > > > >> > > > > > > >> but
> >> > > > >> > > > > > > >> > has
> >> > > > >> > > > > > > >> > > the following benefits.
> >> > > > >> > > > > > > >> > > 1. The api works in a similar way as how
> >> offset
> >> > > > >> management
> >> > > > >> > > > works
> >> > > > >> > > > > > now
> >> > > > >> > > > > > > >> and
> >> > > > >> > > > > > > >> > is
> >> > > > >> > > > > > > >> > > probably what we want in the long term.
> >> > > > >> > > > > > > >> > > 2. It can reset offsets better when there is
> >> data
> >> > > > loss
> >> > > > >> due
> >> > > > >> > > to
> >> > > > >> > > > > > > unclean
> >> > > > >> > > > > > > >> > > leader election or correlated replica
> failure.
> >> > > > >> > > > > > > >> > > 3. It can reset offsets better when topic is
> >> > > > recreated.
> >> > > > >> > > > > > > >> > >
> >> > > > >> > > > > > > >> > > Thanks,
> >> > > > >> > > > > > > >> > >
> >> > > > >> > > > > > > >> > > Jun
> >> > > > >> > > > > > > >> > >
> >> > > > >> > > > > > > >> > >
> >> > > > >> > > > > > > >> > > On Thu, Jan 4, 2018 at 2:05 PM, Dong Lin <
> >> > > > >> > > lindong28@gmail.com
> >> > > > >> > > > >
> >> > > > >> > > > > > > wrote:
> >> > > > >> > > > > > > >> > >
> >> > > > >> > > > > > > >> > > > Hey Jun,
> >> > > > >> > > > > > > >> > > >
> >> > > > >> > > > > > > >> > > > Yeah I agree that ideally we don't want an
> >> ever
> >> > > > >> growing
> >> > > > >> > > > global
> >> > > > >> > > > > > > >> metadata
> >> > > > >> > > > > > > >> > > > version. I just think it may be more
> >> desirable
> >> > to
> >> > > > >> keep
> >> > > > >> > the
> >> > > > >> > > > > > > consumer
> >> > > > >> > > > > > > >> API
> >> > > > >> > > > > > > >> > > > simple.
> >> > > > >> > > > > > > >> > > >
> >> > > > >> > > > > > > >> > > > In my current proposal, metadata version
> >> > returned
> >> > > > in
> >> > > > >> the
> >> > > > >> > > > fetch
> >> > > > >> > > > > > > >> response
> >> > > > >> > > > > > > >> > > > will be stored with the offset together.
> >> More
> >> > > > >> > > specifically,
> >> > > > >> > > > > the
> >> > > > >> > > > > > > >> > > > metadata_epoch in the new offset topic
> >> schema
> >> > > will
> >> > > > be
> >> > > > >> > the
> >> > > > >> > > > > > largest
> >> > > > >> > > > > > > >> > > > metadata_epoch from all the
> MetadataResponse
> >> > and
> >> > > > >> > > > FetchResponse
> >> > > > >> > > > > > > ever
> >> > > > >> > > > > > > >> > > > received by this consumer.
> >> > > > >> > > > > > > >> > > >
> >> > > > >> > > > > > > >> > > > We probably don't have to change the
> >> consumer
> >> > API
> >> > > > for
> >> > > > >> > > > > > > >> > > > commitSync(Map<TopicPartition,
> >> > > OffsetAndMetadata>).
> >> > > > >> If
> >> > > > >> > > user
> >> > > > >> > > > > > calls
> >> > > > >> > > > > > > >> > > > commitSync(...) to commit offset 10 for a
> >> given
> >> > > > >> > partition,
> >> > > > >> > > > for
> >> > > > >> > > > > > > most
> >> > > > >> > > > > > > >> > > > use-cases, this consumer instance should
> >> have
> >> > > > >> consumed
> >> > > > >> > > > message
> >> > > > >> > > > > > > with
> >> > > > >> > > > > > > >> > > offset
> >> > > > >> > > > > > > >> > > > 9 from this partition, in which case the
> >> > consumer
> >> > > > can
> >> > > > >> > > > remember
> >> > > > >> > > > > > and
> >> > > > >> > > > > > > >> use
> >> > > > >> > > > > > > >> > > the
> >> > > > >> > > > > > > >> > > > metadata_epoch from the corresponding
> >> > > FetchResponse
> >> > > > >> when
> >> > > > >> > > > > > > committing
> >> > > > >> > > > > > > >> > > offset.
> >> > > > >> > > > > > > >> > > > If user calls commitSync(..) to commit
> >> offset
> >> > 10
> >> > > > for
> >> > > > >> a
> >> > > > >> > > given
> >> > > > >> > > > > > > >> partition
> >> > > > >> > > > > > > >> > > > without having consumed the message with
> >> > offset 9
> >> > > > >> using
> >> > > > >> > > this
> >> > > > >> > > > > > > >> consumer
> >> > > > >> > > > > > > >> > > > instance, this is probably an advanced
> >> > use-case.
> >> > > In
> >> > > > >> this
> >> > > > >> > > > case
> >> > > > >> > > > > > the
> >> > > > >> > > > > > > >> > > advanced
> >> > > > >> > > > > > > >> > > > user can retrieve the metadata_epoch using
> >> the
> >> > > > newly
> >> > > > >> > added
> >> > > > >> > > > > > > >> > > metadataEpoch()
> >> > > > >> > > > > > > >> > > > API after it fetches the message with
> >> offset 9
> >> > > > >> (probably
> >> > > > >> > > > from
> >> > > > >> > > > > > > >> another
> >> > > > >> > > > > > > >> > > > consumer instance) and encode this
> >> > metadata_epoch
> >> > > > in
> >> > > > >> the
> >> > > > >> > > > > > > >> > > > string OffsetAndMetadata.metadata. Do you
> >> think
> >> > > > this
> >> > > > >> > > > solution
> >> > > > >> > > > > > > would
> >> > > > >> > > > > > > >> > work?
> >> > > > >> > > > > > > >> > > >
> >> > > > >> > > > > > > >> > > > By "not sure that I fully understand your
> >> > latest
> >> > > > >> > > > suggestion",
> >> > > > >> > > > > > are
> >> > > > >> > > > > > > >> you
> >> > > > >> > > > > > > >> > > > referring to solution related to unclean
> >> leader
> >> > > > >> election
> >> > > > >> > > > using
> >> > > > >> > > > > > > >> > > leader_epoch
> >> > > > >> > > > > > > >> > > > in my previous email?
> >> > > > >> > > > > > > >> > > >
> >> > > > >> > > > > > > >> > > > Thanks,
> >> > > > >> > > > > > > >> > > > Dong
> >> > > > >> > > > > > > >> > > >
> >> > > > >> > > > > > > >> > > > On Thu, Jan 4, 2018 at 1:33 PM, Jun Rao <
> >> > > > >> > jun@confluent.io
> >> > > > >> > > >
> >> > > > >> > > > > > wrote:
> >> > > > >> > > > > > > >> > > >
> >> > > > >> > > > > > > >> > > > > Hi, Dong,
> >> > > > >> > > > > > > >> > > > >
> >> > > > >> > > > > > > >> > > > > Not sure that I fully understand your
> >> latest
> >> > > > >> > suggestion.
> >> > > > >> > > > > > > >> Returning an
> >> > > > >> > > > > > > >> > > > ever
> >> > > > >> > > > > > > >> > > > > growing global metadata version itself
> is
> >> no
> >> > > > ideal,
> >> > > > >> > but
> >> > > > >> > > is
> >> > > > >> > > > > > fine.
> >> > > > >> > > > > > > >> My
> >> > > > >> > > > > > > >> > > > > question is whether the metadata version
> >> > > returned
> >> > > > >> in
> >> > > > >> > the
> >> > > > >> > > > > fetch
> >> > > > >> > > > > > > >> > response
> >> > > > >> > > > > > > >> > > > > needs to be stored with the offset
> >> together
> >> > if
> >> > > > >> offsets
> >> > > > >> > > are
> >> > > > >> > > > > > > stored
> >> > > > >> > > > > > > >> > > > > externally. If so, we also have to
> change
> >> the
> >> > > > >> consumer
> >> > > > >> > > API
> >> > > > >> > > > > for
> >> > > > >> > > > > > > >> > > > commitSync()
> >> > > > >> > > > > > > >> > > > > and need to worry about compatibility.
> If
> >> we
> >> > > > don't
> >> > > > >> > store
> >> > > > >> > > > the
> >> > > > >> > > > > > > >> metadata
> >> > > > >> > > > > > > >> > > > > version together with the offset, on a
> >> > consumer
> >> > > > >> > restart,
> >> > > > >> > > > > it's
> >> > > > >> > > > > > > not
> >> > > > >> > > > > > > >> > clear
> >> > > > >> > > > > > > >> > > > how
> >> > > > >> > > > > > > >> > > > > we can ensure the metadata in the
> >> consumer is
> >> > > > high
> >> > > > >> > > enough
> >> > > > >> > > > > > since
> >> > > > >> > > > > > > >> there
> >> > > > >> > > > > > > >> > > is
> >> > > > >> > > > > > > >> > > > no
> >> > > > >> > > > > > > >> > > > > metadata version to compare with.
> >> > > > >> > > > > > > >> > > > >
> >> > > > >> > > > > > > >> > > > > Thanks,
> >> > > > >> > > > > > > >> > > > >
> >> > > > >> > > > > > > >> > > > > Jun
> >> > > > >> > > > > > > >> > > > >
> >> > > > >> > > > > > > >> > > > >
> >> > > > >> > > > > > > >> > > > > On Wed, Jan 3, 2018 at 6:43 PM, Dong
> Lin <
> >> > > > >> > > > > lindong28@gmail.com
> >> > > > >> > > > > > >
> >> > > > >> > > > > > > >> > wrote:
> >> > > > >> > > > > > > >> > > > >
> >> > > > >> > > > > > > >> > > > > > Hey Jun,
> >> > > > >> > > > > > > >> > > > > >
> >> > > > >> > > > > > > >> > > > > > Thanks much for the explanation.
> >> > > > >> > > > > > > >> > > > > >
> >> > > > >> > > > > > > >> > > > > > I understand the advantage of
> >> > partition_epoch
> >> > > > >> over
> >> > > > >> > > > > > > >> metadata_epoch.
> >> > > > >> > > > > > > >> > My
> >> > > > >> > > > > > > >> > > > > > current concern is that the use of
> >> > > leader_epoch
> >> > > > >> and
> >> > > > >> > > the
> >> > > > >> > > > > > > >> > > partition_epoch
> >> > > > >> > > > > > > >> > > > > > requires us considerable change to
> >> > consumer's
> >> > > > >> public
> >> > > > >> > > API
> >> > > > >> > > > > to
> >> > > > >> > > > > > > take
> >> > > > >> > > > > > > >> > care
> >> > > > >> > > > > > > >> > > > of
> >> > > > >> > > > > > > >> > > > > > the case where user stores offset
> >> > externally.
> >> > > > For
> >> > > > >> > > > example,
> >> > > > >> > > > > > > >> > > *consumer*.
> >> > > > >> > > > > > > >> > > > > > *commitSync*(..) would have to take a
> >> map
> >> > > whose
> >> > > > >> > value
> >> > > > >> > > is
> >> > > > >> > > > > > > >> <offset,
> >> > > > >> > > > > > > >> > > > > metadata,
> >> > > > >> > > > > > > >> > > > > > leader epoch, partition epoch>.
> >> > > > >> > *consumer*.*seek*(...)
> >> > > > >> > > > > would
> >> > > > >> > > > > > > >> also
> >> > > > >> > > > > > > >> > > need
> >> > > > >> > > > > > > >> > > > > > leader_epoch and partition_epoch as
> >> > > parameter.
> >> > > > >> > > > Technically
> >> > > > >> > > > > > we
> >> > > > >> > > > > > > >> can
> >> > > > >> > > > > > > >> > > > > probably
> >> > > > >> > > > > > > >> > > > > > still make it work in a backward
> >> compatible
> >> > > > >> manner
> >> > > > >> > > after
> >> > > > >> > > > > > > careful
> >> > > > >> > > > > > > >> > > design
> >> > > > >> > > > > > > >> > > > > and
> >> > > > >> > > > > > > >> > > > > > discussion. But these changes can make
> >> the
> >> > > > >> > consumer's
> >> > > > >> > > > > > > interface
> >> > > > >> > > > > > > >> > > > > > unnecessarily complex for more users
> >> who do
> >> > > not
> >> > > > >> > store
> >> > > > >> > > > > offset
> >> > > > >> > > > > > > >> > > > externally.
> >> > > > >> > > > > > > >> > > > > >
> >> > > > >> > > > > > > >> > > > > > After thinking more about it, we can
> >> > address
> >> > > > all
> >> > > > >> > > > problems
> >> > > > >> > > > > > > >> discussed
> >> > > > >> > > > > > > >> > > by
> >> > > > >> > > > > > > >> > > > > only
> >> > > > >> > > > > > > >> > > > > > using the metadata_epoch without
> >> > introducing
> >> > > > >> > > > leader_epoch
> >> > > > >> > > > > or
> >> > > > >> > > > > > > the
> >> > > > >> > > > > > > >> > > > > > partition_epoch. The current KIP
> >> describes
> >> > > the
> >> > > > >> > changes
> >> > > > >> > > > to
> >> > > > >> > > > > > the
> >> > > > >> > > > > > > >> > > consumer
> >> > > > >> > > > > > > >> > > > > API
> >> > > > >> > > > > > > >> > > > > > and how the new API can be used if
> user
> >> > > stores
> >> > > > >> > offset
> >> > > > >> > > > > > > >> externally.
> >> > > > >> > > > > > > >> > In
> >> > > > >> > > > > > > >> > > > > order
> >> > > > >> > > > > > > >> > > > > > to address the scenario you described
> >> > > earlier,
> >> > > > we
> >> > > > >> > can
> >> > > > >> > > > > > include
> >> > > > >> > > > > > > >> > > > > > metadata_epoch in the FetchResponse
> and
> >> the
> >> > > > >> > > > > > > LeaderAndIsrRequest.
> >> > > > >> > > > > > > >> > > > Consumer
> >> > > > >> > > > > > > >> > > > > > remembers the largest metadata_epoch
> >> from
> >> > all
> >> > > > the
> >> > > > >> > > > > > > FetchResponse
> >> > > > >> > > > > > > >> it
> >> > > > >> > > > > > > >> > > has
> >> > > > >> > > > > > > >> > > > > > received. The metadata_epoch committed
> >> with
> >> > > the
> >> > > > >> > > offset,
> >> > > > >> > > > > > either
> >> > > > >> > > > > > > >> > within
> >> > > > >> > > > > > > >> > > > or
> >> > > > >> > > > > > > >> > > > > > outside Kafka, should be the largest
> >> > > > >> metadata_epoch
> >> > > > >> > > > across
> >> > > > >> > > > > > all
> >> > > > >> > > > > > > >> > > > > > FetchResponse and MetadataResponse
> ever
> >> > > > received
> >> > > > >> by
> >> > > > >> > > this
> >> > > > >> > > > > > > >> consumer.
> >> > > > >> > > > > > > >> > > > > >
> >> > > > >> > > > > > > >> > > > > > The drawback of using only the
> >> > metadata_epoch
> >> > > > is
> >> > > > >> > that
> >> > > > >> > > we
> >> > > > >> > > > > can
> >> > > > >> > > > > > > not
> >> > > > >> > > > > > > >> > > always
> >> > > > >> > > > > > > >> > > > > do
> >> > > > >> > > > > > > >> > > > > > the smart offset reset in case of
> >> unclean
> >> > > > leader
> >> > > > >> > > > election
> >> > > > >> > > > > > > which
> >> > > > >> > > > > > > >> you
> >> > > > >> > > > > > > >> > > > > > mentioned earlier. But in most case,
> >> > unclean
> >> > > > >> leader
> >> > > > >> > > > > election
> >> > > > >> > > > > > > >> > probably
> >> > > > >> > > > > > > >> > > > > > happens when consumer is not
> >> > > > >> rebalancing/restarting.
> >> > > > >> > > In
> >> > > > >> > > > > > these
> >> > > > >> > > > > > > >> > cases,
> >> > > > >> > > > > > > >> > > > > either
> >> > > > >> > > > > > > >> > > > > > consumer is not directly affected by
> >> > unclean
> >> > > > >> leader
> >> > > > >> > > > > election
> >> > > > >> > > > > > > >> since
> >> > > > >> > > > > > > >> > it
> >> > > > >> > > > > > > >> > > > is
> >> > > > >> > > > > > > >> > > > > > not consuming from the end of the log,
> >> or
> >> > > > >> consumer
> >> > > > >> > can
> >> > > > >> > > > > > derive
> >> > > > >> > > > > > > >> the
> >> > > > >> > > > > > > >> > > > > > leader_epoch from the most recent
> >> message
> >> > > > >> received
> >> > > > >> > > > before
> >> > > > >> > > > > it
> >> > > > >> > > > > > > >> sees
> >> > > > >> > > > > > > >> > > > > > OffsetOutOfRangeException. So I am not
> >> sure
> >> > > it
> >> > > > is
> >> > > > >> > > worth
> >> > > > >> > > > > > adding
> >> > > > >> > > > > > > >> the
> >> > > > >> > > > > > > >> > > > > > leader_epoch to consumer API to
> address
> >> the
> >> > > > >> > remaining
> >> > > > >> > > > > corner
> >> > > > >> > > > > > > >> case.
> >> > > > >> > > > > > > >> > > What
> >> > > > >> > > > > > > >> > > > > do
> >> > > > >> > > > > > > >> > > > > > you think?
> >> > > > >> > > > > > > >> > > > > >
> >> > > > >> > > > > > > >> > > > > > Thanks,
> >> > > > >> > > > > > > >> > > > > > Dong
> >> > > > >> > > > > > > >> > > > > >
> >> > > > >> > > > > > > >> > > > > >
> >> > > > >> > > > > > > >> > > > > >
> >> > > > >> > > > > > > >> > > > > > On Tue, Jan 2, 2018 at 6:28 PM, Jun
> Rao
> >> <
> >> > > > >> > > > jun@confluent.io
> >> > > > >> > > > > >
> >> > > > >> > > > > > > >> wrote:
> >> > > > >> > > > > > > >> > > > > >
> >> > > > >> > > > > > > >> > > > > > > Hi, Dong,
> >> > > > >> > > > > > > >> > > > > > >
> >> > > > >> > > > > > > >> > > > > > > Thanks for the reply.
> >> > > > >> > > > > > > >> > > > > > >
> >> > > > >> > > > > > > >> > > > > > > To solve the topic recreation issue,
> >> we
> >> > > could
> >> > > > >> use
> >> > > > >> > > > > either a
> >> > > > >> > > > > > > >> global
> >> > > > >> > > > > > > >> > > > > > metadata
> >> > > > >> > > > > > > >> > > > > > > version or a partition level epoch.
> >> But
> >> > > > either
> >> > > > >> one
> >> > > > >> > > > will
> >> > > > >> > > > > > be a
> >> > > > >> > > > > > > >> new
> >> > > > >> > > > > > > >> > > > > concept,
> >> > > > >> > > > > > > >> > > > > > > right? To me, the latter seems more
> >> > > natural.
> >> > > > It
> >> > > > >> > also
> >> > > > >> > > > > makes
> >> > > > >> > > > > > > it
> >> > > > >> > > > > > > >> > > easier
> >> > > > >> > > > > > > >> > > > to
> >> > > > >> > > > > > > >> > > > > > > detect if a consumer's offset is
> still
> >> > > valid
> >> > > > >> > after a
> >> > > > >> > > > > topic
> >> > > > >> > > > > > > is
> >> > > > >> > > > > > > >> > > > > recreated.
> >> > > > >> > > > > > > >> > > > > > As
> >> > > > >> > > > > > > >> > > > > > > you pointed out, we don't need to
> >> store
> >> > the
> >> > > > >> > > partition
> >> > > > >> > > > > > epoch
> >> > > > >> > > > > > > in
> >> > > > >> > > > > > > >> > the
> >> > > > >> > > > > > > >> > > > > > message.
> >> > > > >> > > > > > > >> > > > > > > The following is what I am thinking.
> >> > When a
> >> > > > >> > > partition
> >> > > > >> > > > is
> >> > > > >> > > > > > > >> created,
> >> > > > >> > > > > > > >> > > we
> >> > > > >> > > > > > > >> > > > > can
> >> > > > >> > > > > > > >> > > > > > > assign a partition epoch from an
> >> > > > >> ever-increasing
> >> > > > >> > > > global
> >> > > > >> > > > > > > >> counter
> >> > > > >> > > > > > > >> > and
> >> > > > >> > > > > > > >> > > > > store
> >> > > > >> > > > > > > >> > > > > > > it in /brokers/topics/[topic]/
> >> > > > >> > > > partitions/[partitionId]
> >> > > > >> > > > > in
> >> > > > >> > > > > > > ZK.
> >> > > > >> > > > > > > >> > The
> >> > > > >> > > > > > > >> > > > > > > partition
> >> > > > >> > > > > > > >> > > > > > > epoch is propagated to every broker.
> >> The
> >> > > > >> consumer
> >> > > > >> > > will
> >> > > > >> > > > > be
> >> > > > >> > > > > > > >> > tracking
> >> > > > >> > > > > > > >> > > a
> >> > > > >> > > > > > > >> > > > > > tuple
> >> > > > >> > > > > > > >> > > > > > > of <offset, leader epoch, partition
> >> > epoch>
> >> > > > for
> >> > > > >> > > > offsets.
> >> > > > >> > > > > > If a
> >> > > > >> > > > > > > >> > topic
> >> > > > >> > > > > > > >> > > is
> >> > > > >> > > > > > > >> > > > > > > recreated, it's possible that a
> >> > consumer's
> >> > > > >> offset
> >> > > > >> > > and
> >> > > > >> > > > > > leader
> >> > > > >> > > > > > > >> > epoch
> >> > > > >> > > > > > > >> > > > > still
> >> > > > >> > > > > > > >> > > > > > > match that in the broker, but
> >> partition
> >> > > epoch
> >> > > > >> > won't
> >> > > > >> > > > be.
> >> > > > >> > > > > In
> >> > > > >> > > > > > > >> this
> >> > > > >> > > > > > > >> > > case,
> >> > > > >> > > > > > > >> > > > > we
> >> > > > >> > > > > > > >> > > > > > > can potentially still treat the
> >> > consumer's
> >> > > > >> offset
> >> > > > >> > as
> >> > > > >> > > > out
> >> > > > >> > > > > > of
> >> > > > >> > > > > > > >> range
> >> > > > >> > > > > > > >> > > and
> >> > > > >> > > > > > > >> > > > > > reset
> >> > > > >> > > > > > > >> > > > > > > the offset based on the offset reset
> >> > policy
> >> > > > in
> >> > > > >> the
> >> > > > >> > > > > > consumer.
> >> > > > >> > > > > > > >> This
> >> > > > >> > > > > > > >> > > > seems
> >> > > > >> > > > > > > >> > > > > > > harder to do with a global metadata
> >> > > version.
> >> > > > >> > > > > > > >> > > > > > >
> >> > > > >> > > > > > > >> > > > > > > Jun
> >> > > > >> > > > > > > >> > > > > > >
> >> > > > >> > > > > > > >> > > > > > >
> >> > > > >> > > > > > > >> > > > > > >
> >> > > > >> > > > > > > >> > > > > > > On Mon, Dec 25, 2017 at 6:56 AM,
> Dong
> >> > Lin <
> >> > > > >> > > > > > > >> lindong28@gmail.com>
> >> > > > >> > > > > > > >> > > > wrote:
> >> > > > >> > > > > > > >> > > > > > >
> >> > > > >> > > > > > > >> > > > > > > > Hey Jun,
> >> > > > >> > > > > > > >> > > > > > > >
> >> > > > >> > > > > > > >> > > > > > > > This is a very good example. After
> >> > > thinking
> >> > > > >> > > through
> >> > > > >> > > > > this
> >> > > > >> > > > > > > in
> >> > > > >> > > > > > > >> > > > detail, I
> >> > > > >> > > > > > > >> > > > > > > agree
> >> > > > >> > > > > > > >> > > > > > > > that we need to commit offset with
> >> > leader
> >> > > > >> epoch
> >> > > > >> > in
> >> > > > >> > > > > order
> >> > > > >> > > > > > > to
> >> > > > >> > > > > > > >> > > address
> >> > > > >> > > > > > > >> > > > > > this
> >> > > > >> > > > > > > >> > > > > > > > example.
> >> > > > >> > > > > > > >> > > > > > > >
> >> > > > >> > > > > > > >> > > > > > > > I think the remaining question is
> >> how
> >> > to
> >> > > > >> address
> >> > > > >> > > the
> >> > > > >> > > > > > > >> scenario
> >> > > > >> > > > > > > >> > > that
> >> > > > >> > > > > > > >> > > > > the
> >> > > > >> > > > > > > >> > > > > > > > topic is deleted and re-created.
> One
> >> > > > possible
> >> > > > >> > > > solution
> >> > > > >> > > > > > is
> >> > > > >> > > > > > > to
> >> > > > >> > > > > > > >> > > commit
> >> > > > >> > > > > > > >> > > > > > > offset
> >> > > > >> > > > > > > >> > > > > > > > with both the leader epoch and the
> >> > > metadata
> >> > > > >> > > version.
> >> > > > >> > > > > The
> >> > > > >> > > > > > > >> logic
> >> > > > >> > > > > > > >> > > and
> >> > > > >> > > > > > > >> > > > > the
> >> > > > >> > > > > > > >> > > > > > > > implementation of this solution
> does
> >> > not
> >> > > > >> > require a
> >> > > > >> > > > new
> >> > > > >> > > > > > > >> concept
> >> > > > >> > > > > > > >> > > > (e.g.
> >> > > > >> > > > > > > >> > > > > > > > partition epoch) and it does not
> >> > require
> >> > > > any
> >> > > > >> > > change
> >> > > > >> > > > to
> >> > > > >> > > > > > the
> >> > > > >> > > > > > > >> > > message
> >> > > > >> > > > > > > >> > > > > > format
> >> > > > >> > > > > > > >> > > > > > > > or leader epoch. It also allows us
> >> to
> >> > > order
> >> > > > >> the
> >> > > > >> > > > > metadata
> >> > > > >> > > > > > > in
> >> > > > >> > > > > > > >> a
> >> > > > >> > > > > > > >> > > > > > > > straightforward manner which may
> be
> >> > > useful
> >> > > > in
> >> > > > >> > the
> >> > > > >> > > > > > future.
> >> > > > >> > > > > > > >> So it
> >> > > > >> > > > > > > >> > > may
> >> > > > >> > > > > > > >> > > > > be
> >> > > > >> > > > > > > >> > > > > > a
> >> > > > >> > > > > > > >> > > > > > > > better solution than generating a
> >> > random
> >> > > > >> > partition
> >> > > > >> > > > > epoch
> >> > > > >> > > > > > > >> every
> >> > > > >> > > > > > > >> > > time
> >> > > > >> > > > > > > >> > > > > we
> >> > > > >> > > > > > > >> > > > > > > > create a partition. Does this
> sound
> >> > > > >> reasonable?
> >> > > > >> > > > > > > >> > > > > > > >
> >> > > > >> > > > > > > >> > > > > > > > Previously one concern with using
> >> the
> >> > > > >> metadata
> >> > > > >> > > > version
> >> > > > >> > > > > > is
> >> > > > >> > > > > > > >> that
> >> > > > >> > > > > > > >> > > > > consumer
> >> > > > >> > > > > > > >> > > > > > > > will be forced to refresh metadata
> >> even
> >> > > if
> >> > > > >> > > metadata
> >> > > > >> > > > > > > version
> >> > > > >> > > > > > > >> is
> >> > > > >> > > > > > > >> > > > > > increased
> >> > > > >> > > > > > > >> > > > > > > > due to topics that the consumer is
> >> not
> >> > > > >> > interested
> >> > > > >> > > > in.
> >> > > > >> > > > > > Now
> >> > > > >> > > > > > > I
> >> > > > >> > > > > > > >> > > > realized
> >> > > > >> > > > > > > >> > > > > > that
> >> > > > >> > > > > > > >> > > > > > > > this is probably not a problem.
> >> > Currently
> >> > > > >> client
> >> > > > >> > > > will
> >> > > > >> > > > > > > >> refresh
> >> > > > >> > > > > > > >> > > > > metadata
> >> > > > >> > > > > > > >> > > > > > > > either due to
> >> InvalidMetadataException
> >> > in
> >> > > > the
> >> > > > >> > > > response
> >> > > > >> > > > > > > from
> >> > > > >> > > > > > > >> > > broker
> >> > > > >> > > > > > > >> > > > or
> >> > > > >> > > > > > > >> > > > > > due
> >> > > > >> > > > > > > >> > > > > > > > to metadata expiry. The addition
> of
> >> the
> >> > > > >> metadata
> >> > > > >> > > > > version
> >> > > > >> > > > > > > >> should
> >> > > > >> > > > > > > >> > > > > > increase
> >> > > > >> > > > > > > >> > > > > > > > the overhead of metadata refresh
> >> caused
> >> > > by
> >> > > > >> > > > > > > >> > > > InvalidMetadataException.
> >> > > > >> > > > > > > >> > > > > If
> >> > > > >> > > > > > > >> > > > > > > > client refresh metadata due to
> >> expiry
> >> > and
> >> > > > it
> >> > > > >> > > > receives
> >> > > > >> > > > > a
> >> > > > >> > > > > > > >> > metadata
> >> > > > >> > > > > > > >> > > > > whose
> >> > > > >> > > > > > > >> > > > > > > > version is lower than the current
> >> > > metadata
> >> > > > >> > > version,
> >> > > > >> > > > we
> >> > > > >> > > > > > can
> >> > > > >> > > > > > > >> > reject
> >> > > > >> > > > > > > >> > > > the
> >> > > > >> > > > > > > >> > > > > > > > metadata but still reset the
> >> metadata
> >> > > age,
> >> > > > >> which
> >> > > > >> > > > > > > essentially
> >> > > > >> > > > > > > >> > keep
> >> > > > >> > > > > > > >> > > > the
> >> > > > >> > > > > > > >> > > > > > > > existing behavior in the client.
> >> > > > >> > > > > > > >> > > > > > > >
> >> > > > >> > > > > > > >> > > > > > > > Thanks much,
> >> > > > >> > > > > > > >> > > > > > > > Dong
> >> > > > >> > > > > > > >> > > > > > > >
> >> > > > >> > > > > > > >> > > > > > >
> >> > > > >> > > > > > > >> > > > > >
> >> > > > >> > > > > > > >> > > > >
> >> > > > >> > > > > > > >> > > >
> >> > > > >> > > > > > > >> > >
> >> > > > >> > > > > > > >> >
> >> > > > >> > > > > > > >>
> >> > > > >> > > > > > > >
> >> > > > >> > > > > > > >
> >> > > > >> > > > > > >
> >> > > > >> > > > > >
> >> > > > >> > > > >
> >> > > > >> > > >
> >> > > > >> > >
> >> > > > >> >
> >> > > > >>
> >> > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Dong Lin <li...@gmail.com>.
Thinking about point 61 more, I realize that the async zookeeper read may
make it less of an issue for controller to read more zookeeper nodes.
Writing partition_epoch in the per-partition znode makes it simpler to
handle the broker failure between zookeeper writes for a topic creation. I
have updated the KIP to use the suggested approach.


On Wed, Jan 17, 2018 at 9:57 AM, Dong Lin <li...@gmail.com> wrote:

> Hey Jun,
>
> Thanks much for the comments. Please see my comments inline.
>
> On Tue, Jan 16, 2018 at 4:38 PM, Jun Rao <ju...@confluent.io> wrote:
>
>> Hi, Dong,
>>
>> Thanks for the updated KIP. Looks good to me overall. Just a few minor
>> comments.
>>
>> 60. OffsetAndMetadata positionAndOffsetEpoch(TopicPartition partition):
>> It
>> seems that there is no need to return metadata. We probably want to return
>> sth like OffsetAndEpoch.
>>
>
> Previously I think we may want to re-use the existing class to keep our
> consumer interface simpler. I have updated the KIP to add class
> OffsetAndOffsetEpoch. I didn't use OffsetAndEpoch because user may confuse
> this name with OffsetEpoch. Does this sound OK?
>
>
>>
>> 61. Should we store partition_epoch in
>> /brokers/topics/[topic]/partitions/[partitionId] in ZK?
>>
>
> I have considered this. I think the advantage of adding the
> partition->partition_epoch map in the existing
> znode /brokers/topics/[topic]/partitions is that controller only needs to
> read one znode per topic to gets its partition_epoch information. Otherwise
> controller may need to read one extra znode per partition to get the same
> information.
>
> When we delete partition or expand partition of a topic, someone needs to
> modify partition->partition_epoch map in znode
> /brokers/topics/[topic]/partitions. This may seem a bit more complicated
> than simply adding or deleting znode /brokers/topics/[topic]/partitions/[partitionId].
> But the complexity is probably similar to the existing operation of
> modifying the partition->replica_list mapping in znode
> /brokers/topics/[topic]. So not sure it is better to store the
> partition_epoch in /brokers/topics/[topic]/partitions/[partitionId]. What
> do you think?
>
>
>>
>> 62. For checking outdated metadata in the client, we probably want to add
>> when max_partition_epoch will be used.
>>
>
> The max_partition_epoch is used in the Proposed Changes -> Client's
> metadata refresh section to determine whether a metadata is outdated. And
> this formula is referenced and re-used in other sections to determine
> whether a metadata is outdated. Does this formula look OK?
>
>
>>
>> 63. "The leader_epoch should be the largest leader_epoch of messages whose
>> offset < the commit offset. If no message has been consumed since consumer
>> initialization, the leader_epoch from seek(...) or OffsetFetchResponse
>> should be used. The partition_epoch should be read from the last
>> FetchResponse corresponding to the given partition and commit offset. ":
>> leader_epoch and partition_epoch are associated with an offset. So, if no
>> message is consumed, there is no offset and therefore there is no need to
>> read leader_epoch and partition_epoch. Also, the leader_epoch associated
>> with the offset should just come from the messages returned in the fetch
>> response.
>>
>
> I am thinking that, if user calls seek(..) and commitSync(...) without
> consuming any messages, we should re-use the leader_epoch and
> partition_epoch provided by the seek(...) in the OffsetCommitRequest. And
> if messages have been successfully consumed, then leader_epoch will come
> from the messages returned in the fetch response. The condition "messages
> whose offset < the commit offset" is needed to take care of the log
> compacted topic which may have offset gap due to log cleaning.
>
> Did I miss something here? Or should I rephrase the paragraph to make it
> less confusing?
>
>
>> 64. Could you include the public methods in the OffsetEpoch class?
>>
>
> I mistakenly deleted the definition of OffsetEpoch class from the KIP. I
> just added it back with the public methods. Could you take another look?
>
>
>>
>> Jun
>>
>>
>> On Thu, Jan 11, 2018 at 5:43 PM, Dong Lin <li...@gmail.com> wrote:
>>
>> > Hey Jun,
>> >
>> > Thanks much. I agree that we can not rely on committed offsets to be
>> always
>> > deleted when we delete topic. So it is necessary to use a per-partition
>> > epoch that does not change unless this partition is deleted. I also
>> agree
>> > that it is very nice to be able to uniquely identify a message with
>> > (offset, leader_epoch, partition_epoch) in face of potential topic
>> deletion
>> > and unclean leader election.
>> >
>> > I agree with all your comments. And I have updated the KIP based on our
>> > latest discussion. In addition, I added InvalidPartitionEpochException
>> > which will be thrown by consumer.poll() if the partition_epoch
>> associated
>> > with the partition, which can be given to consumer using seek(...), is
>> > different from the partition_epoch in the FetchResponse.
>> >
>> > Can you take another look at the latest KIP?
>> >
>> > Thanks!
>> > Dong
>> >
>> >
>> >
>> > On Wed, Jan 10, 2018 at 2:24 PM, Jun Rao <ju...@confluent.io> wrote:
>> >
>> > > Hi, Dong,
>> > >
>> > > My replies are the following.
>> > >
>> > > 60. What you described could also work. The drawback is that we will
>> be
>> > > unnecessarily changing the partition epoch when a partition hasn't
>> really
>> > > changed. I was imagining that the partition epoch will be stored in
>> > > /brokers/topics/[topic]/partitions/[partitionId], instead of at the
>> > topic
>> > > level. So, not sure if ZK size limit is an issue.
>> > >
>> > > 61, 62 and 65. To me, the offset + offset_epoch is a unique identifier
>> > for
>> > > a message. So, if a message hasn't changed, the offset and the
>> associated
>> > > offset_epoch ideally should remain the same (it will be kind of weird
>> if
>> > > two consumer apps save the offset on the same message, but the
>> > offset_epoch
>> > > are different). partition_epoch + leader_epoch give us that.
>> > global_epoch +
>> > > leader_epoch don't. If we use this approach, we can solve not only the
>> > > problem that you have identified, but also other problems when there
>> is
>> > > data loss or topic re-creation more reliably. For example, in the
>> future,
>> > > if we include the partition_epoch and leader_epoch in the fetch
>> request,
>> > > the server can do a more reliable check of whether that offset is
>> valid
>> > or
>> > > not. I am not sure that we can rely upon all external offsets to be
>> > removed
>> > > on topic deletion. For example, a topic may be deleted by an admin who
>> > may
>> > > not know all the applications.
>> > >
>> > > If we agree on the above, the second question is then how to reliably
>> > > propagate the partition_epoch and the leader_epoch to the consumer
>> when
>> > > there are leader or partition changes. The leader_epoch comes from the
>> > > message, which is reliable. So, I was suggesting that when we store an
>> > > offset, we can just store the leader_epoch from the message set
>> > containing
>> > > that offset. Similarly, I was thinking that if the partition_epoch is
>> in
>> > > the fetch response, we can propagate partition_epoch reliably where is
>> > > partition_epoch change.
>> > >
>> > > 63. My point is that once a leader is producing a message in the new
>> > > partition_epoch, ideally, we should associate the new offsets with the
>> > new
>> > > partition_epoch. Otherwise, the offset_epoch won't be the correct
>> unique
>> > > identifier (useful for solving other problems mentioned above). I was
>> > > originally thinking that the leader will include the partition_epoch
>> in
>> > the
>> > > metadata cache in the fetch response. It's just that right now,
>> metadata
>> > > cache is updated on UpdateMetadataRequest, which typically happens
>> after
>> > > the LeaderAndIsrRequest. Another approach is for the leader to cache
>> the
>> > > partition_epoch in the Partition object and return that (instead of
>> the
>> > one
>> > > in metadata cache) in the fetch response.
>> > >
>> > > 65. It seems to me that the global_epoch and the partition_epoch have
>> > > different purposes. A partition_epoch has the benefit that it (1) can
>> be
>> > > used to form a unique identifier for a message and (2) can be used to
>> > > solve other
>> > > corner case problems in the future. I am not sure having just a
>> > > global_epoch can achieve these. global_epoch is useful to determine
>> which
>> > > version of the metadata is newer, especially with topic deletion.
>> > >
>> > > Thanks,
>> > >
>> > > Jun
>> > >
>> > > On Tue, Jan 9, 2018 at 11:34 PM, Dong Lin <li...@gmail.com>
>> wrote:
>> > >
>> > > > Regarding the use of the global epoch in 65), it is very similar to
>> the
>> > > > proposal of the metadata_epoch we discussed earlier. The main
>> > difference
>> > > is
>> > > > that this epoch is incremented when we create/expand/delete topic
>> and
>> > > does
>> > > > not change when controller re-send metadata.
>> > > >
>> > > > I looked at our previous discussion. It seems that we prefer
>> > > > partition_epoch over the metadata_epoch because 1) we prefer not to
>> > have
>> > > an
>> > > > ever growing metadata_epoch and 2) we can reset offset better when
>> > topic
>> > > is
>> > > > re-created. The use of global topic_epoch avoids the drawback of an
>> > ever
>> > > > quickly ever growing metadata_epoch. Though the global epoch does
>> not
>> > > allow
>> > > > us to recognize the invalid offset committed before the topic
>> > > re-creation,
>> > > > we can probably just delete the offset when we delete a topic. Thus
>> I
>> > am
>> > > > not very sure whether it is still worthwhile to have a per-partition
>> > > > partition_epoch if the metadata already has the global epoch.
>> > > >
>> > > >
>> > > > On Tue, Jan 9, 2018 at 6:58 PM, Dong Lin <li...@gmail.com>
>> wrote:
>> > > >
>> > > > > Hey Jun,
>> > > > >
>> > > > > Thanks so much. These comments very useful. Please see below my
>> > > comments.
>> > > > >
>> > > > > On Mon, Jan 8, 2018 at 5:52 PM, Jun Rao <ju...@confluent.io> wrote:
>> > > > >
>> > > > >> Hi, Dong,
>> > > > >>
>> > > > >> Thanks for the updated KIP. A few more comments.
>> > > > >>
>> > > > >> 60. Perhaps having a partition epoch is more flexible since in
>> the
>> > > > future,
>> > > > >> we may support deleting a partition as well.
>> > > > >>
>> > > > >
>> > > > > Yeah I have considered this. I think we can probably still support
>> > > > > deleting a partition by using the topic_epoch -- when partition
>> of a
>> > > > topic
>> > > > > is deleted or created, epoch of all partitions of this topic will
>> be
>> > > > > incremented by 1. Therefore, if that partition is re-created
>> later,
>> > the
>> > > > > epoch of that partition will still be larger than its epoch before
>> > the
>> > > > > deletion, which still allows the client to order the metadata for
>> the
>> > > > > purpose of this KIP. Does this sound reasonable?
>> > > > >
>> > > > > The advantage of using topic_epoch instead of partition_epoch is
>> that
>> > > the
>> > > > > size of the /brokers/topics/[topic] znode and request/response
>> size
>> > can
>> > > > be
>> > > > > smaller. We have a limit on the maximum size of znode (typically
>> > 1MB).
>> > > > Use
>> > > > > partition epoch can effectively reduce the number of partitions
>> that
>> > > can
>> > > > be
>> > > > > described by the /brokers/topics/[topic] znode.
>> > > > >
>> > > > > One use-case of partition_epoch for client to detect that the
>> > committed
>> > > > > offset, either from kafka offset topic or from the external store
>> is
>> > > > > invalid after partition deletion and re-creation. However, it
>> seems
>> > > that
>> > > > we
>> > > > > can also address this use-case with other approaches. For example,
>> > when
>> > > > > AdminClient deletes partitions, it can also delete the committed
>> > > offsets
>> > > > > for those partitions from the offset topic. If user stores offset
>> > > > > externally, it might make sense for user to similarly remove
>> offsets
>> > of
>> > > > > related partitions after these partitions are deleted. So I am not
>> > sure
>> > > > > that we should use partition_epoch in this KIP.
>> > > > >
>> > > > >
>> > > > >>
>> > > > >> 61. It seems that the leader epoch returned in the position()
>> call
>> > > > should
>> > > > >> the the leader epoch returned in the fetch response, not the one
>> in
>> > > the
>> > > > >> metadata cache of the client.
>> > > > >
>> > > > >
>> > > > > I think this is a good idea. Just to double check, this change
>> does
>> > not
>> > > > > affect the correctness or performance of this KIP. But it can be
>> > useful
>> > > > if
>> > > > > we want to use the leader_epoch to better handle the offset rest
>> in
>> > > case
>> > > > of
>> > > > > unclean leader election, which is listed in the future work. Is
>> this
>> > > > > understanding correct?
>> > > > >
>> > > > > I have updated the KIP to specify that the leader_epoch returned
>> by
>> > > > > position() should be the largest leader_epoch of those already
>> > consumed
>> > > > > messages whose offset < position. If no message has been consumed
>> > since
>> > > > > consumer initialization, the leader_epoch from seek() or
>> > > > > OffsetFetchResponse should be used. The offset included in the
>> > > > > OffsetCommitRequest will also be determined in the similar manner.
>> > > > >
>> > > > >
>> > > > >>
>> > > > >> 62. I am wondering if we should return the partition epoch in the
>> > > fetch
>> > > > >> response as well. In the current proposal, if a topic is
>> recreated
>> > and
>> > > > the
>> > > > >> new leader is on the same broker as the old one, there is
>> nothing to
>> > > > force
>> > > > >> the metadata refresh in the client. So, the client may still
>> > associate
>> > > > the
>> > > > >> offset with the old partition epoch.
>> > > > >>
>> > > > >
>> > > > > Could you help me understand the problem if a client associates
>> old
>> > > > > partition_epoch (or the topic_epoch as of the current KIP) with
>> the
>> > > > offset?
>> > > > > The main purpose of the topic_epoch is to be able to drop
>> > leader_epoch
>> > > > to 0
>> > > > > after a partition is deleted and re-created. I guess you may be
>> > > thinking
>> > > > > about using the partition_epoch to detect that the committed
>> offset
>> > is
>> > > > > invalid? In that case, I am wondering if the alternative approach
>> > > > described
>> > > > > in 60) would be reasonable.
>> > > > >
>> > > > >
>> > > > >>
>> > > > >> 63. There is some subtle coordination between the
>> > LeaderAndIsrRequest
>> > > > and
>> > > > >> UpdateMetadataRequest. Currently, when a leader changes, the
>> > > controller
>> > > > >> first sends the LeaderAndIsrRequest to the assigned replicas and
>> the
>> > > > >> UpdateMetadataRequest to every broker. So, there could be a small
>> > > window
>> > > > >> when the leader already receives the new partition epoch in the
>> > > > >> LeaderAndIsrRequest, but the metadata cache in the broker hasn't
>> > been
>> > > > >> updated with the latest partition epoch. Not sure what's the best
>> > way
>> > > to
>> > > > >> address this issue. Perhaps we can update the metadata cache on
>> the
>> > > > broker
>> > > > >> with both LeaderAndIsrRequest and UpdateMetadataRequest. The
>> > challenge
>> > > > is
>> > > > >> that the two have slightly different data. For example, only the
>> > > latter
>> > > > >> has
>> > > > >> all endpoints.
>> > > > >>
>> > > > >
>> > > > > I am not sure whether this is a problem. Could you explain a bit
>> more
>> > > > what
>> > > > > specific problem this small window can cause?
>> > > > >
>> > > > > Since client can fetch metadata from any broker in the cluster,
>> and
>> > > given
>> > > > > that different brokers receive request (e.g. LeaderAndIsrRequest
>> and
>> > > > > UpdateMetadataRequest) in arbitrary order, the metadata received
>> by
>> > > > client
>> > > > > can be in arbitrary order (either newer or older) compared to the
>> > > > broker's
>> > > > > leadership state even if a given broker receives
>> LeaderAndIsrRequest
>> > > and
>> > > > > UpdateMetadataRequest simultaneously. So I am not sure it is
>> useful
>> > to
>> > > > > update broker's cache with LeaderAndIsrRequest.
>> > > > >
>> > > > >
>> > > > >> 64. The enforcement of leader epoch in Offset commit: We allow a
>> > > > consumer
>> > > > >> to set an arbitrary offset. So it's possible for offsets or
>> leader
>> > > epoch
>> > > > >> to
>> > > > >> go backwards. I am not sure if we could always enforce that the
>> > leader
>> > > > >> epoch only goes up on the broker.
>> > > > >>
>> > > > >
>> > > > > Sure. I have removed this check from the KIP.
>> > > > >
>> > > > > BTW, we can probably still ensure that the leader_epoch always
>> > increase
>> > > > if
>> > > > > the leader_epoch used with offset commit is the max(leader_epoch
>> of
>> > the
>> > > > > message with offset = the committed offset - 1, the largest known
>> > > > > leader_epoch from the metadata). But I don't have a good use-case
>> for
>> > > > this
>> > > > > alternative definition. So I choose the keep the KIP simple by
>> > > requiring
>> > > > > leader_epoch to always increase.
>> > > > >
>> > > > >
>> > > > >> 65. Good point on handling missing partition epoch due to topic
>> > > > deletion.
>> > > > >> Another potential way to address this is to additionally
>> propagate
>> > the
>> > > > >> global partition epoch to brokers and the clients. This way,
>> when a
>> > > > >> partition epoch is missing, we can use the global partition
>> epoch to
>> > > > >> reason
>> > > > >> about which metadata is more recent.
>> > > > >>
>> > > > >
>> > > > > This is a great idea. The global epoch can be used to order the
>> > > metadata
>> > > > > and help us recognize the more recent metadata if a topic (or
>> > > partition)
>> > > > is
>> > > > > deleted and re-created.
>> > > > >
>> > > > > Actually, it seems we only need to propagate the global epoch to
>> > > brokers
>> > > > > and clients without propagating this epoch on a per-topic or
>> > > > per-partition
>> > > > > basic. Doing so would simply interface changes made this KIP. Does
>> > this
>> > > > > approach sound reasonable?
>> > > > >
>> > > > >
>> > > > >> 66. A client may also get an offset by time using the
>> > offsetForTimes()
>> > > > >> api.
>> > > > >> So, we probably want to include offsetInternalMetadata in
>> > > > >> OffsetAndTimestamp
>> > > > >> as well.
>> > > > >>
>> > > > >
>> > > > > You are right. This probably also requires us to change the
>> > > > > ListOffsetRequest as well. I will update the KIP after we agree on
>> > the
>> > > > > solution for 65).
>> > > > >
>> > > > >
>> > > > >>
>> > > > >> 67. InteralMetadata can be a bit confusing with the metadata
>> field
>> > > > already
>> > > > >> there. Perhaps we can just call it OffsetEpoch. It might be
>> useful
>> > to
>> > > > make
>> > > > >> OffsetEpoch printable at least for debugging purpose. Once you do
>> > > that,
>> > > > we
>> > > > >> are already exposing the internal fields. So, not sure if it's
>> worth
>> > > > >> hiding
>> > > > >> them. If we do want to hide them, perhaps we can have sth like
>> the
>> > > > >> following. The binary encoding is probably more efficient than
>> JSON
>> > > for
>> > > > >> external storage.
>> > > > >>
>> > > > >> OffsetEpoch {
>> > > > >>  static OffsetEpoch decode(byte[]);
>> > > > >>
>> > > > >>   public byte[] encode();
>> > > > >>
>> > > > >>   public String toString();
>> > > > >> }
>> > > > >>
>> > > > >
>> > > > > Thanks much. I like this solution. I have updated the KIP
>> > accordingly.
>> > > > >
>> > > > >
>> > > > >
>> > > > >>
>> > > > >> Jun
>> > > > >>
>> > > > >> On Mon, Jan 8, 2018 at 4:22 PM, Dong Lin <li...@gmail.com>
>> > wrote:
>> > > > >>
>> > > > >> > Hey Jason,
>> > > > >> >
>> > > > >> > Certainly. This sounds good. I have updated the KIP to clarity
>> > that
>> > > > the
>> > > > >> > global epoch will be incremented by 1 each time a topic is
>> > deleted.
>> > > > >> >
>> > > > >> > Thanks,
>> > > > >> > Dong
>> > > > >> >
>> > > > >> > On Mon, Jan 8, 2018 at 4:09 PM, Jason Gustafson <
>> > jason@confluent.io
>> > > >
>> > > > >> > wrote:
>> > > > >> >
>> > > > >> > > Hi Dong,
>> > > > >> > >
>> > > > >> > >
>> > > > >> > > I think your approach will allow user to distinguish between
>> the
>> > > > >> metadata
>> > > > >> > > > before and after the topic deletion. I also agree that this
>> > can
>> > > be
>> > > > >> > > > potentially be useful to user. I am just not very sure
>> whether
>> > > we
>> > > > >> > already
>> > > > >> > > > have a good use-case to make the additional complexity
>> > > worthwhile.
>> > > > >> It
>> > > > >> > > seems
>> > > > >> > > > that this feature is kind of independent of the main
>> problem
>> > of
>> > > > this
>> > > > >> > KIP.
>> > > > >> > > > Could we add this as a future work?
>> > > > >> > >
>> > > > >> > >
>> > > > >> > > Do you think it's fair if we bump the topic epoch on deletion
>> > and
>> > > > >> leave
>> > > > >> > > propagation of the epoch for deleted topics for future work?
>> I
>> > > don't
>> > > > >> > think
>> > > > >> > > this adds much complexity and it makes the behavior
>> consistent:
>> > > > every
>> > > > >> > topic
>> > > > >> > > mutation results in an epoch bump.
>> > > > >> > >
>> > > > >> > > Thanks,
>> > > > >> > > Jason
>> > > > >> > >
>> > > > >> > > On Mon, Jan 8, 2018 at 3:14 PM, Dong Lin <
>> lindong28@gmail.com>
>> > > > wrote:
>> > > > >> > >
>> > > > >> > > > Hey Ismael,
>> > > > >> > > >
>> > > > >> > > > I guess we actually need user to see this field so that
>> user
>> > can
>> > > > >> store
>> > > > >> > > this
>> > > > >> > > > value in the external store together with the offset. We
>> just
>> > > > prefer
>> > > > >> > the
>> > > > >> > > > value to be opaque to discourage most users from
>> interpreting
>> > > this
>> > > > >> > value.
>> > > > >> > > > One more advantage of using such an opaque field is to be
>> able
>> > > to
>> > > > >> > evolve
>> > > > >> > > > the information (or schema) of this value without changing
>> > > > consumer
>> > > > >> API
>> > > > >> > > in
>> > > > >> > > > the future.
>> > > > >> > > >
>> > > > >> > > > I also thinking it is probably OK for user to be able to
>> > > interpret
>> > > > >> this
>> > > > >> > > > value, particularly for those advanced users.
>> > > > >> > > >
>> > > > >> > > > Thanks,
>> > > > >> > > > Dong
>> > > > >> > > >
>> > > > >> > > > On Mon, Jan 8, 2018 at 2:34 PM, Ismael Juma <
>> > ismael@juma.me.uk>
>> > > > >> wrote:
>> > > > >> > > >
>> > > > >> > > > > On Fri, Jan 5, 2018 at 7:15 PM, Jason Gustafson <
>> > > > >> jason@confluent.io>
>> > > > >> > > > > wrote:
>> > > > >> > > > > >
>> > > > >> > > > > > class OffsetAndMetadata {
>> > > > >> > > > > >   long offset;
>> > > > >> > > > > >   byte[] offsetMetadata;
>> > > > >> > > > > >   String metadata;
>> > > > >> > > > > > }
>> > > > >> > > > >
>> > > > >> > > > >
>> > > > >> > > > > > Admittedly, the naming is a bit annoying, but we can
>> > > probably
>> > > > >> come
>> > > > >> > up
>> > > > >> > > > > with
>> > > > >> > > > > > something better. Internally the byte array would have
>> a
>> > > > >> version.
>> > > > >> > If
>> > > > >> > > in
>> > > > >> > > > > the
>> > > > >> > > > > > future we have anything else we need to add, we can
>> update
>> > > the
>> > > > >> > > version
>> > > > >> > > > > and
>> > > > >> > > > > > we wouldn't need any new APIs.
>> > > > >> > > > > >
>> > > > >> > > > >
>> > > > >> > > > > We can also add fields to a class in a compatible way.
>> So,
>> > it
>> > > > >> seems
>> > > > >> > to
>> > > > >> > > me
>> > > > >> > > > > that the main advantage of the byte array is that it's
>> > opaque
>> > > to
>> > > > >> the
>> > > > >> > > > user.
>> > > > >> > > > > Is that correct? If so, we could also add any opaque
>> > metadata
>> > > > in a
>> > > > >> > > > subclass
>> > > > >> > > > > so that users don't even see it (unless they cast it, but
>> > then
>> > > > >> > they're
>> > > > >> > > on
>> > > > >> > > > > their own).
>> > > > >> > > > >
>> > > > >> > > > > Ismael
>> > > > >> > > > >
>> > > > >> > > > > The corresponding seek() and position() APIs might look
>> > > > something
>> > > > >> > like
>> > > > >> > > > > this:
>> > > > >> > > > > >
>> > > > >> > > > > > void seek(TopicPartition partition, long offset, byte[]
>> > > > >> > > > offsetMetadata);
>> > > > >> > > > > > byte[] positionMetadata(TopicPartition partition);
>> > > > >> > > > > >
>> > > > >> > > > > > What do you think?
>> > > > >> > > > > >
>> > > > >> > > > > > Thanks,
>> > > > >> > > > > > Jason
>> > > > >> > > > > >
>> > > > >> > > > > > On Thu, Jan 4, 2018 at 7:04 PM, Dong Lin <
>> > > lindong28@gmail.com
>> > > > >
>> > > > >> > > wrote:
>> > > > >> > > > > >
>> > > > >> > > > > > > Hey Jun, Jason,
>> > > > >> > > > > > >
>> > > > >> > > > > > > Thanks much for all the feedback. I have updated the
>> KIP
>> > > > >> based on
>> > > > >> > > the
>> > > > >> > > > > > > latest discussion. Can you help check whether it
>> looks
>> > > good?
>> > > > >> > > > > > >
>> > > > >> > > > > > > Thanks,
>> > > > >> > > > > > > Dong
>> > > > >> > > > > > >
>> > > > >> > > > > > > On Thu, Jan 4, 2018 at 5:36 PM, Dong Lin <
>> > > > lindong28@gmail.com
>> > > > >> >
>> > > > >> > > > wrote:
>> > > > >> > > > > > >
>> > > > >> > > > > > > > Hey Jun,
>> > > > >> > > > > > > >
>> > > > >> > > > > > > > Hmm... thinking about this more, I am not sure that
>> > the
>> > > > >> > proposed
>> > > > >> > > > API
>> > > > >> > > > > is
>> > > > >> > > > > > > > sufficient. For users that store offset
>> externally, we
>> > > > >> probably
>> > > > >> > > > need
>> > > > >> > > > > > > extra
>> > > > >> > > > > > > > API to return the leader_epoch and partition_epoch
>> for
>> > > all
>> > > > >> > > > partitions
>> > > > >> > > > > > > that
>> > > > >> > > > > > > > consumers are consuming. I suppose these users
>> > currently
>> > > > use
>> > > > >> > > > > position()
>> > > > >> > > > > > > to
>> > > > >> > > > > > > > get the offset. Thus we probably need a new method
>> > > > >> > > > > > positionWithEpoch(..)
>> > > > >> > > > > > > to
>> > > > >> > > > > > > > return <offset, partition_epoch, leader_epoch>.
>> Does
>> > > this
>> > > > >> sound
>> > > > >> > > > > > > reasonable?
>> > > > >> > > > > > > >
>> > > > >> > > > > > > > Thanks,
>> > > > >> > > > > > > > Dong
>> > > > >> > > > > > > >
>> > > > >> > > > > > > >
>> > > > >> > > > > > > > On Thu, Jan 4, 2018 at 5:26 PM, Jun Rao <
>> > > jun@confluent.io
>> > > > >
>> > > > >> > > wrote:
>> > > > >> > > > > > > >
>> > > > >> > > > > > > >> Hi, Dong,
>> > > > >> > > > > > > >>
>> > > > >> > > > > > > >> Yes, that's what I am thinking. OffsetEpoch will
>> be
>> > > > >> composed
>> > > > >> > of
>> > > > >> > > > > > > >> (partition_epoch,
>> > > > >> > > > > > > >> leader_epoch).
>> > > > >> > > > > > > >>
>> > > > >> > > > > > > >> Thanks,
>> > > > >> > > > > > > >>
>> > > > >> > > > > > > >> Jun
>> > > > >> > > > > > > >>
>> > > > >> > > > > > > >>
>> > > > >> > > > > > > >> On Thu, Jan 4, 2018 at 4:22 PM, Dong Lin <
>> > > > >> lindong28@gmail.com
>> > > > >> > >
>> > > > >> > > > > wrote:
>> > > > >> > > > > > > >>
>> > > > >> > > > > > > >> > Hey Jun,
>> > > > >> > > > > > > >> >
>> > > > >> > > > > > > >> > Thanks much. I like the the new API that you
>> > > proposed.
>> > > > I
>> > > > >> am
>> > > > >> > > not
>> > > > >> > > > > sure
>> > > > >> > > > > > > >> what
>> > > > >> > > > > > > >> > you exactly mean by offset_epoch. I suppose
>> that we
>> > > can
>> > > > >> use
>> > > > >> > > the
>> > > > >> > > > > pair
>> > > > >> > > > > > > of
>> > > > >> > > > > > > >> > (partition_epoch, leader_epoch) as the
>> > offset_epoch,
>> > > > >> right?
>> > > > >> > > > > > > >> >
>> > > > >> > > > > > > >> > Thanks,
>> > > > >> > > > > > > >> > Dong
>> > > > >> > > > > > > >> >
>> > > > >> > > > > > > >> > On Thu, Jan 4, 2018 at 4:02 PM, Jun Rao <
>> > > > >> jun@confluent.io>
>> > > > >> > > > wrote:
>> > > > >> > > > > > > >> >
>> > > > >> > > > > > > >> > > Hi, Dong,
>> > > > >> > > > > > > >> > >
>> > > > >> > > > > > > >> > > Got it. The api that you proposed works. The
>> > > question
>> > > > >> is
>> > > > >> > > > whether
>> > > > >> > > > > > > >> that's
>> > > > >> > > > > > > >> > the
>> > > > >> > > > > > > >> > > api that we want to have in the long term. My
>> > > concern
>> > > > >> is
>> > > > >> > > that
>> > > > >> > > > > > while
>> > > > >> > > > > > > >> the
>> > > > >> > > > > > > >> > api
>> > > > >> > > > > > > >> > > change is simple, the new api seems harder to
>> > > explain
>> > > > >> and
>> > > > >> > > use.
>> > > > >> > > > > For
>> > > > >> > > > > > > >> > example,
>> > > > >> > > > > > > >> > > a consumer storing offsets externally now
>> needs
>> > to
>> > > > call
>> > > > >> > > > > > > >> > > waitForMetadataUpdate() after calling seek().
>> > > > >> > > > > > > >> > >
>> > > > >> > > > > > > >> > > An alternative approach is to make the
>> following
>> > > > >> > compatible
>> > > > >> > > > api
>> > > > >> > > > > > > >> changes
>> > > > >> > > > > > > >> > in
>> > > > >> > > > > > > >> > > Consumer.
>> > > > >> > > > > > > >> > > * Add an additional OffsetEpoch field in
>> > > > >> > OffsetAndMetadata.
>> > > > >> > > > (no
>> > > > >> > > > > > need
>> > > > >> > > > > > > >> to
>> > > > >> > > > > > > >> > > change the CommitSync() api)
>> > > > >> > > > > > > >> > > * Add a new api seek(TopicPartition partition,
>> > long
>> > > > >> > offset,
>> > > > >> > > > > > > >> OffsetEpoch
>> > > > >> > > > > > > >> > > offsetEpoch). We can potentially deprecate the
>> > old
>> > > > api
>> > > > >> > > > > > > >> > seek(TopicPartition
>> > > > >> > > > > > > >> > > partition, long offset) in the future.
>> > > > >> > > > > > > >> > >
>> > > > >> > > > > > > >> > > The alternative approach has similar amount of
>> > api
>> > > > >> changes
>> > > > >> > > as
>> > > > >> > > > > > yours
>> > > > >> > > > > > > >> but
>> > > > >> > > > > > > >> > has
>> > > > >> > > > > > > >> > > the following benefits.
>> > > > >> > > > > > > >> > > 1. The api works in a similar way as how
>> offset
>> > > > >> management
>> > > > >> > > > works
>> > > > >> > > > > > now
>> > > > >> > > > > > > >> and
>> > > > >> > > > > > > >> > is
>> > > > >> > > > > > > >> > > probably what we want in the long term.
>> > > > >> > > > > > > >> > > 2. It can reset offsets better when there is
>> data
>> > > > loss
>> > > > >> due
>> > > > >> > > to
>> > > > >> > > > > > > unclean
>> > > > >> > > > > > > >> > > leader election or correlated replica failure.
>> > > > >> > > > > > > >> > > 3. It can reset offsets better when topic is
>> > > > recreated.
>> > > > >> > > > > > > >> > >
>> > > > >> > > > > > > >> > > Thanks,
>> > > > >> > > > > > > >> > >
>> > > > >> > > > > > > >> > > Jun
>> > > > >> > > > > > > >> > >
>> > > > >> > > > > > > >> > >
>> > > > >> > > > > > > >> > > On Thu, Jan 4, 2018 at 2:05 PM, Dong Lin <
>> > > > >> > > lindong28@gmail.com
>> > > > >> > > > >
>> > > > >> > > > > > > wrote:
>> > > > >> > > > > > > >> > >
>> > > > >> > > > > > > >> > > > Hey Jun,
>> > > > >> > > > > > > >> > > >
>> > > > >> > > > > > > >> > > > Yeah I agree that ideally we don't want an
>> ever
>> > > > >> growing
>> > > > >> > > > global
>> > > > >> > > > > > > >> metadata
>> > > > >> > > > > > > >> > > > version. I just think it may be more
>> desirable
>> > to
>> > > > >> keep
>> > > > >> > the
>> > > > >> > > > > > > consumer
>> > > > >> > > > > > > >> API
>> > > > >> > > > > > > >> > > > simple.
>> > > > >> > > > > > > >> > > >
>> > > > >> > > > > > > >> > > > In my current proposal, metadata version
>> > returned
>> > > > in
>> > > > >> the
>> > > > >> > > > fetch
>> > > > >> > > > > > > >> response
>> > > > >> > > > > > > >> > > > will be stored with the offset together.
>> More
>> > > > >> > > specifically,
>> > > > >> > > > > the
>> > > > >> > > > > > > >> > > > metadata_epoch in the new offset topic
>> schema
>> > > will
>> > > > be
>> > > > >> > the
>> > > > >> > > > > > largest
>> > > > >> > > > > > > >> > > > metadata_epoch from all the MetadataResponse
>> > and
>> > > > >> > > > FetchResponse
>> > > > >> > > > > > > ever
>> > > > >> > > > > > > >> > > > received by this consumer.
>> > > > >> > > > > > > >> > > >
>> > > > >> > > > > > > >> > > > We probably don't have to change the
>> consumer
>> > API
>> > > > for
>> > > > >> > > > > > > >> > > > commitSync(Map<TopicPartition,
>> > > OffsetAndMetadata>).
>> > > > >> If
>> > > > >> > > user
>> > > > >> > > > > > calls
>> > > > >> > > > > > > >> > > > commitSync(...) to commit offset 10 for a
>> given
>> > > > >> > partition,
>> > > > >> > > > for
>> > > > >> > > > > > > most
>> > > > >> > > > > > > >> > > > use-cases, this consumer instance should
>> have
>> > > > >> consumed
>> > > > >> > > > message
>> > > > >> > > > > > > with
>> > > > >> > > > > > > >> > > offset
>> > > > >> > > > > > > >> > > > 9 from this partition, in which case the
>> > consumer
>> > > > can
>> > > > >> > > > remember
>> > > > >> > > > > > and
>> > > > >> > > > > > > >> use
>> > > > >> > > > > > > >> > > the
>> > > > >> > > > > > > >> > > > metadata_epoch from the corresponding
>> > > FetchResponse
>> > > > >> when
>> > > > >> > > > > > > committing
>> > > > >> > > > > > > >> > > offset.
>> > > > >> > > > > > > >> > > > If user calls commitSync(..) to commit
>> offset
>> > 10
>> > > > for
>> > > > >> a
>> > > > >> > > given
>> > > > >> > > > > > > >> partition
>> > > > >> > > > > > > >> > > > without having consumed the message with
>> > offset 9
>> > > > >> using
>> > > > >> > > this
>> > > > >> > > > > > > >> consumer
>> > > > >> > > > > > > >> > > > instance, this is probably an advanced
>> > use-case.
>> > > In
>> > > > >> this
>> > > > >> > > > case
>> > > > >> > > > > > the
>> > > > >> > > > > > > >> > > advanced
>> > > > >> > > > > > > >> > > > user can retrieve the metadata_epoch using
>> the
>> > > > newly
>> > > > >> > added
>> > > > >> > > > > > > >> > > metadataEpoch()
>> > > > >> > > > > > > >> > > > API after it fetches the message with
>> offset 9
>> > > > >> (probably
>> > > > >> > > > from
>> > > > >> > > > > > > >> another
>> > > > >> > > > > > > >> > > > consumer instance) and encode this
>> > metadata_epoch
>> > > > in
>> > > > >> the
>> > > > >> > > > > > > >> > > > string OffsetAndMetadata.metadata. Do you
>> think
>> > > > this
>> > > > >> > > > solution
>> > > > >> > > > > > > would
>> > > > >> > > > > > > >> > work?
>> > > > >> > > > > > > >> > > >
>> > > > >> > > > > > > >> > > > By "not sure that I fully understand your
>> > latest
>> > > > >> > > > suggestion",
>> > > > >> > > > > > are
>> > > > >> > > > > > > >> you
>> > > > >> > > > > > > >> > > > referring to solution related to unclean
>> leader
>> > > > >> election
>> > > > >> > > > using
>> > > > >> > > > > > > >> > > leader_epoch
>> > > > >> > > > > > > >> > > > in my previous email?
>> > > > >> > > > > > > >> > > >
>> > > > >> > > > > > > >> > > > Thanks,
>> > > > >> > > > > > > >> > > > Dong
>> > > > >> > > > > > > >> > > >
>> > > > >> > > > > > > >> > > > On Thu, Jan 4, 2018 at 1:33 PM, Jun Rao <
>> > > > >> > jun@confluent.io
>> > > > >> > > >
>> > > > >> > > > > > wrote:
>> > > > >> > > > > > > >> > > >
>> > > > >> > > > > > > >> > > > > Hi, Dong,
>> > > > >> > > > > > > >> > > > >
>> > > > >> > > > > > > >> > > > > Not sure that I fully understand your
>> latest
>> > > > >> > suggestion.
>> > > > >> > > > > > > >> Returning an
>> > > > >> > > > > > > >> > > > ever
>> > > > >> > > > > > > >> > > > > growing global metadata version itself is
>> no
>> > > > ideal,
>> > > > >> > but
>> > > > >> > > is
>> > > > >> > > > > > fine.
>> > > > >> > > > > > > >> My
>> > > > >> > > > > > > >> > > > > question is whether the metadata version
>> > > returned
>> > > > >> in
>> > > > >> > the
>> > > > >> > > > > fetch
>> > > > >> > > > > > > >> > response
>> > > > >> > > > > > > >> > > > > needs to be stored with the offset
>> together
>> > if
>> > > > >> offsets
>> > > > >> > > are
>> > > > >> > > > > > > stored
>> > > > >> > > > > > > >> > > > > externally. If so, we also have to change
>> the
>> > > > >> consumer
>> > > > >> > > API
>> > > > >> > > > > for
>> > > > >> > > > > > > >> > > > commitSync()
>> > > > >> > > > > > > >> > > > > and need to worry about compatibility. If
>> we
>> > > > don't
>> > > > >> > store
>> > > > >> > > > the
>> > > > >> > > > > > > >> metadata
>> > > > >> > > > > > > >> > > > > version together with the offset, on a
>> > consumer
>> > > > >> > restart,
>> > > > >> > > > > it's
>> > > > >> > > > > > > not
>> > > > >> > > > > > > >> > clear
>> > > > >> > > > > > > >> > > > how
>> > > > >> > > > > > > >> > > > > we can ensure the metadata in the
>> consumer is
>> > > > high
>> > > > >> > > enough
>> > > > >> > > > > > since
>> > > > >> > > > > > > >> there
>> > > > >> > > > > > > >> > > is
>> > > > >> > > > > > > >> > > > no
>> > > > >> > > > > > > >> > > > > metadata version to compare with.
>> > > > >> > > > > > > >> > > > >
>> > > > >> > > > > > > >> > > > > Thanks,
>> > > > >> > > > > > > >> > > > >
>> > > > >> > > > > > > >> > > > > Jun
>> > > > >> > > > > > > >> > > > >
>> > > > >> > > > > > > >> > > > >
>> > > > >> > > > > > > >> > > > > On Wed, Jan 3, 2018 at 6:43 PM, Dong Lin <
>> > > > >> > > > > lindong28@gmail.com
>> > > > >> > > > > > >
>> > > > >> > > > > > > >> > wrote:
>> > > > >> > > > > > > >> > > > >
>> > > > >> > > > > > > >> > > > > > Hey Jun,
>> > > > >> > > > > > > >> > > > > >
>> > > > >> > > > > > > >> > > > > > Thanks much for the explanation.
>> > > > >> > > > > > > >> > > > > >
>> > > > >> > > > > > > >> > > > > > I understand the advantage of
>> > partition_epoch
>> > > > >> over
>> > > > >> > > > > > > >> metadata_epoch.
>> > > > >> > > > > > > >> > My
>> > > > >> > > > > > > >> > > > > > current concern is that the use of
>> > > leader_epoch
>> > > > >> and
>> > > > >> > > the
>> > > > >> > > > > > > >> > > partition_epoch
>> > > > >> > > > > > > >> > > > > > requires us considerable change to
>> > consumer's
>> > > > >> public
>> > > > >> > > API
>> > > > >> > > > > to
>> > > > >> > > > > > > take
>> > > > >> > > > > > > >> > care
>> > > > >> > > > > > > >> > > > of
>> > > > >> > > > > > > >> > > > > > the case where user stores offset
>> > externally.
>> > > > For
>> > > > >> > > > example,
>> > > > >> > > > > > > >> > > *consumer*.
>> > > > >> > > > > > > >> > > > > > *commitSync*(..) would have to take a
>> map
>> > > whose
>> > > > >> > value
>> > > > >> > > is
>> > > > >> > > > > > > >> <offset,
>> > > > >> > > > > > > >> > > > > metadata,
>> > > > >> > > > > > > >> > > > > > leader epoch, partition epoch>.
>> > > > >> > *consumer*.*seek*(...)
>> > > > >> > > > > would
>> > > > >> > > > > > > >> also
>> > > > >> > > > > > > >> > > need
>> > > > >> > > > > > > >> > > > > > leader_epoch and partition_epoch as
>> > > parameter.
>> > > > >> > > > Technically
>> > > > >> > > > > > we
>> > > > >> > > > > > > >> can
>> > > > >> > > > > > > >> > > > > probably
>> > > > >> > > > > > > >> > > > > > still make it work in a backward
>> compatible
>> > > > >> manner
>> > > > >> > > after
>> > > > >> > > > > > > careful
>> > > > >> > > > > > > >> > > design
>> > > > >> > > > > > > >> > > > > and
>> > > > >> > > > > > > >> > > > > > discussion. But these changes can make
>> the
>> > > > >> > consumer's
>> > > > >> > > > > > > interface
>> > > > >> > > > > > > >> > > > > > unnecessarily complex for more users
>> who do
>> > > not
>> > > > >> > store
>> > > > >> > > > > offset
>> > > > >> > > > > > > >> > > > externally.
>> > > > >> > > > > > > >> > > > > >
>> > > > >> > > > > > > >> > > > > > After thinking more about it, we can
>> > address
>> > > > all
>> > > > >> > > > problems
>> > > > >> > > > > > > >> discussed
>> > > > >> > > > > > > >> > > by
>> > > > >> > > > > > > >> > > > > only
>> > > > >> > > > > > > >> > > > > > using the metadata_epoch without
>> > introducing
>> > > > >> > > > leader_epoch
>> > > > >> > > > > or
>> > > > >> > > > > > > the
>> > > > >> > > > > > > >> > > > > > partition_epoch. The current KIP
>> describes
>> > > the
>> > > > >> > changes
>> > > > >> > > > to
>> > > > >> > > > > > the
>> > > > >> > > > > > > >> > > consumer
>> > > > >> > > > > > > >> > > > > API
>> > > > >> > > > > > > >> > > > > > and how the new API can be used if user
>> > > stores
>> > > > >> > offset
>> > > > >> > > > > > > >> externally.
>> > > > >> > > > > > > >> > In
>> > > > >> > > > > > > >> > > > > order
>> > > > >> > > > > > > >> > > > > > to address the scenario you described
>> > > earlier,
>> > > > we
>> > > > >> > can
>> > > > >> > > > > > include
>> > > > >> > > > > > > >> > > > > > metadata_epoch in the FetchResponse and
>> the
>> > > > >> > > > > > > LeaderAndIsrRequest.
>> > > > >> > > > > > > >> > > > Consumer
>> > > > >> > > > > > > >> > > > > > remembers the largest metadata_epoch
>> from
>> > all
>> > > > the
>> > > > >> > > > > > > FetchResponse
>> > > > >> > > > > > > >> it
>> > > > >> > > > > > > >> > > has
>> > > > >> > > > > > > >> > > > > > received. The metadata_epoch committed
>> with
>> > > the
>> > > > >> > > offset,
>> > > > >> > > > > > either
>> > > > >> > > > > > > >> > within
>> > > > >> > > > > > > >> > > > or
>> > > > >> > > > > > > >> > > > > > outside Kafka, should be the largest
>> > > > >> metadata_epoch
>> > > > >> > > > across
>> > > > >> > > > > > all
>> > > > >> > > > > > > >> > > > > > FetchResponse and MetadataResponse ever
>> > > > received
>> > > > >> by
>> > > > >> > > this
>> > > > >> > > > > > > >> consumer.
>> > > > >> > > > > > > >> > > > > >
>> > > > >> > > > > > > >> > > > > > The drawback of using only the
>> > metadata_epoch
>> > > > is
>> > > > >> > that
>> > > > >> > > we
>> > > > >> > > > > can
>> > > > >> > > > > > > not
>> > > > >> > > > > > > >> > > always
>> > > > >> > > > > > > >> > > > > do
>> > > > >> > > > > > > >> > > > > > the smart offset reset in case of
>> unclean
>> > > > leader
>> > > > >> > > > election
>> > > > >> > > > > > > which
>> > > > >> > > > > > > >> you
>> > > > >> > > > > > > >> > > > > > mentioned earlier. But in most case,
>> > unclean
>> > > > >> leader
>> > > > >> > > > > election
>> > > > >> > > > > > > >> > probably
>> > > > >> > > > > > > >> > > > > > happens when consumer is not
>> > > > >> rebalancing/restarting.
>> > > > >> > > In
>> > > > >> > > > > > these
>> > > > >> > > > > > > >> > cases,
>> > > > >> > > > > > > >> > > > > either
>> > > > >> > > > > > > >> > > > > > consumer is not directly affected by
>> > unclean
>> > > > >> leader
>> > > > >> > > > > election
>> > > > >> > > > > > > >> since
>> > > > >> > > > > > > >> > it
>> > > > >> > > > > > > >> > > > is
>> > > > >> > > > > > > >> > > > > > not consuming from the end of the log,
>> or
>> > > > >> consumer
>> > > > >> > can
>> > > > >> > > > > > derive
>> > > > >> > > > > > > >> the
>> > > > >> > > > > > > >> > > > > > leader_epoch from the most recent
>> message
>> > > > >> received
>> > > > >> > > > before
>> > > > >> > > > > it
>> > > > >> > > > > > > >> sees
>> > > > >> > > > > > > >> > > > > > OffsetOutOfRangeException. So I am not
>> sure
>> > > it
>> > > > is
>> > > > >> > > worth
>> > > > >> > > > > > adding
>> > > > >> > > > > > > >> the
>> > > > >> > > > > > > >> > > > > > leader_epoch to consumer API to address
>> the
>> > > > >> > remaining
>> > > > >> > > > > corner
>> > > > >> > > > > > > >> case.
>> > > > >> > > > > > > >> > > What
>> > > > >> > > > > > > >> > > > > do
>> > > > >> > > > > > > >> > > > > > you think?
>> > > > >> > > > > > > >> > > > > >
>> > > > >> > > > > > > >> > > > > > Thanks,
>> > > > >> > > > > > > >> > > > > > Dong
>> > > > >> > > > > > > >> > > > > >
>> > > > >> > > > > > > >> > > > > >
>> > > > >> > > > > > > >> > > > > >
>> > > > >> > > > > > > >> > > > > > On Tue, Jan 2, 2018 at 6:28 PM, Jun Rao
>> <
>> > > > >> > > > jun@confluent.io
>> > > > >> > > > > >
>> > > > >> > > > > > > >> wrote:
>> > > > >> > > > > > > >> > > > > >
>> > > > >> > > > > > > >> > > > > > > Hi, Dong,
>> > > > >> > > > > > > >> > > > > > >
>> > > > >> > > > > > > >> > > > > > > Thanks for the reply.
>> > > > >> > > > > > > >> > > > > > >
>> > > > >> > > > > > > >> > > > > > > To solve the topic recreation issue,
>> we
>> > > could
>> > > > >> use
>> > > > >> > > > > either a
>> > > > >> > > > > > > >> global
>> > > > >> > > > > > > >> > > > > > metadata
>> > > > >> > > > > > > >> > > > > > > version or a partition level epoch.
>> But
>> > > > either
>> > > > >> one
>> > > > >> > > > will
>> > > > >> > > > > > be a
>> > > > >> > > > > > > >> new
>> > > > >> > > > > > > >> > > > > concept,
>> > > > >> > > > > > > >> > > > > > > right? To me, the latter seems more
>> > > natural.
>> > > > It
>> > > > >> > also
>> > > > >> > > > > makes
>> > > > >> > > > > > > it
>> > > > >> > > > > > > >> > > easier
>> > > > >> > > > > > > >> > > > to
>> > > > >> > > > > > > >> > > > > > > detect if a consumer's offset is still
>> > > valid
>> > > > >> > after a
>> > > > >> > > > > topic
>> > > > >> > > > > > > is
>> > > > >> > > > > > > >> > > > > recreated.
>> > > > >> > > > > > > >> > > > > > As
>> > > > >> > > > > > > >> > > > > > > you pointed out, we don't need to
>> store
>> > the
>> > > > >> > > partition
>> > > > >> > > > > > epoch
>> > > > >> > > > > > > in
>> > > > >> > > > > > > >> > the
>> > > > >> > > > > > > >> > > > > > message.
>> > > > >> > > > > > > >> > > > > > > The following is what I am thinking.
>> > When a
>> > > > >> > > partition
>> > > > >> > > > is
>> > > > >> > > > > > > >> created,
>> > > > >> > > > > > > >> > > we
>> > > > >> > > > > > > >> > > > > can
>> > > > >> > > > > > > >> > > > > > > assign a partition epoch from an
>> > > > >> ever-increasing
>> > > > >> > > > global
>> > > > >> > > > > > > >> counter
>> > > > >> > > > > > > >> > and
>> > > > >> > > > > > > >> > > > > store
>> > > > >> > > > > > > >> > > > > > > it in /brokers/topics/[topic]/
>> > > > >> > > > partitions/[partitionId]
>> > > > >> > > > > in
>> > > > >> > > > > > > ZK.
>> > > > >> > > > > > > >> > The
>> > > > >> > > > > > > >> > > > > > > partition
>> > > > >> > > > > > > >> > > > > > > epoch is propagated to every broker.
>> The
>> > > > >> consumer
>> > > > >> > > will
>> > > > >> > > > > be
>> > > > >> > > > > > > >> > tracking
>> > > > >> > > > > > > >> > > a
>> > > > >> > > > > > > >> > > > > > tuple
>> > > > >> > > > > > > >> > > > > > > of <offset, leader epoch, partition
>> > epoch>
>> > > > for
>> > > > >> > > > offsets.
>> > > > >> > > > > > If a
>> > > > >> > > > > > > >> > topic
>> > > > >> > > > > > > >> > > is
>> > > > >> > > > > > > >> > > > > > > recreated, it's possible that a
>> > consumer's
>> > > > >> offset
>> > > > >> > > and
>> > > > >> > > > > > leader
>> > > > >> > > > > > > >> > epoch
>> > > > >> > > > > > > >> > > > > still
>> > > > >> > > > > > > >> > > > > > > match that in the broker, but
>> partition
>> > > epoch
>> > > > >> > won't
>> > > > >> > > > be.
>> > > > >> > > > > In
>> > > > >> > > > > > > >> this
>> > > > >> > > > > > > >> > > case,
>> > > > >> > > > > > > >> > > > > we
>> > > > >> > > > > > > >> > > > > > > can potentially still treat the
>> > consumer's
>> > > > >> offset
>> > > > >> > as
>> > > > >> > > > out
>> > > > >> > > > > > of
>> > > > >> > > > > > > >> range
>> > > > >> > > > > > > >> > > and
>> > > > >> > > > > > > >> > > > > > reset
>> > > > >> > > > > > > >> > > > > > > the offset based on the offset reset
>> > policy
>> > > > in
>> > > > >> the
>> > > > >> > > > > > consumer.
>> > > > >> > > > > > > >> This
>> > > > >> > > > > > > >> > > > seems
>> > > > >> > > > > > > >> > > > > > > harder to do with a global metadata
>> > > version.
>> > > > >> > > > > > > >> > > > > > >
>> > > > >> > > > > > > >> > > > > > > Jun
>> > > > >> > > > > > > >> > > > > > >
>> > > > >> > > > > > > >> > > > > > >
>> > > > >> > > > > > > >> > > > > > >
>> > > > >> > > > > > > >> > > > > > > On Mon, Dec 25, 2017 at 6:56 AM, Dong
>> > Lin <
>> > > > >> > > > > > > >> lindong28@gmail.com>
>> > > > >> > > > > > > >> > > > wrote:
>> > > > >> > > > > > > >> > > > > > >
>> > > > >> > > > > > > >> > > > > > > > Hey Jun,
>> > > > >> > > > > > > >> > > > > > > >
>> > > > >> > > > > > > >> > > > > > > > This is a very good example. After
>> > > thinking
>> > > > >> > > through
>> > > > >> > > > > this
>> > > > >> > > > > > > in
>> > > > >> > > > > > > >> > > > detail, I
>> > > > >> > > > > > > >> > > > > > > agree
>> > > > >> > > > > > > >> > > > > > > > that we need to commit offset with
>> > leader
>> > > > >> epoch
>> > > > >> > in
>> > > > >> > > > > order
>> > > > >> > > > > > > to
>> > > > >> > > > > > > >> > > address
>> > > > >> > > > > > > >> > > > > > this
>> > > > >> > > > > > > >> > > > > > > > example.
>> > > > >> > > > > > > >> > > > > > > >
>> > > > >> > > > > > > >> > > > > > > > I think the remaining question is
>> how
>> > to
>> > > > >> address
>> > > > >> > > the
>> > > > >> > > > > > > >> scenario
>> > > > >> > > > > > > >> > > that
>> > > > >> > > > > > > >> > > > > the
>> > > > >> > > > > > > >> > > > > > > > topic is deleted and re-created. One
>> > > > possible
>> > > > >> > > > solution
>> > > > >> > > > > > is
>> > > > >> > > > > > > to
>> > > > >> > > > > > > >> > > commit
>> > > > >> > > > > > > >> > > > > > > offset
>> > > > >> > > > > > > >> > > > > > > > with both the leader epoch and the
>> > > metadata
>> > > > >> > > version.
>> > > > >> > > > > The
>> > > > >> > > > > > > >> logic
>> > > > >> > > > > > > >> > > and
>> > > > >> > > > > > > >> > > > > the
>> > > > >> > > > > > > >> > > > > > > > implementation of this solution does
>> > not
>> > > > >> > require a
>> > > > >> > > > new
>> > > > >> > > > > > > >> concept
>> > > > >> > > > > > > >> > > > (e.g.
>> > > > >> > > > > > > >> > > > > > > > partition epoch) and it does not
>> > require
>> > > > any
>> > > > >> > > change
>> > > > >> > > > to
>> > > > >> > > > > > the
>> > > > >> > > > > > > >> > > message
>> > > > >> > > > > > > >> > > > > > format
>> > > > >> > > > > > > >> > > > > > > > or leader epoch. It also allows us
>> to
>> > > order
>> > > > >> the
>> > > > >> > > > > metadata
>> > > > >> > > > > > > in
>> > > > >> > > > > > > >> a
>> > > > >> > > > > > > >> > > > > > > > straightforward manner which may be
>> > > useful
>> > > > in
>> > > > >> > the
>> > > > >> > > > > > future.
>> > > > >> > > > > > > >> So it
>> > > > >> > > > > > > >> > > may
>> > > > >> > > > > > > >> > > > > be
>> > > > >> > > > > > > >> > > > > > a
>> > > > >> > > > > > > >> > > > > > > > better solution than generating a
>> > random
>> > > > >> > partition
>> > > > >> > > > > epoch
>> > > > >> > > > > > > >> every
>> > > > >> > > > > > > >> > > time
>> > > > >> > > > > > > >> > > > > we
>> > > > >> > > > > > > >> > > > > > > > create a partition. Does this sound
>> > > > >> reasonable?
>> > > > >> > > > > > > >> > > > > > > >
>> > > > >> > > > > > > >> > > > > > > > Previously one concern with using
>> the
>> > > > >> metadata
>> > > > >> > > > version
>> > > > >> > > > > > is
>> > > > >> > > > > > > >> that
>> > > > >> > > > > > > >> > > > > consumer
>> > > > >> > > > > > > >> > > > > > > > will be forced to refresh metadata
>> even
>> > > if
>> > > > >> > > metadata
>> > > > >> > > > > > > version
>> > > > >> > > > > > > >> is
>> > > > >> > > > > > > >> > > > > > increased
>> > > > >> > > > > > > >> > > > > > > > due to topics that the consumer is
>> not
>> > > > >> > interested
>> > > > >> > > > in.
>> > > > >> > > > > > Now
>> > > > >> > > > > > > I
>> > > > >> > > > > > > >> > > > realized
>> > > > >> > > > > > > >> > > > > > that
>> > > > >> > > > > > > >> > > > > > > > this is probably not a problem.
>> > Currently
>> > > > >> client
>> > > > >> > > > will
>> > > > >> > > > > > > >> refresh
>> > > > >> > > > > > > >> > > > > metadata
>> > > > >> > > > > > > >> > > > > > > > either due to
>> InvalidMetadataException
>> > in
>> > > > the
>> > > > >> > > > response
>> > > > >> > > > > > > from
>> > > > >> > > > > > > >> > > broker
>> > > > >> > > > > > > >> > > > or
>> > > > >> > > > > > > >> > > > > > due
>> > > > >> > > > > > > >> > > > > > > > to metadata expiry. The addition of
>> the
>> > > > >> metadata
>> > > > >> > > > > version
>> > > > >> > > > > > > >> should
>> > > > >> > > > > > > >> > > > > > increase
>> > > > >> > > > > > > >> > > > > > > > the overhead of metadata refresh
>> caused
>> > > by
>> > > > >> > > > > > > >> > > > InvalidMetadataException.
>> > > > >> > > > > > > >> > > > > If
>> > > > >> > > > > > > >> > > > > > > > client refresh metadata due to
>> expiry
>> > and
>> > > > it
>> > > > >> > > > receives
>> > > > >> > > > > a
>> > > > >> > > > > > > >> > metadata
>> > > > >> > > > > > > >> > > > > whose
>> > > > >> > > > > > > >> > > > > > > > version is lower than the current
>> > > metadata
>> > > > >> > > version,
>> > > > >> > > > we
>> > > > >> > > > > > can
>> > > > >> > > > > > > >> > reject
>> > > > >> > > > > > > >> > > > the
>> > > > >> > > > > > > >> > > > > > > > metadata but still reset the
>> metadata
>> > > age,
>> > > > >> which
>> > > > >> > > > > > > essentially
>> > > > >> > > > > > > >> > keep
>> > > > >> > > > > > > >> > > > the
>> > > > >> > > > > > > >> > > > > > > > existing behavior in the client.
>> > > > >> > > > > > > >> > > > > > > >
>> > > > >> > > > > > > >> > > > > > > > Thanks much,
>> > > > >> > > > > > > >> > > > > > > > Dong
>> > > > >> > > > > > > >> > > > > > > >
>> > > > >> > > > > > > >> > > > > > >
>> > > > >> > > > > > > >> > > > > >
>> > > > >> > > > > > > >> > > > >
>> > > > >> > > > > > > >> > > >
>> > > > >> > > > > > > >> > >
>> > > > >> > > > > > > >> >
>> > > > >> > > > > > > >>
>> > > > >> > > > > > > >
>> > > > >> > > > > > > >
>> > > > >> > > > > > >
>> > > > >> > > > > >
>> > > > >> > > > >
>> > > > >> > > >
>> > > > >> > >
>> > > > >> >
>> > > > >>
>> > > > >
>> > > > >
>> > > >
>> > >
>> >
>>
>
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Dong Lin <li...@gmail.com>.
Hey Jun,

Thanks much for the comments. Please see my comments inline.

On Tue, Jan 16, 2018 at 4:38 PM, Jun Rao <ju...@confluent.io> wrote:

> Hi, Dong,
>
> Thanks for the updated KIP. Looks good to me overall. Just a few minor
> comments.
>
> 60. OffsetAndMetadata positionAndOffsetEpoch(TopicPartition partition): It
> seems that there is no need to return metadata. We probably want to return
> sth like OffsetAndEpoch.
>

Previously I think we may want to re-use the existing class to keep our
consumer interface simpler. I have updated the KIP to add class
OffsetAndOffsetEpoch. I didn't use OffsetAndEpoch because user may confuse
this name with OffsetEpoch. Does this sound OK?


>
> 61. Should we store partition_epoch in
> /brokers/topics/[topic]/partitions/[partitionId] in ZK?
>

I have considered this. I think the advantage of adding the
partition->partition_epoch map in the existing
znode /brokers/topics/[topic]/partitions is that controller only needs to
read one znode per topic to gets its partition_epoch information. Otherwise
controller may need to read one extra znode per partition to get the same
information.

When we delete partition or expand partition of a topic, someone needs to
modify partition->partition_epoch map in znode
/brokers/topics/[topic]/partitions.
This may seem a bit more complicated than simply adding or deleting znode
/brokers/topics/[topic]/partitions/[partitionId]. But the complexity is
probably similar to the existing operation of modifying the
partition->replica_list mapping in znode /brokers/topics/[topic]. So not
sure it is better to store the partition_epoch in
/brokers/topics/[topic]/partitions/[partitionId].
What do you think?


>
> 62. For checking outdated metadata in the client, we probably want to add
> when max_partition_epoch will be used.
>

The max_partition_epoch is used in the Proposed Changes -> Client's
metadata refresh section to determine whether a metadata is outdated. And
this formula is referenced and re-used in other sections to determine
whether a metadata is outdated. Does this formula look OK?


>
> 63. "The leader_epoch should be the largest leader_epoch of messages whose
> offset < the commit offset. If no message has been consumed since consumer
> initialization, the leader_epoch from seek(...) or OffsetFetchResponse
> should be used. The partition_epoch should be read from the last
> FetchResponse corresponding to the given partition and commit offset. ":
> leader_epoch and partition_epoch are associated with an offset. So, if no
> message is consumed, there is no offset and therefore there is no need to
> read leader_epoch and partition_epoch. Also, the leader_epoch associated
> with the offset should just come from the messages returned in the fetch
> response.
>

I am thinking that, if user calls seek(..) and commitSync(...) without
consuming any messages, we should re-use the leader_epoch and
partition_epoch provided by the seek(...) in the OffsetCommitRequest. And
if messages have been successfully consumed, then leader_epoch will come
from the messages returned in the fetch response. The condition "messages
whose offset < the commit offset" is needed to take care of the log
compacted topic which may have offset gap due to log cleaning.

Did I miss something here? Or should I rephrase the paragraph to make it
less confusing?


> 64. Could you include the public methods in the OffsetEpoch class?
>

I mistakenly deleted the definition of OffsetEpoch class from the KIP. I
just added it back with the public methods. Could you take another look?


>
> Jun
>
>
> On Thu, Jan 11, 2018 at 5:43 PM, Dong Lin <li...@gmail.com> wrote:
>
> > Hey Jun,
> >
> > Thanks much. I agree that we can not rely on committed offsets to be
> always
> > deleted when we delete topic. So it is necessary to use a per-partition
> > epoch that does not change unless this partition is deleted. I also agree
> > that it is very nice to be able to uniquely identify a message with
> > (offset, leader_epoch, partition_epoch) in face of potential topic
> deletion
> > and unclean leader election.
> >
> > I agree with all your comments. And I have updated the KIP based on our
> > latest discussion. In addition, I added InvalidPartitionEpochException
> > which will be thrown by consumer.poll() if the partition_epoch associated
> > with the partition, which can be given to consumer using seek(...), is
> > different from the partition_epoch in the FetchResponse.
> >
> > Can you take another look at the latest KIP?
> >
> > Thanks!
> > Dong
> >
> >
> >
> > On Wed, Jan 10, 2018 at 2:24 PM, Jun Rao <ju...@confluent.io> wrote:
> >
> > > Hi, Dong,
> > >
> > > My replies are the following.
> > >
> > > 60. What you described could also work. The drawback is that we will be
> > > unnecessarily changing the partition epoch when a partition hasn't
> really
> > > changed. I was imagining that the partition epoch will be stored in
> > > /brokers/topics/[topic]/partitions/[partitionId], instead of at the
> > topic
> > > level. So, not sure if ZK size limit is an issue.
> > >
> > > 61, 62 and 65. To me, the offset + offset_epoch is a unique identifier
> > for
> > > a message. So, if a message hasn't changed, the offset and the
> associated
> > > offset_epoch ideally should remain the same (it will be kind of weird
> if
> > > two consumer apps save the offset on the same message, but the
> > offset_epoch
> > > are different). partition_epoch + leader_epoch give us that.
> > global_epoch +
> > > leader_epoch don't. If we use this approach, we can solve not only the
> > > problem that you have identified, but also other problems when there is
> > > data loss or topic re-creation more reliably. For example, in the
> future,
> > > if we include the partition_epoch and leader_epoch in the fetch
> request,
> > > the server can do a more reliable check of whether that offset is valid
> > or
> > > not. I am not sure that we can rely upon all external offsets to be
> > removed
> > > on topic deletion. For example, a topic may be deleted by an admin who
> > may
> > > not know all the applications.
> > >
> > > If we agree on the above, the second question is then how to reliably
> > > propagate the partition_epoch and the leader_epoch to the consumer when
> > > there are leader or partition changes. The leader_epoch comes from the
> > > message, which is reliable. So, I was suggesting that when we store an
> > > offset, we can just store the leader_epoch from the message set
> > containing
> > > that offset. Similarly, I was thinking that if the partition_epoch is
> in
> > > the fetch response, we can propagate partition_epoch reliably where is
> > > partition_epoch change.
> > >
> > > 63. My point is that once a leader is producing a message in the new
> > > partition_epoch, ideally, we should associate the new offsets with the
> > new
> > > partition_epoch. Otherwise, the offset_epoch won't be the correct
> unique
> > > identifier (useful for solving other problems mentioned above). I was
> > > originally thinking that the leader will include the partition_epoch in
> > the
> > > metadata cache in the fetch response. It's just that right now,
> metadata
> > > cache is updated on UpdateMetadataRequest, which typically happens
> after
> > > the LeaderAndIsrRequest. Another approach is for the leader to cache
> the
> > > partition_epoch in the Partition object and return that (instead of the
> > one
> > > in metadata cache) in the fetch response.
> > >
> > > 65. It seems to me that the global_epoch and the partition_epoch have
> > > different purposes. A partition_epoch has the benefit that it (1) can
> be
> > > used to form a unique identifier for a message and (2) can be used to
> > > solve other
> > > corner case problems in the future. I am not sure having just a
> > > global_epoch can achieve these. global_epoch is useful to determine
> which
> > > version of the metadata is newer, especially with topic deletion.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > > On Tue, Jan 9, 2018 at 11:34 PM, Dong Lin <li...@gmail.com> wrote:
> > >
> > > > Regarding the use of the global epoch in 65), it is very similar to
> the
> > > > proposal of the metadata_epoch we discussed earlier. The main
> > difference
> > > is
> > > > that this epoch is incremented when we create/expand/delete topic and
> > > does
> > > > not change when controller re-send metadata.
> > > >
> > > > I looked at our previous discussion. It seems that we prefer
> > > > partition_epoch over the metadata_epoch because 1) we prefer not to
> > have
> > > an
> > > > ever growing metadata_epoch and 2) we can reset offset better when
> > topic
> > > is
> > > > re-created. The use of global topic_epoch avoids the drawback of an
> > ever
> > > > quickly ever growing metadata_epoch. Though the global epoch does not
> > > allow
> > > > us to recognize the invalid offset committed before the topic
> > > re-creation,
> > > > we can probably just delete the offset when we delete a topic. Thus I
> > am
> > > > not very sure whether it is still worthwhile to have a per-partition
> > > > partition_epoch if the metadata already has the global epoch.
> > > >
> > > >
> > > > On Tue, Jan 9, 2018 at 6:58 PM, Dong Lin <li...@gmail.com>
> wrote:
> > > >
> > > > > Hey Jun,
> > > > >
> > > > > Thanks so much. These comments very useful. Please see below my
> > > comments.
> > > > >
> > > > > On Mon, Jan 8, 2018 at 5:52 PM, Jun Rao <ju...@confluent.io> wrote:
> > > > >
> > > > >> Hi, Dong,
> > > > >>
> > > > >> Thanks for the updated KIP. A few more comments.
> > > > >>
> > > > >> 60. Perhaps having a partition epoch is more flexible since in the
> > > > future,
> > > > >> we may support deleting a partition as well.
> > > > >>
> > > > >
> > > > > Yeah I have considered this. I think we can probably still support
> > > > > deleting a partition by using the topic_epoch -- when partition of
> a
> > > > topic
> > > > > is deleted or created, epoch of all partitions of this topic will
> be
> > > > > incremented by 1. Therefore, if that partition is re-created later,
> > the
> > > > > epoch of that partition will still be larger than its epoch before
> > the
> > > > > deletion, which still allows the client to order the metadata for
> the
> > > > > purpose of this KIP. Does this sound reasonable?
> > > > >
> > > > > The advantage of using topic_epoch instead of partition_epoch is
> that
> > > the
> > > > > size of the /brokers/topics/[topic] znode and request/response size
> > can
> > > > be
> > > > > smaller. We have a limit on the maximum size of znode (typically
> > 1MB).
> > > > Use
> > > > > partition epoch can effectively reduce the number of partitions
> that
> > > can
> > > > be
> > > > > described by the /brokers/topics/[topic] znode.
> > > > >
> > > > > One use-case of partition_epoch for client to detect that the
> > committed
> > > > > offset, either from kafka offset topic or from the external store
> is
> > > > > invalid after partition deletion and re-creation. However, it seems
> > > that
> > > > we
> > > > > can also address this use-case with other approaches. For example,
> > when
> > > > > AdminClient deletes partitions, it can also delete the committed
> > > offsets
> > > > > for those partitions from the offset topic. If user stores offset
> > > > > externally, it might make sense for user to similarly remove
> offsets
> > of
> > > > > related partitions after these partitions are deleted. So I am not
> > sure
> > > > > that we should use partition_epoch in this KIP.
> > > > >
> > > > >
> > > > >>
> > > > >> 61. It seems that the leader epoch returned in the position() call
> > > > should
> > > > >> the the leader epoch returned in the fetch response, not the one
> in
> > > the
> > > > >> metadata cache of the client.
> > > > >
> > > > >
> > > > > I think this is a good idea. Just to double check, this change does
> > not
> > > > > affect the correctness or performance of this KIP. But it can be
> > useful
> > > > if
> > > > > we want to use the leader_epoch to better handle the offset rest in
> > > case
> > > > of
> > > > > unclean leader election, which is listed in the future work. Is
> this
> > > > > understanding correct?
> > > > >
> > > > > I have updated the KIP to specify that the leader_epoch returned by
> > > > > position() should be the largest leader_epoch of those already
> > consumed
> > > > > messages whose offset < position. If no message has been consumed
> > since
> > > > > consumer initialization, the leader_epoch from seek() or
> > > > > OffsetFetchResponse should be used. The offset included in the
> > > > > OffsetCommitRequest will also be determined in the similar manner.
> > > > >
> > > > >
> > > > >>
> > > > >> 62. I am wondering if we should return the partition epoch in the
> > > fetch
> > > > >> response as well. In the current proposal, if a topic is recreated
> > and
> > > > the
> > > > >> new leader is on the same broker as the old one, there is nothing
> to
> > > > force
> > > > >> the metadata refresh in the client. So, the client may still
> > associate
> > > > the
> > > > >> offset with the old partition epoch.
> > > > >>
> > > > >
> > > > > Could you help me understand the problem if a client associates old
> > > > > partition_epoch (or the topic_epoch as of the current KIP) with the
> > > > offset?
> > > > > The main purpose of the topic_epoch is to be able to drop
> > leader_epoch
> > > > to 0
> > > > > after a partition is deleted and re-created. I guess you may be
> > > thinking
> > > > > about using the partition_epoch to detect that the committed offset
> > is
> > > > > invalid? In that case, I am wondering if the alternative approach
> > > > described
> > > > > in 60) would be reasonable.
> > > > >
> > > > >
> > > > >>
> > > > >> 63. There is some subtle coordination between the
> > LeaderAndIsrRequest
> > > > and
> > > > >> UpdateMetadataRequest. Currently, when a leader changes, the
> > > controller
> > > > >> first sends the LeaderAndIsrRequest to the assigned replicas and
> the
> > > > >> UpdateMetadataRequest to every broker. So, there could be a small
> > > window
> > > > >> when the leader already receives the new partition epoch in the
> > > > >> LeaderAndIsrRequest, but the metadata cache in the broker hasn't
> > been
> > > > >> updated with the latest partition epoch. Not sure what's the best
> > way
> > > to
> > > > >> address this issue. Perhaps we can update the metadata cache on
> the
> > > > broker
> > > > >> with both LeaderAndIsrRequest and UpdateMetadataRequest. The
> > challenge
> > > > is
> > > > >> that the two have slightly different data. For example, only the
> > > latter
> > > > >> has
> > > > >> all endpoints.
> > > > >>
> > > > >
> > > > > I am not sure whether this is a problem. Could you explain a bit
> more
> > > > what
> > > > > specific problem this small window can cause?
> > > > >
> > > > > Since client can fetch metadata from any broker in the cluster, and
> > > given
> > > > > that different brokers receive request (e.g. LeaderAndIsrRequest
> and
> > > > > UpdateMetadataRequest) in arbitrary order, the metadata received by
> > > > client
> > > > > can be in arbitrary order (either newer or older) compared to the
> > > > broker's
> > > > > leadership state even if a given broker receives
> LeaderAndIsrRequest
> > > and
> > > > > UpdateMetadataRequest simultaneously. So I am not sure it is useful
> > to
> > > > > update broker's cache with LeaderAndIsrRequest.
> > > > >
> > > > >
> > > > >> 64. The enforcement of leader epoch in Offset commit: We allow a
> > > > consumer
> > > > >> to set an arbitrary offset. So it's possible for offsets or leader
> > > epoch
> > > > >> to
> > > > >> go backwards. I am not sure if we could always enforce that the
> > leader
> > > > >> epoch only goes up on the broker.
> > > > >>
> > > > >
> > > > > Sure. I have removed this check from the KIP.
> > > > >
> > > > > BTW, we can probably still ensure that the leader_epoch always
> > increase
> > > > if
> > > > > the leader_epoch used with offset commit is the max(leader_epoch of
> > the
> > > > > message with offset = the committed offset - 1, the largest known
> > > > > leader_epoch from the metadata). But I don't have a good use-case
> for
> > > > this
> > > > > alternative definition. So I choose the keep the KIP simple by
> > > requiring
> > > > > leader_epoch to always increase.
> > > > >
> > > > >
> > > > >> 65. Good point on handling missing partition epoch due to topic
> > > > deletion.
> > > > >> Another potential way to address this is to additionally propagate
> > the
> > > > >> global partition epoch to brokers and the clients. This way, when
> a
> > > > >> partition epoch is missing, we can use the global partition epoch
> to
> > > > >> reason
> > > > >> about which metadata is more recent.
> > > > >>
> > > > >
> > > > > This is a great idea. The global epoch can be used to order the
> > > metadata
> > > > > and help us recognize the more recent metadata if a topic (or
> > > partition)
> > > > is
> > > > > deleted and re-created.
> > > > >
> > > > > Actually, it seems we only need to propagate the global epoch to
> > > brokers
> > > > > and clients without propagating this epoch on a per-topic or
> > > > per-partition
> > > > > basic. Doing so would simply interface changes made this KIP. Does
> > this
> > > > > approach sound reasonable?
> > > > >
> > > > >
> > > > >> 66. A client may also get an offset by time using the
> > offsetForTimes()
> > > > >> api.
> > > > >> So, we probably want to include offsetInternalMetadata in
> > > > >> OffsetAndTimestamp
> > > > >> as well.
> > > > >>
> > > > >
> > > > > You are right. This probably also requires us to change the
> > > > > ListOffsetRequest as well. I will update the KIP after we agree on
> > the
> > > > > solution for 65).
> > > > >
> > > > >
> > > > >>
> > > > >> 67. InteralMetadata can be a bit confusing with the metadata field
> > > > already
> > > > >> there. Perhaps we can just call it OffsetEpoch. It might be useful
> > to
> > > > make
> > > > >> OffsetEpoch printable at least for debugging purpose. Once you do
> > > that,
> > > > we
> > > > >> are already exposing the internal fields. So, not sure if it's
> worth
> > > > >> hiding
> > > > >> them. If we do want to hide them, perhaps we can have sth like the
> > > > >> following. The binary encoding is probably more efficient than
> JSON
> > > for
> > > > >> external storage.
> > > > >>
> > > > >> OffsetEpoch {
> > > > >>  static OffsetEpoch decode(byte[]);
> > > > >>
> > > > >>   public byte[] encode();
> > > > >>
> > > > >>   public String toString();
> > > > >> }
> > > > >>
> > > > >
> > > > > Thanks much. I like this solution. I have updated the KIP
> > accordingly.
> > > > >
> > > > >
> > > > >
> > > > >>
> > > > >> Jun
> > > > >>
> > > > >> On Mon, Jan 8, 2018 at 4:22 PM, Dong Lin <li...@gmail.com>
> > wrote:
> > > > >>
> > > > >> > Hey Jason,
> > > > >> >
> > > > >> > Certainly. This sounds good. I have updated the KIP to clarity
> > that
> > > > the
> > > > >> > global epoch will be incremented by 1 each time a topic is
> > deleted.
> > > > >> >
> > > > >> > Thanks,
> > > > >> > Dong
> > > > >> >
> > > > >> > On Mon, Jan 8, 2018 at 4:09 PM, Jason Gustafson <
> > jason@confluent.io
> > > >
> > > > >> > wrote:
> > > > >> >
> > > > >> > > Hi Dong,
> > > > >> > >
> > > > >> > >
> > > > >> > > I think your approach will allow user to distinguish between
> the
> > > > >> metadata
> > > > >> > > > before and after the topic deletion. I also agree that this
> > can
> > > be
> > > > >> > > > potentially be useful to user. I am just not very sure
> whether
> > > we
> > > > >> > already
> > > > >> > > > have a good use-case to make the additional complexity
> > > worthwhile.
> > > > >> It
> > > > >> > > seems
> > > > >> > > > that this feature is kind of independent of the main problem
> > of
> > > > this
> > > > >> > KIP.
> > > > >> > > > Could we add this as a future work?
> > > > >> > >
> > > > >> > >
> > > > >> > > Do you think it's fair if we bump the topic epoch on deletion
> > and
> > > > >> leave
> > > > >> > > propagation of the epoch for deleted topics for future work? I
> > > don't
> > > > >> > think
> > > > >> > > this adds much complexity and it makes the behavior
> consistent:
> > > > every
> > > > >> > topic
> > > > >> > > mutation results in an epoch bump.
> > > > >> > >
> > > > >> > > Thanks,
> > > > >> > > Jason
> > > > >> > >
> > > > >> > > On Mon, Jan 8, 2018 at 3:14 PM, Dong Lin <lindong28@gmail.com
> >
> > > > wrote:
> > > > >> > >
> > > > >> > > > Hey Ismael,
> > > > >> > > >
> > > > >> > > > I guess we actually need user to see this field so that user
> > can
> > > > >> store
> > > > >> > > this
> > > > >> > > > value in the external store together with the offset. We
> just
> > > > prefer
> > > > >> > the
> > > > >> > > > value to be opaque to discourage most users from
> interpreting
> > > this
> > > > >> > value.
> > > > >> > > > One more advantage of using such an opaque field is to be
> able
> > > to
> > > > >> > evolve
> > > > >> > > > the information (or schema) of this value without changing
> > > > consumer
> > > > >> API
> > > > >> > > in
> > > > >> > > > the future.
> > > > >> > > >
> > > > >> > > > I also thinking it is probably OK for user to be able to
> > > interpret
> > > > >> this
> > > > >> > > > value, particularly for those advanced users.
> > > > >> > > >
> > > > >> > > > Thanks,
> > > > >> > > > Dong
> > > > >> > > >
> > > > >> > > > On Mon, Jan 8, 2018 at 2:34 PM, Ismael Juma <
> > ismael@juma.me.uk>
> > > > >> wrote:
> > > > >> > > >
> > > > >> > > > > On Fri, Jan 5, 2018 at 7:15 PM, Jason Gustafson <
> > > > >> jason@confluent.io>
> > > > >> > > > > wrote:
> > > > >> > > > > >
> > > > >> > > > > > class OffsetAndMetadata {
> > > > >> > > > > >   long offset;
> > > > >> > > > > >   byte[] offsetMetadata;
> > > > >> > > > > >   String metadata;
> > > > >> > > > > > }
> > > > >> > > > >
> > > > >> > > > >
> > > > >> > > > > > Admittedly, the naming is a bit annoying, but we can
> > > probably
> > > > >> come
> > > > >> > up
> > > > >> > > > > with
> > > > >> > > > > > something better. Internally the byte array would have a
> > > > >> version.
> > > > >> > If
> > > > >> > > in
> > > > >> > > > > the
> > > > >> > > > > > future we have anything else we need to add, we can
> update
> > > the
> > > > >> > > version
> > > > >> > > > > and
> > > > >> > > > > > we wouldn't need any new APIs.
> > > > >> > > > > >
> > > > >> > > > >
> > > > >> > > > > We can also add fields to a class in a compatible way. So,
> > it
> > > > >> seems
> > > > >> > to
> > > > >> > > me
> > > > >> > > > > that the main advantage of the byte array is that it's
> > opaque
> > > to
> > > > >> the
> > > > >> > > > user.
> > > > >> > > > > Is that correct? If so, we could also add any opaque
> > metadata
> > > > in a
> > > > >> > > > subclass
> > > > >> > > > > so that users don't even see it (unless they cast it, but
> > then
> > > > >> > they're
> > > > >> > > on
> > > > >> > > > > their own).
> > > > >> > > > >
> > > > >> > > > > Ismael
> > > > >> > > > >
> > > > >> > > > > The corresponding seek() and position() APIs might look
> > > > something
> > > > >> > like
> > > > >> > > > > this:
> > > > >> > > > > >
> > > > >> > > > > > void seek(TopicPartition partition, long offset, byte[]
> > > > >> > > > offsetMetadata);
> > > > >> > > > > > byte[] positionMetadata(TopicPartition partition);
> > > > >> > > > > >
> > > > >> > > > > > What do you think?
> > > > >> > > > > >
> > > > >> > > > > > Thanks,
> > > > >> > > > > > Jason
> > > > >> > > > > >
> > > > >> > > > > > On Thu, Jan 4, 2018 at 7:04 PM, Dong Lin <
> > > lindong28@gmail.com
> > > > >
> > > > >> > > wrote:
> > > > >> > > > > >
> > > > >> > > > > > > Hey Jun, Jason,
> > > > >> > > > > > >
> > > > >> > > > > > > Thanks much for all the feedback. I have updated the
> KIP
> > > > >> based on
> > > > >> > > the
> > > > >> > > > > > > latest discussion. Can you help check whether it looks
> > > good?
> > > > >> > > > > > >
> > > > >> > > > > > > Thanks,
> > > > >> > > > > > > Dong
> > > > >> > > > > > >
> > > > >> > > > > > > On Thu, Jan 4, 2018 at 5:36 PM, Dong Lin <
> > > > lindong28@gmail.com
> > > > >> >
> > > > >> > > > wrote:
> > > > >> > > > > > >
> > > > >> > > > > > > > Hey Jun,
> > > > >> > > > > > > >
> > > > >> > > > > > > > Hmm... thinking about this more, I am not sure that
> > the
> > > > >> > proposed
> > > > >> > > > API
> > > > >> > > > > is
> > > > >> > > > > > > > sufficient. For users that store offset externally,
> we
> > > > >> probably
> > > > >> > > > need
> > > > >> > > > > > > extra
> > > > >> > > > > > > > API to return the leader_epoch and partition_epoch
> for
> > > all
> > > > >> > > > partitions
> > > > >> > > > > > > that
> > > > >> > > > > > > > consumers are consuming. I suppose these users
> > currently
> > > > use
> > > > >> > > > > position()
> > > > >> > > > > > > to
> > > > >> > > > > > > > get the offset. Thus we probably need a new method
> > > > >> > > > > > positionWithEpoch(..)
> > > > >> > > > > > > to
> > > > >> > > > > > > > return <offset, partition_epoch, leader_epoch>. Does
> > > this
> > > > >> sound
> > > > >> > > > > > > reasonable?
> > > > >> > > > > > > >
> > > > >> > > > > > > > Thanks,
> > > > >> > > > > > > > Dong
> > > > >> > > > > > > >
> > > > >> > > > > > > >
> > > > >> > > > > > > > On Thu, Jan 4, 2018 at 5:26 PM, Jun Rao <
> > > jun@confluent.io
> > > > >
> > > > >> > > wrote:
> > > > >> > > > > > > >
> > > > >> > > > > > > >> Hi, Dong,
> > > > >> > > > > > > >>
> > > > >> > > > > > > >> Yes, that's what I am thinking. OffsetEpoch will be
> > > > >> composed
> > > > >> > of
> > > > >> > > > > > > >> (partition_epoch,
> > > > >> > > > > > > >> leader_epoch).
> > > > >> > > > > > > >>
> > > > >> > > > > > > >> Thanks,
> > > > >> > > > > > > >>
> > > > >> > > > > > > >> Jun
> > > > >> > > > > > > >>
> > > > >> > > > > > > >>
> > > > >> > > > > > > >> On Thu, Jan 4, 2018 at 4:22 PM, Dong Lin <
> > > > >> lindong28@gmail.com
> > > > >> > >
> > > > >> > > > > wrote:
> > > > >> > > > > > > >>
> > > > >> > > > > > > >> > Hey Jun,
> > > > >> > > > > > > >> >
> > > > >> > > > > > > >> > Thanks much. I like the the new API that you
> > > proposed.
> > > > I
> > > > >> am
> > > > >> > > not
> > > > >> > > > > sure
> > > > >> > > > > > > >> what
> > > > >> > > > > > > >> > you exactly mean by offset_epoch. I suppose that
> we
> > > can
> > > > >> use
> > > > >> > > the
> > > > >> > > > > pair
> > > > >> > > > > > > of
> > > > >> > > > > > > >> > (partition_epoch, leader_epoch) as the
> > offset_epoch,
> > > > >> right?
> > > > >> > > > > > > >> >
> > > > >> > > > > > > >> > Thanks,
> > > > >> > > > > > > >> > Dong
> > > > >> > > > > > > >> >
> > > > >> > > > > > > >> > On Thu, Jan 4, 2018 at 4:02 PM, Jun Rao <
> > > > >> jun@confluent.io>
> > > > >> > > > wrote:
> > > > >> > > > > > > >> >
> > > > >> > > > > > > >> > > Hi, Dong,
> > > > >> > > > > > > >> > >
> > > > >> > > > > > > >> > > Got it. The api that you proposed works. The
> > > question
> > > > >> is
> > > > >> > > > whether
> > > > >> > > > > > > >> that's
> > > > >> > > > > > > >> > the
> > > > >> > > > > > > >> > > api that we want to have in the long term. My
> > > concern
> > > > >> is
> > > > >> > > that
> > > > >> > > > > > while
> > > > >> > > > > > > >> the
> > > > >> > > > > > > >> > api
> > > > >> > > > > > > >> > > change is simple, the new api seems harder to
> > > explain
> > > > >> and
> > > > >> > > use.
> > > > >> > > > > For
> > > > >> > > > > > > >> > example,
> > > > >> > > > > > > >> > > a consumer storing offsets externally now needs
> > to
> > > > call
> > > > >> > > > > > > >> > > waitForMetadataUpdate() after calling seek().
> > > > >> > > > > > > >> > >
> > > > >> > > > > > > >> > > An alternative approach is to make the
> following
> > > > >> > compatible
> > > > >> > > > api
> > > > >> > > > > > > >> changes
> > > > >> > > > > > > >> > in
> > > > >> > > > > > > >> > > Consumer.
> > > > >> > > > > > > >> > > * Add an additional OffsetEpoch field in
> > > > >> > OffsetAndMetadata.
> > > > >> > > > (no
> > > > >> > > > > > need
> > > > >> > > > > > > >> to
> > > > >> > > > > > > >> > > change the CommitSync() api)
> > > > >> > > > > > > >> > > * Add a new api seek(TopicPartition partition,
> > long
> > > > >> > offset,
> > > > >> > > > > > > >> OffsetEpoch
> > > > >> > > > > > > >> > > offsetEpoch). We can potentially deprecate the
> > old
> > > > api
> > > > >> > > > > > > >> > seek(TopicPartition
> > > > >> > > > > > > >> > > partition, long offset) in the future.
> > > > >> > > > > > > >> > >
> > > > >> > > > > > > >> > > The alternative approach has similar amount of
> > api
> > > > >> changes
> > > > >> > > as
> > > > >> > > > > > yours
> > > > >> > > > > > > >> but
> > > > >> > > > > > > >> > has
> > > > >> > > > > > > >> > > the following benefits.
> > > > >> > > > > > > >> > > 1. The api works in a similar way as how offset
> > > > >> management
> > > > >> > > > works
> > > > >> > > > > > now
> > > > >> > > > > > > >> and
> > > > >> > > > > > > >> > is
> > > > >> > > > > > > >> > > probably what we want in the long term.
> > > > >> > > > > > > >> > > 2. It can reset offsets better when there is
> data
> > > > loss
> > > > >> due
> > > > >> > > to
> > > > >> > > > > > > unclean
> > > > >> > > > > > > >> > > leader election or correlated replica failure.
> > > > >> > > > > > > >> > > 3. It can reset offsets better when topic is
> > > > recreated.
> > > > >> > > > > > > >> > >
> > > > >> > > > > > > >> > > Thanks,
> > > > >> > > > > > > >> > >
> > > > >> > > > > > > >> > > Jun
> > > > >> > > > > > > >> > >
> > > > >> > > > > > > >> > >
> > > > >> > > > > > > >> > > On Thu, Jan 4, 2018 at 2:05 PM, Dong Lin <
> > > > >> > > lindong28@gmail.com
> > > > >> > > > >
> > > > >> > > > > > > wrote:
> > > > >> > > > > > > >> > >
> > > > >> > > > > > > >> > > > Hey Jun,
> > > > >> > > > > > > >> > > >
> > > > >> > > > > > > >> > > > Yeah I agree that ideally we don't want an
> ever
> > > > >> growing
> > > > >> > > > global
> > > > >> > > > > > > >> metadata
> > > > >> > > > > > > >> > > > version. I just think it may be more
> desirable
> > to
> > > > >> keep
> > > > >> > the
> > > > >> > > > > > > consumer
> > > > >> > > > > > > >> API
> > > > >> > > > > > > >> > > > simple.
> > > > >> > > > > > > >> > > >
> > > > >> > > > > > > >> > > > In my current proposal, metadata version
> > returned
> > > > in
> > > > >> the
> > > > >> > > > fetch
> > > > >> > > > > > > >> response
> > > > >> > > > > > > >> > > > will be stored with the offset together. More
> > > > >> > > specifically,
> > > > >> > > > > the
> > > > >> > > > > > > >> > > > metadata_epoch in the new offset topic schema
> > > will
> > > > be
> > > > >> > the
> > > > >> > > > > > largest
> > > > >> > > > > > > >> > > > metadata_epoch from all the MetadataResponse
> > and
> > > > >> > > > FetchResponse
> > > > >> > > > > > > ever
> > > > >> > > > > > > >> > > > received by this consumer.
> > > > >> > > > > > > >> > > >
> > > > >> > > > > > > >> > > > We probably don't have to change the consumer
> > API
> > > > for
> > > > >> > > > > > > >> > > > commitSync(Map<TopicPartition,
> > > OffsetAndMetadata>).
> > > > >> If
> > > > >> > > user
> > > > >> > > > > > calls
> > > > >> > > > > > > >> > > > commitSync(...) to commit offset 10 for a
> given
> > > > >> > partition,
> > > > >> > > > for
> > > > >> > > > > > > most
> > > > >> > > > > > > >> > > > use-cases, this consumer instance should have
> > > > >> consumed
> > > > >> > > > message
> > > > >> > > > > > > with
> > > > >> > > > > > > >> > > offset
> > > > >> > > > > > > >> > > > 9 from this partition, in which case the
> > consumer
> > > > can
> > > > >> > > > remember
> > > > >> > > > > > and
> > > > >> > > > > > > >> use
> > > > >> > > > > > > >> > > the
> > > > >> > > > > > > >> > > > metadata_epoch from the corresponding
> > > FetchResponse
> > > > >> when
> > > > >> > > > > > > committing
> > > > >> > > > > > > >> > > offset.
> > > > >> > > > > > > >> > > > If user calls commitSync(..) to commit offset
> > 10
> > > > for
> > > > >> a
> > > > >> > > given
> > > > >> > > > > > > >> partition
> > > > >> > > > > > > >> > > > without having consumed the message with
> > offset 9
> > > > >> using
> > > > >> > > this
> > > > >> > > > > > > >> consumer
> > > > >> > > > > > > >> > > > instance, this is probably an advanced
> > use-case.
> > > In
> > > > >> this
> > > > >> > > > case
> > > > >> > > > > > the
> > > > >> > > > > > > >> > > advanced
> > > > >> > > > > > > >> > > > user can retrieve the metadata_epoch using
> the
> > > > newly
> > > > >> > added
> > > > >> > > > > > > >> > > metadataEpoch()
> > > > >> > > > > > > >> > > > API after it fetches the message with offset
> 9
> > > > >> (probably
> > > > >> > > > from
> > > > >> > > > > > > >> another
> > > > >> > > > > > > >> > > > consumer instance) and encode this
> > metadata_epoch
> > > > in
> > > > >> the
> > > > >> > > > > > > >> > > > string OffsetAndMetadata.metadata. Do you
> think
> > > > this
> > > > >> > > > solution
> > > > >> > > > > > > would
> > > > >> > > > > > > >> > work?
> > > > >> > > > > > > >> > > >
> > > > >> > > > > > > >> > > > By "not sure that I fully understand your
> > latest
> > > > >> > > > suggestion",
> > > > >> > > > > > are
> > > > >> > > > > > > >> you
> > > > >> > > > > > > >> > > > referring to solution related to unclean
> leader
> > > > >> election
> > > > >> > > > using
> > > > >> > > > > > > >> > > leader_epoch
> > > > >> > > > > > > >> > > > in my previous email?
> > > > >> > > > > > > >> > > >
> > > > >> > > > > > > >> > > > Thanks,
> > > > >> > > > > > > >> > > > Dong
> > > > >> > > > > > > >> > > >
> > > > >> > > > > > > >> > > > On Thu, Jan 4, 2018 at 1:33 PM, Jun Rao <
> > > > >> > jun@confluent.io
> > > > >> > > >
> > > > >> > > > > > wrote:
> > > > >> > > > > > > >> > > >
> > > > >> > > > > > > >> > > > > Hi, Dong,
> > > > >> > > > > > > >> > > > >
> > > > >> > > > > > > >> > > > > Not sure that I fully understand your
> latest
> > > > >> > suggestion.
> > > > >> > > > > > > >> Returning an
> > > > >> > > > > > > >> > > > ever
> > > > >> > > > > > > >> > > > > growing global metadata version itself is
> no
> > > > ideal,
> > > > >> > but
> > > > >> > > is
> > > > >> > > > > > fine.
> > > > >> > > > > > > >> My
> > > > >> > > > > > > >> > > > > question is whether the metadata version
> > > returned
> > > > >> in
> > > > >> > the
> > > > >> > > > > fetch
> > > > >> > > > > > > >> > response
> > > > >> > > > > > > >> > > > > needs to be stored with the offset together
> > if
> > > > >> offsets
> > > > >> > > are
> > > > >> > > > > > > stored
> > > > >> > > > > > > >> > > > > externally. If so, we also have to change
> the
> > > > >> consumer
> > > > >> > > API
> > > > >> > > > > for
> > > > >> > > > > > > >> > > > commitSync()
> > > > >> > > > > > > >> > > > > and need to worry about compatibility. If
> we
> > > > don't
> > > > >> > store
> > > > >> > > > the
> > > > >> > > > > > > >> metadata
> > > > >> > > > > > > >> > > > > version together with the offset, on a
> > consumer
> > > > >> > restart,
> > > > >> > > > > it's
> > > > >> > > > > > > not
> > > > >> > > > > > > >> > clear
> > > > >> > > > > > > >> > > > how
> > > > >> > > > > > > >> > > > > we can ensure the metadata in the consumer
> is
> > > > high
> > > > >> > > enough
> > > > >> > > > > > since
> > > > >> > > > > > > >> there
> > > > >> > > > > > > >> > > is
> > > > >> > > > > > > >> > > > no
> > > > >> > > > > > > >> > > > > metadata version to compare with.
> > > > >> > > > > > > >> > > > >
> > > > >> > > > > > > >> > > > > Thanks,
> > > > >> > > > > > > >> > > > >
> > > > >> > > > > > > >> > > > > Jun
> > > > >> > > > > > > >> > > > >
> > > > >> > > > > > > >> > > > >
> > > > >> > > > > > > >> > > > > On Wed, Jan 3, 2018 at 6:43 PM, Dong Lin <
> > > > >> > > > > lindong28@gmail.com
> > > > >> > > > > > >
> > > > >> > > > > > > >> > wrote:
> > > > >> > > > > > > >> > > > >
> > > > >> > > > > > > >> > > > > > Hey Jun,
> > > > >> > > > > > > >> > > > > >
> > > > >> > > > > > > >> > > > > > Thanks much for the explanation.
> > > > >> > > > > > > >> > > > > >
> > > > >> > > > > > > >> > > > > > I understand the advantage of
> > partition_epoch
> > > > >> over
> > > > >> > > > > > > >> metadata_epoch.
> > > > >> > > > > > > >> > My
> > > > >> > > > > > > >> > > > > > current concern is that the use of
> > > leader_epoch
> > > > >> and
> > > > >> > > the
> > > > >> > > > > > > >> > > partition_epoch
> > > > >> > > > > > > >> > > > > > requires us considerable change to
> > consumer's
> > > > >> public
> > > > >> > > API
> > > > >> > > > > to
> > > > >> > > > > > > take
> > > > >> > > > > > > >> > care
> > > > >> > > > > > > >> > > > of
> > > > >> > > > > > > >> > > > > > the case where user stores offset
> > externally.
> > > > For
> > > > >> > > > example,
> > > > >> > > > > > > >> > > *consumer*.
> > > > >> > > > > > > >> > > > > > *commitSync*(..) would have to take a map
> > > whose
> > > > >> > value
> > > > >> > > is
> > > > >> > > > > > > >> <offset,
> > > > >> > > > > > > >> > > > > metadata,
> > > > >> > > > > > > >> > > > > > leader epoch, partition epoch>.
> > > > >> > *consumer*.*seek*(...)
> > > > >> > > > > would
> > > > >> > > > > > > >> also
> > > > >> > > > > > > >> > > need
> > > > >> > > > > > > >> > > > > > leader_epoch and partition_epoch as
> > > parameter.
> > > > >> > > > Technically
> > > > >> > > > > > we
> > > > >> > > > > > > >> can
> > > > >> > > > > > > >> > > > > probably
> > > > >> > > > > > > >> > > > > > still make it work in a backward
> compatible
> > > > >> manner
> > > > >> > > after
> > > > >> > > > > > > careful
> > > > >> > > > > > > >> > > design
> > > > >> > > > > > > >> > > > > and
> > > > >> > > > > > > >> > > > > > discussion. But these changes can make
> the
> > > > >> > consumer's
> > > > >> > > > > > > interface
> > > > >> > > > > > > >> > > > > > unnecessarily complex for more users who
> do
> > > not
> > > > >> > store
> > > > >> > > > > offset
> > > > >> > > > > > > >> > > > externally.
> > > > >> > > > > > > >> > > > > >
> > > > >> > > > > > > >> > > > > > After thinking more about it, we can
> > address
> > > > all
> > > > >> > > > problems
> > > > >> > > > > > > >> discussed
> > > > >> > > > > > > >> > > by
> > > > >> > > > > > > >> > > > > only
> > > > >> > > > > > > >> > > > > > using the metadata_epoch without
> > introducing
> > > > >> > > > leader_epoch
> > > > >> > > > > or
> > > > >> > > > > > > the
> > > > >> > > > > > > >> > > > > > partition_epoch. The current KIP
> describes
> > > the
> > > > >> > changes
> > > > >> > > > to
> > > > >> > > > > > the
> > > > >> > > > > > > >> > > consumer
> > > > >> > > > > > > >> > > > > API
> > > > >> > > > > > > >> > > > > > and how the new API can be used if user
> > > stores
> > > > >> > offset
> > > > >> > > > > > > >> externally.
> > > > >> > > > > > > >> > In
> > > > >> > > > > > > >> > > > > order
> > > > >> > > > > > > >> > > > > > to address the scenario you described
> > > earlier,
> > > > we
> > > > >> > can
> > > > >> > > > > > include
> > > > >> > > > > > > >> > > > > > metadata_epoch in the FetchResponse and
> the
> > > > >> > > > > > > LeaderAndIsrRequest.
> > > > >> > > > > > > >> > > > Consumer
> > > > >> > > > > > > >> > > > > > remembers the largest metadata_epoch from
> > all
> > > > the
> > > > >> > > > > > > FetchResponse
> > > > >> > > > > > > >> it
> > > > >> > > > > > > >> > > has
> > > > >> > > > > > > >> > > > > > received. The metadata_epoch committed
> with
> > > the
> > > > >> > > offset,
> > > > >> > > > > > either
> > > > >> > > > > > > >> > within
> > > > >> > > > > > > >> > > > or
> > > > >> > > > > > > >> > > > > > outside Kafka, should be the largest
> > > > >> metadata_epoch
> > > > >> > > > across
> > > > >> > > > > > all
> > > > >> > > > > > > >> > > > > > FetchResponse and MetadataResponse ever
> > > > received
> > > > >> by
> > > > >> > > this
> > > > >> > > > > > > >> consumer.
> > > > >> > > > > > > >> > > > > >
> > > > >> > > > > > > >> > > > > > The drawback of using only the
> > metadata_epoch
> > > > is
> > > > >> > that
> > > > >> > > we
> > > > >> > > > > can
> > > > >> > > > > > > not
> > > > >> > > > > > > >> > > always
> > > > >> > > > > > > >> > > > > do
> > > > >> > > > > > > >> > > > > > the smart offset reset in case of unclean
> > > > leader
> > > > >> > > > election
> > > > >> > > > > > > which
> > > > >> > > > > > > >> you
> > > > >> > > > > > > >> > > > > > mentioned earlier. But in most case,
> > unclean
> > > > >> leader
> > > > >> > > > > election
> > > > >> > > > > > > >> > probably
> > > > >> > > > > > > >> > > > > > happens when consumer is not
> > > > >> rebalancing/restarting.
> > > > >> > > In
> > > > >> > > > > > these
> > > > >> > > > > > > >> > cases,
> > > > >> > > > > > > >> > > > > either
> > > > >> > > > > > > >> > > > > > consumer is not directly affected by
> > unclean
> > > > >> leader
> > > > >> > > > > election
> > > > >> > > > > > > >> since
> > > > >> > > > > > > >> > it
> > > > >> > > > > > > >> > > > is
> > > > >> > > > > > > >> > > > > > not consuming from the end of the log, or
> > > > >> consumer
> > > > >> > can
> > > > >> > > > > > derive
> > > > >> > > > > > > >> the
> > > > >> > > > > > > >> > > > > > leader_epoch from the most recent message
> > > > >> received
> > > > >> > > > before
> > > > >> > > > > it
> > > > >> > > > > > > >> sees
> > > > >> > > > > > > >> > > > > > OffsetOutOfRangeException. So I am not
> sure
> > > it
> > > > is
> > > > >> > > worth
> > > > >> > > > > > adding
> > > > >> > > > > > > >> the
> > > > >> > > > > > > >> > > > > > leader_epoch to consumer API to address
> the
> > > > >> > remaining
> > > > >> > > > > corner
> > > > >> > > > > > > >> case.
> > > > >> > > > > > > >> > > What
> > > > >> > > > > > > >> > > > > do
> > > > >> > > > > > > >> > > > > > you think?
> > > > >> > > > > > > >> > > > > >
> > > > >> > > > > > > >> > > > > > Thanks,
> > > > >> > > > > > > >> > > > > > Dong
> > > > >> > > > > > > >> > > > > >
> > > > >> > > > > > > >> > > > > >
> > > > >> > > > > > > >> > > > > >
> > > > >> > > > > > > >> > > > > > On Tue, Jan 2, 2018 at 6:28 PM, Jun Rao <
> > > > >> > > > jun@confluent.io
> > > > >> > > > > >
> > > > >> > > > > > > >> wrote:
> > > > >> > > > > > > >> > > > > >
> > > > >> > > > > > > >> > > > > > > Hi, Dong,
> > > > >> > > > > > > >> > > > > > >
> > > > >> > > > > > > >> > > > > > > Thanks for the reply.
> > > > >> > > > > > > >> > > > > > >
> > > > >> > > > > > > >> > > > > > > To solve the topic recreation issue, we
> > > could
> > > > >> use
> > > > >> > > > > either a
> > > > >> > > > > > > >> global
> > > > >> > > > > > > >> > > > > > metadata
> > > > >> > > > > > > >> > > > > > > version or a partition level epoch. But
> > > > either
> > > > >> one
> > > > >> > > > will
> > > > >> > > > > > be a
> > > > >> > > > > > > >> new
> > > > >> > > > > > > >> > > > > concept,
> > > > >> > > > > > > >> > > > > > > right? To me, the latter seems more
> > > natural.
> > > > It
> > > > >> > also
> > > > >> > > > > makes
> > > > >> > > > > > > it
> > > > >> > > > > > > >> > > easier
> > > > >> > > > > > > >> > > > to
> > > > >> > > > > > > >> > > > > > > detect if a consumer's offset is still
> > > valid
> > > > >> > after a
> > > > >> > > > > topic
> > > > >> > > > > > > is
> > > > >> > > > > > > >> > > > > recreated.
> > > > >> > > > > > > >> > > > > > As
> > > > >> > > > > > > >> > > > > > > you pointed out, we don't need to store
> > the
> > > > >> > > partition
> > > > >> > > > > > epoch
> > > > >> > > > > > > in
> > > > >> > > > > > > >> > the
> > > > >> > > > > > > >> > > > > > message.
> > > > >> > > > > > > >> > > > > > > The following is what I am thinking.
> > When a
> > > > >> > > partition
> > > > >> > > > is
> > > > >> > > > > > > >> created,
> > > > >> > > > > > > >> > > we
> > > > >> > > > > > > >> > > > > can
> > > > >> > > > > > > >> > > > > > > assign a partition epoch from an
> > > > >> ever-increasing
> > > > >> > > > global
> > > > >> > > > > > > >> counter
> > > > >> > > > > > > >> > and
> > > > >> > > > > > > >> > > > > store
> > > > >> > > > > > > >> > > > > > > it in /brokers/topics/[topic]/
> > > > >> > > > partitions/[partitionId]
> > > > >> > > > > in
> > > > >> > > > > > > ZK.
> > > > >> > > > > > > >> > The
> > > > >> > > > > > > >> > > > > > > partition
> > > > >> > > > > > > >> > > > > > > epoch is propagated to every broker.
> The
> > > > >> consumer
> > > > >> > > will
> > > > >> > > > > be
> > > > >> > > > > > > >> > tracking
> > > > >> > > > > > > >> > > a
> > > > >> > > > > > > >> > > > > > tuple
> > > > >> > > > > > > >> > > > > > > of <offset, leader epoch, partition
> > epoch>
> > > > for
> > > > >> > > > offsets.
> > > > >> > > > > > If a
> > > > >> > > > > > > >> > topic
> > > > >> > > > > > > >> > > is
> > > > >> > > > > > > >> > > > > > > recreated, it's possible that a
> > consumer's
> > > > >> offset
> > > > >> > > and
> > > > >> > > > > > leader
> > > > >> > > > > > > >> > epoch
> > > > >> > > > > > > >> > > > > still
> > > > >> > > > > > > >> > > > > > > match that in the broker, but partition
> > > epoch
> > > > >> > won't
> > > > >> > > > be.
> > > > >> > > > > In
> > > > >> > > > > > > >> this
> > > > >> > > > > > > >> > > case,
> > > > >> > > > > > > >> > > > > we
> > > > >> > > > > > > >> > > > > > > can potentially still treat the
> > consumer's
> > > > >> offset
> > > > >> > as
> > > > >> > > > out
> > > > >> > > > > > of
> > > > >> > > > > > > >> range
> > > > >> > > > > > > >> > > and
> > > > >> > > > > > > >> > > > > > reset
> > > > >> > > > > > > >> > > > > > > the offset based on the offset reset
> > policy
> > > > in
> > > > >> the
> > > > >> > > > > > consumer.
> > > > >> > > > > > > >> This
> > > > >> > > > > > > >> > > > seems
> > > > >> > > > > > > >> > > > > > > harder to do with a global metadata
> > > version.
> > > > >> > > > > > > >> > > > > > >
> > > > >> > > > > > > >> > > > > > > Jun
> > > > >> > > > > > > >> > > > > > >
> > > > >> > > > > > > >> > > > > > >
> > > > >> > > > > > > >> > > > > > >
> > > > >> > > > > > > >> > > > > > > On Mon, Dec 25, 2017 at 6:56 AM, Dong
> > Lin <
> > > > >> > > > > > > >> lindong28@gmail.com>
> > > > >> > > > > > > >> > > > wrote:
> > > > >> > > > > > > >> > > > > > >
> > > > >> > > > > > > >> > > > > > > > Hey Jun,
> > > > >> > > > > > > >> > > > > > > >
> > > > >> > > > > > > >> > > > > > > > This is a very good example. After
> > > thinking
> > > > >> > > through
> > > > >> > > > > this
> > > > >> > > > > > > in
> > > > >> > > > > > > >> > > > detail, I
> > > > >> > > > > > > >> > > > > > > agree
> > > > >> > > > > > > >> > > > > > > > that we need to commit offset with
> > leader
> > > > >> epoch
> > > > >> > in
> > > > >> > > > > order
> > > > >> > > > > > > to
> > > > >> > > > > > > >> > > address
> > > > >> > > > > > > >> > > > > > this
> > > > >> > > > > > > >> > > > > > > > example.
> > > > >> > > > > > > >> > > > > > > >
> > > > >> > > > > > > >> > > > > > > > I think the remaining question is how
> > to
> > > > >> address
> > > > >> > > the
> > > > >> > > > > > > >> scenario
> > > > >> > > > > > > >> > > that
> > > > >> > > > > > > >> > > > > the
> > > > >> > > > > > > >> > > > > > > > topic is deleted and re-created. One
> > > > possible
> > > > >> > > > solution
> > > > >> > > > > > is
> > > > >> > > > > > > to
> > > > >> > > > > > > >> > > commit
> > > > >> > > > > > > >> > > > > > > offset
> > > > >> > > > > > > >> > > > > > > > with both the leader epoch and the
> > > metadata
> > > > >> > > version.
> > > > >> > > > > The
> > > > >> > > > > > > >> logic
> > > > >> > > > > > > >> > > and
> > > > >> > > > > > > >> > > > > the
> > > > >> > > > > > > >> > > > > > > > implementation of this solution does
> > not
> > > > >> > require a
> > > > >> > > > new
> > > > >> > > > > > > >> concept
> > > > >> > > > > > > >> > > > (e.g.
> > > > >> > > > > > > >> > > > > > > > partition epoch) and it does not
> > require
> > > > any
> > > > >> > > change
> > > > >> > > > to
> > > > >> > > > > > the
> > > > >> > > > > > > >> > > message
> > > > >> > > > > > > >> > > > > > format
> > > > >> > > > > > > >> > > > > > > > or leader epoch. It also allows us to
> > > order
> > > > >> the
> > > > >> > > > > metadata
> > > > >> > > > > > > in
> > > > >> > > > > > > >> a
> > > > >> > > > > > > >> > > > > > > > straightforward manner which may be
> > > useful
> > > > in
> > > > >> > the
> > > > >> > > > > > future.
> > > > >> > > > > > > >> So it
> > > > >> > > > > > > >> > > may
> > > > >> > > > > > > >> > > > > be
> > > > >> > > > > > > >> > > > > > a
> > > > >> > > > > > > >> > > > > > > > better solution than generating a
> > random
> > > > >> > partition
> > > > >> > > > > epoch
> > > > >> > > > > > > >> every
> > > > >> > > > > > > >> > > time
> > > > >> > > > > > > >> > > > > we
> > > > >> > > > > > > >> > > > > > > > create a partition. Does this sound
> > > > >> reasonable?
> > > > >> > > > > > > >> > > > > > > >
> > > > >> > > > > > > >> > > > > > > > Previously one concern with using the
> > > > >> metadata
> > > > >> > > > version
> > > > >> > > > > > is
> > > > >> > > > > > > >> that
> > > > >> > > > > > > >> > > > > consumer
> > > > >> > > > > > > >> > > > > > > > will be forced to refresh metadata
> even
> > > if
> > > > >> > > metadata
> > > > >> > > > > > > version
> > > > >> > > > > > > >> is
> > > > >> > > > > > > >> > > > > > increased
> > > > >> > > > > > > >> > > > > > > > due to topics that the consumer is
> not
> > > > >> > interested
> > > > >> > > > in.
> > > > >> > > > > > Now
> > > > >> > > > > > > I
> > > > >> > > > > > > >> > > > realized
> > > > >> > > > > > > >> > > > > > that
> > > > >> > > > > > > >> > > > > > > > this is probably not a problem.
> > Currently
> > > > >> client
> > > > >> > > > will
> > > > >> > > > > > > >> refresh
> > > > >> > > > > > > >> > > > > metadata
> > > > >> > > > > > > >> > > > > > > > either due to
> InvalidMetadataException
> > in
> > > > the
> > > > >> > > > response
> > > > >> > > > > > > from
> > > > >> > > > > > > >> > > broker
> > > > >> > > > > > > >> > > > or
> > > > >> > > > > > > >> > > > > > due
> > > > >> > > > > > > >> > > > > > > > to metadata expiry. The addition of
> the
> > > > >> metadata
> > > > >> > > > > version
> > > > >> > > > > > > >> should
> > > > >> > > > > > > >> > > > > > increase
> > > > >> > > > > > > >> > > > > > > > the overhead of metadata refresh
> caused
> > > by
> > > > >> > > > > > > >> > > > InvalidMetadataException.
> > > > >> > > > > > > >> > > > > If
> > > > >> > > > > > > >> > > > > > > > client refresh metadata due to expiry
> > and
> > > > it
> > > > >> > > > receives
> > > > >> > > > > a
> > > > >> > > > > > > >> > metadata
> > > > >> > > > > > > >> > > > > whose
> > > > >> > > > > > > >> > > > > > > > version is lower than the current
> > > metadata
> > > > >> > > version,
> > > > >> > > > we
> > > > >> > > > > > can
> > > > >> > > > > > > >> > reject
> > > > >> > > > > > > >> > > > the
> > > > >> > > > > > > >> > > > > > > > metadata but still reset the metadata
> > > age,
> > > > >> which
> > > > >> > > > > > > essentially
> > > > >> > > > > > > >> > keep
> > > > >> > > > > > > >> > > > the
> > > > >> > > > > > > >> > > > > > > > existing behavior in the client.
> > > > >> > > > > > > >> > > > > > > >
> > > > >> > > > > > > >> > > > > > > > Thanks much,
> > > > >> > > > > > > >> > > > > > > > Dong
> > > > >> > > > > > > >> > > > > > > >
> > > > >> > > > > > > >> > > > > > >
> > > > >> > > > > > > >> > > > > >
> > > > >> > > > > > > >> > > > >
> > > > >> > > > > > > >> > > >
> > > > >> > > > > > > >> > >
> > > > >> > > > > > > >> >
> > > > >> > > > > > > >>
> > > > >> > > > > > > >
> > > > >> > > > > > > >
> > > > >> > > > > > >
> > > > >> > > > > >
> > > > >> > > > >
> > > > >> > > >
> > > > >> > >
> > > > >> >
> > > > >>
> > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Jun Rao <ju...@confluent.io>.
Hi, Dong,

Thanks for the updated KIP. Looks good to me overall. Just a few minor
comments.

60. OffsetAndMetadata positionAndOffsetEpoch(TopicPartition partition): It
seems that there is no need to return metadata. We probably want to return
sth like OffsetAndEpoch.

61. Should we store partition_epoch in
/brokers/topics/[topic]/partitions/[partitionId] in ZK?

62. For checking outdated metadata in the client, we probably want to add
when max_partition_epoch will be used.

63. "The leader_epoch should be the largest leader_epoch of messages whose
offset < the commit offset. If no message has been consumed since consumer
initialization, the leader_epoch from seek(...) or OffsetFetchResponse
should be used. The partition_epoch should be read from the last
FetchResponse corresponding to the given partition and commit offset. ":
leader_epoch and partition_epoch are associated with an offset. So, if no
message is consumed, there is no offset and therefore there is no need to
read leader_epoch and partition_epoch. Also, the leader_epoch associated
with the offset should just come from the messages returned in the fetch
response.

64. Could you include the public methods in the OffsetEpoch class?

Jun


On Thu, Jan 11, 2018 at 5:43 PM, Dong Lin <li...@gmail.com> wrote:

> Hey Jun,
>
> Thanks much. I agree that we can not rely on committed offsets to be always
> deleted when we delete topic. So it is necessary to use a per-partition
> epoch that does not change unless this partition is deleted. I also agree
> that it is very nice to be able to uniquely identify a message with
> (offset, leader_epoch, partition_epoch) in face of potential topic deletion
> and unclean leader election.
>
> I agree with all your comments. And I have updated the KIP based on our
> latest discussion. In addition, I added InvalidPartitionEpochException
> which will be thrown by consumer.poll() if the partition_epoch associated
> with the partition, which can be given to consumer using seek(...), is
> different from the partition_epoch in the FetchResponse.
>
> Can you take another look at the latest KIP?
>
> Thanks!
> Dong
>
>
>
> On Wed, Jan 10, 2018 at 2:24 PM, Jun Rao <ju...@confluent.io> wrote:
>
> > Hi, Dong,
> >
> > My replies are the following.
> >
> > 60. What you described could also work. The drawback is that we will be
> > unnecessarily changing the partition epoch when a partition hasn't really
> > changed. I was imagining that the partition epoch will be stored in
> > /brokers/topics/[topic]/partitions/[partitionId], instead of at the
> topic
> > level. So, not sure if ZK size limit is an issue.
> >
> > 61, 62 and 65. To me, the offset + offset_epoch is a unique identifier
> for
> > a message. So, if a message hasn't changed, the offset and the associated
> > offset_epoch ideally should remain the same (it will be kind of weird if
> > two consumer apps save the offset on the same message, but the
> offset_epoch
> > are different). partition_epoch + leader_epoch give us that.
> global_epoch +
> > leader_epoch don't. If we use this approach, we can solve not only the
> > problem that you have identified, but also other problems when there is
> > data loss or topic re-creation more reliably. For example, in the future,
> > if we include the partition_epoch and leader_epoch in the fetch request,
> > the server can do a more reliable check of whether that offset is valid
> or
> > not. I am not sure that we can rely upon all external offsets to be
> removed
> > on topic deletion. For example, a topic may be deleted by an admin who
> may
> > not know all the applications.
> >
> > If we agree on the above, the second question is then how to reliably
> > propagate the partition_epoch and the leader_epoch to the consumer when
> > there are leader or partition changes. The leader_epoch comes from the
> > message, which is reliable. So, I was suggesting that when we store an
> > offset, we can just store the leader_epoch from the message set
> containing
> > that offset. Similarly, I was thinking that if the partition_epoch is in
> > the fetch response, we can propagate partition_epoch reliably where is
> > partition_epoch change.
> >
> > 63. My point is that once a leader is producing a message in the new
> > partition_epoch, ideally, we should associate the new offsets with the
> new
> > partition_epoch. Otherwise, the offset_epoch won't be the correct unique
> > identifier (useful for solving other problems mentioned above). I was
> > originally thinking that the leader will include the partition_epoch in
> the
> > metadata cache in the fetch response. It's just that right now, metadata
> > cache is updated on UpdateMetadataRequest, which typically happens after
> > the LeaderAndIsrRequest. Another approach is for the leader to cache the
> > partition_epoch in the Partition object and return that (instead of the
> one
> > in metadata cache) in the fetch response.
> >
> > 65. It seems to me that the global_epoch and the partition_epoch have
> > different purposes. A partition_epoch has the benefit that it (1) can be
> > used to form a unique identifier for a message and (2) can be used to
> > solve other
> > corner case problems in the future. I am not sure having just a
> > global_epoch can achieve these. global_epoch is useful to determine which
> > version of the metadata is newer, especially with topic deletion.
> >
> > Thanks,
> >
> > Jun
> >
> > On Tue, Jan 9, 2018 at 11:34 PM, Dong Lin <li...@gmail.com> wrote:
> >
> > > Regarding the use of the global epoch in 65), it is very similar to the
> > > proposal of the metadata_epoch we discussed earlier. The main
> difference
> > is
> > > that this epoch is incremented when we create/expand/delete topic and
> > does
> > > not change when controller re-send metadata.
> > >
> > > I looked at our previous discussion. It seems that we prefer
> > > partition_epoch over the metadata_epoch because 1) we prefer not to
> have
> > an
> > > ever growing metadata_epoch and 2) we can reset offset better when
> topic
> > is
> > > re-created. The use of global topic_epoch avoids the drawback of an
> ever
> > > quickly ever growing metadata_epoch. Though the global epoch does not
> > allow
> > > us to recognize the invalid offset committed before the topic
> > re-creation,
> > > we can probably just delete the offset when we delete a topic. Thus I
> am
> > > not very sure whether it is still worthwhile to have a per-partition
> > > partition_epoch if the metadata already has the global epoch.
> > >
> > >
> > > On Tue, Jan 9, 2018 at 6:58 PM, Dong Lin <li...@gmail.com> wrote:
> > >
> > > > Hey Jun,
> > > >
> > > > Thanks so much. These comments very useful. Please see below my
> > comments.
> > > >
> > > > On Mon, Jan 8, 2018 at 5:52 PM, Jun Rao <ju...@confluent.io> wrote:
> > > >
> > > >> Hi, Dong,
> > > >>
> > > >> Thanks for the updated KIP. A few more comments.
> > > >>
> > > >> 60. Perhaps having a partition epoch is more flexible since in the
> > > future,
> > > >> we may support deleting a partition as well.
> > > >>
> > > >
> > > > Yeah I have considered this. I think we can probably still support
> > > > deleting a partition by using the topic_epoch -- when partition of a
> > > topic
> > > > is deleted or created, epoch of all partitions of this topic will be
> > > > incremented by 1. Therefore, if that partition is re-created later,
> the
> > > > epoch of that partition will still be larger than its epoch before
> the
> > > > deletion, which still allows the client to order the metadata for the
> > > > purpose of this KIP. Does this sound reasonable?
> > > >
> > > > The advantage of using topic_epoch instead of partition_epoch is that
> > the
> > > > size of the /brokers/topics/[topic] znode and request/response size
> can
> > > be
> > > > smaller. We have a limit on the maximum size of znode (typically
> 1MB).
> > > Use
> > > > partition epoch can effectively reduce the number of partitions that
> > can
> > > be
> > > > described by the /brokers/topics/[topic] znode.
> > > >
> > > > One use-case of partition_epoch for client to detect that the
> committed
> > > > offset, either from kafka offset topic or from the external store is
> > > > invalid after partition deletion and re-creation. However, it seems
> > that
> > > we
> > > > can also address this use-case with other approaches. For example,
> when
> > > > AdminClient deletes partitions, it can also delete the committed
> > offsets
> > > > for those partitions from the offset topic. If user stores offset
> > > > externally, it might make sense for user to similarly remove offsets
> of
> > > > related partitions after these partitions are deleted. So I am not
> sure
> > > > that we should use partition_epoch in this KIP.
> > > >
> > > >
> > > >>
> > > >> 61. It seems that the leader epoch returned in the position() call
> > > should
> > > >> the the leader epoch returned in the fetch response, not the one in
> > the
> > > >> metadata cache of the client.
> > > >
> > > >
> > > > I think this is a good idea. Just to double check, this change does
> not
> > > > affect the correctness or performance of this KIP. But it can be
> useful
> > > if
> > > > we want to use the leader_epoch to better handle the offset rest in
> > case
> > > of
> > > > unclean leader election, which is listed in the future work. Is this
> > > > understanding correct?
> > > >
> > > > I have updated the KIP to specify that the leader_epoch returned by
> > > > position() should be the largest leader_epoch of those already
> consumed
> > > > messages whose offset < position. If no message has been consumed
> since
> > > > consumer initialization, the leader_epoch from seek() or
> > > > OffsetFetchResponse should be used. The offset included in the
> > > > OffsetCommitRequest will also be determined in the similar manner.
> > > >
> > > >
> > > >>
> > > >> 62. I am wondering if we should return the partition epoch in the
> > fetch
> > > >> response as well. In the current proposal, if a topic is recreated
> and
> > > the
> > > >> new leader is on the same broker as the old one, there is nothing to
> > > force
> > > >> the metadata refresh in the client. So, the client may still
> associate
> > > the
> > > >> offset with the old partition epoch.
> > > >>
> > > >
> > > > Could you help me understand the problem if a client associates old
> > > > partition_epoch (or the topic_epoch as of the current KIP) with the
> > > offset?
> > > > The main purpose of the topic_epoch is to be able to drop
> leader_epoch
> > > to 0
> > > > after a partition is deleted and re-created. I guess you may be
> > thinking
> > > > about using the partition_epoch to detect that the committed offset
> is
> > > > invalid? In that case, I am wondering if the alternative approach
> > > described
> > > > in 60) would be reasonable.
> > > >
> > > >
> > > >>
> > > >> 63. There is some subtle coordination between the
> LeaderAndIsrRequest
> > > and
> > > >> UpdateMetadataRequest. Currently, when a leader changes, the
> > controller
> > > >> first sends the LeaderAndIsrRequest to the assigned replicas and the
> > > >> UpdateMetadataRequest to every broker. So, there could be a small
> > window
> > > >> when the leader already receives the new partition epoch in the
> > > >> LeaderAndIsrRequest, but the metadata cache in the broker hasn't
> been
> > > >> updated with the latest partition epoch. Not sure what's the best
> way
> > to
> > > >> address this issue. Perhaps we can update the metadata cache on the
> > > broker
> > > >> with both LeaderAndIsrRequest and UpdateMetadataRequest. The
> challenge
> > > is
> > > >> that the two have slightly different data. For example, only the
> > latter
> > > >> has
> > > >> all endpoints.
> > > >>
> > > >
> > > > I am not sure whether this is a problem. Could you explain a bit more
> > > what
> > > > specific problem this small window can cause?
> > > >
> > > > Since client can fetch metadata from any broker in the cluster, and
> > given
> > > > that different brokers receive request (e.g. LeaderAndIsrRequest and
> > > > UpdateMetadataRequest) in arbitrary order, the metadata received by
> > > client
> > > > can be in arbitrary order (either newer or older) compared to the
> > > broker's
> > > > leadership state even if a given broker receives LeaderAndIsrRequest
> > and
> > > > UpdateMetadataRequest simultaneously. So I am not sure it is useful
> to
> > > > update broker's cache with LeaderAndIsrRequest.
> > > >
> > > >
> > > >> 64. The enforcement of leader epoch in Offset commit: We allow a
> > > consumer
> > > >> to set an arbitrary offset. So it's possible for offsets or leader
> > epoch
> > > >> to
> > > >> go backwards. I am not sure if we could always enforce that the
> leader
> > > >> epoch only goes up on the broker.
> > > >>
> > > >
> > > > Sure. I have removed this check from the KIP.
> > > >
> > > > BTW, we can probably still ensure that the leader_epoch always
> increase
> > > if
> > > > the leader_epoch used with offset commit is the max(leader_epoch of
> the
> > > > message with offset = the committed offset - 1, the largest known
> > > > leader_epoch from the metadata). But I don't have a good use-case for
> > > this
> > > > alternative definition. So I choose the keep the KIP simple by
> > requiring
> > > > leader_epoch to always increase.
> > > >
> > > >
> > > >> 65. Good point on handling missing partition epoch due to topic
> > > deletion.
> > > >> Another potential way to address this is to additionally propagate
> the
> > > >> global partition epoch to brokers and the clients. This way, when a
> > > >> partition epoch is missing, we can use the global partition epoch to
> > > >> reason
> > > >> about which metadata is more recent.
> > > >>
> > > >
> > > > This is a great idea. The global epoch can be used to order the
> > metadata
> > > > and help us recognize the more recent metadata if a topic (or
> > partition)
> > > is
> > > > deleted and re-created.
> > > >
> > > > Actually, it seems we only need to propagate the global epoch to
> > brokers
> > > > and clients without propagating this epoch on a per-topic or
> > > per-partition
> > > > basic. Doing so would simply interface changes made this KIP. Does
> this
> > > > approach sound reasonable?
> > > >
> > > >
> > > >> 66. A client may also get an offset by time using the
> offsetForTimes()
> > > >> api.
> > > >> So, we probably want to include offsetInternalMetadata in
> > > >> OffsetAndTimestamp
> > > >> as well.
> > > >>
> > > >
> > > > You are right. This probably also requires us to change the
> > > > ListOffsetRequest as well. I will update the KIP after we agree on
> the
> > > > solution for 65).
> > > >
> > > >
> > > >>
> > > >> 67. InteralMetadata can be a bit confusing with the metadata field
> > > already
> > > >> there. Perhaps we can just call it OffsetEpoch. It might be useful
> to
> > > make
> > > >> OffsetEpoch printable at least for debugging purpose. Once you do
> > that,
> > > we
> > > >> are already exposing the internal fields. So, not sure if it's worth
> > > >> hiding
> > > >> them. If we do want to hide them, perhaps we can have sth like the
> > > >> following. The binary encoding is probably more efficient than JSON
> > for
> > > >> external storage.
> > > >>
> > > >> OffsetEpoch {
> > > >>  static OffsetEpoch decode(byte[]);
> > > >>
> > > >>   public byte[] encode();
> > > >>
> > > >>   public String toString();
> > > >> }
> > > >>
> > > >
> > > > Thanks much. I like this solution. I have updated the KIP
> accordingly.
> > > >
> > > >
> > > >
> > > >>
> > > >> Jun
> > > >>
> > > >> On Mon, Jan 8, 2018 at 4:22 PM, Dong Lin <li...@gmail.com>
> wrote:
> > > >>
> > > >> > Hey Jason,
> > > >> >
> > > >> > Certainly. This sounds good. I have updated the KIP to clarity
> that
> > > the
> > > >> > global epoch will be incremented by 1 each time a topic is
> deleted.
> > > >> >
> > > >> > Thanks,
> > > >> > Dong
> > > >> >
> > > >> > On Mon, Jan 8, 2018 at 4:09 PM, Jason Gustafson <
> jason@confluent.io
> > >
> > > >> > wrote:
> > > >> >
> > > >> > > Hi Dong,
> > > >> > >
> > > >> > >
> > > >> > > I think your approach will allow user to distinguish between the
> > > >> metadata
> > > >> > > > before and after the topic deletion. I also agree that this
> can
> > be
> > > >> > > > potentially be useful to user. I am just not very sure whether
> > we
> > > >> > already
> > > >> > > > have a good use-case to make the additional complexity
> > worthwhile.
> > > >> It
> > > >> > > seems
> > > >> > > > that this feature is kind of independent of the main problem
> of
> > > this
> > > >> > KIP.
> > > >> > > > Could we add this as a future work?
> > > >> > >
> > > >> > >
> > > >> > > Do you think it's fair if we bump the topic epoch on deletion
> and
> > > >> leave
> > > >> > > propagation of the epoch for deleted topics for future work? I
> > don't
> > > >> > think
> > > >> > > this adds much complexity and it makes the behavior consistent:
> > > every
> > > >> > topic
> > > >> > > mutation results in an epoch bump.
> > > >> > >
> > > >> > > Thanks,
> > > >> > > Jason
> > > >> > >
> > > >> > > On Mon, Jan 8, 2018 at 3:14 PM, Dong Lin <li...@gmail.com>
> > > wrote:
> > > >> > >
> > > >> > > > Hey Ismael,
> > > >> > > >
> > > >> > > > I guess we actually need user to see this field so that user
> can
> > > >> store
> > > >> > > this
> > > >> > > > value in the external store together with the offset. We just
> > > prefer
> > > >> > the
> > > >> > > > value to be opaque to discourage most users from interpreting
> > this
> > > >> > value.
> > > >> > > > One more advantage of using such an opaque field is to be able
> > to
> > > >> > evolve
> > > >> > > > the information (or schema) of this value without changing
> > > consumer
> > > >> API
> > > >> > > in
> > > >> > > > the future.
> > > >> > > >
> > > >> > > > I also thinking it is probably OK for user to be able to
> > interpret
> > > >> this
> > > >> > > > value, particularly for those advanced users.
> > > >> > > >
> > > >> > > > Thanks,
> > > >> > > > Dong
> > > >> > > >
> > > >> > > > On Mon, Jan 8, 2018 at 2:34 PM, Ismael Juma <
> ismael@juma.me.uk>
> > > >> wrote:
> > > >> > > >
> > > >> > > > > On Fri, Jan 5, 2018 at 7:15 PM, Jason Gustafson <
> > > >> jason@confluent.io>
> > > >> > > > > wrote:
> > > >> > > > > >
> > > >> > > > > > class OffsetAndMetadata {
> > > >> > > > > >   long offset;
> > > >> > > > > >   byte[] offsetMetadata;
> > > >> > > > > >   String metadata;
> > > >> > > > > > }
> > > >> > > > >
> > > >> > > > >
> > > >> > > > > > Admittedly, the naming is a bit annoying, but we can
> > probably
> > > >> come
> > > >> > up
> > > >> > > > > with
> > > >> > > > > > something better. Internally the byte array would have a
> > > >> version.
> > > >> > If
> > > >> > > in
> > > >> > > > > the
> > > >> > > > > > future we have anything else we need to add, we can update
> > the
> > > >> > > version
> > > >> > > > > and
> > > >> > > > > > we wouldn't need any new APIs.
> > > >> > > > > >
> > > >> > > > >
> > > >> > > > > We can also add fields to a class in a compatible way. So,
> it
> > > >> seems
> > > >> > to
> > > >> > > me
> > > >> > > > > that the main advantage of the byte array is that it's
> opaque
> > to
> > > >> the
> > > >> > > > user.
> > > >> > > > > Is that correct? If so, we could also add any opaque
> metadata
> > > in a
> > > >> > > > subclass
> > > >> > > > > so that users don't even see it (unless they cast it, but
> then
> > > >> > they're
> > > >> > > on
> > > >> > > > > their own).
> > > >> > > > >
> > > >> > > > > Ismael
> > > >> > > > >
> > > >> > > > > The corresponding seek() and position() APIs might look
> > > something
> > > >> > like
> > > >> > > > > this:
> > > >> > > > > >
> > > >> > > > > > void seek(TopicPartition partition, long offset, byte[]
> > > >> > > > offsetMetadata);
> > > >> > > > > > byte[] positionMetadata(TopicPartition partition);
> > > >> > > > > >
> > > >> > > > > > What do you think?
> > > >> > > > > >
> > > >> > > > > > Thanks,
> > > >> > > > > > Jason
> > > >> > > > > >
> > > >> > > > > > On Thu, Jan 4, 2018 at 7:04 PM, Dong Lin <
> > lindong28@gmail.com
> > > >
> > > >> > > wrote:
> > > >> > > > > >
> > > >> > > > > > > Hey Jun, Jason,
> > > >> > > > > > >
> > > >> > > > > > > Thanks much for all the feedback. I have updated the KIP
> > > >> based on
> > > >> > > the
> > > >> > > > > > > latest discussion. Can you help check whether it looks
> > good?
> > > >> > > > > > >
> > > >> > > > > > > Thanks,
> > > >> > > > > > > Dong
> > > >> > > > > > >
> > > >> > > > > > > On Thu, Jan 4, 2018 at 5:36 PM, Dong Lin <
> > > lindong28@gmail.com
> > > >> >
> > > >> > > > wrote:
> > > >> > > > > > >
> > > >> > > > > > > > Hey Jun,
> > > >> > > > > > > >
> > > >> > > > > > > > Hmm... thinking about this more, I am not sure that
> the
> > > >> > proposed
> > > >> > > > API
> > > >> > > > > is
> > > >> > > > > > > > sufficient. For users that store offset externally, we
> > > >> probably
> > > >> > > > need
> > > >> > > > > > > extra
> > > >> > > > > > > > API to return the leader_epoch and partition_epoch for
> > all
> > > >> > > > partitions
> > > >> > > > > > > that
> > > >> > > > > > > > consumers are consuming. I suppose these users
> currently
> > > use
> > > >> > > > > position()
> > > >> > > > > > > to
> > > >> > > > > > > > get the offset. Thus we probably need a new method
> > > >> > > > > > positionWithEpoch(..)
> > > >> > > > > > > to
> > > >> > > > > > > > return <offset, partition_epoch, leader_epoch>. Does
> > this
> > > >> sound
> > > >> > > > > > > reasonable?
> > > >> > > > > > > >
> > > >> > > > > > > > Thanks,
> > > >> > > > > > > > Dong
> > > >> > > > > > > >
> > > >> > > > > > > >
> > > >> > > > > > > > On Thu, Jan 4, 2018 at 5:26 PM, Jun Rao <
> > jun@confluent.io
> > > >
> > > >> > > wrote:
> > > >> > > > > > > >
> > > >> > > > > > > >> Hi, Dong,
> > > >> > > > > > > >>
> > > >> > > > > > > >> Yes, that's what I am thinking. OffsetEpoch will be
> > > >> composed
> > > >> > of
> > > >> > > > > > > >> (partition_epoch,
> > > >> > > > > > > >> leader_epoch).
> > > >> > > > > > > >>
> > > >> > > > > > > >> Thanks,
> > > >> > > > > > > >>
> > > >> > > > > > > >> Jun
> > > >> > > > > > > >>
> > > >> > > > > > > >>
> > > >> > > > > > > >> On Thu, Jan 4, 2018 at 4:22 PM, Dong Lin <
> > > >> lindong28@gmail.com
> > > >> > >
> > > >> > > > > wrote:
> > > >> > > > > > > >>
> > > >> > > > > > > >> > Hey Jun,
> > > >> > > > > > > >> >
> > > >> > > > > > > >> > Thanks much. I like the the new API that you
> > proposed.
> > > I
> > > >> am
> > > >> > > not
> > > >> > > > > sure
> > > >> > > > > > > >> what
> > > >> > > > > > > >> > you exactly mean by offset_epoch. I suppose that we
> > can
> > > >> use
> > > >> > > the
> > > >> > > > > pair
> > > >> > > > > > > of
> > > >> > > > > > > >> > (partition_epoch, leader_epoch) as the
> offset_epoch,
> > > >> right?
> > > >> > > > > > > >> >
> > > >> > > > > > > >> > Thanks,
> > > >> > > > > > > >> > Dong
> > > >> > > > > > > >> >
> > > >> > > > > > > >> > On Thu, Jan 4, 2018 at 4:02 PM, Jun Rao <
> > > >> jun@confluent.io>
> > > >> > > > wrote:
> > > >> > > > > > > >> >
> > > >> > > > > > > >> > > Hi, Dong,
> > > >> > > > > > > >> > >
> > > >> > > > > > > >> > > Got it. The api that you proposed works. The
> > question
> > > >> is
> > > >> > > > whether
> > > >> > > > > > > >> that's
> > > >> > > > > > > >> > the
> > > >> > > > > > > >> > > api that we want to have in the long term. My
> > concern
> > > >> is
> > > >> > > that
> > > >> > > > > > while
> > > >> > > > > > > >> the
> > > >> > > > > > > >> > api
> > > >> > > > > > > >> > > change is simple, the new api seems harder to
> > explain
> > > >> and
> > > >> > > use.
> > > >> > > > > For
> > > >> > > > > > > >> > example,
> > > >> > > > > > > >> > > a consumer storing offsets externally now needs
> to
> > > call
> > > >> > > > > > > >> > > waitForMetadataUpdate() after calling seek().
> > > >> > > > > > > >> > >
> > > >> > > > > > > >> > > An alternative approach is to make the following
> > > >> > compatible
> > > >> > > > api
> > > >> > > > > > > >> changes
> > > >> > > > > > > >> > in
> > > >> > > > > > > >> > > Consumer.
> > > >> > > > > > > >> > > * Add an additional OffsetEpoch field in
> > > >> > OffsetAndMetadata.
> > > >> > > > (no
> > > >> > > > > > need
> > > >> > > > > > > >> to
> > > >> > > > > > > >> > > change the CommitSync() api)
> > > >> > > > > > > >> > > * Add a new api seek(TopicPartition partition,
> long
> > > >> > offset,
> > > >> > > > > > > >> OffsetEpoch
> > > >> > > > > > > >> > > offsetEpoch). We can potentially deprecate the
> old
> > > api
> > > >> > > > > > > >> > seek(TopicPartition
> > > >> > > > > > > >> > > partition, long offset) in the future.
> > > >> > > > > > > >> > >
> > > >> > > > > > > >> > > The alternative approach has similar amount of
> api
> > > >> changes
> > > >> > > as
> > > >> > > > > > yours
> > > >> > > > > > > >> but
> > > >> > > > > > > >> > has
> > > >> > > > > > > >> > > the following benefits.
> > > >> > > > > > > >> > > 1. The api works in a similar way as how offset
> > > >> management
> > > >> > > > works
> > > >> > > > > > now
> > > >> > > > > > > >> and
> > > >> > > > > > > >> > is
> > > >> > > > > > > >> > > probably what we want in the long term.
> > > >> > > > > > > >> > > 2. It can reset offsets better when there is data
> > > loss
> > > >> due
> > > >> > > to
> > > >> > > > > > > unclean
> > > >> > > > > > > >> > > leader election or correlated replica failure.
> > > >> > > > > > > >> > > 3. It can reset offsets better when topic is
> > > recreated.
> > > >> > > > > > > >> > >
> > > >> > > > > > > >> > > Thanks,
> > > >> > > > > > > >> > >
> > > >> > > > > > > >> > > Jun
> > > >> > > > > > > >> > >
> > > >> > > > > > > >> > >
> > > >> > > > > > > >> > > On Thu, Jan 4, 2018 at 2:05 PM, Dong Lin <
> > > >> > > lindong28@gmail.com
> > > >> > > > >
> > > >> > > > > > > wrote:
> > > >> > > > > > > >> > >
> > > >> > > > > > > >> > > > Hey Jun,
> > > >> > > > > > > >> > > >
> > > >> > > > > > > >> > > > Yeah I agree that ideally we don't want an ever
> > > >> growing
> > > >> > > > global
> > > >> > > > > > > >> metadata
> > > >> > > > > > > >> > > > version. I just think it may be more desirable
> to
> > > >> keep
> > > >> > the
> > > >> > > > > > > consumer
> > > >> > > > > > > >> API
> > > >> > > > > > > >> > > > simple.
> > > >> > > > > > > >> > > >
> > > >> > > > > > > >> > > > In my current proposal, metadata version
> returned
> > > in
> > > >> the
> > > >> > > > fetch
> > > >> > > > > > > >> response
> > > >> > > > > > > >> > > > will be stored with the offset together. More
> > > >> > > specifically,
> > > >> > > > > the
> > > >> > > > > > > >> > > > metadata_epoch in the new offset topic schema
> > will
> > > be
> > > >> > the
> > > >> > > > > > largest
> > > >> > > > > > > >> > > > metadata_epoch from all the MetadataResponse
> and
> > > >> > > > FetchResponse
> > > >> > > > > > > ever
> > > >> > > > > > > >> > > > received by this consumer.
> > > >> > > > > > > >> > > >
> > > >> > > > > > > >> > > > We probably don't have to change the consumer
> API
> > > for
> > > >> > > > > > > >> > > > commitSync(Map<TopicPartition,
> > OffsetAndMetadata>).
> > > >> If
> > > >> > > user
> > > >> > > > > > calls
> > > >> > > > > > > >> > > > commitSync(...) to commit offset 10 for a given
> > > >> > partition,
> > > >> > > > for
> > > >> > > > > > > most
> > > >> > > > > > > >> > > > use-cases, this consumer instance should have
> > > >> consumed
> > > >> > > > message
> > > >> > > > > > > with
> > > >> > > > > > > >> > > offset
> > > >> > > > > > > >> > > > 9 from this partition, in which case the
> consumer
> > > can
> > > >> > > > remember
> > > >> > > > > > and
> > > >> > > > > > > >> use
> > > >> > > > > > > >> > > the
> > > >> > > > > > > >> > > > metadata_epoch from the corresponding
> > FetchResponse
> > > >> when
> > > >> > > > > > > committing
> > > >> > > > > > > >> > > offset.
> > > >> > > > > > > >> > > > If user calls commitSync(..) to commit offset
> 10
> > > for
> > > >> a
> > > >> > > given
> > > >> > > > > > > >> partition
> > > >> > > > > > > >> > > > without having consumed the message with
> offset 9
> > > >> using
> > > >> > > this
> > > >> > > > > > > >> consumer
> > > >> > > > > > > >> > > > instance, this is probably an advanced
> use-case.
> > In
> > > >> this
> > > >> > > > case
> > > >> > > > > > the
> > > >> > > > > > > >> > > advanced
> > > >> > > > > > > >> > > > user can retrieve the metadata_epoch using the
> > > newly
> > > >> > added
> > > >> > > > > > > >> > > metadataEpoch()
> > > >> > > > > > > >> > > > API after it fetches the message with offset 9
> > > >> (probably
> > > >> > > > from
> > > >> > > > > > > >> another
> > > >> > > > > > > >> > > > consumer instance) and encode this
> metadata_epoch
> > > in
> > > >> the
> > > >> > > > > > > >> > > > string OffsetAndMetadata.metadata. Do you think
> > > this
> > > >> > > > solution
> > > >> > > > > > > would
> > > >> > > > > > > >> > work?
> > > >> > > > > > > >> > > >
> > > >> > > > > > > >> > > > By "not sure that I fully understand your
> latest
> > > >> > > > suggestion",
> > > >> > > > > > are
> > > >> > > > > > > >> you
> > > >> > > > > > > >> > > > referring to solution related to unclean leader
> > > >> election
> > > >> > > > using
> > > >> > > > > > > >> > > leader_epoch
> > > >> > > > > > > >> > > > in my previous email?
> > > >> > > > > > > >> > > >
> > > >> > > > > > > >> > > > Thanks,
> > > >> > > > > > > >> > > > Dong
> > > >> > > > > > > >> > > >
> > > >> > > > > > > >> > > > On Thu, Jan 4, 2018 at 1:33 PM, Jun Rao <
> > > >> > jun@confluent.io
> > > >> > > >
> > > >> > > > > > wrote:
> > > >> > > > > > > >> > > >
> > > >> > > > > > > >> > > > > Hi, Dong,
> > > >> > > > > > > >> > > > >
> > > >> > > > > > > >> > > > > Not sure that I fully understand your latest
> > > >> > suggestion.
> > > >> > > > > > > >> Returning an
> > > >> > > > > > > >> > > > ever
> > > >> > > > > > > >> > > > > growing global metadata version itself is no
> > > ideal,
> > > >> > but
> > > >> > > is
> > > >> > > > > > fine.
> > > >> > > > > > > >> My
> > > >> > > > > > > >> > > > > question is whether the metadata version
> > returned
> > > >> in
> > > >> > the
> > > >> > > > > fetch
> > > >> > > > > > > >> > response
> > > >> > > > > > > >> > > > > needs to be stored with the offset together
> if
> > > >> offsets
> > > >> > > are
> > > >> > > > > > > stored
> > > >> > > > > > > >> > > > > externally. If so, we also have to change the
> > > >> consumer
> > > >> > > API
> > > >> > > > > for
> > > >> > > > > > > >> > > > commitSync()
> > > >> > > > > > > >> > > > > and need to worry about compatibility. If we
> > > don't
> > > >> > store
> > > >> > > > the
> > > >> > > > > > > >> metadata
> > > >> > > > > > > >> > > > > version together with the offset, on a
> consumer
> > > >> > restart,
> > > >> > > > > it's
> > > >> > > > > > > not
> > > >> > > > > > > >> > clear
> > > >> > > > > > > >> > > > how
> > > >> > > > > > > >> > > > > we can ensure the metadata in the consumer is
> > > high
> > > >> > > enough
> > > >> > > > > > since
> > > >> > > > > > > >> there
> > > >> > > > > > > >> > > is
> > > >> > > > > > > >> > > > no
> > > >> > > > > > > >> > > > > metadata version to compare with.
> > > >> > > > > > > >> > > > >
> > > >> > > > > > > >> > > > > Thanks,
> > > >> > > > > > > >> > > > >
> > > >> > > > > > > >> > > > > Jun
> > > >> > > > > > > >> > > > >
> > > >> > > > > > > >> > > > >
> > > >> > > > > > > >> > > > > On Wed, Jan 3, 2018 at 6:43 PM, Dong Lin <
> > > >> > > > > lindong28@gmail.com
> > > >> > > > > > >
> > > >> > > > > > > >> > wrote:
> > > >> > > > > > > >> > > > >
> > > >> > > > > > > >> > > > > > Hey Jun,
> > > >> > > > > > > >> > > > > >
> > > >> > > > > > > >> > > > > > Thanks much for the explanation.
> > > >> > > > > > > >> > > > > >
> > > >> > > > > > > >> > > > > > I understand the advantage of
> partition_epoch
> > > >> over
> > > >> > > > > > > >> metadata_epoch.
> > > >> > > > > > > >> > My
> > > >> > > > > > > >> > > > > > current concern is that the use of
> > leader_epoch
> > > >> and
> > > >> > > the
> > > >> > > > > > > >> > > partition_epoch
> > > >> > > > > > > >> > > > > > requires us considerable change to
> consumer's
> > > >> public
> > > >> > > API
> > > >> > > > > to
> > > >> > > > > > > take
> > > >> > > > > > > >> > care
> > > >> > > > > > > >> > > > of
> > > >> > > > > > > >> > > > > > the case where user stores offset
> externally.
> > > For
> > > >> > > > example,
> > > >> > > > > > > >> > > *consumer*.
> > > >> > > > > > > >> > > > > > *commitSync*(..) would have to take a map
> > whose
> > > >> > value
> > > >> > > is
> > > >> > > > > > > >> <offset,
> > > >> > > > > > > >> > > > > metadata,
> > > >> > > > > > > >> > > > > > leader epoch, partition epoch>.
> > > >> > *consumer*.*seek*(...)
> > > >> > > > > would
> > > >> > > > > > > >> also
> > > >> > > > > > > >> > > need
> > > >> > > > > > > >> > > > > > leader_epoch and partition_epoch as
> > parameter.
> > > >> > > > Technically
> > > >> > > > > > we
> > > >> > > > > > > >> can
> > > >> > > > > > > >> > > > > probably
> > > >> > > > > > > >> > > > > > still make it work in a backward compatible
> > > >> manner
> > > >> > > after
> > > >> > > > > > > careful
> > > >> > > > > > > >> > > design
> > > >> > > > > > > >> > > > > and
> > > >> > > > > > > >> > > > > > discussion. But these changes can make the
> > > >> > consumer's
> > > >> > > > > > > interface
> > > >> > > > > > > >> > > > > > unnecessarily complex for more users who do
> > not
> > > >> > store
> > > >> > > > > offset
> > > >> > > > > > > >> > > > externally.
> > > >> > > > > > > >> > > > > >
> > > >> > > > > > > >> > > > > > After thinking more about it, we can
> address
> > > all
> > > >> > > > problems
> > > >> > > > > > > >> discussed
> > > >> > > > > > > >> > > by
> > > >> > > > > > > >> > > > > only
> > > >> > > > > > > >> > > > > > using the metadata_epoch without
> introducing
> > > >> > > > leader_epoch
> > > >> > > > > or
> > > >> > > > > > > the
> > > >> > > > > > > >> > > > > > partition_epoch. The current KIP describes
> > the
> > > >> > changes
> > > >> > > > to
> > > >> > > > > > the
> > > >> > > > > > > >> > > consumer
> > > >> > > > > > > >> > > > > API
> > > >> > > > > > > >> > > > > > and how the new API can be used if user
> > stores
> > > >> > offset
> > > >> > > > > > > >> externally.
> > > >> > > > > > > >> > In
> > > >> > > > > > > >> > > > > order
> > > >> > > > > > > >> > > > > > to address the scenario you described
> > earlier,
> > > we
> > > >> > can
> > > >> > > > > > include
> > > >> > > > > > > >> > > > > > metadata_epoch in the FetchResponse and the
> > > >> > > > > > > LeaderAndIsrRequest.
> > > >> > > > > > > >> > > > Consumer
> > > >> > > > > > > >> > > > > > remembers the largest metadata_epoch from
> all
> > > the
> > > >> > > > > > > FetchResponse
> > > >> > > > > > > >> it
> > > >> > > > > > > >> > > has
> > > >> > > > > > > >> > > > > > received. The metadata_epoch committed with
> > the
> > > >> > > offset,
> > > >> > > > > > either
> > > >> > > > > > > >> > within
> > > >> > > > > > > >> > > > or
> > > >> > > > > > > >> > > > > > outside Kafka, should be the largest
> > > >> metadata_epoch
> > > >> > > > across
> > > >> > > > > > all
> > > >> > > > > > > >> > > > > > FetchResponse and MetadataResponse ever
> > > received
> > > >> by
> > > >> > > this
> > > >> > > > > > > >> consumer.
> > > >> > > > > > > >> > > > > >
> > > >> > > > > > > >> > > > > > The drawback of using only the
> metadata_epoch
> > > is
> > > >> > that
> > > >> > > we
> > > >> > > > > can
> > > >> > > > > > > not
> > > >> > > > > > > >> > > always
> > > >> > > > > > > >> > > > > do
> > > >> > > > > > > >> > > > > > the smart offset reset in case of unclean
> > > leader
> > > >> > > > election
> > > >> > > > > > > which
> > > >> > > > > > > >> you
> > > >> > > > > > > >> > > > > > mentioned earlier. But in most case,
> unclean
> > > >> leader
> > > >> > > > > election
> > > >> > > > > > > >> > probably
> > > >> > > > > > > >> > > > > > happens when consumer is not
> > > >> rebalancing/restarting.
> > > >> > > In
> > > >> > > > > > these
> > > >> > > > > > > >> > cases,
> > > >> > > > > > > >> > > > > either
> > > >> > > > > > > >> > > > > > consumer is not directly affected by
> unclean
> > > >> leader
> > > >> > > > > election
> > > >> > > > > > > >> since
> > > >> > > > > > > >> > it
> > > >> > > > > > > >> > > > is
> > > >> > > > > > > >> > > > > > not consuming from the end of the log, or
> > > >> consumer
> > > >> > can
> > > >> > > > > > derive
> > > >> > > > > > > >> the
> > > >> > > > > > > >> > > > > > leader_epoch from the most recent message
> > > >> received
> > > >> > > > before
> > > >> > > > > it
> > > >> > > > > > > >> sees
> > > >> > > > > > > >> > > > > > OffsetOutOfRangeException. So I am not sure
> > it
> > > is
> > > >> > > worth
> > > >> > > > > > adding
> > > >> > > > > > > >> the
> > > >> > > > > > > >> > > > > > leader_epoch to consumer API to address the
> > > >> > remaining
> > > >> > > > > corner
> > > >> > > > > > > >> case.
> > > >> > > > > > > >> > > What
> > > >> > > > > > > >> > > > > do
> > > >> > > > > > > >> > > > > > you think?
> > > >> > > > > > > >> > > > > >
> > > >> > > > > > > >> > > > > > Thanks,
> > > >> > > > > > > >> > > > > > Dong
> > > >> > > > > > > >> > > > > >
> > > >> > > > > > > >> > > > > >
> > > >> > > > > > > >> > > > > >
> > > >> > > > > > > >> > > > > > On Tue, Jan 2, 2018 at 6:28 PM, Jun Rao <
> > > >> > > > jun@confluent.io
> > > >> > > > > >
> > > >> > > > > > > >> wrote:
> > > >> > > > > > > >> > > > > >
> > > >> > > > > > > >> > > > > > > Hi, Dong,
> > > >> > > > > > > >> > > > > > >
> > > >> > > > > > > >> > > > > > > Thanks for the reply.
> > > >> > > > > > > >> > > > > > >
> > > >> > > > > > > >> > > > > > > To solve the topic recreation issue, we
> > could
> > > >> use
> > > >> > > > > either a
> > > >> > > > > > > >> global
> > > >> > > > > > > >> > > > > > metadata
> > > >> > > > > > > >> > > > > > > version or a partition level epoch. But
> > > either
> > > >> one
> > > >> > > > will
> > > >> > > > > > be a
> > > >> > > > > > > >> new
> > > >> > > > > > > >> > > > > concept,
> > > >> > > > > > > >> > > > > > > right? To me, the latter seems more
> > natural.
> > > It
> > > >> > also
> > > >> > > > > makes
> > > >> > > > > > > it
> > > >> > > > > > > >> > > easier
> > > >> > > > > > > >> > > > to
> > > >> > > > > > > >> > > > > > > detect if a consumer's offset is still
> > valid
> > > >> > after a
> > > >> > > > > topic
> > > >> > > > > > > is
> > > >> > > > > > > >> > > > > recreated.
> > > >> > > > > > > >> > > > > > As
> > > >> > > > > > > >> > > > > > > you pointed out, we don't need to store
> the
> > > >> > > partition
> > > >> > > > > > epoch
> > > >> > > > > > > in
> > > >> > > > > > > >> > the
> > > >> > > > > > > >> > > > > > message.
> > > >> > > > > > > >> > > > > > > The following is what I am thinking.
> When a
> > > >> > > partition
> > > >> > > > is
> > > >> > > > > > > >> created,
> > > >> > > > > > > >> > > we
> > > >> > > > > > > >> > > > > can
> > > >> > > > > > > >> > > > > > > assign a partition epoch from an
> > > >> ever-increasing
> > > >> > > > global
> > > >> > > > > > > >> counter
> > > >> > > > > > > >> > and
> > > >> > > > > > > >> > > > > store
> > > >> > > > > > > >> > > > > > > it in /brokers/topics/[topic]/
> > > >> > > > partitions/[partitionId]
> > > >> > > > > in
> > > >> > > > > > > ZK.
> > > >> > > > > > > >> > The
> > > >> > > > > > > >> > > > > > > partition
> > > >> > > > > > > >> > > > > > > epoch is propagated to every broker. The
> > > >> consumer
> > > >> > > will
> > > >> > > > > be
> > > >> > > > > > > >> > tracking
> > > >> > > > > > > >> > > a
> > > >> > > > > > > >> > > > > > tuple
> > > >> > > > > > > >> > > > > > > of <offset, leader epoch, partition
> epoch>
> > > for
> > > >> > > > offsets.
> > > >> > > > > > If a
> > > >> > > > > > > >> > topic
> > > >> > > > > > > >> > > is
> > > >> > > > > > > >> > > > > > > recreated, it's possible that a
> consumer's
> > > >> offset
> > > >> > > and
> > > >> > > > > > leader
> > > >> > > > > > > >> > epoch
> > > >> > > > > > > >> > > > > still
> > > >> > > > > > > >> > > > > > > match that in the broker, but partition
> > epoch
> > > >> > won't
> > > >> > > > be.
> > > >> > > > > In
> > > >> > > > > > > >> this
> > > >> > > > > > > >> > > case,
> > > >> > > > > > > >> > > > > we
> > > >> > > > > > > >> > > > > > > can potentially still treat the
> consumer's
> > > >> offset
> > > >> > as
> > > >> > > > out
> > > >> > > > > > of
> > > >> > > > > > > >> range
> > > >> > > > > > > >> > > and
> > > >> > > > > > > >> > > > > > reset
> > > >> > > > > > > >> > > > > > > the offset based on the offset reset
> policy
> > > in
> > > >> the
> > > >> > > > > > consumer.
> > > >> > > > > > > >> This
> > > >> > > > > > > >> > > > seems
> > > >> > > > > > > >> > > > > > > harder to do with a global metadata
> > version.
> > > >> > > > > > > >> > > > > > >
> > > >> > > > > > > >> > > > > > > Jun
> > > >> > > > > > > >> > > > > > >
> > > >> > > > > > > >> > > > > > >
> > > >> > > > > > > >> > > > > > >
> > > >> > > > > > > >> > > > > > > On Mon, Dec 25, 2017 at 6:56 AM, Dong
> Lin <
> > > >> > > > > > > >> lindong28@gmail.com>
> > > >> > > > > > > >> > > > wrote:
> > > >> > > > > > > >> > > > > > >
> > > >> > > > > > > >> > > > > > > > Hey Jun,
> > > >> > > > > > > >> > > > > > > >
> > > >> > > > > > > >> > > > > > > > This is a very good example. After
> > thinking
> > > >> > > through
> > > >> > > > > this
> > > >> > > > > > > in
> > > >> > > > > > > >> > > > detail, I
> > > >> > > > > > > >> > > > > > > agree
> > > >> > > > > > > >> > > > > > > > that we need to commit offset with
> leader
> > > >> epoch
> > > >> > in
> > > >> > > > > order
> > > >> > > > > > > to
> > > >> > > > > > > >> > > address
> > > >> > > > > > > >> > > > > > this
> > > >> > > > > > > >> > > > > > > > example.
> > > >> > > > > > > >> > > > > > > >
> > > >> > > > > > > >> > > > > > > > I think the remaining question is how
> to
> > > >> address
> > > >> > > the
> > > >> > > > > > > >> scenario
> > > >> > > > > > > >> > > that
> > > >> > > > > > > >> > > > > the
> > > >> > > > > > > >> > > > > > > > topic is deleted and re-created. One
> > > possible
> > > >> > > > solution
> > > >> > > > > > is
> > > >> > > > > > > to
> > > >> > > > > > > >> > > commit
> > > >> > > > > > > >> > > > > > > offset
> > > >> > > > > > > >> > > > > > > > with both the leader epoch and the
> > metadata
> > > >> > > version.
> > > >> > > > > The
> > > >> > > > > > > >> logic
> > > >> > > > > > > >> > > and
> > > >> > > > > > > >> > > > > the
> > > >> > > > > > > >> > > > > > > > implementation of this solution does
> not
> > > >> > require a
> > > >> > > > new
> > > >> > > > > > > >> concept
> > > >> > > > > > > >> > > > (e.g.
> > > >> > > > > > > >> > > > > > > > partition epoch) and it does not
> require
> > > any
> > > >> > > change
> > > >> > > > to
> > > >> > > > > > the
> > > >> > > > > > > >> > > message
> > > >> > > > > > > >> > > > > > format
> > > >> > > > > > > >> > > > > > > > or leader epoch. It also allows us to
> > order
> > > >> the
> > > >> > > > > metadata
> > > >> > > > > > > in
> > > >> > > > > > > >> a
> > > >> > > > > > > >> > > > > > > > straightforward manner which may be
> > useful
> > > in
> > > >> > the
> > > >> > > > > > future.
> > > >> > > > > > > >> So it
> > > >> > > > > > > >> > > may
> > > >> > > > > > > >> > > > > be
> > > >> > > > > > > >> > > > > > a
> > > >> > > > > > > >> > > > > > > > better solution than generating a
> random
> > > >> > partition
> > > >> > > > > epoch
> > > >> > > > > > > >> every
> > > >> > > > > > > >> > > time
> > > >> > > > > > > >> > > > > we
> > > >> > > > > > > >> > > > > > > > create a partition. Does this sound
> > > >> reasonable?
> > > >> > > > > > > >> > > > > > > >
> > > >> > > > > > > >> > > > > > > > Previously one concern with using the
> > > >> metadata
> > > >> > > > version
> > > >> > > > > > is
> > > >> > > > > > > >> that
> > > >> > > > > > > >> > > > > consumer
> > > >> > > > > > > >> > > > > > > > will be forced to refresh metadata even
> > if
> > > >> > > metadata
> > > >> > > > > > > version
> > > >> > > > > > > >> is
> > > >> > > > > > > >> > > > > > increased
> > > >> > > > > > > >> > > > > > > > due to topics that the consumer is not
> > > >> > interested
> > > >> > > > in.
> > > >> > > > > > Now
> > > >> > > > > > > I
> > > >> > > > > > > >> > > > realized
> > > >> > > > > > > >> > > > > > that
> > > >> > > > > > > >> > > > > > > > this is probably not a problem.
> Currently
> > > >> client
> > > >> > > > will
> > > >> > > > > > > >> refresh
> > > >> > > > > > > >> > > > > metadata
> > > >> > > > > > > >> > > > > > > > either due to InvalidMetadataException
> in
> > > the
> > > >> > > > response
> > > >> > > > > > > from
> > > >> > > > > > > >> > > broker
> > > >> > > > > > > >> > > > or
> > > >> > > > > > > >> > > > > > due
> > > >> > > > > > > >> > > > > > > > to metadata expiry. The addition of the
> > > >> metadata
> > > >> > > > > version
> > > >> > > > > > > >> should
> > > >> > > > > > > >> > > > > > increase
> > > >> > > > > > > >> > > > > > > > the overhead of metadata refresh caused
> > by
> > > >> > > > > > > >> > > > InvalidMetadataException.
> > > >> > > > > > > >> > > > > If
> > > >> > > > > > > >> > > > > > > > client refresh metadata due to expiry
> and
> > > it
> > > >> > > > receives
> > > >> > > > > a
> > > >> > > > > > > >> > metadata
> > > >> > > > > > > >> > > > > whose
> > > >> > > > > > > >> > > > > > > > version is lower than the current
> > metadata
> > > >> > > version,
> > > >> > > > we
> > > >> > > > > > can
> > > >> > > > > > > >> > reject
> > > >> > > > > > > >> > > > the
> > > >> > > > > > > >> > > > > > > > metadata but still reset the metadata
> > age,
> > > >> which
> > > >> > > > > > > essentially
> > > >> > > > > > > >> > keep
> > > >> > > > > > > >> > > > the
> > > >> > > > > > > >> > > > > > > > existing behavior in the client.
> > > >> > > > > > > >> > > > > > > >
> > > >> > > > > > > >> > > > > > > > Thanks much,
> > > >> > > > > > > >> > > > > > > > Dong
> > > >> > > > > > > >> > > > > > > >
> > > >> > > > > > > >> > > > > > >
> > > >> > > > > > > >> > > > > >
> > > >> > > > > > > >> > > > >
> > > >> > > > > > > >> > > >
> > > >> > > > > > > >> > >
> > > >> > > > > > > >> >
> > > >> > > > > > > >>
> > > >> > > > > > > >
> > > >> > > > > > > >
> > > >> > > > > > >
> > > >> > > > > >
> > > >> > > > >
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Jun Rao <ju...@confluent.io>.
Hi, Dong,

Thanks for the updated KIP. Looks good to me overall. Just a few minor
comments.

60. OffsetAndMetadata positionAndOffsetEpoch(TopicPartition partition): It
seems that there is no need to return metadata. We probably want to return
sth like OffsetAndEpoch.

61. Should we store partition_epoch in
/brokers/topics/[topic]/partitions/[partitionId] in ZK?

62. For checking outdated metadata in the client, we probably want to add
when max_partition_epoch will be used.

63. "The leader_epoch should be the largest leader_epoch of messages whose
offset < the commit offset. If no message has been consumed since consumer
initialization, the leader_epoch from seek(...) or OffsetFetchResponse
should be used. The partition_epoch should be read from the last
FetchResponse corresponding to the given partition and commit offset. ":
leader_epoch and partition_epoch are associated with an offset. So, if no
message is consumed, there is no offset and therefore there is no need to
read leader_epoch and partition_epoch. Also, the leader_epoch associated
with the offset should just come from the messages returned in the fetch
response.

64. Could you include the public methods in the OffsetEpoch class?

Jun


On Thu, Jan 11, 2018 at 5:43 PM, Dong Lin <li...@gmail.com> wrote:

> Hey Jun,
>
> Thanks much. I agree that we can not rely on committed offsets to be always
> deleted when we delete topic. So it is necessary to use a per-partition
> epoch that does not change unless this partition is deleted. I also agree
> that it is very nice to be able to uniquely identify a message with
> (offset, leader_epoch, partition_epoch) in face of potential topic deletion
> and unclean leader election.
>
> I agree with all your comments. And I have updated the KIP based on our
> latest discussion. In addition, I added InvalidPartitionEpochException
> which will be thrown by consumer.poll() if the partition_epoch associated
> with the partition, which can be given to consumer using seek(...), is
> different from the partition_epoch in the FetchResponse.
>
> Can you take another look at the latest KIP?
>
> Thanks!
> Dong
>
>
>
> On Wed, Jan 10, 2018 at 2:24 PM, Jun Rao <ju...@confluent.io> wrote:
>
> > Hi, Dong,
> >
> > My replies are the following.
> >
> > 60. What you described could also work. The drawback is that we will be
> > unnecessarily changing the partition epoch when a partition hasn't really
> > changed. I was imagining that the partition epoch will be stored in
> > /brokers/topics/[topic]/partitions/[partitionId], instead of at the
> topic
> > level. So, not sure if ZK size limit is an issue.
> >
> > 61, 62 and 65. To me, the offset + offset_epoch is a unique identifier
> for
> > a message. So, if a message hasn't changed, the offset and the associated
> > offset_epoch ideally should remain the same (it will be kind of weird if
> > two consumer apps save the offset on the same message, but the
> offset_epoch
> > are different). partition_epoch + leader_epoch give us that.
> global_epoch +
> > leader_epoch don't. If we use this approach, we can solve not only the
> > problem that you have identified, but also other problems when there is
> > data loss or topic re-creation more reliably. For example, in the future,
> > if we include the partition_epoch and leader_epoch in the fetch request,
> > the server can do a more reliable check of whether that offset is valid
> or
> > not. I am not sure that we can rely upon all external offsets to be
> removed
> > on topic deletion. For example, a topic may be deleted by an admin who
> may
> > not know all the applications.
> >
> > If we agree on the above, the second question is then how to reliably
> > propagate the partition_epoch and the leader_epoch to the consumer when
> > there are leader or partition changes. The leader_epoch comes from the
> > message, which is reliable. So, I was suggesting that when we store an
> > offset, we can just store the leader_epoch from the message set
> containing
> > that offset. Similarly, I was thinking that if the partition_epoch is in
> > the fetch response, we can propagate partition_epoch reliably where is
> > partition_epoch change.
> >
> > 63. My point is that once a leader is producing a message in the new
> > partition_epoch, ideally, we should associate the new offsets with the
> new
> > partition_epoch. Otherwise, the offset_epoch won't be the correct unique
> > identifier (useful for solving other problems mentioned above). I was
> > originally thinking that the leader will include the partition_epoch in
> the
> > metadata cache in the fetch response. It's just that right now, metadata
> > cache is updated on UpdateMetadataRequest, which typically happens after
> > the LeaderAndIsrRequest. Another approach is for the leader to cache the
> > partition_epoch in the Partition object and return that (instead of the
> one
> > in metadata cache) in the fetch response.
> >
> > 65. It seems to me that the global_epoch and the partition_epoch have
> > different purposes. A partition_epoch has the benefit that it (1) can be
> > used to form a unique identifier for a message and (2) can be used to
> > solve other
> > corner case problems in the future. I am not sure having just a
> > global_epoch can achieve these. global_epoch is useful to determine which
> > version of the metadata is newer, especially with topic deletion.
> >
> > Thanks,
> >
> > Jun
> >
> > On Tue, Jan 9, 2018 at 11:34 PM, Dong Lin <li...@gmail.com> wrote:
> >
> > > Regarding the use of the global epoch in 65), it is very similar to the
> > > proposal of the metadata_epoch we discussed earlier. The main
> difference
> > is
> > > that this epoch is incremented when we create/expand/delete topic and
> > does
> > > not change when controller re-send metadata.
> > >
> > > I looked at our previous discussion. It seems that we prefer
> > > partition_epoch over the metadata_epoch because 1) we prefer not to
> have
> > an
> > > ever growing metadata_epoch and 2) we can reset offset better when
> topic
> > is
> > > re-created. The use of global topic_epoch avoids the drawback of an
> ever
> > > quickly ever growing metadata_epoch. Though the global epoch does not
> > allow
> > > us to recognize the invalid offset committed before the topic
> > re-creation,
> > > we can probably just delete the offset when we delete a topic. Thus I
> am
> > > not very sure whether it is still worthwhile to have a per-partition
> > > partition_epoch if the metadata already has the global epoch.
> > >
> > >
> > > On Tue, Jan 9, 2018 at 6:58 PM, Dong Lin <li...@gmail.com> wrote:
> > >
> > > > Hey Jun,
> > > >
> > > > Thanks so much. These comments very useful. Please see below my
> > comments.
> > > >
> > > > On Mon, Jan 8, 2018 at 5:52 PM, Jun Rao <ju...@confluent.io> wrote:
> > > >
> > > >> Hi, Dong,
> > > >>
> > > >> Thanks for the updated KIP. A few more comments.
> > > >>
> > > >> 60. Perhaps having a partition epoch is more flexible since in the
> > > future,
> > > >> we may support deleting a partition as well.
> > > >>
> > > >
> > > > Yeah I have considered this. I think we can probably still support
> > > > deleting a partition by using the topic_epoch -- when partition of a
> > > topic
> > > > is deleted or created, epoch of all partitions of this topic will be
> > > > incremented by 1. Therefore, if that partition is re-created later,
> the
> > > > epoch of that partition will still be larger than its epoch before
> the
> > > > deletion, which still allows the client to order the metadata for the
> > > > purpose of this KIP. Does this sound reasonable?
> > > >
> > > > The advantage of using topic_epoch instead of partition_epoch is that
> > the
> > > > size of the /brokers/topics/[topic] znode and request/response size
> can
> > > be
> > > > smaller. We have a limit on the maximum size of znode (typically
> 1MB).
> > > Use
> > > > partition epoch can effectively reduce the number of partitions that
> > can
> > > be
> > > > described by the /brokers/topics/[topic] znode.
> > > >
> > > > One use-case of partition_epoch for client to detect that the
> committed
> > > > offset, either from kafka offset topic or from the external store is
> > > > invalid after partition deletion and re-creation. However, it seems
> > that
> > > we
> > > > can also address this use-case with other approaches. For example,
> when
> > > > AdminClient deletes partitions, it can also delete the committed
> > offsets
> > > > for those partitions from the offset topic. If user stores offset
> > > > externally, it might make sense for user to similarly remove offsets
> of
> > > > related partitions after these partitions are deleted. So I am not
> sure
> > > > that we should use partition_epoch in this KIP.
> > > >
> > > >
> > > >>
> > > >> 61. It seems that the leader epoch returned in the position() call
> > > should
> > > >> the the leader epoch returned in the fetch response, not the one in
> > the
> > > >> metadata cache of the client.
> > > >
> > > >
> > > > I think this is a good idea. Just to double check, this change does
> not
> > > > affect the correctness or performance of this KIP. But it can be
> useful
> > > if
> > > > we want to use the leader_epoch to better handle the offset rest in
> > case
> > > of
> > > > unclean leader election, which is listed in the future work. Is this
> > > > understanding correct?
> > > >
> > > > I have updated the KIP to specify that the leader_epoch returned by
> > > > position() should be the largest leader_epoch of those already
> consumed
> > > > messages whose offset < position. If no message has been consumed
> since
> > > > consumer initialization, the leader_epoch from seek() or
> > > > OffsetFetchResponse should be used. The offset included in the
> > > > OffsetCommitRequest will also be determined in the similar manner.
> > > >
> > > >
> > > >>
> > > >> 62. I am wondering if we should return the partition epoch in the
> > fetch
> > > >> response as well. In the current proposal, if a topic is recreated
> and
> > > the
> > > >> new leader is on the same broker as the old one, there is nothing to
> > > force
> > > >> the metadata refresh in the client. So, the client may still
> associate
> > > the
> > > >> offset with the old partition epoch.
> > > >>
> > > >
> > > > Could you help me understand the problem if a client associates old
> > > > partition_epoch (or the topic_epoch as of the current KIP) with the
> > > offset?
> > > > The main purpose of the topic_epoch is to be able to drop
> leader_epoch
> > > to 0
> > > > after a partition is deleted and re-created. I guess you may be
> > thinking
> > > > about using the partition_epoch to detect that the committed offset
> is
> > > > invalid? In that case, I am wondering if the alternative approach
> > > described
> > > > in 60) would be reasonable.
> > > >
> > > >
> > > >>
> > > >> 63. There is some subtle coordination between the
> LeaderAndIsrRequest
> > > and
> > > >> UpdateMetadataRequest. Currently, when a leader changes, the
> > controller
> > > >> first sends the LeaderAndIsrRequest to the assigned replicas and the
> > > >> UpdateMetadataRequest to every broker. So, there could be a small
> > window
> > > >> when the leader already receives the new partition epoch in the
> > > >> LeaderAndIsrRequest, but the metadata cache in the broker hasn't
> been
> > > >> updated with the latest partition epoch. Not sure what's the best
> way
> > to
> > > >> address this issue. Perhaps we can update the metadata cache on the
> > > broker
> > > >> with both LeaderAndIsrRequest and UpdateMetadataRequest. The
> challenge
> > > is
> > > >> that the two have slightly different data. For example, only the
> > latter
> > > >> has
> > > >> all endpoints.
> > > >>
> > > >
> > > > I am not sure whether this is a problem. Could you explain a bit more
> > > what
> > > > specific problem this small window can cause?
> > > >
> > > > Since client can fetch metadata from any broker in the cluster, and
> > given
> > > > that different brokers receive request (e.g. LeaderAndIsrRequest and
> > > > UpdateMetadataRequest) in arbitrary order, the metadata received by
> > > client
> > > > can be in arbitrary order (either newer or older) compared to the
> > > broker's
> > > > leadership state even if a given broker receives LeaderAndIsrRequest
> > and
> > > > UpdateMetadataRequest simultaneously. So I am not sure it is useful
> to
> > > > update broker's cache with LeaderAndIsrRequest.
> > > >
> > > >
> > > >> 64. The enforcement of leader epoch in Offset commit: We allow a
> > > consumer
> > > >> to set an arbitrary offset. So it's possible for offsets or leader
> > epoch
> > > >> to
> > > >> go backwards. I am not sure if we could always enforce that the
> leader
> > > >> epoch only goes up on the broker.
> > > >>
> > > >
> > > > Sure. I have removed this check from the KIP.
> > > >
> > > > BTW, we can probably still ensure that the leader_epoch always
> increase
> > > if
> > > > the leader_epoch used with offset commit is the max(leader_epoch of
> the
> > > > message with offset = the committed offset - 1, the largest known
> > > > leader_epoch from the metadata). But I don't have a good use-case for
> > > this
> > > > alternative definition. So I choose the keep the KIP simple by
> > requiring
> > > > leader_epoch to always increase.
> > > >
> > > >
> > > >> 65. Good point on handling missing partition epoch due to topic
> > > deletion.
> > > >> Another potential way to address this is to additionally propagate
> the
> > > >> global partition epoch to brokers and the clients. This way, when a
> > > >> partition epoch is missing, we can use the global partition epoch to
> > > >> reason
> > > >> about which metadata is more recent.
> > > >>
> > > >
> > > > This is a great idea. The global epoch can be used to order the
> > metadata
> > > > and help us recognize the more recent metadata if a topic (or
> > partition)
> > > is
> > > > deleted and re-created.
> > > >
> > > > Actually, it seems we only need to propagate the global epoch to
> > brokers
> > > > and clients without propagating this epoch on a per-topic or
> > > per-partition
> > > > basic. Doing so would simply interface changes made this KIP. Does
> this
> > > > approach sound reasonable?
> > > >
> > > >
> > > >> 66. A client may also get an offset by time using the
> offsetForTimes()
> > > >> api.
> > > >> So, we probably want to include offsetInternalMetadata in
> > > >> OffsetAndTimestamp
> > > >> as well.
> > > >>
> > > >
> > > > You are right. This probably also requires us to change the
> > > > ListOffsetRequest as well. I will update the KIP after we agree on
> the
> > > > solution for 65).
> > > >
> > > >
> > > >>
> > > >> 67. InteralMetadata can be a bit confusing with the metadata field
> > > already
> > > >> there. Perhaps we can just call it OffsetEpoch. It might be useful
> to
> > > make
> > > >> OffsetEpoch printable at least for debugging purpose. Once you do
> > that,
> > > we
> > > >> are already exposing the internal fields. So, not sure if it's worth
> > > >> hiding
> > > >> them. If we do want to hide them, perhaps we can have sth like the
> > > >> following. The binary encoding is probably more efficient than JSON
> > for
> > > >> external storage.
> > > >>
> > > >> OffsetEpoch {
> > > >>  static OffsetEpoch decode(byte[]);
> > > >>
> > > >>   public byte[] encode();
> > > >>
> > > >>   public String toString();
> > > >> }
> > > >>
> > > >
> > > > Thanks much. I like this solution. I have updated the KIP
> accordingly.
> > > >
> > > >
> > > >
> > > >>
> > > >> Jun
> > > >>
> > > >> On Mon, Jan 8, 2018 at 4:22 PM, Dong Lin <li...@gmail.com>
> wrote:
> > > >>
> > > >> > Hey Jason,
> > > >> >
> > > >> > Certainly. This sounds good. I have updated the KIP to clarity
> that
> > > the
> > > >> > global epoch will be incremented by 1 each time a topic is
> deleted.
> > > >> >
> > > >> > Thanks,
> > > >> > Dong
> > > >> >
> > > >> > On Mon, Jan 8, 2018 at 4:09 PM, Jason Gustafson <
> jason@confluent.io
> > >
> > > >> > wrote:
> > > >> >
> > > >> > > Hi Dong,
> > > >> > >
> > > >> > >
> > > >> > > I think your approach will allow user to distinguish between the
> > > >> metadata
> > > >> > > > before and after the topic deletion. I also agree that this
> can
> > be
> > > >> > > > potentially be useful to user. I am just not very sure whether
> > we
> > > >> > already
> > > >> > > > have a good use-case to make the additional complexity
> > worthwhile.
> > > >> It
> > > >> > > seems
> > > >> > > > that this feature is kind of independent of the main problem
> of
> > > this
> > > >> > KIP.
> > > >> > > > Could we add this as a future work?
> > > >> > >
> > > >> > >
> > > >> > > Do you think it's fair if we bump the topic epoch on deletion
> and
> > > >> leave
> > > >> > > propagation of the epoch for deleted topics for future work? I
> > don't
> > > >> > think
> > > >> > > this adds much complexity and it makes the behavior consistent:
> > > every
> > > >> > topic
> > > >> > > mutation results in an epoch bump.
> > > >> > >
> > > >> > > Thanks,
> > > >> > > Jason
> > > >> > >
> > > >> > > On Mon, Jan 8, 2018 at 3:14 PM, Dong Lin <li...@gmail.com>
> > > wrote:
> > > >> > >
> > > >> > > > Hey Ismael,
> > > >> > > >
> > > >> > > > I guess we actually need user to see this field so that user
> can
> > > >> store
> > > >> > > this
> > > >> > > > value in the external store together with the offset. We just
> > > prefer
> > > >> > the
> > > >> > > > value to be opaque to discourage most users from interpreting
> > this
> > > >> > value.
> > > >> > > > One more advantage of using such an opaque field is to be able
> > to
> > > >> > evolve
> > > >> > > > the information (or schema) of this value without changing
> > > consumer
> > > >> API
> > > >> > > in
> > > >> > > > the future.
> > > >> > > >
> > > >> > > > I also thinking it is probably OK for user to be able to
> > interpret
> > > >> this
> > > >> > > > value, particularly for those advanced users.
> > > >> > > >
> > > >> > > > Thanks,
> > > >> > > > Dong
> > > >> > > >
> > > >> > > > On Mon, Jan 8, 2018 at 2:34 PM, Ismael Juma <
> ismael@juma.me.uk>
> > > >> wrote:
> > > >> > > >
> > > >> > > > > On Fri, Jan 5, 2018 at 7:15 PM, Jason Gustafson <
> > > >> jason@confluent.io>
> > > >> > > > > wrote:
> > > >> > > > > >
> > > >> > > > > > class OffsetAndMetadata {
> > > >> > > > > >   long offset;
> > > >> > > > > >   byte[] offsetMetadata;
> > > >> > > > > >   String metadata;
> > > >> > > > > > }
> > > >> > > > >
> > > >> > > > >
> > > >> > > > > > Admittedly, the naming is a bit annoying, but we can
> > probably
> > > >> come
> > > >> > up
> > > >> > > > > with
> > > >> > > > > > something better. Internally the byte array would have a
> > > >> version.
> > > >> > If
> > > >> > > in
> > > >> > > > > the
> > > >> > > > > > future we have anything else we need to add, we can update
> > the
> > > >> > > version
> > > >> > > > > and
> > > >> > > > > > we wouldn't need any new APIs.
> > > >> > > > > >
> > > >> > > > >
> > > >> > > > > We can also add fields to a class in a compatible way. So,
> it
> > > >> seems
> > > >> > to
> > > >> > > me
> > > >> > > > > that the main advantage of the byte array is that it's
> opaque
> > to
> > > >> the
> > > >> > > > user.
> > > >> > > > > Is that correct? If so, we could also add any opaque
> metadata
> > > in a
> > > >> > > > subclass
> > > >> > > > > so that users don't even see it (unless they cast it, but
> then
> > > >> > they're
> > > >> > > on
> > > >> > > > > their own).
> > > >> > > > >
> > > >> > > > > Ismael
> > > >> > > > >
> > > >> > > > > The corresponding seek() and position() APIs might look
> > > something
> > > >> > like
> > > >> > > > > this:
> > > >> > > > > >
> > > >> > > > > > void seek(TopicPartition partition, long offset, byte[]
> > > >> > > > offsetMetadata);
> > > >> > > > > > byte[] positionMetadata(TopicPartition partition);
> > > >> > > > > >
> > > >> > > > > > What do you think?
> > > >> > > > > >
> > > >> > > > > > Thanks,
> > > >> > > > > > Jason
> > > >> > > > > >
> > > >> > > > > > On Thu, Jan 4, 2018 at 7:04 PM, Dong Lin <
> > lindong28@gmail.com
> > > >
> > > >> > > wrote:
> > > >> > > > > >
> > > >> > > > > > > Hey Jun, Jason,
> > > >> > > > > > >
> > > >> > > > > > > Thanks much for all the feedback. I have updated the KIP
> > > >> based on
> > > >> > > the
> > > >> > > > > > > latest discussion. Can you help check whether it looks
> > good?
> > > >> > > > > > >
> > > >> > > > > > > Thanks,
> > > >> > > > > > > Dong
> > > >> > > > > > >
> > > >> > > > > > > On Thu, Jan 4, 2018 at 5:36 PM, Dong Lin <
> > > lindong28@gmail.com
> > > >> >
> > > >> > > > wrote:
> > > >> > > > > > >
> > > >> > > > > > > > Hey Jun,
> > > >> > > > > > > >
> > > >> > > > > > > > Hmm... thinking about this more, I am not sure that
> the
> > > >> > proposed
> > > >> > > > API
> > > >> > > > > is
> > > >> > > > > > > > sufficient. For users that store offset externally, we
> > > >> probably
> > > >> > > > need
> > > >> > > > > > > extra
> > > >> > > > > > > > API to return the leader_epoch and partition_epoch for
> > all
> > > >> > > > partitions
> > > >> > > > > > > that
> > > >> > > > > > > > consumers are consuming. I suppose these users
> currently
> > > use
> > > >> > > > > position()
> > > >> > > > > > > to
> > > >> > > > > > > > get the offset. Thus we probably need a new method
> > > >> > > > > > positionWithEpoch(..)
> > > >> > > > > > > to
> > > >> > > > > > > > return <offset, partition_epoch, leader_epoch>. Does
> > this
> > > >> sound
> > > >> > > > > > > reasonable?
> > > >> > > > > > > >
> > > >> > > > > > > > Thanks,
> > > >> > > > > > > > Dong
> > > >> > > > > > > >
> > > >> > > > > > > >
> > > >> > > > > > > > On Thu, Jan 4, 2018 at 5:26 PM, Jun Rao <
> > jun@confluent.io
> > > >
> > > >> > > wrote:
> > > >> > > > > > > >
> > > >> > > > > > > >> Hi, Dong,
> > > >> > > > > > > >>
> > > >> > > > > > > >> Yes, that's what I am thinking. OffsetEpoch will be
> > > >> composed
> > > >> > of
> > > >> > > > > > > >> (partition_epoch,
> > > >> > > > > > > >> leader_epoch).
> > > >> > > > > > > >>
> > > >> > > > > > > >> Thanks,
> > > >> > > > > > > >>
> > > >> > > > > > > >> Jun
> > > >> > > > > > > >>
> > > >> > > > > > > >>
> > > >> > > > > > > >> On Thu, Jan 4, 2018 at 4:22 PM, Dong Lin <
> > > >> lindong28@gmail.com
> > > >> > >
> > > >> > > > > wrote:
> > > >> > > > > > > >>
> > > >> > > > > > > >> > Hey Jun,
> > > >> > > > > > > >> >
> > > >> > > > > > > >> > Thanks much. I like the the new API that you
> > proposed.
> > > I
> > > >> am
> > > >> > > not
> > > >> > > > > sure
> > > >> > > > > > > >> what
> > > >> > > > > > > >> > you exactly mean by offset_epoch. I suppose that we
> > can
> > > >> use
> > > >> > > the
> > > >> > > > > pair
> > > >> > > > > > > of
> > > >> > > > > > > >> > (partition_epoch, leader_epoch) as the
> offset_epoch,
> > > >> right?
> > > >> > > > > > > >> >
> > > >> > > > > > > >> > Thanks,
> > > >> > > > > > > >> > Dong
> > > >> > > > > > > >> >
> > > >> > > > > > > >> > On Thu, Jan 4, 2018 at 4:02 PM, Jun Rao <
> > > >> jun@confluent.io>
> > > >> > > > wrote:
> > > >> > > > > > > >> >
> > > >> > > > > > > >> > > Hi, Dong,
> > > >> > > > > > > >> > >
> > > >> > > > > > > >> > > Got it. The api that you proposed works. The
> > question
> > > >> is
> > > >> > > > whether
> > > >> > > > > > > >> that's
> > > >> > > > > > > >> > the
> > > >> > > > > > > >> > > api that we want to have in the long term. My
> > concern
> > > >> is
> > > >> > > that
> > > >> > > > > > while
> > > >> > > > > > > >> the
> > > >> > > > > > > >> > api
> > > >> > > > > > > >> > > change is simple, the new api seems harder to
> > explain
> > > >> and
> > > >> > > use.
> > > >> > > > > For
> > > >> > > > > > > >> > example,
> > > >> > > > > > > >> > > a consumer storing offsets externally now needs
> to
> > > call
> > > >> > > > > > > >> > > waitForMetadataUpdate() after calling seek().
> > > >> > > > > > > >> > >
> > > >> > > > > > > >> > > An alternative approach is to make the following
> > > >> > compatible
> > > >> > > > api
> > > >> > > > > > > >> changes
> > > >> > > > > > > >> > in
> > > >> > > > > > > >> > > Consumer.
> > > >> > > > > > > >> > > * Add an additional OffsetEpoch field in
> > > >> > OffsetAndMetadata.
> > > >> > > > (no
> > > >> > > > > > need
> > > >> > > > > > > >> to
> > > >> > > > > > > >> > > change the CommitSync() api)
> > > >> > > > > > > >> > > * Add a new api seek(TopicPartition partition,
> long
> > > >> > offset,
> > > >> > > > > > > >> OffsetEpoch
> > > >> > > > > > > >> > > offsetEpoch). We can potentially deprecate the
> old
> > > api
> > > >> > > > > > > >> > seek(TopicPartition
> > > >> > > > > > > >> > > partition, long offset) in the future.
> > > >> > > > > > > >> > >
> > > >> > > > > > > >> > > The alternative approach has similar amount of
> api
> > > >> changes
> > > >> > > as
> > > >> > > > > > yours
> > > >> > > > > > > >> but
> > > >> > > > > > > >> > has
> > > >> > > > > > > >> > > the following benefits.
> > > >> > > > > > > >> > > 1. The api works in a similar way as how offset
> > > >> management
> > > >> > > > works
> > > >> > > > > > now
> > > >> > > > > > > >> and
> > > >> > > > > > > >> > is
> > > >> > > > > > > >> > > probably what we want in the long term.
> > > >> > > > > > > >> > > 2. It can reset offsets better when there is data
> > > loss
> > > >> due
> > > >> > > to
> > > >> > > > > > > unclean
> > > >> > > > > > > >> > > leader election or correlated replica failure.
> > > >> > > > > > > >> > > 3. It can reset offsets better when topic is
> > > recreated.
> > > >> > > > > > > >> > >
> > > >> > > > > > > >> > > Thanks,
> > > >> > > > > > > >> > >
> > > >> > > > > > > >> > > Jun
> > > >> > > > > > > >> > >
> > > >> > > > > > > >> > >
> > > >> > > > > > > >> > > On Thu, Jan 4, 2018 at 2:05 PM, Dong Lin <
> > > >> > > lindong28@gmail.com
> > > >> > > > >
> > > >> > > > > > > wrote:
> > > >> > > > > > > >> > >
> > > >> > > > > > > >> > > > Hey Jun,
> > > >> > > > > > > >> > > >
> > > >> > > > > > > >> > > > Yeah I agree that ideally we don't want an ever
> > > >> growing
> > > >> > > > global
> > > >> > > > > > > >> metadata
> > > >> > > > > > > >> > > > version. I just think it may be more desirable
> to
> > > >> keep
> > > >> > the
> > > >> > > > > > > consumer
> > > >> > > > > > > >> API
> > > >> > > > > > > >> > > > simple.
> > > >> > > > > > > >> > > >
> > > >> > > > > > > >> > > > In my current proposal, metadata version
> returned
> > > in
> > > >> the
> > > >> > > > fetch
> > > >> > > > > > > >> response
> > > >> > > > > > > >> > > > will be stored with the offset together. More
> > > >> > > specifically,
> > > >> > > > > the
> > > >> > > > > > > >> > > > metadata_epoch in the new offset topic schema
> > will
> > > be
> > > >> > the
> > > >> > > > > > largest
> > > >> > > > > > > >> > > > metadata_epoch from all the MetadataResponse
> and
> > > >> > > > FetchResponse
> > > >> > > > > > > ever
> > > >> > > > > > > >> > > > received by this consumer.
> > > >> > > > > > > >> > > >
> > > >> > > > > > > >> > > > We probably don't have to change the consumer
> API
> > > for
> > > >> > > > > > > >> > > > commitSync(Map<TopicPartition,
> > OffsetAndMetadata>).
> > > >> If
> > > >> > > user
> > > >> > > > > > calls
> > > >> > > > > > > >> > > > commitSync(...) to commit offset 10 for a given
> > > >> > partition,
> > > >> > > > for
> > > >> > > > > > > most
> > > >> > > > > > > >> > > > use-cases, this consumer instance should have
> > > >> consumed
> > > >> > > > message
> > > >> > > > > > > with
> > > >> > > > > > > >> > > offset
> > > >> > > > > > > >> > > > 9 from this partition, in which case the
> consumer
> > > can
> > > >> > > > remember
> > > >> > > > > > and
> > > >> > > > > > > >> use
> > > >> > > > > > > >> > > the
> > > >> > > > > > > >> > > > metadata_epoch from the corresponding
> > FetchResponse
> > > >> when
> > > >> > > > > > > committing
> > > >> > > > > > > >> > > offset.
> > > >> > > > > > > >> > > > If user calls commitSync(..) to commit offset
> 10
> > > for
> > > >> a
> > > >> > > given
> > > >> > > > > > > >> partition
> > > >> > > > > > > >> > > > without having consumed the message with
> offset 9
> > > >> using
> > > >> > > this
> > > >> > > > > > > >> consumer
> > > >> > > > > > > >> > > > instance, this is probably an advanced
> use-case.
> > In
> > > >> this
> > > >> > > > case
> > > >> > > > > > the
> > > >> > > > > > > >> > > advanced
> > > >> > > > > > > >> > > > user can retrieve the metadata_epoch using the
> > > newly
> > > >> > added
> > > >> > > > > > > >> > > metadataEpoch()
> > > >> > > > > > > >> > > > API after it fetches the message with offset 9
> > > >> (probably
> > > >> > > > from
> > > >> > > > > > > >> another
> > > >> > > > > > > >> > > > consumer instance) and encode this
> metadata_epoch
> > > in
> > > >> the
> > > >> > > > > > > >> > > > string OffsetAndMetadata.metadata. Do you think
> > > this
> > > >> > > > solution
> > > >> > > > > > > would
> > > >> > > > > > > >> > work?
> > > >> > > > > > > >> > > >
> > > >> > > > > > > >> > > > By "not sure that I fully understand your
> latest
> > > >> > > > suggestion",
> > > >> > > > > > are
> > > >> > > > > > > >> you
> > > >> > > > > > > >> > > > referring to solution related to unclean leader
> > > >> election
> > > >> > > > using
> > > >> > > > > > > >> > > leader_epoch
> > > >> > > > > > > >> > > > in my previous email?
> > > >> > > > > > > >> > > >
> > > >> > > > > > > >> > > > Thanks,
> > > >> > > > > > > >> > > > Dong
> > > >> > > > > > > >> > > >
> > > >> > > > > > > >> > > > On Thu, Jan 4, 2018 at 1:33 PM, Jun Rao <
> > > >> > jun@confluent.io
> > > >> > > >
> > > >> > > > > > wrote:
> > > >> > > > > > > >> > > >
> > > >> > > > > > > >> > > > > Hi, Dong,
> > > >> > > > > > > >> > > > >
> > > >> > > > > > > >> > > > > Not sure that I fully understand your latest
> > > >> > suggestion.
> > > >> > > > > > > >> Returning an
> > > >> > > > > > > >> > > > ever
> > > >> > > > > > > >> > > > > growing global metadata version itself is no
> > > ideal,
> > > >> > but
> > > >> > > is
> > > >> > > > > > fine.
> > > >> > > > > > > >> My
> > > >> > > > > > > >> > > > > question is whether the metadata version
> > returned
> > > >> in
> > > >> > the
> > > >> > > > > fetch
> > > >> > > > > > > >> > response
> > > >> > > > > > > >> > > > > needs to be stored with the offset together
> if
> > > >> offsets
> > > >> > > are
> > > >> > > > > > > stored
> > > >> > > > > > > >> > > > > externally. If so, we also have to change the
> > > >> consumer
> > > >> > > API
> > > >> > > > > for
> > > >> > > > > > > >> > > > commitSync()
> > > >> > > > > > > >> > > > > and need to worry about compatibility. If we
> > > don't
> > > >> > store
> > > >> > > > the
> > > >> > > > > > > >> metadata
> > > >> > > > > > > >> > > > > version together with the offset, on a
> consumer
> > > >> > restart,
> > > >> > > > > it's
> > > >> > > > > > > not
> > > >> > > > > > > >> > clear
> > > >> > > > > > > >> > > > how
> > > >> > > > > > > >> > > > > we can ensure the metadata in the consumer is
> > > high
> > > >> > > enough
> > > >> > > > > > since
> > > >> > > > > > > >> there
> > > >> > > > > > > >> > > is
> > > >> > > > > > > >> > > > no
> > > >> > > > > > > >> > > > > metadata version to compare with.
> > > >> > > > > > > >> > > > >
> > > >> > > > > > > >> > > > > Thanks,
> > > >> > > > > > > >> > > > >
> > > >> > > > > > > >> > > > > Jun
> > > >> > > > > > > >> > > > >
> > > >> > > > > > > >> > > > >
> > > >> > > > > > > >> > > > > On Wed, Jan 3, 2018 at 6:43 PM, Dong Lin <
> > > >> > > > > lindong28@gmail.com
> > > >> > > > > > >
> > > >> > > > > > > >> > wrote:
> > > >> > > > > > > >> > > > >
> > > >> > > > > > > >> > > > > > Hey Jun,
> > > >> > > > > > > >> > > > > >
> > > >> > > > > > > >> > > > > > Thanks much for the explanation.
> > > >> > > > > > > >> > > > > >
> > > >> > > > > > > >> > > > > > I understand the advantage of
> partition_epoch
> > > >> over
> > > >> > > > > > > >> metadata_epoch.
> > > >> > > > > > > >> > My
> > > >> > > > > > > >> > > > > > current concern is that the use of
> > leader_epoch
> > > >> and
> > > >> > > the
> > > >> > > > > > > >> > > partition_epoch
> > > >> > > > > > > >> > > > > > requires us considerable change to
> consumer's
> > > >> public
> > > >> > > API
> > > >> > > > > to
> > > >> > > > > > > take
> > > >> > > > > > > >> > care
> > > >> > > > > > > >> > > > of
> > > >> > > > > > > >> > > > > > the case where user stores offset
> externally.
> > > For
> > > >> > > > example,
> > > >> > > > > > > >> > > *consumer*.
> > > >> > > > > > > >> > > > > > *commitSync*(..) would have to take a map
> > whose
> > > >> > value
> > > >> > > is
> > > >> > > > > > > >> <offset,
> > > >> > > > > > > >> > > > > metadata,
> > > >> > > > > > > >> > > > > > leader epoch, partition epoch>.
> > > >> > *consumer*.*seek*(...)
> > > >> > > > > would
> > > >> > > > > > > >> also
> > > >> > > > > > > >> > > need
> > > >> > > > > > > >> > > > > > leader_epoch and partition_epoch as
> > parameter.
> > > >> > > > Technically
> > > >> > > > > > we
> > > >> > > > > > > >> can
> > > >> > > > > > > >> > > > > probably
> > > >> > > > > > > >> > > > > > still make it work in a backward compatible
> > > >> manner
> > > >> > > after
> > > >> > > > > > > careful
> > > >> > > > > > > >> > > design
> > > >> > > > > > > >> > > > > and
> > > >> > > > > > > >> > > > > > discussion. But these changes can make the
> > > >> > consumer's
> > > >> > > > > > > interface
> > > >> > > > > > > >> > > > > > unnecessarily complex for more users who do
> > not
> > > >> > store
> > > >> > > > > offset
> > > >> > > > > > > >> > > > externally.
> > > >> > > > > > > >> > > > > >
> > > >> > > > > > > >> > > > > > After thinking more about it, we can
> address
> > > all
> > > >> > > > problems
> > > >> > > > > > > >> discussed
> > > >> > > > > > > >> > > by
> > > >> > > > > > > >> > > > > only
> > > >> > > > > > > >> > > > > > using the metadata_epoch without
> introducing
> > > >> > > > leader_epoch
> > > >> > > > > or
> > > >> > > > > > > the
> > > >> > > > > > > >> > > > > > partition_epoch. The current KIP describes
> > the
> > > >> > changes
> > > >> > > > to
> > > >> > > > > > the
> > > >> > > > > > > >> > > consumer
> > > >> > > > > > > >> > > > > API
> > > >> > > > > > > >> > > > > > and how the new API can be used if user
> > stores
> > > >> > offset
> > > >> > > > > > > >> externally.
> > > >> > > > > > > >> > In
> > > >> > > > > > > >> > > > > order
> > > >> > > > > > > >> > > > > > to address the scenario you described
> > earlier,
> > > we
> > > >> > can
> > > >> > > > > > include
> > > >> > > > > > > >> > > > > > metadata_epoch in the FetchResponse and the
> > > >> > > > > > > LeaderAndIsrRequest.
> > > >> > > > > > > >> > > > Consumer
> > > >> > > > > > > >> > > > > > remembers the largest metadata_epoch from
> all
> > > the
> > > >> > > > > > > FetchResponse
> > > >> > > > > > > >> it
> > > >> > > > > > > >> > > has
> > > >> > > > > > > >> > > > > > received. The metadata_epoch committed with
> > the
> > > >> > > offset,
> > > >> > > > > > either
> > > >> > > > > > > >> > within
> > > >> > > > > > > >> > > > or
> > > >> > > > > > > >> > > > > > outside Kafka, should be the largest
> > > >> metadata_epoch
> > > >> > > > across
> > > >> > > > > > all
> > > >> > > > > > > >> > > > > > FetchResponse and MetadataResponse ever
> > > received
> > > >> by
> > > >> > > this
> > > >> > > > > > > >> consumer.
> > > >> > > > > > > >> > > > > >
> > > >> > > > > > > >> > > > > > The drawback of using only the
> metadata_epoch
> > > is
> > > >> > that
> > > >> > > we
> > > >> > > > > can
> > > >> > > > > > > not
> > > >> > > > > > > >> > > always
> > > >> > > > > > > >> > > > > do
> > > >> > > > > > > >> > > > > > the smart offset reset in case of unclean
> > > leader
> > > >> > > > election
> > > >> > > > > > > which
> > > >> > > > > > > >> you
> > > >> > > > > > > >> > > > > > mentioned earlier. But in most case,
> unclean
> > > >> leader
> > > >> > > > > election
> > > >> > > > > > > >> > probably
> > > >> > > > > > > >> > > > > > happens when consumer is not
> > > >> rebalancing/restarting.
> > > >> > > In
> > > >> > > > > > these
> > > >> > > > > > > >> > cases,
> > > >> > > > > > > >> > > > > either
> > > >> > > > > > > >> > > > > > consumer is not directly affected by
> unclean
> > > >> leader
> > > >> > > > > election
> > > >> > > > > > > >> since
> > > >> > > > > > > >> > it
> > > >> > > > > > > >> > > > is
> > > >> > > > > > > >> > > > > > not consuming from the end of the log, or
> > > >> consumer
> > > >> > can
> > > >> > > > > > derive
> > > >> > > > > > > >> the
> > > >> > > > > > > >> > > > > > leader_epoch from the most recent message
> > > >> received
> > > >> > > > before
> > > >> > > > > it
> > > >> > > > > > > >> sees
> > > >> > > > > > > >> > > > > > OffsetOutOfRangeException. So I am not sure
> > it
> > > is
> > > >> > > worth
> > > >> > > > > > adding
> > > >> > > > > > > >> the
> > > >> > > > > > > >> > > > > > leader_epoch to consumer API to address the
> > > >> > remaining
> > > >> > > > > corner
> > > >> > > > > > > >> case.
> > > >> > > > > > > >> > > What
> > > >> > > > > > > >> > > > > do
> > > >> > > > > > > >> > > > > > you think?
> > > >> > > > > > > >> > > > > >
> > > >> > > > > > > >> > > > > > Thanks,
> > > >> > > > > > > >> > > > > > Dong
> > > >> > > > > > > >> > > > > >
> > > >> > > > > > > >> > > > > >
> > > >> > > > > > > >> > > > > >
> > > >> > > > > > > >> > > > > > On Tue, Jan 2, 2018 at 6:28 PM, Jun Rao <
> > > >> > > > jun@confluent.io
> > > >> > > > > >
> > > >> > > > > > > >> wrote:
> > > >> > > > > > > >> > > > > >
> > > >> > > > > > > >> > > > > > > Hi, Dong,
> > > >> > > > > > > >> > > > > > >
> > > >> > > > > > > >> > > > > > > Thanks for the reply.
> > > >> > > > > > > >> > > > > > >
> > > >> > > > > > > >> > > > > > > To solve the topic recreation issue, we
> > could
> > > >> use
> > > >> > > > > either a
> > > >> > > > > > > >> global
> > > >> > > > > > > >> > > > > > metadata
> > > >> > > > > > > >> > > > > > > version or a partition level epoch. But
> > > either
> > > >> one
> > > >> > > > will
> > > >> > > > > > be a
> > > >> > > > > > > >> new
> > > >> > > > > > > >> > > > > concept,
> > > >> > > > > > > >> > > > > > > right? To me, the latter seems more
> > natural.
> > > It
> > > >> > also
> > > >> > > > > makes
> > > >> > > > > > > it
> > > >> > > > > > > >> > > easier
> > > >> > > > > > > >> > > > to
> > > >> > > > > > > >> > > > > > > detect if a consumer's offset is still
> > valid
> > > >> > after a
> > > >> > > > > topic
> > > >> > > > > > > is
> > > >> > > > > > > >> > > > > recreated.
> > > >> > > > > > > >> > > > > > As
> > > >> > > > > > > >> > > > > > > you pointed out, we don't need to store
> the
> > > >> > > partition
> > > >> > > > > > epoch
> > > >> > > > > > > in
> > > >> > > > > > > >> > the
> > > >> > > > > > > >> > > > > > message.
> > > >> > > > > > > >> > > > > > > The following is what I am thinking.
> When a
> > > >> > > partition
> > > >> > > > is
> > > >> > > > > > > >> created,
> > > >> > > > > > > >> > > we
> > > >> > > > > > > >> > > > > can
> > > >> > > > > > > >> > > > > > > assign a partition epoch from an
> > > >> ever-increasing
> > > >> > > > global
> > > >> > > > > > > >> counter
> > > >> > > > > > > >> > and
> > > >> > > > > > > >> > > > > store
> > > >> > > > > > > >> > > > > > > it in /brokers/topics/[topic]/
> > > >> > > > partitions/[partitionId]
> > > >> > > > > in
> > > >> > > > > > > ZK.
> > > >> > > > > > > >> > The
> > > >> > > > > > > >> > > > > > > partition
> > > >> > > > > > > >> > > > > > > epoch is propagated to every broker. The
> > > >> consumer
> > > >> > > will
> > > >> > > > > be
> > > >> > > > > > > >> > tracking
> > > >> > > > > > > >> > > a
> > > >> > > > > > > >> > > > > > tuple
> > > >> > > > > > > >> > > > > > > of <offset, leader epoch, partition
> epoch>
> > > for
> > > >> > > > offsets.
> > > >> > > > > > If a
> > > >> > > > > > > >> > topic
> > > >> > > > > > > >> > > is
> > > >> > > > > > > >> > > > > > > recreated, it's possible that a
> consumer's
> > > >> offset
> > > >> > > and
> > > >> > > > > > leader
> > > >> > > > > > > >> > epoch
> > > >> > > > > > > >> > > > > still
> > > >> > > > > > > >> > > > > > > match that in the broker, but partition
> > epoch
> > > >> > won't
> > > >> > > > be.
> > > >> > > > > In
> > > >> > > > > > > >> this
> > > >> > > > > > > >> > > case,
> > > >> > > > > > > >> > > > > we
> > > >> > > > > > > >> > > > > > > can potentially still treat the
> consumer's
> > > >> offset
> > > >> > as
> > > >> > > > out
> > > >> > > > > > of
> > > >> > > > > > > >> range
> > > >> > > > > > > >> > > and
> > > >> > > > > > > >> > > > > > reset
> > > >> > > > > > > >> > > > > > > the offset based on the offset reset
> policy
> > > in
> > > >> the
> > > >> > > > > > consumer.
> > > >> > > > > > > >> This
> > > >> > > > > > > >> > > > seems
> > > >> > > > > > > >> > > > > > > harder to do with a global metadata
> > version.
> > > >> > > > > > > >> > > > > > >
> > > >> > > > > > > >> > > > > > > Jun
> > > >> > > > > > > >> > > > > > >
> > > >> > > > > > > >> > > > > > >
> > > >> > > > > > > >> > > > > > >
> > > >> > > > > > > >> > > > > > > On Mon, Dec 25, 2017 at 6:56 AM, Dong
> Lin <
> > > >> > > > > > > >> lindong28@gmail.com>
> > > >> > > > > > > >> > > > wrote:
> > > >> > > > > > > >> > > > > > >
> > > >> > > > > > > >> > > > > > > > Hey Jun,
> > > >> > > > > > > >> > > > > > > >
> > > >> > > > > > > >> > > > > > > > This is a very good example. After
> > thinking
> > > >> > > through
> > > >> > > > > this
> > > >> > > > > > > in
> > > >> > > > > > > >> > > > detail, I
> > > >> > > > > > > >> > > > > > > agree
> > > >> > > > > > > >> > > > > > > > that we need to commit offset with
> leader
> > > >> epoch
> > > >> > in
> > > >> > > > > order
> > > >> > > > > > > to
> > > >> > > > > > > >> > > address
> > > >> > > > > > > >> > > > > > this
> > > >> > > > > > > >> > > > > > > > example.
> > > >> > > > > > > >> > > > > > > >
> > > >> > > > > > > >> > > > > > > > I think the remaining question is how
> to
> > > >> address
> > > >> > > the
> > > >> > > > > > > >> scenario
> > > >> > > > > > > >> > > that
> > > >> > > > > > > >> > > > > the
> > > >> > > > > > > >> > > > > > > > topic is deleted and re-created. One
> > > possible
> > > >> > > > solution
> > > >> > > > > > is
> > > >> > > > > > > to
> > > >> > > > > > > >> > > commit
> > > >> > > > > > > >> > > > > > > offset
> > > >> > > > > > > >> > > > > > > > with both the leader epoch and the
> > metadata
> > > >> > > version.
> > > >> > > > > The
> > > >> > > > > > > >> logic
> > > >> > > > > > > >> > > and
> > > >> > > > > > > >> > > > > the
> > > >> > > > > > > >> > > > > > > > implementation of this solution does
> not
> > > >> > require a
> > > >> > > > new
> > > >> > > > > > > >> concept
> > > >> > > > > > > >> > > > (e.g.
> > > >> > > > > > > >> > > > > > > > partition epoch) and it does not
> require
> > > any
> > > >> > > change
> > > >> > > > to
> > > >> > > > > > the
> > > >> > > > > > > >> > > message
> > > >> > > > > > > >> > > > > > format
> > > >> > > > > > > >> > > > > > > > or leader epoch. It also allows us to
> > order
> > > >> the
> > > >> > > > > metadata
> > > >> > > > > > > in
> > > >> > > > > > > >> a
> > > >> > > > > > > >> > > > > > > > straightforward manner which may be
> > useful
> > > in
> > > >> > the
> > > >> > > > > > future.
> > > >> > > > > > > >> So it
> > > >> > > > > > > >> > > may
> > > >> > > > > > > >> > > > > be
> > > >> > > > > > > >> > > > > > a
> > > >> > > > > > > >> > > > > > > > better solution than generating a
> random
> > > >> > partition
> > > >> > > > > epoch
> > > >> > > > > > > >> every
> > > >> > > > > > > >> > > time
> > > >> > > > > > > >> > > > > we
> > > >> > > > > > > >> > > > > > > > create a partition. Does this sound
> > > >> reasonable?
> > > >> > > > > > > >> > > > > > > >
> > > >> > > > > > > >> > > > > > > > Previously one concern with using the
> > > >> metadata
> > > >> > > > version
> > > >> > > > > > is
> > > >> > > > > > > >> that
> > > >> > > > > > > >> > > > > consumer
> > > >> > > > > > > >> > > > > > > > will be forced to refresh metadata even
> > if
> > > >> > > metadata
> > > >> > > > > > > version
> > > >> > > > > > > >> is
> > > >> > > > > > > >> > > > > > increased
> > > >> > > > > > > >> > > > > > > > due to topics that the consumer is not
> > > >> > interested
> > > >> > > > in.
> > > >> > > > > > Now
> > > >> > > > > > > I
> > > >> > > > > > > >> > > > realized
> > > >> > > > > > > >> > > > > > that
> > > >> > > > > > > >> > > > > > > > this is probably not a problem.
> Currently
> > > >> client
> > > >> > > > will
> > > >> > > > > > > >> refresh
> > > >> > > > > > > >> > > > > metadata
> > > >> > > > > > > >> > > > > > > > either due to InvalidMetadataException
> in
> > > the
> > > >> > > > response
> > > >> > > > > > > from
> > > >> > > > > > > >> > > broker
> > > >> > > > > > > >> > > > or
> > > >> > > > > > > >> > > > > > due
> > > >> > > > > > > >> > > > > > > > to metadata expiry. The addition of the
> > > >> metadata
> > > >> > > > > version
> > > >> > > > > > > >> should
> > > >> > > > > > > >> > > > > > increase
> > > >> > > > > > > >> > > > > > > > the overhead of metadata refresh caused
> > by
> > > >> > > > > > > >> > > > InvalidMetadataException.
> > > >> > > > > > > >> > > > > If
> > > >> > > > > > > >> > > > > > > > client refresh metadata due to expiry
> and
> > > it
> > > >> > > > receives
> > > >> > > > > a
> > > >> > > > > > > >> > metadata
> > > >> > > > > > > >> > > > > whose
> > > >> > > > > > > >> > > > > > > > version is lower than the current
> > metadata
> > > >> > > version,
> > > >> > > > we
> > > >> > > > > > can
> > > >> > > > > > > >> > reject
> > > >> > > > > > > >> > > > the
> > > >> > > > > > > >> > > > > > > > metadata but still reset the metadata
> > age,
> > > >> which
> > > >> > > > > > > essentially
> > > >> > > > > > > >> > keep
> > > >> > > > > > > >> > > > the
> > > >> > > > > > > >> > > > > > > > existing behavior in the client.
> > > >> > > > > > > >> > > > > > > >
> > > >> > > > > > > >> > > > > > > > Thanks much,
> > > >> > > > > > > >> > > > > > > > Dong
> > > >> > > > > > > >> > > > > > > >
> > > >> > > > > > > >> > > > > > >
> > > >> > > > > > > >> > > > > >
> > > >> > > > > > > >> > > > >
> > > >> > > > > > > >> > > >
> > > >> > > > > > > >> > >
> > > >> > > > > > > >> >
> > > >> > > > > > > >>
> > > >> > > > > > > >
> > > >> > > > > > > >
> > > >> > > > > > >
> > > >> > > > > >
> > > >> > > > >
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Dong Lin <li...@gmail.com>.
Hey Jun,

Thanks much. I agree that we can not rely on committed offsets to be always
deleted when we delete topic. So it is necessary to use a per-partition
epoch that does not change unless this partition is deleted. I also agree
that it is very nice to be able to uniquely identify a message with
(offset, leader_epoch, partition_epoch) in face of potential topic deletion
and unclean leader election.

I agree with all your comments. And I have updated the KIP based on our
latest discussion. In addition, I added InvalidPartitionEpochException
which will be thrown by consumer.poll() if the partition_epoch associated
with the partition, which can be given to consumer using seek(...), is
different from the partition_epoch in the FetchResponse.

Can you take another look at the latest KIP?

Thanks!
Dong



On Wed, Jan 10, 2018 at 2:24 PM, Jun Rao <ju...@confluent.io> wrote:

> Hi, Dong,
>
> My replies are the following.
>
> 60. What you described could also work. The drawback is that we will be
> unnecessarily changing the partition epoch when a partition hasn't really
> changed. I was imagining that the partition epoch will be stored in
> /brokers/topics/[topic]/partitions/[partitionId], instead of at the topic
> level. So, not sure if ZK size limit is an issue.
>
> 61, 62 and 65. To me, the offset + offset_epoch is a unique identifier for
> a message. So, if a message hasn't changed, the offset and the associated
> offset_epoch ideally should remain the same (it will be kind of weird if
> two consumer apps save the offset on the same message, but the offset_epoch
> are different). partition_epoch + leader_epoch give us that. global_epoch +
> leader_epoch don't. If we use this approach, we can solve not only the
> problem that you have identified, but also other problems when there is
> data loss or topic re-creation more reliably. For example, in the future,
> if we include the partition_epoch and leader_epoch in the fetch request,
> the server can do a more reliable check of whether that offset is valid or
> not. I am not sure that we can rely upon all external offsets to be removed
> on topic deletion. For example, a topic may be deleted by an admin who may
> not know all the applications.
>
> If we agree on the above, the second question is then how to reliably
> propagate the partition_epoch and the leader_epoch to the consumer when
> there are leader or partition changes. The leader_epoch comes from the
> message, which is reliable. So, I was suggesting that when we store an
> offset, we can just store the leader_epoch from the message set containing
> that offset. Similarly, I was thinking that if the partition_epoch is in
> the fetch response, we can propagate partition_epoch reliably where is
> partition_epoch change.
>
> 63. My point is that once a leader is producing a message in the new
> partition_epoch, ideally, we should associate the new offsets with the new
> partition_epoch. Otherwise, the offset_epoch won't be the correct unique
> identifier (useful for solving other problems mentioned above). I was
> originally thinking that the leader will include the partition_epoch in the
> metadata cache in the fetch response. It's just that right now, metadata
> cache is updated on UpdateMetadataRequest, which typically happens after
> the LeaderAndIsrRequest. Another approach is for the leader to cache the
> partition_epoch in the Partition object and return that (instead of the one
> in metadata cache) in the fetch response.
>
> 65. It seems to me that the global_epoch and the partition_epoch have
> different purposes. A partition_epoch has the benefit that it (1) can be
> used to form a unique identifier for a message and (2) can be used to
> solve other
> corner case problems in the future. I am not sure having just a
> global_epoch can achieve these. global_epoch is useful to determine which
> version of the metadata is newer, especially with topic deletion.
>
> Thanks,
>
> Jun
>
> On Tue, Jan 9, 2018 at 11:34 PM, Dong Lin <li...@gmail.com> wrote:
>
> > Regarding the use of the global epoch in 65), it is very similar to the
> > proposal of the metadata_epoch we discussed earlier. The main difference
> is
> > that this epoch is incremented when we create/expand/delete topic and
> does
> > not change when controller re-send metadata.
> >
> > I looked at our previous discussion. It seems that we prefer
> > partition_epoch over the metadata_epoch because 1) we prefer not to have
> an
> > ever growing metadata_epoch and 2) we can reset offset better when topic
> is
> > re-created. The use of global topic_epoch avoids the drawback of an ever
> > quickly ever growing metadata_epoch. Though the global epoch does not
> allow
> > us to recognize the invalid offset committed before the topic
> re-creation,
> > we can probably just delete the offset when we delete a topic. Thus I am
> > not very sure whether it is still worthwhile to have a per-partition
> > partition_epoch if the metadata already has the global epoch.
> >
> >
> > On Tue, Jan 9, 2018 at 6:58 PM, Dong Lin <li...@gmail.com> wrote:
> >
> > > Hey Jun,
> > >
> > > Thanks so much. These comments very useful. Please see below my
> comments.
> > >
> > > On Mon, Jan 8, 2018 at 5:52 PM, Jun Rao <ju...@confluent.io> wrote:
> > >
> > >> Hi, Dong,
> > >>
> > >> Thanks for the updated KIP. A few more comments.
> > >>
> > >> 60. Perhaps having a partition epoch is more flexible since in the
> > future,
> > >> we may support deleting a partition as well.
> > >>
> > >
> > > Yeah I have considered this. I think we can probably still support
> > > deleting a partition by using the topic_epoch -- when partition of a
> > topic
> > > is deleted or created, epoch of all partitions of this topic will be
> > > incremented by 1. Therefore, if that partition is re-created later, the
> > > epoch of that partition will still be larger than its epoch before the
> > > deletion, which still allows the client to order the metadata for the
> > > purpose of this KIP. Does this sound reasonable?
> > >
> > > The advantage of using topic_epoch instead of partition_epoch is that
> the
> > > size of the /brokers/topics/[topic] znode and request/response size can
> > be
> > > smaller. We have a limit on the maximum size of znode (typically 1MB).
> > Use
> > > partition epoch can effectively reduce the number of partitions that
> can
> > be
> > > described by the /brokers/topics/[topic] znode.
> > >
> > > One use-case of partition_epoch for client to detect that the committed
> > > offset, either from kafka offset topic or from the external store is
> > > invalid after partition deletion and re-creation. However, it seems
> that
> > we
> > > can also address this use-case with other approaches. For example, when
> > > AdminClient deletes partitions, it can also delete the committed
> offsets
> > > for those partitions from the offset topic. If user stores offset
> > > externally, it might make sense for user to similarly remove offsets of
> > > related partitions after these partitions are deleted. So I am not sure
> > > that we should use partition_epoch in this KIP.
> > >
> > >
> > >>
> > >> 61. It seems that the leader epoch returned in the position() call
> > should
> > >> the the leader epoch returned in the fetch response, not the one in
> the
> > >> metadata cache of the client.
> > >
> > >
> > > I think this is a good idea. Just to double check, this change does not
> > > affect the correctness or performance of this KIP. But it can be useful
> > if
> > > we want to use the leader_epoch to better handle the offset rest in
> case
> > of
> > > unclean leader election, which is listed in the future work. Is this
> > > understanding correct?
> > >
> > > I have updated the KIP to specify that the leader_epoch returned by
> > > position() should be the largest leader_epoch of those already consumed
> > > messages whose offset < position. If no message has been consumed since
> > > consumer initialization, the leader_epoch from seek() or
> > > OffsetFetchResponse should be used. The offset included in the
> > > OffsetCommitRequest will also be determined in the similar manner.
> > >
> > >
> > >>
> > >> 62. I am wondering if we should return the partition epoch in the
> fetch
> > >> response as well. In the current proposal, if a topic is recreated and
> > the
> > >> new leader is on the same broker as the old one, there is nothing to
> > force
> > >> the metadata refresh in the client. So, the client may still associate
> > the
> > >> offset with the old partition epoch.
> > >>
> > >
> > > Could you help me understand the problem if a client associates old
> > > partition_epoch (or the topic_epoch as of the current KIP) with the
> > offset?
> > > The main purpose of the topic_epoch is to be able to drop leader_epoch
> > to 0
> > > after a partition is deleted and re-created. I guess you may be
> thinking
> > > about using the partition_epoch to detect that the committed offset is
> > > invalid? In that case, I am wondering if the alternative approach
> > described
> > > in 60) would be reasonable.
> > >
> > >
> > >>
> > >> 63. There is some subtle coordination between the LeaderAndIsrRequest
> > and
> > >> UpdateMetadataRequest. Currently, when a leader changes, the
> controller
> > >> first sends the LeaderAndIsrRequest to the assigned replicas and the
> > >> UpdateMetadataRequest to every broker. So, there could be a small
> window
> > >> when the leader already receives the new partition epoch in the
> > >> LeaderAndIsrRequest, but the metadata cache in the broker hasn't been
> > >> updated with the latest partition epoch. Not sure what's the best way
> to
> > >> address this issue. Perhaps we can update the metadata cache on the
> > broker
> > >> with both LeaderAndIsrRequest and UpdateMetadataRequest. The challenge
> > is
> > >> that the two have slightly different data. For example, only the
> latter
> > >> has
> > >> all endpoints.
> > >>
> > >
> > > I am not sure whether this is a problem. Could you explain a bit more
> > what
> > > specific problem this small window can cause?
> > >
> > > Since client can fetch metadata from any broker in the cluster, and
> given
> > > that different brokers receive request (e.g. LeaderAndIsrRequest and
> > > UpdateMetadataRequest) in arbitrary order, the metadata received by
> > client
> > > can be in arbitrary order (either newer or older) compared to the
> > broker's
> > > leadership state even if a given broker receives LeaderAndIsrRequest
> and
> > > UpdateMetadataRequest simultaneously. So I am not sure it is useful to
> > > update broker's cache with LeaderAndIsrRequest.
> > >
> > >
> > >> 64. The enforcement of leader epoch in Offset commit: We allow a
> > consumer
> > >> to set an arbitrary offset. So it's possible for offsets or leader
> epoch
> > >> to
> > >> go backwards. I am not sure if we could always enforce that the leader
> > >> epoch only goes up on the broker.
> > >>
> > >
> > > Sure. I have removed this check from the KIP.
> > >
> > > BTW, we can probably still ensure that the leader_epoch always increase
> > if
> > > the leader_epoch used with offset commit is the max(leader_epoch of the
> > > message with offset = the committed offset - 1, the largest known
> > > leader_epoch from the metadata). But I don't have a good use-case for
> > this
> > > alternative definition. So I choose the keep the KIP simple by
> requiring
> > > leader_epoch to always increase.
> > >
> > >
> > >> 65. Good point on handling missing partition epoch due to topic
> > deletion.
> > >> Another potential way to address this is to additionally propagate the
> > >> global partition epoch to brokers and the clients. This way, when a
> > >> partition epoch is missing, we can use the global partition epoch to
> > >> reason
> > >> about which metadata is more recent.
> > >>
> > >
> > > This is a great idea. The global epoch can be used to order the
> metadata
> > > and help us recognize the more recent metadata if a topic (or
> partition)
> > is
> > > deleted and re-created.
> > >
> > > Actually, it seems we only need to propagate the global epoch to
> brokers
> > > and clients without propagating this epoch on a per-topic or
> > per-partition
> > > basic. Doing so would simply interface changes made this KIP. Does this
> > > approach sound reasonable?
> > >
> > >
> > >> 66. A client may also get an offset by time using the offsetForTimes()
> > >> api.
> > >> So, we probably want to include offsetInternalMetadata in
> > >> OffsetAndTimestamp
> > >> as well.
> > >>
> > >
> > > You are right. This probably also requires us to change the
> > > ListOffsetRequest as well. I will update the KIP after we agree on the
> > > solution for 65).
> > >
> > >
> > >>
> > >> 67. InteralMetadata can be a bit confusing with the metadata field
> > already
> > >> there. Perhaps we can just call it OffsetEpoch. It might be useful to
> > make
> > >> OffsetEpoch printable at least for debugging purpose. Once you do
> that,
> > we
> > >> are already exposing the internal fields. So, not sure if it's worth
> > >> hiding
> > >> them. If we do want to hide them, perhaps we can have sth like the
> > >> following. The binary encoding is probably more efficient than JSON
> for
> > >> external storage.
> > >>
> > >> OffsetEpoch {
> > >>  static OffsetEpoch decode(byte[]);
> > >>
> > >>   public byte[] encode();
> > >>
> > >>   public String toString();
> > >> }
> > >>
> > >
> > > Thanks much. I like this solution. I have updated the KIP accordingly.
> > >
> > >
> > >
> > >>
> > >> Jun
> > >>
> > >> On Mon, Jan 8, 2018 at 4:22 PM, Dong Lin <li...@gmail.com> wrote:
> > >>
> > >> > Hey Jason,
> > >> >
> > >> > Certainly. This sounds good. I have updated the KIP to clarity that
> > the
> > >> > global epoch will be incremented by 1 each time a topic is deleted.
> > >> >
> > >> > Thanks,
> > >> > Dong
> > >> >
> > >> > On Mon, Jan 8, 2018 at 4:09 PM, Jason Gustafson <jason@confluent.io
> >
> > >> > wrote:
> > >> >
> > >> > > Hi Dong,
> > >> > >
> > >> > >
> > >> > > I think your approach will allow user to distinguish between the
> > >> metadata
> > >> > > > before and after the topic deletion. I also agree that this can
> be
> > >> > > > potentially be useful to user. I am just not very sure whether
> we
> > >> > already
> > >> > > > have a good use-case to make the additional complexity
> worthwhile.
> > >> It
> > >> > > seems
> > >> > > > that this feature is kind of independent of the main problem of
> > this
> > >> > KIP.
> > >> > > > Could we add this as a future work?
> > >> > >
> > >> > >
> > >> > > Do you think it's fair if we bump the topic epoch on deletion and
> > >> leave
> > >> > > propagation of the epoch for deleted topics for future work? I
> don't
> > >> > think
> > >> > > this adds much complexity and it makes the behavior consistent:
> > every
> > >> > topic
> > >> > > mutation results in an epoch bump.
> > >> > >
> > >> > > Thanks,
> > >> > > Jason
> > >> > >
> > >> > > On Mon, Jan 8, 2018 at 3:14 PM, Dong Lin <li...@gmail.com>
> > wrote:
> > >> > >
> > >> > > > Hey Ismael,
> > >> > > >
> > >> > > > I guess we actually need user to see this field so that user can
> > >> store
> > >> > > this
> > >> > > > value in the external store together with the offset. We just
> > prefer
> > >> > the
> > >> > > > value to be opaque to discourage most users from interpreting
> this
> > >> > value.
> > >> > > > One more advantage of using such an opaque field is to be able
> to
> > >> > evolve
> > >> > > > the information (or schema) of this value without changing
> > consumer
> > >> API
> > >> > > in
> > >> > > > the future.
> > >> > > >
> > >> > > > I also thinking it is probably OK for user to be able to
> interpret
> > >> this
> > >> > > > value, particularly for those advanced users.
> > >> > > >
> > >> > > > Thanks,
> > >> > > > Dong
> > >> > > >
> > >> > > > On Mon, Jan 8, 2018 at 2:34 PM, Ismael Juma <is...@juma.me.uk>
> > >> wrote:
> > >> > > >
> > >> > > > > On Fri, Jan 5, 2018 at 7:15 PM, Jason Gustafson <
> > >> jason@confluent.io>
> > >> > > > > wrote:
> > >> > > > > >
> > >> > > > > > class OffsetAndMetadata {
> > >> > > > > >   long offset;
> > >> > > > > >   byte[] offsetMetadata;
> > >> > > > > >   String metadata;
> > >> > > > > > }
> > >> > > > >
> > >> > > > >
> > >> > > > > > Admittedly, the naming is a bit annoying, but we can
> probably
> > >> come
> > >> > up
> > >> > > > > with
> > >> > > > > > something better. Internally the byte array would have a
> > >> version.
> > >> > If
> > >> > > in
> > >> > > > > the
> > >> > > > > > future we have anything else we need to add, we can update
> the
> > >> > > version
> > >> > > > > and
> > >> > > > > > we wouldn't need any new APIs.
> > >> > > > > >
> > >> > > > >
> > >> > > > > We can also add fields to a class in a compatible way. So, it
> > >> seems
> > >> > to
> > >> > > me
> > >> > > > > that the main advantage of the byte array is that it's opaque
> to
> > >> the
> > >> > > > user.
> > >> > > > > Is that correct? If so, we could also add any opaque metadata
> > in a
> > >> > > > subclass
> > >> > > > > so that users don't even see it (unless they cast it, but then
> > >> > they're
> > >> > > on
> > >> > > > > their own).
> > >> > > > >
> > >> > > > > Ismael
> > >> > > > >
> > >> > > > > The corresponding seek() and position() APIs might look
> > something
> > >> > like
> > >> > > > > this:
> > >> > > > > >
> > >> > > > > > void seek(TopicPartition partition, long offset, byte[]
> > >> > > > offsetMetadata);
> > >> > > > > > byte[] positionMetadata(TopicPartition partition);
> > >> > > > > >
> > >> > > > > > What do you think?
> > >> > > > > >
> > >> > > > > > Thanks,
> > >> > > > > > Jason
> > >> > > > > >
> > >> > > > > > On Thu, Jan 4, 2018 at 7:04 PM, Dong Lin <
> lindong28@gmail.com
> > >
> > >> > > wrote:
> > >> > > > > >
> > >> > > > > > > Hey Jun, Jason,
> > >> > > > > > >
> > >> > > > > > > Thanks much for all the feedback. I have updated the KIP
> > >> based on
> > >> > > the
> > >> > > > > > > latest discussion. Can you help check whether it looks
> good?
> > >> > > > > > >
> > >> > > > > > > Thanks,
> > >> > > > > > > Dong
> > >> > > > > > >
> > >> > > > > > > On Thu, Jan 4, 2018 at 5:36 PM, Dong Lin <
> > lindong28@gmail.com
> > >> >
> > >> > > > wrote:
> > >> > > > > > >
> > >> > > > > > > > Hey Jun,
> > >> > > > > > > >
> > >> > > > > > > > Hmm... thinking about this more, I am not sure that the
> > >> > proposed
> > >> > > > API
> > >> > > > > is
> > >> > > > > > > > sufficient. For users that store offset externally, we
> > >> probably
> > >> > > > need
> > >> > > > > > > extra
> > >> > > > > > > > API to return the leader_epoch and partition_epoch for
> all
> > >> > > > partitions
> > >> > > > > > > that
> > >> > > > > > > > consumers are consuming. I suppose these users currently
> > use
> > >> > > > > position()
> > >> > > > > > > to
> > >> > > > > > > > get the offset. Thus we probably need a new method
> > >> > > > > > positionWithEpoch(..)
> > >> > > > > > > to
> > >> > > > > > > > return <offset, partition_epoch, leader_epoch>. Does
> this
> > >> sound
> > >> > > > > > > reasonable?
> > >> > > > > > > >
> > >> > > > > > > > Thanks,
> > >> > > > > > > > Dong
> > >> > > > > > > >
> > >> > > > > > > >
> > >> > > > > > > > On Thu, Jan 4, 2018 at 5:26 PM, Jun Rao <
> jun@confluent.io
> > >
> > >> > > wrote:
> > >> > > > > > > >
> > >> > > > > > > >> Hi, Dong,
> > >> > > > > > > >>
> > >> > > > > > > >> Yes, that's what I am thinking. OffsetEpoch will be
> > >> composed
> > >> > of
> > >> > > > > > > >> (partition_epoch,
> > >> > > > > > > >> leader_epoch).
> > >> > > > > > > >>
> > >> > > > > > > >> Thanks,
> > >> > > > > > > >>
> > >> > > > > > > >> Jun
> > >> > > > > > > >>
> > >> > > > > > > >>
> > >> > > > > > > >> On Thu, Jan 4, 2018 at 4:22 PM, Dong Lin <
> > >> lindong28@gmail.com
> > >> > >
> > >> > > > > wrote:
> > >> > > > > > > >>
> > >> > > > > > > >> > Hey Jun,
> > >> > > > > > > >> >
> > >> > > > > > > >> > Thanks much. I like the the new API that you
> proposed.
> > I
> > >> am
> > >> > > not
> > >> > > > > sure
> > >> > > > > > > >> what
> > >> > > > > > > >> > you exactly mean by offset_epoch. I suppose that we
> can
> > >> use
> > >> > > the
> > >> > > > > pair
> > >> > > > > > > of
> > >> > > > > > > >> > (partition_epoch, leader_epoch) as the offset_epoch,
> > >> right?
> > >> > > > > > > >> >
> > >> > > > > > > >> > Thanks,
> > >> > > > > > > >> > Dong
> > >> > > > > > > >> >
> > >> > > > > > > >> > On Thu, Jan 4, 2018 at 4:02 PM, Jun Rao <
> > >> jun@confluent.io>
> > >> > > > wrote:
> > >> > > > > > > >> >
> > >> > > > > > > >> > > Hi, Dong,
> > >> > > > > > > >> > >
> > >> > > > > > > >> > > Got it. The api that you proposed works. The
> question
> > >> is
> > >> > > > whether
> > >> > > > > > > >> that's
> > >> > > > > > > >> > the
> > >> > > > > > > >> > > api that we want to have in the long term. My
> concern
> > >> is
> > >> > > that
> > >> > > > > > while
> > >> > > > > > > >> the
> > >> > > > > > > >> > api
> > >> > > > > > > >> > > change is simple, the new api seems harder to
> explain
> > >> and
> > >> > > use.
> > >> > > > > For
> > >> > > > > > > >> > example,
> > >> > > > > > > >> > > a consumer storing offsets externally now needs to
> > call
> > >> > > > > > > >> > > waitForMetadataUpdate() after calling seek().
> > >> > > > > > > >> > >
> > >> > > > > > > >> > > An alternative approach is to make the following
> > >> > compatible
> > >> > > > api
> > >> > > > > > > >> changes
> > >> > > > > > > >> > in
> > >> > > > > > > >> > > Consumer.
> > >> > > > > > > >> > > * Add an additional OffsetEpoch field in
> > >> > OffsetAndMetadata.
> > >> > > > (no
> > >> > > > > > need
> > >> > > > > > > >> to
> > >> > > > > > > >> > > change the CommitSync() api)
> > >> > > > > > > >> > > * Add a new api seek(TopicPartition partition, long
> > >> > offset,
> > >> > > > > > > >> OffsetEpoch
> > >> > > > > > > >> > > offsetEpoch). We can potentially deprecate the old
> > api
> > >> > > > > > > >> > seek(TopicPartition
> > >> > > > > > > >> > > partition, long offset) in the future.
> > >> > > > > > > >> > >
> > >> > > > > > > >> > > The alternative approach has similar amount of api
> > >> changes
> > >> > > as
> > >> > > > > > yours
> > >> > > > > > > >> but
> > >> > > > > > > >> > has
> > >> > > > > > > >> > > the following benefits.
> > >> > > > > > > >> > > 1. The api works in a similar way as how offset
> > >> management
> > >> > > > works
> > >> > > > > > now
> > >> > > > > > > >> and
> > >> > > > > > > >> > is
> > >> > > > > > > >> > > probably what we want in the long term.
> > >> > > > > > > >> > > 2. It can reset offsets better when there is data
> > loss
> > >> due
> > >> > > to
> > >> > > > > > > unclean
> > >> > > > > > > >> > > leader election or correlated replica failure.
> > >> > > > > > > >> > > 3. It can reset offsets better when topic is
> > recreated.
> > >> > > > > > > >> > >
> > >> > > > > > > >> > > Thanks,
> > >> > > > > > > >> > >
> > >> > > > > > > >> > > Jun
> > >> > > > > > > >> > >
> > >> > > > > > > >> > >
> > >> > > > > > > >> > > On Thu, Jan 4, 2018 at 2:05 PM, Dong Lin <
> > >> > > lindong28@gmail.com
> > >> > > > >
> > >> > > > > > > wrote:
> > >> > > > > > > >> > >
> > >> > > > > > > >> > > > Hey Jun,
> > >> > > > > > > >> > > >
> > >> > > > > > > >> > > > Yeah I agree that ideally we don't want an ever
> > >> growing
> > >> > > > global
> > >> > > > > > > >> metadata
> > >> > > > > > > >> > > > version. I just think it may be more desirable to
> > >> keep
> > >> > the
> > >> > > > > > > consumer
> > >> > > > > > > >> API
> > >> > > > > > > >> > > > simple.
> > >> > > > > > > >> > > >
> > >> > > > > > > >> > > > In my current proposal, metadata version returned
> > in
> > >> the
> > >> > > > fetch
> > >> > > > > > > >> response
> > >> > > > > > > >> > > > will be stored with the offset together. More
> > >> > > specifically,
> > >> > > > > the
> > >> > > > > > > >> > > > metadata_epoch in the new offset topic schema
> will
> > be
> > >> > the
> > >> > > > > > largest
> > >> > > > > > > >> > > > metadata_epoch from all the MetadataResponse and
> > >> > > > FetchResponse
> > >> > > > > > > ever
> > >> > > > > > > >> > > > received by this consumer.
> > >> > > > > > > >> > > >
> > >> > > > > > > >> > > > We probably don't have to change the consumer API
> > for
> > >> > > > > > > >> > > > commitSync(Map<TopicPartition,
> OffsetAndMetadata>).
> > >> If
> > >> > > user
> > >> > > > > > calls
> > >> > > > > > > >> > > > commitSync(...) to commit offset 10 for a given
> > >> > partition,
> > >> > > > for
> > >> > > > > > > most
> > >> > > > > > > >> > > > use-cases, this consumer instance should have
> > >> consumed
> > >> > > > message
> > >> > > > > > > with
> > >> > > > > > > >> > > offset
> > >> > > > > > > >> > > > 9 from this partition, in which case the consumer
> > can
> > >> > > > remember
> > >> > > > > > and
> > >> > > > > > > >> use
> > >> > > > > > > >> > > the
> > >> > > > > > > >> > > > metadata_epoch from the corresponding
> FetchResponse
> > >> when
> > >> > > > > > > committing
> > >> > > > > > > >> > > offset.
> > >> > > > > > > >> > > > If user calls commitSync(..) to commit offset 10
> > for
> > >> a
> > >> > > given
> > >> > > > > > > >> partition
> > >> > > > > > > >> > > > without having consumed the message with offset 9
> > >> using
> > >> > > this
> > >> > > > > > > >> consumer
> > >> > > > > > > >> > > > instance, this is probably an advanced use-case.
> In
> > >> this
> > >> > > > case
> > >> > > > > > the
> > >> > > > > > > >> > > advanced
> > >> > > > > > > >> > > > user can retrieve the metadata_epoch using the
> > newly
> > >> > added
> > >> > > > > > > >> > > metadataEpoch()
> > >> > > > > > > >> > > > API after it fetches the message with offset 9
> > >> (probably
> > >> > > > from
> > >> > > > > > > >> another
> > >> > > > > > > >> > > > consumer instance) and encode this metadata_epoch
> > in
> > >> the
> > >> > > > > > > >> > > > string OffsetAndMetadata.metadata. Do you think
> > this
> > >> > > > solution
> > >> > > > > > > would
> > >> > > > > > > >> > work?
> > >> > > > > > > >> > > >
> > >> > > > > > > >> > > > By "not sure that I fully understand your latest
> > >> > > > suggestion",
> > >> > > > > > are
> > >> > > > > > > >> you
> > >> > > > > > > >> > > > referring to solution related to unclean leader
> > >> election
> > >> > > > using
> > >> > > > > > > >> > > leader_epoch
> > >> > > > > > > >> > > > in my previous email?
> > >> > > > > > > >> > > >
> > >> > > > > > > >> > > > Thanks,
> > >> > > > > > > >> > > > Dong
> > >> > > > > > > >> > > >
> > >> > > > > > > >> > > > On Thu, Jan 4, 2018 at 1:33 PM, Jun Rao <
> > >> > jun@confluent.io
> > >> > > >
> > >> > > > > > wrote:
> > >> > > > > > > >> > > >
> > >> > > > > > > >> > > > > Hi, Dong,
> > >> > > > > > > >> > > > >
> > >> > > > > > > >> > > > > Not sure that I fully understand your latest
> > >> > suggestion.
> > >> > > > > > > >> Returning an
> > >> > > > > > > >> > > > ever
> > >> > > > > > > >> > > > > growing global metadata version itself is no
> > ideal,
> > >> > but
> > >> > > is
> > >> > > > > > fine.
> > >> > > > > > > >> My
> > >> > > > > > > >> > > > > question is whether the metadata version
> returned
> > >> in
> > >> > the
> > >> > > > > fetch
> > >> > > > > > > >> > response
> > >> > > > > > > >> > > > > needs to be stored with the offset together if
> > >> offsets
> > >> > > are
> > >> > > > > > > stored
> > >> > > > > > > >> > > > > externally. If so, we also have to change the
> > >> consumer
> > >> > > API
> > >> > > > > for
> > >> > > > > > > >> > > > commitSync()
> > >> > > > > > > >> > > > > and need to worry about compatibility. If we
> > don't
> > >> > store
> > >> > > > the
> > >> > > > > > > >> metadata
> > >> > > > > > > >> > > > > version together with the offset, on a consumer
> > >> > restart,
> > >> > > > > it's
> > >> > > > > > > not
> > >> > > > > > > >> > clear
> > >> > > > > > > >> > > > how
> > >> > > > > > > >> > > > > we can ensure the metadata in the consumer is
> > high
> > >> > > enough
> > >> > > > > > since
> > >> > > > > > > >> there
> > >> > > > > > > >> > > is
> > >> > > > > > > >> > > > no
> > >> > > > > > > >> > > > > metadata version to compare with.
> > >> > > > > > > >> > > > >
> > >> > > > > > > >> > > > > Thanks,
> > >> > > > > > > >> > > > >
> > >> > > > > > > >> > > > > Jun
> > >> > > > > > > >> > > > >
> > >> > > > > > > >> > > > >
> > >> > > > > > > >> > > > > On Wed, Jan 3, 2018 at 6:43 PM, Dong Lin <
> > >> > > > > lindong28@gmail.com
> > >> > > > > > >
> > >> > > > > > > >> > wrote:
> > >> > > > > > > >> > > > >
> > >> > > > > > > >> > > > > > Hey Jun,
> > >> > > > > > > >> > > > > >
> > >> > > > > > > >> > > > > > Thanks much for the explanation.
> > >> > > > > > > >> > > > > >
> > >> > > > > > > >> > > > > > I understand the advantage of partition_epoch
> > >> over
> > >> > > > > > > >> metadata_epoch.
> > >> > > > > > > >> > My
> > >> > > > > > > >> > > > > > current concern is that the use of
> leader_epoch
> > >> and
> > >> > > the
> > >> > > > > > > >> > > partition_epoch
> > >> > > > > > > >> > > > > > requires us considerable change to consumer's
> > >> public
> > >> > > API
> > >> > > > > to
> > >> > > > > > > take
> > >> > > > > > > >> > care
> > >> > > > > > > >> > > > of
> > >> > > > > > > >> > > > > > the case where user stores offset externally.
> > For
> > >> > > > example,
> > >> > > > > > > >> > > *consumer*.
> > >> > > > > > > >> > > > > > *commitSync*(..) would have to take a map
> whose
> > >> > value
> > >> > > is
> > >> > > > > > > >> <offset,
> > >> > > > > > > >> > > > > metadata,
> > >> > > > > > > >> > > > > > leader epoch, partition epoch>.
> > >> > *consumer*.*seek*(...)
> > >> > > > > would
> > >> > > > > > > >> also
> > >> > > > > > > >> > > need
> > >> > > > > > > >> > > > > > leader_epoch and partition_epoch as
> parameter.
> > >> > > > Technically
> > >> > > > > > we
> > >> > > > > > > >> can
> > >> > > > > > > >> > > > > probably
> > >> > > > > > > >> > > > > > still make it work in a backward compatible
> > >> manner
> > >> > > after
> > >> > > > > > > careful
> > >> > > > > > > >> > > design
> > >> > > > > > > >> > > > > and
> > >> > > > > > > >> > > > > > discussion. But these changes can make the
> > >> > consumer's
> > >> > > > > > > interface
> > >> > > > > > > >> > > > > > unnecessarily complex for more users who do
> not
> > >> > store
> > >> > > > > offset
> > >> > > > > > > >> > > > externally.
> > >> > > > > > > >> > > > > >
> > >> > > > > > > >> > > > > > After thinking more about it, we can address
> > all
> > >> > > > problems
> > >> > > > > > > >> discussed
> > >> > > > > > > >> > > by
> > >> > > > > > > >> > > > > only
> > >> > > > > > > >> > > > > > using the metadata_epoch without introducing
> > >> > > > leader_epoch
> > >> > > > > or
> > >> > > > > > > the
> > >> > > > > > > >> > > > > > partition_epoch. The current KIP describes
> the
> > >> > changes
> > >> > > > to
> > >> > > > > > the
> > >> > > > > > > >> > > consumer
> > >> > > > > > > >> > > > > API
> > >> > > > > > > >> > > > > > and how the new API can be used if user
> stores
> > >> > offset
> > >> > > > > > > >> externally.
> > >> > > > > > > >> > In
> > >> > > > > > > >> > > > > order
> > >> > > > > > > >> > > > > > to address the scenario you described
> earlier,
> > we
> > >> > can
> > >> > > > > > include
> > >> > > > > > > >> > > > > > metadata_epoch in the FetchResponse and the
> > >> > > > > > > LeaderAndIsrRequest.
> > >> > > > > > > >> > > > Consumer
> > >> > > > > > > >> > > > > > remembers the largest metadata_epoch from all
> > the
> > >> > > > > > > FetchResponse
> > >> > > > > > > >> it
> > >> > > > > > > >> > > has
> > >> > > > > > > >> > > > > > received. The metadata_epoch committed with
> the
> > >> > > offset,
> > >> > > > > > either
> > >> > > > > > > >> > within
> > >> > > > > > > >> > > > or
> > >> > > > > > > >> > > > > > outside Kafka, should be the largest
> > >> metadata_epoch
> > >> > > > across
> > >> > > > > > all
> > >> > > > > > > >> > > > > > FetchResponse and MetadataResponse ever
> > received
> > >> by
> > >> > > this
> > >> > > > > > > >> consumer.
> > >> > > > > > > >> > > > > >
> > >> > > > > > > >> > > > > > The drawback of using only the metadata_epoch
> > is
> > >> > that
> > >> > > we
> > >> > > > > can
> > >> > > > > > > not
> > >> > > > > > > >> > > always
> > >> > > > > > > >> > > > > do
> > >> > > > > > > >> > > > > > the smart offset reset in case of unclean
> > leader
> > >> > > > election
> > >> > > > > > > which
> > >> > > > > > > >> you
> > >> > > > > > > >> > > > > > mentioned earlier. But in most case, unclean
> > >> leader
> > >> > > > > election
> > >> > > > > > > >> > probably
> > >> > > > > > > >> > > > > > happens when consumer is not
> > >> rebalancing/restarting.
> > >> > > In
> > >> > > > > > these
> > >> > > > > > > >> > cases,
> > >> > > > > > > >> > > > > either
> > >> > > > > > > >> > > > > > consumer is not directly affected by unclean
> > >> leader
> > >> > > > > election
> > >> > > > > > > >> since
> > >> > > > > > > >> > it
> > >> > > > > > > >> > > > is
> > >> > > > > > > >> > > > > > not consuming from the end of the log, or
> > >> consumer
> > >> > can
> > >> > > > > > derive
> > >> > > > > > > >> the
> > >> > > > > > > >> > > > > > leader_epoch from the most recent message
> > >> received
> > >> > > > before
> > >> > > > > it
> > >> > > > > > > >> sees
> > >> > > > > > > >> > > > > > OffsetOutOfRangeException. So I am not sure
> it
> > is
> > >> > > worth
> > >> > > > > > adding
> > >> > > > > > > >> the
> > >> > > > > > > >> > > > > > leader_epoch to consumer API to address the
> > >> > remaining
> > >> > > > > corner
> > >> > > > > > > >> case.
> > >> > > > > > > >> > > What
> > >> > > > > > > >> > > > > do
> > >> > > > > > > >> > > > > > you think?
> > >> > > > > > > >> > > > > >
> > >> > > > > > > >> > > > > > Thanks,
> > >> > > > > > > >> > > > > > Dong
> > >> > > > > > > >> > > > > >
> > >> > > > > > > >> > > > > >
> > >> > > > > > > >> > > > > >
> > >> > > > > > > >> > > > > > On Tue, Jan 2, 2018 at 6:28 PM, Jun Rao <
> > >> > > > jun@confluent.io
> > >> > > > > >
> > >> > > > > > > >> wrote:
> > >> > > > > > > >> > > > > >
> > >> > > > > > > >> > > > > > > Hi, Dong,
> > >> > > > > > > >> > > > > > >
> > >> > > > > > > >> > > > > > > Thanks for the reply.
> > >> > > > > > > >> > > > > > >
> > >> > > > > > > >> > > > > > > To solve the topic recreation issue, we
> could
> > >> use
> > >> > > > > either a
> > >> > > > > > > >> global
> > >> > > > > > > >> > > > > > metadata
> > >> > > > > > > >> > > > > > > version or a partition level epoch. But
> > either
> > >> one
> > >> > > > will
> > >> > > > > > be a
> > >> > > > > > > >> new
> > >> > > > > > > >> > > > > concept,
> > >> > > > > > > >> > > > > > > right? To me, the latter seems more
> natural.
> > It
> > >> > also
> > >> > > > > makes
> > >> > > > > > > it
> > >> > > > > > > >> > > easier
> > >> > > > > > > >> > > > to
> > >> > > > > > > >> > > > > > > detect if a consumer's offset is still
> valid
> > >> > after a
> > >> > > > > topic
> > >> > > > > > > is
> > >> > > > > > > >> > > > > recreated.
> > >> > > > > > > >> > > > > > As
> > >> > > > > > > >> > > > > > > you pointed out, we don't need to store the
> > >> > > partition
> > >> > > > > > epoch
> > >> > > > > > > in
> > >> > > > > > > >> > the
> > >> > > > > > > >> > > > > > message.
> > >> > > > > > > >> > > > > > > The following is what I am thinking. When a
> > >> > > partition
> > >> > > > is
> > >> > > > > > > >> created,
> > >> > > > > > > >> > > we
> > >> > > > > > > >> > > > > can
> > >> > > > > > > >> > > > > > > assign a partition epoch from an
> > >> ever-increasing
> > >> > > > global
> > >> > > > > > > >> counter
> > >> > > > > > > >> > and
> > >> > > > > > > >> > > > > store
> > >> > > > > > > >> > > > > > > it in /brokers/topics/[topic]/
> > >> > > > partitions/[partitionId]
> > >> > > > > in
> > >> > > > > > > ZK.
> > >> > > > > > > >> > The
> > >> > > > > > > >> > > > > > > partition
> > >> > > > > > > >> > > > > > > epoch is propagated to every broker. The
> > >> consumer
> > >> > > will
> > >> > > > > be
> > >> > > > > > > >> > tracking
> > >> > > > > > > >> > > a
> > >> > > > > > > >> > > > > > tuple
> > >> > > > > > > >> > > > > > > of <offset, leader epoch, partition epoch>
> > for
> > >> > > > offsets.
> > >> > > > > > If a
> > >> > > > > > > >> > topic
> > >> > > > > > > >> > > is
> > >> > > > > > > >> > > > > > > recreated, it's possible that a consumer's
> > >> offset
> > >> > > and
> > >> > > > > > leader
> > >> > > > > > > >> > epoch
> > >> > > > > > > >> > > > > still
> > >> > > > > > > >> > > > > > > match that in the broker, but partition
> epoch
> > >> > won't
> > >> > > > be.
> > >> > > > > In
> > >> > > > > > > >> this
> > >> > > > > > > >> > > case,
> > >> > > > > > > >> > > > > we
> > >> > > > > > > >> > > > > > > can potentially still treat the consumer's
> > >> offset
> > >> > as
> > >> > > > out
> > >> > > > > > of
> > >> > > > > > > >> range
> > >> > > > > > > >> > > and
> > >> > > > > > > >> > > > > > reset
> > >> > > > > > > >> > > > > > > the offset based on the offset reset policy
> > in
> > >> the
> > >> > > > > > consumer.
> > >> > > > > > > >> This
> > >> > > > > > > >> > > > seems
> > >> > > > > > > >> > > > > > > harder to do with a global metadata
> version.
> > >> > > > > > > >> > > > > > >
> > >> > > > > > > >> > > > > > > Jun
> > >> > > > > > > >> > > > > > >
> > >> > > > > > > >> > > > > > >
> > >> > > > > > > >> > > > > > >
> > >> > > > > > > >> > > > > > > On Mon, Dec 25, 2017 at 6:56 AM, Dong Lin <
> > >> > > > > > > >> lindong28@gmail.com>
> > >> > > > > > > >> > > > wrote:
> > >> > > > > > > >> > > > > > >
> > >> > > > > > > >> > > > > > > > Hey Jun,
> > >> > > > > > > >> > > > > > > >
> > >> > > > > > > >> > > > > > > > This is a very good example. After
> thinking
> > >> > > through
> > >> > > > > this
> > >> > > > > > > in
> > >> > > > > > > >> > > > detail, I
> > >> > > > > > > >> > > > > > > agree
> > >> > > > > > > >> > > > > > > > that we need to commit offset with leader
> > >> epoch
> > >> > in
> > >> > > > > order
> > >> > > > > > > to
> > >> > > > > > > >> > > address
> > >> > > > > > > >> > > > > > this
> > >> > > > > > > >> > > > > > > > example.
> > >> > > > > > > >> > > > > > > >
> > >> > > > > > > >> > > > > > > > I think the remaining question is how to
> > >> address
> > >> > > the
> > >> > > > > > > >> scenario
> > >> > > > > > > >> > > that
> > >> > > > > > > >> > > > > the
> > >> > > > > > > >> > > > > > > > topic is deleted and re-created. One
> > possible
> > >> > > > solution
> > >> > > > > > is
> > >> > > > > > > to
> > >> > > > > > > >> > > commit
> > >> > > > > > > >> > > > > > > offset
> > >> > > > > > > >> > > > > > > > with both the leader epoch and the
> metadata
> > >> > > version.
> > >> > > > > The
> > >> > > > > > > >> logic
> > >> > > > > > > >> > > and
> > >> > > > > > > >> > > > > the
> > >> > > > > > > >> > > > > > > > implementation of this solution does not
> > >> > require a
> > >> > > > new
> > >> > > > > > > >> concept
> > >> > > > > > > >> > > > (e.g.
> > >> > > > > > > >> > > > > > > > partition epoch) and it does not require
> > any
> > >> > > change
> > >> > > > to
> > >> > > > > > the
> > >> > > > > > > >> > > message
> > >> > > > > > > >> > > > > > format
> > >> > > > > > > >> > > > > > > > or leader epoch. It also allows us to
> order
> > >> the
> > >> > > > > metadata
> > >> > > > > > > in
> > >> > > > > > > >> a
> > >> > > > > > > >> > > > > > > > straightforward manner which may be
> useful
> > in
> > >> > the
> > >> > > > > > future.
> > >> > > > > > > >> So it
> > >> > > > > > > >> > > may
> > >> > > > > > > >> > > > > be
> > >> > > > > > > >> > > > > > a
> > >> > > > > > > >> > > > > > > > better solution than generating a random
> > >> > partition
> > >> > > > > epoch
> > >> > > > > > > >> every
> > >> > > > > > > >> > > time
> > >> > > > > > > >> > > > > we
> > >> > > > > > > >> > > > > > > > create a partition. Does this sound
> > >> reasonable?
> > >> > > > > > > >> > > > > > > >
> > >> > > > > > > >> > > > > > > > Previously one concern with using the
> > >> metadata
> > >> > > > version
> > >> > > > > > is
> > >> > > > > > > >> that
> > >> > > > > > > >> > > > > consumer
> > >> > > > > > > >> > > > > > > > will be forced to refresh metadata even
> if
> > >> > > metadata
> > >> > > > > > > version
> > >> > > > > > > >> is
> > >> > > > > > > >> > > > > > increased
> > >> > > > > > > >> > > > > > > > due to topics that the consumer is not
> > >> > interested
> > >> > > > in.
> > >> > > > > > Now
> > >> > > > > > > I
> > >> > > > > > > >> > > > realized
> > >> > > > > > > >> > > > > > that
> > >> > > > > > > >> > > > > > > > this is probably not a problem. Currently
> > >> client
> > >> > > > will
> > >> > > > > > > >> refresh
> > >> > > > > > > >> > > > > metadata
> > >> > > > > > > >> > > > > > > > either due to InvalidMetadataException in
> > the
> > >> > > > response
> > >> > > > > > > from
> > >> > > > > > > >> > > broker
> > >> > > > > > > >> > > > or
> > >> > > > > > > >> > > > > > due
> > >> > > > > > > >> > > > > > > > to metadata expiry. The addition of the
> > >> metadata
> > >> > > > > version
> > >> > > > > > > >> should
> > >> > > > > > > >> > > > > > increase
> > >> > > > > > > >> > > > > > > > the overhead of metadata refresh caused
> by
> > >> > > > > > > >> > > > InvalidMetadataException.
> > >> > > > > > > >> > > > > If
> > >> > > > > > > >> > > > > > > > client refresh metadata due to expiry and
> > it
> > >> > > > receives
> > >> > > > > a
> > >> > > > > > > >> > metadata
> > >> > > > > > > >> > > > > whose
> > >> > > > > > > >> > > > > > > > version is lower than the current
> metadata
> > >> > > version,
> > >> > > > we
> > >> > > > > > can
> > >> > > > > > > >> > reject
> > >> > > > > > > >> > > > the
> > >> > > > > > > >> > > > > > > > metadata but still reset the metadata
> age,
> > >> which
> > >> > > > > > > essentially
> > >> > > > > > > >> > keep
> > >> > > > > > > >> > > > the
> > >> > > > > > > >> > > > > > > > existing behavior in the client.
> > >> > > > > > > >> > > > > > > >
> > >> > > > > > > >> > > > > > > > Thanks much,
> > >> > > > > > > >> > > > > > > > Dong
> > >> > > > > > > >> > > > > > > >
> > >> > > > > > > >> > > > > > >
> > >> > > > > > > >> > > > > >
> > >> > > > > > > >> > > > >
> > >> > > > > > > >> > > >
> > >> > > > > > > >> > >
> > >> > > > > > > >> >
> > >> > > > > > > >>
> > >> > > > > > > >
> > >> > > > > > > >
> > >> > > > > > >
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Jun Rao <ju...@confluent.io>.
Hi, Dong,

My replies are the following.

60. What you described could also work. The drawback is that we will be
unnecessarily changing the partition epoch when a partition hasn't really
changed. I was imagining that the partition epoch will be stored in
/brokers/topics/[topic]/partitions/[partitionId], instead of at the topic
level. So, not sure if ZK size limit is an issue.

61, 62 and 65. To me, the offset + offset_epoch is a unique identifier for
a message. So, if a message hasn't changed, the offset and the associated
offset_epoch ideally should remain the same (it will be kind of weird if
two consumer apps save the offset on the same message, but the offset_epoch
are different). partition_epoch + leader_epoch give us that. global_epoch +
leader_epoch don't. If we use this approach, we can solve not only the
problem that you have identified, but also other problems when there is
data loss or topic re-creation more reliably. For example, in the future,
if we include the partition_epoch and leader_epoch in the fetch request,
the server can do a more reliable check of whether that offset is valid or
not. I am not sure that we can rely upon all external offsets to be removed
on topic deletion. For example, a topic may be deleted by an admin who may
not know all the applications.

If we agree on the above, the second question is then how to reliably
propagate the partition_epoch and the leader_epoch to the consumer when
there are leader or partition changes. The leader_epoch comes from the
message, which is reliable. So, I was suggesting that when we store an
offset, we can just store the leader_epoch from the message set containing
that offset. Similarly, I was thinking that if the partition_epoch is in
the fetch response, we can propagate partition_epoch reliably where is
partition_epoch change.

63. My point is that once a leader is producing a message in the new
partition_epoch, ideally, we should associate the new offsets with the new
partition_epoch. Otherwise, the offset_epoch won't be the correct unique
identifier (useful for solving other problems mentioned above). I was
originally thinking that the leader will include the partition_epoch in the
metadata cache in the fetch response. It's just that right now, metadata
cache is updated on UpdateMetadataRequest, which typically happens after
the LeaderAndIsrRequest. Another approach is for the leader to cache the
partition_epoch in the Partition object and return that (instead of the one
in metadata cache) in the fetch response.

65. It seems to me that the global_epoch and the partition_epoch have
different purposes. A partition_epoch has the benefit that it (1) can be
used to form a unique identifier for a message and (2) can be used to
solve other
corner case problems in the future. I am not sure having just a
global_epoch can achieve these. global_epoch is useful to determine which
version of the metadata is newer, especially with topic deletion.

Thanks,

Jun

On Tue, Jan 9, 2018 at 11:34 PM, Dong Lin <li...@gmail.com> wrote:

> Regarding the use of the global epoch in 65), it is very similar to the
> proposal of the metadata_epoch we discussed earlier. The main difference is
> that this epoch is incremented when we create/expand/delete topic and does
> not change when controller re-send metadata.
>
> I looked at our previous discussion. It seems that we prefer
> partition_epoch over the metadata_epoch because 1) we prefer not to have an
> ever growing metadata_epoch and 2) we can reset offset better when topic is
> re-created. The use of global topic_epoch avoids the drawback of an ever
> quickly ever growing metadata_epoch. Though the global epoch does not allow
> us to recognize the invalid offset committed before the topic re-creation,
> we can probably just delete the offset when we delete a topic. Thus I am
> not very sure whether it is still worthwhile to have a per-partition
> partition_epoch if the metadata already has the global epoch.
>
>
> On Tue, Jan 9, 2018 at 6:58 PM, Dong Lin <li...@gmail.com> wrote:
>
> > Hey Jun,
> >
> > Thanks so much. These comments very useful. Please see below my comments.
> >
> > On Mon, Jan 8, 2018 at 5:52 PM, Jun Rao <ju...@confluent.io> wrote:
> >
> >> Hi, Dong,
> >>
> >> Thanks for the updated KIP. A few more comments.
> >>
> >> 60. Perhaps having a partition epoch is more flexible since in the
> future,
> >> we may support deleting a partition as well.
> >>
> >
> > Yeah I have considered this. I think we can probably still support
> > deleting a partition by using the topic_epoch -- when partition of a
> topic
> > is deleted or created, epoch of all partitions of this topic will be
> > incremented by 1. Therefore, if that partition is re-created later, the
> > epoch of that partition will still be larger than its epoch before the
> > deletion, which still allows the client to order the metadata for the
> > purpose of this KIP. Does this sound reasonable?
> >
> > The advantage of using topic_epoch instead of partition_epoch is that the
> > size of the /brokers/topics/[topic] znode and request/response size can
> be
> > smaller. We have a limit on the maximum size of znode (typically 1MB).
> Use
> > partition epoch can effectively reduce the number of partitions that can
> be
> > described by the /brokers/topics/[topic] znode.
> >
> > One use-case of partition_epoch for client to detect that the committed
> > offset, either from kafka offset topic or from the external store is
> > invalid after partition deletion and re-creation. However, it seems that
> we
> > can also address this use-case with other approaches. For example, when
> > AdminClient deletes partitions, it can also delete the committed offsets
> > for those partitions from the offset topic. If user stores offset
> > externally, it might make sense for user to similarly remove offsets of
> > related partitions after these partitions are deleted. So I am not sure
> > that we should use partition_epoch in this KIP.
> >
> >
> >>
> >> 61. It seems that the leader epoch returned in the position() call
> should
> >> the the leader epoch returned in the fetch response, not the one in the
> >> metadata cache of the client.
> >
> >
> > I think this is a good idea. Just to double check, this change does not
> > affect the correctness or performance of this KIP. But it can be useful
> if
> > we want to use the leader_epoch to better handle the offset rest in case
> of
> > unclean leader election, which is listed in the future work. Is this
> > understanding correct?
> >
> > I have updated the KIP to specify that the leader_epoch returned by
> > position() should be the largest leader_epoch of those already consumed
> > messages whose offset < position. If no message has been consumed since
> > consumer initialization, the leader_epoch from seek() or
> > OffsetFetchResponse should be used. The offset included in the
> > OffsetCommitRequest will also be determined in the similar manner.
> >
> >
> >>
> >> 62. I am wondering if we should return the partition epoch in the fetch
> >> response as well. In the current proposal, if a topic is recreated and
> the
> >> new leader is on the same broker as the old one, there is nothing to
> force
> >> the metadata refresh in the client. So, the client may still associate
> the
> >> offset with the old partition epoch.
> >>
> >
> > Could you help me understand the problem if a client associates old
> > partition_epoch (or the topic_epoch as of the current KIP) with the
> offset?
> > The main purpose of the topic_epoch is to be able to drop leader_epoch
> to 0
> > after a partition is deleted and re-created. I guess you may be thinking
> > about using the partition_epoch to detect that the committed offset is
> > invalid? In that case, I am wondering if the alternative approach
> described
> > in 60) would be reasonable.
> >
> >
> >>
> >> 63. There is some subtle coordination between the LeaderAndIsrRequest
> and
> >> UpdateMetadataRequest. Currently, when a leader changes, the controller
> >> first sends the LeaderAndIsrRequest to the assigned replicas and the
> >> UpdateMetadataRequest to every broker. So, there could be a small window
> >> when the leader already receives the new partition epoch in the
> >> LeaderAndIsrRequest, but the metadata cache in the broker hasn't been
> >> updated with the latest partition epoch. Not sure what's the best way to
> >> address this issue. Perhaps we can update the metadata cache on the
> broker
> >> with both LeaderAndIsrRequest and UpdateMetadataRequest. The challenge
> is
> >> that the two have slightly different data. For example, only the latter
> >> has
> >> all endpoints.
> >>
> >
> > I am not sure whether this is a problem. Could you explain a bit more
> what
> > specific problem this small window can cause?
> >
> > Since client can fetch metadata from any broker in the cluster, and given
> > that different brokers receive request (e.g. LeaderAndIsrRequest and
> > UpdateMetadataRequest) in arbitrary order, the metadata received by
> client
> > can be in arbitrary order (either newer or older) compared to the
> broker's
> > leadership state even if a given broker receives LeaderAndIsrRequest and
> > UpdateMetadataRequest simultaneously. So I am not sure it is useful to
> > update broker's cache with LeaderAndIsrRequest.
> >
> >
> >> 64. The enforcement of leader epoch in Offset commit: We allow a
> consumer
> >> to set an arbitrary offset. So it's possible for offsets or leader epoch
> >> to
> >> go backwards. I am not sure if we could always enforce that the leader
> >> epoch only goes up on the broker.
> >>
> >
> > Sure. I have removed this check from the KIP.
> >
> > BTW, we can probably still ensure that the leader_epoch always increase
> if
> > the leader_epoch used with offset commit is the max(leader_epoch of the
> > message with offset = the committed offset - 1, the largest known
> > leader_epoch from the metadata). But I don't have a good use-case for
> this
> > alternative definition. So I choose the keep the KIP simple by requiring
> > leader_epoch to always increase.
> >
> >
> >> 65. Good point on handling missing partition epoch due to topic
> deletion.
> >> Another potential way to address this is to additionally propagate the
> >> global partition epoch to brokers and the clients. This way, when a
> >> partition epoch is missing, we can use the global partition epoch to
> >> reason
> >> about which metadata is more recent.
> >>
> >
> > This is a great idea. The global epoch can be used to order the metadata
> > and help us recognize the more recent metadata if a topic (or partition)
> is
> > deleted and re-created.
> >
> > Actually, it seems we only need to propagate the global epoch to brokers
> > and clients without propagating this epoch on a per-topic or
> per-partition
> > basic. Doing so would simply interface changes made this KIP. Does this
> > approach sound reasonable?
> >
> >
> >> 66. A client may also get an offset by time using the offsetForTimes()
> >> api.
> >> So, we probably want to include offsetInternalMetadata in
> >> OffsetAndTimestamp
> >> as well.
> >>
> >
> > You are right. This probably also requires us to change the
> > ListOffsetRequest as well. I will update the KIP after we agree on the
> > solution for 65).
> >
> >
> >>
> >> 67. InteralMetadata can be a bit confusing with the metadata field
> already
> >> there. Perhaps we can just call it OffsetEpoch. It might be useful to
> make
> >> OffsetEpoch printable at least for debugging purpose. Once you do that,
> we
> >> are already exposing the internal fields. So, not sure if it's worth
> >> hiding
> >> them. If we do want to hide them, perhaps we can have sth like the
> >> following. The binary encoding is probably more efficient than JSON for
> >> external storage.
> >>
> >> OffsetEpoch {
> >>  static OffsetEpoch decode(byte[]);
> >>
> >>   public byte[] encode();
> >>
> >>   public String toString();
> >> }
> >>
> >
> > Thanks much. I like this solution. I have updated the KIP accordingly.
> >
> >
> >
> >>
> >> Jun
> >>
> >> On Mon, Jan 8, 2018 at 4:22 PM, Dong Lin <li...@gmail.com> wrote:
> >>
> >> > Hey Jason,
> >> >
> >> > Certainly. This sounds good. I have updated the KIP to clarity that
> the
> >> > global epoch will be incremented by 1 each time a topic is deleted.
> >> >
> >> > Thanks,
> >> > Dong
> >> >
> >> > On Mon, Jan 8, 2018 at 4:09 PM, Jason Gustafson <ja...@confluent.io>
> >> > wrote:
> >> >
> >> > > Hi Dong,
> >> > >
> >> > >
> >> > > I think your approach will allow user to distinguish between the
> >> metadata
> >> > > > before and after the topic deletion. I also agree that this can be
> >> > > > potentially be useful to user. I am just not very sure whether we
> >> > already
> >> > > > have a good use-case to make the additional complexity worthwhile.
> >> It
> >> > > seems
> >> > > > that this feature is kind of independent of the main problem of
> this
> >> > KIP.
> >> > > > Could we add this as a future work?
> >> > >
> >> > >
> >> > > Do you think it's fair if we bump the topic epoch on deletion and
> >> leave
> >> > > propagation of the epoch for deleted topics for future work? I don't
> >> > think
> >> > > this adds much complexity and it makes the behavior consistent:
> every
> >> > topic
> >> > > mutation results in an epoch bump.
> >> > >
> >> > > Thanks,
> >> > > Jason
> >> > >
> >> > > On Mon, Jan 8, 2018 at 3:14 PM, Dong Lin <li...@gmail.com>
> wrote:
> >> > >
> >> > > > Hey Ismael,
> >> > > >
> >> > > > I guess we actually need user to see this field so that user can
> >> store
> >> > > this
> >> > > > value in the external store together with the offset. We just
> prefer
> >> > the
> >> > > > value to be opaque to discourage most users from interpreting this
> >> > value.
> >> > > > One more advantage of using such an opaque field is to be able to
> >> > evolve
> >> > > > the information (or schema) of this value without changing
> consumer
> >> API
> >> > > in
> >> > > > the future.
> >> > > >
> >> > > > I also thinking it is probably OK for user to be able to interpret
> >> this
> >> > > > value, particularly for those advanced users.
> >> > > >
> >> > > > Thanks,
> >> > > > Dong
> >> > > >
> >> > > > On Mon, Jan 8, 2018 at 2:34 PM, Ismael Juma <is...@juma.me.uk>
> >> wrote:
> >> > > >
> >> > > > > On Fri, Jan 5, 2018 at 7:15 PM, Jason Gustafson <
> >> jason@confluent.io>
> >> > > > > wrote:
> >> > > > > >
> >> > > > > > class OffsetAndMetadata {
> >> > > > > >   long offset;
> >> > > > > >   byte[] offsetMetadata;
> >> > > > > >   String metadata;
> >> > > > > > }
> >> > > > >
> >> > > > >
> >> > > > > > Admittedly, the naming is a bit annoying, but we can probably
> >> come
> >> > up
> >> > > > > with
> >> > > > > > something better. Internally the byte array would have a
> >> version.
> >> > If
> >> > > in
> >> > > > > the
> >> > > > > > future we have anything else we need to add, we can update the
> >> > > version
> >> > > > > and
> >> > > > > > we wouldn't need any new APIs.
> >> > > > > >
> >> > > > >
> >> > > > > We can also add fields to a class in a compatible way. So, it
> >> seems
> >> > to
> >> > > me
> >> > > > > that the main advantage of the byte array is that it's opaque to
> >> the
> >> > > > user.
> >> > > > > Is that correct? If so, we could also add any opaque metadata
> in a
> >> > > > subclass
> >> > > > > so that users don't even see it (unless they cast it, but then
> >> > they're
> >> > > on
> >> > > > > their own).
> >> > > > >
> >> > > > > Ismael
> >> > > > >
> >> > > > > The corresponding seek() and position() APIs might look
> something
> >> > like
> >> > > > > this:
> >> > > > > >
> >> > > > > > void seek(TopicPartition partition, long offset, byte[]
> >> > > > offsetMetadata);
> >> > > > > > byte[] positionMetadata(TopicPartition partition);
> >> > > > > >
> >> > > > > > What do you think?
> >> > > > > >
> >> > > > > > Thanks,
> >> > > > > > Jason
> >> > > > > >
> >> > > > > > On Thu, Jan 4, 2018 at 7:04 PM, Dong Lin <lindong28@gmail.com
> >
> >> > > wrote:
> >> > > > > >
> >> > > > > > > Hey Jun, Jason,
> >> > > > > > >
> >> > > > > > > Thanks much for all the feedback. I have updated the KIP
> >> based on
> >> > > the
> >> > > > > > > latest discussion. Can you help check whether it looks good?
> >> > > > > > >
> >> > > > > > > Thanks,
> >> > > > > > > Dong
> >> > > > > > >
> >> > > > > > > On Thu, Jan 4, 2018 at 5:36 PM, Dong Lin <
> lindong28@gmail.com
> >> >
> >> > > > wrote:
> >> > > > > > >
> >> > > > > > > > Hey Jun,
> >> > > > > > > >
> >> > > > > > > > Hmm... thinking about this more, I am not sure that the
> >> > proposed
> >> > > > API
> >> > > > > is
> >> > > > > > > > sufficient. For users that store offset externally, we
> >> probably
> >> > > > need
> >> > > > > > > extra
> >> > > > > > > > API to return the leader_epoch and partition_epoch for all
> >> > > > partitions
> >> > > > > > > that
> >> > > > > > > > consumers are consuming. I suppose these users currently
> use
> >> > > > > position()
> >> > > > > > > to
> >> > > > > > > > get the offset. Thus we probably need a new method
> >> > > > > > positionWithEpoch(..)
> >> > > > > > > to
> >> > > > > > > > return <offset, partition_epoch, leader_epoch>. Does this
> >> sound
> >> > > > > > > reasonable?
> >> > > > > > > >
> >> > > > > > > > Thanks,
> >> > > > > > > > Dong
> >> > > > > > > >
> >> > > > > > > >
> >> > > > > > > > On Thu, Jan 4, 2018 at 5:26 PM, Jun Rao <jun@confluent.io
> >
> >> > > wrote:
> >> > > > > > > >
> >> > > > > > > >> Hi, Dong,
> >> > > > > > > >>
> >> > > > > > > >> Yes, that's what I am thinking. OffsetEpoch will be
> >> composed
> >> > of
> >> > > > > > > >> (partition_epoch,
> >> > > > > > > >> leader_epoch).
> >> > > > > > > >>
> >> > > > > > > >> Thanks,
> >> > > > > > > >>
> >> > > > > > > >> Jun
> >> > > > > > > >>
> >> > > > > > > >>
> >> > > > > > > >> On Thu, Jan 4, 2018 at 4:22 PM, Dong Lin <
> >> lindong28@gmail.com
> >> > >
> >> > > > > wrote:
> >> > > > > > > >>
> >> > > > > > > >> > Hey Jun,
> >> > > > > > > >> >
> >> > > > > > > >> > Thanks much. I like the the new API that you proposed.
> I
> >> am
> >> > > not
> >> > > > > sure
> >> > > > > > > >> what
> >> > > > > > > >> > you exactly mean by offset_epoch. I suppose that we can
> >> use
> >> > > the
> >> > > > > pair
> >> > > > > > > of
> >> > > > > > > >> > (partition_epoch, leader_epoch) as the offset_epoch,
> >> right?
> >> > > > > > > >> >
> >> > > > > > > >> > Thanks,
> >> > > > > > > >> > Dong
> >> > > > > > > >> >
> >> > > > > > > >> > On Thu, Jan 4, 2018 at 4:02 PM, Jun Rao <
> >> jun@confluent.io>
> >> > > > wrote:
> >> > > > > > > >> >
> >> > > > > > > >> > > Hi, Dong,
> >> > > > > > > >> > >
> >> > > > > > > >> > > Got it. The api that you proposed works. The question
> >> is
> >> > > > whether
> >> > > > > > > >> that's
> >> > > > > > > >> > the
> >> > > > > > > >> > > api that we want to have in the long term. My concern
> >> is
> >> > > that
> >> > > > > > while
> >> > > > > > > >> the
> >> > > > > > > >> > api
> >> > > > > > > >> > > change is simple, the new api seems harder to explain
> >> and
> >> > > use.
> >> > > > > For
> >> > > > > > > >> > example,
> >> > > > > > > >> > > a consumer storing offsets externally now needs to
> call
> >> > > > > > > >> > > waitForMetadataUpdate() after calling seek().
> >> > > > > > > >> > >
> >> > > > > > > >> > > An alternative approach is to make the following
> >> > compatible
> >> > > > api
> >> > > > > > > >> changes
> >> > > > > > > >> > in
> >> > > > > > > >> > > Consumer.
> >> > > > > > > >> > > * Add an additional OffsetEpoch field in
> >> > OffsetAndMetadata.
> >> > > > (no
> >> > > > > > need
> >> > > > > > > >> to
> >> > > > > > > >> > > change the CommitSync() api)
> >> > > > > > > >> > > * Add a new api seek(TopicPartition partition, long
> >> > offset,
> >> > > > > > > >> OffsetEpoch
> >> > > > > > > >> > > offsetEpoch). We can potentially deprecate the old
> api
> >> > > > > > > >> > seek(TopicPartition
> >> > > > > > > >> > > partition, long offset) in the future.
> >> > > > > > > >> > >
> >> > > > > > > >> > > The alternative approach has similar amount of api
> >> changes
> >> > > as
> >> > > > > > yours
> >> > > > > > > >> but
> >> > > > > > > >> > has
> >> > > > > > > >> > > the following benefits.
> >> > > > > > > >> > > 1. The api works in a similar way as how offset
> >> management
> >> > > > works
> >> > > > > > now
> >> > > > > > > >> and
> >> > > > > > > >> > is
> >> > > > > > > >> > > probably what we want in the long term.
> >> > > > > > > >> > > 2. It can reset offsets better when there is data
> loss
> >> due
> >> > > to
> >> > > > > > > unclean
> >> > > > > > > >> > > leader election or correlated replica failure.
> >> > > > > > > >> > > 3. It can reset offsets better when topic is
> recreated.
> >> > > > > > > >> > >
> >> > > > > > > >> > > Thanks,
> >> > > > > > > >> > >
> >> > > > > > > >> > > Jun
> >> > > > > > > >> > >
> >> > > > > > > >> > >
> >> > > > > > > >> > > On Thu, Jan 4, 2018 at 2:05 PM, Dong Lin <
> >> > > lindong28@gmail.com
> >> > > > >
> >> > > > > > > wrote:
> >> > > > > > > >> > >
> >> > > > > > > >> > > > Hey Jun,
> >> > > > > > > >> > > >
> >> > > > > > > >> > > > Yeah I agree that ideally we don't want an ever
> >> growing
> >> > > > global
> >> > > > > > > >> metadata
> >> > > > > > > >> > > > version. I just think it may be more desirable to
> >> keep
> >> > the
> >> > > > > > > consumer
> >> > > > > > > >> API
> >> > > > > > > >> > > > simple.
> >> > > > > > > >> > > >
> >> > > > > > > >> > > > In my current proposal, metadata version returned
> in
> >> the
> >> > > > fetch
> >> > > > > > > >> response
> >> > > > > > > >> > > > will be stored with the offset together. More
> >> > > specifically,
> >> > > > > the
> >> > > > > > > >> > > > metadata_epoch in the new offset topic schema will
> be
> >> > the
> >> > > > > > largest
> >> > > > > > > >> > > > metadata_epoch from all the MetadataResponse and
> >> > > > FetchResponse
> >> > > > > > > ever
> >> > > > > > > >> > > > received by this consumer.
> >> > > > > > > >> > > >
> >> > > > > > > >> > > > We probably don't have to change the consumer API
> for
> >> > > > > > > >> > > > commitSync(Map<TopicPartition, OffsetAndMetadata>).
> >> If
> >> > > user
> >> > > > > > calls
> >> > > > > > > >> > > > commitSync(...) to commit offset 10 for a given
> >> > partition,
> >> > > > for
> >> > > > > > > most
> >> > > > > > > >> > > > use-cases, this consumer instance should have
> >> consumed
> >> > > > message
> >> > > > > > > with
> >> > > > > > > >> > > offset
> >> > > > > > > >> > > > 9 from this partition, in which case the consumer
> can
> >> > > > remember
> >> > > > > > and
> >> > > > > > > >> use
> >> > > > > > > >> > > the
> >> > > > > > > >> > > > metadata_epoch from the corresponding FetchResponse
> >> when
> >> > > > > > > committing
> >> > > > > > > >> > > offset.
> >> > > > > > > >> > > > If user calls commitSync(..) to commit offset 10
> for
> >> a
> >> > > given
> >> > > > > > > >> partition
> >> > > > > > > >> > > > without having consumed the message with offset 9
> >> using
> >> > > this
> >> > > > > > > >> consumer
> >> > > > > > > >> > > > instance, this is probably an advanced use-case. In
> >> this
> >> > > > case
> >> > > > > > the
> >> > > > > > > >> > > advanced
> >> > > > > > > >> > > > user can retrieve the metadata_epoch using the
> newly
> >> > added
> >> > > > > > > >> > > metadataEpoch()
> >> > > > > > > >> > > > API after it fetches the message with offset 9
> >> (probably
> >> > > > from
> >> > > > > > > >> another
> >> > > > > > > >> > > > consumer instance) and encode this metadata_epoch
> in
> >> the
> >> > > > > > > >> > > > string OffsetAndMetadata.metadata. Do you think
> this
> >> > > > solution
> >> > > > > > > would
> >> > > > > > > >> > work?
> >> > > > > > > >> > > >
> >> > > > > > > >> > > > By "not sure that I fully understand your latest
> >> > > > suggestion",
> >> > > > > > are
> >> > > > > > > >> you
> >> > > > > > > >> > > > referring to solution related to unclean leader
> >> election
> >> > > > using
> >> > > > > > > >> > > leader_epoch
> >> > > > > > > >> > > > in my previous email?
> >> > > > > > > >> > > >
> >> > > > > > > >> > > > Thanks,
> >> > > > > > > >> > > > Dong
> >> > > > > > > >> > > >
> >> > > > > > > >> > > > On Thu, Jan 4, 2018 at 1:33 PM, Jun Rao <
> >> > jun@confluent.io
> >> > > >
> >> > > > > > wrote:
> >> > > > > > > >> > > >
> >> > > > > > > >> > > > > Hi, Dong,
> >> > > > > > > >> > > > >
> >> > > > > > > >> > > > > Not sure that I fully understand your latest
> >> > suggestion.
> >> > > > > > > >> Returning an
> >> > > > > > > >> > > > ever
> >> > > > > > > >> > > > > growing global metadata version itself is no
> ideal,
> >> > but
> >> > > is
> >> > > > > > fine.
> >> > > > > > > >> My
> >> > > > > > > >> > > > > question is whether the metadata version returned
> >> in
> >> > the
> >> > > > > fetch
> >> > > > > > > >> > response
> >> > > > > > > >> > > > > needs to be stored with the offset together if
> >> offsets
> >> > > are
> >> > > > > > > stored
> >> > > > > > > >> > > > > externally. If so, we also have to change the
> >> consumer
> >> > > API
> >> > > > > for
> >> > > > > > > >> > > > commitSync()
> >> > > > > > > >> > > > > and need to worry about compatibility. If we
> don't
> >> > store
> >> > > > the
> >> > > > > > > >> metadata
> >> > > > > > > >> > > > > version together with the offset, on a consumer
> >> > restart,
> >> > > > > it's
> >> > > > > > > not
> >> > > > > > > >> > clear
> >> > > > > > > >> > > > how
> >> > > > > > > >> > > > > we can ensure the metadata in the consumer is
> high
> >> > > enough
> >> > > > > > since
> >> > > > > > > >> there
> >> > > > > > > >> > > is
> >> > > > > > > >> > > > no
> >> > > > > > > >> > > > > metadata version to compare with.
> >> > > > > > > >> > > > >
> >> > > > > > > >> > > > > Thanks,
> >> > > > > > > >> > > > >
> >> > > > > > > >> > > > > Jun
> >> > > > > > > >> > > > >
> >> > > > > > > >> > > > >
> >> > > > > > > >> > > > > On Wed, Jan 3, 2018 at 6:43 PM, Dong Lin <
> >> > > > > lindong28@gmail.com
> >> > > > > > >
> >> > > > > > > >> > wrote:
> >> > > > > > > >> > > > >
> >> > > > > > > >> > > > > > Hey Jun,
> >> > > > > > > >> > > > > >
> >> > > > > > > >> > > > > > Thanks much for the explanation.
> >> > > > > > > >> > > > > >
> >> > > > > > > >> > > > > > I understand the advantage of partition_epoch
> >> over
> >> > > > > > > >> metadata_epoch.
> >> > > > > > > >> > My
> >> > > > > > > >> > > > > > current concern is that the use of leader_epoch
> >> and
> >> > > the
> >> > > > > > > >> > > partition_epoch
> >> > > > > > > >> > > > > > requires us considerable change to consumer's
> >> public
> >> > > API
> >> > > > > to
> >> > > > > > > take
> >> > > > > > > >> > care
> >> > > > > > > >> > > > of
> >> > > > > > > >> > > > > > the case where user stores offset externally.
> For
> >> > > > example,
> >> > > > > > > >> > > *consumer*.
> >> > > > > > > >> > > > > > *commitSync*(..) would have to take a map whose
> >> > value
> >> > > is
> >> > > > > > > >> <offset,
> >> > > > > > > >> > > > > metadata,
> >> > > > > > > >> > > > > > leader epoch, partition epoch>.
> >> > *consumer*.*seek*(...)
> >> > > > > would
> >> > > > > > > >> also
> >> > > > > > > >> > > need
> >> > > > > > > >> > > > > > leader_epoch and partition_epoch as parameter.
> >> > > > Technically
> >> > > > > > we
> >> > > > > > > >> can
> >> > > > > > > >> > > > > probably
> >> > > > > > > >> > > > > > still make it work in a backward compatible
> >> manner
> >> > > after
> >> > > > > > > careful
> >> > > > > > > >> > > design
> >> > > > > > > >> > > > > and
> >> > > > > > > >> > > > > > discussion. But these changes can make the
> >> > consumer's
> >> > > > > > > interface
> >> > > > > > > >> > > > > > unnecessarily complex for more users who do not
> >> > store
> >> > > > > offset
> >> > > > > > > >> > > > externally.
> >> > > > > > > >> > > > > >
> >> > > > > > > >> > > > > > After thinking more about it, we can address
> all
> >> > > > problems
> >> > > > > > > >> discussed
> >> > > > > > > >> > > by
> >> > > > > > > >> > > > > only
> >> > > > > > > >> > > > > > using the metadata_epoch without introducing
> >> > > > leader_epoch
> >> > > > > or
> >> > > > > > > the
> >> > > > > > > >> > > > > > partition_epoch. The current KIP describes the
> >> > changes
> >> > > > to
> >> > > > > > the
> >> > > > > > > >> > > consumer
> >> > > > > > > >> > > > > API
> >> > > > > > > >> > > > > > and how the new API can be used if user stores
> >> > offset
> >> > > > > > > >> externally.
> >> > > > > > > >> > In
> >> > > > > > > >> > > > > order
> >> > > > > > > >> > > > > > to address the scenario you described earlier,
> we
> >> > can
> >> > > > > > include
> >> > > > > > > >> > > > > > metadata_epoch in the FetchResponse and the
> >> > > > > > > LeaderAndIsrRequest.
> >> > > > > > > >> > > > Consumer
> >> > > > > > > >> > > > > > remembers the largest metadata_epoch from all
> the
> >> > > > > > > FetchResponse
> >> > > > > > > >> it
> >> > > > > > > >> > > has
> >> > > > > > > >> > > > > > received. The metadata_epoch committed with the
> >> > > offset,
> >> > > > > > either
> >> > > > > > > >> > within
> >> > > > > > > >> > > > or
> >> > > > > > > >> > > > > > outside Kafka, should be the largest
> >> metadata_epoch
> >> > > > across
> >> > > > > > all
> >> > > > > > > >> > > > > > FetchResponse and MetadataResponse ever
> received
> >> by
> >> > > this
> >> > > > > > > >> consumer.
> >> > > > > > > >> > > > > >
> >> > > > > > > >> > > > > > The drawback of using only the metadata_epoch
> is
> >> > that
> >> > > we
> >> > > > > can
> >> > > > > > > not
> >> > > > > > > >> > > always
> >> > > > > > > >> > > > > do
> >> > > > > > > >> > > > > > the smart offset reset in case of unclean
> leader
> >> > > > election
> >> > > > > > > which
> >> > > > > > > >> you
> >> > > > > > > >> > > > > > mentioned earlier. But in most case, unclean
> >> leader
> >> > > > > election
> >> > > > > > > >> > probably
> >> > > > > > > >> > > > > > happens when consumer is not
> >> rebalancing/restarting.
> >> > > In
> >> > > > > > these
> >> > > > > > > >> > cases,
> >> > > > > > > >> > > > > either
> >> > > > > > > >> > > > > > consumer is not directly affected by unclean
> >> leader
> >> > > > > election
> >> > > > > > > >> since
> >> > > > > > > >> > it
> >> > > > > > > >> > > > is
> >> > > > > > > >> > > > > > not consuming from the end of the log, or
> >> consumer
> >> > can
> >> > > > > > derive
> >> > > > > > > >> the
> >> > > > > > > >> > > > > > leader_epoch from the most recent message
> >> received
> >> > > > before
> >> > > > > it
> >> > > > > > > >> sees
> >> > > > > > > >> > > > > > OffsetOutOfRangeException. So I am not sure it
> is
> >> > > worth
> >> > > > > > adding
> >> > > > > > > >> the
> >> > > > > > > >> > > > > > leader_epoch to consumer API to address the
> >> > remaining
> >> > > > > corner
> >> > > > > > > >> case.
> >> > > > > > > >> > > What
> >> > > > > > > >> > > > > do
> >> > > > > > > >> > > > > > you think?
> >> > > > > > > >> > > > > >
> >> > > > > > > >> > > > > > Thanks,
> >> > > > > > > >> > > > > > Dong
> >> > > > > > > >> > > > > >
> >> > > > > > > >> > > > > >
> >> > > > > > > >> > > > > >
> >> > > > > > > >> > > > > > On Tue, Jan 2, 2018 at 6:28 PM, Jun Rao <
> >> > > > jun@confluent.io
> >> > > > > >
> >> > > > > > > >> wrote:
> >> > > > > > > >> > > > > >
> >> > > > > > > >> > > > > > > Hi, Dong,
> >> > > > > > > >> > > > > > >
> >> > > > > > > >> > > > > > > Thanks for the reply.
> >> > > > > > > >> > > > > > >
> >> > > > > > > >> > > > > > > To solve the topic recreation issue, we could
> >> use
> >> > > > > either a
> >> > > > > > > >> global
> >> > > > > > > >> > > > > > metadata
> >> > > > > > > >> > > > > > > version or a partition level epoch. But
> either
> >> one
> >> > > > will
> >> > > > > > be a
> >> > > > > > > >> new
> >> > > > > > > >> > > > > concept,
> >> > > > > > > >> > > > > > > right? To me, the latter seems more natural.
> It
> >> > also
> >> > > > > makes
> >> > > > > > > it
> >> > > > > > > >> > > easier
> >> > > > > > > >> > > > to
> >> > > > > > > >> > > > > > > detect if a consumer's offset is still valid
> >> > after a
> >> > > > > topic
> >> > > > > > > is
> >> > > > > > > >> > > > > recreated.
> >> > > > > > > >> > > > > > As
> >> > > > > > > >> > > > > > > you pointed out, we don't need to store the
> >> > > partition
> >> > > > > > epoch
> >> > > > > > > in
> >> > > > > > > >> > the
> >> > > > > > > >> > > > > > message.
> >> > > > > > > >> > > > > > > The following is what I am thinking. When a
> >> > > partition
> >> > > > is
> >> > > > > > > >> created,
> >> > > > > > > >> > > we
> >> > > > > > > >> > > > > can
> >> > > > > > > >> > > > > > > assign a partition epoch from an
> >> ever-increasing
> >> > > > global
> >> > > > > > > >> counter
> >> > > > > > > >> > and
> >> > > > > > > >> > > > > store
> >> > > > > > > >> > > > > > > it in /brokers/topics/[topic]/
> >> > > > partitions/[partitionId]
> >> > > > > in
> >> > > > > > > ZK.
> >> > > > > > > >> > The
> >> > > > > > > >> > > > > > > partition
> >> > > > > > > >> > > > > > > epoch is propagated to every broker. The
> >> consumer
> >> > > will
> >> > > > > be
> >> > > > > > > >> > tracking
> >> > > > > > > >> > > a
> >> > > > > > > >> > > > > > tuple
> >> > > > > > > >> > > > > > > of <offset, leader epoch, partition epoch>
> for
> >> > > > offsets.
> >> > > > > > If a
> >> > > > > > > >> > topic
> >> > > > > > > >> > > is
> >> > > > > > > >> > > > > > > recreated, it's possible that a consumer's
> >> offset
> >> > > and
> >> > > > > > leader
> >> > > > > > > >> > epoch
> >> > > > > > > >> > > > > still
> >> > > > > > > >> > > > > > > match that in the broker, but partition epoch
> >> > won't
> >> > > > be.
> >> > > > > In
> >> > > > > > > >> this
> >> > > > > > > >> > > case,
> >> > > > > > > >> > > > > we
> >> > > > > > > >> > > > > > > can potentially still treat the consumer's
> >> offset
> >> > as
> >> > > > out
> >> > > > > > of
> >> > > > > > > >> range
> >> > > > > > > >> > > and
> >> > > > > > > >> > > > > > reset
> >> > > > > > > >> > > > > > > the offset based on the offset reset policy
> in
> >> the
> >> > > > > > consumer.
> >> > > > > > > >> This
> >> > > > > > > >> > > > seems
> >> > > > > > > >> > > > > > > harder to do with a global metadata version.
> >> > > > > > > >> > > > > > >
> >> > > > > > > >> > > > > > > Jun
> >> > > > > > > >> > > > > > >
> >> > > > > > > >> > > > > > >
> >> > > > > > > >> > > > > > >
> >> > > > > > > >> > > > > > > On Mon, Dec 25, 2017 at 6:56 AM, Dong Lin <
> >> > > > > > > >> lindong28@gmail.com>
> >> > > > > > > >> > > > wrote:
> >> > > > > > > >> > > > > > >
> >> > > > > > > >> > > > > > > > Hey Jun,
> >> > > > > > > >> > > > > > > >
> >> > > > > > > >> > > > > > > > This is a very good example. After thinking
> >> > > through
> >> > > > > this
> >> > > > > > > in
> >> > > > > > > >> > > > detail, I
> >> > > > > > > >> > > > > > > agree
> >> > > > > > > >> > > > > > > > that we need to commit offset with leader
> >> epoch
> >> > in
> >> > > > > order
> >> > > > > > > to
> >> > > > > > > >> > > address
> >> > > > > > > >> > > > > > this
> >> > > > > > > >> > > > > > > > example.
> >> > > > > > > >> > > > > > > >
> >> > > > > > > >> > > > > > > > I think the remaining question is how to
> >> address
> >> > > the
> >> > > > > > > >> scenario
> >> > > > > > > >> > > that
> >> > > > > > > >> > > > > the
> >> > > > > > > >> > > > > > > > topic is deleted and re-created. One
> possible
> >> > > > solution
> >> > > > > > is
> >> > > > > > > to
> >> > > > > > > >> > > commit
> >> > > > > > > >> > > > > > > offset
> >> > > > > > > >> > > > > > > > with both the leader epoch and the metadata
> >> > > version.
> >> > > > > The
> >> > > > > > > >> logic
> >> > > > > > > >> > > and
> >> > > > > > > >> > > > > the
> >> > > > > > > >> > > > > > > > implementation of this solution does not
> >> > require a
> >> > > > new
> >> > > > > > > >> concept
> >> > > > > > > >> > > > (e.g.
> >> > > > > > > >> > > > > > > > partition epoch) and it does not require
> any
> >> > > change
> >> > > > to
> >> > > > > > the
> >> > > > > > > >> > > message
> >> > > > > > > >> > > > > > format
> >> > > > > > > >> > > > > > > > or leader epoch. It also allows us to order
> >> the
> >> > > > > metadata
> >> > > > > > > in
> >> > > > > > > >> a
> >> > > > > > > >> > > > > > > > straightforward manner which may be useful
> in
> >> > the
> >> > > > > > future.
> >> > > > > > > >> So it
> >> > > > > > > >> > > may
> >> > > > > > > >> > > > > be
> >> > > > > > > >> > > > > > a
> >> > > > > > > >> > > > > > > > better solution than generating a random
> >> > partition
> >> > > > > epoch
> >> > > > > > > >> every
> >> > > > > > > >> > > time
> >> > > > > > > >> > > > > we
> >> > > > > > > >> > > > > > > > create a partition. Does this sound
> >> reasonable?
> >> > > > > > > >> > > > > > > >
> >> > > > > > > >> > > > > > > > Previously one concern with using the
> >> metadata
> >> > > > version
> >> > > > > > is
> >> > > > > > > >> that
> >> > > > > > > >> > > > > consumer
> >> > > > > > > >> > > > > > > > will be forced to refresh metadata even if
> >> > > metadata
> >> > > > > > > version
> >> > > > > > > >> is
> >> > > > > > > >> > > > > > increased
> >> > > > > > > >> > > > > > > > due to topics that the consumer is not
> >> > interested
> >> > > > in.
> >> > > > > > Now
> >> > > > > > > I
> >> > > > > > > >> > > > realized
> >> > > > > > > >> > > > > > that
> >> > > > > > > >> > > > > > > > this is probably not a problem. Currently
> >> client
> >> > > > will
> >> > > > > > > >> refresh
> >> > > > > > > >> > > > > metadata
> >> > > > > > > >> > > > > > > > either due to InvalidMetadataException in
> the
> >> > > > response
> >> > > > > > > from
> >> > > > > > > >> > > broker
> >> > > > > > > >> > > > or
> >> > > > > > > >> > > > > > due
> >> > > > > > > >> > > > > > > > to metadata expiry. The addition of the
> >> metadata
> >> > > > > version
> >> > > > > > > >> should
> >> > > > > > > >> > > > > > increase
> >> > > > > > > >> > > > > > > > the overhead of metadata refresh caused by
> >> > > > > > > >> > > > InvalidMetadataException.
> >> > > > > > > >> > > > > If
> >> > > > > > > >> > > > > > > > client refresh metadata due to expiry and
> it
> >> > > > receives
> >> > > > > a
> >> > > > > > > >> > metadata
> >> > > > > > > >> > > > > whose
> >> > > > > > > >> > > > > > > > version is lower than the current metadata
> >> > > version,
> >> > > > we
> >> > > > > > can
> >> > > > > > > >> > reject
> >> > > > > > > >> > > > the
> >> > > > > > > >> > > > > > > > metadata but still reset the metadata age,
> >> which
> >> > > > > > > essentially
> >> > > > > > > >> > keep
> >> > > > > > > >> > > > the
> >> > > > > > > >> > > > > > > > existing behavior in the client.
> >> > > > > > > >> > > > > > > >
> >> > > > > > > >> > > > > > > > Thanks much,
> >> > > > > > > >> > > > > > > > Dong
> >> > > > > > > >> > > > > > > >
> >> > > > > > > >> > > > > > >
> >> > > > > > > >> > > > > >
> >> > > > > > > >> > > > >
> >> > > > > > > >> > > >
> >> > > > > > > >> > >
> >> > > > > > > >> >
> >> > > > > > > >>
> >> > > > > > > >
> >> > > > > > > >
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Dong Lin <li...@gmail.com>.
Regarding the use of the global epoch in 65), it is very similar to the
proposal of the metadata_epoch we discussed earlier. The main difference is
that this epoch is incremented when we create/expand/delete topic and does
not change when controller re-send metadata.

I looked at our previous discussion. It seems that we prefer
partition_epoch over the metadata_epoch because 1) we prefer not to have an
ever growing metadata_epoch and 2) we can reset offset better when topic is
re-created. The use of global topic_epoch avoids the drawback of an ever
quickly ever growing metadata_epoch. Though the global epoch does not allow
us to recognize the invalid offset committed before the topic re-creation,
we can probably just delete the offset when we delete a topic. Thus I am
not very sure whether it is still worthwhile to have a per-partition
partition_epoch if the metadata already has the global epoch.


On Tue, Jan 9, 2018 at 6:58 PM, Dong Lin <li...@gmail.com> wrote:

> Hey Jun,
>
> Thanks so much. These comments very useful. Please see below my comments.
>
> On Mon, Jan 8, 2018 at 5:52 PM, Jun Rao <ju...@confluent.io> wrote:
>
>> Hi, Dong,
>>
>> Thanks for the updated KIP. A few more comments.
>>
>> 60. Perhaps having a partition epoch is more flexible since in the future,
>> we may support deleting a partition as well.
>>
>
> Yeah I have considered this. I think we can probably still support
> deleting a partition by using the topic_epoch -- when partition of a topic
> is deleted or created, epoch of all partitions of this topic will be
> incremented by 1. Therefore, if that partition is re-created later, the
> epoch of that partition will still be larger than its epoch before the
> deletion, which still allows the client to order the metadata for the
> purpose of this KIP. Does this sound reasonable?
>
> The advantage of using topic_epoch instead of partition_epoch is that the
> size of the /brokers/topics/[topic] znode and request/response size can be
> smaller. We have a limit on the maximum size of znode (typically 1MB). Use
> partition epoch can effectively reduce the number of partitions that can be
> described by the /brokers/topics/[topic] znode.
>
> One use-case of partition_epoch for client to detect that the committed
> offset, either from kafka offset topic or from the external store is
> invalid after partition deletion and re-creation. However, it seems that we
> can also address this use-case with other approaches. For example, when
> AdminClient deletes partitions, it can also delete the committed offsets
> for those partitions from the offset topic. If user stores offset
> externally, it might make sense for user to similarly remove offsets of
> related partitions after these partitions are deleted. So I am not sure
> that we should use partition_epoch in this KIP.
>
>
>>
>> 61. It seems that the leader epoch returned in the position() call should
>> the the leader epoch returned in the fetch response, not the one in the
>> metadata cache of the client.
>
>
> I think this is a good idea. Just to double check, this change does not
> affect the correctness or performance of this KIP. But it can be useful if
> we want to use the leader_epoch to better handle the offset rest in case of
> unclean leader election, which is listed in the future work. Is this
> understanding correct?
>
> I have updated the KIP to specify that the leader_epoch returned by
> position() should be the largest leader_epoch of those already consumed
> messages whose offset < position. If no message has been consumed since
> consumer initialization, the leader_epoch from seek() or
> OffsetFetchResponse should be used. The offset included in the
> OffsetCommitRequest will also be determined in the similar manner.
>
>
>>
>> 62. I am wondering if we should return the partition epoch in the fetch
>> response as well. In the current proposal, if a topic is recreated and the
>> new leader is on the same broker as the old one, there is nothing to force
>> the metadata refresh in the client. So, the client may still associate the
>> offset with the old partition epoch.
>>
>
> Could you help me understand the problem if a client associates old
> partition_epoch (or the topic_epoch as of the current KIP) with the offset?
> The main purpose of the topic_epoch is to be able to drop leader_epoch to 0
> after a partition is deleted and re-created. I guess you may be thinking
> about using the partition_epoch to detect that the committed offset is
> invalid? In that case, I am wondering if the alternative approach described
> in 60) would be reasonable.
>
>
>>
>> 63. There is some subtle coordination between the LeaderAndIsrRequest and
>> UpdateMetadataRequest. Currently, when a leader changes, the controller
>> first sends the LeaderAndIsrRequest to the assigned replicas and the
>> UpdateMetadataRequest to every broker. So, there could be a small window
>> when the leader already receives the new partition epoch in the
>> LeaderAndIsrRequest, but the metadata cache in the broker hasn't been
>> updated with the latest partition epoch. Not sure what's the best way to
>> address this issue. Perhaps we can update the metadata cache on the broker
>> with both LeaderAndIsrRequest and UpdateMetadataRequest. The challenge is
>> that the two have slightly different data. For example, only the latter
>> has
>> all endpoints.
>>
>
> I am not sure whether this is a problem. Could you explain a bit more what
> specific problem this small window can cause?
>
> Since client can fetch metadata from any broker in the cluster, and given
> that different brokers receive request (e.g. LeaderAndIsrRequest and
> UpdateMetadataRequest) in arbitrary order, the metadata received by client
> can be in arbitrary order (either newer or older) compared to the broker's
> leadership state even if a given broker receives LeaderAndIsrRequest and
> UpdateMetadataRequest simultaneously. So I am not sure it is useful to
> update broker's cache with LeaderAndIsrRequest.
>
>
>> 64. The enforcement of leader epoch in Offset commit: We allow a consumer
>> to set an arbitrary offset. So it's possible for offsets or leader epoch
>> to
>> go backwards. I am not sure if we could always enforce that the leader
>> epoch only goes up on the broker.
>>
>
> Sure. I have removed this check from the KIP.
>
> BTW, we can probably still ensure that the leader_epoch always increase if
> the leader_epoch used with offset commit is the max(leader_epoch of the
> message with offset = the committed offset - 1, the largest known
> leader_epoch from the metadata). But I don't have a good use-case for this
> alternative definition. So I choose the keep the KIP simple by requiring
> leader_epoch to always increase.
>
>
>> 65. Good point on handling missing partition epoch due to topic deletion.
>> Another potential way to address this is to additionally propagate the
>> global partition epoch to brokers and the clients. This way, when a
>> partition epoch is missing, we can use the global partition epoch to
>> reason
>> about which metadata is more recent.
>>
>
> This is a great idea. The global epoch can be used to order the metadata
> and help us recognize the more recent metadata if a topic (or partition) is
> deleted and re-created.
>
> Actually, it seems we only need to propagate the global epoch to brokers
> and clients without propagating this epoch on a per-topic or per-partition
> basic. Doing so would simply interface changes made this KIP. Does this
> approach sound reasonable?
>
>
>> 66. A client may also get an offset by time using the offsetForTimes()
>> api.
>> So, we probably want to include offsetInternalMetadata in
>> OffsetAndTimestamp
>> as well.
>>
>
> You are right. This probably also requires us to change the
> ListOffsetRequest as well. I will update the KIP after we agree on the
> solution for 65).
>
>
>>
>> 67. InteralMetadata can be a bit confusing with the metadata field already
>> there. Perhaps we can just call it OffsetEpoch. It might be useful to make
>> OffsetEpoch printable at least for debugging purpose. Once you do that, we
>> are already exposing the internal fields. So, not sure if it's worth
>> hiding
>> them. If we do want to hide them, perhaps we can have sth like the
>> following. The binary encoding is probably more efficient than JSON for
>> external storage.
>>
>> OffsetEpoch {
>>  static OffsetEpoch decode(byte[]);
>>
>>   public byte[] encode();
>>
>>   public String toString();
>> }
>>
>
> Thanks much. I like this solution. I have updated the KIP accordingly.
>
>
>
>>
>> Jun
>>
>> On Mon, Jan 8, 2018 at 4:22 PM, Dong Lin <li...@gmail.com> wrote:
>>
>> > Hey Jason,
>> >
>> > Certainly. This sounds good. I have updated the KIP to clarity that the
>> > global epoch will be incremented by 1 each time a topic is deleted.
>> >
>> > Thanks,
>> > Dong
>> >
>> > On Mon, Jan 8, 2018 at 4:09 PM, Jason Gustafson <ja...@confluent.io>
>> > wrote:
>> >
>> > > Hi Dong,
>> > >
>> > >
>> > > I think your approach will allow user to distinguish between the
>> metadata
>> > > > before and after the topic deletion. I also agree that this can be
>> > > > potentially be useful to user. I am just not very sure whether we
>> > already
>> > > > have a good use-case to make the additional complexity worthwhile.
>> It
>> > > seems
>> > > > that this feature is kind of independent of the main problem of this
>> > KIP.
>> > > > Could we add this as a future work?
>> > >
>> > >
>> > > Do you think it's fair if we bump the topic epoch on deletion and
>> leave
>> > > propagation of the epoch for deleted topics for future work? I don't
>> > think
>> > > this adds much complexity and it makes the behavior consistent: every
>> > topic
>> > > mutation results in an epoch bump.
>> > >
>> > > Thanks,
>> > > Jason
>> > >
>> > > On Mon, Jan 8, 2018 at 3:14 PM, Dong Lin <li...@gmail.com> wrote:
>> > >
>> > > > Hey Ismael,
>> > > >
>> > > > I guess we actually need user to see this field so that user can
>> store
>> > > this
>> > > > value in the external store together with the offset. We just prefer
>> > the
>> > > > value to be opaque to discourage most users from interpreting this
>> > value.
>> > > > One more advantage of using such an opaque field is to be able to
>> > evolve
>> > > > the information (or schema) of this value without changing consumer
>> API
>> > > in
>> > > > the future.
>> > > >
>> > > > I also thinking it is probably OK for user to be able to interpret
>> this
>> > > > value, particularly for those advanced users.
>> > > >
>> > > > Thanks,
>> > > > Dong
>> > > >
>> > > > On Mon, Jan 8, 2018 at 2:34 PM, Ismael Juma <is...@juma.me.uk>
>> wrote:
>> > > >
>> > > > > On Fri, Jan 5, 2018 at 7:15 PM, Jason Gustafson <
>> jason@confluent.io>
>> > > > > wrote:
>> > > > > >
>> > > > > > class OffsetAndMetadata {
>> > > > > >   long offset;
>> > > > > >   byte[] offsetMetadata;
>> > > > > >   String metadata;
>> > > > > > }
>> > > > >
>> > > > >
>> > > > > > Admittedly, the naming is a bit annoying, but we can probably
>> come
>> > up
>> > > > > with
>> > > > > > something better. Internally the byte array would have a
>> version.
>> > If
>> > > in
>> > > > > the
>> > > > > > future we have anything else we need to add, we can update the
>> > > version
>> > > > > and
>> > > > > > we wouldn't need any new APIs.
>> > > > > >
>> > > > >
>> > > > > We can also add fields to a class in a compatible way. So, it
>> seems
>> > to
>> > > me
>> > > > > that the main advantage of the byte array is that it's opaque to
>> the
>> > > > user.
>> > > > > Is that correct? If so, we could also add any opaque metadata in a
>> > > > subclass
>> > > > > so that users don't even see it (unless they cast it, but then
>> > they're
>> > > on
>> > > > > their own).
>> > > > >
>> > > > > Ismael
>> > > > >
>> > > > > The corresponding seek() and position() APIs might look something
>> > like
>> > > > > this:
>> > > > > >
>> > > > > > void seek(TopicPartition partition, long offset, byte[]
>> > > > offsetMetadata);
>> > > > > > byte[] positionMetadata(TopicPartition partition);
>> > > > > >
>> > > > > > What do you think?
>> > > > > >
>> > > > > > Thanks,
>> > > > > > Jason
>> > > > > >
>> > > > > > On Thu, Jan 4, 2018 at 7:04 PM, Dong Lin <li...@gmail.com>
>> > > wrote:
>> > > > > >
>> > > > > > > Hey Jun, Jason,
>> > > > > > >
>> > > > > > > Thanks much for all the feedback. I have updated the KIP
>> based on
>> > > the
>> > > > > > > latest discussion. Can you help check whether it looks good?
>> > > > > > >
>> > > > > > > Thanks,
>> > > > > > > Dong
>> > > > > > >
>> > > > > > > On Thu, Jan 4, 2018 at 5:36 PM, Dong Lin <lindong28@gmail.com
>> >
>> > > > wrote:
>> > > > > > >
>> > > > > > > > Hey Jun,
>> > > > > > > >
>> > > > > > > > Hmm... thinking about this more, I am not sure that the
>> > proposed
>> > > > API
>> > > > > is
>> > > > > > > > sufficient. For users that store offset externally, we
>> probably
>> > > > need
>> > > > > > > extra
>> > > > > > > > API to return the leader_epoch and partition_epoch for all
>> > > > partitions
>> > > > > > > that
>> > > > > > > > consumers are consuming. I suppose these users currently use
>> > > > > position()
>> > > > > > > to
>> > > > > > > > get the offset. Thus we probably need a new method
>> > > > > > positionWithEpoch(..)
>> > > > > > > to
>> > > > > > > > return <offset, partition_epoch, leader_epoch>. Does this
>> sound
>> > > > > > > reasonable?
>> > > > > > > >
>> > > > > > > > Thanks,
>> > > > > > > > Dong
>> > > > > > > >
>> > > > > > > >
>> > > > > > > > On Thu, Jan 4, 2018 at 5:26 PM, Jun Rao <ju...@confluent.io>
>> > > wrote:
>> > > > > > > >
>> > > > > > > >> Hi, Dong,
>> > > > > > > >>
>> > > > > > > >> Yes, that's what I am thinking. OffsetEpoch will be
>> composed
>> > of
>> > > > > > > >> (partition_epoch,
>> > > > > > > >> leader_epoch).
>> > > > > > > >>
>> > > > > > > >> Thanks,
>> > > > > > > >>
>> > > > > > > >> Jun
>> > > > > > > >>
>> > > > > > > >>
>> > > > > > > >> On Thu, Jan 4, 2018 at 4:22 PM, Dong Lin <
>> lindong28@gmail.com
>> > >
>> > > > > wrote:
>> > > > > > > >>
>> > > > > > > >> > Hey Jun,
>> > > > > > > >> >
>> > > > > > > >> > Thanks much. I like the the new API that you proposed. I
>> am
>> > > not
>> > > > > sure
>> > > > > > > >> what
>> > > > > > > >> > you exactly mean by offset_epoch. I suppose that we can
>> use
>> > > the
>> > > > > pair
>> > > > > > > of
>> > > > > > > >> > (partition_epoch, leader_epoch) as the offset_epoch,
>> right?
>> > > > > > > >> >
>> > > > > > > >> > Thanks,
>> > > > > > > >> > Dong
>> > > > > > > >> >
>> > > > > > > >> > On Thu, Jan 4, 2018 at 4:02 PM, Jun Rao <
>> jun@confluent.io>
>> > > > wrote:
>> > > > > > > >> >
>> > > > > > > >> > > Hi, Dong,
>> > > > > > > >> > >
>> > > > > > > >> > > Got it. The api that you proposed works. The question
>> is
>> > > > whether
>> > > > > > > >> that's
>> > > > > > > >> > the
>> > > > > > > >> > > api that we want to have in the long term. My concern
>> is
>> > > that
>> > > > > > while
>> > > > > > > >> the
>> > > > > > > >> > api
>> > > > > > > >> > > change is simple, the new api seems harder to explain
>> and
>> > > use.
>> > > > > For
>> > > > > > > >> > example,
>> > > > > > > >> > > a consumer storing offsets externally now needs to call
>> > > > > > > >> > > waitForMetadataUpdate() after calling seek().
>> > > > > > > >> > >
>> > > > > > > >> > > An alternative approach is to make the following
>> > compatible
>> > > > api
>> > > > > > > >> changes
>> > > > > > > >> > in
>> > > > > > > >> > > Consumer.
>> > > > > > > >> > > * Add an additional OffsetEpoch field in
>> > OffsetAndMetadata.
>> > > > (no
>> > > > > > need
>> > > > > > > >> to
>> > > > > > > >> > > change the CommitSync() api)
>> > > > > > > >> > > * Add a new api seek(TopicPartition partition, long
>> > offset,
>> > > > > > > >> OffsetEpoch
>> > > > > > > >> > > offsetEpoch). We can potentially deprecate the old api
>> > > > > > > >> > seek(TopicPartition
>> > > > > > > >> > > partition, long offset) in the future.
>> > > > > > > >> > >
>> > > > > > > >> > > The alternative approach has similar amount of api
>> changes
>> > > as
>> > > > > > yours
>> > > > > > > >> but
>> > > > > > > >> > has
>> > > > > > > >> > > the following benefits.
>> > > > > > > >> > > 1. The api works in a similar way as how offset
>> management
>> > > > works
>> > > > > > now
>> > > > > > > >> and
>> > > > > > > >> > is
>> > > > > > > >> > > probably what we want in the long term.
>> > > > > > > >> > > 2. It can reset offsets better when there is data loss
>> due
>> > > to
>> > > > > > > unclean
>> > > > > > > >> > > leader election or correlated replica failure.
>> > > > > > > >> > > 3. It can reset offsets better when topic is recreated.
>> > > > > > > >> > >
>> > > > > > > >> > > Thanks,
>> > > > > > > >> > >
>> > > > > > > >> > > Jun
>> > > > > > > >> > >
>> > > > > > > >> > >
>> > > > > > > >> > > On Thu, Jan 4, 2018 at 2:05 PM, Dong Lin <
>> > > lindong28@gmail.com
>> > > > >
>> > > > > > > wrote:
>> > > > > > > >> > >
>> > > > > > > >> > > > Hey Jun,
>> > > > > > > >> > > >
>> > > > > > > >> > > > Yeah I agree that ideally we don't want an ever
>> growing
>> > > > global
>> > > > > > > >> metadata
>> > > > > > > >> > > > version. I just think it may be more desirable to
>> keep
>> > the
>> > > > > > > consumer
>> > > > > > > >> API
>> > > > > > > >> > > > simple.
>> > > > > > > >> > > >
>> > > > > > > >> > > > In my current proposal, metadata version returned in
>> the
>> > > > fetch
>> > > > > > > >> response
>> > > > > > > >> > > > will be stored with the offset together. More
>> > > specifically,
>> > > > > the
>> > > > > > > >> > > > metadata_epoch in the new offset topic schema will be
>> > the
>> > > > > > largest
>> > > > > > > >> > > > metadata_epoch from all the MetadataResponse and
>> > > > FetchResponse
>> > > > > > > ever
>> > > > > > > >> > > > received by this consumer.
>> > > > > > > >> > > >
>> > > > > > > >> > > > We probably don't have to change the consumer API for
>> > > > > > > >> > > > commitSync(Map<TopicPartition, OffsetAndMetadata>).
>> If
>> > > user
>> > > > > > calls
>> > > > > > > >> > > > commitSync(...) to commit offset 10 for a given
>> > partition,
>> > > > for
>> > > > > > > most
>> > > > > > > >> > > > use-cases, this consumer instance should have
>> consumed
>> > > > message
>> > > > > > > with
>> > > > > > > >> > > offset
>> > > > > > > >> > > > 9 from this partition, in which case the consumer can
>> > > > remember
>> > > > > > and
>> > > > > > > >> use
>> > > > > > > >> > > the
>> > > > > > > >> > > > metadata_epoch from the corresponding FetchResponse
>> when
>> > > > > > > committing
>> > > > > > > >> > > offset.
>> > > > > > > >> > > > If user calls commitSync(..) to commit offset 10 for
>> a
>> > > given
>> > > > > > > >> partition
>> > > > > > > >> > > > without having consumed the message with offset 9
>> using
>> > > this
>> > > > > > > >> consumer
>> > > > > > > >> > > > instance, this is probably an advanced use-case. In
>> this
>> > > > case
>> > > > > > the
>> > > > > > > >> > > advanced
>> > > > > > > >> > > > user can retrieve the metadata_epoch using the newly
>> > added
>> > > > > > > >> > > metadataEpoch()
>> > > > > > > >> > > > API after it fetches the message with offset 9
>> (probably
>> > > > from
>> > > > > > > >> another
>> > > > > > > >> > > > consumer instance) and encode this metadata_epoch in
>> the
>> > > > > > > >> > > > string OffsetAndMetadata.metadata. Do you think this
>> > > > solution
>> > > > > > > would
>> > > > > > > >> > work?
>> > > > > > > >> > > >
>> > > > > > > >> > > > By "not sure that I fully understand your latest
>> > > > suggestion",
>> > > > > > are
>> > > > > > > >> you
>> > > > > > > >> > > > referring to solution related to unclean leader
>> election
>> > > > using
>> > > > > > > >> > > leader_epoch
>> > > > > > > >> > > > in my previous email?
>> > > > > > > >> > > >
>> > > > > > > >> > > > Thanks,
>> > > > > > > >> > > > Dong
>> > > > > > > >> > > >
>> > > > > > > >> > > > On Thu, Jan 4, 2018 at 1:33 PM, Jun Rao <
>> > jun@confluent.io
>> > > >
>> > > > > > wrote:
>> > > > > > > >> > > >
>> > > > > > > >> > > > > Hi, Dong,
>> > > > > > > >> > > > >
>> > > > > > > >> > > > > Not sure that I fully understand your latest
>> > suggestion.
>> > > > > > > >> Returning an
>> > > > > > > >> > > > ever
>> > > > > > > >> > > > > growing global metadata version itself is no ideal,
>> > but
>> > > is
>> > > > > > fine.
>> > > > > > > >> My
>> > > > > > > >> > > > > question is whether the metadata version returned
>> in
>> > the
>> > > > > fetch
>> > > > > > > >> > response
>> > > > > > > >> > > > > needs to be stored with the offset together if
>> offsets
>> > > are
>> > > > > > > stored
>> > > > > > > >> > > > > externally. If so, we also have to change the
>> consumer
>> > > API
>> > > > > for
>> > > > > > > >> > > > commitSync()
>> > > > > > > >> > > > > and need to worry about compatibility. If we don't
>> > store
>> > > > the
>> > > > > > > >> metadata
>> > > > > > > >> > > > > version together with the offset, on a consumer
>> > restart,
>> > > > > it's
>> > > > > > > not
>> > > > > > > >> > clear
>> > > > > > > >> > > > how
>> > > > > > > >> > > > > we can ensure the metadata in the consumer is high
>> > > enough
>> > > > > > since
>> > > > > > > >> there
>> > > > > > > >> > > is
>> > > > > > > >> > > > no
>> > > > > > > >> > > > > metadata version to compare with.
>> > > > > > > >> > > > >
>> > > > > > > >> > > > > Thanks,
>> > > > > > > >> > > > >
>> > > > > > > >> > > > > Jun
>> > > > > > > >> > > > >
>> > > > > > > >> > > > >
>> > > > > > > >> > > > > On Wed, Jan 3, 2018 at 6:43 PM, Dong Lin <
>> > > > > lindong28@gmail.com
>> > > > > > >
>> > > > > > > >> > wrote:
>> > > > > > > >> > > > >
>> > > > > > > >> > > > > > Hey Jun,
>> > > > > > > >> > > > > >
>> > > > > > > >> > > > > > Thanks much for the explanation.
>> > > > > > > >> > > > > >
>> > > > > > > >> > > > > > I understand the advantage of partition_epoch
>> over
>> > > > > > > >> metadata_epoch.
>> > > > > > > >> > My
>> > > > > > > >> > > > > > current concern is that the use of leader_epoch
>> and
>> > > the
>> > > > > > > >> > > partition_epoch
>> > > > > > > >> > > > > > requires us considerable change to consumer's
>> public
>> > > API
>> > > > > to
>> > > > > > > take
>> > > > > > > >> > care
>> > > > > > > >> > > > of
>> > > > > > > >> > > > > > the case where user stores offset externally. For
>> > > > example,
>> > > > > > > >> > > *consumer*.
>> > > > > > > >> > > > > > *commitSync*(..) would have to take a map whose
>> > value
>> > > is
>> > > > > > > >> <offset,
>> > > > > > > >> > > > > metadata,
>> > > > > > > >> > > > > > leader epoch, partition epoch>.
>> > *consumer*.*seek*(...)
>> > > > > would
>> > > > > > > >> also
>> > > > > > > >> > > need
>> > > > > > > >> > > > > > leader_epoch and partition_epoch as parameter.
>> > > > Technically
>> > > > > > we
>> > > > > > > >> can
>> > > > > > > >> > > > > probably
>> > > > > > > >> > > > > > still make it work in a backward compatible
>> manner
>> > > after
>> > > > > > > careful
>> > > > > > > >> > > design
>> > > > > > > >> > > > > and
>> > > > > > > >> > > > > > discussion. But these changes can make the
>> > consumer's
>> > > > > > > interface
>> > > > > > > >> > > > > > unnecessarily complex for more users who do not
>> > store
>> > > > > offset
>> > > > > > > >> > > > externally.
>> > > > > > > >> > > > > >
>> > > > > > > >> > > > > > After thinking more about it, we can address all
>> > > > problems
>> > > > > > > >> discussed
>> > > > > > > >> > > by
>> > > > > > > >> > > > > only
>> > > > > > > >> > > > > > using the metadata_epoch without introducing
>> > > > leader_epoch
>> > > > > or
>> > > > > > > the
>> > > > > > > >> > > > > > partition_epoch. The current KIP describes the
>> > changes
>> > > > to
>> > > > > > the
>> > > > > > > >> > > consumer
>> > > > > > > >> > > > > API
>> > > > > > > >> > > > > > and how the new API can be used if user stores
>> > offset
>> > > > > > > >> externally.
>> > > > > > > >> > In
>> > > > > > > >> > > > > order
>> > > > > > > >> > > > > > to address the scenario you described earlier, we
>> > can
>> > > > > > include
>> > > > > > > >> > > > > > metadata_epoch in the FetchResponse and the
>> > > > > > > LeaderAndIsrRequest.
>> > > > > > > >> > > > Consumer
>> > > > > > > >> > > > > > remembers the largest metadata_epoch from all the
>> > > > > > > FetchResponse
>> > > > > > > >> it
>> > > > > > > >> > > has
>> > > > > > > >> > > > > > received. The metadata_epoch committed with the
>> > > offset,
>> > > > > > either
>> > > > > > > >> > within
>> > > > > > > >> > > > or
>> > > > > > > >> > > > > > outside Kafka, should be the largest
>> metadata_epoch
>> > > > across
>> > > > > > all
>> > > > > > > >> > > > > > FetchResponse and MetadataResponse ever received
>> by
>> > > this
>> > > > > > > >> consumer.
>> > > > > > > >> > > > > >
>> > > > > > > >> > > > > > The drawback of using only the metadata_epoch is
>> > that
>> > > we
>> > > > > can
>> > > > > > > not
>> > > > > > > >> > > always
>> > > > > > > >> > > > > do
>> > > > > > > >> > > > > > the smart offset reset in case of unclean leader
>> > > > election
>> > > > > > > which
>> > > > > > > >> you
>> > > > > > > >> > > > > > mentioned earlier. But in most case, unclean
>> leader
>> > > > > election
>> > > > > > > >> > probably
>> > > > > > > >> > > > > > happens when consumer is not
>> rebalancing/restarting.
>> > > In
>> > > > > > these
>> > > > > > > >> > cases,
>> > > > > > > >> > > > > either
>> > > > > > > >> > > > > > consumer is not directly affected by unclean
>> leader
>> > > > > election
>> > > > > > > >> since
>> > > > > > > >> > it
>> > > > > > > >> > > > is
>> > > > > > > >> > > > > > not consuming from the end of the log, or
>> consumer
>> > can
>> > > > > > derive
>> > > > > > > >> the
>> > > > > > > >> > > > > > leader_epoch from the most recent message
>> received
>> > > > before
>> > > > > it
>> > > > > > > >> sees
>> > > > > > > >> > > > > > OffsetOutOfRangeException. So I am not sure it is
>> > > worth
>> > > > > > adding
>> > > > > > > >> the
>> > > > > > > >> > > > > > leader_epoch to consumer API to address the
>> > remaining
>> > > > > corner
>> > > > > > > >> case.
>> > > > > > > >> > > What
>> > > > > > > >> > > > > do
>> > > > > > > >> > > > > > you think?
>> > > > > > > >> > > > > >
>> > > > > > > >> > > > > > Thanks,
>> > > > > > > >> > > > > > Dong
>> > > > > > > >> > > > > >
>> > > > > > > >> > > > > >
>> > > > > > > >> > > > > >
>> > > > > > > >> > > > > > On Tue, Jan 2, 2018 at 6:28 PM, Jun Rao <
>> > > > jun@confluent.io
>> > > > > >
>> > > > > > > >> wrote:
>> > > > > > > >> > > > > >
>> > > > > > > >> > > > > > > Hi, Dong,
>> > > > > > > >> > > > > > >
>> > > > > > > >> > > > > > > Thanks for the reply.
>> > > > > > > >> > > > > > >
>> > > > > > > >> > > > > > > To solve the topic recreation issue, we could
>> use
>> > > > > either a
>> > > > > > > >> global
>> > > > > > > >> > > > > > metadata
>> > > > > > > >> > > > > > > version or a partition level epoch. But either
>> one
>> > > > will
>> > > > > > be a
>> > > > > > > >> new
>> > > > > > > >> > > > > concept,
>> > > > > > > >> > > > > > > right? To me, the latter seems more natural. It
>> > also
>> > > > > makes
>> > > > > > > it
>> > > > > > > >> > > easier
>> > > > > > > >> > > > to
>> > > > > > > >> > > > > > > detect if a consumer's offset is still valid
>> > after a
>> > > > > topic
>> > > > > > > is
>> > > > > > > >> > > > > recreated.
>> > > > > > > >> > > > > > As
>> > > > > > > >> > > > > > > you pointed out, we don't need to store the
>> > > partition
>> > > > > > epoch
>> > > > > > > in
>> > > > > > > >> > the
>> > > > > > > >> > > > > > message.
>> > > > > > > >> > > > > > > The following is what I am thinking. When a
>> > > partition
>> > > > is
>> > > > > > > >> created,
>> > > > > > > >> > > we
>> > > > > > > >> > > > > can
>> > > > > > > >> > > > > > > assign a partition epoch from an
>> ever-increasing
>> > > > global
>> > > > > > > >> counter
>> > > > > > > >> > and
>> > > > > > > >> > > > > store
>> > > > > > > >> > > > > > > it in /brokers/topics/[topic]/
>> > > > partitions/[partitionId]
>> > > > > in
>> > > > > > > ZK.
>> > > > > > > >> > The
>> > > > > > > >> > > > > > > partition
>> > > > > > > >> > > > > > > epoch is propagated to every broker. The
>> consumer
>> > > will
>> > > > > be
>> > > > > > > >> > tracking
>> > > > > > > >> > > a
>> > > > > > > >> > > > > > tuple
>> > > > > > > >> > > > > > > of <offset, leader epoch, partition epoch> for
>> > > > offsets.
>> > > > > > If a
>> > > > > > > >> > topic
>> > > > > > > >> > > is
>> > > > > > > >> > > > > > > recreated, it's possible that a consumer's
>> offset
>> > > and
>> > > > > > leader
>> > > > > > > >> > epoch
>> > > > > > > >> > > > > still
>> > > > > > > >> > > > > > > match that in the broker, but partition epoch
>> > won't
>> > > > be.
>> > > > > In
>> > > > > > > >> this
>> > > > > > > >> > > case,
>> > > > > > > >> > > > > we
>> > > > > > > >> > > > > > > can potentially still treat the consumer's
>> offset
>> > as
>> > > > out
>> > > > > > of
>> > > > > > > >> range
>> > > > > > > >> > > and
>> > > > > > > >> > > > > > reset
>> > > > > > > >> > > > > > > the offset based on the offset reset policy in
>> the
>> > > > > > consumer.
>> > > > > > > >> This
>> > > > > > > >> > > > seems
>> > > > > > > >> > > > > > > harder to do with a global metadata version.
>> > > > > > > >> > > > > > >
>> > > > > > > >> > > > > > > Jun
>> > > > > > > >> > > > > > >
>> > > > > > > >> > > > > > >
>> > > > > > > >> > > > > > >
>> > > > > > > >> > > > > > > On Mon, Dec 25, 2017 at 6:56 AM, Dong Lin <
>> > > > > > > >> lindong28@gmail.com>
>> > > > > > > >> > > > wrote:
>> > > > > > > >> > > > > > >
>> > > > > > > >> > > > > > > > Hey Jun,
>> > > > > > > >> > > > > > > >
>> > > > > > > >> > > > > > > > This is a very good example. After thinking
>> > > through
>> > > > > this
>> > > > > > > in
>> > > > > > > >> > > > detail, I
>> > > > > > > >> > > > > > > agree
>> > > > > > > >> > > > > > > > that we need to commit offset with leader
>> epoch
>> > in
>> > > > > order
>> > > > > > > to
>> > > > > > > >> > > address
>> > > > > > > >> > > > > > this
>> > > > > > > >> > > > > > > > example.
>> > > > > > > >> > > > > > > >
>> > > > > > > >> > > > > > > > I think the remaining question is how to
>> address
>> > > the
>> > > > > > > >> scenario
>> > > > > > > >> > > that
>> > > > > > > >> > > > > the
>> > > > > > > >> > > > > > > > topic is deleted and re-created. One possible
>> > > > solution
>> > > > > > is
>> > > > > > > to
>> > > > > > > >> > > commit
>> > > > > > > >> > > > > > > offset
>> > > > > > > >> > > > > > > > with both the leader epoch and the metadata
>> > > version.
>> > > > > The
>> > > > > > > >> logic
>> > > > > > > >> > > and
>> > > > > > > >> > > > > the
>> > > > > > > >> > > > > > > > implementation of this solution does not
>> > require a
>> > > > new
>> > > > > > > >> concept
>> > > > > > > >> > > > (e.g.
>> > > > > > > >> > > > > > > > partition epoch) and it does not require any
>> > > change
>> > > > to
>> > > > > > the
>> > > > > > > >> > > message
>> > > > > > > >> > > > > > format
>> > > > > > > >> > > > > > > > or leader epoch. It also allows us to order
>> the
>> > > > > metadata
>> > > > > > > in
>> > > > > > > >> a
>> > > > > > > >> > > > > > > > straightforward manner which may be useful in
>> > the
>> > > > > > future.
>> > > > > > > >> So it
>> > > > > > > >> > > may
>> > > > > > > >> > > > > be
>> > > > > > > >> > > > > > a
>> > > > > > > >> > > > > > > > better solution than generating a random
>> > partition
>> > > > > epoch
>> > > > > > > >> every
>> > > > > > > >> > > time
>> > > > > > > >> > > > > we
>> > > > > > > >> > > > > > > > create a partition. Does this sound
>> reasonable?
>> > > > > > > >> > > > > > > >
>> > > > > > > >> > > > > > > > Previously one concern with using the
>> metadata
>> > > > version
>> > > > > > is
>> > > > > > > >> that
>> > > > > > > >> > > > > consumer
>> > > > > > > >> > > > > > > > will be forced to refresh metadata even if
>> > > metadata
>> > > > > > > version
>> > > > > > > >> is
>> > > > > > > >> > > > > > increased
>> > > > > > > >> > > > > > > > due to topics that the consumer is not
>> > interested
>> > > > in.
>> > > > > > Now
>> > > > > > > I
>> > > > > > > >> > > > realized
>> > > > > > > >> > > > > > that
>> > > > > > > >> > > > > > > > this is probably not a problem. Currently
>> client
>> > > > will
>> > > > > > > >> refresh
>> > > > > > > >> > > > > metadata
>> > > > > > > >> > > > > > > > either due to InvalidMetadataException in the
>> > > > response
>> > > > > > > from
>> > > > > > > >> > > broker
>> > > > > > > >> > > > or
>> > > > > > > >> > > > > > due
>> > > > > > > >> > > > > > > > to metadata expiry. The addition of the
>> metadata
>> > > > > version
>> > > > > > > >> should
>> > > > > > > >> > > > > > increase
>> > > > > > > >> > > > > > > > the overhead of metadata refresh caused by
>> > > > > > > >> > > > InvalidMetadataException.
>> > > > > > > >> > > > > If
>> > > > > > > >> > > > > > > > client refresh metadata due to expiry and it
>> > > > receives
>> > > > > a
>> > > > > > > >> > metadata
>> > > > > > > >> > > > > whose
>> > > > > > > >> > > > > > > > version is lower than the current metadata
>> > > version,
>> > > > we
>> > > > > > can
>> > > > > > > >> > reject
>> > > > > > > >> > > > the
>> > > > > > > >> > > > > > > > metadata but still reset the metadata age,
>> which
>> > > > > > > essentially
>> > > > > > > >> > keep
>> > > > > > > >> > > > the
>> > > > > > > >> > > > > > > > existing behavior in the client.
>> > > > > > > >> > > > > > > >
>> > > > > > > >> > > > > > > > Thanks much,
>> > > > > > > >> > > > > > > > Dong
>> > > > > > > >> > > > > > > >
>> > > > > > > >> > > > > > >
>> > > > > > > >> > > > > >
>> > > > > > > >> > > > >
>> > > > > > > >> > > >
>> > > > > > > >> > >
>> > > > > > > >> >
>> > > > > > > >>
>> > > > > > > >
>> > > > > > > >
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>>
>
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Dong Lin <li...@gmail.com>.
Hey Jun,

Thanks so much. These comments very useful. Please see below my comments.

On Mon, Jan 8, 2018 at 5:52 PM, Jun Rao <ju...@confluent.io> wrote:

> Hi, Dong,
>
> Thanks for the updated KIP. A few more comments.
>
> 60. Perhaps having a partition epoch is more flexible since in the future,
> we may support deleting a partition as well.
>

Yeah I have considered this. I think we can probably still support deleting
a partition by using the topic_epoch -- when partition of a topic is
deleted or created, epoch of all partitions of this topic will be
incremented by 1. Therefore, if that partition is re-created later, the
epoch of that partition will still be larger than its epoch before the
deletion, which still allows the client to order the metadata for the
purpose of this KIP. Does this sound reasonable?

The advantage of using topic_epoch instead of partition_epoch is that the
size of the /brokers/topics/[topic] znode and request/response size can be
smaller. We have a limit on the maximum size of znode (typically 1MB). Use
partition epoch can effectively reduce the number of partitions that can be
described by the /brokers/topics/[topic] znode.

One use-case of partition_epoch for client to detect that the committed
offset, either from kafka offset topic or from the external store is
invalid after partition deletion and re-creation. However, it seems that we
can also address this use-case with other approaches. For example, when
AdminClient deletes partitions, it can also delete the committed offsets
for those partitions from the offset topic. If user stores offset
externally, it might make sense for user to similarly remove offsets of
related partitions after these partitions are deleted. So I am not sure
that we should use partition_epoch in this KIP.


>
> 61. It seems that the leader epoch returned in the position() call should
> the the leader epoch returned in the fetch response, not the one in the
> metadata cache of the client.


I think this is a good idea. Just to double check, this change does not
affect the correctness or performance of this KIP. But it can be useful if
we want to use the leader_epoch to better handle the offset rest in case of
unclean leader election, which is listed in the future work. Is this
understanding correct?

I have updated the KIP to specify that the leader_epoch returned by
position() should be the largest leader_epoch of those already consumed
messages whose offset < position. If no message has been consumed since
consumer initialization, the leader_epoch from seek() or
OffsetFetchResponse should be used. The offset included in the
OffsetCommitRequest will also be determined in the similar manner.


>
> 62. I am wondering if we should return the partition epoch in the fetch
> response as well. In the current proposal, if a topic is recreated and the
> new leader is on the same broker as the old one, there is nothing to force
> the metadata refresh in the client. So, the client may still associate the
> offset with the old partition epoch.
>

Could you help me understand the problem if a client associates old
partition_epoch (or the topic_epoch as of the current KIP) with the offset?
The main purpose of the topic_epoch is to be able to drop leader_epoch to 0
after a partition is deleted and re-created. I guess you may be thinking
about using the partition_epoch to detect that the committed offset is
invalid? In that case, I am wondering if the alternative approach described
in 60) would be reasonable.


>
> 63. There is some subtle coordination between the LeaderAndIsrRequest and
> UpdateMetadataRequest. Currently, when a leader changes, the controller
> first sends the LeaderAndIsrRequest to the assigned replicas and the
> UpdateMetadataRequest to every broker. So, there could be a small window
> when the leader already receives the new partition epoch in the
> LeaderAndIsrRequest, but the metadata cache in the broker hasn't been
> updated with the latest partition epoch. Not sure what's the best way to
> address this issue. Perhaps we can update the metadata cache on the broker
> with both LeaderAndIsrRequest and UpdateMetadataRequest. The challenge is
> that the two have slightly different data. For example, only the latter has
> all endpoints.
>

I am not sure whether this is a problem. Could you explain a bit more what
specific problem this small window can cause?

Since client can fetch metadata from any broker in the cluster, and given
that different brokers receive request (e.g. LeaderAndIsrRequest and
UpdateMetadataRequest) in arbitrary order, the metadata received by client
can be in arbitrary order (either newer or older) compared to the broker's
leadership state even if a given broker receives LeaderAndIsrRequest and
UpdateMetadataRequest simultaneously. So I am not sure it is useful to
update broker's cache with LeaderAndIsrRequest.


> 64. The enforcement of leader epoch in Offset commit: We allow a consumer
> to set an arbitrary offset. So it's possible for offsets or leader epoch to
> go backwards. I am not sure if we could always enforce that the leader
> epoch only goes up on the broker.
>

Sure. I have removed this check from the KIP.

BTW, we can probably still ensure that the leader_epoch always increase if
the leader_epoch used with offset commit is the max(leader_epoch of the
message with offset = the committed offset - 1, the largest known
leader_epoch from the metadata). But I don't have a good use-case for this
alternative definition. So I choose the keep the KIP simple by requiring
leader_epoch to always increase.


> 65. Good point on handling missing partition epoch due to topic deletion.
> Another potential way to address this is to additionally propagate the
> global partition epoch to brokers and the clients. This way, when a
> partition epoch is missing, we can use the global partition epoch to reason
> about which metadata is more recent.
>

This is a great idea. The global epoch can be used to order the metadata
and help us recognize the more recent metadata if a topic (or partition) is
deleted and re-created.

Actually, it seems we only need to propagate the global epoch to brokers
and clients without propagating this epoch on a per-topic or per-partition
basic. Doing so would simply interface changes made this KIP. Does this
approach sound reasonable?


> 66. A client may also get an offset by time using the offsetForTimes() api.
> So, we probably want to include offsetInternalMetadata in
> OffsetAndTimestamp
> as well.
>

You are right. This probably also requires us to change the
ListOffsetRequest as well. I will update the KIP after we agree on the
solution for 65).


>
> 67. InteralMetadata can be a bit confusing with the metadata field already
> there. Perhaps we can just call it OffsetEpoch. It might be useful to make
> OffsetEpoch printable at least for debugging purpose. Once you do that, we
> are already exposing the internal fields. So, not sure if it's worth hiding
> them. If we do want to hide them, perhaps we can have sth like the
> following. The binary encoding is probably more efficient than JSON for
> external storage.
>
> OffsetEpoch {
>  static OffsetEpoch decode(byte[]);
>
>   public byte[] encode();
>
>   public String toString();
> }
>

Thanks much. I like this solution. I have updated the KIP accordingly.



>
> Jun
>
> On Mon, Jan 8, 2018 at 4:22 PM, Dong Lin <li...@gmail.com> wrote:
>
> > Hey Jason,
> >
> > Certainly. This sounds good. I have updated the KIP to clarity that the
> > global epoch will be incremented by 1 each time a topic is deleted.
> >
> > Thanks,
> > Dong
> >
> > On Mon, Jan 8, 2018 at 4:09 PM, Jason Gustafson <ja...@confluent.io>
> > wrote:
> >
> > > Hi Dong,
> > >
> > >
> > > I think your approach will allow user to distinguish between the
> metadata
> > > > before and after the topic deletion. I also agree that this can be
> > > > potentially be useful to user. I am just not very sure whether we
> > already
> > > > have a good use-case to make the additional complexity worthwhile. It
> > > seems
> > > > that this feature is kind of independent of the main problem of this
> > KIP.
> > > > Could we add this as a future work?
> > >
> > >
> > > Do you think it's fair if we bump the topic epoch on deletion and leave
> > > propagation of the epoch for deleted topics for future work? I don't
> > think
> > > this adds much complexity and it makes the behavior consistent: every
> > topic
> > > mutation results in an epoch bump.
> > >
> > > Thanks,
> > > Jason
> > >
> > > On Mon, Jan 8, 2018 at 3:14 PM, Dong Lin <li...@gmail.com> wrote:
> > >
> > > > Hey Ismael,
> > > >
> > > > I guess we actually need user to see this field so that user can
> store
> > > this
> > > > value in the external store together with the offset. We just prefer
> > the
> > > > value to be opaque to discourage most users from interpreting this
> > value.
> > > > One more advantage of using such an opaque field is to be able to
> > evolve
> > > > the information (or schema) of this value without changing consumer
> API
> > > in
> > > > the future.
> > > >
> > > > I also thinking it is probably OK for user to be able to interpret
> this
> > > > value, particularly for those advanced users.
> > > >
> > > > Thanks,
> > > > Dong
> > > >
> > > > On Mon, Jan 8, 2018 at 2:34 PM, Ismael Juma <is...@juma.me.uk>
> wrote:
> > > >
> > > > > On Fri, Jan 5, 2018 at 7:15 PM, Jason Gustafson <
> jason@confluent.io>
> > > > > wrote:
> > > > > >
> > > > > > class OffsetAndMetadata {
> > > > > >   long offset;
> > > > > >   byte[] offsetMetadata;
> > > > > >   String metadata;
> > > > > > }
> > > > >
> > > > >
> > > > > > Admittedly, the naming is a bit annoying, but we can probably
> come
> > up
> > > > > with
> > > > > > something better. Internally the byte array would have a version.
> > If
> > > in
> > > > > the
> > > > > > future we have anything else we need to add, we can update the
> > > version
> > > > > and
> > > > > > we wouldn't need any new APIs.
> > > > > >
> > > > >
> > > > > We can also add fields to a class in a compatible way. So, it seems
> > to
> > > me
> > > > > that the main advantage of the byte array is that it's opaque to
> the
> > > > user.
> > > > > Is that correct? If so, we could also add any opaque metadata in a
> > > > subclass
> > > > > so that users don't even see it (unless they cast it, but then
> > they're
> > > on
> > > > > their own).
> > > > >
> > > > > Ismael
> > > > >
> > > > > The corresponding seek() and position() APIs might look something
> > like
> > > > > this:
> > > > > >
> > > > > > void seek(TopicPartition partition, long offset, byte[]
> > > > offsetMetadata);
> > > > > > byte[] positionMetadata(TopicPartition partition);
> > > > > >
> > > > > > What do you think?
> > > > > >
> > > > > > Thanks,
> > > > > > Jason
> > > > > >
> > > > > > On Thu, Jan 4, 2018 at 7:04 PM, Dong Lin <li...@gmail.com>
> > > wrote:
> > > > > >
> > > > > > > Hey Jun, Jason,
> > > > > > >
> > > > > > > Thanks much for all the feedback. I have updated the KIP based
> on
> > > the
> > > > > > > latest discussion. Can you help check whether it looks good?
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Dong
> > > > > > >
> > > > > > > On Thu, Jan 4, 2018 at 5:36 PM, Dong Lin <li...@gmail.com>
> > > > wrote:
> > > > > > >
> > > > > > > > Hey Jun,
> > > > > > > >
> > > > > > > > Hmm... thinking about this more, I am not sure that the
> > proposed
> > > > API
> > > > > is
> > > > > > > > sufficient. For users that store offset externally, we
> probably
> > > > need
> > > > > > > extra
> > > > > > > > API to return the leader_epoch and partition_epoch for all
> > > > partitions
> > > > > > > that
> > > > > > > > consumers are consuming. I suppose these users currently use
> > > > > position()
> > > > > > > to
> > > > > > > > get the offset. Thus we probably need a new method
> > > > > > positionWithEpoch(..)
> > > > > > > to
> > > > > > > > return <offset, partition_epoch, leader_epoch>. Does this
> sound
> > > > > > > reasonable?
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Dong
> > > > > > > >
> > > > > > > >
> > > > > > > > On Thu, Jan 4, 2018 at 5:26 PM, Jun Rao <ju...@confluent.io>
> > > wrote:
> > > > > > > >
> > > > > > > >> Hi, Dong,
> > > > > > > >>
> > > > > > > >> Yes, that's what I am thinking. OffsetEpoch will be composed
> > of
> > > > > > > >> (partition_epoch,
> > > > > > > >> leader_epoch).
> > > > > > > >>
> > > > > > > >> Thanks,
> > > > > > > >>
> > > > > > > >> Jun
> > > > > > > >>
> > > > > > > >>
> > > > > > > >> On Thu, Jan 4, 2018 at 4:22 PM, Dong Lin <
> lindong28@gmail.com
> > >
> > > > > wrote:
> > > > > > > >>
> > > > > > > >> > Hey Jun,
> > > > > > > >> >
> > > > > > > >> > Thanks much. I like the the new API that you proposed. I
> am
> > > not
> > > > > sure
> > > > > > > >> what
> > > > > > > >> > you exactly mean by offset_epoch. I suppose that we can
> use
> > > the
> > > > > pair
> > > > > > > of
> > > > > > > >> > (partition_epoch, leader_epoch) as the offset_epoch,
> right?
> > > > > > > >> >
> > > > > > > >> > Thanks,
> > > > > > > >> > Dong
> > > > > > > >> >
> > > > > > > >> > On Thu, Jan 4, 2018 at 4:02 PM, Jun Rao <jun@confluent.io
> >
> > > > wrote:
> > > > > > > >> >
> > > > > > > >> > > Hi, Dong,
> > > > > > > >> > >
> > > > > > > >> > > Got it. The api that you proposed works. The question is
> > > > whether
> > > > > > > >> that's
> > > > > > > >> > the
> > > > > > > >> > > api that we want to have in the long term. My concern is
> > > that
> > > > > > while
> > > > > > > >> the
> > > > > > > >> > api
> > > > > > > >> > > change is simple, the new api seems harder to explain
> and
> > > use.
> > > > > For
> > > > > > > >> > example,
> > > > > > > >> > > a consumer storing offsets externally now needs to call
> > > > > > > >> > > waitForMetadataUpdate() after calling seek().
> > > > > > > >> > >
> > > > > > > >> > > An alternative approach is to make the following
> > compatible
> > > > api
> > > > > > > >> changes
> > > > > > > >> > in
> > > > > > > >> > > Consumer.
> > > > > > > >> > > * Add an additional OffsetEpoch field in
> > OffsetAndMetadata.
> > > > (no
> > > > > > need
> > > > > > > >> to
> > > > > > > >> > > change the CommitSync() api)
> > > > > > > >> > > * Add a new api seek(TopicPartition partition, long
> > offset,
> > > > > > > >> OffsetEpoch
> > > > > > > >> > > offsetEpoch). We can potentially deprecate the old api
> > > > > > > >> > seek(TopicPartition
> > > > > > > >> > > partition, long offset) in the future.
> > > > > > > >> > >
> > > > > > > >> > > The alternative approach has similar amount of api
> changes
> > > as
> > > > > > yours
> > > > > > > >> but
> > > > > > > >> > has
> > > > > > > >> > > the following benefits.
> > > > > > > >> > > 1. The api works in a similar way as how offset
> management
> > > > works
> > > > > > now
> > > > > > > >> and
> > > > > > > >> > is
> > > > > > > >> > > probably what we want in the long term.
> > > > > > > >> > > 2. It can reset offsets better when there is data loss
> due
> > > to
> > > > > > > unclean
> > > > > > > >> > > leader election or correlated replica failure.
> > > > > > > >> > > 3. It can reset offsets better when topic is recreated.
> > > > > > > >> > >
> > > > > > > >> > > Thanks,
> > > > > > > >> > >
> > > > > > > >> > > Jun
> > > > > > > >> > >
> > > > > > > >> > >
> > > > > > > >> > > On Thu, Jan 4, 2018 at 2:05 PM, Dong Lin <
> > > lindong28@gmail.com
> > > > >
> > > > > > > wrote:
> > > > > > > >> > >
> > > > > > > >> > > > Hey Jun,
> > > > > > > >> > > >
> > > > > > > >> > > > Yeah I agree that ideally we don't want an ever
> growing
> > > > global
> > > > > > > >> metadata
> > > > > > > >> > > > version. I just think it may be more desirable to keep
> > the
> > > > > > > consumer
> > > > > > > >> API
> > > > > > > >> > > > simple.
> > > > > > > >> > > >
> > > > > > > >> > > > In my current proposal, metadata version returned in
> the
> > > > fetch
> > > > > > > >> response
> > > > > > > >> > > > will be stored with the offset together. More
> > > specifically,
> > > > > the
> > > > > > > >> > > > metadata_epoch in the new offset topic schema will be
> > the
> > > > > > largest
> > > > > > > >> > > > metadata_epoch from all the MetadataResponse and
> > > > FetchResponse
> > > > > > > ever
> > > > > > > >> > > > received by this consumer.
> > > > > > > >> > > >
> > > > > > > >> > > > We probably don't have to change the consumer API for
> > > > > > > >> > > > commitSync(Map<TopicPartition, OffsetAndMetadata>). If
> > > user
> > > > > > calls
> > > > > > > >> > > > commitSync(...) to commit offset 10 for a given
> > partition,
> > > > for
> > > > > > > most
> > > > > > > >> > > > use-cases, this consumer instance should have consumed
> > > > message
> > > > > > > with
> > > > > > > >> > > offset
> > > > > > > >> > > > 9 from this partition, in which case the consumer can
> > > > remember
> > > > > > and
> > > > > > > >> use
> > > > > > > >> > > the
> > > > > > > >> > > > metadata_epoch from the corresponding FetchResponse
> when
> > > > > > > committing
> > > > > > > >> > > offset.
> > > > > > > >> > > > If user calls commitSync(..) to commit offset 10 for a
> > > given
> > > > > > > >> partition
> > > > > > > >> > > > without having consumed the message with offset 9
> using
> > > this
> > > > > > > >> consumer
> > > > > > > >> > > > instance, this is probably an advanced use-case. In
> this
> > > > case
> > > > > > the
> > > > > > > >> > > advanced
> > > > > > > >> > > > user can retrieve the metadata_epoch using the newly
> > added
> > > > > > > >> > > metadataEpoch()
> > > > > > > >> > > > API after it fetches the message with offset 9
> (probably
> > > > from
> > > > > > > >> another
> > > > > > > >> > > > consumer instance) and encode this metadata_epoch in
> the
> > > > > > > >> > > > string OffsetAndMetadata.metadata. Do you think this
> > > > solution
> > > > > > > would
> > > > > > > >> > work?
> > > > > > > >> > > >
> > > > > > > >> > > > By "not sure that I fully understand your latest
> > > > suggestion",
> > > > > > are
> > > > > > > >> you
> > > > > > > >> > > > referring to solution related to unclean leader
> election
> > > > using
> > > > > > > >> > > leader_epoch
> > > > > > > >> > > > in my previous email?
> > > > > > > >> > > >
> > > > > > > >> > > > Thanks,
> > > > > > > >> > > > Dong
> > > > > > > >> > > >
> > > > > > > >> > > > On Thu, Jan 4, 2018 at 1:33 PM, Jun Rao <
> > jun@confluent.io
> > > >
> > > > > > wrote:
> > > > > > > >> > > >
> > > > > > > >> > > > > Hi, Dong,
> > > > > > > >> > > > >
> > > > > > > >> > > > > Not sure that I fully understand your latest
> > suggestion.
> > > > > > > >> Returning an
> > > > > > > >> > > > ever
> > > > > > > >> > > > > growing global metadata version itself is no ideal,
> > but
> > > is
> > > > > > fine.
> > > > > > > >> My
> > > > > > > >> > > > > question is whether the metadata version returned in
> > the
> > > > > fetch
> > > > > > > >> > response
> > > > > > > >> > > > > needs to be stored with the offset together if
> offsets
> > > are
> > > > > > > stored
> > > > > > > >> > > > > externally. If so, we also have to change the
> consumer
> > > API
> > > > > for
> > > > > > > >> > > > commitSync()
> > > > > > > >> > > > > and need to worry about compatibility. If we don't
> > store
> > > > the
> > > > > > > >> metadata
> > > > > > > >> > > > > version together with the offset, on a consumer
> > restart,
> > > > > it's
> > > > > > > not
> > > > > > > >> > clear
> > > > > > > >> > > > how
> > > > > > > >> > > > > we can ensure the metadata in the consumer is high
> > > enough
> > > > > > since
> > > > > > > >> there
> > > > > > > >> > > is
> > > > > > > >> > > > no
> > > > > > > >> > > > > metadata version to compare with.
> > > > > > > >> > > > >
> > > > > > > >> > > > > Thanks,
> > > > > > > >> > > > >
> > > > > > > >> > > > > Jun
> > > > > > > >> > > > >
> > > > > > > >> > > > >
> > > > > > > >> > > > > On Wed, Jan 3, 2018 at 6:43 PM, Dong Lin <
> > > > > lindong28@gmail.com
> > > > > > >
> > > > > > > >> > wrote:
> > > > > > > >> > > > >
> > > > > > > >> > > > > > Hey Jun,
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > Thanks much for the explanation.
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > I understand the advantage of partition_epoch over
> > > > > > > >> metadata_epoch.
> > > > > > > >> > My
> > > > > > > >> > > > > > current concern is that the use of leader_epoch
> and
> > > the
> > > > > > > >> > > partition_epoch
> > > > > > > >> > > > > > requires us considerable change to consumer's
> public
> > > API
> > > > > to
> > > > > > > take
> > > > > > > >> > care
> > > > > > > >> > > > of
> > > > > > > >> > > > > > the case where user stores offset externally. For
> > > > example,
> > > > > > > >> > > *consumer*.
> > > > > > > >> > > > > > *commitSync*(..) would have to take a map whose
> > value
> > > is
> > > > > > > >> <offset,
> > > > > > > >> > > > > metadata,
> > > > > > > >> > > > > > leader epoch, partition epoch>.
> > *consumer*.*seek*(...)
> > > > > would
> > > > > > > >> also
> > > > > > > >> > > need
> > > > > > > >> > > > > > leader_epoch and partition_epoch as parameter.
> > > > Technically
> > > > > > we
> > > > > > > >> can
> > > > > > > >> > > > > probably
> > > > > > > >> > > > > > still make it work in a backward compatible manner
> > > after
> > > > > > > careful
> > > > > > > >> > > design
> > > > > > > >> > > > > and
> > > > > > > >> > > > > > discussion. But these changes can make the
> > consumer's
> > > > > > > interface
> > > > > > > >> > > > > > unnecessarily complex for more users who do not
> > store
> > > > > offset
> > > > > > > >> > > > externally.
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > After thinking more about it, we can address all
> > > > problems
> > > > > > > >> discussed
> > > > > > > >> > > by
> > > > > > > >> > > > > only
> > > > > > > >> > > > > > using the metadata_epoch without introducing
> > > > leader_epoch
> > > > > or
> > > > > > > the
> > > > > > > >> > > > > > partition_epoch. The current KIP describes the
> > changes
> > > > to
> > > > > > the
> > > > > > > >> > > consumer
> > > > > > > >> > > > > API
> > > > > > > >> > > > > > and how the new API can be used if user stores
> > offset
> > > > > > > >> externally.
> > > > > > > >> > In
> > > > > > > >> > > > > order
> > > > > > > >> > > > > > to address the scenario you described earlier, we
> > can
> > > > > > include
> > > > > > > >> > > > > > metadata_epoch in the FetchResponse and the
> > > > > > > LeaderAndIsrRequest.
> > > > > > > >> > > > Consumer
> > > > > > > >> > > > > > remembers the largest metadata_epoch from all the
> > > > > > > FetchResponse
> > > > > > > >> it
> > > > > > > >> > > has
> > > > > > > >> > > > > > received. The metadata_epoch committed with the
> > > offset,
> > > > > > either
> > > > > > > >> > within
> > > > > > > >> > > > or
> > > > > > > >> > > > > > outside Kafka, should be the largest
> metadata_epoch
> > > > across
> > > > > > all
> > > > > > > >> > > > > > FetchResponse and MetadataResponse ever received
> by
> > > this
> > > > > > > >> consumer.
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > The drawback of using only the metadata_epoch is
> > that
> > > we
> > > > > can
> > > > > > > not
> > > > > > > >> > > always
> > > > > > > >> > > > > do
> > > > > > > >> > > > > > the smart offset reset in case of unclean leader
> > > > election
> > > > > > > which
> > > > > > > >> you
> > > > > > > >> > > > > > mentioned earlier. But in most case, unclean
> leader
> > > > > election
> > > > > > > >> > probably
> > > > > > > >> > > > > > happens when consumer is not
> rebalancing/restarting.
> > > In
> > > > > > these
> > > > > > > >> > cases,
> > > > > > > >> > > > > either
> > > > > > > >> > > > > > consumer is not directly affected by unclean
> leader
> > > > > election
> > > > > > > >> since
> > > > > > > >> > it
> > > > > > > >> > > > is
> > > > > > > >> > > > > > not consuming from the end of the log, or consumer
> > can
> > > > > > derive
> > > > > > > >> the
> > > > > > > >> > > > > > leader_epoch from the most recent message received
> > > > before
> > > > > it
> > > > > > > >> sees
> > > > > > > >> > > > > > OffsetOutOfRangeException. So I am not sure it is
> > > worth
> > > > > > adding
> > > > > > > >> the
> > > > > > > >> > > > > > leader_epoch to consumer API to address the
> > remaining
> > > > > corner
> > > > > > > >> case.
> > > > > > > >> > > What
> > > > > > > >> > > > > do
> > > > > > > >> > > > > > you think?
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > Thanks,
> > > > > > > >> > > > > > Dong
> > > > > > > >> > > > > >
> > > > > > > >> > > > > >
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > On Tue, Jan 2, 2018 at 6:28 PM, Jun Rao <
> > > > jun@confluent.io
> > > > > >
> > > > > > > >> wrote:
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > > Hi, Dong,
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > Thanks for the reply.
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > To solve the topic recreation issue, we could
> use
> > > > > either a
> > > > > > > >> global
> > > > > > > >> > > > > > metadata
> > > > > > > >> > > > > > > version or a partition level epoch. But either
> one
> > > > will
> > > > > > be a
> > > > > > > >> new
> > > > > > > >> > > > > concept,
> > > > > > > >> > > > > > > right? To me, the latter seems more natural. It
> > also
> > > > > makes
> > > > > > > it
> > > > > > > >> > > easier
> > > > > > > >> > > > to
> > > > > > > >> > > > > > > detect if a consumer's offset is still valid
> > after a
> > > > > topic
> > > > > > > is
> > > > > > > >> > > > > recreated.
> > > > > > > >> > > > > > As
> > > > > > > >> > > > > > > you pointed out, we don't need to store the
> > > partition
> > > > > > epoch
> > > > > > > in
> > > > > > > >> > the
> > > > > > > >> > > > > > message.
> > > > > > > >> > > > > > > The following is what I am thinking. When a
> > > partition
> > > > is
> > > > > > > >> created,
> > > > > > > >> > > we
> > > > > > > >> > > > > can
> > > > > > > >> > > > > > > assign a partition epoch from an ever-increasing
> > > > global
> > > > > > > >> counter
> > > > > > > >> > and
> > > > > > > >> > > > > store
> > > > > > > >> > > > > > > it in /brokers/topics/[topic]/
> > > > partitions/[partitionId]
> > > > > in
> > > > > > > ZK.
> > > > > > > >> > The
> > > > > > > >> > > > > > > partition
> > > > > > > >> > > > > > > epoch is propagated to every broker. The
> consumer
> > > will
> > > > > be
> > > > > > > >> > tracking
> > > > > > > >> > > a
> > > > > > > >> > > > > > tuple
> > > > > > > >> > > > > > > of <offset, leader epoch, partition epoch> for
> > > > offsets.
> > > > > > If a
> > > > > > > >> > topic
> > > > > > > >> > > is
> > > > > > > >> > > > > > > recreated, it's possible that a consumer's
> offset
> > > and
> > > > > > leader
> > > > > > > >> > epoch
> > > > > > > >> > > > > still
> > > > > > > >> > > > > > > match that in the broker, but partition epoch
> > won't
> > > > be.
> > > > > In
> > > > > > > >> this
> > > > > > > >> > > case,
> > > > > > > >> > > > > we
> > > > > > > >> > > > > > > can potentially still treat the consumer's
> offset
> > as
> > > > out
> > > > > > of
> > > > > > > >> range
> > > > > > > >> > > and
> > > > > > > >> > > > > > reset
> > > > > > > >> > > > > > > the offset based on the offset reset policy in
> the
> > > > > > consumer.
> > > > > > > >> This
> > > > > > > >> > > > seems
> > > > > > > >> > > > > > > harder to do with a global metadata version.
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > Jun
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > On Mon, Dec 25, 2017 at 6:56 AM, Dong Lin <
> > > > > > > >> lindong28@gmail.com>
> > > > > > > >> > > > wrote:
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > > Hey Jun,
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > This is a very good example. After thinking
> > > through
> > > > > this
> > > > > > > in
> > > > > > > >> > > > detail, I
> > > > > > > >> > > > > > > agree
> > > > > > > >> > > > > > > > that we need to commit offset with leader
> epoch
> > in
> > > > > order
> > > > > > > to
> > > > > > > >> > > address
> > > > > > > >> > > > > > this
> > > > > > > >> > > > > > > > example.
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > I think the remaining question is how to
> address
> > > the
> > > > > > > >> scenario
> > > > > > > >> > > that
> > > > > > > >> > > > > the
> > > > > > > >> > > > > > > > topic is deleted and re-created. One possible
> > > > solution
> > > > > > is
> > > > > > > to
> > > > > > > >> > > commit
> > > > > > > >> > > > > > > offset
> > > > > > > >> > > > > > > > with both the leader epoch and the metadata
> > > version.
> > > > > The
> > > > > > > >> logic
> > > > > > > >> > > and
> > > > > > > >> > > > > the
> > > > > > > >> > > > > > > > implementation of this solution does not
> > require a
> > > > new
> > > > > > > >> concept
> > > > > > > >> > > > (e.g.
> > > > > > > >> > > > > > > > partition epoch) and it does not require any
> > > change
> > > > to
> > > > > > the
> > > > > > > >> > > message
> > > > > > > >> > > > > > format
> > > > > > > >> > > > > > > > or leader epoch. It also allows us to order
> the
> > > > > metadata
> > > > > > > in
> > > > > > > >> a
> > > > > > > >> > > > > > > > straightforward manner which may be useful in
> > the
> > > > > > future.
> > > > > > > >> So it
> > > > > > > >> > > may
> > > > > > > >> > > > > be
> > > > > > > >> > > > > > a
> > > > > > > >> > > > > > > > better solution than generating a random
> > partition
> > > > > epoch
> > > > > > > >> every
> > > > > > > >> > > time
> > > > > > > >> > > > > we
> > > > > > > >> > > > > > > > create a partition. Does this sound
> reasonable?
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > Previously one concern with using the metadata
> > > > version
> > > > > > is
> > > > > > > >> that
> > > > > > > >> > > > > consumer
> > > > > > > >> > > > > > > > will be forced to refresh metadata even if
> > > metadata
> > > > > > > version
> > > > > > > >> is
> > > > > > > >> > > > > > increased
> > > > > > > >> > > > > > > > due to topics that the consumer is not
> > interested
> > > > in.
> > > > > > Now
> > > > > > > I
> > > > > > > >> > > > realized
> > > > > > > >> > > > > > that
> > > > > > > >> > > > > > > > this is probably not a problem. Currently
> client
> > > > will
> > > > > > > >> refresh
> > > > > > > >> > > > > metadata
> > > > > > > >> > > > > > > > either due to InvalidMetadataException in the
> > > > response
> > > > > > > from
> > > > > > > >> > > broker
> > > > > > > >> > > > or
> > > > > > > >> > > > > > due
> > > > > > > >> > > > > > > > to metadata expiry. The addition of the
> metadata
> > > > > version
> > > > > > > >> should
> > > > > > > >> > > > > > increase
> > > > > > > >> > > > > > > > the overhead of metadata refresh caused by
> > > > > > > >> > > > InvalidMetadataException.
> > > > > > > >> > > > > If
> > > > > > > >> > > > > > > > client refresh metadata due to expiry and it
> > > > receives
> > > > > a
> > > > > > > >> > metadata
> > > > > > > >> > > > > whose
> > > > > > > >> > > > > > > > version is lower than the current metadata
> > > version,
> > > > we
> > > > > > can
> > > > > > > >> > reject
> > > > > > > >> > > > the
> > > > > > > >> > > > > > > > metadata but still reset the metadata age,
> which
> > > > > > > essentially
> > > > > > > >> > keep
> > > > > > > >> > > > the
> > > > > > > >> > > > > > > > existing behavior in the client.
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > Thanks much,
> > > > > > > >> > > > > > > > Dong
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > >
> > > > > > > >> > > > >
> > > > > > > >> > > >
> > > > > > > >> > >
> > > > > > > >> >
> > > > > > > >>
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Jun Rao <ju...@confluent.io>.
Hi, Dong,

Thanks for the updated KIP. A few more comments.

60. Perhaps having a partition epoch is more flexible since in the future,
we may support deleting a partition as well.

61. It seems that the leader epoch returned in the position() call should
the the leader epoch returned in the fetch response, not the one in the
metadata cache of the client.

62. I am wondering if we should return the partition epoch in the fetch
response as well. In the current proposal, if a topic is recreated and the
new leader is on the same broker as the old one, there is nothing to force
the metadata refresh in the client. So, the client may still associate the
offset with the old partition epoch.

63. There is some subtle coordination between the LeaderAndIsrRequest and
UpdateMetadataRequest. Currently, when a leader changes, the controller
first sends the LeaderAndIsrRequest to the assigned replicas and the
UpdateMetadataRequest to every broker. So, there could be a small window
when the leader already receives the new partition epoch in the
LeaderAndIsrRequest, but the metadata cache in the broker hasn't been
updated with the latest partition epoch. Not sure what's the best way to
address this issue. Perhaps we can update the metadata cache on the broker
with both LeaderAndIsrRequest and UpdateMetadataRequest. The challenge is
that the two have slightly different data. For example, only the latter has
all endpoints.

64. The enforcement of leader epoch in Offset commit: We allow a consumer
to set an arbitrary offset. So it's possible for offsets or leader epoch to
go backwards. I am not sure if we could always enforce that the leader
epoch only goes up on the broker.

65. Good point on handling missing partition epoch due to topic deletion.
Another potential way to address this is to additionally propagate the
global partition epoch to brokers and the clients. This way, when a
partition epoch is missing, we can use the global partition epoch to reason
about which metadata is more recent.

66. A client may also get an offset by time using the offsetForTimes() api.
So, we probably want to include offsetInternalMetadata in OffsetAndTimestamp
as well.

67. InteralMetadata can be a bit confusing with the metadata field already
there. Perhaps we can just call it OffsetEpoch. It might be useful to make
OffsetEpoch printable at least for debugging purpose. Once you do that, we
are already exposing the internal fields. So, not sure if it's worth hiding
them. If we do want to hide them, perhaps we can have sth like the
following. The binary encoding is probably more efficient than JSON for
external storage.

OffsetEpoch {
 static OffsetEpoch decode(byte[]);

  public byte[] encode();

  public String toString();
}

Jun

On Mon, Jan 8, 2018 at 4:22 PM, Dong Lin <li...@gmail.com> wrote:

> Hey Jason,
>
> Certainly. This sounds good. I have updated the KIP to clarity that the
> global epoch will be incremented by 1 each time a topic is deleted.
>
> Thanks,
> Dong
>
> On Mon, Jan 8, 2018 at 4:09 PM, Jason Gustafson <ja...@confluent.io>
> wrote:
>
> > Hi Dong,
> >
> >
> > I think your approach will allow user to distinguish between the metadata
> > > before and after the topic deletion. I also agree that this can be
> > > potentially be useful to user. I am just not very sure whether we
> already
> > > have a good use-case to make the additional complexity worthwhile. It
> > seems
> > > that this feature is kind of independent of the main problem of this
> KIP.
> > > Could we add this as a future work?
> >
> >
> > Do you think it's fair if we bump the topic epoch on deletion and leave
> > propagation of the epoch for deleted topics for future work? I don't
> think
> > this adds much complexity and it makes the behavior consistent: every
> topic
> > mutation results in an epoch bump.
> >
> > Thanks,
> > Jason
> >
> > On Mon, Jan 8, 2018 at 3:14 PM, Dong Lin <li...@gmail.com> wrote:
> >
> > > Hey Ismael,
> > >
> > > I guess we actually need user to see this field so that user can store
> > this
> > > value in the external store together with the offset. We just prefer
> the
> > > value to be opaque to discourage most users from interpreting this
> value.
> > > One more advantage of using such an opaque field is to be able to
> evolve
> > > the information (or schema) of this value without changing consumer API
> > in
> > > the future.
> > >
> > > I also thinking it is probably OK for user to be able to interpret this
> > > value, particularly for those advanced users.
> > >
> > > Thanks,
> > > Dong
> > >
> > > On Mon, Jan 8, 2018 at 2:34 PM, Ismael Juma <is...@juma.me.uk> wrote:
> > >
> > > > On Fri, Jan 5, 2018 at 7:15 PM, Jason Gustafson <ja...@confluent.io>
> > > > wrote:
> > > > >
> > > > > class OffsetAndMetadata {
> > > > >   long offset;
> > > > >   byte[] offsetMetadata;
> > > > >   String metadata;
> > > > > }
> > > >
> > > >
> > > > > Admittedly, the naming is a bit annoying, but we can probably come
> up
> > > > with
> > > > > something better. Internally the byte array would have a version.
> If
> > in
> > > > the
> > > > > future we have anything else we need to add, we can update the
> > version
> > > > and
> > > > > we wouldn't need any new APIs.
> > > > >
> > > >
> > > > We can also add fields to a class in a compatible way. So, it seems
> to
> > me
> > > > that the main advantage of the byte array is that it's opaque to the
> > > user.
> > > > Is that correct? If so, we could also add any opaque metadata in a
> > > subclass
> > > > so that users don't even see it (unless they cast it, but then
> they're
> > on
> > > > their own).
> > > >
> > > > Ismael
> > > >
> > > > The corresponding seek() and position() APIs might look something
> like
> > > > this:
> > > > >
> > > > > void seek(TopicPartition partition, long offset, byte[]
> > > offsetMetadata);
> > > > > byte[] positionMetadata(TopicPartition partition);
> > > > >
> > > > > What do you think?
> > > > >
> > > > > Thanks,
> > > > > Jason
> > > > >
> > > > > On Thu, Jan 4, 2018 at 7:04 PM, Dong Lin <li...@gmail.com>
> > wrote:
> > > > >
> > > > > > Hey Jun, Jason,
> > > > > >
> > > > > > Thanks much for all the feedback. I have updated the KIP based on
> > the
> > > > > > latest discussion. Can you help check whether it looks good?
> > > > > >
> > > > > > Thanks,
> > > > > > Dong
> > > > > >
> > > > > > On Thu, Jan 4, 2018 at 5:36 PM, Dong Lin <li...@gmail.com>
> > > wrote:
> > > > > >
> > > > > > > Hey Jun,
> > > > > > >
> > > > > > > Hmm... thinking about this more, I am not sure that the
> proposed
> > > API
> > > > is
> > > > > > > sufficient. For users that store offset externally, we probably
> > > need
> > > > > > extra
> > > > > > > API to return the leader_epoch and partition_epoch for all
> > > partitions
> > > > > > that
> > > > > > > consumers are consuming. I suppose these users currently use
> > > > position()
> > > > > > to
> > > > > > > get the offset. Thus we probably need a new method
> > > > > positionWithEpoch(..)
> > > > > > to
> > > > > > > return <offset, partition_epoch, leader_epoch>. Does this sound
> > > > > > reasonable?
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Dong
> > > > > > >
> > > > > > >
> > > > > > > On Thu, Jan 4, 2018 at 5:26 PM, Jun Rao <ju...@confluent.io>
> > wrote:
> > > > > > >
> > > > > > >> Hi, Dong,
> > > > > > >>
> > > > > > >> Yes, that's what I am thinking. OffsetEpoch will be composed
> of
> > > > > > >> (partition_epoch,
> > > > > > >> leader_epoch).
> > > > > > >>
> > > > > > >> Thanks,
> > > > > > >>
> > > > > > >> Jun
> > > > > > >>
> > > > > > >>
> > > > > > >> On Thu, Jan 4, 2018 at 4:22 PM, Dong Lin <lindong28@gmail.com
> >
> > > > wrote:
> > > > > > >>
> > > > > > >> > Hey Jun,
> > > > > > >> >
> > > > > > >> > Thanks much. I like the the new API that you proposed. I am
> > not
> > > > sure
> > > > > > >> what
> > > > > > >> > you exactly mean by offset_epoch. I suppose that we can use
> > the
> > > > pair
> > > > > > of
> > > > > > >> > (partition_epoch, leader_epoch) as the offset_epoch, right?
> > > > > > >> >
> > > > > > >> > Thanks,
> > > > > > >> > Dong
> > > > > > >> >
> > > > > > >> > On Thu, Jan 4, 2018 at 4:02 PM, Jun Rao <ju...@confluent.io>
> > > wrote:
> > > > > > >> >
> > > > > > >> > > Hi, Dong,
> > > > > > >> > >
> > > > > > >> > > Got it. The api that you proposed works. The question is
> > > whether
> > > > > > >> that's
> > > > > > >> > the
> > > > > > >> > > api that we want to have in the long term. My concern is
> > that
> > > > > while
> > > > > > >> the
> > > > > > >> > api
> > > > > > >> > > change is simple, the new api seems harder to explain and
> > use.
> > > > For
> > > > > > >> > example,
> > > > > > >> > > a consumer storing offsets externally now needs to call
> > > > > > >> > > waitForMetadataUpdate() after calling seek().
> > > > > > >> > >
> > > > > > >> > > An alternative approach is to make the following
> compatible
> > > api
> > > > > > >> changes
> > > > > > >> > in
> > > > > > >> > > Consumer.
> > > > > > >> > > * Add an additional OffsetEpoch field in
> OffsetAndMetadata.
> > > (no
> > > > > need
> > > > > > >> to
> > > > > > >> > > change the CommitSync() api)
> > > > > > >> > > * Add a new api seek(TopicPartition partition, long
> offset,
> > > > > > >> OffsetEpoch
> > > > > > >> > > offsetEpoch). We can potentially deprecate the old api
> > > > > > >> > seek(TopicPartition
> > > > > > >> > > partition, long offset) in the future.
> > > > > > >> > >
> > > > > > >> > > The alternative approach has similar amount of api changes
> > as
> > > > > yours
> > > > > > >> but
> > > > > > >> > has
> > > > > > >> > > the following benefits.
> > > > > > >> > > 1. The api works in a similar way as how offset management
> > > works
> > > > > now
> > > > > > >> and
> > > > > > >> > is
> > > > > > >> > > probably what we want in the long term.
> > > > > > >> > > 2. It can reset offsets better when there is data loss due
> > to
> > > > > > unclean
> > > > > > >> > > leader election or correlated replica failure.
> > > > > > >> > > 3. It can reset offsets better when topic is recreated.
> > > > > > >> > >
> > > > > > >> > > Thanks,
> > > > > > >> > >
> > > > > > >> > > Jun
> > > > > > >> > >
> > > > > > >> > >
> > > > > > >> > > On Thu, Jan 4, 2018 at 2:05 PM, Dong Lin <
> > lindong28@gmail.com
> > > >
> > > > > > wrote:
> > > > > > >> > >
> > > > > > >> > > > Hey Jun,
> > > > > > >> > > >
> > > > > > >> > > > Yeah I agree that ideally we don't want an ever growing
> > > global
> > > > > > >> metadata
> > > > > > >> > > > version. I just think it may be more desirable to keep
> the
> > > > > > consumer
> > > > > > >> API
> > > > > > >> > > > simple.
> > > > > > >> > > >
> > > > > > >> > > > In my current proposal, metadata version returned in the
> > > fetch
> > > > > > >> response
> > > > > > >> > > > will be stored with the offset together. More
> > specifically,
> > > > the
> > > > > > >> > > > metadata_epoch in the new offset topic schema will be
> the
> > > > > largest
> > > > > > >> > > > metadata_epoch from all the MetadataResponse and
> > > FetchResponse
> > > > > > ever
> > > > > > >> > > > received by this consumer.
> > > > > > >> > > >
> > > > > > >> > > > We probably don't have to change the consumer API for
> > > > > > >> > > > commitSync(Map<TopicPartition, OffsetAndMetadata>). If
> > user
> > > > > calls
> > > > > > >> > > > commitSync(...) to commit offset 10 for a given
> partition,
> > > for
> > > > > > most
> > > > > > >> > > > use-cases, this consumer instance should have consumed
> > > message
> > > > > > with
> > > > > > >> > > offset
> > > > > > >> > > > 9 from this partition, in which case the consumer can
> > > remember
> > > > > and
> > > > > > >> use
> > > > > > >> > > the
> > > > > > >> > > > metadata_epoch from the corresponding FetchResponse when
> > > > > > committing
> > > > > > >> > > offset.
> > > > > > >> > > > If user calls commitSync(..) to commit offset 10 for a
> > given
> > > > > > >> partition
> > > > > > >> > > > without having consumed the message with offset 9 using
> > this
> > > > > > >> consumer
> > > > > > >> > > > instance, this is probably an advanced use-case. In this
> > > case
> > > > > the
> > > > > > >> > > advanced
> > > > > > >> > > > user can retrieve the metadata_epoch using the newly
> added
> > > > > > >> > > metadataEpoch()
> > > > > > >> > > > API after it fetches the message with offset 9 (probably
> > > from
> > > > > > >> another
> > > > > > >> > > > consumer instance) and encode this metadata_epoch in the
> > > > > > >> > > > string OffsetAndMetadata.metadata. Do you think this
> > > solution
> > > > > > would
> > > > > > >> > work?
> > > > > > >> > > >
> > > > > > >> > > > By "not sure that I fully understand your latest
> > > suggestion",
> > > > > are
> > > > > > >> you
> > > > > > >> > > > referring to solution related to unclean leader election
> > > using
> > > > > > >> > > leader_epoch
> > > > > > >> > > > in my previous email?
> > > > > > >> > > >
> > > > > > >> > > > Thanks,
> > > > > > >> > > > Dong
> > > > > > >> > > >
> > > > > > >> > > > On Thu, Jan 4, 2018 at 1:33 PM, Jun Rao <
> jun@confluent.io
> > >
> > > > > wrote:
> > > > > > >> > > >
> > > > > > >> > > > > Hi, Dong,
> > > > > > >> > > > >
> > > > > > >> > > > > Not sure that I fully understand your latest
> suggestion.
> > > > > > >> Returning an
> > > > > > >> > > > ever
> > > > > > >> > > > > growing global metadata version itself is no ideal,
> but
> > is
> > > > > fine.
> > > > > > >> My
> > > > > > >> > > > > question is whether the metadata version returned in
> the
> > > > fetch
> > > > > > >> > response
> > > > > > >> > > > > needs to be stored with the offset together if offsets
> > are
> > > > > > stored
> > > > > > >> > > > > externally. If so, we also have to change the consumer
> > API
> > > > for
> > > > > > >> > > > commitSync()
> > > > > > >> > > > > and need to worry about compatibility. If we don't
> store
> > > the
> > > > > > >> metadata
> > > > > > >> > > > > version together with the offset, on a consumer
> restart,
> > > > it's
> > > > > > not
> > > > > > >> > clear
> > > > > > >> > > > how
> > > > > > >> > > > > we can ensure the metadata in the consumer is high
> > enough
> > > > > since
> > > > > > >> there
> > > > > > >> > > is
> > > > > > >> > > > no
> > > > > > >> > > > > metadata version to compare with.
> > > > > > >> > > > >
> > > > > > >> > > > > Thanks,
> > > > > > >> > > > >
> > > > > > >> > > > > Jun
> > > > > > >> > > > >
> > > > > > >> > > > >
> > > > > > >> > > > > On Wed, Jan 3, 2018 at 6:43 PM, Dong Lin <
> > > > lindong28@gmail.com
> > > > > >
> > > > > > >> > wrote:
> > > > > > >> > > > >
> > > > > > >> > > > > > Hey Jun,
> > > > > > >> > > > > >
> > > > > > >> > > > > > Thanks much for the explanation.
> > > > > > >> > > > > >
> > > > > > >> > > > > > I understand the advantage of partition_epoch over
> > > > > > >> metadata_epoch.
> > > > > > >> > My
> > > > > > >> > > > > > current concern is that the use of leader_epoch and
> > the
> > > > > > >> > > partition_epoch
> > > > > > >> > > > > > requires us considerable change to consumer's public
> > API
> > > > to
> > > > > > take
> > > > > > >> > care
> > > > > > >> > > > of
> > > > > > >> > > > > > the case where user stores offset externally. For
> > > example,
> > > > > > >> > > *consumer*.
> > > > > > >> > > > > > *commitSync*(..) would have to take a map whose
> value
> > is
> > > > > > >> <offset,
> > > > > > >> > > > > metadata,
> > > > > > >> > > > > > leader epoch, partition epoch>.
> *consumer*.*seek*(...)
> > > > would
> > > > > > >> also
> > > > > > >> > > need
> > > > > > >> > > > > > leader_epoch and partition_epoch as parameter.
> > > Technically
> > > > > we
> > > > > > >> can
> > > > > > >> > > > > probably
> > > > > > >> > > > > > still make it work in a backward compatible manner
> > after
> > > > > > careful
> > > > > > >> > > design
> > > > > > >> > > > > and
> > > > > > >> > > > > > discussion. But these changes can make the
> consumer's
> > > > > > interface
> > > > > > >> > > > > > unnecessarily complex for more users who do not
> store
> > > > offset
> > > > > > >> > > > externally.
> > > > > > >> > > > > >
> > > > > > >> > > > > > After thinking more about it, we can address all
> > > problems
> > > > > > >> discussed
> > > > > > >> > > by
> > > > > > >> > > > > only
> > > > > > >> > > > > > using the metadata_epoch without introducing
> > > leader_epoch
> > > > or
> > > > > > the
> > > > > > >> > > > > > partition_epoch. The current KIP describes the
> changes
> > > to
> > > > > the
> > > > > > >> > > consumer
> > > > > > >> > > > > API
> > > > > > >> > > > > > and how the new API can be used if user stores
> offset
> > > > > > >> externally.
> > > > > > >> > In
> > > > > > >> > > > > order
> > > > > > >> > > > > > to address the scenario you described earlier, we
> can
> > > > > include
> > > > > > >> > > > > > metadata_epoch in the FetchResponse and the
> > > > > > LeaderAndIsrRequest.
> > > > > > >> > > > Consumer
> > > > > > >> > > > > > remembers the largest metadata_epoch from all the
> > > > > > FetchResponse
> > > > > > >> it
> > > > > > >> > > has
> > > > > > >> > > > > > received. The metadata_epoch committed with the
> > offset,
> > > > > either
> > > > > > >> > within
> > > > > > >> > > > or
> > > > > > >> > > > > > outside Kafka, should be the largest metadata_epoch
> > > across
> > > > > all
> > > > > > >> > > > > > FetchResponse and MetadataResponse ever received by
> > this
> > > > > > >> consumer.
> > > > > > >> > > > > >
> > > > > > >> > > > > > The drawback of using only the metadata_epoch is
> that
> > we
> > > > can
> > > > > > not
> > > > > > >> > > always
> > > > > > >> > > > > do
> > > > > > >> > > > > > the smart offset reset in case of unclean leader
> > > election
> > > > > > which
> > > > > > >> you
> > > > > > >> > > > > > mentioned earlier. But in most case, unclean leader
> > > > election
> > > > > > >> > probably
> > > > > > >> > > > > > happens when consumer is not rebalancing/restarting.
> > In
> > > > > these
> > > > > > >> > cases,
> > > > > > >> > > > > either
> > > > > > >> > > > > > consumer is not directly affected by unclean leader
> > > > election
> > > > > > >> since
> > > > > > >> > it
> > > > > > >> > > > is
> > > > > > >> > > > > > not consuming from the end of the log, or consumer
> can
> > > > > derive
> > > > > > >> the
> > > > > > >> > > > > > leader_epoch from the most recent message received
> > > before
> > > > it
> > > > > > >> sees
> > > > > > >> > > > > > OffsetOutOfRangeException. So I am not sure it is
> > worth
> > > > > adding
> > > > > > >> the
> > > > > > >> > > > > > leader_epoch to consumer API to address the
> remaining
> > > > corner
> > > > > > >> case.
> > > > > > >> > > What
> > > > > > >> > > > > do
> > > > > > >> > > > > > you think?
> > > > > > >> > > > > >
> > > > > > >> > > > > > Thanks,
> > > > > > >> > > > > > Dong
> > > > > > >> > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > > > On Tue, Jan 2, 2018 at 6:28 PM, Jun Rao <
> > > jun@confluent.io
> > > > >
> > > > > > >> wrote:
> > > > > > >> > > > > >
> > > > > > >> > > > > > > Hi, Dong,
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > Thanks for the reply.
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > To solve the topic recreation issue, we could use
> > > > either a
> > > > > > >> global
> > > > > > >> > > > > > metadata
> > > > > > >> > > > > > > version or a partition level epoch. But either one
> > > will
> > > > > be a
> > > > > > >> new
> > > > > > >> > > > > concept,
> > > > > > >> > > > > > > right? To me, the latter seems more natural. It
> also
> > > > makes
> > > > > > it
> > > > > > >> > > easier
> > > > > > >> > > > to
> > > > > > >> > > > > > > detect if a consumer's offset is still valid
> after a
> > > > topic
> > > > > > is
> > > > > > >> > > > > recreated.
> > > > > > >> > > > > > As
> > > > > > >> > > > > > > you pointed out, we don't need to store the
> > partition
> > > > > epoch
> > > > > > in
> > > > > > >> > the
> > > > > > >> > > > > > message.
> > > > > > >> > > > > > > The following is what I am thinking. When a
> > partition
> > > is
> > > > > > >> created,
> > > > > > >> > > we
> > > > > > >> > > > > can
> > > > > > >> > > > > > > assign a partition epoch from an ever-increasing
> > > global
> > > > > > >> counter
> > > > > > >> > and
> > > > > > >> > > > > store
> > > > > > >> > > > > > > it in /brokers/topics/[topic]/
> > > partitions/[partitionId]
> > > > in
> > > > > > ZK.
> > > > > > >> > The
> > > > > > >> > > > > > > partition
> > > > > > >> > > > > > > epoch is propagated to every broker. The consumer
> > will
> > > > be
> > > > > > >> > tracking
> > > > > > >> > > a
> > > > > > >> > > > > > tuple
> > > > > > >> > > > > > > of <offset, leader epoch, partition epoch> for
> > > offsets.
> > > > > If a
> > > > > > >> > topic
> > > > > > >> > > is
> > > > > > >> > > > > > > recreated, it's possible that a consumer's offset
> > and
> > > > > leader
> > > > > > >> > epoch
> > > > > > >> > > > > still
> > > > > > >> > > > > > > match that in the broker, but partition epoch
> won't
> > > be.
> > > > In
> > > > > > >> this
> > > > > > >> > > case,
> > > > > > >> > > > > we
> > > > > > >> > > > > > > can potentially still treat the consumer's offset
> as
> > > out
> > > > > of
> > > > > > >> range
> > > > > > >> > > and
> > > > > > >> > > > > > reset
> > > > > > >> > > > > > > the offset based on the offset reset policy in the
> > > > > consumer.
> > > > > > >> This
> > > > > > >> > > > seems
> > > > > > >> > > > > > > harder to do with a global metadata version.
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > Jun
> > > > > > >> > > > > > >
> > > > > > >> > > > > > >
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > On Mon, Dec 25, 2017 at 6:56 AM, Dong Lin <
> > > > > > >> lindong28@gmail.com>
> > > > > > >> > > > wrote:
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > > Hey Jun,
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > This is a very good example. After thinking
> > through
> > > > this
> > > > > > in
> > > > > > >> > > > detail, I
> > > > > > >> > > > > > > agree
> > > > > > >> > > > > > > > that we need to commit offset with leader epoch
> in
> > > > order
> > > > > > to
> > > > > > >> > > address
> > > > > > >> > > > > > this
> > > > > > >> > > > > > > > example.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > I think the remaining question is how to address
> > the
> > > > > > >> scenario
> > > > > > >> > > that
> > > > > > >> > > > > the
> > > > > > >> > > > > > > > topic is deleted and re-created. One possible
> > > solution
> > > > > is
> > > > > > to
> > > > > > >> > > commit
> > > > > > >> > > > > > > offset
> > > > > > >> > > > > > > > with both the leader epoch and the metadata
> > version.
> > > > The
> > > > > > >> logic
> > > > > > >> > > and
> > > > > > >> > > > > the
> > > > > > >> > > > > > > > implementation of this solution does not
> require a
> > > new
> > > > > > >> concept
> > > > > > >> > > > (e.g.
> > > > > > >> > > > > > > > partition epoch) and it does not require any
> > change
> > > to
> > > > > the
> > > > > > >> > > message
> > > > > > >> > > > > > format
> > > > > > >> > > > > > > > or leader epoch. It also allows us to order the
> > > > metadata
> > > > > > in
> > > > > > >> a
> > > > > > >> > > > > > > > straightforward manner which may be useful in
> the
> > > > > future.
> > > > > > >> So it
> > > > > > >> > > may
> > > > > > >> > > > > be
> > > > > > >> > > > > > a
> > > > > > >> > > > > > > > better solution than generating a random
> partition
> > > > epoch
> > > > > > >> every
> > > > > > >> > > time
> > > > > > >> > > > > we
> > > > > > >> > > > > > > > create a partition. Does this sound reasonable?
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > Previously one concern with using the metadata
> > > version
> > > > > is
> > > > > > >> that
> > > > > > >> > > > > consumer
> > > > > > >> > > > > > > > will be forced to refresh metadata even if
> > metadata
> > > > > > version
> > > > > > >> is
> > > > > > >> > > > > > increased
> > > > > > >> > > > > > > > due to topics that the consumer is not
> interested
> > > in.
> > > > > Now
> > > > > > I
> > > > > > >> > > > realized
> > > > > > >> > > > > > that
> > > > > > >> > > > > > > > this is probably not a problem. Currently client
> > > will
> > > > > > >> refresh
> > > > > > >> > > > > metadata
> > > > > > >> > > > > > > > either due to InvalidMetadataException in the
> > > response
> > > > > > from
> > > > > > >> > > broker
> > > > > > >> > > > or
> > > > > > >> > > > > > due
> > > > > > >> > > > > > > > to metadata expiry. The addition of the metadata
> > > > version
> > > > > > >> should
> > > > > > >> > > > > > increase
> > > > > > >> > > > > > > > the overhead of metadata refresh caused by
> > > > > > >> > > > InvalidMetadataException.
> > > > > > >> > > > > If
> > > > > > >> > > > > > > > client refresh metadata due to expiry and it
> > > receives
> > > > a
> > > > > > >> > metadata
> > > > > > >> > > > > whose
> > > > > > >> > > > > > > > version is lower than the current metadata
> > version,
> > > we
> > > > > can
> > > > > > >> > reject
> > > > > > >> > > > the
> > > > > > >> > > > > > > > metadata but still reset the metadata age, which
> > > > > > essentially
> > > > > > >> > keep
> > > > > > >> > > > the
> > > > > > >> > > > > > > > existing behavior in the client.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > Thanks much,
> > > > > > >> > > > > > > > Dong
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > >
> > > > > > >> > > >
> > > > > > >> > >
> > > > > > >> >
> > > > > > >>
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Dong Lin <li...@gmail.com>.
Hey Jason,

Certainly. This sounds good. I have updated the KIP to clarity that the
global epoch will be incremented by 1 each time a topic is deleted.

Thanks,
Dong

On Mon, Jan 8, 2018 at 4:09 PM, Jason Gustafson <ja...@confluent.io> wrote:

> Hi Dong,
>
>
> I think your approach will allow user to distinguish between the metadata
> > before and after the topic deletion. I also agree that this can be
> > potentially be useful to user. I am just not very sure whether we already
> > have a good use-case to make the additional complexity worthwhile. It
> seems
> > that this feature is kind of independent of the main problem of this KIP.
> > Could we add this as a future work?
>
>
> Do you think it's fair if we bump the topic epoch on deletion and leave
> propagation of the epoch for deleted topics for future work? I don't think
> this adds much complexity and it makes the behavior consistent: every topic
> mutation results in an epoch bump.
>
> Thanks,
> Jason
>
> On Mon, Jan 8, 2018 at 3:14 PM, Dong Lin <li...@gmail.com> wrote:
>
> > Hey Ismael,
> >
> > I guess we actually need user to see this field so that user can store
> this
> > value in the external store together with the offset. We just prefer the
> > value to be opaque to discourage most users from interpreting this value.
> > One more advantage of using such an opaque field is to be able to evolve
> > the information (or schema) of this value without changing consumer API
> in
> > the future.
> >
> > I also thinking it is probably OK for user to be able to interpret this
> > value, particularly for those advanced users.
> >
> > Thanks,
> > Dong
> >
> > On Mon, Jan 8, 2018 at 2:34 PM, Ismael Juma <is...@juma.me.uk> wrote:
> >
> > > On Fri, Jan 5, 2018 at 7:15 PM, Jason Gustafson <ja...@confluent.io>
> > > wrote:
> > > >
> > > > class OffsetAndMetadata {
> > > >   long offset;
> > > >   byte[] offsetMetadata;
> > > >   String metadata;
> > > > }
> > >
> > >
> > > > Admittedly, the naming is a bit annoying, but we can probably come up
> > > with
> > > > something better. Internally the byte array would have a version. If
> in
> > > the
> > > > future we have anything else we need to add, we can update the
> version
> > > and
> > > > we wouldn't need any new APIs.
> > > >
> > >
> > > We can also add fields to a class in a compatible way. So, it seems to
> me
> > > that the main advantage of the byte array is that it's opaque to the
> > user.
> > > Is that correct? If so, we could also add any opaque metadata in a
> > subclass
> > > so that users don't even see it (unless they cast it, but then they're
> on
> > > their own).
> > >
> > > Ismael
> > >
> > > The corresponding seek() and position() APIs might look something like
> > > this:
> > > >
> > > > void seek(TopicPartition partition, long offset, byte[]
> > offsetMetadata);
> > > > byte[] positionMetadata(TopicPartition partition);
> > > >
> > > > What do you think?
> > > >
> > > > Thanks,
> > > > Jason
> > > >
> > > > On Thu, Jan 4, 2018 at 7:04 PM, Dong Lin <li...@gmail.com>
> wrote:
> > > >
> > > > > Hey Jun, Jason,
> > > > >
> > > > > Thanks much for all the feedback. I have updated the KIP based on
> the
> > > > > latest discussion. Can you help check whether it looks good?
> > > > >
> > > > > Thanks,
> > > > > Dong
> > > > >
> > > > > On Thu, Jan 4, 2018 at 5:36 PM, Dong Lin <li...@gmail.com>
> > wrote:
> > > > >
> > > > > > Hey Jun,
> > > > > >
> > > > > > Hmm... thinking about this more, I am not sure that the proposed
> > API
> > > is
> > > > > > sufficient. For users that store offset externally, we probably
> > need
> > > > > extra
> > > > > > API to return the leader_epoch and partition_epoch for all
> > partitions
> > > > > that
> > > > > > consumers are consuming. I suppose these users currently use
> > > position()
> > > > > to
> > > > > > get the offset. Thus we probably need a new method
> > > > positionWithEpoch(..)
> > > > > to
> > > > > > return <offset, partition_epoch, leader_epoch>. Does this sound
> > > > > reasonable?
> > > > > >
> > > > > > Thanks,
> > > > > > Dong
> > > > > >
> > > > > >
> > > > > > On Thu, Jan 4, 2018 at 5:26 PM, Jun Rao <ju...@confluent.io>
> wrote:
> > > > > >
> > > > > >> Hi, Dong,
> > > > > >>
> > > > > >> Yes, that's what I am thinking. OffsetEpoch will be composed of
> > > > > >> (partition_epoch,
> > > > > >> leader_epoch).
> > > > > >>
> > > > > >> Thanks,
> > > > > >>
> > > > > >> Jun
> > > > > >>
> > > > > >>
> > > > > >> On Thu, Jan 4, 2018 at 4:22 PM, Dong Lin <li...@gmail.com>
> > > wrote:
> > > > > >>
> > > > > >> > Hey Jun,
> > > > > >> >
> > > > > >> > Thanks much. I like the the new API that you proposed. I am
> not
> > > sure
> > > > > >> what
> > > > > >> > you exactly mean by offset_epoch. I suppose that we can use
> the
> > > pair
> > > > > of
> > > > > >> > (partition_epoch, leader_epoch) as the offset_epoch, right?
> > > > > >> >
> > > > > >> > Thanks,
> > > > > >> > Dong
> > > > > >> >
> > > > > >> > On Thu, Jan 4, 2018 at 4:02 PM, Jun Rao <ju...@confluent.io>
> > wrote:
> > > > > >> >
> > > > > >> > > Hi, Dong,
> > > > > >> > >
> > > > > >> > > Got it. The api that you proposed works. The question is
> > whether
> > > > > >> that's
> > > > > >> > the
> > > > > >> > > api that we want to have in the long term. My concern is
> that
> > > > while
> > > > > >> the
> > > > > >> > api
> > > > > >> > > change is simple, the new api seems harder to explain and
> use.
> > > For
> > > > > >> > example,
> > > > > >> > > a consumer storing offsets externally now needs to call
> > > > > >> > > waitForMetadataUpdate() after calling seek().
> > > > > >> > >
> > > > > >> > > An alternative approach is to make the following compatible
> > api
> > > > > >> changes
> > > > > >> > in
> > > > > >> > > Consumer.
> > > > > >> > > * Add an additional OffsetEpoch field in OffsetAndMetadata.
> > (no
> > > > need
> > > > > >> to
> > > > > >> > > change the CommitSync() api)
> > > > > >> > > * Add a new api seek(TopicPartition partition, long offset,
> > > > > >> OffsetEpoch
> > > > > >> > > offsetEpoch). We can potentially deprecate the old api
> > > > > >> > seek(TopicPartition
> > > > > >> > > partition, long offset) in the future.
> > > > > >> > >
> > > > > >> > > The alternative approach has similar amount of api changes
> as
> > > > yours
> > > > > >> but
> > > > > >> > has
> > > > > >> > > the following benefits.
> > > > > >> > > 1. The api works in a similar way as how offset management
> > works
> > > > now
> > > > > >> and
> > > > > >> > is
> > > > > >> > > probably what we want in the long term.
> > > > > >> > > 2. It can reset offsets better when there is data loss due
> to
> > > > > unclean
> > > > > >> > > leader election or correlated replica failure.
> > > > > >> > > 3. It can reset offsets better when topic is recreated.
> > > > > >> > >
> > > > > >> > > Thanks,
> > > > > >> > >
> > > > > >> > > Jun
> > > > > >> > >
> > > > > >> > >
> > > > > >> > > On Thu, Jan 4, 2018 at 2:05 PM, Dong Lin <
> lindong28@gmail.com
> > >
> > > > > wrote:
> > > > > >> > >
> > > > > >> > > > Hey Jun,
> > > > > >> > > >
> > > > > >> > > > Yeah I agree that ideally we don't want an ever growing
> > global
> > > > > >> metadata
> > > > > >> > > > version. I just think it may be more desirable to keep the
> > > > > consumer
> > > > > >> API
> > > > > >> > > > simple.
> > > > > >> > > >
> > > > > >> > > > In my current proposal, metadata version returned in the
> > fetch
> > > > > >> response
> > > > > >> > > > will be stored with the offset together. More
> specifically,
> > > the
> > > > > >> > > > metadata_epoch in the new offset topic schema will be the
> > > > largest
> > > > > >> > > > metadata_epoch from all the MetadataResponse and
> > FetchResponse
> > > > > ever
> > > > > >> > > > received by this consumer.
> > > > > >> > > >
> > > > > >> > > > We probably don't have to change the consumer API for
> > > > > >> > > > commitSync(Map<TopicPartition, OffsetAndMetadata>). If
> user
> > > > calls
> > > > > >> > > > commitSync(...) to commit offset 10 for a given partition,
> > for
> > > > > most
> > > > > >> > > > use-cases, this consumer instance should have consumed
> > message
> > > > > with
> > > > > >> > > offset
> > > > > >> > > > 9 from this partition, in which case the consumer can
> > remember
> > > > and
> > > > > >> use
> > > > > >> > > the
> > > > > >> > > > metadata_epoch from the corresponding FetchResponse when
> > > > > committing
> > > > > >> > > offset.
> > > > > >> > > > If user calls commitSync(..) to commit offset 10 for a
> given
> > > > > >> partition
> > > > > >> > > > without having consumed the message with offset 9 using
> this
> > > > > >> consumer
> > > > > >> > > > instance, this is probably an advanced use-case. In this
> > case
> > > > the
> > > > > >> > > advanced
> > > > > >> > > > user can retrieve the metadata_epoch using the newly added
> > > > > >> > > metadataEpoch()
> > > > > >> > > > API after it fetches the message with offset 9 (probably
> > from
> > > > > >> another
> > > > > >> > > > consumer instance) and encode this metadata_epoch in the
> > > > > >> > > > string OffsetAndMetadata.metadata. Do you think this
> > solution
> > > > > would
> > > > > >> > work?
> > > > > >> > > >
> > > > > >> > > > By "not sure that I fully understand your latest
> > suggestion",
> > > > are
> > > > > >> you
> > > > > >> > > > referring to solution related to unclean leader election
> > using
> > > > > >> > > leader_epoch
> > > > > >> > > > in my previous email?
> > > > > >> > > >
> > > > > >> > > > Thanks,
> > > > > >> > > > Dong
> > > > > >> > > >
> > > > > >> > > > On Thu, Jan 4, 2018 at 1:33 PM, Jun Rao <jun@confluent.io
> >
> > > > wrote:
> > > > > >> > > >
> > > > > >> > > > > Hi, Dong,
> > > > > >> > > > >
> > > > > >> > > > > Not sure that I fully understand your latest suggestion.
> > > > > >> Returning an
> > > > > >> > > > ever
> > > > > >> > > > > growing global metadata version itself is no ideal, but
> is
> > > > fine.
> > > > > >> My
> > > > > >> > > > > question is whether the metadata version returned in the
> > > fetch
> > > > > >> > response
> > > > > >> > > > > needs to be stored with the offset together if offsets
> are
> > > > > stored
> > > > > >> > > > > externally. If so, we also have to change the consumer
> API
> > > for
> > > > > >> > > > commitSync()
> > > > > >> > > > > and need to worry about compatibility. If we don't store
> > the
> > > > > >> metadata
> > > > > >> > > > > version together with the offset, on a consumer restart,
> > > it's
> > > > > not
> > > > > >> > clear
> > > > > >> > > > how
> > > > > >> > > > > we can ensure the metadata in the consumer is high
> enough
> > > > since
> > > > > >> there
> > > > > >> > > is
> > > > > >> > > > no
> > > > > >> > > > > metadata version to compare with.
> > > > > >> > > > >
> > > > > >> > > > > Thanks,
> > > > > >> > > > >
> > > > > >> > > > > Jun
> > > > > >> > > > >
> > > > > >> > > > >
> > > > > >> > > > > On Wed, Jan 3, 2018 at 6:43 PM, Dong Lin <
> > > lindong28@gmail.com
> > > > >
> > > > > >> > wrote:
> > > > > >> > > > >
> > > > > >> > > > > > Hey Jun,
> > > > > >> > > > > >
> > > > > >> > > > > > Thanks much for the explanation.
> > > > > >> > > > > >
> > > > > >> > > > > > I understand the advantage of partition_epoch over
> > > > > >> metadata_epoch.
> > > > > >> > My
> > > > > >> > > > > > current concern is that the use of leader_epoch and
> the
> > > > > >> > > partition_epoch
> > > > > >> > > > > > requires us considerable change to consumer's public
> API
> > > to
> > > > > take
> > > > > >> > care
> > > > > >> > > > of
> > > > > >> > > > > > the case where user stores offset externally. For
> > example,
> > > > > >> > > *consumer*.
> > > > > >> > > > > > *commitSync*(..) would have to take a map whose value
> is
> > > > > >> <offset,
> > > > > >> > > > > metadata,
> > > > > >> > > > > > leader epoch, partition epoch>. *consumer*.*seek*(...)
> > > would
> > > > > >> also
> > > > > >> > > need
> > > > > >> > > > > > leader_epoch and partition_epoch as parameter.
> > Technically
> > > > we
> > > > > >> can
> > > > > >> > > > > probably
> > > > > >> > > > > > still make it work in a backward compatible manner
> after
> > > > > careful
> > > > > >> > > design
> > > > > >> > > > > and
> > > > > >> > > > > > discussion. But these changes can make the consumer's
> > > > > interface
> > > > > >> > > > > > unnecessarily complex for more users who do not store
> > > offset
> > > > > >> > > > externally.
> > > > > >> > > > > >
> > > > > >> > > > > > After thinking more about it, we can address all
> > problems
> > > > > >> discussed
> > > > > >> > > by
> > > > > >> > > > > only
> > > > > >> > > > > > using the metadata_epoch without introducing
> > leader_epoch
> > > or
> > > > > the
> > > > > >> > > > > > partition_epoch. The current KIP describes the changes
> > to
> > > > the
> > > > > >> > > consumer
> > > > > >> > > > > API
> > > > > >> > > > > > and how the new API can be used if user stores offset
> > > > > >> externally.
> > > > > >> > In
> > > > > >> > > > > order
> > > > > >> > > > > > to address the scenario you described earlier, we can
> > > > include
> > > > > >> > > > > > metadata_epoch in the FetchResponse and the
> > > > > LeaderAndIsrRequest.
> > > > > >> > > > Consumer
> > > > > >> > > > > > remembers the largest metadata_epoch from all the
> > > > > FetchResponse
> > > > > >> it
> > > > > >> > > has
> > > > > >> > > > > > received. The metadata_epoch committed with the
> offset,
> > > > either
> > > > > >> > within
> > > > > >> > > > or
> > > > > >> > > > > > outside Kafka, should be the largest metadata_epoch
> > across
> > > > all
> > > > > >> > > > > > FetchResponse and MetadataResponse ever received by
> this
> > > > > >> consumer.
> > > > > >> > > > > >
> > > > > >> > > > > > The drawback of using only the metadata_epoch is that
> we
> > > can
> > > > > not
> > > > > >> > > always
> > > > > >> > > > > do
> > > > > >> > > > > > the smart offset reset in case of unclean leader
> > election
> > > > > which
> > > > > >> you
> > > > > >> > > > > > mentioned earlier. But in most case, unclean leader
> > > election
> > > > > >> > probably
> > > > > >> > > > > > happens when consumer is not rebalancing/restarting.
> In
> > > > these
> > > > > >> > cases,
> > > > > >> > > > > either
> > > > > >> > > > > > consumer is not directly affected by unclean leader
> > > election
> > > > > >> since
> > > > > >> > it
> > > > > >> > > > is
> > > > > >> > > > > > not consuming from the end of the log, or consumer can
> > > > derive
> > > > > >> the
> > > > > >> > > > > > leader_epoch from the most recent message received
> > before
> > > it
> > > > > >> sees
> > > > > >> > > > > > OffsetOutOfRangeException. So I am not sure it is
> worth
> > > > adding
> > > > > >> the
> > > > > >> > > > > > leader_epoch to consumer API to address the remaining
> > > corner
> > > > > >> case.
> > > > > >> > > What
> > > > > >> > > > > do
> > > > > >> > > > > > you think?
> > > > > >> > > > > >
> > > > > >> > > > > > Thanks,
> > > > > >> > > > > > Dong
> > > > > >> > > > > >
> > > > > >> > > > > >
> > > > > >> > > > > >
> > > > > >> > > > > > On Tue, Jan 2, 2018 at 6:28 PM, Jun Rao <
> > jun@confluent.io
> > > >
> > > > > >> wrote:
> > > > > >> > > > > >
> > > > > >> > > > > > > Hi, Dong,
> > > > > >> > > > > > >
> > > > > >> > > > > > > Thanks for the reply.
> > > > > >> > > > > > >
> > > > > >> > > > > > > To solve the topic recreation issue, we could use
> > > either a
> > > > > >> global
> > > > > >> > > > > > metadata
> > > > > >> > > > > > > version or a partition level epoch. But either one
> > will
> > > > be a
> > > > > >> new
> > > > > >> > > > > concept,
> > > > > >> > > > > > > right? To me, the latter seems more natural. It also
> > > makes
> > > > > it
> > > > > >> > > easier
> > > > > >> > > > to
> > > > > >> > > > > > > detect if a consumer's offset is still valid after a
> > > topic
> > > > > is
> > > > > >> > > > > recreated.
> > > > > >> > > > > > As
> > > > > >> > > > > > > you pointed out, we don't need to store the
> partition
> > > > epoch
> > > > > in
> > > > > >> > the
> > > > > >> > > > > > message.
> > > > > >> > > > > > > The following is what I am thinking. When a
> partition
> > is
> > > > > >> created,
> > > > > >> > > we
> > > > > >> > > > > can
> > > > > >> > > > > > > assign a partition epoch from an ever-increasing
> > global
> > > > > >> counter
> > > > > >> > and
> > > > > >> > > > > store
> > > > > >> > > > > > > it in /brokers/topics/[topic]/
> > partitions/[partitionId]
> > > in
> > > > > ZK.
> > > > > >> > The
> > > > > >> > > > > > > partition
> > > > > >> > > > > > > epoch is propagated to every broker. The consumer
> will
> > > be
> > > > > >> > tracking
> > > > > >> > > a
> > > > > >> > > > > > tuple
> > > > > >> > > > > > > of <offset, leader epoch, partition epoch> for
> > offsets.
> > > > If a
> > > > > >> > topic
> > > > > >> > > is
> > > > > >> > > > > > > recreated, it's possible that a consumer's offset
> and
> > > > leader
> > > > > >> > epoch
> > > > > >> > > > > still
> > > > > >> > > > > > > match that in the broker, but partition epoch won't
> > be.
> > > In
> > > > > >> this
> > > > > >> > > case,
> > > > > >> > > > > we
> > > > > >> > > > > > > can potentially still treat the consumer's offset as
> > out
> > > > of
> > > > > >> range
> > > > > >> > > and
> > > > > >> > > > > > reset
> > > > > >> > > > > > > the offset based on the offset reset policy in the
> > > > consumer.
> > > > > >> This
> > > > > >> > > > seems
> > > > > >> > > > > > > harder to do with a global metadata version.
> > > > > >> > > > > > >
> > > > > >> > > > > > > Jun
> > > > > >> > > > > > >
> > > > > >> > > > > > >
> > > > > >> > > > > > >
> > > > > >> > > > > > > On Mon, Dec 25, 2017 at 6:56 AM, Dong Lin <
> > > > > >> lindong28@gmail.com>
> > > > > >> > > > wrote:
> > > > > >> > > > > > >
> > > > > >> > > > > > > > Hey Jun,
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > This is a very good example. After thinking
> through
> > > this
> > > > > in
> > > > > >> > > > detail, I
> > > > > >> > > > > > > agree
> > > > > >> > > > > > > > that we need to commit offset with leader epoch in
> > > order
> > > > > to
> > > > > >> > > address
> > > > > >> > > > > > this
> > > > > >> > > > > > > > example.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > I think the remaining question is how to address
> the
> > > > > >> scenario
> > > > > >> > > that
> > > > > >> > > > > the
> > > > > >> > > > > > > > topic is deleted and re-created. One possible
> > solution
> > > > is
> > > > > to
> > > > > >> > > commit
> > > > > >> > > > > > > offset
> > > > > >> > > > > > > > with both the leader epoch and the metadata
> version.
> > > The
> > > > > >> logic
> > > > > >> > > and
> > > > > >> > > > > the
> > > > > >> > > > > > > > implementation of this solution does not require a
> > new
> > > > > >> concept
> > > > > >> > > > (e.g.
> > > > > >> > > > > > > > partition epoch) and it does not require any
> change
> > to
> > > > the
> > > > > >> > > message
> > > > > >> > > > > > format
> > > > > >> > > > > > > > or leader epoch. It also allows us to order the
> > > metadata
> > > > > in
> > > > > >> a
> > > > > >> > > > > > > > straightforward manner which may be useful in the
> > > > future.
> > > > > >> So it
> > > > > >> > > may
> > > > > >> > > > > be
> > > > > >> > > > > > a
> > > > > >> > > > > > > > better solution than generating a random partition
> > > epoch
> > > > > >> every
> > > > > >> > > time
> > > > > >> > > > > we
> > > > > >> > > > > > > > create a partition. Does this sound reasonable?
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > Previously one concern with using the metadata
> > version
> > > > is
> > > > > >> that
> > > > > >> > > > > consumer
> > > > > >> > > > > > > > will be forced to refresh metadata even if
> metadata
> > > > > version
> > > > > >> is
> > > > > >> > > > > > increased
> > > > > >> > > > > > > > due to topics that the consumer is not interested
> > in.
> > > > Now
> > > > > I
> > > > > >> > > > realized
> > > > > >> > > > > > that
> > > > > >> > > > > > > > this is probably not a problem. Currently client
> > will
> > > > > >> refresh
> > > > > >> > > > > metadata
> > > > > >> > > > > > > > either due to InvalidMetadataException in the
> > response
> > > > > from
> > > > > >> > > broker
> > > > > >> > > > or
> > > > > >> > > > > > due
> > > > > >> > > > > > > > to metadata expiry. The addition of the metadata
> > > version
> > > > > >> should
> > > > > >> > > > > > increase
> > > > > >> > > > > > > > the overhead of metadata refresh caused by
> > > > > >> > > > InvalidMetadataException.
> > > > > >> > > > > If
> > > > > >> > > > > > > > client refresh metadata due to expiry and it
> > receives
> > > a
> > > > > >> > metadata
> > > > > >> > > > > whose
> > > > > >> > > > > > > > version is lower than the current metadata
> version,
> > we
> > > > can
> > > > > >> > reject
> > > > > >> > > > the
> > > > > >> > > > > > > > metadata but still reset the metadata age, which
> > > > > essentially
> > > > > >> > keep
> > > > > >> > > > the
> > > > > >> > > > > > > > existing behavior in the client.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > Thanks much,
> > > > > >> > > > > > > > Dong
> > > > > >> > > > > > > >
> > > > > >> > > > > > >
> > > > > >> > > > > >
> > > > > >> > > > >
> > > > > >> > > >
> > > > > >> > >
> > > > > >> >
> > > > > >>
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Jason Gustafson <ja...@confluent.io>.
Hi Dong,


I think your approach will allow user to distinguish between the metadata
> before and after the topic deletion. I also agree that this can be
> potentially be useful to user. I am just not very sure whether we already
> have a good use-case to make the additional complexity worthwhile. It seems
> that this feature is kind of independent of the main problem of this KIP.
> Could we add this as a future work?


Do you think it's fair if we bump the topic epoch on deletion and leave
propagation of the epoch for deleted topics for future work? I don't think
this adds much complexity and it makes the behavior consistent: every topic
mutation results in an epoch bump.

Thanks,
Jason

On Mon, Jan 8, 2018 at 3:14 PM, Dong Lin <li...@gmail.com> wrote:

> Hey Ismael,
>
> I guess we actually need user to see this field so that user can store this
> value in the external store together with the offset. We just prefer the
> value to be opaque to discourage most users from interpreting this value.
> One more advantage of using such an opaque field is to be able to evolve
> the information (or schema) of this value without changing consumer API in
> the future.
>
> I also thinking it is probably OK for user to be able to interpret this
> value, particularly for those advanced users.
>
> Thanks,
> Dong
>
> On Mon, Jan 8, 2018 at 2:34 PM, Ismael Juma <is...@juma.me.uk> wrote:
>
> > On Fri, Jan 5, 2018 at 7:15 PM, Jason Gustafson <ja...@confluent.io>
> > wrote:
> > >
> > > class OffsetAndMetadata {
> > >   long offset;
> > >   byte[] offsetMetadata;
> > >   String metadata;
> > > }
> >
> >
> > > Admittedly, the naming is a bit annoying, but we can probably come up
> > with
> > > something better. Internally the byte array would have a version. If in
> > the
> > > future we have anything else we need to add, we can update the version
> > and
> > > we wouldn't need any new APIs.
> > >
> >
> > We can also add fields to a class in a compatible way. So, it seems to me
> > that the main advantage of the byte array is that it's opaque to the
> user.
> > Is that correct? If so, we could also add any opaque metadata in a
> subclass
> > so that users don't even see it (unless they cast it, but then they're on
> > their own).
> >
> > Ismael
> >
> > The corresponding seek() and position() APIs might look something like
> > this:
> > >
> > > void seek(TopicPartition partition, long offset, byte[]
> offsetMetadata);
> > > byte[] positionMetadata(TopicPartition partition);
> > >
> > > What do you think?
> > >
> > > Thanks,
> > > Jason
> > >
> > > On Thu, Jan 4, 2018 at 7:04 PM, Dong Lin <li...@gmail.com> wrote:
> > >
> > > > Hey Jun, Jason,
> > > >
> > > > Thanks much for all the feedback. I have updated the KIP based on the
> > > > latest discussion. Can you help check whether it looks good?
> > > >
> > > > Thanks,
> > > > Dong
> > > >
> > > > On Thu, Jan 4, 2018 at 5:36 PM, Dong Lin <li...@gmail.com>
> wrote:
> > > >
> > > > > Hey Jun,
> > > > >
> > > > > Hmm... thinking about this more, I am not sure that the proposed
> API
> > is
> > > > > sufficient. For users that store offset externally, we probably
> need
> > > > extra
> > > > > API to return the leader_epoch and partition_epoch for all
> partitions
> > > > that
> > > > > consumers are consuming. I suppose these users currently use
> > position()
> > > > to
> > > > > get the offset. Thus we probably need a new method
> > > positionWithEpoch(..)
> > > > to
> > > > > return <offset, partition_epoch, leader_epoch>. Does this sound
> > > > reasonable?
> > > > >
> > > > > Thanks,
> > > > > Dong
> > > > >
> > > > >
> > > > > On Thu, Jan 4, 2018 at 5:26 PM, Jun Rao <ju...@confluent.io> wrote:
> > > > >
> > > > >> Hi, Dong,
> > > > >>
> > > > >> Yes, that's what I am thinking. OffsetEpoch will be composed of
> > > > >> (partition_epoch,
> > > > >> leader_epoch).
> > > > >>
> > > > >> Thanks,
> > > > >>
> > > > >> Jun
> > > > >>
> > > > >>
> > > > >> On Thu, Jan 4, 2018 at 4:22 PM, Dong Lin <li...@gmail.com>
> > wrote:
> > > > >>
> > > > >> > Hey Jun,
> > > > >> >
> > > > >> > Thanks much. I like the the new API that you proposed. I am not
> > sure
> > > > >> what
> > > > >> > you exactly mean by offset_epoch. I suppose that we can use the
> > pair
> > > > of
> > > > >> > (partition_epoch, leader_epoch) as the offset_epoch, right?
> > > > >> >
> > > > >> > Thanks,
> > > > >> > Dong
> > > > >> >
> > > > >> > On Thu, Jan 4, 2018 at 4:02 PM, Jun Rao <ju...@confluent.io>
> wrote:
> > > > >> >
> > > > >> > > Hi, Dong,
> > > > >> > >
> > > > >> > > Got it. The api that you proposed works. The question is
> whether
> > > > >> that's
> > > > >> > the
> > > > >> > > api that we want to have in the long term. My concern is that
> > > while
> > > > >> the
> > > > >> > api
> > > > >> > > change is simple, the new api seems harder to explain and use.
> > For
> > > > >> > example,
> > > > >> > > a consumer storing offsets externally now needs to call
> > > > >> > > waitForMetadataUpdate() after calling seek().
> > > > >> > >
> > > > >> > > An alternative approach is to make the following compatible
> api
> > > > >> changes
> > > > >> > in
> > > > >> > > Consumer.
> > > > >> > > * Add an additional OffsetEpoch field in OffsetAndMetadata.
> (no
> > > need
> > > > >> to
> > > > >> > > change the CommitSync() api)
> > > > >> > > * Add a new api seek(TopicPartition partition, long offset,
> > > > >> OffsetEpoch
> > > > >> > > offsetEpoch). We can potentially deprecate the old api
> > > > >> > seek(TopicPartition
> > > > >> > > partition, long offset) in the future.
> > > > >> > >
> > > > >> > > The alternative approach has similar amount of api changes as
> > > yours
> > > > >> but
> > > > >> > has
> > > > >> > > the following benefits.
> > > > >> > > 1. The api works in a similar way as how offset management
> works
> > > now
> > > > >> and
> > > > >> > is
> > > > >> > > probably what we want in the long term.
> > > > >> > > 2. It can reset offsets better when there is data loss due to
> > > > unclean
> > > > >> > > leader election or correlated replica failure.
> > > > >> > > 3. It can reset offsets better when topic is recreated.
> > > > >> > >
> > > > >> > > Thanks,
> > > > >> > >
> > > > >> > > Jun
> > > > >> > >
> > > > >> > >
> > > > >> > > On Thu, Jan 4, 2018 at 2:05 PM, Dong Lin <lindong28@gmail.com
> >
> > > > wrote:
> > > > >> > >
> > > > >> > > > Hey Jun,
> > > > >> > > >
> > > > >> > > > Yeah I agree that ideally we don't want an ever growing
> global
> > > > >> metadata
> > > > >> > > > version. I just think it may be more desirable to keep the
> > > > consumer
> > > > >> API
> > > > >> > > > simple.
> > > > >> > > >
> > > > >> > > > In my current proposal, metadata version returned in the
> fetch
> > > > >> response
> > > > >> > > > will be stored with the offset together. More specifically,
> > the
> > > > >> > > > metadata_epoch in the new offset topic schema will be the
> > > largest
> > > > >> > > > metadata_epoch from all the MetadataResponse and
> FetchResponse
> > > > ever
> > > > >> > > > received by this consumer.
> > > > >> > > >
> > > > >> > > > We probably don't have to change the consumer API for
> > > > >> > > > commitSync(Map<TopicPartition, OffsetAndMetadata>). If user
> > > calls
> > > > >> > > > commitSync(...) to commit offset 10 for a given partition,
> for
> > > > most
> > > > >> > > > use-cases, this consumer instance should have consumed
> message
> > > > with
> > > > >> > > offset
> > > > >> > > > 9 from this partition, in which case the consumer can
> remember
> > > and
> > > > >> use
> > > > >> > > the
> > > > >> > > > metadata_epoch from the corresponding FetchResponse when
> > > > committing
> > > > >> > > offset.
> > > > >> > > > If user calls commitSync(..) to commit offset 10 for a given
> > > > >> partition
> > > > >> > > > without having consumed the message with offset 9 using this
> > > > >> consumer
> > > > >> > > > instance, this is probably an advanced use-case. In this
> case
> > > the
> > > > >> > > advanced
> > > > >> > > > user can retrieve the metadata_epoch using the newly added
> > > > >> > > metadataEpoch()
> > > > >> > > > API after it fetches the message with offset 9 (probably
> from
> > > > >> another
> > > > >> > > > consumer instance) and encode this metadata_epoch in the
> > > > >> > > > string OffsetAndMetadata.metadata. Do you think this
> solution
> > > > would
> > > > >> > work?
> > > > >> > > >
> > > > >> > > > By "not sure that I fully understand your latest
> suggestion",
> > > are
> > > > >> you
> > > > >> > > > referring to solution related to unclean leader election
> using
> > > > >> > > leader_epoch
> > > > >> > > > in my previous email?
> > > > >> > > >
> > > > >> > > > Thanks,
> > > > >> > > > Dong
> > > > >> > > >
> > > > >> > > > On Thu, Jan 4, 2018 at 1:33 PM, Jun Rao <ju...@confluent.io>
> > > wrote:
> > > > >> > > >
> > > > >> > > > > Hi, Dong,
> > > > >> > > > >
> > > > >> > > > > Not sure that I fully understand your latest suggestion.
> > > > >> Returning an
> > > > >> > > > ever
> > > > >> > > > > growing global metadata version itself is no ideal, but is
> > > fine.
> > > > >> My
> > > > >> > > > > question is whether the metadata version returned in the
> > fetch
> > > > >> > response
> > > > >> > > > > needs to be stored with the offset together if offsets are
> > > > stored
> > > > >> > > > > externally. If so, we also have to change the consumer API
> > for
> > > > >> > > > commitSync()
> > > > >> > > > > and need to worry about compatibility. If we don't store
> the
> > > > >> metadata
> > > > >> > > > > version together with the offset, on a consumer restart,
> > it's
> > > > not
> > > > >> > clear
> > > > >> > > > how
> > > > >> > > > > we can ensure the metadata in the consumer is high enough
> > > since
> > > > >> there
> > > > >> > > is
> > > > >> > > > no
> > > > >> > > > > metadata version to compare with.
> > > > >> > > > >
> > > > >> > > > > Thanks,
> > > > >> > > > >
> > > > >> > > > > Jun
> > > > >> > > > >
> > > > >> > > > >
> > > > >> > > > > On Wed, Jan 3, 2018 at 6:43 PM, Dong Lin <
> > lindong28@gmail.com
> > > >
> > > > >> > wrote:
> > > > >> > > > >
> > > > >> > > > > > Hey Jun,
> > > > >> > > > > >
> > > > >> > > > > > Thanks much for the explanation.
> > > > >> > > > > >
> > > > >> > > > > > I understand the advantage of partition_epoch over
> > > > >> metadata_epoch.
> > > > >> > My
> > > > >> > > > > > current concern is that the use of leader_epoch and the
> > > > >> > > partition_epoch
> > > > >> > > > > > requires us considerable change to consumer's public API
> > to
> > > > take
> > > > >> > care
> > > > >> > > > of
> > > > >> > > > > > the case where user stores offset externally. For
> example,
> > > > >> > > *consumer*.
> > > > >> > > > > > *commitSync*(..) would have to take a map whose value is
> > > > >> <offset,
> > > > >> > > > > metadata,
> > > > >> > > > > > leader epoch, partition epoch>. *consumer*.*seek*(...)
> > would
> > > > >> also
> > > > >> > > need
> > > > >> > > > > > leader_epoch and partition_epoch as parameter.
> Technically
> > > we
> > > > >> can
> > > > >> > > > > probably
> > > > >> > > > > > still make it work in a backward compatible manner after
> > > > careful
> > > > >> > > design
> > > > >> > > > > and
> > > > >> > > > > > discussion. But these changes can make the consumer's
> > > > interface
> > > > >> > > > > > unnecessarily complex for more users who do not store
> > offset
> > > > >> > > > externally.
> > > > >> > > > > >
> > > > >> > > > > > After thinking more about it, we can address all
> problems
> > > > >> discussed
> > > > >> > > by
> > > > >> > > > > only
> > > > >> > > > > > using the metadata_epoch without introducing
> leader_epoch
> > or
> > > > the
> > > > >> > > > > > partition_epoch. The current KIP describes the changes
> to
> > > the
> > > > >> > > consumer
> > > > >> > > > > API
> > > > >> > > > > > and how the new API can be used if user stores offset
> > > > >> externally.
> > > > >> > In
> > > > >> > > > > order
> > > > >> > > > > > to address the scenario you described earlier, we can
> > > include
> > > > >> > > > > > metadata_epoch in the FetchResponse and the
> > > > LeaderAndIsrRequest.
> > > > >> > > > Consumer
> > > > >> > > > > > remembers the largest metadata_epoch from all the
> > > > FetchResponse
> > > > >> it
> > > > >> > > has
> > > > >> > > > > > received. The metadata_epoch committed with the offset,
> > > either
> > > > >> > within
> > > > >> > > > or
> > > > >> > > > > > outside Kafka, should be the largest metadata_epoch
> across
> > > all
> > > > >> > > > > > FetchResponse and MetadataResponse ever received by this
> > > > >> consumer.
> > > > >> > > > > >
> > > > >> > > > > > The drawback of using only the metadata_epoch is that we
> > can
> > > > not
> > > > >> > > always
> > > > >> > > > > do
> > > > >> > > > > > the smart offset reset in case of unclean leader
> election
> > > > which
> > > > >> you
> > > > >> > > > > > mentioned earlier. But in most case, unclean leader
> > election
> > > > >> > probably
> > > > >> > > > > > happens when consumer is not rebalancing/restarting. In
> > > these
> > > > >> > cases,
> > > > >> > > > > either
> > > > >> > > > > > consumer is not directly affected by unclean leader
> > election
> > > > >> since
> > > > >> > it
> > > > >> > > > is
> > > > >> > > > > > not consuming from the end of the log, or consumer can
> > > derive
> > > > >> the
> > > > >> > > > > > leader_epoch from the most recent message received
> before
> > it
> > > > >> sees
> > > > >> > > > > > OffsetOutOfRangeException. So I am not sure it is worth
> > > adding
> > > > >> the
> > > > >> > > > > > leader_epoch to consumer API to address the remaining
> > corner
> > > > >> case.
> > > > >> > > What
> > > > >> > > > > do
> > > > >> > > > > > you think?
> > > > >> > > > > >
> > > > >> > > > > > Thanks,
> > > > >> > > > > > Dong
> > > > >> > > > > >
> > > > >> > > > > >
> > > > >> > > > > >
> > > > >> > > > > > On Tue, Jan 2, 2018 at 6:28 PM, Jun Rao <
> jun@confluent.io
> > >
> > > > >> wrote:
> > > > >> > > > > >
> > > > >> > > > > > > Hi, Dong,
> > > > >> > > > > > >
> > > > >> > > > > > > Thanks for the reply.
> > > > >> > > > > > >
> > > > >> > > > > > > To solve the topic recreation issue, we could use
> > either a
> > > > >> global
> > > > >> > > > > > metadata
> > > > >> > > > > > > version or a partition level epoch. But either one
> will
> > > be a
> > > > >> new
> > > > >> > > > > concept,
> > > > >> > > > > > > right? To me, the latter seems more natural. It also
> > makes
> > > > it
> > > > >> > > easier
> > > > >> > > > to
> > > > >> > > > > > > detect if a consumer's offset is still valid after a
> > topic
> > > > is
> > > > >> > > > > recreated.
> > > > >> > > > > > As
> > > > >> > > > > > > you pointed out, we don't need to store the partition
> > > epoch
> > > > in
> > > > >> > the
> > > > >> > > > > > message.
> > > > >> > > > > > > The following is what I am thinking. When a partition
> is
> > > > >> created,
> > > > >> > > we
> > > > >> > > > > can
> > > > >> > > > > > > assign a partition epoch from an ever-increasing
> global
> > > > >> counter
> > > > >> > and
> > > > >> > > > > store
> > > > >> > > > > > > it in /brokers/topics/[topic]/
> partitions/[partitionId]
> > in
> > > > ZK.
> > > > >> > The
> > > > >> > > > > > > partition
> > > > >> > > > > > > epoch is propagated to every broker. The consumer will
> > be
> > > > >> > tracking
> > > > >> > > a
> > > > >> > > > > > tuple
> > > > >> > > > > > > of <offset, leader epoch, partition epoch> for
> offsets.
> > > If a
> > > > >> > topic
> > > > >> > > is
> > > > >> > > > > > > recreated, it's possible that a consumer's offset and
> > > leader
> > > > >> > epoch
> > > > >> > > > > still
> > > > >> > > > > > > match that in the broker, but partition epoch won't
> be.
> > In
> > > > >> this
> > > > >> > > case,
> > > > >> > > > > we
> > > > >> > > > > > > can potentially still treat the consumer's offset as
> out
> > > of
> > > > >> range
> > > > >> > > and
> > > > >> > > > > > reset
> > > > >> > > > > > > the offset based on the offset reset policy in the
> > > consumer.
> > > > >> This
> > > > >> > > > seems
> > > > >> > > > > > > harder to do with a global metadata version.
> > > > >> > > > > > >
> > > > >> > > > > > > Jun
> > > > >> > > > > > >
> > > > >> > > > > > >
> > > > >> > > > > > >
> > > > >> > > > > > > On Mon, Dec 25, 2017 at 6:56 AM, Dong Lin <
> > > > >> lindong28@gmail.com>
> > > > >> > > > wrote:
> > > > >> > > > > > >
> > > > >> > > > > > > > Hey Jun,
> > > > >> > > > > > > >
> > > > >> > > > > > > > This is a very good example. After thinking through
> > this
> > > > in
> > > > >> > > > detail, I
> > > > >> > > > > > > agree
> > > > >> > > > > > > > that we need to commit offset with leader epoch in
> > order
> > > > to
> > > > >> > > address
> > > > >> > > > > > this
> > > > >> > > > > > > > example.
> > > > >> > > > > > > >
> > > > >> > > > > > > > I think the remaining question is how to address the
> > > > >> scenario
> > > > >> > > that
> > > > >> > > > > the
> > > > >> > > > > > > > topic is deleted and re-created. One possible
> solution
> > > is
> > > > to
> > > > >> > > commit
> > > > >> > > > > > > offset
> > > > >> > > > > > > > with both the leader epoch and the metadata version.
> > The
> > > > >> logic
> > > > >> > > and
> > > > >> > > > > the
> > > > >> > > > > > > > implementation of this solution does not require a
> new
> > > > >> concept
> > > > >> > > > (e.g.
> > > > >> > > > > > > > partition epoch) and it does not require any change
> to
> > > the
> > > > >> > > message
> > > > >> > > > > > format
> > > > >> > > > > > > > or leader epoch. It also allows us to order the
> > metadata
> > > > in
> > > > >> a
> > > > >> > > > > > > > straightforward manner which may be useful in the
> > > future.
> > > > >> So it
> > > > >> > > may
> > > > >> > > > > be
> > > > >> > > > > > a
> > > > >> > > > > > > > better solution than generating a random partition
> > epoch
> > > > >> every
> > > > >> > > time
> > > > >> > > > > we
> > > > >> > > > > > > > create a partition. Does this sound reasonable?
> > > > >> > > > > > > >
> > > > >> > > > > > > > Previously one concern with using the metadata
> version
> > > is
> > > > >> that
> > > > >> > > > > consumer
> > > > >> > > > > > > > will be forced to refresh metadata even if metadata
> > > > version
> > > > >> is
> > > > >> > > > > > increased
> > > > >> > > > > > > > due to topics that the consumer is not interested
> in.
> > > Now
> > > > I
> > > > >> > > > realized
> > > > >> > > > > > that
> > > > >> > > > > > > > this is probably not a problem. Currently client
> will
> > > > >> refresh
> > > > >> > > > > metadata
> > > > >> > > > > > > > either due to InvalidMetadataException in the
> response
> > > > from
> > > > >> > > broker
> > > > >> > > > or
> > > > >> > > > > > due
> > > > >> > > > > > > > to metadata expiry. The addition of the metadata
> > version
> > > > >> should
> > > > >> > > > > > increase
> > > > >> > > > > > > > the overhead of metadata refresh caused by
> > > > >> > > > InvalidMetadataException.
> > > > >> > > > > If
> > > > >> > > > > > > > client refresh metadata due to expiry and it
> receives
> > a
> > > > >> > metadata
> > > > >> > > > > whose
> > > > >> > > > > > > > version is lower than the current metadata version,
> we
> > > can
> > > > >> > reject
> > > > >> > > > the
> > > > >> > > > > > > > metadata but still reset the metadata age, which
> > > > essentially
> > > > >> > keep
> > > > >> > > > the
> > > > >> > > > > > > > existing behavior in the client.
> > > > >> > > > > > > >
> > > > >> > > > > > > > Thanks much,
> > > > >> > > > > > > > Dong
> > > > >> > > > > > > >
> > > > >> > > > > > >
> > > > >> > > > > >
> > > > >> > > > >
> > > > >> > > >
> > > > >> > >
> > > > >> >
> > > > >>
> > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Dong Lin <li...@gmail.com>.
Hey Ismael,

I guess we actually need user to see this field so that user can store this
value in the external store together with the offset. We just prefer the
value to be opaque to discourage most users from interpreting this value.
One more advantage of using such an opaque field is to be able to evolve
the information (or schema) of this value without changing consumer API in
the future.

I also thinking it is probably OK for user to be able to interpret this
value, particularly for those advanced users.

Thanks,
Dong

On Mon, Jan 8, 2018 at 2:34 PM, Ismael Juma <is...@juma.me.uk> wrote:

> On Fri, Jan 5, 2018 at 7:15 PM, Jason Gustafson <ja...@confluent.io>
> wrote:
> >
> > class OffsetAndMetadata {
> >   long offset;
> >   byte[] offsetMetadata;
> >   String metadata;
> > }
>
>
> > Admittedly, the naming is a bit annoying, but we can probably come up
> with
> > something better. Internally the byte array would have a version. If in
> the
> > future we have anything else we need to add, we can update the version
> and
> > we wouldn't need any new APIs.
> >
>
> We can also add fields to a class in a compatible way. So, it seems to me
> that the main advantage of the byte array is that it's opaque to the user.
> Is that correct? If so, we could also add any opaque metadata in a subclass
> so that users don't even see it (unless they cast it, but then they're on
> their own).
>
> Ismael
>
> The corresponding seek() and position() APIs might look something like
> this:
> >
> > void seek(TopicPartition partition, long offset, byte[] offsetMetadata);
> > byte[] positionMetadata(TopicPartition partition);
> >
> > What do you think?
> >
> > Thanks,
> > Jason
> >
> > On Thu, Jan 4, 2018 at 7:04 PM, Dong Lin <li...@gmail.com> wrote:
> >
> > > Hey Jun, Jason,
> > >
> > > Thanks much for all the feedback. I have updated the KIP based on the
> > > latest discussion. Can you help check whether it looks good?
> > >
> > > Thanks,
> > > Dong
> > >
> > > On Thu, Jan 4, 2018 at 5:36 PM, Dong Lin <li...@gmail.com> wrote:
> > >
> > > > Hey Jun,
> > > >
> > > > Hmm... thinking about this more, I am not sure that the proposed API
> is
> > > > sufficient. For users that store offset externally, we probably need
> > > extra
> > > > API to return the leader_epoch and partition_epoch for all partitions
> > > that
> > > > consumers are consuming. I suppose these users currently use
> position()
> > > to
> > > > get the offset. Thus we probably need a new method
> > positionWithEpoch(..)
> > > to
> > > > return <offset, partition_epoch, leader_epoch>. Does this sound
> > > reasonable?
> > > >
> > > > Thanks,
> > > > Dong
> > > >
> > > >
> > > > On Thu, Jan 4, 2018 at 5:26 PM, Jun Rao <ju...@confluent.io> wrote:
> > > >
> > > >> Hi, Dong,
> > > >>
> > > >> Yes, that's what I am thinking. OffsetEpoch will be composed of
> > > >> (partition_epoch,
> > > >> leader_epoch).
> > > >>
> > > >> Thanks,
> > > >>
> > > >> Jun
> > > >>
> > > >>
> > > >> On Thu, Jan 4, 2018 at 4:22 PM, Dong Lin <li...@gmail.com>
> wrote:
> > > >>
> > > >> > Hey Jun,
> > > >> >
> > > >> > Thanks much. I like the the new API that you proposed. I am not
> sure
> > > >> what
> > > >> > you exactly mean by offset_epoch. I suppose that we can use the
> pair
> > > of
> > > >> > (partition_epoch, leader_epoch) as the offset_epoch, right?
> > > >> >
> > > >> > Thanks,
> > > >> > Dong
> > > >> >
> > > >> > On Thu, Jan 4, 2018 at 4:02 PM, Jun Rao <ju...@confluent.io> wrote:
> > > >> >
> > > >> > > Hi, Dong,
> > > >> > >
> > > >> > > Got it. The api that you proposed works. The question is whether
> > > >> that's
> > > >> > the
> > > >> > > api that we want to have in the long term. My concern is that
> > while
> > > >> the
> > > >> > api
> > > >> > > change is simple, the new api seems harder to explain and use.
> For
> > > >> > example,
> > > >> > > a consumer storing offsets externally now needs to call
> > > >> > > waitForMetadataUpdate() after calling seek().
> > > >> > >
> > > >> > > An alternative approach is to make the following compatible api
> > > >> changes
> > > >> > in
> > > >> > > Consumer.
> > > >> > > * Add an additional OffsetEpoch field in OffsetAndMetadata. (no
> > need
> > > >> to
> > > >> > > change the CommitSync() api)
> > > >> > > * Add a new api seek(TopicPartition partition, long offset,
> > > >> OffsetEpoch
> > > >> > > offsetEpoch). We can potentially deprecate the old api
> > > >> > seek(TopicPartition
> > > >> > > partition, long offset) in the future.
> > > >> > >
> > > >> > > The alternative approach has similar amount of api changes as
> > yours
> > > >> but
> > > >> > has
> > > >> > > the following benefits.
> > > >> > > 1. The api works in a similar way as how offset management works
> > now
> > > >> and
> > > >> > is
> > > >> > > probably what we want in the long term.
> > > >> > > 2. It can reset offsets better when there is data loss due to
> > > unclean
> > > >> > > leader election or correlated replica failure.
> > > >> > > 3. It can reset offsets better when topic is recreated.
> > > >> > >
> > > >> > > Thanks,
> > > >> > >
> > > >> > > Jun
> > > >> > >
> > > >> > >
> > > >> > > On Thu, Jan 4, 2018 at 2:05 PM, Dong Lin <li...@gmail.com>
> > > wrote:
> > > >> > >
> > > >> > > > Hey Jun,
> > > >> > > >
> > > >> > > > Yeah I agree that ideally we don't want an ever growing global
> > > >> metadata
> > > >> > > > version. I just think it may be more desirable to keep the
> > > consumer
> > > >> API
> > > >> > > > simple.
> > > >> > > >
> > > >> > > > In my current proposal, metadata version returned in the fetch
> > > >> response
> > > >> > > > will be stored with the offset together. More specifically,
> the
> > > >> > > > metadata_epoch in the new offset topic schema will be the
> > largest
> > > >> > > > metadata_epoch from all the MetadataResponse and FetchResponse
> > > ever
> > > >> > > > received by this consumer.
> > > >> > > >
> > > >> > > > We probably don't have to change the consumer API for
> > > >> > > > commitSync(Map<TopicPartition, OffsetAndMetadata>). If user
> > calls
> > > >> > > > commitSync(...) to commit offset 10 for a given partition, for
> > > most
> > > >> > > > use-cases, this consumer instance should have consumed message
> > > with
> > > >> > > offset
> > > >> > > > 9 from this partition, in which case the consumer can remember
> > and
> > > >> use
> > > >> > > the
> > > >> > > > metadata_epoch from the corresponding FetchResponse when
> > > committing
> > > >> > > offset.
> > > >> > > > If user calls commitSync(..) to commit offset 10 for a given
> > > >> partition
> > > >> > > > without having consumed the message with offset 9 using this
> > > >> consumer
> > > >> > > > instance, this is probably an advanced use-case. In this case
> > the
> > > >> > > advanced
> > > >> > > > user can retrieve the metadata_epoch using the newly added
> > > >> > > metadataEpoch()
> > > >> > > > API after it fetches the message with offset 9 (probably from
> > > >> another
> > > >> > > > consumer instance) and encode this metadata_epoch in the
> > > >> > > > string OffsetAndMetadata.metadata. Do you think this solution
> > > would
> > > >> > work?
> > > >> > > >
> > > >> > > > By "not sure that I fully understand your latest suggestion",
> > are
> > > >> you
> > > >> > > > referring to solution related to unclean leader election using
> > > >> > > leader_epoch
> > > >> > > > in my previous email?
> > > >> > > >
> > > >> > > > Thanks,
> > > >> > > > Dong
> > > >> > > >
> > > >> > > > On Thu, Jan 4, 2018 at 1:33 PM, Jun Rao <ju...@confluent.io>
> > wrote:
> > > >> > > >
> > > >> > > > > Hi, Dong,
> > > >> > > > >
> > > >> > > > > Not sure that I fully understand your latest suggestion.
> > > >> Returning an
> > > >> > > > ever
> > > >> > > > > growing global metadata version itself is no ideal, but is
> > fine.
> > > >> My
> > > >> > > > > question is whether the metadata version returned in the
> fetch
> > > >> > response
> > > >> > > > > needs to be stored with the offset together if offsets are
> > > stored
> > > >> > > > > externally. If so, we also have to change the consumer API
> for
> > > >> > > > commitSync()
> > > >> > > > > and need to worry about compatibility. If we don't store the
> > > >> metadata
> > > >> > > > > version together with the offset, on a consumer restart,
> it's
> > > not
> > > >> > clear
> > > >> > > > how
> > > >> > > > > we can ensure the metadata in the consumer is high enough
> > since
> > > >> there
> > > >> > > is
> > > >> > > > no
> > > >> > > > > metadata version to compare with.
> > > >> > > > >
> > > >> > > > > Thanks,
> > > >> > > > >
> > > >> > > > > Jun
> > > >> > > > >
> > > >> > > > >
> > > >> > > > > On Wed, Jan 3, 2018 at 6:43 PM, Dong Lin <
> lindong28@gmail.com
> > >
> > > >> > wrote:
> > > >> > > > >
> > > >> > > > > > Hey Jun,
> > > >> > > > > >
> > > >> > > > > > Thanks much for the explanation.
> > > >> > > > > >
> > > >> > > > > > I understand the advantage of partition_epoch over
> > > >> metadata_epoch.
> > > >> > My
> > > >> > > > > > current concern is that the use of leader_epoch and the
> > > >> > > partition_epoch
> > > >> > > > > > requires us considerable change to consumer's public API
> to
> > > take
> > > >> > care
> > > >> > > > of
> > > >> > > > > > the case where user stores offset externally. For example,
> > > >> > > *consumer*.
> > > >> > > > > > *commitSync*(..) would have to take a map whose value is
> > > >> <offset,
> > > >> > > > > metadata,
> > > >> > > > > > leader epoch, partition epoch>. *consumer*.*seek*(...)
> would
> > > >> also
> > > >> > > need
> > > >> > > > > > leader_epoch and partition_epoch as parameter. Technically
> > we
> > > >> can
> > > >> > > > > probably
> > > >> > > > > > still make it work in a backward compatible manner after
> > > careful
> > > >> > > design
> > > >> > > > > and
> > > >> > > > > > discussion. But these changes can make the consumer's
> > > interface
> > > >> > > > > > unnecessarily complex for more users who do not store
> offset
> > > >> > > > externally.
> > > >> > > > > >
> > > >> > > > > > After thinking more about it, we can address all problems
> > > >> discussed
> > > >> > > by
> > > >> > > > > only
> > > >> > > > > > using the metadata_epoch without introducing leader_epoch
> or
> > > the
> > > >> > > > > > partition_epoch. The current KIP describes the changes to
> > the
> > > >> > > consumer
> > > >> > > > > API
> > > >> > > > > > and how the new API can be used if user stores offset
> > > >> externally.
> > > >> > In
> > > >> > > > > order
> > > >> > > > > > to address the scenario you described earlier, we can
> > include
> > > >> > > > > > metadata_epoch in the FetchResponse and the
> > > LeaderAndIsrRequest.
> > > >> > > > Consumer
> > > >> > > > > > remembers the largest metadata_epoch from all the
> > > FetchResponse
> > > >> it
> > > >> > > has
> > > >> > > > > > received. The metadata_epoch committed with the offset,
> > either
> > > >> > within
> > > >> > > > or
> > > >> > > > > > outside Kafka, should be the largest metadata_epoch across
> > all
> > > >> > > > > > FetchResponse and MetadataResponse ever received by this
> > > >> consumer.
> > > >> > > > > >
> > > >> > > > > > The drawback of using only the metadata_epoch is that we
> can
> > > not
> > > >> > > always
> > > >> > > > > do
> > > >> > > > > > the smart offset reset in case of unclean leader election
> > > which
> > > >> you
> > > >> > > > > > mentioned earlier. But in most case, unclean leader
> election
> > > >> > probably
> > > >> > > > > > happens when consumer is not rebalancing/restarting. In
> > these
> > > >> > cases,
> > > >> > > > > either
> > > >> > > > > > consumer is not directly affected by unclean leader
> election
> > > >> since
> > > >> > it
> > > >> > > > is
> > > >> > > > > > not consuming from the end of the log, or consumer can
> > derive
> > > >> the
> > > >> > > > > > leader_epoch from the most recent message received before
> it
> > > >> sees
> > > >> > > > > > OffsetOutOfRangeException. So I am not sure it is worth
> > adding
> > > >> the
> > > >> > > > > > leader_epoch to consumer API to address the remaining
> corner
> > > >> case.
> > > >> > > What
> > > >> > > > > do
> > > >> > > > > > you think?
> > > >> > > > > >
> > > >> > > > > > Thanks,
> > > >> > > > > > Dong
> > > >> > > > > >
> > > >> > > > > >
> > > >> > > > > >
> > > >> > > > > > On Tue, Jan 2, 2018 at 6:28 PM, Jun Rao <jun@confluent.io
> >
> > > >> wrote:
> > > >> > > > > >
> > > >> > > > > > > Hi, Dong,
> > > >> > > > > > >
> > > >> > > > > > > Thanks for the reply.
> > > >> > > > > > >
> > > >> > > > > > > To solve the topic recreation issue, we could use
> either a
> > > >> global
> > > >> > > > > > metadata
> > > >> > > > > > > version or a partition level epoch. But either one will
> > be a
> > > >> new
> > > >> > > > > concept,
> > > >> > > > > > > right? To me, the latter seems more natural. It also
> makes
> > > it
> > > >> > > easier
> > > >> > > > to
> > > >> > > > > > > detect if a consumer's offset is still valid after a
> topic
> > > is
> > > >> > > > > recreated.
> > > >> > > > > > As
> > > >> > > > > > > you pointed out, we don't need to store the partition
> > epoch
> > > in
> > > >> > the
> > > >> > > > > > message.
> > > >> > > > > > > The following is what I am thinking. When a partition is
> > > >> created,
> > > >> > > we
> > > >> > > > > can
> > > >> > > > > > > assign a partition epoch from an ever-increasing global
> > > >> counter
> > > >> > and
> > > >> > > > > store
> > > >> > > > > > > it in /brokers/topics/[topic]/partitions/[partitionId]
> in
> > > ZK.
> > > >> > The
> > > >> > > > > > > partition
> > > >> > > > > > > epoch is propagated to every broker. The consumer will
> be
> > > >> > tracking
> > > >> > > a
> > > >> > > > > > tuple
> > > >> > > > > > > of <offset, leader epoch, partition epoch> for offsets.
> > If a
> > > >> > topic
> > > >> > > is
> > > >> > > > > > > recreated, it's possible that a consumer's offset and
> > leader
> > > >> > epoch
> > > >> > > > > still
> > > >> > > > > > > match that in the broker, but partition epoch won't be.
> In
> > > >> this
> > > >> > > case,
> > > >> > > > > we
> > > >> > > > > > > can potentially still treat the consumer's offset as out
> > of
> > > >> range
> > > >> > > and
> > > >> > > > > > reset
> > > >> > > > > > > the offset based on the offset reset policy in the
> > consumer.
> > > >> This
> > > >> > > > seems
> > > >> > > > > > > harder to do with a global metadata version.
> > > >> > > > > > >
> > > >> > > > > > > Jun
> > > >> > > > > > >
> > > >> > > > > > >
> > > >> > > > > > >
> > > >> > > > > > > On Mon, Dec 25, 2017 at 6:56 AM, Dong Lin <
> > > >> lindong28@gmail.com>
> > > >> > > > wrote:
> > > >> > > > > > >
> > > >> > > > > > > > Hey Jun,
> > > >> > > > > > > >
> > > >> > > > > > > > This is a very good example. After thinking through
> this
> > > in
> > > >> > > > detail, I
> > > >> > > > > > > agree
> > > >> > > > > > > > that we need to commit offset with leader epoch in
> order
> > > to
> > > >> > > address
> > > >> > > > > > this
> > > >> > > > > > > > example.
> > > >> > > > > > > >
> > > >> > > > > > > > I think the remaining question is how to address the
> > > >> scenario
> > > >> > > that
> > > >> > > > > the
> > > >> > > > > > > > topic is deleted and re-created. One possible solution
> > is
> > > to
> > > >> > > commit
> > > >> > > > > > > offset
> > > >> > > > > > > > with both the leader epoch and the metadata version.
> The
> > > >> logic
> > > >> > > and
> > > >> > > > > the
> > > >> > > > > > > > implementation of this solution does not require a new
> > > >> concept
> > > >> > > > (e.g.
> > > >> > > > > > > > partition epoch) and it does not require any change to
> > the
> > > >> > > message
> > > >> > > > > > format
> > > >> > > > > > > > or leader epoch. It also allows us to order the
> metadata
> > > in
> > > >> a
> > > >> > > > > > > > straightforward manner which may be useful in the
> > future.
> > > >> So it
> > > >> > > may
> > > >> > > > > be
> > > >> > > > > > a
> > > >> > > > > > > > better solution than generating a random partition
> epoch
> > > >> every
> > > >> > > time
> > > >> > > > > we
> > > >> > > > > > > > create a partition. Does this sound reasonable?
> > > >> > > > > > > >
> > > >> > > > > > > > Previously one concern with using the metadata version
> > is
> > > >> that
> > > >> > > > > consumer
> > > >> > > > > > > > will be forced to refresh metadata even if metadata
> > > version
> > > >> is
> > > >> > > > > > increased
> > > >> > > > > > > > due to topics that the consumer is not interested in.
> > Now
> > > I
> > > >> > > > realized
> > > >> > > > > > that
> > > >> > > > > > > > this is probably not a problem. Currently client will
> > > >> refresh
> > > >> > > > > metadata
> > > >> > > > > > > > either due to InvalidMetadataException in the response
> > > from
> > > >> > > broker
> > > >> > > > or
> > > >> > > > > > due
> > > >> > > > > > > > to metadata expiry. The addition of the metadata
> version
> > > >> should
> > > >> > > > > > increase
> > > >> > > > > > > > the overhead of metadata refresh caused by
> > > >> > > > InvalidMetadataException.
> > > >> > > > > If
> > > >> > > > > > > > client refresh metadata due to expiry and it receives
> a
> > > >> > metadata
> > > >> > > > > whose
> > > >> > > > > > > > version is lower than the current metadata version, we
> > can
> > > >> > reject
> > > >> > > > the
> > > >> > > > > > > > metadata but still reset the metadata age, which
> > > essentially
> > > >> > keep
> > > >> > > > the
> > > >> > > > > > > > existing behavior in the client.
> > > >> > > > > > > >
> > > >> > > > > > > > Thanks much,
> > > >> > > > > > > > Dong
> > > >> > > > > > > >
> > > >> > > > > > >
> > > >> > > > > >
> > > >> > > > >
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Ismael Juma <is...@juma.me.uk>.
On Fri, Jan 5, 2018 at 7:15 PM, Jason Gustafson <ja...@confluent.io> wrote:
>
> class OffsetAndMetadata {
>   long offset;
>   byte[] offsetMetadata;
>   String metadata;
> }


> Admittedly, the naming is a bit annoying, but we can probably come up with
> something better. Internally the byte array would have a version. If in the
> future we have anything else we need to add, we can update the version and
> we wouldn't need any new APIs.
>

We can also add fields to a class in a compatible way. So, it seems to me
that the main advantage of the byte array is that it's opaque to the user.
Is that correct? If so, we could also add any opaque metadata in a subclass
so that users don't even see it (unless they cast it, but then they're on
their own).

Ismael

The corresponding seek() and position() APIs might look something like this:
>
> void seek(TopicPartition partition, long offset, byte[] offsetMetadata);
> byte[] positionMetadata(TopicPartition partition);
>
> What do you think?
>
> Thanks,
> Jason
>
> On Thu, Jan 4, 2018 at 7:04 PM, Dong Lin <li...@gmail.com> wrote:
>
> > Hey Jun, Jason,
> >
> > Thanks much for all the feedback. I have updated the KIP based on the
> > latest discussion. Can you help check whether it looks good?
> >
> > Thanks,
> > Dong
> >
> > On Thu, Jan 4, 2018 at 5:36 PM, Dong Lin <li...@gmail.com> wrote:
> >
> > > Hey Jun,
> > >
> > > Hmm... thinking about this more, I am not sure that the proposed API is
> > > sufficient. For users that store offset externally, we probably need
> > extra
> > > API to return the leader_epoch and partition_epoch for all partitions
> > that
> > > consumers are consuming. I suppose these users currently use position()
> > to
> > > get the offset. Thus we probably need a new method
> positionWithEpoch(..)
> > to
> > > return <offset, partition_epoch, leader_epoch>. Does this sound
> > reasonable?
> > >
> > > Thanks,
> > > Dong
> > >
> > >
> > > On Thu, Jan 4, 2018 at 5:26 PM, Jun Rao <ju...@confluent.io> wrote:
> > >
> > >> Hi, Dong,
> > >>
> > >> Yes, that's what I am thinking. OffsetEpoch will be composed of
> > >> (partition_epoch,
> > >> leader_epoch).
> > >>
> > >> Thanks,
> > >>
> > >> Jun
> > >>
> > >>
> > >> On Thu, Jan 4, 2018 at 4:22 PM, Dong Lin <li...@gmail.com> wrote:
> > >>
> > >> > Hey Jun,
> > >> >
> > >> > Thanks much. I like the the new API that you proposed. I am not sure
> > >> what
> > >> > you exactly mean by offset_epoch. I suppose that we can use the pair
> > of
> > >> > (partition_epoch, leader_epoch) as the offset_epoch, right?
> > >> >
> > >> > Thanks,
> > >> > Dong
> > >> >
> > >> > On Thu, Jan 4, 2018 at 4:02 PM, Jun Rao <ju...@confluent.io> wrote:
> > >> >
> > >> > > Hi, Dong,
> > >> > >
> > >> > > Got it. The api that you proposed works. The question is whether
> > >> that's
> > >> > the
> > >> > > api that we want to have in the long term. My concern is that
> while
> > >> the
> > >> > api
> > >> > > change is simple, the new api seems harder to explain and use. For
> > >> > example,
> > >> > > a consumer storing offsets externally now needs to call
> > >> > > waitForMetadataUpdate() after calling seek().
> > >> > >
> > >> > > An alternative approach is to make the following compatible api
> > >> changes
> > >> > in
> > >> > > Consumer.
> > >> > > * Add an additional OffsetEpoch field in OffsetAndMetadata. (no
> need
> > >> to
> > >> > > change the CommitSync() api)
> > >> > > * Add a new api seek(TopicPartition partition, long offset,
> > >> OffsetEpoch
> > >> > > offsetEpoch). We can potentially deprecate the old api
> > >> > seek(TopicPartition
> > >> > > partition, long offset) in the future.
> > >> > >
> > >> > > The alternative approach has similar amount of api changes as
> yours
> > >> but
> > >> > has
> > >> > > the following benefits.
> > >> > > 1. The api works in a similar way as how offset management works
> now
> > >> and
> > >> > is
> > >> > > probably what we want in the long term.
> > >> > > 2. It can reset offsets better when there is data loss due to
> > unclean
> > >> > > leader election or correlated replica failure.
> > >> > > 3. It can reset offsets better when topic is recreated.
> > >> > >
> > >> > > Thanks,
> > >> > >
> > >> > > Jun
> > >> > >
> > >> > >
> > >> > > On Thu, Jan 4, 2018 at 2:05 PM, Dong Lin <li...@gmail.com>
> > wrote:
> > >> > >
> > >> > > > Hey Jun,
> > >> > > >
> > >> > > > Yeah I agree that ideally we don't want an ever growing global
> > >> metadata
> > >> > > > version. I just think it may be more desirable to keep the
> > consumer
> > >> API
> > >> > > > simple.
> > >> > > >
> > >> > > > In my current proposal, metadata version returned in the fetch
> > >> response
> > >> > > > will be stored with the offset together. More specifically, the
> > >> > > > metadata_epoch in the new offset topic schema will be the
> largest
> > >> > > > metadata_epoch from all the MetadataResponse and FetchResponse
> > ever
> > >> > > > received by this consumer.
> > >> > > >
> > >> > > > We probably don't have to change the consumer API for
> > >> > > > commitSync(Map<TopicPartition, OffsetAndMetadata>). If user
> calls
> > >> > > > commitSync(...) to commit offset 10 for a given partition, for
> > most
> > >> > > > use-cases, this consumer instance should have consumed message
> > with
> > >> > > offset
> > >> > > > 9 from this partition, in which case the consumer can remember
> and
> > >> use
> > >> > > the
> > >> > > > metadata_epoch from the corresponding FetchResponse when
> > committing
> > >> > > offset.
> > >> > > > If user calls commitSync(..) to commit offset 10 for a given
> > >> partition
> > >> > > > without having consumed the message with offset 9 using this
> > >> consumer
> > >> > > > instance, this is probably an advanced use-case. In this case
> the
> > >> > > advanced
> > >> > > > user can retrieve the metadata_epoch using the newly added
> > >> > > metadataEpoch()
> > >> > > > API after it fetches the message with offset 9 (probably from
> > >> another
> > >> > > > consumer instance) and encode this metadata_epoch in the
> > >> > > > string OffsetAndMetadata.metadata. Do you think this solution
> > would
> > >> > work?
> > >> > > >
> > >> > > > By "not sure that I fully understand your latest suggestion",
> are
> > >> you
> > >> > > > referring to solution related to unclean leader election using
> > >> > > leader_epoch
> > >> > > > in my previous email?
> > >> > > >
> > >> > > > Thanks,
> > >> > > > Dong
> > >> > > >
> > >> > > > On Thu, Jan 4, 2018 at 1:33 PM, Jun Rao <ju...@confluent.io>
> wrote:
> > >> > > >
> > >> > > > > Hi, Dong,
> > >> > > > >
> > >> > > > > Not sure that I fully understand your latest suggestion.
> > >> Returning an
> > >> > > > ever
> > >> > > > > growing global metadata version itself is no ideal, but is
> fine.
> > >> My
> > >> > > > > question is whether the metadata version returned in the fetch
> > >> > response
> > >> > > > > needs to be stored with the offset together if offsets are
> > stored
> > >> > > > > externally. If so, we also have to change the consumer API for
> > >> > > > commitSync()
> > >> > > > > and need to worry about compatibility. If we don't store the
> > >> metadata
> > >> > > > > version together with the offset, on a consumer restart, it's
> > not
> > >> > clear
> > >> > > > how
> > >> > > > > we can ensure the metadata in the consumer is high enough
> since
> > >> there
> > >> > > is
> > >> > > > no
> > >> > > > > metadata version to compare with.
> > >> > > > >
> > >> > > > > Thanks,
> > >> > > > >
> > >> > > > > Jun
> > >> > > > >
> > >> > > > >
> > >> > > > > On Wed, Jan 3, 2018 at 6:43 PM, Dong Lin <lindong28@gmail.com
> >
> > >> > wrote:
> > >> > > > >
> > >> > > > > > Hey Jun,
> > >> > > > > >
> > >> > > > > > Thanks much for the explanation.
> > >> > > > > >
> > >> > > > > > I understand the advantage of partition_epoch over
> > >> metadata_epoch.
> > >> > My
> > >> > > > > > current concern is that the use of leader_epoch and the
> > >> > > partition_epoch
> > >> > > > > > requires us considerable change to consumer's public API to
> > take
> > >> > care
> > >> > > > of
> > >> > > > > > the case where user stores offset externally. For example,
> > >> > > *consumer*.
> > >> > > > > > *commitSync*(..) would have to take a map whose value is
> > >> <offset,
> > >> > > > > metadata,
> > >> > > > > > leader epoch, partition epoch>. *consumer*.*seek*(...) would
> > >> also
> > >> > > need
> > >> > > > > > leader_epoch and partition_epoch as parameter. Technically
> we
> > >> can
> > >> > > > > probably
> > >> > > > > > still make it work in a backward compatible manner after
> > careful
> > >> > > design
> > >> > > > > and
> > >> > > > > > discussion. But these changes can make the consumer's
> > interface
> > >> > > > > > unnecessarily complex for more users who do not store offset
> > >> > > > externally.
> > >> > > > > >
> > >> > > > > > After thinking more about it, we can address all problems
> > >> discussed
> > >> > > by
> > >> > > > > only
> > >> > > > > > using the metadata_epoch without introducing leader_epoch or
> > the
> > >> > > > > > partition_epoch. The current KIP describes the changes to
> the
> > >> > > consumer
> > >> > > > > API
> > >> > > > > > and how the new API can be used if user stores offset
> > >> externally.
> > >> > In
> > >> > > > > order
> > >> > > > > > to address the scenario you described earlier, we can
> include
> > >> > > > > > metadata_epoch in the FetchResponse and the
> > LeaderAndIsrRequest.
> > >> > > > Consumer
> > >> > > > > > remembers the largest metadata_epoch from all the
> > FetchResponse
> > >> it
> > >> > > has
> > >> > > > > > received. The metadata_epoch committed with the offset,
> either
> > >> > within
> > >> > > > or
> > >> > > > > > outside Kafka, should be the largest metadata_epoch across
> all
> > >> > > > > > FetchResponse and MetadataResponse ever received by this
> > >> consumer.
> > >> > > > > >
> > >> > > > > > The drawback of using only the metadata_epoch is that we can
> > not
> > >> > > always
> > >> > > > > do
> > >> > > > > > the smart offset reset in case of unclean leader election
> > which
> > >> you
> > >> > > > > > mentioned earlier. But in most case, unclean leader election
> > >> > probably
> > >> > > > > > happens when consumer is not rebalancing/restarting. In
> these
> > >> > cases,
> > >> > > > > either
> > >> > > > > > consumer is not directly affected by unclean leader election
> > >> since
> > >> > it
> > >> > > > is
> > >> > > > > > not consuming from the end of the log, or consumer can
> derive
> > >> the
> > >> > > > > > leader_epoch from the most recent message received before it
> > >> sees
> > >> > > > > > OffsetOutOfRangeException. So I am not sure it is worth
> adding
> > >> the
> > >> > > > > > leader_epoch to consumer API to address the remaining corner
> > >> case.
> > >> > > What
> > >> > > > > do
> > >> > > > > > you think?
> > >> > > > > >
> > >> > > > > > Thanks,
> > >> > > > > > Dong
> > >> > > > > >
> > >> > > > > >
> > >> > > > > >
> > >> > > > > > On Tue, Jan 2, 2018 at 6:28 PM, Jun Rao <ju...@confluent.io>
> > >> wrote:
> > >> > > > > >
> > >> > > > > > > Hi, Dong,
> > >> > > > > > >
> > >> > > > > > > Thanks for the reply.
> > >> > > > > > >
> > >> > > > > > > To solve the topic recreation issue, we could use either a
> > >> global
> > >> > > > > > metadata
> > >> > > > > > > version or a partition level epoch. But either one will
> be a
> > >> new
> > >> > > > > concept,
> > >> > > > > > > right? To me, the latter seems more natural. It also makes
> > it
> > >> > > easier
> > >> > > > to
> > >> > > > > > > detect if a consumer's offset is still valid after a topic
> > is
> > >> > > > > recreated.
> > >> > > > > > As
> > >> > > > > > > you pointed out, we don't need to store the partition
> epoch
> > in
> > >> > the
> > >> > > > > > message.
> > >> > > > > > > The following is what I am thinking. When a partition is
> > >> created,
> > >> > > we
> > >> > > > > can
> > >> > > > > > > assign a partition epoch from an ever-increasing global
> > >> counter
> > >> > and
> > >> > > > > store
> > >> > > > > > > it in /brokers/topics/[topic]/partitions/[partitionId] in
> > ZK.
> > >> > The
> > >> > > > > > > partition
> > >> > > > > > > epoch is propagated to every broker. The consumer will be
> > >> > tracking
> > >> > > a
> > >> > > > > > tuple
> > >> > > > > > > of <offset, leader epoch, partition epoch> for offsets.
> If a
> > >> > topic
> > >> > > is
> > >> > > > > > > recreated, it's possible that a consumer's offset and
> leader
> > >> > epoch
> > >> > > > > still
> > >> > > > > > > match that in the broker, but partition epoch won't be. In
> > >> this
> > >> > > case,
> > >> > > > > we
> > >> > > > > > > can potentially still treat the consumer's offset as out
> of
> > >> range
> > >> > > and
> > >> > > > > > reset
> > >> > > > > > > the offset based on the offset reset policy in the
> consumer.
> > >> This
> > >> > > > seems
> > >> > > > > > > harder to do with a global metadata version.
> > >> > > > > > >
> > >> > > > > > > Jun
> > >> > > > > > >
> > >> > > > > > >
> > >> > > > > > >
> > >> > > > > > > On Mon, Dec 25, 2017 at 6:56 AM, Dong Lin <
> > >> lindong28@gmail.com>
> > >> > > > wrote:
> > >> > > > > > >
> > >> > > > > > > > Hey Jun,
> > >> > > > > > > >
> > >> > > > > > > > This is a very good example. After thinking through this
> > in
> > >> > > > detail, I
> > >> > > > > > > agree
> > >> > > > > > > > that we need to commit offset with leader epoch in order
> > to
> > >> > > address
> > >> > > > > > this
> > >> > > > > > > > example.
> > >> > > > > > > >
> > >> > > > > > > > I think the remaining question is how to address the
> > >> scenario
> > >> > > that
> > >> > > > > the
> > >> > > > > > > > topic is deleted and re-created. One possible solution
> is
> > to
> > >> > > commit
> > >> > > > > > > offset
> > >> > > > > > > > with both the leader epoch and the metadata version. The
> > >> logic
> > >> > > and
> > >> > > > > the
> > >> > > > > > > > implementation of this solution does not require a new
> > >> concept
> > >> > > > (e.g.
> > >> > > > > > > > partition epoch) and it does not require any change to
> the
> > >> > > message
> > >> > > > > > format
> > >> > > > > > > > or leader epoch. It also allows us to order the metadata
> > in
> > >> a
> > >> > > > > > > > straightforward manner which may be useful in the
> future.
> > >> So it
> > >> > > may
> > >> > > > > be
> > >> > > > > > a
> > >> > > > > > > > better solution than generating a random partition epoch
> > >> every
> > >> > > time
> > >> > > > > we
> > >> > > > > > > > create a partition. Does this sound reasonable?
> > >> > > > > > > >
> > >> > > > > > > > Previously one concern with using the metadata version
> is
> > >> that
> > >> > > > > consumer
> > >> > > > > > > > will be forced to refresh metadata even if metadata
> > version
> > >> is
> > >> > > > > > increased
> > >> > > > > > > > due to topics that the consumer is not interested in.
> Now
> > I
> > >> > > > realized
> > >> > > > > > that
> > >> > > > > > > > this is probably not a problem. Currently client will
> > >> refresh
> > >> > > > > metadata
> > >> > > > > > > > either due to InvalidMetadataException in the response
> > from
> > >> > > broker
> > >> > > > or
> > >> > > > > > due
> > >> > > > > > > > to metadata expiry. The addition of the metadata version
> > >> should
> > >> > > > > > increase
> > >> > > > > > > > the overhead of metadata refresh caused by
> > >> > > > InvalidMetadataException.
> > >> > > > > If
> > >> > > > > > > > client refresh metadata due to expiry and it receives a
> > >> > metadata
> > >> > > > > whose
> > >> > > > > > > > version is lower than the current metadata version, we
> can
> > >> > reject
> > >> > > > the
> > >> > > > > > > > metadata but still reset the metadata age, which
> > essentially
> > >> > keep
> > >> > > > the
> > >> > > > > > > > existing behavior in the client.
> > >> > > > > > > >
> > >> > > > > > > > Thanks much,
> > >> > > > > > > > Dong
> > >> > > > > > > >
> > >> > > > > > >
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Jason Gustafson <ja...@confluent.io>.
Hi Dong,

Sorry for the late reply. I think the latest revision is looking good. I
have a few minor suggestions:

1. The name "partition_epoch" makes me think it changes independently at
the partition level, but all partitions for a topic should have the same
epoch. Maybe "topic_epoch" is nearer the mark?
2. Should we increment this epoch when a topic is deleted also? When the
broker returns an UNKNOWN_TOPIC_OR_PARTITION error in a metadata response,
we can also include the latest partition epoch, which would allow the
client to disambiguate the error if it has seen more recent metadata.
3. I am still wondering whether it is a good idea to expose these epochs in
the consumer API. As an alternative, have you considered representing the
data as an opaque blob of bytes? For example:

class OffsetAndMetadata {
  long offset;
  byte[] offsetMetadata;
  String metadata;
}

Admittedly, the naming is a bit annoying, but we can probably come up with
something better. Internally the byte array would have a version. If in the
future we have anything else we need to add, we can update the version and
we wouldn't need any new APIs.

The corresponding seek() and position() APIs might look something like this:

void seek(TopicPartition partition, long offset, byte[] offsetMetadata);
byte[] positionMetadata(TopicPartition partition);

What do you think?

Thanks,
Jason

On Thu, Jan 4, 2018 at 7:04 PM, Dong Lin <li...@gmail.com> wrote:

> Hey Jun, Jason,
>
> Thanks much for all the feedback. I have updated the KIP based on the
> latest discussion. Can you help check whether it looks good?
>
> Thanks,
> Dong
>
> On Thu, Jan 4, 2018 at 5:36 PM, Dong Lin <li...@gmail.com> wrote:
>
> > Hey Jun,
> >
> > Hmm... thinking about this more, I am not sure that the proposed API is
> > sufficient. For users that store offset externally, we probably need
> extra
> > API to return the leader_epoch and partition_epoch for all partitions
> that
> > consumers are consuming. I suppose these users currently use position()
> to
> > get the offset. Thus we probably need a new method positionWithEpoch(..)
> to
> > return <offset, partition_epoch, leader_epoch>. Does this sound
> reasonable?
> >
> > Thanks,
> > Dong
> >
> >
> > On Thu, Jan 4, 2018 at 5:26 PM, Jun Rao <ju...@confluent.io> wrote:
> >
> >> Hi, Dong,
> >>
> >> Yes, that's what I am thinking. OffsetEpoch will be composed of
> >> (partition_epoch,
> >> leader_epoch).
> >>
> >> Thanks,
> >>
> >> Jun
> >>
> >>
> >> On Thu, Jan 4, 2018 at 4:22 PM, Dong Lin <li...@gmail.com> wrote:
> >>
> >> > Hey Jun,
> >> >
> >> > Thanks much. I like the the new API that you proposed. I am not sure
> >> what
> >> > you exactly mean by offset_epoch. I suppose that we can use the pair
> of
> >> > (partition_epoch, leader_epoch) as the offset_epoch, right?
> >> >
> >> > Thanks,
> >> > Dong
> >> >
> >> > On Thu, Jan 4, 2018 at 4:02 PM, Jun Rao <ju...@confluent.io> wrote:
> >> >
> >> > > Hi, Dong,
> >> > >
> >> > > Got it. The api that you proposed works. The question is whether
> >> that's
> >> > the
> >> > > api that we want to have in the long term. My concern is that while
> >> the
> >> > api
> >> > > change is simple, the new api seems harder to explain and use. For
> >> > example,
> >> > > a consumer storing offsets externally now needs to call
> >> > > waitForMetadataUpdate() after calling seek().
> >> > >
> >> > > An alternative approach is to make the following compatible api
> >> changes
> >> > in
> >> > > Consumer.
> >> > > * Add an additional OffsetEpoch field in OffsetAndMetadata. (no need
> >> to
> >> > > change the CommitSync() api)
> >> > > * Add a new api seek(TopicPartition partition, long offset,
> >> OffsetEpoch
> >> > > offsetEpoch). We can potentially deprecate the old api
> >> > seek(TopicPartition
> >> > > partition, long offset) in the future.
> >> > >
> >> > > The alternative approach has similar amount of api changes as yours
> >> but
> >> > has
> >> > > the following benefits.
> >> > > 1. The api works in a similar way as how offset management works now
> >> and
> >> > is
> >> > > probably what we want in the long term.
> >> > > 2. It can reset offsets better when there is data loss due to
> unclean
> >> > > leader election or correlated replica failure.
> >> > > 3. It can reset offsets better when topic is recreated.
> >> > >
> >> > > Thanks,
> >> > >
> >> > > Jun
> >> > >
> >> > >
> >> > > On Thu, Jan 4, 2018 at 2:05 PM, Dong Lin <li...@gmail.com>
> wrote:
> >> > >
> >> > > > Hey Jun,
> >> > > >
> >> > > > Yeah I agree that ideally we don't want an ever growing global
> >> metadata
> >> > > > version. I just think it may be more desirable to keep the
> consumer
> >> API
> >> > > > simple.
> >> > > >
> >> > > > In my current proposal, metadata version returned in the fetch
> >> response
> >> > > > will be stored with the offset together. More specifically, the
> >> > > > metadata_epoch in the new offset topic schema will be the largest
> >> > > > metadata_epoch from all the MetadataResponse and FetchResponse
> ever
> >> > > > received by this consumer.
> >> > > >
> >> > > > We probably don't have to change the consumer API for
> >> > > > commitSync(Map<TopicPartition, OffsetAndMetadata>). If user calls
> >> > > > commitSync(...) to commit offset 10 for a given partition, for
> most
> >> > > > use-cases, this consumer instance should have consumed message
> with
> >> > > offset
> >> > > > 9 from this partition, in which case the consumer can remember and
> >> use
> >> > > the
> >> > > > metadata_epoch from the corresponding FetchResponse when
> committing
> >> > > offset.
> >> > > > If user calls commitSync(..) to commit offset 10 for a given
> >> partition
> >> > > > without having consumed the message with offset 9 using this
> >> consumer
> >> > > > instance, this is probably an advanced use-case. In this case the
> >> > > advanced
> >> > > > user can retrieve the metadata_epoch using the newly added
> >> > > metadataEpoch()
> >> > > > API after it fetches the message with offset 9 (probably from
> >> another
> >> > > > consumer instance) and encode this metadata_epoch in the
> >> > > > string OffsetAndMetadata.metadata. Do you think this solution
> would
> >> > work?
> >> > > >
> >> > > > By "not sure that I fully understand your latest suggestion", are
> >> you
> >> > > > referring to solution related to unclean leader election using
> >> > > leader_epoch
> >> > > > in my previous email?
> >> > > >
> >> > > > Thanks,
> >> > > > Dong
> >> > > >
> >> > > > On Thu, Jan 4, 2018 at 1:33 PM, Jun Rao <ju...@confluent.io> wrote:
> >> > > >
> >> > > > > Hi, Dong,
> >> > > > >
> >> > > > > Not sure that I fully understand your latest suggestion.
> >> Returning an
> >> > > > ever
> >> > > > > growing global metadata version itself is no ideal, but is fine.
> >> My
> >> > > > > question is whether the metadata version returned in the fetch
> >> > response
> >> > > > > needs to be stored with the offset together if offsets are
> stored
> >> > > > > externally. If so, we also have to change the consumer API for
> >> > > > commitSync()
> >> > > > > and need to worry about compatibility. If we don't store the
> >> metadata
> >> > > > > version together with the offset, on a consumer restart, it's
> not
> >> > clear
> >> > > > how
> >> > > > > we can ensure the metadata in the consumer is high enough since
> >> there
> >> > > is
> >> > > > no
> >> > > > > metadata version to compare with.
> >> > > > >
> >> > > > > Thanks,
> >> > > > >
> >> > > > > Jun
> >> > > > >
> >> > > > >
> >> > > > > On Wed, Jan 3, 2018 at 6:43 PM, Dong Lin <li...@gmail.com>
> >> > wrote:
> >> > > > >
> >> > > > > > Hey Jun,
> >> > > > > >
> >> > > > > > Thanks much for the explanation.
> >> > > > > >
> >> > > > > > I understand the advantage of partition_epoch over
> >> metadata_epoch.
> >> > My
> >> > > > > > current concern is that the use of leader_epoch and the
> >> > > partition_epoch
> >> > > > > > requires us considerable change to consumer's public API to
> take
> >> > care
> >> > > > of
> >> > > > > > the case where user stores offset externally. For example,
> >> > > *consumer*.
> >> > > > > > *commitSync*(..) would have to take a map whose value is
> >> <offset,
> >> > > > > metadata,
> >> > > > > > leader epoch, partition epoch>. *consumer*.*seek*(...) would
> >> also
> >> > > need
> >> > > > > > leader_epoch and partition_epoch as parameter. Technically we
> >> can
> >> > > > > probably
> >> > > > > > still make it work in a backward compatible manner after
> careful
> >> > > design
> >> > > > > and
> >> > > > > > discussion. But these changes can make the consumer's
> interface
> >> > > > > > unnecessarily complex for more users who do not store offset
> >> > > > externally.
> >> > > > > >
> >> > > > > > After thinking more about it, we can address all problems
> >> discussed
> >> > > by
> >> > > > > only
> >> > > > > > using the metadata_epoch without introducing leader_epoch or
> the
> >> > > > > > partition_epoch. The current KIP describes the changes to the
> >> > > consumer
> >> > > > > API
> >> > > > > > and how the new API can be used if user stores offset
> >> externally.
> >> > In
> >> > > > > order
> >> > > > > > to address the scenario you described earlier, we can include
> >> > > > > > metadata_epoch in the FetchResponse and the
> LeaderAndIsrRequest.
> >> > > > Consumer
> >> > > > > > remembers the largest metadata_epoch from all the
> FetchResponse
> >> it
> >> > > has
> >> > > > > > received. The metadata_epoch committed with the offset, either
> >> > within
> >> > > > or
> >> > > > > > outside Kafka, should be the largest metadata_epoch across all
> >> > > > > > FetchResponse and MetadataResponse ever received by this
> >> consumer.
> >> > > > > >
> >> > > > > > The drawback of using only the metadata_epoch is that we can
> not
> >> > > always
> >> > > > > do
> >> > > > > > the smart offset reset in case of unclean leader election
> which
> >> you
> >> > > > > > mentioned earlier. But in most case, unclean leader election
> >> > probably
> >> > > > > > happens when consumer is not rebalancing/restarting. In these
> >> > cases,
> >> > > > > either
> >> > > > > > consumer is not directly affected by unclean leader election
> >> since
> >> > it
> >> > > > is
> >> > > > > > not consuming from the end of the log, or consumer can derive
> >> the
> >> > > > > > leader_epoch from the most recent message received before it
> >> sees
> >> > > > > > OffsetOutOfRangeException. So I am not sure it is worth adding
> >> the
> >> > > > > > leader_epoch to consumer API to address the remaining corner
> >> case.
> >> > > What
> >> > > > > do
> >> > > > > > you think?
> >> > > > > >
> >> > > > > > Thanks,
> >> > > > > > Dong
> >> > > > > >
> >> > > > > >
> >> > > > > >
> >> > > > > > On Tue, Jan 2, 2018 at 6:28 PM, Jun Rao <ju...@confluent.io>
> >> wrote:
> >> > > > > >
> >> > > > > > > Hi, Dong,
> >> > > > > > >
> >> > > > > > > Thanks for the reply.
> >> > > > > > >
> >> > > > > > > To solve the topic recreation issue, we could use either a
> >> global
> >> > > > > > metadata
> >> > > > > > > version or a partition level epoch. But either one will be a
> >> new
> >> > > > > concept,
> >> > > > > > > right? To me, the latter seems more natural. It also makes
> it
> >> > > easier
> >> > > > to
> >> > > > > > > detect if a consumer's offset is still valid after a topic
> is
> >> > > > > recreated.
> >> > > > > > As
> >> > > > > > > you pointed out, we don't need to store the partition epoch
> in
> >> > the
> >> > > > > > message.
> >> > > > > > > The following is what I am thinking. When a partition is
> >> created,
> >> > > we
> >> > > > > can
> >> > > > > > > assign a partition epoch from an ever-increasing global
> >> counter
> >> > and
> >> > > > > store
> >> > > > > > > it in /brokers/topics/[topic]/partitions/[partitionId] in
> ZK.
> >> > The
> >> > > > > > > partition
> >> > > > > > > epoch is propagated to every broker. The consumer will be
> >> > tracking
> >> > > a
> >> > > > > > tuple
> >> > > > > > > of <offset, leader epoch, partition epoch> for offsets. If a
> >> > topic
> >> > > is
> >> > > > > > > recreated, it's possible that a consumer's offset and leader
> >> > epoch
> >> > > > > still
> >> > > > > > > match that in the broker, but partition epoch won't be. In
> >> this
> >> > > case,
> >> > > > > we
> >> > > > > > > can potentially still treat the consumer's offset as out of
> >> range
> >> > > and
> >> > > > > > reset
> >> > > > > > > the offset based on the offset reset policy in the consumer.
> >> This
> >> > > > seems
> >> > > > > > > harder to do with a global metadata version.
> >> > > > > > >
> >> > > > > > > Jun
> >> > > > > > >
> >> > > > > > >
> >> > > > > > >
> >> > > > > > > On Mon, Dec 25, 2017 at 6:56 AM, Dong Lin <
> >> lindong28@gmail.com>
> >> > > > wrote:
> >> > > > > > >
> >> > > > > > > > Hey Jun,
> >> > > > > > > >
> >> > > > > > > > This is a very good example. After thinking through this
> in
> >> > > > detail, I
> >> > > > > > > agree
> >> > > > > > > > that we need to commit offset with leader epoch in order
> to
> >> > > address
> >> > > > > > this
> >> > > > > > > > example.
> >> > > > > > > >
> >> > > > > > > > I think the remaining question is how to address the
> >> scenario
> >> > > that
> >> > > > > the
> >> > > > > > > > topic is deleted and re-created. One possible solution is
> to
> >> > > commit
> >> > > > > > > offset
> >> > > > > > > > with both the leader epoch and the metadata version. The
> >> logic
> >> > > and
> >> > > > > the
> >> > > > > > > > implementation of this solution does not require a new
> >> concept
> >> > > > (e.g.
> >> > > > > > > > partition epoch) and it does not require any change to the
> >> > > message
> >> > > > > > format
> >> > > > > > > > or leader epoch. It also allows us to order the metadata
> in
> >> a
> >> > > > > > > > straightforward manner which may be useful in the future.
> >> So it
> >> > > may
> >> > > > > be
> >> > > > > > a
> >> > > > > > > > better solution than generating a random partition epoch
> >> every
> >> > > time
> >> > > > > we
> >> > > > > > > > create a partition. Does this sound reasonable?
> >> > > > > > > >
> >> > > > > > > > Previously one concern with using the metadata version is
> >> that
> >> > > > > consumer
> >> > > > > > > > will be forced to refresh metadata even if metadata
> version
> >> is
> >> > > > > > increased
> >> > > > > > > > due to topics that the consumer is not interested in. Now
> I
> >> > > > realized
> >> > > > > > that
> >> > > > > > > > this is probably not a problem. Currently client will
> >> refresh
> >> > > > > metadata
> >> > > > > > > > either due to InvalidMetadataException in the response
> from
> >> > > broker
> >> > > > or
> >> > > > > > due
> >> > > > > > > > to metadata expiry. The addition of the metadata version
> >> should
> >> > > > > > increase
> >> > > > > > > > the overhead of metadata refresh caused by
> >> > > > InvalidMetadataException.
> >> > > > > If
> >> > > > > > > > client refresh metadata due to expiry and it receives a
> >> > metadata
> >> > > > > whose
> >> > > > > > > > version is lower than the current metadata version, we can
> >> > reject
> >> > > > the
> >> > > > > > > > metadata but still reset the metadata age, which
> essentially
> >> > keep
> >> > > > the
> >> > > > > > > > existing behavior in the client.
> >> > > > > > > >
> >> > > > > > > > Thanks much,
> >> > > > > > > > Dong
> >> > > > > > > >
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Dong Lin <li...@gmail.com>.
Hey Jun, Jason,

Thanks much for all the feedback. I have updated the KIP based on the
latest discussion. Can you help check whether it looks good?

Thanks,
Dong

On Thu, Jan 4, 2018 at 5:36 PM, Dong Lin <li...@gmail.com> wrote:

> Hey Jun,
>
> Hmm... thinking about this more, I am not sure that the proposed API is
> sufficient. For users that store offset externally, we probably need extra
> API to return the leader_epoch and partition_epoch for all partitions that
> consumers are consuming. I suppose these users currently use position() to
> get the offset. Thus we probably need a new method positionWithEpoch(..) to
> return <offset, partition_epoch, leader_epoch>. Does this sound reasonable?
>
> Thanks,
> Dong
>
>
> On Thu, Jan 4, 2018 at 5:26 PM, Jun Rao <ju...@confluent.io> wrote:
>
>> Hi, Dong,
>>
>> Yes, that's what I am thinking. OffsetEpoch will be composed of
>> (partition_epoch,
>> leader_epoch).
>>
>> Thanks,
>>
>> Jun
>>
>>
>> On Thu, Jan 4, 2018 at 4:22 PM, Dong Lin <li...@gmail.com> wrote:
>>
>> > Hey Jun,
>> >
>> > Thanks much. I like the the new API that you proposed. I am not sure
>> what
>> > you exactly mean by offset_epoch. I suppose that we can use the pair of
>> > (partition_epoch, leader_epoch) as the offset_epoch, right?
>> >
>> > Thanks,
>> > Dong
>> >
>> > On Thu, Jan 4, 2018 at 4:02 PM, Jun Rao <ju...@confluent.io> wrote:
>> >
>> > > Hi, Dong,
>> > >
>> > > Got it. The api that you proposed works. The question is whether
>> that's
>> > the
>> > > api that we want to have in the long term. My concern is that while
>> the
>> > api
>> > > change is simple, the new api seems harder to explain and use. For
>> > example,
>> > > a consumer storing offsets externally now needs to call
>> > > waitForMetadataUpdate() after calling seek().
>> > >
>> > > An alternative approach is to make the following compatible api
>> changes
>> > in
>> > > Consumer.
>> > > * Add an additional OffsetEpoch field in OffsetAndMetadata. (no need
>> to
>> > > change the CommitSync() api)
>> > > * Add a new api seek(TopicPartition partition, long offset,
>> OffsetEpoch
>> > > offsetEpoch). We can potentially deprecate the old api
>> > seek(TopicPartition
>> > > partition, long offset) in the future.
>> > >
>> > > The alternative approach has similar amount of api changes as yours
>> but
>> > has
>> > > the following benefits.
>> > > 1. The api works in a similar way as how offset management works now
>> and
>> > is
>> > > probably what we want in the long term.
>> > > 2. It can reset offsets better when there is data loss due to unclean
>> > > leader election or correlated replica failure.
>> > > 3. It can reset offsets better when topic is recreated.
>> > >
>> > > Thanks,
>> > >
>> > > Jun
>> > >
>> > >
>> > > On Thu, Jan 4, 2018 at 2:05 PM, Dong Lin <li...@gmail.com> wrote:
>> > >
>> > > > Hey Jun,
>> > > >
>> > > > Yeah I agree that ideally we don't want an ever growing global
>> metadata
>> > > > version. I just think it may be more desirable to keep the consumer
>> API
>> > > > simple.
>> > > >
>> > > > In my current proposal, metadata version returned in the fetch
>> response
>> > > > will be stored with the offset together. More specifically, the
>> > > > metadata_epoch in the new offset topic schema will be the largest
>> > > > metadata_epoch from all the MetadataResponse and FetchResponse ever
>> > > > received by this consumer.
>> > > >
>> > > > We probably don't have to change the consumer API for
>> > > > commitSync(Map<TopicPartition, OffsetAndMetadata>). If user calls
>> > > > commitSync(...) to commit offset 10 for a given partition, for most
>> > > > use-cases, this consumer instance should have consumed message with
>> > > offset
>> > > > 9 from this partition, in which case the consumer can remember and
>> use
>> > > the
>> > > > metadata_epoch from the corresponding FetchResponse when committing
>> > > offset.
>> > > > If user calls commitSync(..) to commit offset 10 for a given
>> partition
>> > > > without having consumed the message with offset 9 using this
>> consumer
>> > > > instance, this is probably an advanced use-case. In this case the
>> > > advanced
>> > > > user can retrieve the metadata_epoch using the newly added
>> > > metadataEpoch()
>> > > > API after it fetches the message with offset 9 (probably from
>> another
>> > > > consumer instance) and encode this metadata_epoch in the
>> > > > string OffsetAndMetadata.metadata. Do you think this solution would
>> > work?
>> > > >
>> > > > By "not sure that I fully understand your latest suggestion", are
>> you
>> > > > referring to solution related to unclean leader election using
>> > > leader_epoch
>> > > > in my previous email?
>> > > >
>> > > > Thanks,
>> > > > Dong
>> > > >
>> > > > On Thu, Jan 4, 2018 at 1:33 PM, Jun Rao <ju...@confluent.io> wrote:
>> > > >
>> > > > > Hi, Dong,
>> > > > >
>> > > > > Not sure that I fully understand your latest suggestion.
>> Returning an
>> > > > ever
>> > > > > growing global metadata version itself is no ideal, but is fine.
>> My
>> > > > > question is whether the metadata version returned in the fetch
>> > response
>> > > > > needs to be stored with the offset together if offsets are stored
>> > > > > externally. If so, we also have to change the consumer API for
>> > > > commitSync()
>> > > > > and need to worry about compatibility. If we don't store the
>> metadata
>> > > > > version together with the offset, on a consumer restart, it's not
>> > clear
>> > > > how
>> > > > > we can ensure the metadata in the consumer is high enough since
>> there
>> > > is
>> > > > no
>> > > > > metadata version to compare with.
>> > > > >
>> > > > > Thanks,
>> > > > >
>> > > > > Jun
>> > > > >
>> > > > >
>> > > > > On Wed, Jan 3, 2018 at 6:43 PM, Dong Lin <li...@gmail.com>
>> > wrote:
>> > > > >
>> > > > > > Hey Jun,
>> > > > > >
>> > > > > > Thanks much for the explanation.
>> > > > > >
>> > > > > > I understand the advantage of partition_epoch over
>> metadata_epoch.
>> > My
>> > > > > > current concern is that the use of leader_epoch and the
>> > > partition_epoch
>> > > > > > requires us considerable change to consumer's public API to take
>> > care
>> > > > of
>> > > > > > the case where user stores offset externally. For example,
>> > > *consumer*.
>> > > > > > *commitSync*(..) would have to take a map whose value is
>> <offset,
>> > > > > metadata,
>> > > > > > leader epoch, partition epoch>. *consumer*.*seek*(...) would
>> also
>> > > need
>> > > > > > leader_epoch and partition_epoch as parameter. Technically we
>> can
>> > > > > probably
>> > > > > > still make it work in a backward compatible manner after careful
>> > > design
>> > > > > and
>> > > > > > discussion. But these changes can make the consumer's interface
>> > > > > > unnecessarily complex for more users who do not store offset
>> > > > externally.
>> > > > > >
>> > > > > > After thinking more about it, we can address all problems
>> discussed
>> > > by
>> > > > > only
>> > > > > > using the metadata_epoch without introducing leader_epoch or the
>> > > > > > partition_epoch. The current KIP describes the changes to the
>> > > consumer
>> > > > > API
>> > > > > > and how the new API can be used if user stores offset
>> externally.
>> > In
>> > > > > order
>> > > > > > to address the scenario you described earlier, we can include
>> > > > > > metadata_epoch in the FetchResponse and the LeaderAndIsrRequest.
>> > > > Consumer
>> > > > > > remembers the largest metadata_epoch from all the FetchResponse
>> it
>> > > has
>> > > > > > received. The metadata_epoch committed with the offset, either
>> > within
>> > > > or
>> > > > > > outside Kafka, should be the largest metadata_epoch across all
>> > > > > > FetchResponse and MetadataResponse ever received by this
>> consumer.
>> > > > > >
>> > > > > > The drawback of using only the metadata_epoch is that we can not
>> > > always
>> > > > > do
>> > > > > > the smart offset reset in case of unclean leader election which
>> you
>> > > > > > mentioned earlier. But in most case, unclean leader election
>> > probably
>> > > > > > happens when consumer is not rebalancing/restarting. In these
>> > cases,
>> > > > > either
>> > > > > > consumer is not directly affected by unclean leader election
>> since
>> > it
>> > > > is
>> > > > > > not consuming from the end of the log, or consumer can derive
>> the
>> > > > > > leader_epoch from the most recent message received before it
>> sees
>> > > > > > OffsetOutOfRangeException. So I am not sure it is worth adding
>> the
>> > > > > > leader_epoch to consumer API to address the remaining corner
>> case.
>> > > What
>> > > > > do
>> > > > > > you think?
>> > > > > >
>> > > > > > Thanks,
>> > > > > > Dong
>> > > > > >
>> > > > > >
>> > > > > >
>> > > > > > On Tue, Jan 2, 2018 at 6:28 PM, Jun Rao <ju...@confluent.io>
>> wrote:
>> > > > > >
>> > > > > > > Hi, Dong,
>> > > > > > >
>> > > > > > > Thanks for the reply.
>> > > > > > >
>> > > > > > > To solve the topic recreation issue, we could use either a
>> global
>> > > > > > metadata
>> > > > > > > version or a partition level epoch. But either one will be a
>> new
>> > > > > concept,
>> > > > > > > right? To me, the latter seems more natural. It also makes it
>> > > easier
>> > > > to
>> > > > > > > detect if a consumer's offset is still valid after a topic is
>> > > > > recreated.
>> > > > > > As
>> > > > > > > you pointed out, we don't need to store the partition epoch in
>> > the
>> > > > > > message.
>> > > > > > > The following is what I am thinking. When a partition is
>> created,
>> > > we
>> > > > > can
>> > > > > > > assign a partition epoch from an ever-increasing global
>> counter
>> > and
>> > > > > store
>> > > > > > > it in /brokers/topics/[topic]/partitions/[partitionId] in ZK.
>> > The
>> > > > > > > partition
>> > > > > > > epoch is propagated to every broker. The consumer will be
>> > tracking
>> > > a
>> > > > > > tuple
>> > > > > > > of <offset, leader epoch, partition epoch> for offsets. If a
>> > topic
>> > > is
>> > > > > > > recreated, it's possible that a consumer's offset and leader
>> > epoch
>> > > > > still
>> > > > > > > match that in the broker, but partition epoch won't be. In
>> this
>> > > case,
>> > > > > we
>> > > > > > > can potentially still treat the consumer's offset as out of
>> range
>> > > and
>> > > > > > reset
>> > > > > > > the offset based on the offset reset policy in the consumer.
>> This
>> > > > seems
>> > > > > > > harder to do with a global metadata version.
>> > > > > > >
>> > > > > > > Jun
>> > > > > > >
>> > > > > > >
>> > > > > > >
>> > > > > > > On Mon, Dec 25, 2017 at 6:56 AM, Dong Lin <
>> lindong28@gmail.com>
>> > > > wrote:
>> > > > > > >
>> > > > > > > > Hey Jun,
>> > > > > > > >
>> > > > > > > > This is a very good example. After thinking through this in
>> > > > detail, I
>> > > > > > > agree
>> > > > > > > > that we need to commit offset with leader epoch in order to
>> > > address
>> > > > > > this
>> > > > > > > > example.
>> > > > > > > >
>> > > > > > > > I think the remaining question is how to address the
>> scenario
>> > > that
>> > > > > the
>> > > > > > > > topic is deleted and re-created. One possible solution is to
>> > > commit
>> > > > > > > offset
>> > > > > > > > with both the leader epoch and the metadata version. The
>> logic
>> > > and
>> > > > > the
>> > > > > > > > implementation of this solution does not require a new
>> concept
>> > > > (e.g.
>> > > > > > > > partition epoch) and it does not require any change to the
>> > > message
>> > > > > > format
>> > > > > > > > or leader epoch. It also allows us to order the metadata in
>> a
>> > > > > > > > straightforward manner which may be useful in the future.
>> So it
>> > > may
>> > > > > be
>> > > > > > a
>> > > > > > > > better solution than generating a random partition epoch
>> every
>> > > time
>> > > > > we
>> > > > > > > > create a partition. Does this sound reasonable?
>> > > > > > > >
>> > > > > > > > Previously one concern with using the metadata version is
>> that
>> > > > > consumer
>> > > > > > > > will be forced to refresh metadata even if metadata version
>> is
>> > > > > > increased
>> > > > > > > > due to topics that the consumer is not interested in. Now I
>> > > > realized
>> > > > > > that
>> > > > > > > > this is probably not a problem. Currently client will
>> refresh
>> > > > > metadata
>> > > > > > > > either due to InvalidMetadataException in the response from
>> > > broker
>> > > > or
>> > > > > > due
>> > > > > > > > to metadata expiry. The addition of the metadata version
>> should
>> > > > > > increase
>> > > > > > > > the overhead of metadata refresh caused by
>> > > > InvalidMetadataException.
>> > > > > If
>> > > > > > > > client refresh metadata due to expiry and it receives a
>> > metadata
>> > > > > whose
>> > > > > > > > version is lower than the current metadata version, we can
>> > reject
>> > > > the
>> > > > > > > > metadata but still reset the metadata age, which essentially
>> > keep
>> > > > the
>> > > > > > > > existing behavior in the client.
>> > > > > > > >
>> > > > > > > > Thanks much,
>> > > > > > > > Dong
>> > > > > > > >
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>>
>
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Dong Lin <li...@gmail.com>.
Hey Jun,

Hmm... thinking about this more, I am not sure that the proposed API is
sufficient. For users that store offset externally, we probably need extra
API to return the leader_epoch and partition_epoch for all partitions that
consumers are consuming. I suppose these users currently use position() to
get the offset. Thus we probably need a new method positionWithEpoch(..) to
return <offset, partition_epoch, leader_epoch>. Does this sound reasonable?

Thanks,
Dong


On Thu, Jan 4, 2018 at 5:26 PM, Jun Rao <ju...@confluent.io> wrote:

> Hi, Dong,
>
> Yes, that's what I am thinking. OffsetEpoch will be composed of
> (partition_epoch,
> leader_epoch).
>
> Thanks,
>
> Jun
>
>
> On Thu, Jan 4, 2018 at 4:22 PM, Dong Lin <li...@gmail.com> wrote:
>
> > Hey Jun,
> >
> > Thanks much. I like the the new API that you proposed. I am not sure what
> > you exactly mean by offset_epoch. I suppose that we can use the pair of
> > (partition_epoch, leader_epoch) as the offset_epoch, right?
> >
> > Thanks,
> > Dong
> >
> > On Thu, Jan 4, 2018 at 4:02 PM, Jun Rao <ju...@confluent.io> wrote:
> >
> > > Hi, Dong,
> > >
> > > Got it. The api that you proposed works. The question is whether that's
> > the
> > > api that we want to have in the long term. My concern is that while the
> > api
> > > change is simple, the new api seems harder to explain and use. For
> > example,
> > > a consumer storing offsets externally now needs to call
> > > waitForMetadataUpdate() after calling seek().
> > >
> > > An alternative approach is to make the following compatible api changes
> > in
> > > Consumer.
> > > * Add an additional OffsetEpoch field in OffsetAndMetadata. (no need to
> > > change the CommitSync() api)
> > > * Add a new api seek(TopicPartition partition, long offset, OffsetEpoch
> > > offsetEpoch). We can potentially deprecate the old api
> > seek(TopicPartition
> > > partition, long offset) in the future.
> > >
> > > The alternative approach has similar amount of api changes as yours but
> > has
> > > the following benefits.
> > > 1. The api works in a similar way as how offset management works now
> and
> > is
> > > probably what we want in the long term.
> > > 2. It can reset offsets better when there is data loss due to unclean
> > > leader election or correlated replica failure.
> > > 3. It can reset offsets better when topic is recreated.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > >
> > > On Thu, Jan 4, 2018 at 2:05 PM, Dong Lin <li...@gmail.com> wrote:
> > >
> > > > Hey Jun,
> > > >
> > > > Yeah I agree that ideally we don't want an ever growing global
> metadata
> > > > version. I just think it may be more desirable to keep the consumer
> API
> > > > simple.
> > > >
> > > > In my current proposal, metadata version returned in the fetch
> response
> > > > will be stored with the offset together. More specifically, the
> > > > metadata_epoch in the new offset topic schema will be the largest
> > > > metadata_epoch from all the MetadataResponse and FetchResponse ever
> > > > received by this consumer.
> > > >
> > > > We probably don't have to change the consumer API for
> > > > commitSync(Map<TopicPartition, OffsetAndMetadata>). If user calls
> > > > commitSync(...) to commit offset 10 for a given partition, for most
> > > > use-cases, this consumer instance should have consumed message with
> > > offset
> > > > 9 from this partition, in which case the consumer can remember and
> use
> > > the
> > > > metadata_epoch from the corresponding FetchResponse when committing
> > > offset.
> > > > If user calls commitSync(..) to commit offset 10 for a given
> partition
> > > > without having consumed the message with offset 9 using this consumer
> > > > instance, this is probably an advanced use-case. In this case the
> > > advanced
> > > > user can retrieve the metadata_epoch using the newly added
> > > metadataEpoch()
> > > > API after it fetches the message with offset 9 (probably from another
> > > > consumer instance) and encode this metadata_epoch in the
> > > > string OffsetAndMetadata.metadata. Do you think this solution would
> > work?
> > > >
> > > > By "not sure that I fully understand your latest suggestion", are you
> > > > referring to solution related to unclean leader election using
> > > leader_epoch
> > > > in my previous email?
> > > >
> > > > Thanks,
> > > > Dong
> > > >
> > > > On Thu, Jan 4, 2018 at 1:33 PM, Jun Rao <ju...@confluent.io> wrote:
> > > >
> > > > > Hi, Dong,
> > > > >
> > > > > Not sure that I fully understand your latest suggestion. Returning
> an
> > > > ever
> > > > > growing global metadata version itself is no ideal, but is fine. My
> > > > > question is whether the metadata version returned in the fetch
> > response
> > > > > needs to be stored with the offset together if offsets are stored
> > > > > externally. If so, we also have to change the consumer API for
> > > > commitSync()
> > > > > and need to worry about compatibility. If we don't store the
> metadata
> > > > > version together with the offset, on a consumer restart, it's not
> > clear
> > > > how
> > > > > we can ensure the metadata in the consumer is high enough since
> there
> > > is
> > > > no
> > > > > metadata version to compare with.
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Jun
> > > > >
> > > > >
> > > > > On Wed, Jan 3, 2018 at 6:43 PM, Dong Lin <li...@gmail.com>
> > wrote:
> > > > >
> > > > > > Hey Jun,
> > > > > >
> > > > > > Thanks much for the explanation.
> > > > > >
> > > > > > I understand the advantage of partition_epoch over
> metadata_epoch.
> > My
> > > > > > current concern is that the use of leader_epoch and the
> > > partition_epoch
> > > > > > requires us considerable change to consumer's public API to take
> > care
> > > > of
> > > > > > the case where user stores offset externally. For example,
> > > *consumer*.
> > > > > > *commitSync*(..) would have to take a map whose value is <offset,
> > > > > metadata,
> > > > > > leader epoch, partition epoch>. *consumer*.*seek*(...) would also
> > > need
> > > > > > leader_epoch and partition_epoch as parameter. Technically we can
> > > > > probably
> > > > > > still make it work in a backward compatible manner after careful
> > > design
> > > > > and
> > > > > > discussion. But these changes can make the consumer's interface
> > > > > > unnecessarily complex for more users who do not store offset
> > > > externally.
> > > > > >
> > > > > > After thinking more about it, we can address all problems
> discussed
> > > by
> > > > > only
> > > > > > using the metadata_epoch without introducing leader_epoch or the
> > > > > > partition_epoch. The current KIP describes the changes to the
> > > consumer
> > > > > API
> > > > > > and how the new API can be used if user stores offset externally.
> > In
> > > > > order
> > > > > > to address the scenario you described earlier, we can include
> > > > > > metadata_epoch in the FetchResponse and the LeaderAndIsrRequest.
> > > > Consumer
> > > > > > remembers the largest metadata_epoch from all the FetchResponse
> it
> > > has
> > > > > > received. The metadata_epoch committed with the offset, either
> > within
> > > > or
> > > > > > outside Kafka, should be the largest metadata_epoch across all
> > > > > > FetchResponse and MetadataResponse ever received by this
> consumer.
> > > > > >
> > > > > > The drawback of using only the metadata_epoch is that we can not
> > > always
> > > > > do
> > > > > > the smart offset reset in case of unclean leader election which
> you
> > > > > > mentioned earlier. But in most case, unclean leader election
> > probably
> > > > > > happens when consumer is not rebalancing/restarting. In these
> > cases,
> > > > > either
> > > > > > consumer is not directly affected by unclean leader election
> since
> > it
> > > > is
> > > > > > not consuming from the end of the log, or consumer can derive the
> > > > > > leader_epoch from the most recent message received before it sees
> > > > > > OffsetOutOfRangeException. So I am not sure it is worth adding
> the
> > > > > > leader_epoch to consumer API to address the remaining corner
> case.
> > > What
> > > > > do
> > > > > > you think?
> > > > > >
> > > > > > Thanks,
> > > > > > Dong
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Tue, Jan 2, 2018 at 6:28 PM, Jun Rao <ju...@confluent.io>
> wrote:
> > > > > >
> > > > > > > Hi, Dong,
> > > > > > >
> > > > > > > Thanks for the reply.
> > > > > > >
> > > > > > > To solve the topic recreation issue, we could use either a
> global
> > > > > > metadata
> > > > > > > version or a partition level epoch. But either one will be a
> new
> > > > > concept,
> > > > > > > right? To me, the latter seems more natural. It also makes it
> > > easier
> > > > to
> > > > > > > detect if a consumer's offset is still valid after a topic is
> > > > > recreated.
> > > > > > As
> > > > > > > you pointed out, we don't need to store the partition epoch in
> > the
> > > > > > message.
> > > > > > > The following is what I am thinking. When a partition is
> created,
> > > we
> > > > > can
> > > > > > > assign a partition epoch from an ever-increasing global counter
> > and
> > > > > store
> > > > > > > it in /brokers/topics/[topic]/partitions/[partitionId] in ZK.
> > The
> > > > > > > partition
> > > > > > > epoch is propagated to every broker. The consumer will be
> > tracking
> > > a
> > > > > > tuple
> > > > > > > of <offset, leader epoch, partition epoch> for offsets. If a
> > topic
> > > is
> > > > > > > recreated, it's possible that a consumer's offset and leader
> > epoch
> > > > > still
> > > > > > > match that in the broker, but partition epoch won't be. In this
> > > case,
> > > > > we
> > > > > > > can potentially still treat the consumer's offset as out of
> range
> > > and
> > > > > > reset
> > > > > > > the offset based on the offset reset policy in the consumer.
> This
> > > > seems
> > > > > > > harder to do with a global metadata version.
> > > > > > >
> > > > > > > Jun
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > On Mon, Dec 25, 2017 at 6:56 AM, Dong Lin <lindong28@gmail.com
> >
> > > > wrote:
> > > > > > >
> > > > > > > > Hey Jun,
> > > > > > > >
> > > > > > > > This is a very good example. After thinking through this in
> > > > detail, I
> > > > > > > agree
> > > > > > > > that we need to commit offset with leader epoch in order to
> > > address
> > > > > > this
> > > > > > > > example.
> > > > > > > >
> > > > > > > > I think the remaining question is how to address the scenario
> > > that
> > > > > the
> > > > > > > > topic is deleted and re-created. One possible solution is to
> > > commit
> > > > > > > offset
> > > > > > > > with both the leader epoch and the metadata version. The
> logic
> > > and
> > > > > the
> > > > > > > > implementation of this solution does not require a new
> concept
> > > > (e.g.
> > > > > > > > partition epoch) and it does not require any change to the
> > > message
> > > > > > format
> > > > > > > > or leader epoch. It also allows us to order the metadata in a
> > > > > > > > straightforward manner which may be useful in the future. So
> it
> > > may
> > > > > be
> > > > > > a
> > > > > > > > better solution than generating a random partition epoch
> every
> > > time
> > > > > we
> > > > > > > > create a partition. Does this sound reasonable?
> > > > > > > >
> > > > > > > > Previously one concern with using the metadata version is
> that
> > > > > consumer
> > > > > > > > will be forced to refresh metadata even if metadata version
> is
> > > > > > increased
> > > > > > > > due to topics that the consumer is not interested in. Now I
> > > > realized
> > > > > > that
> > > > > > > > this is probably not a problem. Currently client will refresh
> > > > > metadata
> > > > > > > > either due to InvalidMetadataException in the response from
> > > broker
> > > > or
> > > > > > due
> > > > > > > > to metadata expiry. The addition of the metadata version
> should
> > > > > > increase
> > > > > > > > the overhead of metadata refresh caused by
> > > > InvalidMetadataException.
> > > > > If
> > > > > > > > client refresh metadata due to expiry and it receives a
> > metadata
> > > > > whose
> > > > > > > > version is lower than the current metadata version, we can
> > reject
> > > > the
> > > > > > > > metadata but still reset the metadata age, which essentially
> > keep
> > > > the
> > > > > > > > existing behavior in the client.
> > > > > > > >
> > > > > > > > Thanks much,
> > > > > > > > Dong
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Jun Rao <ju...@confluent.io>.
Hi, Dong,

Yes, that's what I am thinking. OffsetEpoch will be composed of
(partition_epoch,
leader_epoch).

Thanks,

Jun


On Thu, Jan 4, 2018 at 4:22 PM, Dong Lin <li...@gmail.com> wrote:

> Hey Jun,
>
> Thanks much. I like the the new API that you proposed. I am not sure what
> you exactly mean by offset_epoch. I suppose that we can use the pair of
> (partition_epoch, leader_epoch) as the offset_epoch, right?
>
> Thanks,
> Dong
>
> On Thu, Jan 4, 2018 at 4:02 PM, Jun Rao <ju...@confluent.io> wrote:
>
> > Hi, Dong,
> >
> > Got it. The api that you proposed works. The question is whether that's
> the
> > api that we want to have in the long term. My concern is that while the
> api
> > change is simple, the new api seems harder to explain and use. For
> example,
> > a consumer storing offsets externally now needs to call
> > waitForMetadataUpdate() after calling seek().
> >
> > An alternative approach is to make the following compatible api changes
> in
> > Consumer.
> > * Add an additional OffsetEpoch field in OffsetAndMetadata. (no need to
> > change the CommitSync() api)
> > * Add a new api seek(TopicPartition partition, long offset, OffsetEpoch
> > offsetEpoch). We can potentially deprecate the old api
> seek(TopicPartition
> > partition, long offset) in the future.
> >
> > The alternative approach has similar amount of api changes as yours but
> has
> > the following benefits.
> > 1. The api works in a similar way as how offset management works now and
> is
> > probably what we want in the long term.
> > 2. It can reset offsets better when there is data loss due to unclean
> > leader election or correlated replica failure.
> > 3. It can reset offsets better when topic is recreated.
> >
> > Thanks,
> >
> > Jun
> >
> >
> > On Thu, Jan 4, 2018 at 2:05 PM, Dong Lin <li...@gmail.com> wrote:
> >
> > > Hey Jun,
> > >
> > > Yeah I agree that ideally we don't want an ever growing global metadata
> > > version. I just think it may be more desirable to keep the consumer API
> > > simple.
> > >
> > > In my current proposal, metadata version returned in the fetch response
> > > will be stored with the offset together. More specifically, the
> > > metadata_epoch in the new offset topic schema will be the largest
> > > metadata_epoch from all the MetadataResponse and FetchResponse ever
> > > received by this consumer.
> > >
> > > We probably don't have to change the consumer API for
> > > commitSync(Map<TopicPartition, OffsetAndMetadata>). If user calls
> > > commitSync(...) to commit offset 10 for a given partition, for most
> > > use-cases, this consumer instance should have consumed message with
> > offset
> > > 9 from this partition, in which case the consumer can remember and use
> > the
> > > metadata_epoch from the corresponding FetchResponse when committing
> > offset.
> > > If user calls commitSync(..) to commit offset 10 for a given partition
> > > without having consumed the message with offset 9 using this consumer
> > > instance, this is probably an advanced use-case. In this case the
> > advanced
> > > user can retrieve the metadata_epoch using the newly added
> > metadataEpoch()
> > > API after it fetches the message with offset 9 (probably from another
> > > consumer instance) and encode this metadata_epoch in the
> > > string OffsetAndMetadata.metadata. Do you think this solution would
> work?
> > >
> > > By "not sure that I fully understand your latest suggestion", are you
> > > referring to solution related to unclean leader election using
> > leader_epoch
> > > in my previous email?
> > >
> > > Thanks,
> > > Dong
> > >
> > > On Thu, Jan 4, 2018 at 1:33 PM, Jun Rao <ju...@confluent.io> wrote:
> > >
> > > > Hi, Dong,
> > > >
> > > > Not sure that I fully understand your latest suggestion. Returning an
> > > ever
> > > > growing global metadata version itself is no ideal, but is fine. My
> > > > question is whether the metadata version returned in the fetch
> response
> > > > needs to be stored with the offset together if offsets are stored
> > > > externally. If so, we also have to change the consumer API for
> > > commitSync()
> > > > and need to worry about compatibility. If we don't store the metadata
> > > > version together with the offset, on a consumer restart, it's not
> clear
> > > how
> > > > we can ensure the metadata in the consumer is high enough since there
> > is
> > > no
> > > > metadata version to compare with.
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > >
> > > > On Wed, Jan 3, 2018 at 6:43 PM, Dong Lin <li...@gmail.com>
> wrote:
> > > >
> > > > > Hey Jun,
> > > > >
> > > > > Thanks much for the explanation.
> > > > >
> > > > > I understand the advantage of partition_epoch over metadata_epoch.
> My
> > > > > current concern is that the use of leader_epoch and the
> > partition_epoch
> > > > > requires us considerable change to consumer's public API to take
> care
> > > of
> > > > > the case where user stores offset externally. For example,
> > *consumer*.
> > > > > *commitSync*(..) would have to take a map whose value is <offset,
> > > > metadata,
> > > > > leader epoch, partition epoch>. *consumer*.*seek*(...) would also
> > need
> > > > > leader_epoch and partition_epoch as parameter. Technically we can
> > > > probably
> > > > > still make it work in a backward compatible manner after careful
> > design
> > > > and
> > > > > discussion. But these changes can make the consumer's interface
> > > > > unnecessarily complex for more users who do not store offset
> > > externally.
> > > > >
> > > > > After thinking more about it, we can address all problems discussed
> > by
> > > > only
> > > > > using the metadata_epoch without introducing leader_epoch or the
> > > > > partition_epoch. The current KIP describes the changes to the
> > consumer
> > > > API
> > > > > and how the new API can be used if user stores offset externally.
> In
> > > > order
> > > > > to address the scenario you described earlier, we can include
> > > > > metadata_epoch in the FetchResponse and the LeaderAndIsrRequest.
> > > Consumer
> > > > > remembers the largest metadata_epoch from all the FetchResponse it
> > has
> > > > > received. The metadata_epoch committed with the offset, either
> within
> > > or
> > > > > outside Kafka, should be the largest metadata_epoch across all
> > > > > FetchResponse and MetadataResponse ever received by this consumer.
> > > > >
> > > > > The drawback of using only the metadata_epoch is that we can not
> > always
> > > > do
> > > > > the smart offset reset in case of unclean leader election which you
> > > > > mentioned earlier. But in most case, unclean leader election
> probably
> > > > > happens when consumer is not rebalancing/restarting. In these
> cases,
> > > > either
> > > > > consumer is not directly affected by unclean leader election since
> it
> > > is
> > > > > not consuming from the end of the log, or consumer can derive the
> > > > > leader_epoch from the most recent message received before it sees
> > > > > OffsetOutOfRangeException. So I am not sure it is worth adding the
> > > > > leader_epoch to consumer API to address the remaining corner case.
> > What
> > > > do
> > > > > you think?
> > > > >
> > > > > Thanks,
> > > > > Dong
> > > > >
> > > > >
> > > > >
> > > > > On Tue, Jan 2, 2018 at 6:28 PM, Jun Rao <ju...@confluent.io> wrote:
> > > > >
> > > > > > Hi, Dong,
> > > > > >
> > > > > > Thanks for the reply.
> > > > > >
> > > > > > To solve the topic recreation issue, we could use either a global
> > > > > metadata
> > > > > > version or a partition level epoch. But either one will be a new
> > > > concept,
> > > > > > right? To me, the latter seems more natural. It also makes it
> > easier
> > > to
> > > > > > detect if a consumer's offset is still valid after a topic is
> > > > recreated.
> > > > > As
> > > > > > you pointed out, we don't need to store the partition epoch in
> the
> > > > > message.
> > > > > > The following is what I am thinking. When a partition is created,
> > we
> > > > can
> > > > > > assign a partition epoch from an ever-increasing global counter
> and
> > > > store
> > > > > > it in /brokers/topics/[topic]/partitions/[partitionId] in ZK.
> The
> > > > > > partition
> > > > > > epoch is propagated to every broker. The consumer will be
> tracking
> > a
> > > > > tuple
> > > > > > of <offset, leader epoch, partition epoch> for offsets. If a
> topic
> > is
> > > > > > recreated, it's possible that a consumer's offset and leader
> epoch
> > > > still
> > > > > > match that in the broker, but partition epoch won't be. In this
> > case,
> > > > we
> > > > > > can potentially still treat the consumer's offset as out of range
> > and
> > > > > reset
> > > > > > the offset based on the offset reset policy in the consumer. This
> > > seems
> > > > > > harder to do with a global metadata version.
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Mon, Dec 25, 2017 at 6:56 AM, Dong Lin <li...@gmail.com>
> > > wrote:
> > > > > >
> > > > > > > Hey Jun,
> > > > > > >
> > > > > > > This is a very good example. After thinking through this in
> > > detail, I
> > > > > > agree
> > > > > > > that we need to commit offset with leader epoch in order to
> > address
> > > > > this
> > > > > > > example.
> > > > > > >
> > > > > > > I think the remaining question is how to address the scenario
> > that
> > > > the
> > > > > > > topic is deleted and re-created. One possible solution is to
> > commit
> > > > > > offset
> > > > > > > with both the leader epoch and the metadata version. The logic
> > and
> > > > the
> > > > > > > implementation of this solution does not require a new concept
> > > (e.g.
> > > > > > > partition epoch) and it does not require any change to the
> > message
> > > > > format
> > > > > > > or leader epoch. It also allows us to order the metadata in a
> > > > > > > straightforward manner which may be useful in the future. So it
> > may
> > > > be
> > > > > a
> > > > > > > better solution than generating a random partition epoch every
> > time
> > > > we
> > > > > > > create a partition. Does this sound reasonable?
> > > > > > >
> > > > > > > Previously one concern with using the metadata version is that
> > > > consumer
> > > > > > > will be forced to refresh metadata even if metadata version is
> > > > > increased
> > > > > > > due to topics that the consumer is not interested in. Now I
> > > realized
> > > > > that
> > > > > > > this is probably not a problem. Currently client will refresh
> > > > metadata
> > > > > > > either due to InvalidMetadataException in the response from
> > broker
> > > or
> > > > > due
> > > > > > > to metadata expiry. The addition of the metadata version should
> > > > > increase
> > > > > > > the overhead of metadata refresh caused by
> > > InvalidMetadataException.
> > > > If
> > > > > > > client refresh metadata due to expiry and it receives a
> metadata
> > > > whose
> > > > > > > version is lower than the current metadata version, we can
> reject
> > > the
> > > > > > > metadata but still reset the metadata age, which essentially
> keep
> > > the
> > > > > > > existing behavior in the client.
> > > > > > >
> > > > > > > Thanks much,
> > > > > > > Dong
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Dong Lin <li...@gmail.com>.
Hey Jun,

Thanks much. I like the the new API that you proposed. I am not sure what
you exactly mean by offset_epoch. I suppose that we can use the pair of
(partition_epoch, leader_epoch) as the offset_epoch, right?

Thanks,
Dong

On Thu, Jan 4, 2018 at 4:02 PM, Jun Rao <ju...@confluent.io> wrote:

> Hi, Dong,
>
> Got it. The api that you proposed works. The question is whether that's the
> api that we want to have in the long term. My concern is that while the api
> change is simple, the new api seems harder to explain and use. For example,
> a consumer storing offsets externally now needs to call
> waitForMetadataUpdate() after calling seek().
>
> An alternative approach is to make the following compatible api changes in
> Consumer.
> * Add an additional OffsetEpoch field in OffsetAndMetadata. (no need to
> change the CommitSync() api)
> * Add a new api seek(TopicPartition partition, long offset, OffsetEpoch
> offsetEpoch). We can potentially deprecate the old api seek(TopicPartition
> partition, long offset) in the future.
>
> The alternative approach has similar amount of api changes as yours but has
> the following benefits.
> 1. The api works in a similar way as how offset management works now and is
> probably what we want in the long term.
> 2. It can reset offsets better when there is data loss due to unclean
> leader election or correlated replica failure.
> 3. It can reset offsets better when topic is recreated.
>
> Thanks,
>
> Jun
>
>
> On Thu, Jan 4, 2018 at 2:05 PM, Dong Lin <li...@gmail.com> wrote:
>
> > Hey Jun,
> >
> > Yeah I agree that ideally we don't want an ever growing global metadata
> > version. I just think it may be more desirable to keep the consumer API
> > simple.
> >
> > In my current proposal, metadata version returned in the fetch response
> > will be stored with the offset together. More specifically, the
> > metadata_epoch in the new offset topic schema will be the largest
> > metadata_epoch from all the MetadataResponse and FetchResponse ever
> > received by this consumer.
> >
> > We probably don't have to change the consumer API for
> > commitSync(Map<TopicPartition, OffsetAndMetadata>). If user calls
> > commitSync(...) to commit offset 10 for a given partition, for most
> > use-cases, this consumer instance should have consumed message with
> offset
> > 9 from this partition, in which case the consumer can remember and use
> the
> > metadata_epoch from the corresponding FetchResponse when committing
> offset.
> > If user calls commitSync(..) to commit offset 10 for a given partition
> > without having consumed the message with offset 9 using this consumer
> > instance, this is probably an advanced use-case. In this case the
> advanced
> > user can retrieve the metadata_epoch using the newly added
> metadataEpoch()
> > API after it fetches the message with offset 9 (probably from another
> > consumer instance) and encode this metadata_epoch in the
> > string OffsetAndMetadata.metadata. Do you think this solution would work?
> >
> > By "not sure that I fully understand your latest suggestion", are you
> > referring to solution related to unclean leader election using
> leader_epoch
> > in my previous email?
> >
> > Thanks,
> > Dong
> >
> > On Thu, Jan 4, 2018 at 1:33 PM, Jun Rao <ju...@confluent.io> wrote:
> >
> > > Hi, Dong,
> > >
> > > Not sure that I fully understand your latest suggestion. Returning an
> > ever
> > > growing global metadata version itself is no ideal, but is fine. My
> > > question is whether the metadata version returned in the fetch response
> > > needs to be stored with the offset together if offsets are stored
> > > externally. If so, we also have to change the consumer API for
> > commitSync()
> > > and need to worry about compatibility. If we don't store the metadata
> > > version together with the offset, on a consumer restart, it's not clear
> > how
> > > we can ensure the metadata in the consumer is high enough since there
> is
> > no
> > > metadata version to compare with.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > >
> > > On Wed, Jan 3, 2018 at 6:43 PM, Dong Lin <li...@gmail.com> wrote:
> > >
> > > > Hey Jun,
> > > >
> > > > Thanks much for the explanation.
> > > >
> > > > I understand the advantage of partition_epoch over metadata_epoch. My
> > > > current concern is that the use of leader_epoch and the
> partition_epoch
> > > > requires us considerable change to consumer's public API to take care
> > of
> > > > the case where user stores offset externally. For example,
> *consumer*.
> > > > *commitSync*(..) would have to take a map whose value is <offset,
> > > metadata,
> > > > leader epoch, partition epoch>. *consumer*.*seek*(...) would also
> need
> > > > leader_epoch and partition_epoch as parameter. Technically we can
> > > probably
> > > > still make it work in a backward compatible manner after careful
> design
> > > and
> > > > discussion. But these changes can make the consumer's interface
> > > > unnecessarily complex for more users who do not store offset
> > externally.
> > > >
> > > > After thinking more about it, we can address all problems discussed
> by
> > > only
> > > > using the metadata_epoch without introducing leader_epoch or the
> > > > partition_epoch. The current KIP describes the changes to the
> consumer
> > > API
> > > > and how the new API can be used if user stores offset externally. In
> > > order
> > > > to address the scenario you described earlier, we can include
> > > > metadata_epoch in the FetchResponse and the LeaderAndIsrRequest.
> > Consumer
> > > > remembers the largest metadata_epoch from all the FetchResponse it
> has
> > > > received. The metadata_epoch committed with the offset, either within
> > or
> > > > outside Kafka, should be the largest metadata_epoch across all
> > > > FetchResponse and MetadataResponse ever received by this consumer.
> > > >
> > > > The drawback of using only the metadata_epoch is that we can not
> always
> > > do
> > > > the smart offset reset in case of unclean leader election which you
> > > > mentioned earlier. But in most case, unclean leader election probably
> > > > happens when consumer is not rebalancing/restarting. In these cases,
> > > either
> > > > consumer is not directly affected by unclean leader election since it
> > is
> > > > not consuming from the end of the log, or consumer can derive the
> > > > leader_epoch from the most recent message received before it sees
> > > > OffsetOutOfRangeException. So I am not sure it is worth adding the
> > > > leader_epoch to consumer API to address the remaining corner case.
> What
> > > do
> > > > you think?
> > > >
> > > > Thanks,
> > > > Dong
> > > >
> > > >
> > > >
> > > > On Tue, Jan 2, 2018 at 6:28 PM, Jun Rao <ju...@confluent.io> wrote:
> > > >
> > > > > Hi, Dong,
> > > > >
> > > > > Thanks for the reply.
> > > > >
> > > > > To solve the topic recreation issue, we could use either a global
> > > > metadata
> > > > > version or a partition level epoch. But either one will be a new
> > > concept,
> > > > > right? To me, the latter seems more natural. It also makes it
> easier
> > to
> > > > > detect if a consumer's offset is still valid after a topic is
> > > recreated.
> > > > As
> > > > > you pointed out, we don't need to store the partition epoch in the
> > > > message.
> > > > > The following is what I am thinking. When a partition is created,
> we
> > > can
> > > > > assign a partition epoch from an ever-increasing global counter and
> > > store
> > > > > it in /brokers/topics/[topic]/partitions/[partitionId] in ZK. The
> > > > > partition
> > > > > epoch is propagated to every broker. The consumer will be tracking
> a
> > > > tuple
> > > > > of <offset, leader epoch, partition epoch> for offsets. If a topic
> is
> > > > > recreated, it's possible that a consumer's offset and leader epoch
> > > still
> > > > > match that in the broker, but partition epoch won't be. In this
> case,
> > > we
> > > > > can potentially still treat the consumer's offset as out of range
> and
> > > > reset
> > > > > the offset based on the offset reset policy in the consumer. This
> > seems
> > > > > harder to do with a global metadata version.
> > > > >
> > > > > Jun
> > > > >
> > > > >
> > > > >
> > > > > On Mon, Dec 25, 2017 at 6:56 AM, Dong Lin <li...@gmail.com>
> > wrote:
> > > > >
> > > > > > Hey Jun,
> > > > > >
> > > > > > This is a very good example. After thinking through this in
> > detail, I
> > > > > agree
> > > > > > that we need to commit offset with leader epoch in order to
> address
> > > > this
> > > > > > example.
> > > > > >
> > > > > > I think the remaining question is how to address the scenario
> that
> > > the
> > > > > > topic is deleted and re-created. One possible solution is to
> commit
> > > > > offset
> > > > > > with both the leader epoch and the metadata version. The logic
> and
> > > the
> > > > > > implementation of this solution does not require a new concept
> > (e.g.
> > > > > > partition epoch) and it does not require any change to the
> message
> > > > format
> > > > > > or leader epoch. It also allows us to order the metadata in a
> > > > > > straightforward manner which may be useful in the future. So it
> may
> > > be
> > > > a
> > > > > > better solution than generating a random partition epoch every
> time
> > > we
> > > > > > create a partition. Does this sound reasonable?
> > > > > >
> > > > > > Previously one concern with using the metadata version is that
> > > consumer
> > > > > > will be forced to refresh metadata even if metadata version is
> > > > increased
> > > > > > due to topics that the consumer is not interested in. Now I
> > realized
> > > > that
> > > > > > this is probably not a problem. Currently client will refresh
> > > metadata
> > > > > > either due to InvalidMetadataException in the response from
> broker
> > or
> > > > due
> > > > > > to metadata expiry. The addition of the metadata version should
> > > > increase
> > > > > > the overhead of metadata refresh caused by
> > InvalidMetadataException.
> > > If
> > > > > > client refresh metadata due to expiry and it receives a metadata
> > > whose
> > > > > > version is lower than the current metadata version, we can reject
> > the
> > > > > > metadata but still reset the metadata age, which essentially keep
> > the
> > > > > > existing behavior in the client.
> > > > > >
> > > > > > Thanks much,
> > > > > > Dong
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Jun Rao <ju...@confluent.io>.
Hi, Dong,

Got it. The api that you proposed works. The question is whether that's the
api that we want to have in the long term. My concern is that while the api
change is simple, the new api seems harder to explain and use. For example,
a consumer storing offsets externally now needs to call
waitForMetadataUpdate() after calling seek().

An alternative approach is to make the following compatible api changes in
Consumer.
* Add an additional OffsetEpoch field in OffsetAndMetadata. (no need to
change the CommitSync() api)
* Add a new api seek(TopicPartition partition, long offset, OffsetEpoch
offsetEpoch). We can potentially deprecate the old api seek(TopicPartition
partition, long offset) in the future.

The alternative approach has similar amount of api changes as yours but has
the following benefits.
1. The api works in a similar way as how offset management works now and is
probably what we want in the long term.
2. It can reset offsets better when there is data loss due to unclean
leader election or correlated replica failure.
3. It can reset offsets better when topic is recreated.

Thanks,

Jun


On Thu, Jan 4, 2018 at 2:05 PM, Dong Lin <li...@gmail.com> wrote:

> Hey Jun,
>
> Yeah I agree that ideally we don't want an ever growing global metadata
> version. I just think it may be more desirable to keep the consumer API
> simple.
>
> In my current proposal, metadata version returned in the fetch response
> will be stored with the offset together. More specifically, the
> metadata_epoch in the new offset topic schema will be the largest
> metadata_epoch from all the MetadataResponse and FetchResponse ever
> received by this consumer.
>
> We probably don't have to change the consumer API for
> commitSync(Map<TopicPartition, OffsetAndMetadata>). If user calls
> commitSync(...) to commit offset 10 for a given partition, for most
> use-cases, this consumer instance should have consumed message with offset
> 9 from this partition, in which case the consumer can remember and use the
> metadata_epoch from the corresponding FetchResponse when committing offset.
> If user calls commitSync(..) to commit offset 10 for a given partition
> without having consumed the message with offset 9 using this consumer
> instance, this is probably an advanced use-case. In this case the advanced
> user can retrieve the metadata_epoch using the newly added metadataEpoch()
> API after it fetches the message with offset 9 (probably from another
> consumer instance) and encode this metadata_epoch in the
> string OffsetAndMetadata.metadata. Do you think this solution would work?
>
> By "not sure that I fully understand your latest suggestion", are you
> referring to solution related to unclean leader election using leader_epoch
> in my previous email?
>
> Thanks,
> Dong
>
> On Thu, Jan 4, 2018 at 1:33 PM, Jun Rao <ju...@confluent.io> wrote:
>
> > Hi, Dong,
> >
> > Not sure that I fully understand your latest suggestion. Returning an
> ever
> > growing global metadata version itself is no ideal, but is fine. My
> > question is whether the metadata version returned in the fetch response
> > needs to be stored with the offset together if offsets are stored
> > externally. If so, we also have to change the consumer API for
> commitSync()
> > and need to worry about compatibility. If we don't store the metadata
> > version together with the offset, on a consumer restart, it's not clear
> how
> > we can ensure the metadata in the consumer is high enough since there is
> no
> > metadata version to compare with.
> >
> > Thanks,
> >
> > Jun
> >
> >
> > On Wed, Jan 3, 2018 at 6:43 PM, Dong Lin <li...@gmail.com> wrote:
> >
> > > Hey Jun,
> > >
> > > Thanks much for the explanation.
> > >
> > > I understand the advantage of partition_epoch over metadata_epoch. My
> > > current concern is that the use of leader_epoch and the partition_epoch
> > > requires us considerable change to consumer's public API to take care
> of
> > > the case where user stores offset externally. For example, *consumer*.
> > > *commitSync*(..) would have to take a map whose value is <offset,
> > metadata,
> > > leader epoch, partition epoch>. *consumer*.*seek*(...) would also need
> > > leader_epoch and partition_epoch as parameter. Technically we can
> > probably
> > > still make it work in a backward compatible manner after careful design
> > and
> > > discussion. But these changes can make the consumer's interface
> > > unnecessarily complex for more users who do not store offset
> externally.
> > >
> > > After thinking more about it, we can address all problems discussed by
> > only
> > > using the metadata_epoch without introducing leader_epoch or the
> > > partition_epoch. The current KIP describes the changes to the consumer
> > API
> > > and how the new API can be used if user stores offset externally. In
> > order
> > > to address the scenario you described earlier, we can include
> > > metadata_epoch in the FetchResponse and the LeaderAndIsrRequest.
> Consumer
> > > remembers the largest metadata_epoch from all the FetchResponse it has
> > > received. The metadata_epoch committed with the offset, either within
> or
> > > outside Kafka, should be the largest metadata_epoch across all
> > > FetchResponse and MetadataResponse ever received by this consumer.
> > >
> > > The drawback of using only the metadata_epoch is that we can not always
> > do
> > > the smart offset reset in case of unclean leader election which you
> > > mentioned earlier. But in most case, unclean leader election probably
> > > happens when consumer is not rebalancing/restarting. In these cases,
> > either
> > > consumer is not directly affected by unclean leader election since it
> is
> > > not consuming from the end of the log, or consumer can derive the
> > > leader_epoch from the most recent message received before it sees
> > > OffsetOutOfRangeException. So I am not sure it is worth adding the
> > > leader_epoch to consumer API to address the remaining corner case. What
> > do
> > > you think?
> > >
> > > Thanks,
> > > Dong
> > >
> > >
> > >
> > > On Tue, Jan 2, 2018 at 6:28 PM, Jun Rao <ju...@confluent.io> wrote:
> > >
> > > > Hi, Dong,
> > > >
> > > > Thanks for the reply.
> > > >
> > > > To solve the topic recreation issue, we could use either a global
> > > metadata
> > > > version or a partition level epoch. But either one will be a new
> > concept,
> > > > right? To me, the latter seems more natural. It also makes it easier
> to
> > > > detect if a consumer's offset is still valid after a topic is
> > recreated.
> > > As
> > > > you pointed out, we don't need to store the partition epoch in the
> > > message.
> > > > The following is what I am thinking. When a partition is created, we
> > can
> > > > assign a partition epoch from an ever-increasing global counter and
> > store
> > > > it in /brokers/topics/[topic]/partitions/[partitionId] in ZK. The
> > > > partition
> > > > epoch is propagated to every broker. The consumer will be tracking a
> > > tuple
> > > > of <offset, leader epoch, partition epoch> for offsets. If a topic is
> > > > recreated, it's possible that a consumer's offset and leader epoch
> > still
> > > > match that in the broker, but partition epoch won't be. In this case,
> > we
> > > > can potentially still treat the consumer's offset as out of range and
> > > reset
> > > > the offset based on the offset reset policy in the consumer. This
> seems
> > > > harder to do with a global metadata version.
> > > >
> > > > Jun
> > > >
> > > >
> > > >
> > > > On Mon, Dec 25, 2017 at 6:56 AM, Dong Lin <li...@gmail.com>
> wrote:
> > > >
> > > > > Hey Jun,
> > > > >
> > > > > This is a very good example. After thinking through this in
> detail, I
> > > > agree
> > > > > that we need to commit offset with leader epoch in order to address
> > > this
> > > > > example.
> > > > >
> > > > > I think the remaining question is how to address the scenario that
> > the
> > > > > topic is deleted and re-created. One possible solution is to commit
> > > > offset
> > > > > with both the leader epoch and the metadata version. The logic and
> > the
> > > > > implementation of this solution does not require a new concept
> (e.g.
> > > > > partition epoch) and it does not require any change to the message
> > > format
> > > > > or leader epoch. It also allows us to order the metadata in a
> > > > > straightforward manner which may be useful in the future. So it may
> > be
> > > a
> > > > > better solution than generating a random partition epoch every time
> > we
> > > > > create a partition. Does this sound reasonable?
> > > > >
> > > > > Previously one concern with using the metadata version is that
> > consumer
> > > > > will be forced to refresh metadata even if metadata version is
> > > increased
> > > > > due to topics that the consumer is not interested in. Now I
> realized
> > > that
> > > > > this is probably not a problem. Currently client will refresh
> > metadata
> > > > > either due to InvalidMetadataException in the response from broker
> or
> > > due
> > > > > to metadata expiry. The addition of the metadata version should
> > > increase
> > > > > the overhead of metadata refresh caused by
> InvalidMetadataException.
> > If
> > > > > client refresh metadata due to expiry and it receives a metadata
> > whose
> > > > > version is lower than the current metadata version, we can reject
> the
> > > > > metadata but still reset the metadata age, which essentially keep
> the
> > > > > existing behavior in the client.
> > > > >
> > > > > Thanks much,
> > > > > Dong
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Dong Lin <li...@gmail.com>.
Hey Jun,

Yeah I agree that ideally we don't want an ever growing global metadata
version. I just think it may be more desirable to keep the consumer API
simple.

In my current proposal, metadata version returned in the fetch response
will be stored with the offset together. More specifically, the
metadata_epoch in the new offset topic schema will be the largest
metadata_epoch from all the MetadataResponse and FetchResponse ever
received by this consumer.

We probably don't have to change the consumer API for
commitSync(Map<TopicPartition, OffsetAndMetadata>). If user calls
commitSync(...) to commit offset 10 for a given partition, for most
use-cases, this consumer instance should have consumed message with offset
9 from this partition, in which case the consumer can remember and use the
metadata_epoch from the corresponding FetchResponse when committing offset.
If user calls commitSync(..) to commit offset 10 for a given partition
without having consumed the message with offset 9 using this consumer
instance, this is probably an advanced use-case. In this case the advanced
user can retrieve the metadata_epoch using the newly added metadataEpoch()
API after it fetches the message with offset 9 (probably from another
consumer instance) and encode this metadata_epoch in the
string OffsetAndMetadata.metadata. Do you think this solution would work?

By "not sure that I fully understand your latest suggestion", are you
referring to solution related to unclean leader election using leader_epoch
in my previous email?

Thanks,
Dong

On Thu, Jan 4, 2018 at 1:33 PM, Jun Rao <ju...@confluent.io> wrote:

> Hi, Dong,
>
> Not sure that I fully understand your latest suggestion. Returning an ever
> growing global metadata version itself is no ideal, but is fine. My
> question is whether the metadata version returned in the fetch response
> needs to be stored with the offset together if offsets are stored
> externally. If so, we also have to change the consumer API for commitSync()
> and need to worry about compatibility. If we don't store the metadata
> version together with the offset, on a consumer restart, it's not clear how
> we can ensure the metadata in the consumer is high enough since there is no
> metadata version to compare with.
>
> Thanks,
>
> Jun
>
>
> On Wed, Jan 3, 2018 at 6:43 PM, Dong Lin <li...@gmail.com> wrote:
>
> > Hey Jun,
> >
> > Thanks much for the explanation.
> >
> > I understand the advantage of partition_epoch over metadata_epoch. My
> > current concern is that the use of leader_epoch and the partition_epoch
> > requires us considerable change to consumer's public API to take care of
> > the case where user stores offset externally. For example, *consumer*.
> > *commitSync*(..) would have to take a map whose value is <offset,
> metadata,
> > leader epoch, partition epoch>. *consumer*.*seek*(...) would also need
> > leader_epoch and partition_epoch as parameter. Technically we can
> probably
> > still make it work in a backward compatible manner after careful design
> and
> > discussion. But these changes can make the consumer's interface
> > unnecessarily complex for more users who do not store offset externally.
> >
> > After thinking more about it, we can address all problems discussed by
> only
> > using the metadata_epoch without introducing leader_epoch or the
> > partition_epoch. The current KIP describes the changes to the consumer
> API
> > and how the new API can be used if user stores offset externally. In
> order
> > to address the scenario you described earlier, we can include
> > metadata_epoch in the FetchResponse and the LeaderAndIsrRequest. Consumer
> > remembers the largest metadata_epoch from all the FetchResponse it has
> > received. The metadata_epoch committed with the offset, either within or
> > outside Kafka, should be the largest metadata_epoch across all
> > FetchResponse and MetadataResponse ever received by this consumer.
> >
> > The drawback of using only the metadata_epoch is that we can not always
> do
> > the smart offset reset in case of unclean leader election which you
> > mentioned earlier. But in most case, unclean leader election probably
> > happens when consumer is not rebalancing/restarting. In these cases,
> either
> > consumer is not directly affected by unclean leader election since it is
> > not consuming from the end of the log, or consumer can derive the
> > leader_epoch from the most recent message received before it sees
> > OffsetOutOfRangeException. So I am not sure it is worth adding the
> > leader_epoch to consumer API to address the remaining corner case. What
> do
> > you think?
> >
> > Thanks,
> > Dong
> >
> >
> >
> > On Tue, Jan 2, 2018 at 6:28 PM, Jun Rao <ju...@confluent.io> wrote:
> >
> > > Hi, Dong,
> > >
> > > Thanks for the reply.
> > >
> > > To solve the topic recreation issue, we could use either a global
> > metadata
> > > version or a partition level epoch. But either one will be a new
> concept,
> > > right? To me, the latter seems more natural. It also makes it easier to
> > > detect if a consumer's offset is still valid after a topic is
> recreated.
> > As
> > > you pointed out, we don't need to store the partition epoch in the
> > message.
> > > The following is what I am thinking. When a partition is created, we
> can
> > > assign a partition epoch from an ever-increasing global counter and
> store
> > > it in /brokers/topics/[topic]/partitions/[partitionId] in ZK. The
> > > partition
> > > epoch is propagated to every broker. The consumer will be tracking a
> > tuple
> > > of <offset, leader epoch, partition epoch> for offsets. If a topic is
> > > recreated, it's possible that a consumer's offset and leader epoch
> still
> > > match that in the broker, but partition epoch won't be. In this case,
> we
> > > can potentially still treat the consumer's offset as out of range and
> > reset
> > > the offset based on the offset reset policy in the consumer. This seems
> > > harder to do with a global metadata version.
> > >
> > > Jun
> > >
> > >
> > >
> > > On Mon, Dec 25, 2017 at 6:56 AM, Dong Lin <li...@gmail.com> wrote:
> > >
> > > > Hey Jun,
> > > >
> > > > This is a very good example. After thinking through this in detail, I
> > > agree
> > > > that we need to commit offset with leader epoch in order to address
> > this
> > > > example.
> > > >
> > > > I think the remaining question is how to address the scenario that
> the
> > > > topic is deleted and re-created. One possible solution is to commit
> > > offset
> > > > with both the leader epoch and the metadata version. The logic and
> the
> > > > implementation of this solution does not require a new concept (e.g.
> > > > partition epoch) and it does not require any change to the message
> > format
> > > > or leader epoch. It also allows us to order the metadata in a
> > > > straightforward manner which may be useful in the future. So it may
> be
> > a
> > > > better solution than generating a random partition epoch every time
> we
> > > > create a partition. Does this sound reasonable?
> > > >
> > > > Previously one concern with using the metadata version is that
> consumer
> > > > will be forced to refresh metadata even if metadata version is
> > increased
> > > > due to topics that the consumer is not interested in. Now I realized
> > that
> > > > this is probably not a problem. Currently client will refresh
> metadata
> > > > either due to InvalidMetadataException in the response from broker or
> > due
> > > > to metadata expiry. The addition of the metadata version should
> > increase
> > > > the overhead of metadata refresh caused by InvalidMetadataException.
> If
> > > > client refresh metadata due to expiry and it receives a metadata
> whose
> > > > version is lower than the current metadata version, we can reject the
> > > > metadata but still reset the metadata age, which essentially keep the
> > > > existing behavior in the client.
> > > >
> > > > Thanks much,
> > > > Dong
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Jun Rao <ju...@confluent.io>.
Hi, Dong,

Not sure that I fully understand your latest suggestion. Returning an ever
growing global metadata version itself is no ideal, but is fine. My
question is whether the metadata version returned in the fetch response
needs to be stored with the offset together if offsets are stored
externally. If so, we also have to change the consumer API for commitSync()
and need to worry about compatibility. If we don't store the metadata
version together with the offset, on a consumer restart, it's not clear how
we can ensure the metadata in the consumer is high enough since there is no
metadata version to compare with.

Thanks,

Jun


On Wed, Jan 3, 2018 at 6:43 PM, Dong Lin <li...@gmail.com> wrote:

> Hey Jun,
>
> Thanks much for the explanation.
>
> I understand the advantage of partition_epoch over metadata_epoch. My
> current concern is that the use of leader_epoch and the partition_epoch
> requires us considerable change to consumer's public API to take care of
> the case where user stores offset externally. For example, *consumer*.
> *commitSync*(..) would have to take a map whose value is <offset, metadata,
> leader epoch, partition epoch>. *consumer*.*seek*(...) would also need
> leader_epoch and partition_epoch as parameter. Technically we can probably
> still make it work in a backward compatible manner after careful design and
> discussion. But these changes can make the consumer's interface
> unnecessarily complex for more users who do not store offset externally.
>
> After thinking more about it, we can address all problems discussed by only
> using the metadata_epoch without introducing leader_epoch or the
> partition_epoch. The current KIP describes the changes to the consumer API
> and how the new API can be used if user stores offset externally. In order
> to address the scenario you described earlier, we can include
> metadata_epoch in the FetchResponse and the LeaderAndIsrRequest. Consumer
> remembers the largest metadata_epoch from all the FetchResponse it has
> received. The metadata_epoch committed with the offset, either within or
> outside Kafka, should be the largest metadata_epoch across all
> FetchResponse and MetadataResponse ever received by this consumer.
>
> The drawback of using only the metadata_epoch is that we can not always do
> the smart offset reset in case of unclean leader election which you
> mentioned earlier. But in most case, unclean leader election probably
> happens when consumer is not rebalancing/restarting. In these cases, either
> consumer is not directly affected by unclean leader election since it is
> not consuming from the end of the log, or consumer can derive the
> leader_epoch from the most recent message received before it sees
> OffsetOutOfRangeException. So I am not sure it is worth adding the
> leader_epoch to consumer API to address the remaining corner case. What do
> you think?
>
> Thanks,
> Dong
>
>
>
> On Tue, Jan 2, 2018 at 6:28 PM, Jun Rao <ju...@confluent.io> wrote:
>
> > Hi, Dong,
> >
> > Thanks for the reply.
> >
> > To solve the topic recreation issue, we could use either a global
> metadata
> > version or a partition level epoch. But either one will be a new concept,
> > right? To me, the latter seems more natural. It also makes it easier to
> > detect if a consumer's offset is still valid after a topic is recreated.
> As
> > you pointed out, we don't need to store the partition epoch in the
> message.
> > The following is what I am thinking. When a partition is created, we can
> > assign a partition epoch from an ever-increasing global counter and store
> > it in /brokers/topics/[topic]/partitions/[partitionId] in ZK. The
> > partition
> > epoch is propagated to every broker. The consumer will be tracking a
> tuple
> > of <offset, leader epoch, partition epoch> for offsets. If a topic is
> > recreated, it's possible that a consumer's offset and leader epoch still
> > match that in the broker, but partition epoch won't be. In this case, we
> > can potentially still treat the consumer's offset as out of range and
> reset
> > the offset based on the offset reset policy in the consumer. This seems
> > harder to do with a global metadata version.
> >
> > Jun
> >
> >
> >
> > On Mon, Dec 25, 2017 at 6:56 AM, Dong Lin <li...@gmail.com> wrote:
> >
> > > Hey Jun,
> > >
> > > This is a very good example. After thinking through this in detail, I
> > agree
> > > that we need to commit offset with leader epoch in order to address
> this
> > > example.
> > >
> > > I think the remaining question is how to address the scenario that the
> > > topic is deleted and re-created. One possible solution is to commit
> > offset
> > > with both the leader epoch and the metadata version. The logic and the
> > > implementation of this solution does not require a new concept (e.g.
> > > partition epoch) and it does not require any change to the message
> format
> > > or leader epoch. It also allows us to order the metadata in a
> > > straightforward manner which may be useful in the future. So it may be
> a
> > > better solution than generating a random partition epoch every time we
> > > create a partition. Does this sound reasonable?
> > >
> > > Previously one concern with using the metadata version is that consumer
> > > will be forced to refresh metadata even if metadata version is
> increased
> > > due to topics that the consumer is not interested in. Now I realized
> that
> > > this is probably not a problem. Currently client will refresh metadata
> > > either due to InvalidMetadataException in the response from broker or
> due
> > > to metadata expiry. The addition of the metadata version should
> increase
> > > the overhead of metadata refresh caused by InvalidMetadataException. If
> > > client refresh metadata due to expiry and it receives a metadata whose
> > > version is lower than the current metadata version, we can reject the
> > > metadata but still reset the metadata age, which essentially keep the
> > > existing behavior in the client.
> > >
> > > Thanks much,
> > > Dong
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Dong Lin <li...@gmail.com>.
Hey Jun,

Thanks much for the explanation.

I understand the advantage of partition_epoch over metadata_epoch. My
current concern is that the use of leader_epoch and the partition_epoch
requires us considerable change to consumer's public API to take care of
the case where user stores offset externally. For example, *consumer*.
*commitSync*(..) would have to take a map whose value is <offset, metadata,
leader epoch, partition epoch>. *consumer*.*seek*(...) would also need
leader_epoch and partition_epoch as parameter. Technically we can probably
still make it work in a backward compatible manner after careful design and
discussion. But these changes can make the consumer's interface
unnecessarily complex for more users who do not store offset externally.

After thinking more about it, we can address all problems discussed by only
using the metadata_epoch without introducing leader_epoch or the
partition_epoch. The current KIP describes the changes to the consumer API
and how the new API can be used if user stores offset externally. In order
to address the scenario you described earlier, we can include
metadata_epoch in the FetchResponse and the LeaderAndIsrRequest. Consumer
remembers the largest metadata_epoch from all the FetchResponse it has
received. The metadata_epoch committed with the offset, either within or
outside Kafka, should be the largest metadata_epoch across all
FetchResponse and MetadataResponse ever received by this consumer.

The drawback of using only the metadata_epoch is that we can not always do
the smart offset reset in case of unclean leader election which you
mentioned earlier. But in most case, unclean leader election probably
happens when consumer is not rebalancing/restarting. In these cases, either
consumer is not directly affected by unclean leader election since it is
not consuming from the end of the log, or consumer can derive the
leader_epoch from the most recent message received before it sees
OffsetOutOfRangeException. So I am not sure it is worth adding the
leader_epoch to consumer API to address the remaining corner case. What do
you think?

Thanks,
Dong



On Tue, Jan 2, 2018 at 6:28 PM, Jun Rao <ju...@confluent.io> wrote:

> Hi, Dong,
>
> Thanks for the reply.
>
> To solve the topic recreation issue, we could use either a global metadata
> version or a partition level epoch. But either one will be a new concept,
> right? To me, the latter seems more natural. It also makes it easier to
> detect if a consumer's offset is still valid after a topic is recreated. As
> you pointed out, we don't need to store the partition epoch in the message.
> The following is what I am thinking. When a partition is created, we can
> assign a partition epoch from an ever-increasing global counter and store
> it in /brokers/topics/[topic]/partitions/[partitionId] in ZK. The
> partition
> epoch is propagated to every broker. The consumer will be tracking a tuple
> of <offset, leader epoch, partition epoch> for offsets. If a topic is
> recreated, it's possible that a consumer's offset and leader epoch still
> match that in the broker, but partition epoch won't be. In this case, we
> can potentially still treat the consumer's offset as out of range and reset
> the offset based on the offset reset policy in the consumer. This seems
> harder to do with a global metadata version.
>
> Jun
>
>
>
> On Mon, Dec 25, 2017 at 6:56 AM, Dong Lin <li...@gmail.com> wrote:
>
> > Hey Jun,
> >
> > This is a very good example. After thinking through this in detail, I
> agree
> > that we need to commit offset with leader epoch in order to address this
> > example.
> >
> > I think the remaining question is how to address the scenario that the
> > topic is deleted and re-created. One possible solution is to commit
> offset
> > with both the leader epoch and the metadata version. The logic and the
> > implementation of this solution does not require a new concept (e.g.
> > partition epoch) and it does not require any change to the message format
> > or leader epoch. It also allows us to order the metadata in a
> > straightforward manner which may be useful in the future. So it may be a
> > better solution than generating a random partition epoch every time we
> > create a partition. Does this sound reasonable?
> >
> > Previously one concern with using the metadata version is that consumer
> > will be forced to refresh metadata even if metadata version is increased
> > due to topics that the consumer is not interested in. Now I realized that
> > this is probably not a problem. Currently client will refresh metadata
> > either due to InvalidMetadataException in the response from broker or due
> > to metadata expiry. The addition of the metadata version should increase
> > the overhead of metadata refresh caused by InvalidMetadataException. If
> > client refresh metadata due to expiry and it receives a metadata whose
> > version is lower than the current metadata version, we can reject the
> > metadata but still reset the metadata age, which essentially keep the
> > existing behavior in the client.
> >
> > Thanks much,
> > Dong
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Jun Rao <ju...@confluent.io>.
Hi, Dong,

Thanks for the reply.

To solve the topic recreation issue, we could use either a global metadata
version or a partition level epoch. But either one will be a new concept,
right? To me, the latter seems more natural. It also makes it easier to
detect if a consumer's offset is still valid after a topic is recreated. As
you pointed out, we don't need to store the partition epoch in the message.
The following is what I am thinking. When a partition is created, we can
assign a partition epoch from an ever-increasing global counter and store
it in /brokers/topics/[topic]/partitions/[partitionId] in ZK. The partition
epoch is propagated to every broker. The consumer will be tracking a tuple
of <offset, leader epoch, partition epoch> for offsets. If a topic is
recreated, it's possible that a consumer's offset and leader epoch still
match that in the broker, but partition epoch won't be. In this case, we
can potentially still treat the consumer's offset as out of range and reset
the offset based on the offset reset policy in the consumer. This seems
harder to do with a global metadata version.

Jun



On Mon, Dec 25, 2017 at 6:56 AM, Dong Lin <li...@gmail.com> wrote:

> Hey Jun,
>
> This is a very good example. After thinking through this in detail, I agree
> that we need to commit offset with leader epoch in order to address this
> example.
>
> I think the remaining question is how to address the scenario that the
> topic is deleted and re-created. One possible solution is to commit offset
> with both the leader epoch and the metadata version. The logic and the
> implementation of this solution does not require a new concept (e.g.
> partition epoch) and it does not require any change to the message format
> or leader epoch. It also allows us to order the metadata in a
> straightforward manner which may be useful in the future. So it may be a
> better solution than generating a random partition epoch every time we
> create a partition. Does this sound reasonable?
>
> Previously one concern with using the metadata version is that consumer
> will be forced to refresh metadata even if metadata version is increased
> due to topics that the consumer is not interested in. Now I realized that
> this is probably not a problem. Currently client will refresh metadata
> either due to InvalidMetadataException in the response from broker or due
> to metadata expiry. The addition of the metadata version should increase
> the overhead of metadata refresh caused by InvalidMetadataException. If
> client refresh metadata due to expiry and it receives a metadata whose
> version is lower than the current metadata version, we can reject the
> metadata but still reset the metadata age, which essentially keep the
> existing behavior in the client.
>
> Thanks much,
> Dong
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Dong Lin <li...@gmail.com>.
Hey Jun,

This is a very good example. After thinking through this in detail, I agree
that we need to commit offset with leader epoch in order to address this
example.

I think the remaining question is how to address the scenario that the
topic is deleted and re-created. One possible solution is to commit offset
with both the leader epoch and the metadata version. The logic and the
implementation of this solution does not require a new concept (e.g.
partition epoch) and it does not require any change to the message format
or leader epoch. It also allows us to order the metadata in a
straightforward manner which may be useful in the future. So it may be a
better solution than generating a random partition epoch every time we
create a partition. Does this sound reasonable?

Previously one concern with using the metadata version is that consumer
will be forced to refresh metadata even if metadata version is increased
due to topics that the consumer is not interested in. Now I realized that
this is probably not a problem. Currently client will refresh metadata
either due to InvalidMetadataException in the response from broker or due
to metadata expiry. The addition of the metadata version should increase
the overhead of metadata refresh caused by InvalidMetadataException. If
client refresh metadata due to expiry and it receives a metadata whose
version is lower than the current metadata version, we can reject the
metadata but still reset the metadata age, which essentially keep the
existing behavior in the client.

Thanks much,
Dong

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Jun Rao <ju...@confluent.io>.
Hi, Dong,

My previous reply has a couple of typos. So, sending it again with the fix.

Consider the following scenario. In metadata v1, the leader for a partition
is at broker 1. In metadata v2, leader is at broker 2. In metadata v3,
leader is at broker 1 again. The last committed offset in v1, v2 and v3 are
10, 20 and 30, respectively. A consumer is started and reads metadata v1
and reads messages from offset 0 to 25 from broker 1. My understanding is
that in the current proposal, the metadata version associated with offset
25 is v1. The consumer is then restarted and fetches metadata v2. The
consumer tries to read from broker 2, which is the old leader with the last
offset at 20. In this case, the consumer will still get
OffsetOutOfRangeException incorrectly.

Fundamentally, it seems that the metadata version that we to want associate
with the offset should be the version when the data is published, not the
one that the consumer caches. In the above example, the metadata version
when offset 25 is published is v3. If the consumer uses that version, it
will be able to avoid the above issue. So, the metadata version ideally
probably should be stored with the message. We already store leader epoch
in each message set. So, reusing that will be the most convenient. It is
true that leader epoch is reset to 0 after the topic is re-created. To
address that issue, we can probably store a partition epoch in each message
set. The partition epoch can be generated randomly every time a partition
is created. If we want to avoid message format change, we can potentially
reserve the first byte of leader epoch as the partition epoch.

As I mentioned earlier, using leader epoch also has the benefit that it can
better handle the other corner case when existing message is lost due to
multiple replica failure (e.g. power outage) or unclean leader election.
Suppose in the above example, if the messages from offset 22 to 30 are lost
and republished with new messages at metadata v4, the consumer will know
that offset 25 is not valid since it's associated with v3. In this case,
the consumer can potentially reset its offset to the last offset in v3,
which is 22. This way, the consumer can re-consume all the republished
messages.

Thanks,

Jun

On Fri, Dec 22, 2017 at 5:04 PM, Jun Rao <ju...@confluent.io> wrote:

> Hi, Dong,
>
> Thanks for the updated KIP. Still have some questions on the latest
> approach in the KIP.
>
> Consider the following scenario. In metadata v1, the leader for a
> partition is at broker 1. In metadata v2, leader is at broker 2. In
> metadata v3, leader is at broker 1 again. The last committed offset in v1,
> v2 and v3 are 10, 20 and 30, respectively. A consumer is started and reads
> metadata v1 and reads messages from offset 0 to 25 from broker 1. My
> understanding is that in the current proposal, the metadata version
> associated with offset 25 is v6. The consumer is then restarted and fetches
> metadata v7. The consumer tries to read from broker 2, which is the old
> leader with the last offset at 20. In this case, the consumer will still
> get OffsetOutOfRangeException incorrectly.
>
> Fundamentally, it seems that the metadata version that we to want
> associate with the offset should be the version when the data is published,
> not the one that the consumer caches. In the above example, the metadata
> version when offset 25 is published is v3. If the consumer uses that
> version, it will be able to avoid the above issue. So, the metadata version
> ideally probably should be stored with the message. We already store leader
> epoch in each message set. So, reusing that will be the most convenient. It
> is true that leader epoch is reset to 0 after the topic is re-created. To
> address that issue, we can probably store a partition epoch in each message
> set. The partition epoch can be generated randomly every time a partition
> is created. If we want to avoid message format change, we can potentially
> reserve the first byte of leader epoch as the partition epoch.
>
> As I mentioned earlier, using leader epoch also has the benefit that it
> can better handle the other corner case when existing message is lost due
> to multiple replica failure (e.g. power outage) or unclean leader election.
> Suppose in the above example, if the messages from offset 22 to 30 are lost
> and republished with new messages at metadata v4, the consumer will know
> that offset 25 is not valid since it's associated with v3. In this case,
> the consumer can potentially reset its offset to the last offset in v3,
> which is 22. This way, the consumer can re-consume all the republished
> messages.
>
> Thanks,
>
> Jun
>
>
> On Thu, Dec 21, 2017 at 2:46 PM, Dong Lin <li...@gmail.com> wrote:
>
>> Hey Jun,
>>
>> Thanks much for your comments. Yeah I have not considered the case where
>> the offset is stored externally.
>>
>> Based Jason's question, I think we probably have to use a global
>> metadata_epoch. And since we have a global metadata_epoch, this KIP
>> probably no longer needs the per-partition leader_epoch. Then we can use
>> two newly-added API in consumer that allows user to get the metadata_epoch
>> from consumer and wait for consumer to receive MetadataResponse whose
>> metadata_epoch >= the given metadata_epoch. These two APIs should address
>> the case where user stored offset externally. I have updated the KIP
>> accordingly. Could you take another look?
>>
>> Thanks for all the comments.
>>
>> Dong
>>
>>
>> On Tue, Dec 19, 2017 at 3:09 PM, Jun Rao <ju...@confluent.io> wrote:
>>
>> > Hi, Dong,
>> >
>> > Thanks for the reply.
>> >
>> > 10. I was actually just thinking the case when the consumer consumes old
>> > data. If the current leader epoch is 3 and the consumer is consuming
>> > records generated in leader epoch 1, the epoch associated with the
>> offset
>> > should be 1. However, as you pointed out, the fetch response currently
>> > includes the leader epoch for fetched data. So, this is already covered.
>> >
>> > 11. That's an interesting thought. What about the case when the offsets
>> are
>> > stored externally? When we restart a consumer and seek to an externally
>> > stored offset, we won't know the leader epoch in the consumer. Do we
>> need
>> > another request to retrieve the leader epoch based on an offset and make
>> > sure the info is up to date? Another related thing is that the leader
>> epoch
>> > that we want to associate the offset with ideally should be the epoch
>> when
>> > the data is fetched. For example, when all replicas lost data due to a
>> > power failure or when there is an unclean leader election, the leader
>> epoch
>> > for a given offset may change over time on the broker. In those cases, a
>> > consumer's offset may be in range, but is not in the same leader epoch
>> for
>> > the time when the data is fetched. We can potentially do a smarter
>> offset
>> > reset in those cases if we remember the epoch when the data is fetched.
>> >
>> > Jun
>> >
>> >
>> >
>> > On Mon, Dec 18, 2017 at 1:58 PM, Dong Lin <li...@gmail.com> wrote:
>> >
>> > > Hey Jun,
>> > >
>> > > Thanks much for your comments. These are very thoughtful ideas. Please
>> > see
>> > > my comments below.
>> > >
>> > > On Thu, Dec 14, 2017 at 6:38 PM, Jun Rao <ju...@confluent.io> wrote:
>> > >
>> > > > Hi, Dong,
>> > > >
>> > > > Thanks for the update. A few more comments below.
>> > > >
>> > > > 10. It seems that we need to return the leader epoch in the fetch
>> > > response
>> > > > as well When fetching data, we could be fetching data from a leader
>> > epoch
>> > > > older than what's returned in the metadata response. So, we want to
>> use
>> > > the
>> > > > leader epoch associated with the offset being fetched for committing
>> > > > offsets.
>> > > >
>> > >
>> > > It seems that we may have two separate issues here. The first issue is
>> > that
>> > > consumer uses metadata that is older than the one it uses before. The
>> > > second issue is that consumer uses metadata which is newer than the
>> > > corresponding leader epoch in the leader broker. We know that the
>> > > OffsetOutOfRangeException described in this KIP can be prevented by
>> > > avoiding the first issue. On the other hand, it seems that the
>> > > OffsetOffsetOutOfRangeException can still happen even if we avoid the
>> > > second issue -- if consumer uses an older version of metadata, the
>> leader
>> > > epoch in its metadata may equal the leader epoch in the broker even if
>> > the
>> > > leader epoch in the broker is oudated.
>> > >
>> > > Given this understanding, I am not sure why we need to return the
>> leader
>> > > epoch in the fetch response. As long as consumer's metadata is not
>> going
>> > > back in version, I think we are good. Did I miss something here?
>> > >
>> > >
>> > > >
>> > > > 11. Should we now extend OffsetAndMetadata used in the offset commit
>> > api
>> > > in
>> > > > KafkaConsumer to include leader epoch? Similarly, should we return
>> > leader
>> > > > epoch in endOffsets(), beginningOffsets() and position()? We
>> probably
>> > > need
>> > > > to think about how to make the api backward compatible.
>> > > >
>> > >
>> > > After thinking through this carefully, I think we probably don't want
>> to
>> > > extend OffsetAndMetadata to include leader epoch because leader epoch
>> is
>> > > kind of implementation detail which ideally should be hidden from
>> user.
>> > The
>> > > consumer can include leader epoch in the OffsetCommitRequest after
>> taking
>> > > offset from commitSync(final Map<TopicPartition, OffsetAndMetadata>
>> > > offsets). Similarly consumer can store leader epoch from
>> > > OffsetFetchResponse and only provide offset to user via
>> > > consumer.committed(topicPartition). This solution seems to work well
>> and
>> > > we
>> > > don't have to make changes to consumer's public API. Does this sound
>> OK?
>> > >
>> > >
>> > > >
>> > > > 12. It seems that we now need to store leader epoch in the offset
>> > topic.
>> > > > Could you include the new schema for the value of the offset topic
>> and
>> > > add
>> > > > upgrade notes?
>> > >
>> > >
>> > > You are right. I have updated the KIP to specify the new schema for
>> the
>> > > value of the offset topic. Can you take another look?
>> > >
>> > > For existing messages in the offset topic, leader_epoch will be
>> missing.
>> > We
>> > > will use leader_epoch = -1 to indicate the missing leader_epoch. Then
>> the
>> > > consumer behavior will be the same as it is now because any
>> leader_epoch
>> > in
>> > > the MetadataResponse will be larger than the leader_epoch = -1 in the
>> > > OffetFetchResponse. Thus we don't need specific procedure for upgrades
>> > due
>> > > to this change in the offset topic schema. By "upgrade nodes", do you
>> > mean
>> > > the sentences we need to include in the upgrade.html in the PR later?
>> > >
>> > >
>> > > >
>> > > > Jun
>> > > >
>> > > >
>> > > > On Tue, Dec 12, 2017 at 5:19 PM, Dong Lin <li...@gmail.com>
>> wrote:
>> > > >
>> > > > > Hey Jun,
>> > > > >
>> > > > > I see. Sounds good. Yeah it is probably simpler to leave this to
>> > > another
>> > > > > KIP in the future.
>> > > > >
>> > > > > Thanks for all the comments. Since there is no further comment in
>> the
>> > > > > community, I will open the voting thread.
>> > > > >
>> > > > > Thanks,
>> > > > > Dong
>> > > > >
>> > > > > On Mon, Dec 11, 2017 at 5:37 PM, Jun Rao <ju...@confluent.io>
>> wrote:
>> > > > >
>> > > > > > Hi, Dong,
>> > > > > >
>> > > > > > The case that I am thinking is network partitioning. Suppose one
>> > > > deploys
>> > > > > a
>> > > > > > stretched cluster across multiple AZs in the same region. If the
>> > > > machines
>> > > > > > in one AZ can't communicate to brokers in other AZs due to a
>> > network
>> > > > > issue,
>> > > > > > the brokers in that AZ won't get any new metadata.
>> > > > > >
>> > > > > > We can potentially solve this problem by requiring some kind of
>> > > regular
>> > > > > > heartbeats between the controller and the broker. This may need
>> > some
>> > > > more
>> > > > > > thoughts. So, it's probably fine to leave this to another KIP in
>> > the
>> > > > > > future.
>> > > > > >
>> > > > > > Thanks,
>> > > > > >
>> > > > > > Jun
>> > > > > >
>> > > > > > On Mon, Dec 11, 2017 at 2:55 PM, Dong Lin <li...@gmail.com>
>> > > wrote:
>> > > > > >
>> > > > > > > Hey Jun,
>> > > > > > >
>> > > > > > > Thanks for the comment. I am open to improve this KIP to
>> address
>> > > more
>> > > > > > > problems. I probably need more help in understanding what is
>> the
>> > > > > current
>> > > > > > > problem with consumer using outdated metadata and whether it
>> is
>> > > > easier
>> > > > > to
>> > > > > > > address it together with this KIP.
>> > > > > > >
>> > > > > > > I agree that a consumer can potentially talk to old leader
>> for a
>> > > long
>> > > > > > time
>> > > > > > > even after this KIP. But after this KIP, the consumer probably
>> > > should
>> > > > > not
>> > > > > > > get OffetOutofRangeException and therefore will not cause
>> offset
>> > > > rewind
>> > > > > > > issue. So the only problem is that consumer will not be able
>> to
>> > > fetch
>> > > > > > data
>> > > > > > > until it has updated metadata. It seems that this situation
>> can
>> > > only
>> > > > > > happen
>> > > > > > > if the broker is too slow in processing LeaderAndIsrRequest
>> since
>> > > > > > otherwise
>> > > > > > > the consumer will be forced to update metadata due to
>> > > > > > > NotLeaderForPartitionException. So the problem we are having
>> > here
>> > > is
>> > > > > > that
>> > > > > > > consumer will not be able to fetch data if some broker is too
>> > slow
>> > > in
>> > > > > > > processing LeaderAndIsrRequest.
>> > > > > > >
>> > > > > > > Because Kafka propagates LeaderAndIsrRequest asynchronously to
>> > all
>> > > > > > brokers
>> > > > > > > in the cluster, there will always be a period of time when
>> > consumer
>> > > > can
>> > > > > > not
>> > > > > > > fetch data for the partition during the leadership change.
>> Thus
>> > it
>> > > > > seems
>> > > > > > > more like a broker-side performance issue instead of
>> client-side
>> > > > > > > correctness issue. My gut feel is that it is not causing a
>> much a
>> > > > > problem
>> > > > > > > as the problem to be fixed in this KIP. And if we were to
>> address
>> > > it,
>> > > > > we
>> > > > > > > probably need to make change in the broker side, e.g. with
>> > > > prioritized
>> > > > > > > queue for controller-related requests, which may be kind of
>> > > > orthogonal
>> > > > > to
>> > > > > > > this KIP. I am not very sure it will be easier to address it
>> with
>> > > the
>> > > > > > > change in this KIP. Do you have any recommendation?
>> > > > > > >
>> > > > > > > Thanks,
>> > > > > > > Dong
>> > > > > > >
>> > > > > > >
>> > > > > > > On Mon, Dec 11, 2017 at 1:51 PM, Jun Rao <ju...@confluent.io>
>> > wrote:
>> > > > > > >
>> > > > > > > > Hi, Dong,
>> > > > > > > >
>> > > > > > > > Thanks for the reply.
>> > > > > > > >
>> > > > > > > > My suggestion of forcing the metadata refresh from the
>> > controller
>> > > > may
>> > > > > > not
>> > > > > > > > work in general since the cached controller could be
>> outdated
>> > > too.
>> > > > > The
>> > > > > > > > general problem is that if a consumer's metadata is
>> outdated,
>> > it
>> > > > may
>> > > > > > get
>> > > > > > > > stuck with the old leader for a long time. We can address
>> the
>> > > issue
>> > > > > of
>> > > > > > > > detecting outdated metadata in a separate KIP in the future
>> if
>> > > you
>> > > > > > didn't
>> > > > > > > > intend to address it in this KIP.
>> > > > > > > >
>> > > > > > > > Thanks,
>> > > > > > > >
>> > > > > > > > Jun
>> > > > > > > >
>> > > > > > > >
>> > > > > > > > On Sat, Dec 9, 2017 at 10:12 PM, Dong Lin <
>> lindong28@gmail.com
>> > >
>> > > > > wrote:
>> > > > > > > >
>> > > > > > > > > Hey Jun,
>> > > > > > > > >
>> > > > > > > > > Thanks much for your comments. Given that client needs to
>> > > > > > de-serialize
>> > > > > > > > the
>> > > > > > > > > metadata anyway, the extra overhead of checking the
>> > > per-partition
>> > > > > > > version
>> > > > > > > > > for every partition should not be a big concern. Thus it
>> > makes
>> > > > > sense
>> > > > > > to
>> > > > > > > > use
>> > > > > > > > > leader epoch as the per-partition version instead of
>> > creating a
>> > > > > > global
>> > > > > > > > > metadata version. I will update the KIP to do that.
>> > > > > > > > >
>> > > > > > > > > Regarding the detection of outdated metadata, I think it
>> is
>> > > > > possible
>> > > > > > to
>> > > > > > > > > ensure that client gets latest metadata by fetching from
>> > > > > controller.
>> > > > > > > Note
>> > > > > > > > > that this requires extra logic in the controller such that
>> > > > > controller
>> > > > > > > > > updates metadata directly in memory without requiring
>> > > > > > > > > UpdateMetadataRequest. But I am not sure the main
>> motivation
>> > of
>> > > > > this
>> > > > > > at
>> > > > > > > > > this moment. But this makes controller more like a
>> bottleneck
>> > > in
>> > > > > the
>> > > > > > > > > cluster which we probably want to avoid.
>> > > > > > > > >
>> > > > > > > > > I think we can probably keep the current way of ensuring
>> > > metadata
>> > > > > > > > > freshness. Currently client will be forced to refresh
>> > metadata
>> > > if
>> > > > > > > broker
>> > > > > > > > > returns error (e.g. NotLeaderForPartition) due to outdated
>> > > > metadata
>> > > > > > or
>> > > > > > > if
>> > > > > > > > > the metadata does not contain the partition that the
>> client
>> > > > needs.
>> > > > > In
>> > > > > > > the
>> > > > > > > > > future, as you previously suggested, we can include
>> > > per-partition
>> > > > > > > > > leaderEpoch in the FetchRequest/ProduceRequest such that
>> > broker
>> > > > can
>> > > > > > > > return
>> > > > > > > > > error if the epoch is smaller than cached epoch in the
>> > broker.
>> > > > > Given
>> > > > > > > that
>> > > > > > > > > this adds more complexity to Kafka, I think we can
>> probably
>> > > think
>> > > > > > about
>> > > > > > > > > that leader when we have a specific use-case or problem to
>> > > solve
>> > > > > with
>> > > > > > > > > up-to-date metadata. Does this sound OK?
>> > > > > > > > >
>> > > > > > > > > Thanks,
>> > > > > > > > > Dong
>> > > > > > > > >
>> > > > > > > > >
>> > > > > > > > >
>> > > > > > > > > On Fri, Dec 8, 2017 at 3:53 PM, Jun Rao <jun@confluent.io
>> >
>> > > > wrote:
>> > > > > > > > >
>> > > > > > > > > > Hi, Dong,
>> > > > > > > > > >
>> > > > > > > > > > Thanks for the reply. A few more points below.
>> > > > > > > > > >
>> > > > > > > > > > For dealing with how to prevent a consumer switching
>> from a
>> > > new
>> > > > > > > leader
>> > > > > > > > to
>> > > > > > > > > > an old leader, you suggestion that refreshes metadata on
>> > > > consumer
>> > > > > > > > restart
>> > > > > > > > > > until it sees a metadata version >= the one associated
>> with
>> > > the
>> > > > > > > offset
>> > > > > > > > > > works too, as long as we guarantee that the cached
>> metadata
>> > > > > > versions
>> > > > > > > on
>> > > > > > > > > the
>> > > > > > > > > > brokers only go up.
>> > > > > > > > > >
>> > > > > > > > > > The second discussion point is on whether the metadata
>> > > > versioning
>> > > > > > > > should
>> > > > > > > > > be
>> > > > > > > > > > per partition or global. For the partition level
>> > versioning,
>> > > > you
>> > > > > > were
>> > > > > > > > > > concerned about the performance. Given that metadata
>> > updates
>> > > > are
>> > > > > > > rare,
>> > > > > > > > I
>> > > > > > > > > am
>> > > > > > > > > > not sure if it's a big concern though. Doing a million
>> if
>> > > tests
>> > > > > is
>> > > > > > > > > probably
>> > > > > > > > > > going to take less than 1ms. Another thing is that the
>> > > metadata
>> > > > > > > version
>> > > > > > > > > > seems to need to survive controller failover. In your
>> > current
>> > > > > > > > approach, a
>> > > > > > > > > > consumer may not be able to wait on the right version of
>> > the
>> > > > > > metadata
>> > > > > > > > > after
>> > > > > > > > > > the consumer restart since the metadata version may have
>> > been
>> > > > > > > recycled
>> > > > > > > > on
>> > > > > > > > > > the server side due to a controller failover while the
>> > > consumer
>> > > > > is
>> > > > > > > > down.
>> > > > > > > > > > The partition level leaderEpoch survives controller
>> failure
>> > > and
>> > > > > > won't
>> > > > > > > > > have
>> > > > > > > > > > this issue.
>> > > > > > > > > >
>> > > > > > > > > > Lastly, neither your proposal nor mine addresses the
>> issue
>> > > how
>> > > > to
>> > > > > > > > > guarantee
>> > > > > > > > > > a consumer to detect that is metadata is outdated.
>> > Currently,
>> > > > the
>> > > > > > > > > consumer
>> > > > > > > > > > is not guaranteed to fetch metadata from every broker
>> > within
>> > > > some
>> > > > > > > > bounded
>> > > > > > > > > > period of time. Maybe this is out of the scope of your
>> KIP.
>> > > But
>> > > > > one
>> > > > > > > > idea
>> > > > > > > > > is
>> > > > > > > > > > force the consumer to refresh metadata from the
>> controller
>> > > > > > > > periodically.
>> > > > > > > > > >
>> > > > > > > > > > Jun
>> > > > > > > > > >
>> > > > > > > > > >
>> > > > > > > > > > On Thu, Dec 7, 2017 at 11:25 AM, Dong Lin <
>> > > lindong28@gmail.com
>> > > > >
>> > > > > > > wrote:
>> > > > > > > > > >
>> > > > > > > > > > > Hey Jun,
>> > > > > > > > > > >
>> > > > > > > > > > > Thanks much for the comments. Great point particularly
>> > > > > regarding
>> > > > > > > > (3). I
>> > > > > > > > > > > haven't thought about this before.
>> > > > > > > > > > >
>> > > > > > > > > > > It seems that there are two possible ways where the
>> > version
>> > > > > > number
>> > > > > > > > can
>> > > > > > > > > be
>> > > > > > > > > > > used. One solution is for client to check the version
>> > > number
>> > > > at
>> > > > > > the
>> > > > > > > > > time
>> > > > > > > > > > it
>> > > > > > > > > > > receives MetadataResponse. And if the version number
>> in
>> > the
>> > > > > > > > > > > MetadataResponse is smaller than the version number in
>> > the
>> > > > > > client's
>> > > > > > > > > > cache,
>> > > > > > > > > > > the client will be forced to fetch metadata again.
>> > Another
>> > > > > > > solution,
>> > > > > > > > > as
>> > > > > > > > > > > you have suggested, is for broker to check the version
>> > > number
>> > > > > at
>> > > > > > > the
>> > > > > > > > > time
>> > > > > > > > > > > it receives a request from client. The broker will
>> reject
>> > > the
>> > > > > > > request
>> > > > > > > > > if
>> > > > > > > > > > > the version is smaller than the version in broker's
>> > cache.
>> > > > > > > > > > >
>> > > > > > > > > > > I am not very sure that the second solution can
>> address
>> > the
>> > > > > > problem
>> > > > > > > > > here.
>> > > > > > > > > > > In the scenario described in the JIRA ticket, broker's
>> > > cache
>> > > > > may
>> > > > > > be
>> > > > > > > > > > > outdated because it has not processed the
>> > > LeaderAndIsrRequest
>> > > > > > from
>> > > > > > > > the
>> > > > > > > > > > > controller. Thus it may still process client's request
>> > even
>> > > > if
>> > > > > > the
>> > > > > > > > > > version
>> > > > > > > > > > > in client's request is actually outdated. Does this
>> make
>> > > > sense?
>> > > > > > > > > > >
>> > > > > > > > > > > IMO, it seems that we can address problem (3) by
>> saving
>> > the
>> > > > > > > metadata
>> > > > > > > > > > > version together with the offset. After consumer
>> starts,
>> > it
>> > > > > will
>> > > > > > > keep
>> > > > > > > > > > > fetching metadata until the metadata version >= the
>> > version
>> > > > > saved
>> > > > > > > > with
>> > > > > > > > > > the
>> > > > > > > > > > > offset of this partition.
>> > > > > > > > > > >
>> > > > > > > > > > > Regarding problems (1) and (2): Currently we use the
>> > > version
>> > > > > > number
>> > > > > > > > in
>> > > > > > > > > > the
>> > > > > > > > > > > MetadataResponse to ensure that the metadata does not
>> go
>> > > back
>> > > > > in
>> > > > > > > > time.
>> > > > > > > > > > > There are two alternative solutions to address
>> problems
>> > (1)
>> > > > and
>> > > > > > > (2).
>> > > > > > > > > One
>> > > > > > > > > > > solution is for client to enumerate all partitions in
>> the
>> > > > > > > > > > MetadataResponse,
>> > > > > > > > > > > compare their epoch with those in the cached metadata,
>> > and
>> > > > > > rejects
>> > > > > > > > the
>> > > > > > > > > > > MetadataResponse iff any leader epoch is smaller. The
>> > main
>> > > > > > concern
>> > > > > > > is
>> > > > > > > > > > that
>> > > > > > > > > > > MetadataResponse currently cached information of all
>> > > > partitions
>> > > > > > in
>> > > > > > > > the
>> > > > > > > > > > > entire cluster. It may slow down client's performance
>> if
>> > we
>> > > > > were
>> > > > > > to
>> > > > > > > > do
>> > > > > > > > > > it.
>> > > > > > > > > > > The other solution is for client to enumerate
>> partitions
>> > > for
>> > > > > only
>> > > > > > > > > topics
>> > > > > > > > > > > registered in the org.apache.kafka.clients.Metadata,
>> > which
>> > > > > will
>> > > > > > be
>> > > > > > > > an
>> > > > > > > > > > > empty
>> > > > > > > > > > > set for producer and the set of subscribed partitions
>> for
>> > > > > > consumer.
>> > > > > > > > But
>> > > > > > > > > > > this degrades to all topics if consumer subscribes to
>> > > topics
>> > > > in
>> > > > > > the
>> > > > > > > > > > cluster
>> > > > > > > > > > > by pattern.
>> > > > > > > > > > >
>> > > > > > > > > > > Note that client will only be forced to update
>> metadata
>> > if
>> > > > the
>> > > > > > > > version
>> > > > > > > > > in
>> > > > > > > > > > > the MetadataResponse is smaller than the version in
>> the
>> > > > cached
>> > > > > > > > > metadata.
>> > > > > > > > > > In
>> > > > > > > > > > > general it should not be a problem. It can be a
>> problem
>> > > only
>> > > > if
>> > > > > > > some
>> > > > > > > > > > broker
>> > > > > > > > > > > is particularly slower than other brokers in
>> processing
>> > > > > > > > > > > UpdateMetadataRequest. When this is the case, it means
>> > that
>> > > > the
>> > > > > > > > broker
>> > > > > > > > > is
>> > > > > > > > > > > also particularly slower in processing
>> > LeaderAndIsrRequest,
>> > > > > which
>> > > > > > > can
>> > > > > > > > > > cause
>> > > > > > > > > > > problem anyway because some partition will probably
>> have
>> > no
>> > > > > > leader
>> > > > > > > > > during
>> > > > > > > > > > > this period. I am not sure problems (1) and (2) cause
>> > more
>> > > > > > problem
>> > > > > > > > than
>> > > > > > > > > > > what we already have.
>> > > > > > > > > > >
>> > > > > > > > > > > Thanks,
>> > > > > > > > > > > Dong
>> > > > > > > > > > >
>> > > > > > > > > > >
>> > > > > > > > > > > On Wed, Dec 6, 2017 at 6:42 PM, Jun Rao <
>> > jun@confluent.io>
>> > > > > > wrote:
>> > > > > > > > > > >
>> > > > > > > > > > > > Hi, Dong,
>> > > > > > > > > > > >
>> > > > > > > > > > > > Great finding on the issue. It's a real problem. A
>> few
>> > > > > comments
>> > > > > > > > about
>> > > > > > > > > > the
>> > > > > > > > > > > > KIP. (1) I am not sure about updating
>> > > > > controller_metadata_epoch
>> > > > > > > on
>> > > > > > > > > > every
>> > > > > > > > > > > > UpdateMetadataRequest. Currently, the controller can
>> > send
>> > > > > > > > > > > > UpdateMetadataRequest when there is no actual
>> metadata
>> > > > > change.
>> > > > > > > > Doing
>> > > > > > > > > > this
>> > > > > > > > > > > > may require unnecessary metadata refresh on the
>> client.
>> > > (2)
>> > > > > > > > > > > > controller_metadata_epoch is global across all
>> topics.
>> > > This
>> > > > > > means
>> > > > > > > > > that
>> > > > > > > > > > a
>> > > > > > > > > > > > client may be forced to update its metadata even
>> when
>> > the
>> > > > > > > metadata
>> > > > > > > > > for
>> > > > > > > > > > > the
>> > > > > > > > > > > > topics that it cares haven't changed. (3) It doesn't
>> > seem
>> > > > > that
>> > > > > > > the
>> > > > > > > > > KIP
>> > > > > > > > > > > > handles the corner case when a consumer is
>> restarted.
>> > > Say a
>> > > > > > > > consumer
>> > > > > > > > > > > reads
>> > > > > > > > > > > > from the new leader, commits the offset and then is
>> > > > > restarted.
>> > > > > > On
>> > > > > > > > > > > restart,
>> > > > > > > > > > > > the consumer gets an outdated metadata and fetches
>> from
>> > > the
>> > > > > old
>> > > > > > > > > leader.
>> > > > > > > > > > > > Then, the consumer will get into the offset out of
>> > range
>> > > > > issue.
>> > > > > > > > > > > >
>> > > > > > > > > > > > Given the above, I am thinking of the following
>> > approach.
>> > > > We
>> > > > > > > > actually
>> > > > > > > > > > > > already have metadata versioning at the partition
>> > level.
>> > > > Each
>> > > > > > > > leader
>> > > > > > > > > > has
>> > > > > > > > > > > a
>> > > > > > > > > > > > leader epoch which is monotonically increasing. We
>> can
>> > > > > > > potentially
>> > > > > > > > > > > > propagate leader epoch back in the metadata response
>> > and
>> > > > the
>> > > > > > > > clients
>> > > > > > > > > > can
>> > > > > > > > > > > > cache that. This solves the issue of (1) and (2). To
>> > > solve
>> > > > > (3),
>> > > > > > > > when
>> > > > > > > > > > > saving
>> > > > > > > > > > > > an offset, we could save both an offset and the
>> > > > corresponding
>> > > > > > > > leader
>> > > > > > > > > > > epoch.
>> > > > > > > > > > > > When fetching the data, the consumer provides both
>> the
>> > > > offset
>> > > > > > and
>> > > > > > > > the
>> > > > > > > > > > > > leader epoch. A leader will only serve the request
>> if
>> > its
>> > > > > > leader
>> > > > > > > > > epoch
>> > > > > > > > > > is
>> > > > > > > > > > > > equal to or greater than the leader epoch from the
>> > > > consumer.
>> > > > > To
>> > > > > > > > > achieve
>> > > > > > > > > > > > this, we need to change the fetch request protocol
>> and
>> > > the
>> > > > > > offset
>> > > > > > > > > > commit
>> > > > > > > > > > > > api, which requires some more thoughts.
>> > > > > > > > > > > >
>> > > > > > > > > > > > Thanks,
>> > > > > > > > > > > >
>> > > > > > > > > > > > Jun
>> > > > > > > > > > > >
>> > > > > > > > > > > >
>> > > > > > > > > > > > On Wed, Dec 6, 2017 at 10:57 AM, Dong Lin <
>> > > > > lindong28@gmail.com
>> > > > > > >
>> > > > > > > > > wrote:
>> > > > > > > > > > > >
>> > > > > > > > > > > > > Bump up the thread.
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > It will be great to have more comments on whether
>> we
>> > > > should
>> > > > > > do
>> > > > > > > it
>> > > > > > > > > or
>> > > > > > > > > > > > > whether there is better way to address the
>> motivation
>> > > of
>> > > > > this
>> > > > > > > > KIP.
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > On Mon, Dec 4, 2017 at 3:09 PM, Dong Lin <
>> > > > > > lindong28@gmail.com>
>> > > > > > > > > > wrote:
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > > I don't have an interesting rejected alternative
>> > > > solution
>> > > > > > to
>> > > > > > > > put
>> > > > > > > > > in
>> > > > > > > > > > > the
>> > > > > > > > > > > > > > KIP. If there is good alternative solution from
>> > > anyone
>> > > > in
>> > > > > > > this
>> > > > > > > > > > > thread,
>> > > > > > > > > > > > I
>> > > > > > > > > > > > > am
>> > > > > > > > > > > > > > happy to discuss this and update the KIP
>> > accordingly.
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > > Thanks,
>> > > > > > > > > > > > > > Dong
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > > On Mon, Dec 4, 2017 at 1:12 PM, Ted Yu <
>> > > > > > yuzhihong@gmail.com>
>> > > > > > > > > > wrote:
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > >> It is clearer now.
>> > > > > > > > > > > > > >>
>> > > > > > > > > > > > > >> I noticed that Rejected Alternatives section is
>> > > empty.
>> > > > > > > > > > > > > >> Have you considered any alternative ?
>> > > > > > > > > > > > > >>
>> > > > > > > > > > > > > >> Cheers
>> > > > > > > > > > > > > >>
>> > > > > > > > > > > > > >> On Mon, Dec 4, 2017 at 1:07 PM, Dong Lin <
>> > > > > > > lindong28@gmail.com
>> > > > > > > > >
>> > > > > > > > > > > wrote:
>> > > > > > > > > > > > > >>
>> > > > > > > > > > > > > >> > Ted, thanks for catching this. I have updated
>> > the
>> > > > > > sentence
>> > > > > > > > to
>> > > > > > > > > > make
>> > > > > > > > > > > > it
>> > > > > > > > > > > > > >> > readable.
>> > > > > > > > > > > > > >> >
>> > > > > > > > > > > > > >> > Thanks,
>> > > > > > > > > > > > > >> > Dong
>> > > > > > > > > > > > > >> >
>> > > > > > > > > > > > > >> > On Sat, Dec 2, 2017 at 3:05 PM, Ted Yu <
>> > > > > > > yuzhihong@gmail.com
>> > > > > > > > >
>> > > > > > > > > > > wrote:
>> > > > > > > > > > > > > >> >
>> > > > > > > > > > > > > >> > > bq. It the controller_epoch of the incoming
>> > > > > > > > > MetadataResponse,
>> > > > > > > > > > or
>> > > > > > > > > > > > if
>> > > > > > > > > > > > > >> the
>> > > > > > > > > > > > > >> > > controller_epoch is the same but the
>> > > > > > > > > controller_metadata_epoch
>> > > > > > > > > > > > > >> > >
>> > > > > > > > > > > > > >> > > Can you update the above sentence so that
>> the
>> > > > > > intention
>> > > > > > > is
>> > > > > > > > > > > > clearer ?
>> > > > > > > > > > > > > >> > >
>> > > > > > > > > > > > > >> > > Thanks
>> > > > > > > > > > > > > >> > >
>> > > > > > > > > > > > > >> > > On Fri, Dec 1, 2017 at 6:33 PM, Dong Lin <
>> > > > > > > > > lindong28@gmail.com
>> > > > > > > > > > >
>> > > > > > > > > > > > > wrote:
>> > > > > > > > > > > > > >> > >
>> > > > > > > > > > > > > >> > > > Hi all,
>> > > > > > > > > > > > > >> > > >
>> > > > > > > > > > > > > >> > > > I have created KIP-232: Detect outdated
>> > > metadata
>> > > > > by
>> > > > > > > > adding
>> > > > > > > > > > > > > >> > > > ControllerMetadataEpoch field:
>> > > > > > > > > > > > > >> > > > https://cwiki.apache.org/
>> > > > > > > confluence/display/KAFKA/KIP-
>> > > > > > > > > > > > > >> > > > 232%3A+Detect+outdated+metadat
>> a+by+adding+
>> > > > > > > > > > > > > >> > ControllerMetadataEpoch+field
>> > > > > > > > > > > > > >> > > > .
>> > > > > > > > > > > > > >> > > >
>> > > > > > > > > > > > > >> > > > The KIP proposes to add fields in
>> > > > MetadataResponse
>> > > > > > and
>> > > > > > > > > > > > > >> > > > UpdateMetadataRequest so that client can
>> > > reject
>> > > > > > > outdated
>> > > > > > > > > > > > metadata
>> > > > > > > > > > > > > >> and
>> > > > > > > > > > > > > >> > > avoid
>> > > > > > > > > > > > > >> > > > unnecessary OffsetOutOfRangeException.
>> > > Otherwise
>> > > > > > there
>> > > > > > > > is
>> > > > > > > > > > > > > currently
>> > > > > > > > > > > > > >> > race
>> > > > > > > > > > > > > >> > > > condition that can cause consumer to
>> reset
>> > > > offset
>> > > > > > > which
>> > > > > > > > > > > > negatively
>> > > > > > > > > > > > > >> > affect
>> > > > > > > > > > > > > >> > > > the consumer's availability.
>> > > > > > > > > > > > > >> > > >
>> > > > > > > > > > > > > >> > > > Feedback and suggestions are welcome!
>> > > > > > > > > > > > > >> > > >
>> > > > > > > > > > > > > >> > > > Regards,
>> > > > > > > > > > > > > >> > > > Dong
>> > > > > > > > > > > > > >> > > >
>> > > > > > > > > > > > > >> > >
>> > > > > > > > > > > > > >> >
>> > > > > > > > > > > > > >>
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > >
>> > > > > > > > > > > >
>> > > > > > > > > > >
>> > > > > > > > > >
>> > > > > > > > >
>> > > > > > > >
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>>
>
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Jun Rao <ju...@confluent.io>.
Hi, Dong,

Thanks for the updated KIP. Still have some questions on the latest
approach in the KIP.

Consider the following scenario. In metadata v1, the leader for a partition
is at broker 1. In metadata v2, leader is at broker 2. In metadata v3,
leader is at broker 1 again. The last committed offset in v1, v2 and v3 are
10, 20 and 30, respectively. A consumer is started and reads metadata v1
and reads messages from offset 0 to 25 from broker 1. My understanding is
that in the current proposal, the metadata version associated with offset
25 is v6. The consumer is then restarted and fetches metadata v7. The
consumer tries to read from broker 2, which is the old leader with the last
offset at 20. In this case, the consumer will still get
OffsetOutOfRangeException incorrectly.

Fundamentally, it seems that the metadata version that we to want associate
with the offset should be the version when the data is published, not the
one that the consumer caches. In the above example, the metadata version
when offset 25 is published is v3. If the consumer uses that version, it
will be able to avoid the above issue. So, the metadata version ideally
probably should be stored with the message. We already store leader epoch
in each message set. So, reusing that will be the most convenient. It is
true that leader epoch is reset to 0 after the topic is re-created. To
address that issue, we can probably store a partition epoch in each message
set. The partition epoch can be generated randomly every time a partition
is created. If we want to avoid message format change, we can potentially
reserve the first byte of leader epoch as the partition epoch.

As I mentioned earlier, using leader epoch also has the benefit that it can
better handle the other corner case when existing message is lost due to
multiple replica failure (e.g. power outage) or unclean leader election.
Suppose in the above example, if the messages from offset 22 to 30 are lost
and republished with new messages at metadata v4, the consumer will know
that offset 25 is not valid since it's associated with v3. In this case,
the consumer can potentially reset its offset to the last offset in v3,
which is 22. This way, the consumer can re-consume all the republished
messages.

Thanks,

Jun


On Thu, Dec 21, 2017 at 2:46 PM, Dong Lin <li...@gmail.com> wrote:

> Hey Jun,
>
> Thanks much for your comments. Yeah I have not considered the case where
> the offset is stored externally.
>
> Based Jason's question, I think we probably have to use a global
> metadata_epoch. And since we have a global metadata_epoch, this KIP
> probably no longer needs the per-partition leader_epoch. Then we can use
> two newly-added API in consumer that allows user to get the metadata_epoch
> from consumer and wait for consumer to receive MetadataResponse whose
> metadata_epoch >= the given metadata_epoch. These two APIs should address
> the case where user stored offset externally. I have updated the KIP
> accordingly. Could you take another look?
>
> Thanks for all the comments.
>
> Dong
>
>
> On Tue, Dec 19, 2017 at 3:09 PM, Jun Rao <ju...@confluent.io> wrote:
>
> > Hi, Dong,
> >
> > Thanks for the reply.
> >
> > 10. I was actually just thinking the case when the consumer consumes old
> > data. If the current leader epoch is 3 and the consumer is consuming
> > records generated in leader epoch 1, the epoch associated with the offset
> > should be 1. However, as you pointed out, the fetch response currently
> > includes the leader epoch for fetched data. So, this is already covered.
> >
> > 11. That's an interesting thought. What about the case when the offsets
> are
> > stored externally? When we restart a consumer and seek to an externally
> > stored offset, we won't know the leader epoch in the consumer. Do we need
> > another request to retrieve the leader epoch based on an offset and make
> > sure the info is up to date? Another related thing is that the leader
> epoch
> > that we want to associate the offset with ideally should be the epoch
> when
> > the data is fetched. For example, when all replicas lost data due to a
> > power failure or when there is an unclean leader election, the leader
> epoch
> > for a given offset may change over time on the broker. In those cases, a
> > consumer's offset may be in range, but is not in the same leader epoch
> for
> > the time when the data is fetched. We can potentially do a smarter offset
> > reset in those cases if we remember the epoch when the data is fetched.
> >
> > Jun
> >
> >
> >
> > On Mon, Dec 18, 2017 at 1:58 PM, Dong Lin <li...@gmail.com> wrote:
> >
> > > Hey Jun,
> > >
> > > Thanks much for your comments. These are very thoughtful ideas. Please
> > see
> > > my comments below.
> > >
> > > On Thu, Dec 14, 2017 at 6:38 PM, Jun Rao <ju...@confluent.io> wrote:
> > >
> > > > Hi, Dong,
> > > >
> > > > Thanks for the update. A few more comments below.
> > > >
> > > > 10. It seems that we need to return the leader epoch in the fetch
> > > response
> > > > as well When fetching data, we could be fetching data from a leader
> > epoch
> > > > older than what's returned in the metadata response. So, we want to
> use
> > > the
> > > > leader epoch associated with the offset being fetched for committing
> > > > offsets.
> > > >
> > >
> > > It seems that we may have two separate issues here. The first issue is
> > that
> > > consumer uses metadata that is older than the one it uses before. The
> > > second issue is that consumer uses metadata which is newer than the
> > > corresponding leader epoch in the leader broker. We know that the
> > > OffsetOutOfRangeException described in this KIP can be prevented by
> > > avoiding the first issue. On the other hand, it seems that the
> > > OffsetOffsetOutOfRangeException can still happen even if we avoid the
> > > second issue -- if consumer uses an older version of metadata, the
> leader
> > > epoch in its metadata may equal the leader epoch in the broker even if
> > the
> > > leader epoch in the broker is oudated.
> > >
> > > Given this understanding, I am not sure why we need to return the
> leader
> > > epoch in the fetch response. As long as consumer's metadata is not
> going
> > > back in version, I think we are good. Did I miss something here?
> > >
> > >
> > > >
> > > > 11. Should we now extend OffsetAndMetadata used in the offset commit
> > api
> > > in
> > > > KafkaConsumer to include leader epoch? Similarly, should we return
> > leader
> > > > epoch in endOffsets(), beginningOffsets() and position()? We probably
> > > need
> > > > to think about how to make the api backward compatible.
> > > >
> > >
> > > After thinking through this carefully, I think we probably don't want
> to
> > > extend OffsetAndMetadata to include leader epoch because leader epoch
> is
> > > kind of implementation detail which ideally should be hidden from user.
> > The
> > > consumer can include leader epoch in the OffsetCommitRequest after
> taking
> > > offset from commitSync(final Map<TopicPartition, OffsetAndMetadata>
> > > offsets). Similarly consumer can store leader epoch from
> > > OffsetFetchResponse and only provide offset to user via
> > > consumer.committed(topicPartition). This solution seems to work well
> and
> > > we
> > > don't have to make changes to consumer's public API. Does this sound
> OK?
> > >
> > >
> > > >
> > > > 12. It seems that we now need to store leader epoch in the offset
> > topic.
> > > > Could you include the new schema for the value of the offset topic
> and
> > > add
> > > > upgrade notes?
> > >
> > >
> > > You are right. I have updated the KIP to specify the new schema for the
> > > value of the offset topic. Can you take another look?
> > >
> > > For existing messages in the offset topic, leader_epoch will be
> missing.
> > We
> > > will use leader_epoch = -1 to indicate the missing leader_epoch. Then
> the
> > > consumer behavior will be the same as it is now because any
> leader_epoch
> > in
> > > the MetadataResponse will be larger than the leader_epoch = -1 in the
> > > OffetFetchResponse. Thus we don't need specific procedure for upgrades
> > due
> > > to this change in the offset topic schema. By "upgrade nodes", do you
> > mean
> > > the sentences we need to include in the upgrade.html in the PR later?
> > >
> > >
> > > >
> > > > Jun
> > > >
> > > >
> > > > On Tue, Dec 12, 2017 at 5:19 PM, Dong Lin <li...@gmail.com>
> wrote:
> > > >
> > > > > Hey Jun,
> > > > >
> > > > > I see. Sounds good. Yeah it is probably simpler to leave this to
> > > another
> > > > > KIP in the future.
> > > > >
> > > > > Thanks for all the comments. Since there is no further comment in
> the
> > > > > community, I will open the voting thread.
> > > > >
> > > > > Thanks,
> > > > > Dong
> > > > >
> > > > > On Mon, Dec 11, 2017 at 5:37 PM, Jun Rao <ju...@confluent.io> wrote:
> > > > >
> > > > > > Hi, Dong,
> > > > > >
> > > > > > The case that I am thinking is network partitioning. Suppose one
> > > > deploys
> > > > > a
> > > > > > stretched cluster across multiple AZs in the same region. If the
> > > > machines
> > > > > > in one AZ can't communicate to brokers in other AZs due to a
> > network
> > > > > issue,
> > > > > > the brokers in that AZ won't get any new metadata.
> > > > > >
> > > > > > We can potentially solve this problem by requiring some kind of
> > > regular
> > > > > > heartbeats between the controller and the broker. This may need
> > some
> > > > more
> > > > > > thoughts. So, it's probably fine to leave this to another KIP in
> > the
> > > > > > future.
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > > On Mon, Dec 11, 2017 at 2:55 PM, Dong Lin <li...@gmail.com>
> > > wrote:
> > > > > >
> > > > > > > Hey Jun,
> > > > > > >
> > > > > > > Thanks for the comment. I am open to improve this KIP to
> address
> > > more
> > > > > > > problems. I probably need more help in understanding what is
> the
> > > > > current
> > > > > > > problem with consumer using outdated metadata and whether it is
> > > > easier
> > > > > to
> > > > > > > address it together with this KIP.
> > > > > > >
> > > > > > > I agree that a consumer can potentially talk to old leader for
> a
> > > long
> > > > > > time
> > > > > > > even after this KIP. But after this KIP, the consumer probably
> > > should
> > > > > not
> > > > > > > get OffetOutofRangeException and therefore will not cause
> offset
> > > > rewind
> > > > > > > issue. So the only problem is that consumer will not be able to
> > > fetch
> > > > > > data
> > > > > > > until it has updated metadata. It seems that this situation can
> > > only
> > > > > > happen
> > > > > > > if the broker is too slow in processing LeaderAndIsrRequest
> since
> > > > > > otherwise
> > > > > > > the consumer will be forced to update metadata due to
> > > > > > > NotLeaderForPartitionException. So the problem we are having
> > here
> > > is
> > > > > > that
> > > > > > > consumer will not be able to fetch data if some broker is too
> > slow
> > > in
> > > > > > > processing LeaderAndIsrRequest.
> > > > > > >
> > > > > > > Because Kafka propagates LeaderAndIsrRequest asynchronously to
> > all
> > > > > > brokers
> > > > > > > in the cluster, there will always be a period of time when
> > consumer
> > > > can
> > > > > > not
> > > > > > > fetch data for the partition during the leadership change. Thus
> > it
> > > > > seems
> > > > > > > more like a broker-side performance issue instead of
> client-side
> > > > > > > correctness issue. My gut feel is that it is not causing a
> much a
> > > > > problem
> > > > > > > as the problem to be fixed in this KIP. And if we were to
> address
> > > it,
> > > > > we
> > > > > > > probably need to make change in the broker side, e.g. with
> > > > prioritized
> > > > > > > queue for controller-related requests, which may be kind of
> > > > orthogonal
> > > > > to
> > > > > > > this KIP. I am not very sure it will be easier to address it
> with
> > > the
> > > > > > > change in this KIP. Do you have any recommendation?
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Dong
> > > > > > >
> > > > > > >
> > > > > > > On Mon, Dec 11, 2017 at 1:51 PM, Jun Rao <ju...@confluent.io>
> > wrote:
> > > > > > >
> > > > > > > > Hi, Dong,
> > > > > > > >
> > > > > > > > Thanks for the reply.
> > > > > > > >
> > > > > > > > My suggestion of forcing the metadata refresh from the
> > controller
> > > > may
> > > > > > not
> > > > > > > > work in general since the cached controller could be outdated
> > > too.
> > > > > The
> > > > > > > > general problem is that if a consumer's metadata is outdated,
> > it
> > > > may
> > > > > > get
> > > > > > > > stuck with the old leader for a long time. We can address the
> > > issue
> > > > > of
> > > > > > > > detecting outdated metadata in a separate KIP in the future
> if
> > > you
> > > > > > didn't
> > > > > > > > intend to address it in this KIP.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > >
> > > > > > > > Jun
> > > > > > > >
> > > > > > > >
> > > > > > > > On Sat, Dec 9, 2017 at 10:12 PM, Dong Lin <
> lindong28@gmail.com
> > >
> > > > > wrote:
> > > > > > > >
> > > > > > > > > Hey Jun,
> > > > > > > > >
> > > > > > > > > Thanks much for your comments. Given that client needs to
> > > > > > de-serialize
> > > > > > > > the
> > > > > > > > > metadata anyway, the extra overhead of checking the
> > > per-partition
> > > > > > > version
> > > > > > > > > for every partition should not be a big concern. Thus it
> > makes
> > > > > sense
> > > > > > to
> > > > > > > > use
> > > > > > > > > leader epoch as the per-partition version instead of
> > creating a
> > > > > > global
> > > > > > > > > metadata version. I will update the KIP to do that.
> > > > > > > > >
> > > > > > > > > Regarding the detection of outdated metadata, I think it is
> > > > > possible
> > > > > > to
> > > > > > > > > ensure that client gets latest metadata by fetching from
> > > > > controller.
> > > > > > > Note
> > > > > > > > > that this requires extra logic in the controller such that
> > > > > controller
> > > > > > > > > updates metadata directly in memory without requiring
> > > > > > > > > UpdateMetadataRequest. But I am not sure the main
> motivation
> > of
> > > > > this
> > > > > > at
> > > > > > > > > this moment. But this makes controller more like a
> bottleneck
> > > in
> > > > > the
> > > > > > > > > cluster which we probably want to avoid.
> > > > > > > > >
> > > > > > > > > I think we can probably keep the current way of ensuring
> > > metadata
> > > > > > > > > freshness. Currently client will be forced to refresh
> > metadata
> > > if
> > > > > > > broker
> > > > > > > > > returns error (e.g. NotLeaderForPartition) due to outdated
> > > > metadata
> > > > > > or
> > > > > > > if
> > > > > > > > > the metadata does not contain the partition that the client
> > > > needs.
> > > > > In
> > > > > > > the
> > > > > > > > > future, as you previously suggested, we can include
> > > per-partition
> > > > > > > > > leaderEpoch in the FetchRequest/ProduceRequest such that
> > broker
> > > > can
> > > > > > > > return
> > > > > > > > > error if the epoch is smaller than cached epoch in the
> > broker.
> > > > > Given
> > > > > > > that
> > > > > > > > > this adds more complexity to Kafka, I think we can probably
> > > think
> > > > > > about
> > > > > > > > > that leader when we have a specific use-case or problem to
> > > solve
> > > > > with
> > > > > > > > > up-to-date metadata. Does this sound OK?
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > > Dong
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Fri, Dec 8, 2017 at 3:53 PM, Jun Rao <ju...@confluent.io>
> > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hi, Dong,
> > > > > > > > > >
> > > > > > > > > > Thanks for the reply. A few more points below.
> > > > > > > > > >
> > > > > > > > > > For dealing with how to prevent a consumer switching
> from a
> > > new
> > > > > > > leader
> > > > > > > > to
> > > > > > > > > > an old leader, you suggestion that refreshes metadata on
> > > > consumer
> > > > > > > > restart
> > > > > > > > > > until it sees a metadata version >= the one associated
> with
> > > the
> > > > > > > offset
> > > > > > > > > > works too, as long as we guarantee that the cached
> metadata
> > > > > > versions
> > > > > > > on
> > > > > > > > > the
> > > > > > > > > > brokers only go up.
> > > > > > > > > >
> > > > > > > > > > The second discussion point is on whether the metadata
> > > > versioning
> > > > > > > > should
> > > > > > > > > be
> > > > > > > > > > per partition or global. For the partition level
> > versioning,
> > > > you
> > > > > > were
> > > > > > > > > > concerned about the performance. Given that metadata
> > updates
> > > > are
> > > > > > > rare,
> > > > > > > > I
> > > > > > > > > am
> > > > > > > > > > not sure if it's a big concern though. Doing a million if
> > > tests
> > > > > is
> > > > > > > > > probably
> > > > > > > > > > going to take less than 1ms. Another thing is that the
> > > metadata
> > > > > > > version
> > > > > > > > > > seems to need to survive controller failover. In your
> > current
> > > > > > > > approach, a
> > > > > > > > > > consumer may not be able to wait on the right version of
> > the
> > > > > > metadata
> > > > > > > > > after
> > > > > > > > > > the consumer restart since the metadata version may have
> > been
> > > > > > > recycled
> > > > > > > > on
> > > > > > > > > > the server side due to a controller failover while the
> > > consumer
> > > > > is
> > > > > > > > down.
> > > > > > > > > > The partition level leaderEpoch survives controller
> failure
> > > and
> > > > > > won't
> > > > > > > > > have
> > > > > > > > > > this issue.
> > > > > > > > > >
> > > > > > > > > > Lastly, neither your proposal nor mine addresses the
> issue
> > > how
> > > > to
> > > > > > > > > guarantee
> > > > > > > > > > a consumer to detect that is metadata is outdated.
> > Currently,
> > > > the
> > > > > > > > > consumer
> > > > > > > > > > is not guaranteed to fetch metadata from every broker
> > within
> > > > some
> > > > > > > > bounded
> > > > > > > > > > period of time. Maybe this is out of the scope of your
> KIP.
> > > But
> > > > > one
> > > > > > > > idea
> > > > > > > > > is
> > > > > > > > > > force the consumer to refresh metadata from the
> controller
> > > > > > > > periodically.
> > > > > > > > > >
> > > > > > > > > > Jun
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Thu, Dec 7, 2017 at 11:25 AM, Dong Lin <
> > > lindong28@gmail.com
> > > > >
> > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Hey Jun,
> > > > > > > > > > >
> > > > > > > > > > > Thanks much for the comments. Great point particularly
> > > > > regarding
> > > > > > > > (3). I
> > > > > > > > > > > haven't thought about this before.
> > > > > > > > > > >
> > > > > > > > > > > It seems that there are two possible ways where the
> > version
> > > > > > number
> > > > > > > > can
> > > > > > > > > be
> > > > > > > > > > > used. One solution is for client to check the version
> > > number
> > > > at
> > > > > > the
> > > > > > > > > time
> > > > > > > > > > it
> > > > > > > > > > > receives MetadataResponse. And if the version number in
> > the
> > > > > > > > > > > MetadataResponse is smaller than the version number in
> > the
> > > > > > client's
> > > > > > > > > > cache,
> > > > > > > > > > > the client will be forced to fetch metadata again.
> > Another
> > > > > > > solution,
> > > > > > > > > as
> > > > > > > > > > > you have suggested, is for broker to check the version
> > > number
> > > > > at
> > > > > > > the
> > > > > > > > > time
> > > > > > > > > > > it receives a request from client. The broker will
> reject
> > > the
> > > > > > > request
> > > > > > > > > if
> > > > > > > > > > > the version is smaller than the version in broker's
> > cache.
> > > > > > > > > > >
> > > > > > > > > > > I am not very sure that the second solution can address
> > the
> > > > > > problem
> > > > > > > > > here.
> > > > > > > > > > > In the scenario described in the JIRA ticket, broker's
> > > cache
> > > > > may
> > > > > > be
> > > > > > > > > > > outdated because it has not processed the
> > > LeaderAndIsrRequest
> > > > > > from
> > > > > > > > the
> > > > > > > > > > > controller. Thus it may still process client's request
> > even
> > > > if
> > > > > > the
> > > > > > > > > > version
> > > > > > > > > > > in client's request is actually outdated. Does this
> make
> > > > sense?
> > > > > > > > > > >
> > > > > > > > > > > IMO, it seems that we can address problem (3) by saving
> > the
> > > > > > > metadata
> > > > > > > > > > > version together with the offset. After consumer
> starts,
> > it
> > > > > will
> > > > > > > keep
> > > > > > > > > > > fetching metadata until the metadata version >= the
> > version
> > > > > saved
> > > > > > > > with
> > > > > > > > > > the
> > > > > > > > > > > offset of this partition.
> > > > > > > > > > >
> > > > > > > > > > > Regarding problems (1) and (2): Currently we use the
> > > version
> > > > > > number
> > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > > MetadataResponse to ensure that the metadata does not
> go
> > > back
> > > > > in
> > > > > > > > time.
> > > > > > > > > > > There are two alternative solutions to address problems
> > (1)
> > > > and
> > > > > > > (2).
> > > > > > > > > One
> > > > > > > > > > > solution is for client to enumerate all partitions in
> the
> > > > > > > > > > MetadataResponse,
> > > > > > > > > > > compare their epoch with those in the cached metadata,
> > and
> > > > > > rejects
> > > > > > > > the
> > > > > > > > > > > MetadataResponse iff any leader epoch is smaller. The
> > main
> > > > > > concern
> > > > > > > is
> > > > > > > > > > that
> > > > > > > > > > > MetadataResponse currently cached information of all
> > > > partitions
> > > > > > in
> > > > > > > > the
> > > > > > > > > > > entire cluster. It may slow down client's performance
> if
> > we
> > > > > were
> > > > > > to
> > > > > > > > do
> > > > > > > > > > it.
> > > > > > > > > > > The other solution is for client to enumerate
> partitions
> > > for
> > > > > only
> > > > > > > > > topics
> > > > > > > > > > > registered in the org.apache.kafka.clients.Metadata,
> > which
> > > > > will
> > > > > > be
> > > > > > > > an
> > > > > > > > > > > empty
> > > > > > > > > > > set for producer and the set of subscribed partitions
> for
> > > > > > consumer.
> > > > > > > > But
> > > > > > > > > > > this degrades to all topics if consumer subscribes to
> > > topics
> > > > in
> > > > > > the
> > > > > > > > > > cluster
> > > > > > > > > > > by pattern.
> > > > > > > > > > >
> > > > > > > > > > > Note that client will only be forced to update metadata
> > if
> > > > the
> > > > > > > > version
> > > > > > > > > in
> > > > > > > > > > > the MetadataResponse is smaller than the version in the
> > > > cached
> > > > > > > > > metadata.
> > > > > > > > > > In
> > > > > > > > > > > general it should not be a problem. It can be a problem
> > > only
> > > > if
> > > > > > > some
> > > > > > > > > > broker
> > > > > > > > > > > is particularly slower than other brokers in processing
> > > > > > > > > > > UpdateMetadataRequest. When this is the case, it means
> > that
> > > > the
> > > > > > > > broker
> > > > > > > > > is
> > > > > > > > > > > also particularly slower in processing
> > LeaderAndIsrRequest,
> > > > > which
> > > > > > > can
> > > > > > > > > > cause
> > > > > > > > > > > problem anyway because some partition will probably
> have
> > no
> > > > > > leader
> > > > > > > > > during
> > > > > > > > > > > this period. I am not sure problems (1) and (2) cause
> > more
> > > > > > problem
> > > > > > > > than
> > > > > > > > > > > what we already have.
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > > Dong
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Wed, Dec 6, 2017 at 6:42 PM, Jun Rao <
> > jun@confluent.io>
> > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Hi, Dong,
> > > > > > > > > > > >
> > > > > > > > > > > > Great finding on the issue. It's a real problem. A
> few
> > > > > comments
> > > > > > > > about
> > > > > > > > > > the
> > > > > > > > > > > > KIP. (1) I am not sure about updating
> > > > > controller_metadata_epoch
> > > > > > > on
> > > > > > > > > > every
> > > > > > > > > > > > UpdateMetadataRequest. Currently, the controller can
> > send
> > > > > > > > > > > > UpdateMetadataRequest when there is no actual
> metadata
> > > > > change.
> > > > > > > > Doing
> > > > > > > > > > this
> > > > > > > > > > > > may require unnecessary metadata refresh on the
> client.
> > > (2)
> > > > > > > > > > > > controller_metadata_epoch is global across all
> topics.
> > > This
> > > > > > means
> > > > > > > > > that
> > > > > > > > > > a
> > > > > > > > > > > > client may be forced to update its metadata even when
> > the
> > > > > > > metadata
> > > > > > > > > for
> > > > > > > > > > > the
> > > > > > > > > > > > topics that it cares haven't changed. (3) It doesn't
> > seem
> > > > > that
> > > > > > > the
> > > > > > > > > KIP
> > > > > > > > > > > > handles the corner case when a consumer is restarted.
> > > Say a
> > > > > > > > consumer
> > > > > > > > > > > reads
> > > > > > > > > > > > from the new leader, commits the offset and then is
> > > > > restarted.
> > > > > > On
> > > > > > > > > > > restart,
> > > > > > > > > > > > the consumer gets an outdated metadata and fetches
> from
> > > the
> > > > > old
> > > > > > > > > leader.
> > > > > > > > > > > > Then, the consumer will get into the offset out of
> > range
> > > > > issue.
> > > > > > > > > > > >
> > > > > > > > > > > > Given the above, I am thinking of the following
> > approach.
> > > > We
> > > > > > > > actually
> > > > > > > > > > > > already have metadata versioning at the partition
> > level.
> > > > Each
> > > > > > > > leader
> > > > > > > > > > has
> > > > > > > > > > > a
> > > > > > > > > > > > leader epoch which is monotonically increasing. We
> can
> > > > > > > potentially
> > > > > > > > > > > > propagate leader epoch back in the metadata response
> > and
> > > > the
> > > > > > > > clients
> > > > > > > > > > can
> > > > > > > > > > > > cache that. This solves the issue of (1) and (2). To
> > > solve
> > > > > (3),
> > > > > > > > when
> > > > > > > > > > > saving
> > > > > > > > > > > > an offset, we could save both an offset and the
> > > > corresponding
> > > > > > > > leader
> > > > > > > > > > > epoch.
> > > > > > > > > > > > When fetching the data, the consumer provides both
> the
> > > > offset
> > > > > > and
> > > > > > > > the
> > > > > > > > > > > > leader epoch. A leader will only serve the request if
> > its
> > > > > > leader
> > > > > > > > > epoch
> > > > > > > > > > is
> > > > > > > > > > > > equal to or greater than the leader epoch from the
> > > > consumer.
> > > > > To
> > > > > > > > > achieve
> > > > > > > > > > > > this, we need to change the fetch request protocol
> and
> > > the
> > > > > > offset
> > > > > > > > > > commit
> > > > > > > > > > > > api, which requires some more thoughts.
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks,
> > > > > > > > > > > >
> > > > > > > > > > > > Jun
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On Wed, Dec 6, 2017 at 10:57 AM, Dong Lin <
> > > > > lindong28@gmail.com
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Bump up the thread.
> > > > > > > > > > > > >
> > > > > > > > > > > > > It will be great to have more comments on whether
> we
> > > > should
> > > > > > do
> > > > > > > it
> > > > > > > > > or
> > > > > > > > > > > > > whether there is better way to address the
> motivation
> > > of
> > > > > this
> > > > > > > > KIP.
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Mon, Dec 4, 2017 at 3:09 PM, Dong Lin <
> > > > > > lindong28@gmail.com>
> > > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > I don't have an interesting rejected alternative
> > > > solution
> > > > > > to
> > > > > > > > put
> > > > > > > > > in
> > > > > > > > > > > the
> > > > > > > > > > > > > > KIP. If there is good alternative solution from
> > > anyone
> > > > in
> > > > > > > this
> > > > > > > > > > > thread,
> > > > > > > > > > > > I
> > > > > > > > > > > > > am
> > > > > > > > > > > > > > happy to discuss this and update the KIP
> > accordingly.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > Dong
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Mon, Dec 4, 2017 at 1:12 PM, Ted Yu <
> > > > > > yuzhihong@gmail.com>
> > > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >> It is clearer now.
> > > > > > > > > > > > > >>
> > > > > > > > > > > > > >> I noticed that Rejected Alternatives section is
> > > empty.
> > > > > > > > > > > > > >> Have you considered any alternative ?
> > > > > > > > > > > > > >>
> > > > > > > > > > > > > >> Cheers
> > > > > > > > > > > > > >>
> > > > > > > > > > > > > >> On Mon, Dec 4, 2017 at 1:07 PM, Dong Lin <
> > > > > > > lindong28@gmail.com
> > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > > > > > >>
> > > > > > > > > > > > > >> > Ted, thanks for catching this. I have updated
> > the
> > > > > > sentence
> > > > > > > > to
> > > > > > > > > > make
> > > > > > > > > > > > it
> > > > > > > > > > > > > >> > readable.
> > > > > > > > > > > > > >> >
> > > > > > > > > > > > > >> > Thanks,
> > > > > > > > > > > > > >> > Dong
> > > > > > > > > > > > > >> >
> > > > > > > > > > > > > >> > On Sat, Dec 2, 2017 at 3:05 PM, Ted Yu <
> > > > > > > yuzhihong@gmail.com
> > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > > > > > >> >
> > > > > > > > > > > > > >> > > bq. It the controller_epoch of the incoming
> > > > > > > > > MetadataResponse,
> > > > > > > > > > or
> > > > > > > > > > > > if
> > > > > > > > > > > > > >> the
> > > > > > > > > > > > > >> > > controller_epoch is the same but the
> > > > > > > > > controller_metadata_epoch
> > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > >> > > Can you update the above sentence so that
> the
> > > > > > intention
> > > > > > > is
> > > > > > > > > > > > clearer ?
> > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > >> > > Thanks
> > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > >> > > On Fri, Dec 1, 2017 at 6:33 PM, Dong Lin <
> > > > > > > > > lindong28@gmail.com
> > > > > > > > > > >
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > >> > > > Hi all,
> > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > >> > > > I have created KIP-232: Detect outdated
> > > metadata
> > > > > by
> > > > > > > > adding
> > > > > > > > > > > > > >> > > > ControllerMetadataEpoch field:
> > > > > > > > > > > > > >> > > > https://cwiki.apache.org/
> > > > > > > confluence/display/KAFKA/KIP-
> > > > > > > > > > > > > >> > > > 232%3A+Detect+outdated+
> metadata+by+adding+
> > > > > > > > > > > > > >> > ControllerMetadataEpoch+field
> > > > > > > > > > > > > >> > > > .
> > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > >> > > > The KIP proposes to add fields in
> > > > MetadataResponse
> > > > > > and
> > > > > > > > > > > > > >> > > > UpdateMetadataRequest so that client can
> > > reject
> > > > > > > outdated
> > > > > > > > > > > > metadata
> > > > > > > > > > > > > >> and
> > > > > > > > > > > > > >> > > avoid
> > > > > > > > > > > > > >> > > > unnecessary OffsetOutOfRangeException.
> > > Otherwise
> > > > > > there
> > > > > > > > is
> > > > > > > > > > > > > currently
> > > > > > > > > > > > > >> > race
> > > > > > > > > > > > > >> > > > condition that can cause consumer to reset
> > > > offset
> > > > > > > which
> > > > > > > > > > > > negatively
> > > > > > > > > > > > > >> > affect
> > > > > > > > > > > > > >> > > > the consumer's availability.
> > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > >> > > > Feedback and suggestions are welcome!
> > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > >> > > > Regards,
> > > > > > > > > > > > > >> > > > Dong
> > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > >> >
> > > > > > > > > > > > > >>
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Dong Lin <li...@gmail.com>.
Hey Jun,

Thanks much for your comments. Yeah I have not considered the case where
the offset is stored externally.

Based Jason's question, I think we probably have to use a global
metadata_epoch. And since we have a global metadata_epoch, this KIP
probably no longer needs the per-partition leader_epoch. Then we can use
two newly-added API in consumer that allows user to get the metadata_epoch
from consumer and wait for consumer to receive MetadataResponse whose
metadata_epoch >= the given metadata_epoch. These two APIs should address
the case where user stored offset externally. I have updated the KIP
accordingly. Could you take another look?

Thanks for all the comments.

Dong


On Tue, Dec 19, 2017 at 3:09 PM, Jun Rao <ju...@confluent.io> wrote:

> Hi, Dong,
>
> Thanks for the reply.
>
> 10. I was actually just thinking the case when the consumer consumes old
> data. If the current leader epoch is 3 and the consumer is consuming
> records generated in leader epoch 1, the epoch associated with the offset
> should be 1. However, as you pointed out, the fetch response currently
> includes the leader epoch for fetched data. So, this is already covered.
>
> 11. That's an interesting thought. What about the case when the offsets are
> stored externally? When we restart a consumer and seek to an externally
> stored offset, we won't know the leader epoch in the consumer. Do we need
> another request to retrieve the leader epoch based on an offset and make
> sure the info is up to date? Another related thing is that the leader epoch
> that we want to associate the offset with ideally should be the epoch when
> the data is fetched. For example, when all replicas lost data due to a
> power failure or when there is an unclean leader election, the leader epoch
> for a given offset may change over time on the broker. In those cases, a
> consumer's offset may be in range, but is not in the same leader epoch for
> the time when the data is fetched. We can potentially do a smarter offset
> reset in those cases if we remember the epoch when the data is fetched.
>
> Jun
>
>
>
> On Mon, Dec 18, 2017 at 1:58 PM, Dong Lin <li...@gmail.com> wrote:
>
> > Hey Jun,
> >
> > Thanks much for your comments. These are very thoughtful ideas. Please
> see
> > my comments below.
> >
> > On Thu, Dec 14, 2017 at 6:38 PM, Jun Rao <ju...@confluent.io> wrote:
> >
> > > Hi, Dong,
> > >
> > > Thanks for the update. A few more comments below.
> > >
> > > 10. It seems that we need to return the leader epoch in the fetch
> > response
> > > as well When fetching data, we could be fetching data from a leader
> epoch
> > > older than what's returned in the metadata response. So, we want to use
> > the
> > > leader epoch associated with the offset being fetched for committing
> > > offsets.
> > >
> >
> > It seems that we may have two separate issues here. The first issue is
> that
> > consumer uses metadata that is older than the one it uses before. The
> > second issue is that consumer uses metadata which is newer than the
> > corresponding leader epoch in the leader broker. We know that the
> > OffsetOutOfRangeException described in this KIP can be prevented by
> > avoiding the first issue. On the other hand, it seems that the
> > OffsetOffsetOutOfRangeException can still happen even if we avoid the
> > second issue -- if consumer uses an older version of metadata, the leader
> > epoch in its metadata may equal the leader epoch in the broker even if
> the
> > leader epoch in the broker is oudated.
> >
> > Given this understanding, I am not sure why we need to return the leader
> > epoch in the fetch response. As long as consumer's metadata is not going
> > back in version, I think we are good. Did I miss something here?
> >
> >
> > >
> > > 11. Should we now extend OffsetAndMetadata used in the offset commit
> api
> > in
> > > KafkaConsumer to include leader epoch? Similarly, should we return
> leader
> > > epoch in endOffsets(), beginningOffsets() and position()? We probably
> > need
> > > to think about how to make the api backward compatible.
> > >
> >
> > After thinking through this carefully, I think we probably don't want to
> > extend OffsetAndMetadata to include leader epoch because leader epoch is
> > kind of implementation detail which ideally should be hidden from user.
> The
> > consumer can include leader epoch in the OffsetCommitRequest after taking
> > offset from commitSync(final Map<TopicPartition, OffsetAndMetadata>
> > offsets). Similarly consumer can store leader epoch from
> > OffsetFetchResponse and only provide offset to user via
> > consumer.committed(topicPartition). This solution seems to work well and
> > we
> > don't have to make changes to consumer's public API. Does this sound OK?
> >
> >
> > >
> > > 12. It seems that we now need to store leader epoch in the offset
> topic.
> > > Could you include the new schema for the value of the offset topic and
> > add
> > > upgrade notes?
> >
> >
> > You are right. I have updated the KIP to specify the new schema for the
> > value of the offset topic. Can you take another look?
> >
> > For existing messages in the offset topic, leader_epoch will be missing.
> We
> > will use leader_epoch = -1 to indicate the missing leader_epoch. Then the
> > consumer behavior will be the same as it is now because any leader_epoch
> in
> > the MetadataResponse will be larger than the leader_epoch = -1 in the
> > OffetFetchResponse. Thus we don't need specific procedure for upgrades
> due
> > to this change in the offset topic schema. By "upgrade nodes", do you
> mean
> > the sentences we need to include in the upgrade.html in the PR later?
> >
> >
> > >
> > > Jun
> > >
> > >
> > > On Tue, Dec 12, 2017 at 5:19 PM, Dong Lin <li...@gmail.com> wrote:
> > >
> > > > Hey Jun,
> > > >
> > > > I see. Sounds good. Yeah it is probably simpler to leave this to
> > another
> > > > KIP in the future.
> > > >
> > > > Thanks for all the comments. Since there is no further comment in the
> > > > community, I will open the voting thread.
> > > >
> > > > Thanks,
> > > > Dong
> > > >
> > > > On Mon, Dec 11, 2017 at 5:37 PM, Jun Rao <ju...@confluent.io> wrote:
> > > >
> > > > > Hi, Dong,
> > > > >
> > > > > The case that I am thinking is network partitioning. Suppose one
> > > deploys
> > > > a
> > > > > stretched cluster across multiple AZs in the same region. If the
> > > machines
> > > > > in one AZ can't communicate to brokers in other AZs due to a
> network
> > > > issue,
> > > > > the brokers in that AZ won't get any new metadata.
> > > > >
> > > > > We can potentially solve this problem by requiring some kind of
> > regular
> > > > > heartbeats between the controller and the broker. This may need
> some
> > > more
> > > > > thoughts. So, it's probably fine to leave this to another KIP in
> the
> > > > > future.
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Jun
> > > > >
> > > > > On Mon, Dec 11, 2017 at 2:55 PM, Dong Lin <li...@gmail.com>
> > wrote:
> > > > >
> > > > > > Hey Jun,
> > > > > >
> > > > > > Thanks for the comment. I am open to improve this KIP to address
> > more
> > > > > > problems. I probably need more help in understanding what is the
> > > > current
> > > > > > problem with consumer using outdated metadata and whether it is
> > > easier
> > > > to
> > > > > > address it together with this KIP.
> > > > > >
> > > > > > I agree that a consumer can potentially talk to old leader for a
> > long
> > > > > time
> > > > > > even after this KIP. But after this KIP, the consumer probably
> > should
> > > > not
> > > > > > get OffetOutofRangeException and therefore will not cause offset
> > > rewind
> > > > > > issue. So the only problem is that consumer will not be able to
> > fetch
> > > > > data
> > > > > > until it has updated metadata. It seems that this situation can
> > only
> > > > > happen
> > > > > > if the broker is too slow in processing LeaderAndIsrRequest since
> > > > > otherwise
> > > > > > the consumer will be forced to update metadata due to
> > > > > > NotLeaderForPartitionException. So the problem we are having
> here
> > is
> > > > > that
> > > > > > consumer will not be able to fetch data if some broker is too
> slow
> > in
> > > > > > processing LeaderAndIsrRequest.
> > > > > >
> > > > > > Because Kafka propagates LeaderAndIsrRequest asynchronously to
> all
> > > > > brokers
> > > > > > in the cluster, there will always be a period of time when
> consumer
> > > can
> > > > > not
> > > > > > fetch data for the partition during the leadership change. Thus
> it
> > > > seems
> > > > > > more like a broker-side performance issue instead of client-side
> > > > > > correctness issue. My gut feel is that it is not causing a much a
> > > > problem
> > > > > > as the problem to be fixed in this KIP. And if we were to address
> > it,
> > > > we
> > > > > > probably need to make change in the broker side, e.g. with
> > > prioritized
> > > > > > queue for controller-related requests, which may be kind of
> > > orthogonal
> > > > to
> > > > > > this KIP. I am not very sure it will be easier to address it with
> > the
> > > > > > change in this KIP. Do you have any recommendation?
> > > > > >
> > > > > > Thanks,
> > > > > > Dong
> > > > > >
> > > > > >
> > > > > > On Mon, Dec 11, 2017 at 1:51 PM, Jun Rao <ju...@confluent.io>
> wrote:
> > > > > >
> > > > > > > Hi, Dong,
> > > > > > >
> > > > > > > Thanks for the reply.
> > > > > > >
> > > > > > > My suggestion of forcing the metadata refresh from the
> controller
> > > may
> > > > > not
> > > > > > > work in general since the cached controller could be outdated
> > too.
> > > > The
> > > > > > > general problem is that if a consumer's metadata is outdated,
> it
> > > may
> > > > > get
> > > > > > > stuck with the old leader for a long time. We can address the
> > issue
> > > > of
> > > > > > > detecting outdated metadata in a separate KIP in the future if
> > you
> > > > > didn't
> > > > > > > intend to address it in this KIP.
> > > > > > >
> > > > > > > Thanks,
> > > > > > >
> > > > > > > Jun
> > > > > > >
> > > > > > >
> > > > > > > On Sat, Dec 9, 2017 at 10:12 PM, Dong Lin <lindong28@gmail.com
> >
> > > > wrote:
> > > > > > >
> > > > > > > > Hey Jun,
> > > > > > > >
> > > > > > > > Thanks much for your comments. Given that client needs to
> > > > > de-serialize
> > > > > > > the
> > > > > > > > metadata anyway, the extra overhead of checking the
> > per-partition
> > > > > > version
> > > > > > > > for every partition should not be a big concern. Thus it
> makes
> > > > sense
> > > > > to
> > > > > > > use
> > > > > > > > leader epoch as the per-partition version instead of
> creating a
> > > > > global
> > > > > > > > metadata version. I will update the KIP to do that.
> > > > > > > >
> > > > > > > > Regarding the detection of outdated metadata, I think it is
> > > > possible
> > > > > to
> > > > > > > > ensure that client gets latest metadata by fetching from
> > > > controller.
> > > > > > Note
> > > > > > > > that this requires extra logic in the controller such that
> > > > controller
> > > > > > > > updates metadata directly in memory without requiring
> > > > > > > > UpdateMetadataRequest. But I am not sure the main motivation
> of
> > > > this
> > > > > at
> > > > > > > > this moment. But this makes controller more like a bottleneck
> > in
> > > > the
> > > > > > > > cluster which we probably want to avoid.
> > > > > > > >
> > > > > > > > I think we can probably keep the current way of ensuring
> > metadata
> > > > > > > > freshness. Currently client will be forced to refresh
> metadata
> > if
> > > > > > broker
> > > > > > > > returns error (e.g. NotLeaderForPartition) due to outdated
> > > metadata
> > > > > or
> > > > > > if
> > > > > > > > the metadata does not contain the partition that the client
> > > needs.
> > > > In
> > > > > > the
> > > > > > > > future, as you previously suggested, we can include
> > per-partition
> > > > > > > > leaderEpoch in the FetchRequest/ProduceRequest such that
> broker
> > > can
> > > > > > > return
> > > > > > > > error if the epoch is smaller than cached epoch in the
> broker.
> > > > Given
> > > > > > that
> > > > > > > > this adds more complexity to Kafka, I think we can probably
> > think
> > > > > about
> > > > > > > > that leader when we have a specific use-case or problem to
> > solve
> > > > with
> > > > > > > > up-to-date metadata. Does this sound OK?
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Dong
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > On Fri, Dec 8, 2017 at 3:53 PM, Jun Rao <ju...@confluent.io>
> > > wrote:
> > > > > > > >
> > > > > > > > > Hi, Dong,
> > > > > > > > >
> > > > > > > > > Thanks for the reply. A few more points below.
> > > > > > > > >
> > > > > > > > > For dealing with how to prevent a consumer switching from a
> > new
> > > > > > leader
> > > > > > > to
> > > > > > > > > an old leader, you suggestion that refreshes metadata on
> > > consumer
> > > > > > > restart
> > > > > > > > > until it sees a metadata version >= the one associated with
> > the
> > > > > > offset
> > > > > > > > > works too, as long as we guarantee that the cached metadata
> > > > > versions
> > > > > > on
> > > > > > > > the
> > > > > > > > > brokers only go up.
> > > > > > > > >
> > > > > > > > > The second discussion point is on whether the metadata
> > > versioning
> > > > > > > should
> > > > > > > > be
> > > > > > > > > per partition or global. For the partition level
> versioning,
> > > you
> > > > > were
> > > > > > > > > concerned about the performance. Given that metadata
> updates
> > > are
> > > > > > rare,
> > > > > > > I
> > > > > > > > am
> > > > > > > > > not sure if it's a big concern though. Doing a million if
> > tests
> > > > is
> > > > > > > > probably
> > > > > > > > > going to take less than 1ms. Another thing is that the
> > metadata
> > > > > > version
> > > > > > > > > seems to need to survive controller failover. In your
> current
> > > > > > > approach, a
> > > > > > > > > consumer may not be able to wait on the right version of
> the
> > > > > metadata
> > > > > > > > after
> > > > > > > > > the consumer restart since the metadata version may have
> been
> > > > > > recycled
> > > > > > > on
> > > > > > > > > the server side due to a controller failover while the
> > consumer
> > > > is
> > > > > > > down.
> > > > > > > > > The partition level leaderEpoch survives controller failure
> > and
> > > > > won't
> > > > > > > > have
> > > > > > > > > this issue.
> > > > > > > > >
> > > > > > > > > Lastly, neither your proposal nor mine addresses the issue
> > how
> > > to
> > > > > > > > guarantee
> > > > > > > > > a consumer to detect that is metadata is outdated.
> Currently,
> > > the
> > > > > > > > consumer
> > > > > > > > > is not guaranteed to fetch metadata from every broker
> within
> > > some
> > > > > > > bounded
> > > > > > > > > period of time. Maybe this is out of the scope of your KIP.
> > But
> > > > one
> > > > > > > idea
> > > > > > > > is
> > > > > > > > > force the consumer to refresh metadata from the controller
> > > > > > > periodically.
> > > > > > > > >
> > > > > > > > > Jun
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Thu, Dec 7, 2017 at 11:25 AM, Dong Lin <
> > lindong28@gmail.com
> > > >
> > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hey Jun,
> > > > > > > > > >
> > > > > > > > > > Thanks much for the comments. Great point particularly
> > > > regarding
> > > > > > > (3). I
> > > > > > > > > > haven't thought about this before.
> > > > > > > > > >
> > > > > > > > > > It seems that there are two possible ways where the
> version
> > > > > number
> > > > > > > can
> > > > > > > > be
> > > > > > > > > > used. One solution is for client to check the version
> > number
> > > at
> > > > > the
> > > > > > > > time
> > > > > > > > > it
> > > > > > > > > > receives MetadataResponse. And if the version number in
> the
> > > > > > > > > > MetadataResponse is smaller than the version number in
> the
> > > > > client's
> > > > > > > > > cache,
> > > > > > > > > > the client will be forced to fetch metadata again.
> Another
> > > > > > solution,
> > > > > > > > as
> > > > > > > > > > you have suggested, is for broker to check the version
> > number
> > > > at
> > > > > > the
> > > > > > > > time
> > > > > > > > > > it receives a request from client. The broker will reject
> > the
> > > > > > request
> > > > > > > > if
> > > > > > > > > > the version is smaller than the version in broker's
> cache.
> > > > > > > > > >
> > > > > > > > > > I am not very sure that the second solution can address
> the
> > > > > problem
> > > > > > > > here.
> > > > > > > > > > In the scenario described in the JIRA ticket, broker's
> > cache
> > > > may
> > > > > be
> > > > > > > > > > outdated because it has not processed the
> > LeaderAndIsrRequest
> > > > > from
> > > > > > > the
> > > > > > > > > > controller. Thus it may still process client's request
> even
> > > if
> > > > > the
> > > > > > > > > version
> > > > > > > > > > in client's request is actually outdated. Does this make
> > > sense?
> > > > > > > > > >
> > > > > > > > > > IMO, it seems that we can address problem (3) by saving
> the
> > > > > > metadata
> > > > > > > > > > version together with the offset. After consumer starts,
> it
> > > > will
> > > > > > keep
> > > > > > > > > > fetching metadata until the metadata version >= the
> version
> > > > saved
> > > > > > > with
> > > > > > > > > the
> > > > > > > > > > offset of this partition.
> > > > > > > > > >
> > > > > > > > > > Regarding problems (1) and (2): Currently we use the
> > version
> > > > > number
> > > > > > > in
> > > > > > > > > the
> > > > > > > > > > MetadataResponse to ensure that the metadata does not go
> > back
> > > > in
> > > > > > > time.
> > > > > > > > > > There are two alternative solutions to address problems
> (1)
> > > and
> > > > > > (2).
> > > > > > > > One
> > > > > > > > > > solution is for client to enumerate all partitions in the
> > > > > > > > > MetadataResponse,
> > > > > > > > > > compare their epoch with those in the cached metadata,
> and
> > > > > rejects
> > > > > > > the
> > > > > > > > > > MetadataResponse iff any leader epoch is smaller. The
> main
> > > > > concern
> > > > > > is
> > > > > > > > > that
> > > > > > > > > > MetadataResponse currently cached information of all
> > > partitions
> > > > > in
> > > > > > > the
> > > > > > > > > > entire cluster. It may slow down client's performance if
> we
> > > > were
> > > > > to
> > > > > > > do
> > > > > > > > > it.
> > > > > > > > > > The other solution is for client to enumerate partitions
> > for
> > > > only
> > > > > > > > topics
> > > > > > > > > > registered in the org.apache.kafka.clients.Metadata,
> which
> > > > will
> > > > > be
> > > > > > > an
> > > > > > > > > > empty
> > > > > > > > > > set for producer and the set of subscribed partitions for
> > > > > consumer.
> > > > > > > But
> > > > > > > > > > this degrades to all topics if consumer subscribes to
> > topics
> > > in
> > > > > the
> > > > > > > > > cluster
> > > > > > > > > > by pattern.
> > > > > > > > > >
> > > > > > > > > > Note that client will only be forced to update metadata
> if
> > > the
> > > > > > > version
> > > > > > > > in
> > > > > > > > > > the MetadataResponse is smaller than the version in the
> > > cached
> > > > > > > > metadata.
> > > > > > > > > In
> > > > > > > > > > general it should not be a problem. It can be a problem
> > only
> > > if
> > > > > > some
> > > > > > > > > broker
> > > > > > > > > > is particularly slower than other brokers in processing
> > > > > > > > > > UpdateMetadataRequest. When this is the case, it means
> that
> > > the
> > > > > > > broker
> > > > > > > > is
> > > > > > > > > > also particularly slower in processing
> LeaderAndIsrRequest,
> > > > which
> > > > > > can
> > > > > > > > > cause
> > > > > > > > > > problem anyway because some partition will probably have
> no
> > > > > leader
> > > > > > > > during
> > > > > > > > > > this period. I am not sure problems (1) and (2) cause
> more
> > > > > problem
> > > > > > > than
> > > > > > > > > > what we already have.
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > > Dong
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Wed, Dec 6, 2017 at 6:42 PM, Jun Rao <
> jun@confluent.io>
> > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Hi, Dong,
> > > > > > > > > > >
> > > > > > > > > > > Great finding on the issue. It's a real problem. A few
> > > > comments
> > > > > > > about
> > > > > > > > > the
> > > > > > > > > > > KIP. (1) I am not sure about updating
> > > > controller_metadata_epoch
> > > > > > on
> > > > > > > > > every
> > > > > > > > > > > UpdateMetadataRequest. Currently, the controller can
> send
> > > > > > > > > > > UpdateMetadataRequest when there is no actual metadata
> > > > change.
> > > > > > > Doing
> > > > > > > > > this
> > > > > > > > > > > may require unnecessary metadata refresh on the client.
> > (2)
> > > > > > > > > > > controller_metadata_epoch is global across all topics.
> > This
> > > > > means
> > > > > > > > that
> > > > > > > > > a
> > > > > > > > > > > client may be forced to update its metadata even when
> the
> > > > > > metadata
> > > > > > > > for
> > > > > > > > > > the
> > > > > > > > > > > topics that it cares haven't changed. (3) It doesn't
> seem
> > > > that
> > > > > > the
> > > > > > > > KIP
> > > > > > > > > > > handles the corner case when a consumer is restarted.
> > Say a
> > > > > > > consumer
> > > > > > > > > > reads
> > > > > > > > > > > from the new leader, commits the offset and then is
> > > > restarted.
> > > > > On
> > > > > > > > > > restart,
> > > > > > > > > > > the consumer gets an outdated metadata and fetches from
> > the
> > > > old
> > > > > > > > leader.
> > > > > > > > > > > Then, the consumer will get into the offset out of
> range
> > > > issue.
> > > > > > > > > > >
> > > > > > > > > > > Given the above, I am thinking of the following
> approach.
> > > We
> > > > > > > actually
> > > > > > > > > > > already have metadata versioning at the partition
> level.
> > > Each
> > > > > > > leader
> > > > > > > > > has
> > > > > > > > > > a
> > > > > > > > > > > leader epoch which is monotonically increasing. We can
> > > > > > potentially
> > > > > > > > > > > propagate leader epoch back in the metadata response
> and
> > > the
> > > > > > > clients
> > > > > > > > > can
> > > > > > > > > > > cache that. This solves the issue of (1) and (2). To
> > solve
> > > > (3),
> > > > > > > when
> > > > > > > > > > saving
> > > > > > > > > > > an offset, we could save both an offset and the
> > > corresponding
> > > > > > > leader
> > > > > > > > > > epoch.
> > > > > > > > > > > When fetching the data, the consumer provides both the
> > > offset
> > > > > and
> > > > > > > the
> > > > > > > > > > > leader epoch. A leader will only serve the request if
> its
> > > > > leader
> > > > > > > > epoch
> > > > > > > > > is
> > > > > > > > > > > equal to or greater than the leader epoch from the
> > > consumer.
> > > > To
> > > > > > > > achieve
> > > > > > > > > > > this, we need to change the fetch request protocol and
> > the
> > > > > offset
> > > > > > > > > commit
> > > > > > > > > > > api, which requires some more thoughts.
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > >
> > > > > > > > > > > Jun
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Wed, Dec 6, 2017 at 10:57 AM, Dong Lin <
> > > > lindong28@gmail.com
> > > > > >
> > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Bump up the thread.
> > > > > > > > > > > >
> > > > > > > > > > > > It will be great to have more comments on whether we
> > > should
> > > > > do
> > > > > > it
> > > > > > > > or
> > > > > > > > > > > > whether there is better way to address the motivation
> > of
> > > > this
> > > > > > > KIP.
> > > > > > > > > > > >
> > > > > > > > > > > > On Mon, Dec 4, 2017 at 3:09 PM, Dong Lin <
> > > > > lindong28@gmail.com>
> > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > I don't have an interesting rejected alternative
> > > solution
> > > > > to
> > > > > > > put
> > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > > > > KIP. If there is good alternative solution from
> > anyone
> > > in
> > > > > > this
> > > > > > > > > > thread,
> > > > > > > > > > > I
> > > > > > > > > > > > am
> > > > > > > > > > > > > happy to discuss this and update the KIP
> accordingly.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > Dong
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Mon, Dec 4, 2017 at 1:12 PM, Ted Yu <
> > > > > yuzhihong@gmail.com>
> > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > >> It is clearer now.
> > > > > > > > > > > > >>
> > > > > > > > > > > > >> I noticed that Rejected Alternatives section is
> > empty.
> > > > > > > > > > > > >> Have you considered any alternative ?
> > > > > > > > > > > > >>
> > > > > > > > > > > > >> Cheers
> > > > > > > > > > > > >>
> > > > > > > > > > > > >> On Mon, Dec 4, 2017 at 1:07 PM, Dong Lin <
> > > > > > lindong28@gmail.com
> > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > > > > >>
> > > > > > > > > > > > >> > Ted, thanks for catching this. I have updated
> the
> > > > > sentence
> > > > > > > to
> > > > > > > > > make
> > > > > > > > > > > it
> > > > > > > > > > > > >> > readable.
> > > > > > > > > > > > >> >
> > > > > > > > > > > > >> > Thanks,
> > > > > > > > > > > > >> > Dong
> > > > > > > > > > > > >> >
> > > > > > > > > > > > >> > On Sat, Dec 2, 2017 at 3:05 PM, Ted Yu <
> > > > > > yuzhihong@gmail.com
> > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > > > > >> >
> > > > > > > > > > > > >> > > bq. It the controller_epoch of the incoming
> > > > > > > > MetadataResponse,
> > > > > > > > > or
> > > > > > > > > > > if
> > > > > > > > > > > > >> the
> > > > > > > > > > > > >> > > controller_epoch is the same but the
> > > > > > > > controller_metadata_epoch
> > > > > > > > > > > > >> > >
> > > > > > > > > > > > >> > > Can you update the above sentence so that the
> > > > > intention
> > > > > > is
> > > > > > > > > > > clearer ?
> > > > > > > > > > > > >> > >
> > > > > > > > > > > > >> > > Thanks
> > > > > > > > > > > > >> > >
> > > > > > > > > > > > >> > > On Fri, Dec 1, 2017 at 6:33 PM, Dong Lin <
> > > > > > > > lindong28@gmail.com
> > > > > > > > > >
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > >> > >
> > > > > > > > > > > > >> > > > Hi all,
> > > > > > > > > > > > >> > > >
> > > > > > > > > > > > >> > > > I have created KIP-232: Detect outdated
> > metadata
> > > > by
> > > > > > > adding
> > > > > > > > > > > > >> > > > ControllerMetadataEpoch field:
> > > > > > > > > > > > >> > > > https://cwiki.apache.org/
> > > > > > confluence/display/KAFKA/KIP-
> > > > > > > > > > > > >> > > > 232%3A+Detect+outdated+metadata+by+adding+
> > > > > > > > > > > > >> > ControllerMetadataEpoch+field
> > > > > > > > > > > > >> > > > .
> > > > > > > > > > > > >> > > >
> > > > > > > > > > > > >> > > > The KIP proposes to add fields in
> > > MetadataResponse
> > > > > and
> > > > > > > > > > > > >> > > > UpdateMetadataRequest so that client can
> > reject
> > > > > > outdated
> > > > > > > > > > > metadata
> > > > > > > > > > > > >> and
> > > > > > > > > > > > >> > > avoid
> > > > > > > > > > > > >> > > > unnecessary OffsetOutOfRangeException.
> > Otherwise
> > > > > there
> > > > > > > is
> > > > > > > > > > > > currently
> > > > > > > > > > > > >> > race
> > > > > > > > > > > > >> > > > condition that can cause consumer to reset
> > > offset
> > > > > > which
> > > > > > > > > > > negatively
> > > > > > > > > > > > >> > affect
> > > > > > > > > > > > >> > > > the consumer's availability.
> > > > > > > > > > > > >> > > >
> > > > > > > > > > > > >> > > > Feedback and suggestions are welcome!
> > > > > > > > > > > > >> > > >
> > > > > > > > > > > > >> > > > Regards,
> > > > > > > > > > > > >> > > > Dong
> > > > > > > > > > > > >> > > >
> > > > > > > > > > > > >> > >
> > > > > > > > > > > > >> >
> > > > > > > > > > > > >>
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Jun Rao <ju...@confluent.io>.
Hi, Dong,

Thanks for the reply.

10. I was actually just thinking the case when the consumer consumes old
data. If the current leader epoch is 3 and the consumer is consuming
records generated in leader epoch 1, the epoch associated with the offset
should be 1. However, as you pointed out, the fetch response currently
includes the leader epoch for fetched data. So, this is already covered.

11. That's an interesting thought. What about the case when the offsets are
stored externally? When we restart a consumer and seek to an externally
stored offset, we won't know the leader epoch in the consumer. Do we need
another request to retrieve the leader epoch based on an offset and make
sure the info is up to date? Another related thing is that the leader epoch
that we want to associate the offset with ideally should be the epoch when
the data is fetched. For example, when all replicas lost data due to a
power failure or when there is an unclean leader election, the leader epoch
for a given offset may change over time on the broker. In those cases, a
consumer's offset may be in range, but is not in the same leader epoch for
the time when the data is fetched. We can potentially do a smarter offset
reset in those cases if we remember the epoch when the data is fetched.

Jun



On Mon, Dec 18, 2017 at 1:58 PM, Dong Lin <li...@gmail.com> wrote:

> Hey Jun,
>
> Thanks much for your comments. These are very thoughtful ideas. Please see
> my comments below.
>
> On Thu, Dec 14, 2017 at 6:38 PM, Jun Rao <ju...@confluent.io> wrote:
>
> > Hi, Dong,
> >
> > Thanks for the update. A few more comments below.
> >
> > 10. It seems that we need to return the leader epoch in the fetch
> response
> > as well When fetching data, we could be fetching data from a leader epoch
> > older than what's returned in the metadata response. So, we want to use
> the
> > leader epoch associated with the offset being fetched for committing
> > offsets.
> >
>
> It seems that we may have two separate issues here. The first issue is that
> consumer uses metadata that is older than the one it uses before. The
> second issue is that consumer uses metadata which is newer than the
> corresponding leader epoch in the leader broker. We know that the
> OffsetOutOfRangeException described in this KIP can be prevented by
> avoiding the first issue. On the other hand, it seems that the
> OffsetOffsetOutOfRangeException can still happen even if we avoid the
> second issue -- if consumer uses an older version of metadata, the leader
> epoch in its metadata may equal the leader epoch in the broker even if the
> leader epoch in the broker is oudated.
>
> Given this understanding, I am not sure why we need to return the leader
> epoch in the fetch response. As long as consumer's metadata is not going
> back in version, I think we are good. Did I miss something here?
>
>
> >
> > 11. Should we now extend OffsetAndMetadata used in the offset commit api
> in
> > KafkaConsumer to include leader epoch? Similarly, should we return leader
> > epoch in endOffsets(), beginningOffsets() and position()? We probably
> need
> > to think about how to make the api backward compatible.
> >
>
> After thinking through this carefully, I think we probably don't want to
> extend OffsetAndMetadata to include leader epoch because leader epoch is
> kind of implementation detail which ideally should be hidden from user. The
> consumer can include leader epoch in the OffsetCommitRequest after taking
> offset from commitSync(final Map<TopicPartition, OffsetAndMetadata>
> offsets). Similarly consumer can store leader epoch from
> OffsetFetchResponse and only provide offset to user via
> consumer.committed(topicPartition). This solution seems to work well and
> we
> don't have to make changes to consumer's public API. Does this sound OK?
>
>
> >
> > 12. It seems that we now need to store leader epoch in the offset topic.
> > Could you include the new schema for the value of the offset topic and
> add
> > upgrade notes?
>
>
> You are right. I have updated the KIP to specify the new schema for the
> value of the offset topic. Can you take another look?
>
> For existing messages in the offset topic, leader_epoch will be missing. We
> will use leader_epoch = -1 to indicate the missing leader_epoch. Then the
> consumer behavior will be the same as it is now because any leader_epoch in
> the MetadataResponse will be larger than the leader_epoch = -1 in the
> OffetFetchResponse. Thus we don't need specific procedure for upgrades due
> to this change in the offset topic schema. By "upgrade nodes", do you mean
> the sentences we need to include in the upgrade.html in the PR later?
>
>
> >
> > Jun
> >
> >
> > On Tue, Dec 12, 2017 at 5:19 PM, Dong Lin <li...@gmail.com> wrote:
> >
> > > Hey Jun,
> > >
> > > I see. Sounds good. Yeah it is probably simpler to leave this to
> another
> > > KIP in the future.
> > >
> > > Thanks for all the comments. Since there is no further comment in the
> > > community, I will open the voting thread.
> > >
> > > Thanks,
> > > Dong
> > >
> > > On Mon, Dec 11, 2017 at 5:37 PM, Jun Rao <ju...@confluent.io> wrote:
> > >
> > > > Hi, Dong,
> > > >
> > > > The case that I am thinking is network partitioning. Suppose one
> > deploys
> > > a
> > > > stretched cluster across multiple AZs in the same region. If the
> > machines
> > > > in one AZ can't communicate to brokers in other AZs due to a network
> > > issue,
> > > > the brokers in that AZ won't get any new metadata.
> > > >
> > > > We can potentially solve this problem by requiring some kind of
> regular
> > > > heartbeats between the controller and the broker. This may need some
> > more
> > > > thoughts. So, it's probably fine to leave this to another KIP in the
> > > > future.
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > > On Mon, Dec 11, 2017 at 2:55 PM, Dong Lin <li...@gmail.com>
> wrote:
> > > >
> > > > > Hey Jun,
> > > > >
> > > > > Thanks for the comment. I am open to improve this KIP to address
> more
> > > > > problems. I probably need more help in understanding what is the
> > > current
> > > > > problem with consumer using outdated metadata and whether it is
> > easier
> > > to
> > > > > address it together with this KIP.
> > > > >
> > > > > I agree that a consumer can potentially talk to old leader for a
> long
> > > > time
> > > > > even after this KIP. But after this KIP, the consumer probably
> should
> > > not
> > > > > get OffetOutofRangeException and therefore will not cause offset
> > rewind
> > > > > issue. So the only problem is that consumer will not be able to
> fetch
> > > > data
> > > > > until it has updated metadata. It seems that this situation can
> only
> > > > happen
> > > > > if the broker is too slow in processing LeaderAndIsrRequest since
> > > > otherwise
> > > > > the consumer will be forced to update metadata due to
> > > > > NotLeaderForPartitionException. So the problem we are having here
> is
> > > > that
> > > > > consumer will not be able to fetch data if some broker is too slow
> in
> > > > > processing LeaderAndIsrRequest.
> > > > >
> > > > > Because Kafka propagates LeaderAndIsrRequest asynchronously to all
> > > > brokers
> > > > > in the cluster, there will always be a period of time when consumer
> > can
> > > > not
> > > > > fetch data for the partition during the leadership change. Thus it
> > > seems
> > > > > more like a broker-side performance issue instead of client-side
> > > > > correctness issue. My gut feel is that it is not causing a much a
> > > problem
> > > > > as the problem to be fixed in this KIP. And if we were to address
> it,
> > > we
> > > > > probably need to make change in the broker side, e.g. with
> > prioritized
> > > > > queue for controller-related requests, which may be kind of
> > orthogonal
> > > to
> > > > > this KIP. I am not very sure it will be easier to address it with
> the
> > > > > change in this KIP. Do you have any recommendation?
> > > > >
> > > > > Thanks,
> > > > > Dong
> > > > >
> > > > >
> > > > > On Mon, Dec 11, 2017 at 1:51 PM, Jun Rao <ju...@confluent.io> wrote:
> > > > >
> > > > > > Hi, Dong,
> > > > > >
> > > > > > Thanks for the reply.
> > > > > >
> > > > > > My suggestion of forcing the metadata refresh from the controller
> > may
> > > > not
> > > > > > work in general since the cached controller could be outdated
> too.
> > > The
> > > > > > general problem is that if a consumer's metadata is outdated, it
> > may
> > > > get
> > > > > > stuck with the old leader for a long time. We can address the
> issue
> > > of
> > > > > > detecting outdated metadata in a separate KIP in the future if
> you
> > > > didn't
> > > > > > intend to address it in this KIP.
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > >
> > > > > > On Sat, Dec 9, 2017 at 10:12 PM, Dong Lin <li...@gmail.com>
> > > wrote:
> > > > > >
> > > > > > > Hey Jun,
> > > > > > >
> > > > > > > Thanks much for your comments. Given that client needs to
> > > > de-serialize
> > > > > > the
> > > > > > > metadata anyway, the extra overhead of checking the
> per-partition
> > > > > version
> > > > > > > for every partition should not be a big concern. Thus it makes
> > > sense
> > > > to
> > > > > > use
> > > > > > > leader epoch as the per-partition version instead of creating a
> > > > global
> > > > > > > metadata version. I will update the KIP to do that.
> > > > > > >
> > > > > > > Regarding the detection of outdated metadata, I think it is
> > > possible
> > > > to
> > > > > > > ensure that client gets latest metadata by fetching from
> > > controller.
> > > > > Note
> > > > > > > that this requires extra logic in the controller such that
> > > controller
> > > > > > > updates metadata directly in memory without requiring
> > > > > > > UpdateMetadataRequest. But I am not sure the main motivation of
> > > this
> > > > at
> > > > > > > this moment. But this makes controller more like a bottleneck
> in
> > > the
> > > > > > > cluster which we probably want to avoid.
> > > > > > >
> > > > > > > I think we can probably keep the current way of ensuring
> metadata
> > > > > > > freshness. Currently client will be forced to refresh metadata
> if
> > > > > broker
> > > > > > > returns error (e.g. NotLeaderForPartition) due to outdated
> > metadata
> > > > or
> > > > > if
> > > > > > > the metadata does not contain the partition that the client
> > needs.
> > > In
> > > > > the
> > > > > > > future, as you previously suggested, we can include
> per-partition
> > > > > > > leaderEpoch in the FetchRequest/ProduceRequest such that broker
> > can
> > > > > > return
> > > > > > > error if the epoch is smaller than cached epoch in the broker.
> > > Given
> > > > > that
> > > > > > > this adds more complexity to Kafka, I think we can probably
> think
> > > > about
> > > > > > > that leader when we have a specific use-case or problem to
> solve
> > > with
> > > > > > > up-to-date metadata. Does this sound OK?
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Dong
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > On Fri, Dec 8, 2017 at 3:53 PM, Jun Rao <ju...@confluent.io>
> > wrote:
> > > > > > >
> > > > > > > > Hi, Dong,
> > > > > > > >
> > > > > > > > Thanks for the reply. A few more points below.
> > > > > > > >
> > > > > > > > For dealing with how to prevent a consumer switching from a
> new
> > > > > leader
> > > > > > to
> > > > > > > > an old leader, you suggestion that refreshes metadata on
> > consumer
> > > > > > restart
> > > > > > > > until it sees a metadata version >= the one associated with
> the
> > > > > offset
> > > > > > > > works too, as long as we guarantee that the cached metadata
> > > > versions
> > > > > on
> > > > > > > the
> > > > > > > > brokers only go up.
> > > > > > > >
> > > > > > > > The second discussion point is on whether the metadata
> > versioning
> > > > > > should
> > > > > > > be
> > > > > > > > per partition or global. For the partition level versioning,
> > you
> > > > were
> > > > > > > > concerned about the performance. Given that metadata updates
> > are
> > > > > rare,
> > > > > > I
> > > > > > > am
> > > > > > > > not sure if it's a big concern though. Doing a million if
> tests
> > > is
> > > > > > > probably
> > > > > > > > going to take less than 1ms. Another thing is that the
> metadata
> > > > > version
> > > > > > > > seems to need to survive controller failover. In your current
> > > > > > approach, a
> > > > > > > > consumer may not be able to wait on the right version of the
> > > > metadata
> > > > > > > after
> > > > > > > > the consumer restart since the metadata version may have been
> > > > > recycled
> > > > > > on
> > > > > > > > the server side due to a controller failover while the
> consumer
> > > is
> > > > > > down.
> > > > > > > > The partition level leaderEpoch survives controller failure
> and
> > > > won't
> > > > > > > have
> > > > > > > > this issue.
> > > > > > > >
> > > > > > > > Lastly, neither your proposal nor mine addresses the issue
> how
> > to
> > > > > > > guarantee
> > > > > > > > a consumer to detect that is metadata is outdated. Currently,
> > the
> > > > > > > consumer
> > > > > > > > is not guaranteed to fetch metadata from every broker within
> > some
> > > > > > bounded
> > > > > > > > period of time. Maybe this is out of the scope of your KIP.
> But
> > > one
> > > > > > idea
> > > > > > > is
> > > > > > > > force the consumer to refresh metadata from the controller
> > > > > > periodically.
> > > > > > > >
> > > > > > > > Jun
> > > > > > > >
> > > > > > > >
> > > > > > > > On Thu, Dec 7, 2017 at 11:25 AM, Dong Lin <
> lindong28@gmail.com
> > >
> > > > > wrote:
> > > > > > > >
> > > > > > > > > Hey Jun,
> > > > > > > > >
> > > > > > > > > Thanks much for the comments. Great point particularly
> > > regarding
> > > > > > (3). I
> > > > > > > > > haven't thought about this before.
> > > > > > > > >
> > > > > > > > > It seems that there are two possible ways where the version
> > > > number
> > > > > > can
> > > > > > > be
> > > > > > > > > used. One solution is for client to check the version
> number
> > at
> > > > the
> > > > > > > time
> > > > > > > > it
> > > > > > > > > receives MetadataResponse. And if the version number in the
> > > > > > > > > MetadataResponse is smaller than the version number in the
> > > > client's
> > > > > > > > cache,
> > > > > > > > > the client will be forced to fetch metadata again.  Another
> > > > > solution,
> > > > > > > as
> > > > > > > > > you have suggested, is for broker to check the version
> number
> > > at
> > > > > the
> > > > > > > time
> > > > > > > > > it receives a request from client. The broker will reject
> the
> > > > > request
> > > > > > > if
> > > > > > > > > the version is smaller than the version in broker's cache.
> > > > > > > > >
> > > > > > > > > I am not very sure that the second solution can address the
> > > > problem
> > > > > > > here.
> > > > > > > > > In the scenario described in the JIRA ticket, broker's
> cache
> > > may
> > > > be
> > > > > > > > > outdated because it has not processed the
> LeaderAndIsrRequest
> > > > from
> > > > > > the
> > > > > > > > > controller. Thus it may still process client's request even
> > if
> > > > the
> > > > > > > > version
> > > > > > > > > in client's request is actually outdated. Does this make
> > sense?
> > > > > > > > >
> > > > > > > > > IMO, it seems that we can address problem (3) by saving the
> > > > > metadata
> > > > > > > > > version together with the offset. After consumer starts, it
> > > will
> > > > > keep
> > > > > > > > > fetching metadata until the metadata version >= the version
> > > saved
> > > > > > with
> > > > > > > > the
> > > > > > > > > offset of this partition.
> > > > > > > > >
> > > > > > > > > Regarding problems (1) and (2): Currently we use the
> version
> > > > number
> > > > > > in
> > > > > > > > the
> > > > > > > > > MetadataResponse to ensure that the metadata does not go
> back
> > > in
> > > > > > time.
> > > > > > > > > There are two alternative solutions to address problems (1)
> > and
> > > > > (2).
> > > > > > > One
> > > > > > > > > solution is for client to enumerate all partitions in the
> > > > > > > > MetadataResponse,
> > > > > > > > > compare their epoch with those in the cached metadata, and
> > > > rejects
> > > > > > the
> > > > > > > > > MetadataResponse iff any leader epoch is smaller. The main
> > > > concern
> > > > > is
> > > > > > > > that
> > > > > > > > > MetadataResponse currently cached information of all
> > partitions
> > > > in
> > > > > > the
> > > > > > > > > entire cluster. It may slow down client's performance if we
> > > were
> > > > to
> > > > > > do
> > > > > > > > it.
> > > > > > > > > The other solution is for client to enumerate partitions
> for
> > > only
> > > > > > > topics
> > > > > > > > > registered in the org.apache.kafka.clients.Metadata, which
> > > will
> > > > be
> > > > > > an
> > > > > > > > > empty
> > > > > > > > > set for producer and the set of subscribed partitions for
> > > > consumer.
> > > > > > But
> > > > > > > > > this degrades to all topics if consumer subscribes to
> topics
> > in
> > > > the
> > > > > > > > cluster
> > > > > > > > > by pattern.
> > > > > > > > >
> > > > > > > > > Note that client will only be forced to update metadata if
> > the
> > > > > > version
> > > > > > > in
> > > > > > > > > the MetadataResponse is smaller than the version in the
> > cached
> > > > > > > metadata.
> > > > > > > > In
> > > > > > > > > general it should not be a problem. It can be a problem
> only
> > if
> > > > > some
> > > > > > > > broker
> > > > > > > > > is particularly slower than other brokers in processing
> > > > > > > > > UpdateMetadataRequest. When this is the case, it means that
> > the
> > > > > > broker
> > > > > > > is
> > > > > > > > > also particularly slower in processing LeaderAndIsrRequest,
> > > which
> > > > > can
> > > > > > > > cause
> > > > > > > > > problem anyway because some partition will probably have no
> > > > leader
> > > > > > > during
> > > > > > > > > this period. I am not sure problems (1) and (2) cause more
> > > > problem
> > > > > > than
> > > > > > > > > what we already have.
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > > Dong
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Wed, Dec 6, 2017 at 6:42 PM, Jun Rao <ju...@confluent.io>
> > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hi, Dong,
> > > > > > > > > >
> > > > > > > > > > Great finding on the issue. It's a real problem. A few
> > > comments
> > > > > > about
> > > > > > > > the
> > > > > > > > > > KIP. (1) I am not sure about updating
> > > controller_metadata_epoch
> > > > > on
> > > > > > > > every
> > > > > > > > > > UpdateMetadataRequest. Currently, the controller can send
> > > > > > > > > > UpdateMetadataRequest when there is no actual metadata
> > > change.
> > > > > > Doing
> > > > > > > > this
> > > > > > > > > > may require unnecessary metadata refresh on the client.
> (2)
> > > > > > > > > > controller_metadata_epoch is global across all topics.
> This
> > > > means
> > > > > > > that
> > > > > > > > a
> > > > > > > > > > client may be forced to update its metadata even when the
> > > > > metadata
> > > > > > > for
> > > > > > > > > the
> > > > > > > > > > topics that it cares haven't changed. (3) It doesn't seem
> > > that
> > > > > the
> > > > > > > KIP
> > > > > > > > > > handles the corner case when a consumer is restarted.
> Say a
> > > > > > consumer
> > > > > > > > > reads
> > > > > > > > > > from the new leader, commits the offset and then is
> > > restarted.
> > > > On
> > > > > > > > > restart,
> > > > > > > > > > the consumer gets an outdated metadata and fetches from
> the
> > > old
> > > > > > > leader.
> > > > > > > > > > Then, the consumer will get into the offset out of range
> > > issue.
> > > > > > > > > >
> > > > > > > > > > Given the above, I am thinking of the following approach.
> > We
> > > > > > actually
> > > > > > > > > > already have metadata versioning at the partition level.
> > Each
> > > > > > leader
> > > > > > > > has
> > > > > > > > > a
> > > > > > > > > > leader epoch which is monotonically increasing. We can
> > > > > potentially
> > > > > > > > > > propagate leader epoch back in the metadata response and
> > the
> > > > > > clients
> > > > > > > > can
> > > > > > > > > > cache that. This solves the issue of (1) and (2). To
> solve
> > > (3),
> > > > > > when
> > > > > > > > > saving
> > > > > > > > > > an offset, we could save both an offset and the
> > corresponding
> > > > > > leader
> > > > > > > > > epoch.
> > > > > > > > > > When fetching the data, the consumer provides both the
> > offset
> > > > and
> > > > > > the
> > > > > > > > > > leader epoch. A leader will only serve the request if its
> > > > leader
> > > > > > > epoch
> > > > > > > > is
> > > > > > > > > > equal to or greater than the leader epoch from the
> > consumer.
> > > To
> > > > > > > achieve
> > > > > > > > > > this, we need to change the fetch request protocol and
> the
> > > > offset
> > > > > > > > commit
> > > > > > > > > > api, which requires some more thoughts.
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > >
> > > > > > > > > > Jun
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Wed, Dec 6, 2017 at 10:57 AM, Dong Lin <
> > > lindong28@gmail.com
> > > > >
> > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Bump up the thread.
> > > > > > > > > > >
> > > > > > > > > > > It will be great to have more comments on whether we
> > should
> > > > do
> > > > > it
> > > > > > > or
> > > > > > > > > > > whether there is better way to address the motivation
> of
> > > this
> > > > > > KIP.
> > > > > > > > > > >
> > > > > > > > > > > On Mon, Dec 4, 2017 at 3:09 PM, Dong Lin <
> > > > lindong28@gmail.com>
> > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > I don't have an interesting rejected alternative
> > solution
> > > > to
> > > > > > put
> > > > > > > in
> > > > > > > > > the
> > > > > > > > > > > > KIP. If there is good alternative solution from
> anyone
> > in
> > > > > this
> > > > > > > > > thread,
> > > > > > > > > > I
> > > > > > > > > > > am
> > > > > > > > > > > > happy to discuss this and update the KIP accordingly.
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks,
> > > > > > > > > > > > Dong
> > > > > > > > > > > >
> > > > > > > > > > > > On Mon, Dec 4, 2017 at 1:12 PM, Ted Yu <
> > > > yuzhihong@gmail.com>
> > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > >> It is clearer now.
> > > > > > > > > > > >>
> > > > > > > > > > > >> I noticed that Rejected Alternatives section is
> empty.
> > > > > > > > > > > >> Have you considered any alternative ?
> > > > > > > > > > > >>
> > > > > > > > > > > >> Cheers
> > > > > > > > > > > >>
> > > > > > > > > > > >> On Mon, Dec 4, 2017 at 1:07 PM, Dong Lin <
> > > > > lindong28@gmail.com
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > > > >>
> > > > > > > > > > > >> > Ted, thanks for catching this. I have updated the
> > > > sentence
> > > > > > to
> > > > > > > > make
> > > > > > > > > > it
> > > > > > > > > > > >> > readable.
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > Thanks,
> > > > > > > > > > > >> > Dong
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > On Sat, Dec 2, 2017 at 3:05 PM, Ted Yu <
> > > > > yuzhihong@gmail.com
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > > bq. It the controller_epoch of the incoming
> > > > > > > MetadataResponse,
> > > > > > > > or
> > > > > > > > > > if
> > > > > > > > > > > >> the
> > > > > > > > > > > >> > > controller_epoch is the same but the
> > > > > > > controller_metadata_epoch
> > > > > > > > > > > >> > >
> > > > > > > > > > > >> > > Can you update the above sentence so that the
> > > > intention
> > > > > is
> > > > > > > > > > clearer ?
> > > > > > > > > > > >> > >
> > > > > > > > > > > >> > > Thanks
> > > > > > > > > > > >> > >
> > > > > > > > > > > >> > > On Fri, Dec 1, 2017 at 6:33 PM, Dong Lin <
> > > > > > > lindong28@gmail.com
> > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > > > >> > >
> > > > > > > > > > > >> > > > Hi all,
> > > > > > > > > > > >> > > >
> > > > > > > > > > > >> > > > I have created KIP-232: Detect outdated
> metadata
> > > by
> > > > > > adding
> > > > > > > > > > > >> > > > ControllerMetadataEpoch field:
> > > > > > > > > > > >> > > > https://cwiki.apache.org/
> > > > > confluence/display/KAFKA/KIP-
> > > > > > > > > > > >> > > > 232%3A+Detect+outdated+metadata+by+adding+
> > > > > > > > > > > >> > ControllerMetadataEpoch+field
> > > > > > > > > > > >> > > > .
> > > > > > > > > > > >> > > >
> > > > > > > > > > > >> > > > The KIP proposes to add fields in
> > MetadataResponse
> > > > and
> > > > > > > > > > > >> > > > UpdateMetadataRequest so that client can
> reject
> > > > > outdated
> > > > > > > > > > metadata
> > > > > > > > > > > >> and
> > > > > > > > > > > >> > > avoid
> > > > > > > > > > > >> > > > unnecessary OffsetOutOfRangeException.
> Otherwise
> > > > there
> > > > > > is
> > > > > > > > > > > currently
> > > > > > > > > > > >> > race
> > > > > > > > > > > >> > > > condition that can cause consumer to reset
> > offset
> > > > > which
> > > > > > > > > > negatively
> > > > > > > > > > > >> > affect
> > > > > > > > > > > >> > > > the consumer's availability.
> > > > > > > > > > > >> > > >
> > > > > > > > > > > >> > > > Feedback and suggestions are welcome!
> > > > > > > > > > > >> > > >
> > > > > > > > > > > >> > > > Regards,
> > > > > > > > > > > >> > > > Dong
> > > > > > > > > > > >> > > >
> > > > > > > > > > > >> > >
> > > > > > > > > > > >> >
> > > > > > > > > > > >>
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Dong Lin <li...@gmail.com>.
Hey Jun,

Thanks much for your comments. These are very thoughtful ideas. Please see
my comments below.

On Thu, Dec 14, 2017 at 6:38 PM, Jun Rao <ju...@confluent.io> wrote:

> Hi, Dong,
>
> Thanks for the update. A few more comments below.
>
> 10. It seems that we need to return the leader epoch in the fetch response
> as well When fetching data, we could be fetching data from a leader epoch
> older than what's returned in the metadata response. So, we want to use the
> leader epoch associated with the offset being fetched for committing
> offsets.
>

It seems that we may have two separate issues here. The first issue is that
consumer uses metadata that is older than the one it uses before. The
second issue is that consumer uses metadata which is newer than the
corresponding leader epoch in the leader broker. We know that the
OffsetOutOfRangeException described in this KIP can be prevented by
avoiding the first issue. On the other hand, it seems that the
OffsetOffsetOutOfRangeException can still happen even if we avoid the
second issue -- if consumer uses an older version of metadata, the leader
epoch in its metadata may equal the leader epoch in the broker even if the
leader epoch in the broker is oudated.

Given this understanding, I am not sure why we need to return the leader
epoch in the fetch response. As long as consumer's metadata is not going
back in version, I think we are good. Did I miss something here?


>
> 11. Should we now extend OffsetAndMetadata used in the offset commit api in
> KafkaConsumer to include leader epoch? Similarly, should we return leader
> epoch in endOffsets(), beginningOffsets() and position()? We probably need
> to think about how to make the api backward compatible.
>

After thinking through this carefully, I think we probably don't want to
extend OffsetAndMetadata to include leader epoch because leader epoch is
kind of implementation detail which ideally should be hidden from user. The
consumer can include leader epoch in the OffsetCommitRequest after taking
offset from commitSync(final Map<TopicPartition, OffsetAndMetadata>
offsets). Similarly consumer can store leader epoch from
OffsetFetchResponse and only provide offset to user via
consumer.committed(topicPartition). This solution seems to work well and we
don't have to make changes to consumer's public API. Does this sound OK?


>
> 12. It seems that we now need to store leader epoch in the offset topic.
> Could you include the new schema for the value of the offset topic and add
> upgrade notes?


You are right. I have updated the KIP to specify the new schema for the
value of the offset topic. Can you take another look?

For existing messages in the offset topic, leader_epoch will be missing. We
will use leader_epoch = -1 to indicate the missing leader_epoch. Then the
consumer behavior will be the same as it is now because any leader_epoch in
the MetadataResponse will be larger than the leader_epoch = -1 in the
OffetFetchResponse. Thus we don't need specific procedure for upgrades due
to this change in the offset topic schema. By "upgrade nodes", do you mean
the sentences we need to include in the upgrade.html in the PR later?


>
> Jun
>
>
> On Tue, Dec 12, 2017 at 5:19 PM, Dong Lin <li...@gmail.com> wrote:
>
> > Hey Jun,
> >
> > I see. Sounds good. Yeah it is probably simpler to leave this to another
> > KIP in the future.
> >
> > Thanks for all the comments. Since there is no further comment in the
> > community, I will open the voting thread.
> >
> > Thanks,
> > Dong
> >
> > On Mon, Dec 11, 2017 at 5:37 PM, Jun Rao <ju...@confluent.io> wrote:
> >
> > > Hi, Dong,
> > >
> > > The case that I am thinking is network partitioning. Suppose one
> deploys
> > a
> > > stretched cluster across multiple AZs in the same region. If the
> machines
> > > in one AZ can't communicate to brokers in other AZs due to a network
> > issue,
> > > the brokers in that AZ won't get any new metadata.
> > >
> > > We can potentially solve this problem by requiring some kind of regular
> > > heartbeats between the controller and the broker. This may need some
> more
> > > thoughts. So, it's probably fine to leave this to another KIP in the
> > > future.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > > On Mon, Dec 11, 2017 at 2:55 PM, Dong Lin <li...@gmail.com> wrote:
> > >
> > > > Hey Jun,
> > > >
> > > > Thanks for the comment. I am open to improve this KIP to address more
> > > > problems. I probably need more help in understanding what is the
> > current
> > > > problem with consumer using outdated metadata and whether it is
> easier
> > to
> > > > address it together with this KIP.
> > > >
> > > > I agree that a consumer can potentially talk to old leader for a long
> > > time
> > > > even after this KIP. But after this KIP, the consumer probably should
> > not
> > > > get OffetOutofRangeException and therefore will not cause offset
> rewind
> > > > issue. So the only problem is that consumer will not be able to fetch
> > > data
> > > > until it has updated metadata. It seems that this situation can only
> > > happen
> > > > if the broker is too slow in processing LeaderAndIsrRequest since
> > > otherwise
> > > > the consumer will be forced to update metadata due to
> > > > NotLeaderForPartitionException. So the problem we are having here is
> > > that
> > > > consumer will not be able to fetch data if some broker is too slow in
> > > > processing LeaderAndIsrRequest.
> > > >
> > > > Because Kafka propagates LeaderAndIsrRequest asynchronously to all
> > > brokers
> > > > in the cluster, there will always be a period of time when consumer
> can
> > > not
> > > > fetch data for the partition during the leadership change. Thus it
> > seems
> > > > more like a broker-side performance issue instead of client-side
> > > > correctness issue. My gut feel is that it is not causing a much a
> > problem
> > > > as the problem to be fixed in this KIP. And if we were to address it,
> > we
> > > > probably need to make change in the broker side, e.g. with
> prioritized
> > > > queue for controller-related requests, which may be kind of
> orthogonal
> > to
> > > > this KIP. I am not very sure it will be easier to address it with the
> > > > change in this KIP. Do you have any recommendation?
> > > >
> > > > Thanks,
> > > > Dong
> > > >
> > > >
> > > > On Mon, Dec 11, 2017 at 1:51 PM, Jun Rao <ju...@confluent.io> wrote:
> > > >
> > > > > Hi, Dong,
> > > > >
> > > > > Thanks for the reply.
> > > > >
> > > > > My suggestion of forcing the metadata refresh from the controller
> may
> > > not
> > > > > work in general since the cached controller could be outdated too.
> > The
> > > > > general problem is that if a consumer's metadata is outdated, it
> may
> > > get
> > > > > stuck with the old leader for a long time. We can address the issue
> > of
> > > > > detecting outdated metadata in a separate KIP in the future if you
> > > didn't
> > > > > intend to address it in this KIP.
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Jun
> > > > >
> > > > >
> > > > > On Sat, Dec 9, 2017 at 10:12 PM, Dong Lin <li...@gmail.com>
> > wrote:
> > > > >
> > > > > > Hey Jun,
> > > > > >
> > > > > > Thanks much for your comments. Given that client needs to
> > > de-serialize
> > > > > the
> > > > > > metadata anyway, the extra overhead of checking the per-partition
> > > > version
> > > > > > for every partition should not be a big concern. Thus it makes
> > sense
> > > to
> > > > > use
> > > > > > leader epoch as the per-partition version instead of creating a
> > > global
> > > > > > metadata version. I will update the KIP to do that.
> > > > > >
> > > > > > Regarding the detection of outdated metadata, I think it is
> > possible
> > > to
> > > > > > ensure that client gets latest metadata by fetching from
> > controller.
> > > > Note
> > > > > > that this requires extra logic in the controller such that
> > controller
> > > > > > updates metadata directly in memory without requiring
> > > > > > UpdateMetadataRequest. But I am not sure the main motivation of
> > this
> > > at
> > > > > > this moment. But this makes controller more like a bottleneck in
> > the
> > > > > > cluster which we probably want to avoid.
> > > > > >
> > > > > > I think we can probably keep the current way of ensuring metadata
> > > > > > freshness. Currently client will be forced to refresh metadata if
> > > > broker
> > > > > > returns error (e.g. NotLeaderForPartition) due to outdated
> metadata
> > > or
> > > > if
> > > > > > the metadata does not contain the partition that the client
> needs.
> > In
> > > > the
> > > > > > future, as you previously suggested, we can include per-partition
> > > > > > leaderEpoch in the FetchRequest/ProduceRequest such that broker
> can
> > > > > return
> > > > > > error if the epoch is smaller than cached epoch in the broker.
> > Given
> > > > that
> > > > > > this adds more complexity to Kafka, I think we can probably think
> > > about
> > > > > > that leader when we have a specific use-case or problem to solve
> > with
> > > > > > up-to-date metadata. Does this sound OK?
> > > > > >
> > > > > > Thanks,
> > > > > > Dong
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Fri, Dec 8, 2017 at 3:53 PM, Jun Rao <ju...@confluent.io>
> wrote:
> > > > > >
> > > > > > > Hi, Dong,
> > > > > > >
> > > > > > > Thanks for the reply. A few more points below.
> > > > > > >
> > > > > > > For dealing with how to prevent a consumer switching from a new
> > > > leader
> > > > > to
> > > > > > > an old leader, you suggestion that refreshes metadata on
> consumer
> > > > > restart
> > > > > > > until it sees a metadata version >= the one associated with the
> > > > offset
> > > > > > > works too, as long as we guarantee that the cached metadata
> > > versions
> > > > on
> > > > > > the
> > > > > > > brokers only go up.
> > > > > > >
> > > > > > > The second discussion point is on whether the metadata
> versioning
> > > > > should
> > > > > > be
> > > > > > > per partition or global. For the partition level versioning,
> you
> > > were
> > > > > > > concerned about the performance. Given that metadata updates
> are
> > > > rare,
> > > > > I
> > > > > > am
> > > > > > > not sure if it's a big concern though. Doing a million if tests
> > is
> > > > > > probably
> > > > > > > going to take less than 1ms. Another thing is that the metadata
> > > > version
> > > > > > > seems to need to survive controller failover. In your current
> > > > > approach, a
> > > > > > > consumer may not be able to wait on the right version of the
> > > metadata
> > > > > > after
> > > > > > > the consumer restart since the metadata version may have been
> > > > recycled
> > > > > on
> > > > > > > the server side due to a controller failover while the consumer
> > is
> > > > > down.
> > > > > > > The partition level leaderEpoch survives controller failure and
> > > won't
> > > > > > have
> > > > > > > this issue.
> > > > > > >
> > > > > > > Lastly, neither your proposal nor mine addresses the issue how
> to
> > > > > > guarantee
> > > > > > > a consumer to detect that is metadata is outdated. Currently,
> the
> > > > > > consumer
> > > > > > > is not guaranteed to fetch metadata from every broker within
> some
> > > > > bounded
> > > > > > > period of time. Maybe this is out of the scope of your KIP. But
> > one
> > > > > idea
> > > > > > is
> > > > > > > force the consumer to refresh metadata from the controller
> > > > > periodically.
> > > > > > >
> > > > > > > Jun
> > > > > > >
> > > > > > >
> > > > > > > On Thu, Dec 7, 2017 at 11:25 AM, Dong Lin <lindong28@gmail.com
> >
> > > > wrote:
> > > > > > >
> > > > > > > > Hey Jun,
> > > > > > > >
> > > > > > > > Thanks much for the comments. Great point particularly
> > regarding
> > > > > (3). I
> > > > > > > > haven't thought about this before.
> > > > > > > >
> > > > > > > > It seems that there are two possible ways where the version
> > > number
> > > > > can
> > > > > > be
> > > > > > > > used. One solution is for client to check the version number
> at
> > > the
> > > > > > time
> > > > > > > it
> > > > > > > > receives MetadataResponse. And if the version number in the
> > > > > > > > MetadataResponse is smaller than the version number in the
> > > client's
> > > > > > > cache,
> > > > > > > > the client will be forced to fetch metadata again.  Another
> > > > solution,
> > > > > > as
> > > > > > > > you have suggested, is for broker to check the version number
> > at
> > > > the
> > > > > > time
> > > > > > > > it receives a request from client. The broker will reject the
> > > > request
> > > > > > if
> > > > > > > > the version is smaller than the version in broker's cache.
> > > > > > > >
> > > > > > > > I am not very sure that the second solution can address the
> > > problem
> > > > > > here.
> > > > > > > > In the scenario described in the JIRA ticket, broker's cache
> > may
> > > be
> > > > > > > > outdated because it has not processed the LeaderAndIsrRequest
> > > from
> > > > > the
> > > > > > > > controller. Thus it may still process client's request even
> if
> > > the
> > > > > > > version
> > > > > > > > in client's request is actually outdated. Does this make
> sense?
> > > > > > > >
> > > > > > > > IMO, it seems that we can address problem (3) by saving the
> > > > metadata
> > > > > > > > version together with the offset. After consumer starts, it
> > will
> > > > keep
> > > > > > > > fetching metadata until the metadata version >= the version
> > saved
> > > > > with
> > > > > > > the
> > > > > > > > offset of this partition.
> > > > > > > >
> > > > > > > > Regarding problems (1) and (2): Currently we use the version
> > > number
> > > > > in
> > > > > > > the
> > > > > > > > MetadataResponse to ensure that the metadata does not go back
> > in
> > > > > time.
> > > > > > > > There are two alternative solutions to address problems (1)
> and
> > > > (2).
> > > > > > One
> > > > > > > > solution is for client to enumerate all partitions in the
> > > > > > > MetadataResponse,
> > > > > > > > compare their epoch with those in the cached metadata, and
> > > rejects
> > > > > the
> > > > > > > > MetadataResponse iff any leader epoch is smaller. The main
> > > concern
> > > > is
> > > > > > > that
> > > > > > > > MetadataResponse currently cached information of all
> partitions
> > > in
> > > > > the
> > > > > > > > entire cluster. It may slow down client's performance if we
> > were
> > > to
> > > > > do
> > > > > > > it.
> > > > > > > > The other solution is for client to enumerate partitions for
> > only
> > > > > > topics
> > > > > > > > registered in the org.apache.kafka.clients.Metadata, which
> > will
> > > be
> > > > > an
> > > > > > > > empty
> > > > > > > > set for producer and the set of subscribed partitions for
> > > consumer.
> > > > > But
> > > > > > > > this degrades to all topics if consumer subscribes to topics
> in
> > > the
> > > > > > > cluster
> > > > > > > > by pattern.
> > > > > > > >
> > > > > > > > Note that client will only be forced to update metadata if
> the
> > > > > version
> > > > > > in
> > > > > > > > the MetadataResponse is smaller than the version in the
> cached
> > > > > > metadata.
> > > > > > > In
> > > > > > > > general it should not be a problem. It can be a problem only
> if
> > > > some
> > > > > > > broker
> > > > > > > > is particularly slower than other brokers in processing
> > > > > > > > UpdateMetadataRequest. When this is the case, it means that
> the
> > > > > broker
> > > > > > is
> > > > > > > > also particularly slower in processing LeaderAndIsrRequest,
> > which
> > > > can
> > > > > > > cause
> > > > > > > > problem anyway because some partition will probably have no
> > > leader
> > > > > > during
> > > > > > > > this period. I am not sure problems (1) and (2) cause more
> > > problem
> > > > > than
> > > > > > > > what we already have.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Dong
> > > > > > > >
> > > > > > > >
> > > > > > > > On Wed, Dec 6, 2017 at 6:42 PM, Jun Rao <ju...@confluent.io>
> > > wrote:
> > > > > > > >
> > > > > > > > > Hi, Dong,
> > > > > > > > >
> > > > > > > > > Great finding on the issue. It's a real problem. A few
> > comments
> > > > > about
> > > > > > > the
> > > > > > > > > KIP. (1) I am not sure about updating
> > controller_metadata_epoch
> > > > on
> > > > > > > every
> > > > > > > > > UpdateMetadataRequest. Currently, the controller can send
> > > > > > > > > UpdateMetadataRequest when there is no actual metadata
> > change.
> > > > > Doing
> > > > > > > this
> > > > > > > > > may require unnecessary metadata refresh on the client. (2)
> > > > > > > > > controller_metadata_epoch is global across all topics. This
> > > means
> > > > > > that
> > > > > > > a
> > > > > > > > > client may be forced to update its metadata even when the
> > > > metadata
> > > > > > for
> > > > > > > > the
> > > > > > > > > topics that it cares haven't changed. (3) It doesn't seem
> > that
> > > > the
> > > > > > KIP
> > > > > > > > > handles the corner case when a consumer is restarted. Say a
> > > > > consumer
> > > > > > > > reads
> > > > > > > > > from the new leader, commits the offset and then is
> > restarted.
> > > On
> > > > > > > > restart,
> > > > > > > > > the consumer gets an outdated metadata and fetches from the
> > old
> > > > > > leader.
> > > > > > > > > Then, the consumer will get into the offset out of range
> > issue.
> > > > > > > > >
> > > > > > > > > Given the above, I am thinking of the following approach.
> We
> > > > > actually
> > > > > > > > > already have metadata versioning at the partition level.
> Each
> > > > > leader
> > > > > > > has
> > > > > > > > a
> > > > > > > > > leader epoch which is monotonically increasing. We can
> > > > potentially
> > > > > > > > > propagate leader epoch back in the metadata response and
> the
> > > > > clients
> > > > > > > can
> > > > > > > > > cache that. This solves the issue of (1) and (2). To solve
> > (3),
> > > > > when
> > > > > > > > saving
> > > > > > > > > an offset, we could save both an offset and the
> corresponding
> > > > > leader
> > > > > > > > epoch.
> > > > > > > > > When fetching the data, the consumer provides both the
> offset
> > > and
> > > > > the
> > > > > > > > > leader epoch. A leader will only serve the request if its
> > > leader
> > > > > > epoch
> > > > > > > is
> > > > > > > > > equal to or greater than the leader epoch from the
> consumer.
> > To
> > > > > > achieve
> > > > > > > > > this, we need to change the fetch request protocol and the
> > > offset
> > > > > > > commit
> > > > > > > > > api, which requires some more thoughts.
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > >
> > > > > > > > > Jun
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Wed, Dec 6, 2017 at 10:57 AM, Dong Lin <
> > lindong28@gmail.com
> > > >
> > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Bump up the thread.
> > > > > > > > > >
> > > > > > > > > > It will be great to have more comments on whether we
> should
> > > do
> > > > it
> > > > > > or
> > > > > > > > > > whether there is better way to address the motivation of
> > this
> > > > > KIP.
> > > > > > > > > >
> > > > > > > > > > On Mon, Dec 4, 2017 at 3:09 PM, Dong Lin <
> > > lindong28@gmail.com>
> > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > I don't have an interesting rejected alternative
> solution
> > > to
> > > > > put
> > > > > > in
> > > > > > > > the
> > > > > > > > > > > KIP. If there is good alternative solution from anyone
> in
> > > > this
> > > > > > > > thread,
> > > > > > > > > I
> > > > > > > > > > am
> > > > > > > > > > > happy to discuss this and update the KIP accordingly.
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > > Dong
> > > > > > > > > > >
> > > > > > > > > > > On Mon, Dec 4, 2017 at 1:12 PM, Ted Yu <
> > > yuzhihong@gmail.com>
> > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > >> It is clearer now.
> > > > > > > > > > >>
> > > > > > > > > > >> I noticed that Rejected Alternatives section is empty.
> > > > > > > > > > >> Have you considered any alternative ?
> > > > > > > > > > >>
> > > > > > > > > > >> Cheers
> > > > > > > > > > >>
> > > > > > > > > > >> On Mon, Dec 4, 2017 at 1:07 PM, Dong Lin <
> > > > lindong28@gmail.com
> > > > > >
> > > > > > > > wrote:
> > > > > > > > > > >>
> > > > > > > > > > >> > Ted, thanks for catching this. I have updated the
> > > sentence
> > > > > to
> > > > > > > make
> > > > > > > > > it
> > > > > > > > > > >> > readable.
> > > > > > > > > > >> >
> > > > > > > > > > >> > Thanks,
> > > > > > > > > > >> > Dong
> > > > > > > > > > >> >
> > > > > > > > > > >> > On Sat, Dec 2, 2017 at 3:05 PM, Ted Yu <
> > > > yuzhihong@gmail.com
> > > > > >
> > > > > > > > wrote:
> > > > > > > > > > >> >
> > > > > > > > > > >> > > bq. It the controller_epoch of the incoming
> > > > > > MetadataResponse,
> > > > > > > or
> > > > > > > > > if
> > > > > > > > > > >> the
> > > > > > > > > > >> > > controller_epoch is the same but the
> > > > > > controller_metadata_epoch
> > > > > > > > > > >> > >
> > > > > > > > > > >> > > Can you update the above sentence so that the
> > > intention
> > > > is
> > > > > > > > > clearer ?
> > > > > > > > > > >> > >
> > > > > > > > > > >> > > Thanks
> > > > > > > > > > >> > >
> > > > > > > > > > >> > > On Fri, Dec 1, 2017 at 6:33 PM, Dong Lin <
> > > > > > lindong28@gmail.com
> > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > > >> > >
> > > > > > > > > > >> > > > Hi all,
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > I have created KIP-232: Detect outdated metadata
> > by
> > > > > adding
> > > > > > > > > > >> > > > ControllerMetadataEpoch field:
> > > > > > > > > > >> > > > https://cwiki.apache.org/
> > > > confluence/display/KAFKA/KIP-
> > > > > > > > > > >> > > > 232%3A+Detect+outdated+metadata+by+adding+
> > > > > > > > > > >> > ControllerMetadataEpoch+field
> > > > > > > > > > >> > > > .
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > The KIP proposes to add fields in
> MetadataResponse
> > > and
> > > > > > > > > > >> > > > UpdateMetadataRequest so that client can reject
> > > > outdated
> > > > > > > > > metadata
> > > > > > > > > > >> and
> > > > > > > > > > >> > > avoid
> > > > > > > > > > >> > > > unnecessary OffsetOutOfRangeException. Otherwise
> > > there
> > > > > is
> > > > > > > > > > currently
> > > > > > > > > > >> > race
> > > > > > > > > > >> > > > condition that can cause consumer to reset
> offset
> > > > which
> > > > > > > > > negatively
> > > > > > > > > > >> > affect
> > > > > > > > > > >> > > > the consumer's availability.
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > Feedback and suggestions are welcome!
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > Regards,
> > > > > > > > > > >> > > > Dong
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > >
> > > > > > > > > > >> >
> > > > > > > > > > >>
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Jun Rao <ju...@confluent.io>.
Hi, Dong,

Thanks for the update. A few more comments below.

10. It seems that we need to return the leader epoch in the fetch response
as well When fetching data, we could be fetching data from a leader epoch
older than what's returned in the metadata response. So, we want to use the
leader epoch associated with the offset being fetched for committing
offsets.

11. Should we now extend OffsetAndMetadata used in the offset commit api in
KafkaConsumer to include leader epoch? Similarly, should we return leader
epoch in endOffsets(), beginningOffsets() and position()? We probably need
to think about how to make the api backward compatible.

12. It seems that we now need to store leader epoch in the offset topic.
Could you include the new schema for the value of the offset topic and add
upgrade notes?

Jun


On Tue, Dec 12, 2017 at 5:19 PM, Dong Lin <li...@gmail.com> wrote:

> Hey Jun,
>
> I see. Sounds good. Yeah it is probably simpler to leave this to another
> KIP in the future.
>
> Thanks for all the comments. Since there is no further comment in the
> community, I will open the voting thread.
>
> Thanks,
> Dong
>
> On Mon, Dec 11, 2017 at 5:37 PM, Jun Rao <ju...@confluent.io> wrote:
>
> > Hi, Dong,
> >
> > The case that I am thinking is network partitioning. Suppose one deploys
> a
> > stretched cluster across multiple AZs in the same region. If the machines
> > in one AZ can't communicate to brokers in other AZs due to a network
> issue,
> > the brokers in that AZ won't get any new metadata.
> >
> > We can potentially solve this problem by requiring some kind of regular
> > heartbeats between the controller and the broker. This may need some more
> > thoughts. So, it's probably fine to leave this to another KIP in the
> > future.
> >
> > Thanks,
> >
> > Jun
> >
> > On Mon, Dec 11, 2017 at 2:55 PM, Dong Lin <li...@gmail.com> wrote:
> >
> > > Hey Jun,
> > >
> > > Thanks for the comment. I am open to improve this KIP to address more
> > > problems. I probably need more help in understanding what is the
> current
> > > problem with consumer using outdated metadata and whether it is easier
> to
> > > address it together with this KIP.
> > >
> > > I agree that a consumer can potentially talk to old leader for a long
> > time
> > > even after this KIP. But after this KIP, the consumer probably should
> not
> > > get OffetOutofRangeException and therefore will not cause offset rewind
> > > issue. So the only problem is that consumer will not be able to fetch
> > data
> > > until it has updated metadata. It seems that this situation can only
> > happen
> > > if the broker is too slow in processing LeaderAndIsrRequest since
> > otherwise
> > > the consumer will be forced to update metadata due to
> > > NotLeaderForPartitionException. So the problem we are having here is
> > that
> > > consumer will not be able to fetch data if some broker is too slow in
> > > processing LeaderAndIsrRequest.
> > >
> > > Because Kafka propagates LeaderAndIsrRequest asynchronously to all
> > brokers
> > > in the cluster, there will always be a period of time when consumer can
> > not
> > > fetch data for the partition during the leadership change. Thus it
> seems
> > > more like a broker-side performance issue instead of client-side
> > > correctness issue. My gut feel is that it is not causing a much a
> problem
> > > as the problem to be fixed in this KIP. And if we were to address it,
> we
> > > probably need to make change in the broker side, e.g. with prioritized
> > > queue for controller-related requests, which may be kind of orthogonal
> to
> > > this KIP. I am not very sure it will be easier to address it with the
> > > change in this KIP. Do you have any recommendation?
> > >
> > > Thanks,
> > > Dong
> > >
> > >
> > > On Mon, Dec 11, 2017 at 1:51 PM, Jun Rao <ju...@confluent.io> wrote:
> > >
> > > > Hi, Dong,
> > > >
> > > > Thanks for the reply.
> > > >
> > > > My suggestion of forcing the metadata refresh from the controller may
> > not
> > > > work in general since the cached controller could be outdated too.
> The
> > > > general problem is that if a consumer's metadata is outdated, it may
> > get
> > > > stuck with the old leader for a long time. We can address the issue
> of
> > > > detecting outdated metadata in a separate KIP in the future if you
> > didn't
> > > > intend to address it in this KIP.
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > >
> > > > On Sat, Dec 9, 2017 at 10:12 PM, Dong Lin <li...@gmail.com>
> wrote:
> > > >
> > > > > Hey Jun,
> > > > >
> > > > > Thanks much for your comments. Given that client needs to
> > de-serialize
> > > > the
> > > > > metadata anyway, the extra overhead of checking the per-partition
> > > version
> > > > > for every partition should not be a big concern. Thus it makes
> sense
> > to
> > > > use
> > > > > leader epoch as the per-partition version instead of creating a
> > global
> > > > > metadata version. I will update the KIP to do that.
> > > > >
> > > > > Regarding the detection of outdated metadata, I think it is
> possible
> > to
> > > > > ensure that client gets latest metadata by fetching from
> controller.
> > > Note
> > > > > that this requires extra logic in the controller such that
> controller
> > > > > updates metadata directly in memory without requiring
> > > > > UpdateMetadataRequest. But I am not sure the main motivation of
> this
> > at
> > > > > this moment. But this makes controller more like a bottleneck in
> the
> > > > > cluster which we probably want to avoid.
> > > > >
> > > > > I think we can probably keep the current way of ensuring metadata
> > > > > freshness. Currently client will be forced to refresh metadata if
> > > broker
> > > > > returns error (e.g. NotLeaderForPartition) due to outdated metadata
> > or
> > > if
> > > > > the metadata does not contain the partition that the client needs.
> In
> > > the
> > > > > future, as you previously suggested, we can include per-partition
> > > > > leaderEpoch in the FetchRequest/ProduceRequest such that broker can
> > > > return
> > > > > error if the epoch is smaller than cached epoch in the broker.
> Given
> > > that
> > > > > this adds more complexity to Kafka, I think we can probably think
> > about
> > > > > that leader when we have a specific use-case or problem to solve
> with
> > > > > up-to-date metadata. Does this sound OK?
> > > > >
> > > > > Thanks,
> > > > > Dong
> > > > >
> > > > >
> > > > >
> > > > > On Fri, Dec 8, 2017 at 3:53 PM, Jun Rao <ju...@confluent.io> wrote:
> > > > >
> > > > > > Hi, Dong,
> > > > > >
> > > > > > Thanks for the reply. A few more points below.
> > > > > >
> > > > > > For dealing with how to prevent a consumer switching from a new
> > > leader
> > > > to
> > > > > > an old leader, you suggestion that refreshes metadata on consumer
> > > > restart
> > > > > > until it sees a metadata version >= the one associated with the
> > > offset
> > > > > > works too, as long as we guarantee that the cached metadata
> > versions
> > > on
> > > > > the
> > > > > > brokers only go up.
> > > > > >
> > > > > > The second discussion point is on whether the metadata versioning
> > > > should
> > > > > be
> > > > > > per partition or global. For the partition level versioning, you
> > were
> > > > > > concerned about the performance. Given that metadata updates are
> > > rare,
> > > > I
> > > > > am
> > > > > > not sure if it's a big concern though. Doing a million if tests
> is
> > > > > probably
> > > > > > going to take less than 1ms. Another thing is that the metadata
> > > version
> > > > > > seems to need to survive controller failover. In your current
> > > > approach, a
> > > > > > consumer may not be able to wait on the right version of the
> > metadata
> > > > > after
> > > > > > the consumer restart since the metadata version may have been
> > > recycled
> > > > on
> > > > > > the server side due to a controller failover while the consumer
> is
> > > > down.
> > > > > > The partition level leaderEpoch survives controller failure and
> > won't
> > > > > have
> > > > > > this issue.
> > > > > >
> > > > > > Lastly, neither your proposal nor mine addresses the issue how to
> > > > > guarantee
> > > > > > a consumer to detect that is metadata is outdated. Currently, the
> > > > > consumer
> > > > > > is not guaranteed to fetch metadata from every broker within some
> > > > bounded
> > > > > > period of time. Maybe this is out of the scope of your KIP. But
> one
> > > > idea
> > > > > is
> > > > > > force the consumer to refresh metadata from the controller
> > > > periodically.
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > >
> > > > > > On Thu, Dec 7, 2017 at 11:25 AM, Dong Lin <li...@gmail.com>
> > > wrote:
> > > > > >
> > > > > > > Hey Jun,
> > > > > > >
> > > > > > > Thanks much for the comments. Great point particularly
> regarding
> > > > (3). I
> > > > > > > haven't thought about this before.
> > > > > > >
> > > > > > > It seems that there are two possible ways where the version
> > number
> > > > can
> > > > > be
> > > > > > > used. One solution is for client to check the version number at
> > the
> > > > > time
> > > > > > it
> > > > > > > receives MetadataResponse. And if the version number in the
> > > > > > > MetadataResponse is smaller than the version number in the
> > client's
> > > > > > cache,
> > > > > > > the client will be forced to fetch metadata again.  Another
> > > solution,
> > > > > as
> > > > > > > you have suggested, is for broker to check the version number
> at
> > > the
> > > > > time
> > > > > > > it receives a request from client. The broker will reject the
> > > request
> > > > > if
> > > > > > > the version is smaller than the version in broker's cache.
> > > > > > >
> > > > > > > I am not very sure that the second solution can address the
> > problem
> > > > > here.
> > > > > > > In the scenario described in the JIRA ticket, broker's cache
> may
> > be
> > > > > > > outdated because it has not processed the LeaderAndIsrRequest
> > from
> > > > the
> > > > > > > controller. Thus it may still process client's request even if
> > the
> > > > > > version
> > > > > > > in client's request is actually outdated. Does this make sense?
> > > > > > >
> > > > > > > IMO, it seems that we can address problem (3) by saving the
> > > metadata
> > > > > > > version together with the offset. After consumer starts, it
> will
> > > keep
> > > > > > > fetching metadata until the metadata version >= the version
> saved
> > > > with
> > > > > > the
> > > > > > > offset of this partition.
> > > > > > >
> > > > > > > Regarding problems (1) and (2): Currently we use the version
> > number
> > > > in
> > > > > > the
> > > > > > > MetadataResponse to ensure that the metadata does not go back
> in
> > > > time.
> > > > > > > There are two alternative solutions to address problems (1) and
> > > (2).
> > > > > One
> > > > > > > solution is for client to enumerate all partitions in the
> > > > > > MetadataResponse,
> > > > > > > compare their epoch with those in the cached metadata, and
> > rejects
> > > > the
> > > > > > > MetadataResponse iff any leader epoch is smaller. The main
> > concern
> > > is
> > > > > > that
> > > > > > > MetadataResponse currently cached information of all partitions
> > in
> > > > the
> > > > > > > entire cluster. It may slow down client's performance if we
> were
> > to
> > > > do
> > > > > > it.
> > > > > > > The other solution is for client to enumerate partitions for
> only
> > > > > topics
> > > > > > > registered in the org.apache.kafka.clients.Metadata, which
> will
> > be
> > > > an
> > > > > > > empty
> > > > > > > set for producer and the set of subscribed partitions for
> > consumer.
> > > > But
> > > > > > > this degrades to all topics if consumer subscribes to topics in
> > the
> > > > > > cluster
> > > > > > > by pattern.
> > > > > > >
> > > > > > > Note that client will only be forced to update metadata if the
> > > > version
> > > > > in
> > > > > > > the MetadataResponse is smaller than the version in the cached
> > > > > metadata.
> > > > > > In
> > > > > > > general it should not be a problem. It can be a problem only if
> > > some
> > > > > > broker
> > > > > > > is particularly slower than other brokers in processing
> > > > > > > UpdateMetadataRequest. When this is the case, it means that the
> > > > broker
> > > > > is
> > > > > > > also particularly slower in processing LeaderAndIsrRequest,
> which
> > > can
> > > > > > cause
> > > > > > > problem anyway because some partition will probably have no
> > leader
> > > > > during
> > > > > > > this period. I am not sure problems (1) and (2) cause more
> > problem
> > > > than
> > > > > > > what we already have.
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Dong
> > > > > > >
> > > > > > >
> > > > > > > On Wed, Dec 6, 2017 at 6:42 PM, Jun Rao <ju...@confluent.io>
> > wrote:
> > > > > > >
> > > > > > > > Hi, Dong,
> > > > > > > >
> > > > > > > > Great finding on the issue. It's a real problem. A few
> comments
> > > > about
> > > > > > the
> > > > > > > > KIP. (1) I am not sure about updating
> controller_metadata_epoch
> > > on
> > > > > > every
> > > > > > > > UpdateMetadataRequest. Currently, the controller can send
> > > > > > > > UpdateMetadataRequest when there is no actual metadata
> change.
> > > > Doing
> > > > > > this
> > > > > > > > may require unnecessary metadata refresh on the client. (2)
> > > > > > > > controller_metadata_epoch is global across all topics. This
> > means
> > > > > that
> > > > > > a
> > > > > > > > client may be forced to update its metadata even when the
> > > metadata
> > > > > for
> > > > > > > the
> > > > > > > > topics that it cares haven't changed. (3) It doesn't seem
> that
> > > the
> > > > > KIP
> > > > > > > > handles the corner case when a consumer is restarted. Say a
> > > > consumer
> > > > > > > reads
> > > > > > > > from the new leader, commits the offset and then is
> restarted.
> > On
> > > > > > > restart,
> > > > > > > > the consumer gets an outdated metadata and fetches from the
> old
> > > > > leader.
> > > > > > > > Then, the consumer will get into the offset out of range
> issue.
> > > > > > > >
> > > > > > > > Given the above, I am thinking of the following approach. We
> > > > actually
> > > > > > > > already have metadata versioning at the partition level. Each
> > > > leader
> > > > > > has
> > > > > > > a
> > > > > > > > leader epoch which is monotonically increasing. We can
> > > potentially
> > > > > > > > propagate leader epoch back in the metadata response and the
> > > > clients
> > > > > > can
> > > > > > > > cache that. This solves the issue of (1) and (2). To solve
> (3),
> > > > when
> > > > > > > saving
> > > > > > > > an offset, we could save both an offset and the corresponding
> > > > leader
> > > > > > > epoch.
> > > > > > > > When fetching the data, the consumer provides both the offset
> > and
> > > > the
> > > > > > > > leader epoch. A leader will only serve the request if its
> > leader
> > > > > epoch
> > > > > > is
> > > > > > > > equal to or greater than the leader epoch from the consumer.
> To
> > > > > achieve
> > > > > > > > this, we need to change the fetch request protocol and the
> > offset
> > > > > > commit
> > > > > > > > api, which requires some more thoughts.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > >
> > > > > > > > Jun
> > > > > > > >
> > > > > > > >
> > > > > > > > On Wed, Dec 6, 2017 at 10:57 AM, Dong Lin <
> lindong28@gmail.com
> > >
> > > > > wrote:
> > > > > > > >
> > > > > > > > > Bump up the thread.
> > > > > > > > >
> > > > > > > > > It will be great to have more comments on whether we should
> > do
> > > it
> > > > > or
> > > > > > > > > whether there is better way to address the motivation of
> this
> > > > KIP.
> > > > > > > > >
> > > > > > > > > On Mon, Dec 4, 2017 at 3:09 PM, Dong Lin <
> > lindong28@gmail.com>
> > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > I don't have an interesting rejected alternative solution
> > to
> > > > put
> > > > > in
> > > > > > > the
> > > > > > > > > > KIP. If there is good alternative solution from anyone in
> > > this
> > > > > > > thread,
> > > > > > > > I
> > > > > > > > > am
> > > > > > > > > > happy to discuss this and update the KIP accordingly.
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > > Dong
> > > > > > > > > >
> > > > > > > > > > On Mon, Dec 4, 2017 at 1:12 PM, Ted Yu <
> > yuzhihong@gmail.com>
> > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > >> It is clearer now.
> > > > > > > > > >>
> > > > > > > > > >> I noticed that Rejected Alternatives section is empty.
> > > > > > > > > >> Have you considered any alternative ?
> > > > > > > > > >>
> > > > > > > > > >> Cheers
> > > > > > > > > >>
> > > > > > > > > >> On Mon, Dec 4, 2017 at 1:07 PM, Dong Lin <
> > > lindong28@gmail.com
> > > > >
> > > > > > > wrote:
> > > > > > > > > >>
> > > > > > > > > >> > Ted, thanks for catching this. I have updated the
> > sentence
> > > > to
> > > > > > make
> > > > > > > > it
> > > > > > > > > >> > readable.
> > > > > > > > > >> >
> > > > > > > > > >> > Thanks,
> > > > > > > > > >> > Dong
> > > > > > > > > >> >
> > > > > > > > > >> > On Sat, Dec 2, 2017 at 3:05 PM, Ted Yu <
> > > yuzhihong@gmail.com
> > > > >
> > > > > > > wrote:
> > > > > > > > > >> >
> > > > > > > > > >> > > bq. It the controller_epoch of the incoming
> > > > > MetadataResponse,
> > > > > > or
> > > > > > > > if
> > > > > > > > > >> the
> > > > > > > > > >> > > controller_epoch is the same but the
> > > > > controller_metadata_epoch
> > > > > > > > > >> > >
> > > > > > > > > >> > > Can you update the above sentence so that the
> > intention
> > > is
> > > > > > > > clearer ?
> > > > > > > > > >> > >
> > > > > > > > > >> > > Thanks
> > > > > > > > > >> > >
> > > > > > > > > >> > > On Fri, Dec 1, 2017 at 6:33 PM, Dong Lin <
> > > > > lindong28@gmail.com
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > >> > >
> > > > > > > > > >> > > > Hi all,
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > I have created KIP-232: Detect outdated metadata
> by
> > > > adding
> > > > > > > > > >> > > > ControllerMetadataEpoch field:
> > > > > > > > > >> > > > https://cwiki.apache.org/
> > > confluence/display/KAFKA/KIP-
> > > > > > > > > >> > > > 232%3A+Detect+outdated+metadata+by+adding+
> > > > > > > > > >> > ControllerMetadataEpoch+field
> > > > > > > > > >> > > > .
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > The KIP proposes to add fields in MetadataResponse
> > and
> > > > > > > > > >> > > > UpdateMetadataRequest so that client can reject
> > > outdated
> > > > > > > > metadata
> > > > > > > > > >> and
> > > > > > > > > >> > > avoid
> > > > > > > > > >> > > > unnecessary OffsetOutOfRangeException. Otherwise
> > there
> > > > is
> > > > > > > > > currently
> > > > > > > > > >> > race
> > > > > > > > > >> > > > condition that can cause consumer to reset offset
> > > which
> > > > > > > > negatively
> > > > > > > > > >> > affect
> > > > > > > > > >> > > > the consumer's availability.
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > Feedback and suggestions are welcome!
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > Regards,
> > > > > > > > > >> > > > Dong
> > > > > > > > > >> > > >
> > > > > > > > > >> > >
> > > > > > > > > >> >
> > > > > > > > > >>
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Dong Lin <li...@gmail.com>.
Hey Jun,

I see. Sounds good. Yeah it is probably simpler to leave this to another
KIP in the future.

Thanks for all the comments. Since there is no further comment in the
community, I will open the voting thread.

Thanks,
Dong

On Mon, Dec 11, 2017 at 5:37 PM, Jun Rao <ju...@confluent.io> wrote:

> Hi, Dong,
>
> The case that I am thinking is network partitioning. Suppose one deploys a
> stretched cluster across multiple AZs in the same region. If the machines
> in one AZ can't communicate to brokers in other AZs due to a network issue,
> the brokers in that AZ won't get any new metadata.
>
> We can potentially solve this problem by requiring some kind of regular
> heartbeats between the controller and the broker. This may need some more
> thoughts. So, it's probably fine to leave this to another KIP in the
> future.
>
> Thanks,
>
> Jun
>
> On Mon, Dec 11, 2017 at 2:55 PM, Dong Lin <li...@gmail.com> wrote:
>
> > Hey Jun,
> >
> > Thanks for the comment. I am open to improve this KIP to address more
> > problems. I probably need more help in understanding what is the current
> > problem with consumer using outdated metadata and whether it is easier to
> > address it together with this KIP.
> >
> > I agree that a consumer can potentially talk to old leader for a long
> time
> > even after this KIP. But after this KIP, the consumer probably should not
> > get OffetOutofRangeException and therefore will not cause offset rewind
> > issue. So the only problem is that consumer will not be able to fetch
> data
> > until it has updated metadata. It seems that this situation can only
> happen
> > if the broker is too slow in processing LeaderAndIsrRequest since
> otherwise
> > the consumer will be forced to update metadata due to
> > NotLeaderForPartitionException. So the problem we are having here is
> that
> > consumer will not be able to fetch data if some broker is too slow in
> > processing LeaderAndIsrRequest.
> >
> > Because Kafka propagates LeaderAndIsrRequest asynchronously to all
> brokers
> > in the cluster, there will always be a period of time when consumer can
> not
> > fetch data for the partition during the leadership change. Thus it seems
> > more like a broker-side performance issue instead of client-side
> > correctness issue. My gut feel is that it is not causing a much a problem
> > as the problem to be fixed in this KIP. And if we were to address it, we
> > probably need to make change in the broker side, e.g. with prioritized
> > queue for controller-related requests, which may be kind of orthogonal to
> > this KIP. I am not very sure it will be easier to address it with the
> > change in this KIP. Do you have any recommendation?
> >
> > Thanks,
> > Dong
> >
> >
> > On Mon, Dec 11, 2017 at 1:51 PM, Jun Rao <ju...@confluent.io> wrote:
> >
> > > Hi, Dong,
> > >
> > > Thanks for the reply.
> > >
> > > My suggestion of forcing the metadata refresh from the controller may
> not
> > > work in general since the cached controller could be outdated too. The
> > > general problem is that if a consumer's metadata is outdated, it may
> get
> > > stuck with the old leader for a long time. We can address the issue of
> > > detecting outdated metadata in a separate KIP in the future if you
> didn't
> > > intend to address it in this KIP.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > >
> > > On Sat, Dec 9, 2017 at 10:12 PM, Dong Lin <li...@gmail.com> wrote:
> > >
> > > > Hey Jun,
> > > >
> > > > Thanks much for your comments. Given that client needs to
> de-serialize
> > > the
> > > > metadata anyway, the extra overhead of checking the per-partition
> > version
> > > > for every partition should not be a big concern. Thus it makes sense
> to
> > > use
> > > > leader epoch as the per-partition version instead of creating a
> global
> > > > metadata version. I will update the KIP to do that.
> > > >
> > > > Regarding the detection of outdated metadata, I think it is possible
> to
> > > > ensure that client gets latest metadata by fetching from controller.
> > Note
> > > > that this requires extra logic in the controller such that controller
> > > > updates metadata directly in memory without requiring
> > > > UpdateMetadataRequest. But I am not sure the main motivation of this
> at
> > > > this moment. But this makes controller more like a bottleneck in the
> > > > cluster which we probably want to avoid.
> > > >
> > > > I think we can probably keep the current way of ensuring metadata
> > > > freshness. Currently client will be forced to refresh metadata if
> > broker
> > > > returns error (e.g. NotLeaderForPartition) due to outdated metadata
> or
> > if
> > > > the metadata does not contain the partition that the client needs. In
> > the
> > > > future, as you previously suggested, we can include per-partition
> > > > leaderEpoch in the FetchRequest/ProduceRequest such that broker can
> > > return
> > > > error if the epoch is smaller than cached epoch in the broker. Given
> > that
> > > > this adds more complexity to Kafka, I think we can probably think
> about
> > > > that leader when we have a specific use-case or problem to solve with
> > > > up-to-date metadata. Does this sound OK?
> > > >
> > > > Thanks,
> > > > Dong
> > > >
> > > >
> > > >
> > > > On Fri, Dec 8, 2017 at 3:53 PM, Jun Rao <ju...@confluent.io> wrote:
> > > >
> > > > > Hi, Dong,
> > > > >
> > > > > Thanks for the reply. A few more points below.
> > > > >
> > > > > For dealing with how to prevent a consumer switching from a new
> > leader
> > > to
> > > > > an old leader, you suggestion that refreshes metadata on consumer
> > > restart
> > > > > until it sees a metadata version >= the one associated with the
> > offset
> > > > > works too, as long as we guarantee that the cached metadata
> versions
> > on
> > > > the
> > > > > brokers only go up.
> > > > >
> > > > > The second discussion point is on whether the metadata versioning
> > > should
> > > > be
> > > > > per partition or global. For the partition level versioning, you
> were
> > > > > concerned about the performance. Given that metadata updates are
> > rare,
> > > I
> > > > am
> > > > > not sure if it's a big concern though. Doing a million if tests is
> > > > probably
> > > > > going to take less than 1ms. Another thing is that the metadata
> > version
> > > > > seems to need to survive controller failover. In your current
> > > approach, a
> > > > > consumer may not be able to wait on the right version of the
> metadata
> > > > after
> > > > > the consumer restart since the metadata version may have been
> > recycled
> > > on
> > > > > the server side due to a controller failover while the consumer is
> > > down.
> > > > > The partition level leaderEpoch survives controller failure and
> won't
> > > > have
> > > > > this issue.
> > > > >
> > > > > Lastly, neither your proposal nor mine addresses the issue how to
> > > > guarantee
> > > > > a consumer to detect that is metadata is outdated. Currently, the
> > > > consumer
> > > > > is not guaranteed to fetch metadata from every broker within some
> > > bounded
> > > > > period of time. Maybe this is out of the scope of your KIP. But one
> > > idea
> > > > is
> > > > > force the consumer to refresh metadata from the controller
> > > periodically.
> > > > >
> > > > > Jun
> > > > >
> > > > >
> > > > > On Thu, Dec 7, 2017 at 11:25 AM, Dong Lin <li...@gmail.com>
> > wrote:
> > > > >
> > > > > > Hey Jun,
> > > > > >
> > > > > > Thanks much for the comments. Great point particularly regarding
> > > (3). I
> > > > > > haven't thought about this before.
> > > > > >
> > > > > > It seems that there are two possible ways where the version
> number
> > > can
> > > > be
> > > > > > used. One solution is for client to check the version number at
> the
> > > > time
> > > > > it
> > > > > > receives MetadataResponse. And if the version number in the
> > > > > > MetadataResponse is smaller than the version number in the
> client's
> > > > > cache,
> > > > > > the client will be forced to fetch metadata again.  Another
> > solution,
> > > > as
> > > > > > you have suggested, is for broker to check the version number at
> > the
> > > > time
> > > > > > it receives a request from client. The broker will reject the
> > request
> > > > if
> > > > > > the version is smaller than the version in broker's cache.
> > > > > >
> > > > > > I am not very sure that the second solution can address the
> problem
> > > > here.
> > > > > > In the scenario described in the JIRA ticket, broker's cache may
> be
> > > > > > outdated because it has not processed the LeaderAndIsrRequest
> from
> > > the
> > > > > > controller. Thus it may still process client's request even if
> the
> > > > > version
> > > > > > in client's request is actually outdated. Does this make sense?
> > > > > >
> > > > > > IMO, it seems that we can address problem (3) by saving the
> > metadata
> > > > > > version together with the offset. After consumer starts, it will
> > keep
> > > > > > fetching metadata until the metadata version >= the version saved
> > > with
> > > > > the
> > > > > > offset of this partition.
> > > > > >
> > > > > > Regarding problems (1) and (2): Currently we use the version
> number
> > > in
> > > > > the
> > > > > > MetadataResponse to ensure that the metadata does not go back in
> > > time.
> > > > > > There are two alternative solutions to address problems (1) and
> > (2).
> > > > One
> > > > > > solution is for client to enumerate all partitions in the
> > > > > MetadataResponse,
> > > > > > compare their epoch with those in the cached metadata, and
> rejects
> > > the
> > > > > > MetadataResponse iff any leader epoch is smaller. The main
> concern
> > is
> > > > > that
> > > > > > MetadataResponse currently cached information of all partitions
> in
> > > the
> > > > > > entire cluster. It may slow down client's performance if we were
> to
> > > do
> > > > > it.
> > > > > > The other solution is for client to enumerate partitions for only
> > > > topics
> > > > > > registered in the org.apache.kafka.clients.Metadata, which will
> be
> > > an
> > > > > > empty
> > > > > > set for producer and the set of subscribed partitions for
> consumer.
> > > But
> > > > > > this degrades to all topics if consumer subscribes to topics in
> the
> > > > > cluster
> > > > > > by pattern.
> > > > > >
> > > > > > Note that client will only be forced to update metadata if the
> > > version
> > > > in
> > > > > > the MetadataResponse is smaller than the version in the cached
> > > > metadata.
> > > > > In
> > > > > > general it should not be a problem. It can be a problem only if
> > some
> > > > > broker
> > > > > > is particularly slower than other brokers in processing
> > > > > > UpdateMetadataRequest. When this is the case, it means that the
> > > broker
> > > > is
> > > > > > also particularly slower in processing LeaderAndIsrRequest, which
> > can
> > > > > cause
> > > > > > problem anyway because some partition will probably have no
> leader
> > > > during
> > > > > > this period. I am not sure problems (1) and (2) cause more
> problem
> > > than
> > > > > > what we already have.
> > > > > >
> > > > > > Thanks,
> > > > > > Dong
> > > > > >
> > > > > >
> > > > > > On Wed, Dec 6, 2017 at 6:42 PM, Jun Rao <ju...@confluent.io>
> wrote:
> > > > > >
> > > > > > > Hi, Dong,
> > > > > > >
> > > > > > > Great finding on the issue. It's a real problem. A few comments
> > > about
> > > > > the
> > > > > > > KIP. (1) I am not sure about updating controller_metadata_epoch
> > on
> > > > > every
> > > > > > > UpdateMetadataRequest. Currently, the controller can send
> > > > > > > UpdateMetadataRequest when there is no actual metadata change.
> > > Doing
> > > > > this
> > > > > > > may require unnecessary metadata refresh on the client. (2)
> > > > > > > controller_metadata_epoch is global across all topics. This
> means
> > > > that
> > > > > a
> > > > > > > client may be forced to update its metadata even when the
> > metadata
> > > > for
> > > > > > the
> > > > > > > topics that it cares haven't changed. (3) It doesn't seem that
> > the
> > > > KIP
> > > > > > > handles the corner case when a consumer is restarted. Say a
> > > consumer
> > > > > > reads
> > > > > > > from the new leader, commits the offset and then is restarted.
> On
> > > > > > restart,
> > > > > > > the consumer gets an outdated metadata and fetches from the old
> > > > leader.
> > > > > > > Then, the consumer will get into the offset out of range issue.
> > > > > > >
> > > > > > > Given the above, I am thinking of the following approach. We
> > > actually
> > > > > > > already have metadata versioning at the partition level. Each
> > > leader
> > > > > has
> > > > > > a
> > > > > > > leader epoch which is monotonically increasing. We can
> > potentially
> > > > > > > propagate leader epoch back in the metadata response and the
> > > clients
> > > > > can
> > > > > > > cache that. This solves the issue of (1) and (2). To solve (3),
> > > when
> > > > > > saving
> > > > > > > an offset, we could save both an offset and the corresponding
> > > leader
> > > > > > epoch.
> > > > > > > When fetching the data, the consumer provides both the offset
> and
> > > the
> > > > > > > leader epoch. A leader will only serve the request if its
> leader
> > > > epoch
> > > > > is
> > > > > > > equal to or greater than the leader epoch from the consumer. To
> > > > achieve
> > > > > > > this, we need to change the fetch request protocol and the
> offset
> > > > > commit
> > > > > > > api, which requires some more thoughts.
> > > > > > >
> > > > > > > Thanks,
> > > > > > >
> > > > > > > Jun
> > > > > > >
> > > > > > >
> > > > > > > On Wed, Dec 6, 2017 at 10:57 AM, Dong Lin <lindong28@gmail.com
> >
> > > > wrote:
> > > > > > >
> > > > > > > > Bump up the thread.
> > > > > > > >
> > > > > > > > It will be great to have more comments on whether we should
> do
> > it
> > > > or
> > > > > > > > whether there is better way to address the motivation of this
> > > KIP.
> > > > > > > >
> > > > > > > > On Mon, Dec 4, 2017 at 3:09 PM, Dong Lin <
> lindong28@gmail.com>
> > > > > wrote:
> > > > > > > >
> > > > > > > > > I don't have an interesting rejected alternative solution
> to
> > > put
> > > > in
> > > > > > the
> > > > > > > > > KIP. If there is good alternative solution from anyone in
> > this
> > > > > > thread,
> > > > > > > I
> > > > > > > > am
> > > > > > > > > happy to discuss this and update the KIP accordingly.
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > > Dong
> > > > > > > > >
> > > > > > > > > On Mon, Dec 4, 2017 at 1:12 PM, Ted Yu <
> yuzhihong@gmail.com>
> > > > > wrote:
> > > > > > > > >
> > > > > > > > >> It is clearer now.
> > > > > > > > >>
> > > > > > > > >> I noticed that Rejected Alternatives section is empty.
> > > > > > > > >> Have you considered any alternative ?
> > > > > > > > >>
> > > > > > > > >> Cheers
> > > > > > > > >>
> > > > > > > > >> On Mon, Dec 4, 2017 at 1:07 PM, Dong Lin <
> > lindong28@gmail.com
> > > >
> > > > > > wrote:
> > > > > > > > >>
> > > > > > > > >> > Ted, thanks for catching this. I have updated the
> sentence
> > > to
> > > > > make
> > > > > > > it
> > > > > > > > >> > readable.
> > > > > > > > >> >
> > > > > > > > >> > Thanks,
> > > > > > > > >> > Dong
> > > > > > > > >> >
> > > > > > > > >> > On Sat, Dec 2, 2017 at 3:05 PM, Ted Yu <
> > yuzhihong@gmail.com
> > > >
> > > > > > wrote:
> > > > > > > > >> >
> > > > > > > > >> > > bq. It the controller_epoch of the incoming
> > > > MetadataResponse,
> > > > > or
> > > > > > > if
> > > > > > > > >> the
> > > > > > > > >> > > controller_epoch is the same but the
> > > > controller_metadata_epoch
> > > > > > > > >> > >
> > > > > > > > >> > > Can you update the above sentence so that the
> intention
> > is
> > > > > > > clearer ?
> > > > > > > > >> > >
> > > > > > > > >> > > Thanks
> > > > > > > > >> > >
> > > > > > > > >> > > On Fri, Dec 1, 2017 at 6:33 PM, Dong Lin <
> > > > lindong28@gmail.com
> > > > > >
> > > > > > > > wrote:
> > > > > > > > >> > >
> > > > > > > > >> > > > Hi all,
> > > > > > > > >> > > >
> > > > > > > > >> > > > I have created KIP-232: Detect outdated metadata by
> > > adding
> > > > > > > > >> > > > ControllerMetadataEpoch field:
> > > > > > > > >> > > > https://cwiki.apache.org/
> > confluence/display/KAFKA/KIP-
> > > > > > > > >> > > > 232%3A+Detect+outdated+metadata+by+adding+
> > > > > > > > >> > ControllerMetadataEpoch+field
> > > > > > > > >> > > > .
> > > > > > > > >> > > >
> > > > > > > > >> > > > The KIP proposes to add fields in MetadataResponse
> and
> > > > > > > > >> > > > UpdateMetadataRequest so that client can reject
> > outdated
> > > > > > > metadata
> > > > > > > > >> and
> > > > > > > > >> > > avoid
> > > > > > > > >> > > > unnecessary OffsetOutOfRangeException. Otherwise
> there
> > > is
> > > > > > > > currently
> > > > > > > > >> > race
> > > > > > > > >> > > > condition that can cause consumer to reset offset
> > which
> > > > > > > negatively
> > > > > > > > >> > affect
> > > > > > > > >> > > > the consumer's availability.
> > > > > > > > >> > > >
> > > > > > > > >> > > > Feedback and suggestions are welcome!
> > > > > > > > >> > > >
> > > > > > > > >> > > > Regards,
> > > > > > > > >> > > > Dong
> > > > > > > > >> > > >
> > > > > > > > >> > >
> > > > > > > > >> >
> > > > > > > > >>
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Jun Rao <ju...@confluent.io>.
Hi, Dong,

The case that I am thinking is network partitioning. Suppose one deploys a
stretched cluster across multiple AZs in the same region. If the machines
in one AZ can't communicate to brokers in other AZs due to a network issue,
the brokers in that AZ won't get any new metadata.

We can potentially solve this problem by requiring some kind of regular
heartbeats between the controller and the broker. This may need some more
thoughts. So, it's probably fine to leave this to another KIP in the future.

Thanks,

Jun

On Mon, Dec 11, 2017 at 2:55 PM, Dong Lin <li...@gmail.com> wrote:

> Hey Jun,
>
> Thanks for the comment. I am open to improve this KIP to address more
> problems. I probably need more help in understanding what is the current
> problem with consumer using outdated metadata and whether it is easier to
> address it together with this KIP.
>
> I agree that a consumer can potentially talk to old leader for a long time
> even after this KIP. But after this KIP, the consumer probably should not
> get OffetOutofRangeException and therefore will not cause offset rewind
> issue. So the only problem is that consumer will not be able to fetch data
> until it has updated metadata. It seems that this situation can only happen
> if the broker is too slow in processing LeaderAndIsrRequest since otherwise
> the consumer will be forced to update metadata due to
> NotLeaderForPartitionException. So the problem we are having here is that
> consumer will not be able to fetch data if some broker is too slow in
> processing LeaderAndIsrRequest.
>
> Because Kafka propagates LeaderAndIsrRequest asynchronously to all brokers
> in the cluster, there will always be a period of time when consumer can not
> fetch data for the partition during the leadership change. Thus it seems
> more like a broker-side performance issue instead of client-side
> correctness issue. My gut feel is that it is not causing a much a problem
> as the problem to be fixed in this KIP. And if we were to address it, we
> probably need to make change in the broker side, e.g. with prioritized
> queue for controller-related requests, which may be kind of orthogonal to
> this KIP. I am not very sure it will be easier to address it with the
> change in this KIP. Do you have any recommendation?
>
> Thanks,
> Dong
>
>
> On Mon, Dec 11, 2017 at 1:51 PM, Jun Rao <ju...@confluent.io> wrote:
>
> > Hi, Dong,
> >
> > Thanks for the reply.
> >
> > My suggestion of forcing the metadata refresh from the controller may not
> > work in general since the cached controller could be outdated too. The
> > general problem is that if a consumer's metadata is outdated, it may get
> > stuck with the old leader for a long time. We can address the issue of
> > detecting outdated metadata in a separate KIP in the future if you didn't
> > intend to address it in this KIP.
> >
> > Thanks,
> >
> > Jun
> >
> >
> > On Sat, Dec 9, 2017 at 10:12 PM, Dong Lin <li...@gmail.com> wrote:
> >
> > > Hey Jun,
> > >
> > > Thanks much for your comments. Given that client needs to de-serialize
> > the
> > > metadata anyway, the extra overhead of checking the per-partition
> version
> > > for every partition should not be a big concern. Thus it makes sense to
> > use
> > > leader epoch as the per-partition version instead of creating a global
> > > metadata version. I will update the KIP to do that.
> > >
> > > Regarding the detection of outdated metadata, I think it is possible to
> > > ensure that client gets latest metadata by fetching from controller.
> Note
> > > that this requires extra logic in the controller such that controller
> > > updates metadata directly in memory without requiring
> > > UpdateMetadataRequest. But I am not sure the main motivation of this at
> > > this moment. But this makes controller more like a bottleneck in the
> > > cluster which we probably want to avoid.
> > >
> > > I think we can probably keep the current way of ensuring metadata
> > > freshness. Currently client will be forced to refresh metadata if
> broker
> > > returns error (e.g. NotLeaderForPartition) due to outdated metadata or
> if
> > > the metadata does not contain the partition that the client needs. In
> the
> > > future, as you previously suggested, we can include per-partition
> > > leaderEpoch in the FetchRequest/ProduceRequest such that broker can
> > return
> > > error if the epoch is smaller than cached epoch in the broker. Given
> that
> > > this adds more complexity to Kafka, I think we can probably think about
> > > that leader when we have a specific use-case or problem to solve with
> > > up-to-date metadata. Does this sound OK?
> > >
> > > Thanks,
> > > Dong
> > >
> > >
> > >
> > > On Fri, Dec 8, 2017 at 3:53 PM, Jun Rao <ju...@confluent.io> wrote:
> > >
> > > > Hi, Dong,
> > > >
> > > > Thanks for the reply. A few more points below.
> > > >
> > > > For dealing with how to prevent a consumer switching from a new
> leader
> > to
> > > > an old leader, you suggestion that refreshes metadata on consumer
> > restart
> > > > until it sees a metadata version >= the one associated with the
> offset
> > > > works too, as long as we guarantee that the cached metadata versions
> on
> > > the
> > > > brokers only go up.
> > > >
> > > > The second discussion point is on whether the metadata versioning
> > should
> > > be
> > > > per partition or global. For the partition level versioning, you were
> > > > concerned about the performance. Given that metadata updates are
> rare,
> > I
> > > am
> > > > not sure if it's a big concern though. Doing a million if tests is
> > > probably
> > > > going to take less than 1ms. Another thing is that the metadata
> version
> > > > seems to need to survive controller failover. In your current
> > approach, a
> > > > consumer may not be able to wait on the right version of the metadata
> > > after
> > > > the consumer restart since the metadata version may have been
> recycled
> > on
> > > > the server side due to a controller failover while the consumer is
> > down.
> > > > The partition level leaderEpoch survives controller failure and won't
> > > have
> > > > this issue.
> > > >
> > > > Lastly, neither your proposal nor mine addresses the issue how to
> > > guarantee
> > > > a consumer to detect that is metadata is outdated. Currently, the
> > > consumer
> > > > is not guaranteed to fetch metadata from every broker within some
> > bounded
> > > > period of time. Maybe this is out of the scope of your KIP. But one
> > idea
> > > is
> > > > force the consumer to refresh metadata from the controller
> > periodically.
> > > >
> > > > Jun
> > > >
> > > >
> > > > On Thu, Dec 7, 2017 at 11:25 AM, Dong Lin <li...@gmail.com>
> wrote:
> > > >
> > > > > Hey Jun,
> > > > >
> > > > > Thanks much for the comments. Great point particularly regarding
> > (3). I
> > > > > haven't thought about this before.
> > > > >
> > > > > It seems that there are two possible ways where the version number
> > can
> > > be
> > > > > used. One solution is for client to check the version number at the
> > > time
> > > > it
> > > > > receives MetadataResponse. And if the version number in the
> > > > > MetadataResponse is smaller than the version number in the client's
> > > > cache,
> > > > > the client will be forced to fetch metadata again.  Another
> solution,
> > > as
> > > > > you have suggested, is for broker to check the version number at
> the
> > > time
> > > > > it receives a request from client. The broker will reject the
> request
> > > if
> > > > > the version is smaller than the version in broker's cache.
> > > > >
> > > > > I am not very sure that the second solution can address the problem
> > > here.
> > > > > In the scenario described in the JIRA ticket, broker's cache may be
> > > > > outdated because it has not processed the LeaderAndIsrRequest from
> > the
> > > > > controller. Thus it may still process client's request even if the
> > > > version
> > > > > in client's request is actually outdated. Does this make sense?
> > > > >
> > > > > IMO, it seems that we can address problem (3) by saving the
> metadata
> > > > > version together with the offset. After consumer starts, it will
> keep
> > > > > fetching metadata until the metadata version >= the version saved
> > with
> > > > the
> > > > > offset of this partition.
> > > > >
> > > > > Regarding problems (1) and (2): Currently we use the version number
> > in
> > > > the
> > > > > MetadataResponse to ensure that the metadata does not go back in
> > time.
> > > > > There are two alternative solutions to address problems (1) and
> (2).
> > > One
> > > > > solution is for client to enumerate all partitions in the
> > > > MetadataResponse,
> > > > > compare their epoch with those in the cached metadata, and rejects
> > the
> > > > > MetadataResponse iff any leader epoch is smaller. The main concern
> is
> > > > that
> > > > > MetadataResponse currently cached information of all partitions in
> > the
> > > > > entire cluster. It may slow down client's performance if we were to
> > do
> > > > it.
> > > > > The other solution is for client to enumerate partitions for only
> > > topics
> > > > > registered in the org.apache.kafka.clients.Metadata, which will be
> > an
> > > > > empty
> > > > > set for producer and the set of subscribed partitions for consumer.
> > But
> > > > > this degrades to all topics if consumer subscribes to topics in the
> > > > cluster
> > > > > by pattern.
> > > > >
> > > > > Note that client will only be forced to update metadata if the
> > version
> > > in
> > > > > the MetadataResponse is smaller than the version in the cached
> > > metadata.
> > > > In
> > > > > general it should not be a problem. It can be a problem only if
> some
> > > > broker
> > > > > is particularly slower than other brokers in processing
> > > > > UpdateMetadataRequest. When this is the case, it means that the
> > broker
> > > is
> > > > > also particularly slower in processing LeaderAndIsrRequest, which
> can
> > > > cause
> > > > > problem anyway because some partition will probably have no leader
> > > during
> > > > > this period. I am not sure problems (1) and (2) cause more problem
> > than
> > > > > what we already have.
> > > > >
> > > > > Thanks,
> > > > > Dong
> > > > >
> > > > >
> > > > > On Wed, Dec 6, 2017 at 6:42 PM, Jun Rao <ju...@confluent.io> wrote:
> > > > >
> > > > > > Hi, Dong,
> > > > > >
> > > > > > Great finding on the issue. It's a real problem. A few comments
> > about
> > > > the
> > > > > > KIP. (1) I am not sure about updating controller_metadata_epoch
> on
> > > > every
> > > > > > UpdateMetadataRequest. Currently, the controller can send
> > > > > > UpdateMetadataRequest when there is no actual metadata change.
> > Doing
> > > > this
> > > > > > may require unnecessary metadata refresh on the client. (2)
> > > > > > controller_metadata_epoch is global across all topics. This means
> > > that
> > > > a
> > > > > > client may be forced to update its metadata even when the
> metadata
> > > for
> > > > > the
> > > > > > topics that it cares haven't changed. (3) It doesn't seem that
> the
> > > KIP
> > > > > > handles the corner case when a consumer is restarted. Say a
> > consumer
> > > > > reads
> > > > > > from the new leader, commits the offset and then is restarted. On
> > > > > restart,
> > > > > > the consumer gets an outdated metadata and fetches from the old
> > > leader.
> > > > > > Then, the consumer will get into the offset out of range issue.
> > > > > >
> > > > > > Given the above, I am thinking of the following approach. We
> > actually
> > > > > > already have metadata versioning at the partition level. Each
> > leader
> > > > has
> > > > > a
> > > > > > leader epoch which is monotonically increasing. We can
> potentially
> > > > > > propagate leader epoch back in the metadata response and the
> > clients
> > > > can
> > > > > > cache that. This solves the issue of (1) and (2). To solve (3),
> > when
> > > > > saving
> > > > > > an offset, we could save both an offset and the corresponding
> > leader
> > > > > epoch.
> > > > > > When fetching the data, the consumer provides both the offset and
> > the
> > > > > > leader epoch. A leader will only serve the request if its leader
> > > epoch
> > > > is
> > > > > > equal to or greater than the leader epoch from the consumer. To
> > > achieve
> > > > > > this, we need to change the fetch request protocol and the offset
> > > > commit
> > > > > > api, which requires some more thoughts.
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > >
> > > > > > On Wed, Dec 6, 2017 at 10:57 AM, Dong Lin <li...@gmail.com>
> > > wrote:
> > > > > >
> > > > > > > Bump up the thread.
> > > > > > >
> > > > > > > It will be great to have more comments on whether we should do
> it
> > > or
> > > > > > > whether there is better way to address the motivation of this
> > KIP.
> > > > > > >
> > > > > > > On Mon, Dec 4, 2017 at 3:09 PM, Dong Lin <li...@gmail.com>
> > > > wrote:
> > > > > > >
> > > > > > > > I don't have an interesting rejected alternative solution to
> > put
> > > in
> > > > > the
> > > > > > > > KIP. If there is good alternative solution from anyone in
> this
> > > > > thread,
> > > > > > I
> > > > > > > am
> > > > > > > > happy to discuss this and update the KIP accordingly.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Dong
> > > > > > > >
> > > > > > > > On Mon, Dec 4, 2017 at 1:12 PM, Ted Yu <yu...@gmail.com>
> > > > wrote:
> > > > > > > >
> > > > > > > >> It is clearer now.
> > > > > > > >>
> > > > > > > >> I noticed that Rejected Alternatives section is empty.
> > > > > > > >> Have you considered any alternative ?
> > > > > > > >>
> > > > > > > >> Cheers
> > > > > > > >>
> > > > > > > >> On Mon, Dec 4, 2017 at 1:07 PM, Dong Lin <
> lindong28@gmail.com
> > >
> > > > > wrote:
> > > > > > > >>
> > > > > > > >> > Ted, thanks for catching this. I have updated the sentence
> > to
> > > > make
> > > > > > it
> > > > > > > >> > readable.
> > > > > > > >> >
> > > > > > > >> > Thanks,
> > > > > > > >> > Dong
> > > > > > > >> >
> > > > > > > >> > On Sat, Dec 2, 2017 at 3:05 PM, Ted Yu <
> yuzhihong@gmail.com
> > >
> > > > > wrote:
> > > > > > > >> >
> > > > > > > >> > > bq. It the controller_epoch of the incoming
> > > MetadataResponse,
> > > > or
> > > > > > if
> > > > > > > >> the
> > > > > > > >> > > controller_epoch is the same but the
> > > controller_metadata_epoch
> > > > > > > >> > >
> > > > > > > >> > > Can you update the above sentence so that the intention
> is
> > > > > > clearer ?
> > > > > > > >> > >
> > > > > > > >> > > Thanks
> > > > > > > >> > >
> > > > > > > >> > > On Fri, Dec 1, 2017 at 6:33 PM, Dong Lin <
> > > lindong28@gmail.com
> > > > >
> > > > > > > wrote:
> > > > > > > >> > >
> > > > > > > >> > > > Hi all,
> > > > > > > >> > > >
> > > > > > > >> > > > I have created KIP-232: Detect outdated metadata by
> > adding
> > > > > > > >> > > > ControllerMetadataEpoch field:
> > > > > > > >> > > > https://cwiki.apache.org/
> confluence/display/KAFKA/KIP-
> > > > > > > >> > > > 232%3A+Detect+outdated+metadata+by+adding+
> > > > > > > >> > ControllerMetadataEpoch+field
> > > > > > > >> > > > .
> > > > > > > >> > > >
> > > > > > > >> > > > The KIP proposes to add fields in MetadataResponse and
> > > > > > > >> > > > UpdateMetadataRequest so that client can reject
> outdated
> > > > > > metadata
> > > > > > > >> and
> > > > > > > >> > > avoid
> > > > > > > >> > > > unnecessary OffsetOutOfRangeException. Otherwise there
> > is
> > > > > > > currently
> > > > > > > >> > race
> > > > > > > >> > > > condition that can cause consumer to reset offset
> which
> > > > > > negatively
> > > > > > > >> > affect
> > > > > > > >> > > > the consumer's availability.
> > > > > > > >> > > >
> > > > > > > >> > > > Feedback and suggestions are welcome!
> > > > > > > >> > > >
> > > > > > > >> > > > Regards,
> > > > > > > >> > > > Dong
> > > > > > > >> > > >
> > > > > > > >> > >
> > > > > > > >> >
> > > > > > > >>
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Dong Lin <li...@gmail.com>.
Hey Jun,

Thanks for the comment. I am open to improve this KIP to address more
problems. I probably need more help in understanding what is the current
problem with consumer using outdated metadata and whether it is easier to
address it together with this KIP.

I agree that a consumer can potentially talk to old leader for a long time
even after this KIP. But after this KIP, the consumer probably should not
get OffetOutofRangeException and therefore will not cause offset rewind
issue. So the only problem is that consumer will not be able to fetch data
until it has updated metadata. It seems that this situation can only happen
if the broker is too slow in processing LeaderAndIsrRequest since otherwise
the consumer will be forced to update metadata due to
NotLeaderForPartitionException. So the problem we are having here is that
consumer will not be able to fetch data if some broker is too slow in
processing LeaderAndIsrRequest.

Because Kafka propagates LeaderAndIsrRequest asynchronously to all brokers
in the cluster, there will always be a period of time when consumer can not
fetch data for the partition during the leadership change. Thus it seems
more like a broker-side performance issue instead of client-side
correctness issue. My gut feel is that it is not causing a much a problem
as the problem to be fixed in this KIP. And if we were to address it, we
probably need to make change in the broker side, e.g. with prioritized
queue for controller-related requests, which may be kind of orthogonal to
this KIP. I am not very sure it will be easier to address it with the
change in this KIP. Do you have any recommendation?

Thanks,
Dong


On Mon, Dec 11, 2017 at 1:51 PM, Jun Rao <ju...@confluent.io> wrote:

> Hi, Dong,
>
> Thanks for the reply.
>
> My suggestion of forcing the metadata refresh from the controller may not
> work in general since the cached controller could be outdated too. The
> general problem is that if a consumer's metadata is outdated, it may get
> stuck with the old leader for a long time. We can address the issue of
> detecting outdated metadata in a separate KIP in the future if you didn't
> intend to address it in this KIP.
>
> Thanks,
>
> Jun
>
>
> On Sat, Dec 9, 2017 at 10:12 PM, Dong Lin <li...@gmail.com> wrote:
>
> > Hey Jun,
> >
> > Thanks much for your comments. Given that client needs to de-serialize
> the
> > metadata anyway, the extra overhead of checking the per-partition version
> > for every partition should not be a big concern. Thus it makes sense to
> use
> > leader epoch as the per-partition version instead of creating a global
> > metadata version. I will update the KIP to do that.
> >
> > Regarding the detection of outdated metadata, I think it is possible to
> > ensure that client gets latest metadata by fetching from controller. Note
> > that this requires extra logic in the controller such that controller
> > updates metadata directly in memory without requiring
> > UpdateMetadataRequest. But I am not sure the main motivation of this at
> > this moment. But this makes controller more like a bottleneck in the
> > cluster which we probably want to avoid.
> >
> > I think we can probably keep the current way of ensuring metadata
> > freshness. Currently client will be forced to refresh metadata if broker
> > returns error (e.g. NotLeaderForPartition) due to outdated metadata or if
> > the metadata does not contain the partition that the client needs. In the
> > future, as you previously suggested, we can include per-partition
> > leaderEpoch in the FetchRequest/ProduceRequest such that broker can
> return
> > error if the epoch is smaller than cached epoch in the broker. Given that
> > this adds more complexity to Kafka, I think we can probably think about
> > that leader when we have a specific use-case or problem to solve with
> > up-to-date metadata. Does this sound OK?
> >
> > Thanks,
> > Dong
> >
> >
> >
> > On Fri, Dec 8, 2017 at 3:53 PM, Jun Rao <ju...@confluent.io> wrote:
> >
> > > Hi, Dong,
> > >
> > > Thanks for the reply. A few more points below.
> > >
> > > For dealing with how to prevent a consumer switching from a new leader
> to
> > > an old leader, you suggestion that refreshes metadata on consumer
> restart
> > > until it sees a metadata version >= the one associated with the offset
> > > works too, as long as we guarantee that the cached metadata versions on
> > the
> > > brokers only go up.
> > >
> > > The second discussion point is on whether the metadata versioning
> should
> > be
> > > per partition or global. For the partition level versioning, you were
> > > concerned about the performance. Given that metadata updates are rare,
> I
> > am
> > > not sure if it's a big concern though. Doing a million if tests is
> > probably
> > > going to take less than 1ms. Another thing is that the metadata version
> > > seems to need to survive controller failover. In your current
> approach, a
> > > consumer may not be able to wait on the right version of the metadata
> > after
> > > the consumer restart since the metadata version may have been recycled
> on
> > > the server side due to a controller failover while the consumer is
> down.
> > > The partition level leaderEpoch survives controller failure and won't
> > have
> > > this issue.
> > >
> > > Lastly, neither your proposal nor mine addresses the issue how to
> > guarantee
> > > a consumer to detect that is metadata is outdated. Currently, the
> > consumer
> > > is not guaranteed to fetch metadata from every broker within some
> bounded
> > > period of time. Maybe this is out of the scope of your KIP. But one
> idea
> > is
> > > force the consumer to refresh metadata from the controller
> periodically.
> > >
> > > Jun
> > >
> > >
> > > On Thu, Dec 7, 2017 at 11:25 AM, Dong Lin <li...@gmail.com> wrote:
> > >
> > > > Hey Jun,
> > > >
> > > > Thanks much for the comments. Great point particularly regarding
> (3). I
> > > > haven't thought about this before.
> > > >
> > > > It seems that there are two possible ways where the version number
> can
> > be
> > > > used. One solution is for client to check the version number at the
> > time
> > > it
> > > > receives MetadataResponse. And if the version number in the
> > > > MetadataResponse is smaller than the version number in the client's
> > > cache,
> > > > the client will be forced to fetch metadata again.  Another solution,
> > as
> > > > you have suggested, is for broker to check the version number at the
> > time
> > > > it receives a request from client. The broker will reject the request
> > if
> > > > the version is smaller than the version in broker's cache.
> > > >
> > > > I am not very sure that the second solution can address the problem
> > here.
> > > > In the scenario described in the JIRA ticket, broker's cache may be
> > > > outdated because it has not processed the LeaderAndIsrRequest from
> the
> > > > controller. Thus it may still process client's request even if the
> > > version
> > > > in client's request is actually outdated. Does this make sense?
> > > >
> > > > IMO, it seems that we can address problem (3) by saving the metadata
> > > > version together with the offset. After consumer starts, it will keep
> > > > fetching metadata until the metadata version >= the version saved
> with
> > > the
> > > > offset of this partition.
> > > >
> > > > Regarding problems (1) and (2): Currently we use the version number
> in
> > > the
> > > > MetadataResponse to ensure that the metadata does not go back in
> time.
> > > > There are two alternative solutions to address problems (1) and (2).
> > One
> > > > solution is for client to enumerate all partitions in the
> > > MetadataResponse,
> > > > compare their epoch with those in the cached metadata, and rejects
> the
> > > > MetadataResponse iff any leader epoch is smaller. The main concern is
> > > that
> > > > MetadataResponse currently cached information of all partitions in
> the
> > > > entire cluster. It may slow down client's performance if we were to
> do
> > > it.
> > > > The other solution is for client to enumerate partitions for only
> > topics
> > > > registered in the org.apache.kafka.clients.Metadata, which will be
> an
> > > > empty
> > > > set for producer and the set of subscribed partitions for consumer.
> But
> > > > this degrades to all topics if consumer subscribes to topics in the
> > > cluster
> > > > by pattern.
> > > >
> > > > Note that client will only be forced to update metadata if the
> version
> > in
> > > > the MetadataResponse is smaller than the version in the cached
> > metadata.
> > > In
> > > > general it should not be a problem. It can be a problem only if some
> > > broker
> > > > is particularly slower than other brokers in processing
> > > > UpdateMetadataRequest. When this is the case, it means that the
> broker
> > is
> > > > also particularly slower in processing LeaderAndIsrRequest, which can
> > > cause
> > > > problem anyway because some partition will probably have no leader
> > during
> > > > this period. I am not sure problems (1) and (2) cause more problem
> than
> > > > what we already have.
> > > >
> > > > Thanks,
> > > > Dong
> > > >
> > > >
> > > > On Wed, Dec 6, 2017 at 6:42 PM, Jun Rao <ju...@confluent.io> wrote:
> > > >
> > > > > Hi, Dong,
> > > > >
> > > > > Great finding on the issue. It's a real problem. A few comments
> about
> > > the
> > > > > KIP. (1) I am not sure about updating controller_metadata_epoch on
> > > every
> > > > > UpdateMetadataRequest. Currently, the controller can send
> > > > > UpdateMetadataRequest when there is no actual metadata change.
> Doing
> > > this
> > > > > may require unnecessary metadata refresh on the client. (2)
> > > > > controller_metadata_epoch is global across all topics. This means
> > that
> > > a
> > > > > client may be forced to update its metadata even when the metadata
> > for
> > > > the
> > > > > topics that it cares haven't changed. (3) It doesn't seem that the
> > KIP
> > > > > handles the corner case when a consumer is restarted. Say a
> consumer
> > > > reads
> > > > > from the new leader, commits the offset and then is restarted. On
> > > > restart,
> > > > > the consumer gets an outdated metadata and fetches from the old
> > leader.
> > > > > Then, the consumer will get into the offset out of range issue.
> > > > >
> > > > > Given the above, I am thinking of the following approach. We
> actually
> > > > > already have metadata versioning at the partition level. Each
> leader
> > > has
> > > > a
> > > > > leader epoch which is monotonically increasing. We can potentially
> > > > > propagate leader epoch back in the metadata response and the
> clients
> > > can
> > > > > cache that. This solves the issue of (1) and (2). To solve (3),
> when
> > > > saving
> > > > > an offset, we could save both an offset and the corresponding
> leader
> > > > epoch.
> > > > > When fetching the data, the consumer provides both the offset and
> the
> > > > > leader epoch. A leader will only serve the request if its leader
> > epoch
> > > is
> > > > > equal to or greater than the leader epoch from the consumer. To
> > achieve
> > > > > this, we need to change the fetch request protocol and the offset
> > > commit
> > > > > api, which requires some more thoughts.
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Jun
> > > > >
> > > > >
> > > > > On Wed, Dec 6, 2017 at 10:57 AM, Dong Lin <li...@gmail.com>
> > wrote:
> > > > >
> > > > > > Bump up the thread.
> > > > > >
> > > > > > It will be great to have more comments on whether we should do it
> > or
> > > > > > whether there is better way to address the motivation of this
> KIP.
> > > > > >
> > > > > > On Mon, Dec 4, 2017 at 3:09 PM, Dong Lin <li...@gmail.com>
> > > wrote:
> > > > > >
> > > > > > > I don't have an interesting rejected alternative solution to
> put
> > in
> > > > the
> > > > > > > KIP. If there is good alternative solution from anyone in this
> > > > thread,
> > > > > I
> > > > > > am
> > > > > > > happy to discuss this and update the KIP accordingly.
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Dong
> > > > > > >
> > > > > > > On Mon, Dec 4, 2017 at 1:12 PM, Ted Yu <yu...@gmail.com>
> > > wrote:
> > > > > > >
> > > > > > >> It is clearer now.
> > > > > > >>
> > > > > > >> I noticed that Rejected Alternatives section is empty.
> > > > > > >> Have you considered any alternative ?
> > > > > > >>
> > > > > > >> Cheers
> > > > > > >>
> > > > > > >> On Mon, Dec 4, 2017 at 1:07 PM, Dong Lin <lindong28@gmail.com
> >
> > > > wrote:
> > > > > > >>
> > > > > > >> > Ted, thanks for catching this. I have updated the sentence
> to
> > > make
> > > > > it
> > > > > > >> > readable.
> > > > > > >> >
> > > > > > >> > Thanks,
> > > > > > >> > Dong
> > > > > > >> >
> > > > > > >> > On Sat, Dec 2, 2017 at 3:05 PM, Ted Yu <yuzhihong@gmail.com
> >
> > > > wrote:
> > > > > > >> >
> > > > > > >> > > bq. It the controller_epoch of the incoming
> > MetadataResponse,
> > > or
> > > > > if
> > > > > > >> the
> > > > > > >> > > controller_epoch is the same but the
> > controller_metadata_epoch
> > > > > > >> > >
> > > > > > >> > > Can you update the above sentence so that the intention is
> > > > > clearer ?
> > > > > > >> > >
> > > > > > >> > > Thanks
> > > > > > >> > >
> > > > > > >> > > On Fri, Dec 1, 2017 at 6:33 PM, Dong Lin <
> > lindong28@gmail.com
> > > >
> > > > > > wrote:
> > > > > > >> > >
> > > > > > >> > > > Hi all,
> > > > > > >> > > >
> > > > > > >> > > > I have created KIP-232: Detect outdated metadata by
> adding
> > > > > > >> > > > ControllerMetadataEpoch field:
> > > > > > >> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > > > >> > > > 232%3A+Detect+outdated+metadata+by+adding+
> > > > > > >> > ControllerMetadataEpoch+field
> > > > > > >> > > > .
> > > > > > >> > > >
> > > > > > >> > > > The KIP proposes to add fields in MetadataResponse and
> > > > > > >> > > > UpdateMetadataRequest so that client can reject outdated
> > > > > metadata
> > > > > > >> and
> > > > > > >> > > avoid
> > > > > > >> > > > unnecessary OffsetOutOfRangeException. Otherwise there
> is
> > > > > > currently
> > > > > > >> > race
> > > > > > >> > > > condition that can cause consumer to reset offset which
> > > > > negatively
> > > > > > >> > affect
> > > > > > >> > > > the consumer's availability.
> > > > > > >> > > >
> > > > > > >> > > > Feedback and suggestions are welcome!
> > > > > > >> > > >
> > > > > > >> > > > Regards,
> > > > > > >> > > > Dong
> > > > > > >> > > >
> > > > > > >> > >
> > > > > > >> >
> > > > > > >>
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Jun Rao <ju...@confluent.io>.
Hi, Dong,

Thanks for the reply.

My suggestion of forcing the metadata refresh from the controller may not
work in general since the cached controller could be outdated too. The
general problem is that if a consumer's metadata is outdated, it may get
stuck with the old leader for a long time. We can address the issue of
detecting outdated metadata in a separate KIP in the future if you didn't
intend to address it in this KIP.

Thanks,

Jun


On Sat, Dec 9, 2017 at 10:12 PM, Dong Lin <li...@gmail.com> wrote:

> Hey Jun,
>
> Thanks much for your comments. Given that client needs to de-serialize the
> metadata anyway, the extra overhead of checking the per-partition version
> for every partition should not be a big concern. Thus it makes sense to use
> leader epoch as the per-partition version instead of creating a global
> metadata version. I will update the KIP to do that.
>
> Regarding the detection of outdated metadata, I think it is possible to
> ensure that client gets latest metadata by fetching from controller. Note
> that this requires extra logic in the controller such that controller
> updates metadata directly in memory without requiring
> UpdateMetadataRequest. But I am not sure the main motivation of this at
> this moment. But this makes controller more like a bottleneck in the
> cluster which we probably want to avoid.
>
> I think we can probably keep the current way of ensuring metadata
> freshness. Currently client will be forced to refresh metadata if broker
> returns error (e.g. NotLeaderForPartition) due to outdated metadata or if
> the metadata does not contain the partition that the client needs. In the
> future, as you previously suggested, we can include per-partition
> leaderEpoch in the FetchRequest/ProduceRequest such that broker can return
> error if the epoch is smaller than cached epoch in the broker. Given that
> this adds more complexity to Kafka, I think we can probably think about
> that leader when we have a specific use-case or problem to solve with
> up-to-date metadata. Does this sound OK?
>
> Thanks,
> Dong
>
>
>
> On Fri, Dec 8, 2017 at 3:53 PM, Jun Rao <ju...@confluent.io> wrote:
>
> > Hi, Dong,
> >
> > Thanks for the reply. A few more points below.
> >
> > For dealing with how to prevent a consumer switching from a new leader to
> > an old leader, you suggestion that refreshes metadata on consumer restart
> > until it sees a metadata version >= the one associated with the offset
> > works too, as long as we guarantee that the cached metadata versions on
> the
> > brokers only go up.
> >
> > The second discussion point is on whether the metadata versioning should
> be
> > per partition or global. For the partition level versioning, you were
> > concerned about the performance. Given that metadata updates are rare, I
> am
> > not sure if it's a big concern though. Doing a million if tests is
> probably
> > going to take less than 1ms. Another thing is that the metadata version
> > seems to need to survive controller failover. In your current approach, a
> > consumer may not be able to wait on the right version of the metadata
> after
> > the consumer restart since the metadata version may have been recycled on
> > the server side due to a controller failover while the consumer is down.
> > The partition level leaderEpoch survives controller failure and won't
> have
> > this issue.
> >
> > Lastly, neither your proposal nor mine addresses the issue how to
> guarantee
> > a consumer to detect that is metadata is outdated. Currently, the
> consumer
> > is not guaranteed to fetch metadata from every broker within some bounded
> > period of time. Maybe this is out of the scope of your KIP. But one idea
> is
> > force the consumer to refresh metadata from the controller periodically.
> >
> > Jun
> >
> >
> > On Thu, Dec 7, 2017 at 11:25 AM, Dong Lin <li...@gmail.com> wrote:
> >
> > > Hey Jun,
> > >
> > > Thanks much for the comments. Great point particularly regarding (3). I
> > > haven't thought about this before.
> > >
> > > It seems that there are two possible ways where the version number can
> be
> > > used. One solution is for client to check the version number at the
> time
> > it
> > > receives MetadataResponse. And if the version number in the
> > > MetadataResponse is smaller than the version number in the client's
> > cache,
> > > the client will be forced to fetch metadata again.  Another solution,
> as
> > > you have suggested, is for broker to check the version number at the
> time
> > > it receives a request from client. The broker will reject the request
> if
> > > the version is smaller than the version in broker's cache.
> > >
> > > I am not very sure that the second solution can address the problem
> here.
> > > In the scenario described in the JIRA ticket, broker's cache may be
> > > outdated because it has not processed the LeaderAndIsrRequest from the
> > > controller. Thus it may still process client's request even if the
> > version
> > > in client's request is actually outdated. Does this make sense?
> > >
> > > IMO, it seems that we can address problem (3) by saving the metadata
> > > version together with the offset. After consumer starts, it will keep
> > > fetching metadata until the metadata version >= the version saved with
> > the
> > > offset of this partition.
> > >
> > > Regarding problems (1) and (2): Currently we use the version number in
> > the
> > > MetadataResponse to ensure that the metadata does not go back in time.
> > > There are two alternative solutions to address problems (1) and (2).
> One
> > > solution is for client to enumerate all partitions in the
> > MetadataResponse,
> > > compare their epoch with those in the cached metadata, and rejects the
> > > MetadataResponse iff any leader epoch is smaller. The main concern is
> > that
> > > MetadataResponse currently cached information of all partitions in the
> > > entire cluster. It may slow down client's performance if we were to do
> > it.
> > > The other solution is for client to enumerate partitions for only
> topics
> > > registered in the org.apache.kafka.clients.Metadata, which will be an
> > > empty
> > > set for producer and the set of subscribed partitions for consumer. But
> > > this degrades to all topics if consumer subscribes to topics in the
> > cluster
> > > by pattern.
> > >
> > > Note that client will only be forced to update metadata if the version
> in
> > > the MetadataResponse is smaller than the version in the cached
> metadata.
> > In
> > > general it should not be a problem. It can be a problem only if some
> > broker
> > > is particularly slower than other brokers in processing
> > > UpdateMetadataRequest. When this is the case, it means that the broker
> is
> > > also particularly slower in processing LeaderAndIsrRequest, which can
> > cause
> > > problem anyway because some partition will probably have no leader
> during
> > > this period. I am not sure problems (1) and (2) cause more problem than
> > > what we already have.
> > >
> > > Thanks,
> > > Dong
> > >
> > >
> > > On Wed, Dec 6, 2017 at 6:42 PM, Jun Rao <ju...@confluent.io> wrote:
> > >
> > > > Hi, Dong,
> > > >
> > > > Great finding on the issue. It's a real problem. A few comments about
> > the
> > > > KIP. (1) I am not sure about updating controller_metadata_epoch on
> > every
> > > > UpdateMetadataRequest. Currently, the controller can send
> > > > UpdateMetadataRequest when there is no actual metadata change. Doing
> > this
> > > > may require unnecessary metadata refresh on the client. (2)
> > > > controller_metadata_epoch is global across all topics. This means
> that
> > a
> > > > client may be forced to update its metadata even when the metadata
> for
> > > the
> > > > topics that it cares haven't changed. (3) It doesn't seem that the
> KIP
> > > > handles the corner case when a consumer is restarted. Say a consumer
> > > reads
> > > > from the new leader, commits the offset and then is restarted. On
> > > restart,
> > > > the consumer gets an outdated metadata and fetches from the old
> leader.
> > > > Then, the consumer will get into the offset out of range issue.
> > > >
> > > > Given the above, I am thinking of the following approach. We actually
> > > > already have metadata versioning at the partition level. Each leader
> > has
> > > a
> > > > leader epoch which is monotonically increasing. We can potentially
> > > > propagate leader epoch back in the metadata response and the clients
> > can
> > > > cache that. This solves the issue of (1) and (2). To solve (3), when
> > > saving
> > > > an offset, we could save both an offset and the corresponding leader
> > > epoch.
> > > > When fetching the data, the consumer provides both the offset and the
> > > > leader epoch. A leader will only serve the request if its leader
> epoch
> > is
> > > > equal to or greater than the leader epoch from the consumer. To
> achieve
> > > > this, we need to change the fetch request protocol and the offset
> > commit
> > > > api, which requires some more thoughts.
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > >
> > > > On Wed, Dec 6, 2017 at 10:57 AM, Dong Lin <li...@gmail.com>
> wrote:
> > > >
> > > > > Bump up the thread.
> > > > >
> > > > > It will be great to have more comments on whether we should do it
> or
> > > > > whether there is better way to address the motivation of this KIP.
> > > > >
> > > > > On Mon, Dec 4, 2017 at 3:09 PM, Dong Lin <li...@gmail.com>
> > wrote:
> > > > >
> > > > > > I don't have an interesting rejected alternative solution to put
> in
> > > the
> > > > > > KIP. If there is good alternative solution from anyone in this
> > > thread,
> > > > I
> > > > > am
> > > > > > happy to discuss this and update the KIP accordingly.
> > > > > >
> > > > > > Thanks,
> > > > > > Dong
> > > > > >
> > > > > > On Mon, Dec 4, 2017 at 1:12 PM, Ted Yu <yu...@gmail.com>
> > wrote:
> > > > > >
> > > > > >> It is clearer now.
> > > > > >>
> > > > > >> I noticed that Rejected Alternatives section is empty.
> > > > > >> Have you considered any alternative ?
> > > > > >>
> > > > > >> Cheers
> > > > > >>
> > > > > >> On Mon, Dec 4, 2017 at 1:07 PM, Dong Lin <li...@gmail.com>
> > > wrote:
> > > > > >>
> > > > > >> > Ted, thanks for catching this. I have updated the sentence to
> > make
> > > > it
> > > > > >> > readable.
> > > > > >> >
> > > > > >> > Thanks,
> > > > > >> > Dong
> > > > > >> >
> > > > > >> > On Sat, Dec 2, 2017 at 3:05 PM, Ted Yu <yu...@gmail.com>
> > > wrote:
> > > > > >> >
> > > > > >> > > bq. It the controller_epoch of the incoming
> MetadataResponse,
> > or
> > > > if
> > > > > >> the
> > > > > >> > > controller_epoch is the same but the
> controller_metadata_epoch
> > > > > >> > >
> > > > > >> > > Can you update the above sentence so that the intention is
> > > > clearer ?
> > > > > >> > >
> > > > > >> > > Thanks
> > > > > >> > >
> > > > > >> > > On Fri, Dec 1, 2017 at 6:33 PM, Dong Lin <
> lindong28@gmail.com
> > >
> > > > > wrote:
> > > > > >> > >
> > > > > >> > > > Hi all,
> > > > > >> > > >
> > > > > >> > > > I have created KIP-232: Detect outdated metadata by adding
> > > > > >> > > > ControllerMetadataEpoch field:
> > > > > >> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > > >> > > > 232%3A+Detect+outdated+metadata+by+adding+
> > > > > >> > ControllerMetadataEpoch+field
> > > > > >> > > > .
> > > > > >> > > >
> > > > > >> > > > The KIP proposes to add fields in MetadataResponse and
> > > > > >> > > > UpdateMetadataRequest so that client can reject outdated
> > > > metadata
> > > > > >> and
> > > > > >> > > avoid
> > > > > >> > > > unnecessary OffsetOutOfRangeException. Otherwise there is
> > > > > currently
> > > > > >> > race
> > > > > >> > > > condition that can cause consumer to reset offset which
> > > > negatively
> > > > > >> > affect
> > > > > >> > > > the consumer's availability.
> > > > > >> > > >
> > > > > >> > > > Feedback and suggestions are welcome!
> > > > > >> > > >
> > > > > >> > > > Regards,
> > > > > >> > > > Dong
> > > > > >> > > >
> > > > > >> > >
> > > > > >> >
> > > > > >>
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Dong Lin <li...@gmail.com>.
Hey Jun,

I have updated the KIP based on our discussion. Thanks!

Dong

On Sat, Dec 9, 2017 at 10:12 PM, Dong Lin <li...@gmail.com> wrote:

> Hey Jun,
>
> Thanks much for your comments. Given that client needs to de-serialize the
> metadata anyway, the extra overhead of checking the per-partition version
> for every partition should not be a big concern. Thus it makes sense to use
> leader epoch as the per-partition version instead of creating a global
> metadata version. I will update the KIP to do that.
>
> Regarding the detection of outdated metadata, I think it is possible to
> ensure that client gets latest metadata by fetching from controller. Note
> that this requires extra logic in the controller such that controller
> updates metadata directly in memory without requiring
> UpdateMetadataRequest. But I am not sure the main motivation of this at
> this moment. But this makes controller more like a bottleneck in the
> cluster which we probably want to avoid.
>
> I think we can probably keep the current way of ensuring metadata
> freshness. Currently client will be forced to refresh metadata if broker
> returns error (e.g. NotLeaderForPartition) due to outdated metadata or if
> the metadata does not contain the partition that the client needs. In the
> future, as you previously suggested, we can include per-partition
> leaderEpoch in the FetchRequest/ProduceRequest such that broker can return
> error if the epoch is smaller than cached epoch in the broker. Given that
> this adds more complexity to Kafka, I think we can probably think about
> that leader when we have a specific use-case or problem to solve with
> up-to-date metadata. Does this sound OK?
>
> Thanks,
> Dong
>
>
>
> On Fri, Dec 8, 2017 at 3:53 PM, Jun Rao <ju...@confluent.io> wrote:
>
>> Hi, Dong,
>>
>> Thanks for the reply. A few more points below.
>>
>> For dealing with how to prevent a consumer switching from a new leader to
>> an old leader, you suggestion that refreshes metadata on consumer restart
>> until it sees a metadata version >= the one associated with the offset
>> works too, as long as we guarantee that the cached metadata versions on
>> the
>> brokers only go up.
>>
>> The second discussion point is on whether the metadata versioning should
>> be
>> per partition or global. For the partition level versioning, you were
>> concerned about the performance. Given that metadata updates are rare, I
>> am
>> not sure if it's a big concern though. Doing a million if tests is
>> probably
>> going to take less than 1ms. Another thing is that the metadata version
>> seems to need to survive controller failover. In your current approach, a
>> consumer may not be able to wait on the right version of the metadata
>> after
>> the consumer restart since the metadata version may have been recycled on
>> the server side due to a controller failover while the consumer is down.
>> The partition level leaderEpoch survives controller failure and won't have
>> this issue.
>>
>> Lastly, neither your proposal nor mine addresses the issue how to
>> guarantee
>> a consumer to detect that is metadata is outdated. Currently, the consumer
>> is not guaranteed to fetch metadata from every broker within some bounded
>> period of time. Maybe this is out of the scope of your KIP. But one idea
>> is
>> force the consumer to refresh metadata from the controller periodically.
>>
>> Jun
>>
>>
>> On Thu, Dec 7, 2017 at 11:25 AM, Dong Lin <li...@gmail.com> wrote:
>>
>> > Hey Jun,
>> >
>> > Thanks much for the comments. Great point particularly regarding (3). I
>> > haven't thought about this before.
>> >
>> > It seems that there are two possible ways where the version number can
>> be
>> > used. One solution is for client to check the version number at the
>> time it
>> > receives MetadataResponse. And if the version number in the
>> > MetadataResponse is smaller than the version number in the client's
>> cache,
>> > the client will be forced to fetch metadata again.  Another solution, as
>> > you have suggested, is for broker to check the version number at the
>> time
>> > it receives a request from client. The broker will reject the request if
>> > the version is smaller than the version in broker's cache.
>> >
>> > I am not very sure that the second solution can address the problem
>> here.
>> > In the scenario described in the JIRA ticket, broker's cache may be
>> > outdated because it has not processed the LeaderAndIsrRequest from the
>> > controller. Thus it may still process client's request even if the
>> version
>> > in client's request is actually outdated. Does this make sense?
>> >
>> > IMO, it seems that we can address problem (3) by saving the metadata
>> > version together with the offset. After consumer starts, it will keep
>> > fetching metadata until the metadata version >= the version saved with
>> the
>> > offset of this partition.
>> >
>> > Regarding problems (1) and (2): Currently we use the version number in
>> the
>> > MetadataResponse to ensure that the metadata does not go back in time.
>> > There are two alternative solutions to address problems (1) and (2). One
>> > solution is for client to enumerate all partitions in the
>> MetadataResponse,
>> > compare their epoch with those in the cached metadata, and rejects the
>> > MetadataResponse iff any leader epoch is smaller. The main concern is
>> that
>> > MetadataResponse currently cached information of all partitions in the
>> > entire cluster. It may slow down client's performance if we were to do
>> it.
>> > The other solution is for client to enumerate partitions for only topics
>> > registered in the org.apache.kafka.clients.Metadata, which will be an
>> > empty
>> > set for producer and the set of subscribed partitions for consumer. But
>> > this degrades to all topics if consumer subscribes to topics in the
>> cluster
>> > by pattern.
>> >
>> > Note that client will only be forced to update metadata if the version
>> in
>> > the MetadataResponse is smaller than the version in the cached
>> metadata. In
>> > general it should not be a problem. It can be a problem only if some
>> broker
>> > is particularly slower than other brokers in processing
>> > UpdateMetadataRequest. When this is the case, it means that the broker
>> is
>> > also particularly slower in processing LeaderAndIsrRequest, which can
>> cause
>> > problem anyway because some partition will probably have no leader
>> during
>> > this period. I am not sure problems (1) and (2) cause more problem than
>> > what we already have.
>> >
>> > Thanks,
>> > Dong
>> >
>> >
>> > On Wed, Dec 6, 2017 at 6:42 PM, Jun Rao <ju...@confluent.io> wrote:
>> >
>> > > Hi, Dong,
>> > >
>> > > Great finding on the issue. It's a real problem. A few comments about
>> the
>> > > KIP. (1) I am not sure about updating controller_metadata_epoch on
>> every
>> > > UpdateMetadataRequest. Currently, the controller can send
>> > > UpdateMetadataRequest when there is no actual metadata change. Doing
>> this
>> > > may require unnecessary metadata refresh on the client. (2)
>> > > controller_metadata_epoch is global across all topics. This means
>> that a
>> > > client may be forced to update its metadata even when the metadata for
>> > the
>> > > topics that it cares haven't changed. (3) It doesn't seem that the KIP
>> > > handles the corner case when a consumer is restarted. Say a consumer
>> > reads
>> > > from the new leader, commits the offset and then is restarted. On
>> > restart,
>> > > the consumer gets an outdated metadata and fetches from the old
>> leader.
>> > > Then, the consumer will get into the offset out of range issue.
>> > >
>> > > Given the above, I am thinking of the following approach. We actually
>> > > already have metadata versioning at the partition level. Each leader
>> has
>> > a
>> > > leader epoch which is monotonically increasing. We can potentially
>> > > propagate leader epoch back in the metadata response and the clients
>> can
>> > > cache that. This solves the issue of (1) and (2). To solve (3), when
>> > saving
>> > > an offset, we could save both an offset and the corresponding leader
>> > epoch.
>> > > When fetching the data, the consumer provides both the offset and the
>> > > leader epoch. A leader will only serve the request if its leader
>> epoch is
>> > > equal to or greater than the leader epoch from the consumer. To
>> achieve
>> > > this, we need to change the fetch request protocol and the offset
>> commit
>> > > api, which requires some more thoughts.
>> > >
>> > > Thanks,
>> > >
>> > > Jun
>> > >
>> > >
>> > > On Wed, Dec 6, 2017 at 10:57 AM, Dong Lin <li...@gmail.com>
>> wrote:
>> > >
>> > > > Bump up the thread.
>> > > >
>> > > > It will be great to have more comments on whether we should do it or
>> > > > whether there is better way to address the motivation of this KIP.
>> > > >
>> > > > On Mon, Dec 4, 2017 at 3:09 PM, Dong Lin <li...@gmail.com>
>> wrote:
>> > > >
>> > > > > I don't have an interesting rejected alternative solution to put
>> in
>> > the
>> > > > > KIP. If there is good alternative solution from anyone in this
>> > thread,
>> > > I
>> > > > am
>> > > > > happy to discuss this and update the KIP accordingly.
>> > > > >
>> > > > > Thanks,
>> > > > > Dong
>> > > > >
>> > > > > On Mon, Dec 4, 2017 at 1:12 PM, Ted Yu <yu...@gmail.com>
>> wrote:
>> > > > >
>> > > > >> It is clearer now.
>> > > > >>
>> > > > >> I noticed that Rejected Alternatives section is empty.
>> > > > >> Have you considered any alternative ?
>> > > > >>
>> > > > >> Cheers
>> > > > >>
>> > > > >> On Mon, Dec 4, 2017 at 1:07 PM, Dong Lin <li...@gmail.com>
>> > wrote:
>> > > > >>
>> > > > >> > Ted, thanks for catching this. I have updated the sentence to
>> make
>> > > it
>> > > > >> > readable.
>> > > > >> >
>> > > > >> > Thanks,
>> > > > >> > Dong
>> > > > >> >
>> > > > >> > On Sat, Dec 2, 2017 at 3:05 PM, Ted Yu <yu...@gmail.com>
>> > wrote:
>> > > > >> >
>> > > > >> > > bq. It the controller_epoch of the incoming
>> MetadataResponse, or
>> > > if
>> > > > >> the
>> > > > >> > > controller_epoch is the same but the
>> controller_metadata_epoch
>> > > > >> > >
>> > > > >> > > Can you update the above sentence so that the intention is
>> > > clearer ?
>> > > > >> > >
>> > > > >> > > Thanks
>> > > > >> > >
>> > > > >> > > On Fri, Dec 1, 2017 at 6:33 PM, Dong Lin <
>> lindong28@gmail.com>
>> > > > wrote:
>> > > > >> > >
>> > > > >> > > > Hi all,
>> > > > >> > > >
>> > > > >> > > > I have created KIP-232: Detect outdated metadata by adding
>> > > > >> > > > ControllerMetadataEpoch field:
>> > > > >> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> > > > >> > > > 232%3A+Detect+outdated+metadata+by+adding+
>> > > > >> > ControllerMetadataEpoch+field
>> > > > >> > > > .
>> > > > >> > > >
>> > > > >> > > > The KIP proposes to add fields in MetadataResponse and
>> > > > >> > > > UpdateMetadataRequest so that client can reject outdated
>> > > metadata
>> > > > >> and
>> > > > >> > > avoid
>> > > > >> > > > unnecessary OffsetOutOfRangeException. Otherwise there is
>> > > > currently
>> > > > >> > race
>> > > > >> > > > condition that can cause consumer to reset offset which
>> > > negatively
>> > > > >> > affect
>> > > > >> > > > the consumer's availability.
>> > > > >> > > >
>> > > > >> > > > Feedback and suggestions are welcome!
>> > > > >> > > >
>> > > > >> > > > Regards,
>> > > > >> > > > Dong
>> > > > >> > > >
>> > > > >> > >
>> > > > >> >
>> > > > >>
>> > > > >
>> > > > >
>> > > >
>> > >
>> >
>>
>
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Dong Lin <li...@gmail.com>.
Hey Jun,

Thanks much for your comments. Given that client needs to de-serialize the
metadata anyway, the extra overhead of checking the per-partition version
for every partition should not be a big concern. Thus it makes sense to use
leader epoch as the per-partition version instead of creating a global
metadata version. I will update the KIP to do that.

Regarding the detection of outdated metadata, I think it is possible to
ensure that client gets latest metadata by fetching from controller. Note
that this requires extra logic in the controller such that controller
updates metadata directly in memory without requiring
UpdateMetadataRequest. But I am not sure the main motivation of this at
this moment. But this makes controller more like a bottleneck in the
cluster which we probably want to avoid.

I think we can probably keep the current way of ensuring metadata
freshness. Currently client will be forced to refresh metadata if broker
returns error (e.g. NotLeaderForPartition) due to outdated metadata or if
the metadata does not contain the partition that the client needs. In the
future, as you previously suggested, we can include per-partition
leaderEpoch in the FetchRequest/ProduceRequest such that broker can return
error if the epoch is smaller than cached epoch in the broker. Given that
this adds more complexity to Kafka, I think we can probably think about
that leader when we have a specific use-case or problem to solve with
up-to-date metadata. Does this sound OK?

Thanks,
Dong



On Fri, Dec 8, 2017 at 3:53 PM, Jun Rao <ju...@confluent.io> wrote:

> Hi, Dong,
>
> Thanks for the reply. A few more points below.
>
> For dealing with how to prevent a consumer switching from a new leader to
> an old leader, you suggestion that refreshes metadata on consumer restart
> until it sees a metadata version >= the one associated with the offset
> works too, as long as we guarantee that the cached metadata versions on the
> brokers only go up.
>
> The second discussion point is on whether the metadata versioning should be
> per partition or global. For the partition level versioning, you were
> concerned about the performance. Given that metadata updates are rare, I am
> not sure if it's a big concern though. Doing a million if tests is probably
> going to take less than 1ms. Another thing is that the metadata version
> seems to need to survive controller failover. In your current approach, a
> consumer may not be able to wait on the right version of the metadata after
> the consumer restart since the metadata version may have been recycled on
> the server side due to a controller failover while the consumer is down.
> The partition level leaderEpoch survives controller failure and won't have
> this issue.
>
> Lastly, neither your proposal nor mine addresses the issue how to guarantee
> a consumer to detect that is metadata is outdated. Currently, the consumer
> is not guaranteed to fetch metadata from every broker within some bounded
> period of time. Maybe this is out of the scope of your KIP. But one idea is
> force the consumer to refresh metadata from the controller periodically.
>
> Jun
>
>
> On Thu, Dec 7, 2017 at 11:25 AM, Dong Lin <li...@gmail.com> wrote:
>
> > Hey Jun,
> >
> > Thanks much for the comments. Great point particularly regarding (3). I
> > haven't thought about this before.
> >
> > It seems that there are two possible ways where the version number can be
> > used. One solution is for client to check the version number at the time
> it
> > receives MetadataResponse. And if the version number in the
> > MetadataResponse is smaller than the version number in the client's
> cache,
> > the client will be forced to fetch metadata again.  Another solution, as
> > you have suggested, is for broker to check the version number at the time
> > it receives a request from client. The broker will reject the request if
> > the version is smaller than the version in broker's cache.
> >
> > I am not very sure that the second solution can address the problem here.
> > In the scenario described in the JIRA ticket, broker's cache may be
> > outdated because it has not processed the LeaderAndIsrRequest from the
> > controller. Thus it may still process client's request even if the
> version
> > in client's request is actually outdated. Does this make sense?
> >
> > IMO, it seems that we can address problem (3) by saving the metadata
> > version together with the offset. After consumer starts, it will keep
> > fetching metadata until the metadata version >= the version saved with
> the
> > offset of this partition.
> >
> > Regarding problems (1) and (2): Currently we use the version number in
> the
> > MetadataResponse to ensure that the metadata does not go back in time.
> > There are two alternative solutions to address problems (1) and (2). One
> > solution is for client to enumerate all partitions in the
> MetadataResponse,
> > compare their epoch with those in the cached metadata, and rejects the
> > MetadataResponse iff any leader epoch is smaller. The main concern is
> that
> > MetadataResponse currently cached information of all partitions in the
> > entire cluster. It may slow down client's performance if we were to do
> it.
> > The other solution is for client to enumerate partitions for only topics
> > registered in the org.apache.kafka.clients.Metadata, which will be an
> > empty
> > set for producer and the set of subscribed partitions for consumer. But
> > this degrades to all topics if consumer subscribes to topics in the
> cluster
> > by pattern.
> >
> > Note that client will only be forced to update metadata if the version in
> > the MetadataResponse is smaller than the version in the cached metadata.
> In
> > general it should not be a problem. It can be a problem only if some
> broker
> > is particularly slower than other brokers in processing
> > UpdateMetadataRequest. When this is the case, it means that the broker is
> > also particularly slower in processing LeaderAndIsrRequest, which can
> cause
> > problem anyway because some partition will probably have no leader during
> > this period. I am not sure problems (1) and (2) cause more problem than
> > what we already have.
> >
> > Thanks,
> > Dong
> >
> >
> > On Wed, Dec 6, 2017 at 6:42 PM, Jun Rao <ju...@confluent.io> wrote:
> >
> > > Hi, Dong,
> > >
> > > Great finding on the issue. It's a real problem. A few comments about
> the
> > > KIP. (1) I am not sure about updating controller_metadata_epoch on
> every
> > > UpdateMetadataRequest. Currently, the controller can send
> > > UpdateMetadataRequest when there is no actual metadata change. Doing
> this
> > > may require unnecessary metadata refresh on the client. (2)
> > > controller_metadata_epoch is global across all topics. This means that
> a
> > > client may be forced to update its metadata even when the metadata for
> > the
> > > topics that it cares haven't changed. (3) It doesn't seem that the KIP
> > > handles the corner case when a consumer is restarted. Say a consumer
> > reads
> > > from the new leader, commits the offset and then is restarted. On
> > restart,
> > > the consumer gets an outdated metadata and fetches from the old leader.
> > > Then, the consumer will get into the offset out of range issue.
> > >
> > > Given the above, I am thinking of the following approach. We actually
> > > already have metadata versioning at the partition level. Each leader
> has
> > a
> > > leader epoch which is monotonically increasing. We can potentially
> > > propagate leader epoch back in the metadata response and the clients
> can
> > > cache that. This solves the issue of (1) and (2). To solve (3), when
> > saving
> > > an offset, we could save both an offset and the corresponding leader
> > epoch.
> > > When fetching the data, the consumer provides both the offset and the
> > > leader epoch. A leader will only serve the request if its leader epoch
> is
> > > equal to or greater than the leader epoch from the consumer. To achieve
> > > this, we need to change the fetch request protocol and the offset
> commit
> > > api, which requires some more thoughts.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > >
> > > On Wed, Dec 6, 2017 at 10:57 AM, Dong Lin <li...@gmail.com> wrote:
> > >
> > > > Bump up the thread.
> > > >
> > > > It will be great to have more comments on whether we should do it or
> > > > whether there is better way to address the motivation of this KIP.
> > > >
> > > > On Mon, Dec 4, 2017 at 3:09 PM, Dong Lin <li...@gmail.com>
> wrote:
> > > >
> > > > > I don't have an interesting rejected alternative solution to put in
> > the
> > > > > KIP. If there is good alternative solution from anyone in this
> > thread,
> > > I
> > > > am
> > > > > happy to discuss this and update the KIP accordingly.
> > > > >
> > > > > Thanks,
> > > > > Dong
> > > > >
> > > > > On Mon, Dec 4, 2017 at 1:12 PM, Ted Yu <yu...@gmail.com>
> wrote:
> > > > >
> > > > >> It is clearer now.
> > > > >>
> > > > >> I noticed that Rejected Alternatives section is empty.
> > > > >> Have you considered any alternative ?
> > > > >>
> > > > >> Cheers
> > > > >>
> > > > >> On Mon, Dec 4, 2017 at 1:07 PM, Dong Lin <li...@gmail.com>
> > wrote:
> > > > >>
> > > > >> > Ted, thanks for catching this. I have updated the sentence to
> make
> > > it
> > > > >> > readable.
> > > > >> >
> > > > >> > Thanks,
> > > > >> > Dong
> > > > >> >
> > > > >> > On Sat, Dec 2, 2017 at 3:05 PM, Ted Yu <yu...@gmail.com>
> > wrote:
> > > > >> >
> > > > >> > > bq. It the controller_epoch of the incoming MetadataResponse,
> or
> > > if
> > > > >> the
> > > > >> > > controller_epoch is the same but the controller_metadata_epoch
> > > > >> > >
> > > > >> > > Can you update the above sentence so that the intention is
> > > clearer ?
> > > > >> > >
> > > > >> > > Thanks
> > > > >> > >
> > > > >> > > On Fri, Dec 1, 2017 at 6:33 PM, Dong Lin <lindong28@gmail.com
> >
> > > > wrote:
> > > > >> > >
> > > > >> > > > Hi all,
> > > > >> > > >
> > > > >> > > > I have created KIP-232: Detect outdated metadata by adding
> > > > >> > > > ControllerMetadataEpoch field:
> > > > >> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > >> > > > 232%3A+Detect+outdated+metadata+by+adding+
> > > > >> > ControllerMetadataEpoch+field
> > > > >> > > > .
> > > > >> > > >
> > > > >> > > > The KIP proposes to add fields in MetadataResponse and
> > > > >> > > > UpdateMetadataRequest so that client can reject outdated
> > > metadata
> > > > >> and
> > > > >> > > avoid
> > > > >> > > > unnecessary OffsetOutOfRangeException. Otherwise there is
> > > > currently
> > > > >> > race
> > > > >> > > > condition that can cause consumer to reset offset which
> > > negatively
> > > > >> > affect
> > > > >> > > > the consumer's availability.
> > > > >> > > >
> > > > >> > > > Feedback and suggestions are welcome!
> > > > >> > > >
> > > > >> > > > Regards,
> > > > >> > > > Dong
> > > > >> > > >
> > > > >> > >
> > > > >> >
> > > > >>
> > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Jun Rao <ju...@confluent.io>.
Hi, Dong,

Thanks for the reply. A few more points below.

For dealing with how to prevent a consumer switching from a new leader to
an old leader, you suggestion that refreshes metadata on consumer restart
until it sees a metadata version >= the one associated with the offset
works too, as long as we guarantee that the cached metadata versions on the
brokers only go up.

The second discussion point is on whether the metadata versioning should be
per partition or global. For the partition level versioning, you were
concerned about the performance. Given that metadata updates are rare, I am
not sure if it's a big concern though. Doing a million if tests is probably
going to take less than 1ms. Another thing is that the metadata version
seems to need to survive controller failover. In your current approach, a
consumer may not be able to wait on the right version of the metadata after
the consumer restart since the metadata version may have been recycled on
the server side due to a controller failover while the consumer is down.
The partition level leaderEpoch survives controller failure and won't have
this issue.

Lastly, neither your proposal nor mine addresses the issue how to guarantee
a consumer to detect that is metadata is outdated. Currently, the consumer
is not guaranteed to fetch metadata from every broker within some bounded
period of time. Maybe this is out of the scope of your KIP. But one idea is
force the consumer to refresh metadata from the controller periodically.

Jun


On Thu, Dec 7, 2017 at 11:25 AM, Dong Lin <li...@gmail.com> wrote:

> Hey Jun,
>
> Thanks much for the comments. Great point particularly regarding (3). I
> haven't thought about this before.
>
> It seems that there are two possible ways where the version number can be
> used. One solution is for client to check the version number at the time it
> receives MetadataResponse. And if the version number in the
> MetadataResponse is smaller than the version number in the client's cache,
> the client will be forced to fetch metadata again.  Another solution, as
> you have suggested, is for broker to check the version number at the time
> it receives a request from client. The broker will reject the request if
> the version is smaller than the version in broker's cache.
>
> I am not very sure that the second solution can address the problem here.
> In the scenario described in the JIRA ticket, broker's cache may be
> outdated because it has not processed the LeaderAndIsrRequest from the
> controller. Thus it may still process client's request even if the version
> in client's request is actually outdated. Does this make sense?
>
> IMO, it seems that we can address problem (3) by saving the metadata
> version together with the offset. After consumer starts, it will keep
> fetching metadata until the metadata version >= the version saved with the
> offset of this partition.
>
> Regarding problems (1) and (2): Currently we use the version number in the
> MetadataResponse to ensure that the metadata does not go back in time.
> There are two alternative solutions to address problems (1) and (2). One
> solution is for client to enumerate all partitions in the MetadataResponse,
> compare their epoch with those in the cached metadata, and rejects the
> MetadataResponse iff any leader epoch is smaller. The main concern is that
> MetadataResponse currently cached information of all partitions in the
> entire cluster. It may slow down client's performance if we were to do it.
> The other solution is for client to enumerate partitions for only topics
> registered in the org.apache.kafka.clients.Metadata, which will be an
> empty
> set for producer and the set of subscribed partitions for consumer. But
> this degrades to all topics if consumer subscribes to topics in the cluster
> by pattern.
>
> Note that client will only be forced to update metadata if the version in
> the MetadataResponse is smaller than the version in the cached metadata. In
> general it should not be a problem. It can be a problem only if some broker
> is particularly slower than other brokers in processing
> UpdateMetadataRequest. When this is the case, it means that the broker is
> also particularly slower in processing LeaderAndIsrRequest, which can cause
> problem anyway because some partition will probably have no leader during
> this period. I am not sure problems (1) and (2) cause more problem than
> what we already have.
>
> Thanks,
> Dong
>
>
> On Wed, Dec 6, 2017 at 6:42 PM, Jun Rao <ju...@confluent.io> wrote:
>
> > Hi, Dong,
> >
> > Great finding on the issue. It's a real problem. A few comments about the
> > KIP. (1) I am not sure about updating controller_metadata_epoch on every
> > UpdateMetadataRequest. Currently, the controller can send
> > UpdateMetadataRequest when there is no actual metadata change. Doing this
> > may require unnecessary metadata refresh on the client. (2)
> > controller_metadata_epoch is global across all topics. This means that a
> > client may be forced to update its metadata even when the metadata for
> the
> > topics that it cares haven't changed. (3) It doesn't seem that the KIP
> > handles the corner case when a consumer is restarted. Say a consumer
> reads
> > from the new leader, commits the offset and then is restarted. On
> restart,
> > the consumer gets an outdated metadata and fetches from the old leader.
> > Then, the consumer will get into the offset out of range issue.
> >
> > Given the above, I am thinking of the following approach. We actually
> > already have metadata versioning at the partition level. Each leader has
> a
> > leader epoch which is monotonically increasing. We can potentially
> > propagate leader epoch back in the metadata response and the clients can
> > cache that. This solves the issue of (1) and (2). To solve (3), when
> saving
> > an offset, we could save both an offset and the corresponding leader
> epoch.
> > When fetching the data, the consumer provides both the offset and the
> > leader epoch. A leader will only serve the request if its leader epoch is
> > equal to or greater than the leader epoch from the consumer. To achieve
> > this, we need to change the fetch request protocol and the offset commit
> > api, which requires some more thoughts.
> >
> > Thanks,
> >
> > Jun
> >
> >
> > On Wed, Dec 6, 2017 at 10:57 AM, Dong Lin <li...@gmail.com> wrote:
> >
> > > Bump up the thread.
> > >
> > > It will be great to have more comments on whether we should do it or
> > > whether there is better way to address the motivation of this KIP.
> > >
> > > On Mon, Dec 4, 2017 at 3:09 PM, Dong Lin <li...@gmail.com> wrote:
> > >
> > > > I don't have an interesting rejected alternative solution to put in
> the
> > > > KIP. If there is good alternative solution from anyone in this
> thread,
> > I
> > > am
> > > > happy to discuss this and update the KIP accordingly.
> > > >
> > > > Thanks,
> > > > Dong
> > > >
> > > > On Mon, Dec 4, 2017 at 1:12 PM, Ted Yu <yu...@gmail.com> wrote:
> > > >
> > > >> It is clearer now.
> > > >>
> > > >> I noticed that Rejected Alternatives section is empty.
> > > >> Have you considered any alternative ?
> > > >>
> > > >> Cheers
> > > >>
> > > >> On Mon, Dec 4, 2017 at 1:07 PM, Dong Lin <li...@gmail.com>
> wrote:
> > > >>
> > > >> > Ted, thanks for catching this. I have updated the sentence to make
> > it
> > > >> > readable.
> > > >> >
> > > >> > Thanks,
> > > >> > Dong
> > > >> >
> > > >> > On Sat, Dec 2, 2017 at 3:05 PM, Ted Yu <yu...@gmail.com>
> wrote:
> > > >> >
> > > >> > > bq. It the controller_epoch of the incoming MetadataResponse, or
> > if
> > > >> the
> > > >> > > controller_epoch is the same but the controller_metadata_epoch
> > > >> > >
> > > >> > > Can you update the above sentence so that the intention is
> > clearer ?
> > > >> > >
> > > >> > > Thanks
> > > >> > >
> > > >> > > On Fri, Dec 1, 2017 at 6:33 PM, Dong Lin <li...@gmail.com>
> > > wrote:
> > > >> > >
> > > >> > > > Hi all,
> > > >> > > >
> > > >> > > > I have created KIP-232: Detect outdated metadata by adding
> > > >> > > > ControllerMetadataEpoch field:
> > > >> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > >> > > > 232%3A+Detect+outdated+metadata+by+adding+
> > > >> > ControllerMetadataEpoch+field
> > > >> > > > .
> > > >> > > >
> > > >> > > > The KIP proposes to add fields in MetadataResponse and
> > > >> > > > UpdateMetadataRequest so that client can reject outdated
> > metadata
> > > >> and
> > > >> > > avoid
> > > >> > > > unnecessary OffsetOutOfRangeException. Otherwise there is
> > > currently
> > > >> > race
> > > >> > > > condition that can cause consumer to reset offset which
> > negatively
> > > >> > affect
> > > >> > > > the consumer's availability.
> > > >> > > >
> > > >> > > > Feedback and suggestions are welcome!
> > > >> > > >
> > > >> > > > Regards,
> > > >> > > > Dong
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Dong Lin <li...@gmail.com>.
Hey Jun,

Thanks much for the comments. Great point particularly regarding (3). I
haven't thought about this before.

It seems that there are two possible ways where the version number can be
used. One solution is for client to check the version number at the time it
receives MetadataResponse. And if the version number in the
MetadataResponse is smaller than the version number in the client's cache,
the client will be forced to fetch metadata again.  Another solution, as
you have suggested, is for broker to check the version number at the time
it receives a request from client. The broker will reject the request if
the version is smaller than the version in broker's cache.

I am not very sure that the second solution can address the problem here.
In the scenario described in the JIRA ticket, broker's cache may be
outdated because it has not processed the LeaderAndIsrRequest from the
controller. Thus it may still process client's request even if the version
in client's request is actually outdated. Does this make sense?

IMO, it seems that we can address problem (3) by saving the metadata
version together with the offset. After consumer starts, it will keep
fetching metadata until the metadata version >= the version saved with the
offset of this partition.

Regarding problems (1) and (2): Currently we use the version number in the
MetadataResponse to ensure that the metadata does not go back in time.
There are two alternative solutions to address problems (1) and (2). One
solution is for client to enumerate all partitions in the MetadataResponse,
compare their epoch with those in the cached metadata, and rejects the
MetadataResponse iff any leader epoch is smaller. The main concern is that
MetadataResponse currently cached information of all partitions in the
entire cluster. It may slow down client's performance if we were to do it.
The other solution is for client to enumerate partitions for only topics
registered in the org.apache.kafka.clients.Metadata, which will be an empty
set for producer and the set of subscribed partitions for consumer. But
this degrades to all topics if consumer subscribes to topics in the cluster
by pattern.

Note that client will only be forced to update metadata if the version in
the MetadataResponse is smaller than the version in the cached metadata. In
general it should not be a problem. It can be a problem only if some broker
is particularly slower than other brokers in processing
UpdateMetadataRequest. When this is the case, it means that the broker is
also particularly slower in processing LeaderAndIsrRequest, which can cause
problem anyway because some partition will probably have no leader during
this period. I am not sure problems (1) and (2) cause more problem than
what we already have.

Thanks,
Dong


On Wed, Dec 6, 2017 at 6:42 PM, Jun Rao <ju...@confluent.io> wrote:

> Hi, Dong,
>
> Great finding on the issue. It's a real problem. A few comments about the
> KIP. (1) I am not sure about updating controller_metadata_epoch on every
> UpdateMetadataRequest. Currently, the controller can send
> UpdateMetadataRequest when there is no actual metadata change. Doing this
> may require unnecessary metadata refresh on the client. (2)
> controller_metadata_epoch is global across all topics. This means that a
> client may be forced to update its metadata even when the metadata for the
> topics that it cares haven't changed. (3) It doesn't seem that the KIP
> handles the corner case when a consumer is restarted. Say a consumer reads
> from the new leader, commits the offset and then is restarted. On restart,
> the consumer gets an outdated metadata and fetches from the old leader.
> Then, the consumer will get into the offset out of range issue.
>
> Given the above, I am thinking of the following approach. We actually
> already have metadata versioning at the partition level. Each leader has a
> leader epoch which is monotonically increasing. We can potentially
> propagate leader epoch back in the metadata response and the clients can
> cache that. This solves the issue of (1) and (2). To solve (3), when saving
> an offset, we could save both an offset and the corresponding leader epoch.
> When fetching the data, the consumer provides both the offset and the
> leader epoch. A leader will only serve the request if its leader epoch is
> equal to or greater than the leader epoch from the consumer. To achieve
> this, we need to change the fetch request protocol and the offset commit
> api, which requires some more thoughts.
>
> Thanks,
>
> Jun
>
>
> On Wed, Dec 6, 2017 at 10:57 AM, Dong Lin <li...@gmail.com> wrote:
>
> > Bump up the thread.
> >
> > It will be great to have more comments on whether we should do it or
> > whether there is better way to address the motivation of this KIP.
> >
> > On Mon, Dec 4, 2017 at 3:09 PM, Dong Lin <li...@gmail.com> wrote:
> >
> > > I don't have an interesting rejected alternative solution to put in the
> > > KIP. If there is good alternative solution from anyone in this thread,
> I
> > am
> > > happy to discuss this and update the KIP accordingly.
> > >
> > > Thanks,
> > > Dong
> > >
> > > On Mon, Dec 4, 2017 at 1:12 PM, Ted Yu <yu...@gmail.com> wrote:
> > >
> > >> It is clearer now.
> > >>
> > >> I noticed that Rejected Alternatives section is empty.
> > >> Have you considered any alternative ?
> > >>
> > >> Cheers
> > >>
> > >> On Mon, Dec 4, 2017 at 1:07 PM, Dong Lin <li...@gmail.com> wrote:
> > >>
> > >> > Ted, thanks for catching this. I have updated the sentence to make
> it
> > >> > readable.
> > >> >
> > >> > Thanks,
> > >> > Dong
> > >> >
> > >> > On Sat, Dec 2, 2017 at 3:05 PM, Ted Yu <yu...@gmail.com> wrote:
> > >> >
> > >> > > bq. It the controller_epoch of the incoming MetadataResponse, or
> if
> > >> the
> > >> > > controller_epoch is the same but the controller_metadata_epoch
> > >> > >
> > >> > > Can you update the above sentence so that the intention is
> clearer ?
> > >> > >
> > >> > > Thanks
> > >> > >
> > >> > > On Fri, Dec 1, 2017 at 6:33 PM, Dong Lin <li...@gmail.com>
> > wrote:
> > >> > >
> > >> > > > Hi all,
> > >> > > >
> > >> > > > I have created KIP-232: Detect outdated metadata by adding
> > >> > > > ControllerMetadataEpoch field:
> > >> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > >> > > > 232%3A+Detect+outdated+metadata+by+adding+
> > >> > ControllerMetadataEpoch+field
> > >> > > > .
> > >> > > >
> > >> > > > The KIP proposes to add fields in MetadataResponse and
> > >> > > > UpdateMetadataRequest so that client can reject outdated
> metadata
> > >> and
> > >> > > avoid
> > >> > > > unnecessary OffsetOutOfRangeException. Otherwise there is
> > currently
> > >> > race
> > >> > > > condition that can cause consumer to reset offset which
> negatively
> > >> > affect
> > >> > > > the consumer's availability.
> > >> > > >
> > >> > > > Feedback and suggestions are welcome!
> > >> > > >
> > >> > > > Regards,
> > >> > > > Dong
> > >> > > >
> > >> > >
> > >> >
> > >>
> > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Jun Rao <ju...@confluent.io>.
Hi, Dong,

Great finding on the issue. It's a real problem. A few comments about the
KIP. (1) I am not sure about updating controller_metadata_epoch on every
UpdateMetadataRequest. Currently, the controller can send
UpdateMetadataRequest when there is no actual metadata change. Doing this
may require unnecessary metadata refresh on the client. (2)
controller_metadata_epoch is global across all topics. This means that a
client may be forced to update its metadata even when the metadata for the
topics that it cares haven't changed. (3) It doesn't seem that the KIP
handles the corner case when a consumer is restarted. Say a consumer reads
from the new leader, commits the offset and then is restarted. On restart,
the consumer gets an outdated metadata and fetches from the old leader.
Then, the consumer will get into the offset out of range issue.

Given the above, I am thinking of the following approach. We actually
already have metadata versioning at the partition level. Each leader has a
leader epoch which is monotonically increasing. We can potentially
propagate leader epoch back in the metadata response and the clients can
cache that. This solves the issue of (1) and (2). To solve (3), when saving
an offset, we could save both an offset and the corresponding leader epoch.
When fetching the data, the consumer provides both the offset and the
leader epoch. A leader will only serve the request if its leader epoch is
equal to or greater than the leader epoch from the consumer. To achieve
this, we need to change the fetch request protocol and the offset commit
api, which requires some more thoughts.

Thanks,

Jun


On Wed, Dec 6, 2017 at 10:57 AM, Dong Lin <li...@gmail.com> wrote:

> Bump up the thread.
>
> It will be great to have more comments on whether we should do it or
> whether there is better way to address the motivation of this KIP.
>
> On Mon, Dec 4, 2017 at 3:09 PM, Dong Lin <li...@gmail.com> wrote:
>
> > I don't have an interesting rejected alternative solution to put in the
> > KIP. If there is good alternative solution from anyone in this thread, I
> am
> > happy to discuss this and update the KIP accordingly.
> >
> > Thanks,
> > Dong
> >
> > On Mon, Dec 4, 2017 at 1:12 PM, Ted Yu <yu...@gmail.com> wrote:
> >
> >> It is clearer now.
> >>
> >> I noticed that Rejected Alternatives section is empty.
> >> Have you considered any alternative ?
> >>
> >> Cheers
> >>
> >> On Mon, Dec 4, 2017 at 1:07 PM, Dong Lin <li...@gmail.com> wrote:
> >>
> >> > Ted, thanks for catching this. I have updated the sentence to make it
> >> > readable.
> >> >
> >> > Thanks,
> >> > Dong
> >> >
> >> > On Sat, Dec 2, 2017 at 3:05 PM, Ted Yu <yu...@gmail.com> wrote:
> >> >
> >> > > bq. It the controller_epoch of the incoming MetadataResponse, or if
> >> the
> >> > > controller_epoch is the same but the controller_metadata_epoch
> >> > >
> >> > > Can you update the above sentence so that the intention is clearer ?
> >> > >
> >> > > Thanks
> >> > >
> >> > > On Fri, Dec 1, 2017 at 6:33 PM, Dong Lin <li...@gmail.com>
> wrote:
> >> > >
> >> > > > Hi all,
> >> > > >
> >> > > > I have created KIP-232: Detect outdated metadata by adding
> >> > > > ControllerMetadataEpoch field:
> >> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >> > > > 232%3A+Detect+outdated+metadata+by+adding+
> >> > ControllerMetadataEpoch+field
> >> > > > .
> >> > > >
> >> > > > The KIP proposes to add fields in MetadataResponse and
> >> > > > UpdateMetadataRequest so that client can reject outdated metadata
> >> and
> >> > > avoid
> >> > > > unnecessary OffsetOutOfRangeException. Otherwise there is
> currently
> >> > race
> >> > > > condition that can cause consumer to reset offset which negatively
> >> > affect
> >> > > > the consumer's availability.
> >> > > >
> >> > > > Feedback and suggestions are welcome!
> >> > > >
> >> > > > Regards,
> >> > > > Dong
> >> > > >
> >> > >
> >> >
> >>
> >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Dong Lin <li...@gmail.com>.
Bump up the thread.

It will be great to have more comments on whether we should do it or
whether there is better way to address the motivation of this KIP.

On Mon, Dec 4, 2017 at 3:09 PM, Dong Lin <li...@gmail.com> wrote:

> I don't have an interesting rejected alternative solution to put in the
> KIP. If there is good alternative solution from anyone in this thread, I am
> happy to discuss this and update the KIP accordingly.
>
> Thanks,
> Dong
>
> On Mon, Dec 4, 2017 at 1:12 PM, Ted Yu <yu...@gmail.com> wrote:
>
>> It is clearer now.
>>
>> I noticed that Rejected Alternatives section is empty.
>> Have you considered any alternative ?
>>
>> Cheers
>>
>> On Mon, Dec 4, 2017 at 1:07 PM, Dong Lin <li...@gmail.com> wrote:
>>
>> > Ted, thanks for catching this. I have updated the sentence to make it
>> > readable.
>> >
>> > Thanks,
>> > Dong
>> >
>> > On Sat, Dec 2, 2017 at 3:05 PM, Ted Yu <yu...@gmail.com> wrote:
>> >
>> > > bq. It the controller_epoch of the incoming MetadataResponse, or if
>> the
>> > > controller_epoch is the same but the controller_metadata_epoch
>> > >
>> > > Can you update the above sentence so that the intention is clearer ?
>> > >
>> > > Thanks
>> > >
>> > > On Fri, Dec 1, 2017 at 6:33 PM, Dong Lin <li...@gmail.com> wrote:
>> > >
>> > > > Hi all,
>> > > >
>> > > > I have created KIP-232: Detect outdated metadata by adding
>> > > > ControllerMetadataEpoch field:
>> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> > > > 232%3A+Detect+outdated+metadata+by+adding+
>> > ControllerMetadataEpoch+field
>> > > > .
>> > > >
>> > > > The KIP proposes to add fields in MetadataResponse and
>> > > > UpdateMetadataRequest so that client can reject outdated metadata
>> and
>> > > avoid
>> > > > unnecessary OffsetOutOfRangeException. Otherwise there is currently
>> > race
>> > > > condition that can cause consumer to reset offset which negatively
>> > affect
>> > > > the consumer's availability.
>> > > >
>> > > > Feedback and suggestions are welcome!
>> > > >
>> > > > Regards,
>> > > > Dong
>> > > >
>> > >
>> >
>>
>
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Dong Lin <li...@gmail.com>.
I don't have an interesting rejected alternative solution to put in the
KIP. If there is good alternative solution from anyone in this thread, I am
happy to discuss this and update the KIP accordingly.

Thanks,
Dong

On Mon, Dec 4, 2017 at 1:12 PM, Ted Yu <yu...@gmail.com> wrote:

> It is clearer now.
>
> I noticed that Rejected Alternatives section is empty.
> Have you considered any alternative ?
>
> Cheers
>
> On Mon, Dec 4, 2017 at 1:07 PM, Dong Lin <li...@gmail.com> wrote:
>
> > Ted, thanks for catching this. I have updated the sentence to make it
> > readable.
> >
> > Thanks,
> > Dong
> >
> > On Sat, Dec 2, 2017 at 3:05 PM, Ted Yu <yu...@gmail.com> wrote:
> >
> > > bq. It the controller_epoch of the incoming MetadataResponse, or if the
> > > controller_epoch is the same but the controller_metadata_epoch
> > >
> > > Can you update the above sentence so that the intention is clearer ?
> > >
> > > Thanks
> > >
> > > On Fri, Dec 1, 2017 at 6:33 PM, Dong Lin <li...@gmail.com> wrote:
> > >
> > > > Hi all,
> > > >
> > > > I have created KIP-232: Detect outdated metadata by adding
> > > > ControllerMetadataEpoch field:
> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > 232%3A+Detect+outdated+metadata+by+adding+
> > ControllerMetadataEpoch+field
> > > > .
> > > >
> > > > The KIP proposes to add fields in MetadataResponse and
> > > > UpdateMetadataRequest so that client can reject outdated metadata and
> > > avoid
> > > > unnecessary OffsetOutOfRangeException. Otherwise there is currently
> > race
> > > > condition that can cause consumer to reset offset which negatively
> > affect
> > > > the consumer's availability.
> > > >
> > > > Feedback and suggestions are welcome!
> > > >
> > > > Regards,
> > > > Dong
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Ted Yu <yu...@gmail.com>.
It is clearer now.

I noticed that Rejected Alternatives section is empty.
Have you considered any alternative ?

Cheers

On Mon, Dec 4, 2017 at 1:07 PM, Dong Lin <li...@gmail.com> wrote:

> Ted, thanks for catching this. I have updated the sentence to make it
> readable.
>
> Thanks,
> Dong
>
> On Sat, Dec 2, 2017 at 3:05 PM, Ted Yu <yu...@gmail.com> wrote:
>
> > bq. It the controller_epoch of the incoming MetadataResponse, or if the
> > controller_epoch is the same but the controller_metadata_epoch
> >
> > Can you update the above sentence so that the intention is clearer ?
> >
> > Thanks
> >
> > On Fri, Dec 1, 2017 at 6:33 PM, Dong Lin <li...@gmail.com> wrote:
> >
> > > Hi all,
> > >
> > > I have created KIP-232: Detect outdated metadata by adding
> > > ControllerMetadataEpoch field:
> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > 232%3A+Detect+outdated+metadata+by+adding+
> ControllerMetadataEpoch+field
> > > .
> > >
> > > The KIP proposes to add fields in MetadataResponse and
> > > UpdateMetadataRequest so that client can reject outdated metadata and
> > avoid
> > > unnecessary OffsetOutOfRangeException. Otherwise there is currently
> race
> > > condition that can cause consumer to reset offset which negatively
> affect
> > > the consumer's availability.
> > >
> > > Feedback and suggestions are welcome!
> > >
> > > Regards,
> > > Dong
> > >
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Dong Lin <li...@gmail.com>.
Ted, thanks for catching this. I have updated the sentence to make it
readable.

Thanks,
Dong

On Sat, Dec 2, 2017 at 3:05 PM, Ted Yu <yu...@gmail.com> wrote:

> bq. It the controller_epoch of the incoming MetadataResponse, or if the
> controller_epoch is the same but the controller_metadata_epoch
>
> Can you update the above sentence so that the intention is clearer ?
>
> Thanks
>
> On Fri, Dec 1, 2017 at 6:33 PM, Dong Lin <li...@gmail.com> wrote:
>
> > Hi all,
> >
> > I have created KIP-232: Detect outdated metadata by adding
> > ControllerMetadataEpoch field:
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > 232%3A+Detect+outdated+metadata+by+adding+ControllerMetadataEpoch+field
> > .
> >
> > The KIP proposes to add fields in MetadataResponse and
> > UpdateMetadataRequest so that client can reject outdated metadata and
> avoid
> > unnecessary OffsetOutOfRangeException. Otherwise there is currently race
> > condition that can cause consumer to reset offset which negatively affect
> > the consumer's availability.
> >
> > Feedback and suggestions are welcome!
> >
> > Regards,
> > Dong
> >
>

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

Posted by Ted Yu <yu...@gmail.com>.
bq. It the controller_epoch of the incoming MetadataResponse, or if the
controller_epoch is the same but the controller_metadata_epoch

Can you update the above sentence so that the intention is clearer ?

Thanks

On Fri, Dec 1, 2017 at 6:33 PM, Dong Lin <li...@gmail.com> wrote:

> Hi all,
>
> I have created KIP-232: Detect outdated metadata by adding
> ControllerMetadataEpoch field:
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 232%3A+Detect+outdated+metadata+by+adding+ControllerMetadataEpoch+field
> .
>
> The KIP proposes to add fields in MetadataResponse and
> UpdateMetadataRequest so that client can reject outdated metadata and avoid
> unnecessary OffsetOutOfRangeException. Otherwise there is currently race
> condition that can cause consumer to reset offset which negatively affect
> the consumer's availability.
>
> Feedback and suggestions are welcome!
>
> Regards,
> Dong
>