Re: [DISCUSS] KIP-876: Time based cluster metadata snapshots

2022-10-12 Thread Niket Goel
Thanks for the KIP Jose! Adding this ability makes sense to me. Just a few
quick wonderments:
1. Do we need this value to be of the order of `ms`. Is it better off being
tunable to a minute granularity?
2. Somewhat related to 1 - Do we need to make any tweaks to the way we
cleanup the metadata directory (time/size based cleanup)? My concern is if
the metadata-dir cleanup will trigger fast enough in all cases today. e.g.
If I were to configure the time-based snapshotting to something like every
10 seconds, with enough load of course, will I end up flooding the disk.?

On Wed, Oct 12, 2022 at 1:28 AM David Jacot 
wrote:

> Hi José,
>
> Thanks for the KIP. That makes total sense. On nit, I would name the
> new property `metadata.log.snapshot.interval.ms` as `between` is
> implied by the `interval`.
>
> Best,
> David
>
> On Tue, Oct 11, 2022 at 9:16 PM José Armando García Sancio
>  wrote:
> >
> > Hey all,
> >
> > I am interested in allowing brokers and controllers in KRaft to
> > generate snapshots for the cluster metadata partition on a timely
> > basis. This would better allow Kafka users to use cluster metadata
> > snapshots as a solution for backing up the cluster's metadata.
> >
> > Let's use this thread to discuss KIP-876:
> > https://cwiki.apache.org/confluence/x/MY3GDQ
> >
> > Thanks!
> > --
> > -José
>


-- 
- Niket


[jira] [Resolved] (KAFKA-14275) KRaft Controllers should crash after failing to apply any metadata record

2022-10-11 Thread Niket Goel (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-14275?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Niket Goel resolved KAFKA-14275.

Fix Version/s: 3.3
 Reviewer: Jason Gustafson
   Resolution: Fixed

> KRaft Controllers should crash after failing to apply any metadata record 
> --
>
> Key: KAFKA-14275
> URL: https://issues.apache.org/jira/browse/KAFKA-14275
> Project: Kafka
>  Issue Type: Bug
>  Components: kraft
>Affects Versions: 3.3.1
>    Reporter: Niket Goel
>Assignee: Niket Goel
>Priority: Major
> Fix For: 3.3
>
>
> When replaying records on a standby controller, any error encountered will 
> halt further processing of that batch. Currently we log an error and allow 
> the controller to continue normal operation. In contrast a similar error on 
> the active controller causes it to halt and exit the jvm. This is 
> inconsistent behavior as nothing prevents a standby from eventually becoming 
> the active controller (even when it had skipped over a record batch). We 
> should halt the process in the case of a standby controller as well.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14275) KRaft Controllers should crash after failing to apply any metadata record

2022-10-03 Thread Niket Goel (Jira)
Niket Goel created KAFKA-14275:
--

 Summary: KRaft Controllers should crash after failing to apply any 
metadata record 
 Key: KAFKA-14275
 URL: https://issues.apache.org/jira/browse/KAFKA-14275
 Project: Kafka
  Issue Type: Bug
  Components: kraft
Affects Versions: 3.3.1
Reporter: Niket Goel


When replaying records on a standby controller, any error encountered will halt 
further processing of that batch. Currently we log an error and allow the 
controller to continue normal operation. In contrast a similar error on the 
active controller causes it to halt and exit the jvm. This is inconsistent 
behavior as nothing prevents a standby from eventually becoming the active 
controller (even when it had skipped over a record batch). We should halt the 
process in the case of a standby controller as well.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-13888) KIP-836: Addition of Information in DescribeQuorumResponse about Voter Lag

2022-08-19 Thread Niket Goel (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-13888?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Niket Goel resolved KAFKA-13888.

Resolution: Fixed

> KIP-836: Addition of Information in DescribeQuorumResponse about Voter Lag
> --
>
> Key: KAFKA-13888
> URL: https://issues.apache.org/jira/browse/KAFKA-13888
> Project: Kafka
>  Issue Type: Improvement
>  Components: kraft
>    Reporter: Niket Goel
>    Assignee: Niket Goel
>Priority: Blocker
> Fix For: 3.3.0
>
>
> Tracking issue for the implementation of KIP:836



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-14114) KIP-859: Add Metadata Log Processing Error Related Metrics

