Re: [DISCUSS] KIP-836: Addition of Information in DescribeQuorumResponse about Voter Lag

2022-05-24 Thread José Armando García Sancio
On Fri, May 20, 2022 at 4:04 PM Niket Goel  wrote:
>
> So Ideally the LastCatchUpTimeMs is supposed to be a proxy for lag. We could 
> report this field as lag, but I am personally not in favor of doing that as 
> the LastCaughtUpTimeMs is just an approximation of lag, but not actual lag.
>
> I guess it depends on the experience we want through the tool. Do we want to 
> isolate the user from internal details and report a “lag” which we can change 
> the definition of, or just report the actual metric for the user of the tool 
> to decide for themselves how to interpret that info.
>
> Do you have a strong opinion here?

Thanks for the explanation Niket. I think it is okay to only report
the `LastCaughtUpTimeMs` since the leader will compute this value
based on the LEO and not the high-watermark.


Re: [DISCUSS] KIP-836: Addition of Information in DescribeQuorumResponse about Voter Lag

2022-05-20 Thread Niket Goel
So Ideally the LastCatchUpTimeMs is supposed to be a proxy for lag. We could 
report this field as lag, but I am personally not in favor of doing that as the 
LastCaughtUpTimeMs is just an approximation of lag, but not actual lag. 

I guess it depends on the experience we want through the tool. Do we want to 
isolate the user from internal details and report a “lag” which we can change 
the definition of, or just report the actual metric for the user of the tool to 
decide for themselves how to interpret that info.

Do you have a strong opinion here?

- Niket

> On May 20, 2022, at 11:29 AM, José Armando García Sancio 
>  wrote:
> 
> Thanks for the changes to KIP-836 Niket.
> 
> KIP-836 has the following output for "--describe replication":
>> bin/kafka-metadata-quorum.sh --describe replication
> ReplicaId   LogEndOffsetLag LastFetchTimeMs
> LastCaughtUpTimeMsStatus
> 0   234134  0   tnowtnow
>   Leader
> 1   234130  4   t2  t6
>   Follower
> 2   234100  34  t3  t7
>   Follower
> 3   234124  10  t4  t8
>   Observer
> 4   234130  4   t5  t9
>   Observer
> 
> KIP-595 has the following output for "--descripbe replication":
>> bin/kafka-metadata-quorum.sh --describe replication
> ReplicaId   LogEndOffsetLag LagTimeMs   Status
> 0   234134  0   0   Leader
> 1   234130  4   10  Follower
> 2   234100  34  15  Follower
> 3   234124  10  12  Observer
> 4   234130  4   15  Observer
> 
> Should we update KIP-836 to also include "LagTimeMs" in the output?



Re: [DISCUSS] KIP-836: Addition of Information in DescribeQuorumResponse about Voter Lag

2022-05-20 Thread José Armando García Sancio
Thanks for the changes to KIP-836 Niket.

KIP-836 has the following output for "--describe replication":
> bin/kafka-metadata-quorum.sh --describe replication
ReplicaId   LogEndOffsetLag LastFetchTimeMs
LastCaughtUpTimeMsStatus
0   234134  0   tnowtnow
   Leader
1   234130  4   t2  t6
   Follower
2   234100  34  t3  t7
   Follower
3   234124  10  t4  t8
   Observer
4   234130  4   t5  t9
   Observer

KIP-595 has the following output for "--descripbe replication":
> bin/kafka-metadata-quorum.sh --describe replication
ReplicaId   LogEndOffsetLag LagTimeMs   Status
0   234134  0   0   Leader
1   234130  4   10  Follower
2   234100  34  15  Follower
3   234124  10  12  Observer
4   234130  4   15  Observer

Should we update KIP-836 to also include "LagTimeMs" in the output?


Re: [DISCUSS] KIP-836: Addition of Information in DescribeQuorumResponse about Voter Lag

