Hi Niket,

Thanks for the KIP. I have a few minor comments:

1. We should keep DescribeQuorumResult's constructor package-private
for now in my opinion. We have been debating about this in KIP-692 and
KIP-777 but as we haven't reached a consensus yet, we should be on the
conservative side here.

2. Regarding gettings in QuorumInfo and ReplicaState, we usually don't
prefix getters with `get`.

3. Regarding the new fields in DescribeQuorumResponse, should they
have default values (e.g. -1) and be ignorable? The default values are
useful when talking to a controller which does not support those fields.

Best,
David

On Thu, May 19, 2022 at 12:59 AM Niket Goel <ng...@confluent.io.invalid> wrote:
>
> I did miss updating the KIP on Jose's comment. Have done that now, thanks
> for the reminder.
>
> For the `kafka-metadata-quorum.sh` tool, it seems that the tool's
> dependence on the DescribeQuorum API is implicit given the original KIP
> [1]. I will add a section in this KIP demonstrating that the tool's output
> should contain the newly added fields as well.
> The tool itself is tracked under this JIRA [2]
>
> Thanks
> Niket
>
> [1]
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum#KIP595%3AARaftProtocolfortheMetadataQuorum-ToolingSupport=
> [2] https://issues.apache.org/jira/browse/KAFKA-13914
>
> On Tue, May 17, 2022 at 7:31 PM deng ziming <dengziming1...@gmail.com>
> wrote:
>
> > Hello Niket,
> >
> >
> > 1. I find the DescribeQuorumResult still contains an
> > DescribeQuorumResponseData, which is not allowed as Jose commented, have
> > you forgot to change it?
> >
> > 2. You only add an Handle in AdminClient, can you also add an
> > `kafka-metadata-quorum.sh` tool to help this?
> >
> >
> > > On May 17, 2022, at 9:50 AM, Niket Goel <ng...@confluent.io.INVALID>
> > wrote:
> > >
> > > Thanks for the call out David. We will populate these fields for the
> > > Observers as well. I will clarify this in the KIP.
> > >
> > > On Mon, May 16, 2022 at 1:50 PM David Arthur <davidart...@apache.org>
> > wrote:
> > >
> > >> Niket, thanks for the KIP!
> > >>
> > >> Sorry for the late feedback on this, but I just had a quick question.
> > The
> > >> KIP indicates the two new fields will be set for voters only, however
> > this
> > >> ReplicaState struct is also used by the Observers in
> > >> DescribeQuorumResponse. Will we simply fill in -1 for these values, or
> > do
> > >> we intend to report the last fetch and caught-up time of the observers
> > as
> > >> well?
> > >>
> > >> Thanks!
> > >> David
> > >>
> > >>
> > >> On Mon, May 16, 2022 at 1:46 PM Niket Goel <ng...@confluent.io.invalid>
> > >> wrote:
> > >>
> > >>> Hi all,
> > >>>
> > >>> Thank you for the feedback on this. I have started a voting thread for
> > >>> this KIP here:
> > >>> https://lists.apache.org/thread/bkb7gsbxpljh5qh014ztffq7bldjrb2x
> > >>>
> > >>> Thanks
> > >>> Niket Goel
> > >>>
> > >>>
> > >>> From: Niket Goel <ng...@confluent.io>
> > >>> Date: Thursday, May 12, 2022 at 5:25 PM
> > >>> To: dev@kafka.apache.org <dev@kafka.apache.org>
> > >>> Subject: Re: [DISCUSS] KIP-836: Addition of Information in
> > >>> DescribeQuorumResponse about Voter Lag
> > >>> Appreciate the careful review Jose.!
> > >>>
> > >>> Ack on 1 and 2. Will fix.
> > >>>
> > >>> For number 3 (and I am using [1] as a reference for this discussion), I
> > >>> think the correct language to use would be:
> > >>>
> > >>> "Whenever a new fetch request
> > >>> comes in the replica's last caught up time is updated to the time of
> > >>> this fetch request if it requests an offset greater than or equal to
> > the
> > >>> leader's
> > >>> current end offset"
> > >>> Does that sound right now?
> > >>>
> > >>> Although I think I will go ahead and rewrite the explanation in a way
> > >> that
> > >>> is more understandable. Thanks for pointing this out.
> > >>>
> > >>> Thanks
> > >>>
> > >>> [1]
> > >>>
> > >>
> > https://github.com/apache/kafka/blob/fa59be4e770627cd34cef85986b58ad7f606928d/core/src/main/scala/kafka/cluster/Replica.scala#L97
> > >>>
> > >>>
> > >>>
> > >>> On Thu, May 12, 2022 at 3:20 PM José Armando García Sancio
> > >>> <jsan...@confluent.io.invalid> wrote:
> > >>> Thanks for the Kafka improvement Niket.
> > >>>
> > >>> 1. For the fields `LastFetchTime` and `LastCaughtUpTime`, Kafka tends
> > >>> to use the suffix "Timestamp" when the value is an absolute wall clock
> > >>> value.
> > >>>
> > >>> 2. The method `result()` for the type `DescribeQuorumResult` returns
> > >>> the type `DescribeQuorumResponseData`. The types generated from the
> > >>> RPC JSON schema are internal to Kafka and not exposed to clients. For
> > >>> the admin client we should use a different type that is explicitly
> > >>> public. See `org.apache.kafka.client.admin.DescribeTopicsResult` for
> > >>> an example.
> > >>>
> > >>> 3. The proposed section has his sentence "Whenever a new fetch request
> > >>> comes in the replica's last caught up time is updated to the time of
> > >>> the fetch request if it requests an offset greater than the leader's
> > >>> current end offset." Did you mean "previous fetch time" instead of
> > >>> "last caught up time"? What do you mean by "requests an offset greater
> > >>> than the leader's current end offset.?" Excluding diverging logs the
> > >>> follower fetch offset should never be greater than the leader LEO.
> > >>>
> > >>> Thanks,
> > >>> -José
> > >>>
> > >>>
> > >>> --
> > >>> - Niket
> > >>>
> > >>
> > >
> > >
> > > --
> > > - Niket
> >
> >
>
> --
> - Niket

Reply via email to