2022-08-10 Thread Niket Goel (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-14114?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Niket Goel resolved KAFKA-14114.

Resolution: Fixed

[https://github.com/apache/kafka/commit/ac64693434a66ad76bfc401a1a89ab2dbeaaeaa8]

> KIP-859: Add Metadata Log Processing Error Related Metrics
> --
>
> Key: KAFKA-14114
> URL: https://issues.apache.org/jira/browse/KAFKA-14114
> Project: Kafka
>  Issue Type: Improvement
>Affects Versions: 3.2.0
>    Reporter: Niket Goel
>    Assignee: Niket Goel
>Priority: Blocker
> Fix For: 3.3.0
>
>
> Tracking Jira for KIP:859



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Re: [DISCUSS] Apache Kafka 3.3.0 Release

2022-08-04 Thread Niket Goel
(hit send too soon)
KIP --
https://cwiki.apache.org/confluence/display/KAFKA/KIP-859%3A+Add+Metadata+Log+Processing+Error+Related+Metrics
JIRA -- https://issues.apache.org/jira/browse/KAFKA-14114

On Thu, Aug 4, 2022 at 2:33 PM Niket Goel  wrote:

> Hey Jose,
>
> I would like to request adding KIP-859 [1] to the 3.3.0 release. The KIP
> adds some important metrics to allow visibility into KRaft log processing
> related errors.
> The KIP was approved today and I have a PR ready for review, which I hope
> should get reviewed and merged within next week.
>
> Thanks
> Niket Goel
>
> On Thu, Aug 4, 2022 at 8:58 AM José Armando García Sancio
>  wrote:
>
>> On Thu, Aug 4, 2022 at 8:37 AM Justine Olshan
>>  wrote:
>> >
>> > Hey Jose.
>> > I found a gap in handling ISR changes in ZK mode. We just need to
>> prevent
>> > brokers that are offline from being added to ISR. Since KIP-841 is part
>> of
>> > this release and the fix should be small (a few lines), I propose adding
>> > https://issues.apache.org/jira/browse/KAFKA-14140 to the 3.3 release.
>> > I'm hoping to have the PR reviewed and completed next week.
>>
>> I think we should include this fix in 3.3.0 for ZK mode. We
>> implemented this fix for KRaft mode and it will make Apache Kafka
>> safer when handling broker shutdowns.
>>
>> Thanks for volunteering to fix this.
>> --
>> -José
>>
>
>
> --
> - Niket
>


-- 
- Niket


Re: [DISCUSS] Apache Kafka 3.3.0 Release

2022-08-04 Thread Niket Goel
Hey Jose,

I would like to request adding KIP-859 [1] to the 3.3.0 release. The KIP
adds some important metrics to allow visibility into KRaft log processing
related errors.
The KIP was approved today and I have a PR ready for review, which I hope
should get reviewed and merged within next week.

Thanks
Niket Goel

On Thu, Aug 4, 2022 at 8:58 AM José Armando García Sancio
 wrote:

> On Thu, Aug 4, 2022 at 8:37 AM Justine Olshan
>  wrote:
> >
> > Hey Jose.
> > I found a gap in handling ISR changes in ZK mode. We just need to prevent
> > brokers that are offline from being added to ISR. Since KIP-841 is part
> of
> > this release and the fix should be small (a few lines), I propose adding
> > https://issues.apache.org/jira/browse/KAFKA-14140 to the 3.3 release.
> > I'm hoping to have the PR reviewed and completed next week.
>
> I think we should include this fix in 3.3.0 for ZK mode. We
> implemented this fix for KRaft mode and it will make Apache Kafka
> safer when handling broker shutdowns.
>
> Thanks for volunteering to fix this.
> --
> -José
>


-- 
- Niket


Re: [VOTE] KIP-859: Add Metadata Log Processing Error Related Metrics

2022-08-04 Thread Niket Goel
Thanks everyone for the feedback and votes. I have three +1s (David, Colin,
Jose).
Closing this vote now.

On Thu, Aug 4, 2022 at 2:09 PM José Armando García Sancio
 wrote:

> Thanks for the improvement. LGTM. +1 (binding).
>
> --
> -José
>

- Niket


Re: [VOTE] KIP-859: Add Metadata Log Processing Error Related Metrics

2022-08-04 Thread Niket Goel
Hey Jose,

> How about the inactive controller? Are inactive controllers going to
update this metric when they encounter an error when replaying a
record?


Yes, this metric will be reported for both active and inactive controllers.
The Inactive controllers will update this metric when they encounter any
error replaying the records.

- Niket

On Thu, Aug 4, 2022 at 11:30 AM José Armando García Sancio
 wrote:

> Thanks for the KIP Niket.
>
> > kafka.controller:type=KafkaController,name=MetadataErrorCountReports the
> number of times this controller node has renounced leadership of the
> metadata quorum owing to an error encountered during event processing
>
> How about the inactive controller? Are inactive controllers going to
> update this metric when they encounter an error when replaying a
> record?
>
> Thanks!
> José
>


-- 
- Niket


Re: [VOTE] KIP-859: Add Metadata Log Processing Error Related Metrics

2022-08-03 Thread Niket Goel
Thanks for the explanation Colin.

> ForceRenounceCount => 
> kafka.controller:type=KafkaController,name=MetadataErrorCount
> publisher-error-count => metadata-load-error-count
> listener-batch-load-error-count => metadata-apply-error-count

Yeah, this makes sense. I have made the changes in naming you suggested and 
updated the KIP.

- Niket


> On Aug 3, 2022, at 2:00 PM, Colin McCabe  wrote:
> 
> I think there are a few different cases here:
> 
> 1a. We hit an ApiException PREPARING metadata records on the active 
> controller. This is normal and expected. For example, someone tried to create 
> a topic that already exists. We translate the ApiException to an ApiError and 
> return the appropriate error code to the user. (NotControllerException is 
> also included here in 1A)
> 
> 1b. We hit a non-ApiException PREPARING metadata records on the active 
> controller. This is not expected. It would be something like we get a 
> NullPointerException in a createTopics RPC. In this case, we resign 
> leadership and increment your ForceRenounceCount metric.
> 
> 2. We hit any exception APPLYING metadata records on the active controller. 
> In this case, we just want to exit the process. This prevents the 
> (potentially) bad records from ever being commited to the log, since the 
> active controller applies the records locally before sending them out to its 
> peers.
> 
> 3. We hit any exception APPLYING metadata records on a standby controller. In 
> this case, we want to increment a metric so that we know that this happened. 
> But we don't want to exit the process, in case all the controllers are 
> hitting the same problem. Remember that the standby controller only ever 
> applies committed records.
> 
> 4a. We hit any exception constructing the MetadataDelta / new MetadataImage 
> on the broker. Increment listener-batch-load-error-count.
> 
> 4b. We hit any exception applying the MetadataDelta / new MetadataImage on 
> the broker. Increment publisher-error-count
> 
> The KIP is solid on 4a and 4b, but we have no metric here that we can use for 
> case #3. The standby controller can't resign because it's not active in the 
> first place. So I would suggest rather than having a resignation metric, we 
> just have a general controller metadata error metric.
> 
> How do you feel about:
> 
> ForceRenounceCount => 
> kafka.controller:type=KafkaController,name=MetadataErrorCount
> publisher-error-count => metadata-load-error-count
> listener-batch-load-error-count => metadata-apply-error-count
> 
> best,
> Colin
> 
> 
> On Tue, Aug 2, 2022, at 18:05, Niket Goel wrote:
>> Thanks for taking the time to go over the KIP Colin.
>> 
>> While I agree with both your points about error handling, I think this 
>> KIP focuses on just exposing these errors via the proposed metrics and 
>> does not alter the error handling behavior on either the brokers or the 
>> controllers. The metrics (as proposed) are somewhat independent of that 
>> and should just hook into whatever we are doing. If we change the error 
>> handling then the metrics should be hooked into the new code as well.
>> 
>> Maybe the point you are trying to make is that having the properties 
>> which you mention below will generally make it so that errors are not 
>> committed to the log and hence having the metrics for the is sufficient 
>> (as the error will likely be transient and recoverable?).
>> 
>> Apologies if I did not understand the intent of your comment.
>> 
>> - Niket
>> 
>>> On Aug 2, 2022, at 3:34 PM, Colin McCabe  wrote:
>>> 
>>> Hi Niket,
>>> 
>>> Thanks for the KIP -- much appreciated! The new metrics look very useful.
>>> 
>>> I agree with the proposed error handling for errors on standby controllers 
>>> and brokers. For active controllers, I think we should establish two points:
>>> 
>>> 1. the active controller replays metadata before submitting it to the Raft 
>>> quorum
>>> 2. metadata replay errors on the active cause the process to exit, prior to 
>>> attempting to commit the record
>>> 
>>> This will allow most of these metadata replay errors to be noticed and NOT 
>>> committed to the metadata log, which I think will make things much more 
>>> robust. Since the controller process can be restarted very quickly, it 
>>> shouldn't be an undue operational burden. (It's true that when in combined 
>>> mode, restarts will take longer, but this kind of tradeoff is integral to 
>>> combined mode -- you get reduced fault isolation in exchange for the lowe

Re: [VOTE] KIP-859: Add Metadata Log Processing Error Related Metrics

2022-08-02 Thread Niket Goel
Thanks for taking the time to go over the KIP Colin.

While I agree with both your points about error handling, I think this KIP 
focuses on just exposing these errors via the proposed metrics and does not 
alter the error handling behavior on either the brokers or the controllers. The 
metrics (as proposed) are somewhat independent of that and should just hook 
into whatever we are doing. If we change the error handling then the metrics 
should be hooked into the new code as well.

Maybe the point you are trying to make is that having the properties which you 
mention below will generally make it so that errors are not committed to the 
log and hence having the metrics for the is sufficient (as the error will 
likely be transient and recoverable?).

Apologies if I did not understand the intent of your comment.

- Niket

> On Aug 2, 2022, at 3:34 PM, Colin McCabe  wrote:
> 
> Hi Niket,
> 
> Thanks for the KIP -- much appreciated! The new metrics look very useful.
> 
> I agree with the proposed error handling for errors on standby controllers 
> and brokers. For active controllers, I think we should establish two points:
> 
> 1. the active controller replays metadata before submitting it to the Raft 
> quorum
> 2. metadata replay errors on the active cause the process to exit, prior to 
> attempting to commit the record
> 
> This will allow most of these metadata replay errors to be noticed and NOT 
> committed to the metadata log, which I think will make things much more 
> robust. Since the controller process can be restarted very quickly, it 
> shouldn't be an undue operational burden. (It's true that when in combined 
> mode, restarts will take longer, but this kind of tradeoff is integral to 
> combined mode -- you get reduced fault isolation in exchange for the lower 
> overhead of one fewer JVM process).
> 
> best,
> Colin
> 
> 
> On Mon, Aug 1, 2022, at 18:05, David Arthur wrote:
>> Thanks, Niket.
>> 
>> +1 binding from me
>> 
>> -David
>> 
>> On Mon, Aug 1, 2022 at 8:15 PM Niket Goel  wrote:
>>> 
>>> Hi all,
>>> 
>>> I would like to start a vote on KIP-859 which adds some new metrics to 
>>> KRaft to allow for better visibility into log processing errors.
>>> 
>>> KIP 
>>> —ttps://cwiki.apache.org/confluence/display/KAFKA/KIP-859%3A+Add+Metadata+Log+Processing+Error+Related+Metrics
>>>  
>>> <https://www.google.com/url?q=https://cwiki.apache.org/confluence/display/KAFKA/KIP-859%253A%2BAdd%2BMetadata%2BLog%2BProcessing%2BError%2BRelated%2BMetrics&source=gmail-imap&ust=166008449400&usg=AOvVaw3enbkssNlSaDhPrm6faYby>
>>> Discussion Thread — 
>>> https://www.google.com/url?q=https://lists.apache.org/thread/yl87h1s484yc09yjo1no46hwpbv0qkwt&source=gmail-imap&ust=166008449400&usg=AOvVaw2R3Sj0u0NlQOG9XHh-Wgzs
>>> 
>>> Thanks
>>> Niket
>>> 



Re: [DISCUSS] KIP-859: Add Metadata Log Processing Error Related Metrics

2022-08-01 Thread Niket Goel
Hi all,

Thanks for the comments on this KIP. I have started a vote for this KIP here [1]

- Niket

[1] https://lists.apache.org/thread/1kqv8wjcj9w64xxy3p1qxll2ws7xqo3w 

> On Jul 27, 2022, at 11:39 AM, David Arthur 
>  wrote:
> 
> Thanks for the KIP, Niket! I definitely agree we need to surface metadata
> processing errors to the operator. I have some questions about the
> semantics of the new metrics:
> 
> 1) Does "publisher-error-count" represent the number of errors
> encountered only when loading the most recent image? Or will this value be
> the cumulative number of publisher errors since the broker started?
> 2) Same question for "listener-batch-load-error-count"
> 3) Will ForceRenounceCount be zero for non-leader controllers? Or will this
> value remain between elections and only get reset to zero upon a restart
> 
> Thanks!
> David
> 
> On Wed, Jul 27, 2022 at 2:20 PM Niket Goel 
> wrote:
> 
>> 
>> Hi all,
>> 
>> I would like to start a discussion on adding some new metrics to KRaft to
>> allow for better visibility into log processing errors.
>> 
>> KIP URL:
>> https://www.google.com/url?q=https://cwiki.apache.org/confluence/display/KAFKA/KIP-859%253A%2BAdd%2BMetadata%2BLog%2BProcessing%2BError%2BRelated%2BMetrics&source=gmail-imap&ust=165955196500&usg=AOvVaw2Uzcu-JIs-OZSdfTavNjn7
>> 
>> Thanks!
>> Niket
>> 
>> 
> 
> -- 
> -David