2022-05-19 Thread David Jacot
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  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 
> 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 
> > 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 
> > 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 
> > >> 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 
> > >>> Date: Thursday, May 12, 2022 at 5:25 PM
> > >>> To: 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
> > >>>
> > >>>
&

Re: [DISCUSS] KIP-836: Addition of Information in DescribeQuorumResponse about Voter Lag

2022-05-18 Thread Niket Goel
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 
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 
> 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 
> 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 
> >> 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 
> >>> Date: Thursday, May 12, 2022 at 5:25 PM
> >>> To: 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
> >>>  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


Re: [DISCUSS] KIP-836: Addition of Information in DescribeQuorumResponse about Voter Lag

2022-05-17 Thread deng ziming
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  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  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 
>> 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 
>>> Date: Thursday, May 12, 2022 at 5:25 PM
>>> To: 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
>>>  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



Re: [DISCUSS] KIP-836: Addition of Information in DescribeQuorumResponse about Voter Lag

2022-05-16 Thread Niket Goel
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  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 
> 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 
> > Date: Thursday, May 12, 2022 at 5:25 PM
> > To: 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
> >  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


Re: [DISCUSS] KIP-836: Addition of Information in DescribeQuorumResponse about Voter Lag

2022-05-16 Thread David Arthur
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 
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 
> Date: Thursday, May 12, 2022 at 5:25 PM
> To: 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
>  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
>


Re: [DISCUSS] KIP-836: Addition of Information in DescribeQuorumResponse about Voter Lag

2022-05-16 Thread Niket Goel
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 
Date: Thursday, May 12, 2022 at 5:25 PM
To: 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 
 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


Re: [DISCUSS] KIP-836: Addition of Information in DescribeQuorumResponse about Voter Lag

2022-05-12 Thread Niket Goel
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
 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


Re: [DISCUSS] KIP-836: Addition of Information in DescribeQuorumResponse about Voter Lag

2022-05-12 Thread José Armando García Sancio
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é


Re: [DISCUSS] KIP-836: Addition of Information in DescribeQuorumResponse about Voter Lag

2022-05-12 Thread Niket Goel
Hey Ron,

That's a good callout. Just a minor call out so that we are on the same
page - This API is always responded to by the Raft leader.
Now as you pointed out there is a possibility that the leader has not heard
from the voters yet. We will need to add in a state describing this UNKNOWN
fetch time for the voters. I will update the KIP to reflect this.

Thanks


On Thu, May 12, 2022 at 10:57 AM Ron Dagostino  wrote:

> Hi Niket.  Thanks for the KIP.  Are all the fields you specified
> always known?  For example, might a new controller not have a last
> fetch time for other voters, and then what would it send in the
> response?  If this is possible then we should be explicit about what
> is to be sent in this case.
>
> Ron
>
> On Thu, May 12, 2022 at 12:54 PM Niket Goel 
> wrote:
> >
> > Thanks for the suggestion Colin.
> >
> > > One minor point: I suspect that whatever we end up naming the
> additional
> > fields here, should also be the name of the metrics in KIP-835. So if we
> go
> > with a metric named "last-applied-offset" we'd want a lastAppliedOffset
> > field here, and so on.
> >
> > This is a good point. Will respond to the discussion thread on KIP-835
> > about the dependency here.
> >
> > > I also wonder if it makes sense for us to report the timestamp of the
> > latest batch that has been fetched (and not necessarily applied) rather
> > than the wall clock time at which the leader made the latest fetch.
> >
> > In theory I am onboard with your suggestion and honestly I too wanted to
> > add something similar. However, from what I understand (and please
> correct
> > me if my understanding is off), the `DescribeQuorum` API as it is
> > implemented lives in the Raft layer and utilizes the data available
> within
> > that layer to fill out the response. To achieve a more accurate info on
> > what was applied etc like you recommend, we would need to look into the
> > log.
> > This leaves us two with options high level options --
> > 1. Peek into the log in the raft layer:
> >   I think this is definitely not the way to go as it breaks the isolation
> > the raft layer has from the contents of the log and also introduces more
> > computational work which would hurt performance.
> > 2. Have the layer above the Raft Client (so the controller) provide the
> > required information:
> >   We can consider this approach, however it will break the separation
> > between the layers. IIUC, the `DescribeQuorum` API is intended to be a
> Raft
> > API, but doing this will result in it being dependent on the controller
> (or
> > some layer driving the raft client). I am not sure if that is the
> direction
> > we want to go in the long term.
> >
> > I think my meta point is that there might be a way to get more accurate
> > information of "lag" into the response, but the question is that if that
> > additional fidelity in the accuracy of the lag is worth the cost we will
> > end up paying to add it.
> >
> > Let me know your thoughts on this.
> >
> > On Wed, May 11, 2022 at 12:56 PM Colin McCabe 
> wrote:
> >
> > > Thanks, Niket. I also agree with Jason that this is a public API
> despite
> > > the lack of command-line tool, so we do indeed need a KIP. :)
> > >
> > > One minor point: I suspect that whatever we end up naming the
> additional
> > > fields here, should also be the name of the metrics in KIP-835. So if
> we go
> > > with a metric named "last-applied-offset" we'd want a lastAppliedOffset
> > > field here, and so on.
> > >
> > > I also wonder if it makes sense for us to report the timestamp of the
> > > latest batch that has been fetched (and not necessarily applied) rather
> > > than the wall clock time at which the leader made the latest fetch. If
> we
> > > take both timestamps directly from the metadata log, we know they'll be
> > > comparable even in the presence of clock skew. And we know because of
> > > KIP-835 that the metadata log won't go quiet for prolonged periods.
> > >
> > > best,
> > > Colin
> > >
> > >
> > > On Tue, May 10, 2022, at 13:30, Niket Goel wrote:
> > > >> @Niket does it make sense to add the Admin API to this KIP?
> > > >
> > > > Thanks Deng for pointing this out. I agree with Jason's suggestion. I
> > > will
> > > > go ahead and add the admin API to this KIP.
> > > >
> > > > - Niket
> > > >
> > > > On Tue, May 10, 2022 at 11:44 AM Jason Gustafson
> > > 
> > > > wrote:
> > > >
> > > >> > Hello Niket, currently DescribeQuorumResponse is not a public
> API, we
> > > >> don’t have a Admin api or shell script to get
> DescribeQuorumResponse, so
> > > >> it’s unnecessary to submit a KIP to change it, you can just submit
> a PR
> > > to
> > > >> accomplish this.
> > > >>
> > > >> Hey Ziming, I think it is public. It was documented in KIP-595 and
> we
> > > have
> > > >> implemented the API on the server. However, it looks like I never
> added
> > > >> the Admin API (even though it is assumed by the
> > > `kafka-metadata-quorum.sh`
> > > >> tool). @Niket does it make sense to add 

Re: [DISCUSS] KIP-836: Addition of Information in DescribeQuorumResponse about Voter Lag

2022-05-12 Thread Ron Dagostino
Hi Niket.  Thanks for the KIP.  Are all the fields you specified
always known?  For example, might a new controller not have a last
fetch time for other voters, and then what would it send in the
response?  If this is possible then we should be explicit about what
is to be sent in this case.

Ron

