Hi Jun,

Thanks for the comments. I apologize for some of the typos and confusion.
I’ve updated the KIP to fix some of the issues you mentioned.

20.2 I’ve changed the type to String.
20.1/3 I’ve updated the TopicZNode to fix formatting and reflect the latest
version before this change.

21. You are correct and I’ve removed this line. I’ve also added a line
mentioning an IBP bump is necessary for migration

22. I think the wording was unclear but your summary is what was intended
by this line. I’ve updated to make this point more clear. “Remove deleted
topics from replicas by sending StopReplicaRequest V3 before the IBP bump
using the old logic, and using V4 and the new logic with topic IDs after
the IBP bump.”

23. I’ve removed the unspecified type from the KIP and mention that IBP
will be used to handle this request. “IBP will be used to determine whether
this new form of the request will be used. For older requests, we will
ignore this field and default to previous behavior.”

24. I’ve fixed this typo.

25. I've added a topics at a higher level for LeaderAndIsrResponse v5,
StopReplicaResponse v4. I've also changed StopReplicaRequest v4 to have
topics at a higher level.

26. I’ve updated forgotten_topics_data--added the topic ID and removed the
topic name

27. I’ve decided on plain text, and I’ve added an example.

28. I’ve added this idea to future work.

Thanks again for taking a look,

Justine

On Wed, Sep 23, 2020 at 10:28 AM Jun Rao <j...@confluent.io> wrote:

> Hi, Justine,
>
> Thanks for the response. Made another pass. A few more comments below.
>
> 20. znode schema:
> 20.1 It seems that {"name": "version", "type": "int", "id": "UUID", "doc":
> "version id"} should be {"name": "version", "type": "int"}, {"name": "id",
> "type": "UUID", "doc": "version id"}.
> 20.2 The znode format is JSON which doesn't have UUID type. So the type
> probably should be string?
> 20.3 Also, the existing format used seems outdated. It should have the
> following format.
>     Json.encodeAsBytes(Map(
>       "version" -> 2,
>       "partitions" -> replicaAssignmentJson.asJava,
>       "adding_replicas" -> addingReplicasAssignmentJson.asJava,
>       "removing_replicas" -> removingReplicasAssignmentJson.asJava
>     ).asJava)
>   }
>
> 21. Migration: The KIP says "The migration process can take place without
> an inter-broker protocol bump, as the format stored in
> /brokers/topics/[topic] will be compatible with older broker versions."
> However, since we are bumping up the version of inter-broker requests, it
> seems that we need to use IBP for migration.
>
> 22. The KIP says "Remove deleted topics from replicas by sending
> StopReplicaRequest V3 for any topics which do not contain a topic ID, and
> V4 for any topics which do contain a topic ID.". However, if we use IBP, it
> seems that the controller will either send StopReplicaRequest V3
> or StopReplicaRequest V4, but never mixed V3 and V4 for different topics.
> Basically, before the IBP bump, V3 will be used. After the IBP bump,
> topicId will be created and V4 will be used.
>
> 23. Given that we depend on IBP, do we still need "0 UNSPECIFIED"
> in LeaderAndIsr?
>
> 24. LeaderAndIsrResponse v5 : It still has the topic field.
>
> 25. LeaderAndIsrResponse v5, StopReplicaResponse v4: Could we use this
> opportunity to organize the response in 2 levels, first by topic, then by
> partition, as most other requests/responses?
>
> 26. FetchRequest v13 : Should forgotten_topics_data use topicId too?
>
> 27. "This file can either be plain text (key/value pairs) or JSON." Have we
> decided which one to use? Also, it would be helpful to provide an example.
>
> 28. Future improvement: Another future improvement opportunity is to use
> topicId in GroupMetadataManager.offsetCommitKey in the offset_commit topic.
> This may save some space.
>
> Thanks,
>
> Jun
>
> On Wed, Sep 23, 2020 at 8:50 AM Justine Olshan <jols...@confluent.io>
> wrote:
>
> > Hi Tom,
> >
> > Thanks for the comment. I think this is a really good idea and it has
> been
> > added to the KIP under the newly added tooling section.
> >
> > Thanks again,
> > Justine
> >
> > On Wed, Sep 23, 2020 at 3:17 AM Tom Bentley <tbent...@redhat.com> wrote:
> >
> > > Hi Justine,
> > >
> > > I know you started the vote thread, but on re-reading the KIP I noticed
> > > that although the topic id is included in the MetadataResponse it's not
> > > surfaced in the output from `kafka-topics.sh --describe`. Maybe that
> was
> > > intentional because ids are intentionally not really something the user
> > > should care deeply about, but it would also make life harder for anyone
> > > debugging Kafka and this would likely get worse the more topic ids got
> > > rolled out across the protocols, clients etc. It seems likely that
> > > `kafka-topics.sh` will eventually need the ability to show the id of a
> > > topic and perhaps find a topic name given an id. Is there any reason
> not
> > to
> > > implement that in this KIP?
> > >
> > > Many thanks,
> > >
> > > Tom
> > >
> > > On Mon, Sep 21, 2020 at 9:54 PM Justine Olshan <jols...@confluent.io>
> > > wrote:
> > >
> > > > Hi all,
> > > >
> > > > After thinking about it, I've decided to remove the topic name from
> the
> > > > Fetch Request and Response after all. Since there are so many of
> these
> > > > requests per second, it is worth removing the extra information. I've
> > > > updated the KIP to reflect this change.
> > > >
> > > > Please let me know if there is anything else we should discuss before
> > > > voting.
> > > >
> > > > Thank you,
> > > > Justine
> > > >
> > > > On Fri, Sep 18, 2020 at 9:46 AM Justine Olshan <jols...@confluent.io
> >
> > > > wrote:
> > > >
> > > > > Hi Jun,
> > > > >
> > > > > I see what you are saying. For now we can remove the extra
> > information.
> > > > > I'll leave the option to add more fields to the file in the future.
> > The
> > > > KIP
> > > > > has been updated to reflect this change.
> > > > >
> > > > > Thanks,
> > > > > Justine
> > > > >
> > > > > On Fri, Sep 18, 2020 at 8:46 AM Jun Rao <j...@confluent.io> wrote:
> > > > >
> > > > >> Hi, Justine,
> > > > >>
> > > > >> Thanks for the reply.
> > > > >>
> > > > >> 13. If the log directory is the source of truth, it means that the
> > > > >> redundant info in the metadata file will be ignored. Then the
> > question
> > > > is
> > > > >> why do we need to put the redundant info in the metadata file now?
> > > > >>
> > > > >> Thanks,
> > > > >>
> > > > >> Jun
> > > > >>
> > > > >> On Thu, Sep 17, 2020 at 5:07 PM Justine Olshan <
> > jols...@confluent.io>
> > > > >> wrote:
> > > > >>
> > > > >> > Hi Jun,
> > > > >> > Thanks for the quick response!
> > > > >> >
> > > > >> > 12. I've decided to bump up the versions on the requests and
> > updated
> > > > the
> > > > >> > KIP. I think it's good we thoroughly discussed the options here,
> > so
> > > we
> > > > >> know
> > > > >> > we made a good choice. :)
> > > > >> >
> > > > >> > 13. This is an interesting situation. I think if this does occur
> > we
> > > > >> should
> > > > >> > give a warning. I agree that it's hard to know the source of
> truth
> > > for
> > > > >> sure
> > > > >> > since the directory or the file could be manually modified. I
> > guess
> > > > the
> > > > >> > directory could be used as the source of truth. To be honest,
> I'm
> > > not
> > > > >> > really sure what happens in kafka when the log directory is
> > renamed
> > > > >> > manually in such a way. I'm also wondering if the situation is
> > > > >> recoverable
> > > > >> > in this scenario.
> > > > >> >
> > > > >> > Thanks,
> > > > >> > Justine
> > > > >> >
> > > > >> > On Thu, Sep 17, 2020 at 4:28 PM Jun Rao <j...@confluent.io>
> wrote:
> > > > >> >
> > > > >> > > Hi, Justine,
> > > > >> > >
> > > > >> > > Thanks for the reply.
> > > > >> > >
> > > > >> > > 12. I don't have a strong preference either. However, if we
> need
> > > IBP
> > > > >> > > anyway, maybe it's easier to just bump up the version for all
> > > inter
> > > > >> > broker
> > > > >> > > requests and add the topic id field as a regular field. A
> > regular
> > > > >> field
> > > > >> > is
> > > > >> > > a bit more concise in wire transfer than a flexible field.
> > > > >> > >
> > > > >> > > 13. The confusion that I was referring to is between the topic
> > > name
> > > > >> and
> > > > >> > > partition number between the log dir and the metadata file.
> For
> > > > >> example,
> > > > >> > if
> > > > >> > > the log dir is topicA-1 and the metadata file in it has topicB
> > and
> > > > >> > > partition 0 (say due to a bug or manual modification), which
> one
> > > do
> > > > we
> > > > >> > use
> > > > >> > > as the source of truth?
> > > > >> > >
> > > > >> > > Jun
> > > > >> > >
> > > > >> > > On Thu, Sep 17, 2020 at 3:43 PM Justine Olshan <
> > > > jols...@confluent.io>
> > > > >> > > wrote:
> > > > >> > >
> > > > >> > > > Hi Jun,
> > > > >> > > > Thanks for the comments.
> > > > >> > > >
> > > > >> > > > 12. I bumped the LeaderAndIsrRequest because I removed the
> > topic
> > > > >> name
> > > > >> > > field
> > > > >> > > > in the response. It may be possible to avoid bumping the
> > version
> > > > >> > without
> > > > >> > > > that change, but I may be missing something.
> > > > >> > > > I believe StopReplica is actually on version 3 now, but
> > because
> > > > >> > version 2
> > > > >> > > > is flexible, I kept that listed as version 2 on the KIP
> page.
> > > > >> However,
> > > > >> > > you
> > > > >> > > > may be right in that we may need to bump the version on
> > > > StopReplica
> > > > >> to
> > > > >> > > deal
> > > > >> > > > with deletion differently as mentioned above. I don't know
> if
> > I
> > > > >> have a
> > > > >> > > big
> > > > >> > > > preference over used tagged fields or not.
> > > > >> > > >
> > > > >> > > > 13. I was thinking that in the case where the file and the
> > > request
> > > > >> > topic
> > > > >> > > > ids don't match, it means that the broker's topic/the one in
> > the
> > > > >> file
> > > > >> > has
> > > > >> > > > been deleted. In that case, we would need to delete the old
> > > topic
> > > > >> and
> > > > >> > > start
> > > > >> > > > receiving the new version. If the topic name were to change,
> > but
> > > > the
> > > > >> > ids
> > > > >> > > > still matched, the file would also need to update. Am I
> > missing
> > > a
> > > > >> case
> > > > >> > > > where the file would be correct and not the request?
> > > > >> > > >
> > > > >> > > > Thanks,
> > > > >> > > > Justine
> > > > >> > > >
> > > > >> > > > On Thu, Sep 17, 2020 at 3:18 PM Jun Rao <j...@confluent.io>
> > > wrote:
> > > > >> > > >
> > > > >> > > > > Hi, Justine,
> > > > >> > > > >
> > > > >> > > > > Thanks for the reply. A couple of more comments below.
> > > > >> > > > >
> > > > >> > > > > 12. ListOffset and OffsetForLeader currently don't support
> > > > >> flexible
> > > > >> > > > fields.
> > > > >> > > > > So, we have to bump up the version number and use IBP at
> > least
> > > > for
> > > > >> > > these
> > > > >> > > > > two requests. Note that it seems 2.7.0 will require IBP
> > anyway
> > > > >> > because
> > > > >> > > of
> > > > >> > > > > changes in KAFKA-10435. Also, it seems that the version
> for
> > > > >> > > > > LeaderAndIsrRequest and StopReplica are bumped even though
> > we
> > > > only
> > > > >> > > added
> > > > >> > > > a
> > > > >> > > > > tagged field. But since IBP is needed anyway, we may want
> to
> > > > >> revisit
> > > > >> > > the
> > > > >> > > > > overall tagged field choice.
> > > > >> > > > >
> > > > >> > > > > 13. The only downside is the potential confusion on which
> > one
> > > is
> > > > >> the
> > > > >> > > > source
> > > > >> > > > > of truth if they don't match. Another option is to include
> > > those
> > > > >> > fields
> > > > >> > > > in
> > > > >> > > > > the metadata file when we actually change the directory
> > > > structure.
> > > > >> > > > >
> > > > >> > > > > Thanks,
> > > > >> > > > >
> > > > >> > > > > Jun
> > > > >> > > > >
> > > > >> > > > > On Thu, Sep 17, 2020 at 2:01 PM Justine Olshan <
> > > > >> jols...@confluent.io
> > > > >> > >
> > > > >> > > > > wrote:
> > > > >> > > > >
> > > > >> > > > > > Hello all,
> > > > >> > > > > >
> > > > >> > > > > > I've thought some more about removing the topic name
> field
> > > > from
> > > > >> > some
> > > > >> > > of
> > > > >> > > > > the
> > > > >> > > > > > requests. On closer inspection of the
> requests/responses,
> > it
> > > > >> seems
> > > > >> > > that
> > > > >> > > > > the
> > > > >> > > > > > internal changes would be much larger than I expected.
> > Some
> > > > >> > protocols
> > > > >> > > > > > involve clients, so they would require changes too. I'm
> > > > thinking
> > > > >> > that
> > > > >> > > > for
> > > > >> > > > > > now, removing the topic name from these requests and
> > > responses
> > > > >> are
> > > > >> > > out
> > > > >> > > > of
> > > > >> > > > > > scope.
> > > > >> > > > > >
> > > > >> > > > > > I have decided to just keep the change
> > LeaderAndIsrResponse
> > > to
> > > > >> > remove
> > > > >> > > > the
> > > > >> > > > > > topic name, and have updated the KIP to reflect this
> > > change. I
> > > > >> have
> > > > >> > > > also
> > > > >> > > > > > mentioned the other requests and responses in future
> work.
> > > > >> > > > > >
> > > > >> > > > > > I'm hoping to start the voting process soon, so let me
> > know
> > > if
> > > > >> > there
> > > > >> > > is
> > > > >> > > > > > anything else we should discuss.
> > > > >> > > > > >
> > > > >> > > > > > Thank you,
> > > > >> > > > > > Justine
> > > > >> > > > > >
> > > > >> > > > > > On Tue, Sep 15, 2020 at 3:57 PM Justine Olshan <
> > > > >> > jols...@confluent.io
> > > > >> > > >
> > > > >> > > > > > wrote:
> > > > >> > > > > >
> > > > >> > > > > > > Hello again,
> > > > >> > > > > > > To follow up on some of the other comments:
> > > > >> > > > > > >
> > > > >> > > > > > > 10/11) We can remove the topic name from these
> > > > >> > requests/responses,
> > > > >> > > > and
> > > > >> > > > > > > that means we will just have to make a few internal
> > > changes
> > > > to
> > > > >> > make
> > > > >> > > > > > > partitions accessible by topic id and partition. I can
> > > > update
> > > > >> the
> > > > >> > > KIP
> > > > >> > > > > to
> > > > >> > > > > > > remove them unless anyone thinks they should stay.
> > > > >> > > > > > >
> > > > >> > > > > > > 12) Addressed in the previous email. I've updated the
> > KIP
> > > to
> > > > >> > > include
> > > > >> > > > > > > tagged fields for the requests and responses. (More on
> > > that
> > > > >> > below)
> > > > >> > > > > > >
> > > > >> > > > > > > 13) I think part of the idea for including this
> > > information
> > > > >> is to
> > > > >> > > > > prepare
> > > > >> > > > > > > for future changes. Perhaps the directory structure
> > might
> > > > >> change
> > > > >> > > from
> > > > >> > > > > > > topicName_partitionNumber to something like
> > > > >> > > topicID_partitionNumber.
> > > > >> > > > > Then
> > > > >> > > > > > > it would be useful to have the topic name in the file
> > > since
> > > > it
> > > > >> > > would
> > > > >> > > > > not
> > > > >> > > > > > be
> > > > >> > > > > > > in the directory structure. Supporting topic renames
> > might
> > > > be
> > > > >> > > easier
> > > > >> > > > if
> > > > >> > > > > > the
> > > > >> > > > > > > other fields are included. Would there be any
> downsides
> > to
> > > > >> > > including
> > > > >> > > > > this
> > > > >> > > > > > > information?
> > > > >> > > > > > >
> > > > >> > > > > > > 14)  Yes, we would need to copy the partition metadata
> > > file
> > > > in
> > > > >> > this
> > > > >> > > > > > > process. I've updated the KIP to include this.
> > > > >> > > > > > >
> > > > >> > > > > > > 15) I believe Lucas meant v1 and v2 here. He was
> > referring
> > > > to
> > > > >> how
> > > > >> > > the
> > > > >> > > > > > > requests would fall under different IBP and meant that
> > > older
> > > > >> > > brokers
> > > > >> > > > > > would
> > > > >> > > > > > > have to use the older version of the request and the
> > > > existing
> > > > >> > topic
> > > > >> > > > > > > deletion process. At first, it seemed like tagged
> fields
> > > > would
> > > > >> > > > resolve
> > > > >> > > > > > > the IBP issue. However, we may need IBP for this
> request
> > > > after
> > > > >> > all
> > > > >> > > > > since
> > > > >> > > > > > > the controller handles the topic deletion differently
> > > > >> depending
> > > > >> > on
> > > > >> > > > the
> > > > >> > > > > > IBP
> > > > >> > > > > > > version. In an older version, we can't just send a
> > > > StopReplica
> > > > >> > > delete
> > > > >> > > > > the
> > > > >> > > > > > > topic immediately like we'd want to for this KIP.
> > > > >> > > > > > >
> > > > >> > > > > > > This makes me wonder if we want tagged fields on all
> the
> > > > >> requests
> > > > >> > > > after
> > > > >> > > > > > > all. Let me know your thoughts!
> > > > >> > > > > > >
> > > > >> > > > > > > Justine
> > > > >> > > > > > >
> > > > >> > > > > > > On Tue, Sep 15, 2020 at 1:03 PM Justine Olshan <
> > > > >> > > jols...@confluent.io
> > > > >> > > > >
> > > > >> > > > > > > wrote:
> > > > >> > > > > > >
> > > > >> > > > > > >> Hi all,
> > > > >> > > > > > >> Jun brought up a good point in his last email about
> > > tagged
> > > > >> > fields,
> > > > >> > > > and
> > > > >> > > > > > >> I've updated the KIP to reflect that the changes to
> > > > requests
> > > > >> and
> > > > >> > > > > > responses
> > > > >> > > > > > >> will be in the form of tagged fields to avoid
> changing
> > > IBP.
> > > > >> > > > > > >>
> > > > >> > > > > > >> Jun: I plan on sending a followup email to address
> some
> > > of
> > > > >> the
> > > > >> > > other
> > > > >> > > > > > >> points.
> > > > >> > > > > > >>
> > > > >> > > > > > >> Thanks,
> > > > >> > > > > > >> Justine
> > > > >> > > > > > >>
> > > > >> > > > > > >> On Mon, Sep 14, 2020 at 4:25 PM Jun Rao <
> > > j...@confluent.io>
> > > > >> > wrote:
> > > > >> > > > > > >>
> > > > >> > > > > > >>> Hi, Justine,
> > > > >> > > > > > >>>
> > > > >> > > > > > >>> Thanks for the updated KIP. A few comments below.
> > > > >> > > > > > >>>
> > > > >> > > > > > >>> 10. LeaderAndIsr Response: Do we need the topic
> name?
> > > > >> > > > > > >>>
> > > > >> > > > > > >>> 11. For the changed request/response, other than
> > > > >> LeaderAndIsr,
> > > > >> > > > > > >>> UpdateMetadata, Metadata, do we need to include the
> > > topic
> > > > >> name?
> > > > >> > > > > > >>>
> > > > >> > > > > > >>> 12. It seems that upgrades don't require IBP. Does
> > that
> > > > mean
> > > > >> > the
> > > > >> > > > new
> > > > >> > > > > > >>> fields
> > > > >> > > > > > >>> in all the request/response are added as tagged
> fields
> > > > >> without
> > > > >> > > > > bumping
> > > > >> > > > > > up
> > > > >> > > > > > >>> the request version? It would be useful to make that
> > > > clear.
> > > > >> > > > > > >>>
> > > > >> > > > > > >>> 13. Partition Metadata file: Do we need to include
> the
> > > > topic
> > > > >> > name
> > > > >> > > > and
> > > > >> > > > > > the
> > > > >> > > > > > >>> partition id since they are implied in the directory
> > > name?
> > > > >> > > > > > >>>
> > > > >> > > > > > >>> 14. In the JBOD mode, we support moving a
> partition's
> > > data
> > > > >> from
> > > > >> > > one
> > > > >> > > > > > disk
> > > > >> > > > > > >>> to
> > > > >> > > > > > >>> another. Will the new partition metadata file be
> > copied
> > > > >> during
> > > > >> > > that
> > > > >> > > > > > >>> process?
> > > > >> > > > > > >>>
> > > > >> > > > > > >>> 15. The KIP says "Remove deleted topics from
> replicas
> > by
> > > > >> > sending
> > > > >> > > > > > >>> StopReplicaRequest V2 for any topics which do not
> > > contain
> > > > a
> > > > >> > topic
> > > > >> > > > ID,
> > > > >> > > > > > and
> > > > >> > > > > > >>> V3 for any topics which do contain a topic ID.".
> > > However,
> > > > it
> > > > >> > > seems
> > > > >> > > > > the
> > > > >> > > > > > >>> updated controller will create all missing topic IDs
> > > first
> > > > >> > before
> > > > >> > > > > doing
> > > > >> > > > > > >>> other actions. So, is StopReplicaRequest V2 needed?
> > > > >> > > > > > >>>
> > > > >> > > > > > >>> Jun
> > > > >> > > > > > >>>
> > > > >> > > > > > >>> On Fri, Sep 11, 2020 at 10:31 AM John Roesler <
> > > > >> > > vvcep...@apache.org
> > > > >> > > > >
> > > > >> > > > > > >>> wrote:
> > > > >> > > > > > >>>
> > > > >> > > > > > >>> > Thanks, Justine!
> > > > >> > > > > > >>> >
> > > > >> > > > > > >>> > Your response seems compelling to me.
> > > > >> > > > > > >>> >
> > > > >> > > > > > >>> > -John
> > > > >> > > > > > >>> >
> > > > >> > > > > > >>> > On Fri, 2020-09-11 at 10:17 -0700, Justine Olshan
> > > wrote:
> > > > >> > > > > > >>> > > Hello all,
> > > > >> > > > > > >>> > > Thanks for continuing the discussion! I have a
> few
> > > > >> > responses
> > > > >> > > to
> > > > >> > > > > > your
> > > > >> > > > > > >>> > points.
> > > > >> > > > > > >>> > >
> > > > >> > > > > > >>> > > Tom: You are correct in that this KIP has not
> > > > mentioned
> > > > >> the
> > > > >> > > > > > >>> > > DeleteTopicsRequest. I think that this would be
> > out
> > > of
> > > > >> > scope
> > > > >> > > > for
> > > > >> > > > > > >>> now, but
> > > > >> > > > > > >>> > > may be something worth adding in the future.
> > > > >> > > > > > >>> > >
> > > > >> > > > > > >>> > > John: We did consider sequence ids, but there
> are
> > a
> > > > few
> > > > >> > > reasons
> > > > >> > > > > to
> > > > >> > > > > > >>> favor
> > > > >> > > > > > >>> > > UUIDs. There are several cases where topics from
> > > > >> different
> > > > >> > > > > clusters
> > > > >> > > > > > >>> may
> > > > >> > > > > > >>> > > interact now and in the future. For example,
> > Mirror
> > > > >> Maker 2
> > > > >> > > may
> > > > >> > > > > > >>> benefit
> > > > >> > > > > > >>> > > from being able to detect when a cluster being
> > > > mirrored
> > > > >> is
> > > > >> > > > > deleted
> > > > >> > > > > > >>> and
> > > > >> > > > > > >>> > > recreated and globally unique identifiers would
> > make
> > > > >> > > resolving
> > > > >> > > > > > issues
> > > > >> > > > > > >>> > > easier than sequence IDs which may collide
> between
> > > > >> > clusters.
> > > > >> > > > > > KIP-405
> > > > >> > > > > > >>> > > (tiered storage) will also benefit from globally
> > > > unique
> > > > >> IDs
> > > > >> > > as
> > > > >> > > > > > shared
> > > > >> > > > > > >>> > > buckets may be used between clusters.
> > > > >> > > > > > >>> > >
> > > > >> > > > > > >>> > > Globally unique IDs would also make
> functionality
> > > like
> > > > >> > moving
> > > > >> > > > > > topics
> > > > >> > > > > > >>> > > between disparate clusters easier in the future,
> > > > >> simplify
> > > > >> > any
> > > > >> > > > > > future
> > > > >> > > > > > >>> > > implementations of backups and restores, and
> more.
> > > In
> > > > >> > > general,
> > > > >> > > > > > >>> unique IDs
> > > > >> > > > > > >>> > > would ensure that the source cluster topics do
> not
> > > > >> conflict
> > > > >> > > > with
> > > > >> > > > > > the
> > > > >> > > > > > >>> > > destination cluster topics.
> > > > >> > > > > > >>> > >
> > > > >> > > > > > >>> > > If we were to use sequence ids, we would need
> > > > >> sufficiently
> > > > >> > > > large
> > > > >> > > > > > >>> cluster
> > > > >> > > > > > >>> > > ids to be stored with the topic identifiers or
> we
> > > run
> > > > >> the
> > > > >> > > risk
> > > > >> > > > of
> > > > >> > > > > > >>> > > collisions. This will give up any advantage in
> > > > >> compactness
> > > > >> > > that
> > > > >> > > > > > >>> sequence
> > > > >> > > > > > >>> > > numbers may bring. Given these advantages I
> think
> > it
> > > > >> makes
> > > > >> > > > sense
> > > > >> > > > > to
> > > > >> > > > > > >>> use
> > > > >> > > > > > >>> > > UUIDs.
> > > > >> > > > > > >>> > >
> > > > >> > > > > > >>> > > Gokul: This is an interesting idea, but this is
> a
> > > > >> breaking
> > > > >> > > > > change.
> > > > >> > > > > > >>> Out of
> > > > >> > > > > > >>> > > scope for now, but maybe worth discussing in the
> > > > future.
> > > > >> > > > > > >>> > >
> > > > >> > > > > > >>> > > Hope this explains some of the decisions,
> > > > >> > > > > > >>> > >
> > > > >> > > > > > >>> > > Justine
> > > > >> > > > > > >>> > >
> > > > >> > > > > > >>> > >
> > > > >> > > > > > >>> > >
> > > > >> > > > > > >>> > > On Fri, Sep 11, 2020 at 8:27 AM Gokul Ramanan
> > > > >> Subramanian <
> > > > >> > > > > > >>> > > gokul24...@gmail.com> wrote:
> > > > >> > > > > > >>> > >
> > > > >> > > > > > >>> > > > Hi.
> > > > >> > > > > > >>> > > >
> > > > >> > > > > > >>> > > > Thanks for the KIP.
> > > > >> > > > > > >>> > > >
> > > > >> > > > > > >>> > > > Have you thought about whether it makes sense
> to
> > > > >> support
> > > > >> > > > > > >>> authorizing a
> > > > >> > > > > > >>> > > > principal for a topic ID rather than a topic
> > name
> > > to
> > > > >> > > achieve
> > > > >> > > > > > >>> tighter
> > > > >> > > > > > >>> > > > security?
> > > > >> > > > > > >>> > > >
> > > > >> > > > > > >>> > > > Or is the topic ID fundamentally an internal
> > > detail
> > > > >> > similar
> > > > >> > > > to
> > > > >> > > > > > >>> epochs
> > > > >> > > > > > >>> > used
> > > > >> > > > > > >>> > > > in a bunch of other places in Kafka?
> > > > >> > > > > > >>> > > >
> > > > >> > > > > > >>> > > > Thanks.
> > > > >> > > > > > >>> > > >
> > > > >> > > > > > >>> > > > On Fri, Sep 11, 2020 at 4:06 PM John Roesler <
> > > > >> > > > > > vvcep...@apache.org>
> > > > >> > > > > > >>> > wrote:
> > > > >> > > > > > >>> > > >
> > > > >> > > > > > >>> > > > > Hello Justine,
> > > > >> > > > > > >>> > > > >
> > > > >> > > > > > >>> > > > > Thanks for the KIP!
> > > > >> > > > > > >>> > > > >
> > > > >> > > > > > >>> > > > > I happen to have been confronted recently
> with
> > > the
> > > > >> need
> > > > >> > > to
> > > > >> > > > > keep
> > > > >> > > > > > >>> > track of
> > > > >> > > > > > >>> > > > a
> > > > >> > > > > > >>> > > > > large number of topics as compactly as
> > > possible. I
> > > > >> was
> > > > >> > > > going
> > > > >> > > > > to
> > > > >> > > > > > >>> come
> > > > >> > > > > > >>> > up
> > > > >> > > > > > >>> > > > > with some way to dictionary encode the topic
> > > names
> > > > >> as
> > > > >> > > > > integers,
> > > > >> > > > > > >>> but
> > > > >> > > > > > >>> > this
> > > > >> > > > > > >>> > > > > seems much better!
> > > > >> > > > > > >>> > > > >
> > > > >> > > > > > >>> > > > > Apologies if this has been raised before,
> but
> > > I’m
> > > > >> > > wondering
> > > > >> > > > > > >>> about the
> > > > >> > > > > > >>> > > > > choice of UUID vs sequence number for the
> ids.
> > > > >> > Typically,
> > > > >> > > > > I’ve
> > > > >> > > > > > >>> seen
> > > > >> > > > > > >>> > UUIDs
> > > > >> > > > > > >>> > > > > in two situations:
> > > > >> > > > > > >>> > > > > 1. When processes need to generate
> > non-colliding
> > > > >> > > > identifiers
> > > > >> > > > > > >>> without
> > > > >> > > > > > >>> > > > > coordination.
> > > > >> > > > > > >>> > > > > 2. When the identifier needs to be
> > “universally
> > > > >> > unique”;
> > > > >> > > > > I.e.,
> > > > >> > > > > > >>> the
> > > > >> > > > > > >>> > > > > identifier needs to distinguish the entity
> > from
> > > > all
> > > > >> > other
> > > > >> > > > > > >>> entities
> > > > >> > > > > > >>> > that
> > > > >> > > > > > >>> > > > > could ever exist. This is useful in cases
> > where
> > > > >> > entities
> > > > >> > > > from
> > > > >> > > > > > all
> > > > >> > > > > > >>> > kinds
> > > > >> > > > > > >>> > > > of
> > > > >> > > > > > >>> > > > > systems get mixed together, such as when
> > dumping
> > > > >> logs
> > > > >> > > from
> > > > >> > > > > all
> > > > >> > > > > > >>> > processes
> > > > >> > > > > > >>> > > > in
> > > > >> > > > > > >>> > > > > a company into a common system.
> > > > >> > > > > > >>> > > > >
> > > > >> > > > > > >>> > > > > Maybe I’m being short-sighted, but it
> doesn’t
> > > seem
> > > > >> like
> > > > >> > > > > either
> > > > >> > > > > > >>> really
> > > > >> > > > > > >>> > > > > applies here. It seems like the brokers
> could
> > > and
> > > > >> would
> > > > >> > > > > achieve
> > > > >> > > > > > >>> > consensus
> > > > >> > > > > > >>> > > > > when creating a topic anyway, which is all
> > > that’s
> > > > >> > > required
> > > > >> > > > to
> > > > >> > > > > > >>> > generate
> > > > >> > > > > > >>> > > > > non-colliding sequence ids. For the second,
> as
> > > you
> > > > >> > > mention,
> > > > >> > > > > we
> > > > >> > > > > > >>> could
> > > > >> > > > > > >>> > > > assign
> > > > >> > > > > > >>> > > > > a UUID to the cluster as a whole, which
> would
> > > > render
> > > > >> > any
> > > > >> > > > > > resource
> > > > >> > > > > > >>> > scoped
> > > > >> > > > > > >>> > > > to
> > > > >> > > > > > >>> > > > > the broker universally unique as well.
> > > > >> > > > > > >>> > > > >
> > > > >> > > > > > >>> > > > > The reason I mention this is that, although
> a
> > > UUID
> > > > >> is
> > > > >> > way
> > > > >> > > > > more
> > > > >> > > > > > >>> > compact
> > > > >> > > > > > >>> > > > > than topic names, it’s still 16 bytes. In
> > > > contrast,
> > > > >> a
> > > > >> > > > 4-byte
> > > > >> > > > > > >>> integer
> > > > >> > > > > > >>> > > > > sequence id would give us 4 billion unique
> > > topics
> > > > >> per
> > > > >> > > > > cluster,
> > > > >> > > > > > >>> which
> > > > >> > > > > > >>> > > > seems
> > > > >> > > > > > >>> > > > > like enough ;)
> > > > >> > > > > > >>> > > > >
> > > > >> > > > > > >>> > > > > Considering the number of different times
> > these
> > > > >> topic
> > > > >> > > > > > >>> identifiers are
> > > > >> > > > > > >>> > > > sent
> > > > >> > > > > > >>> > > > > over the wire or stored in memory, it seems
> > like
> > > > it
> > > > >> > might
> > > > >> > > > be
> > > > >> > > > > > >>> worth
> > > > >> > > > > > >>> > the
> > > > >> > > > > > >>> > > > > additional 4x space savings.
> > > > >> > > > > > >>> > > > >
> > > > >> > > > > > >>> > > > > What do you think about this?
> > > > >> > > > > > >>> > > > >
> > > > >> > > > > > >>> > > > > Thanks,
> > > > >> > > > > > >>> > > > > John
> > > > >> > > > > > >>> > > > >
> > > > >> > > > > > >>> > > > > On Fri, Sep 11, 2020, at 03:20, Tom Bentley
> > > wrote:
> > > > >> > > > > > >>> > > > > > Hi Justine,
> > > > >> > > > > > >>> > > > > >
> > > > >> > > > > > >>> > > > > > This looks like a very welcome
> improvement.
> > > > >> Thanks!
> > > > >> > > > > > >>> > > > > >
> > > > >> > > > > > >>> > > > > > Maybe I missed it, but the KIP doesn't
> seem
> > to
> > > > >> > mention
> > > > >> > > > > > changing
> > > > >> > > > > > >>> > > > > > DeleteTopicsRequest to identify the topic
> > > using
> > > > an
> > > > >> > id.
> > > > >> > > > > Maybe
> > > > >> > > > > > >>> > that's out
> > > > >> > > > > > >>> > > > > of
> > > > >> > > > > > >>> > > > > > scope, but DeleteTopicsRequest is not
> listed
> > > > among
> > > > >> > the
> > > > >> > > > > Future
> > > > >> > > > > > >>> Work
> > > > >> > > > > > >>> > APIs
> > > > >> > > > > > >>> > > > > > either.
> > > > >> > > > > > >>> > > > > >
> > > > >> > > > > > >>> > > > > > Kind regards,
> > > > >> > > > > > >>> > > > > >
> > > > >> > > > > > >>> > > > > > Tom
> > > > >> > > > > > >>> > > > > >
> > > > >> > > > > > >>> > > > > > On Thu, Sep 10, 2020 at 3:59 PM Satish
> > > Duggana <
> > > > >> > > > > > >>> > > > satish.dugg...@gmail.com
> > > > >> > > > > > >>> > > > > > wrote:
> > > > >> > > > > > >>> > > > > >
> > > > >> > > > > > >>> > > > > > > Thanks Lucas/Justine for the nice KIP.
> > > > >> > > > > > >>> > > > > > >
> > > > >> > > > > > >>> > > > > > > It has several benefits which also
> include
> > > > >> > > simplifying
> > > > >> > > > > the
> > > > >> > > > > > >>> topic
> > > > >> > > > > > >>> > > > > > > deletion process by controller and logs
> > > > cleanup
> > > > >> by
> > > > >> > > > > brokers
> > > > >> > > > > > in
> > > > >> > > > > > >>> > corner
> > > > >> > > > > > >>> > > > > > > cases.
> > > > >> > > > > > >>> > > > > > >
> > > > >> > > > > > >>> > > > > > > Best,
> > > > >> > > > > > >>> > > > > > > Satish.
> > > > >> > > > > > >>> > > > > > >
> > > > >> > > > > > >>> > > > > > > On Wed, Sep 9, 2020 at 10:07 PM Justine
> > > > Olshan <
> > > > >> > > > > > >>> > jols...@confluent.io
> > > > >> > > > > > >>> > > > > > > wrote:
> > > > >> > > > > > >>> > > > > > > > Hello all, it's been almost a year!
> I've
> > > > made
> > > > >> > some
> > > > >> > > > > > changes
> > > > >> > > > > > >>> to
> > > > >> > > > > > >>> > this
> > > > >> > > > > > >>> > > > > KIP
> > > > >> > > > > > >>> > > > > > > and hope to continue the discussion.
> > > > >> > > > > > >>> > > > > > > > One of the main changes I've added is
> > now
> > > > the
> > > > >> > > > metadata
> > > > >> > > > > > >>> response
> > > > >> > > > > > >>> > > > will
> > > > >> > > > > > >>> > > > > > > include the topic ID (as Colin
> suggested).
> > > > >> Clients
> > > > >> > > can
> > > > >> > > > > > >>> obtain the
> > > > >> > > > > > >>> > > > > topicID
> > > > >> > > > > > >>> > > > > > > of a given topic through a
> > TopicDescription.
> > > > The
> > > > >> > > > topicId
> > > > >> > > > > > will
> > > > >> > > > > > >>> > also be
> > > > >> > > > > > >>> > > > > > > included with the UpdateMetadata
> request.
> > > > >> > > > > > >>> > > > > > > > Let me know what you all think.
> > > > >> > > > > > >>> > > > > > > > Thank you,
> > > > >> > > > > > >>> > > > > > > > Justine
> > > > >> > > > > > >>> > > > > > > >
> > > > >> > > > > > >>> > > > > > > > On 2019/09/13 16:38:26, "Colin
> McCabe" <
> > > > >> > > > > > cmcc...@apache.org
> > > > >> > > > > > >>> >
> > > > >> > > > > > >>> > wrote:
> > > > >> > > > > > >>> > > > > > > > > Hi Lucas,
> > > > >> > > > > > >>> > > > > > > > >
> > > > >> > > > > > >>> > > > > > > > > Thanks for tackling this.  Topic IDs
> > > are a
> > > > >> > great
> > > > >> > > > > idea,
> > > > >> > > > > > >>> and
> > > > >> > > > > > >>> > this
> > > > >> > > > > > >>> > > > is
> > > > >> > > > > > >>> > > > > a
> > > > >> > > > > > >>> > > > > > > really good writeup.
> > > > >> > > > > > >>> > > > > > > > > For /brokers/topics/[topic], the
> > schema
> > > > >> version
> > > > >> > > > > should
> > > > >> > > > > > be
> > > > >> > > > > > >>> > bumped
> > > > >> > > > > > >>> > > > to
> > > > >> > > > > > >>> > > > > > > version 3, rather than 2.  KIP-455
> bumped
> > > the
> > > > >> > version
> > > > >> > > > of
> > > > >> > > > > > this
> > > > >> > > > > > >>> > znode
> > > > >> > > > > > >>> > > > to
> > > > >> > > > > > >>> > > > > 2
> > > > >> > > > > > >>> > > > > > > already :)
> > > > >> > > > > > >>> > > > > > > > > Given that we're going to be seeing
> > > these
> > > > >> > things
> > > > >> > > as
> > > > >> > > > > > >>> strings
> > > > >> > > > > > >>> > as
> > > > >> > > > > > >>> > > > lot
> > > > >> > > > > > >>> > > > > (in
> > > > >> > > > > > >>> > > > > > > logs, in ZooKeeper, on the command-line,
> > > > etc.),
> > > > >> > does
> > > > >> > > it
> > > > >> > > > > > make
> > > > >> > > > > > >>> > sense to
> > > > >> > > > > > >>> > > > > use
> > > > >> > > > > > >>> > > > > > > base64 when converting them to strings?
> > > > >> > > > > > >>> > > > > > > > > Here is an example of the hex
> > > > >> representation:
> > > > >> > > > > > >>> > > > > > > > > 6fcb514b-b878-4c9d-95b7-8dc3a7ce6fd8
> > > > >> > > > > > >>> > > > > > > > >
> > > > >> > > > > > >>> > > > > > > > > And here is an example in base64.
> > > > >> > > > > > >>> > > > > > > > > b8tRS7h4TJ2Vt43Dp85v2A
> > > > >> > > > > > >>> > > > > > > > >
> > > > >> > > > > > >>> > > > > > > > > The base64 version saves 15 letters
> > (to
> > > be
> > > > >> > fair,
> > > > >> > > 4
> > > > >> > > > of
> > > > >> > > > > > >>> those
> > > > >> > > > > > >>> > were
> > > > >> > > > > > >>> > > > > > > dashes that we could have elided in the
> > hex
> > > > >> > > > > > representation.)
> > > > >> > > > > > >>> > > > > > > > > Another thing to consider is that we
> > > > should
> > > > >> > > specify
> > > > >> > > > > > that
> > > > >> > > > > > >>> the
> > > > >> > > > > > >>> > > > > > > all-zeroes UUID is not a valid topic
> UUID.
> > > >  We
> > > > >> > can't
> > > > >> > > > use
> > > > >> > > > > > >>> null
> > > > >> > > > > > >>> > for
> > > > >> > > > > > >>> > > > this
> > > > >> > > > > > >>> > > > > > > because we can't pass a null UUID over
> the
> > > RPC
> > > > >> > > protocol
> > > > >> > > > > > >>> (there
> > > > >> > > > > > >>> > is no
> > > > >> > > > > > >>> > > > > > > special pattern for null, nor do we want
> > to
> > > > >> waste
> > > > >> > > space
> > > > >> > > > > > >>> reserving
> > > > >> > > > > > >>> > > > such
> > > > >> > > > > > >>> > > > > a
> > > > >> > > > > > >>> > > > > > > pattern.)
> > > > >> > > > > > >>> > > > > > > > > Maybe I missed it, but did you
> > describe
> > > > >> > > "migration
> > > > >> > > > > > of...
> > > > >> > > > > > >>> > existing
> > > > >> > > > > > >>> > > > > > > topic[s] without topic IDs" in detail in
> > any
> > > > >> > section?
> > > > >> > > > It
> > > > >> > > > > > >>> seems
> > > > >> > > > > > >>> > like
> > > > >> > > > > > >>> > > > > when
> > > > >> > > > > > >>> > > > > > > the new controller becomes active, it
> > should
> > > > >> just
> > > > >> > > > > generate
> > > > >> > > > > > >>> random
> > > > >> > > > > > >>> > > > > UUIDs for
> > > > >> > > > > > >>> > > > > > > these, and write the random UUIDs back
> to
> > > > >> > ZooKeeper.
> > > > >> > > > It
> > > > >> > > > > > >>> would be
> > > > >> > > > > > >>> > > > good
> > > > >> > > > > > >>> > > > > to
> > > > >> > > > > > >>> > > > > > > spell that out.  We should make it clear
> > > that
> > > > >> this
> > > > >> > > > > happens
> > > > >> > > > > > >>> > regardless
> > > > >> > > > > > >>> > > > > of
> > > > >> > > > > > >>> > > > > > > the inter-broker protocol version (it's
> a
> > > > >> > compatible
> > > > >> > > > > > change).
> > > > >> > > > > > >>> > > > > > > > > "LeaderAndIsrRequests including an
> > > > >> > > > is_every_partition
> > > > >> > > > > > >>> flag"
> > > > >> > > > > > >>> > > > seems a
> > > > >> > > > > > >>> > > > > > > bit wordy.  Can we just call these "full
> > > > >> > > > > > >>> LeaderAndIsrRequests"?
> > > > >> > > > > > >>> > Then
> > > > >> > > > > > >>> > > > > the
> > > > >> > > > > > >>> > > > > > > RPC field could be named "full".  Also,
> it
> > > > would
> > > > >> > > > probably
> > > > >> > > > > > be
> > > > >> > > > > > >>> > better
> > > > >> > > > > > >>> > > > > for the
> > > > >> > > > > > >>> > > > > > > RPC field to be an enum of {
> UNSPECIFIED,
> > > > >> > > INCREMENTAL,
> > > > >> > > > > FULL
> > > > >> > > > > > >>> }, so
> > > > >> > > > > > >>> > > > that
> > > > >> > > > > > >>> > > > > we
> > > > >> > > > > > >>> > > > > > > can cleanly handle old versions (by
> > treating
> > > > >> them
> > > > >> > as
> > > > >> > > > > > >>> UNSPECIFIED)
> > > > >> > > > > > >>> > > > > > > > > In the LeaderAndIsrRequest section,
> > you
> > > > >> write
> > > > >> > "A
> > > > >> > > > > final
> > > > >> > > > > > >>> > deletion
> > > > >> > > > > > >>> > > > > event
> > > > >> > > > > > >>> > > > > > > will be secheduled for X ms after the
> > > > >> > > > LeaderAndIsrRequest
> > > > >> > > > > > was
> > > > >> > > > > > >>> > first
> > > > >> > > > > > >>> > > > > > > received..."  I guess the X was a
> > > placeholder
> > > > >> that
> > > > >> > > you
> > > > >> > > > > > >>> intended
> > > > >> > > > > > >>> > to
> > > > >> > > > > > >>> > > > > replace
> > > > >> > > > > > >>> > > > > > > before posting? :)  In any case, this
> > seems
> > > > like
> > > > >> > the
> > > > >> > > > kind
> > > > >> > > > > > of
> > > > >> > > > > > >>> > thing
> > > > >> > > > > > >>> > > > we'd
> > > > >> > > > > > >>> > > > > > > want a configuration for.  Let's
> describe
> > > that
> > > > >> > > > > > configuration
> > > > >> > > > > > >>> key
> > > > >> > > > > > >>> > > > > somewhere
> > > > >> > > > > > >>> > > > > > > in this KIP, including what its default
> > > value
> > > > >> is.
> > > > >> > > > > > >>> > > > > > > > > We should probably also log a bunch
> of
> > > > >> messages
> > > > >> > > at
> > > > >> > > > > WARN
> > > > >> > > > > > >>> level
> > > > >> > > > > > >>> > > > when
> > > > >> > > > > > >>> > > > > > > something is scheduled for deletion, as
> > > well.
> > > > >> > (Maybe
> > > > >> > > > > this
> > > > >> > > > > > >>> was
> > > > >> > > > > > >>> > > > > assumed, but
> > > > >> > > > > > >>> > > > > > > it would be good to mention it).
> > > > >> > > > > > >>> > > > > > > > > I feel like there are a few sections
> > > that
> > > > >> > should
> > > > >> > > be
> > > > >> > > > > > >>> moved to
> > > > >> > > > > > >>> > > > > "rejected
> > > > >> > > > > > >>> > > > > > > alternatives."  For example, in the
> > > > DeleteTopics
> > > > >> > > > section,
> > > > >> > > > > > >>> since
> > > > >> > > > > > >>> > we're
> > > > >> > > > > > >>> > > > > not
> > > > >> > > > > > >>> > > > > > > going to do option 1 or 2, these should
> be
> > > > moved
> > > > >> > into
> > > > >> > > > > > >>> "rejected
> > > > >> > > > > > >>> > > > > > > alternatives,"  rather than appearing
> > > inline.
> > > > >> > > Another
> > > > >> > > > > case
> > > > >> > > > > > >>> is
> > > > >> > > > > > >>> > the
> > > > >> > > > > > >>> > > > > "Should
> > > > >> > > > > > >>> > > > > > > we remove topic name from the protocol
> > where
> > > > >> > > possible"
> > > > >> > > > > > >>> section.
> > > > >> > > > > > >>> > This
> > > > >> > > > > > >>> > > > > is
> > > > >> > > > > > >>> > > > > > > clearly discussing a design alternative
> > that
> > > > >> we're
> > > > >> > > not
> > > > >> > > > > > >>> proposing
> > > > >> > > > > > >>> > to
> > > > >> > > > > > >>> > > > > > > implement: removing the topic name from
> > > those
> > > > >> > > > protocols.
> > > > >> > > > > > >>> > > > > > > > > Is it really necessary to have a new
> > > > >> > > > > > >>> > /admin/delete_topics_by_id
> > > > >> > > > > > >>> > > > > path
> > > > >> > > > > > >>> > > > > > > in ZooKeeper?  It seems like we don't
> > really
> > > > >> need
> > > > >> > > this.
> > > > >> > > > > > >>> Whenever
> > > > >> > > > > > >>> > > > > there is
> > > > >> > > > > > >>> > > > > > > a new controller, we'll send out full
> > > > >> > > > > LeaderAndIsrRequests
> > > > >> > > > > > >>> which
> > > > >> > > > > > >>> > will
> > > > >> > > > > > >>> > > > > > > trigger the stale topics to be cleaned
> up.
> > > >  The
> > > > >> > > active
> > > > >> > > > > > >>> > controller
> > > > >> > > > > > >>> > > > will
> > > > >> > > > > > >>> > > > > > > also send the full LeaderAndIsrRequest
> to
> > > > >> brokers
> > > > >> > > that
> > > > >> > > > > are
> > > > >> > > > > > >>> just
> > > > >> > > > > > >>> > > > > starting
> > > > >> > > > > > >>> > > > > > > up.    So we don't really need this kind
> > of
> > > > >> > two-phase
> > > > >> > > > > > commit
> > > > >> > > > > > >>> > (send
> > > > >> > > > > > >>> > > > out
> > > > >> > > > > > >>> > > > > > > StopReplicasRequest, get ACKs from all
> > > nodes,
> > > > >> > commit
> > > > >> > > by
> > > > >> > > > > > >>> removing
> > > > >> > > > > > >>> > > > > > > /admin/delete_topics node) any more.
> > > > >> > > > > > >>> > > > > > > > > You mention that FetchRequest will
> now
> > > > >> include
> > > > >> > > UUID
> > > > >> > > > > to
> > > > >> > > > > > >>> avoid
> > > > >> > > > > > >>> > > > issues
> > > > >> > > > > > >>> > > > > > > where requests are made to stale
> > partitions.
> > > > >> > > However,
> > > > >> > > > > > >>> adding a
> > > > >> > > > > > >>> > UUID
> > > > >> > > > > > >>> > > > to
> > > > >> > > > > > >>> > > > > > > MetadataRequest is listed as future
> work,
> > > out
> > > > of
> > > > >> > > scope
> > > > >> > > > > for
> > > > >> > > > > > >>> this
> > > > >> > > > > > >>> > KIP.
> > > > >> > > > > > >>> > > > > How
> > > > >> > > > > > >>> > > > > > > will the client learn what the topic
> UUID
> > > is,
> > > > if
> > > > >> > the
> > > > >> > > > > > metadata
> > > > >> > > > > > >>> > > > response
> > > > >> > > > > > >>> > > > > > > doesn't include that information?  It
> > seems
> > > > like
> > > > >> > > adding
> > > > >> > > > > the
> > > > >> > > > > > >>> UUID
> > > > >> > > > > > >>> > to
> > > > >> > > > > > >>> > > > > > > MetadataResponse would be an improvement
> > > here
> > > > >> that
> > > > >> > > > might
> > > > >> > > > > > not
> > > > >> > > > > > >>> be
> > > > >> > > > > > >>> > too
> > > > >> > > > > > >>> > > > > hard to
> > > > >> > > > > > >>> > > > > > > make.
> > > > >> > > > > > >>> > > > > > > > > best,
> > > > >> > > > > > >>> > > > > > > > > Colin
> > > > >> > > > > > >>> > > > > > > > >
> > > > >> > > > > > >>> > > > > > > > >
> > > > >> > > > > > >>> > > > > > > > > On Mon, Sep 9, 2019, at 17:48,
> Ryanne
> > > > Dolan
> > > > >> > > wrote:
> > > > >> > > > > > >>> > > > > > > > > > Lucas, this would be great. I've
> run
> > > > into
> > > > >> > > issues
> > > > >> > > > > with
> > > > >> > > > > > >>> > topics
> > > > >> > > > > > >>> > > > > being
> > > > >> > > > > > >>> > > > > > > > > > resurrected accidentally, since a
> > > client
> > > > >> > cannot
> > > > >> > > > > > easily
> > > > >> > > > > > >>> > > > > distinguish
> > > > >> > > > > > >>> > > > > > > between
> > > > >> > > > > > >>> > > > > > > > > > a deleted topic and a new topic
> with
> > > the
> > > > >> same
> > > > >> > > > name.
> > > > >> > > > > > I'd
> > > > >> > > > > > >>> > need
> > > > >> > > > > > >>> > > > the
> > > > >> > > > > > >>> > > > > ID
> > > > >> > > > > > >>> > > > > > > > > > accessible from the client to
> solve
> > > that
> > > > >> > issue,
> > > > >> > > > but
> > > > >> > > > > > >>> this
> > > > >> > > > > > >>> > is a
> > > > >> > > > > > >>> > > > > good
> > > > >> > > > > > >>> > > > > > > first
> > > > >> > > > > > >>> > > > > > > > > > step.
> > > > >> > > > > > >>> > > > > > > > > >
> > > > >> > > > > > >>> > > > > > > > > > Ryanne
> > > > >> > > > > > >>> > > > > > > > > >
> > > > >> > > > > > >>> > > > > > > > > > On Wed, Sep 4, 2019 at 1:41 PM
> Lucas
> > > > >> > > Bradstreet <
> > > > >> > > > > > >>> > > > > lu...@confluent.io>
> > > > >> > > > > > >>> > > > > > > wrote:
> > > > >> > > > > > >>> > > > > > > > > > > Hi all,
> > > > >> > > > > > >>> > > > > > > > > > >
> > > > >> > > > > > >>> > > > > > > > > > > I would like to kick off
> > discussion
> > > of
> > > > >> > > KIP-516,
> > > > >> > > > > an
> > > > >> > > > > > >>> > > > > implementation
> > > > >> > > > > > >>> > > > > > > of topic
> > > > >> > > > > > >>> > > > > > > > > > > IDs for Kafka. Topic IDs aim to
> > > solve
> > > > >> topic
> > > > >> > > > > > >>> uniqueness
> > > > >> > > > > > >>> > > > > problems in
> > > > >> > > > > > >>> > > > > > > Kafka,
> > > > >> > > > > > >>> > > > > > > > > > > where referring to a topic by
> name
> > > > >> alone is
> > > > >> > > > > > >>> insufficient.
> > > > >> > > > > > >>> > > > Such
> > > > >> > > > > > >>> > > > > > > cases
> > > > >> > > > > > >>> > > > > > > > > > > include when a topic has been
> > > deleted
> > > > >> and
> > > > >> > > > > recreated
> > > > >> > > > > > >>> with
> > > > >> > > > > > >>> > the
> > > > >> > > > > > >>> > > > > same
> > > > >> > > > > > >>> > > > > > > name.
> > > > >> > > > > > >>> > > > > > > > > > > Unique identifiers will help
> > > simplify
> > > > >> and
> > > > >> > > > improve
> > > > >> > > > > > >>> Kafka's
> > > > >> > > > > > >>> > > > topic
> > > > >> > > > > > >>> > > > > > > deletion
> > > > >> > > > > > >>> > > > > > > > > > > process, as well as prevent
> cases
> > > > where
> > > > >> > > brokers
> > > > >> > > > > may
> > > > >> > > > > > >>> > > > incorrectly
> > > > >> > > > > > >>> > > > > > > interact
> > > > >> > > > > > >>> > > > > > > > > > > with stale versions of topics.
> > > > >> > > > > > >>> > > > > > > > > > >
> > > > >> > > > > > >>> > > > > > > > > > >
> > > > >> > > > > > >>> > > > > > > > > > >
> > > > >> > > > > > >>> > > >
> > > > >> > > > > > >>> >
> > > > >> > > > > > >>>
> > > > >> > > > > >
> > > > >> > > > >
> > > > >> > > >
> > > > >> > >
> > > > >> >
> > > > >>
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-516%3A+Topic+Identifiers
> > > > >> > > > > > >>> > > > > > > > > > > Looking forward to your
> thoughts.
> > > > >> > > > > > >>> > > > > > > > > > >
> > > > >> > > > > > >>> > > > > > > > > > > Lucas
> > > > >> > > > > > >>> > > > > > > > > > >
> > > > >> > > > > > >>> >
> > > > >> > > > > > >>> >
> > > > >> > > > > > >>>
> > > > >> > > > > > >>
> > > > >> > > > > >
> > > > >> > > > >
> > > > >> > > >
> > > > >> > >
> > > > >> >
> > > > >>
> > > > >
> > > >
> > >
> >
>

Reply via email to