[VOTE] KIP-859: Add Metadata Log Processing Error Related Metrics

2022-08-01 Thread Niket Goel
Hi all,

I would like to start a vote on KIP-859 which adds some new metrics to KRaft to 
allow for better visibility into log processing errors.

KIP 
—ttps://cwiki.apache.org/confluence/display/KAFKA/KIP-859%3A+Add+Metadata+Log+Processing+Error+Related+Metrics
 

Discussion Thread — 
https://lists.apache.org/thread/yl87h1s484yc09yjo1no46hwpbv0qkwt 

Thanks
Niket



Re: [DISCUSS] KIP-859: Add Metadata Log Processing Error Related Metrics

2022-07-27 Thread Niket Goel
Thanks for the review David.

Here are the answers to your questions. I will update the KIP to make the info 
clearer.

> 1) Does "publisher-error-count" represent the number of errors
> encountered only when loading the most recent image? Or will this value be
> the cumulative number of publisher errors since the broker started?
> 2) Same question for "listener-batch-load-error-count"
The intent is to have a cumulative number for both of these. The rationale is 
that any fault in loading an image (even if a subsequent load was OK) is worthy 
of inspection. It would be good to have a way to bring the count down to zero 
through an operator initiated signal, but that could be a follow up.

> 3) Will ForceRenounceCount be zero for non-leader controllers? Or will this
> value remain between elections and only get reset to zero upon a restart
I think it makes sense to keep these metrics for all controllers in the system. 
A forced resignation is usually looked at after it has happened, and at that 
point, the controller might not be the leader anymore.