On Thu, May 12, 2022 at 12:54 PM Niket Goel  wrote:
>
> Thanks for the suggestion Colin.
>
> > One minor point: I suspect that whatever we end up naming the additional
> fields here, should also be the name of the metrics in KIP-835. So if we go
> with a metric named "last-applied-offset" we'd want a lastAppliedOffset
> field here, and so on.
>
> This is a good point. Will respond to the discussion thread on KIP-835
> about the dependency here.
>
> > I also wonder if it makes sense for us to report the timestamp of the
> latest batch that has been fetched (and not necessarily applied) rather
> than the wall clock time at which the leader made the latest fetch.
>
> In theory I am onboard with your suggestion and honestly I too wanted to
> add something similar. However, from what I understand (and please correct
> me if my understanding is off), the `DescribeQuorum` API as it is
> implemented lives in the Raft layer and utilizes the data available within
> that layer to fill out the response. To achieve a more accurate info on
> what was applied etc like you recommend, we would need to look into the
> log.
> This leaves us two with options high level options --
> 1. Peek into the log in the raft layer:
>   I think this is definitely not the way to go as it breaks the isolation
> the raft layer has from the contents of the log and also introduces more
> computational work which would hurt performance.
> 2. Have the layer above the Raft Client (so the controller) provide the
> required information:
>   We can consider this approach, however it will break the separation
> between the layers. IIUC, the `DescribeQuorum` API is intended to be a Raft
> API, but doing this will result in it being dependent on the controller (or
> some layer driving the raft client). I am not sure if that is the direction
> we want to go in the long term.
>
> I think my meta point is that there might be a way to get more accurate
> information of "lag" into the response, but the question is that if that
> additional fidelity in the accuracy of the lag is worth the cost we will
> end up paying to add it.
>
> Let me know your thoughts on this.
>
> On Wed, May 11, 2022 at 12:56 PM Colin McCabe  wrote:
>
> > Thanks, Niket. I also agree with Jason that this is a public API despite
> > the lack of command-line tool, so we do indeed need a KIP. :)
> >
> > One minor point: I suspect that whatever we end up naming the additional
> > fields here, should also be the name of the metrics in KIP-835. So if we go
> > with a metric named "last-applied-offset" we'd want a lastAppliedOffset
> > field here, and so on.
> >
> > I also wonder if it makes sense for us to report the timestamp of the
> > latest batch that has been fetched (and not necessarily applied) rather
> > than the wall clock time at which the leader made the latest fetch. If we
> > take both timestamps directly from the metadata log, we know they'll be
> > comparable even in the presence of clock skew. And we know because of
> > KIP-835 that the metadata log won't go quiet for prolonged periods.
> >
> > best,
> > Colin
> >
> >
> > On Tue, May 10, 2022, at 13:30, Niket Goel wrote:
> > >> @Niket does it make sense to add the Admin API to this KIP?
> > >
> > > Thanks Deng for pointing this out. I agree with Jason's suggestion. I
> > will
> > > go ahead and add the admin API to this KIP.
> > >
> > > - Niket
> > >
> > > On Tue, May 10, 2022 at 11:44 AM Jason Gustafson
> > 
> > > wrote:
> > >
> > >> > Hello Niket, currently DescribeQuorumResponse is not a public API, we
> > >> don’t have a Admin api or shell script to get DescribeQuorumResponse, so
> > >> it’s unnecessary to submit a KIP to change it, you can just submit a PR
> > to
> > >> accomplish this.
> > >>
> > >> Hey Ziming, I think it is public. It was documented in KIP-595 and we
> > have
> > >> implemented the API on the server. However, it looks like I never added
> > >> the Admin API (even though it is assumed by the
> > `kafka-metadata-quorum.sh`
> > >> tool). @Niket does it make sense to add the Admin API to this KIP?
> > >>
> > >> Best,
> > >> Jason
> > >>
> > >> On Mon, May 9, 2022 at 8:09 PM deng ziming 
> > >> wrote:
> > >>
> > >> > Hello Niket, currently DescribeQuorumResponse is not a public API, we
> > >> > don’t have a Admin api or shell script to get DescribeQuorumResponse,
> > so
> > >> > it’s unnecessary to submit a KIP to change it, you can just submit a
> > PR
> > >> to
> > >> > accomplish this.
> > >> >
> > >> > --
> > >> > Thanks
> > >> > Ziming
> > >> >
> > >> > > On May 10, 2022, at 1:33 AM, Niket Goel  > >
> > >> > wrote:
> > >> > >
> > >> > > Hi all,
> > >> > >
> > >> > > I created a KIP to add some more informati

Re: [DISCUSS] KIP-836: Addition of Information in DescribeQuorumResponse about Voter Lag