> On Jul 27, 2022, at 11:39 AM, David Arthur 
>  wrote:
> 
> Thanks for the KIP, Niket! I definitely agree we need to surface metadata
> processing errors to the operator. I have some questions about the
> semantics of the new metrics:
> 
> 1) Does "publisher-error-count" represent the number of errors
> encountered only when loading the most recent image? Or will this value be
> the cumulative number of publisher errors since the broker started?
> 2) Same question for "listener-batch-load-error-count"
> 3) Will ForceRenounceCount be zero for non-leader controllers? Or will this
> value remain between elections and only get reset to zero upon a restart
> 
> Thanks!
> David
> 
> On Wed, Jul 27, 2022 at 2:20 PM Niket Goel 
> wrote:
> 
>> 
>> Hi all,
>> 
>> I would like to start a discussion on adding some new metrics to KRaft to
>> allow for better visibility into log processing errors.
>> 
>> KIP URL:
>> https://www.google.com/url?q=https://cwiki.apache.org/confluence/display/KAFKA/KIP-859%253A%2BAdd%2BMetadata%2BLog%2BProcessing%2BError%2BRelated%2BMetrics&source=gmail-imap&ust=165955196500&usg=AOvVaw2Uzcu-JIs-OZSdfTavNjn7
>> 
>> Thanks!
>> Niket
>> 
>> 
> 
> -- 
> -David



[DISCUSS] KIP-859: Add Metadata Log Processing Error Related Metrics

2022-07-27 Thread Niket Goel

Hi all,

I would like to start a discussion on adding some new metrics to KRaft to allow 
for better visibility into log processing errors.

KIP URL: 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-859%3A+Add+Metadata+Log+Processing+Error+Related+Metrics
 

Thanks!
Niket



[jira] [Created] (KAFKA-14114) KIP-859: Add Metadata Log Processing Error Related Metrics

2022-07-27 Thread Niket Goel (Jira)
Niket Goel created KAFKA-14114:
--

 Summary: KIP-859: Add Metadata Log Processing Error Related Metrics
 Key: KAFKA-14114
 URL: https://issues.apache.org/jira/browse/KAFKA-14114
 Project: Kafka
  Issue Type: Improvement
Affects Versions: 3.2.0
Reporter: Niket Goel
Assignee: Niket Goel
 Fix For: 3.3


Tracking Jira for KIP:859



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-13806) Check CRC when reading snapshots

2022-07-26 Thread Niket Goel (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-13806?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Niket Goel resolved KAFKA-13806.

Fix Version/s: 3.3
   Resolution: Duplicate

> Check CRC when reading snapshots
> 
>
> Key: KAFKA-13806
> URL: https://issues.apache.org/jira/browse/KAFKA-13806
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: Jose Armando Garcia Sancio
>Assignee: Jose Armando Garcia Sancio
>Priority: Major
> Fix For: 3.3
>
>




--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14104) Perform CRC validation on KRaft Batch Records and Snapshots

2022-07-25 Thread Niket Goel (Jira)
Niket Goel created KAFKA-14104:
--

 Summary: Perform CRC validation on KRaft Batch Records and 
Snapshots
 Key: KAFKA-14104
 URL: https://issues.apache.org/jira/browse/KAFKA-14104
 Project: Kafka
  Issue Type: Bug
Affects Versions: 3.2.0
Reporter: Niket Goel
 Fix For: 3.3


Today we stamp the BatchRecord header with a CRC [1] and verify that CRC before 
the log is written to disk [2]. The CRC checks should also be verified when the 
records are read back from disk. The same procedure should be followed for 
KRaft snapshots as well.

[1] 
[https://github.com/apache/kafka/blob/6b76c01cf895db0651e2cdcc07c2c392f00a8ceb/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java#L501=]
 

[2] 
[https://github.com/apache/kafka/blob/679e9e0cee67e7d3d2ece204a421ea7da31d73e9/core/src/main/scala/kafka/log/UnifiedLog.scala#L1143]



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Re: [DISCUSS] KIP-853: KRaft Voters Change

2022-07-21 Thread Niket Goel
Hey Jose,

Thanks for the KIP. This is a good improvement and will make the KRaft 
implementation much more robust in the face of failures and generally make it 
more flexible for users.

I did a first pass through the KIP and here are some comments (some of these 
might just be a little uninformed, so feel free to direct me to supplemental 
reading):
Overall protocol safety wise, the reconfiguration operations look safe.

> This UUID will be generated once and persisted as part of the quorum state 
> for the topic partition
Do we mean that it will be generated every time the disk on the replica is 
primed (so every disk incarnation will have UUID). I think you describe it in a 
section further below. Best to pull it up to here — “the replica uuid is 
automatically generated once by the replica when persisting the quorum state 
for the first time.”

> If there are any pending voter change operations the leader will wait for 
> them to finish.
Will new requests be rejected or queued up behind the pending operation. (I am 
assuming rejected, but want to confirm)

> When this option is used the leader of the KRaft topic partition will not 
> allow the AddVoter RPC to add replica IDs that are not describe in the 
> configuration and it would not allow the RemoveVoter RPC to remove replica 
> IDs that are described in the configuration.
Bootstrapping is a little tricky I think. Is it safer/simpler to say that the 
any add/remove RPC operations are blocked until all nodes in the config are 
processed? The way it is worded above makes it seem like the leader will accept 
adds of the same node from outside. Is that the case?

> The KRaft leader will not perform writes from the state machine (active 
> controller) until is has written to the log an AddVoterRecord for every 
> replica id in the controller.quorum.voters  configuration.
Just thinking through - One of the safety requirements for the protocol is for 
a leader to commit at least one write in an epoch before doing config changes, 
right? In this special case we should be ok because the quorum has no one but 
the leader in the beginning. Is that the thought process?

> controller.quorum.bootstrap.servers vs controller.quorum.voters
I understand the use of quorum.voters, but the bootstrap.servers is not 
entirely clear to me. So in the example of starting the cluster with one voter, 
will that one voter be listed here? And when using this configuration, is the 
expectation that quorum.voters is empty, or will it eventually get populated 
with the new quorum members? e.g. further in the kip we say — “Replica 3 will 
discover the partition leader using controller.quorum.voters”; so I guess it 
will be populated?

> This check will be removed and replicas will reply to votes request when the 
> candidate is not in the voter set or the voting replica is not in the voter 
> set.
This is a major change IMO and I think it would be good if we could somehow 
highlight it in the KIP to aid a future reader.

> This also means that the KRaft implementation needs to handle this 
> uncommitted state getting truncated and reverted.
Do we need to talk about the specific behavior a little more here? I mean how 
does this affect any inflight messages with quorums moving between different 
values. (Just a brief except to why it works)

> This state can be discovered by a client by using the DescribeQuorum RPC, the 
> Admin client or the kafka-quorum.sh CLI.
The describeQuorum RPC does not respond with a list of observers today. We 
would need a section to fix that.

> The client can now decide to add replica (3, UUID3') to the set of voters 
> using the AddVoter RPC, the Admin client or the kafka-quorum.sh CLI.
Trying the understand the general thought process‚ the addition of this node 
back into the quorum will be a manually triggered operation and not something 
the node will attempt by itself?

This is a general wonderment, and feel free to ignore it:
Might be good to list some scenarios demonstrating the safety , e.g. how do we 
ensure that there is no loss of availability during an addVoter operation when 
the leader goes down. Or how a failed operation is safely removed from the log 
and reverted.

> On Jul 21, 2022, at 9:49 AM, José Armando García Sancio 
>  wrote:
> 
> Hi all,
> 
> I would like to start the discussion on my design to support
> dynamically changing the set of voters in the KRaft cluster metadata
> topic partition.
> 
> KIP URL: 
> https://www.google.com/url?q=https://cwiki.apache.org/confluence/x/nyH1D&source=gmail-imap&ust=165902699300&usg=AOvVaw12sPgdPT9X6LeINEVmj-iO
> 
> Thanks!
> -José



[jira] [Created] (KAFKA-14035) QuorumController handleRenounce throws NPE

2022-06-30 Thread Niket Goel (Jira)
Niket Goel created KAFKA-14035:
--

 Summary: QuorumController handleRenounce throws NPE
 Key: KAFKA-14035
 URL: https://issues.apache.org/jira/browse/KAFKA-14035
 Project: Kafka
  Issue Type: Bug
Affects Versions: 3.2.0
Reporter: Niket Goel
Assignee: Niket Goel
 Fix For: 3.3.0


Sometimes when the controller is rolled you can encounter the following 
exception, after which the controller in-memory state seems to become 
inconsistent with the Metadata Log.

 

[Controller 1] handleRenounce[23]: failed with unknown server exception 
NullPointerException at epoch -1 in  us. Reverting to last committed offset 
.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


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

2022-06-13 Thread Niket Goel
Thanks for taking a look Gouzhang.
That is an accurate summary of the changes.

- Niket

> On Jun 13, 2022, at 2:19 PM, Guozhang Wang  wrote:
> 
> Hello Niket,
> 
> I took a look at the diffs of the wiki page history and it seems the
> updates are:
> 
> 1) Removing the topic name from QuorumInfo.
> 2) Making ReplicaState public APIs more aligned with existing ones.
> 3) Renaming from DescribeQuorumResult to DescribeMetadataQuorumResult.
> 
> Is that all? If yes, I'm +1 on the changes.
> 
> 
> Guozhang
> 
> On Mon, Jun 13, 2022 at 2:09 PM Niket Goel  <mailto:ng...@confluent.io.invalid>>
> wrote:
> 
>> Hi all,
>> 
>> An update on KIP-836; while iterating on the PR some finer details in the
>> naming of the API handler and related classes in the Admin Client has
>> changed.
>> I have updated the KIP with the changes. Please provide any feedback you
>> may have on the changes made.
>> 
>> PR - 
>> https://www.google.com/url?q=https://github.com/apache/kafka/pull/12206&source=gmail-imap&ust=165576000900&usg=AOvVaw0Av1rIpaRYGZsVAJkgSFwx
>>  
>> <https://www.google.com/url?q=https://github.com/apache/kafka/pull/12206&source=gmail-imap&ust=165576000900&usg=AOvVaw0Av1rIpaRYGZsVAJkgSFwx>
>> KIP -
>> https://www.google.com/url?q=https://cwiki.apache.org/confluence/display/KAFKA/KIP-836%253A%2BAddition%2Bof%2BInformation%2Bin%2BDescribeQuorumResponse%2Babout%2BVoter%2BLag&source=gmail-imap&ust=165576000900&usg=AOvVaw3ZhA-Qvc0rZBYddc4u9667
>>  
>> <https://www.google.com/url?q=https://cwiki.apache.org/confluence/display/KAFKA/KIP-836%253A%2BAddition%2Bof%2BInformation%2Bin%2BDescribeQuorumResponse%2Babout%2BVoter%2BLag&source=gmail-imap&ust=165576000900&usg=AOvVaw3ZhA-Qvc0rZBYddc4u9667>
>> 
>> Thanks
>> - Niket
>> 
>> 
>>> On May 24, 2022, at 2:30 PM, José Armando García Sancio
>>  wrote:
>>> 
>>> +1 (binding).
>>> 
>>> On Mon, May 23, 2022 at 11:52 AM Niket Goel 
>> wrote:
>>>> 
>>>> Thanks everyone for the feedback on this. KIP-836 was approved with
>>>> 3 binding votes from Luke, David Arthur and David Jacot.
>>>> - Niket
>>>> 
>>>>> On May 20, 2022, at 11:44 PM, David Jacot  wrote:
>>>>> 
>>>>> +1 (binding)
>>>>> 
>>>>> Thanks for the KIP!
>>>>> 
>>>>> David
>>>>> 
>>>>> Le ven. 20 mai 2022 à 21:08, David Arthur  a écrit :
>>>>> 
>>>>>> Hey Niket, +1 (binding) from me.
>>>>>> 
>>>>>> -David
>>>>>> 
>>>>>> On Thu, May 19, 2022 at 2:07 PM José Armando García Sancio
>>>>>>  wrote:
>>>>>> 
>>>>>>> Hey Niket,
>>>>>>> 
>>>>>>> I took a look at the latest KIP. It looks like
>> QuorumInfo.ReplicaState
>>>>>>> is missing the RPC fields added by this PR. Is the plan to return
>> them
>>>>>>> to the Admin Client? E.g. it is missing LastFetchTimestamp and
>>>>>>> LastCaughtUpTimestamp.
>>>>>>> 
>>>>>>> For those fields what will the admin client return when the RPC
>>>>>>> version doesn't support those features?
>>>>>>> 
>>>>>>> --
>>>>>>> -José
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> --
>>>>>> David Arthur
>>>>>> 
>>>> 
>>> 
>>> 
>>> --
>>> -José
>> 
>> 
> 
> -- 
> -- Guozhang