2022-05-12 Thread Niket Goel
Thanks for the suggestion Colin.

> One minor point: I suspect that whatever we end up naming the additional
fields here, should also be the name of the metrics in KIP-835. So if we go
with a metric named "last-applied-offset" we'd want a lastAppliedOffset
field here, and so on.

This is a good point. Will respond to the discussion thread on KIP-835
about the dependency here.

> I also wonder if it makes sense for us to report the timestamp of the
latest batch that has been fetched (and not necessarily applied) rather
than the wall clock time at which the leader made the latest fetch.

In theory I am onboard with your suggestion and honestly I too wanted to
add something similar. However, from what I understand (and please correct
me if my understanding is off), the `DescribeQuorum` API as it is
implemented lives in the Raft layer and utilizes the data available within
that layer to fill out the response. To achieve a more accurate info on
what was applied etc like you recommend, we would need to look into the
log.
This leaves us two with options high level options --
1. Peek into the log in the raft layer:
  I think this is definitely not the way to go as it breaks the isolation
the raft layer has from the contents of the log and also introduces more
computational work which would hurt performance.
2. Have the layer above the Raft Client (so the controller) provide the
required information:
  We can consider this approach, however it will break the separation
between the layers. IIUC, the `DescribeQuorum` API is intended to be a Raft
API, but doing this will result in it being dependent on the controller (or
some layer driving the raft client). I am not sure if that is the direction
we want to go in the long term.

I think my meta point is that there might be a way to get more accurate
information of "lag" into the response, but the question is that if that
additional fidelity in the accuracy of the lag is worth the cost we will
end up paying to add it.

Let me know your thoughts on this.

On Wed, May 11, 2022 at 12:56 PM Colin McCabe  wrote:

> Thanks, Niket. I also agree with Jason that this is a public API despite
> the lack of command-line tool, so we do indeed need a KIP. :)
>
> One minor point: I suspect that whatever we end up naming the additional
> fields here, should also be the name of the metrics in KIP-835. So if we go
> with a metric named "last-applied-offset" we'd want a lastAppliedOffset
> field here, and so on.
>
> I also wonder if it makes sense for us to report the timestamp of the
> latest batch that has been fetched (and not necessarily applied) rather
> than the wall clock time at which the leader made the latest fetch. If we
> take both timestamps directly from the metadata log, we know they'll be
> comparable even in the presence of clock skew. And we know because of
> KIP-835 that the metadata log won't go quiet for prolonged periods.
>
> best,
> Colin
>
>
> On Tue, May 10, 2022, at 13:30, Niket Goel wrote:
> >> @Niket does it make sense to add the Admin API to this KIP?
> >
> > Thanks Deng for pointing this out. I agree with Jason's suggestion. I
> will
> > go ahead and add the admin API to this KIP.
> >
> > - Niket
> >
> > On Tue, May 10, 2022 at 11:44 AM Jason Gustafson
> 
> > wrote:
> >
> >> > Hello Niket, currently DescribeQuorumResponse is not a public API, we
> >> don’t have a Admin api or shell script to get DescribeQuorumResponse, so
> >> it’s unnecessary to submit a KIP to change it, you can just submit a PR
> to
> >> accomplish this.
> >>
> >> Hey Ziming, I think it is public. It was documented in KIP-595 and we
> have
> >> implemented the API on the server. However, it looks like I never added
> >> the Admin API (even though it is assumed by the
> `kafka-metadata-quorum.sh`
> >> tool). @Niket does it make sense to add the Admin API to this KIP?
> >>
> >> Best,
> >> Jason
> >>
> >> On Mon, May 9, 2022 at 8:09 PM deng ziming 
> >> wrote:
> >>
> >> > Hello Niket, currently DescribeQuorumResponse is not a public API, we
> >> > don’t have a Admin api or shell script to get DescribeQuorumResponse,
> so
> >> > it’s unnecessary to submit a KIP to change it, you can just submit a
> PR
> >> to
> >> > accomplish this.
> >> >
> >> > --
> >> > Thanks
> >> > Ziming
> >> >
> >> > > On May 10, 2022, at 1:33 AM, Niket Goel  >
> >> > wrote:
> >> > >
> >> > > Hi all,
> >> > >
> >> > > I created a KIP to add some more information to
> >> > `DesscribeQuorumResponse` to enable ascertaining voter lag in the
> quorum
> >> a
> >> > little better.
> >> > > Please see KIP --
> >> >
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-836%3A+Additional+Information+in+DescribeQuorumResponse+about+Voter+Lag
> >> > >
> >> > > Thanks for your feedback,
> >> > > Niket Goel
> >> >
> >> >
> >>
> >
> >
> > --
> > - Niket
>