[jira] [Created] (KAFKA-13986) DescribeQuorum does not return the observers (brokers) for the Metadata log

2022-06-13 Thread Niket Goel (Jira)
Niket Goel created KAFKA-13986:
--

 Summary: DescribeQuorum does not return the observers (brokers) 
for the Metadata log
 Key: KAFKA-13986
 URL: https://issues.apache.org/jira/browse/KAFKA-13986
 Project: Kafka
  Issue Type: Improvement
  Components: kraft
Affects Versions: 3.0.1, 3.0.0
Reporter: Niket Goel


h2. Background

While working on the [PR|https://github.com/apache/kafka/pull/12206] for 
KIP-836, we realized that the `DescribeQuorum` API does not return the brokers 
as observers for the metadata log.

As noted by [~dengziming] :
_We set nodeId=-1 if it's a broker so observers.size==0_

The related code is:

[https://github.com/apache/kafka/blob/4c9eeef5b2dff9a4f0977fbc5ac7eaaf930d0d0e/core/src/main/scala/kafka/raft/RaftManager.scala#L185-L189]
{code:java}
val nodeId = if (config.processRoles.contains(ControllerRole))
{ OptionalInt.of(config.nodeId) }
else
{ OptionalInt.empty() }
{code}
h2. ToDo

We should fix this and have the DescribeMetadata API return the brokers as 
observers for the metadata log.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)


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

2022-06-13 Thread Niket Goel
Hi all,

An update on KIP-836; while iterating on the PR some finer details in the 
naming of the API handler and related classes in the Admin Client has changed.
I have updated the KIP with the changes. Please provide any feedback you may 
have on the changes made.

PR - https://github.com/apache/kafka/pull/12206
KIP - 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-836%3A+Addition+of+Information+in+DescribeQuorumResponse+about+Voter+Lag

Thanks
- Niket


> On May 24, 2022, at 2:30 PM, José Armando García Sancio 
>  wrote:
> 
> +1 (binding).
> 
> On Mon, May 23, 2022 at 11:52 AM Niket Goel  
> wrote:
>> 
>> Thanks everyone for the feedback on this. KIP-836 was approved with
>> 3 binding votes from Luke, David Arthur and David Jacot.
>> - Niket
>> 
>>> On May 20, 2022, at 11:44 PM, David Jacot  wrote:
>>> 
>>> +1 (binding)
>>> 
>>> Thanks for the KIP!
>>> 
>>> David
>>> 
>>> Le ven. 20 mai 2022 à 21:08, David Arthur  a écrit :
>>> 
>>>> Hey Niket, +1 (binding) from me.
>>>> 
>>>> -David
>>>> 
>>>> On Thu, May 19, 2022 at 2:07 PM José Armando García Sancio
>>>>  wrote:
>>>> 
>>>>> Hey Niket,
>>>>> 
>>>>> I took a look at the latest KIP. It looks like QuorumInfo.ReplicaState
>>>>> is missing the RPC fields added by this PR. Is the plan to return them
>>>>> to the Admin Client? E.g. it is missing LastFetchTimestamp and
>>>>> LastCaughtUpTimestamp.
>>>>> 
>>>>> For those fields what will the admin client return when the RPC
>>>>> version doesn't support those features?
>>>>> 
>>>>> --
>>>>> -José
>>>>> 
>>>> 
>>>> 
>>>> --
>>>> David Arthur
>>>> 
>> 
> 
> 
> -- 
> -José



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

2022-05-23 Thread Niket Goel
Thanks everyone for the feedback on this. KIP-836 was approved with
3 binding votes from Luke, David Arthur and David Jacot.
- Niket

> On May 20, 2022, at 11:44 PM, David Jacot  wrote:
> 
> +1 (binding)
> 
> Thanks for the KIP!
> 
> David
> 
> Le ven. 20 mai 2022 à 21:08, David Arthur  a écrit :
> 
>> Hey Niket, +1 (binding) from me.
>> 
>> -David
>> 
>> On Thu, May 19, 2022 at 2:07 PM José Armando García Sancio
>>  wrote:
>> 
>>> Hey Niket,
>>> 
>>> I took a look at the latest KIP. It looks like QuorumInfo.ReplicaState
>>> is missing the RPC fields added by this PR. Is the plan to return them
>>> to the Admin Client? E.g. it is missing LastFetchTimestamp and
>>> LastCaughtUpTimestamp.
>>> 
>>> For those fields what will the admin client return when the RPC
>>> version doesn't support those features?
>>> 
>>> --
>>> -José
>>> 
>> 
>> 
>> --
>> David Arthur
>> 



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-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-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 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


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

2022-05-16 Thread Niket Goel
Hi all,

I would like to start a vote for KIP-836:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-836%3A+Addition+of+Information+in+DescribeQuorumResponse+about+Voter+Lag

Thanks
Niket Goel


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 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

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-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


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

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


[jira] [Created] (KAFKA-13888) KIP-836: Addition of Information in DescribeQuorumResponse about Voter Lag

2022-05-09 Thread Niket Goel (Jira)
Niket Goel created KAFKA-13888:
--

 Summary: KIP-836: Addition of Information in 
DescribeQuorumResponse about Voter Lag
 Key: KAFKA-13888
 URL: https://issues.apache.org/jira/browse/KAFKA-13888
 Project: Kafka
  Issue Type: Improvement
  Components: kraft
Reporter: Niket Goel


Tracking issue for the implementation of KIP:836



--
This message was sent by Atlassian Jira
(v8.20.7#820007)


Conditional Produce in Kafka (KAFKA-2260)

2021-08-23 Thread Niket Goel
Hi all,

I am new to the Kafka project and while having a discussion with a colleague, I 
was pointed to the JIRA KAFKA-2260 
<https://issues.apache.org/jira/browse/KAFKA-2260> [1] which talks about adding 
the ability to perform "conditional produce" requests to Kafka. This idea 
sounded very exciting to me and I can see it draw parallels with Optimistic 
Concurrency Control in Traditional Databases wherein Kafka could be used as a 
"system of record”, or as an arbiter for disparate systems working on the same 
data. The natural scale-out architecture of Kafka makes this ability even more 
interesting than the more monolithic databases that generally offer this. I 
found a system which is doing something like this - Waltz 
<https://wepay.github.io/waltz/docs/introduction>altz 
<https://wepay.github.io/waltz/docs/introduction> [2] today already.

Following the discussion in the JIRA [1] itself and then on KIP-27 
<https://cwiki.apache.org/confluence/display/KAFKA/KIP-27+-+Conditional+Publish>
 [3], it seems like there was a lot of interest in this proposal, but it never 
materialized. Also IIUC, with the introduction of KIP-98 
<https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging>
 [4], some of the use cases that were motivating the discussion seem to have 
been addressed (e.g. idempotent producer). 

In theory, introducing this ability to Kafka seems pretty great to me, but 
being new to Kafka I am not able to articulate real-life use cases where users 
of Kafka would utilize such an ability. This is where I would like help.

My questions to the group here today are:
1. Is there a use case that you are using/or want to use Kafka for, where 
having the ability to "conditionally produce” or "optimistically lock a key" 
would simplify your life?
2. If there are such scenarios today, how are people making it work? Is it 
through other Kafka features, or through application side logic.

Please feel to also add your thoughts on this feature’s usefulness, or even 
point me towards more suggested reading if I am missing things. :) 


Thanks
Niket Goel

[1] https://issues.apache.org/jira/browse/KAFKA-2260 
<https://issues.apache.org/jira/browse/KAFKA-2260>
[2] https://wepay.github.io/waltz/docs/introduction 
<https://wepay.github.io/waltz/docs/introduction> 
[3] 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-27+-+Conditional+Publish 
<https://cwiki.apache.org/confluence/display/KAFKA/KIP-27+-+Conditional+Publish>
 
[4] 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
 
<https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging>
 



[jira] [Resolved] (KAFKA-13189) Revisit the CreateTopic API behavior when number of Unfenced Brokers available for placement is less than min_isr

2021-08-10 Thread Niket Goel (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-13189?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Niket Goel resolved KAFKA-13189.

Resolution: Fixed

> Revisit the CreateTopic API behavior when number of Unfenced Brokers 
> available for placement is less than min_isr
> -
>
> Key: KAFKA-13189
> URL: https://issues.apache.org/jira/browse/KAFKA-13189
> Project: Kafka
>  Issue Type: Task
>  Components: controller, kraft
>Affects Versions: 3.0.0
>Reporter: Niket Goel
>Priority: Critical
>
> Today when a CreateTopic call is made while the number of brokers available 
> to create the topic is non-zero, the creation of the topic succeeds (under 
> certain conditions). The behavior remains the same even if the min_isr and 
> replication factor for the topic are greater than 1. This would lead to the 
> create succeeding, however a subsequent on the topic immediately failing. 
> This behavior might not be the most intuitive or expected by some users.
> This Jira is cut to ensure that this behavior is revisited and the expected 
> experience is agreed upon prior to KRaft GA.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (KAFKA-13190) Revisit the CreateTopic API behavior when number of Unfenced Brokers available for placement is less than min_isr

2021-08-10 Thread Niket Goel (Jira)
Niket Goel created KAFKA-13190:
--

 Summary: Revisit the CreateTopic API behavior when number of 
Unfenced Brokers available for placement is less than min_isr
 Key: KAFKA-13190
 URL: https://issues.apache.org/jira/browse/KAFKA-13190
 Project: Kafka
  Issue Type: Task
  Components: controller, kraft
Affects Versions: 3.0.0
Reporter: Niket Goel


Today when a CreateTopic call is made while the number of brokers available to 
create the topic is non-zero, the creation of the topic succeeds (under certain 
conditions). The behavior remains the same even if the min_isr and replication 
factor for the topic are greater than 1. This would lead to the create 
succeeding, however a subsequent on the topic immediately failing. This 
behavior might not be the most intuitive or expected by some users.

This Jira is cut to ensure that this behavior is revisited and the expected 
experience is agreed upon prior to KRaft GA.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (KAFKA-13189) Revisit the CreateTopic API behavior when number of Unfenced Brokers available for placement is less than min_isr

2021-08-10 Thread Niket Goel (Jira)
Niket Goel created KAFKA-13189:
--

 Summary: Revisit the CreateTopic API behavior when number of 
Unfenced Brokers available for placement is less than min_isr
 Key: KAFKA-13189
 URL: https://issues.apache.org/jira/browse/KAFKA-13189
 Project: Kafka
  Issue Type: Task
  Components: controller, kraft
Affects Versions: 3.0.0
Reporter: Niket Goel


Today when a CreateTopic call is made while the number of brokers available to 
create the topic is non-zero, the creation of the topic succeeds (under certain 
conditions). The behavior remains the same even if the min_isr and replication 
factor for the topic are greater than 1. This would lead to the create 
succeeding, however a subsequent on the topic immediately failing. This 
behavior might not be the most intuitive or expected by some users.

This Jira is cut to ensure that this behavior is revisited and the expected 
experience is agreed upon prior to KRaft GA.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Resolved] (KAFKA-13015) Create System tests for Metadata Snapshots

2021-07-30 Thread Niket Goel (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-13015?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Niket Goel resolved KAFKA-13015.

Fix Version/s: 3.0.0
 Reviewer: Jose Armando Garcia Sancio
   Resolution: Fixed

> Create System tests for Metadata Snapshots
> --
>
> Key: KAFKA-13015
> URL: https://issues.apache.org/jira/browse/KAFKA-13015
> Project: Kafka
>  Issue Type: Sub-task
>    Reporter: Niket Goel
>    Assignee: Niket Goel
>Priority: Major
>  Labels: kip-500
> Fix For: 3.0.0
>
>
> Create system tests in ducktape to test the ability of brokers and 
> controllers to generate and consume snapshots and catch up with the metadata 
> log.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (KAFKA-13110) Review and Optimize KIP-500 related Settings

2021-07-20 Thread Niket Goel (Jira)
Niket Goel created KAFKA-13110:
--

 Summary: Review and Optimize KIP-500 related Settings
 Key: KAFKA-13110
 URL: https://issues.apache.org/jira/browse/KAFKA-13110
 Project: Kafka
  Issue Type: Improvement
  Components: kraft
Affects Versions: 3.0.0
Reporter: Niket Goel


Some configurations/settings for Kraft have been inherited from the ZK era and 
have not necessarily been thought about again.

This JIRA tracks reviewing configuration settings related to KIP-500 and 
optimizing them where possible.

An example of settings that need an overhaul:
val BrokerHeartbeatIntervalMs = 2000
val BrokerSessionTimeoutMs = 9000
A 2 second heartbeat interval and consequently a 9 second session timeout are 
pretty high values and can be optimized to shorter durations.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (KAFKA-13025) Add more information to the Snapshot header and footer

2021-07-01 Thread Niket Goel (Jira)
Niket Goel created KAFKA-13025:
--

 Summary: Add more information to the Snapshot header and footer
 Key: KAFKA-13025
 URL: https://issues.apache.org/jira/browse/KAFKA-13025
 Project: Kafka
  Issue Type: Improvement
  Components: kraft
Reporter: Niket Goel


https://issues.apache.org/jira/browse/KAFKA-12952 Adds a header and footer to 
metadata snapshots. This Jira follows up with more fields that should be added 
to the delimiters.

Header Fields:
 # -*Version :* Schema version for the snapshot header-
 # -*Last Contained Log Time* : The append time of the highest record contained 
in this snapshot-
 # *End Offset* : End offset of the snapshot from the snapshot ID
 # *Epoch :* Epoch of the snapshot ** from the Snapshot ID**
 # *Creator ID* : (Optional) ID of the broker/Controller that created the 
snapshot
 # *Cluster ID :* (Optional) ID of the cluster that created the snapshot
 # *Create Time :* Timestamp of the snapshot creation (might not be needed as 
each record batch has a timestamp already.

Footer Fields:
 # -*Version* : Schema version of the snapshot footer (same as header)-
 # *Record Type* : A type fields indicating this is the end record for the 
snapshot file.

 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


Request to be added as a contributor to Kafka JIRA board

2021-06-30 Thread Niket Goel
Hi kafka-dev team,

Please add me as a contributor to the Kafka JIRA and Confluence wiki. My JIRA 
username is 'Niket Goel' and my Confluent ID is ngoel.

Thanks
Niket Goel





[jira] [Created] (KAFKA-13016) Interpret snapshot header version to correctly parse the snapshot

2021-06-30 Thread Niket Goel (Jira)
Niket Goel created KAFKA-13016:
--

 Summary: Interpret snapshot header version to correctly parse the 
snapshot
 Key: KAFKA-13016
 URL: https://issues.apache.org/jira/browse/KAFKA-13016
 Project: Kafka
  Issue Type: Sub-task
Reporter: Niket Goel


https://issues.apache.org/jira/browse/KAFKA-12952 adds delimiters to the 
snapshot files. These delimiters serve as start and end markers for the 
snapshots and also contain some metadata information about the snapshots. The 
snapshot consumers need to interpret the version within the header to correctly 
parse the schema of the snapshot being consumed or throw meaningful errors when 
consuming incompatible snapshot versions.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (KAFKA-13015) Create System tests for Metadata Snapshots

2021-06-30 Thread Niket Goel (Jira)
Niket Goel created KAFKA-13015:
--

 Summary: Create System tests for Metadata Snapshots
 Key: KAFKA-13015
 URL: https://issues.apache.org/jira/browse/KAFKA-13015
 Project: Kafka
  Issue Type: Sub-task
Reporter: Niket Goel






--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (KAFKA-12997) Expose log record append time to the controller/broker

2021-06-25 Thread Niket Goel (Jira)
Niket Goel created KAFKA-12997:
--

 Summary: Expose log record append time to the controller/broker
 Key: KAFKA-12997
 URL: https://issues.apache.org/jira/browse/KAFKA-12997
 Project: Kafka
  Issue Type: Sub-task
Reporter: Niket Goel


The snapshot records are generated by each individual quorum participant which 
also stamps the append time in the records. These appends times are generated 
from a different clock (except in the case of the quorum leader) as compared to 
the metadata log records (where timestamps are stamped by the leader).

To enable having a single clock to compare timestamps, 
https://issues.apache.org/jira/browse/KAFKA-12952 adds a timestamp field to the 
snapshot header which should contain the append time of the highest record 
contained in the snapshot (which will be in leader time).

This JIRA tracks exposing and wiring the batch timestamp such that it can be 
provided to the SnapshotWriter at the time of snapshot creation.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (KAFKA-12952) Metadata Snapshot File Header

2021-06-15 Thread Niket Goel (Jira)
Niket Goel created KAFKA-12952:
--

 Summary: Metadata Snapshot File Header
 Key: KAFKA-12952
 URL: https://issues.apache.org/jira/browse/KAFKA-12952
 Project: Kafka
  Issue Type: Sub-task
  Components: controller, kraft
Reporter: Niket Goel


Create a new Control Record that will serve as the header for a Metadata 
Snapshot File. This header will be contained at the beginning of each Snapshot 
File.

Add the following fields to the header record:
 # *Version :* Schema version for the snapshot header
 # *End Offset* : End offset of the snapshot from the snapshot ID
 # *Epoch :* Epoch of the snapshot ** from the Snapshot ID
 # *Creator ID* : (Optional) ID of the broker/Controller that created the 
snapshot
 # *Cluster ID :* (Optional) ID of the cluster that created the snapshot
 # *Create Time :* Timestamp of the snapshot creation (might not be needed as 
each record batch has a timestamp already)
 # *HasFooter :* Boolean indicating if a control record/batch will be included 
at the end of the file

 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)