-- 
- Niket


Re: [DISCUSS] KIP-836: Addition of Information in DescribeQuorumResponse about Voter Lag

2022-05-11 Thread Colin McCabe
Thanks, Niket. I also agree with Jason that this is a public API despite the 
lack of command-line tool, so we do indeed need a KIP. :)

One minor point: I suspect that whatever we end up naming the additional fields 
here, should also be the name of the metrics in KIP-835. So if we go with a 
metric named "last-applied-offset" we'd want a lastAppliedOffset field here, 
and so on.

I also wonder if it makes sense for us to report the timestamp of the latest 
batch that has been fetched (and not necessarily applied) rather than the wall 
clock time at which the leader made the latest fetch. If we take both 
timestamps directly from the metadata log, we know they'll be comparable even 
in the presence of clock skew. And we know because of KIP-835 that the metadata 
log won't go quiet for prolonged periods.

best,
Colin


On Tue, May 10, 2022, at 13:30, Niket Goel wrote:
>> @Niket does it make sense to add the Admin API to this KIP?
>
> Thanks Deng for pointing this out. I agree with Jason's suggestion. I will
> go ahead and add the admin API to this KIP.
>
> - Niket
>
> On Tue, May 10, 2022 at 11:44 AM Jason Gustafson 
> wrote:
>
>> > Hello Niket, currently DescribeQuorumResponse is not a public API, we
>> don’t have a Admin api or shell script to get DescribeQuorumResponse, so
>> it’s unnecessary to submit a KIP to change it, you can just submit a PR to
>> accomplish this.
>>
>> Hey Ziming, I think it is public. It was documented in KIP-595 and we have
>> implemented the API on the server. However, it looks like I never added
>> the Admin API (even though it is assumed by the `kafka-metadata-quorum.sh`
>> tool). @Niket does it make sense to add the Admin API to this KIP?
>>
>> Best,
>> Jason
>>
>> On Mon, May 9, 2022 at 8:09 PM deng ziming 
>> wrote:
>>
>> > Hello Niket, currently DescribeQuorumResponse is not a public API, we
>> > don’t have a Admin api or shell script to get DescribeQuorumResponse, so
>> > it’s unnecessary to submit a KIP to change it, you can just submit a PR
>> to
>> > accomplish this.
>> >
>> > --
>> > Thanks
>> > Ziming
>> >
>> > > On May 10, 2022, at 1:33 AM, Niket Goel 
>> > wrote:
>> > >
>> > > Hi all,
>> > >
>> > > I created a KIP to add some more information to
>> > `DesscribeQuorumResponse` to enable ascertaining voter lag in the quorum
>> a
>> > little better.
>> > > Please see KIP --
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-836%3A+Additional+Information+in+DescribeQuorumResponse+about+Voter+Lag
>> > >
>> > > Thanks for your feedback,
>> > > Niket Goel
>> >
>> >
>>
>
>
> -- 
> - Niket


Re: [DISCUSS] KIP-836: Addition of Information in DescribeQuorumResponse about Voter Lag

2022-05-10 Thread Niket Goel
> @Niket does it make sense to add the Admin API to this KIP?

Thanks Deng for pointing this out. I agree with Jason's suggestion. I will
go ahead and add the admin API to this KIP.

- Niket

On Tue, May 10, 2022 at 11:44 AM Jason Gustafson 
wrote:

> > Hello Niket, currently DescribeQuorumResponse is not a public API, we
> don’t have a Admin api or shell script to get DescribeQuorumResponse, so
> it’s unnecessary to submit a KIP to change it, you can just submit a PR to
> accomplish this.
>
> Hey Ziming, I think it is public. It was documented in KIP-595 and we have
> implemented the API on the server. However, it looks like I never added
> the Admin API (even though it is assumed by the `kafka-metadata-quorum.sh`
> tool). @Niket does it make sense to add the Admin API to this KIP?
>
> Best,
> Jason
>
> On Mon, May 9, 2022 at 8:09 PM deng ziming 
> wrote:
>
> > Hello Niket, currently DescribeQuorumResponse is not a public API, we
> > don’t have a Admin api or shell script to get DescribeQuorumResponse, so
> > it’s unnecessary to submit a KIP to change it, you can just submit a PR
> to
> > accomplish this.
> >
> > --
> > Thanks
> > Ziming
> >
> > > On May 10, 2022, at 1:33 AM, Niket Goel 
> > wrote:
> > >
> > > Hi all,
> > >
> > > I created a KIP to add some more information to
> > `DesscribeQuorumResponse` to enable ascertaining voter lag in the quorum
> a
> > little better.
> > > Please see KIP --
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-836%3A+Additional+Information+in+DescribeQuorumResponse+about+Voter+Lag
> > >
> > > Thanks for your feedback,
> > > Niket Goel
> >
> >
>


-- 
- Niket


Re: [DISCUSS] KIP-836: Addition of Information in DescribeQuorumResponse about Voter Lag

2022-05-10 Thread Jason Gustafson
> Hello Niket, currently DescribeQuorumResponse is not a public API, we
don’t have a Admin api or shell script to get DescribeQuorumResponse, so
it’s unnecessary to submit a KIP to change it, you can just submit a PR to
accomplish this.

Hey Ziming, I think it is public. It was documented in KIP-595 and we have
implemented the API on the server. However, it looks like I never added
the Admin API (even though it is assumed by the `kafka-metadata-quorum.sh`
tool). @Niket does it make sense to add the Admin API to this KIP?

Best,
Jason

On Mon, May 9, 2022 at 8:09 PM deng ziming  wrote:

> Hello Niket, currently DescribeQuorumResponse is not a public API, we
> don’t have a Admin api or shell script to get DescribeQuorumResponse, so
> it’s unnecessary to submit a KIP to change it, you can just submit a PR to
> accomplish this.
>
> --
> Thanks
> Ziming
>
> > On May 10, 2022, at 1:33 AM, Niket Goel 
> wrote:
> >
> > Hi all,
> >
> > I created a KIP to add some more information to
> `DesscribeQuorumResponse` to enable ascertaining voter lag in the quorum a
> little better.
> > Please see KIP --
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-836%3A+Additional+Information+in+DescribeQuorumResponse+about+Voter+Lag
> >
> > Thanks for your feedback,
> > Niket Goel
>
>


Re: [DISCUSS] KIP-836: Addition of Information in DescribeQuorumResponse about Voter Lag

2022-05-09 Thread deng ziming
Hello Niket, currently DescribeQuorumResponse is not a public API, we don’t 
have a Admin api or shell script to get DescribeQuorumResponse, so it’s 
unnecessary to submit a KIP to change it, you can just submit a PR to 
accomplish this.

--
Thanks
Ziming

> On May 10, 2022, at 1:33 AM, Niket Goel  wrote:
> 
> Hi all,
> 
> I created a KIP to add some more information to `DesscribeQuorumResponse` to 
> enable ascertaining voter lag in the quorum a little better.
> Please see KIP -- 
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-836%3A+Additional+Information+in+DescribeQuorumResponse+about+Voter+Lag
> 
> Thanks for your feedback,
> Niket Goel